You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hu...@apache.org on 2022/09/07 10:37:20 UTC

[iotdb] 04/07: remove SymbolAllocator

This is an automated email from the ASF dual-hosted git repository.

hui pushed a commit to branch lmh/TypeProviderOpt
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 9f3e3ccf1b97500ed839545465475afd3d256880
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Wed Sep 7 14:08:43 2022 +0800

    remove SymbolAllocator
---
 .../db/mpp/plan/planner/LogicalPlanBuilder.java    |  5 +-
 .../db/mpp/plan/planner/LogicalPlanVisitor.java    | 29 +++++-----
 .../apache/iotdb/db/mpp/plan/planner/Symbol.java   | 29 ----------
 .../iotdb/db/mpp/plan/planner/SymbolAllocator.java | 66 ----------------------
 .../mpp/plan/plan/distribution/LastQueryTest.java  |  3 +-
 5 files changed, 16 insertions(+), 116 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java
index d2ead1276d..afa7e34806 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanBuilder.java
@@ -101,11 +101,8 @@ public class LogicalPlanBuilder {
 
   private final MPPQueryContext context;
 
-  private final SymbolAllocator symbolAllocator;
-
-  public LogicalPlanBuilder(MPPQueryContext context, SymbolAllocator symbolAllocator) {
+  public LogicalPlanBuilder(MPPQueryContext context) {
     this.context = context;
-    this.symbolAllocator = symbolAllocator;
   }
 
   public PlanNode getRoot() {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
index b66c3cfcf2..87d59a3e2e 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/LogicalPlanVisitor.java
@@ -78,7 +78,6 @@ import java.util.stream.Collectors;
 public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryContext> {
 
   private final Analysis analysis;
-  private final SymbolAllocator symbolAllocator = new SymbolAllocator();
 
   public LogicalPlanVisitor(Analysis analysis) {
     this.analysis = analysis;
@@ -92,7 +91,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
 
   @Override
   public PlanNode visitQuery(QueryStatement queryStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
 
     if (queryStatement.isLastQuery()) {
       return planBuilder
@@ -106,7 +105,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
     if (queryStatement.isAlignByDevice()) {
       Map<String, PlanNode> deviceToSubPlanMap = new TreeMap<>();
       for (String deviceName : analysis.getDeviceToSourceExpressions().keySet()) {
-        LogicalPlanBuilder subPlanBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+        LogicalPlanBuilder subPlanBuilder = new LogicalPlanBuilder(context);
         subPlanBuilder =
             subPlanBuilder.withNewRoot(
                 visitQueryBody(
@@ -172,7 +171,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
       Expression havingExpression,
       List<Integer> measurementIndexes, // only used in ALIGN BY DEVICE
       MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
 
     // plan data source node
     if (isRawDataSource) {
@@ -484,7 +483,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   @Override
   public PlanNode visitShowTimeSeries(
       ShowTimeSeriesStatement showTimeSeriesStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
 
     // If there is only one region, we can push down the offset and limit operation to
     // source operator.
@@ -520,7 +519,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
         && null != analysis.getDataPartitionInfo()
         && 0 != analysis.getDataPartitionInfo().getDataPartitionMap().size()) {
       PlanNode lastPlanNode =
-          new LogicalPlanBuilder(context, symbolAllocator)
+          new LogicalPlanBuilder(context)
               .planLast(
                   analysis.getSourceExpressions(),
                   analysis.getGlobalTimeFilter(),
@@ -542,7 +541,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   @Override
   public PlanNode visitShowDevices(
       ShowDevicesStatement showDevicesStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
 
     // If there is only one region, we can push down the offset and limit operation to
     // source operator.
@@ -579,7 +578,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   @Override
   public PlanNode visitCountDevices(
       CountDevicesStatement countDevicesStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
     return planBuilder
         .planDevicesCountSource(
             countDevicesStatement.getPathPattern(), countDevicesStatement.isPrefixPath())
@@ -590,7 +589,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   @Override
   public PlanNode visitCountTimeSeries(
       CountTimeSeriesStatement countTimeSeriesStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
     return planBuilder
         .planTimeSeriesCountSource(
             countTimeSeriesStatement.getPathPattern(),
@@ -606,7 +605,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   @Override
   public PlanNode visitCountLevelTimeSeries(
       CountLevelTimeSeriesStatement countLevelTimeSeriesStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
     return planBuilder
         .planLevelTimeSeriesCountSource(
             countLevelTimeSeriesStatement.getPathPattern(),
@@ -621,7 +620,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
 
   @Override
   public PlanNode visitCountNodes(CountNodesStatement countStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
     return planBuilder
         .planNodePathsSchemaSource(countStatement.getPathPattern(), countStatement.getLevel())
         .planSchemaQueryMerge(false)
@@ -711,7 +710,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   @Override
   public PlanNode visitSchemaFetch(
       SchemaFetchStatement schemaFetchStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
     List<String> storageGroupList =
         new ArrayList<>(analysis.getSchemaPartitionInfo().getSchemaPartitionMap().keySet());
     return planBuilder
@@ -726,7 +725,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   @Override
   public PlanNode visitShowChildPaths(
       ShowChildPathsStatement showChildPathsStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
     return planBuilder
         .planNodePathsSchemaSource(showChildPathsStatement.getPartialPath(), -1)
         .planSchemaQueryMerge(false)
@@ -737,7 +736,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   @Override
   public PlanNode visitShowChildNodes(
       ShowChildNodesStatement showChildNodesStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
     return planBuilder
         .planNodePathsSchemaSource(showChildNodesStatement.getPartialPath(), -1)
         .planSchemaQueryMerge(false)
@@ -769,7 +768,7 @@ public class LogicalPlanVisitor extends StatementVisitor<PlanNode, MPPQueryConte
   @Override
   public PlanNode visitShowPathsUsingTemplate(
       ShowPathsUsingTemplateStatement showPathsUsingTemplateStatement, MPPQueryContext context) {
-    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context, symbolAllocator);
+    LogicalPlanBuilder planBuilder = new LogicalPlanBuilder(context);
     planBuilder =
         planBuilder
             .planPathsUsingTemplateSource(analysis.getTemplateSetInfo().left.getId())
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/Symbol.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/Symbol.java
deleted file mode 100644
index ef8e1e2b44..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/Symbol.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.mpp.plan.planner;
-
-public class Symbol {
-
-  private final String name;
-
-  public Symbol(String name) {
-    this.name = name;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/SymbolAllocator.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/SymbolAllocator.java
deleted file mode 100644
index f596800d85..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/SymbolAllocator.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.mpp.plan.planner;
-
-import org.apache.iotdb.db.mpp.plan.expression.Expression;
-import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
-import org.apache.iotdb.db.mpp.plan.expression.multi.FunctionExpression;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class SymbolAllocator {
-
-  private final Map<Symbol, TSDataType> symbols;
-  private int nextId;
-
-  public SymbolAllocator() {
-    symbols = new HashMap<>();
-  }
-
-  public Symbol newSymbol(Expression expression, TSDataType type) {
-    String nameHint = "expr";
-    if (expression instanceof TimeSeriesOperand) {
-      nameHint = "series";
-    } else if (expression instanceof FunctionExpression) {
-      FunctionExpression functionExpression = (FunctionExpression) expression;
-      nameHint = functionExpression.getFunctionName();
-    }
-    return newSymbol(nameHint, type);
-  }
-
-  public Symbol newSymbol(String nameHint, TSDataType type) {
-    nameHint = nameHint.toLowerCase();
-
-    String unique = nameHint;
-
-    Symbol symbol = new Symbol(unique);
-    while (symbols.putIfAbsent(symbol, type) != null) {
-      symbol = new Symbol(unique + "_" + nextId());
-    }
-
-    return symbol;
-  }
-
-  private int nextId() {
-    return nextId++;
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/LastQueryTest.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/LastQueryTest.java
index 31243734aa..b150d8d6f8 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/LastQueryTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/LastQueryTest.java
@@ -27,7 +27,6 @@ import org.apache.iotdb.db.mpp.common.QueryId;
 import org.apache.iotdb.db.mpp.plan.expression.Expression;
 import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
 import org.apache.iotdb.db.mpp.plan.planner.LogicalPlanBuilder;
-import org.apache.iotdb.db.mpp.plan.planner.SymbolAllocator;
 import org.apache.iotdb.db.mpp.plan.planner.distribution.DistributionPlanner;
 import org.apache.iotdb.db.mpp.plan.planner.plan.DistributedQueryPlan;
 import org.apache.iotdb.db.mpp.plan.planner.plan.LogicalQueryPlan;
@@ -198,7 +197,7 @@ public class LastQueryTest {
 
   private LogicalQueryPlan constructLastQuery(List<String> paths, MPPQueryContext context)
       throws IllegalPathException {
-    LogicalPlanBuilder builder = new LogicalPlanBuilder(context, new SymbolAllocator());
+    LogicalPlanBuilder builder = new LogicalPlanBuilder(context);
     Set<Expression> expressions = new HashSet<>();
     for (String path : paths) {
       expressions.add(new TimeSeriesOperand(new MeasurementPath(path)));