You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2023/08/25 09:13:14 UTC
[iotdb] branch auth updated: [auth] Adaptation of schema query authentication interface (#10952)
This is an automated email from the ASF dual-hosted git repository.
jackietien pushed a commit to branch auth
in repository https://gitbox.apache.org/repos/asf/iotdb.git
The following commit(s) were added to refs/heads/auth by this push:
new d7baf328972 [auth] Adaptation of schema query authentication interface (#10952)
d7baf328972 is described below
commit d7baf32897279d0c1c703c7423f0ef73365f546c
Author: Chen YZ <43...@users.noreply.github.com>
AuthorDate: Fri Aug 25 17:13:07 2023 +0800
[auth] Adaptation of schema query authentication interface (#10952)
---
.../it/database/IoTDBDatabaseSetAndDeleteIT.java | 23 ++-
.../it/partition/IoTDBPartitionGetterIT.java | 8 +-
.../request/read/database/CountDatabasePlan.java | 15 +-
.../request/read/database/GetDatabasePlan.java | 5 +-
.../read/partition/GetNodePathsPartitionPlan.java | 12 ++
.../read/template/GetPathsSetTemplatePlan.java | 11 +-
.../iotdb/confignode/manager/ConfigManager.java | 22 ++-
.../apache/iotdb/confignode/manager/IManager.java | 13 +-
.../manager/schema/ClusterSchemaManager.java | 16 +-
.../persistence/executor/ConfigPlanExecutor.java | 3 +-
.../persistence/schema/ClusterSchemaInfo.java | 23 +--
.../confignode/persistence/schema/ConfigMTree.java | 38 +++--
.../thrift/ConfigNodeRPCServiceProcessor.java | 34 ++--
.../request/ConfigPhysicalPlanSerDeTest.java | 8 +-
.../persistence/schema/ClusterSchemaInfoTest.java | 6 +-
.../persistence/schema/ConfigMTreeTest.java | 33 ++--
.../iotdb/db/protocol/client/ConfigNodeClient.java | 22 ++-
.../impl/DataNodeInternalRPCServiceImpl.java | 3 +-
.../operator/schema/source/DeviceSchemaSource.java | 8 +-
.../schema/source/LogicalViewSchemaSource.java | 12 +-
.../operator/schema/source/NodeSchemaSource.java | 11 +-
.../schema/source/PathsUsingTemplateSource.java | 8 +-
.../schema/source/SchemaSourceFactory.java | 42 +++--
.../schema/source/TimeSeriesSchemaSource.java | 9 +-
.../queryengine/plan/analyze/AnalyzeVisitor.java | 8 +-
.../plan/analyze/ClusterPartitionFetcher.java | 23 +--
.../plan/analyze/IPartitionFetcher.java | 6 +-
.../analyze/cache/partition/PartitionCache.java | 5 +-
.../config/executor/ClusterConfigTaskExecutor.java | 30 ++--
.../plan/planner/LogicalPlanBuilder.java | 48 ++++--
.../plan/planner/LogicalPlanVisitor.java | 37 +++--
.../plan/planner/OperatorTreeGenerator.java | 32 +++-
.../plan/node/metedata/read/DevicesCountNode.java | 13 +-
.../node/metedata/read/DevicesSchemaScanNode.java | 13 +-
.../metedata/read/LevelTimeSeriesCountNode.java | 13 +-
.../metedata/read/LogicalViewSchemaScanNode.java | 17 +-
.../metedata/read/NodePathsSchemaScanNode.java | 12 +-
.../metedata/read/PathsUsingTemplateScanNode.java | 11 +-
.../node/metedata/read/SchemaQueryScanNode.java | 43 ++++--
.../node/metedata/read/TimeSeriesCountNode.java | 15 +-
.../metedata/read/TimeSeriesSchemaScanNode.java | 29 +++-
.../statement/metadata/CountDatabaseStatement.java | 12 ++
.../statement/metadata/CountDevicesStatement.java | 12 ++
.../metadata/CountLevelTimeSeriesStatement.java | 12 ++
.../statement/metadata/CountNodesStatement.java | 12 ++
.../metadata/CountTimeSeriesStatement.java | 12 ++
.../metadata/ShowChildNodesStatement.java | 11 ++
.../metadata/ShowChildPathsStatement.java | 11 ++
.../statement/metadata/ShowDatabaseStatement.java | 12 ++
.../statement/metadata/ShowDevicesStatement.java | 12 ++
.../plan/statement/metadata/ShowTTLStatement.java | 11 ++
.../metadata/ShowTimeSeriesStatement.java | 12 ++
.../template/ShowPathSetTemplateStatement.java | 11 ++
.../template/ShowPathsUsingTemplateStatement.java | 11 ++
.../metadata/view/ShowLogicalViewStatement.java | 11 ++
.../mtree/impl/mem/MTreeBelowSGMemoryImpl.java | 54 +++++--
.../mtree/impl/pbtree/MTreeBelowSGCachedImpl.java | 56 +++++--
.../schemaregion/mtree/traverser/Traverser.java | 11 +-
.../mtree/traverser/basic/DatabaseTraverser.java | 10 +-
.../mtree/traverser/basic/EntityTraverser.java | 11 +-
.../mtree/traverser/basic/MNodeTraverser.java | 11 +-
.../traverser/basic/MeasurementTraverser.java | 12 +-
.../traverser/collector/DatabaseCollector.java | 9 +-
.../mtree/traverser/collector/EntityCollector.java | 9 +-
.../traverser/collector/MNodeAboveDBCollector.java | 9 +-
.../mtree/traverser/collector/MNodeCollector.java | 9 +-
.../traverser/collector/MeasurementCollector.java | 9 +-
.../mtree/traverser/counter/DatabaseCounter.java | 10 +-
.../mtree/traverser/counter/EntityCounter.java | 10 +-
.../traverser/counter/MeasurementCounter.java | 9 +-
.../mtree/traverser/updater/EntityUpdater.java | 11 +-
.../traverser/updater/MeasurementUpdater.java | 10 +-
.../schemaregion/read/req/IShowSchemaPlan.java | 3 +
.../read/req/SchemaRegionReadPlanFactory.java | 69 +++------
.../read/req/impl/AbstractShowSchemaPlanImpl.java | 13 +-
.../read/req/impl/ShowDevicesPlanImpl.java | 6 +-
.../read/req/impl/ShowNodesPlanImpl.java | 6 +-
.../read/req/impl/ShowTimeSeriesPlanImpl.java | 6 +-
.../template/ClusterTemplateManager.java | 8 +-
.../db/schemaengine/template/ITemplateManager.java | 4 +-
.../metrics/IoTDBInternalLocalReporter.java | 8 +-
.../schemaRegion/SchemaRegionAliasAndTagTest.java | 21 +--
.../schemaRegion/SchemaRegionBasicTest.java | 172 ++++++++-------------
.../schemaRegion/SchemaRegionManagementTest.java | 22 +--
.../schemaRegion/SchemaRegionTemplateTest.java | 6 +-
.../schemaRegion/SchemaRegionTestUtil.java | 90 +++++++++--
.../schema/SchemaQueryScanOperatorTest.java | 12 +-
.../plan/analyze/FakePartitionFetcherImpl.java | 2 +-
.../distribution/DistributionPlannerBasicTest.java | 10 +-
.../queryengine/plan/plan/distribution/Util.java | 2 +-
.../read/DeviceSchemaScanNodeSerdeTest.java | 4 +-
.../NodeManagementMemoryMergeNodeSerdeTest.java | 6 +-
.../read/PathsUsingTemplateScanNodeTest.java | 6 +-
.../metadata/read/SchemaCountNodeSerdeTest.java | 12 +-
.../read/TimeSeriesSchemaScanNodeSerdeTest.java | 4 +-
.../apache/iotdb/commons/path/PathPatternTree.java | 9 ++
.../apache/iotdb/commons/path/fa/FAFactory.java | 2 +-
.../iotdb/commons/schema/SchemaConstant.java | 18 ++-
.../iotdb/commons/path/PathPatternTreeTest.java | 7 +-
.../src/main/thrift/confignode.thrift | 19 ++-
pom.xml | 10 +-
101 files changed, 1178 insertions(+), 541 deletions(-)
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java
index e39a1a14624..f19e2acabe3 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/database/IoTDBDatabaseSetAndDeleteIT.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TCountDatabaseResp;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchemaResp;
import org.apache.iotdb.confignode.rpc.thrift.TDeleteDatabasesReq;
+import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq;
import org.apache.iotdb.confignode.rpc.thrift.TSetDataReplicationFactorReq;
import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaReplicationFactorReq;
import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionIntervalReq;
@@ -47,6 +48,8 @@ import java.util.Arrays;
import java.util.List;
import java.util.Map;
+import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_SCOPE_BINARY;
+
@RunWith(IoTDBTestRunner.class)
@Category({LocalStandaloneIT.class, ClusterIT.class})
public class IoTDBDatabaseSetAndDeleteIT {
@@ -86,19 +89,25 @@ public class IoTDBDatabaseSetAndDeleteIT {
Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
// test count all Databases
- TCountDatabaseResp countResp = client.countMatchedDatabases(Arrays.asList("root", "**"));
+ TCountDatabaseResp countResp =
+ client.countMatchedDatabases(
+ new TGetDatabaseReq(Arrays.asList("root", "**"), ALL_MATCH_SCOPE_BINARY));
Assert.assertEquals(
TSStatusCode.SUCCESS_STATUS.getStatusCode(), countResp.getStatus().getCode());
Assert.assertEquals(2, countResp.getCount());
// test count one Database
- countResp = client.countMatchedDatabases(Arrays.asList("root", "sg0"));
+ countResp =
+ client.countMatchedDatabases(
+ new TGetDatabaseReq(Arrays.asList("root", "sg0"), ALL_MATCH_SCOPE_BINARY));
Assert.assertEquals(
TSStatusCode.SUCCESS_STATUS.getStatusCode(), countResp.getStatus().getCode());
Assert.assertEquals(1, countResp.getCount());
// test query all DatabaseSchemas
- TDatabaseSchemaResp getResp = client.getMatchedDatabaseSchemas(Arrays.asList("root", "**"));
+ TDatabaseSchemaResp getResp =
+ client.getMatchedDatabaseSchemas(
+ new TGetDatabaseReq(Arrays.asList("root", "**"), ALL_MATCH_SCOPE_BINARY));
Assert.assertEquals(
TSStatusCode.SUCCESS_STATUS.getStatusCode(), getResp.getStatus().getCode());
Map<String, TDatabaseSchema> schemaMap = getResp.getDatabaseSchemaMap();
@@ -134,7 +143,9 @@ public class IoTDBDatabaseSetAndDeleteIT {
Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
// test setter results
- getResp = client.getMatchedDatabaseSchemas(Arrays.asList("root", "sg1"));
+ getResp =
+ client.getMatchedDatabaseSchemas(
+ new TGetDatabaseReq(Arrays.asList("root", "sg1"), ALL_MATCH_SCOPE_BINARY));
Assert.assertEquals(
TSStatusCode.SUCCESS_STATUS.getStatusCode(), getResp.getStatus().getCode());
schemaMap = getResp.getDatabaseSchemaMap();
@@ -169,7 +180,9 @@ public class IoTDBDatabaseSetAndDeleteIT {
List<String> sgs = Arrays.asList(sg0, sg1);
deleteStorageGroupsReq.setPrefixPathList(sgs);
TSStatus deleteSgStatus = client.deleteDatabases(deleteStorageGroupsReq);
- TDatabaseSchemaResp root = client.getMatchedDatabaseSchemas(Arrays.asList("root", "*"));
+ TDatabaseSchemaResp root =
+ client.getMatchedDatabaseSchemas(
+ new TGetDatabaseReq(Arrays.asList("root", "*"), ALL_MATCH_SCOPE_BINARY));
Assert.assertTrue(root.getDatabaseSchemaMap().isEmpty());
Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), deleteSgStatus.getCode());
}
diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
index 4df3da3ca80..d4d7c0125b6 100644
--- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBPartitionGetterIT.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.confignode.it.utils.ConfigNodeTestUtils;
import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListReq;
import org.apache.iotdb.confignode.rpc.thrift.TCountTimeSlotListResp;
@@ -30,6 +31,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TDataNodeInfo;
import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq;
import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionTableResp;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
+import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetRegionIdReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetRegionIdResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetSeriesSlotListReq;
@@ -326,8 +328,10 @@ public class IoTDBPartitionGetterIT {
// Check the number of DataRegionGroup.
// And this number should be greater than or equal to leastDataRegionGroupNum
- TShowDatabaseResp showStorageGroupResp =
- client.showDatabase(Arrays.asList(storageGroup.split("\\.")));
+ TGetDatabaseReq req =
+ new TGetDatabaseReq(
+ Arrays.asList(storageGroup.split("\\.")), SchemaConstant.ALL_MATCH_SCOPE_BINARY);
+ TShowDatabaseResp showStorageGroupResp = client.showDatabase(req);
Assert.assertTrue(
showStorageGroupResp.getDatabaseInfoMap().get(storageGroup).getDataRegionNum()
>= leastDataRegionGroupNum);
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/database/CountDatabasePlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/database/CountDatabasePlan.java
index 0973490be04..3e838c08a06 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/database/CountDatabasePlan.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/database/CountDatabasePlan.java
@@ -19,6 +19,7 @@
package org.apache.iotdb.confignode.consensus.request.read.database;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil;
import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan;
import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
@@ -32,6 +33,7 @@ import java.util.List;
public class CountDatabasePlan extends ConfigPhysicalPlan {
private String[] storageGroupPattern;
+ private PathPatternTree scope;
public CountDatabasePlan() {
super(ConfigPhysicalPlanType.CountDatabase);
@@ -41,20 +43,27 @@ public class CountDatabasePlan extends ConfigPhysicalPlan {
super(type);
}
- public CountDatabasePlan(List<String> storageGroupPattern) {
+ public CountDatabasePlan(List<String> storageGroupPattern, PathPatternTree scope) {
this();
this.storageGroupPattern = storageGroupPattern.toArray(new String[0]);
+ this.scope = scope;
}
- public CountDatabasePlan(ConfigPhysicalPlanType type, List<String> storageGroupPattern) {
+ public CountDatabasePlan(
+ ConfigPhysicalPlanType type, List<String> storageGroupPattern, PathPatternTree scope) {
super(type);
this.storageGroupPattern = storageGroupPattern.toArray(new String[0]);
+ this.scope = scope;
}
public String[] getDatabasePattern() {
return storageGroupPattern;
}
+ public PathPatternTree getScope() {
+ return scope;
+ }
+
@Override
protected void serializeImpl(DataOutputStream stream) throws IOException {
stream.writeShort(getType().getPlanType());
@@ -63,6 +72,7 @@ public class CountDatabasePlan extends ConfigPhysicalPlan {
for (String node : storageGroupPattern) {
BasicStructureSerDeUtil.write(node, stream);
}
+ scope.serialize(stream);
}
@Override
@@ -72,6 +82,7 @@ public class CountDatabasePlan extends ConfigPhysicalPlan {
for (int i = 0; i < length; i++) {
storageGroupPattern[i] = BasicStructureSerDeUtil.readString(buffer);
}
+ scope = PathPatternTree.deserialize(buffer);
}
@Override
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/database/GetDatabasePlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/database/GetDatabasePlan.java
index 908d0cbb81a..161e58776f5 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/database/GetDatabasePlan.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/database/GetDatabasePlan.java
@@ -19,6 +19,7 @@
package org.apache.iotdb.confignode.consensus.request.read.database;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
import java.util.List;
@@ -29,7 +30,7 @@ public class GetDatabasePlan extends CountDatabasePlan {
super(ConfigPhysicalPlanType.GetDatabase);
}
- public GetDatabasePlan(List<String> storageGroupPathPattern) {
- super(ConfigPhysicalPlanType.GetDatabase, storageGroupPathPattern);
+ public GetDatabasePlan(List<String> storageGroupPathPattern, PathPatternTree scope) {
+ super(ConfigPhysicalPlanType.GetDatabase, storageGroupPathPattern, scope);
}
}
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/partition/GetNodePathsPartitionPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/partition/GetNodePathsPartitionPlan.java
index de99ce2cf91..978a70853b2 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/partition/GetNodePathsPartitionPlan.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/partition/GetNodePathsPartitionPlan.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.confignode.consensus.request.read.partition;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PathDeserializeUtil;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan;
import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
@@ -31,12 +32,21 @@ import java.util.Objects;
public class GetNodePathsPartitionPlan extends ConfigPhysicalPlan {
private PartialPath partialPath;
+ private PathPatternTree scope;
private int level = -1;
public GetNodePathsPartitionPlan() {
super(ConfigPhysicalPlanType.GetNodePathsPartition);
}
+ public PathPatternTree getScope() {
+ return scope;
+ }
+
+ public void setScope(PathPatternTree scope) {
+ this.scope = scope;
+ }
+
public PartialPath getPartialPath() {
return partialPath;
}
@@ -57,12 +67,14 @@ public class GetNodePathsPartitionPlan extends ConfigPhysicalPlan {
protected void serializeImpl(DataOutputStream stream) throws IOException {
stream.writeShort(getType().getPlanType());
partialPath.serialize(stream);
+ scope.serialize(stream);
stream.writeInt(level);
}
@Override
protected void deserializeImpl(ByteBuffer buffer) throws IOException {
partialPath = (PartialPath) PathDeserializeUtil.deserialize(buffer);
+ scope = PathPatternTree.deserialize(buffer);
level = buffer.getInt();
}
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/template/GetPathsSetTemplatePlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/template/GetPathsSetTemplatePlan.java
index 0496ca3acf2..dfb05a7d222 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/template/GetPathsSetTemplatePlan.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/template/GetPathsSetTemplatePlan.java
@@ -19,6 +19,7 @@
package org.apache.iotdb.confignode.consensus.request.read.template;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan;
import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
@@ -30,28 +31,36 @@ import java.nio.ByteBuffer;
public class GetPathsSetTemplatePlan extends ConfigPhysicalPlan {
private String name;
+ private PathPatternTree scope;
public GetPathsSetTemplatePlan() {
super(ConfigPhysicalPlanType.GetPathsSetTemplate);
}
- public GetPathsSetTemplatePlan(String name) {
+ public GetPathsSetTemplatePlan(String name, PathPatternTree scope) {
super(ConfigPhysicalPlanType.GetPathsSetTemplate);
this.name = name;
+ this.scope = scope;
}
public String getName() {
return name;
}
+ public PathPatternTree getScope() {
+ return scope;
+ }
+
@Override
protected void serializeImpl(DataOutputStream stream) throws IOException {
stream.writeShort(getType().getPlanType());
ReadWriteIOUtils.write(name, stream);
+ scope.serialize(stream);
}
@Override
protected void deserializeImpl(ByteBuffer buffer) throws IOException {
this.name = ReadWriteIOUtils.readString(buffer);
+ this.scope = PathPatternTree.deserialize(buffer);
}
}
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index e6c5500d007..285243a796c 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -38,6 +38,7 @@ 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.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.commons.service.metric.MetricService;
import org.apache.iotdb.commons.utils.AuthUtils;
import org.apache.iotdb.commons.utils.PathUtils;
@@ -129,9 +130,11 @@ import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetAllPipeInfoResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetDataNodeLocationsResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetJarInListReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetJarInListResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetLocationForTriggerResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetPipePluginTableResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetRegionIdReq;
@@ -756,11 +759,13 @@ public class ConfigManager implements IManager {
}
@Override
- public TSchemaNodeManagementResp getNodePathsPartition(PartialPath partialPath, Integer level) {
+ public TSchemaNodeManagementResp getNodePathsPartition(
+ PartialPath partialPath, PathPatternTree scope, Integer level) {
TSStatus status = confirmLeader();
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
GetNodePathsPartitionPlan getNodePathsPartitionPlan = new GetNodePathsPartitionPlan();
getNodePathsPartitionPlan.setPartialPath(partialPath);
+ getNodePathsPartitionPlan.setScope(scope);
if (null != level) {
getNodePathsPartitionPlan.setLevel(level);
}
@@ -1405,9 +1410,14 @@ public class ConfigManager implements IManager {
}
@Override
- public TShowDatabaseResp showDatabase(GetDatabasePlan getDatabasePlan) {
+ public TShowDatabaseResp showDatabase(TGetDatabaseReq req) {
TSStatus status = confirmLeader();
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+ PathPatternTree scope =
+ req.getScopePatternTree() == null
+ ? SchemaConstant.ALL_MATCH_SCOPE
+ : PathPatternTree.deserialize(ByteBuffer.wrap(req.getScopePatternTree()));
+ GetDatabasePlan getDatabasePlan = new GetDatabasePlan(req.getDatabasePathPattern(), scope);
return getClusterSchemaManager().showDatabase(getDatabasePlan);
} else {
return new TShowDatabaseResp().setStatus(status);
@@ -1483,10 +1493,14 @@ public class ConfigManager implements IManager {
}
@Override
- public TGetPathsSetTemplatesResp getPathsSetTemplate(String req) {
+ public TGetPathsSetTemplatesResp getPathsSetTemplate(TGetPathsSetTemplatesReq req) {
TSStatus status = confirmLeader();
if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
- return clusterSchemaManager.getPathsSetTemplate(req);
+ PathPatternTree scope =
+ req.getScopePatternTree() == null
+ ? SchemaConstant.ALL_MATCH_SCOPE
+ : PathPatternTree.deserialize(ByteBuffer.wrap(req.getScopePatternTree()));
+ return clusterSchemaManager.getPathsSetTemplate(req.getTemplateName(), scope);
} else {
return new TGetPathsSetTemplatesResp(status);
}
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
index f30957975f0..f6ad13907be 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
@@ -74,9 +74,11 @@ import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetAllPipeInfoResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetDataNodeLocationsResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetJarInListReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetJarInListResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetLocationForTriggerResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetPipePluginTableResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetRegionIdReq;
@@ -343,7 +345,8 @@ public interface IManager {
*
* @return TSchemaNodeManagementResp
*/
- TSchemaNodeManagementResp getNodePathsPartition(PartialPath partialPath, Integer level);
+ TSchemaNodeManagementResp getNodePathsPartition(
+ PartialPath partialPath, PathPatternTree scope, Integer level);
/**
* Get DataPartition.
@@ -492,10 +495,10 @@ public interface IManager {
/**
* Show StorageGroup.
*
- * @param getStorageGroupPlan GetStorageGroupPlan, including path patterns about StorageGroups
+ * @param req TShowDatabaseReq
* @return TShowStorageGroupResp
*/
- TShowDatabaseResp showDatabase(GetDatabasePlan getStorageGroupPlan);
+ TShowDatabaseResp showDatabase(TGetDatabaseReq req);
/**
* Create schemaengine template.
@@ -531,10 +534,10 @@ public interface IManager {
/**
* show paths set schemaengine template xx.
*
- * @param req String
+ * @param req req
* @return TGetPathsSetTemplatesResp
*/
- TGetPathsSetTemplatesResp getPathsSetTemplate(String req);
+ TGetPathsSetTemplatesResp getPathsSetTemplate(TGetPathsSetTemplatesReq req);
/** Deactivate schemaengine template. */
TSStatus deactivateSchemaTemplate(TDeactivateSchemaTemplateReq req);
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java
index 46490089453..43dbec4f107 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.common.rpc.thrift.TSetTTLReq;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.commons.service.metric.MetricService;
import org.apache.iotdb.commons.utils.PathUtils;
@@ -285,10 +286,10 @@ public class ClusterSchemaManager {
*
* @return DatabaseSchemaResp
*/
- public DatabaseSchemaResp getMatchedDatabaseSchema(GetDatabasePlan getStorageGroupPlan) {
+ public DatabaseSchemaResp getMatchedDatabaseSchema(GetDatabasePlan getDatabasePlan) {
DatabaseSchemaResp resp;
try {
- resp = (DatabaseSchemaResp) getConsensusManager().read(getStorageGroupPlan);
+ resp = (DatabaseSchemaResp) getConsensusManager().read(getDatabasePlan);
} catch (ConsensusException e) {
LOGGER.warn(CONSENSUS_READ_ERROR, e);
TSStatus res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode());
@@ -778,8 +779,9 @@ public class ClusterSchemaManager {
}
/** show path set template xx */
- public TGetPathsSetTemplatesResp getPathsSetTemplate(String templateName) {
- GetPathsSetTemplatePlan getPathsSetTemplatePlan = new GetPathsSetTemplatePlan(templateName);
+ public TGetPathsSetTemplatesResp getPathsSetTemplate(String templateName, PathPatternTree scope) {
+ GetPathsSetTemplatePlan getPathsSetTemplatePlan =
+ new GetPathsSetTemplatePlan(templateName, scope);
PathInfoResp pathInfoResp;
try {
pathInfoResp = (PathInfoResp) getConsensusManager().read(getPathsSetTemplatePlan);
@@ -853,7 +855,8 @@ public class ClusterSchemaManager {
return new Pair<>(templateResp.getStatus(), null);
}
- GetPathsSetTemplatePlan getPathsSetTemplatePlan = new GetPathsSetTemplatePlan(templateName);
+ GetPathsSetTemplatePlan getPathsSetTemplatePlan =
+ new GetPathsSetTemplatePlan(templateName, SchemaConstant.ALL_MATCH_SCOPE);
PathInfoResp pathInfoResp;
try {
pathInfoResp = (PathInfoResp) getConsensusManager().read(getPathsSetTemplatePlan);
@@ -939,7 +942,8 @@ public class ClusterSchemaManager {
}
// check is template set on some path, block all template set operation
- GetPathsSetTemplatePlan getPathsSetTemplatePlan = new GetPathsSetTemplatePlan(templateName);
+ GetPathsSetTemplatePlan getPathsSetTemplatePlan =
+ new GetPathsSetTemplatePlan(templateName, SchemaConstant.ALL_MATCH_SCOPE);
PathInfoResp pathInfoResp;
try {
pathInfoResp = (PathInfoResp) getConsensusManager().read(getPathsSetTemplatePlan);
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java
index 2aff2dc81f0..2e226cd05b8 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java
@@ -559,7 +559,8 @@ public class ConfigPlanExecutor {
if (-1 == level) {
// get child paths
Pair<Set<TSchemaNode>, Set<PartialPath>> matchedChildInNextLevel =
- clusterSchemaInfo.getChildNodePathInNextLevel(partialPath);
+ clusterSchemaInfo.getChildNodePathInNextLevel(
+ partialPath, getNodePathsPartitionPlan.getScope());
alreadyMatchedNode = matchedChildInNextLevel.left;
if (!partialPath.hasMultiLevelMatchWildcard()) {
needMatchedNode = new HashSet<>();
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
index 555b6b5a406..82a8ac616ab 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
import org.apache.iotdb.commons.utils.StatusUtils;
import org.apache.iotdb.commons.utils.TestOnly;
@@ -89,6 +90,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_PATTERN;
+import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_SCOPE;
import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_TEMPLATE;
import static org.apache.iotdb.commons.schema.SchemaConstant.SYSTEM_DATABASE_PATTERN;
@@ -261,8 +263,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
databaseReadWriteLock.readLock().lock();
try {
int count =
- mTree.getDatabaseNum(ALL_MATCH_PATTERN, false)
- - mTree.getDatabaseNum(SYSTEM_DATABASE_PATTERN, false);
+ mTree.getDatabaseNum(ALL_MATCH_PATTERN, ALL_MATCH_SCOPE, false)
+ - mTree.getDatabaseNum(SYSTEM_DATABASE_PATTERN, ALL_MATCH_SCOPE, false);
if (count >= limit) {
throw new SchemaQuotaExceededException(limit);
}
@@ -278,7 +280,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
databaseReadWriteLock.readLock().lock();
try {
PartialPath patternPath = new PartialPath(plan.getDatabasePattern());
- result.setCount(mTree.getDatabaseNum(patternPath, false));
+ result.setCount(mTree.getDatabaseNum(patternPath, plan.getScope(), false));
result.setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()));
} catch (MetadataException e) {
LOGGER.error(ERROR_NAME, e);
@@ -298,7 +300,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
try {
Map<String, TDatabaseSchema> schemaMap = new HashMap<>();
PartialPath patternPath = new PartialPath(plan.getDatabasePattern());
- List<PartialPath> matchedPaths = mTree.getMatchedDatabases(patternPath, false);
+ List<PartialPath> matchedPaths =
+ mTree.getMatchedDatabases(patternPath, plan.getScope(), false);
for (PartialPath path : matchedPaths) {
schemaMap.put(
path.getFullPath(),
@@ -514,7 +517,8 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
try {
for (String rawPath : rawPathList) {
PartialPath patternPath = new PartialPath(rawPath);
- List<PartialPath> matchedPaths = mTree.getMatchedDatabases(patternPath, false);
+ List<PartialPath> matchedPaths =
+ mTree.getMatchedDatabases(patternPath, ALL_MATCH_SCOPE, false);
for (PartialPath path : matchedPaths) {
schemaMap.put(
path.getFullPath(),
@@ -668,12 +672,12 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
}
public Pair<Set<TSchemaNode>, Set<PartialPath>> getChildNodePathInNextLevel(
- PartialPath partialPath) {
+ PartialPath partialPath, PathPatternTree scope) {
Pair<Set<TSchemaNode>, Set<PartialPath>> matchedPathsInNextLevel =
new Pair<>(new HashSet<>(), new HashSet<>());
databaseReadWriteLock.readLock().lock();
try {
- matchedPathsInNextLevel = mTree.getChildNodePathInNextLevel(partialPath);
+ matchedPathsInNextLevel = mTree.getChildNodePathInNextLevel(partialPath, scope);
} catch (MetadataException e) {
LOGGER.error("Error get matched paths in next level.", e);
} finally {
@@ -841,13 +845,14 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
TSStatus status;
try {
String templateName = getPathsSetTemplatePlan.getName();
+ PathPatternTree scope = getPathsSetTemplatePlan.getScope();
int templateId;
if (templateName.equals(ONE_LEVEL_PATH_WILDCARD)) {
templateId = ALL_TEMPLATE;
} else {
templateId = templateTable.getTemplate(templateName).getId();
}
- pathInfoResp.setPathList(mTree.getPathsSetOnTemplate(templateId, false));
+ pathInfoResp.setPathList(mTree.getPathsSetOnTemplate(templateId, scope, false));
status = new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
} catch (MetadataException e) {
status = RpcUtils.getStatus(e.getErrorCode(), e.getMessage());
@@ -863,7 +868,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
for (Template template : templateList) {
id = template.getId();
try {
- List<String> pathList = mTree.getPathsSetOnTemplate(id, true);
+ List<String> pathList = mTree.getPathsSetOnTemplate(id, ALL_MATCH_SCOPE, true);
if (!pathList.isEmpty()) {
List<Pair<String, Boolean>> pathSetInfoList = new ArrayList<>();
for (String path : pathList) {
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTree.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTree.java
index 054c896097b..fdd482106b2 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTree.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTree.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
import org.apache.iotdb.commons.schema.node.utils.IMNodeIterator;
@@ -61,6 +62,7 @@ import java.util.TreeSet;
import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_SCOPE;
import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_RESULT_NODES;
import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_TEMPLATE;
import static org.apache.iotdb.commons.schema.SchemaConstant.INTERNAL_MNODE_TYPE;
@@ -169,7 +171,7 @@ public class ConfigMTree {
* @return a list contains all databases related to given path
*/
public List<PartialPath> getBelongedDatabases(PartialPath pathPattern) throws MetadataException {
- return collectDatabases(pathPattern, false, true);
+ return collectDatabases(pathPattern, ALL_MATCH_SCOPE, false, true);
}
/**
@@ -178,21 +180,26 @@ public class ConfigMTree {
* collected.
*
* @param pathPattern a path pattern or a full path
+ * @param scope traversing scope
* @param isPrefixMatch if true, the path pattern is used to match prefix path
* @return a list contains all database names under given path pattern
*/
- public List<PartialPath> getMatchedDatabases(PartialPath pathPattern, boolean isPrefixMatch)
+ public List<PartialPath> getMatchedDatabases(
+ PartialPath pathPattern, PathPatternTree scope, boolean isPrefixMatch)
throws MetadataException {
- return collectDatabases(pathPattern, isPrefixMatch, false);
+ return collectDatabases(pathPattern, scope, isPrefixMatch, false);
}
private List<PartialPath> collectDatabases(
- PartialPath pathPattern, boolean isPrefixMatch, boolean collectInternal)
+ PartialPath pathPattern,
+ PathPatternTree scope,
+ boolean isPrefixMatch,
+ boolean collectInternal)
throws MetadataException {
List<PartialPath> result = new LinkedList<>();
try (DatabaseCollector<?, ?> collector =
new DatabaseCollector<List<PartialPath>, IConfigMNode>(
- root, pathPattern, store, isPrefixMatch) {
+ root, pathPattern, store, isPrefixMatch, scope) {
@Override
protected void collectDatabase(IDatabaseMNode<IConfigMNode> node) {
@@ -230,12 +237,13 @@ public class ConfigMTree {
* used to match prefix path. All timeseries start with the matched prefix path will be counted.
*
* @param pathPattern a path pattern or a full path, may contain wildcard.
+ * @param scope traversing scope
* @param isPrefixMatch if true, the path pattern is used to match prefix path
*/
- public int getDatabaseNum(PartialPath pathPattern, boolean isPrefixMatch)
+ public int getDatabaseNum(PartialPath pathPattern, PathPatternTree scope, boolean isPrefixMatch)
throws MetadataException {
try (DatabaseCounter<IConfigMNode> counter =
- new DatabaseCounter<>(root, pathPattern, store, isPrefixMatch)) {
+ new DatabaseCounter<>(root, pathPattern, store, isPrefixMatch, scope)) {
return (int) counter.count();
}
}
@@ -378,7 +386,8 @@ public class ConfigMTree {
PartialPath pathPattern, int nodeLevel, boolean isPrefixMatch) throws MetadataException {
List<PartialPath> result = new LinkedList<>();
try (MNodeAboveDBCollector<Void, IConfigMNode> collector =
- new MNodeAboveDBCollector<Void, IConfigMNode>(root, pathPattern, store, isPrefixMatch) {
+ new MNodeAboveDBCollector<Void, IConfigMNode>(
+ root, pathPattern, store, isPrefixMatch, ALL_MATCH_SCOPE) {
@Override
protected Void collectMNode(IConfigMNode node) {
result.add(getPartialPathFromRootToNode(node));
@@ -403,14 +412,15 @@ public class ConfigMTree {
* return [root.a, root.b]
*
* @param pathPattern The given path
+ * @param scope traversing scope
* @return All child nodes' seriesPath(s) of given seriesPath.
*/
public Pair<Set<TSchemaNode>, Set<PartialPath>> getChildNodePathInNextLevel(
- PartialPath pathPattern) throws MetadataException {
+ PartialPath pathPattern, PathPatternTree scope) throws MetadataException {
Set<TSchemaNode> result = new TreeSet<>();
try (MNodeAboveDBCollector<Void, IConfigMNode> collector =
new MNodeAboveDBCollector<Void, IConfigMNode>(
- root, pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD), store, false) {
+ root, pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD), store, false, scope) {
@Override
protected Void collectMNode(IConfigMNode node) {
result.add(
@@ -481,12 +491,12 @@ public class ConfigMTree {
}
}
- public List<String> getPathsSetOnTemplate(int templateId, boolean filterPreUnset)
- throws MetadataException {
+ public List<String> getPathsSetOnTemplate(
+ int templateId, PathPatternTree scope, boolean filterPreUnset) throws MetadataException {
List<String> resSet = new ArrayList<>();
try (MNodeCollector<Void, IConfigMNode> collector =
new MNodeCollector<Void, IConfigMNode>(
- root, new PartialPath(ALL_RESULT_NODES), store, false) {
+ root, new PartialPath(ALL_RESULT_NODES), store, false, scope) {
@Override
protected boolean acceptFullMatchedNode(IConfigMNode node) {
if (super.acceptFullMatchedNode(node)) {
@@ -533,7 +543,7 @@ public class ConfigMTree {
throws MetadataException {
Map<Integer, Set<PartialPath>> result = new HashMap<>();
try (MNodeCollector<Void, IConfigMNode> collector =
- new MNodeCollector<Void, IConfigMNode>(root, pathPattern, store, false) {
+ new MNodeCollector<Void, IConfigMNode>(root, pathPattern, store, false, ALL_MATCH_SCOPE) {
@Override
protected boolean acceptFullMatchedNode(IConfigMNode node) {
return (node.getSchemaTemplateId() != NON_TEMPLATE)
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
index 6f3984be406..544c64d86d6 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
@@ -110,9 +110,11 @@ import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetAllPipeInfoResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetDataNodeLocationsResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetJarInListReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetJarInListResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetLocationForTriggerResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetPipePluginTableResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetPipeSinkReq;
@@ -446,10 +448,14 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
}
@Override
- public TCountDatabaseResp countMatchedDatabases(List<String> storageGroupPathPattern) {
+ public TCountDatabaseResp countMatchedDatabases(TGetDatabaseReq req) {
+ PathPatternTree scope =
+ req.getScopePatternTree() == null
+ ? SchemaConstant.ALL_MATCH_SCOPE
+ : PathPatternTree.deserialize(ByteBuffer.wrap(req.getScopePatternTree()));
+ CountDatabasePlan plan = new CountDatabasePlan(req.getDatabasePathPattern(), scope);
CountDatabaseResp countDatabaseResp =
- (CountDatabaseResp)
- configManager.countMatchedDatabases(new CountDatabasePlan(storageGroupPathPattern));
+ (CountDatabaseResp) configManager.countMatchedDatabases(plan);
TCountDatabaseResp resp = new TCountDatabaseResp();
countDatabaseResp.convertToRPCCountStorageGroupResp(resp);
@@ -457,10 +463,14 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
}
@Override
- public TDatabaseSchemaResp getMatchedDatabaseSchemas(List<String> storageGroupPathPattern) {
+ public TDatabaseSchemaResp getMatchedDatabaseSchemas(TGetDatabaseReq req) {
+ PathPatternTree scope =
+ req.getScopePatternTree() == null
+ ? SchemaConstant.ALL_MATCH_SCOPE
+ : PathPatternTree.deserialize(ByteBuffer.wrap(req.getScopePatternTree()));
+ GetDatabasePlan plan = new GetDatabasePlan(req.getDatabasePathPattern(), scope);
DatabaseSchemaResp databaseSchemaResp =
- (DatabaseSchemaResp)
- configManager.getMatchedDatabaseSchemas(new GetDatabasePlan(storageGroupPathPattern));
+ (DatabaseSchemaResp) configManager.getMatchedDatabaseSchemas(plan);
return databaseSchemaResp.convertToRPCStorageGroupSchemaResp();
}
@@ -483,8 +493,12 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
public TSchemaNodeManagementResp getSchemaNodeManagementPartition(TSchemaNodeManagementReq req) {
PathPatternTree patternTree =
PathPatternTree.deserialize(ByteBuffer.wrap(req.getPathPatternTree()));
+ PathPatternTree scope =
+ req.getScopePatternTree() == null
+ ? SchemaConstant.ALL_MATCH_SCOPE
+ : PathPatternTree.deserialize(ByteBuffer.wrap(req.getScopePatternTree()));
PartialPath partialPath = patternTree.getAllPathPatterns().get(0);
- return configManager.getNodePathsPartition(partialPath, req.getLevel());
+ return configManager.getNodePathsPartition(partialPath, scope, req.getLevel());
}
@Override
@@ -816,8 +830,8 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
}
@Override
- public TShowDatabaseResp showDatabase(List<String> storageGroupPathPattern) {
- return configManager.showDatabase(new GetDatabasePlan(storageGroupPathPattern));
+ public TShowDatabaseResp showDatabase(TGetDatabaseReq req) {
+ return configManager.showDatabase(req);
}
@Override
@@ -841,7 +855,7 @@ public class ConfigNodeRPCServiceProcessor implements IConfigNodeRPCService.Ifac
}
@Override
- public TGetPathsSetTemplatesResp getPathsSetTemplate(String req) {
+ public TGetPathsSetTemplatesResp getPathsSetTemplate(TGetPathsSetTemplatesReq req) {
return configManager.getPathsSetTemplate(req);
}
diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
index f30d46dd117..7bf61af210b 100644
--- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
+++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
@@ -169,6 +169,7 @@ import java.util.Set;
import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.ConfigRegion;
import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.DataRegion;
import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.SchemaRegion;
+import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_SCOPE;
import static org.junit.Assert.assertEquals;
public class ConfigPhysicalPlanSerDeTest {
@@ -320,7 +321,7 @@ public class ConfigPhysicalPlanSerDeTest {
@Test
public void CountStorageGroupPlanTest() throws IOException {
- CountDatabasePlan req0 = new CountDatabasePlan(Arrays.asList("root", "sg"));
+ CountDatabasePlan req0 = new CountDatabasePlan(Arrays.asList("root", "sg"), ALL_MATCH_SCOPE);
CountDatabasePlan req1 =
(CountDatabasePlan) ConfigPhysicalPlan.Factory.create(req0.serializeToByteBuffer());
Assert.assertEquals(req0, req1);
@@ -328,7 +329,7 @@ public class ConfigPhysicalPlanSerDeTest {
@Test
public void GetStorageGroupPlanTest() throws IOException {
- GetDatabasePlan req0 = new GetDatabasePlan(Arrays.asList("root", "sg"));
+ GetDatabasePlan req0 = new GetDatabasePlan(Arrays.asList("root", "sg"), ALL_MATCH_SCOPE);
CountDatabasePlan req1 =
(CountDatabasePlan) ConfigPhysicalPlan.Factory.create(req0.serializeToByteBuffer());
Assert.assertEquals(req0, req1);
@@ -923,6 +924,7 @@ public class ConfigPhysicalPlanSerDeTest {
public void GetNodePathsPartitionPlanTest() throws IOException, IllegalPathException {
GetNodePathsPartitionPlan getNodePathsPartitionPlan0 = new GetNodePathsPartitionPlan();
getNodePathsPartitionPlan0.setPartialPath(new PartialPath("root.sg1.**"));
+ getNodePathsPartitionPlan0.setScope(ALL_MATCH_SCOPE);
GetNodePathsPartitionPlan getNodePathsPartitionPlan1 =
(GetNodePathsPartitionPlan)
ConfigPhysicalPlan.Factory.create(getNodePathsPartitionPlan0.serializeToByteBuffer());
@@ -952,7 +954,7 @@ public class ConfigPhysicalPlanSerDeTest {
@Test
public void ShowPathSetTemplatePlanTest() throws IOException {
GetPathsSetTemplatePlan getPathsSetTemplatePlan0 =
- new GetPathsSetTemplatePlan("template_name_test");
+ new GetPathsSetTemplatePlan("template_name_test", ALL_MATCH_SCOPE);
GetPathsSetTemplatePlan getPathsSetTemplatePlan1 =
(GetPathsSetTemplatePlan)
ConfigPhysicalPlan.Factory.create(getPathsSetTemplatePlan0.serializeToByteBuffer());
diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java
index 507f8f99be0..cb00cca843f 100644
--- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java
+++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java
@@ -58,6 +58,7 @@ import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
+import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_SCOPE;
import static org.apache.iotdb.db.utils.constant.TestConstant.BASE_OUTPUT_PATH;
public class ClusterSchemaInfoTest {
@@ -110,7 +111,8 @@ public class ClusterSchemaInfoTest {
Assert.assertEquals(storageGroupPathList.size(), clusterSchemaInfo.getDatabaseNames().size());
GetDatabasePlan getStorageGroupReq =
- new GetDatabasePlan(Arrays.asList(PathUtils.splitPathToDetachedNodes("root.**")));
+ new GetDatabasePlan(
+ Arrays.asList(PathUtils.splitPathToDetachedNodes("root.**")), ALL_MATCH_SCOPE);
Map<String, TDatabaseSchema> reloadResult =
clusterSchemaInfo.getMatchedDatabaseSchemas(getStorageGroupReq).getSchemaMap();
Assert.assertEquals(testMap, reloadResult);
@@ -143,7 +145,7 @@ public class ClusterSchemaInfoTest {
List<String> pathList =
clusterSchemaInfo
- .getPathsSetTemplate(new GetPathsSetTemplatePlan(templateName))
+ .getPathsSetTemplate(new GetPathsSetTemplatePlan(templateName, ALL_MATCH_SCOPE))
.getPathList();
Assert.assertEquals(3, pathList.size());
Assert.assertTrue(pathList.contains("root.test1.template"));
diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTreeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTreeTest.java
index 925c6cc6cea..77cb392708a 100644
--- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTreeTest.java
+++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTreeTest.java
@@ -37,6 +37,7 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Set;
+import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_SCOPE;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@@ -192,14 +193,14 @@ public class ConfigMTreeTest {
root.setStorageGroup(new PartialPath("root.sg3"));
root.setStorageGroup(new PartialPath("root.a.b.sg3"));
- assertEquals(7, root.getDatabaseNum(new PartialPath("root.**"), false));
- assertEquals(3, root.getDatabaseNum(new PartialPath("root.*"), false));
- assertEquals(2, root.getDatabaseNum(new PartialPath("root.*.*"), false));
- assertEquals(2, root.getDatabaseNum(new PartialPath("root.*.*.*"), false));
- assertEquals(1, root.getDatabaseNum(new PartialPath("root.*.sg1"), false));
- assertEquals(2, root.getDatabaseNum(new PartialPath("root.**.sg1"), false));
- assertEquals(1, root.getDatabaseNum(new PartialPath("root.sg3"), false));
- assertEquals(2, root.getDatabaseNum(new PartialPath("root.*.b.*"), false));
+ assertEquals(7, root.getDatabaseNum(new PartialPath("root.**"), ALL_MATCH_SCOPE, false));
+ assertEquals(3, root.getDatabaseNum(new PartialPath("root.*"), ALL_MATCH_SCOPE, false));
+ assertEquals(2, root.getDatabaseNum(new PartialPath("root.*.*"), ALL_MATCH_SCOPE, false));
+ assertEquals(2, root.getDatabaseNum(new PartialPath("root.*.*.*"), ALL_MATCH_SCOPE, false));
+ assertEquals(1, root.getDatabaseNum(new PartialPath("root.*.sg1"), ALL_MATCH_SCOPE, false));
+ assertEquals(2, root.getDatabaseNum(new PartialPath("root.**.sg1"), ALL_MATCH_SCOPE, false));
+ assertEquals(1, root.getDatabaseNum(new PartialPath("root.sg3"), ALL_MATCH_SCOPE, false));
+ assertEquals(2, root.getDatabaseNum(new PartialPath("root.*.b.*"), ALL_MATCH_SCOPE, false));
}
@Test
@@ -278,9 +279,17 @@ public class ConfigMTreeTest {
i, newTree.getNodeWithAutoCreate(pathList[i].concatNode("a")).getSchemaTemplateId());
}
- assertEquals(3, newTree.getMatchedDatabases(new PartialPath("root.**.sg"), false).size());
- assertEquals(2, newTree.getMatchedDatabases(new PartialPath("root.**.b.sg"), false).size());
- assertEquals(1, newTree.getMatchedDatabases(new PartialPath("root.*.*.sg"), false).size());
+ assertEquals(
+ 3,
+ newTree.getMatchedDatabases(new PartialPath("root.**.sg"), ALL_MATCH_SCOPE, false).size());
+ assertEquals(
+ 2,
+ newTree
+ .getMatchedDatabases(new PartialPath("root.**.b.sg"), ALL_MATCH_SCOPE, false)
+ .size());
+ assertEquals(
+ 1,
+ newTree.getMatchedDatabases(new PartialPath("root.*.*.sg"), ALL_MATCH_SCOPE, false).size());
}
@Test
@@ -313,7 +322,7 @@ public class ConfigMTreeTest {
}
try {
- List<String> pathList = root.getPathsSetOnTemplate(0, false);
+ List<String> pathList = root.getPathsSetOnTemplate(0, ALL_MATCH_SCOPE, false);
Assert.assertTrue(pathList.contains("root.a.template0"));
Assert.assertTrue(pathList.contains("root.a.b.template0"));
} catch (MetadataException e) {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java
index d9390ab1976..1e087c2e052 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java
@@ -78,9 +78,11 @@ import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetAllPipeInfoResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetDataNodeLocationsResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetJarInListReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetJarInListResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetLocationForTriggerResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetPipePluginTableResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetPipeSinkReq;
@@ -458,19 +460,15 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
}
@Override
- public TCountDatabaseResp countMatchedDatabases(List<String> storageGroupPathPattern)
- throws TException {
+ public TCountDatabaseResp countMatchedDatabases(TGetDatabaseReq req) throws TException {
return executeRemoteCallWithRetry(
- () -> client.countMatchedDatabases(storageGroupPathPattern),
- resp -> !updateConfigNodeLeader(resp.status));
+ () -> client.countMatchedDatabases(req), resp -> !updateConfigNodeLeader(resp.status));
}
@Override
- public TDatabaseSchemaResp getMatchedDatabaseSchemas(List<String> storageGroupPathPattern)
- throws TException {
+ public TDatabaseSchemaResp getMatchedDatabaseSchemas(TGetDatabaseReq req) throws TException {
return executeRemoteCallWithRetry(
- () -> client.getMatchedDatabaseSchemas(storageGroupPathPattern),
- resp -> !updateConfigNodeLeader(resp.status));
+ () -> client.getMatchedDatabaseSchemas(req), resp -> !updateConfigNodeLeader(resp.status));
}
@Override
@@ -665,10 +663,9 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
}
@Override
- public TShowDatabaseResp showDatabase(List<String> storageGroupPathPattern) throws TException {
+ public TShowDatabaseResp showDatabase(TGetDatabaseReq req) throws TException {
return executeRemoteCallWithRetry(
- () -> client.showDatabase(storageGroupPathPattern),
- resp -> !updateConfigNodeLeader(resp.status));
+ () -> client.showDatabase(req), resp -> !updateConfigNodeLeader(resp.status));
}
@Override
@@ -792,7 +789,8 @@ public class ConfigNodeClient implements IConfigNodeRPCService.Iface, ThriftClie
}
@Override
- public TGetPathsSetTemplatesResp getPathsSetTemplate(String req) throws TException {
+ public TGetPathsSetTemplatesResp getPathsSetTemplate(TGetPathsSetTemplatesReq req)
+ throws TException {
return executeRemoteCallWithRetry(
() -> client.getPathsSetTemplate(req), resp -> !updateConfigNodeLeader(resp.status));
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java
index ce5cbe6d15c..697181a7a19 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java
@@ -43,6 +43,7 @@ import org.apache.iotdb.commons.path.PathDeserializeUtil;
import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.pipe.plugin.meta.PipePluginMeta;
import org.apache.iotdb.commons.pipe.task.meta.PipeMeta;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression;
import org.apache.iotdb.commons.service.metric.MetricService;
import org.apache.iotdb.commons.service.metric.enums.Tag;
@@ -782,7 +783,7 @@ public class DataNodeInternalRPCServiceImpl implements IDataNodeRPCService.Iface
for (PartialPath pattern : filteredPatternTree.getAllPathPatterns()) {
ISchemaSource<ITimeSeriesSchemaInfo> schemaSource =
SchemaSourceFactory.getTimeSeriesSchemaCountSource(
- pattern, false, null, null);
+ pattern, false, null, null, SchemaConstant.ALL_MATCH_SCOPE);
try (ISchemaReader<ITimeSeriesSchemaInfo> schemaReader =
schemaSource.getSchemaReader(schemaRegion)) {
if (schemaReader.hasNext()) {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/DeviceSchemaSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/DeviceSchemaSource.java
index 5f9d67bedcd..68ef273da4f 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/DeviceSchemaSource.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/DeviceSchemaSource.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.schema.source;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.exception.runtime.SchemaExecutionException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
@@ -39,6 +40,7 @@ import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_PATTERN;
public class DeviceSchemaSource implements ISchemaSource<IDeviceSchemaInfo> {
private final PartialPath pathPattern;
+ private final PathPatternTree scope;
private final boolean isPrefixMatch;
private final long limit;
@@ -54,7 +56,8 @@ public class DeviceSchemaSource implements ISchemaSource<IDeviceSchemaInfo> {
long limit,
long offset,
boolean hasSgCol,
- SchemaFilter schemaFilter) {
+ SchemaFilter schemaFilter,
+ PathPatternTree scope) {
this.pathPattern = pathPattern;
this.isPrefixMatch = isPrefixPath;
@@ -63,6 +66,7 @@ public class DeviceSchemaSource implements ISchemaSource<IDeviceSchemaInfo> {
this.hasSgCol = hasSgCol;
this.schemaFilter = schemaFilter;
+ this.scope = scope;
}
@Override
@@ -70,7 +74,7 @@ public class DeviceSchemaSource implements ISchemaSource<IDeviceSchemaInfo> {
try {
return schemaRegion.getDeviceReader(
SchemaRegionReadPlanFactory.getShowDevicesPlan(
- pathPattern, limit, offset, isPrefixMatch, schemaFilter));
+ pathPattern, limit, offset, isPrefixMatch, schemaFilter, scope));
} 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/LogicalViewSchemaSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/LogicalViewSchemaSource.java
index 02f00b05c52..cdfa4d6b7b9 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
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.schema.source;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.exception.runtime.SchemaExecutionException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.commons.schema.filter.SchemaFilterFactory;
import org.apache.iotdb.commons.schema.view.LogicalViewSchema;
@@ -44,6 +45,7 @@ import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_PATTERN;
public class LogicalViewSchemaSource implements ISchemaSource<ITimeSeriesSchemaInfo> {
private final PartialPath pathPattern;
+ private final PathPatternTree scope;
private final long limit;
private final long offset;
@@ -51,8 +53,13 @@ public class LogicalViewSchemaSource implements ISchemaSource<ITimeSeriesSchemaI
private final SchemaFilter schemaFilter;
LogicalViewSchemaSource(
- PartialPath pathPattern, long limit, long offset, SchemaFilter schemaFilter) {
+ PartialPath pathPattern,
+ long limit,
+ long offset,
+ SchemaFilter schemaFilter,
+ PathPatternTree scope) {
this.pathPattern = pathPattern;
+ this.scope = scope;
this.limit = limit;
this.offset = offset;
@@ -72,7 +79,8 @@ public class LogicalViewSchemaSource implements ISchemaSource<ITimeSeriesSchemaI
false,
SchemaFilterFactory.and(
schemaFilter, SchemaFilterFactory.createViewTypeFilter(ViewType.VIEW)),
- true));
+ true,
+ scope));
} 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/NodeSchemaSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/NodeSchemaSource.java
index 1e41396043f..be0972f887f 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/NodeSchemaSource.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/NodeSchemaSource.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.schema.source;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.exception.runtime.SchemaExecutionException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion;
@@ -39,12 +40,13 @@ import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCAR
public class NodeSchemaSource implements ISchemaSource<INodeSchemaInfo> {
private final PartialPath pathPattern;
-
+ private final PathPatternTree scope;
private final int level;
- NodeSchemaSource(PartialPath pathPattern, int level) {
+ NodeSchemaSource(PartialPath pathPattern, int level, PathPatternTree scope) {
this.pathPattern = pathPattern;
this.level = level;
+ this.scope = scope;
}
@Override
@@ -53,9 +55,10 @@ public class NodeSchemaSource implements ISchemaSource<INodeSchemaInfo> {
if (-1 == level) {
showNodesPlan =
SchemaRegionReadPlanFactory.getShowNodesPlan(
- pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD));
+ pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD), scope);
} else {
- showNodesPlan = SchemaRegionReadPlanFactory.getShowNodesPlan(pathPattern, level, false);
+ showNodesPlan =
+ SchemaRegionReadPlanFactory.getShowNodesPlan(pathPattern, level, false, scope);
}
try {
return schemaRegion.getNodeReader(showNodesPlan);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/PathsUsingTemplateSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/PathsUsingTemplateSource.java
index fd1364abf22..28b8ad94ef4 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/PathsUsingTemplateSource.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/PathsUsingTemplateSource.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.schema.source;
import org.apache.iotdb.commons.exception.runtime.SchemaExecutionException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion;
@@ -39,12 +40,15 @@ import java.util.NoSuchElementException;
public class PathsUsingTemplateSource implements ISchemaSource<IDeviceSchemaInfo> {
private final List<PartialPath> pathPatternList;
+ private final PathPatternTree scope;
private final int templateId;
- PathsUsingTemplateSource(List<PartialPath> pathPatternList, int templateId) {
+ PathsUsingTemplateSource(
+ List<PartialPath> pathPatternList, int templateId, PathPatternTree scope) {
this.pathPatternList = pathPatternList;
this.templateId = templateId;
+ this.scope = scope;
}
@Override
@@ -126,7 +130,7 @@ public class PathsUsingTemplateSource implements ISchemaSource<IDeviceSchemaInfo
currentDeviceReader =
schemaRegion.getDeviceReader(
SchemaRegionReadPlanFactory.getShowDevicesPlan(
- pathPatternIterator.next(), 0, 0, false, templateId));
+ pathPatternIterator.next(), 0, 0, false, templateId, scope));
if (currentDeviceReader.hasNext()) {
return true;
} else {
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 7b5a8933cc2..7fe10842579 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
@@ -20,6 +20,7 @@
package org.apache.iotdb.db.queryengine.execution.operator.schema.source;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.IDeviceSchemaInfo;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.INodeSchemaInfo;
@@ -35,53 +36,66 @@ public class SchemaSourceFactory {
// Empty constructor
}
+ // count time series
public static ISchemaSource<ITimeSeriesSchemaInfo> getTimeSeriesSchemaCountSource(
PartialPath pathPattern,
boolean isPrefixMatch,
SchemaFilter schemaFilter,
- Map<Integer, Template> templateMap) {
+ Map<Integer, Template> templateMap,
+ PathPatternTree scope) {
return new TimeSeriesSchemaSource(
- pathPattern, isPrefixMatch, 0, 0, schemaFilter, templateMap, false);
+ pathPattern, isPrefixMatch, 0, 0, schemaFilter, templateMap, false, scope);
}
+ // show time series
public static ISchemaSource<ITimeSeriesSchemaInfo> getTimeSeriesSchemaScanSource(
PartialPath pathPattern,
boolean isPrefixMatch,
long limit,
long offset,
SchemaFilter schemaFilter,
- Map<Integer, Template> templateMap) {
+ Map<Integer, Template> templateMap,
+ PathPatternTree scope) {
return new TimeSeriesSchemaSource(
- pathPattern, isPrefixMatch, limit, offset, schemaFilter, templateMap, true);
+ pathPattern, isPrefixMatch, limit, offset, schemaFilter, templateMap, true, scope);
}
+ // count device
public static ISchemaSource<IDeviceSchemaInfo> getDeviceSchemaSource(
- PartialPath pathPattern, boolean isPrefixPath) {
- return new DeviceSchemaSource(pathPattern, isPrefixPath, 0, 0, false, null);
+ PartialPath pathPattern, boolean isPrefixPath, PathPatternTree scope) {
+ return new DeviceSchemaSource(pathPattern, isPrefixPath, 0, 0, false, null, scope);
}
+ // show device
public static ISchemaSource<IDeviceSchemaInfo> getDeviceSchemaSource(
PartialPath pathPattern,
boolean isPrefixPath,
long limit,
long offset,
boolean hasSgCol,
- SchemaFilter schemaFilter) {
- return new DeviceSchemaSource(pathPattern, isPrefixPath, limit, offset, hasSgCol, schemaFilter);
+ SchemaFilter schemaFilter,
+ PathPatternTree scope) {
+ return new DeviceSchemaSource(
+ pathPattern, isPrefixPath, limit, offset, hasSgCol, schemaFilter, scope);
}
+ // show nodes
public static ISchemaSource<INodeSchemaInfo> getNodeSchemaSource(
- PartialPath pathPattern, int level) {
- return new NodeSchemaSource(pathPattern, level);
+ PartialPath pathPattern, int level, PathPatternTree scope) {
+ return new NodeSchemaSource(pathPattern, level, scope);
}
public static ISchemaSource<IDeviceSchemaInfo> getPathsUsingTemplateSource(
- List<PartialPath> pathPatternList, int templateId) {
- return new PathsUsingTemplateSource(pathPatternList, templateId);
+ List<PartialPath> pathPatternList, int templateId, PathPatternTree scope) {
+ return new PathsUsingTemplateSource(pathPatternList, templateId, scope);
}
public static ISchemaSource<ITimeSeriesSchemaInfo> getLogicalViewSchemaSource(
- PartialPath pathPattern, long limit, long offset, SchemaFilter schemaFilter) {
- return new LogicalViewSchemaSource(pathPattern, limit, offset, schemaFilter);
+ PartialPath pathPattern,
+ long limit,
+ long offset,
+ SchemaFilter schemaFilter,
+ PathPatternTree scope) {
+ return new LogicalViewSchemaSource(pathPattern, limit, offset, schemaFilter, scope);
}
}
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 17f1c2b6192..343e522759b 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
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.schema.source;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.exception.runtime.SchemaExecutionException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.commons.schema.view.ViewType;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
@@ -44,6 +45,7 @@ import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_PATTERN;
public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaInfo> {
private final PartialPath pathPattern;
+ private final PathPatternTree scope;
private final boolean isPrefixMatch;
private final long limit;
private final long offset;
@@ -58,7 +60,8 @@ public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaIn
long offset,
SchemaFilter schemaFilter,
Map<Integer, Template> templateMap,
- boolean needViewDetail) {
+ boolean needViewDetail,
+ PathPatternTree scope) {
this.pathPattern = pathPattern;
this.isPrefixMatch = isPrefixMatch;
this.limit = limit;
@@ -66,6 +69,7 @@ public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaIn
this.schemaFilter = schemaFilter;
this.templateMap = templateMap;
this.needViewDetail = needViewDetail;
+ this.scope = scope;
}
@Override
@@ -79,7 +83,8 @@ public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaIn
offset,
isPrefixMatch,
schemaFilter,
- needViewDetail));
+ needViewDetail,
+ scope));
} catch (MetadataException e) {
throw new SchemaExecutionException(e.getMessage(), e);
}
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 7701e6b8dda..836ff976639 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
@@ -2768,7 +2768,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
patternTree.appendPathPattern(countStatement.getPathPattern());
SchemaNodeManagementPartition schemaNodeManagementPartition =
partitionFetcher.getSchemaNodeManagementPartitionWithLevel(
- patternTree, countStatement.getLevel());
+ patternTree, countStatement.getAuthorityScope(), countStatement.getLevel());
if (schemaNodeManagementPartition == null) {
return analysis;
@@ -2789,6 +2789,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
return visitSchemaNodeManagementPartition(
showChildPathsStatement,
showChildPathsStatement.getPartialPath(),
+ showChildPathsStatement.getAuthorityScope(),
DatasetHeaderFactory.getShowChildPathsHeader());
}
@@ -2798,6 +2799,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
return visitSchemaNodeManagementPartition(
showChildNodesStatement,
showChildNodesStatement.getPartialPath(),
+ showChildNodesStatement.getAuthorityScope(),
DatasetHeaderFactory.getShowChildNodesHeader());
}
@@ -2812,14 +2814,14 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
}
private Analysis visitSchemaNodeManagementPartition(
- Statement statement, PartialPath path, DatasetHeader header) {
+ Statement statement, PartialPath path, PathPatternTree scope, DatasetHeader header) {
Analysis analysis = new Analysis();
analysis.setStatement(statement);
PathPatternTree patternTree = new PathPatternTree();
patternTree.appendPathPattern(path);
SchemaNodeManagementPartition schemaNodeManagementPartition =
- partitionFetcher.getSchemaNodeManagementPartition(patternTree);
+ partitionFetcher.getSchemaNodeManagementPartition(patternTree, scope);
if (schemaNodeManagementPartition == null) {
return analysis;
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java
index c13d30e780c..8b6ff56e754 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java
@@ -49,14 +49,12 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeInfo;
import org.apache.iotdb.db.queryengine.plan.analyze.cache.partition.PartitionCache;
import org.apache.iotdb.mpp.rpc.thrift.TRegionRouteReq;
import org.apache.iotdb.rpc.TSStatusCode;
-import org.apache.iotdb.tsfile.utils.PublicBAOS;
import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
@@ -159,13 +157,13 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
@Override
public SchemaNodeManagementPartition getSchemaNodeManagementPartitionWithLevel(
- PathPatternTree patternTree, Integer level) {
+ PathPatternTree patternTree, PathPatternTree scope, Integer level) {
try (ConfigNodeClient client =
configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
patternTree.constructTree();
TSchemaNodeManagementResp schemaNodeManagementResp =
client.getSchemaNodeManagementPartition(
- constructSchemaNodeManagementPartitionReq(patternTree, level));
+ constructSchemaNodeManagementPartitionReq(patternTree, scope, level));
return parseSchemaNodeManagementPartitionResp(schemaNodeManagementResp);
} catch (ClientManagerException | TException e) {
@@ -314,28 +312,19 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
}
private TSchemaPartitionReq constructSchemaPartitionReq(PathPatternTree patternTree) {
- PublicBAOS baos = new PublicBAOS();
try {
- patternTree.serialize(baos);
- ByteBuffer serializedPatternTree = ByteBuffer.allocate(baos.size());
- serializedPatternTree.put(baos.getBuf(), 0, baos.size());
- serializedPatternTree.flip();
- return new TSchemaPartitionReq(serializedPatternTree);
+ return new TSchemaPartitionReq(patternTree.serialize());
} catch (IOException e) {
throw new StatementAnalyzeException("An error occurred when serializing pattern tree");
}
}
private TSchemaNodeManagementReq constructSchemaNodeManagementPartitionReq(
- PathPatternTree patternTree, Integer level) {
- PublicBAOS baos = new PublicBAOS();
+ PathPatternTree patternTree, PathPatternTree scope, Integer level) {
try {
- patternTree.serialize(baos);
- ByteBuffer serializedPatternTree = ByteBuffer.allocate(baos.size());
- serializedPatternTree.put(baos.getBuf(), 0, baos.size());
- serializedPatternTree.flip();
TSchemaNodeManagementReq schemaNodeManagementReq =
- new TSchemaNodeManagementReq(serializedPatternTree);
+ new TSchemaNodeManagementReq(patternTree.serialize());
+ schemaNodeManagementReq.setScopePatternTree(scope.serialize());
if (null == level) {
schemaNodeManagementReq.setLevel(-1);
} else {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/IPartitionFetcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/IPartitionFetcher.java
index 8930dccb21d..5750b251cff 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/IPartitionFetcher.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/IPartitionFetcher.java
@@ -75,13 +75,13 @@ public interface IPartitionFetcher {
/** Get schema partition and matched nodes according to path pattern tree. */
default SchemaNodeManagementPartition getSchemaNodeManagementPartition(
- PathPatternTree patternTree) {
- return getSchemaNodeManagementPartitionWithLevel(patternTree, null);
+ PathPatternTree patternTree, PathPatternTree scope) {
+ return getSchemaNodeManagementPartitionWithLevel(patternTree, scope, null);
}
/** Get schema partition and matched nodes according to path pattern tree and node level. */
SchemaNodeManagementPartition getSchemaNodeManagementPartitionWithLevel(
- PathPatternTree patternTree, Integer level);
+ PathPatternTree patternTree, PathPatternTree scope, Integer level);
/** Update region cache in partition cache when receive request from config node */
boolean updateRegionCache(TRegionRouteReq req);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java
index 478033b335e..16ce3628bea 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java
@@ -37,9 +37,11 @@ import org.apache.iotdb.commons.partition.SchemaPartitionTable;
import org.apache.iotdb.commons.partition.SeriesPartitionTable;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.commons.utils.PathUtils;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchemaResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq;
import org.apache.iotdb.confignode.rpc.thrift.TRegionRouteMapResp;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -190,7 +192,8 @@ public class PartitionCache {
result.reset();
getStorageGroupMap(result, devicePaths, true);
if (!result.isSuccess()) {
- TDatabaseSchemaResp storageGroupSchemaResp = client.getMatchedDatabaseSchemas(ROOT_PATH);
+ TGetDatabaseReq req = new TGetDatabaseReq(ROOT_PATH, SchemaConstant.ALL_MATCH_SCOPE_BINARY);
+ TDatabaseSchemaResp storageGroupSchemaResp = client.getMatchedDatabaseSchemas(req);
if (storageGroupSchemaResp.getStatus().getCode()
== TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
Set<String> storageGroupNames = storageGroupSchemaResp.getDatabaseSchemaMap().keySet();
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
index 1f25a869d04..8afb2a02bf4 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
@@ -67,6 +67,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TDropFunctionReq;
import org.apache.iotdb.confignode.rpc.thrift.TDropModelReq;
import org.apache.iotdb.confignode.rpc.thrift.TDropPipePluginReq;
import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
+import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetPipePluginTableResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetRegionIdReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetRegionIdResp;
@@ -327,10 +328,13 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
try (ConfigNodeClient client =
CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
// Send request to some API server
- TShowDatabaseResp resp = client.showDatabase(databasePathPattern);
+ TGetDatabaseReq req =
+ new TGetDatabaseReq(
+ databasePathPattern, showDatabaseStatement.getAuthorityScope().serialize());
+ TShowDatabaseResp resp = client.showDatabase(req);
// build TSBlock
showDatabaseStatement.buildTSBlock(resp.getDatabaseInfoMap(), future);
- } catch (ClientManagerException | TException e) {
+ } catch (IOException | ClientManagerException | TException e) {
future.setException(e);
}
return future;
@@ -345,11 +349,14 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
Arrays.asList(countDatabaseStatement.getPathPattern().getNodes());
try (ConfigNodeClient client =
CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
- TCountDatabaseResp resp = client.countMatchedDatabases(databasePathPattern);
+ TGetDatabaseReq req =
+ new TGetDatabaseReq(
+ databasePathPattern, countDatabaseStatement.getAuthorityScope().serialize());
+ TCountDatabaseResp resp = client.countMatchedDatabases(req);
databaseNum = resp.getCount();
// build TSBlock
CountDatabaseTask.buildTSBlock(databaseNum, future);
- } catch (ClientManagerException | TException e) {
+ } catch (IOException | ClientManagerException | TException e) {
future.setException(e);
}
return future;
@@ -1126,16 +1133,19 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
Map<String, Long> databaseToTTL = new HashMap<>();
try (ConfigNodeClient client =
CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
+ ByteBuffer scope = showTTLStatement.getAuthorityScope().serialize();
if (showTTLStatement.isAll()) {
List<String> allStorageGroupPathPattern = Arrays.asList("root", "**");
- TDatabaseSchemaResp resp = client.getMatchedDatabaseSchemas(allStorageGroupPathPattern);
+ TGetDatabaseReq req = new TGetDatabaseReq(allStorageGroupPathPattern, scope);
+ TDatabaseSchemaResp resp = client.getMatchedDatabaseSchemas(req);
for (Map.Entry<String, TDatabaseSchema> entry : resp.getDatabaseSchemaMap().entrySet()) {
databaseToTTL.put(entry.getKey(), entry.getValue().getTTL());
}
} else {
for (PartialPath databasePath : databasePaths) {
List<String> databasePathPattern = Arrays.asList(databasePath.getNodes());
- TDatabaseSchemaResp resp = client.getMatchedDatabaseSchemas(databasePathPattern);
+ TGetDatabaseReq req = new TGetDatabaseReq(databasePathPattern, scope);
+ TDatabaseSchemaResp resp = client.getMatchedDatabaseSchemas(req);
for (Map.Entry<String, TDatabaseSchema> entry : resp.getDatabaseSchemaMap().entrySet()) {
if (!databaseToTTL.containsKey(entry.getKey())) {
databaseToTTL.put(entry.getKey(), entry.getValue().getTTL());
@@ -1143,7 +1153,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
}
}
}
- } catch (ClientManagerException | TException e) {
+ } catch (IOException | ClientManagerException | TException e) {
future.setException(e);
}
// build TSBlock
@@ -1320,11 +1330,13 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
public SettableFuture<ConfigTaskResult> showPathSetTemplate(
ShowPathSetTemplateStatement showPathSetTemplateStatement) {
SettableFuture<ConfigTaskResult> future = SettableFuture.create();
- String templateName = showPathSetTemplateStatement.getTemplateName();
try {
// Send request to some API server
List<PartialPath> listPath =
- ClusterTemplateManager.getInstance().getPathsSetTemplate(templateName);
+ ClusterTemplateManager.getInstance()
+ .getPathsSetTemplate(
+ showPathSetTemplateStatement.getTemplateName(),
+ showPathSetTemplateStatement.getAuthorityScope());
// Build TSBlock
ShowPathSetTemplateTask.buildTSBlock(listPath, future);
} catch (Exception e) {
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 54f2ce71dbc..fd3d1bc8f9e 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
@@ -1106,7 +1106,8 @@ public class LogicalPlanBuilder {
long offset,
boolean orderByHeat,
boolean prefixPath,
- Map<Integer, Template> templateMap) {
+ Map<Integer, Template> templateMap,
+ PathPatternTree scope) {
this.root =
new TimeSeriesSchemaScanNode(
context.getQueryId().genPlanNodeId(),
@@ -1116,7 +1117,8 @@ public class LogicalPlanBuilder {
offset,
orderByHeat,
prefixPath,
- templateMap);
+ templateMap,
+ scope);
return this;
}
@@ -1126,7 +1128,8 @@ public class LogicalPlanBuilder {
long offset,
boolean prefixPath,
boolean hasSgCol,
- SchemaFilter schemaFilter) {
+ SchemaFilter schemaFilter,
+ PathPatternTree scope) {
this.root =
new DevicesSchemaScanNode(
context.getQueryId().genPlanNodeId(),
@@ -1135,7 +1138,8 @@ public class LogicalPlanBuilder {
offset,
prefixPath,
hasSgCol,
- schemaFilter);
+ schemaFilter,
+ scope);
return this;
}
@@ -1201,8 +1205,10 @@ public class LogicalPlanBuilder {
return this;
}
- public LogicalPlanBuilder planDevicesCountSource(PartialPath partialPath, boolean prefixPath) {
- this.root = new DevicesCountNode(context.getQueryId().genPlanNodeId(), partialPath, prefixPath);
+ public LogicalPlanBuilder planDevicesCountSource(
+ PartialPath partialPath, boolean prefixPath, PathPatternTree scope) {
+ this.root =
+ new DevicesCountNode(context.getQueryId().genPlanNodeId(), partialPath, prefixPath, scope);
return this;
}
@@ -1210,14 +1216,16 @@ public class LogicalPlanBuilder {
PartialPath partialPath,
boolean prefixPath,
SchemaFilter schemaFilter,
- Map<Integer, Template> templateMap) {
+ Map<Integer, Template> templateMap,
+ PathPatternTree scope) {
this.root =
new TimeSeriesCountNode(
context.getQueryId().genPlanNodeId(),
partialPath,
prefixPath,
schemaFilter,
- templateMap);
+ templateMap,
+ scope);
return this;
}
@@ -1226,7 +1234,8 @@ public class LogicalPlanBuilder {
boolean prefixPath,
int level,
SchemaFilter schemaFilter,
- Map<Integer, Template> templateMap) {
+ Map<Integer, Template> templateMap,
+ PathPatternTree scope) {
this.root =
new LevelTimeSeriesCountNode(
context.getQueryId().genPlanNodeId(),
@@ -1234,13 +1243,16 @@ public class LogicalPlanBuilder {
prefixPath,
level,
schemaFilter,
- templateMap);
+ templateMap,
+ scope);
return this;
}
- public LogicalPlanBuilder planNodePathsSchemaSource(PartialPath partialPath, Integer level) {
+ public LogicalPlanBuilder planNodePathsSchemaSource(
+ PartialPath partialPath, Integer level, PathPatternTree scope) {
this.root =
- new NodePathsSchemaScanNode(context.getQueryId().genPlanNodeId(), partialPath, level);
+ new NodePathsSchemaScanNode(
+ context.getQueryId().genPlanNodeId(), partialPath, level, scope);
return this;
}
@@ -1269,18 +1281,22 @@ public class LogicalPlanBuilder {
}
public LogicalPlanBuilder planPathsUsingTemplateSource(
- List<PartialPath> pathPatternList, int templateId) {
+ List<PartialPath> pathPatternList, int templateId, PathPatternTree scope) {
this.root =
new PathsUsingTemplateScanNode(
- context.getQueryId().genPlanNodeId(), pathPatternList, templateId);
+ context.getQueryId().genPlanNodeId(), pathPatternList, templateId, scope);
return this;
}
public LogicalPlanBuilder planLogicalViewSchemaSource(
- PartialPath pathPattern, SchemaFilter schemaFilter, long limit, long offset) {
+ PartialPath pathPattern,
+ SchemaFilter schemaFilter,
+ long limit,
+ long offset,
+ PathPatternTree scope) {
this.root =
new LogicalViewSchemaScanNode(
- context.getQueryId().genPlanNodeId(), pathPattern, schemaFilter, limit, offset);
+ context.getQueryId().genPlanNodeId(), pathPattern, schemaFilter, limit, offset, scope);
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 d5180aacf46..ad2ffbdd500 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
@@ -591,7 +591,8 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
offset,
showTimeSeriesStatement.isOrderByHeat(),
showTimeSeriesStatement.isPrefixPath(),
- analysis.getRelatedTemplateInfo())
+ analysis.getRelatedTemplateInfo(),
+ showTimeSeriesStatement.getAuthorityScope())
.planSchemaQueryMerge(showTimeSeriesStatement.isOrderByHeat());
// show latest timeseries
if (showTimeSeriesStatement.isOrderByHeat()
@@ -640,7 +641,8 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
offset,
showDevicesStatement.isPrefixPath(),
showDevicesStatement.hasSgCol(),
- showDevicesStatement.getSchemaFilter())
+ showDevicesStatement.getSchemaFilter(),
+ showDevicesStatement.getAuthorityScope())
.planSchemaQueryMerge(false);
if (!canPushDownOffsetLimit) {
@@ -658,7 +660,9 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(analysis, context);
return planBuilder
.planDevicesCountSource(
- countDevicesStatement.getPathPattern(), countDevicesStatement.isPrefixPath())
+ countDevicesStatement.getPathPattern(),
+ countDevicesStatement.isPrefixPath(),
+ countDevicesStatement.getAuthorityScope())
.planCountMerge()
.getRoot();
}
@@ -672,7 +676,8 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
countTimeSeriesStatement.getPathPattern(),
countTimeSeriesStatement.isPrefixPath(),
countTimeSeriesStatement.getSchemaFilter(),
- analysis.getRelatedTemplateInfo())
+ analysis.getRelatedTemplateInfo(),
+ countTimeSeriesStatement.getAuthorityScope())
.planCountMerge()
.getRoot();
}
@@ -687,7 +692,8 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
countLevelTimeSeriesStatement.isPrefixPath(),
countLevelTimeSeriesStatement.getLevel(),
countLevelTimeSeriesStatement.getSchemaFilter(),
- analysis.getRelatedTemplateInfo())
+ analysis.getRelatedTemplateInfo(),
+ countLevelTimeSeriesStatement.getAuthorityScope())
.planCountMerge()
.getRoot();
}
@@ -696,7 +702,10 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
public PlanNode visitCountNodes(CountNodesStatement countStatement, MPPQueryContext context) {
LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(analysis, context);
return planBuilder
- .planNodePathsSchemaSource(countStatement.getPathPattern(), countStatement.getLevel())
+ .planNodePathsSchemaSource(
+ countStatement.getPathPattern(),
+ countStatement.getLevel(),
+ countStatement.getAuthorityScope())
.planSchemaQueryMerge(false)
.planNodeManagementMemoryMerge(analysis.getMatchedNodes())
.planNodePathsCount()
@@ -810,7 +819,10 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
ShowChildPathsStatement showChildPathsStatement, MPPQueryContext context) {
LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(analysis, context);
return planBuilder
- .planNodePathsSchemaSource(showChildPathsStatement.getPartialPath(), -1)
+ .planNodePathsSchemaSource(
+ showChildPathsStatement.getPartialPath(),
+ -1,
+ showChildPathsStatement.getAuthorityScope())
.planSchemaQueryMerge(false)
.planNodeManagementMemoryMerge(analysis.getMatchedNodes())
.getRoot();
@@ -821,7 +833,10 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
ShowChildNodesStatement showChildNodesStatement, MPPQueryContext context) {
LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(analysis, context);
return planBuilder
- .planNodePathsSchemaSource(showChildNodesStatement.getPartialPath(), -1)
+ .planNodePathsSchemaSource(
+ showChildNodesStatement.getPartialPath(),
+ -1,
+ showChildNodesStatement.getAuthorityScope())
.planSchemaQueryMerge(false)
.planNodeManagementMemoryMerge(analysis.getMatchedNodes())
.planNodePathsConvert()
@@ -885,7 +900,8 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
planBuilder
.planPathsUsingTemplateSource(
analysis.getSpecifiedTemplateRelatedPathPatternList(),
- analysis.getTemplateSetInfo().left.getId())
+ analysis.getTemplateSetInfo().left.getId(),
+ showPathsUsingTemplateStatement.getAuthorityScope())
.planSchemaQueryMerge(false);
return planBuilder.getRoot();
}
@@ -947,7 +963,8 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
showLogicalViewStatement.getPathPattern(),
showLogicalViewStatement.getSchemaFilter(),
limit,
- offset)
+ offset,
+ showLogicalViewStatement.getAuthorityScope())
.planSchemaQueryMerge(false);
if (canPushDownOffsetLimit) {
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 3585b301e10..d216be93417 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
@@ -576,7 +576,8 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
node.getLimit(),
node.getOffset(),
node.getSchemaFilter(),
- node.getTemplateMap()));
+ node.getTemplateMap(),
+ node.getScope()));
}
@Override
@@ -599,7 +600,8 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
node.getLimit(),
node.getOffset(),
node.isHasSgCol(),
- node.getSchemaFilter()));
+ node.getSchemaFilter(),
+ node.getScope()));
}
@Override
@@ -648,7 +650,8 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
return new SchemaCountOperator<>(
node.getPlanNodeId(),
operatorContext,
- SchemaSourceFactory.getDeviceSchemaSource(node.getPath(), node.isPrefixPath()));
+ SchemaSourceFactory.getDeviceSchemaSource(
+ node.getPath(), node.isPrefixPath(), node.getScope()));
}
@Override
@@ -666,7 +669,11 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
node.getPlanNodeId(),
operatorContext,
SchemaSourceFactory.getTimeSeriesSchemaCountSource(
- node.getPath(), node.isPrefixPath(), node.getSchemaFilter(), node.getTemplateMap()));
+ node.getPath(),
+ node.isPrefixPath(),
+ node.getSchemaFilter(),
+ node.getTemplateMap(),
+ node.getScope()));
}
@Override
@@ -685,7 +692,11 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
operatorContext,
node.getLevel(),
SchemaSourceFactory.getTimeSeriesSchemaCountSource(
- node.getPath(), node.isPrefixPath(), node.getSchemaFilter(), node.getTemplateMap()));
+ node.getPath(),
+ node.isPrefixPath(),
+ node.getSchemaFilter(),
+ node.getTemplateMap(),
+ node.getScope()));
}
@Override
@@ -702,7 +713,8 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
return new SchemaQueryScanOperator<>(
node.getPlanNodeId(),
operatorContext,
- SchemaSourceFactory.getNodeSchemaSource(node.getPrefixPath(), node.getLevel()));
+ SchemaSourceFactory.getNodeSchemaSource(
+ node.getPrefixPath(), node.getLevel(), node.getScope()));
}
@Override
@@ -2446,7 +2458,7 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
node.getPlanNodeId(),
operatorContext,
SchemaSourceFactory.getPathsUsingTemplateSource(
- node.getPathPatternList(), node.getTemplateId()));
+ node.getPathPatternList(), node.getTemplateId(), node.getScope()));
}
public Operator visitLogicalViewSchemaScan(
@@ -2463,7 +2475,11 @@ public class OperatorTreeGenerator extends PlanVisitor<Operator, LocalExecutionP
node.getPlanNodeId(),
operatorContext,
SchemaSourceFactory.getLogicalViewSchemaSource(
- node.getPath(), node.getLimit(), node.getOffset(), node.getSchemaFilter()));
+ node.getPath(),
+ node.getLimit(),
+ node.getOffset(),
+ node.getSchemaFilter(),
+ node.getScope()));
}
public List<Operator> dealWithConsumeAllChildrenPipelineBreaker(
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/DevicesCountNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/DevicesCountNode.java
index 415aafcb78e..1e7ec8c323a 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/DevicesCountNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/DevicesCountNode.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode;
@@ -36,13 +37,14 @@ import java.util.stream.Collectors;
public class DevicesCountNode extends SchemaQueryScanNode {
- public DevicesCountNode(PlanNodeId id, PartialPath partialPath, boolean isPrefixPath) {
- super(id, partialPath, isPrefixPath);
+ public DevicesCountNode(
+ PlanNodeId id, PartialPath partialPath, boolean isPrefixPath, PathPatternTree scope) {
+ super(id, partialPath, isPrefixPath, scope);
}
@Override
public PlanNode clone() {
- return new DevicesCountNode(getPlanNodeId(), path, isPrefixPath);
+ return new DevicesCountNode(getPlanNodeId(), path, isPrefixPath, scope);
}
@Override
@@ -56,6 +58,7 @@ public class DevicesCountNode extends SchemaQueryScanNode {
protected void serializeAttributes(ByteBuffer byteBuffer) {
PlanNodeType.DEVICES_COUNT.serialize(byteBuffer);
ReadWriteIOUtils.write(path.getFullPath(), byteBuffer);
+ scope.serialize(byteBuffer);
ReadWriteIOUtils.write(isPrefixPath, byteBuffer);
}
@@ -63,6 +66,7 @@ public class DevicesCountNode extends SchemaQueryScanNode {
protected void serializeAttributes(DataOutputStream stream) throws IOException {
PlanNodeType.DEVICES_COUNT.serialize(stream);
ReadWriteIOUtils.write(path.getFullPath(), stream);
+ scope.serialize(stream);
ReadWriteIOUtils.write(isPrefixPath, stream);
}
@@ -74,9 +78,10 @@ public class DevicesCountNode extends SchemaQueryScanNode {
} catch (IllegalPathException e) {
throw new IllegalArgumentException("Cannot deserialize DevicesSchemaScanNode", e);
}
+ PathPatternTree scope = PathPatternTree.deserialize(buffer);
boolean isPrefixPath = ReadWriteIOUtils.readBool(buffer);
PlanNodeId planNodeId = PlanNodeId.deserialize(buffer);
- return new DevicesCountNode(planNodeId, path, isPrefixPath);
+ return new DevicesCountNode(planNodeId, path, isPrefixPath, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/DevicesSchemaScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/DevicesSchemaScanNode.java
index f1af2c44843..5a0fd91871d 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/DevicesSchemaScanNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/DevicesSchemaScanNode.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
@@ -47,8 +48,9 @@ public class DevicesSchemaScanNode extends SchemaQueryScanNode {
long offset,
boolean isPrefixPath,
boolean hasSgCol,
- SchemaFilter schemaFilter) {
- super(id, path, limit, offset, isPrefixPath);
+ SchemaFilter schemaFilter,
+ PathPatternTree scope) {
+ super(id, path, limit, offset, isPrefixPath, scope);
this.hasSgCol = hasSgCol;
this.schemaFilter = schemaFilter;
}
@@ -64,7 +66,7 @@ public class DevicesSchemaScanNode extends SchemaQueryScanNode {
@Override
public PlanNode clone() {
return new DevicesSchemaScanNode(
- getPlanNodeId(), path, limit, offset, isPrefixPath, hasSgCol, schemaFilter);
+ getPlanNodeId(), path, limit, offset, isPrefixPath, hasSgCol, schemaFilter, scope);
}
@Override
@@ -83,6 +85,7 @@ public class DevicesSchemaScanNode extends SchemaQueryScanNode {
protected void serializeAttributes(ByteBuffer byteBuffer) {
PlanNodeType.DEVICES_SCHEMA_SCAN.serialize(byteBuffer);
ReadWriteIOUtils.write(path.getFullPath(), byteBuffer);
+ scope.serialize(byteBuffer);
ReadWriteIOUtils.write(limit, byteBuffer);
ReadWriteIOUtils.write(offset, byteBuffer);
ReadWriteIOUtils.write(isPrefixPath, byteBuffer);
@@ -94,6 +97,7 @@ public class DevicesSchemaScanNode extends SchemaQueryScanNode {
protected void serializeAttributes(DataOutputStream stream) throws IOException {
PlanNodeType.DEVICES_SCHEMA_SCAN.serialize(stream);
ReadWriteIOUtils.write(path.getFullPath(), stream);
+ scope.serialize(stream);
ReadWriteIOUtils.write(limit, stream);
ReadWriteIOUtils.write(offset, stream);
ReadWriteIOUtils.write(isPrefixPath, stream);
@@ -109,6 +113,7 @@ public class DevicesSchemaScanNode extends SchemaQueryScanNode {
} catch (IllegalPathException e) {
throw new IllegalArgumentException("Cannot deserialize DevicesSchemaScanNode", e);
}
+ PathPatternTree scope = PathPatternTree.deserialize(byteBuffer);
long limit = ReadWriteIOUtils.readLong(byteBuffer);
long offset = ReadWriteIOUtils.readLong(byteBuffer);
boolean isPrefixPath = ReadWriteIOUtils.readBool(byteBuffer);
@@ -116,7 +121,7 @@ public class DevicesSchemaScanNode extends SchemaQueryScanNode {
SchemaFilter schemaFilter = SchemaFilter.deserialize(byteBuffer);
PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
return new DevicesSchemaScanNode(
- planNodeId, path, limit, offset, isPrefixPath, hasSgCol, schemaFilter);
+ planNodeId, path, limit, offset, isPrefixPath, hasSgCol, schemaFilter, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LevelTimeSeriesCountNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LevelTimeSeriesCountNode.java
index 9bf5962973a..c4089f60605 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LevelTimeSeriesCountNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LevelTimeSeriesCountNode.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
@@ -52,8 +53,9 @@ public class LevelTimeSeriesCountNode extends SchemaQueryScanNode {
boolean isPrefixPath,
int level,
SchemaFilter schemaFilter,
- @NotNull Map<Integer, Template> templateMap) {
- super(id, partialPath, isPrefixPath);
+ @NotNull Map<Integer, Template> templateMap,
+ @NotNull PathPatternTree scope) {
+ super(id, partialPath, isPrefixPath, scope);
this.level = level;
this.schemaFilter = schemaFilter;
this.templateMap = templateMap;
@@ -74,7 +76,7 @@ public class LevelTimeSeriesCountNode extends SchemaQueryScanNode {
@Override
public PlanNode clone() {
return new LevelTimeSeriesCountNode(
- getPlanNodeId(), path, isPrefixPath, level, schemaFilter, templateMap);
+ getPlanNodeId(), path, isPrefixPath, level, schemaFilter, templateMap, scope);
}
@Override
@@ -88,6 +90,7 @@ public class LevelTimeSeriesCountNode extends SchemaQueryScanNode {
protected void serializeAttributes(ByteBuffer byteBuffer) {
PlanNodeType.LEVEL_TIME_SERIES_COUNT.serialize(byteBuffer);
ReadWriteIOUtils.write(path.getFullPath(), byteBuffer);
+ scope.serialize(byteBuffer);
ReadWriteIOUtils.write(isPrefixPath, byteBuffer);
ReadWriteIOUtils.write(level, byteBuffer);
SchemaFilter.serialize(schemaFilter, byteBuffer);
@@ -101,6 +104,7 @@ public class LevelTimeSeriesCountNode extends SchemaQueryScanNode {
protected void serializeAttributes(DataOutputStream stream) throws IOException {
PlanNodeType.LEVEL_TIME_SERIES_COUNT.serialize(stream);
ReadWriteIOUtils.write(path.getFullPath(), stream);
+ scope.serialize(stream);
ReadWriteIOUtils.write(isPrefixPath, stream);
ReadWriteIOUtils.write(level, stream);
SchemaFilter.serialize(schemaFilter, stream);
@@ -118,6 +122,7 @@ public class LevelTimeSeriesCountNode extends SchemaQueryScanNode {
} catch (IllegalPathException e) {
throw new IllegalArgumentException("Cannot deserialize DevicesSchemaScanNode", e);
}
+ PathPatternTree scope = PathPatternTree.deserialize(buffer);
boolean isPrefixPath = ReadWriteIOUtils.readBool(buffer);
int level = ReadWriteIOUtils.readInt(buffer);
SchemaFilter schemaFilter = SchemaFilter.deserialize(buffer);
@@ -131,7 +136,7 @@ public class LevelTimeSeriesCountNode extends SchemaQueryScanNode {
}
PlanNodeId planNodeId = PlanNodeId.deserialize(buffer);
return new LevelTimeSeriesCountNode(
- planNodeId, path, isPrefixPath, level, schemaFilter, templateMap);
+ planNodeId, path, isPrefixPath, level, schemaFilter, templateMap, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LogicalViewSchemaScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LogicalViewSchemaScanNode.java
index 8bf7042526e..9f7efebcc54 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LogicalViewSchemaScanNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LogicalViewSchemaScanNode.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
@@ -41,8 +42,13 @@ public class LogicalViewSchemaScanNode extends SchemaQueryScanNode {
private final SchemaFilter schemaFilter;
public LogicalViewSchemaScanNode(
- PlanNodeId id, PartialPath partialPath, SchemaFilter schemaFilter, long limit, long offset) {
- super(id, partialPath, limit, offset, false);
+ PlanNodeId id,
+ PartialPath partialPath,
+ SchemaFilter schemaFilter,
+ long limit,
+ long offset,
+ PathPatternTree scope) {
+ super(id, partialPath, limit, offset, false, scope);
this.schemaFilter = schemaFilter;
}
@@ -54,6 +60,7 @@ public class LogicalViewSchemaScanNode extends SchemaQueryScanNode {
protected void serializeAttributes(ByteBuffer byteBuffer) {
PlanNodeType.LOGICAL_VIEW_SCHEMA_SCAN.serialize(byteBuffer);
ReadWriteIOUtils.write(path.getFullPath(), byteBuffer);
+ scope.serialize(byteBuffer);
SchemaFilter.serialize(schemaFilter, byteBuffer);
ReadWriteIOUtils.write(limit, byteBuffer);
ReadWriteIOUtils.write(offset, byteBuffer);
@@ -63,6 +70,7 @@ public class LogicalViewSchemaScanNode extends SchemaQueryScanNode {
protected void serializeAttributes(DataOutputStream stream) throws IOException {
PlanNodeType.LOGICAL_VIEW_SCHEMA_SCAN.serialize(stream);
ReadWriteIOUtils.write(path.getFullPath(), stream);
+ scope.serialize(stream);
SchemaFilter.serialize(schemaFilter, stream);
ReadWriteIOUtils.write(limit, stream);
ReadWriteIOUtils.write(offset, stream);
@@ -76,18 +84,19 @@ public class LogicalViewSchemaScanNode extends SchemaQueryScanNode {
} catch (IllegalPathException e) {
throw new IllegalArgumentException("Cannot deserialize TimeSeriesSchemaScanNode", e);
}
+ PathPatternTree scope = PathPatternTree.deserialize(byteBuffer);
SchemaFilter schemaFilter = SchemaFilter.deserialize(byteBuffer);
long limit = ReadWriteIOUtils.readLong(byteBuffer);
long offset = ReadWriteIOUtils.readLong(byteBuffer);
PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
- return new LogicalViewSchemaScanNode(planNodeId, path, schemaFilter, limit, offset);
+ return new LogicalViewSchemaScanNode(planNodeId, path, schemaFilter, limit, offset, scope);
}
@Override
public PlanNode clone() {
- return new LogicalViewSchemaScanNode(getPlanNodeId(), path, schemaFilter, limit, offset);
+ return new LogicalViewSchemaScanNode(getPlanNodeId(), path, schemaFilter, limit, offset, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/NodePathsSchemaScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/NodePathsSchemaScanNode.java
index 2698f8a3c9d..2fd68cb7bd5 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/NodePathsSchemaScanNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/NodePathsSchemaScanNode.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PathDeserializeUtil;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode;
@@ -39,8 +40,10 @@ public class NodePathsSchemaScanNode extends SchemaQueryScanNode {
private PartialPath prefixPath;
private int level = -1;
- public NodePathsSchemaScanNode(PlanNodeId id, PartialPath prefixPath, int level) {
+ public NodePathsSchemaScanNode(
+ PlanNodeId id, PartialPath prefixPath, int level, PathPatternTree scope) {
super(id);
+ setScope(scope);
this.prefixPath = prefixPath;
this.level = level;
}
@@ -55,7 +58,7 @@ public class NodePathsSchemaScanNode extends SchemaQueryScanNode {
@Override
public PlanNode clone() {
- return new NodePathsSchemaScanNode(getPlanNodeId(), prefixPath, level);
+ return new NodePathsSchemaScanNode(getPlanNodeId(), prefixPath, level, scope);
}
@Override
@@ -69,6 +72,7 @@ public class NodePathsSchemaScanNode extends SchemaQueryScanNode {
protected void serializeAttributes(ByteBuffer byteBuffer) {
PlanNodeType.NODE_PATHS_SCAN.serialize(byteBuffer);
prefixPath.serialize(byteBuffer);
+ scope.serialize(byteBuffer);
byteBuffer.putInt(level);
}
@@ -76,14 +80,16 @@ public class NodePathsSchemaScanNode extends SchemaQueryScanNode {
protected void serializeAttributes(DataOutputStream stream) throws IOException {
PlanNodeType.NODE_PATHS_SCAN.serialize(stream);
prefixPath.serialize(stream);
+ scope.serialize(stream);
stream.writeInt(level);
}
public static PlanNode deserialize(ByteBuffer buffer) {
PartialPath path = (PartialPath) PathDeserializeUtil.deserialize(buffer);
+ PathPatternTree scope = PathPatternTree.deserialize(buffer);
int level = buffer.getInt();
PlanNodeId planNodeId = PlanNodeId.deserialize(buffer);
- return new NodePathsSchemaScanNode(planNodeId, path, level);
+ return new NodePathsSchemaScanNode(planNodeId, path, level, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/PathsUsingTemplateScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/PathsUsingTemplateScanNode.java
index 3020a954714..65bbceabcce 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/PathsUsingTemplateScanNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/PathsUsingTemplateScanNode.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PathDeserializeUtil;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode;
@@ -43,8 +44,9 @@ public class PathsUsingTemplateScanNode extends SchemaQueryScanNode {
private final int templateId;
public PathsUsingTemplateScanNode(
- PlanNodeId id, List<PartialPath> pathPatternList, int templateId) {
+ PlanNodeId id, List<PartialPath> pathPatternList, int templateId, PathPatternTree scope) {
super(id);
+ setScope(scope);
this.pathPatternList = pathPatternList;
this.templateId = templateId;
}
@@ -65,7 +67,7 @@ public class PathsUsingTemplateScanNode extends SchemaQueryScanNode {
@Override
public PlanNode clone() {
- return new PathsUsingTemplateScanNode(getPlanNodeId(), pathPatternList, templateId);
+ return new PathsUsingTemplateScanNode(getPlanNodeId(), pathPatternList, templateId, scope);
}
@Override
@@ -82,6 +84,7 @@ public class PathsUsingTemplateScanNode extends SchemaQueryScanNode {
for (PartialPath pathPattern : pathPatternList) {
pathPattern.serialize(byteBuffer);
}
+ scope.serialize(byteBuffer);
ReadWriteIOUtils.write(templateId, byteBuffer);
}
@@ -92,6 +95,7 @@ public class PathsUsingTemplateScanNode extends SchemaQueryScanNode {
for (PartialPath pathPattern : pathPatternList) {
pathPattern.serialize(stream);
}
+ scope.serialize(stream);
ReadWriteIOUtils.write(templateId, stream);
}
@@ -101,9 +105,10 @@ public class PathsUsingTemplateScanNode extends SchemaQueryScanNode {
for (int i = 0; i < size; i++) {
pathPatternList.add((PartialPath) PathDeserializeUtil.deserialize(buffer));
}
+ PathPatternTree scope = PathPatternTree.deserialize(buffer);
int templateId = ReadWriteIOUtils.readInt(buffer);
PlanNodeId planNodeId = PlanNodeId.deserialize(buffer);
- return new PathsUsingTemplateScanNode(planNodeId, pathPatternList, templateId);
+ return new PathsUsingTemplateScanNode(planNodeId, pathPatternList, templateId, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/SchemaQueryScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/SchemaQueryScanNode.java
index 71b658ec3cc..0e913505a83 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/SchemaQueryScanNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/SchemaQueryScanNode.java
@@ -21,6 +21,8 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read;
import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
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.PlanNodeUtil;
@@ -35,26 +37,34 @@ public abstract class SchemaQueryScanNode extends SourceNode {
protected long limit;
protected long offset;
protected PartialPath path;
+ protected PathPatternTree scope;
private boolean hasLimit;
protected boolean isPrefixPath;
private TRegionReplicaSet schemaRegionReplicaSet;
protected SchemaQueryScanNode(PlanNodeId id) {
- this(id, null, false);
+ this(id, null, false, SchemaConstant.ALL_MATCH_SCOPE);
}
protected SchemaQueryScanNode(
- PlanNodeId id, PartialPath partialPath, long limit, long offset, boolean isPrefixPath) {
+ PlanNodeId id,
+ PartialPath partialPath,
+ long limit,
+ long offset,
+ boolean isPrefixPath,
+ PathPatternTree scope) {
super(id);
this.path = partialPath;
+ this.scope = scope;
setLimit(limit);
this.offset = offset;
this.isPrefixPath = isPrefixPath;
}
- protected SchemaQueryScanNode(PlanNodeId id, PartialPath partialPath, boolean isPrefixPath) {
- this(id, partialPath, 0, 0, isPrefixPath);
+ protected SchemaQueryScanNode(
+ PlanNodeId id, PartialPath partialPath, boolean isPrefixPath, PathPatternTree scope) {
+ this(id, partialPath, 0, 0, isPrefixPath, scope);
}
@Override
@@ -129,6 +139,14 @@ public abstract class SchemaQueryScanNode extends SourceNode {
this.path = path;
}
+ public PathPatternTree getScope() {
+ return scope;
+ }
+
+ public void setScope(PathPatternTree scope) {
+ this.scope = scope;
+ }
+
public boolean isHasLimit() {
return hasLimit;
}
@@ -151,24 +169,19 @@ public abstract class SchemaQueryScanNode extends SourceNode {
@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;
- }
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ if (!super.equals(o)) return false;
SchemaQueryScanNode that = (SchemaQueryScanNode) o;
return limit == that.limit
&& offset == that.offset
&& isPrefixPath == that.isPrefixPath
- && Objects.equals(path, that.path);
+ && Objects.equals(path, that.path)
+ && Objects.equals(scope, that.scope);
}
@Override
public int hashCode() {
- return Objects.hash(super.hashCode(), limit, offset, path, isPrefixPath);
+ return Objects.hash(super.hashCode(), limit, offset, path, scope, isPrefixPath);
}
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesCountNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesCountNode.java
index 080ada4c2cb..ccbe3246ba2 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesCountNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesCountNode.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
@@ -52,8 +53,9 @@ public class TimeSeriesCountNode extends SchemaQueryScanNode {
PartialPath partialPath,
boolean isPrefixPath,
SchemaFilter schemaFilter,
- @NotNull Map<Integer, Template> templateMap) {
- super(id, partialPath, isPrefixPath);
+ @NotNull Map<Integer, Template> templateMap,
+ @NotNull PathPatternTree scope) {
+ super(id, partialPath, isPrefixPath, scope);
this.schemaFilter = schemaFilter;
this.templateMap = templateMap;
}
@@ -68,7 +70,8 @@ public class TimeSeriesCountNode extends SchemaQueryScanNode {
@Override
public PlanNode clone() {
- return new TimeSeriesCountNode(getPlanNodeId(), path, isPrefixPath, schemaFilter, templateMap);
+ return new TimeSeriesCountNode(
+ getPlanNodeId(), path, isPrefixPath, schemaFilter, templateMap, scope);
}
@Override
@@ -82,6 +85,7 @@ public class TimeSeriesCountNode extends SchemaQueryScanNode {
protected void serializeAttributes(ByteBuffer byteBuffer) {
PlanNodeType.TIME_SERIES_COUNT.serialize(byteBuffer);
ReadWriteIOUtils.write(path.getFullPath(), byteBuffer);
+ scope.serialize(byteBuffer);
ReadWriteIOUtils.write(isPrefixPath, byteBuffer);
SchemaFilter.serialize(schemaFilter, byteBuffer);
ReadWriteIOUtils.write(templateMap.size(), byteBuffer);
@@ -94,6 +98,7 @@ public class TimeSeriesCountNode extends SchemaQueryScanNode {
protected void serializeAttributes(DataOutputStream stream) throws IOException {
PlanNodeType.TIME_SERIES_COUNT.serialize(stream);
ReadWriteIOUtils.write(path.getFullPath(), stream);
+ scope.serialize(stream);
ReadWriteIOUtils.write(isPrefixPath, stream);
SchemaFilter.serialize(schemaFilter, stream);
ReadWriteIOUtils.write(templateMap.size(), stream);
@@ -110,6 +115,7 @@ public class TimeSeriesCountNode extends SchemaQueryScanNode {
} catch (IllegalPathException e) {
throw new IllegalArgumentException("Cannot deserialize DevicesSchemaScanNode", e);
}
+ PathPatternTree scope = PathPatternTree.deserialize(buffer);
boolean isPrefixPath = ReadWriteIOUtils.readBool(buffer);
SchemaFilter schemaFilter = SchemaFilter.deserialize(buffer);
@@ -123,7 +129,8 @@ public class TimeSeriesCountNode extends SchemaQueryScanNode {
}
PlanNodeId planNodeId = PlanNodeId.deserialize(buffer);
- return new TimeSeriesCountNode(planNodeId, path, isPrefixPath, schemaFilter, templateMap);
+ return new TimeSeriesCountNode(
+ planNodeId, path, isPrefixPath, schemaFilter, templateMap, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesSchemaScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesSchemaScanNode.java
index ccd8c8d1ccc..f6ce20e9822 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesSchemaScanNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesSchemaScanNode.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.read;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
@@ -57,8 +58,9 @@ public class TimeSeriesSchemaScanNode extends SchemaQueryScanNode {
long offset,
boolean orderByHeat,
boolean isPrefixPath,
- @NotNull Map<Integer, Template> templateMap) {
- super(id, partialPath, limit, offset, isPrefixPath);
+ @NotNull Map<Integer, Template> templateMap,
+ @NotNull PathPatternTree scope) {
+ super(id, partialPath, limit, offset, isPrefixPath, scope);
this.schemaFilter = schemaFilter;
this.orderByHeat = orderByHeat;
this.templateMap = templateMap;
@@ -72,6 +74,7 @@ public class TimeSeriesSchemaScanNode extends SchemaQueryScanNode {
protected void serializeAttributes(ByteBuffer byteBuffer) {
PlanNodeType.TIME_SERIES_SCHEMA_SCAN.serialize(byteBuffer);
ReadWriteIOUtils.write(path.getFullPath(), byteBuffer);
+ scope.serialize(byteBuffer);
SchemaFilter.serialize(schemaFilter, byteBuffer);
ReadWriteIOUtils.write(limit, byteBuffer);
ReadWriteIOUtils.write(offset, byteBuffer);
@@ -88,6 +91,7 @@ public class TimeSeriesSchemaScanNode extends SchemaQueryScanNode {
protected void serializeAttributes(DataOutputStream stream) throws IOException {
PlanNodeType.TIME_SERIES_SCHEMA_SCAN.serialize(stream);
ReadWriteIOUtils.write(path.getFullPath(), stream);
+ scope.serialize(stream);
SchemaFilter.serialize(schemaFilter, stream);
ReadWriteIOUtils.write(limit, stream);
ReadWriteIOUtils.write(offset, stream);
@@ -108,6 +112,7 @@ public class TimeSeriesSchemaScanNode extends SchemaQueryScanNode {
} catch (IllegalPathException e) {
throw new IllegalArgumentException("Cannot deserialize TimeSeriesSchemaScanNode", e);
}
+ PathPatternTree scope = PathPatternTree.deserialize(byteBuffer);
SchemaFilter schemaFilter = SchemaFilter.deserialize(byteBuffer);
long limit = ReadWriteIOUtils.readLong(byteBuffer);
long offset = ReadWriteIOUtils.readLong(byteBuffer);
@@ -126,7 +131,15 @@ public class TimeSeriesSchemaScanNode extends SchemaQueryScanNode {
PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer);
return new TimeSeriesSchemaScanNode(
- planNodeId, path, schemaFilter, limit, offset, oderByHeat, isPrefixPath, templateMap);
+ planNodeId,
+ path,
+ schemaFilter,
+ limit,
+ offset,
+ oderByHeat,
+ isPrefixPath,
+ templateMap,
+ scope);
}
public boolean isOrderByHeat() {
@@ -140,7 +153,15 @@ public class TimeSeriesSchemaScanNode extends SchemaQueryScanNode {
@Override
public PlanNode clone() {
return new TimeSeriesSchemaScanNode(
- getPlanNodeId(), path, schemaFilter, limit, offset, orderByHeat, isPrefixPath, templateMap);
+ getPlanNodeId(),
+ path,
+ schemaFilter,
+ limit,
+ offset,
+ orderByHeat,
+ isPrefixPath,
+ templateMap,
+ scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
index 4ff683f4436..10c8d7f5278 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDatabaseStatement.java
@@ -20,16 +20,28 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.plan.analyze.QueryType;
import org.apache.iotdb.db.queryengine.plan.statement.IConfigStatement;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
public class CountDatabaseStatement extends CountStatement implements IConfigStatement {
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
+
public CountDatabaseStatement(PartialPath partialPath) {
super(partialPath);
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
@Override
public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
return visitor.visitCountStorageGroup(this, context);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDevicesStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDevicesStatement.java
index 84ef6b7bdb9..5aa0fff837f 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDevicesStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountDevicesStatement.java
@@ -20,13 +20,25 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
public class CountDevicesStatement extends CountStatement {
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
+
public CountDevicesStatement(PartialPath partialPath) {
super(partialPath);
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
@Override
public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
return visitor.visitCountDevices(this, context);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountLevelTimeSeriesStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountLevelTimeSeriesStatement.java
index 6cc6fd9fd2e..f822baa5294 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountLevelTimeSeriesStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountLevelTimeSeriesStatement.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
@@ -27,11 +29,21 @@ public class CountLevelTimeSeriesStatement extends CountStatement {
private int level;
private SchemaFilter schemaFilter;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
+
public CountLevelTimeSeriesStatement(PartialPath partialPath, int level) {
super(partialPath);
this.level = level;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
public int getLevel() {
return level;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountNodesStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountNodesStatement.java
index 06c9c7ef75d..0ea68aae09d 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountNodesStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountNodesStatement.java
@@ -20,16 +20,28 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
public class CountNodesStatement extends CountStatement {
private final int level;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
+
public CountNodesStatement(PartialPath partialPath, int level) {
super(partialPath);
this.level = level;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
public int getLevel() {
return level;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountTimeSeriesStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountTimeSeriesStatement.java
index 50f7c49e531..ff7413b6e4f 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountTimeSeriesStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountTimeSeriesStatement.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
@@ -27,10 +29,20 @@ public class CountTimeSeriesStatement extends CountStatement {
private SchemaFilter schemaFilter;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
+
public CountTimeSeriesStatement(PartialPath partialPath) {
super(partialPath);
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
public SchemaFilter getSchemaFilter() {
return schemaFilter;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowChildNodesStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowChildNodesStatement.java
index 56c21a4cf03..7c3b0fa8104 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowChildNodesStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowChildNodesStatement.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
import java.util.Collections;
@@ -27,12 +29,21 @@ import java.util.List;
public class ShowChildNodesStatement extends ShowStatement {
private final PartialPath partialPath;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
public ShowChildNodesStatement(PartialPath partialPath) {
super();
this.partialPath = partialPath;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
public PartialPath getPartialPath() {
return partialPath;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowChildPathsStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowChildPathsStatement.java
index c49a8553c30..23f59717d05 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowChildPathsStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowChildPathsStatement.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
import java.util.Collections;
@@ -27,12 +29,21 @@ import java.util.List;
public class ShowChildPathsStatement extends ShowStatement {
private final PartialPath partialPath;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
public ShowChildPathsStatement(PartialPath partialPath) {
super();
this.partialPath = partialPath;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
@Override
public List<PartialPath> getPaths() {
return Collections.singletonList(partialPath);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java
index 711202a3027..e7a7cbd08d5 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDatabaseStatement.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseInfo;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeader;
import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant;
@@ -51,6 +53,8 @@ import java.util.stream.Collectors;
public class ShowDatabaseStatement extends ShowStatement implements IConfigStatement {
private final PartialPath pathPattern;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
+
private boolean isDetailed;
public ShowDatabaseStatement(PartialPath pathPattern) {
@@ -59,6 +63,14 @@ public class ShowDatabaseStatement extends ShowStatement implements IConfigState
this.isDetailed = false;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
public PartialPath getPathPattern() {
return pathPattern;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDevicesStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDevicesStatement.java
index 1911d1644c1..d03d57235d2 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDevicesStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDevicesStatement.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
@@ -39,11 +41,21 @@ public class ShowDevicesStatement extends ShowStatement {
private boolean hasSgCol;
private SchemaFilter schemaFilter;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
+
public ShowDevicesStatement(PartialPath pathPattern) {
super();
this.pathPattern = pathPattern;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
public SchemaFilter getSchemaFilter() {
return schemaFilter;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTTLStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTTLStatement.java
index 399e8619232..417a45fc7b4 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTTLStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTTLStatement.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.plan.analyze.QueryType;
import org.apache.iotdb.db.queryengine.plan.statement.IConfigStatement;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
@@ -29,12 +31,21 @@ import java.util.List;
public class ShowTTLStatement extends ShowStatement implements IConfigStatement {
private List<PartialPath> pathPatterns = new ArrayList<>();
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
private boolean isAll = false;
public boolean isAll() {
return isAll;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
@Override
public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
return visitor.visitShowTTL(this, context);
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 ca7fddb8764..b09a1e6a45e 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
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor;
@@ -43,6 +45,8 @@ 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 PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
+
public ShowTimeSeriesStatement(PartialPath pathPattern, boolean orderByHeat) {
super();
this.pathPattern = pathPattern;
@@ -57,6 +61,14 @@ public class ShowTimeSeriesStatement extends ShowStatement {
return schemaFilter;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
public void setSchemaFilter(SchemaFilter schemaFilter) {
this.schemaFilter = schemaFilter;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/template/ShowPathSetTemplateStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/template/ShowPathSetTemplateStatement.java
index be4a7ef1af5..6c5826f02c8 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/template/ShowPathSetTemplateStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/template/ShowPathSetTemplateStatement.java
@@ -19,6 +19,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata.template;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.plan.analyze.QueryType;
import org.apache.iotdb.db.queryengine.plan.statement.IConfigStatement;
import org.apache.iotdb.db.queryengine.plan.statement.StatementType;
@@ -28,6 +30,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.metadata.ShowStatement;
public class ShowPathSetTemplateStatement extends ShowStatement implements IConfigStatement {
private String templateName;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
public ShowPathSetTemplateStatement(String templateName) {
super();
@@ -35,6 +38,14 @@ public class ShowPathSetTemplateStatement extends ShowStatement implements IConf
this.templateName = templateName;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
public String getTemplateName() {
return templateName;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java
index 14433ed164d..780a8e0fb95 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/template/ShowPathsUsingTemplateStatement.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata.template;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
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.metadata.ShowStatement;
@@ -31,6 +33,7 @@ public class ShowPathsUsingTemplateStatement extends ShowStatement {
private PartialPath pathPattern;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
private String templateName;
public ShowPathsUsingTemplateStatement(PartialPath pathPattern, String templateName) {
@@ -40,6 +43,14 @@ public class ShowPathsUsingTemplateStatement extends ShowStatement {
this.templateName = templateName;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
@Override
public List<PartialPath> getPaths() {
return Collections.singletonList(pathPattern);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/view/ShowLogicalViewStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/view/ShowLogicalViewStatement.java
index 2eedd33c24c..0ad3aa00195 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/view/ShowLogicalViewStatement.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/view/ShowLogicalViewStatement.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.queryengine.plan.statement.metadata.view;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
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.metadata.ShowStatement;
@@ -30,6 +32,7 @@ import java.util.List;
public class ShowLogicalViewStatement extends ShowStatement {
private final PartialPath pathPattern;
+ private PathPatternTree authorityScope = SchemaConstant.ALL_MATCH_SCOPE;
private SchemaFilter schemaFilter;
@@ -38,6 +41,14 @@ public class ShowLogicalViewStatement extends ShowStatement {
this.pathPattern = pathPattern;
}
+ public PathPatternTree getAuthorityScope() {
+ return authorityScope;
+ }
+
+ public void setAuthorityScope(PathPatternTree authorityScope) {
+ this.authorityScope = authorityScope;
+ }
+
public PartialPath getPathPattern() {
return pathPattern;
}
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 b86f84e1e1f..21b52ccfc51 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
@@ -578,7 +578,8 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
List<PartialPath> result = new ArrayList<>();
try (MeasurementUpdater<IMemMNode> updater =
- new MeasurementUpdater<IMemMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementUpdater<IMemMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateMeasurement(IMeasurementMNode<IMemMNode> node) {
node.setPreDeleted(true);
@@ -594,7 +595,8 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
List<PartialPath> result = new ArrayList<>();
try (MeasurementUpdater<IMemMNode> updater =
- new MeasurementUpdater<IMemMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementUpdater<IMemMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateMeasurement(IMeasurementMNode<IMemMNode> node) {
node.setPreDeleted(false);
@@ -610,7 +612,8 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
List<PartialPath> result = new LinkedList<>();
try (MeasurementCollector<Void, IMemMNode> collector =
- new MeasurementCollector<Void, IMemMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementCollector<Void, IMemMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected Void collectMeasurement(IMeasurementMNode<IMemMNode> node) {
if (node.isPreDeleted()) {
@@ -624,11 +627,13 @@ public class MTreeBelowSGMemoryImpl {
return result;
}
+ // TODO: seems useless
public Set<PartialPath> getDevicesOfPreDeletedTimeseries(PartialPath pathPattern)
throws MetadataException {
Set<PartialPath> result = new HashSet<>();
try (MeasurementCollector<Void, IMemMNode> collector =
- new MeasurementCollector<Void, IMemMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementCollector<Void, IMemMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected Void collectMeasurement(IMeasurementMNode<IMemMNode> node) {
if (node.isPreDeleted()) {
@@ -697,7 +702,8 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
List<MeasurementPath> result = new LinkedList<>();
try (MeasurementCollector<Void, IMemMNode> collector =
- new MeasurementCollector<Void, IMemMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementCollector<Void, IMemMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected Void collectMeasurement(IMeasurementMNode<IMemMNode> node) {
MeasurementPath path = getCurrentMeasurementPathInTraverse(node);
@@ -804,7 +810,8 @@ public class MTreeBelowSGMemoryImpl {
Map<PartialPath, List<Integer>> resultTemplateSetInfo = new HashMap<>();
for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
try (EntityUpdater<?> updater =
- new EntityUpdater<IMemMNode>(rootNode, entry.getKey(), store, false) {
+ new EntityUpdater<IMemMNode>(
+ rootNode, entry.getKey(), store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateEntity(IDeviceMNode<IMemMNode> node) {
if (entry.getValue().contains(node.getSchemaTemplateId())) {
@@ -825,7 +832,8 @@ public class MTreeBelowSGMemoryImpl {
Map<PartialPath, List<Integer>> resultTemplateSetInfo = new HashMap<>();
for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
try (EntityUpdater<IMemMNode> updater =
- new EntityUpdater<IMemMNode>(rootNode, entry.getKey(), store, false) {
+ new EntityUpdater<IMemMNode>(
+ rootNode, entry.getKey(), store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateEntity(IDeviceMNode<IMemMNode> node) {
if (entry.getValue().contains(node.getSchemaTemplateId())
@@ -847,7 +855,8 @@ public class MTreeBelowSGMemoryImpl {
Map<PartialPath, List<Integer>> resultTemplateSetInfo = new HashMap<>();
for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
try (EntityUpdater<?> collector =
- new EntityUpdater<IMemMNode>(rootNode, entry.getKey(), store, false) {
+ new EntityUpdater<IMemMNode>(
+ rootNode, entry.getKey(), store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateEntity(IDeviceMNode<IMemMNode> node) {
if (entry.getValue().contains(node.getSchemaTemplateId())
@@ -895,7 +904,7 @@ public class MTreeBelowSGMemoryImpl {
public long countPathsUsingTemplate(PartialPath pathPattern, int templateId)
throws MetadataException {
try (EntityCounter<IMemMNode> counter =
- new EntityCounter<>(rootNode, pathPattern, store, false)) {
+ new EntityCounter<>(rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE)) {
counter.setSchemaTemplateFilter(templateId);
return counter.count();
}
@@ -910,7 +919,11 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
EntityCollector<IDeviceSchemaInfo, IMemMNode> collector =
new EntityCollector<IDeviceSchemaInfo, IMemMNode>(
- rootNode, showDevicesPlan.getPath(), store, showDevicesPlan.isPrefixMatch()) {
+ rootNode,
+ showDevicesPlan.getPath(),
+ store,
+ showDevicesPlan.isPrefixMatch(),
+ SchemaConstant.ALL_MATCH_SCOPE) {
protected IDeviceSchemaInfo collectEntity(IDeviceMNode<IMemMNode> node) {
PartialPath device = getPartialPathFromRootToNode(node.getAsMNode());
@@ -975,7 +988,11 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
MeasurementCollector<ITimeSeriesSchemaInfo, IMemMNode> collector =
new MeasurementCollector<ITimeSeriesSchemaInfo, IMemMNode>(
- rootNode, showTimeSeriesPlan.getPath(), store, showTimeSeriesPlan.isPrefixMatch()) {
+ rootNode,
+ showTimeSeriesPlan.getPath(),
+ store,
+ showTimeSeriesPlan.isPrefixMatch(),
+ SchemaConstant.ALL_MATCH_SCOPE) {
protected ITimeSeriesSchemaInfo collectMeasurement(IMeasurementMNode<IMemMNode> node) {
return new ITimeSeriesSchemaInfo() {
@@ -1047,7 +1064,11 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
MNodeCollector<INodeSchemaInfo, IMemMNode> collector =
new MNodeCollector<INodeSchemaInfo, IMemMNode>(
- rootNode, showNodesPlan.getPath(), store, showNodesPlan.isPrefixMatch()) {
+ rootNode,
+ showNodesPlan.getPath(),
+ store,
+ showNodesPlan.isPrefixMatch(),
+ SchemaConstant.ALL_MATCH_SCOPE) {
protected INodeSchemaInfo collectMNode(IMemMNode node) {
return new ShowNodesResult(
@@ -1147,7 +1168,8 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
List<PartialPath> result = new ArrayList<>();
try (MeasurementUpdater<IMemMNode> updater =
- new MeasurementUpdater<IMemMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementUpdater<IMemMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateMeasurement(IMeasurementMNode<IMemMNode> node) {
if (node.isLogicalView()) {
@@ -1165,7 +1187,8 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
List<PartialPath> result = new ArrayList<>();
try (MeasurementUpdater<IMemMNode> updater =
- new MeasurementUpdater<IMemMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementUpdater<IMemMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateMeasurement(IMeasurementMNode<IMemMNode> node) {
if (node.isLogicalView()) {
@@ -1183,7 +1206,8 @@ public class MTreeBelowSGMemoryImpl {
throws MetadataException {
List<PartialPath> result = new LinkedList<>();
try (MeasurementCollector<Void, IMemMNode> collector =
- new MeasurementCollector<Void, IMemMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementCollector<Void, IMemMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected Void collectMeasurement(IMeasurementMNode<IMemMNode> node) {
if (node.isLogicalView() && node.isPreDeleted()) {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java
index d484d4e0344..1ae6907fa3d 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java
@@ -143,7 +143,11 @@ public class MTreeBelowSGCachedImpl {
// recover MNode
try (MNodeCollector<Void, ICachedMNode> collector =
new MNodeCollector<Void, ICachedMNode>(
- this.rootNode, new PartialPath(storageGroupMNode.getFullPath()), this.store, true) {
+ this.rootNode,
+ new PartialPath(storageGroupMNode.getFullPath()),
+ this.store,
+ true,
+ SchemaConstant.ALL_MATCH_SCOPE) {
@Override
protected Void collectMNode(ICachedMNode node) {
if (node.isMeasurement()) {
@@ -177,7 +181,11 @@ public class MTreeBelowSGCachedImpl {
// recover MNode
try (MNodeCollector<Void, ICachedMNode> collector =
new MNodeCollector<Void, ICachedMNode>(
- this.rootNode, new PartialPath(storageGroupMNode.getFullPath()), this.store, true) {
+ this.rootNode,
+ new PartialPath(storageGroupMNode.getFullPath()),
+ this.store,
+ true,
+ SchemaConstant.ALL_MATCH_SCOPE) {
@Override
protected Void collectMNode(ICachedMNode node) {
if (node.isMeasurement()) {
@@ -647,7 +655,8 @@ public class MTreeBelowSGCachedImpl {
throws MetadataException {
List<PartialPath> result = new ArrayList<>();
try (MeasurementUpdater<ICachedMNode> updater =
- new MeasurementUpdater<ICachedMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementUpdater<ICachedMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateMeasurement(IMeasurementMNode<ICachedMNode> node)
throws MetadataException {
@@ -665,7 +674,8 @@ public class MTreeBelowSGCachedImpl {
throws MetadataException {
List<PartialPath> result = new ArrayList<>();
try (MeasurementUpdater<ICachedMNode> updater =
- new MeasurementUpdater<ICachedMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementUpdater<ICachedMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateMeasurement(IMeasurementMNode<ICachedMNode> node)
throws MetadataException {
@@ -683,7 +693,8 @@ public class MTreeBelowSGCachedImpl {
throws MetadataException {
List<PartialPath> result = new LinkedList<>();
try (MeasurementCollector<Void, ICachedMNode> collector =
- new MeasurementCollector<Void, ICachedMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementCollector<Void, ICachedMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected Void collectMeasurement(IMeasurementMNode<ICachedMNode> node) {
if (node.isPreDeleted()) {
@@ -701,7 +712,8 @@ public class MTreeBelowSGCachedImpl {
throws MetadataException {
Set<PartialPath> result = new HashSet<>();
try (MeasurementCollector<Void, ICachedMNode> collector =
- new MeasurementCollector<Void, ICachedMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementCollector<Void, ICachedMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected Void collectMeasurement(IMeasurementMNode<ICachedMNode> node) {
if (node.isPreDeleted()) {
@@ -780,7 +792,8 @@ public class MTreeBelowSGCachedImpl {
throws MetadataException {
List<MeasurementPath> result = new LinkedList<>();
try (MeasurementCollector<Void, ICachedMNode> collector =
- new MeasurementCollector<Void, ICachedMNode>(rootNode, pathPattern, store, false) {
+ new MeasurementCollector<Void, ICachedMNode>(
+ rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected Void collectMeasurement(IMeasurementMNode<ICachedMNode> node) {
if (node.isPreDeleted()) {
@@ -941,7 +954,8 @@ public class MTreeBelowSGCachedImpl {
Map<PartialPath, List<Integer>> resultTemplateSetInfo = new HashMap<>();
for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
try (EntityUpdater<ICachedMNode> updater =
- new EntityUpdater<ICachedMNode>(rootNode, entry.getKey(), store, false) {
+ new EntityUpdater<ICachedMNode>(
+ rootNode, entry.getKey(), store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateEntity(IDeviceMNode<ICachedMNode> node) throws MetadataException {
if (entry.getValue().contains(node.getSchemaTemplateId())) {
@@ -963,7 +977,8 @@ public class MTreeBelowSGCachedImpl {
Map<PartialPath, List<Integer>> resultTemplateSetInfo = new HashMap<>();
for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
try (EntityUpdater<ICachedMNode> updater =
- new EntityUpdater<ICachedMNode>(rootNode, entry.getKey(), store, false) {
+ new EntityUpdater<ICachedMNode>(
+ rootNode, entry.getKey(), store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateEntity(IDeviceMNode<ICachedMNode> node) throws MetadataException {
if (entry.getValue().contains(node.getSchemaTemplateId())
@@ -986,7 +1001,8 @@ public class MTreeBelowSGCachedImpl {
Map<PartialPath, List<Integer>> resultTemplateSetInfo = new HashMap<>();
for (Map.Entry<PartialPath, List<Integer>> entry : templateSetInfo.entrySet()) {
try (EntityUpdater<ICachedMNode> collector =
- new EntityUpdater<ICachedMNode>(rootNode, entry.getKey(), store, false) {
+ new EntityUpdater<ICachedMNode>(
+ rootNode, entry.getKey(), store, false, SchemaConstant.ALL_MATCH_SCOPE) {
protected void updateEntity(IDeviceMNode<ICachedMNode> node) throws MetadataException {
if (entry.getValue().contains(node.getSchemaTemplateId())
@@ -1011,7 +1027,7 @@ public class MTreeBelowSGCachedImpl {
public long countPathsUsingTemplate(PartialPath pathPattern, int templateId)
throws MetadataException {
try (EntityCounter<ICachedMNode> counter =
- new EntityCounter<>(rootNode, pathPattern, store, false)) {
+ new EntityCounter<>(rootNode, pathPattern, store, false, SchemaConstant.ALL_MATCH_SCOPE)) {
counter.setSchemaTemplateFilter(templateId);
return counter.count();
}
@@ -1053,7 +1069,11 @@ public class MTreeBelowSGCachedImpl {
throws MetadataException {
EntityCollector<IDeviceSchemaInfo, ICachedMNode> collector =
new EntityCollector<IDeviceSchemaInfo, ICachedMNode>(
- rootNode, showDevicesPlan.getPath(), store, showDevicesPlan.isPrefixMatch()) {
+ rootNode,
+ showDevicesPlan.getPath(),
+ store,
+ showDevicesPlan.isPrefixMatch(),
+ SchemaConstant.ALL_MATCH_SCOPE) {
protected IDeviceSchemaInfo collectEntity(IDeviceMNode<ICachedMNode> node) {
PartialPath device = getPartialPathFromRootToNode(node.getAsMNode());
@@ -1118,7 +1138,11 @@ public class MTreeBelowSGCachedImpl {
throws MetadataException {
MeasurementCollector<ITimeSeriesSchemaInfo, ICachedMNode> collector =
new MeasurementCollector<ITimeSeriesSchemaInfo, ICachedMNode>(
- rootNode, showTimeSeriesPlan.getPath(), store, showTimeSeriesPlan.isPrefixMatch()) {
+ rootNode,
+ showTimeSeriesPlan.getPath(),
+ store,
+ showTimeSeriesPlan.isPrefixMatch(),
+ SchemaConstant.ALL_MATCH_SCOPE) {
protected ITimeSeriesSchemaInfo collectMeasurement(IMeasurementMNode<ICachedMNode> node) {
return new ITimeSeriesSchemaInfo() {
@@ -1190,7 +1214,11 @@ public class MTreeBelowSGCachedImpl {
throws MetadataException {
MNodeCollector<INodeSchemaInfo, ICachedMNode> collector =
new MNodeCollector<INodeSchemaInfo, ICachedMNode>(
- rootNode, showNodesPlan.getPath(), store, showNodesPlan.isPrefixMatch()) {
+ rootNode,
+ showNodesPlan.getPath(),
+ store,
+ showNodesPlan.isPrefixMatch(),
+ SchemaConstant.ALL_MATCH_SCOPE) {
protected INodeSchemaInfo collectMNode(ICachedMNode node) {
return new ShowNodesResult(
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java
index 115ce1d27df..a07a990c704 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.path.fa.IFAState;
import org.apache.iotdb.commons.path.fa.IFATransition;
import org.apache.iotdb.commons.schema.node.IMNode;
@@ -81,11 +82,17 @@ public abstract class Traverser<R, N extends IMNode<N>> extends AbstractTreeVisi
* @param path use wildcard to specify which part to traverse
* @param store MTree store to traverse
* @param isPrefixMatch prefix match or not
+ * @param scope traversing scope
* @throws MetadataException path does not meet the expected rules
*/
- protected Traverser(N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ protected Traverser(
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, isPrefixMatch);
+ super(startNode, path, isPrefixMatch, scope);
this.store = store.getWithReentrantReadLock();
initStack();
String[] nodes = path.getNodes();
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/DatabaseTraverser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/DatabaseTraverser.java
index 8ed4460eb96..005afa3d46c 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/DatabaseTraverser.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/DatabaseTraverser.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.Traverser;
@@ -35,12 +36,17 @@ public abstract class DatabaseTraverser<R, N extends IMNode<N>> extends Traverse
* @param path use wildcard to specify which part to traverse
* @param store MTree store to traverse
* @param isPrefixMatch prefix match or not
+ * @param scope traversing scope
* @throws MetadataException path does not meet the expected rules
*/
public DatabaseTraverser(
- N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/EntityTraverser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/EntityTraverser.java
index 8b85e939dab..25f5a7c19ec 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/EntityTraverser.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/EntityTraverser.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.Traverser;
@@ -36,11 +37,17 @@ public abstract class EntityTraverser<R, N extends IMNode<N>> extends Traverser<
* @param path use wildcard to specify which part to traverse
* @param store MTree store to traverse
* @param isPrefixMatch prefix match or not
+ * @param scope traversing scope
* @throws MetadataException path does not meet the expected rules
*/
- public EntityTraverser(N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ public EntityTraverser(
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MNodeTraverser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MNodeTraverser.java
index e327171cf45..97dba7fc410 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MNodeTraverser.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MNodeTraverser.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.Traverser;
@@ -43,11 +44,17 @@ public abstract class MNodeTraverser<R, N extends IMNode<N>> extends Traverser<R
* @param path use wildcard to specify which part to traverse
* @param store MTree store to traverse
* @param isPrefixMatch prefix match or not
+ * @param scope traversing scope
* @throws MetadataException path does not meet the expected rules
*/
- public MNodeTraverser(N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ public MNodeTraverser(
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MeasurementTraverser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MeasurementTraverser.java
index 7dee7dcf8ff..05bc6a1acce 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MeasurementTraverser.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MeasurementTraverser.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.Traverser;
@@ -33,12 +34,17 @@ public abstract class MeasurementTraverser<R, N extends IMNode<N>> extends Trave
* @param path use wildcard to specify which part to traverse
* @param store MTree store to traverse
* @param isPrefixMatch prefix match or not
+ * @param scope traversing scope
* @throws MetadataException path does not meet the expected rules
*/
- public MeasurementTraverser(
- N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ protected MeasurementTraverser(
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/DatabaseCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/DatabaseCollector.java
index 3c86b080a8a..d9f8aee3026 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/DatabaseCollector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/DatabaseCollector.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.collector;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
@@ -28,9 +29,13 @@ import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic.Datab
// This class implements database path collection function.
public abstract class DatabaseCollector<R, N extends IMNode<N>> extends DatabaseTraverser<R, N> {
protected DatabaseCollector(
- N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/EntityCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/EntityCollector.java
index 3e07b61b8a3..7a588949845 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/EntityCollector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/EntityCollector.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.collector;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
@@ -30,9 +31,13 @@ import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic.Entit
public abstract class EntityCollector<R, N extends IMNode<N>> extends EntityTraverser<R, N> {
protected EntityCollector(
- N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MNodeAboveDBCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MNodeAboveDBCollector.java
index 1c4f671b28e..0c092e8acf3 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MNodeAboveDBCollector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MNodeAboveDBCollector.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.collector;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
@@ -31,9 +32,13 @@ public abstract class MNodeAboveDBCollector<T, N extends IMNode<N>> extends MNod
protected Set<PartialPath> involvedDatabaseMNodes = new HashSet<>();
protected MNodeAboveDBCollector(
- N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MNodeCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MNodeCollector.java
index 67bf11795d7..fdbb2419c78 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MNodeCollector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MNodeCollector.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.collector;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic.MNodeTraverser;
@@ -34,9 +35,13 @@ import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic.MNode
public abstract class MNodeCollector<R, N extends IMNode<N>> extends MNodeTraverser<R, N> {
protected MNodeCollector(
- N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
protected final R transferToResult(N node) {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MeasurementCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MeasurementCollector.java
index b22e7cd0494..b591ae26d05 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MeasurementCollector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/collector/MeasurementCollector.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.collector;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.MeasurementPath;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
@@ -32,9 +33,13 @@ public abstract class MeasurementCollector<R, N extends IMNode<N>>
extends MeasurementTraverser<R, N> {
protected MeasurementCollector(
- N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/DatabaseCounter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/DatabaseCounter.java
index 5953289837a..2e16b188e21 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/DatabaseCounter.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/DatabaseCounter.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.counter;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic.DatabaseTraverser;
@@ -30,9 +31,14 @@ public class DatabaseCounter<N extends IMNode<N>> extends DatabaseTraverser<Void
private int count;
- public DatabaseCounter(N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ public DatabaseCounter(
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/EntityCounter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/EntityCounter.java
index be1083100f1..af71484e7e0 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/EntityCounter.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/EntityCounter.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.counter;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic.EntityTraverser;
@@ -29,9 +30,14 @@ public class EntityCounter<N extends IMNode<N>> extends EntityTraverser<Void, N>
implements Counter {
private int count;
- public EntityCounter(N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ public EntityCounter(
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/MeasurementCounter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/MeasurementCounter.java
index eab6b0df1df..8ecc25685cd 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/MeasurementCounter.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/counter/MeasurementCounter.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.counter;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.basic.MeasurementTraverser;
@@ -30,9 +31,13 @@ public class MeasurementCounter<N extends IMNode<N>> extends MeasurementTraverse
private int count;
public MeasurementCounter(
- N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/updater/EntityUpdater.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/updater/EntityUpdater.java
index 6dfd6ee1137..d11cdb29143 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/updater/EntityUpdater.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/updater/EntityUpdater.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.updater;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.commons.schema.node.role.IDeviceMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
@@ -34,11 +35,17 @@ public abstract class EntityUpdater<N extends IMNode<N>> extends EntityTraverser
* @param path use wildcard to specify which part to traverse
* @param store MTree store to traverse
* @param isPrefixMatch prefix match or not
+ * @param scope traversing scope
* @throws MetadataException path does not meet the expected rules
*/
- public EntityUpdater(N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ protected EntityUpdater(
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/updater/MeasurementUpdater.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/updater/MeasurementUpdater.java
index 160e49805c5..d8b6ef03621 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/updater/MeasurementUpdater.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/updater/MeasurementUpdater.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.updater;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore;
@@ -34,12 +35,17 @@ public abstract class MeasurementUpdater<N extends IMNode<N>> extends Measuremen
* @param path use wildcard to specify which part to traverse
* @param store MTree store to traverse
* @param isPrefixMatch prefix match or not
+ * @param scope the scope of the path
* @throws MetadataException path does not meet the expected rules
*/
public MeasurementUpdater(
- N startNode, PartialPath path, IMTreeStore<N> store, boolean isPrefixMatch)
+ N startNode,
+ PartialPath path,
+ IMTreeStore<N> store,
+ boolean isPrefixMatch,
+ PathPatternTree scope)
throws MetadataException {
- super(startNode, path, store, isPrefixMatch);
+ super(startNode, path, store, isPrefixMatch, scope);
}
@Override
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/IShowSchemaPlan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/IShowSchemaPlan.java
index b34107532f3..49d36dfde90 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/IShowSchemaPlan.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/IShowSchemaPlan.java
@@ -21,6 +21,7 @@
package org.apache.iotdb.db.schemaengine.schemaregion.read.req;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegionPlan;
import org.apache.iotdb.db.schemaengine.schemaregion.SchemaRegionPlanType;
import org.apache.iotdb.db.schemaengine.schemaregion.SchemaRegionPlanVisitor;
@@ -39,6 +40,8 @@ public interface IShowSchemaPlan extends ISchemaRegionPlan {
PartialPath getPath();
+ PathPatternTree getScope();
+
long getLimit();
long getOffset();
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 df32455535f..eb0c2f64acc 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
@@ -20,63 +20,37 @@
package org.apache.iotdb.db.schemaengine.schemaregion.read.req;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
-import org.apache.iotdb.commons.schema.filter.SchemaFilterFactory;
-import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.db.schemaengine.schemaregion.read.req.impl.ShowDevicesPlanImpl;
import org.apache.iotdb.db.schemaengine.schemaregion.read.req.impl.ShowNodesPlanImpl;
import org.apache.iotdb.db.schemaengine.schemaregion.read.req.impl.ShowTimeSeriesPlanImpl;
import org.apache.iotdb.db.schemaengine.template.Template;
-import java.util.Collections;
import java.util.Map;
public class SchemaRegionReadPlanFactory {
private SchemaRegionReadPlanFactory() {}
- @TestOnly
- public static IShowDevicesPlan getShowDevicesPlan(PartialPath path) {
- return new ShowDevicesPlanImpl(path, 0, 0, false, -1, null);
- }
-
- @TestOnly
- public static IShowDevicesPlan getShowDevicesPlan(PartialPath path, boolean isPrefixMatch) {
- return new ShowDevicesPlanImpl(path, 0, 0, isPrefixMatch, -1, null);
- }
-
public static IShowDevicesPlan getShowDevicesPlan(
- PartialPath path, long limit, long offset, boolean isPrefixMatch, SchemaFilter schemaFilter) {
- return new ShowDevicesPlanImpl(path, limit, offset, isPrefixMatch, -1, schemaFilter);
+ PartialPath path,
+ long limit,
+ long offset,
+ boolean isPrefixMatch,
+ SchemaFilter schemaFilter,
+ PathPatternTree scope) {
+ return new ShowDevicesPlanImpl(path, limit, offset, isPrefixMatch, -1, schemaFilter, scope);
}
public static IShowDevicesPlan getShowDevicesPlan(
- PartialPath path, int limit, int offset, boolean isPrefixMatch, int templateId) {
- return new ShowDevicesPlanImpl(path, limit, offset, isPrefixMatch, templateId, null);
- }
-
- @TestOnly
- public static IShowTimeSeriesPlan getShowTimeSeriesPlan(PartialPath path) {
- return new ShowTimeSeriesPlanImpl(path, Collections.emptyMap(), 0, 0, false, null, false);
- }
-
- @TestOnly
- public static IShowTimeSeriesPlan getShowTimeSeriesPlan(
- PartialPath path, Map<Integer, Template> relatedTemplate) {
- return new ShowTimeSeriesPlanImpl(path, relatedTemplate, 0, 0, false, null, false);
- }
-
- @TestOnly
- public static IShowTimeSeriesPlan getShowTimeSeriesPlan(
- PartialPath path, boolean isContains, String key, String value) {
- return new ShowTimeSeriesPlanImpl(
- path,
- Collections.emptyMap(),
- 0,
- 0,
- false,
- SchemaFilterFactory.createTagFilter(key, value, isContains),
- false);
+ PartialPath path,
+ int limit,
+ int offset,
+ boolean isPrefixMatch,
+ int templateId,
+ PathPatternTree scope) {
+ return new ShowDevicesPlanImpl(path, limit, offset, isPrefixMatch, templateId, null, scope);
}
public static IShowTimeSeriesPlan getShowTimeSeriesPlan(
@@ -86,17 +60,18 @@ public class SchemaRegionReadPlanFactory {
long offset,
boolean isPrefixMatch,
SchemaFilter schemaFilter,
- boolean needViewDetail) {
+ boolean needViewDetail,
+ PathPatternTree scope) {
return new ShowTimeSeriesPlanImpl(
- path, relatedTemplate, limit, offset, isPrefixMatch, schemaFilter, needViewDetail);
+ path, relatedTemplate, limit, offset, isPrefixMatch, schemaFilter, needViewDetail, scope);
}
- public static IShowNodesPlan getShowNodesPlan(PartialPath path) {
- return new ShowNodesPlanImpl(path, -1, false);
+ public static IShowNodesPlan getShowNodesPlan(PartialPath path, PathPatternTree scope) {
+ return new ShowNodesPlanImpl(path, -1, false, scope);
}
public static IShowNodesPlan getShowNodesPlan(
- PartialPath path, int level, boolean isPrefixMatch) {
- return new ShowNodesPlanImpl(path, level, isPrefixMatch);
+ PartialPath path, int level, boolean isPrefixMatch, PathPatternTree scope) {
+ return new ShowNodesPlanImpl(path, level, isPrefixMatch, scope);
}
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/AbstractShowSchemaPlanImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/AbstractShowSchemaPlanImpl.java
index c60246cb68d..8694d8c09ec 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/AbstractShowSchemaPlanImpl.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/AbstractShowSchemaPlanImpl.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.schemaengine.schemaregion.read.req.impl;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.schemaengine.schemaregion.read.req.IShowSchemaPlan;
import java.util.Objects;
@@ -27,22 +29,26 @@ import java.util.Objects;
public abstract class AbstractShowSchemaPlanImpl implements IShowSchemaPlan {
protected final PartialPath path;
+ protected final PathPatternTree scope;
protected final long limit;
protected final long offset;
protected final boolean isPrefixMatch;
protected AbstractShowSchemaPlanImpl(PartialPath path) {
this.path = path;
+ this.scope = SchemaConstant.ALL_MATCH_SCOPE;
this.limit = 0;
this.offset = 0;
this.isPrefixMatch = false;
}
- AbstractShowSchemaPlanImpl(PartialPath path, long limit, long offset, boolean isPrefixMatch) {
+ AbstractShowSchemaPlanImpl(
+ PartialPath path, long limit, long offset, boolean isPrefixMatch, PathPatternTree scope) {
this.path = path;
this.limit = limit;
this.offset = offset;
this.isPrefixMatch = isPrefixMatch;
+ this.scope = scope;
}
@Override
@@ -50,6 +56,11 @@ public abstract class AbstractShowSchemaPlanImpl implements IShowSchemaPlan {
return path;
}
+ @Override
+ public PathPatternTree getScope() {
+ return scope;
+ }
+
@Override
public long getLimit() {
return limit;
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowDevicesPlanImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowDevicesPlanImpl.java
index 5a0e9616343..c75734747d6 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowDevicesPlanImpl.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowDevicesPlanImpl.java
@@ -21,6 +21,7 @@
package org.apache.iotdb.db.schemaengine.schemaregion.read.req.impl;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.schemaengine.schemaregion.read.req.IShowDevicesPlan;
@@ -38,8 +39,9 @@ public class ShowDevicesPlanImpl extends AbstractShowSchemaPlanImpl implements I
long offset,
boolean isPrefixMatch,
int schemaTemplateId,
- SchemaFilter schemaFilter) {
- super(path, limit, offset, isPrefixMatch);
+ SchemaFilter schemaFilter,
+ PathPatternTree scope) {
+ super(path, limit, offset, isPrefixMatch, scope);
this.schemaTemplateId = schemaTemplateId;
this.schemaFilter = schemaFilter;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowNodesPlanImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowNodesPlanImpl.java
index 34ed3c088ba..fb36d7bc549 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowNodesPlanImpl.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/read/req/impl/ShowNodesPlanImpl.java
@@ -21,14 +21,16 @@
package org.apache.iotdb.db.schemaengine.schemaregion.read.req.impl;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.schemaengine.schemaregion.read.req.IShowNodesPlan;
public class ShowNodesPlanImpl extends AbstractShowSchemaPlanImpl implements IShowNodesPlan {
private final int level;
- public ShowNodesPlanImpl(PartialPath pathPattern, int level, boolean isPrefixMatch) {
- super(pathPattern, 0, 0, isPrefixMatch);
+ public ShowNodesPlanImpl(
+ PartialPath pathPattern, int level, boolean isPrefixMatch, PathPatternTree scope) {
+ super(pathPattern, 0, 0, isPrefixMatch, scope);
this.level = level;
}
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 5798d340b5b..ba9ea23cd3a 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
@@ -21,6 +21,7 @@
package org.apache.iotdb.db.schemaengine.schemaregion.read.req.impl;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.schema.filter.SchemaFilter;
import org.apache.iotdb.db.schemaengine.schemaregion.read.req.IShowTimeSeriesPlan;
import org.apache.iotdb.db.schemaengine.template.Template;
@@ -43,8 +44,9 @@ public class ShowTimeSeriesPlanImpl extends AbstractShowSchemaPlanImpl
long offset,
boolean isPrefixMatch,
SchemaFilter schemaFilter,
- boolean needViewDetail) {
- super(path, limit, offset, isPrefixMatch);
+ boolean needViewDetail,
+ PathPatternTree scope) {
+ super(path, limit, offset, isPrefixMatch, scope);
this.relatedTemplate = relatedTemplate;
this.schemaFilter = schemaFilter;
this.needViewDetail = needViewDetail;
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/template/ClusterTemplateManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/template/ClusterTemplateManager.java
index 42a907fa5f8..d82172c816b 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/template/ClusterTemplateManager.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/template/ClusterTemplateManager.java
@@ -28,10 +28,12 @@ import org.apache.iotdb.commons.exception.IoTDBException;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.exception.runtime.SchemaExecutionException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.path.PathPatternUtil;
import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.confignode.rpc.thrift.TCreateSchemaTemplateReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp;
+import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesReq;
import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesResp;
import org.apache.iotdb.confignode.rpc.thrift.TGetTemplateResp;
import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq;
@@ -239,11 +241,13 @@ public class ClusterTemplateManager implements ITemplateManager {
}
@Override
- public List<PartialPath> getPathsSetTemplate(String name) {
+ public List<PartialPath> getPathsSetTemplate(String name, PathPatternTree scope) {
List<PartialPath> listPath = new ArrayList<>();
try (ConfigNodeClient configNodeClient =
CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
- TGetPathsSetTemplatesResp resp = configNodeClient.getPathsSetTemplate(name);
+ TGetPathsSetTemplatesResp resp =
+ configNodeClient.getPathsSetTemplate(
+ new TGetPathsSetTemplatesReq(name, scope.serialize()));
if (resp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
if (resp.getPathList() != null) {
resp.getPathList()
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/template/ITemplateManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/template/ITemplateManager.java
index 1b86ba50b62..f2bf7e6d673 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/template/ITemplateManager.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/template/ITemplateManager.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.schemaengine.template;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
import org.apache.iotdb.commons.exception.IoTDBException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.queryengine.plan.statement.metadata.template.CreateSchemaTemplateStatement;
import org.apache.iotdb.tsfile.utils.Pair;
@@ -63,8 +64,9 @@ public interface ITemplateManager {
* Get info of mounted template.
*
* @param name template name
+ * @param scope scope
*/
- List<PartialPath> getPathsSetTemplate(String name);
+ List<PartialPath> getPathsSetTemplate(String name, PathPatternTree scope);
Pair<Template, PartialPath> checkTemplateSetInfo(PartialPath devicePath);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/IoTDBInternalLocalReporter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/IoTDBInternalLocalReporter.java
index 303c9dbe7b8..da7a72206eb 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/IoTDBInternalLocalReporter.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/metrics/IoTDBInternalLocalReporter.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.commons.consensus.ConfigRegionId;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
+import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq;
import org.apache.iotdb.confignode.rpc.thrift.TShowDatabaseResp;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.protocol.client.ConfigNodeClient;
@@ -86,8 +87,11 @@ public class IoTDBInternalLocalReporter extends IoTDBInternalReporter {
ConfigNodeClientManager.getInstance();
try (ConfigNodeClient client =
configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
- TShowDatabaseResp showDatabaseResp =
- client.showDatabase(Arrays.asList(SchemaConstant.SYSTEM_DATABASE.split("\\.")));
+ TGetDatabaseReq req =
+ new TGetDatabaseReq(
+ Arrays.asList(SchemaConstant.SYSTEM_DATABASE.split("\\.")),
+ SchemaConstant.ALL_MATCH_SCOPE_BINARY);
+ TShowDatabaseResp showDatabaseResp = client.showDatabase(req);
if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == showDatabaseResp.getStatus().getCode()
&& showDatabaseResp.getDatabaseInfoMapSize() == 0) {
TDatabaseSchema databaseSchema = new TDatabaseSchema();
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionAliasAndTagTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionAliasAndTagTest.java
index 567d6ca73b5..40311fc1321 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionAliasAndTagTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionAliasAndTagTest.java
@@ -23,7 +23,6 @@ import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion;
-import org.apache.iotdb.db.schemaengine.schemaregion.read.req.SchemaRegionReadPlanFactory;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ITimeSeriesSchemaInfo;
import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -145,9 +144,7 @@ public class SchemaRegionAliasAndTagTest extends AbstractSchemaRegionTest {
String fullPath, String alias, Map<String, String> tags, Map<String, String> attributes) {
try {
List<ITimeSeriesSchemaInfo> result =
- SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(new PartialPath(fullPath)));
+ SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath(fullPath));
Assert.assertEquals(1, result.size());
Assert.assertEquals(fullPath, result.get(0).getFullPath());
Assert.assertEquals(alias, result.get(0).getAlias());
@@ -162,9 +159,7 @@ public class SchemaRegionAliasAndTagTest extends AbstractSchemaRegionTest {
private void checkAttributes(String fullPath, Map<String, String> attributes) {
try {
List<ITimeSeriesSchemaInfo> result =
- SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(new PartialPath(fullPath)));
+ SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath(fullPath));
Assert.assertEquals(1, result.size());
Assert.assertEquals(fullPath, result.get(0).getFullPath());
Assert.assertEquals(attributes, result.get(0).getAttributes());
@@ -177,9 +172,7 @@ public class SchemaRegionAliasAndTagTest extends AbstractSchemaRegionTest {
private void checkTags(String fullPath, Map<String, String> tags) {
try {
List<ITimeSeriesSchemaInfo> result =
- SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(new PartialPath(fullPath)));
+ SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath(fullPath));
Assert.assertEquals(1, result.size());
Assert.assertEquals(fullPath, result.get(0).getFullPath());
Assert.assertEquals(tags, result.get(0).getTags());
@@ -553,9 +546,7 @@ public class SchemaRegionAliasAndTagTest extends AbstractSchemaRegionTest {
prepareTimeseries();
List<ITimeSeriesSchemaInfo> result =
SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.sg.wf01.wt01.v1.temp")));
+ schemaRegion, new PartialPath("root.sg.wf01.wt01.v1.temp"));
Assert.assertEquals(1, result.size());
// delete timeseries
PathPatternTree patternTree = new PathPatternTree();
@@ -565,9 +556,7 @@ public class SchemaRegionAliasAndTagTest extends AbstractSchemaRegionTest {
schemaRegion.deleteTimeseriesInBlackList(patternTree);
result =
SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.sg.wf01.wt01.v1.temp")));
+ schemaRegion, new PartialPath("root.sg.wf01.wt01.v1.temp"));
Assert.assertEquals(0, result.size());
} catch (Exception e) {
logger.error(e.getMessage(), e);
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionBasicTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionBasicTest.java
index 1794d24df7a..0c692300f73 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionBasicTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionBasicTest.java
@@ -28,7 +28,6 @@ import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
import org.apache.iotdb.db.exception.metadata.MeasurementAlreadyExistException;
import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion;
-import org.apache.iotdb.db.schemaengine.schemaregion.read.req.SchemaRegionReadPlanFactory;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.IDeviceSchemaInfo;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ITimeSeriesSchemaInfo;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.impl.ShowDevicesResult;
@@ -622,35 +621,25 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
// CASE 01. Query a timeseries, result should be empty set.
Assert.assertEquals(
Collections.emptyList(),
- SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(new PartialPath("root.laptop.d0"))));
+ SchemaRegionTestUtil.getMatchedDevices(schemaRegion, new PartialPath("root.laptop.d0")));
// CASE 02. Query an existing device.
Assert.assertEquals(
Collections.singletonList(new ShowDevicesResult("root.laptop.d1", false)),
- SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(new PartialPath("root.laptop.d1"))));
+ SchemaRegionTestUtil.getMatchedDevices(schemaRegion, new PartialPath("root.laptop.d1")));
Assert.assertEquals(
Collections.singletonList(new ShowDevicesResult("root.laptop.d2", false)),
- SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(new PartialPath("root.laptop.d2"))));
+ SchemaRegionTestUtil.getMatchedDevices(schemaRegion, new PartialPath("root.laptop.d2")));
// CASE 03. Query an existing device, which has a sub device
Assert.assertEquals(
Collections.singletonList(new ShowDevicesResult("root.laptop", false)),
- SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(new PartialPath("root.laptop"))));
+ SchemaRegionTestUtil.getMatchedDevices(schemaRegion, new PartialPath("root.laptop")));
// CASE 04. Query devices using '*'
Assert.assertEquals(
Collections.singletonList(new ShowDevicesResult("root.laptop", false)),
- SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(new PartialPath("root.*"))));
+ SchemaRegionTestUtil.getMatchedDevices(schemaRegion, new PartialPath("root.*")));
// CASE 05. Query all devices using 'root.**'
List<IDeviceSchemaInfo> expectedList =
@@ -660,9 +649,7 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
new ShowDevicesResult("root.laptop.d2", false),
new ShowDevicesResult("root.laptop.d1.s2", false));
List<IDeviceSchemaInfo> actualResult =
- SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(new PartialPath("root.**")));
+ SchemaRegionTestUtil.getMatchedDevices(schemaRegion, new PartialPath("root.**"));
// Compare hash sets because the order does not matter.
HashSet<IDeviceSchemaInfo> expectedHashset = new HashSet<>(expectedList);
HashSet<IDeviceSchemaInfo> actualHashset = new HashSet<>(actualResult);
@@ -674,9 +661,7 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
new ShowDevicesResult("root.laptop.d1", false),
new ShowDevicesResult("root.laptop.d2", false));
actualResult =
- SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(new PartialPath("root.**.d*")));
+ SchemaRegionTestUtil.getMatchedDevices(schemaRegion, new PartialPath("root.**.d*"));
// Compare hash sets because the order does not matter.
expectedHashset = new HashSet<>(expectedList);
actualHashset = new HashSet<>(actualResult);
@@ -685,30 +670,22 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
// CASE 07. show devices root.** limit 3 offset 0
actualResult =
SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(
- new PartialPath("root.**"), 3, 0, false, null));
+ schemaRegion, new PartialPath("root.**"), 3, 0, false);
Assert.assertEquals(3, actualResult.size());
// CASE 08. show devices root.** limit 3 offset 1
actualResult =
SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(
- new PartialPath("root.**"), 3, 1, false, null));
+ schemaRegion, new PartialPath("root.**"), 3, 1, false);
Assert.assertEquals(3, actualResult.size());
// CASE 09. show devices root.** limit 3 offset 2
actualResult =
SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(
- new PartialPath("root.**"), 3, 2, false, null));
+ schemaRegion, new PartialPath("root.**"), 3, 2, false);
Assert.assertEquals(2, actualResult.size());
// CASE 10. show devices root.** limit 3 offset 99
actualResult =
SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(
- new PartialPath("root.**"), 3, 99, false, null));
+ schemaRegion, new PartialPath("root.**"), 3, 99, false);
Assert.assertEquals(0, actualResult.size());
// CASE 11. show devices root.** where device contains 'laptop'
expectedList =
@@ -720,12 +697,11 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
actualResult =
SchemaRegionTestUtil.getMatchedDevices(
schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(
- new PartialPath("root.**"),
- 0,
- 0,
- false,
- SchemaFilterFactory.createPathContainsFilter("laptop")));
+ new PartialPath("root.**"),
+ 0,
+ 0,
+ false,
+ SchemaFilterFactory.createPathContainsFilter("laptop"));
expectedHashset = new HashSet<>(expectedList);
actualHashset = new HashSet<>(actualResult);
Assert.assertEquals(expectedHashset, actualHashset);
@@ -737,12 +713,11 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
actualResult =
SchemaRegionTestUtil.getMatchedDevices(
schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(
- new PartialPath("root.**"),
- 2,
- 0,
- false,
- SchemaFilterFactory.createPathContainsFilter("laptop.d")));
+ new PartialPath("root.**"),
+ 2,
+ 0,
+ false,
+ SchemaFilterFactory.createPathContainsFilter("laptop.d"));
expectedHashset = new HashSet<>(expectedList);
actualHashset = new HashSet<>(actualResult);
Assert.assertEquals(expectedHashset, actualHashset);
@@ -764,9 +739,7 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
// CASE 01: all timeseries
List<ITimeSeriesSchemaInfo> result =
- SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(new PartialPath("root.**")));
+ SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath("root.**"));
Set<String> expectedPathList =
new HashSet<>(
Arrays.asList(
@@ -785,10 +758,7 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
Assert.assertEquals(expectedPathList, actualPathList);
// CASE 02: some timeseries, pattern "root.**.s*"
- result =
- SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(new PartialPath("root.**.s*")));
+ result = SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath("root.**.s*"));
expectedPathList =
new HashSet<>(
Arrays.asList(
@@ -808,14 +778,13 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
result =
SchemaRegionTestUtil.showTimeseries(
schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.**"),
- Collections.emptyMap(),
- 0,
- 0,
- false,
- SchemaFilterFactory.createPathContainsFilter("s"),
- false));
+ new PartialPath("root.**"),
+ Collections.emptyMap(),
+ 0,
+ 0,
+ false,
+ SchemaFilterFactory.createPathContainsFilter("s"),
+ false);
expectedPathList =
new HashSet<>(
Arrays.asList(
@@ -836,14 +805,13 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
result =
SchemaRegionTestUtil.showTimeseries(
schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.**"),
- Collections.emptyMap(),
- 0,
- 0,
- false,
- SchemaFilterFactory.createPathContainsFilter("1"),
- false));
+ new PartialPath("root.**"),
+ Collections.emptyMap(),
+ 0,
+ 0,
+ false,
+ SchemaFilterFactory.createPathContainsFilter("1"),
+ false);
expectedPathList =
new HashSet<>(
Arrays.asList(
@@ -863,14 +831,13 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
result =
SchemaRegionTestUtil.showTimeseries(
schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.**"),
- Collections.emptyMap(),
- 0,
- 0,
- false,
- SchemaFilterFactory.createPathContainsFilter("laptop.d"),
- false));
+ new PartialPath("root.**"),
+ Collections.emptyMap(),
+ 0,
+ 0,
+ false,
+ SchemaFilterFactory.createPathContainsFilter("laptop.d"),
+ false);
expectedPathList =
new HashSet<>(
Arrays.asList(
@@ -892,14 +859,13 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
result =
SchemaRegionTestUtil.showTimeseries(
schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.**"),
- Collections.emptyMap(),
- 0,
- 0,
- false,
- SchemaFilterFactory.createDataTypeFilter(TSDataType.INT64),
- false));
+ new PartialPath("root.**"),
+ Collections.emptyMap(),
+ 0,
+ 0,
+ false,
+ SchemaFilterFactory.createDataTypeFilter(TSDataType.INT64),
+ false);
expectedPathList =
new HashSet<>(
Arrays.asList(
@@ -921,14 +887,13 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
result =
SchemaRegionTestUtil.showTimeseries(
schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.**"),
- Collections.emptyMap(),
- 0,
- 0,
- false,
- SchemaFilterFactory.createDataTypeFilter(TSDataType.BOOLEAN),
- false));
+ new PartialPath("root.**"),
+ Collections.emptyMap(),
+ 0,
+ 0,
+ false,
+ SchemaFilterFactory.createDataTypeFilter(TSDataType.BOOLEAN),
+ false);
expectedPathList = new HashSet<>(Collections.emptyList());
expectedSize = expectedPathList.size();
Assert.assertEquals(expectedSize, result.size());
@@ -953,18 +918,14 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
new ShowDevicesResult("root.test.dac.device1", false),
new ShowDevicesResult("root.test.dac.device1.d1", false));
List<IDeviceSchemaInfo> actualResult =
- SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(new PartialPath("root.**.d*")));
+ SchemaRegionTestUtil.getMatchedDevices(schemaRegion, new PartialPath("root.**.d*"));
// Compare hash sets because the order does not matter.
Set<IDeviceSchemaInfo> expectedHashset = new HashSet<>(expectedList);
Set<IDeviceSchemaInfo> actualHashset = new HashSet<>(actualResult);
Assert.assertEquals(expectedHashset, actualHashset);
List<ITimeSeriesSchemaInfo> result =
- SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(new PartialPath("root.**.d*.*")));
+ SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath("root.**.d*.*"));
Set<String> expectedPathList =
new HashSet<>(
Arrays.asList(
@@ -996,9 +957,7 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
new ShowDevicesResult("root.test.abc57.bcde22.def89", false),
new ShowDevicesResult("root.test.abc57.bcd22.def89", false));
List<IDeviceSchemaInfo> actualResult =
- SchemaRegionTestUtil.getMatchedDevices(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowDevicesPlan(new PartialPath("root.**.*b*.*")));
+ SchemaRegionTestUtil.getMatchedDevices(schemaRegion, new PartialPath("root.**.*b*.*"));
// Compare hash sets because the order does not matter.
Set<IDeviceSchemaInfo> expectedHashset = new HashSet<>(expectedList);
Set<IDeviceSchemaInfo> actualHashset = new HashSet<>(actualResult);
@@ -1006,9 +965,7 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
// case2: show timeseries root.**.*e*.*e*
List<ITimeSeriesSchemaInfo> result =
- SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(new PartialPath("root.**.*e*.*e*")));
+ SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath("root.**.*e*.*e*"));
Set<String> expectedPathList =
new HashSet<>(
Arrays.asList(
@@ -1025,10 +982,7 @@ public class SchemaRegionBasicTest extends AbstractSchemaRegionTest {
Assert.assertEquals(expectedPathList, actualPathList);
// case3: show timeseries root.**.*e*
- result =
- SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(new PartialPath("root.**.*e*")));
+ result = SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath("root.**.*e*"));
Assert.assertEquals(expectedSize, result.size());
actualPathList = new HashSet<>();
for (int index = 0; index < expectedSize; index++) {
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionManagementTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionManagementTest.java
index e604803af52..31c600bbdc6 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionManagementTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionManagementTest.java
@@ -27,7 +27,6 @@ import org.apache.iotdb.consensus.ConsensusFactory;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion;
-import org.apache.iotdb.db.schemaengine.schemaregion.read.req.SchemaRegionReadPlanFactory;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ISchemaInfo;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ITimeSeriesSchemaInfo;
import org.apache.iotdb.db.schemaengine.schemaregion.write.req.SchemaRegionWritePlanFactory;
@@ -98,9 +97,7 @@ public class SchemaRegionManagementTest extends AbstractSchemaRegionTest {
List<ITimeSeriesSchemaInfo> result =
SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.sg.**"), false, "tag-key", "tag-value"));
+ schemaRegion, new PartialPath("root.sg.**"), false, "tag-key", "tag-value");
ITimeSeriesSchemaInfo seriesResult = result.get(0);
Assert.assertEquals(
@@ -115,9 +112,7 @@ public class SchemaRegionManagementTest extends AbstractSchemaRegionTest {
newSchemaRegion.loadSnapshot(snapshotDir);
result =
SchemaRegionTestUtil.showTimeseries(
- newSchemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.sg.**"), false, "tag-key", "tag-value"));
+ newSchemaRegion, new PartialPath("root.sg.**"), false, "tag-key", "tag-value");
seriesResult = result.get(0);
Assert.assertEquals(
@@ -129,9 +124,8 @@ public class SchemaRegionManagementTest extends AbstractSchemaRegionTest {
result =
SchemaRegionTestUtil.showTimeseries(
newSchemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.sg.*.s1"),
- Collections.singletonMap(template.getId(), template)));
+ new PartialPath("root.sg.*.s1"),
+ Collections.singletonMap(template.getId(), template));
result.sort(Comparator.comparing(ISchemaInfo::getFullPath));
Assert.assertEquals(
new PartialPath("root.sg.d1.s1").getFullPath(), result.get(0).getFullPath());
@@ -178,9 +172,7 @@ public class SchemaRegionManagementTest extends AbstractSchemaRegionTest {
List<ITimeSeriesSchemaInfo> result =
SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.sg.**"), false, "tag-key", "tag-value"));
+ schemaRegion, new PartialPath("root.sg.**"), false, "tag-key", "tag-value");
Assert.assertEquals(0, result.size());
@@ -190,9 +182,7 @@ public class SchemaRegionManagementTest extends AbstractSchemaRegionTest {
newSchemaRegion.loadSnapshot(snapshotDir);
result =
SchemaRegionTestUtil.showTimeseries(
- newSchemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.sg.**"), false, "tag-key", "tag-value"));
+ newSchemaRegion, new PartialPath("root.sg.**"), false, "tag-key", "tag-value");
Assert.assertEquals(0, result.size());
} finally {
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionTemplateTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionTemplateTest.java
index 733fcd286fb..e612a6ea7c3 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionTemplateTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionTemplateTest.java
@@ -23,7 +23,6 @@ import org.apache.iotdb.commons.path.MeasurementPath;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion;
-import org.apache.iotdb.db.schemaengine.schemaregion.read.req.SchemaRegionReadPlanFactory;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ISchemaInfo;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ITimeSeriesSchemaInfo;
import org.apache.iotdb.db.schemaengine.schemaregion.write.req.SchemaRegionWritePlanFactory;
@@ -216,10 +215,7 @@ public class SchemaRegionTemplateTest extends AbstractSchemaRegionTest {
// check show timeseries
List<ITimeSeriesSchemaInfo> result =
- SchemaRegionTestUtil.showTimeseries(
- schemaRegion,
- SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- new PartialPath("root.**"), templateMap));
+ SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath("root.**"), templateMap);
result.sort(Comparator.comparing(ISchemaInfo::getFullPath));
for (int i = 0; i < result.size(); i++) {
Assert.assertEquals(expectedTimeseries.get(i), result.get(i).getFullPath());
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 ecc040a3db4..d863e8270aa 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
@@ -21,9 +21,9 @@ package org.apache.iotdb.db.metadata.schemaRegion;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.filter.SchemaFilter;
+import org.apache.iotdb.commons.schema.filter.SchemaFilterFactory;
import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion;
-import org.apache.iotdb.db.schemaengine.schemaregion.read.req.IShowDevicesPlan;
-import org.apache.iotdb.db.schemaengine.schemaregion.read.req.IShowTimeSeriesPlan;
import org.apache.iotdb.db.schemaengine.schemaregion.read.req.SchemaRegionReadPlanFactory;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.IDeviceSchemaInfo;
import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.INodeSchemaInfo;
@@ -38,6 +38,7 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -45,6 +46,7 @@ import java.util.Map;
import java.util.Set;
import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
+import static org.apache.iotdb.commons.schema.SchemaConstant.ALL_MATCH_SCOPE;
public class SchemaRegionTestUtil {
@@ -162,7 +164,7 @@ public class SchemaRegionTestUtil {
try (ISchemaReader<ITimeSeriesSchemaInfo> timeSeriesReader =
schemaRegion.getTimeSeriesReader(
SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- pathPattern, templateMap, 0, 0, isPrefixMatch, null, false))) {
+ pathPattern, templateMap, 0, 0, isPrefixMatch, null, false, ALL_MATCH_SCOPE))) {
long count = 0;
while (timeSeriesReader.hasNext()) {
timeSeriesReader.next();
@@ -178,7 +180,8 @@ public class SchemaRegionTestUtil {
ISchemaRegion schemaRegion, PartialPath pathPattern, boolean isPrefixMatch) {
try (ISchemaReader<IDeviceSchemaInfo> deviceReader =
schemaRegion.getDeviceReader(
- SchemaRegionReadPlanFactory.getShowDevicesPlan(pathPattern, isPrefixMatch))) {
+ SchemaRegionReadPlanFactory.getShowDevicesPlan(
+ pathPattern, 0, 0, isPrefixMatch, null, ALL_MATCH_SCOPE))) {
long count = 0;
while (deviceReader.hasNext()) {
deviceReader.next();
@@ -195,7 +198,7 @@ public class SchemaRegionTestUtil {
try (ISchemaReader<ITimeSeriesSchemaInfo> timeSeriesReader =
schemaRegion.getTimeSeriesReader(
SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
- pathPattern, null, 0, 0, isPrefixMatch, null, false))) {
+ pathPattern, null, 0, 0, isPrefixMatch, null, false, ALL_MATCH_SCOPE))) {
Map<PartialPath, Long> countMap = new HashMap<>();
while (timeSeriesReader.hasNext()) {
ITimeSeriesSchemaInfo timeSeriesSchemaInfo = timeSeriesReader.next();
@@ -224,7 +227,7 @@ public class SchemaRegionTestUtil {
try (ISchemaReader<IDeviceSchemaInfo> deviceReader =
schemaRegion.getDeviceReader(
SchemaRegionReadPlanFactory.getShowDevicesPlan(
- pathPattern, 0, 0, false, templateId)); ) {
+ pathPattern, 0, 0, false, templateId, ALL_MATCH_SCOPE))) {
while (deviceReader.hasNext()) {
result.add(deviceReader.next().getFullPath());
}
@@ -239,7 +242,8 @@ public class SchemaRegionTestUtil {
List<PartialPath> result = new ArrayList<>();
try (ISchemaReader<INodeSchemaInfo> nodeReader =
schemaRegion.getNodeReader(
- SchemaRegionReadPlanFactory.getShowNodesPlan(pathPattern, nodeLevel, isPrefixMatch))) {
+ SchemaRegionReadPlanFactory.getShowNodesPlan(
+ pathPattern, nodeLevel, isPrefixMatch, ALL_MATCH_SCOPE))) {
while (nodeReader.hasNext()) {
result.add(nodeReader.next().getPartialPath());
}
@@ -255,7 +259,7 @@ public class SchemaRegionTestUtil {
try (ISchemaReader<INodeSchemaInfo> nodeReader =
schemaRegion.getNodeReader(
SchemaRegionReadPlanFactory.getShowNodesPlan(
- pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD)))) {
+ pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD), ALL_MATCH_SCOPE))) {
while (nodeReader.hasNext()) {
result.add(nodeReader.next());
}
@@ -266,10 +270,54 @@ public class SchemaRegionTestUtil {
}
public static List<ITimeSeriesSchemaInfo> showTimeseries(
- ISchemaRegion schemaRegion, IShowTimeSeriesPlan plan) {
+ ISchemaRegion schemaRegion, PartialPath path) {
+ return showTimeseries(schemaRegion, path, Collections.emptyMap(), 0, 0, false, null, false);
+ }
+
+ public static List<ITimeSeriesSchemaInfo> showTimeseries(
+ ISchemaRegion schemaRegion, PartialPath path, Map<Integer, Template> relatedTemplate) {
+ return showTimeseries(schemaRegion, path, relatedTemplate, 0, 0, false, null, false);
+ }
+
+ public static List<ITimeSeriesSchemaInfo> showTimeseries(
+ ISchemaRegion schemaRegion,
+ PartialPath path,
+ boolean isContains,
+ String tagKey,
+ String tagValue) {
+ return showTimeseries(
+ schemaRegion,
+ path,
+ Collections.emptyMap(),
+ 0,
+ 0,
+ false,
+ SchemaFilterFactory.createTagFilter(tagKey, tagValue, isContains),
+ false);
+ }
+
+ public static List<ITimeSeriesSchemaInfo> showTimeseries(
+ ISchemaRegion schemaRegion,
+ PartialPath path,
+ Map<Integer, Template> relatedTemplate,
+ long limit,
+ long offset,
+ boolean isPrefixMatch,
+ SchemaFilter schemaFilter,
+ boolean needViewDetail) {
List<ITimeSeriesSchemaInfo> result = new ArrayList<>();
ITimeSeriesSchemaInfo timeSeriesSchemaInfo;
- try (ISchemaReader<ITimeSeriesSchemaInfo> reader = schemaRegion.getTimeSeriesReader(plan)) {
+ try (ISchemaReader<ITimeSeriesSchemaInfo> reader =
+ schemaRegion.getTimeSeriesReader(
+ SchemaRegionReadPlanFactory.getShowTimeSeriesPlan(
+ path,
+ relatedTemplate,
+ limit,
+ offset,
+ isPrefixMatch,
+ schemaFilter,
+ needViewDetail,
+ ALL_MATCH_SCOPE))) {
while (reader.hasNext()) {
timeSeriesSchemaInfo = reader.next();
result.add(
@@ -288,9 +336,27 @@ public class SchemaRegionTestUtil {
}
public static List<IDeviceSchemaInfo> getMatchedDevices(
- ISchemaRegion schemaRegion, IShowDevicesPlan plan) {
+ ISchemaRegion schemaRegion, PartialPath path) {
+ return getMatchedDevices(schemaRegion, path, 0, 0, false);
+ }
+
+ public static List<IDeviceSchemaInfo> getMatchedDevices(
+ ISchemaRegion schemaRegion, PartialPath path, int limit, int offset, boolean isPrefixMatch) {
+ return getMatchedDevices(schemaRegion, path, limit, offset, isPrefixMatch, null);
+ }
+
+ public static List<IDeviceSchemaInfo> getMatchedDevices(
+ ISchemaRegion schemaRegion,
+ PartialPath path,
+ int limit,
+ int offset,
+ boolean isPrefixMatch,
+ SchemaFilter filter) {
List<IDeviceSchemaInfo> result = new ArrayList<>();
- try (ISchemaReader<IDeviceSchemaInfo> reader = schemaRegion.getDeviceReader(plan)) {
+ try (ISchemaReader<IDeviceSchemaInfo> reader =
+ schemaRegion.getDeviceReader(
+ SchemaRegionReadPlanFactory.getShowDevicesPlan(
+ path, limit, offset, isPrefixMatch, filter, ALL_MATCH_SCOPE))) {
while (reader.hasNext()) {
result.add(reader.next());
}
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 6961c05e9d8..d64cb543c03 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
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.execution.operator.schema;
import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.common.FragmentInstanceId;
import org.apache.iotdb.db.queryengine.common.PlanFragmentId;
import org.apache.iotdb.db.queryengine.common.QueryId;
@@ -90,7 +91,8 @@ public class SchemaQueryScanOperatorTest {
operatorContext.setDriverContext(
new SchemaDriverContext(fragmentInstanceContext, schemaRegion, 0));
ISchemaSource<IDeviceSchemaInfo> deviceSchemaSource =
- SchemaSourceFactory.getDeviceSchemaSource(partialPath, false, 10, 0, true, null);
+ SchemaSourceFactory.getDeviceSchemaSource(
+ partialPath, false, 10, 0, true, null, SchemaConstant.ALL_MATCH_SCOPE);
SchemaOperatorTestUtil.mockGetSchemaReader(
deviceSchemaSource,
Collections.singletonList(deviceSchemaInfo).iterator(),
@@ -196,7 +198,13 @@ public class SchemaQueryScanOperatorTest {
new SchemaDriverContext(fragmentInstanceContext, schemaRegion, 0));
ISchemaSource<ITimeSeriesSchemaInfo> timeSeriesSchemaSource =
SchemaSourceFactory.getTimeSeriesSchemaScanSource(
- partialPath, false, 10, 0, null, Collections.emptyMap());
+ partialPath,
+ false,
+ 10,
+ 0,
+ null,
+ Collections.emptyMap(),
+ SchemaConstant.ALL_MATCH_SCOPE);
SchemaOperatorTestUtil.mockGetSchemaReader(
timeSeriesSchemaSource, showTimeSeriesResults.iterator(), schemaRegion, true);
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/FakePartitionFetcherImpl.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/FakePartitionFetcherImpl.java
index 3bb4e323bf1..e7ae7ed4421 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/FakePartitionFetcherImpl.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/FakePartitionFetcherImpl.java
@@ -116,7 +116,7 @@ public class FakePartitionFetcherImpl implements IPartitionFetcher {
@Override
public SchemaNodeManagementPartition getSchemaNodeManagementPartitionWithLevel(
- PathPatternTree patternTree, Integer level) {
+ PathPatternTree patternTree, PathPatternTree scope, Integer level) {
return null;
}
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/distribution/DistributionPlannerBasicTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/distribution/DistributionPlannerBasicTest.java
index 28b315a0289..f9fb9ff3be8 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/distribution/DistributionPlannerBasicTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/distribution/DistributionPlannerBasicTest.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.MeasurementPath;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.SchemaConstant;
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;
@@ -119,7 +120,8 @@ public class DistributionPlannerBasicTest {
0,
false,
false,
- Collections.emptyMap()));
+ Collections.emptyMap(),
+ SchemaConstant.ALL_MATCH_SCOPE));
metaMergeNode.addChild(
new TimeSeriesSchemaScanNode(
queryId.genPlanNodeId(),
@@ -129,7 +131,8 @@ public class DistributionPlannerBasicTest {
0,
false,
false,
- Collections.emptyMap()));
+ Collections.emptyMap(),
+ SchemaConstant.ALL_MATCH_SCOPE));
metaMergeNode.addChild(
new TimeSeriesSchemaScanNode(
queryId.genPlanNodeId(),
@@ -139,7 +142,8 @@ public class DistributionPlannerBasicTest {
0,
false,
false,
- Collections.emptyMap()));
+ Collections.emptyMap(),
+ SchemaConstant.ALL_MATCH_SCOPE));
LimitNode root2 = new LimitNode(queryId.genPlanNodeId(), metaMergeNode, 10);
Analysis analysis = Util.constructAnalysis();
DistributionPlanner planner2 =
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/distribution/Util.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/distribution/Util.java
index 28c8547001f..b3b87c32da4 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/distribution/Util.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/distribution/Util.java
@@ -392,7 +392,7 @@ public class Util {
@Override
public SchemaNodeManagementPartition getSchemaNodeManagementPartitionWithLevel(
- PathPatternTree patternTree, Integer level) {
+ PathPatternTree patternTree, PathPatternTree scope, Integer level) {
return null;
}
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
index 83bd766f245..163224079a9 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/DeviceSchemaScanNodeSerdeTest.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.plan.plan.node.metadata.read;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.common.FragmentInstanceId;
import org.apache.iotdb.db.queryengine.common.PlanFragmentId;
import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelLocation;
@@ -55,7 +56,8 @@ public class DeviceSchemaScanNodeSerdeTest {
10,
false,
false,
- null);
+ null,
+ SchemaConstant.ALL_MATCH_SCOPE);
IdentitySinkNode sinkNode =
new IdentitySinkNode(
new PlanNodeId("sink"),
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/NodeManagementMemoryMergeNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/NodeManagementMemoryMergeNodeSerdeTest.java
index c55cc9cf4d3..bf1aaaef0ae 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/NodeManagementMemoryMergeNodeSerdeTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/NodeManagementMemoryMergeNodeSerdeTest.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TSchemaNode;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.commons.schema.node.MNodeType;
import org.apache.iotdb.db.queryengine.common.FragmentInstanceId;
import org.apache.iotdb.db.queryengine.common.PlanFragmentId;
@@ -95,7 +96,10 @@ public class NodeManagementMemoryMergeNodeSerdeTest {
ExchangeNode exchangeNode = new ExchangeNode(new PlanNodeId("exchange"));
NodePathsSchemaScanNode childPathsSchemaScanNode =
new NodePathsSchemaScanNode(
- new PlanNodeId("NodePathsScan"), new PartialPath("root.ln"), -1);
+ new PlanNodeId("NodePathsScan"),
+ new PartialPath("root.ln"),
+ -1,
+ SchemaConstant.ALL_MATCH_SCOPE);
IdentitySinkNode sinkNode =
new IdentitySinkNode(
new PlanNodeId("sink"),
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/PathsUsingTemplateScanNodeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/PathsUsingTemplateScanNodeTest.java
index 4ebbdd2b866..1dd2572231f 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/PathsUsingTemplateScanNodeTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/PathsUsingTemplateScanNodeTest.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.db.queryengine.plan.plan.node.metadata.read;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.schema.SchemaConstant;
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.metedata.read.PathsUsingTemplateScanNode;
@@ -39,7 +40,10 @@ public class PathsUsingTemplateScanNodeTest {
patternTree.appendPathPattern(new PartialPath("root.sg.**.*"));
PathsUsingTemplateScanNode pathsUsingTemplateScanNode =
new PathsUsingTemplateScanNode(
- new PlanNodeId("0"), Collections.singletonList(new PartialPath("root.sg")), 1);
+ new PlanNodeId("0"),
+ Collections.singletonList(new PartialPath("root.sg")),
+ 1,
+ SchemaConstant.ALL_MATCH_SCOPE);
ByteBuffer byteBuffer = ByteBuffer.allocate(1024 * 1024);
pathsUsingTemplateScanNode.serialize(byteBuffer);
byteBuffer.flip();
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/SchemaCountNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/SchemaCountNodeSerdeTest.java
index c3a51c4cd8f..524e1daa8fc 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/SchemaCountNodeSerdeTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/SchemaCountNodeSerdeTest.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.plan.plan.node.metadata.read;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.common.FragmentInstanceId;
import org.apache.iotdb.db.queryengine.common.PlanFragmentId;
import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelLocation;
@@ -47,7 +48,10 @@ public class SchemaCountNodeSerdeTest {
ExchangeNode exchangeNode = new ExchangeNode(new PlanNodeId("exchange"));
DevicesCountNode devicesCountNode =
new DevicesCountNode(
- new PlanNodeId("devicesCount"), new PartialPath("root.sg.device0"), true);
+ new PlanNodeId("devicesCount"),
+ new PartialPath("root.sg.device0"),
+ true,
+ SchemaConstant.ALL_MATCH_SCOPE);
IdentitySinkNode sinkNode =
new IdentitySinkNode(
new PlanNodeId("sink"),
@@ -82,7 +86,8 @@ public class SchemaCountNodeSerdeTest {
true,
10,
null,
- Collections.emptyMap());
+ Collections.emptyMap(),
+ SchemaConstant.ALL_MATCH_SCOPE);
IdentitySinkNode sinkNode =
new IdentitySinkNode(
new PlanNodeId("sink"),
@@ -116,7 +121,8 @@ public class SchemaCountNodeSerdeTest {
new PartialPath("root.sg.device0"),
true,
null,
- Collections.emptyMap());
+ Collections.emptyMap(),
+ SchemaConstant.ALL_MATCH_SCOPE);
IdentitySinkNode sinkNode =
new IdentitySinkNode(
new PlanNodeId("sink"),
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
index b6ac654f9e3..20e02eb2a00 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/plan/node/metadata/read/TimeSeriesSchemaScanNodeSerdeTest.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.queryengine.plan.plan.node.metadata.read;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.schema.SchemaConstant;
import org.apache.iotdb.db.queryengine.common.FragmentInstanceId;
import org.apache.iotdb.db.queryengine.common.PlanFragmentId;
import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelLocation;
@@ -56,7 +57,8 @@ public class TimeSeriesSchemaScanNodeSerdeTest {
0,
false,
false,
- Collections.emptyMap());
+ Collections.emptyMap(),
+ SchemaConstant.ALL_MATCH_SCOPE);
IdentitySinkNode sinkNode =
new IdentitySinkNode(
new PlanNodeId("sink"),
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java
index d91246b4374..0cf17f8c99d 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/PathPatternTree.java
@@ -318,6 +318,15 @@ public class PathPatternTree {
root.serialize(buffer);
}
+ public ByteBuffer serialize() throws IOException {
+ PublicBAOS baos = new PublicBAOS();
+ serialize(baos);
+ ByteBuffer serializedPatternTree = ByteBuffer.allocate(baos.size());
+ serializedPatternTree.put(baos.getBuf(), 0, baos.size());
+ serializedPatternTree.flip();
+ return serializedPatternTree;
+ }
+
public static PathPatternTree deserialize(ByteBuffer buffer) {
PathPatternNode<Void, VoidSerializer> root =
PathPatternNode.deserializeNode(buffer, VoidSerializer.getInstance());
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/FAFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/FAFactory.java
index ea6c2d02f80..d4da9d65810 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/FAFactory.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/path/fa/FAFactory.java
@@ -45,7 +45,7 @@ public class FAFactory {
.build(
builder -> {
if (builder.getPatternTree() != null) {
- if (builder.getPatternTree().equals(SchemaConstant.ALL_MATCH_PATTERN_TREE)) {
+ if (builder.getPatternTree().equals(SchemaConstant.ALL_MATCH_SCOPE)) {
// always return the same instance for root.**
return SchemaConstant.ALL_MATCH_DFA;
}
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/SchemaConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/SchemaConstant.java
index 7717b1c679b..d7b40a37d84 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/SchemaConstant.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/SchemaConstant.java
@@ -23,6 +23,9 @@ import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PathPatternTree;
import org.apache.iotdb.commons.path.fa.dfa.PatternDFA;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
public class SchemaConstant {
private SchemaConstant() {
@@ -54,17 +57,22 @@ public class SchemaConstant {
public static final String MTREE_SNAPSHOT = "mtree.snapshot";
public static final String MTREE_SNAPSHOT_TMP = "mtree.snapshot.tmp";
public static final String SYSTEM_DATABASE = "root.__system";
-
public static final String[] ALL_RESULT_NODES = new String[] {"root", "**"};
public static final PartialPath ALL_MATCH_PATTERN = new PartialPath(ALL_RESULT_NODES);
- public static final PathPatternTree ALL_MATCH_PATTERN_TREE = new PathPatternTree();
+ public static final PatternDFA ALL_MATCH_DFA = new PatternDFA(ALL_MATCH_PATTERN, false);
+ public static final PathPatternTree ALL_MATCH_SCOPE = new PathPatternTree();
+ public static final ByteBuffer ALL_MATCH_SCOPE_BINARY;
static {
- ALL_MATCH_PATTERN_TREE.appendPathPattern(ALL_MATCH_PATTERN);
- ALL_MATCH_PATTERN_TREE.constructTree();
+ ALL_MATCH_SCOPE.appendPathPattern(ALL_MATCH_PATTERN);
+ ALL_MATCH_SCOPE.constructTree();
+ try {
+ ALL_MATCH_SCOPE_BINARY = ALL_MATCH_SCOPE.serialize();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
}
- public static final PatternDFA ALL_MATCH_DFA = new PatternDFA(ALL_MATCH_PATTERN, false);
public static final PartialPath SYSTEM_DATABASE_PATTERN =
new PartialPath(SYSTEM_DATABASE.split("\\."));
diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/path/PathPatternTreeTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/path/PathPatternTreeTest.java
index 1566a82f75a..989bbf94698 100644
--- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/path/PathPatternTreeTest.java
+++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/path/PathPatternTreeTest.java
@@ -20,7 +20,6 @@
package org.apache.iotdb.commons.path;
import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.tsfile.utils.PublicBAOS;
import org.junit.Assert;
import org.junit.Test;
@@ -261,11 +260,7 @@ public class PathPatternTreeTest {
compressedDevicePaths.stream().sorted().collect(Collectors.toList()),
patternTree.getAllDevicePaths().stream().sorted().collect(Collectors.toList()));
- PublicBAOS outputStream = new PublicBAOS();
- resultPatternTree.serialize(outputStream);
- ByteBuffer buffer = ByteBuffer.allocate(outputStream.size());
- buffer.put(outputStream.getBuf(), 0, outputStream.size());
- buffer.flip();
+ ByteBuffer buffer = resultPatternTree.serialize();
PathPatternTree tmpPathPatternTree = PathPatternTree.deserialize(buffer);
Assert.assertEquals(resultPatternTree, tmpPathPatternTree);
}
diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
index dff1a0736ed..472210e063c 100644
--- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
@@ -212,6 +212,7 @@ struct TSchemaPartitionTableResp {
struct TSchemaNodeManagementReq {
1: required binary pathPatternTree
2: optional i32 level
+ 3: optional binary scopePatternTree
}
struct TSchemaNodeManagementResp {
@@ -544,6 +545,11 @@ struct TDatabaseInfo {
11: required i32 maxDataRegionNum
}
+struct TGetDatabaseReq{
+ 1: required list<string> databasePathPattern
+ 2: required binary scopePatternTree
+}
+
struct TShowDatabaseResp {
1: required common.TSStatus status
// map<DatabaseName, TDatabaseInfo>
@@ -611,6 +617,11 @@ struct TSetSchemaTemplateReq {
3: required string path
}
+struct TGetPathsSetTemplatesReq {
+ 1: required string templateName
+ 2: required binary scopePatternTree
+}
+
struct TGetPathsSetTemplatesResp {
1: required common.TSStatus status
2: optional list<string> pathList
@@ -908,10 +919,10 @@ service IConfigNodeRPCService {
common.TSStatus setTimePartitionInterval(TSetTimePartitionIntervalReq req)
/** Count the matched Databases */
- TCountDatabaseResp countMatchedDatabases(list<string> DatabasePathPattern)
+ TCountDatabaseResp countMatchedDatabases(TGetDatabaseReq req)
/** Get the matched Databases' TDatabaseSchema */
- TDatabaseSchemaResp getMatchedDatabaseSchemas(list<string> DatabasePathPattern)
+ TDatabaseSchemaResp getMatchedDatabaseSchemas(TGetDatabaseReq req)
// ======================================================
// SchemaPartition
@@ -1215,7 +1226,7 @@ service IConfigNodeRPCService {
TShowConfigNodesResp showConfigNodes()
/** Show cluster Databases' information */
- TShowDatabaseResp showDatabase(list<string> databasePathPattern)
+ TShowDatabaseResp showDatabase(TGetDatabaseReq req)
/**
* Show the matched cluster Regions' information
@@ -1257,7 +1268,7 @@ service IConfigNodeRPCService {
/**
* Get paths setting given schema template
*/
- TGetPathsSetTemplatesResp getPathsSetTemplate(string req)
+ TGetPathsSetTemplatesResp getPathsSetTemplate(TGetPathsSetTemplatesReq req)
/**
* Deactivate schema template from paths matched by given pattern tree in cluster
diff --git a/pom.xml b/pom.xml
index 6f95b5b89db..53e77c56fce 100644
--- a/pom.xml
+++ b/pom.xml
@@ -79,6 +79,7 @@
</mailingList>
</mailingLists>
<modules>
+ <module>integration-test</module>
<module>iotdb-api</module>
<module>iotdb-client</module>
<module>iotdb-connector</module>
@@ -1292,9 +1293,9 @@
<configuration>
<artifactItems>
<artifactItem>
- <groupId>org.apache.iotdb</groupId>
+ <groupId>org.apache.iotdb.tools</groupId>
<artifactId>iotdb-tools-thrift</artifactId>
- <version>${project.version}</version>
+ <version>${iotdb-tools-thrift.version}</version>
<classifier>${os.classifier}</classifier>
<type>zip</type>
<overWrite>true</overWrite>
@@ -1355,11 +1356,10 @@
<dependencies>
<!-- This ensures the iotdb-tools-thift module is build first -->
<dependency>
- <groupId>org.apache.iotdb</groupId>
+ <groupId>org.apache.iotdb.tools</groupId>
<artifactId>iotdb-tools-thrift</artifactId>
- <version>1.3.0-SNAPSHOT</version>
+ <version>${iotdb-tools-thrift.version}</version>
<type>pom</type>
- <scope>provided</scope>
</dependency>
</dependencies>
</profile>