JackieTien97 commented on code in PR #12446:
URL: https://github.com/apache/iotdb/pull/12446#discussion_r1595030678


##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/schema/ISchemaFetcher.java:
##########
@@ -47,6 +47,22 @@ public interface ISchemaFetcher {
   ISchemaTree fetchSchema(
       PathPatternTree patternTree, boolean withTemplate, MPPQueryContext 
context);
 
+  /**
+   * TODO need to be implemented in schema engine
+   *
+   * <p>Fetch all the schema by the given patternTree in device level
+   *
+   * @return schemaTree without measurement nodes
+   */
+  default ISchemaTree fetchSchemaInDeviceLevel(
+      PathPatternTree patternTree, MPPQueryContext context) {
+    ISchemaTree schemaTree = fetchSchema(patternTree, false, context);

Review Comment:
   at least should call `fetchSchemaWithTags`?



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDevicesStatement.java:
##########
@@ -38,12 +40,23 @@ public class ShowDevicesStatement extends ShowStatement {
   private final PartialPath pathPattern;
   private boolean hasSgCol;
   private SchemaFilter schemaFilter;
+  private WhereCondition timeCondition;
+  private Map<PartialPath, Boolean> devicePathToAlignedStatus;

Review Comment:
   put this into `Analysis`



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/TimeseriesSchemaInfo.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.queryengine.common;
+
+import 
org.apache.iotdb.db.queryengine.common.schematree.IMeasurementSchemaInfo;
+import org.apache.iotdb.db.schemaengine.schemaregion.utils.MetaUtils;
+
+import org.apache.tsfile.utils.Pair;
+import org.apache.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+import static 
org.apache.iotdb.db.queryengine.execution.operator.schema.source.TimeSeriesSchemaSource.mapToString;
+
+public class TimeseriesSchemaInfo {
+  boolean isAligned;
+  String dataType;
+  String encoding;
+  String compression;
+  String tags;
+
+  // TODO: Currently we can't get attributes from fetchSchema in query
+  // String attributes;
+  String deadband;
+  String deadbandParameters;

Review Comment:
   Add `private final` for these fields and if you want to get them, add getter 
for each of this fields.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/DistributionPlanContext.java:
##########
@@ -38,6 +38,8 @@ public class DistributionPlanContext {
   // DataRegions
   protected boolean queryMultiRegion;
 
+  private boolean isAllDeviceOnlyInOneRegion;

Review Comment:
   no need to add a new field, reuse the `oneSeriesInMultiRegion` field



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java:
##########
@@ -2796,12 +2798,49 @@ public Analysis visitShowTimeSeries(
 
     PathPatternTree patternTree = new PathPatternTree();
     patternTree.appendPathPattern(showTimeSeriesStatement.getPathPattern());
-    SchemaPartition schemaPartitionInfo = 
partitionFetcher.getSchemaPartition(patternTree);
-    analysis.setSchemaPartitionInfo(schemaPartitionInfo);
 
-    Map<Integer, Template> templateMap =
-        
schemaFetcher.checkAllRelatedTemplate(showTimeSeriesStatement.getPathPattern());
-    analysis.setRelatedTemplateInfo(templateMap);
+    if (showTimeSeriesStatement.hasTimeCondition()) {
+      // If there is time condition in SHOW TIMESERIES, we need to scan the 
raw data
+      WhereCondition timeCondition = 
showTimeSeriesStatement.getTimeCondition();
+      analyzeGlobalTimeConditionInShowMetaData(timeCondition, analysis);
+      context.generateGlobalTimeFilter(analysis);
+
+      ISchemaTree schemaTree = schemaFetcher.fetchSchema(patternTree, false, 
context);
+      if (schemaTree.isEmpty()) {
+        analysis.setFinishQueryAfterAnalyze(true);
+        return analysis;
+      }
+      removeLogicViewMeasurement(schemaTree);
+
+      Map<PartialPath, TimeseriesSchemaInfo> timeseriesToSchemaInfo = new 
HashMap<>();
+      List<DeviceSchemaInfo> deviceSchemaInfoList = 
schemaTree.getMatchedDevices(ALL_MATCH_PATTERN);
+      Set<String> deviceSet = new HashSet<>();
+      for (DeviceSchemaInfo deviceSchemaInfo : deviceSchemaInfoList) {
+        boolean isAligned = deviceSchemaInfo.isAligned();
+        PartialPath devicePath = deviceSchemaInfo.getDevicePath();
+        deviceSet.add(devicePath.getFullPath());
+        for (IMeasurementSchemaInfo measurementSchemaInfo :
+            deviceSchemaInfo.getMeasurementSchemaInfoList()) {
+          MeasurementPath measurementPath =

Review Comment:
   Why aligned device still use `MeasurementPath`?



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/source/RegionScanNode.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.queryengine.plan.planner.plan.node.source;
+
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+public abstract class RegionScanNode extends SourceNode {
+
+  protected TRegionReplicaSet regionReplicaSet;
+  protected boolean outputCount = false;
+
+  protected RegionScanNode(PlanNodeId id) {
+    super(id);
+  }
+
+  @Override
+  public void setRegionReplicaSet(TRegionReplicaSet regionReplicaSet) {
+    this.regionReplicaSet = regionReplicaSet;
+  }
+
+  @Override
+  public TRegionReplicaSet getRegionReplicaSet() {
+    return regionReplicaSet;
+  }
+
+  public boolean isOutputCount() {
+    return outputCount;
+  }
+
+  public void setOutputCount(boolean outputCount) {
+    this.outputCount = outputCount;
+  }
+
+  public abstract List<PartialPath> getDevicePaths();
+
+  public abstract void setDevicePaths(Set<PartialPath> devicePaths);

Review Comment:
   ```suggestion
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java:
##########
@@ -756,6 +763,13 @@ public Statement visitCountDevices(CountDevicesContext 
ctx) {
     } else {
       path = new PartialPath(SqlConstant.getSingleRootArray());
     }
+    if (ctx.timeConditionClause() != null) {
+      WhereCondition timeCondition = 
parseWhereClause(ctx.timeConditionClause().whereClause());
+      ShowDevicesStatement statement = new ShowDevicesStatement(path);

Review Comment:
   use `CountDeviceStatement`



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowDevicesStatement.java:
##########
@@ -38,12 +40,23 @@ public class ShowDevicesStatement extends ShowStatement {
   private final PartialPath pathPattern;
   private boolean hasSgCol;
   private SchemaFilter schemaFilter;
+  private WhereCondition timeCondition;
+  private Map<PartialPath, Boolean> devicePathToAlignedStatus;
+  boolean outputCount = false;

Review Comment:
   ```suggestion
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java:
##########
@@ -719,6 +751,43 @@ private List<PlanNode> processRawSeriesScan(
     return Collections.singletonList(parent);
   }
 
+  private List<PlanNode> splitRegionScanNodeByRegion(
+      RegionScanNode node, DistributionPlanContext context) {
+    List<PlanNode> planNodesForRegions = new ArrayList<>();
+    Map<TRegionReplicaSet, Set<PartialPath>> devicesInRegion = new HashMap<>();
+    List<PartialPath> devicesList = node.getDevicePaths();
+    boolean isAllDeviceOnlyInOneRegion = true;
+
+    for (PartialPath device : devicesList) {
+      List<TRegionReplicaSet> dataDistribution =
+          analysis.getPartitionInfoByDevice(device, 
context.getPartitionTimeFilter());
+      isAllDeviceOnlyInOneRegion = isAllDeviceOnlyInOneRegion && 
dataDistribution.size() == 1;
+      for (TRegionReplicaSet dataRegion : dataDistribution) {
+        devicesInRegion.computeIfAbsent(dataRegion, k -> new 
HashSet<>()).add(device);
+      }
+    }
+
+    context.setAllDeviceOnlyInOneRegion(isAllDeviceOnlyInOneRegion);
+
+    // If there is only one region, return directly
+    if (devicesInRegion.size() == 1) {
+      node.setRegionReplicaSet(devicesInRegion.keySet().iterator().next());
+      planNodesForRegions.add(node);
+      return planNodesForRegions;
+    }
+
+    for (Map.Entry<TRegionReplicaSet, Set<PartialPath>> entry : 
devicesInRegion.entrySet()) {
+      RegionScanNode split = (RegionScanNode) node.clone();
+      split.setPlanNodeId(context.queryContext.getQueryId().genPlanNodeId());
+      split.setRegionReplicaSet(entry.getKey());
+      split.setDevicePaths(entry.getValue());
+      split.setOutputCount(isAllDeviceOnlyInOneRegion);
+      planNodesForRegions.add(split);
+    }

Review Comment:
   Since you've got all the devices previously, you can directly group them 
instead of this ineffective method.



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java:
##########
@@ -769,6 +783,13 @@ public Statement 
visitCountTimeseries(CountTimeseriesContext ctx) {
     } else {
       path = new PartialPath(SqlConstant.getSingleRootArray());
     }
+    if (ctx.timeConditionClause() != null) {
+      WhereCondition timeCondition = 
parseWhereClause(ctx.timeConditionClause().whereClause());
+      ShowTimeSeriesStatement timeseriesStatement = new 
ShowTimeSeriesStatement(path, false);

Review Comment:
   use `CountTimeSeriesStatement`



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTimeSeriesStatement.java:
##########
@@ -37,11 +40,12 @@
 public class ShowTimeSeriesStatement extends ShowStatement {
 
   private final PartialPath pathPattern;
-
   private SchemaFilter schemaFilter;
-
   // if is true, the result will be sorted according to the inserting 
frequency of the time series
   private final boolean orderByHeat;
+  private WhereCondition timeCondition;
+  private Map<PartialPath, TimeseriesSchemaInfo> timeseriesToSchemas;

Review Comment:
   put this into `Analysis`



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/ShowTimeSeriesStatement.java:
##########
@@ -37,11 +40,12 @@
 public class ShowTimeSeriesStatement extends ShowStatement {
 
   private final PartialPath pathPattern;
-
   private SchemaFilter schemaFilter;
-
   // if is true, the result will be sorted according to the inserting 
frequency of the time series
   private final boolean orderByHeat;
+  private WhereCondition timeCondition;
+  private Map<PartialPath, TimeseriesSchemaInfo> timeseriesToSchemas;
+  boolean outputCount = false;

Review Comment:
   ```suggestion
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java:
##########
@@ -709,6 +713,34 @@ public List<PlanNode> visitAlignedLastQueryScan(
     return processRawSeriesScan(node, context, mergeNode);
   }
 
+  List<PlanNode> processRegionScan(RegionScanNode node, 
DistributionPlanContext context) {

Review Comment:
   ```suggestion
     private List<PlanNode> processRegionScan(RegionScanNode node, 
DistributionPlanContext context) {
   ```



##########
iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java:
##########
@@ -2796,12 +2798,49 @@ public Analysis visitShowTimeSeries(
 
     PathPatternTree patternTree = new PathPatternTree();
     patternTree.appendPathPattern(showTimeSeriesStatement.getPathPattern());
-    SchemaPartition schemaPartitionInfo = 
partitionFetcher.getSchemaPartition(patternTree);
-    analysis.setSchemaPartitionInfo(schemaPartitionInfo);
 
-    Map<Integer, Template> templateMap =
-        
schemaFetcher.checkAllRelatedTemplate(showTimeSeriesStatement.getPathPattern());
-    analysis.setRelatedTemplateInfo(templateMap);
+    if (showTimeSeriesStatement.hasTimeCondition()) {
+      // If there is time condition in SHOW TIMESERIES, we need to scan the 
raw data
+      WhereCondition timeCondition = 
showTimeSeriesStatement.getTimeCondition();
+      analyzeGlobalTimeConditionInShowMetaData(timeCondition, analysis);
+      context.generateGlobalTimeFilter(analysis);
+
+      ISchemaTree schemaTree = schemaFetcher.fetchSchema(patternTree, false, 
context);

Review Comment:
   Why use `fetchSchema`, shoud at least be `fetchSchemaWithTags`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to