diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBShowDiskUsageIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBShowDiskUsageIT.java new file mode 100644 index 0000000000000..81b9098d9ef13 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBShowDiskUsageIT.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.it; + +import org.apache.iotdb.isession.ISession; +import org.apache.iotdb.isession.SessionDataSet; +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.ClusterIT; +import org.apache.iotdb.itbase.category.LocalStandaloneIT; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.fail; + +@RunWith(IoTDBTestRunner.class) +@Category({LocalStandaloneIT.class, ClusterIT.class}) +public class IoTDBShowDiskUsageIT { + + @BeforeClass + public static void setUp() throws Exception { + EnvFactory.getEnv().initClusterEnvironment(); + + try (ISession session = EnvFactory.getEnv().getSessionConnection()) { + for (int i = 0; i < 20; i++) { + session.executeNonQueryStatement( + "insert into root.test.d" + i + "(time, s0, s1, s2) values (" + i + ",1, 1, 1)"); + } + session.executeNonQueryStatement( + "insert into root.test.d0(time,s0,s1) aligned values (-1,1,1)"); + session.executeNonQueryStatement("flush"); + } + } + + @AfterClass + public static void tearDown() throws Exception { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + @Test + public void test1() { + + try (ISession session = EnvFactory.getEnv().getSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement("show disk_usage from root.test.**"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + long sum = 0; + Map timePartitionSizes = new HashMap<>(); + while (iterator.next()) { + long timePartition = iterator.getLong("TimePartition"); + long size = iterator.getLong("SizeInBytes"); + timePartitionSizes.compute(timePartition, (k, v) -> v == null ? size : v + size); + sum += size; + } + Assert.assertTrue(sum > 0); + Assert.assertTrue(timePartitionSizes.containsKey(0L)); + Assert.assertTrue(timePartitionSizes.containsKey(-1L)); + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void test2() { + try (ISession session = EnvFactory.getEnv().getSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement("show disk_usage from root.test.** limit 2"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + int count = 0; + while (iterator.next()) { + count++; + } + Assert.assertEquals(2, count); + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void test3() { + try (ISession session = EnvFactory.getEnv().getSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement( + "show disk_usage from root.test.** order by TimePartition desc"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + long previousTimePartition = Long.MAX_VALUE; + while (iterator.next()) { + long currentTimePartition = iterator.getLong("TimePartition"); + Assert.assertTrue(currentTimePartition <= previousTimePartition); + previousTimePartition = currentTimePartition; + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void test4() { + try (ISession session = EnvFactory.getEnv().getSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement( + "show disk_usage from root.test.** where TimePartition < 0 or DataNodeId >= 2"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + while (iterator.next()) { + Assert.assertTrue( + iterator.getLong("TimePartition") < 0 || iterator.getLong("DataNodeId") >= 2); + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void test5() { + try (ISession session = EnvFactory.getEnv().getSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement( + "show disk_usage from root.test.** order by TimePartition desc limit 2"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + long previousTimePartition = Long.MAX_VALUE; + int count = 0; + while (iterator.next()) { + count++; + long currentTimePartition = iterator.getLong("TimePartition"); + Assert.assertTrue(currentTimePartition <= previousTimePartition); + previousTimePartition = currentTimePartition; + } + Assert.assertEquals(2, count); + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } +} diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/IoTDBShowDiskUsageTableIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/IoTDBShowDiskUsageTableIT.java new file mode 100644 index 0000000000000..76ac3ccc7587a --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/IoTDBShowDiskUsageTableIT.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.relational.it; + +import org.apache.iotdb.isession.ITableSession; +import org.apache.iotdb.isession.SessionDataSet; +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.TableClusterIT; +import org.apache.iotdb.itbase.category.TableLocalStandaloneIT; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.fail; + +@RunWith(IoTDBTestRunner.class) +@Category({TableLocalStandaloneIT.class, TableClusterIT.class}) +public class IoTDBShowDiskUsageTableIT { + + @BeforeClass + public static void setUp() throws Exception { + EnvFactory.getEnv().initClusterEnvironment(); + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + session.executeNonQueryStatement("create database test"); + session.executeNonQueryStatement("use test"); + session.executeNonQueryStatement( + "create table t1(tag1 string tag, s0 int32 field, s1 int32 field)"); + session.executeNonQueryStatement( + "create table t2(tag1 string tag, s0 int32 field, s1 int32 field)"); + session.executeNonQueryStatement( + "create view v1(tag1 string tag, s0 int32 field, s1 int32 field) as root.test.**"); + for (int i = 0; i < 20; i++) { + session.executeNonQueryStatement( + "insert into t1(time,tag1,s0,s1) values (" + i + ", 'd" + i + "', 1, 1)"); + } + session.executeNonQueryStatement("insert into t1(time,tag1,s0,s1) values (-1,'d1',1,1)"); + session.executeNonQueryStatement("flush"); + } + } + + @AfterClass + public static void tearDown() throws Exception { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + @Test + public void test1() { + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement( + "select * from information_schema.table_disk_usage where database = 'test'"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + long sum = 0; + Map timePartitionSizes = new HashMap<>(); + Map tableSizes = new HashMap<>(); + while (iterator.next()) { + String table = iterator.getString("table_name"); + long timePartition = iterator.getLong("time_partition"); + long size = iterator.getLong("size_in_bytes"); + timePartitionSizes.compute(timePartition, (k, v) -> v == null ? size : v + size); + tableSizes.compute(table, (k, v) -> v == null ? size : v + size); + sum += size; + } + Assert.assertTrue(sum > 0); + Assert.assertEquals(2, tableSizes.size()); + Assert.assertEquals(0L, (long) tableSizes.get("t2")); + Assert.assertFalse(tableSizes.containsKey("v1")); + Assert.assertTrue(timePartitionSizes.containsKey(0L)); + Assert.assertTrue(timePartitionSizes.containsKey(-1L)); + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void test2() { + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement( + "select * from information_schema.table_disk_usage where database = 'test' limit 2"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + int count = 0; + while (iterator.next()) { + count++; + } + Assert.assertEquals(2, count); + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void test3() { + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement( + "select * from information_schema.table_disk_usage where database = 'test' order by time_partition desc"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + long previousTimePartition = Long.MAX_VALUE; + while (iterator.next()) { + long currentTimePartition = iterator.getLong("time_partition"); + Assert.assertTrue(currentTimePartition <= previousTimePartition); + previousTimePartition = currentTimePartition; + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void test4() { + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement( + "select * from information_schema.table_disk_usage where database = 'test' and (time_partition < 0 or datanode_id >= 2)"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + while (iterator.next()) { + Assert.assertTrue( + iterator.getLong("time_partition") < 0 || iterator.getLong("datanode_id") >= 2); + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void test5() { + try (ITableSession session = EnvFactory.getEnv().getTableSessionConnection()) { + SessionDataSet sessionDataSet = + session.executeQueryStatement( + "select * from information_schema.table_disk_usage where database = 'test' order by time_partition desc limit 2"); + SessionDataSet.DataIterator iterator = sessionDataSet.iterator(); + long previousTimePartition = Long.MAX_VALUE; + int count = 0; + while (iterator.next()) { + count++; + long currentTimePartition = iterator.getLong("time_partition"); + Assert.assertTrue(currentTimePartition <= previousTimePartition); + previousTimePartition = currentTimePartition; + } + Assert.assertEquals(2, count); + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } +} diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java index a5144d92586f7..886d6ef10ac80 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java @@ -414,6 +414,7 @@ public void testInformationSchema() throws SQLException { "regions,INF,", "services,INF,", "subscriptions,INF,", + "table_disk_usage,INF,", "tables,INF,", "topics,INF,", "views,INF,")); @@ -566,6 +567,17 @@ public void testInformationSchema() throws SQLException { "mpp_port,INT32,ATTRIBUTE,", "data_consensus_port,INT32,ATTRIBUTE,", "schema_consensus_port,INT32,ATTRIBUTE,"))); + TestUtils.assertResultSetEqual( + statement.executeQuery("desc table_disk_usage"), + "ColumnName,DataType,Category,", + new HashSet<>( + Arrays.asList( + "database,STRING,FIELD,", + "table_name,STRING,FIELD,", + "datanode_id,INT32,FIELD,", + "region_id,INT32,FIELD,", + "time_partition,INT64,FIELD,", + "size_in_bytes,INT64,FIELD,"))); // Only root user is allowed Assert.assertThrows(SQLException.class, () -> statement.execute("select * from regions")); @@ -580,6 +592,8 @@ public void testInformationSchema() throws SQLException { Assert.assertThrows(SQLException.class, () -> statement.execute("select * from data_nodes")); Assert.assertThrows( SQLException.class, () -> statement.executeQuery("select * from pipe_plugins")); + Assert.assertThrows( + SQLException.class, () -> statement.executeQuery("select * from table_disk_usage")); // Filter out not self-created pipes TestUtils.assertResultSetEqual( @@ -645,6 +659,7 @@ public void testInformationSchema() throws SQLException { "information_schema,configurations,INF,USING,null,SYSTEM VIEW,", "information_schema,keywords,INF,USING,null,SYSTEM VIEW,", "information_schema,nodes,INF,USING,null,SYSTEM VIEW,", + "information_schema,table_disk_usage,INF,USING,null,SYSTEM VIEW,", "information_schema,config_nodes,INF,USING,null,SYSTEM VIEW,", "information_schema,data_nodes,INF,USING,null,SYSTEM VIEW,", "information_schema,connections,INF,USING,null,SYSTEM VIEW,", @@ -655,7 +670,7 @@ public void testInformationSchema() throws SQLException { TestUtils.assertResultSetEqual( statement.executeQuery("count devices from tables where status = 'USING'"), "count(devices),", - Collections.singleton("22,")); + Collections.singleton("23,")); TestUtils.assertResultSetEqual( statement.executeQuery( "select * from columns where table_name = 'queries' or database = 'test'"), diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 index 70f9fced2b364..07b79998a8556 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 @@ -99,6 +99,7 @@ keyWords | DEVICES | DISABLE | DISCARD + | DISK_USAGE | DROP | ELAPSEDTIME | ELSE @@ -223,6 +224,7 @@ keyWords | SGLEVEL | SHOW | SINK + | SIZEINBYTES | SLIMIT | SOFFSET | SOURCE diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 index efe661e05430c..4253d863b399a 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 @@ -91,7 +91,7 @@ dclStatement utilityStatement : flush | clearCache | setConfiguration | settle | startRepairData | stopRepairData | explain | setSystemStatus | showVersion | showFlushInfo | showLockInfo | showQueryResource - | showQueries | showCurrentTimestamp | killQuery | grantWatermarkEmbedding + | showQueries | showDiskUsage | showCurrentTimestamp | killQuery | grantWatermarkEmbedding | revokeWatermarkEmbedding | loadConfiguration | loadTimeseries | loadFile | removeFile | unloadFile | setSqlDialect | showCurrentSqlDialect | showCurrentUser ; @@ -998,6 +998,10 @@ sortKey | DATANODEID | ELAPSEDTIME | STATEMENT + | DATABASE + | REGIONID + | TIMEPARTITION + | SIZEINBYTES ; // ---- Fill Clause @@ -1268,6 +1272,13 @@ showQueries rowPaginationClause? ; +showDiskUsage + : SHOW DISK_USAGE FROM prefixPath + whereClause? + orderByClause? + rowPaginationClause? + ; + // Show Current Timestamp showCurrentTimestamp : SHOW CURRENT_TIMESTAMP diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 index 4ff6f0dc12906..65dcd5810f335 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 @@ -314,6 +314,10 @@ DISCARD : D I S C A R D ; +DISK_USAGE + : D I S K '_' U S A G E + ; + DROP : D R O P ; @@ -823,6 +827,10 @@ SINK : S I N K ; +SIZEINBYTES + : S I Z E I N B Y T E S + ; + SLIMIT : S L I M I T ; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 98c15a2d9bf06..00171f3754845 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -1219,6 +1219,8 @@ public class IoTDBConfig { private long maxObjectSizeInByte = 4 * 1024 * 1024 * 1024L; + private int maxSubTaskNumForInformationTableScan = 4; + IoTDBConfig() {} public int getMaxLogEntriesNumPerBatch() { @@ -4367,4 +4369,12 @@ public long getMaxObjectSizeInByte() { public void setMaxObjectSizeInByte(long maxObjectSizeInByte) { this.maxObjectSizeInByte = maxObjectSizeInByte; } + + public int getMaxSubTaskNumForInformationTableScan() { + return maxSubTaskNumForInformationTableScan; + } + + public void setMaxSubTaskNumForInformationTableScan(int maxSubTaskNumForInformationTableScan) { + this.maxSubTaskNumForInformationTableScan = maxSubTaskNumForInformationTableScan; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 6730138b2af5c..a352b7f9816ad 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -2208,6 +2208,16 @@ public synchronized void loadHotModifiedProps(TrimProperties properties) conf.setMaxRowsInCteBuffer(maxRowsInCteBuffer); } + // max sub-task num for information table scan + int maxSubTaskNumForInformationTableScan = + Integer.parseInt( + properties.getProperty( + "max_sub_task_num_for_information_table_scan", + Integer.toString(conf.getMaxSubTaskNumForInformationTableScan()))); + if (maxSubTaskNumForInformationTableScan > 0) { + conf.setMaxSubTaskNumForInformationTableScan(maxSubTaskNumForInformationTableScan); + } + } catch (Exception e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java index b250046d5c046..ededb9d580cf6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java @@ -60,6 +60,7 @@ import org.apache.iotdb.db.storageengine.dataregion.DataRegion; 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.utils.TableDiskUsageStatisticUtil; import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileResourceUtils; import org.apache.iotdb.db.storageengine.load.LoadTsFileManager; import org.apache.iotdb.db.storageengine.rescon.disk.FolderManager; @@ -657,7 +658,14 @@ private TSStatus loadFileToDataRegion(String filePath, ProgressIndex progressInd StorageEngine.getInstance().getDataRegion(((DataRegionId) consensusGroupId)); if (region != null) { TsFileResource resource = generateTsFileResource(filePath, progressIndex); - region.loadNewTsFile(resource, true, false, true); + region.loadNewTsFile( + resource, + true, + false, + true, + region.isTableModel() + ? TableDiskUsageStatisticUtil.calculateTableSizeMap(resource) + : Optional.empty()); } else { // Data region is null indicates that dr has been removed or migrated. In those cases, there // is no need to replicate data. we just return success to avoid leader keeping retry diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/DatasetHeaderFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/DatasetHeaderFactory.java index 4d19daca70d53..fde6b997ec07c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/DatasetHeaderFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/header/DatasetHeaderFactory.java @@ -207,6 +207,10 @@ public static DatasetHeader getShowQueriesHeader() { return new DatasetHeader(ColumnHeaderConstant.showQueriesColumnHeaders, false); } + public static DatasetHeader getShowDiskUsageHeader() { + return new DatasetHeader(ColumnHeaderConstant.showDiskUsageColumnHeaders, true); + } + public static DatasetHeader getShowSpaceQuotaHeader() { return new DatasetHeader(ColumnHeaderConstant.showSpaceQuotaColumnHeaders, true); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/DataNodeQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/DataNodeQueryContext.java index 814fcc7df63a6..259afd8814ea2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/DataNodeQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/DataNodeQueryContext.java @@ -19,17 +19,24 @@ package org.apache.iotdb.db.queryengine.execution.fragment; +import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.confignode.rpc.thrift.TTableInfo; +import org.apache.iotdb.db.protocol.client.ConfigNodeClient; +import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; +import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.db.queryengine.plan.relational.metadata.fetcher.cache.TableDeviceSchemaCache; +import org.apache.thrift.TException; import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.utils.Pair; import javax.annotation.concurrent.GuardedBy; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; @@ -59,6 +66,10 @@ public class DataNodeQueryContext { private final AtomicInteger dataNodeFINum; + // Used for TableModel information table_disk_usage scan + @GuardedBy("lock") + private Map> databaseTableInfoMap; + // TODO consider more fine-grained locks, now the AtomicInteger in uncachedPathToSeriesScanInfo is // unnecessary private final ReentrantLock lock = new ReentrantLock(); @@ -77,6 +88,25 @@ public void addUnCachePath(PartialPath path, AtomicInteger dataNodeSeriesScanNum uncachedPathToSeriesScanInfo.put(path, new Pair<>(dataNodeSeriesScanNum, null)); } + public Map> getDatabaseTableInfoMap() + throws ClientManagerException, TException { + if (databaseTableInfoMap != null) { + return databaseTableInfoMap; + } + lock.lock(); + if (databaseTableInfoMap != null) { + lock.unlock(); + return databaseTableInfoMap; + } + try (final ConfigNodeClient client = + ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + this.databaseTableInfoMap = client.showTables4InformationSchema().getDatabaseTableInfoMap(); + } finally { + lock.unlock(); + } + return databaseTableInfoMap; + } + public void decreaseDeviceAndMayUpdateLastCache( QualifiedObjectName tableName, DeviceEntry deviceEntry, Integer initialCount) { checkArgument(initialCount != null, "initialCount shouldn't be null here"); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorContext.java index beb25030400fb..013ed167574f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorContext.java @@ -112,7 +112,12 @@ public FragmentInstanceContext getInstanceContext() { return driverContext.getFragmentInstanceContext(); } - public Duration getMaxRunTime() { + public static Duration getMaxRunTime() { + return maxRunTime; + } + + @TestOnly + public Duration getMaxRunTimeForTest() { return maxRunTime; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ShowDiskUsageOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ShowDiskUsageOperator.java new file mode 100644 index 0000000000000..082d787777dc0 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ShowDiskUsageOperator.java @@ -0,0 +1,198 @@ +/* + * 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; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.queryengine.common.header.DatasetHeaderFactory; +import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.utils.StorageEngineTimePartitionIterator; +import org.apache.iotdb.db.storageengine.dataregion.utils.TreeDiskUsageStatisticUtil; + +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.common.conf.TSFileDescriptor; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.read.common.block.TsBlockBuilder; +import org.apache.tsfile.read.filter.basic.Filter; +import org.apache.tsfile.read.reader.series.PaginationController; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +public class ShowDiskUsageOperator implements SourceOperator { + + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(ShowDiskUsageOperator.class); + + private final OperatorContext operatorContext; + private final PlanNodeId sourceId; + private final PartialPath pathPattern; + private final StorageEngineTimePartitionIterator timePartitionIterator; + private final PaginationController paginationController; + private final TsBlockBuilder tsBlockBuilder = + new TsBlockBuilder(DatasetHeaderFactory.getShowDiskUsageHeader().getRespDataTypes()); + private TreeDiskUsageStatisticUtil statisticUtil; + private boolean allConsumed = false; + + public ShowDiskUsageOperator( + OperatorContext operatorContext, + PlanNodeId sourceId, + PartialPath pathPattern, + Filter pushDownFilter, + PaginationController paginationController) { + this.operatorContext = operatorContext; + this.sourceId = sourceId; + this.pathPattern = pathPattern; + this.paginationController = paginationController; + this.timePartitionIterator = + new StorageEngineTimePartitionIterator( + Optional.of( + dataRegion -> { + String databaseName = dataRegion.getDatabaseName(); + return !dataRegion.isTableModel() + && pathPattern.matchPrefixPath(new PartialPath(databaseName)); + }), + Optional.of( + (dataRegion, timePartition) -> { + if (pushDownFilter != null) { + Object[] row = new Object[4]; + row[0] = new Binary(dataRegion.getDatabaseName(), TSFileConfig.STRING_CHARSET); + row[1] = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + row[2] = dataRegion.getDataRegionId().getId(); + row[3] = timePartition; + if (!pushDownFilter.satisfyRow(0, row)) { + return false; + } + } + if (paginationController.hasCurOffset()) { + paginationController.consumeOffset(); + return false; + } + return paginationController.hasCurLimit(); + })); + } + + @Override + public PlanNodeId getSourceId() { + return sourceId; + } + + @Override + public OperatorContext getOperatorContext() { + return operatorContext; + } + + @Override + public TsBlock next() throws Exception { + if (!hasNext()) { + throw new NoSuchElementException(); + } + long maxRuntime = operatorContext.getMaxRunTime().roundTo(TimeUnit.NANOSECONDS); + long start = System.nanoTime(); + do { + if (statisticUtil != null && statisticUtil.hasNextFile()) { + statisticUtil.calculateNextFile(); + continue; + } + if (statisticUtil != null) { + tsBlockBuilder.getTimeColumnBuilder().writeLong(0); + tsBlockBuilder.getValueColumnBuilders()[0].writeBinary( + new Binary( + timePartitionIterator.currentDataRegion().getDatabaseName(), + TSFileConfig.STRING_CHARSET)); + tsBlockBuilder.getValueColumnBuilders()[1].writeInt( + IoTDBDescriptor.getInstance().getConfig().getDataNodeId()); + tsBlockBuilder.getValueColumnBuilders()[2].writeInt( + Integer.parseInt(timePartitionIterator.currentDataRegion().getDataRegionIdString())); + tsBlockBuilder.getValueColumnBuilders()[3].writeLong( + timePartitionIterator.currentTimePartition()); + tsBlockBuilder.getValueColumnBuilders()[4].writeLong(statisticUtil.getResult()[0]); + tsBlockBuilder.declarePosition(); + paginationController.consumeLimit(); + statisticUtil.close(); + } + if (paginationController.hasCurLimit() && timePartitionIterator.nextTimePartition()) { + DataRegion dataRegion = timePartitionIterator.currentDataRegion(); + long timePartition = timePartitionIterator.currentTimePartition(); + statisticUtil = + new TreeDiskUsageStatisticUtil( + dataRegion.getTsFileManager(), + timePartition, + pathPattern, + Optional.ofNullable(operatorContext.getInstanceContext())); + } else { + allConsumed = true; + } + } while (System.nanoTime() - start < maxRuntime && !allConsumed); + + if (!allConsumed) { + return null; + } + return tsBlockBuilder.build(); + } + + @Override + public boolean hasNext() throws Exception { + return !allConsumed; + } + + @Override + public void close() throws Exception { + if (statisticUtil != null) { + statisticUtil.close(); + } + } + + @Override + public boolean isFinished() throws Exception { + return allConsumed; + } + + @Override + public long calculateMaxPeekMemory() { + return TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); + } + + @Override + public long calculateMaxReturnSize() { + return TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); + } + + @Override + public long calculateRetainedSizeAfterCallingNext() { + return 0; + } + + @Override + public long ramBytesUsed() { + return INSTANCE_SIZE + + TreeDiskUsageStatisticUtil.SHALLOW_SIZE + + RamUsageEstimator.sizeOfObject(timePartitionIterator) + + MemoryEstimationHelper.getEstimatedSizeOfPartialPath(pathPattern) + + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(operatorContext) + + MemoryEstimationHelper.getEstimatedSizeOfAccountableObject(sourceId); + } +} 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 8774b670e7a31..d9cdb0cef45ad 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 @@ -63,6 +63,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowTopicReq; import org.apache.iotdb.confignode.rpc.thrift.TTableInfo; import org.apache.iotdb.db.auth.AuthorityChecker; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; @@ -72,10 +73,14 @@ import org.apache.iotdb.db.queryengine.common.ConnectionInfo; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.execution.QueryState; +import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.plan.Coordinator; 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.TableBuiltinTableFunction; +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; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; @@ -83,8 +88,14 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.util.ReservedIdentifiers; import org.apache.iotdb.db.relational.grammar.sql.RelationalSqlKeywords; import org.apache.iotdb.db.schemaengine.table.InformationSchemaUtils; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.utils.StorageEngineTimePartitionIterator; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.DataRegionTableSizeQueryContext; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCacheReader; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TimePartitionTableSizeQueryContext; import org.apache.iotdb.db.utils.MathUtils; import org.apache.iotdb.db.utils.TimestampPrecisionUtils; +import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.thrift.TException; @@ -94,19 +105,27 @@ import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.block.TsBlockBuilder; import org.apache.tsfile.read.common.block.column.RunLengthEncodedColumn; +import org.apache.tsfile.read.filter.basic.Filter; +import org.apache.tsfile.read.reader.series.PaginationController; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BytesUtils; import org.apache.tsfile.utils.Pair; +import java.io.Closeable; +import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.Optional; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -121,6 +140,7 @@ import static org.apache.iotdb.commons.schema.column.ColumnHeaderConstant.NODE_TYPE_CONFIG_NODE; import static org.apache.iotdb.commons.schema.column.ColumnHeaderConstant.NODE_TYPE_DATA_NODE; import static org.apache.iotdb.commons.schema.table.TsTable.TTL_PROPERTY; +import static org.apache.iotdb.db.queryengine.plan.analyze.PredicateUtils.convertPredicateToFilter; import static org.apache.iotdb.db.queryengine.plan.execution.config.TableConfigTaskVisitor.canShowDB; import static org.apache.iotdb.db.queryengine.plan.execution.config.TableConfigTaskVisitor.canShowTable; import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.ShowFunctionsTask.BINARY_MAP; @@ -136,11 +156,12 @@ public class InformationSchemaContentSupplierFactory { private InformationSchemaContentSupplierFactory() {} - public static Iterator getSupplier( - final String tableName, + public static IInformationSchemaContentSupplier getSupplier( + final OperatorContext context, final List dataTypes, - Expression predicate, - final UserEntity userEntity) { + final UserEntity userEntity, + final InformationSchemaTableScanNode node) { + String tableName = node.getQualifiedObjectName().getObjectName(); try { switch (tableName) { case InformationSchema.QUERIES: @@ -175,10 +196,37 @@ public static Iterator getSupplier( return new ConfigNodesSupplier(dataTypes, userEntity); case InformationSchema.DATA_NODES: return new DataNodesSupplier(dataTypes, userEntity); + case InformationSchema.TABLE_DISK_USAGE: + Filter pushDownFilter = null; + if (!InformationSchema.getColumnsSupportPushDownPredicate( + node.getQualifiedObjectName().getObjectName()) + .isEmpty() + && node.getPushDownPredicate() != null) { + Map measurementColumnsIndexMap = + new HashMap<>(node.getOutputColumnNames().size()); + for (int i = 0; i < node.getOutputColumnNames().size(); i++) { + measurementColumnsIndexMap.put(node.getOutputColumnNames().get(i), i); + } + pushDownFilter = + convertPredicateToFilter( + node.getPushDownPredicate(), + measurementColumnsIndexMap, + node.getAssignments(), + null, + context.getSessionInfo().getZoneId(), + TimestampPrecisionUtils.currPrecision); + } + return new TableDiskUsageSupplier( + dataTypes, + userEntity, + pushDownFilter, + new PaginationController(node.getPushDownLimit(), node.getPushDownOffset()), + context, + ((TableDiskUsageInformationSchemaTableScanNode) node).getRegions()); case InformationSchema.CONNECTIONS: return new ConnectionsSupplier(dataTypes, userEntity); case InformationSchema.CURRENT_QUERIES: - return new CurrentQueriesSupplier(dataTypes, predicate, userEntity); + return new CurrentQueriesSupplier(dataTypes, node.getPushDownPredicate(), userEntity); case InformationSchema.QUERIES_COSTS_HISTOGRAM: return new QueriesCostsHistogramSupplier(dataTypes, userEntity); case InformationSchema.SERVICES: @@ -191,6 +239,8 @@ public static Iterator getSupplier( } } + public interface IInformationSchemaContentSupplier extends Iterator, Closeable {} + private static class QueriesSupplier extends TsBlockSupplier { private final long currTime = System.currentTimeMillis(); // We initialize it later for the convenience of data preparation @@ -1165,7 +1215,297 @@ public boolean hasNext() { } } - private abstract static class TsBlockSupplier implements Iterator { + private static class TableDiskUsageSupplier implements IInformationSchemaContentSupplier { + private final List dataTypes; + private final Map> databaseTableInfoMap; + private final Filter pushDownFilter; + private final PaginationController paginationController; + private final OperatorContext operatorContext; + + private DataRegion currentDataRegion; + private boolean currentDatabaseOnlyHasOneTable; + + private TableDiskUsageCacheReader currentDataRegionCacheReader; + private DataRegionTableSizeQueryContext currentDataRegionTableSizeQueryContext; + + private final StorageEngineTimePartitionIterator dataRegionIterator; + + private long prepareCacheReaderCostInNS = 0; + private long loadObjectFileCostInNS = 0; + private long prepareCachedTsFileIDCostInNS = 0; + private long checkAllFilesInTsFileManagerCostInNS = 0; + private long readTsFileCacheValueFilesCostInNS = 0; + + private TableDiskUsageSupplier( + final List dataTypes, + final UserEntity userEntity, + final Filter pushDownFilter, + final PaginationController paginationController, + final OperatorContext operatorContext, + final List regionsForCurrentSubTask) + throws TException, ClientManagerException { + this.dataTypes = dataTypes; + this.pushDownFilter = pushDownFilter; + this.paginationController = paginationController; + this.operatorContext = operatorContext; + AuthorityChecker.getAccessControl().checkUserGlobalSysPrivilege(userEntity); + this.databaseTableInfoMap = + operatorContext.getInstanceContext().getDataNodeQueryContext().getDatabaseTableInfoMap(); + Set regions = new HashSet<>(regionsForCurrentSubTask); + operatorContext.recordSpecifiedInfo( + PlanGraphPrinter.REGIONS_OF_CURRENT_SUB_TASK, regionsForCurrentSubTask.toString()); + this.dataRegionIterator = + new StorageEngineTimePartitionIterator( + Optional.of( + dataRegion -> { + List tTableInfos = + databaseTableInfoMap.get(dataRegion.getDatabaseName()); + if (tTableInfos == null || tTableInfos.isEmpty()) { + return false; + } + if (!regions.contains(dataRegion.getDataRegionId().getId())) { + return false; + } + currentDataRegionTableSizeQueryContext = + new DataRegionTableSizeQueryContext( + false, operatorContext.getInstanceContext()); + return true; + }), + Optional.empty()); + } + + @Override + public boolean hasNext() { + boolean result = hasNextInternal(); + if (!result) { + updateSpecifiedInfo(); + } + return result; + } + + private boolean hasNextInternal() { + if (currentDataRegionCacheReader != null) { + return true; + } + if (!paginationController.hasCurLimit()) { + return false; + } + try { + while (dataRegionIterator.nextDataRegion()) { + currentDataRegion = dataRegionIterator.currentDataRegion(); + for (Long timePartition : currentDataRegion.getTsFileManager().getTimePartitions()) { + Map tablesToScan = getTablesToScan(currentDataRegion, timePartition); + if (!tablesToScan.isEmpty()) { + currentDataRegionTableSizeQueryContext.addTimePartition( + timePartition, new TimePartitionTableSizeQueryContext(tablesToScan)); + } + } + if (currentDataRegionTableSizeQueryContext.isEmpty()) { + continue; + } + currentDataRegionCacheReader = + new TableDiskUsageCacheReader( + currentDataRegion, + currentDataRegionTableSizeQueryContext, + currentDatabaseOnlyHasOneTable); + return true; + } + } catch (Exception e) { + closeDataRegionReader(); + throw new IoTDBRuntimeException( + e.getMessage(), e, TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + return false; + } + + private void updateSpecifiedInfo() { + if (operatorContext + .getSpecifiedInfo() + .containsKey(PlanGraphPrinter.PREPARE_CACHE_READER_COST)) { + return; + } + operatorContext.recordSpecifiedInfo( + PlanGraphPrinter.PREPARE_CACHE_READER_COST, + TimeUnit.NANOSECONDS.toMillis(prepareCacheReaderCostInNS) + + IoTDBConstant.SPACE + + RpcUtils.MILLISECOND); + operatorContext.recordSpecifiedInfo( + PlanGraphPrinter.LOAD_OBJECT_FILE_COST, + TimeUnit.NANOSECONDS.toMillis(loadObjectFileCostInNS) + + IoTDBConstant.SPACE + + RpcUtils.MILLISECOND); + operatorContext.recordSpecifiedInfo( + PlanGraphPrinter.PREPARE_CACHED_TSFILE_ID_COST, + TimeUnit.NANOSECONDS.toMillis(prepareCachedTsFileIDCostInNS) + + IoTDBConstant.SPACE + + RpcUtils.MILLISECOND); + operatorContext.recordSpecifiedInfo( + PlanGraphPrinter.CHECK_ALL_FILES_IN_TSFILE_MANAGER_COST, + TimeUnit.NANOSECONDS.toMillis(checkAllFilesInTsFileManagerCostInNS) + + IoTDBConstant.SPACE + + RpcUtils.MILLISECOND); + operatorContext.recordSpecifiedInfo( + PlanGraphPrinter.READ_TSFILE_CACHE_VALUE_FILES_COST, + TimeUnit.NANOSECONDS.toMillis(readTsFileCacheValueFilesCostInNS) + + IoTDBConstant.SPACE + + RpcUtils.MILLISECOND); + } + + private Map getTablesToScan(DataRegion dataRegion, long timePartition) { + String databaseName = dataRegion.getDatabaseName(); + List tTableInfos = databaseTableInfoMap.get(databaseName); + if (tTableInfos == null || tTableInfos.isEmpty()) { + return Collections.emptyMap(); + } + + Map tablesToScan = new TreeMap<>(); + int totalValidTableCount = 0; + for (TTableInfo tTableInfo : tTableInfos) { + if (tTableInfo.getType() != TableType.BASE_TABLE.ordinal()) { + continue; + } + totalValidTableCount++; + if (pushDownFilter != null) { + Object[] row = new Object[5]; + row[0] = new Binary(dataRegion.getDatabaseName(), TSFileConfig.STRING_CHARSET); + row[1] = new Binary(tTableInfo.getTableName(), TSFileConfig.STRING_CHARSET); + row[2] = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + row[3] = dataRegion.getDataRegionId().getId(); + row[4] = timePartition; + if (!pushDownFilter.satisfyRow(0, row)) { + continue; + } + } + if (!paginationController.hasCurLimit()) { + break; + } + if (paginationController.hasCurOffset()) { + paginationController.consumeOffset(); + continue; + } + paginationController.consumeLimit(); + tablesToScan.put(tTableInfo.getTableName(), 0L); + } + currentDatabaseOnlyHasOneTable = totalValidTableCount == 1; + return tablesToScan; + } + + @Override + public TsBlock next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + long maxRuntime = OperatorContext.getMaxRunTime().roundTo(TimeUnit.NANOSECONDS); + long start = System.nanoTime(); + long prevStageEndTime = start; + + try { + try { + if (!currentDataRegionCacheReader.prepareCacheReader(start, maxRuntime)) { + return null; + } + } finally { + long now = System.nanoTime(); + prepareCacheReaderCostInNS += now - prevStageEndTime; + prevStageEndTime = now; + } + + try { + if (!currentDataRegionCacheReader.loadObjectFileTableSizeCache(start, maxRuntime)) { + return null; + } + } finally { + long now = System.nanoTime(); + loadObjectFileCostInNS += now - prevStageEndTime; + prevStageEndTime = now; + } + + try { + if (!currentDataRegionCacheReader.prepareCachedTsFileIDKeys(start, maxRuntime)) { + return null; + } + } finally { + long now = System.nanoTime(); + prepareCachedTsFileIDCostInNS += now - prevStageEndTime; + prevStageEndTime = now; + } + + try { + if (!currentDataRegionCacheReader.checkAllFilesInTsFileManager(start, maxRuntime)) { + return null; + } + } finally { + long now = System.nanoTime(); + checkAllFilesInTsFileManagerCostInNS += now - prevStageEndTime; + prevStageEndTime = now; + } + + try { + if (!currentDataRegionCacheReader.readCacheValueFilesAndUpdateResultMap( + start, maxRuntime)) { + return null; + } + } finally { + readTsFileCacheValueFilesCostInNS += System.nanoTime() - prevStageEndTime; + } + + return buildTsBlock(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return null; + } catch (Exception e) { + throw new IoTDBRuntimeException( + e.getMessage(), e, TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + } + + private TsBlock buildTsBlock() { + TsBlockBuilder builder = new TsBlockBuilder(dataTypes); + for (Map.Entry entry : + currentDataRegionTableSizeQueryContext + .getTimePartitionTableSizeQueryContextMap() + .entrySet()) { + long timePartition = entry.getKey(); + for (Map.Entry tableSizeEntry : + entry.getValue().getTableSizeResultMap().entrySet()) { + String tableName = tableSizeEntry.getKey(); + long size = tableSizeEntry.getValue(); + builder.getTimeColumnBuilder().writeLong(0); + ColumnBuilder[] columns = builder.getValueColumnBuilders(); + + columns[0].writeBinary( + new Binary(currentDataRegion.getDatabaseName(), TSFileConfig.STRING_CHARSET)); + columns[1].writeBinary(new Binary(tableName, TSFileConfig.STRING_CHARSET)); + columns[2].writeInt(IoTDBDescriptor.getInstance().getConfig().getDataNodeId()); + columns[3].writeInt(currentDataRegion.getDataRegionId().getId()); + columns[4].writeLong(timePartition); + columns[5].writeLong(size); + builder.declarePosition(); + } + } + closeDataRegionReader(); + return builder.build(); + } + + @Override + public void close() throws IOException { + closeDataRegionReader(); + } + + private void closeDataRegionReader() { + if (currentDataRegionCacheReader == null) { + return; + } + try { + currentDataRegionCacheReader.close(); + currentDataRegionCacheReader = null; + } catch (IOException ignored) { + } + } + } + + private abstract static class TsBlockSupplier implements IInformationSchemaContentSupplier { protected final TsBlockBuilder resultBuilder; protected final ColumnBuilder[] columnBuilders; @@ -1194,6 +1534,11 @@ public TsBlock next() { } protected abstract void constructLine(); + + @Override + public void close() throws IOException { + // do nothing + } } private static class ConnectionsSupplier extends TsBlockSupplier { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaTableScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaTableScanOperator.java index 0b2c5275a1b92..d8a625fb92b6e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaTableScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaTableScanOperator.java @@ -28,15 +28,14 @@ import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.utils.RamUsageEstimator; -import java.util.Iterator; - public class InformationSchemaTableScanOperator implements SourceOperator { private final OperatorContext operatorContext; private final PlanNodeId sourceId; - private final Iterator contentSupplier; + private final InformationSchemaContentSupplierFactory.IInformationSchemaContentSupplier + contentSupplier; private static final int DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES = TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); @@ -45,7 +44,9 @@ public class InformationSchemaTableScanOperator implements SourceOperator { RamUsageEstimator.shallowSizeOfInstance(InformationSchemaTableScanOperator.class); public InformationSchemaTableScanOperator( - OperatorContext operatorContext, PlanNodeId sourceId, Iterator contentSupplier) { + OperatorContext operatorContext, + PlanNodeId sourceId, + InformationSchemaContentSupplierFactory.IInformationSchemaContentSupplier contentSupplier) { this.operatorContext = operatorContext; this.sourceId = sourceId; this.contentSupplier = contentSupplier; @@ -73,7 +74,9 @@ public boolean isFinished() throws Exception { @Override public void close() throws Exception { - // do nothing + if (contentSupplier != null) { + contentSupplier.close(); + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analysis.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analysis.java index 37603ee2dbca8..ec1bdeed0c629 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analysis.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analysis.java @@ -58,6 +58,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.component.SortItem; import org.apache.iotdb.db.queryengine.plan.statement.crud.QueryStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ExplainAnalyzeStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowDiskUsageStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement; import org.apache.tsfile.enums.TSDataType; @@ -487,6 +488,7 @@ private boolean hasDataSource() { return (dataPartition != null && !dataPartition.isEmpty()) || (schemaPartition != null && !schemaPartition.isEmpty()) || statement instanceof ShowQueriesStatement + || statement instanceof ShowDiskUsageStatement || (statement instanceof QueryStatement && ((QueryStatement) statement).isAggregationQuery()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index f0f5d419f38ca..8ff6875b18f04 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -137,6 +137,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.pipe.PipeEnrichedStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ExplainAnalyzeStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ExplainStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowDiskUsageStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowVersionStatement; import org.apache.iotdb.db.utils.constant.SqlConstant; @@ -3652,29 +3653,82 @@ public Analysis visitShowQueries( analysis.setSourceExpressions(sourceExpressions); sourceExpressions.forEach(expression -> analyzeExpressionType(analysis, expression)); - analyzeWhere(analysis, showQueriesStatement); + if (!analyzeWhere( + analysis, + showQueriesStatement.getWhereCondition(), + ColumnHeaderConstant.showQueriesColumnHeaders)) { + showQueriesStatement.setWhereCondition(null); + } analysis.setMergeOrderParameter(new OrderByParameter(showQueriesStatement.getSortItemList())); return analysis; } - private void analyzeWhere(Analysis analysis, ShowQueriesStatement showQueriesStatement) { - WhereCondition whereCondition = showQueriesStatement.getWhereCondition(); + @Override + public Analysis visitShowDiskUsage( + ShowDiskUsageStatement showDiskUsageStatement, MPPQueryContext context) { + Analysis analysis = new Analysis(); + analysis.setRealStatement(showDiskUsageStatement); + analysis.setRespDatasetHeader(DatasetHeaderFactory.getShowDiskUsageHeader()); + analysis.setVirtualSource(true); + + List allReadableDataNodeLocations = getReadableDataNodeLocations(); + if (allReadableDataNodeLocations.isEmpty()) { + throw new StatementAnalyzeException("no Running DataNodes"); + } + analysis.setReadableDataNodeLocations(allReadableDataNodeLocations); + + Set sourceExpressions = new HashSet<>(); + for (ColumnHeader columnHeader : analysis.getRespDatasetHeader().getColumnHeaders()) { + sourceExpressions.add( + TimeSeriesOperand.constructColumnHeaderExpression( + columnHeader.getColumnName(), columnHeader.getColumnType())); + } + analysis.setSourceExpressions(sourceExpressions); + sourceExpressions.forEach(expression -> analyzeExpressionType(analysis, expression)); + + if (!analyzeWhere( + analysis, + showDiskUsageStatement.getWhereCondition(), + ColumnHeaderConstant.showDiskUsageColumnHeaders)) { + showDiskUsageStatement.setWhereCondition(null); + } + + analysis.setMergeOrderParameter(new OrderByParameter(showDiskUsageStatement.getSortItemList())); + + return analysis; + } + + private boolean analyzeWhere( + Analysis analysis, WhereCondition whereCondition, List statementColumnHeaders) { if (whereCondition == null) { - return; + return true; } - Expression whereExpression = + Expression predicate = ExpressionAnalyzer.bindTypeForTimeSeriesOperand( - whereCondition.getPredicate(), ColumnHeaderConstant.showQueriesColumnHeaders); + whereCondition.getPredicate(), statementColumnHeaders); + Pair resultPair = + PredicateUtils.extractGlobalTimePredicate(predicate, true, true); + boolean hasValueFilter = resultPair.getRight(); - TSDataType outputType = analyzeExpressionType(analysis, whereExpression); + predicate = PredicateUtils.simplifyPredicate(predicate); + + // set where condition to null if predicate is true or don't have value filter + if (!hasValueFilter || predicate.equals(ConstantOperand.TRUE)) { + return false; + } else { + whereCondition.setPredicate(predicate); + } + TSDataType outputType = analyzeExpressionType(analysis, predicate); if (outputType != TSDataType.BOOLEAN) { throw new SemanticException(String.format(WHERE_WRONG_TYPE_ERROR_MSG, outputType)); } - analysis.setWhereExpression(whereExpression); + analysis.setWhereExpression(predicate); + analysis.setHasValueFilter(true); + return true; } // Region view diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/LimitOffsetPushDown.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/LimitOffsetPushDown.java index e041bb4457a72..6ee46831bc8a0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/LimitOffsetPushDown.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/LimitOffsetPushDown.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.queryengine.plan.expression.Expression; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.CollectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.FillNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.FilterNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.LimitNode; @@ -37,6 +38,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.join.LeftOuterTimeJoinNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.AlignedSeriesScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesScanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowDiskUsageNode; import org.apache.iotdb.db.queryengine.plan.statement.StatementType; import org.apache.iotdb.db.queryengine.plan.statement.component.FillPolicy; import org.apache.iotdb.db.queryengine.plan.statement.component.GroupByTimeComponent; @@ -69,7 +71,11 @@ public class LimitOffsetPushDown implements PlanOptimizer { @Override public PlanNode optimize(PlanNode plan, Analysis analysis, MPPQueryContext context) { - if (analysis.getTreeStatement().getType() != StatementType.QUERY) { + StatementType statementType = analysis.getTreeStatement().getType(); + if (statementType == StatementType.SHOW_DISK_USAGE) { + return plan.accept(new Rewriter(), new RewriterContext(analysis)); + } + if (statementType != StatementType.QUERY) { return plan; } QueryStatement queryStatement = analysis.getQueryStatement(); @@ -174,6 +180,23 @@ public PlanNode visitMultiChildProcess(MultiChildProcessNode node, RewriterConte return node; } + @Override + public PlanNode visitCollect(CollectNode node, RewriterContext context) { + PlanNode newNode = node.clone(); + RewriterContext subContext = new RewriterContext(context.getAnalysis()); + if (context.getLimit() > 0) { + subContext.setLimit(context.getLimit() + context.getOffset()); + } + for (PlanNode child : node.getChildren()) { + newNode.addChild(child.accept(this, subContext)); + } + if (node.getChildren().size() > 1) { + // keep parent limit/offset node + context.setEnablePushDown(false); + } + return newNode; + } + @Override public PlanNode visitTwoChildProcess(TwoChildProcessNode node, RewriterContext context) { context.setEnablePushDown(false); @@ -204,6 +227,15 @@ public PlanNode visitAlignedSeriesScan(AlignedSeriesScanNode node, RewriterConte } return node; } + + @Override + public PlanNode visitShowDiskUsage(ShowDiskUsageNode node, RewriterContext context) { + if (context.isEnablePushDown()) { + node.setPushDownLimit(context.getLimit()); + node.setPushDownOffset(context.getOffset()); + } + return node; + } } private static class RewriterContext { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/PredicatePushDown.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/PredicatePushDown.java index ed15f82ea4be6..4a03287439e73 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/PredicatePushDown.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/optimization/PredicatePushDown.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.path.AlignedPath; import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.plan.analyze.Analysis; @@ -29,6 +30,7 @@ import org.apache.iotdb.db.queryengine.plan.analyze.PredicateUtils; import org.apache.iotdb.db.queryengine.plan.analyze.TemplatedInfo; import org.apache.iotdb.db.queryengine.plan.expression.Expression; +import org.apache.iotdb.db.queryengine.plan.expression.leaf.TimeSeriesOperand; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; @@ -43,6 +45,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.AlignedSeriesScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesScanSourceNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowDiskUsageNode; import org.apache.iotdb.db.queryengine.plan.statement.StatementType; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.iotdb.db.queryengine.plan.statement.crud.QueryStatement; @@ -52,7 +55,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.Function; +import java.util.stream.Collectors; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -62,7 +67,11 @@ public class PredicatePushDown implements PlanOptimizer { @Override public PlanNode optimize(PlanNode plan, Analysis analysis, MPPQueryContext context) { - if (analysis.getTreeStatement().getType() != StatementType.QUERY) { + StatementType statementType = analysis.getTreeStatement().getType(); + if (statementType == StatementType.SHOW_DISK_USAGE) { + return plan.accept(new Rewriter(), new RewriterContext(analysis, context, false)); + } + if (statementType != StatementType.QUERY) { return plan; } QueryStatement queryStatement = analysis.getQueryStatement(); @@ -340,6 +349,53 @@ public PlanNode visitSeriesScanSource(SeriesScanSourceNode node, RewriterContext } } + @Override + public PlanNode visitShowDiskUsage(ShowDiskUsageNode node, RewriterContext context) { + if (context.hasNotInheritedPredicate()) { + return node; + } + Expression inheritedPredicate = context.getInheritedPredicate(); + + List conjuncts = PredicateUtils.extractConjuncts(inheritedPredicate); + List canPushDownConjuncts = new ArrayList<>(); + List cannotPushDownConjuncts = new ArrayList<>(); + for (Expression conjunct : conjuncts) { + + if (PredicateUtils.predicateCanPushDownToSource(conjunct) + && !extractSymbolsFromExpression(conjunct) + .contains(ColumnHeaderConstant.SIZE_IN_BYTES)) { + canPushDownConjuncts.add(conjunct); + } else { + cannotPushDownConjuncts.add(conjunct); + } + } + + if (canPushDownConjuncts.isEmpty()) { + // cannot push down + return node; + } + + node.setPushDownPredicate(PredicateUtils.combineConjuncts(canPushDownConjuncts)); + context.setEnablePushDown(true); + + if (cannotPushDownConjuncts.isEmpty()) { + // all conjuncts can be push down + PlanNode resultNode = planTransform(node, context); + resultNode = planProject(resultNode, context); + return resultNode; + } else { + return planFilter( + node, PredicateUtils.combineConjuncts(cannotPushDownConjuncts), context, true); + } + } + + private Set extractSymbolsFromExpression(Expression expression) { + List sourceExpressions = ExpressionAnalyzer.searchSourceExpressions(expression); + return sourceExpressions.stream() + .map(e -> ((TimeSeriesOperand) e).getPath().toString()) + .collect(Collectors.toSet()); + } + private PlanNode planTransform(PlanNode resultNode, RewriterContext context) { FilterNode pushDownFilterNode = context.getPushDownFilterNode(); Expression[] outputExpressions = pushDownFilterNode.getOutputExpressions(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java index 70325257b085e..a84fff8bb9635 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java @@ -243,6 +243,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.sys.SetSystemStatusStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowCurrentSqlDialectStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowCurrentUserStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowDiskUsageStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowVersionStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.StartRepairDataStatement; @@ -3818,6 +3819,41 @@ public Statement visitShowQueries(IoTDBSqlParser.ShowQueriesContext ctx) { return showQueriesStatement; } + @Override + public Statement visitShowDiskUsage(IoTDBSqlParser.ShowDiskUsageContext ctx) { + PartialPath pathPattern = parsePrefixPath(ctx.prefixPath()); + ShowDiskUsageStatement showDiskUsageStatement = new ShowDiskUsageStatement(pathPattern); + if (ctx.whereClause() != null) { + showDiskUsageStatement.setWhereCondition(parseWhereClause(ctx.whereClause())); + } + + // parse ORDER BY + if (ctx.orderByClause() != null) { + showDiskUsageStatement.setOrderByComponent( + parseOrderByClause( + ctx.orderByClause(), + ImmutableSet.of( + OrderByKey.DATABASE, + OrderByKey.DATANODEID, + OrderByKey.REGIONID, + OrderByKey.TIMEPARTITION, + OrderByKey.SIZEINBYTES), + false)); + } + + // parse LIMIT & OFFSET + if (ctx.rowPaginationClause() != null) { + if (ctx.rowPaginationClause().limitClause() != null) { + showDiskUsageStatement.setLimit(parseLimitClause(ctx.rowPaginationClause().limitClause())); + } + if (ctx.rowPaginationClause().offsetClause() != null) { + showDiskUsageStatement.setOffset( + parseOffsetClause(ctx.rowPaginationClause().offsetClause())); + } + } + return showDiskUsageStatement; + } + // show region @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanBuilder.java index 0aaecc3f13d15..7a1507beb4ffe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanBuilder.java @@ -61,6 +61,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.read.TimeSeriesCountNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.read.TimeSeriesSchemaScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AI.InferenceNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.CollectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ColumnInjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceViewIntoNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceViewNode; @@ -86,6 +87,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.DeviceRegionScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesSourceNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowDiskUsageNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowQueriesNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.TimeseriesRegionScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.AggregationDescriptor; @@ -1315,6 +1317,72 @@ private LogicalPlanBuilder planSingleShowQueries( return this; } + public LogicalPlanBuilder planShowDiskUsage(Analysis analysis, PartialPath pathPattern) { + List dataNodeLocations = analysis.getReadableDataNodeLocations(); + if (dataNodeLocations.size() == 1) { + this.root = + planSingleShowDiskUsage(dataNodeLocations.get(0), pathPattern) + .planFilterAndTransform( + analysis.getWhereExpression(), + analysis.getSourceExpressions(), + false, + Ordering.ASC, + true) + .planSort(analysis.getMergeOrderParameter()) + .getRoot(); + } else if (analysis.getMergeOrderParameter().isEmpty()) { + CollectNode collectNode = + new CollectNode( + context.getQueryId().genPlanNodeId(), + ShowDiskUsageNode.SHOW_DISK_USAGE_HEADER_COLUMNS); + dataNodeLocations.forEach( + dataNodeLocation -> + collectNode.addChild( + planSingleShowDiskUsage(dataNodeLocation, pathPattern) + .planFilterAndTransform( + analysis.getWhereExpression(), + analysis.getSourceExpressions(), + false, + Ordering.ASC, + true) + .getRoot())); + this.root = collectNode; + } else { + MergeSortNode mergeSortNode = + new MergeSortNode( + context.getQueryId().genPlanNodeId(), + analysis.getMergeOrderParameter(), + ShowDiskUsageNode.SHOW_DISK_USAGE_HEADER_COLUMNS); + dataNodeLocations.forEach( + dataNodeLocation -> + mergeSortNode.addChild( + planSingleShowDiskUsage(dataNodeLocation, pathPattern) + .planFilterAndTransform( + analysis.getWhereExpression(), + analysis.getSourceExpressions(), + false, + Ordering.ASC, + true) + .planSort(analysis.getMergeOrderParameter()) + .getRoot())); + this.root = mergeSortNode; + } + + ColumnHeaderConstant.showDiskUsageColumnHeaders.forEach( + columnHeader -> + context + .getTypeProvider() + .setTreeModelType(columnHeader.getColumnName(), columnHeader.getColumnType())); + return this; + } + + private LogicalPlanBuilder planSingleShowDiskUsage( + TDataNodeLocation dataNodeLocation, PartialPath pathPattern) { + this.root = + new ShowDiskUsageNode(context.getQueryId().genPlanNodeId(), dataNodeLocation, pathPattern); + return this; + } + public LogicalPlanBuilder planOrderBy(List sortItemList) { if (sortItemList.isEmpty()) { return this; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java index ea7be0c2c3330..2867d8093ade7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java @@ -87,6 +87,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.metadata.view.ShowLogicalViewStatement; import org.apache.iotdb.db.queryengine.plan.statement.pipe.PipeEnrichedStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ExplainAnalyzeStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowDiskUsageStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement; import org.apache.tsfile.enums.TSDataType; @@ -979,6 +980,18 @@ public PlanNode visitShowQueries( return planBuilder.getRoot(); } + @Override + public PlanNode visitShowDiskUsage( + ShowDiskUsageStatement showDiskUsageStatement, MPPQueryContext context) { + LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(analysis, context); + planBuilder = + planBuilder + .planShowDiskUsage(analysis, showDiskUsageStatement.getPathPattern()) + .planOffset(showDiskUsageStatement.getOffset()) + .planLimit(showDiskUsageStatement.getLimit()); + return planBuilder.getRoot(); + } + @Override public PlanNode visitCreateLogicalView( CreateLogicalViewStatement createLogicalViewStatement, MPPQueryContext context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java index d0e0ff37ca296..be6613c174de8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java @@ -57,6 +57,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.process.ActiveRegionScanMergeOperator; import org.apache.iotdb.db.queryengine.execution.operator.process.AggregationMergeSortOperator; import org.apache.iotdb.db.queryengine.execution.operator.process.AggregationOperator; +import org.apache.iotdb.db.queryengine.execution.operator.process.CollectOperator; import org.apache.iotdb.db.queryengine.execution.operator.process.ColumnInjectOperator; import org.apache.iotdb.db.queryengine.execution.operator.process.DeviceViewIntoOperator; import org.apache.iotdb.db.queryengine.execution.operator.process.DeviceViewOperator; @@ -150,6 +151,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.source.ExchangeOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.SeriesAggregationScanOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.SeriesScanOperator; +import org.apache.iotdb.db.queryengine.execution.operator.source.ShowDiskUsageOperator; import org.apache.iotdb.db.queryengine.execution.operator.source.ShowQueriesOperator; import org.apache.iotdb.db.queryengine.execution.operator.window.ConditionWindowParameter; import org.apache.iotdb.db.queryengine.execution.operator.window.CountWindowParameter; @@ -198,6 +200,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ActiveRegionScanMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationMergeSortNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.CollectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ColumnInjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceViewIntoNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceViewNode; @@ -236,6 +239,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesAggregationScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesAggregationSourceNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesScanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowDiskUsageNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowQueriesNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.TimeseriesRegionScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.AggregationDescriptor; @@ -283,6 +287,7 @@ import org.apache.tsfile.read.filter.basic.Filter; import org.apache.tsfile.read.filter.operator.TimeFilterOperators.TimeGt; import org.apache.tsfile.read.filter.operator.TimeFilterOperators.TimeGtEq; +import org.apache.tsfile.read.reader.series.PaginationController; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.TimeDuration; @@ -1170,6 +1175,18 @@ public Operator visitMergeSort(MergeSortNode node, LocalExecutionPlanContext con getComparator(sortItemList, sortItemIndexList, sortItemDataTypeList)); } + @Override + public Operator visitCollect(CollectNode node, LocalExecutionPlanContext context) { + OperatorContext operatorContext = + context + .getDriverContext() + .addOperatorContext( + context.getNextOperatorId(), + node.getPlanNodeId(), + CollectOperator.class.getSimpleName()); + return new CollectOperator(operatorContext, dealWithConsumeChildrenOneByOneNode(node, context)); + } + @Override public Operator visitAggregationMergeSort( AggregationMergeSortNode node, LocalExecutionPlanContext context) { @@ -2595,6 +2612,32 @@ public Operator visitShowQueries(ShowQueriesNode node, LocalExecutionPlanContext node.getAllowedUsername()); } + @Override + public Operator visitShowDiskUsage(ShowDiskUsageNode node, LocalExecutionPlanContext context) { + OperatorContext operatorContext = + context + .getDriverContext() + .addOperatorContext( + context.getNextOperatorId(), + node.getPlanNodeId(), + ShowDiskUsageOperator.class.getSimpleName()); + Filter pushDownFilter = + convertPredicateToFilter( + node.getPushDownPredicate(), + node.getOutputColumnNames(), + false, + context.getTypeProvider(), + context.getZoneId()); + PaginationController paginationController = + new PaginationController(node.getPushDownLimit(), node.getPushDownOffset()); + return new ShowDiskUsageOperator( + operatorContext, + node.getPlanNodeId(), + node.getPathPattern(), + pushDownFilter, + paginationController); + } + private List generateOutputColumnsFromChildren(MultiChildProcessNode node) { // TODO we should also sort the InputLocation for each column if they are not overlapped return makeLayout(node).entrySet().stream() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java index 1bc788251a764..63705c78910f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java @@ -1319,14 +1319,14 @@ public Operator visitInformationSchemaTableScan( operatorContext, node.getPlanNodeId(), getSupplier( - node.getQualifiedObjectName().getObjectName(), + operatorContext, dataTypes, - node.getPushDownPredicate(), context .getDriverContext() .getFragmentInstanceContext() .getSessionInfo() - .getUserEntity())); + .getUserEntity(), + node)); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/ExchangeNodeAdder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/ExchangeNodeAdder.java index 69105a094d151..009d45ae54008 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/ExchangeNodeAdder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/ExchangeNodeAdder.java @@ -37,6 +37,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ActiveRegionScanMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationMergeSortNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.CollectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceViewNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ExchangeNode; @@ -239,6 +240,11 @@ public PlanNode visitMergeSort(MergeSortNode node, NodeGroupContext context) { return processMultiChildNode(node, context); } + @Override + public PlanNode visitCollect(CollectNode node, NodeGroupContext context) { + return processMultiChildNode(node, context); + } + @Override public PlanNode visitTopK(TopKNode node, NodeGroupContext context) { return processMultiChildNode(node, context); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SimpleFragmentParallelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SimpleFragmentParallelPlanner.java index 334d1973f5345..e1ef9921cf96d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SimpleFragmentParallelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SimpleFragmentParallelPlanner.java @@ -37,6 +37,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.crud.QueryStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.ShowTimeSeriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ExplainAnalyzeStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowDiskUsageStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement; import org.apache.tsfile.read.common.Path; @@ -156,6 +157,7 @@ private void produceFragmentInstance(PlanFragment fragment) { if (analysis.getTreeStatement() instanceof QueryStatement || analysis.getTreeStatement() instanceof ExplainAnalyzeStatement || analysis.getTreeStatement() instanceof ShowQueriesStatement + || analysis.getTreeStatement() instanceof ShowDiskUsageStatement || (analysis.getTreeStatement() instanceof ShowTimeSeriesStatement && ((ShowTimeSeriesStatement) analysis.getTreeStatement()).isOrderByHeat())) { fragmentInstance.getFragment().generateTypeProvider(queryContext.getTypeProvider()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java index bc6475b23236f..c6e10efda78cf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java @@ -48,6 +48,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ActiveRegionScanMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationMergeSortNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.CollectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceViewNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.GroupByLevelNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.GroupByTagNode; @@ -140,6 +141,18 @@ public List visitMergeSort(MergeSortNode node, DistributionPlanContext return Collections.singletonList(newRoot); } + @Override + public List visitCollect(CollectNode node, DistributionPlanContext context) { + CollectNode newRoot = + new CollectNode( + context.queryContext.getQueryId().genPlanNodeId(), node.getOutputColumnNames()); + for (int i = 0; i < node.getChildren().size(); i++) { + List rewroteNodes = rewrite(node.getChildren().get(i), context); + rewroteNodes.forEach(newRoot::addChild); + } + return Collections.singletonList(newRoot); + } + private MergeSortNode cloneMergeSortNodeWithoutChild( MergeSortNode node, DistributionPlanContext context) { return new MergeSortNode( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/memory/PipelineMemoryEstimatorFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/memory/PipelineMemoryEstimatorFactory.java index ea637b586e4b0..f02a5d109f6c7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/memory/PipelineMemoryEstimatorFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/memory/PipelineMemoryEstimatorFactory.java @@ -43,6 +43,7 @@ public static boolean isConsumeChildrenOneByOneNode(final PlanNode node) { case IDENTITY_SINK: case SCHEMA_FETCH_MERGE: case LAST_QUERY_COLLECT: + case TREE_COLLECT: return true; default: return false; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/AbstractFragmentParallelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/AbstractFragmentParallelPlanner.java index d7415320e048a..0dcc7018fc60d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/AbstractFragmentParallelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/AbstractFragmentParallelPlanner.java @@ -85,7 +85,7 @@ protected void selectExecutorAndHost( if (regionReplicaSet == null || regionReplicaSet.getRegionId() == null) { TDataNodeLocation dataNodeLocation = fragment.getTargetLocation(); if (dataNodeLocation != null) { - // now only the case ShowQueries will enter here + // now only the case ShowQueries and ShowDiskUsage will enter here fragmentInstance.setExecutorAndHost(new QueryExecutor(dataNodeLocation)); } else { // no data region && no dataNodeLocation, we need to execute this FI on local 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 180241b4192cd..180b4b9f1be11 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 @@ -27,6 +27,7 @@ import org.apache.iotdb.db.queryengine.plan.expression.Expression; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationMergeSortNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.CollectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ColumnInjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceViewIntoNode; @@ -85,6 +86,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.PreviousFillNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.RowNumberNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SemiJoinNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableFunctionProcessorNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKRankingNode; @@ -129,6 +131,13 @@ public class PlanGraphPrinter extends PlanVisitor, PlanGraphPrinter public static final String CURRENT_USED_MEMORY = "CurrentUsedMemory"; public static final String MAX_USED_MEMORY = "MaxUsedMemory"; public static final String MAX_RESERVED_MEMORY = "MaxReservedMemory"; + public static final String REGIONS_OF_CURRENT_SUB_TASK = "RegionsOfCurrentSubTask"; + public static final String PREPARE_CACHE_READER_COST = "PrepareCacheReaderCost"; + public static final String LOAD_OBJECT_FILE_COST = "LoadObjectFileCost"; + public static final String PREPARE_CACHED_TSFILE_ID_COST = "PrepareCachedTsFileIdCost"; + public static final String CHECK_ALL_FILES_IN_TSFILE_MANAGER_COST = + "CheckAllFilesInTSFileManagerCost"; + public static final String READ_TSFILE_CACHE_VALUE_FILES_COST = "ReadTsFileCacheValueFilesCost"; @Override public List visitPlan(PlanNode node, GraphContext context) { @@ -270,6 +279,14 @@ public List visitMergeSort(MergeSortNode node, GraphContext context) { return render(node, boxValue, context); } + @Override + public List visitCollect(CollectNode node, GraphContext context) { + List boxValue = new ArrayList<>(); + boxValue.add(String.format("Collect-%s", node.getPlanNodeId().getId())); + boxValue.add(String.format("ChildrenCount: %d", node.getChildren().size())); + return render(node, boxValue, context); + } + @Override public List visitTopK(TopKNode node, GraphContext context) { List boxValue = new ArrayList<>(); @@ -685,6 +702,13 @@ public List visitTableScan(TableScanNode node, GraphContext context) { String.format( "MeasurementToColumnName: %s", treeDeviceViewScanNode.getMeasurementColumnNameMap())); } + if (node instanceof TableDiskUsageInformationSchemaTableScanNode) { + boxValue.add( + String.format( + "%s: %s", + REGIONS_OF_CURRENT_SUB_TASK, + ((TableDiskUsageInformationSchemaTableScanNode) node).getRegions())); + } return render(node, boxValue, context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java index 55aaefe8be618..dfa2d7c44961f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeType.java @@ -68,6 +68,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ActiveRegionScanMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationMergeSortNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.CollectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ColumnInjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceViewIntoNode; @@ -103,6 +104,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.LastQueryScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesAggregationScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesScanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowDiskUsageNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowQueriesNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.TimeseriesRegionScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ContinuousSameSearchIndexSeparatorNode; @@ -131,6 +133,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.PreviousFillNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.RowNumberNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SemiJoinNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableFunctionNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableFunctionProcessorNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKRankingNode; @@ -269,6 +272,8 @@ public enum PlanNodeType { LAST_QUERY_SCAN((short) 98), ALTER_ENCODING_COMPRESSOR((short) 99), + SHOW_DISK_USAGE((short) 100), + TREE_COLLECT((short) 101), CREATE_OR_UPDATE_TABLE_DEVICE((short) 902), TABLE_DEVICE_QUERY_SCAN((short) 903), @@ -323,6 +328,7 @@ public enum PlanNodeType { TABLE_TOPK_RANKING_NODE((short) 1037), TABLE_ROW_NUMBER_NODE((short) 1038), TABLE_VALUES_NODE((short) 1039), + TABLE_DISK_USAGE_INFORMATION_SCHEMA_TABLE_SCAN_NODE((short) 1040), RELATIONAL_INSERT_TABLET((short) 2000), RELATIONAL_INSERT_ROW((short) 2001), @@ -607,6 +613,10 @@ public static PlanNode deserialize(ByteBuffer buffer, short nodeType) { return LastQueryScanNode.deserialize(buffer); case 99: return AlterEncodingCompressorNode.deserialize(buffer); + case 100: + return ShowDiskUsageNode.deserialize(buffer); + case 101: + return CollectNode.deserialize(buffer); case 902: return CreateOrUpdateTableDeviceNode.deserialize(buffer); case 903: @@ -727,6 +737,8 @@ public static PlanNode deserialize(ByteBuffer buffer, short nodeType) { return RowNumberNode.deserialize(buffer); case 1039: return ValuesNode.deserialize(buffer); + case 1040: + return TableDiskUsageInformationSchemaTableScanNode.deserialize(buffer); case 2000: return RelationalInsertTabletNode.deserialize(buffer); case 2001: 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 44f1cd8bc1f67..957bfde5121d4 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 @@ -65,6 +65,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ActiveRegionScanMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationMergeSortNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.AggregationNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.CollectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.ColumnInjectNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceMergeNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.DeviceViewIntoNode; @@ -107,6 +108,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesAggregationSourceNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesScanSourceNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowDiskUsageNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowQueriesNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SourceNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.TimeseriesRegionScanNode; @@ -365,6 +367,10 @@ public R visitMergeSort(MergeSortNode node, C context) { return visitMultiChildProcess(node, context); } + public R visitCollect(CollectNode node, C context) { + return visitMultiChildProcess(node, context); + } + public R visitTopK(TopKNode node, C context) { return visitMultiChildProcess(node, context); } @@ -383,6 +389,10 @@ public R visitShowQueries(ShowQueriesNode node, C context) { return visitPlan(node, context); } + public R visitShowDiskUsage(ShowDiskUsageNode node, C context) { + return visitPlan(node, context); + } + public R visitIdentitySink(IdentitySinkNode node, C context) { return visitPlan(node, context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/process/CollectNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/process/CollectNode.java new file mode 100644 index 0000000000000..d44c6412f86fe --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/process/CollectNode.java @@ -0,0 +1,121 @@ +/* + * 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.planner.plan.node.process; + +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; + +import com.google.common.base.Objects; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; + +public class CollectNode extends MultiChildProcessNode { + + private final List outputColumnNames; + + public CollectNode(PlanNodeId id, List outputColumnNames) { + super(id); + this.outputColumnNames = outputColumnNames; + } + + public CollectNode(PlanNodeId id, List children, List outputColumnNames) { + super(id, children); + this.outputColumnNames = outputColumnNames; + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCollect(this, context); + } + + @Override + public PlanNode clone() { + return new CollectNode(id, outputColumnNames); + } + + @Override + public PlanNodeType getType() { + return PlanNodeType.TREE_COLLECT; + } + + @Override + public List getOutputColumnNames() { + return outputColumnNames; + } + + @Override + public PlanNode replaceChildren(List newChildren) { + checkArgument(children.size() == newChildren.size(), "wrong number of new children"); + return new CollectNode(id, newChildren, outputColumnNames); + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + PlanNodeType.TREE_COLLECT.serialize(byteBuffer); + ReadWriteIOUtils.write(outputColumnNames.size(), byteBuffer); + outputColumnNames.forEach(column -> ReadWriteIOUtils.write(column, byteBuffer)); + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + PlanNodeType.TREE_COLLECT.serialize(stream); + ReadWriteIOUtils.write(outputColumnNames.size(), stream); + for (String outputColumnName : outputColumnNames) { + ReadWriteIOUtils.write(outputColumnName, stream); + } + } + + public static CollectNode deserialize(ByteBuffer byteBuffer) { + int size = ReadWriteIOUtils.readInt(byteBuffer); + List outputColumnNames = new ArrayList<>(size); + while (size-- > 0) { + outputColumnNames.add(ReadWriteIOUtils.readString(byteBuffer)); + } + PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer); + return new CollectNode(planNodeId, outputColumnNames); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(super.hashCode()); + } + + @Override + public String toString() { + return "CollectNode-" + this.getPlanNodeId(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/source/ShowDiskUsageNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/source/ShowDiskUsageNode.java new file mode 100644 index 0000000000000..18a0e3687e065 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/source/ShowDiskUsageNode.java @@ -0,0 +1,214 @@ +/* + * 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.planner.plan.node.source; + +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathDeserializeUtil; +import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; +import org.apache.iotdb.db.queryengine.plan.expression.Expression; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; + +import com.google.common.collect.ImmutableList; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public class ShowDiskUsageNode extends VirtualSourceNode { + + public static final List SHOW_DISK_USAGE_HEADER_COLUMNS = + ImmutableList.of( + ColumnHeaderConstant.DATABASE, + ColumnHeaderConstant.DATA_NODE_ID, + ColumnHeaderConstant.REGION_ID, + ColumnHeaderConstant.TIME_PARTITION, + ColumnHeaderConstant.SIZE_IN_BYTES); + + private final PartialPath pathPattern; + private Expression pushDownPredicate; + private long pushDownLimit; + private long pushDownOffset; + + public ShowDiskUsageNode( + PlanNodeId id, TDataNodeLocation dataNodeLocation, PartialPath pathPattern) { + super(id, dataNodeLocation); + this.pathPattern = pathPattern; + } + + public ShowDiskUsageNode( + PlanNodeId id, + TDataNodeLocation dataNodeLocation, + PartialPath pathPattern, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset) { + super(id, dataNodeLocation); + this.pathPattern = pathPattern; + this.pushDownPredicate = pushDownPredicate; + this.pushDownLimit = pushDownLimit; + this.pushDownOffset = pushDownOffset; + } + + public PartialPath getPathPattern() { + return pathPattern; + } + + public Expression getPushDownPredicate() { + return pushDownPredicate; + } + + public void setPushDownPredicate(Expression pushDownPredicate) { + this.pushDownPredicate = pushDownPredicate; + } + + public long getPushDownLimit() { + return pushDownLimit; + } + + public void setPushDownLimit(long pushDownLimit) { + this.pushDownLimit = pushDownLimit; + } + + public long getPushDownOffset() { + return pushDownOffset; + } + + public void setPushDownOffset(long pushDownOffset) { + this.pushDownOffset = pushDownOffset; + } + + @Override + public List getChildren() { + return Collections.emptyList(); + } + + @Override + public void addChild(PlanNode child) { + throw new UnsupportedOperationException("no child is allowed for ShowDiskUsageNode"); + } + + @Override + public PlanNodeType getType() { + return PlanNodeType.SHOW_DISK_USAGE; + } + + @Override + public PlanNode clone() { + return new ShowDiskUsageNode( + getPlanNodeId(), + getDataNodeLocation(), + pathPattern, + pushDownPredicate, + pushDownLimit, + pushDownOffset); + } + + @Override + public int allowedChildCount() { + return NO_CHILD_ALLOWED; + } + + @Override + public List getOutputColumnNames() { + return SHOW_DISK_USAGE_HEADER_COLUMNS; + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitShowDiskUsage(this, context); + } + + // We only use DataNodeLocation when do distributionPlan, so DataNodeLocation is no need to + // serialize + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + PlanNodeType.SHOW_DISK_USAGE.serialize(byteBuffer); + pathPattern.serialize(byteBuffer); + if (pushDownPredicate == null) { + ReadWriteIOUtils.write((byte) 0, byteBuffer); + } else { + ReadWriteIOUtils.write((byte) 1, byteBuffer); + Expression.serialize(pushDownPredicate, byteBuffer); + } + ReadWriteIOUtils.write(pushDownLimit, byteBuffer); + ReadWriteIOUtils.write(pushDownOffset, byteBuffer); + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + PlanNodeType.SHOW_DISK_USAGE.serialize(stream); + pathPattern.serialize(stream); + if (pushDownPredicate == null) { + ReadWriteIOUtils.write((byte) 0, stream); + } else { + ReadWriteIOUtils.write((byte) 1, stream); + Expression.serialize(pushDownPredicate, stream); + } + ReadWriteIOUtils.write(pushDownLimit, stream); + ReadWriteIOUtils.write(pushDownOffset, stream); + } + + public static ShowDiskUsageNode deserialize(ByteBuffer byteBuffer) { + PartialPath pathPattern = (PartialPath) PathDeserializeUtil.deserialize(byteBuffer); + byte isNull = ReadWriteIOUtils.readByte(byteBuffer); + Expression pushDownPredicate = null; + if (isNull == 1) { + pushDownPredicate = Expression.deserialize(byteBuffer); + } + long limit = ReadWriteIOUtils.readLong(byteBuffer); + long offset = ReadWriteIOUtils.readLong(byteBuffer); + PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer); + return new ShowDiskUsageNode(planNodeId, null, pathPattern, pushDownPredicate, limit, offset); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + ShowDiskUsageNode that = (ShowDiskUsageNode) o; + return Objects.equals(this.pathPattern, that.pathPattern); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), pathPattern); + } + + @Override + public String toString() { + return String.format( + "ShowDiskUsageNode-%s: [pathPattern: %s]", this.getPlanNodeId(), pathPattern); + } +} 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 ded699588c1c2..47dc87499b975 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 @@ -385,16 +385,20 @@ private RelationPlan processPhysicalTable(Table table, Scope scope) { outerContext); } - TableScanNode tableScanNode = - qualifiedObjectName.getDatabaseName().equals(INFORMATION_DATABASE) - ? new InformationSchemaTableScanNode( - idAllocator.genPlanNodeId(), qualifiedObjectName, outputSymbols, tableColumnSchema) - : new DeviceTableScanNode( - idAllocator.genPlanNodeId(), - qualifiedObjectName, - outputSymbols, - tableColumnSchema, - tagAndAttributeIndexMap); + TableScanNode tableScanNode; + if (qualifiedObjectName.getDatabaseName().equals(INFORMATION_DATABASE)) { + tableScanNode = + new InformationSchemaTableScanNode( + idAllocator.genPlanNodeId(), qualifiedObjectName, outputSymbols, tableColumnSchema); + } else { + tableScanNode = + new DeviceTableScanNode( + idAllocator.genPlanNodeId(), + qualifiedObjectName, + outputSymbols, + tableColumnSchema, + tagAndAttributeIndexMap); + } return new RelationPlan(tableScanNode, scope, outputSymbols, outerContext); } 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 7072b5f519f73..dcc7c65b120cc 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 @@ -26,10 +26,14 @@ import org.apache.iotdb.commons.exception.IoTDBRuntimeException; import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.partition.SchemaPartition; +import org.apache.iotdb.commons.schema.table.InformationSchema; import org.apache.iotdb.commons.schema.table.TsTable; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.utils.TimePartitionUtils; +import org.apache.iotdb.confignode.rpc.thrift.TRegionInfo; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.sql.SemanticException; +import org.apache.iotdb.db.queryengine.common.DataNodeEndPoints; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.plan.ClusterTopology; @@ -80,6 +84,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SemiJoinNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SortNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.StreamSortNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableFunctionProcessorNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKRankingNode; @@ -1006,15 +1011,23 @@ public List visitTreeDeviceViewScan(TreeDeviceViewScanNode node, PlanC @Override public List visitInformationSchemaTableScan( InformationSchemaTableScanNode node, PlanContext context) { + final String tableName = node.getQualifiedObjectName().getObjectName(); List dataNodeLocations = - dataNodeLocationSupplier.getDataNodeLocations( - node.getQualifiedObjectName().getObjectName()); + dataNodeLocationSupplier.getDataNodeLocations(tableName); if (dataNodeLocations.isEmpty()) { throw new IoTDBRuntimeException( "No available dataNodes, may be the cluster is closing", TSStatusCode.NO_AVAILABLE_REPLICA.getStatusCode()); } + if (InformationSchema.TABLE_DISK_USAGE.equals(tableName)) { + return buildTableDiskUsageScanNodes(node, dataNodeLocations); + } + return buildNormalInformationSchemaScanNodes(node, dataNodeLocations); + } + + private List buildNormalInformationSchemaScanNodes( + InformationSchemaTableScanNode node, List dataNodeLocations) { List resultTableScanNodeList = new ArrayList<>(); dataNodeLocations.forEach( dataNodeLocation -> @@ -1031,6 +1044,66 @@ public List visitInformationSchemaTableScan( return resultTableScanNodeList; } + private List buildTableDiskUsageScanNodes( + InformationSchemaTableScanNode node, List dataNodeLocations) { + final int maxSubTaskNum = + Math.max( + 1, IoTDBDescriptor.getInstance().getConfig().getMaxSubTaskNumForInformationTableScan()); + final Map> regionsByDataNode = + DataNodeLocationSupplierFactory.getReadableRegionsForTableDiskUsageInformationSchemaTable(); + final List result = new ArrayList<>(dataNodeLocations.size() * maxSubTaskNum); + for (TDataNodeLocation dataNodeLocation : dataNodeLocations) { + final List regionInfos = regionsByDataNode.get(dataNodeLocation.getDataNodeId()); + if (regionInfos == null || regionInfos.isEmpty()) { + continue; + } + List> subTaskRegions = splitRegionsBySubTask(regionInfos, maxSubTaskNum); + for (List regionIds : subTaskRegions) { + result.add( + new TableDiskUsageInformationSchemaTableScanNode( + queryId.genPlanNodeId(), + node.getQualifiedObjectName(), + node.getOutputSymbols(), + node.getAssignments(), + node.getPushDownPredicate(), + node.getPushDownLimit(), + node.getPushDownOffset(), + new TRegionReplicaSet(null, ImmutableList.of(dataNodeLocation)), + regionIds)); + } + } + if (result.isEmpty()) { + result.add( + new TableDiskUsageInformationSchemaTableScanNode( + queryId.genPlanNodeId(), + node.getQualifiedObjectName(), + node.getOutputSymbols(), + node.getAssignments(), + node.getPushDownPredicate(), + node.getPushDownLimit(), + node.getPushDownOffset(), + new TRegionReplicaSet( + null, ImmutableList.of(DataNodeEndPoints.getLocalDataNodeLocation())), + Collections.emptyList())); + } + return result; + } + + private List> splitRegionsBySubTask( + List regionInfos, int maxSubTaskNum) { + int subTaskNum = Math.min(maxSubTaskNum, regionInfos.size()); + final List> result = new ArrayList<>(subTaskNum); + for (int i = 0; i < subTaskNum; i++) { + result.add(new ArrayList<>()); + } + for (int i = 0; i < regionInfos.size(); i++) { + int groupIndex = i % subTaskNum; + result.get(groupIndex).add(regionInfos.get(i).getConsensusGroupId().getId()); + } + + return result; + } + @Override public List visitAggregation(AggregationNode node, PlanContext context) { List preGroupedSymbols = node.getPreGroupedSymbols(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushDownOffsetIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushDownOffsetIntoTableScan.java index 77143f9037428..67b50ab31df4c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushDownOffsetIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/iterative/rule/PushDownOffsetIntoTableScan.java @@ -22,6 +22,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.Rule; 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.InformationSchemaTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.OffsetNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.utils.matching.Capture; @@ -54,9 +55,10 @@ public Pattern getPattern() { @Override public Result apply(OffsetNode parent, Captures captures, Context context) { TableScanNode tableScanNode = captures.get(CHILD); - if ((tableScanNode instanceof DeviceTableScanNode - && !(tableScanNode instanceof AggregationTableScanNode)) - && !((DeviceTableScanNode) tableScanNode).isPushLimitToEachDevice()) { + if (tableScanNode instanceof InformationSchemaTableScanNode + || ((tableScanNode instanceof DeviceTableScanNode + && !(tableScanNode instanceof AggregationTableScanNode)) + && !((DeviceTableScanNode) tableScanNode).isPushLimitToEachDevice())) { tableScanNode.setPushDownOffset(parent.getCount()); // consider case that there is no limit tableScanNode.setPushDownLimit( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/InformationSchemaTableScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/InformationSchemaTableScanNode.java index 08a11adbbdb07..cca7dd4df0f7a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/InformationSchemaTableScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/InformationSchemaTableScanNode.java @@ -81,7 +81,7 @@ public InformationSchemaTableScanNode( this.regionReplicaSet = regionReplicaSet; } - private InformationSchemaTableScanNode() {} + protected InformationSchemaTableScanNode() {} @Override public R accept(PlanVisitor visitor, C context) { @@ -101,16 +101,21 @@ public PlanNode clone() { regionReplicaSet); } + @Override + public PlanNodeType getType() { + return PlanNodeType.INFORMATION_SCHEMA_TABLE_SCAN_NODE; + } + @Override protected void serializeAttributes(ByteBuffer byteBuffer) { - PlanNodeType.INFORMATION_SCHEMA_TABLE_SCAN_NODE.serialize(byteBuffer); + getType().serialize(byteBuffer); TableScanNode.serializeMemberVariables(this, byteBuffer, true); } @Override protected void serializeAttributes(DataOutputStream stream) throws IOException { - PlanNodeType.INFORMATION_SCHEMA_TABLE_SCAN_NODE.serialize(stream); + getType().serialize(stream); TableScanNode.serializeMemberVariables(this, stream, true); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/TableDiskUsageInformationSchemaTableScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/TableDiskUsageInformationSchemaTableScanNode.java new file mode 100644 index 0000000000000..a135da76c62b5 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/node/TableDiskUsageInformationSchemaTableScanNode.java @@ -0,0 +1,129 @@ +/* + * 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.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class TableDiskUsageInformationSchemaTableScanNode extends InformationSchemaTableScanNode { + + private List regions; + + public TableDiskUsageInformationSchemaTableScanNode( + PlanNodeId id, + QualifiedObjectName qualifiedObjectName, + List outputSymbols, + Map assignments, + Expression pushDownPredicate, + long pushDownLimit, + long pushDownOffset, + TRegionReplicaSet regionReplicaSet, + List regions) { + super( + id, + qualifiedObjectName, + outputSymbols, + assignments, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + regionReplicaSet); + this.regions = regions; + } + + private TableDiskUsageInformationSchemaTableScanNode() { + super(); + } + + @Override + public PlanNodeType getType() { + return PlanNodeType.TABLE_DISK_USAGE_INFORMATION_SCHEMA_TABLE_SCAN_NODE; + } + + public void setRegions(List regions) { + this.regions = regions; + } + + public List getRegions() { + return regions; + } + + @Override + public PlanNode clone() { + return new TableDiskUsageInformationSchemaTableScanNode( + id, + qualifiedObjectName, + outputSymbols, + assignments, + pushDownPredicate, + pushDownLimit, + pushDownOffset, + regionReplicaSet, + regions); + } + + @Override + protected void serializeAttributes(ByteBuffer byteBuffer) { + super.serializeAttributes(byteBuffer); + ReadWriteIOUtils.write(regions.size(), byteBuffer); + for (Integer region : regions) { + ReadWriteIOUtils.write(region, byteBuffer); + } + } + + @Override + protected void serializeAttributes(DataOutputStream stream) throws IOException { + super.serializeAttributes(stream); + ReadWriteIOUtils.write(regions.size(), stream); + for (Integer region : regions) { + ReadWriteIOUtils.write(region, stream); + } + } + + public static InformationSchemaTableScanNode deserialize(ByteBuffer byteBuffer) { + TableDiskUsageInformationSchemaTableScanNode node = + new TableDiskUsageInformationSchemaTableScanNode(); + TableScanNode.deserializeMemberVariables(byteBuffer, node, true); + int length = ReadWriteIOUtils.readInt(byteBuffer); + List regions = new ArrayList<>(length); + for (int i = 0; i < length; i++) { + regions.add(ReadWriteIOUtils.readInt(byteBuffer)); + } + + node.setPlanNodeId(PlanNodeId.deserialize(byteBuffer)); + node.regions = regions; + return node; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java index 8e73872f5ba8f..a0632017091ae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java @@ -19,11 +19,15 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.exception.IoTDBRuntimeException; import org.apache.iotdb.commons.schema.table.InformationSchema; import org.apache.iotdb.confignode.rpc.thrift.TGetDataNodeLocationsResp; +import org.apache.iotdb.confignode.rpc.thrift.TRegionInfo; +import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq; +import org.apache.iotdb.confignode.rpc.thrift.TShowRegionResp; import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; @@ -32,8 +36,11 @@ import org.apache.thrift.TException; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.apache.iotdb.rpc.TSStatusCode.QUERY_PROCESS_ERROR; @@ -49,6 +56,32 @@ public interface DataNodeLocationSupplier { List getDataNodeLocations(String table); } + public static Map> + getReadableRegionsForTableDiskUsageInformationSchemaTable() { + try (final ConfigNodeClient client = + ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + final TShowRegionReq req = new TShowRegionReq(); + req.setIsTableModel(true); + req.setConsensusGroupType(TConsensusGroupType.DataRegion); + TShowRegionResp tShowRegionResp = client.showRegion(req); + if (tShowRegionResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + throw new IoTDBRuntimeException( + "An error occurred when executing getReadableDataRegions():" + + tShowRegionResp.getStatus().getMessage(), + QUERY_PROCESS_ERROR.getStatusCode()); + } + Map> map = new HashMap<>(); + for (TRegionInfo tRegionInfo : tShowRegionResp.getRegionInfoList()) { + map.computeIfAbsent(tRegionInfo.getDataNodeId(), k -> new ArrayList<>()).add(tRegionInfo); + } + return map; + } catch (final ClientManagerException | TException e) { + throw new IoTDBRuntimeException( + "An error occurred when executing getReadableDataNodeLocations():" + e.getMessage(), + QUERY_PROCESS_ERROR.getStatusCode()); + } + } + /** DataNode in these states is readable: Running, ReadOnly, Removing */ public static List getReadableDataNodeLocations() { try (final ConfigNodeClient client = @@ -85,6 +118,7 @@ private static InformationSchemaTableDataNodeLocationSupplier getInstance() { public List getDataNodeLocations(final String tableName) { switch (tableName) { case InformationSchema.QUERIES: + case InformationSchema.TABLE_DISK_USAGE: case InformationSchema.CONNECTIONS: case InformationSchema.CURRENT_QUERIES: case InformationSchema.QUERIES_COSTS_HISTOGRAM: 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 ce8a002a465da..89403b59f9795 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 @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations; +import org.apache.iotdb.commons.schema.table.InformationSchema; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; @@ -41,6 +42,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SortNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.StreamSortNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableFunctionProcessorNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKNode; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; @@ -53,7 +55,7 @@ /** * Optimization phase: Logical plan planning. * - *

The LIMIT OFFSET condition can be pushed down to the DeviceTableScanNode, when the following + *

The LIMIT OFFSET condition can be pushed down to the TableScanNode, when the following * conditions are met: *

  • Time series query (not aggregation query). *
  • The query expressions are all scalar expression. @@ -105,9 +107,9 @@ public PlanNode visitFilter(FilterNode node, Context context) { // In Filter-TableScan and Filter-Project-TableScan case, limit can not be pushed down. // In later, we need consider other case such as Filter-Values. // FilterNode in outer query can not be pushed down. - if (node.getChild() instanceof DeviceTableScanNode + if (node.getChild() instanceof TableScanNode || (node.getChild() instanceof ProjectNode - && ((ProjectNode) node.getChild()).getChild() instanceof DeviceTableScanNode)) { + && ((ProjectNode) node.getChild()).getChild() instanceof TableScanNode)) { context.enablePushDown = false; return node; } @@ -162,10 +164,10 @@ public PlanNode visitLimit(LimitNode node, Context context) { context.enablePushDown = false; return node; } else { - DeviceTableScanNode deviceTableScanNode = subContext.deviceTableScanNode; - context.deviceTableScanNode = deviceTableScanNode; - if (deviceTableScanNode != null) { - deviceTableScanNode.setPushDownLimit(node.getCount()); + TableScanNode tableScanNode = subContext.tableScanNode; + context.tableScanNode = tableScanNode; + if (tableScanNode != null) { + tableScanNode.setPushDownLimit(node.getCount()); } return node; } @@ -183,14 +185,19 @@ public PlanNode visitSort(SortNode node, Context context) { return node; } - DeviceTableScanNode deviceTableScanNode = subContext.deviceTableScanNode; - context.deviceTableScanNode = deviceTableScanNode; + TableScanNode tableScanNode = subContext.tableScanNode; + context.tableScanNode = tableScanNode; + + if (!(tableScanNode instanceof DeviceTableScanNode)) { + context.enablePushDown = false; + return node; + } OrderingScheme orderingScheme = node.getOrderingScheme(); Map tableColumnSchema = - analysis.getTableColumnSchema(deviceTableScanNode.getQualifiedObjectName()); + analysis.getTableColumnSchema(tableScanNode.getQualifiedObjectName()); Set sortSymbols = new HashSet<>(); for (Symbol orderBy : orderingScheme.getOrderBy()) { - if (deviceTableScanNode.isTimeColumn(orderBy)) { + if (tableScanNode.isTimeColumn(orderBy)) { break; } @@ -212,7 +219,7 @@ public PlanNode visitSort(SortNode node, Context context) { break; } } - deviceTableScanNode.setPushLimitToEachDevice(pushLimitToEachDevice); + ((DeviceTableScanNode) tableScanNode).setPushLimitToEachDevice(pushLimitToEachDevice); return node; } @@ -234,7 +241,7 @@ public PlanNode visitAggregation(AggregationNode node, Context context) { @Override public PlanNode visitDeviceTableScan(DeviceTableScanNode node, Context context) { - context.deviceTableScanNode = node; + context.tableScanNode = node; return node; } @@ -247,7 +254,12 @@ public PlanNode visitCteScan(CteScanNode node, Context context) { @Override public PlanNode visitInformationSchemaTableScan( InformationSchemaTableScanNode node, Context context) { - context.enablePushDown = false; + if (InformationSchema.supportsPushDownLimitOffset( + node.getQualifiedObjectName().getObjectName())) { + context.tableScanNode = node; + } else { + context.enablePushDown = false; + } return node; } @@ -269,9 +281,9 @@ public PlanNode visitTopK(TopKNode node, Context context) { } private static class Context { - // means if limit and offset can be pushed down into DeviceTableScanNode + // means if limit and offset can be pushed down into TableScanNode private boolean enablePushDown = true; - private DeviceTableScanNode deviceTableScanNode; + private TableScanNode tableScanNode; private boolean existSortNode = false; private boolean existLimitNode = false; } 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 55905f6768247..7bd4530afd194 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 @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.partition.DataPartitionQueryParam; +import org.apache.iotdb.commons.schema.table.InformationSchema; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.sql.SemanticException; @@ -61,6 +62,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ProjectNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SemiJoinNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SortNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.UnionNode; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; @@ -447,97 +449,12 @@ && extractUnique(conjunct).contains(node.getGroupIdSymbol().get())) { @Override public PlanNode visitInformationSchemaTableScan( InformationSchemaTableScanNode node, RewriteContext context) { - // no predicate or table is not current_queries, just return dierectly if (TRUE_LITERAL.equals(context.inheritedPredicate)) { return node; } - // push-down for CURRENT_QUERIES - if (CURRENT_QUERIES.equals(node.getQualifiedObjectName().getObjectName())) { - SplitExpression splitExpression = splitCurrentQueriesPredicate(context.inheritedPredicate); - // exist expressions can push down to scan operator - if (!splitExpression.getExpressionsCanPushDown().isEmpty()) { - List expressions = splitExpression.getExpressionsCanPushDown(); - checkState(expressions.size() == 1, "Unexpected number of expressions in table scan"); - node.setPushDownPredicate(expressions.get(0)); - } - - // exist expressions cannot push down - if (!splitExpression.getExpressionsCannotPushDown().isEmpty()) { - List expressions = splitExpression.getExpressionsCannotPushDown(); - return new FilterNode( - queryId.genPlanNodeId(), - node, - expressions.size() == 1 - ? expressions.get(0) - : new LogicalExpression(LogicalExpression.Operator.AND, expressions)); - } - return node; - } - - FilterNode filterNode = - new FilterNode(queryId.genPlanNodeId(), node, context.inheritedPredicate); - context.inheritedPredicate = TRUE_LITERAL; - return filterNode; - } - - private SplitExpression splitCurrentQueriesPredicate(Expression predicate) { - List expressionsCanPushDown = new ArrayList<>(); - List expressionsCannotPushDown = new ArrayList<>(); - - if (predicate instanceof LogicalExpression - && ((LogicalExpression) predicate).getOperator() == LogicalExpression.Operator.AND) { - - // predicate like state = 'xxx' can be push down - // Note: the optimizer CanonicalizeExpressionRewriter will ensure the predicate like 'xxx' = - // state will be canonicalized to state = 'xxx' - boolean hasExpressionPushDown = false; - for (Expression expression : ((LogicalExpression) predicate).getTerms()) { - if (isStateComparedWithConstant(expression) && !hasExpressionPushDown) { - // if there are more than one state = 'xxx' terms, only add first to push-down candidate - expressionsCanPushDown.add(expression); - hasExpressionPushDown = true; - } else { - expressionsCannotPushDown.add(expression); - } - } - - return new SplitExpression( - Collections.emptyList(), expressionsCanPushDown, expressionsCannotPushDown, null); - } - - if (isStateComparedWithConstant(predicate)) { - expressionsCanPushDown.add(predicate); - } else { - expressionsCannotPushDown.add(predicate); - } - - return new SplitExpression( - Collections.emptyList(), expressionsCanPushDown, expressionsCannotPushDown, null); - } - - private boolean isStateComparedWithConstant(Expression expression) { - if (!(expression instanceof ComparisonExpression)) { - return false; - } - - ComparisonExpression comparisonExpression = (ComparisonExpression) expression; - - if (ComparisonExpression.Operator.EQUAL != comparisonExpression.getOperator()) { - return false; - } - - if (!(comparisonExpression.getLeft() instanceof SymbolReference) - || !STATE_TABLE_MODEL.equals( - ((SymbolReference) comparisonExpression.getLeft()).getName())) { - return false; - } - - if (!(comparisonExpression.getRight() instanceof StringLiteral)) { - return false; - } - - return true; + // push down for information schema tables + return combineFilterAndScan(node, context.inheritedPredicate); } @Override @@ -554,8 +471,12 @@ public PlanNode visitDeviceTableScan( return combineFilterAndScan(tableScanNode, context.inheritedPredicate); } - public PlanNode combineFilterAndScan(DeviceTableScanNode tableScanNode, Expression predicate) { - SplitExpression splitExpression = splitPredicate(tableScanNode, predicate); + public PlanNode combineFilterAndScan(TableScanNode tableScanNode, Expression predicate) { + SplitExpression splitExpression = + tableScanNode instanceof InformationSchemaTableScanNode + ? splitPredicateForInformationSchemaTable( + (InformationSchemaTableScanNode) tableScanNode, predicate) + : splitPredicate((DeviceTableScanNode) tableScanNode, predicate); // exist expressions can push down to scan operator if (!splitExpression.getExpressionsCanPushDown().isEmpty()) { @@ -568,10 +489,11 @@ public PlanNode combineFilterAndScan(DeviceTableScanNode tableScanNode, Expressi // extract global time filter and set it to DeviceTableScanNode Pair resultPair = extractGlobalTimeFilter(pushDownPredicate, splitExpression.getTimeColumnName()); - if (resultPair.left != null) { - tableScanNode.setTimePredicate(resultPair.left); + Boolean hasValueFilter = resultPair.getRight(); + if (tableScanNode instanceof DeviceTableScanNode && resultPair.left != null) { + ((DeviceTableScanNode) tableScanNode).setTimePredicate(resultPair.left); } - if (Boolean.TRUE.equals(resultPair.right)) { + if (Boolean.TRUE.equals(hasValueFilter)) { if (pushDownPredicate instanceof LogicalExpression && ((LogicalExpression) pushDownPredicate).getTerms().size() == 1) { tableScanNode.setPushDownPredicate( @@ -585,7 +507,10 @@ public PlanNode combineFilterAndScan(DeviceTableScanNode tableScanNode, Expressi } // do index scan after expressionCanPushDown is processed - getDeviceEntriesWithDataPartitions(tableScanNode, splitExpression.getMetadataExpressions()); + if (tableScanNode instanceof DeviceTableScanNode) { + getDeviceEntriesWithDataPartitions( + (DeviceTableScanNode) tableScanNode, splitExpression.getMetadataExpressions()); + } // exist expressions can not push down to scan operator if (!splitExpression.getExpressionsCannotPushDown().isEmpty()) { @@ -601,6 +526,101 @@ public PlanNode combineFilterAndScan(DeviceTableScanNode tableScanNode, Expressi return tableScanNode; } + interface InformationSchemaTablePredicatePushDownChecker { + boolean canPushDown(Expression expression); + } + + private SplitExpression splitPredicateForInformationSchemaTable( + InformationSchemaTableScanNode tableScanNode, Expression predicate) { + String informationSchemaTable = tableScanNode.getQualifiedObjectName().getObjectName(); + InformationSchemaTablePredicatePushDownChecker checker; + switch (informationSchemaTable) { + case CURRENT_QUERIES: + checker = + new InformationSchemaTablePredicatePushDownChecker() { + // predicate like state = 'xxx' can be push down + // Note: the optimizer CanonicalizeExpressionRewriter will ensure the predicate like + // 'xxx' = + // state will be canonicalized to state = 'xxx' + boolean hasExpressionPushDown = false; + + @Override + public boolean canPushDown(Expression expression) { + if (isStateComparedWithConstant(expression) && !hasExpressionPushDown) { + // if there are more than one state = 'xxx' terms, only add first to push-down + // candidate + hasExpressionPushDown = true; + return true; + } + return false; + } + + private boolean isStateComparedWithConstant(Expression expression) { + if (!(expression instanceof ComparisonExpression)) { + return false; + } + + ComparisonExpression comparisonExpression = (ComparisonExpression) expression; + + if (ComparisonExpression.Operator.EQUAL != comparisonExpression.getOperator()) { + return false; + } + + if (!(comparisonExpression.getLeft() instanceof SymbolReference) + || !STATE_TABLE_MODEL.equals( + ((SymbolReference) comparisonExpression.getLeft()).getName())) { + return false; + } + + return comparisonExpression.getRight() instanceof StringLiteral; + } + }; + break; + default: + checker = + new InformationSchemaTablePredicatePushDownChecker() { + final Set columnsThatSupportPushDownPredicate = + InformationSchema.getColumnsSupportPushDownPredicate(informationSchemaTable); + + @Override + public boolean canPushDown(Expression expression) { + return PredicateCombineIntoTableScanChecker.check( + columnsThatSupportPushDownPredicate, expression); + } + }; + } + return splitPredicateForInformationSchemaTable(predicate, checker); + } + + private SplitExpression splitPredicateForInformationSchemaTable( + Expression predicate, InformationSchemaTablePredicatePushDownChecker checker) { + List expressionsCanPushDown = new ArrayList<>(); + List expressionsCannotPushDown = new ArrayList<>(); + if (predicate instanceof LogicalExpression + && ((LogicalExpression) predicate).getOperator() == LogicalExpression.Operator.AND) { + + for (Expression expression : ((LogicalExpression) predicate).getTerms()) { + if (checker.canPushDown(expression)) { + expressionsCanPushDown.add(expression); + } else { + expressionsCannotPushDown.add(expression); + } + } + + return new SplitExpression( + Collections.emptyList(), expressionsCanPushDown, expressionsCannotPushDown, null); + } + + if (checker.canPushDown(predicate)) { + expressionsCanPushDown.add(predicate); + } else { + expressionsCannotPushDown.add(predicate); + } + + return new SplitExpression( + Collections.emptyList(), expressionsCanPushDown, expressionsCannotPushDown, null); + } + private SplitExpression splitPredicate(DeviceTableScanNode node, Expression predicate) { Set idOrAttributeColumnNames = new HashSet<>(node.getAssignments().size()); Set timeOrMeasurementColumnNames = new HashSet<>(node.getAssignments().size()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java index 6e456e2220aab..95bb70872ff02 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/security/TreeAccessCheckVisitor.java @@ -152,6 +152,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.sys.SetSystemStatusStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowCurrentSqlDialectStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowCurrentUserStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowDiskUsageStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowVersionStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.StartRepairDataStatement; @@ -1790,6 +1791,15 @@ public TSStatus visitShowQueries(ShowQueriesStatement statement, TreeAccessCheck return SUCCEED; } + @Override + public TSStatus visitShowDiskUsage( + ShowDiskUsageStatement showDiskUsageStatement, TreeAccessCheckContext context) { + return checkGlobalAuth( + context.setAuditLogOperation(AuditLogOperation.QUERY), + PrivilegeType.SYSTEM, + () -> showDiskUsageStatement.getPathPattern().toString()); + } + @Override public TSStatus visitShowRegion(ShowRegionStatement statement, TreeAccessCheckContext context) { return checkGlobalAuth(context, PrivilegeType.MAINTAIN, () -> ""); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java index 7301ce0f406a2..87fc8f2e5eff0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java @@ -44,7 +44,9 @@ import org.apache.iotdb.db.queryengine.plan.scheduler.FragInstanceDispatchResult; import org.apache.iotdb.db.queryengine.plan.scheduler.IFragInstanceDispatcher; import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.utils.TableDiskUsageStatisticUtil; import org.apache.iotdb.db.utils.SetThreadName; import org.apache.iotdb.mpp.rpc.thrift.TLoadCommandReq; import org.apache.iotdb.mpp.rpc.thrift.TLoadResp; @@ -62,6 +64,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -182,13 +185,15 @@ public void dispatchLocally(FragmentInstance instance) throws FragmentInstanceDi cloneTsFileResource = tsFileResource.shallowClone(); } - StorageEngine.getInstance() - .getDataRegion((DataRegionId) groupId) - .loadNewTsFile( - cloneTsFileResource, - ((LoadSingleTsFileNode) planNode).isDeleteAfterLoad(), - isGeneratedByPipe, - false); + DataRegion dataRegion = StorageEngine.getInstance().getDataRegion((DataRegionId) groupId); + dataRegion.loadNewTsFile( + cloneTsFileResource, + ((LoadSingleTsFileNode) planNode).isDeleteAfterLoad(), + isGeneratedByPipe, + false, + dataRegion.isTableModel() + ? TableDiskUsageStatisticUtil.calculateTableSizeMap(cloneTsFileResource) + : Optional.empty()); } catch (LoadFileException e) { LOGGER.warn("Load TsFile Node {} error.", planNode, e); TSStatus resultStatus = new TSStatus(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementType.java index 721459ce84d37..19e597ef03197 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementType.java @@ -198,4 +198,6 @@ public enum StatementType { STOP_EXTERNAL_SERVICE, DROP_EXTERNAL_SERVICE, SHOW_EXTERNAL_SERVICE, + + SHOW_DISK_USAGE, } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java index 29d49453f4445..3617c4bae8d56 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/StatementVisitor.java @@ -144,6 +144,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowConfigurationStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowCurrentSqlDialectStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowCurrentUserStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowDiskUsageStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowVersionStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.StartRepairDataStatement; @@ -536,6 +537,10 @@ public R visitShowQueries(ShowQueriesStatement showQueriesStatement, C context) return visitStatement(showQueriesStatement, context); } + public R visitShowDiskUsage(ShowDiskUsageStatement showDiskUsageStatement, C context) { + return visitStatement(showDiskUsageStatement, context); + } + public R visitShowRegion(ShowRegionStatement showRegionStatement, C context) { return visitStatement(showRegionStatement, context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/OrderByComponent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/OrderByComponent.java index b6aa8c67c0a9d..236a939dc9aef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/OrderByComponent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/OrderByComponent.java @@ -67,7 +67,11 @@ public void addSortItem(SortItem sortItem) { case OrderByKey.DATANODEID: case OrderByKey.ELAPSEDTIME: case OrderByKey.STATEMENT: - // show queries statement + case OrderByKey.DATABASE: + case OrderByKey.REGIONID: + case OrderByKey.TIMEPARTITION: + case OrderByKey.SIZEINBYTES: + // show queries statement or show disk usage statement break; default: throw new IllegalArgumentException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/OrderByKey.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/OrderByKey.java index 7ee7002495fb9..6cd7296fd24cf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/OrderByKey.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/component/OrderByKey.java @@ -27,6 +27,11 @@ public class OrderByKey { public static final String DATANODEID = "DATANODEID"; public static final String ELAPSEDTIME = "ELAPSEDTIME"; public static final String STATEMENT = "STATEMENT"; + public static final String DATABASE = "DATABASE"; + public static final String REGIONID = "REGIONID"; + public static final String TIMEPARTITION = "TIMEPARTITION"; + public static final String TIMESERIESPARTITION = "TIMESERIESPARTITION"; + public static final String SIZEINBYTES = "SIZEINBYTES"; private OrderByKey() { // forbidding instantiation diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/sys/ShowDiskUsageStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/sys/ShowDiskUsageStatement.java new file mode 100644 index 0000000000000..313c8aabb00f9 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/sys/ShowDiskUsageStatement.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.statement.sys; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.queryengine.plan.statement.StatementType; +import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; +import org.apache.iotdb.db.queryengine.plan.statement.component.OrderByComponent; +import org.apache.iotdb.db.queryengine.plan.statement.component.SortItem; +import org.apache.iotdb.db.queryengine.plan.statement.component.WhereCondition; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.ShowStatement; + +import java.util.Collections; +import java.util.List; + +public class ShowDiskUsageStatement extends ShowStatement { + private final PartialPath pathPattern; + private WhereCondition whereCondition; + private OrderByComponent orderByComponent; + + public ShowDiskUsageStatement(PartialPath pathPattern) { + this.statementType = StatementType.SHOW_DISK_USAGE; + this.pathPattern = pathPattern; + } + + public PartialPath getPathPattern() { + return pathPattern; + } + + public void setWhereCondition(WhereCondition whereCondition) { + this.whereCondition = whereCondition; + } + + public WhereCondition getWhereCondition() { + return whereCondition; + } + + public void setOrderByComponent(OrderByComponent orderByComponent) { + this.orderByComponent = orderByComponent; + } + + public List getSortItemList() { + return orderByComponent == null ? Collections.emptyList() : orderByComponent.getSortItemList(); + } + + @Override + public boolean isQuery() { + return true; + } + + @Override + public R accept(StatementVisitor visitor, C context) { + return visitor.visitShowDiskUsage(this, context); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/WritingMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/WritingMetrics.java index c6251c4e6c846..9ee13e01b9915 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/WritingMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/WritingMetrics.java @@ -26,12 +26,14 @@ import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.flush.FlushManager; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; import org.apache.iotdb.db.storageengine.dataregion.wal.WALManager; import org.apache.iotdb.db.storageengine.dataregion.wal.checkpoint.CheckpointType; import org.apache.iotdb.db.storageengine.rescon.memory.SystemInfo; import org.apache.iotdb.metrics.AbstractMetricService; import org.apache.iotdb.metrics.impl.DoNothingMetricManager; import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.type.AutoGauge; import org.apache.iotdb.metrics.type.Counter; import org.apache.iotdb.metrics.type.Gauge; import org.apache.iotdb.metrics.type.Histogram; @@ -446,6 +448,7 @@ private void unbindWALCostMetrics(AbstractMetricService metricService) { public static final String WAL_FLUSH_MEMTABLE_COUNT = "wal_flush_memtable_count"; public static final String MANUAL_FLUSH_MEMTABLE_COUNT = "manual_flush_memtable_count"; public static final String MEM_CONTROL_FLUSH_MEMTABLE_COUNT = "mem_control_flush_memtable_count"; + public static final String BLOCKED_OPERATION_NUM = "blocked_operation_num"; private Gauge flushThreholdGauge = DoNothingMetricManager.DO_NOTHING_GAUGE; private Gauge rejectThreholdGauge = DoNothingMetricManager.DO_NOTHING_GAUGE; @@ -459,6 +462,9 @@ private void unbindWALCostMetrics(AbstractMetricService metricService) { private Histogram avgPointHistogram = DoNothingMetricManager.DO_NOTHING_HISTOGRAM; + private AutoGauge tableDiskUsageCacheBlockedRequestNumGauge = + DoNothingMetricManager.DO_NOTHING_AUTO_GAUGE; + public void bindDataRegionMetrics() { List allDataRegions = StorageEngine.getInstance().getAllDataRegions(); List allDataRegionIds = StorageEngine.getInstance().getAllDataRegionIds(); @@ -489,6 +495,16 @@ public void bindDataRegionMetrics() { memtableLiveTimer = MetricService.getInstance() .getOrCreateTimer(Metric.MEMTABLE_LIVE_DURATION.toString(), MetricLevel.IMPORTANT); + + tableDiskUsageCacheBlockedRequestNumGauge = + MetricService.getInstance() + .createAutoGauge( + Metric.TABLE_DISK_USAGE_CACHE.toString(), + MetricLevel.IMPORTANT, + TableDiskUsageCache.getInstance(), + TableDiskUsageCache::getQueueSize, + Tag.NAME.toString(), + BLOCKED_OPERATION_NUM); } public void unbindDataRegionMetrics() { @@ -517,6 +533,12 @@ public void unbindDataRegionMetrics() { Tag.TYPE.toString(), REJECT_THRESHOLD); MetricService.getInstance().remove(MetricType.TIMER, Metric.MEMTABLE_LIVE_DURATION.toString()); + MetricService.getInstance() + .remove( + MetricType.AUTO_GAUGE, + Metric.TABLE_DISK_USAGE_CACHE.toString(), + Tag.NAME.toString(), + BLOCKED_OPERATION_NUM); } public void createDataRegionMemoryCostMetrics(DataRegion dataRegion) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java index 964f6edcd763a..66d1d6d0d791b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java @@ -71,6 +71,7 @@ import org.apache.iotdb.db.storageengine.dataregion.flush.FlushListener; import org.apache.iotdb.db.storageengine.dataregion.flush.TsFileFlushPolicy; import org.apache.iotdb.db.storageengine.dataregion.flush.TsFileFlushPolicy.DirectFlushPolicy; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; import org.apache.iotdb.db.storageengine.dataregion.wal.WALManager; import org.apache.iotdb.db.storageengine.dataregion.wal.exception.WALException; import org.apache.iotdb.db.storageengine.dataregion.wal.recover.WALRecoverManager; @@ -419,6 +420,7 @@ public void stop() { if (cachedThreadPool != null) { cachedThreadPool.shutdownNow(); } + TableDiskUsageCache.getInstance().close(); dataRegionMap.clear(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index d95d51e390eb7..7fecc6b03428a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -137,6 +137,7 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.FileTimeIndexCacheRecorder; import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.ITimeIndex; import org.apache.iotdb.db.storageengine.dataregion.utils.fileTimeIndexCache.FileTimeIndexCacheReader; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; import org.apache.iotdb.db.storageengine.dataregion.utils.validate.TsFileValidator; import org.apache.iotdb.db.storageengine.dataregion.wal.WALManager; import org.apache.iotdb.db.storageengine.dataregion.wal.node.IWALNode; @@ -378,6 +379,8 @@ public class DataRegion implements IDataRegionForQuery { private ILoadDiskSelector ordinaryLoadDiskSelector; private ILoadDiskSelector pipeAndIoTV2LoadDiskSelector; + private final boolean isTableModel; + /** * Construct a database processor. * @@ -396,6 +399,7 @@ public DataRegion( this.dataRegionId = new DataRegionId(Integer.parseInt(dataRegionIdString)); this.databaseName = databaseName; this.fileFlushPolicy = fileFlushPolicy; + this.isTableModel = isTableModelDatabase(databaseName); acquireDirectBufferMemory(); dataRegionSysDir = SystemFileFactory.INSTANCE.getFile(systemDir, dataRegionIdString); @@ -413,6 +417,8 @@ public DataRegion( IoTDBThreadPoolFactory.newSingleThreadExecutor( databaseName + "-" + dataRegionIdString + "-UpgradeMod"); + TableDiskUsageCache.getInstance().registerRegion(this); + // recover tsfiles unless consensus protocol is ratis and storage engine is not ready if (config.getDataRegionConsensusProtocolClass().equals(ConsensusFactory.RATIS_CONSENSUS) && !StorageEngine.getInstance().isReadyForReadAndWrite()) { @@ -455,6 +461,7 @@ public DataRegion( @TestOnly public DataRegion(String databaseName, String dataRegionIdString) { this.databaseName = databaseName; + this.isTableModel = isTableModelDatabase(databaseName); this.dataRegionIdString = dataRegionIdString; this.dataRegionId = new DataRegionId(Integer.parseInt(this.dataRegionIdString)); this.tsFileManager = new TsFileManager(databaseName, dataRegionIdString, ""); @@ -498,6 +505,10 @@ public String getDatabaseName() { return databaseName; } + public boolean isTableModel() { + return isTableModel; + } + public boolean isReady() { return isReady; } @@ -1994,8 +2005,9 @@ public void deleteFolder(String systemDir) { "{} will close all files for deleting data folder {}", databaseName + "-" + dataRegionIdString, systemDir); - FileTimeIndexCacheRecorder.getInstance() - .removeFileTimeIndexCache(Integer.parseInt(dataRegionIdString)); + int regionId = dataRegionId.getId(); + TableDiskUsageCache.getInstance().remove(databaseName, regionId); + FileTimeIndexCacheRecorder.getInstance().removeFileTimeIndexCache(regionId); writeLock("deleteFolder"); try { File dataRegionSystemFolder = @@ -3825,7 +3837,8 @@ public void loadNewTsFile( final TsFileResource newTsFileResource, final boolean deleteOriginFile, final boolean isGeneratedByPipe, - final boolean isFromConsensus) + final boolean isFromConsensus, + final Optional> tableSizeMap) throws LoadFileException { if (DataRegionConsensusImpl.getInstance() instanceof IoTConsensus) { final IoTConsensusServerImpl impl = @@ -3895,6 +3908,11 @@ public void loadNewTsFile( deleteOriginFile, isGeneratedByPipe); + tableSizeMap.ifPresent( + stringLongMap -> + TableDiskUsageCache.getInstance() + .write(databaseName, newTsFileResource.getTsFileID(), stringLongMap)); + FileMetrics.getInstance() .addTsFile( newTsFileResource.getDatabaseName(), @@ -4333,6 +4351,10 @@ public String getDataRegionIdString() { return dataRegionIdString; } + public DataRegionId getDataRegionId() { + return dataRegionId; + } + /** * Get the storageGroupPath with dataRegionId. * diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java index 54b21ddd382fe..b6791830e64be 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/FastCompactionPerformer.java @@ -165,6 +165,7 @@ public void perform() throws Exception { ? new FastCrossCompactionWriter( targetFiles, seqFiles, readerCacheMap, encryptParameter) : new FastInnerCompactionWriter(targetFiles, encryptParameter)) { + compactionWriter.setCompactionTaskSummary(subTaskSummary); List schemas = CompactionTableSchemaCollector.collectSchema( seqFiles, unseqFiles, readerCacheMap, deviceIterator.getDeprecatedTableSchemaMap()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadChunkCompactionPerformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadChunkCompactionPerformer.java index d406286e37f64..3bccceeddfbd6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadChunkCompactionPerformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadChunkCompactionPerformer.java @@ -210,6 +210,8 @@ private void useNewWriter() throws IOException { memoryBudgetForFileWriter, CompactionType.INNER_SEQ_COMPACTION, firstEncryptParameter); + summary.recordTargetTsFileTableSizeMap( + targetResources.get(currentTargetFileIndex), currentWriter.getTableSizeMap()); currentWriter.setSchema(CompactionTableSchemaCollector.copySchema(schema)); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadPointCompactionPerformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadPointCompactionPerformer.java index c58870357d915..d4c075da5293e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadPointCompactionPerformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/performer/impl/ReadPointCompactionPerformer.java @@ -150,6 +150,7 @@ public void perform() throws Exception { summary.setTemporalFileNum(targetFiles.size()); try (AbstractCompactionWriter compactionWriter = getCompactionWriter(seqFiles, unseqFiles, targetFiles)) { + compactionWriter.setCompactionTaskSummary(summary); // Do not close device iterator, because tsfile reader is managed by FileReaderManager. MultiTsFileDeviceIterator deviceIterator = new MultiTsFileDeviceIterator(seqFiles, unseqFiles); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CompactionTaskSummary.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CompactionTaskSummary.java index b9814d86701e9..d30f37c9c950d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CompactionTaskSummary.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CompactionTaskSummary.java @@ -19,8 +19,13 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + import java.text.SimpleDateFormat; import java.util.Date; +import java.util.HashMap; +import java.util.Map; /** The summary of one {@link AbstractCompactionTask} execution. */ public class CompactionTaskSummary { @@ -37,6 +42,7 @@ public class CompactionTaskSummary { protected long rewritePointNum = 0; protected long temporalFileSize = 0; protected int temporalFileNum = 0; + protected Map> targetFileTableSizeMap = new HashMap<>(); public void start() { this.status = Status.STARTED; @@ -189,6 +195,15 @@ public int getTemporalFileNum() { return temporalFileNum; } + public void recordTargetTsFileTableSizeMap( + TsFileResource resource, Map tableSizeMap) { + this.targetFileTableSizeMap.put(resource.getTsFileID(), tableSizeMap); + } + + public Map getTableSizeMapOfTargetResource(TsFileID targetTsFileId) { + return targetFileTableSizeMap.get(targetTsFileId); + } + @Override public String toString() { String startTimeInStr = new SimpleDateFormat().format(new Date(startTime)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CrossSpaceCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CrossSpaceCompactionTask.java index 41d859924f073..a7a3c0b6b9058 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CrossSpaceCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/CrossSpaceCompactionTask.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task; import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.service.metrics.CompactionMetrics; import org.apache.iotdb.db.service.metrics.FileMetrics; @@ -38,6 +39,7 @@ 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.generator.TsFileNameGenerator; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; import org.apache.tsfile.utils.TsFileUtils; import org.slf4j.Logger; @@ -236,6 +238,8 @@ public boolean doCompaction() { } } + updateTableSizeCache(); + CompactionMetrics.getInstance().recordSummaryInfo(summary); double costTime = (System.currentTimeMillis() - startTime) / 1000.0d; @@ -273,6 +277,21 @@ public boolean doCompaction() { return isSuccess; } + protected void updateTableSizeCache() { + if (!PathUtils.isTableModelDatabase(this.storageGroupName)) { + return; + } + for (TsFileResource resource : targetTsfileResourceList) { + if (!resource.isDeleted()) { + TableDiskUsageCache.getInstance() + .write( + storageGroupName, + resource.getTsFileID(), + summary.getTableSizeMapOfTargetResource(resource.getTsFileID())); + } + } + } + public void recover() { try { if (needRecoverTaskInfoFromLogFile) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java index c61d2275ac1af..a4b4cf497214d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InnerSpaceCompactionTask.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task; +import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; @@ -43,6 +44,7 @@ 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.generator.TsFileNameGenerator; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.exception.StopReadTsFileByInterruptException; @@ -432,7 +434,7 @@ protected void compact(SimpleCompactionLogger compactionLogger) throws Exception CompactionUtils.deleteSourceTsFileAndUpdateFileMetrics( filesView.sourceFilesInLog, filesView.sequence); - + updateTableSizeCache(); CompactionMetrics.getInstance().recordSummaryInfo(summary); } @@ -458,6 +460,28 @@ protected void prepareTargetFiles() throws IOException { filesView.sourceFilesInCompactionPerformer, filesView.targetFilesInPerformer); } + protected void updateTableSizeCache() { + if (!PathUtils.isTableModelDatabase(this.storageGroupName)) { + return; + } + for (int i = 0; i < filesView.renamedTargetFiles.size(); i++) { + TableDiskUsageCache.getInstance() + .write( + this.storageGroupName, + filesView.skippedSourceFiles.get(i).getTsFileID(), + filesView.renamedTargetFiles.get(i).getTsFileID()); + } + for (TsFileResource resource : filesView.targetFilesInPerformer) { + if (!resource.isDeleted()) { + TableDiskUsageCache.getInstance() + .write( + this.storageGroupName, + resource.getTsFileID(), + summary.getTableSizeMapOfTargetResource(resource.getTsFileID())); + } + } + } + public void recover() { try { if (needRecoverTaskInfoFromLogFile) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InsertionCrossSpaceCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InsertionCrossSpaceCompactionTask.java index c2aee2bde9016..d89c722f6f4b5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InsertionCrossSpaceCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InsertionCrossSpaceCompactionTask.java @@ -33,6 +33,7 @@ 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.generator.TsFileNameGenerator; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; import java.io.File; import java.io.IOException; @@ -166,6 +167,9 @@ protected boolean doCompaction() { lockWrite(Collections.singletonList(unseqFileToInsert)); CompactionUtils.deleteTsFileResourceWithoutLock(unseqFileToInsert); + TableDiskUsageCache.getInstance() + .write(storageGroupName, unseqFileToInsert.getTsFileID(), targetFile.getTsFileID()); + double costTime = (System.currentTimeMillis() - startTime) / 1000.0d; LOGGER.info( "{}-{} [Compaction] InsertionCrossSpaceCompaction task finishes successfully, " diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCompactionWriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCompactionWriter.java index f3cd5185b58af..96ab986552d91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCompactionWriter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCompactionWriter.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.exception.CompactionLastTimeCheckFailedException; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.CompactionTaskSummary; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.fast.element.AlignedPageElement; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.executor.fast.element.ChunkMetadataElement; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.writer.flushcontroller.AbstractCompactionFlushController; @@ -105,6 +106,8 @@ public abstract class AbstractCompactionWriter implements AutoCloseable { private EncryptParameter encryptParameter; + protected CompactionTaskSummary compactionTaskSummary; + public abstract void startChunkGroup(IDeviceID deviceId, boolean isAlign) throws IOException; public abstract void endChunkGroup() throws IOException; @@ -340,4 +343,8 @@ protected void checkPreviousTimestamp(long currentWritingTimestamp, int subTaskI } public abstract void setSchemaForAllTargetFile(List schemas); + + public void setCompactionTaskSummary(CompactionTaskSummary compactionTaskSummary) { + this.compactionTaskSummary = compactionTaskSummary; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCrossCompactionWriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCrossCompactionWriter.java index f970ad65e56c3..61cec47d39a24 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCrossCompactionWriter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractCrossCompactionWriter.java @@ -171,6 +171,9 @@ public void endFile() throws IOException { // set empty target file to DELETED if (isEmptyFile[i]) { targetResources.get(i).forceMarkDeleted(); + } else if (compactionTaskSummary != null) { + compactionTaskSummary.recordTargetTsFileTableSizeMap( + targetResources.get(i), targetFileWriters.get(i).getTableSizeMap()); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractInnerCompactionWriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractInnerCompactionWriter.java index 6573bb7e96e86..0ace4d5cb9735 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractInnerCompactionWriter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/writer/AbstractInnerCompactionWriter.java @@ -121,6 +121,10 @@ private void useNewWriter() throws IOException { ? CompactionType.INNER_SEQ_COMPACTION : CompactionType.INNER_UNSEQ_COMPACTION, encryptParameter); + if (compactionTaskSummary != null) { + compactionTaskSummary.recordTargetTsFileTableSizeMap( + targetResources.get(currentFileIndex), fileWriter.getTableSizeMap()); + } fileWriter.setSchema(CompactionTableSchemaCollector.copySchema(schemas.get(0))); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index acdac61180bf0..91511ad745f06 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -66,6 +66,7 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.FileTimeIndexCacheRecorder; import org.apache.iotdb.db.storageengine.dataregion.utils.SharedTimeDataBuffer; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; import org.apache.iotdb.db.storageengine.dataregion.wal.WALManager; import org.apache.iotdb.db.storageengine.dataregion.wal.node.IWALNode; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.listener.AbstractResultListener; @@ -1753,6 +1754,11 @@ private void endFile() throws IOException, TsFileProcessorException { tsFileResource.serialize(); FileTimeIndexCacheRecorder.getInstance().logFileTimeIndex(tsFileResource); + TableDiskUsageCache.getInstance() + .write( + tsFileResource.getDatabaseName(), + tsFileResource.getTsFileID(), + writer.getTableSizeMap()); if (logger.isDebugEnabled()) { logger.debug("Ended file {}", tsFileResource); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileID.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileID.java index c9656382e3d68..e654cbebe9c5c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileID.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileID.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.storageengine.dataregion.tsfile; +import org.apache.tsfile.utils.RamUsageEstimator; + import java.util.Objects; import static org.apache.iotdb.commons.conf.IoTDBConstant.FILE_NAME_SEPARATOR; @@ -26,6 +28,8 @@ public class TsFileID { + public static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(TsFileID.class); + public final int regionId; public final long timePartitionId; public final long timestamp; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java index b7c1ba2c14fb4..86890370c313b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java @@ -146,6 +146,18 @@ public List getTsFileListSnapshot(long timePartition, boolean se } } + public Pair, List> getTsFileListSnapshot( + long timePartition) { + readLock(); + try { + return new Pair<>( + new ArrayList<>(sequenceFiles.getOrDefault(timePartition, new TsFileResourceList())), + new ArrayList<>(unsequenceFiles.getOrDefault(timePartition, new TsFileResourceList()))); + } finally { + readUnlock(); + } + } + public List getTsFileList(boolean sequence, long startTime, long endTime) { // the iteration of ConcurrentSkipListMap is not concurrent secure // so we must add read lock here diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index b84cce9e8d21b..4a0f1e4b72c7e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -101,7 +101,7 @@ public class TsFileResource implements PersistentResource, Cloneable { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(TsFileResource.class) + RamUsageEstimator.shallowSizeOfInstance(TsFileRepairStatus.class) - + RamUsageEstimator.shallowSizeOfInstance(TsFileID.class); + + TsFileID.SHALLOW_SIZE; private static final Logger LOGGER = LoggerFactory.getLogger(TsFileResource.class); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/DiskUsageStatisticUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/DiskUsageStatisticUtil.java new file mode 100644 index 0000000000000..a913e417c8522 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/DiskUsageStatisticUtil.java @@ -0,0 +1,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.IMetadataIndexEntry; +import org.apache.tsfile.file.header.ChunkGroupHeader; +import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.MetadataIndexNode; +import org.apache.tsfile.file.metadata.enums.MetadataIndexNodeType; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Queue; +import java.util.function.LongConsumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public abstract class DiskUsageStatisticUtil implements Closeable { + + protected static final Logger logger = LoggerFactory.getLogger(DiskUsageStatisticUtil.class); + protected Queue resourcesWithReadLock; + protected final long timePartition; + protected final Iterator iterator; + protected final LongConsumer timeSeriesMetadataIoSizeRecorder; + protected final LongConsumer timeSeriesMetadataCountRecorder; + + public DiskUsageStatisticUtil( + TsFileManager tsFileManager, + long timePartition, + Optional fragmentInstanceContext) { + this.timePartition = timePartition; + this.timeSeriesMetadataIoSizeRecorder = + fragmentInstanceContext + .map( + context -> + context.getQueryStatistics().getLoadTimeSeriesMetadataActualIOSize()::addAndGet) + .orElse(null); + this.timeSeriesMetadataCountRecorder = + fragmentInstanceContext + .map( + context -> + context.getQueryStatistics().getLoadTimeSeriesMetadataFromDiskCount() + ::addAndGet) + .orElse(null); + List seqResources = tsFileManager.getTsFileListSnapshot(timePartition, true); + List unseqResources = tsFileManager.getTsFileListSnapshot(timePartition, false); + List resources = + Stream.concat(seqResources.stream(), unseqResources.stream()).collect(Collectors.toList()); + acquireReadLocks(resources); + iterator = resourcesWithReadLock.iterator(); + } + + public long getTimePartition() { + return timePartition; + } + + public boolean hasNextFile() { + return iterator.hasNext(); + } + + protected void acquireReadLocks(List resources) { + this.resourcesWithReadLock = new LinkedList<>(); + try { + for (TsFileResource resource : resources) { + if (!resource.isClosed()) { + continue; + } + resource.readLock(); + if (resource.isDeleted() || !resource.isClosed()) { + resource.readUnlock(); + continue; + } + resourcesWithReadLock.add(resource); + } + } catch (Exception e) { + releaseReadLocks(); + throw e; + } + } + + protected void releaseReadLocks() { + if (resourcesWithReadLock == null) { + return; + } + for (TsFileResource resource : resourcesWithReadLock) { + resource.readUnlock(); + } + resourcesWithReadLock = null; + } + + public void calculateNextFile() { + TsFileResource tsFileResource = iterator.next(); + if (tsFileResource.isDeleted() || calculateWithoutOpenFile(tsFileResource)) { + iterator.remove(); + tsFileResource.readUnlock(); + return; + } + FileReaderManager.getInstance().increaseFileReaderReference(tsFileResource, true); + try { + TsFileSequenceReader reader = + FileReaderManager.getInstance() + .get( + tsFileResource.getTsFilePath(), + tsFileResource.getTsFileID(), + true, + timeSeriesMetadataIoSizeRecorder); + calculateNextFile(tsFileResource, reader); + } catch (Exception e) { + logger.error("Failed to scan file {}", tsFileResource.getTsFile().getAbsolutePath(), e); + } finally { + // this operation including readUnlock + FileReaderManager.getInstance().decreaseFileReaderReference(tsFileResource, true); + iterator.remove(); + } + } + + protected abstract boolean calculateWithoutOpenFile(TsFileResource tsFileResource); + + protected abstract void calculateNextFile( + TsFileResource tsFileResource, TsFileSequenceReader reader) + throws IOException, IllegalPathException; + + protected static Offsets calculateStartOffsetOfChunkGroupAndTimeseriesMetadata( + TsFileSequenceReader reader, + MetadataIndexNode firstMeasurementNodeOfCurrentDevice, + Pair deviceIsAlignedPair, + long rootMeasurementNodeStartOffset, + LongConsumer timeSeriesMetadataCountRecorder, + LongConsumer timeSeriesMetadataIoSizeRecorder) + throws IOException { + int chunkGroupHeaderSize = + new ChunkGroupHeader(deviceIsAlignedPair.getLeft()).getSerializedSize(); + if (deviceIsAlignedPair.getRight()) { + Pair timeseriesMetadataOffsetPair = + getTimeColumnMetadataOffset( + reader, firstMeasurementNodeOfCurrentDevice, timeSeriesMetadataIoSizeRecorder); + IChunkMetadata firstChunkMetadata = + reader + .getChunkMetadataListByTimeseriesMetadataOffset( + timeseriesMetadataOffsetPair.getLeft(), timeseriesMetadataOffsetPair.getRight()) + .get(0); + if (timeSeriesMetadataCountRecorder != null) { + timeSeriesMetadataIoSizeRecorder.accept( + timeseriesMetadataOffsetPair.getRight() - timeseriesMetadataOffsetPair.getLeft()); + timeSeriesMetadataCountRecorder.accept(1); + } + return new Offsets( + firstChunkMetadata.getOffsetOfChunkHeader() - chunkGroupHeaderSize, + timeseriesMetadataOffsetPair.getLeft(), + rootMeasurementNodeStartOffset); + } else { + Map, Pair>> timeseriesMetadataOffsetByDevice = + reader.getTimeseriesMetadataOffsetByDevice( + firstMeasurementNodeOfCurrentDevice, Collections.emptySet(), true); + long minTimeseriesMetadataOffset = 0; + long minChunkOffset = Long.MAX_VALUE; + for (Map.Entry, Pair>> entry : + timeseriesMetadataOffsetByDevice.entrySet()) { + minTimeseriesMetadataOffset = + minTimeseriesMetadataOffset == 0 + ? entry.getValue().getRight().getLeft() + : minTimeseriesMetadataOffset; + if (timeSeriesMetadataIoSizeRecorder != null) { + timeSeriesMetadataIoSizeRecorder.accept( + entry.getValue().getRight().getRight() - entry.getValue().getRight().getLeft()); + timeSeriesMetadataCountRecorder.accept(1); + } + for (IChunkMetadata chunkMetadata : entry.getValue().getLeft()) { + minChunkOffset = Math.min(minChunkOffset, chunkMetadata.getOffsetOfChunkHeader()); + break; + } + } + return new Offsets( + minChunkOffset - chunkGroupHeaderSize, + minTimeseriesMetadataOffset, + rootMeasurementNodeStartOffset); + } + } + + private static Pair getTimeColumnMetadataOffset( + TsFileSequenceReader reader, + MetadataIndexNode measurementNode, + LongConsumer timeSeriesMetadataIoSizeRecorder) + throws IOException { + if (measurementNode.isDeviceLevel()) { + throw new IllegalArgumentException("device level metadata index node is not supported"); + } + List children = measurementNode.getChildren(); + long startOffset = children.get(0).getOffset(); + long endOffset = + children.size() > 1 ? children.get(1).getOffset() : measurementNode.getEndOffset(); + if (measurementNode.getNodeType().equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) { + return new Pair<>(startOffset, endOffset); + } else { + MetadataIndexNode metadataIndexNode = + reader.readMetadataIndexNode( + startOffset, endOffset, false, timeSeriesMetadataIoSizeRecorder); + return getTimeColumnMetadataOffset( + reader, metadataIndexNode, timeSeriesMetadataIoSizeRecorder); + } + } + + @Override + public void close() { + releaseReadLocks(); + } + + protected static class Offsets { + protected final long firstChunkOffset; + protected final long firstTimeseriesMetadataOffset; + protected final long firstMeasurementNodeOffset; + + public Offsets( + long firstChunkOffset, + long firstTimeseriesMetadataOffset, + long firstMeasurementNodeOffset) { + this.firstChunkOffset = firstChunkOffset; + this.firstTimeseriesMetadataOffset = firstTimeseriesMetadataOffset; + this.firstMeasurementNodeOffset = firstMeasurementNodeOffset; + } + + protected long minusOffsetForTableModel(Offsets other) { + return firstChunkOffset + - other.firstChunkOffset + + firstTimeseriesMetadataOffset + - other.firstTimeseriesMetadataOffset + + firstMeasurementNodeOffset + - other.firstMeasurementNodeOffset; + } + + protected long minusOffsetForTreeModel(Offsets other) { + return firstChunkOffset + - other.firstChunkOffset + + firstTimeseriesMetadataOffset + - other.firstTimeseriesMetadataOffset; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/StorageEngineTimePartitionIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/StorageEngineTimePartitionIterator.java new file mode 100644 index 0000000000000..a7a0096f2bcac --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/StorageEngineTimePartitionIterator.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils; + +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; + +import org.apache.tsfile.utils.Accountable; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.util.Iterator; +import java.util.Optional; + +public class StorageEngineTimePartitionIterator implements Accountable { + public static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(StorageEngineTimePartitionIterator.class); + + private final Iterator dataRegionIterator; + private final Optional dataRegionFilter; + private final Optional timePartitionFilter; + private DataRegion currentDataRegion; + private long currentTimePartition; + private Iterator timePartitionIterator; + + public StorageEngineTimePartitionIterator( + Optional dataRegionFilter, + Optional timePartitionFilter) { + this.dataRegionIterator = StorageEngine.getInstance().getAllDataRegions().iterator(); + this.dataRegionFilter = dataRegionFilter; + this.timePartitionFilter = timePartitionFilter; + } + + public boolean nextTimePartition() throws Exception { + while (true) { + if (timePartitionIterator != null && timePartitionIterator.hasNext()) { + currentTimePartition = timePartitionIterator.next(); + if (timePartitionFilter.isPresent() + && !timePartitionFilter.get().apply(currentDataRegion, currentTimePartition)) { + continue; + } + return true; + } else if (!nextDataRegion()) { + return false; + } // should not have else branch + } + } + + public boolean nextDataRegion() throws Exception { + while (dataRegionIterator.hasNext()) { + currentDataRegion = dataRegionIterator.next(); + if (currentDataRegion == null || currentDataRegion.isDeleted()) { + continue; + } + if (dataRegionFilter.isPresent() && !dataRegionFilter.get().apply(currentDataRegion)) { + continue; + } + timePartitionIterator = currentDataRegion.getTimePartitions().iterator(); + if (timePartitionIterator.hasNext()) { + return true; + } + } + return false; + } + + public DataRegion currentDataRegion() { + return currentDataRegion; + } + + public long currentTimePartition() { + return currentTimePartition; + } + + @Override + public long ramBytesUsed() { + return SHALLOW_SIZE; + } + + @FunctionalInterface + public interface DataRegionFilterFunc { + boolean apply(DataRegion currentDataRegion) throws Exception; + } + + @FunctionalInterface + public interface TimePartitionFilterFunc { + boolean apply(DataRegion currentDataRegion, long currentTimePartition) throws Exception; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TableDiskUsageStatisticUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TableDiskUsageStatisticUtil.java new file mode 100644 index 0000000000000..7357cf86bfc55 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TableDiskUsageStatisticUtil.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils; + +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TimePartitionTableSizeQueryContext; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.MetadataIndexNode; +import org.apache.tsfile.file.metadata.TsFileMetadata; +import org.apache.tsfile.read.TsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.RamUsageEstimator; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.LongConsumer; + +public class TableDiskUsageStatisticUtil extends DiskUsageStatisticUtil { + public static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(TableDiskUsageStatisticUtil.class); + private final String database; + private final List> tsFilesToQueryInCache; + private final TimePartitionTableSizeQueryContext tableSizeQueryContext; + private final boolean databaseHasOnlyOneTable; + private final boolean needAllData; + + public TableDiskUsageStatisticUtil( + DataRegion dataRegion, + long timePartition, + TimePartitionTableSizeQueryContext tableSizeQueryContext, + boolean needAllData, + boolean databaseHasOnlyOneTable, + List> tsFilesToQueryInCache, + Optional context) { + super(dataRegion.getTsFileManager(), timePartition, context); + this.database = dataRegion.getDatabaseName(); + this.tableSizeQueryContext = tableSizeQueryContext; + this.tsFilesToQueryInCache = tsFilesToQueryInCache; + this.databaseHasOnlyOneTable = databaseHasOnlyOneTable; + this.needAllData = needAllData; + } + + @Override + protected boolean calculateWithoutOpenFile(TsFileResource tsFileResource) { + TsFileID tsFileID = tsFileResource.getTsFileID(); + Long cachedValueOffset = tableSizeQueryContext.getCachedTsFileIdOffset(tsFileID); + if (cachedValueOffset != null) { + tsFilesToQueryInCache.add(new Pair<>(tsFileID, cachedValueOffset)); + return true; + } + + if (!databaseHasOnlyOneTable || tsFileResource.anyModFileExists()) { + return false; + } + String table = tableSizeQueryContext.getTableSizeResultMap().keySet().iterator().next(); + tableSizeQueryContext.updateResult(table, tsFileResource.getTsFileSize(), needAllData); + TableDiskUsageCache.getInstance() + .write( + database, + tsFileResource.getTsFileID(), + Collections.singletonMap(table, tsFileResource.getTsFileSize())); + return true; + } + + @Override + protected void calculateNextFile(TsFileResource tsFileResource, TsFileSequenceReader reader) + throws IOException { + TsFileMetadata tsFileMetadata = reader.readFileMetadata(); + if (!needAllData && !hasSatisfiedData(tsFileMetadata)) { + return; + } + + if (tsFileMetadata.getTableMetadataIndexNodeMap().size() == 1) { + String satisfiedTable = + tsFileMetadata.getTableMetadataIndexNodeMap().keySet().iterator().next(); + tableSizeQueryContext.updateResult( + satisfiedTable, tsFileResource.getTsFileSize(), needAllData); + TableDiskUsageCache.getInstance() + .write( + database, + tsFileResource.getTsFileID(), + Collections.singletonMap( + tsFileMetadata.getTableMetadataIndexNodeMap().keySet().iterator().next(), + tsFileResource.getTsFileSize())); + return; + } + + calculateDiskUsageInBytesByOffset(tsFileResource, reader); + } + + private boolean hasSatisfiedData(TsFileMetadata tsFileMetadata) { + Map tableMetadataIndexNodeMap = + tsFileMetadata.getTableMetadataIndexNodeMap(); + return tableSizeQueryContext.getTableSizeResultMap().keySet().stream() + .anyMatch(tableMetadataIndexNodeMap::containsKey); + } + + private void calculateDiskUsageInBytesByOffset( + TsFileResource resource, TsFileSequenceReader reader) throws IOException { + Map tsFileTableSizeMap = + calculateTableSizeMap( + reader, timeSeriesMetadataCountRecorder, timeSeriesMetadataIoSizeRecorder); + for (Map.Entry entry : tsFileTableSizeMap.entrySet()) { + tableSizeQueryContext.updateResult(entry.getKey(), entry.getValue(), needAllData); + } + TableDiskUsageCache.getInstance().write(database, resource.getTsFileID(), tsFileTableSizeMap); + } + + public static Optional> calculateTableSizeMap(TsFileResource resource) { + if (!resource.getTsFile().exists()) { + return Optional.empty(); + } + try (TsFileSequenceReader reader = new TsFileSequenceReader(resource.getTsFilePath())) { + return Optional.of(calculateTableSizeMap(reader, null, null)); + } catch (Exception e) { + logger.error("Failed to calculate tsfile table sizes", e); + return Optional.empty(); + } + } + + public static Map calculateTableSizeMap( + TsFileSequenceReader reader, + @Nullable LongConsumer timeSeriesMetadataCountRecorder, + @Nullable LongConsumer timeSeriesMetadataIoSizeRecorder) + throws IOException { + TsFileMetadata tsFileMetadata = reader.readFileMetadata(); + Map tableMetadataIndexNodeMap = + tsFileMetadata.getTableMetadataIndexNodeMap(); + String currentTable = null, nextTable = null; + Iterator iterator = tableMetadataIndexNodeMap.keySet().iterator(); + Map tableOffsetMap = new HashMap<>(); + Map tsFileTableSizeMap = new HashMap<>(); + while (currentTable != null || iterator.hasNext()) { + currentTable = currentTable == null ? iterator.next() : currentTable; + nextTable = iterator.hasNext() ? iterator.next() : null; + long tableSize = + calculateTableSize( + tableOffsetMap, + tsFileMetadata, + reader, + currentTable, + nextTable, + timeSeriesMetadataCountRecorder, + timeSeriesMetadataIoSizeRecorder); + tsFileTableSizeMap.put(currentTable, tableSize); + currentTable = nextTable; + } + return tsFileTableSizeMap; + } + + private static long calculateTableSize( + Map tableOffsetMap, + TsFileMetadata tsFileMetadata, + TsFileSequenceReader reader, + String tableName, + String nextTable, + LongConsumer timeSeriesMetadataCountRecorder, + LongConsumer timeSeriesMetadataIoSizeRecorder) + throws IOException { + Offsets startOffset = + getTableOffset( + tableOffsetMap, + reader, + tableName, + timeSeriesMetadataCountRecorder, + timeSeriesMetadataIoSizeRecorder); + Offsets endOffset; + if (nextTable == null) { + long firstMeasurementNodeOffsetOfFirstTable; + String firstTableName = + tsFileMetadata.getTableMetadataIndexNodeMap().keySet().iterator().next(); + Offsets firstTableOffset = tableOffsetMap.get(firstTableName); + if (firstTableOffset != null) { + firstMeasurementNodeOffsetOfFirstTable = firstTableOffset.firstMeasurementNodeOffset; + } else { + TsFileDeviceIterator deviceIterator = + reader.getTableDevicesIteratorWithIsAligned( + tableName, timeSeriesMetadataIoSizeRecorder); + deviceIterator.next(); + firstMeasurementNodeOffsetOfFirstTable = + deviceIterator.getCurrentDeviceMeasurementNodeOffset()[0]; + } + endOffset = + new Offsets( + tsFileMetadata.getMetaOffset(), + firstMeasurementNodeOffsetOfFirstTable, + reader.getFileMetadataPos()); + } else { + endOffset = + getTableOffset( + tableOffsetMap, + reader, + nextTable, + timeSeriesMetadataCountRecorder, + timeSeriesMetadataIoSizeRecorder); + } + return endOffset.minusOffsetForTableModel(startOffset); + } + + private static Offsets getTableOffset( + Map tableOffsetMap, + TsFileSequenceReader reader, + String tableName, + LongConsumer timeSeriesMetadataCountRecorder, + LongConsumer timeSeriesMetadataIoSizeRecorder) { + return tableOffsetMap.computeIfAbsent( + tableName, + k -> { + try { + TsFileDeviceIterator deviceIterator = + reader.getTableDevicesIteratorWithIsAligned(k, timeSeriesMetadataIoSizeRecorder); + Pair pair = deviceIterator.next(); + return calculateStartOffsetOfChunkGroupAndTimeseriesMetadata( + reader, + deviceIterator.getFirstMeasurementNodeOfCurrentDevice(), + pair, + deviceIterator.getCurrentDeviceMeasurementNodeOffset()[0], + timeSeriesMetadataCountRecorder, + timeSeriesMetadataIoSizeRecorder); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TreeDiskUsageStatisticUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TreeDiskUsageStatisticUtil.java new file mode 100644 index 0000000000000..ec4d8f03d8683 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/TreeDiskUsageStatisticUtil.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils; + +import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.CompactionPathUtils; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.MetadataIndexNode; +import org.apache.tsfile.read.LazyTsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.io.IOException; +import java.util.Optional; + +public class TreeDiskUsageStatisticUtil extends DiskUsageStatisticUtil { + + public static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOfInstance(TreeDiskUsageStatisticUtil.class); + + private final PartialPath pathPattern; + private final boolean isMatchedDeviceSequential; + private long result; + + public TreeDiskUsageStatisticUtil( + TsFileManager tsFileManager, + long timePartition, + PartialPath pathPattern, + Optional context) { + super(tsFileManager, timePartition, context); + this.pathPattern = pathPattern; + this.result = 0; + String[] nodes = pathPattern.getNodes(); + boolean hasWildcardInPath = false; + for (int i = 0; i < nodes.length; i++) { + if (nodes[i].equals(IoTDBConstant.ONE_LEVEL_PATH_WILDCARD) + || nodes[i].equals(IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD)) { + hasWildcardInPath = true; + continue; + } + if (hasWildcardInPath) { + this.isMatchedDeviceSequential = false; + return; + } + } + this.isMatchedDeviceSequential = true; + } + + public long[] getResult() { + return new long[] {result}; + } + + @Override + protected boolean calculateWithoutOpenFile(TsFileResource tsFileResource) { + return false; + } + + @Override + protected void calculateNextFile(TsFileResource tsFileResource, TsFileSequenceReader reader) + throws IOException, IllegalPathException { + long firstDeviceMeasurementNodeOffset = -1; + LazyTsFileDeviceIterator deviceIterator = reader.getLazyDeviceIterator(); + while (deviceIterator.hasNext()) { + IDeviceID deviceIsAlignedPair = deviceIterator.next(); + firstDeviceMeasurementNodeOffset = + firstDeviceMeasurementNodeOffset == -1 + ? deviceIterator.getCurrentDeviceMeasurementNodeOffset()[0] + : firstDeviceMeasurementNodeOffset; + if (!matchPathPattern(deviceIsAlignedPair)) { + continue; + } + MetadataIndexNode nodeOfFirstMatchedDevice = + deviceIterator.getFirstMeasurementNodeOfCurrentDevice(); + addMeasurementNodeSizeForCurrentDevice(deviceIterator); + IDeviceID nextNotMatchedDevice = null; + MetadataIndexNode nodeOfNextNotMatchedDevice = null; + while (deviceIterator.hasNext()) { + IDeviceID currentDevice = deviceIterator.next(); + if (!matchPathPattern(currentDevice)) { + nextNotMatchedDevice = currentDevice; + nodeOfNextNotMatchedDevice = deviceIterator.getFirstMeasurementNodeOfCurrentDevice(); + break; + } + addMeasurementNodeSizeForCurrentDevice(deviceIterator); + } + result += + calculatePathPatternSize( + reader, + new Pair<>(deviceIsAlignedPair, reader.isAlignedDevice(nodeOfFirstMatchedDevice)), + nodeOfFirstMatchedDevice, + nodeOfNextNotMatchedDevice == null + ? null + : new Pair<>( + nextNotMatchedDevice, reader.isAlignedDevice(nodeOfNextNotMatchedDevice)), + nodeOfNextNotMatchedDevice, + firstDeviceMeasurementNodeOffset); + if (isMatchedDeviceSequential) { + break; + } + } + } + + private void addMeasurementNodeSizeForCurrentDevice(LazyTsFileDeviceIterator iterator) { + long[] startEndPair = iterator.getCurrentDeviceMeasurementNodeOffset(); + result += startEndPair[1] - startEndPair[0]; + } + + private long calculatePathPatternSize( + TsFileSequenceReader reader, + Pair firstMatchedDevice, + MetadataIndexNode nodeOfFirstMatchedDevice, + Pair nextNotMatchedDevice, + MetadataIndexNode nodeOfNextNotMatchedDevice, + long firstDeviceMeasurementNodeOffset) + throws IOException { + Offsets chunkGroupTimeseriesMetadataStartOffsetPair, chunkGroupTimeseriesMetadataEndOffsetPair; + if (nextNotMatchedDevice == null) { + chunkGroupTimeseriesMetadataEndOffsetPair = + new Offsets( + reader.readFileMetadata().getMetaOffset(), firstDeviceMeasurementNodeOffset, 0); + } else { + chunkGroupTimeseriesMetadataEndOffsetPair = + calculateStartOffsetOfChunkGroupAndTimeseriesMetadata( + reader, + nodeOfNextNotMatchedDevice, + nextNotMatchedDevice, + 0, + timeSeriesMetadataCountRecorder, + timeSeriesMetadataIoSizeRecorder); + } + chunkGroupTimeseriesMetadataStartOffsetPair = + calculateStartOffsetOfChunkGroupAndTimeseriesMetadata( + reader, + nodeOfFirstMatchedDevice, + firstMatchedDevice, + 0, + timeSeriesMetadataCountRecorder, + timeSeriesMetadataIoSizeRecorder); + return chunkGroupTimeseriesMetadataEndOffsetPair.minusOffsetForTreeModel( + chunkGroupTimeseriesMetadataStartOffsetPair); + } + + private boolean matchPathPattern(IDeviceID deviceID) throws IllegalPathException { + return pathPattern.matchFullPath(CompactionPathUtils.getPath(deviceID)); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/AbstractTableSizeCacheWriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/AbstractTableSizeCacheWriter.java new file mode 100644 index 0000000000000..4f080ca50739b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/AbstractTableSizeCacheWriter.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache; + +import org.apache.iotdb.db.storageengine.StorageEngine; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.List; +import java.util.concurrent.TimeUnit; + +public abstract class AbstractTableSizeCacheWriter { + protected static final Logger logger = + LoggerFactory.getLogger(AbstractTableSizeCacheWriter.class); + public static final String TEMP_CACHE_FILE_SUBFIX = ".tmp"; + protected final int regionId; + protected long previousCompactionTimestamp = System.currentTimeMillis(); + protected long lastWriteTimestamp = System.currentTimeMillis(); + protected int currentIndexFileVersion = 0; + protected final File dir; + + public AbstractTableSizeCacheWriter(String database, int regionId) { + this.regionId = regionId; + this.dir = StorageEngine.getDataRegionSystemDir(database, regionId + ""); + } + + protected void failedToRecover(Exception e) { + TableDiskUsageCache.getInstance().failedToRecover(e); + } + + protected void deleteOldVersionFiles(int maxVersion, String prefix, List files) { + for (File file : files) { + try { + int version = Integer.parseInt(file.getName().substring(prefix.length())); + if (version != maxVersion) { + Files.deleteIfExists(file.toPath()); + } + } catch (Exception e) { + logger.warn( + "Failed to delete old version table size cache file {}", file.getAbsolutePath()); + } + } + } + + public void closeIfIdle() { + if (System.currentTimeMillis() - lastWriteTimestamp >= TimeUnit.MINUTES.toMillis(1)) { + close(); + } + } + + public void markWritten() { + lastWriteTimestamp = System.currentTimeMillis(); + } + + public abstract boolean needCompact(); + + public abstract void compact(); + + public abstract void flush() throws IOException; + + public abstract void sync() throws IOException; + + public abstract void close(); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/DataRegionTableSizeQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/DataRegionTableSizeQueryContext.java new file mode 100644 index 0000000000000..960d6cec41907 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/DataRegionTableSizeQueryContext.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.storageengine.dataregion.utils.tableDiskUsageCache; + +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; + +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Optional; + +public class DataRegionTableSizeQueryContext { + + private final boolean needAllData; + + private final Map + timePartitionTableSizeQueryContextMap = new LinkedHashMap<>(); + private int objectFileNum = 0; + + private long previousUsedTimePartition = Long.MIN_VALUE; + private TimePartitionTableSizeQueryContext previousUsedTimePartitionContext = null; + + private final Optional fragmentInstanceContext; + private long acquiredMemory; + + public DataRegionTableSizeQueryContext(boolean needAllData) { + this(needAllData, null); + } + + public DataRegionTableSizeQueryContext( + boolean needAllData, FragmentInstanceContext fragmentInstanceContext) { + this.needAllData = needAllData; + this.fragmentInstanceContext = Optional.ofNullable(fragmentInstanceContext); + } + + public Map getTimePartitionTableSizeQueryContextMap() { + return timePartitionTableSizeQueryContextMap; + } + + public boolean isNeedAllData() { + return needAllData; + } + + public boolean isEmpty() { + return timePartitionTableSizeQueryContextMap.isEmpty(); + } + + public Optional getFragmentInstanceContext() { + return fragmentInstanceContext; + } + + public void addCachedTsFileIDAndOffsetInValueFile(TsFileID tsFileID, long offset) { + if (useTimePartition(tsFileID.timePartitionId)) { + previousUsedTimePartitionContext.addCachedTsFileIDAndOffsetInValueFile(tsFileID, offset); + } + } + + public void replaceCachedTsFileID(TsFileID originTsFileID, TsFileID newTsFileID) { + if (useTimePartition(originTsFileID.timePartitionId)) { + previousUsedTimePartitionContext.replaceCachedTsFileID(originTsFileID, newTsFileID); + } + } + + public void updateResult(String table, long size, long currentTimePartition) { + if (useTimePartition(currentTimePartition)) { + previousUsedTimePartitionContext.updateResult(table, size, needAllData); + } + } + + /** + * useTimePartition must be called before accessing previousUsedTimePartitionContext. When it + * returns false, the caller must skip any operation on the context. + */ + private boolean useTimePartition(long currentTimePartition) { + if (currentTimePartition != previousUsedTimePartition + || previousUsedTimePartitionContext == null) { + TimePartitionTableSizeQueryContext currentTimePartitionContext = + timePartitionTableSizeQueryContextMap.compute( + currentTimePartition, + (k, v) -> + (v == null && needAllData) + ? new TimePartitionTableSizeQueryContext(new HashMap<>()) + : v); + if (currentTimePartitionContext == null) { + return false; + } + previousUsedTimePartition = currentTimePartition; + previousUsedTimePartitionContext = currentTimePartitionContext; + } + return true; + } + + public void addAllTimePartitionsInTsFileManager(TsFileManager tsFileManager) { + for (Long timePartition : tsFileManager.getTimePartitions()) { + addTimePartition(timePartition, new TimePartitionTableSizeQueryContext(new HashMap<>())); + } + } + + public void addTimePartition( + long timePartition, TimePartitionTableSizeQueryContext timePartitionTableSizeQueryContext) { + timePartitionTableSizeQueryContextMap.put(timePartition, timePartitionTableSizeQueryContext); + } + + public int getObjectFileNum() { + return objectFileNum; + } + + public long getObjectFileSize() { + long totalSize = 0; + for (TimePartitionTableSizeQueryContext timePartitionContext : + timePartitionTableSizeQueryContextMap.values()) { + totalSize += timePartitionContext.getObjectFileSizeOfCurrentTimePartition(); + } + return totalSize; + } + + public void updateObjectFileNum(int delta) { + this.objectFileNum += delta; + } + + public void reserveMemoryForResultMap() { + long cost = + RamUsageEstimator.sizeOfMapWithKnownShallowSize( + timePartitionTableSizeQueryContextMap, + RamUsageEstimator.SHALLOW_SIZE_OF_HASHMAP, + RamUsageEstimator.SHALLOW_SIZE_OF_HASHMAP_ENTRY); + reserveMemory(cost); + } + + public void reserveMemoryForTsFileIDs() { + long cost = + timePartitionTableSizeQueryContextMap.values().stream() + .mapToLong(TimePartitionTableSizeQueryContext::ramBytesUsedOfTsFileIDOffsetMap) + .sum(); + reserveMemory(cost); + } + + public void reserveMemory(long size) { + if (!fragmentInstanceContext.isPresent()) { + return; + } + MemoryReservationManager memoryReservationContext = + fragmentInstanceContext.get().getMemoryReservationContext(); + memoryReservationContext.reserveMemoryCumulatively(size); + memoryReservationContext.reserveMemoryImmediately(); + acquiredMemory += size; + } + + public void releaseMemory() { + if (!fragmentInstanceContext.isPresent() || acquiredMemory <= 0) { + return; + } + fragmentInstanceContext + .get() + .getMemoryReservationContext() + .releaseMemoryCumulatively(acquiredMemory); + acquiredMemory = 0; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/DefaultTableDiskUsageCacheProvider.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/DefaultTableDiskUsageCacheProvider.java new file mode 100644 index 0000000000000..87992040fb101 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/DefaultTableDiskUsageCacheProvider.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache; + +public class DefaultTableDiskUsageCacheProvider implements TableDiskUsageCacheProvider { + + @Override + public TableDiskUsageCache create() { + return Holder.INSTANCE; + } + + private static class Holder { + private static final TableDiskUsageCache INSTANCE = new TableDiskUsageCache(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TableDiskUsageCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TableDiskUsageCache.java new file mode 100644 index 0000000000000..e1dc29f9958e2 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TableDiskUsageCache.java @@ -0,0 +1,512 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache; + +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; +import org.apache.iotdb.commons.concurrent.ThreadName; +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.object.EmptyObjectTableSizeCacheReader; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.object.IObjectTableSizeCacheReader; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile.TsFileTableDiskUsageCacheWriter; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile.TsFileTableSizeCacheReader; + +import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.ServiceLoader; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class TableDiskUsageCache { + protected static final Logger LOGGER = LoggerFactory.getLogger(TableDiskUsageCache.class); + protected final BlockingQueue queue = new LinkedBlockingQueue<>(1000); + // regionId -> writer mapping + protected final Map writerMap = new HashMap<>(); + protected ScheduledExecutorService scheduledExecutorService; + private int processedOperationCountSinceLastPeriodicCheck = 0; + protected volatile boolean failedToRecover = false; + private volatile boolean stop = false; + + protected TableDiskUsageCache() { + scheduledExecutorService = + IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor( + ThreadName.FILE_TIME_INDEX_RECORD.getName()); + scheduledExecutorService.submit(this::run); + } + + protected void run() { + try { + while (!stop) { + try { + for (DataRegionTableSizeCacheWriter writer : writerMap.values()) { + syncTsFileTableSizeCacheIfNecessary(writer); + persistPendingObjectDeltasIfNecessary(writer); + } + Operation operation = queue.poll(1, TimeUnit.SECONDS); + if (operation != null) { + operation.apply(this); + processedOperationCountSinceLastPeriodicCheck++; + } + if (operation == null || processedOperationCountSinceLastPeriodicCheck % 1000 == 0) { + performPeriodicMaintenance(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return; + } catch (Exception e) { + LOGGER.error("Meet exception when apply TableDiskUsageCache operation.", e); + } + } + } finally { + writerMap.values().forEach(DataRegionTableSizeCacheWriter::close); + } + } + + private void performPeriodicMaintenance() { + checkAndMayCloseIdleWriter(); + compactIfNecessary(TimeUnit.SECONDS.toMillis(1)); + processedOperationCountSinceLastPeriodicCheck = 0; + } + + /** + * Any unrecoverable error in a single writer will mark the whole TableDiskUsageCache as failed + * and disable further operations. + */ + protected void failedToRecover(Exception e) { + failedToRecover = true; + LOGGER.error("Failed to recover TableDiskUsageCache", e); + } + + protected void syncTsFileTableSizeCacheIfNecessary(DataRegionTableSizeCacheWriter writer) { + try { + writer.tsFileCacheWriter.syncIfNecessary(); + } catch (IOException e) { + LOGGER.warn("Failed to sync tsfile table size cache.", e); + } + } + + // Hook for subclasses to persist pending object table size deltas. No-op by default. + protected void persistPendingObjectDeltasIfNecessary(DataRegionTableSizeCacheWriter writer) {} + + protected void compactIfNecessary(long maxRunTime) { + if (!StorageEngine.getInstance().isReadyForReadAndWrite()) { + return; + } + long startTime = System.currentTimeMillis(); + for (DataRegionTableSizeCacheWriter writer : writerMap.values()) { + if (System.currentTimeMillis() - startTime > maxRunTime) { + break; + } + if (writer.getActiveReaderNum() > 0) { + continue; + } + writer.compactIfNecessary(); + } + } + + protected void checkAndMayCloseIdleWriter() { + for (DataRegionTableSizeCacheWriter writer : writerMap.values()) { + writer.closeIfIdle(); + } + } + + public void write(String database, TsFileID tsFileID, Map tableSizeMap) { + if (tableSizeMap == null || tableSizeMap.isEmpty()) { + // tree model + return; + } + addOperationToQueue(new WriteOperation(database, tsFileID, tableSizeMap)); + } + + public void write(String database, TsFileID originTsFileID, TsFileID newTsFileID) { + addOperationToQueue(new ReplaceTsFileOperation(database, originTsFileID, newTsFileID)); + } + + public void writeObjectDelta( + String database, int regionId, long timePartition, String table, long size, int num) { + throw new UnsupportedOperationException("writeObjectDelta"); + } + + public CompletableFuture> startRead( + DataRegion dataRegion, boolean readTsFileCache, boolean readObjectFileCache) { + StartReadOperation operation = + new StartReadOperation(dataRegion, readTsFileCache, readObjectFileCache); + if (!addOperationToQueue(operation)) { + operation.future.complete( + new Pair<>( + new TsFileTableSizeCacheReader( + 0, null, 0, null, dataRegion.getDataRegionId().getId()), + new EmptyObjectTableSizeCacheReader())); + } + return operation.future; + } + + public void endRead(DataRegion dataRegion) { + EndReadOperation operation = new EndReadOperation(dataRegion); + addOperationToQueue(operation); + } + + public void registerRegion(DataRegion region) { + RegisterRegionOperation operation = new RegisterRegionOperation(region); + if (!region.isTableModel()) { + return; + } + addOperationToQueue(operation); + } + + public void remove(String database, int regionId) { + RemoveRegionOperation operation = new RemoveRegionOperation(database, regionId); + if (!addOperationToQueue(operation)) { + return; + } + try { + operation.future.get(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (Exception e) { + LOGGER.error("Meet exception when remove TableDiskUsageCache.", e); + } + } + + protected boolean addOperationToQueue(Operation operation) { + if (failedToRecover || stop) { + return false; + } + try { + queue.put(operation); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return false; + } + return true; + } + + public int getQueueSize() { + return queue.size(); + } + + public void close() { + if (scheduledExecutorService == null) { + return; + } + try { + stop = true; + scheduledExecutorService.shutdown(); + scheduledExecutorService.awaitTermination(5, TimeUnit.SECONDS); + writerMap.values().forEach(DataRegionTableSizeCacheWriter::close); + writerMap.clear(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + @TestOnly + public void ensureRunning() { + stop = false; + failedToRecover = false; + if (scheduledExecutorService.isTerminated()) { + scheduledExecutorService = + IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor( + ThreadName.FILE_TIME_INDEX_RECORD.getName()); + scheduledExecutorService.submit(this::run); + } + } + + protected DataRegionTableSizeCacheWriter createWriter( + String database, int regionId, DataRegion region) { + return new DataRegionTableSizeCacheWriter(database, regionId, region); + } + + protected TsFileTableSizeCacheReader createTsFileCacheReader( + DataRegionTableSizeCacheWriter dataRegionWriter, int regionId) { + TsFileTableDiskUsageCacheWriter tsFileCacheWriter = dataRegionWriter.tsFileCacheWriter; + return new TsFileTableSizeCacheReader( + tsFileCacheWriter.keyFileLength(), + tsFileCacheWriter.getKeyFile(), + tsFileCacheWriter.valueFileLength(), + tsFileCacheWriter.getValueFile(), + regionId); + } + + protected IObjectTableSizeCacheReader createObjectFileCacheReader( + DataRegionTableSizeCacheWriter dataRegionWriter, int regionId) { + return new EmptyObjectTableSizeCacheReader(); + } + + protected abstract static class Operation { + protected final String database; + protected final int regionId; + + protected Operation(String database, int regionId) { + this.database = database; + this.regionId = regionId; + } + + public int getRegionId() { + return regionId; + } + + public String getDatabase() { + return database; + } + + public abstract void apply(TableDiskUsageCache tableDiskUsageCache) throws IOException; + } + + protected static class StartReadOperation extends Operation { + protected final DataRegion region; + protected final boolean readTsFileCache; + protected final boolean readObjectFileCache; + public CompletableFuture> future = + new CompletableFuture<>(); + + public StartReadOperation( + DataRegion dataRegion, boolean readTsFileCache, boolean readObjectFileCache) { + super(dataRegion.getDatabaseName(), dataRegion.getDataRegionId().getId()); + this.region = dataRegion; + this.readTsFileCache = readTsFileCache; + this.readObjectFileCache = readObjectFileCache; + } + + @Override + public void apply(TableDiskUsageCache tableDiskUsageCache) throws IOException { + DataRegionTableSizeCacheWriter writer = tableDiskUsageCache.writerMap.get(regionId); + try { + if (writer == null || writer.getRemovedFuture() != null) { + // region is removing or removed + future.complete( + new Pair<>( + new TsFileTableSizeCacheReader(0, null, 0, null, regionId), + new EmptyObjectTableSizeCacheReader())); + return; + } + writer.increaseActiveReaderNum(); + // Flush buffered writes to ensure readers observe a consistent snapshot + writer.flush(); + TsFileTableSizeCacheReader tsFileTableSizeCacheReader = + readTsFileCache ? tableDiskUsageCache.createTsFileCacheReader(writer, regionId) : null; + IObjectTableSizeCacheReader objectTableSizeCacheReader = + readObjectFileCache + ? tableDiskUsageCache.createObjectFileCacheReader(writer, regionId) + : null; + future.complete(new Pair<>(tsFileTableSizeCacheReader, objectTableSizeCacheReader)); + } catch (Throwable t) { + future.completeExceptionally(t); + } + } + } + + private static class EndReadOperation extends Operation { + protected final DataRegion region; + + public EndReadOperation(DataRegion dataRegion) { + super(dataRegion.getDatabaseName(), dataRegion.getDataRegionId().getId()); + this.region = dataRegion; + } + + @Override + public void apply(TableDiskUsageCache tableDiskUsageCache) throws IOException { + tableDiskUsageCache.writerMap.computeIfPresent( + regionId, + (k, writer) -> { + if (writer.dataRegion != region) { + return writer; + } + writer.decreaseActiveReaderNum(); + // Complete pending remove when the last reader exits + if (writer.getRemovedFuture() != null) { + writer.close(); + writer.getRemovedFuture().complete(null); + writer.setRemovedFuture(null); + return null; + } + return writer; + }); + } + } + + private static class WriteOperation extends Operation { + + private final TsFileID tsFileID; + private final Map tableSizeMap; + + protected WriteOperation(String database, TsFileID tsFileID, Map tableSizeMap) { + super(database, tsFileID.regionId); + this.tsFileID = tsFileID; + this.tableSizeMap = tableSizeMap; + } + + @Override + public void apply(TableDiskUsageCache tableDiskUsageCache) throws IOException { + DataRegionTableSizeCacheWriter dataRegionTableSizeCacheWriter = + tableDiskUsageCache.writerMap.get(regionId); + if (dataRegionTableSizeCacheWriter != null) { + dataRegionTableSizeCacheWriter.tsFileCacheWriter.write(tsFileID, tableSizeMap); + } + } + } + + private static class ReplaceTsFileOperation extends Operation { + private final TsFileID originTsFileID; + private final TsFileID newTsFileID; + + public ReplaceTsFileOperation(String database, TsFileID originTsFileID, TsFileID newTsFileID) { + super(database, originTsFileID.regionId); + this.originTsFileID = originTsFileID; + this.newTsFileID = newTsFileID; + } + + @Override + public void apply(TableDiskUsageCache tableDiskUsageCache) throws IOException { + DataRegionTableSizeCacheWriter writer = tableDiskUsageCache.writerMap.get(regionId); + if (writer != null) { + writer.tsFileCacheWriter.write(originTsFileID, newTsFileID); + } + } + } + + protected static class RegisterRegionOperation extends Operation { + + protected final DataRegion dataRegion; + protected final CompletableFuture future = new CompletableFuture<>(); + + public RegisterRegionOperation(DataRegion dataRegion) { + super(dataRegion.getDatabaseName(), dataRegion.getDataRegionId().getId()); + this.dataRegion = dataRegion; + } + + @Override + public void apply(TableDiskUsageCache tableDiskUsageCache) { + tableDiskUsageCache.writerMap.computeIfAbsent( + regionId, regionId -> tableDiskUsageCache.createWriter(database, regionId, dataRegion)); + future.complete(null); + } + + public CompletableFuture getFuture() { + return future; + } + } + + private static class RemoveRegionOperation extends Operation { + + private final CompletableFuture future = new CompletableFuture<>(); + + private RemoveRegionOperation(String database, int regionId) { + super(database, regionId); + } + + @Override + public void apply(TableDiskUsageCache tableDiskUsageCache) { + tableDiskUsageCache.writerMap.computeIfPresent( + regionId, + (k, writer) -> { + if (writer.getActiveReaderNum() > 0) { + // If there are active readers, defer removal until all readers finish + writer.setRemovedFuture(future); + return writer; + } + writer.close(); + future.complete(null); + return null; + }); + } + } + + public static TableDiskUsageCache getInstance() { + return TableDiskUsageCache.InstanceHolder.INSTANCE; + } + + private static class InstanceHolder { + private InstanceHolder() {} + + private static final TableDiskUsageCache INSTANCE = loadInstance(); + + private static TableDiskUsageCache loadInstance() { + ServiceLoader loader = + ServiceLoader.load(TableDiskUsageCacheProvider.class); + for (TableDiskUsageCacheProvider provider : loader) { + return provider.create(); + } + return new DefaultTableDiskUsageCacheProvider().create(); + } + } + + protected static class DataRegionTableSizeCacheWriter { + protected final DataRegion dataRegion; + protected final TsFileTableDiskUsageCacheWriter tsFileCacheWriter; + protected int activeReaderNum = 0; + protected CompletableFuture removedFuture; + + protected DataRegionTableSizeCacheWriter(String database, int regionId, DataRegion dataRegion) { + this.dataRegion = dataRegion; + this.tsFileCacheWriter = new TsFileTableDiskUsageCacheWriter(database, regionId); + } + + public void increaseActiveReaderNum() { + activeReaderNum++; + } + + public void decreaseActiveReaderNum() { + if (activeReaderNum > 0) { + activeReaderNum--; + } + } + + public int getActiveReaderNum() { + return activeReaderNum; + } + + public void compactIfNecessary() { + if (tsFileCacheWriter.needCompact()) { + tsFileCacheWriter.compact(); + } + } + + public void closeIfIdle() { + tsFileCacheWriter.closeIfIdle(); + } + + public void flush() throws IOException { + tsFileCacheWriter.flush(); + } + + public void setRemovedFuture(CompletableFuture removedFuture) { + this.removedFuture = removedFuture; + } + + public CompletableFuture getRemovedFuture() { + return removedFuture; + } + + public void close() { + tsFileCacheWriter.close(); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TableDiskUsageCacheProvider.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TableDiskUsageCacheProvider.java new file mode 100644 index 0000000000000..cba896ed78159 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TableDiskUsageCacheProvider.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache; + +public interface TableDiskUsageCacheProvider { + TableDiskUsageCache create(); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TableDiskUsageCacheReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TableDiskUsageCacheReader.java new file mode 100644 index 0000000000000..d4f9a88bba8db --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TableDiskUsageCacheReader.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache; + +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; +import org.apache.iotdb.db.storageengine.dataregion.utils.TableDiskUsageStatisticUtil; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.object.IObjectTableSizeCacheReader; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile.TsFileTableSizeCacheReader; + +import org.apache.tsfile.utils.Pair; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +public class TableDiskUsageCacheReader implements Closeable { + + private final DataRegion dataRegion; + private final DataRegionTableSizeQueryContext dataRegionContext; + + private CompletableFuture> + prepareReaderFuture; + private TsFileTableSizeCacheReader tsFileTableSizeCacheReader; + private IObjectTableSizeCacheReader objectTableSizeCacheReader; + + private boolean objectFileSizeLoaded = false; + private boolean tsFileIdKeysPrepared = false; + private boolean allTsFileResourceChecked = false; + + private final Iterator> timePartitionIterator; + + private final boolean currentDatabaseOnlyHasOneTable; + private TableDiskUsageStatisticUtil tableDiskUsageStatisticUtil; + + private final List> tsFilesToQueryInCache = new ArrayList<>(); + private Iterator> tsFilesToQueryInCacheIterator = null; + + public TableDiskUsageCacheReader( + DataRegion dataRegion, + DataRegionTableSizeQueryContext dataRegionContext, + boolean databaseHasOnlyOneTable) { + this.dataRegion = dataRegion; + this.dataRegionContext = dataRegionContext; + this.currentDatabaseOnlyHasOneTable = databaseHasOnlyOneTable; + if (dataRegionContext.isNeedAllData()) { + dataRegionContext.addAllTimePartitionsInTsFileManager(dataRegion.getTsFileManager()); + } + this.timePartitionIterator = + dataRegionContext.getTimePartitionTableSizeQueryContextMap().entrySet().iterator(); + dataRegionContext.reserveMemoryForResultMap(); + } + + public boolean prepareCacheReader(long startTime, long maxRunTime) throws Exception { + if (this.tsFileTableSizeCacheReader == null) { + this.prepareReaderFuture = + this.prepareReaderFuture == null + ? TableDiskUsageCache.getInstance().startRead(dataRegion, true, true) + : this.prepareReaderFuture; + do { + if (prepareReaderFuture.isDone()) { + Pair readerPair = + prepareReaderFuture.get(); + this.tsFileTableSizeCacheReader = readerPair.left; + this.tsFileTableSizeCacheReader.openKeyFile(); + this.objectTableSizeCacheReader = readerPair.right; + break; + } else { + Thread.sleep(1); + } + } while (System.nanoTime() - startTime < maxRunTime); + } + return this.tsFileTableSizeCacheReader != null; + } + + public boolean loadObjectFileTableSizeCache(long startTime, long maxRunTime) throws Exception { + if (objectFileSizeLoaded) { + return true; + } + if (objectTableSizeCacheReader.loadObjectFileTableSize( + dataRegionContext, startTime, maxRunTime)) { + closeObjectFileTableSizeCacheReader(); + objectFileSizeLoaded = true; + return true; + } + return false; + } + + public boolean prepareCachedTsFileIDKeys(long startTime, long maxRunTime) throws Exception { + if (tsFileIdKeysPrepared) { + return true; + } + if (tsFileTableSizeCacheReader.readFromKeyFile(dataRegionContext, startTime, maxRunTime)) { + dataRegionContext.reserveMemoryForTsFileIDs(); + tsFileIdKeysPrepared = true; + return true; + } + return false; + } + + public boolean checkAllFilesInTsFileManager(long start, long maxRunTime) { + if (allTsFileResourceChecked) { + return true; + } + do { + if (!calculateNextFile()) { + allTsFileResourceChecked = true; + break; + } + } while (System.nanoTime() - start < maxRunTime); + return allTsFileResourceChecked; + } + + private boolean calculateNextFile() { + while (true) { + if (tableDiskUsageStatisticUtil != null && tableDiskUsageStatisticUtil.hasNextFile()) { + tableDiskUsageStatisticUtil.calculateNextFile(); + return true; + } + if (timePartitionIterator.hasNext()) { + Map.Entry currentTimePartitionEntry = + timePartitionIterator.next(); + long timePartition = currentTimePartitionEntry.getKey(); + closeTableDiskUsageStatisticUtil(); + tableDiskUsageStatisticUtil = + new TableDiskUsageStatisticUtil( + dataRegion, + timePartition, + currentTimePartitionEntry.getValue(), + dataRegionContext.isNeedAllData(), + currentDatabaseOnlyHasOneTable, + tsFilesToQueryInCache, + dataRegionContext.getFragmentInstanceContext()); + } else { + closeTableDiskUsageStatisticUtil(); + return false; + } + } + } + + public boolean readCacheValueFilesAndUpdateResultMap(long startTime, long maxRunTime) + throws IOException { + if (this.tsFilesToQueryInCacheIterator == null) { + this.tsFilesToQueryInCache.sort(Comparator.comparingLong(Pair::getRight)); + this.tsFilesToQueryInCacheIterator = tsFilesToQueryInCache.iterator(); + this.tsFileTableSizeCacheReader.openValueFile(); + } + return tsFileTableSizeCacheReader.readFromValueFile( + tsFilesToQueryInCacheIterator, dataRegionContext, startTime, maxRunTime); + } + + public DataRegion getDataRegion() { + return dataRegion; + } + + @Override + public void close() throws IOException { + closeTableDiskUsageStatisticUtil(); + closeTsFileTableSizeCacheReader(); + closeObjectFileTableSizeCacheReader(); + if (prepareReaderFuture != null) { + TableDiskUsageCache.getInstance().endRead(dataRegion); + prepareReaderFuture = null; + } + dataRegionContext.releaseMemory(); + } + + private void closeTableDiskUsageStatisticUtil() { + if (tableDiskUsageStatisticUtil != null) { + tableDiskUsageStatisticUtil.close(); + tableDiskUsageStatisticUtil = null; + } + } + + private void closeTsFileTableSizeCacheReader() { + if (tsFileTableSizeCacheReader != null) { + tsFileTableSizeCacheReader.closeCurrentFile(); + } + } + + private void closeObjectFileTableSizeCacheReader() { + if (objectTableSizeCacheReader != null) { + objectTableSizeCacheReader.close(); + objectTableSizeCacheReader = null; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TimePartitionTableSizeQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TimePartitionTableSizeQueryContext.java new file mode 100644 index 0000000000000..454fb4b93418e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/TimePartitionTableSizeQueryContext.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache; + +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; + +import org.apache.tsfile.utils.Accountable; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.util.HashMap; +import java.util.Map; + +public class TimePartitionTableSizeQueryContext implements Accountable { + private static final long SHALLOW_SIZE = + RamUsageEstimator.shallowSizeOf(TimePartitionTableSizeQueryContext.class); + + /** + * tableSizeResultMap serves as both: 1) result container 2) table filter when needAllData is + * false + */ + private final Map tableSizeResultMap; + + // tsFileIDOffsetInValueFileMap should be null at first + private Map tsFileIDOffsetInValueFileMap; + + public TimePartitionTableSizeQueryContext(Map tableSizeResultMap) { + this.tableSizeResultMap = tableSizeResultMap; + } + + public void addCachedTsFileIDAndOffsetInValueFile(TsFileID tsFileID, long offset) { + if (tsFileIDOffsetInValueFileMap == null) { + tsFileIDOffsetInValueFileMap = new HashMap<>(); + } + tsFileIDOffsetInValueFileMap.put(tsFileID, offset); + } + + public void replaceCachedTsFileID(TsFileID originTsFileID, TsFileID newTsFileID) { + if (tsFileIDOffsetInValueFileMap == null) { + return; + } + Long offset = tsFileIDOffsetInValueFileMap.remove(originTsFileID); + if (offset != null) { + tsFileIDOffsetInValueFileMap.put(newTsFileID, offset); + } + } + + public void updateResult(String table, long size, boolean needAllData) { + if (needAllData) { + tableSizeResultMap.compute(table, (k, v) -> (v == null ? 0 : v) + size); + } else { + tableSizeResultMap.computeIfPresent(table, (k, v) -> v + size); + } + } + + public Map getTableSizeResultMap() { + return tableSizeResultMap; + } + + public Long getCachedTsFileIdOffset(TsFileID tsFileID) { + return tsFileIDOffsetInValueFileMap == null ? null : tsFileIDOffsetInValueFileMap.get(tsFileID); + } + + public long getObjectFileSizeOfCurrentTimePartition() { + long size = 0; + for (Long value : tableSizeResultMap.values()) { + size += value; + } + return size; + } + + @Override + public long ramBytesUsed() { + return SHALLOW_SIZE + + RamUsageEstimator.sizeOfMapWithKnownShallowSize( + tableSizeResultMap, + RamUsageEstimator.SHALLOW_SIZE_OF_HASHMAP, + RamUsageEstimator.SHALLOW_SIZE_OF_HASHMAP_ENTRY) + + ramBytesUsedOfTsFileIDOffsetMap(); + } + + // tsFileIDOffsetInValueFileMap should be null at first + public long ramBytesUsedOfTsFileIDOffsetMap() { + if (tsFileIDOffsetInValueFileMap == null) { + return 0; + } + return RamUsageEstimator.SHALLOW_SIZE_OF_HASHMAP + + tsFileIDOffsetInValueFileMap.size() + * (RamUsageEstimator.SHALLOW_SIZE_OF_HASHMAP_ENTRY + + Long.BYTES + + TsFileID.SHALLOW_SIZE); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/object/EmptyObjectTableSizeCacheReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/object/EmptyObjectTableSizeCacheReader.java new file mode 100644 index 0000000000000..2020ab10a2337 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/object/EmptyObjectTableSizeCacheReader.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.object; + +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.DataRegionTableSizeQueryContext; + +public class EmptyObjectTableSizeCacheReader implements IObjectTableSizeCacheReader { + + @Override + public boolean loadObjectFileTableSize( + DataRegionTableSizeQueryContext dataRegionContext, long startTime, long maxRunTime) { + return true; + } + + @Override + public void close() {} +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/object/IObjectTableSizeCacheReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/object/IObjectTableSizeCacheReader.java new file mode 100644 index 0000000000000..6183601e719f3 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/object/IObjectTableSizeCacheReader.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.object; + +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.DataRegionTableSizeQueryContext; + +import java.io.IOException; + +public interface IObjectTableSizeCacheReader extends AutoCloseable { + boolean loadObjectFileTableSize( + DataRegionTableSizeQueryContext dataRegionContext, long startTime, long maxRunTime) + throws IOException; + + @Override + void close(); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/tsfile/TsFileTableDiskUsageCacheWriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/tsfile/TsFileTableDiskUsageCacheWriter.java new file mode 100644 index 0000000000000..7c60d38fc134f --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/tsfile/TsFileTableDiskUsageCacheWriter.java @@ -0,0 +1,311 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile; + +import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.AbstractTableSizeCacheWriter; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TimePartitionTableSizeQueryContext; + +import org.apache.tsfile.utils.Pair; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +public class TsFileTableDiskUsageCacheWriter extends AbstractTableSizeCacheWriter { + public static final String TSFILE_CACHE_KEY_FILENAME_PREFIX = "TableSizeKeyFile_"; + public static final String TSFILE_CACHE_VALUE_FILENAME_PREFIX = "TableSizeValueFile_"; + public static final int KEY_FILE_OFFSET_RECORD_LENGTH = 5 * Long.BYTES + 1; + public static final int KEY_FILE_REDIRECT_RECORD_LENGTH = 7 * Long.BYTES + 1; + public static final byte KEY_FILE_RECORD_TYPE_OFFSET = 1; + public static final byte KEY_FILE_RECORD_TYPE_REDIRECT = 2; + + private TsFileTableSizeIndexFileWriter tsFileTableSizeIndexFileWriter; + private long lastSyncTimestamp = System.currentTimeMillis(); + + public TsFileTableDiskUsageCacheWriter(String database, int regionId) { + super(database, regionId); + recoverTsFileTableSizeIndexFile(true); + } + + private void recoverTsFileTableSizeIndexFile(boolean needRecover) { + dir.mkdirs(); + File[] files = dir.listFiles(); + currentIndexFileVersion = 0; + List keyFiles = new ArrayList<>(); + List valueFiles = new ArrayList<>(); + if (files != null) { + for (File file : files) { + String fileName = file.getName(); + boolean isKeyFile = fileName.startsWith(TSFILE_CACHE_KEY_FILENAME_PREFIX); + boolean isValueFile = !isKeyFile && fileName.startsWith(TSFILE_CACHE_VALUE_FILENAME_PREFIX); + boolean isTempFile = fileName.endsWith(TEMP_CACHE_FILE_SUBFIX); + if (!isKeyFile) { + if (isValueFile && !isTempFile) { + valueFiles.add(file); + } + continue; + } + int version; + try { + version = getVersion(fileName); + } catch (NumberFormatException ignored) { + continue; + } + if (isTempFile) { + FileUtils.deleteFileIfExist(file); + FileUtils.deleteFileIfExist(generateValueFile(version, true)); + continue; + } + File valueFile = + new File(dir + File.separator + TSFILE_CACHE_VALUE_FILENAME_PREFIX + version); + // may have a valid value index file + if (!valueFile.exists()) { + File tempValueFile = new File(valueFile.getPath() + TEMP_CACHE_FILE_SUBFIX); + if (tempValueFile.exists()) { + try { + Files.move(tempValueFile.toPath(), valueFile.toPath()); + } catch (IOException e) { + logger.warn("Failed to move {} to {}", tempValueFile, valueFile, e); + continue; + } + valueFiles.add(valueFile); + } else { + // lost value file + FileUtils.deleteFileIfExist(file); + continue; + } + } + currentIndexFileVersion = Math.max(currentIndexFileVersion, version); + keyFiles.add(file); + } + if (keyFiles.size() > 1) { + deleteOldVersionFiles(currentIndexFileVersion, TSFILE_CACHE_KEY_FILENAME_PREFIX, keyFiles); + } + if (valueFiles.size() > 1) { + deleteOldVersionFiles( + currentIndexFileVersion, TSFILE_CACHE_VALUE_FILENAME_PREFIX, valueFiles); + } + } + File currentKeyIndexFile = generateKeyFile(currentIndexFileVersion, false); + File currentValueIndexFile = generateValueFile(currentIndexFileVersion, false); + try { + this.tsFileTableSizeIndexFileWriter = + new TsFileTableSizeIndexFileWriter( + regionId, currentKeyIndexFile, currentValueIndexFile, needRecover); + } catch (IOException e) { + failedToRecover(e); + } + } + + private int getVersion(String fileName) throws NumberFormatException { + String removePrefixStr = fileName.substring(TSFILE_CACHE_KEY_FILENAME_PREFIX.length()); + int suffixIdx = removePrefixStr.indexOf('.'); + return Integer.parseInt( + suffixIdx > 0 ? removePrefixStr.substring(0, suffixIdx) : removePrefixStr); + } + + public void write(TsFileID tsFileID, Map tableSizeMap) throws IOException { + tsFileTableSizeIndexFileWriter.write(tsFileID, tableSizeMap); + markWritten(); + } + + public void write(TsFileID originTsFileID, TsFileID newTsFileID) throws IOException { + tsFileTableSizeIndexFileWriter.write(originTsFileID, newTsFileID); + markWritten(); + } + + @Override + public boolean needCompact() { + if (System.currentTimeMillis() - previousCompactionTimestamp <= TimeUnit.MINUTES.toMillis(2)) { + return false; + } + DataRegion dataRegion = StorageEngine.getInstance().getDataRegion(new DataRegionId(regionId)); + if (dataRegion == null || dataRegion.isDeleted()) { + return false; + } + TsFileManager tsFileManager = dataRegion.getTsFileManager(); + int fileNum = tsFileManager.size(true) + tsFileManager.size(false); + int estimatedEntryNumInCacheFile = (int) (keyFileLength() / KEY_FILE_OFFSET_RECORD_LENGTH); + int delta = estimatedEntryNumInCacheFile - fileNum; + return delta >= 1000; + } + + @Override + public void compact() { + previousCompactionTimestamp = System.currentTimeMillis(); + this.tsFileTableSizeIndexFileWriter.close(); + TsFileTableSizeCacheReader cacheFileReader = + new TsFileTableSizeCacheReader( + tsFileTableSizeIndexFileWriter.getKeyFile().length(), + tsFileTableSizeIndexFileWriter.getKeyFile(), + tsFileTableSizeIndexFileWriter.getValueFile().length(), + tsFileTableSizeIndexFileWriter.getValueFile(), + regionId); + Map contextMap = new HashMap<>(); + try { + cacheFileReader.openKeyFile(); + while (cacheFileReader.hasNextEntryInKeyFile()) { + TsFileTableSizeCacheReader.KeyFileEntry keyFileEntry = + cacheFileReader.readOneEntryFromKeyFile(); + TimePartitionTableSizeQueryContext context = + contextMap.computeIfAbsent( + keyFileEntry.getTimePartitionId(), + k -> new TimePartitionTableSizeQueryContext(Collections.emptyMap())); + if (keyFileEntry.originTsFileID == null) { + context.addCachedTsFileIDAndOffsetInValueFile(keyFileEntry.tsFileID, keyFileEntry.offset); + } else { + context.replaceCachedTsFileID(keyFileEntry.originTsFileID, keyFileEntry.tsFileID); + } + } + } catch (IOException e) { + logger.error("Failed to read key file during compaction", e); + return; + } finally { + cacheFileReader.closeCurrentFile(); + } + + List> validFilesOrderByOffset = new ArrayList<>(); + DataRegion dataRegion = StorageEngine.getInstance().getDataRegion(new DataRegionId(regionId)); + if (dataRegion == null || dataRegion.isDeleted()) { + return; + } + TsFileManager tsFileManager = dataRegion.getTsFileManager(); + for (Long timePartition : tsFileManager.getTimePartitions()) { + TimePartitionTableSizeQueryContext context = contextMap.get(timePartition); + if (context == null) { + continue; + } + Pair, List> resources = + tsFileManager.getTsFileListSnapshot(timePartition); + Stream.concat(resources.left.stream(), resources.right.stream()) + .forEach( + resource -> { + Long offset = context.getCachedTsFileIdOffset(resource.getTsFileID()); + if (offset != null) { + validFilesOrderByOffset.add(new Pair<>(resource.getTsFileID(), offset)); + } + }); + } + validFilesOrderByOffset.sort(Comparator.comparingLong(Pair::getRight)); + + TsFileTableSizeIndexFileWriter targetFileWriter = null; + try { + targetFileWriter = + new TsFileTableSizeIndexFileWriter( + regionId, + generateKeyFile(currentIndexFileVersion + 1, true), + generateValueFile(currentIndexFileVersion + 1, true)); + cacheFileReader.openValueFile(); + for (Pair pair : validFilesOrderByOffset) { + TsFileID tsFileID = pair.getLeft(); + long offset = pair.getRight(); + Map tableSizeMap = cacheFileReader.readOneEntryFromValueFile(offset, true); + targetFileWriter.write(tsFileID, tableSizeMap); + } + targetFileWriter.close(); + + // replace + File targetKeyFile = generateKeyFile(currentIndexFileVersion + 1, false); + File targetValueFile = generateValueFile(currentIndexFileVersion + 1, false); + Files.move(targetFileWriter.getKeyFile().toPath(), targetKeyFile.toPath()); + Files.move(targetFileWriter.getValueFile().toPath(), targetValueFile.toPath()); + } catch (Exception e) { + logger.error("Failed to execute compaction for tsfile table size cache file", e); + } finally { + if (targetFileWriter != null) { + targetFileWriter.close(); + } + cacheFileReader.closeCurrentFile(); + this.recoverTsFileTableSizeIndexFile(false); + } + } + + private File generateKeyFile(int version, boolean isTempFile) { + return new File( + dir + + File.separator + + TSFILE_CACHE_KEY_FILENAME_PREFIX + + version + + (isTempFile ? TEMP_CACHE_FILE_SUBFIX : "")); + } + + private File generateValueFile(int version, boolean isTempFile) { + return new File( + dir + + File.separator + + TSFILE_CACHE_VALUE_FILENAME_PREFIX + + version + + (isTempFile ? TEMP_CACHE_FILE_SUBFIX : "")); + } + + @Override + public void flush() throws IOException { + tsFileTableSizeIndexFileWriter.flush(); + } + + public File getKeyFile() { + return tsFileTableSizeIndexFileWriter.getKeyFile(); + } + + public File getValueFile() { + return tsFileTableSizeIndexFileWriter.getValueFile(); + } + + public long keyFileLength() { + return tsFileTableSizeIndexFileWriter.keyFileLength(); + } + + public long valueFileLength() { + return tsFileTableSizeIndexFileWriter.valueFileLength(); + } + + public void syncIfNecessary() throws IOException { + if (lastWriteTimestamp > lastSyncTimestamp + && System.currentTimeMillis() - lastSyncTimestamp >= TimeUnit.MINUTES.toMillis(10)) { + sync(); + } + } + + @Override + public void sync() throws IOException { + tsFileTableSizeIndexFileWriter.sync(); + lastSyncTimestamp = System.currentTimeMillis(); + } + + @Override + public void close() { + this.tsFileTableSizeIndexFileWriter.close(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/tsfile/TsFileTableSizeCacheReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/tsfile/TsFileTableSizeCacheReader.java new file mode 100644 index 0000000000000..c4e20b2d36e3c --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/tsfile/TsFileTableSizeCacheReader.java @@ -0,0 +1,401 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile; + +import org.apache.iotdb.commons.exception.IoTDBRuntimeException; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.DataRegionTableSizeQueryContext; +import org.apache.iotdb.db.utils.MmapUtil; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.ReadWriteForEncodingUtils; +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class TsFileTableSizeCacheReader { + + private static final Logger logger = LoggerFactory.getLogger(TsFileTableSizeCacheReader.class); + + private final File keyFile; + private final long keyFileLength; + private final File valueFile; + private final long valueFileLength; + private DirectBufferedSeekableFileInputStream inputStream; + private final int regionId; + + public TsFileTableSizeCacheReader( + long keyFileLength, File keyFile, long valueFileLength, File valueFile, int regionId) { + this.keyFile = keyFile; + this.keyFileLength = keyFileLength; + this.valueFile = valueFile; + this.valueFileLength = valueFileLength; + this.regionId = regionId; + } + + public void openKeyFile() throws IOException { + if (keyFileLength > 0 && inputStream == null) { + this.inputStream = new DirectBufferedSeekableFileInputStream(keyFile.toPath(), 4 * 1024); + } + } + + public void openValueFile() throws IOException { + if (valueFileLength > 0 && inputStream == null) { + this.inputStream = new DirectBufferedSeekableFileInputStream(valueFile.toPath(), 4 * 1024); + } + } + + public Pair selfCheck() { + if (keyFileLength == 0 || valueFileLength == 0) { + return new Pair<>(0L, 0L); + } + List offsetsInKeyFile = new ArrayList<>(); + List lastCompleteKeyOffsets = new ArrayList<>(); + long lastCompleteEntryEndOffsetInKeyFile = 0; + try { + openKeyFile(); + while (hasNextEntryInKeyFile()) { + KeyFileEntry keyFileEntry = readOneEntryFromKeyFile(); + lastCompleteEntryEndOffsetInKeyFile = inputStream.position(); + if (keyFileEntry.originTsFileID != null) { + if (!lastCompleteKeyOffsets.isEmpty()) { + lastCompleteKeyOffsets.set( + lastCompleteKeyOffsets.size() - 1, lastCompleteEntryEndOffsetInKeyFile); + } + continue; + } + offsetsInKeyFile.add(keyFileEntry.offset); + lastCompleteKeyOffsets.add(lastCompleteEntryEndOffsetInKeyFile); + } + } catch (Exception e) { + logger.warn("Failed to read table tsfile size cache file {}", keyFile, e); + } finally { + closeCurrentFile(); + } + + if (offsetsInKeyFile.isEmpty()) { + return new Pair<>(0L, 0L); + } + + int keyIterIndex = 0; + long keyFileTruncateSize = 0; + long valueFileTruncateSize = 0; + + try { + openValueFile(); + while (inputStream.position() < valueFileLength && keyIterIndex < offsetsInKeyFile.size()) { + long startOffsetInKeyFile = offsetsInKeyFile.get(keyIterIndex); + long endOffsetInKeyFile = lastCompleteKeyOffsets.get(keyIterIndex); + keyIterIndex++; + long startOffset = inputStream.position(); + if (startOffset != startOffsetInKeyFile) { + break; + } + readOneEntryFromValueFile(startOffset, false); + keyFileTruncateSize = endOffsetInKeyFile; + valueFileTruncateSize = inputStream.position(); + } + } catch (Exception e) { + logger.warn( + "Failed to read table tsfile size cache {} after position: {} and {} after position: {}", + keyFile, + valueFile, + keyFileTruncateSize, + valueFileTruncateSize, + e); + } finally { + closeCurrentFile(); + } + return new Pair<>(keyFileTruncateSize, valueFileTruncateSize); + } + + public boolean readFromKeyFile( + DataRegionTableSizeQueryContext dataRegionContext, long startTime, long maxRunTime) + throws IOException { + do { + if (keyFileLength == 0) { + return true; + } + if (!hasNextEntryInKeyFile()) { + closeCurrentFile(); + return true; + } + try { + KeyFileEntry keyFileEntry = readOneEntryFromKeyFile(); + if (keyFileEntry.originTsFileID == null) { + dataRegionContext.addCachedTsFileIDAndOffsetInValueFile( + keyFileEntry.tsFileID, keyFileEntry.offset); + } else { + dataRegionContext.replaceCachedTsFileID( + keyFileEntry.originTsFileID, keyFileEntry.tsFileID); + } + } catch (IOException e) { + closeCurrentFile(); + throw e; + } + } while (System.nanoTime() - startTime < maxRunTime); + return false; + } + + public boolean hasNextEntryInKeyFile() { + return keyFileLength > 0 && inputStream.position() < keyFileLength; + } + + public KeyFileEntry readOneEntryFromKeyFile() throws IOException { + byte type = ReadWriteIOUtils.readByte(inputStream); + long timePartition = ReadWriteIOUtils.readLong(inputStream); + long timestamp = ReadWriteIOUtils.readLong(inputStream); + long fileVersion = ReadWriteIOUtils.readLong(inputStream); + long compactionVersion = ReadWriteIOUtils.readLong(inputStream); + TsFileID tsFileID = + new TsFileID(regionId, timePartition, timestamp, fileVersion, compactionVersion); + KeyFileEntry keyFileEntry; + if (type == TsFileTableDiskUsageCacheWriter.KEY_FILE_RECORD_TYPE_OFFSET) { + long offset = ReadWriteIOUtils.readLong(inputStream); + keyFileEntry = new KeyFileEntry(tsFileID, offset); + } else if (type == TsFileTableDiskUsageCacheWriter.KEY_FILE_RECORD_TYPE_REDIRECT) { + long originTimestamp = ReadWriteIOUtils.readLong(inputStream); + long originFileVersion = ReadWriteIOUtils.readLong(inputStream); + long originCompactionVersion = ReadWriteIOUtils.readLong(inputStream); + TsFileID originTsFileID = + new TsFileID( + regionId, timePartition, originTimestamp, originFileVersion, originCompactionVersion); + keyFileEntry = new KeyFileEntry(tsFileID, originTsFileID); + } else { + throw new IoTDBRuntimeException( + "Unsupported record type in file: " + keyFile.getPath() + ", type: " + type, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()); + } + return keyFileEntry; + } + + public boolean readFromValueFile( + Iterator> tsFilesToQueryInCache, + DataRegionTableSizeQueryContext dataRegionContext, + long startTime, + long maxRunTime) + throws IOException { + do { + if (!tsFilesToQueryInCache.hasNext()) { + closeCurrentFile(); + return true; + } + Pair pair = tsFilesToQueryInCache.next(); + long timePartition = pair.left.timePartitionId; + long offset = pair.right; + inputStream.seek(offset); + + int tableNum = ReadWriteForEncodingUtils.readVarInt(inputStream); + for (int i = 0; i < tableNum; i++) { + String tableName = ReadWriteIOUtils.readVarIntString(inputStream); + long size = ReadWriteIOUtils.readLong(inputStream); + dataRegionContext.updateResult(tableName, size, timePartition); + } + } while (System.nanoTime() - startTime < maxRunTime); + return false; + } + + public Map readOneEntryFromValueFile(long offset, boolean needResult) + throws IOException { + inputStream.seek(offset); + int tableNum = ReadWriteForEncodingUtils.readVarInt(inputStream); + if (tableNum <= 0) { + throw new IllegalArgumentException("tableNum should be greater than 0"); + } + Map tableSizeMap = needResult ? new HashMap<>(tableNum) : null; + for (int i = 0; i < tableNum; i++) { + String tableName = ReadWriteIOUtils.readVarIntString(inputStream); + long size = ReadWriteIOUtils.readLong(inputStream); + if (needResult) { + tableSizeMap.put(tableName, size); + } + } + return tableSizeMap; + } + + public void closeCurrentFile() { + if (inputStream != null) { + try { + inputStream.close(); + } catch (IOException ignored) { + } + inputStream = null; + } + } + + public static class KeyFileEntry { + public TsFileID tsFileID; + public TsFileID originTsFileID; + public long offset; + + public KeyFileEntry(TsFileID tsFileID, long offset) { + this.tsFileID = tsFileID; + this.offset = offset; + } + + public KeyFileEntry(TsFileID tsFileID, TsFileID originTsFileID) { + this.tsFileID = tsFileID; + this.originTsFileID = originTsFileID; + } + + public long getTimePartitionId() { + return tsFileID.timePartitionId; + } + } + + public static final class DirectBufferedSeekableFileInputStream extends InputStream { + + private final FileChannel channel; + private final ByteBuffer buffer; + + // file offset of buffer[0] + private long bufferStartPos = 0; + + // next read position + private long position = 0; + + public DirectBufferedSeekableFileInputStream(Path path, int bufferSize) throws IOException { + this.channel = FileChannel.open(path, StandardOpenOption.READ); + this.buffer = ByteBuffer.allocateDirect(bufferSize); + this.buffer.limit(0); // mark empty + } + + /** Only support forward seek: newPos >= position */ + public void seek(long newPos) throws IOException { + if (newPos < position) { + throw new UnsupportedOperationException("Backward seek is not supported"); + } + + // Fast path 0: no-op + if (newPos == position) { + return; + } + + long delta = newPos - position; + + // Fast path 1: consume remaining buffer + if (delta <= buffer.remaining()) { + buffer.position(buffer.position() + (int) delta); + position = newPos; + return; + } + + // Fast path 2: still inside buffer window (rare but safe) + long bufferEnd = bufferStartPos + buffer.limit(); + if (newPos >= bufferStartPos && newPos < bufferEnd) { + buffer.position((int) (newPos - bufferStartPos)); + position = newPos; + return; + } + + // Slow path: invalidate buffer and jump + buffer.clear(); + buffer.limit(0); + + channel.position(newPos); + bufferStartPos = newPos; + position = newPos; + } + + @Override + public int read() throws IOException { + if (!buffer.hasRemaining()) { + if (!refill()) { + return -1; + } + } + position++; + return buffer.get() & 0xFF; + } + + @Override + public int read(byte[] dst, int off, int len) throws IOException { + if (len == 0) { + return 0; + } + + int total = 0; + while (len > 0) { + if (!buffer.hasRemaining()) { + if (!refill()) { + return total == 0 ? -1 : total; + } + } + int n = Math.min(len, buffer.remaining()); + buffer.get(dst, off, n); + off += n; + len -= n; + total += n; + position += n; + } + return total; + } + + private boolean refill() throws IOException { + buffer.clear(); + channel.position(position); + bufferStartPos = position; + + int read = channel.read(buffer); + if (read <= 0) { + buffer.limit(0); + return false; + } + buffer.flip(); + return true; + } + + public long position() { + return position; + } + + @Override + public int available() throws IOException { + long remainingInFile = channel.size() - position; + if (remainingInFile <= 0) { + return 0; + } + return (int) Math.min(Integer.MAX_VALUE, remainingInFile); + } + + @Override + public void close() throws IOException { + try { + MmapUtil.clean(buffer); + } finally { + channel.close(); + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/tsfile/TsFileTableSizeIndexFileWriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/tsfile/TsFileTableSizeIndexFileWriter.java new file mode 100644 index 0000000000000..06db0869a0822 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/utils/tableDiskUsageCache/tsfile/TsFileTableSizeIndexFileWriter.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile; + +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; + +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.ReadWriteForEncodingUtils; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.BufferedOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.util.Map; + +import static org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile.TsFileTableDiskUsageCacheWriter.KEY_FILE_RECORD_TYPE_OFFSET; +import static org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile.TsFileTableDiskUsageCacheWriter.KEY_FILE_RECORD_TYPE_REDIRECT; + +public class TsFileTableSizeIndexFileWriter { + private final int regionId; + private final File currentKeyIndexFile; + private final File currentValueIndexFile; + private FileOutputStream keyFileOutputStream; + private FileOutputStream valueFileOutputStream; + private BufferedOutputStream keyBufferedOutputStream; + private BufferedOutputStream valueBufferedOutputStream; + private long keyFileSize; + private long valueFileSize; + + public TsFileTableSizeIndexFileWriter( + int regionId, File currentKeyIndexFile, File currentValueIndexFile) throws IOException { + this(regionId, currentKeyIndexFile, currentValueIndexFile, true); + } + + public TsFileTableSizeIndexFileWriter( + int regionId, File currentKeyIndexFile, File currentValueIndexFile, boolean recover) + throws IOException { + this.regionId = regionId; + this.currentKeyIndexFile = currentKeyIndexFile; + this.currentValueIndexFile = currentValueIndexFile; + currentKeyIndexFile.createNewFile(); + currentValueIndexFile.createNewFile(); + if (recover) { + recover(); + } else { + keyFileSize = currentKeyIndexFile.length(); + valueFileSize = currentValueIndexFile.length(); + } + } + + private void recover() throws IOException { + TsFileTableSizeCacheReader cacheFileReader = + new TsFileTableSizeCacheReader( + currentKeyIndexFile.length(), + currentKeyIndexFile, + currentValueIndexFile.length(), + currentValueIndexFile, + regionId); + Pair truncateSize = cacheFileReader.selfCheck(); + if (truncateSize.left != currentKeyIndexFile.length()) { + try (FileChannel channel = + FileChannel.open(currentKeyIndexFile.toPath(), StandardOpenOption.WRITE)) { + channel.truncate(truncateSize.left); + } + } + if (truncateSize.right != currentValueIndexFile.length()) { + try (FileChannel channel = + FileChannel.open(currentValueIndexFile.toPath(), StandardOpenOption.WRITE)) { + channel.truncate(truncateSize.right); + } + } + this.keyFileSize = truncateSize.left; + this.valueFileSize = truncateSize.right; + } + + public void write(TsFileID tsFileID, Map tableSizeMap) throws IOException { + ensureKeyFileOpened(); + ensureValueFileOpened(); + long valueOffset = valueFileSize; + valueFileSize += + ReadWriteForEncodingUtils.writeVarInt(tableSizeMap.size(), valueBufferedOutputStream); + for (Map.Entry entry : tableSizeMap.entrySet()) { + valueFileSize += ReadWriteIOUtils.writeVar(entry.getKey(), valueBufferedOutputStream); + valueFileSize += ReadWriteIOUtils.write(entry.getValue(), valueBufferedOutputStream); + } + keyFileSize += ReadWriteIOUtils.write(KEY_FILE_RECORD_TYPE_OFFSET, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(tsFileID.timePartitionId, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(tsFileID.timestamp, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(tsFileID.fileVersion, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(tsFileID.compactionVersion, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(valueOffset, keyBufferedOutputStream); + } + + public void write(TsFileID originTsFileID, TsFileID newTsFileID) throws IOException { + ensureKeyFileOpened(); + keyFileSize += ReadWriteIOUtils.write(KEY_FILE_RECORD_TYPE_REDIRECT, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(newTsFileID.timePartitionId, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(newTsFileID.timestamp, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(newTsFileID.fileVersion, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(newTsFileID.compactionVersion, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(originTsFileID.timestamp, keyBufferedOutputStream); + keyFileSize += ReadWriteIOUtils.write(originTsFileID.fileVersion, keyBufferedOutputStream); + keyFileSize += + ReadWriteIOUtils.write(originTsFileID.compactionVersion, keyBufferedOutputStream); + } + + private void ensureKeyFileOpened() throws IOException { + if (keyFileOutputStream == null) { + keyFileOutputStream = new FileOutputStream(currentKeyIndexFile, true); + keyFileSize = currentKeyIndexFile.length(); + keyBufferedOutputStream = new BufferedOutputStream(keyFileOutputStream); + } + } + + private void ensureValueFileOpened() throws IOException { + if (valueFileOutputStream == null) { + valueFileOutputStream = new FileOutputStream(currentValueIndexFile, true); + valueFileSize = currentValueIndexFile.length(); + valueBufferedOutputStream = new BufferedOutputStream(valueFileOutputStream); + } + } + + public void flush() throws IOException { + if (valueBufferedOutputStream != null) { + valueBufferedOutputStream.flush(); + } + if (keyFileOutputStream != null) { + keyBufferedOutputStream.flush(); + } + } + + public File getKeyFile() { + return currentKeyIndexFile; + } + + public File getValueFile() { + return currentValueIndexFile; + } + + public long keyFileLength() { + return keyFileSize; + } + + public long valueFileLength() { + return valueFileSize; + } + + public void sync() throws IOException { + flush(); + if (valueFileOutputStream != null) { + valueFileOutputStream.getFD().sync(); + } + if (keyFileOutputStream != null) { + keyFileOutputStream.getFD().sync(); + } + } + + public void close() { + try { + sync(); + } catch (IOException ignored) { + } + try { + if (valueBufferedOutputStream != null) { + valueBufferedOutputStream.close(); + valueBufferedOutputStream = null; + valueFileOutputStream = null; + } + } catch (IOException ignored) { + } + try { + if (keyBufferedOutputStream != null) { + keyBufferedOutputStream.close(); + keyBufferedOutputStream = null; + keyFileOutputStream = null; + } + } catch (IOException ignored) { + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java index 83372a5a6c0b6..daf16bcc2d399 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java @@ -586,7 +586,12 @@ private void loadAll( tsFileResource, timePartitionProgressIndexMap.getOrDefault( entry.getKey().getTimePartitionSlot(), MinimumProgressIndex.INSTANCE)); - dataRegion.loadNewTsFile(tsFileResource, true, isGeneratedByPipe, false); + dataRegion.loadNewTsFile( + tsFileResource, + true, + isGeneratedByPipe, + false, + Optional.ofNullable(writer.getTableSizeMap())); // Metrics dataRegion diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/InnerTimeJoinOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/InnerTimeJoinOperatorTest.java index 5aacf5674b53d..e0b8148bb5d02 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/InnerTimeJoinOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/InnerTimeJoinOperatorTest.java @@ -87,7 +87,8 @@ public void testInnerJoin1() { // 13 13, 130 OperatorContext operatorContext = Mockito.mock(OperatorContext.class); - Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getMaxRunTimeForTest()) + .thenReturn(new Duration(1, TimeUnit.SECONDS)); Operator leftChild = new Operator() { @@ -375,7 +376,8 @@ public void testInnerJoin2() { // 3 3 4 30.0 false OperatorContext operatorContext = Mockito.mock(OperatorContext.class); - Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1000, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getMaxRunTimeForTest()) + .thenReturn(new Duration(1000, TimeUnit.SECONDS)); Operator leftChild = new Operator() { @@ -692,7 +694,8 @@ public void testInnerJoin3() { // 10 10 20 null false OperatorContext operatorContext = Mockito.mock(OperatorContext.class); - Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1000, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getMaxRunTimeForTest()) + .thenReturn(new Duration(1000, TimeUnit.SECONDS)); Operator child1 = new Operator() { @@ -996,7 +999,8 @@ public void testInnerJoin4() { // empty OperatorContext operatorContext = Mockito.mock(OperatorContext.class); - Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1000, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getMaxRunTimeForTest()) + .thenReturn(new Duration(1000, TimeUnit.SECONDS)); Operator child1 = new Operator() { @@ -1375,7 +1379,8 @@ public void testInnerJoin5() { // 90 90 180 null false "love" OperatorContext operatorContext = Mockito.mock(OperatorContext.class); - Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1000, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getMaxRunTimeForTest()) + .thenReturn(new Duration(1000, TimeUnit.SECONDS)); Operator child1 = new Operator() { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/LeftOuterTimeJoinOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/LeftOuterTimeJoinOperatorTest.java index a71c31e65cd94..3970d0a636831 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/LeftOuterTimeJoinOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/process/join/LeftOuterTimeJoinOperatorTest.java @@ -85,7 +85,8 @@ public void testLeftOuterJoin1() { // 25 25 null OperatorContext operatorContext = Mockito.mock(OperatorContext.class); - Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getMaxRunTimeForTest()) + .thenReturn(new Duration(1, TimeUnit.SECONDS)); Operator leftChild = new Operator() { @@ -374,7 +375,8 @@ public void testLeftOuterJoin2() { // 3 3 4 30.0 false OperatorContext operatorContext = Mockito.mock(OperatorContext.class); - Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getMaxRunTimeForTest()) + .thenReturn(new Duration(1, TimeUnit.SECONDS)); Operator leftChild = new Operator() { @@ -702,7 +704,8 @@ public void testLeftOuterJoin3() { // 111 111 null OperatorContext operatorContext = Mockito.mock(OperatorContext.class); - Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getMaxRunTimeForTest()) + .thenReturn(new Duration(1, TimeUnit.SECONDS)); Operator leftChild = new Operator() { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/parser/StatementGeneratorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/parser/StatementGeneratorTest.java index 4d27792e5328b..168923c7a4285 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/parser/StatementGeneratorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/parser/StatementGeneratorTest.java @@ -63,6 +63,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.metadata.template.UnsetSchemaTemplateStatement; import org.apache.iotdb.db.queryengine.plan.statement.metadata.view.CreateLogicalViewStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.AuthorStatement; +import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowDiskUsageStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement; import org.apache.iotdb.isession.template.TemplateNode; import org.apache.iotdb.rpc.StatementExecutionException; @@ -120,6 +121,37 @@ public class StatementGeneratorTest { + @Test + public void testShowDiskUsage() { + + Statement showDiskUsage = + StatementGenerator.createStatement( + "show disk_usage from root.test.** order by database, datanodeid, regionid, timepartition, sizeinbytes", + ZonedDateTime.now().getOffset()); + Assert.assertTrue(showDiskUsage instanceof ShowDiskUsageStatement); + Assert.assertEquals( + ((ShowDiskUsageStatement) showDiskUsage).getSortItemList().get(0), + new SortItem(OrderByKey.DATABASE, Ordering.ASC)); + Assert.assertEquals( + ((ShowDiskUsageStatement) showDiskUsage).getSortItemList().get(1), + new SortItem(OrderByKey.DATANODEID, Ordering.ASC)); + Assert.assertEquals( + ((ShowDiskUsageStatement) showDiskUsage).getSortItemList().get(2), + new SortItem(OrderByKey.REGIONID, Ordering.ASC)); + Assert.assertEquals( + ((ShowDiskUsageStatement) showDiskUsage).getSortItemList().get(3), + new SortItem(OrderByKey.TIMEPARTITION, Ordering.ASC)); + Assert.assertEquals( + ((ShowDiskUsageStatement) showDiskUsage).getSortItemList().get(4), + new SortItem(OrderByKey.SIZEINBYTES, Ordering.ASC)); + + Assert.assertThrows( + SemanticException.class, + () -> + StatementGenerator.createStatement( + "show disk_usage from root.test.** order by a", ZonedDateTime.now().getOffset())); + } + @Test public void testShowQueries() { Statement showQueries = diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/source/SourceNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/source/SourceNodeSerdeTest.java index 1b68a32c4c5c2..e3f781453d4cb 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/source/SourceNodeSerdeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/source/SourceNodeSerdeTest.java @@ -19,19 +19,32 @@ package org.apache.iotdb.db.queryengine.plan.planner.node.source; +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.db.queryengine.plan.planner.node.PlanNodeDeserializeHelper; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.LastQueryScanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowDiskUsageNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.ShowQueriesNode; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableDiskUsageInformationSchemaTableScanNode; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.read.common.type.IntType; +import org.apache.tsfile.read.common.type.LongType; +import org.apache.tsfile.read.common.type.StringType; import org.apache.tsfile.write.schema.MeasurementSchema; import org.junit.Test; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import static org.junit.Assert.assertEquals; @@ -73,6 +86,22 @@ public void testLastQueryScanNode() throws IllegalPathException { assertEquals(PlanNodeDeserializeHelper.deserialize(byteBuffer), node); } + @Test + public void testShowDiskUsageNode() throws IllegalPathException { + ShowDiskUsageNode node = + new ShowDiskUsageNode(new PlanNodeId("test"), null, new PartialPath("root.test.d1")); + + ByteBuffer byteBuffer = ByteBuffer.allocate(2048); + node.serialize(byteBuffer); + byteBuffer.flip(); + assertEquals(PlanNodeDeserializeHelper.deserialize(byteBuffer), node); + node = new ShowDiskUsageNode(new PlanNodeId("test"), null, new PartialPath("root.test.d1")); + byteBuffer = ByteBuffer.allocate(2048); + node.serialize(byteBuffer); + byteBuffer.flip(); + assertEquals(PlanNodeDeserializeHelper.deserialize(byteBuffer), node); + } + @Test public void testShowQueriesNode() throws IllegalPathException { ShowQueriesNode node = new ShowQueriesNode(new PlanNodeId("test"), null, "root"); @@ -87,4 +116,50 @@ public void testShowQueriesNode() throws IllegalPathException { byteBuffer.flip(); assertEquals(PlanNodeDeserializeHelper.deserialize(byteBuffer), node); } + + @Test + public void testTableDiskUsageInformationTableScanNode() throws IllegalPathException { + List symbols = Arrays.asList(new Symbol("database"), new Symbol("size_in_bytes")); + Map assignments = new HashMap<>(); + assignments.put( + new Symbol("database"), + new ColumnSchema("database", StringType.getInstance(), false, TsTableColumnCategory.FIELD)); + assignments.put( + new Symbol("table_name"), + new ColumnSchema( + "table_name", StringType.getInstance(), false, TsTableColumnCategory.FIELD)); + assignments.put( + new Symbol("datanode_id"), + new ColumnSchema("datanode_id", IntType.getInstance(), false, TsTableColumnCategory.FIELD)); + assignments.put( + new Symbol("region_id"), + new ColumnSchema("region_id", IntType.getInstance(), false, TsTableColumnCategory.FIELD)); + assignments.put( + new Symbol("time_partition"), + new ColumnSchema( + "time_partition", LongType.getInstance(), false, TsTableColumnCategory.FIELD)); + assignments.put( + new Symbol("size_in_bytes"), + new ColumnSchema( + "size_in_bytes", LongType.getInstance(), false, TsTableColumnCategory.FIELD)); + TableDiskUsageInformationSchemaTableScanNode node = + new TableDiskUsageInformationSchemaTableScanNode( + new PlanNodeId("test"), + new QualifiedObjectName("test", "table1"), + symbols, + assignments, + null, + 1, + 1, + new TRegionReplicaSet(), + Arrays.asList(1)); + ByteBuffer byteBuffer = ByteBuffer.allocate(2048); + node.serialize(byteBuffer); + byteBuffer.flip(); + assertEquals( + ((TableDiskUsageInformationSchemaTableScanNode) + PlanNodeDeserializeHelper.deserialize(byteBuffer)) + .getRegions(), + node.getRegions()); + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofInnerJoinOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofInnerJoinOperatorTest.java index de54d7faa0270..9fe263685549f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofInnerJoinOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/AsofInnerJoinOperatorTest.java @@ -332,7 +332,8 @@ private void testExecute( DriverContext driverContext = Mockito.mock(DriverContext.class); Mockito.when(driverContext.getFragmentInstanceContext()).thenReturn(fragmentInstanceContext); OperatorContext operatorContext = Mockito.mock(OperatorContext.class); - Mockito.when(operatorContext.getMaxRunTime()).thenReturn(new Duration(1, TimeUnit.SECONDS)); + Mockito.when(operatorContext.getMaxRunTimeForTest()) + .thenReturn(new Duration(1, TimeUnit.SECONDS)); Mockito.when(operatorContext.getDriverContext()).thenReturn(driverContext); Operator leftChild = diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/service/DataNodeInternalRPCServiceImplTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/service/DataNodeInternalRPCServiceImplTest.java index 622c2c4ebbfe7..22a2d6cdc461f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/service/DataNodeInternalRPCServiceImplTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/service/DataNodeInternalRPCServiceImplTest.java @@ -78,6 +78,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; public class DataNodeInternalRPCServiceImplTest { @@ -209,7 +210,7 @@ public void testRejectLoad4NonActiveImpl() throws LoadFileException { ((IoTConsensus) DataRegionConsensusImpl.getInstance()) .getImpl(new DataRegionId(1)) .setActive(false); - dataRegion.loadNewTsFile(new TsFileResource(), false, false, false); + dataRegion.loadNewTsFile(new TsFileResource(), false, false, false, Optional.empty()); ((IoTConsensus) DataRegionConsensusImpl.getInstance()) .getImpl(new DataRegionId(1)) .setActive(true); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/utils/TableDiskUsageTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/utils/TableDiskUsageTest.java new file mode 100644 index 0000000000000..4ec7a8b932d60 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/utils/TableDiskUsageTest.java @@ -0,0 +1,300 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils; + +import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.compaction.AbstractCompactionTest; +import org.apache.iotdb.db.storageengine.dataregion.compaction.tablemodel.CompactionTableModelTestFileWriter; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.DataRegionTableSizeQueryContext; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCacheReader; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TimePartitionTableSizeQueryContext; + +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.utils.Pair; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class TableDiskUsageTest extends AbstractCompactionTest { + + private DataRegion mockDataRegion; + private TsFileManager mockTsFileManager; + + @Before + @Override + public void setUp() + throws IOException, WriteProcessException, MetadataException, InterruptedException { + super.setUp(); + TableDiskUsageCache.getInstance().ensureRunning(); + mockDataRegion = Mockito.mock(DataRegion.class); + Mockito.when(mockDataRegion.getDatabaseName()).thenReturn("test"); + Mockito.when(mockDataRegion.getDataRegionId()).thenReturn(new DataRegionId(0)); + Mockito.when(mockDataRegion.getDataRegionIdString()).thenReturn("0"); + Mockito.when(mockDataRegion.isTableModel()).thenReturn(true); + StorageEngine.getInstance().setDataRegion(new DataRegionId(0), mockDataRegion); + mockTsFileManager = new TsFileManager("test", "0", ""); + Mockito.when(mockDataRegion.getTsFileManager()).thenReturn(mockTsFileManager); + TableDiskUsageCache.getInstance().registerRegion(mockDataRegion); + } + + @After + @Override + public void tearDown() throws IOException, StorageEngineException { + super.tearDown(); + StorageEngine.getInstance().deleteDataRegion(new DataRegionId(0)); + } + + @Test + public void test1() throws Exception { + TsFileResource resource = prepareFile(4); + mockTsFileManager.add(resource, true); + + DataRegionTableSizeQueryContext context = new DataRegionTableSizeQueryContext(false); + // only query table1 and table2 + Map timePartitionTableSizeMap = new HashMap<>(); + timePartitionTableSizeMap.put("table1", 0L); + timePartitionTableSizeMap.put("table2", 0L); + context.addTimePartition(0, new TimePartitionTableSizeQueryContext(timePartitionTableSizeMap)); + queryTableSize(context); + int entryNum = 0; + for (Map.Entry timePartitionEntry : + context.getTimePartitionTableSizeQueryContextMap().entrySet()) { + TimePartitionTableSizeQueryContext timePartitionContext = timePartitionEntry.getValue(); + for (Map.Entry entry : + timePartitionContext.getTableSizeResultMap().entrySet()) { + String tableName = entry.getKey(); + long size = entry.getValue(); + Assert.assertNotEquals("table3", tableName); + Assert.assertNotEquals("table4", tableName); + Assert.assertTrue(size > 0); + entryNum++; + } + } + Assert.assertEquals(2, entryNum); + } + + @Test + public void test2() throws Exception { + // cached + TsFileResource resource1 = prepareFile(4); + mockTsFileManager.add(resource1, true); + Map tableSizeMap = new HashMap<>(); + tableSizeMap.put("table1", 10000000L); + tableSizeMap.put("table2", 10000000L); + TableDiskUsageCache.getInstance() + .write(mockDataRegion.getDatabaseName(), resource1.getTsFileID(), tableSizeMap); + + TsFileResource resource2 = prepareFile(4); + mockTsFileManager.add(resource2, true); + + DataRegionTableSizeQueryContext context = new DataRegionTableSizeQueryContext(false); + // only query table1 and table2 + Map timePartitionTableSizeMap = new HashMap<>(); + timePartitionTableSizeMap.put("table1", 0L); + timePartitionTableSizeMap.put("table2", 0L); + context.addTimePartition(0, new TimePartitionTableSizeQueryContext(timePartitionTableSizeMap)); + queryTableSize(context); + int entryNum = 0; + for (Map.Entry timePartitionEntry : + context.getTimePartitionTableSizeQueryContextMap().entrySet()) { + TimePartitionTableSizeQueryContext timePartitionContext = timePartitionEntry.getValue(); + for (Map.Entry entry : + timePartitionContext.getTableSizeResultMap().entrySet()) { + String tableName = entry.getKey(); + long size = entry.getValue(); + Assert.assertNotEquals("table3", tableName); + Assert.assertNotEquals("table4", tableName); + Assert.assertTrue(size > 10000000L); + entryNum++; + } + } + Assert.assertEquals(2, entryNum); + } + + @Test + public void test3() throws Exception { + // deleted + TsFileResource resource1 = prepareFile(4); + Map tableSizeMap = new HashMap<>(); + tableSizeMap.put("table1", 10000000L); + tableSizeMap.put("table2", 10000000L); + TableDiskUsageCache.getInstance() + .write(mockDataRegion.getDatabaseName(), resource1.getTsFileID(), tableSizeMap); + + TsFileResource resource2 = prepareFile(4); + mockTsFileManager.add(resource2, true); + + DataRegionTableSizeQueryContext context = new DataRegionTableSizeQueryContext(false); + // only query table1 and table2 + Map timePartitionTableSizeMap = new HashMap<>(); + timePartitionTableSizeMap.put("table1", 0L); + timePartitionTableSizeMap.put("table2", 0L); + context.addTimePartition(0, new TimePartitionTableSizeQueryContext(timePartitionTableSizeMap)); + TableDiskUsageCacheReader reader = + new TableDiskUsageCacheReader(mockDataRegion, context, false); + queryTableSize(context); + int entryNum = 0; + for (Map.Entry timePartitionEntry : + context.getTimePartitionTableSizeQueryContextMap().entrySet()) { + TimePartitionTableSizeQueryContext timePartitionContext = timePartitionEntry.getValue(); + for (Map.Entry entry : + timePartitionContext.getTableSizeResultMap().entrySet()) { + String tableName = entry.getKey(); + long size = entry.getValue(); + Assert.assertNotEquals("table3", tableName); + Assert.assertNotEquals("table4", tableName); + Assert.assertTrue(size < 10000000L && size > 0); + entryNum++; + } + } + Assert.assertEquals(2, entryNum); + } + + @Test + public void test4() throws Exception { + TsFileResource resource1 = prepareFile(4); + Map tableSizeMap = new HashMap<>(); + tableSizeMap.put("table1", 10000000L); + tableSizeMap.put("table2", 10000000L); + TableDiskUsageCache.getInstance() + .write(mockDataRegion.getDatabaseName(), resource1.getTsFileID(), tableSizeMap); + + TsFileResource resource2 = prepareFile(4); + mockTsFileManager.add(resource2, true); + // resource1 renamed to resource2 and recorded in cache + TableDiskUsageCache.getInstance() + .write(mockDataRegion.getDatabaseName(), resource1.getTsFileID(), resource2.getTsFileID()); + + DataRegionTableSizeQueryContext context = new DataRegionTableSizeQueryContext(false); + // only query table1 and table2 + Map timePartitionTableSizeMap = new HashMap<>(); + timePartitionTableSizeMap.put("table1", 0L); + timePartitionTableSizeMap.put("table2", 0L); + context.addTimePartition(0, new TimePartitionTableSizeQueryContext(timePartitionTableSizeMap)); + TableDiskUsageCacheReader reader = + new TableDiskUsageCacheReader(mockDataRegion, context, false); + queryTableSize(context); + int entryNum = 0; + for (Map.Entry timePartitionEntry : + context.getTimePartitionTableSizeQueryContextMap().entrySet()) { + TimePartitionTableSizeQueryContext timePartitionContext = timePartitionEntry.getValue(); + for (Map.Entry entry : + timePartitionContext.getTableSizeResultMap().entrySet()) { + String tableName = entry.getKey(); + long size = entry.getValue(); + Assert.assertNotEquals("table3", tableName); + Assert.assertNotEquals("table4", tableName); + Assert.assertEquals(10000000L, size); + entryNum++; + } + } + Assert.assertEquals(2, entryNum); + } + + @Test + public void testCalculateTableSizeFromFile() throws Exception { + Pair> resourceTableSizeMapPair = + prepareFileAndTableSizeMap(10, 100000); + TsFileResource resource1 = resourceTableSizeMapPair.getLeft(); + Map tableSizeMapGeneratedByWrite = resourceTableSizeMapPair.getRight(); + Assert.assertEquals(10, tableSizeMapGeneratedByWrite.size()); + for (Long value : tableSizeMapGeneratedByWrite.values()) { + Assert.assertTrue(value > 0); + } + mockTsFileManager.add(resource1, true); + + DataRegionTableSizeQueryContext context = new DataRegionTableSizeQueryContext(true); + queryTableSize(context); + Assert.assertEquals(1, context.getTimePartitionTableSizeQueryContextMap().size()); + TimePartitionTableSizeQueryContext timePartitionContext = + context.getTimePartitionTableSizeQueryContextMap().values().iterator().next(); + Map tableSizeMapGeneratedByQuery = timePartitionContext.getTableSizeResultMap(); + Assert.assertEquals(10, tableSizeMapGeneratedByQuery.size()); + + for (Map.Entry entry : tableSizeMapGeneratedByQuery.entrySet()) { + String tableName = entry.getKey(); + Long size = tableSizeMapGeneratedByWrite.get(tableName); + Assert.assertNotNull(size); + Assert.assertTrue(Math.abs(size - entry.getValue()) < 1000); + } + } + + private void queryTableSize(DataRegionTableSizeQueryContext queryContext) throws Exception { + TableDiskUsageCacheReader reader = + new TableDiskUsageCacheReader(mockDataRegion, queryContext, false); + try { + Assert.assertTrue(reader.prepareCacheReader(System.nanoTime(), Long.MAX_VALUE)); + Assert.assertTrue(reader.loadObjectFileTableSizeCache(System.nanoTime(), Long.MAX_VALUE)); + Assert.assertTrue(reader.prepareCachedTsFileIDKeys(System.nanoTime(), Long.MAX_VALUE)); + Assert.assertTrue(reader.checkAllFilesInTsFileManager(System.nanoTime(), Long.MAX_VALUE)); + Assert.assertTrue( + reader.readCacheValueFilesAndUpdateResultMap(System.nanoTime(), Long.MAX_VALUE)); + } finally { + reader.close(); + } + } + + private TsFileResource prepareFile(int tableNum) throws IOException { + return prepareFileAndTableSizeMap(tableNum, 100).getLeft(); + } + + private Pair> prepareFileAndTableSizeMap( + int tableNum, int pointNum) throws IOException { + TsFileResource resource1 = createEmptyFileAndResource(true); + Map tableSizeMap = null; + try (CompactionTableModelTestFileWriter writer = + new CompactionTableModelTestFileWriter(resource1)) { + for (int i = 0; i < tableNum; i++) { + writer.registerTableSchema("table" + i, Collections.singletonList("device")); + writer.startChunkGroup("table" + i, Collections.singletonList("d1")); + writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue( + Arrays.asList("s0", "s1"), + new TimeRange[][][] { + new TimeRange[][] {new TimeRange[] {new TimeRange(10, 10 + pointNum - 1)}} + }, + TSEncoding.PLAIN, + CompressionType.LZ4, + Arrays.asList(false, false)); + writer.endChunkGroup(); + } + writer.endFile(); + tableSizeMap = writer.getFileWriter().getTableSizeMap(); + } + return new Pair<>(resource1, tableSizeMap); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/utils/TsFileTableSizeCacheWriterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/utils/TsFileTableSizeCacheWriterTest.java new file mode 100644 index 0000000000000..b173188268ff2 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/utils/TsFileTableSizeCacheWriterTest.java @@ -0,0 +1,468 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.utils; + +import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.compaction.AbstractCompactionTest; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileID; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.AbstractTableSizeCacheWriter; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.DataRegionTableSizeQueryContext; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TimePartitionTableSizeQueryContext; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile.TsFileTableDiskUsageCacheWriter; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.tsfile.TsFileTableSizeCacheReader; + +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.ReadWriteForEncodingUtils; +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TsFileTableSizeCacheWriterTest extends AbstractCompactionTest { + + private DataRegion mockDataRegion; + private TsFileManager mockTsFileManager; + + @Before + @Override + public void setUp() + throws IOException, WriteProcessException, MetadataException, InterruptedException { + super.setUp(); + mockDataRegion = Mockito.mock(DataRegion.class); + Mockito.when(mockDataRegion.getDatabaseName()).thenReturn("test"); + Mockito.when(mockDataRegion.getDataRegionIdString()).thenReturn("0"); + StorageEngine.getInstance().setDataRegion(new DataRegionId(0), mockDataRegion); + mockTsFileManager = new TsFileManager("test", "0", ""); + Mockito.when(mockDataRegion.getTsFileManager()).thenReturn(mockTsFileManager); + } + + @After + @Override + public void tearDown() throws IOException, StorageEngineException { + super.tearDown(); + StorageEngine.getInstance().deleteDataRegion(new DataRegionId(0)); + } + + @Test + public void testCompactEmptyTargetFile() throws IOException { + TsFileTableDiskUsageCacheWriter writer = + new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + File oldKeyFile = writer.getKeyFile(); + File oldValueFile = writer.getValueFile(); + Assert.assertEquals("TableSizeKeyFile_0", oldKeyFile.getName()); + Assert.assertEquals("TableSizeValueFile_0", oldValueFile.getName()); + + TsFileResource resource1 = createEmptyFileAndResourceWithName("1-1-0-0.tsfile", 1, true); + TsFileResource resource2 = createEmptyFileAndResourceWithName("2-2-0-0.tsfile", 1, true); + TsFileResource resource3 = createEmptyFileAndResourceWithName("3-3-0-0.tsfile", 1, false); + + writer.write(resource1.getTsFileID(), Collections.singletonMap("table1", 10L)); + writer.write(resource2.getTsFileID(), Collections.singletonMap("table1", 20L)); + writer.write(resource3.getTsFileID(), Collections.singletonMap("table2", 200L)); + + writer.compact(); + + Assert.assertFalse(oldKeyFile.exists()); + Assert.assertEquals("TableSizeKeyFile_1", writer.getKeyFile().getName()); + Assert.assertEquals("TableSizeValueFile_1", writer.getValueFile().getName()); + Assert.assertEquals(0, writer.keyFileLength()); + Assert.assertEquals(0, writer.valueFileLength()); + writer.close(); + } + + @Test + public void testCompactTargetFile1() throws IOException { + TsFileTableDiskUsageCacheWriter writer = + new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + File oldKeyFile = writer.getKeyFile(); + File oldValueFile = writer.getValueFile(); + Assert.assertEquals("TableSizeKeyFile_0", oldKeyFile.getName()); + Assert.assertEquals("TableSizeValueFile_0", oldValueFile.getName()); + + TsFileResource resource1 = createEmptyFileAndResourceWithName("1-1-0-0.tsfile", 1, true); + TsFileResource resource2 = createEmptyFileAndResourceWithName("2-2-0-0.tsfile", 1, true); + TsFileResource resource3 = createEmptyFileAndResourceWithName("3-3-0-0.tsfile", 1, false); + mockTsFileManager.add(resource1, true); + mockTsFileManager.add(resource3, false); + + writer.write(resource1.getTsFileID(), Collections.singletonMap("table1", 10L)); + writer.write(resource2.getTsFileID(), Collections.singletonMap("table1", 10L)); + writer.write(resource3.getTsFileID(), Collections.singletonMap("table2", 200L)); + + writer.compact(); + + File targetKeyFile = writer.getKeyFile(); + File targetValueFile = writer.getValueFile(); + Assert.assertFalse(oldKeyFile.exists()); + Assert.assertFalse(oldValueFile.exists()); + Assert.assertEquals("TableSizeKeyFile_1", targetKeyFile.getName()); + Assert.assertEquals("TableSizeValueFile_1", targetValueFile.getName()); + writer.close(); + + TsFileTableSizeCacheReader reader = + new TsFileTableSizeCacheReader( + targetKeyFile.length(), targetKeyFile, targetValueFile.length(), targetValueFile, 1); + reader.openKeyFile(); + int count = 0; + while (reader.hasNextEntryInKeyFile()) { + TsFileTableSizeCacheReader.KeyFileEntry keyFileEntry = reader.readOneEntryFromKeyFile(); + count++; + } + reader.closeCurrentFile(); + Assert.assertEquals(2, count); + } + + @Test + public void testCompactTargetFile2() throws IOException { + TsFileTableDiskUsageCacheWriter writer = + new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + File oldKeyFile = writer.getKeyFile(); + File oldValueFile = writer.getValueFile(); + Assert.assertEquals("TableSizeKeyFile_0", oldKeyFile.getName()); + Assert.assertEquals("TableSizeValueFile_0", oldValueFile.getName()); + + TsFileResource resource1 = createEmptyFileAndResourceWithName("1-1-0-0.tsfile", 1, true); + TsFileResource resource2 = createEmptyFileAndResourceWithName("2-2-0-0.tsfile", 1, true); + TsFileResource resource3 = createEmptyFileAndResourceWithName("3-3-0-0.tsfile", 1, false); + mockTsFileManager.add(resource1, true); + mockTsFileManager.add(resource2, true); + + writer.write(resource1.getTsFileID(), Collections.singletonMap("table1", 10L)); + writer.write(resource3.getTsFileID(), Collections.singletonMap("table2", 200L)); + writer.write(resource3.getTsFileID(), resource2.getTsFileID()); + + writer.compact(); + + File targetKeyFile = writer.getKeyFile(); + File targetValueFile = writer.getValueFile(); + Assert.assertFalse(oldKeyFile.exists()); + Assert.assertFalse(oldValueFile.exists()); + Assert.assertEquals("TableSizeKeyFile_1", targetKeyFile.getName()); + Assert.assertEquals("TableSizeValueFile_1", targetValueFile.getName()); + writer.close(); + + TsFileTableSizeCacheReader reader = + new TsFileTableSizeCacheReader( + targetKeyFile.length(), targetKeyFile, targetValueFile.length(), targetValueFile, 1); + reader.openKeyFile(); + int count = 0; + while (reader.hasNextEntryInKeyFile()) { + TsFileTableSizeCacheReader.KeyFileEntry keyFileEntry = reader.readOneEntryFromKeyFile(); + Assert.assertNotEquals(3, keyFileEntry.tsFileID.fileVersion); + count++; + } + reader.closeCurrentFile(); + Assert.assertEquals(2, count); + } + + @Test + public void testReadPerformance() throws IOException { + TsFileTableDiskUsageCacheWriter writer = + new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + for (int i = 1; i <= 10; i++) { + for (int j = 1; j <= 10000; j++) { + TsFileResource resource = + createEmptyFileAndResourceWithName(j + "-" + j + "-0-0.tsfile", i, true); + writer.write(resource.getTsFileID(), generateTableSizeMap(i)); + } + } + Assert.assertNotEquals(writer.getKeyFile().length(), writer.keyFileLength()); + Assert.assertNotEquals(writer.getValueFile().length(), writer.valueFileLength()); + writer.close(); + Assert.assertEquals(writer.getKeyFile().length(), writer.keyFileLength()); + Assert.assertEquals(writer.getValueFile().length(), writer.valueFileLength()); + File keyFile = writer.getKeyFile(); + File valueFile = writer.getValueFile(); + DataRegionTableSizeQueryContext context = new DataRegionTableSizeQueryContext(true); + TsFileTableSizeCacheReader reader = + new TsFileTableSizeCacheReader(keyFile.length(), keyFile, valueFile.length(), valueFile, 0); + List> offsets = new ArrayList<>(); + long start = System.currentTimeMillis(); + reader.openKeyFile(); + for (int i = 1; i <= 10; i++) { + for (int j = 1; j <= 10000; j++) { + Assert.assertTrue(reader.hasNextEntryInKeyFile()); + TsFileTableSizeCacheReader.KeyFileEntry entry = reader.readOneEntryFromKeyFile(); + Assert.assertNotNull(entry); + Assert.assertNull(entry.originTsFileID); + Assert.assertEquals(new TsFileID(0, i, j, j, 0L), entry.tsFileID); + Assert.assertTrue(entry.offset >= 0); + offsets.add(new Pair<>(entry.tsFileID, entry.offset)); + } + } + Assert.assertFalse(reader.hasNextEntryInKeyFile()); + reader.closeCurrentFile(); + reader.openValueFile(); + reader.readFromValueFile(offsets.iterator(), context, System.nanoTime(), Long.MAX_VALUE); + reader.closeCurrentFile(); + System.out.println("cost: " + (System.currentTimeMillis() - start) + "ms"); + System.out.println("keyFileLength: " + keyFile.length()); + System.out.println("valueFileLength: " + valueFile.length()); + + Assert.assertEquals(10, context.getTimePartitionTableSizeQueryContextMap().size()); + for (Map.Entry entry : + context.getTimePartitionTableSizeQueryContextMap().entrySet()) { + Map tableSizeResultMap = entry.getValue().getTableSizeResultMap(); + for (Map.Entry tableSizeEntry : tableSizeResultMap.entrySet()) { + int i = Integer.parseInt(tableSizeEntry.getKey().substring(5)); + Assert.assertEquals(Long.valueOf(i * 10000L), tableSizeEntry.getValue()); + } + } + } + + @Test + public void testRecoverWriter1() throws IOException { + TsFileTableDiskUsageCacheWriter writer = + new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + for (int i = 1; i <= 10; i++) { + for (int j = 1; j <= 10; j++) { + TsFileResource resource = + createEmptyFileAndResourceWithName(j + "-" + j + "-0-0.tsfile", i, true); + writer.write(resource.getTsFileID(), generateTableSizeMap(i)); + } + } + writer.close(); + File keyFile = writer.getKeyFile(); + long keyFileValidLength = keyFile.length(); + File valueFile = writer.getValueFile(); + long valueFileValidLength = valueFile.length(); + Files.write(keyFile.toPath(), new byte[] {1, 2, 3, 4}, StandardOpenOption.APPEND); + Files.write(valueFile.toPath(), new byte[] {1, 2, 3, 4}, StandardOpenOption.APPEND); + Assert.assertEquals(keyFileValidLength + 4, keyFile.length()); + Assert.assertEquals(valueFileValidLength + 4, valueFile.length()); + + writer = new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + writer.close(); + Assert.assertEquals(keyFileValidLength, keyFile.length()); + Assert.assertEquals(valueFileValidLength, valueFile.length()); + + Files.write(keyFile.toPath(), new byte[] {0, 0, 0, 0}, StandardOpenOption.APPEND); + writer = new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + writer.close(); + Assert.assertEquals(keyFileValidLength, keyFile.length()); + Assert.assertEquals(valueFileValidLength, valueFile.length()); + + ByteBuffer buffer = + ByteBuffer.allocate(TsFileTableDiskUsageCacheWriter.KEY_FILE_OFFSET_RECORD_LENGTH); + TsFileID tsFileID = new TsFileID(0, 1, 2, 3, 4); + ReadWriteIOUtils.write(TsFileTableDiskUsageCacheWriter.KEY_FILE_RECORD_TYPE_OFFSET, buffer); + ReadWriteIOUtils.write(tsFileID.timePartitionId, buffer); + ReadWriteIOUtils.write(tsFileID.timestamp, buffer); + ReadWriteIOUtils.write(tsFileID.fileVersion, buffer); + ReadWriteIOUtils.write(tsFileID.compactionVersion, buffer); + ReadWriteIOUtils.write(valueFileValidLength, buffer); + Files.write( + keyFile.toPath(), + Arrays.copyOf(buffer.array(), buffer.position()), + StandardOpenOption.APPEND); + writer = new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + writer.close(); + Assert.assertEquals(keyFileValidLength, keyFile.length()); + Assert.assertEquals(valueFileValidLength, valueFile.length()); + + buffer.clear(); + ReadWriteForEncodingUtils.writeVarInt(1, buffer); + ReadWriteIOUtils.writeVar("table1", buffer); + ReadWriteIOUtils.write(10L, buffer); + Files.write( + valueFile.toPath(), + Arrays.copyOf(buffer.array(), buffer.position()), + StandardOpenOption.APPEND); + writer = new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + writer.close(); + Assert.assertEquals(keyFileValidLength, keyFile.length()); + Assert.assertEquals(valueFileValidLength, valueFile.length()); + } + + @Test + public void testRecoverWriter2() throws IOException { + File dir = + StorageEngine.getDataRegionSystemDir( + mockDataRegion.getDatabaseName(), mockDataRegion.getDataRegionIdString()); + dir.mkdirs(); + File keyFile1 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_KEY_FILENAME_PREFIX + "0"); + File valueFile1 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + "0"); + File tempKeyFile2 = + new File( + dir, + TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_KEY_FILENAME_PREFIX + + "1" + + AbstractTableSizeCacheWriter.TEMP_CACHE_FILE_SUBFIX); + File tempValueFile2 = + new File( + dir, + TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + + "1" + + AbstractTableSizeCacheWriter.TEMP_CACHE_FILE_SUBFIX); + + keyFile1.createNewFile(); + valueFile1.createNewFile(); + tempKeyFile2.createNewFile(); + tempValueFile2.createNewFile(); + TsFileTableDiskUsageCacheWriter writer = + new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + writer.close(); + Assert.assertTrue(keyFile1.exists()); + Assert.assertTrue(valueFile1.exists()); + Assert.assertFalse(tempKeyFile2.exists()); + Assert.assertFalse(tempValueFile2.exists()); + Assert.assertEquals(keyFile1, writer.getKeyFile()); + Assert.assertEquals(valueFile1, writer.getValueFile()); + } + + @Test + public void testRecoverWriter3() throws IOException { + File dir = + StorageEngine.getDataRegionSystemDir( + mockDataRegion.getDatabaseName(), mockDataRegion.getDataRegionIdString()); + dir.mkdirs(); + File keyFile1 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_KEY_FILENAME_PREFIX + "0"); + File valueFile1 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + "0"); + File keyFile2 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_KEY_FILENAME_PREFIX + "1"); + File valueFile2 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + "1"); + + keyFile1.createNewFile(); + valueFile1.createNewFile(); + keyFile2.createNewFile(); + valueFile2.createNewFile(); + TsFileTableDiskUsageCacheWriter writer = + new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + writer.close(); + Assert.assertFalse(keyFile1.exists()); + Assert.assertFalse(valueFile1.exists()); + Assert.assertTrue(keyFile2.exists()); + Assert.assertTrue(valueFile2.exists()); + Assert.assertEquals(keyFile2, writer.getKeyFile()); + Assert.assertEquals(valueFile2, writer.getValueFile()); + } + + @Test + public void testRecoverWriter4() throws IOException { + File dir = + StorageEngine.getDataRegionSystemDir( + mockDataRegion.getDatabaseName(), mockDataRegion.getDataRegionIdString()); + dir.mkdirs(); + File keyFile1 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_KEY_FILENAME_PREFIX + "0"); + File valueFile1 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + "0"); + File keyFile2 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_KEY_FILENAME_PREFIX + "1"); + File tempValueFile2 = + new File( + dir, + TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + + "1" + + AbstractTableSizeCacheWriter.TEMP_CACHE_FILE_SUBFIX); + File valueFile2 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + "1"); + + keyFile1.createNewFile(); + valueFile1.createNewFile(); + keyFile2.createNewFile(); + tempValueFile2.createNewFile(); + TsFileTableDiskUsageCacheWriter writer = + new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + writer.close(); + Assert.assertFalse(keyFile1.exists()); + Assert.assertFalse(valueFile1.exists()); + Assert.assertFalse(tempValueFile2.exists()); + Assert.assertTrue(keyFile2.exists()); + Assert.assertTrue(valueFile2.exists()); + Assert.assertEquals(keyFile2, writer.getKeyFile()); + Assert.assertEquals(valueFile2, writer.getValueFile()); + } + + @Test + public void testRecoverWriter5() throws IOException { + File dir = + StorageEngine.getDataRegionSystemDir( + mockDataRegion.getDatabaseName(), mockDataRegion.getDataRegionIdString()); + dir.mkdirs(); + File keyFile1 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_KEY_FILENAME_PREFIX + "1"); + File valueFile1 = + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + "1"); + File tempKeyFile2 = + new File( + dir, + TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_KEY_FILENAME_PREFIX + + "2" + + TsFileTableDiskUsageCacheWriter.TEMP_CACHE_FILE_SUBFIX); + File tempValueFile2 = + new File( + dir, + TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + + "2" + + AbstractTableSizeCacheWriter.TEMP_CACHE_FILE_SUBFIX); + + keyFile1.createNewFile(); + tempKeyFile2.createNewFile(); + tempValueFile2.createNewFile(); + TsFileTableDiskUsageCacheWriter writer = + new TsFileTableDiskUsageCacheWriter(mockDataRegion.getDatabaseName(), 0); + writer.close(); + Assert.assertFalse(keyFile1.exists()); + Assert.assertFalse(valueFile1.exists()); + Assert.assertFalse(tempValueFile2.exists()); + Assert.assertFalse(tempKeyFile2.exists()); + Assert.assertEquals( + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_KEY_FILENAME_PREFIX + "0"), + writer.getKeyFile()); + Assert.assertEquals( + new File(dir, TsFileTableDiskUsageCacheWriter.TSFILE_CACHE_VALUE_FILENAME_PREFIX + "0"), + writer.getValueFile()); + } + + private Map generateTableSizeMap(int tableNum) { + Map map = new HashMap<>(); + for (int i = 1; i <= tableNum; i++) { + map.put("table" + i, (long) i); + } + return map; + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java index ebd4d3b7ae036..0de593390f442 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java @@ -39,6 +39,7 @@ import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; import org.apache.iotdb.db.storageengine.dataregion.read.control.QueryResourceManager; import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.FileTimeIndexCacheRecorder; +import org.apache.iotdb.db.storageengine.dataregion.utils.tableDiskUsageCache.TableDiskUsageCache; import org.apache.iotdb.db.storageengine.dataregion.wal.WALManager; import org.apache.iotdb.db.storageengine.dataregion.wal.recover.WALRecoverManager; import org.apache.iotdb.db.storageengine.rescon.disk.TierManager; @@ -228,6 +229,7 @@ public static void cleanAllDir() throws IOException { cleanDir(path); } FileTimeIndexCacheRecorder.getInstance().close(); + TableDiskUsageCache.getInstance().close(); // delete system info cleanDir(config.getSystemDir()); // delete query diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index f4ebae2fb807e..c2093618a81cd 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -832,6 +832,11 @@ io_task_queue_size_for_flushing=10 # Datatype: bool enable_query_memory_estimation=true +# max sub task num for InformationTableScan +# effectiveMode: hot_reload +# Datatype: int +max_sub_task_num_for_information_table_scan=4 + #################### ### Schema Engine Configuration #################### diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index 6f9f95ca8fe88..3201213dd892a 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -200,6 +200,7 @@ public enum ThreadName { REGION_MIGRATE("Region-Migrate-Pool"), STORAGE_ENGINE_RECOVER_TRIGGER("StorageEngine-RecoverTrigger"), FILE_TIME_INDEX_RECORD("FileTimeIndexRecord"), + TABLE_SIZE_INDEX_RECORD("TableSizeIndexRecord"), BINARY_ALLOCATOR_SAMPLE_EVICTOR("BinaryAllocator-SampleEvictor"), BINARY_ALLOCATOR_AUTO_RELEASER("BinaryAllocator-Auto-Releaser"), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java index 55fecde7c3db1..a7259ee590bf9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java @@ -349,6 +349,10 @@ private ColumnHeaderConstant() { public static final String SHOW_CONFIGURATIONS_DEFAULT_VALUE = "default_value"; public static final String SHOW_CONFIGURATIONS_DESCRIPTION = "description"; + public static final String SIZE_IN_BYTES = "SizeInBytes"; + public static final String SIZE_IN_BYTES_TABLE_MODEL = "size_in_bytes"; + public static final String TIME_PARTITION_TABLE_MODEL = "time_partition"; + public static final List lastQueryColumnHeaders = ImmutableList.of( new ColumnHeader(TIMESERIES, TSDataType.TEXT), @@ -646,6 +650,14 @@ private ColumnHeaderConstant() { new ColumnHeader(ELAPSED_TIME, TSDataType.FLOAT), new ColumnHeader(STATEMENT, TSDataType.TEXT)); + public static final List showDiskUsageColumnHeaders = + ImmutableList.of( + new ColumnHeader(DATABASE, TSDataType.TEXT), + new ColumnHeader(DATA_NODE_ID, TSDataType.INT32), + new ColumnHeader(REGION_ID, TSDataType.INT32), + new ColumnHeader(TIME_PARTITION, TSDataType.INT64), + new ColumnHeader(SIZE_IN_BYTES, TSDataType.INT64)); + public static final List showSpaceQuotaColumnHeaders = ImmutableList.of( new ColumnHeader(DATABASE, TSDataType.TEXT), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java index ac6c8ebab24f2..cfa3e279b1eb1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java @@ -21,17 +21,25 @@ import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; import org.apache.iotdb.commons.schema.table.column.AttributeColumnSchema; +import org.apache.iotdb.commons.schema.table.column.FieldColumnSchema; import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; +import com.google.common.collect.ImmutableSet; import org.apache.tsfile.enums.TSDataType; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Locale; import java.util.Map; +import java.util.Set; public class InformationSchema { public static final String INFORMATION_DATABASE = "information_schema"; private static final Map schemaTables = new HashMap<>(); + private static final Map> columnsThatSupportPushDownPredicate = + new HashMap<>(); + private static final Set tablesThatSupportPushDownLimitOffset = new HashSet<>(); public static final String QUERIES = "queries"; public static final String DATABASES = "databases"; @@ -49,6 +57,7 @@ public class InformationSchema { public static final String NODES = "nodes"; public static final String CONFIG_NODES = "config_nodes"; public static final String DATA_NODES = "data_nodes"; + public static final String TABLE_DISK_USAGE = "table_disk_usage"; public static final String CONNECTIONS = "connections"; public static final String CURRENT_QUERIES = "current_queries"; public static final String QUERIES_COSTS_HISTOGRAM = "queries_costs_histogram"; @@ -331,6 +340,23 @@ public class InformationSchema { ColumnHeaderConstant.SCHEMA_CONSENSUS_PORT_TABLE_MODEL, TSDataType.INT32)); schemaTables.put(DATA_NODES, dataNodesTable); + final TsTable tableDiskUsageTable = new TsTable(TABLE_DISK_USAGE); + tableDiskUsageTable.addColumnSchema( + new FieldColumnSchema( + ColumnHeaderConstant.DATABASE.toLowerCase(Locale.ENGLISH), TSDataType.STRING)); + tableDiskUsageTable.addColumnSchema( + new FieldColumnSchema(ColumnHeaderConstant.TABLE_NAME_TABLE_MODEL, TSDataType.STRING)); + tableDiskUsageTable.addColumnSchema( + new FieldColumnSchema(ColumnHeaderConstant.DATA_NODE_ID_TABLE_MODEL, TSDataType.INT32)); + tableDiskUsageTable.addColumnSchema( + new FieldColumnSchema(ColumnHeaderConstant.REGION_ID_TABLE_MODEL, TSDataType.INT32)); + tableDiskUsageTable.addColumnSchema( + new FieldColumnSchema(ColumnHeaderConstant.TIME_PARTITION_TABLE_MODEL, TSDataType.INT64)); + tableDiskUsageTable.addColumnSchema( + new FieldColumnSchema(ColumnHeaderConstant.SIZE_IN_BYTES_TABLE_MODEL, TSDataType.INT64)); + tableDiskUsageTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); + schemaTables.put(TABLE_DISK_USAGE, tableDiskUsageTable); + final TsTable connectionsTable = new TsTable(CONNECTIONS); connectionsTable.addColumnSchema( new TagColumnSchema(ColumnHeaderConstant.DATANODE_ID, TSDataType.STRING)); @@ -388,10 +414,31 @@ public class InformationSchema { schemaTables.put(SERVICES, servicesTable); } + static { + columnsThatSupportPushDownPredicate.put( + TABLE_DISK_USAGE, + ImmutableSet.of( + ColumnHeaderConstant.DATABASE, + ColumnHeaderConstant.TABLE_NAME_TABLE_MODEL, + ColumnHeaderConstant.DATA_NODE_ID_TABLE_MODEL, + ColumnHeaderConstant.REGION_ID_TABLE_MODEL, + ColumnHeaderConstant.TIME_PARTITION_TABLE_MODEL)); + + tablesThatSupportPushDownLimitOffset.add(TABLE_DISK_USAGE); + } + public static Map getSchemaTables() { return schemaTables; } + public static Set getColumnsSupportPushDownPredicate(String tableName) { + return columnsThatSupportPushDownPredicate.getOrDefault(tableName, Collections.emptySet()); + } + + public static boolean supportsPushDownLimitOffset(String tableName) { + return tablesThatSupportPushDownLimitOffset.contains(tableName); + } + private InformationSchema() { // Utils } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java index ad14e90cd57de..74b5cafbae111 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java @@ -91,6 +91,7 @@ public enum Metric { PAM_RELEASED_COUNT("primitive_array_manager_released_count"), PAM_ALLOCATED_FAILURE_COUNT("primitive_array_manager_allocated_failure_count"), PAM_RELEASED_FAILURE_COUNT("primitive_array_manager_released_failure_count"), + TABLE_DISK_USAGE_CACHE("table_disk_usage_cache"), // compaction related DATA_WRITTEN("data_written"), diff --git a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 index 07186fd422219..8023c00b94b9e 100644 --- a/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 +++ b/iotdb-core/relational-grammar/src/main/antlr4/org/apache/iotdb/db/relational/grammar/sql/RelationalSql.g4 @@ -679,7 +679,6 @@ showQueriesStatement limitOffsetClause ; - killQueryStatement : KILL (QUERY queryId=string | ALL QUERIES) ; diff --git a/pom.xml b/pom.xml index 9be2fa2359ab4..bf41ba90d4352 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 0.14.1 1.9 1.5.6-3 - 2.2.1-260203-SNAPSHOT + 2.2.1-260205-SNAPSHOT