diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBShowTimeseriesOrderByTimeseriesIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBShowTimeseriesOrderByTimeseriesIT.java new file mode 100644 index 0000000000000..e4edb54eb4659 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBShowTimeseriesOrderByTimeseriesIT.java @@ -0,0 +1,221 @@ +/* + * 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.schema; + +import org.apache.iotdb.commons.schema.column.ColumnHeaderConstant; +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.itbase.category.ClusterIT; +import org.apache.iotdb.itbase.category.LocalStandaloneIT; +import org.apache.iotdb.util.AbstractSchemaIT; + +import org.junit.After; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runners.Parameterized; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +@Category({LocalStandaloneIT.class, ClusterIT.class}) +public class IoTDBShowTimeseriesOrderByTimeseriesIT extends AbstractSchemaIT { + + private static final List BASE_TIMESERIES = + Arrays.asList( + "root.db1.devA.m1", + "root.db1.devA.m2", + "root.db1.devB.m1", + "root.db1.devB.x", + "root.db2.devA.m1", + "root.db2.devC.m0", + "root.db2.devC.m3", + "root.db3.z.m1", + "root.db3.z.m10", + "root.db3.z.m2"); + + public IoTDBShowTimeseriesOrderByTimeseriesIT(SchemaTestMode schemaTestMode) { + super(schemaTestMode); + } + + @Parameterized.BeforeParam + public static void before() throws Exception { + setUpEnvironment(); + EnvFactory.getEnv().initClusterEnvironment(); + } + + @Parameterized.AfterParam + public static void after() throws Exception { + EnvFactory.getEnv().cleanClusterEnvironment(); + tearDownEnvironment(); + } + + @After + public void tearDown() throws Exception { + clearSchema(); + } + + private void prepareComplexSchema() throws Exception { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + statement.execute("CREATE DATABASE root.db1"); + statement.execute("CREATE DATABASE root.db2"); + statement.execute("CREATE DATABASE root.db3"); + + for (String ts : BASE_TIMESERIES) { + statement.execute( + String.format( + "create timeseries %s with datatype=INT32, encoding=RLE, compression=SNAPPY", ts)); + } + } + } + + private List queryTimeseries(final String sql) throws Exception { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery(sql)) { + List result = new ArrayList<>(); + while (resultSet.next()) { + result.add(resultSet.getString(ColumnHeaderConstant.TIMESERIES)); + } + return result; + } + } + + @Test + public void testOrderAscWithoutLimit() throws Exception { + prepareComplexSchema(); + List expected = new ArrayList<>(BASE_TIMESERIES); + Collections.sort(expected); + + List actual = queryTimeseries("show timeseries root.db*.** order by timeseries"); + assertEquals(expected, actual); + } + + @Test + public void testOrderDescWithOffsetLimit() throws Exception { + prepareComplexSchema(); + List expected = new ArrayList<>(BASE_TIMESERIES); + Collections.sort(expected); + Collections.reverse(expected); + expected = expected.subList(2, 6); // offset 2 limit 4 + + List actual = + queryTimeseries("show timeseries root.db*.** order by timeseries desc offset 2 limit 4"); + assertEquals(expected, actual); + } + + @Test + public void testInsertThenQueryOrder() throws Exception { + prepareComplexSchema(); + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + statement.execute( + "create timeseries root.db0.devX.a with datatype=INT32, encoding=RLE, compression=SNAPPY"); + } + + List expected = new ArrayList<>(BASE_TIMESERIES); + expected.add("root.db0.devX.a"); + Collections.sort(expected); + + List actual = queryTimeseries("show timeseries root.db*.** order by timeseries"); + assertEquals(expected, actual); + } + + @Test + public void testDeleteSubtreeThenQueryOrder() throws Exception { + prepareComplexSchema(); + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + statement.execute("delete timeseries root.db2.devC.**"); + } + + List expected = new ArrayList<>(BASE_TIMESERIES); + expected.remove("root.db2.devC.m0"); + expected.remove("root.db2.devC.m3"); + Collections.sort(expected); + + List actual = queryTimeseries("show timeseries root.db*.** order by timeseries"); + assertEquals(expected, actual); + } + + @Test + public void testOffsetLimitAfterDeletesAndAdds() throws Exception { + prepareComplexSchema(); + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + statement.execute("delete timeseries root.db1.devB.x"); + statement.execute( + "create timeseries root.db1.devC.m0 with datatype=INT32, encoding=RLE, compression=SNAPPY"); + statement.execute( + "create timeseries root.db4.devZ.z with datatype=INT32, encoding=RLE, compression=SNAPPY"); + } + + List expected = new ArrayList<>(BASE_TIMESERIES); + expected.remove("root.db1.devB.x"); + expected.add("root.db1.devC.m0"); + expected.add("root.db4.devZ.z"); + Collections.sort(expected); + expected = expected.subList(5, 10); // offset 5 limit 5 + + List actual = + queryTimeseries("show timeseries root.db*.** order by timeseries offset 5 limit 5"); + assertEquals(expected, actual); + } + + @Test + public void testConflictWithLatest() throws Exception { + prepareComplexSchema(); + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + try (ResultSet ignored = + statement.executeQuery("show latest timeseries order by timeseries")) { + fail("Expected exception for conflict between LATEST and ORDER BY TIMESERIES"); + } catch (SQLException e) { + assertTrue( + e.getMessage().toLowerCase().contains("latest") + && e.getMessage().toLowerCase().contains("order by timeseries")); + } + } + } + + @Test + public void testConflictWithTimeCondition() throws Exception { + prepareComplexSchema(); + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + try (ResultSet ignored = + statement.executeQuery("show timeseries where time > 0 order by timeseries")) { + fail("Expected exception for conflict between TIME condition and ORDER BY TIMESERIES"); + } catch (SQLException e) { + assertTrue( + e.getMessage().toLowerCase().contains("time condition") + && e.getMessage().toLowerCase().contains("order by timeseries")); + } + } + } +} 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..2308d3b81b7d2 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 @@ -200,7 +200,12 @@ showDevices // ---- Show Timeseries showTimeseries - : SHOW LATEST? TIMESERIES prefixPath? timeseriesWhereClause? timeConditionClause? rowPaginationClause? + : SHOW LATEST? TIMESERIES prefixPath? timeseriesWhereClause? timeConditionClause? orderByTimeseriesClause? rowPaginationClause? + ; + +// order by timeseries for SHOW TIMESERIES +orderByTimeseriesClause + : ORDER BY TIMESERIES (ASC | DESC)? ; // ---- Show Child Paths @@ -1586,4 +1591,4 @@ subStringExpression signedIntegerLiteral : (PLUS|MINUS)?INTEGER_LITERAL - ; \ No newline at end of file + ; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/LogicalViewSchemaSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/LogicalViewSchemaSource.java index f1eaaebbd1ddc..cd5dd52e8046d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/LogicalViewSchemaSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/LogicalViewSchemaSource.java @@ -79,7 +79,9 @@ public ISchemaReader getSchemaReader(ISchemaRegion schema SchemaFilterFactory.and( schemaFilter, SchemaFilterFactory.createViewTypeFilter(ViewType.VIEW)), true, - scope)); + scope, + false, + false)); } catch (MetadataException e) { throw new SchemaExecutionException(e.getMessage(), e); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java index 2ef0ab9e18a78..4cb22e3578200 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java @@ -47,7 +47,7 @@ public static ISchemaSource getTimeSeriesSchemaCountSourc Map templateMap, PathPatternTree scope) { return new TimeSeriesSchemaSource( - pathPattern, isPrefixMatch, 0, 0, schemaFilter, templateMap, false, scope); + pathPattern, isPrefixMatch, 0, 0, schemaFilter, templateMap, false, scope, false, false); } // show time series @@ -58,9 +58,20 @@ public static ISchemaSource getTimeSeriesSchemaScanSource long offset, SchemaFilter schemaFilter, Map templateMap, - PathPatternTree scope) { + PathPatternTree scope, + boolean orderByTimeseries, + boolean orderByTimeseriesDesc) { return new TimeSeriesSchemaSource( - pathPattern, isPrefixMatch, limit, offset, schemaFilter, templateMap, true, scope); + pathPattern, + isPrefixMatch, + limit, + offset, + schemaFilter, + templateMap, + true, + scope, + orderByTimeseries, + orderByTimeseriesDesc); } // count device diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java index 40799c548eeba..59aaa0a1964e3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java @@ -54,6 +54,8 @@ public class TimeSeriesSchemaSource implements ISchemaSource templateMap; private final boolean needViewDetail; + private final boolean orderByTimeseries; + private final boolean orderByTimeseriesDesc; TimeSeriesSchemaSource( PartialPath pathPattern, @@ -63,7 +65,9 @@ public class TimeSeriesSchemaSource implements ISchemaSource templateMap, boolean needViewDetail, - PathPatternTree scope) { + PathPatternTree scope, + boolean orderByTimeseries, + boolean orderByTimeseriesDesc) { this.pathPattern = pathPattern; this.isPrefixMatch = isPrefixMatch; this.limit = limit; @@ -72,6 +76,8 @@ public class TimeSeriesSchemaSource implements ISchemaSource getSchemaReader(ISchemaRegion schema isPrefixMatch, schemaFilter, needViewDetail, - scope)); + scope, + orderByTimeseries, + orderByTimeseriesDesc)); } catch (MetadataException e) { throw new SchemaExecutionException(e.getMessage(), e); } 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..4c477ae28768f 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 @@ -789,6 +789,22 @@ public Statement visitShowTimeseries(IoTDBSqlParser.ShowTimeseriesContext ctx) { showTimeSeriesStatement.setTimeCondition( parseWhereClause(ctx.timeConditionClause().whereClause())); } + + // ORDER BY TIMESERIES [ASC|DESC] + if (ctx.orderByTimeseriesClause() != null) { + if (orderByHeat) { + throw new SemanticException( + "LATEST and ORDER BY TIMESERIES cannot be used at the same time."); + } + if (ctx.timeConditionClause() != null) { + throw new SemanticException("ORDER BY TIMESERIES does not support TIME condition."); + } + Ordering ordering = Ordering.ASC; + if (ctx.orderByTimeseriesClause().DESC() != null) { + ordering = Ordering.DESC; + } + showTimeSeriesStatement.setOrderByTimeseries(true, ordering); + } if (ctx.rowPaginationClause() != null) { if (ctx.rowPaginationClause().limitClause() != null) { showTimeSeriesStatement.setLimit(parseLimitClause(ctx.rowPaginationClause().limitClause())); 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 5414b5ac02003..66c55a54c0f62 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 @@ -905,7 +905,7 @@ public LogicalPlanBuilder planFill(FillDescriptor fillDescriptor, Ordering scanO } public LogicalPlanBuilder planLimit(long rowLimit) { - if (rowLimit == 0) { + if (rowLimit <= 0) { return this; } @@ -995,7 +995,9 @@ public LogicalPlanBuilder planTimeSeriesSchemaSource( boolean orderByHeat, boolean prefixPath, Map templateMap, - PathPatternTree scope) { + PathPatternTree scope, + boolean orderByTimeseries, + boolean orderByTimeseriesDesc) { this.root = new TimeSeriesSchemaScanNode( context.getQueryId().genPlanNodeId(), @@ -1006,7 +1008,9 @@ public LogicalPlanBuilder planTimeSeriesSchemaSource( orderByHeat, prefixPath, templateMap, - scope); + scope, + orderByTimeseries, + orderByTimeseriesDesc); 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 2274869c35543..263cc717c46cf 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 @@ -54,6 +54,8 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.AggregationStep; import org.apache.iotdb.db.queryengine.plan.statement.StatementNode; import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; +import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; +import org.apache.iotdb.db.queryengine.plan.statement.component.SortItem; import org.apache.iotdb.db.queryengine.plan.statement.crud.DeleteDataStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertMultiTabletsStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowStatement; @@ -95,6 +97,7 @@ import org.apache.tsfile.write.schema.MeasurementSchema; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -556,6 +559,13 @@ public PlanNode visitShowTimeSeries( ShowTimeSeriesStatement showTimeSeriesStatement, MPPQueryContext context) { LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(analysis, context); + // Ensure TypeProvider has schema query column types for later SortNode + ColumnHeaderConstant.showTimeSeriesColumnHeaders.forEach( + columnHeader -> + context + .getTypeProvider() + .setTreeModelType(columnHeader.getColumnName(), columnHeader.getColumnType())); + long limit = showTimeSeriesStatement.getLimit(); long offset = showTimeSeriesStatement.getOffset(); if (showTimeSeriesStatement.hasTimeCondition()) { @@ -569,18 +579,22 @@ public PlanNode visitShowTimeSeries( // If there is only one region, we can push down the offset and limit operation to // source operator. - boolean canPushDownOffsetLimit = + boolean singleSchemaRegion = analysis.getSchemaPartitionInfo() != null - && analysis.getSchemaPartitionInfo().getDistributionInfo().size() == 1 - && !showTimeSeriesStatement.isOrderByHeat(); + && analysis.getSchemaPartitionInfo().getDistributionInfo().size() == 1; + boolean canPushDownOffsetLimit = singleSchemaRegion && !showTimeSeriesStatement.isOrderByHeat(); if (showTimeSeriesStatement.isOrderByHeat()) { limit = 0; offset = 0; } else if (!canPushDownOffsetLimit) { - limit = showTimeSeriesStatement.getLimit() + showTimeSeriesStatement.getOffset(); + limit = showTimeSeriesStatement.getLimitWithOffset(); offset = 0; } + boolean orderByTimeseries = showTimeSeriesStatement.isOrderByTimeseries(); + boolean orderByTimeseriesDesc = + orderByTimeseries && showTimeSeriesStatement.getNameOrdering() == Ordering.DESC; + planBuilder = planBuilder .planTimeSeriesSchemaSource( @@ -591,9 +605,18 @@ public PlanNode visitShowTimeSeries( showTimeSeriesStatement.isOrderByHeat(), showTimeSeriesStatement.isPrefixPath(), analysis.getRelatedTemplateInfo(), - showTimeSeriesStatement.getAuthorityScope()) + showTimeSeriesStatement.getAuthorityScope(), + orderByTimeseries, + orderByTimeseriesDesc) .planSchemaQueryMerge(showTimeSeriesStatement.isOrderByHeat()); + // order by timeseries name in multi-region case: still need global SortNode + if (showTimeSeriesStatement.isOrderByTimeseries() && !singleSchemaRegion) { + SortItem sortItem = + new SortItem(ColumnHeaderConstant.TIMESERIES, showTimeSeriesStatement.getNameOrdering()); + planBuilder = planBuilder.planOrderBy(Collections.singletonList(sortItem)); + } + // show latest timeseries if (showTimeSeriesStatement.isOrderByHeat() && null != analysis.getDataPartitionInfo() @@ -636,7 +659,7 @@ public PlanNode visitShowDevices( long limit = showDevicesStatement.getLimit(); long offset = showDevicesStatement.getOffset(); if (!canPushDownOffsetLimit) { - limit = showDevicesStatement.getLimit() + showDevicesStatement.getOffset(); + limit = showDevicesStatement.getLimitWithOffset(); offset = 0; } @@ -1008,7 +1031,7 @@ public PlanNode visitShowLogicalView( long limit = showLogicalViewStatement.getLimit(); long offset = showLogicalViewStatement.getOffset(); if (!canPushDownOffsetLimit) { - limit = showLogicalViewStatement.getLimit() + showLogicalViewStatement.getOffset(); + limit = showLogicalViewStatement.getLimitWithOffset(); offset = 0; } planBuilder = 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 f869ce59d470f..261cad23a2c0f 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 @@ -913,7 +913,9 @@ public Operator visitTimeSeriesSchemaScan( node.getOffset(), node.getSchemaFilter(), node.getTemplateMap(), - node.getScope())); + node.getScope(), + node.isOrderByTimeseries(), + node.isOrderByTimeseriesDesc())); } @Override 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..25f51f88d9eae 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 @@ -157,7 +157,9 @@ private void produceFragmentInstance(PlanFragment fragment) { || analysis.getTreeStatement() instanceof ExplainAnalyzeStatement || analysis.getTreeStatement() instanceof ShowQueriesStatement || (analysis.getTreeStatement() instanceof ShowTimeSeriesStatement - && ((ShowTimeSeriesStatement) analysis.getTreeStatement()).isOrderByHeat())) { + && (((ShowTimeSeriesStatement) analysis.getTreeStatement()).isOrderByHeat() + || ((ShowTimeSeriesStatement) analysis.getTreeStatement()) + .isOrderByTimeseries()))) { fragmentInstance.getFragment().generateTypeProvider(queryContext.getTypeProvider()); } instanceMap.putIfAbsent(fragment.getId(), fragmentInstance); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/read/TimeSeriesSchemaScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/read/TimeSeriesSchemaScanNode.java index 1997a3caefe84..ddf36b9bb78c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/read/TimeSeriesSchemaScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/read/TimeSeriesSchemaScanNode.java @@ -48,6 +48,11 @@ public class TimeSeriesSchemaScanNode extends SchemaQueryScanNode { // if is true, the result will be sorted according to the inserting frequency of the timeseries private final boolean orderByHeat; + // Whether to order result by timeseries full path in this region. + private final boolean orderByTimeseries; + // When orderByTimeseries is true, whether the ordering is descending. + private final boolean orderByTimeseriesDesc; + private final SchemaFilter schemaFilter; private final Map templateMap; @@ -66,6 +71,28 @@ public TimeSeriesSchemaScanNode( this.schemaFilter = schemaFilter; this.orderByHeat = orderByHeat; this.templateMap = templateMap; + this.orderByTimeseries = false; + this.orderByTimeseriesDesc = false; + } + + public TimeSeriesSchemaScanNode( + PlanNodeId id, + PartialPath partialPath, + SchemaFilter schemaFilter, + long limit, + long offset, + boolean orderByHeat, + boolean isPrefixPath, + @NotNull Map templateMap, + @NotNull PathPatternTree scope, + boolean orderByTimeseries, + boolean orderByTimeseriesDesc) { + super(id, partialPath, limit, offset, isPrefixPath, scope); + this.schemaFilter = schemaFilter; + this.orderByHeat = orderByHeat; + this.templateMap = templateMap; + this.orderByTimeseries = orderByTimeseries; + this.orderByTimeseriesDesc = orderByTimeseriesDesc; } public SchemaFilter getSchemaFilter() { @@ -82,6 +109,8 @@ protected void serializeAttributes(ByteBuffer byteBuffer) { ReadWriteIOUtils.write(offset, byteBuffer); ReadWriteIOUtils.write(orderByHeat, byteBuffer); ReadWriteIOUtils.write(isPrefixPath, byteBuffer); + ReadWriteIOUtils.write(orderByTimeseries, byteBuffer); + ReadWriteIOUtils.write(orderByTimeseriesDesc, byteBuffer); ReadWriteIOUtils.write(templateMap.size(), byteBuffer); for (Template template : templateMap.values()) { @@ -99,6 +128,8 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(offset, stream); ReadWriteIOUtils.write(orderByHeat, stream); ReadWriteIOUtils.write(isPrefixPath, stream); + ReadWriteIOUtils.write(orderByTimeseries, stream); + ReadWriteIOUtils.write(orderByTimeseriesDesc, stream); ReadWriteIOUtils.write(templateMap.size(), stream); for (Template template : templateMap.values()) { @@ -120,6 +151,8 @@ public static TimeSeriesSchemaScanNode deserialize(ByteBuffer byteBuffer) { long offset = ReadWriteIOUtils.readLong(byteBuffer); boolean oderByHeat = ReadWriteIOUtils.readBool(byteBuffer); boolean isPrefixPath = ReadWriteIOUtils.readBool(byteBuffer); + boolean orderByTimeseries = ReadWriteIOUtils.readBool(byteBuffer); + boolean orderByTimeseriesDesc = ReadWriteIOUtils.readBool(byteBuffer); int templateNum = ReadWriteIOUtils.readInt(byteBuffer); Map templateMap = new HashMap<>(); @@ -141,13 +174,23 @@ public static TimeSeriesSchemaScanNode deserialize(ByteBuffer byteBuffer) { oderByHeat, isPrefixPath, templateMap, - scope); + scope, + orderByTimeseries, + orderByTimeseriesDesc); } public boolean isOrderByHeat() { return orderByHeat; } + public boolean isOrderByTimeseries() { + return orderByTimeseries; + } + + public boolean isOrderByTimeseriesDesc() { + return orderByTimeseriesDesc; + } + public Map getTemplateMap() { return templateMap; } @@ -168,7 +211,9 @@ public PlanNode clone() { orderByHeat, isPrefixPath, templateMap, - scope); + scope, + orderByTimeseries, + orderByTimeseriesDesc); } @Override @@ -190,12 +235,16 @@ public boolean equals(Object o) { return false; } TimeSeriesSchemaScanNode that = (TimeSeriesSchemaScanNode) o; - return orderByHeat == that.orderByHeat && Objects.equals(schemaFilter, that.schemaFilter); + return orderByHeat == that.orderByHeat + && orderByTimeseries == that.orderByTimeseries + && orderByTimeseriesDesc == that.orderByTimeseriesDesc + && Objects.equals(schemaFilter, that.schemaFilter); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), schemaFilter, orderByHeat); + return Objects.hash( + super.hashCode(), schemaFilter, orderByHeat, orderByTimeseries, orderByTimeseriesDesc); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowStatement.java index 971fae6698be6..b839791821767 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowStatement.java @@ -59,6 +59,13 @@ public void setOffset(long offset) { this.offset = offset; } + public long getLimitWithOffset() { + if (limit <= 0) { + return limit; + } + return limit + offset; + } + public boolean isPrefixPath() { return isPrefixPath; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTimeSeriesStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTimeSeriesStatement.java index 82a389d622499..bb07ca6a141ef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTimeSeriesStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTimeSeriesStatement.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.schema.filter.SchemaFilter; import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; +import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.iotdb.db.queryengine.plan.statement.component.WhereCondition; import java.util.Collections; @@ -42,6 +43,9 @@ public class ShowTimeSeriesStatement extends ShowStatement { // if is true, the result will be sorted according to the inserting frequency of the time series private final boolean orderByHeat; private WhereCondition timeCondition; + // order by timeseries name + private boolean orderByTimeseries; + private Ordering nameOrdering = Ordering.ASC; public ShowTimeSeriesStatement(PartialPath pathPattern, boolean orderByHeat) { super(); @@ -65,6 +69,21 @@ public boolean isOrderByHeat() { return orderByHeat; } + public boolean isOrderByTimeseries() { + return orderByTimeseries; + } + + public Ordering getNameOrdering() { + return nameOrdering; + } + + public void setOrderByTimeseries(boolean orderByTimeseries, Ordering ordering) { + this.orderByTimeseries = orderByTimeseries; + if (ordering != null) { + this.nameOrdering = ordering; + } + } + public void setTimeCondition(WhereCondition timeCondition) { this.timeCondition = timeCondition; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java index 9aec147ace1fe..868a47d6143cd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java @@ -73,6 +73,7 @@ import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.reader.impl.TimeseriesReaderWithViewFetch; import org.apache.iotdb.db.schemaengine.schemaregion.utils.MetaFormatUtils; import org.apache.iotdb.db.schemaengine.schemaregion.utils.filter.DeviceFilterVisitor; +import org.apache.iotdb.db.schemaengine.template.ClusterTemplateManager; import org.apache.iotdb.db.storageengine.rescon.quotas.DataNodeSpaceQuotaManager; import org.apache.iotdb.db.utils.SchemaUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -192,6 +193,22 @@ public synchronized boolean createSnapshot(final File snapshotDir) { return store.createSnapshot(snapshotDir); } + private void applySubtreeMeasurementDelta(IMemMNode startNode, final long delta) { + if (delta == 0 || startNode == null) { + return; + } + IMemMNode current = startNode; + while (current != null) { + current.setSubtreeMeasurementCount(current.getSubtreeMeasurementCount() + delta); + current = current.getParent(); + } + } + + private long getTemplateMeasurementCount(final int templateId) { + final Template template = ClusterTemplateManager.getInstance().getTemplate(templateId); + return template == null ? 0L : template.getMeasurementNumber(); + } + public static MTreeBelowSGMemoryImpl loadFromSnapshot( final File snapshotDir, final String databaseFullPath, @@ -203,18 +220,21 @@ public static MTreeBelowSGMemoryImpl loadFromSnapshot( final Function, Map> tagGetter, final Function, Map> attributeGetter) throws IOException, IllegalPathException { - return new MTreeBelowSGMemoryImpl( - PartialPath.getQualifiedDatabasePartialPath(databaseFullPath), - MemMTreeStore.loadFromSnapshot( - snapshotDir, - measurementProcess, - deviceProcess, - tableDeviceProcess, - regionStatistics, - metric), - tagGetter, - attributeGetter, - regionStatistics); + final MTreeBelowSGMemoryImpl mtree = + new MTreeBelowSGMemoryImpl( + PartialPath.getQualifiedDatabasePartialPath(databaseFullPath), + MemMTreeStore.loadFromSnapshot( + snapshotDir, + measurementProcess, + deviceProcess, + tableDeviceProcess, + regionStatistics, + metric), + tagGetter, + attributeGetter, + regionStatistics); + mtree.rebuildSubtreeMeasurementCount(); + return mtree; } // endregion @@ -314,6 +334,7 @@ public IMeasurementMNode createTimeSeries( entityMNode.addAlias(alias, measurementMNode); } + applySubtreeMeasurementDelta(measurementMNode.getAsMNode(), 1L); return measurementMNode; } } @@ -411,6 +432,7 @@ public List> createAlignedTimeSeries( if (aliasList != null && aliasList.get(i) != null) { entityMNode.addAlias(aliasList.get(i), measurementMNode); } + applySubtreeMeasurementDelta(measurementMNode.getAsMNode(), 1L); measurementMNodeList.add(measurementMNode); } return measurementMNodeList; @@ -619,6 +641,7 @@ public IMeasurementMNode deleteTimeSeries(final PartialPath path) if (deletedNode.getAlias() != null) { parent.getAsDeviceMNode().deleteAliasChild(deletedNode.getAlias()); } + applySubtreeMeasurementDelta(parent, -1L); } deleteEmptyInternalMNode(parent.getAsDeviceMNode()); return deletedNode; @@ -1016,6 +1039,7 @@ public void activateTemplate(final PartialPath activatePath, final Template temp entityMNode.setUseTemplate(true); entityMNode.setSchemaTemplateId(template.getId()); regionStatistics.activateTemplate(template.getId()); + applySubtreeMeasurementDelta(entityMNode.getAsMNode(), (long) template.getMeasurementNumber()); } public Map> constructSchemaBlackListWithTemplate( @@ -1080,6 +1104,8 @@ protected void updateEntity(final IDeviceMNode node) { resultTemplateSetInfo.put( node.getPartialPath(), Collections.singletonList(node.getSchemaTemplateId())); regionStatistics.deactivateTemplate(node.getSchemaTemplateId()); + applySubtreeMeasurementDelta( + node.getAsMNode(), -getTemplateMeasurementCount(node.getSchemaTemplateId())); node.deactivateTemplate(); deleteEmptyInternalMNode(node); } @@ -1112,6 +1138,7 @@ public void activateTemplateWithoutCheck( entityMNode.setUseTemplate(true); entityMNode.setSchemaTemplateId(templateId); regionStatistics.activateTemplate(templateId); + applySubtreeMeasurementDelta(entityMNode.getAsMNode(), getTemplateMeasurementCount(templateId)); } public long countPathsUsingTemplate(final PartialPath pathPattern, final int templateId) @@ -1123,6 +1150,23 @@ public long countPathsUsingTemplate(final PartialPath pathPattern, final int tem } } + public void rebuildSubtreeMeasurementCount() { + rebuildSubtreeMeasurementCountFromNode(rootNode); + } + + private long rebuildSubtreeMeasurementCountFromNode(final IMemMNode node) { + long count = node.isMeasurement() ? 1L : 0L; + final IMNodeIterator iterator = store.getChildrenIterator(node); + while (iterator.hasNext()) { + count += rebuildSubtreeMeasurementCountFromNode(iterator.next()); + } + if (node.isDevice() && node.getAsDeviceMNode().isUseTemplate()) { + count += getTemplateMeasurementCount(node.getAsDeviceMNode().getSchemaTemplateId()); + } + node.setSubtreeMeasurementCount(count); + return count; + } + // endregion // region Interfaces for schema reader @@ -1435,6 +1479,46 @@ public ISchemaReader getTimeSeriesReader( showTimeSeriesPlan.isPrefixMatch(), showTimeSeriesPlan.getScope()) { + private long remainingOffset = Math.max(0, showTimeSeriesPlan.getOffset()); + + private boolean shouldPruneSubtree(final IMemMNode node) { + if (remainingOffset <= 0) { + return false; + } + final long subtreeCount = node.getSubtreeMeasurementCount(); + if (subtreeCount <= remainingOffset) { + remainingOffset -= subtreeCount; + return true; + } + return false; + } + + @Override + protected boolean acceptFullMatchedNode(final IMemMNode node) { + if (!node.isMeasurement()) { + return false; + } + if (remainingOffset > 0) { + // skip this measurement + remainingOffset--; + return false; + } + return true; + } + + @Override + protected boolean shouldVisitSubtreeOfInternalMatchedNode(final IMemMNode node) { + if (shouldPruneSubtree(node)) { + return false; + } + return !node.isMeasurement(); + } + + @Override + protected boolean shouldVisitSubtreeOfFullMatchedNode(final IMemMNode node) { + return !node.isMeasurement() && !shouldPruneSubtree(node); + } + @Override protected ITimeSeriesSchemaInfo collectMeasurement( final IMeasurementMNode node) { @@ -1495,15 +1579,44 @@ public ITimeSeriesSchemaInfo snapshot() { } }; } + + @Override + protected Iterator getChildrenIterator(final IMemMNode parent) + throws MetadataException { + Iterator baseIterator = super.getChildrenIterator(parent); + + if (!showTimeSeriesPlan.isOrderByTimeseries()) { + return baseIterator; + } + + List children = new ArrayList<>(); + if (baseIterator instanceof IMNodeIterator) { + IMNodeIterator it = (IMNodeIterator) baseIterator; + while (it.hasNext()) { + children.add(it.next()); + } + it.close(); + } else { + while (baseIterator.hasNext()) { + children.add(baseIterator.next()); + } + } + + children.sort( + (a, b) -> { + int cmp = a.getName().compareTo(b.getName()); + return showTimeSeriesPlan.isOrderByTimeseriesDesc() ? -cmp : cmp; + }); + return children.iterator(); + } }; collector.setTemplateMap(showTimeSeriesPlan.getRelatedTemplate(), nodeFactory); final ISchemaReader reader = new TimeseriesReaderWithViewFetch( collector, showTimeSeriesPlan.getSchemaFilter(), showTimeSeriesPlan.needViewDetail()); - if (showTimeSeriesPlan.getLimit() > 0 || showTimeSeriesPlan.getOffset() > 0) { - return new SchemaReaderLimitOffsetWrapper<>( - reader, showTimeSeriesPlan.getLimit(), showTimeSeriesPlan.getOffset()); + if (showTimeSeriesPlan.getLimit() > 0) { + return new SchemaReaderLimitOffsetWrapper<>(reader, showTimeSeriesPlan.getLimit(), 0); } else { return reader; } @@ -1615,6 +1728,7 @@ public IMeasurementMNode createLogicalView( measurementMNode.setParent(entityMNode.getAsMNode()); store.addChild(entityMNode.getAsMNode(), leafName, measurementMNode.getAsMNode()); + applySubtreeMeasurementDelta(measurementMNode.getAsMNode(), 1L); return measurementMNode; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/IMemMNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/IMemMNode.java index d3d055928b14a..6cd14800f3e3f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/IMemMNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/IMemMNode.java @@ -20,4 +20,13 @@ import org.apache.iotdb.commons.schema.node.IMNode; -public interface IMemMNode extends IMNode {} +public interface IMemMNode extends IMNode { + + /** + * The count of measurement nodes contained in the subtree rooted at this node. The counter is + * maintained in memory only. + */ + long getSubtreeMeasurementCount(); + + void setSubtreeMeasurementCount(long subtreeMeasurementCount); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/basic/BasicMNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/basic/BasicMNode.java index a033c56c67e0e..2860bea6e9445 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/basic/BasicMNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/basic/BasicMNode.java @@ -46,6 +46,9 @@ public class BasicMNode implements IMemMNode { private IMemMNode parent; private final BasicMNodeInfo basicMNodeInfo; + /** Cached count of measurements in this node's subtree, restored on restart. */ + private long subtreeMeasurementCount = 0L; + /** from root to this node, only be set when used once for InternalMNode */ private String fullPath; @@ -99,6 +102,16 @@ public void setFullPath(final String fullPath) { this.fullPath = fullPath; } + @Override + public long getSubtreeMeasurementCount() { + return subtreeMeasurementCount; + } + + @Override + public void setSubtreeMeasurementCount(final long subtreeMeasurementCount) { + this.subtreeMeasurementCount = subtreeMeasurementCount; + } + @Override public PartialPath getPartialPath() { final List detachedPath = new ArrayList<>(); @@ -225,6 +238,7 @@ public R accept(final MNodeVisitor visitor, final C context) { *
  • basicMNodeInfo reference, 8B *
  • parent reference, 8B *
  • fullPath reference, 8B + *
  • subtreeMeasurementCount, 8B * *
  • MapEntry in parent *
      @@ -236,7 +250,7 @@ public R accept(final MNodeVisitor visitor, final C context) { */ @Override public int estimateSize() { - return 8 + 8 + 8 + 8 + 8 + 8 + 28 + basicMNodeInfo.estimateSize(); + return 8 + 8 + 8 + 8 + 8 + 8 + 8 + 28 + basicMNodeInfo.estimateSize(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/AboveDatabaseMNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/AboveDatabaseMNode.java index cff30d8b8c462..87144d4954a5d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/AboveDatabaseMNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/AboveDatabaseMNode.java @@ -33,4 +33,14 @@ public AboveDatabaseMNode(IMemMNode parent, String name) { public IMemMNode getAsMNode() { return this; } + + @Override + public long getSubtreeMeasurementCount() { + return basicMNode.getSubtreeMeasurementCount(); + } + + @Override + public void setSubtreeMeasurementCount(long subtreeMeasurementCount) { + basicMNode.setSubtreeMeasurementCount(subtreeMeasurementCount); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/DatabaseMNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/DatabaseMNode.java index c6b2f5e2427bc..290cf4273601b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/DatabaseMNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/DatabaseMNode.java @@ -45,4 +45,14 @@ public IDeviceInfo getDeviceInfo() { public void setDeviceInfo(IDeviceInfo deviceInfo) { basicMNode.setDeviceInfo(deviceInfo); } + + @Override + public long getSubtreeMeasurementCount() { + return basicMNode.getSubtreeMeasurementCount(); + } + + @Override + public void setSubtreeMeasurementCount(long subtreeMeasurementCount) { + basicMNode.setSubtreeMeasurementCount(subtreeMeasurementCount); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/MeasurementMNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/MeasurementMNode.java index a40cbe6bc0fc9..d2a2cbd80c90c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/MeasurementMNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/impl/MeasurementMNode.java @@ -52,4 +52,14 @@ public IMNodeContainer getChildren() { public final boolean isLogicalView() { return false; } + + @Override + public long getSubtreeMeasurementCount() { + return basicMNode.getSubtreeMeasurementCount(); + } + + @Override + public void setSubtreeMeasurementCount(long subtreeMeasurementCount) { + basicMNode.setSubtreeMeasurementCount(subtreeMeasurementCount); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/IShowTimeSeriesPlan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/IShowTimeSeriesPlan.java index e05ee54f253cc..0226e0dca9c7c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/IShowTimeSeriesPlan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/IShowTimeSeriesPlan.java @@ -32,4 +32,10 @@ public interface IShowTimeSeriesPlan extends IShowSchemaPlan { SchemaFilter getSchemaFilter(); Map getRelatedTemplate(); + + /** Whether to order result by timeseries full path in this region. */ + boolean isOrderByTimeseries(); + + /** Whether the timeseries ordering is descending when {@link #isOrderByTimeseries()} is true. */ + boolean isOrderByTimeseriesDesc(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/SchemaRegionReadPlanFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/SchemaRegionReadPlanFactory.java index 84a2ca9cdf7cc..3707fe6f43edd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/SchemaRegionReadPlanFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/SchemaRegionReadPlanFactory.java @@ -61,9 +61,20 @@ public static IShowTimeSeriesPlan getShowTimeSeriesPlan( boolean isPrefixMatch, SchemaFilter schemaFilter, boolean needViewDetail, - PathPatternTree scope) { + PathPatternTree scope, + boolean orderByTimeseries, + boolean orderByTimeseriesDesc) { return new ShowTimeSeriesPlanImpl( - path, relatedTemplate, limit, offset, isPrefixMatch, schemaFilter, needViewDetail, scope); + path, + relatedTemplate, + limit, + offset, + isPrefixMatch, + schemaFilter, + needViewDetail, + scope, + orderByTimeseries, + orderByTimeseriesDesc); } public static IShowNodesPlan getShowNodesPlan(PartialPath path, PathPatternTree scope) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowTimeSeriesPlanImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowTimeSeriesPlanImpl.java index 8aeb0e837bedd..ea22e22d8d61b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowTimeSeriesPlanImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowTimeSeriesPlanImpl.java @@ -37,6 +37,10 @@ public class ShowTimeSeriesPlanImpl extends AbstractShowSchemaPlanImpl private final SchemaFilter schemaFilter; private final boolean needViewDetail; + // order-by-timeseries pushdown flags inside a single SchemaRegion + private final boolean orderByTimeseries; + private final boolean orderByTimeseriesDesc; + public ShowTimeSeriesPlanImpl( PartialPath path, Map relatedTemplate, @@ -45,11 +49,15 @@ public ShowTimeSeriesPlanImpl( boolean isPrefixMatch, SchemaFilter schemaFilter, boolean needViewDetail, - PathPatternTree scope) { + PathPatternTree scope, + boolean orderByTimeseries, + boolean orderByTimeseriesDesc) { super(path, limit, offset, isPrefixMatch, scope); this.relatedTemplate = relatedTemplate; this.schemaFilter = schemaFilter; this.needViewDetail = needViewDetail; + this.orderByTimeseries = orderByTimeseries; + this.orderByTimeseriesDesc = orderByTimeseriesDesc; } @Override @@ -67,18 +75,31 @@ public Map getRelatedTemplate() { return relatedTemplate; } + @Override + public boolean isOrderByTimeseries() { + return orderByTimeseries; + } + + @Override + public boolean isOrderByTimeseriesDesc() { + return orderByTimeseriesDesc; + } + @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; ShowTimeSeriesPlanImpl that = (ShowTimeSeriesPlanImpl) o; - return Objects.equals(relatedTemplate, that.relatedTemplate) + return orderByTimeseries == that.orderByTimeseries + && orderByTimeseriesDesc == that.orderByTimeseriesDesc + && Objects.equals(relatedTemplate, that.relatedTemplate) && Objects.equals(schemaFilter, that.schemaFilter); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), relatedTemplate, schemaFilter); + return Objects.hash( + super.hashCode(), relatedTemplate, schemaFilter, orderByTimeseries, orderByTimeseriesDesc); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionTestUtil.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionTestUtil.java index 42800c4e58630..45b4c9137881e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionTestUtil.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionTestUtil.java @@ -175,7 +175,16 @@ public static long getAllTimeSeriesCount( try (ISchemaReader timeSeriesReader = schemaRegion.getTimeSeriesReader( SchemaRegionReadPlanFactory.getShowTimeSeriesPlan( - pathPattern, templateMap, 0, 0, isPrefixMatch, null, false, ALL_MATCH_SCOPE))) { + pathPattern, + templateMap, + 0, + 0, + isPrefixMatch, + null, + false, + ALL_MATCH_SCOPE, + false, + false))) { long count = 0; while (timeSeriesReader.hasNext()) { timeSeriesReader.next(); @@ -200,7 +209,16 @@ public static void checkSingleTimeSeries( try (final ISchemaReader timeSeriesReader = schemaRegion.getTimeSeriesReader( SchemaRegionReadPlanFactory.getShowTimeSeriesPlan( - pathPattern, Collections.emptyMap(), 0, 0, false, null, false, ALL_MATCH_SCOPE))) { + pathPattern, + Collections.emptyMap(), + 0, + 0, + false, + null, + false, + ALL_MATCH_SCOPE, + false, + false))) { Assert.assertTrue(timeSeriesReader.hasNext()); final ITimeSeriesSchemaInfo info = timeSeriesReader.next(); Assert.assertEquals(isAligned, info.isUnderAlignedDevice()); @@ -237,7 +255,16 @@ public static Map getMeasurementCountGroupByLevel( try (ISchemaReader timeSeriesReader = schemaRegion.getTimeSeriesReader( SchemaRegionReadPlanFactory.getShowTimeSeriesPlan( - pathPattern, null, 0, 0, isPrefixMatch, null, false, ALL_MATCH_SCOPE))) { + pathPattern, + null, + 0, + 0, + isPrefixMatch, + null, + false, + ALL_MATCH_SCOPE, + false, + false))) { Map countMap = new HashMap<>(); while (timeSeriesReader.hasNext()) { ITimeSeriesSchemaInfo timeSeriesSchemaInfo = timeSeriesReader.next(); @@ -356,7 +383,9 @@ public static List showTimeseries( isPrefixMatch, schemaFilter, needViewDetail, - ALL_MATCH_SCOPE))) { + ALL_MATCH_SCOPE, + false, + false))) { while (reader.hasNext()) { timeSeriesSchemaInfo = reader.next(); result.add( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaQueryScanOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaQueryScanOperatorTest.java index a5d4a6e6acef6..aab01b785c799 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaQueryScanOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaQueryScanOperatorTest.java @@ -215,7 +215,9 @@ public void testTimeSeriesSchemaScan() throws Exception { 0, null, Collections.emptyMap(), - SchemaConstant.ALL_MATCH_SCOPE); + SchemaConstant.ALL_MATCH_SCOPE, + false, + false); SchemaOperatorTestUtil.mockGetSchemaReader( timeSeriesSchemaSource, showTimeSeriesResults.iterator(), schemaRegion, true);