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/10/13 10:57:32 UTC

[iotdb] branch lmh/mppSelectInto updated (fb63d17c6b -> 66a200c55b)

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

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


    from fb63d17c6b tmp save (analyzer for SELECT INTO)
     new fa3290a015 finish analyzer
     new 4904bea0ad refactor analyzer
     new 9113b516a6 add header
     new 66a200c55b fix bugs

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../db/mpp/common/header/ColumnHeaderConstant.java |  19 ++
 .../db/mpp/common/header/DatasetHeaderFactory.java |   6 +
 .../apache/iotdb/db/mpp/plan/analyze/Analysis.java |  31 +--
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  | 267 ++++++---------------
 .../iotdb/db/mpp/plan/analyze/SelectIntoUtils.java | 105 ++++++++
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       |   2 +-
 .../parameter/IntoDeviceMeasurementDescriptor.java |  62 +++++
 .../planner/plan/parameter/IntoPathDescriptor.java |  60 +++++
 .../plan/statement/component/IntoComponent.java    | 198 ++++++++++++++-
 .../tsfile/common/constant/TsFileConstant.java     |   2 +-
 10 files changed, 535 insertions(+), 217 deletions(-)
 create mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SelectIntoUtils.java
 create mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoDeviceMeasurementDescriptor.java
 create mode 100644 server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoPathDescriptor.java


[iotdb] 03/04: add header

Posted by hu...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9113b516a6d306049e98c6d5bafb5028901ee111
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Thu Oct 13 16:39:37 2022 +0800

    add header
---
 .../db/mpp/common/header/ColumnHeaderConstant.java    | 19 +++++++++++++++++++
 .../db/mpp/common/header/DatasetHeaderFactory.java    |  6 ++++++
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java     |  6 ++++++
 3 files changed, 31 insertions(+)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java
index d91eb57ca7..4b2738b8ad 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java
@@ -115,6 +115,12 @@ public class ColumnHeaderConstant {
   public static final String COLUMN_PIPE_STATUS = "status";
   public static final String COLUMN_PIPE_MESSAGE = "message";
 
+  // column names for select into
+  public static final String COLUMN_SOURCE_DEVICE = "source device";
+  public static final String COLUMN_SOURCE_COLUMN = "source column";
+  public static final String COLUMN_TARGET_TIMESERIES = "target timeseries";
+  public static final String COLUMN_WRITTEN = "written";
+
   public static final List<ColumnHeader> lastQueryColumnHeaders =
       ImmutableList.of(
           new ColumnHeader(COLUMN_TIMESERIES, TSDataType.TEXT),
@@ -275,4 +281,17 @@ public class ColumnHeaderConstant {
           new ColumnHeader(COLUMN_PIPE_REMOTE, TSDataType.TEXT),
           new ColumnHeader(COLUMN_PIPE_STATUS, TSDataType.TEXT),
           new ColumnHeader(COLUMN_PIPE_MESSAGE, TSDataType.TEXT));
+
+  public static final List<ColumnHeader> selectIntoColumnHeaders =
+      ImmutableList.of(
+          new ColumnHeader(COLUMN_SOURCE_COLUMN, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_TARGET_TIMESERIES, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_WRITTEN, TSDataType.INT32));
+
+  public static final List<ColumnHeader> selectIntoAlignByDeviceColumnHeaders =
+      ImmutableList.of(
+          new ColumnHeader(COLUMN_SOURCE_DEVICE, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_SOURCE_COLUMN, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_TARGET_TIMESERIES, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_WRITTEN, TSDataType.INT32));
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java
index c8ccb8a1e8..9698d4370a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java
@@ -128,4 +128,10 @@ public class DatasetHeaderFactory {
   public static DatasetHeader getShowPipeHeader() {
     return new DatasetHeader(ColumnHeaderConstant.showPipeColumnHeaders, true);
   }
+
+  public static DatasetHeader getSelectIntoHeader(boolean isAlignByDevice) {
+    return isAlignByDevice
+        ? new DatasetHeader(ColumnHeaderConstant.selectIntoAlignByDeviceColumnHeaders, true)
+        : new DatasetHeader(ColumnHeaderConstant.selectIntoColumnHeaders, true);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
index dcd90bf3fe..8837208abc 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
@@ -918,6 +918,12 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
       Analysis analysis,
       QueryStatement queryStatement,
       List<Pair<Expression, String>> outputExpressions) {
+    if (queryStatement.isSelectInto()) {
+      analysis.setRespDatasetHeader(
+          DatasetHeaderFactory.getSelectIntoHeader(queryStatement.isAlignByDevice()));
+      return;
+    }
+
     boolean isIgnoreTimestamp =
         queryStatement.isAggregationQuery() && !queryStatement.isGroupByTime();
     List<ColumnHeader> columnHeaders = new ArrayList<>();


[iotdb] 04/04: fix bugs

Posted by hu...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 66a200c55b30c35513ef87975a593f661acfac0b
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Thu Oct 13 18:56:55 2022 +0800

    fix bugs
---
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  | 11 +++---
 .../iotdb/db/mpp/plan/parser/ASTVisitor.java       |  2 +-
 .../parameter/IntoDeviceMeasurementDescriptor.java | 39 ++++++++++++----------
 .../plan/statement/component/IntoComponent.java    |  5 +--
 .../tsfile/common/constant/TsFileConstant.java     |  2 +-
 5 files changed, 34 insertions(+), 25 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
index 8837208abc..30d945044a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
@@ -201,6 +201,10 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
       logger.info("[EndFetchSchema]");
       // If there is no leaf node in the schema tree, the query should be completed immediately
       if (schemaTree.isEmpty()) {
+        if (queryStatement.isSelectInto()) {
+          analysis.setRespDatasetHeader(
+              DatasetHeaderFactory.getSelectIntoHeader(queryStatement.isAlignByDevice()));
+        }
         if (queryStatement.isLastQuery()) {
           analysis.setRespDatasetHeader(DatasetHeaderFactory.getLastQueryHeader());
         }
@@ -1026,7 +1030,6 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
       PartialPath deviceTemplate = intoDeviceMeasurementIterator.getDeviceTemplate();
       boolean isAlignedDevice = intoDeviceMeasurementIterator.isAlignedDevice();
       PartialPath targetDevice = constructTargetDevice(sourceDevice, deviceTemplate);
-      intoDeviceMeasurementDescriptor.specifyTargetDevice(sourceDevice, targetDevice);
       intoDeviceMeasurementDescriptor.specifyDeviceAlignment(targetDevice, isAlignedDevice);
 
       for (Expression sourceColumn : sourceColumns) {
@@ -1037,10 +1040,10 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
               constructTargetMeasurement(
                   sourceDevice.concatNode(sourceColumn.toString()), measurementTemplate);
         } else {
-          targetMeasurement = sourceColumn.toString();
+          targetMeasurement = measurementTemplate;
         }
-        intoDeviceMeasurementDescriptor.specifyTargetMeasurement(
-            targetDevice, sourceColumn.toString(), targetMeasurement);
+        intoDeviceMeasurementDescriptor.specifyTargetDeviceMeasurement(
+            sourceDevice, targetDevice, sourceColumn.toString(), targetMeasurement);
         intoDeviceMeasurementIterator.nextMeasurement();
       }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
index c4f9ddf867..dc28cae4e2 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
@@ -1645,7 +1645,7 @@ public class ASTVisitor extends IoTDBSqlParserBaseVisitor<Statement> {
   private void checkNodeName(String src) {
     // node name could start with * and end with *
     if (!TsFileConstant.NODE_NAME_PATTERN.matcher(src).matches()) {
-      throw new SQLParserException(
+      throw new SemanticException(
           String.format(
               "%s is illegal, unquoted node name can only consist of digits, characters and underscore, or start or end with wildcard",
               src));
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoDeviceMeasurementDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoDeviceMeasurementDescriptor.java
index c0a0ec120e..abfdca58a5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoDeviceMeasurementDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoDeviceMeasurementDescriptor.java
@@ -21,37 +21,42 @@ package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.tsfile.utils.Pair;
 
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.apache.iotdb.db.mpp.plan.statement.component.IntoComponent.DUPLICATE_TARGET_DEVICE_ERROR_MSG;
 import static org.apache.iotdb.db.mpp.plan.statement.component.IntoComponent.DUPLICATE_TARGET_PATH_ERROR_MSG;
 
-public class IntoDeviceMeasurementDescriptor extends IntoPathDescriptor {
+public class IntoDeviceMeasurementDescriptor {
 
-  private final Map<PartialPath, PartialPath> targetDeviceToSourceDeviceMap;
+  private final Map<PartialPath, Map<String, Pair<PartialPath, String>>> targetPathToSourceMap;
+  protected final Map<PartialPath, Boolean> deviceToAlignedMap;
 
   public IntoDeviceMeasurementDescriptor() {
-    super();
-    this.targetDeviceToSourceDeviceMap = new HashMap<>();
+    this.targetPathToSourceMap = new HashMap<>();
+    this.deviceToAlignedMap = new HashMap<>();
   }
 
-  public void specifyTargetDevice(PartialPath sourceDevice, PartialPath targetDevice) {
-    if (targetDeviceToSourceDeviceMap.containsKey(targetDevice)
-        && !targetDeviceToSourceDeviceMap.get(targetDevice).equals(sourceDevice)) {
-      throw new SemanticException(DUPLICATE_TARGET_DEVICE_ERROR_MSG);
-    }
-    targetDeviceToSourceDeviceMap.put(targetDevice, sourceDevice);
-  }
-
-  public void specifyTargetMeasurement(
-      PartialPath targetDevice, String sourceColumn, String targetMeasurement) {
-    Map<String, String> measurementToSourceColumnMap =
+  public void specifyTargetDeviceMeasurement(
+      PartialPath sourceDevice,
+      PartialPath targetDevice,
+      String sourceColumn,
+      String targetMeasurement) {
+    Map<String, Pair<PartialPath, String>> measurementToSourceColumnMap =
         targetPathToSourceMap.computeIfAbsent(targetDevice, key -> new HashMap<>());
     if (measurementToSourceColumnMap.containsKey(targetMeasurement)) {
       throw new SemanticException(DUPLICATE_TARGET_PATH_ERROR_MSG);
     }
-    measurementToSourceColumnMap.put(targetMeasurement, sourceColumn);
+    measurementToSourceColumnMap.put(targetMeasurement, new Pair<>(sourceDevice, sourceColumn));
+  }
+
+  public void specifyDeviceAlignment(PartialPath targetDevice, boolean isAligned) {
+    if (deviceToAlignedMap.containsKey(targetDevice)
+        && deviceToAlignedMap.get(targetDevice) != isAligned) {
+      throw new SemanticException(
+          "select into: alignment property must be the same for the same device.");
+    }
+    deviceToAlignedMap.put(targetDevice, isAligned);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java
index 223e10602f..240a5333e4 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java
@@ -39,8 +39,6 @@ public class IntoComponent extends StatementNode {
       "select into: the number of source devices and the number of target devices should be the same.";
   public static String PATH_NUM_MISMATCH_ERROR_MSG =
       "select into: the number of source columns and the number of target paths should be the same.";
-  public static String DUPLICATE_TARGET_DEVICE_ERROR_MSG =
-      "select into: target devices in into clause should be different.";
   public static String DUPLICATE_TARGET_PATH_ERROR_MSG =
       "select into: target paths in into clause should be different.";
 
@@ -196,6 +194,9 @@ public class IntoComponent extends StatementNode {
     public void nextMeasurement() {
       if (!intoItems.get(deviceIndex).isMeasurementsExistPlaceholder()) {
         measurementIndex++;
+        if (measurementIndex == intoItems.get(deviceIndex).getIntoMeasurements().size()) {
+          measurementIndex = 0;
+        }
       }
     }
   }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java
index 956a34a953..7917c3a1cb 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/constant/TsFileConstant.java
@@ -41,7 +41,7 @@ public class TsFileConstant {
   private static final String IDENTIFIER_MATCHER = "([a-zA-Z0-9_\\u2E80-\\u9FFF]+)";
   public static final Pattern IDENTIFIER_PATTERN = Pattern.compile(IDENTIFIER_MATCHER);
 
-  private static final String NODE_NAME_MATCHER = "(\\*{0,2}[a-zA-Z0-9_\\u2E80-\\u9FFF]+\\*{0,2})";
+  private static final String NODE_NAME_MATCHER = "(\\*{0,2}[a-zA-Z0-9_\\u2E80-\\u9FFF]*\\*{0,2})";
   public static final Pattern NODE_NAME_PATTERN = Pattern.compile(NODE_NAME_MATCHER);
 
   private static final String NODE_NAME_IN_INTO_PATH_MATCHER = "([a-zA-Z0-9_${}\\u2E80-\\u9FFF]+)";


[iotdb] 02/04: refactor analyzer

Posted by hu...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 4904bea0ad14151e73ac4a3ebe10d1432b73728b
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Thu Oct 13 16:03:29 2022 +0800

    refactor analyzer
---
 .../apache/iotdb/db/mpp/plan/analyze/Analysis.java |  31 +-
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  | 398 +++------------------
 .../iotdb/db/mpp/plan/analyze/SelectIntoUtils.java | 105 ++++++
 .../parameter/IntoDeviceMeasurementDescriptor.java |  57 +++
 .../planner/plan/parameter/IntoPathDescriptor.java |  60 ++++
 .../plan/statement/component/IntoComponent.java    | 188 +++++++++-
 6 files changed, 479 insertions(+), 360 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java
index 119f67654a..ea344ca9fe 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/Analysis.java
@@ -24,8 +24,6 @@ import org.apache.iotdb.common.rpc.thrift.TSchemaNode;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.SchemaPartition;
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.path.PatternTreeMap;
-import org.apache.iotdb.db.metadata.path.PatternTreeMapFactory;
 import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.mpp.common.NodeRef;
 import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
@@ -33,6 +31,8 @@ import org.apache.iotdb.db.mpp.common.schematree.ISchemaTree;
 import org.apache.iotdb.db.mpp.plan.expression.Expression;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.FillDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.IntoDeviceMeasurementDescriptor;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.IntoPathDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.OrderByParameter;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -140,12 +140,14 @@ public class Analysis {
   private DatasetHeader respDatasetHeader;
 
   /////////////////////////////////////////////////////////////////////////////////////////////////
-  // SELECT INTO Analysis (used in ALIGN BY DEVICE)
+  // SELECT INTO Analysis
   /////////////////////////////////////////////////////////////////////////////////////////////////
 
-  private PatternTreeMap<String, PatternTreeMapFactory.StringSerializer> intoPathPatternTreeMap;
+  // used in ALIGN BY DEVICE
+  private IntoDeviceMeasurementDescriptor intoDeviceMeasurementDescriptor;
 
-  private Map<PartialPath, Boolean> intoDeviceToAlignedMap;
+  // used in ALIGN BY TIME
+  private IntoPathDescriptor intoPathDescriptor;
 
   /////////////////////////////////////////////////////////////////////////////////////////////////
   // Schema Query Analysis
@@ -414,21 +416,20 @@ public class Analysis {
     this.deviceViewOutputExpressions = deviceViewOutputExpressions;
   }
 
-  public PatternTreeMap<String, PatternTreeMapFactory.StringSerializer>
-      getIntoPathPatternTreeMap() {
-    return intoPathPatternTreeMap;
+  public IntoDeviceMeasurementDescriptor getIntoDeviceMeasurementDescriptor() {
+    return intoDeviceMeasurementDescriptor;
   }
 
-  public void setIntoPathPatternTreeMap(
-      PatternTreeMap<String, PatternTreeMapFactory.StringSerializer> intoPathPatternTreeMap) {
-    this.intoPathPatternTreeMap = intoPathPatternTreeMap;
+  public void setIntoDeviceMeasurementDescriptor(
+      IntoDeviceMeasurementDescriptor intoDeviceMeasurementDescriptor) {
+    this.intoDeviceMeasurementDescriptor = intoDeviceMeasurementDescriptor;
   }
 
-  public Map<PartialPath, Boolean> getIntoDeviceToAlignedMap() {
-    return intoDeviceToAlignedMap;
+  public IntoPathDescriptor getIntoPathDescriptor() {
+    return intoPathDescriptor;
   }
 
-  public void setIntoDeviceToAlignedMap(Map<PartialPath, Boolean> intoDeviceToAlignedMap) {
-    this.intoDeviceToAlignedMap = intoDeviceToAlignedMap;
+  public void setIntoPathDescriptor(IntoPathDescriptor intoPathDescriptor) {
+    this.intoPathDescriptor = intoPathDescriptor;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
index f28f1064fb..dcd90bf3fe 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
@@ -29,7 +29,6 @@ import org.apache.iotdb.commons.partition.SchemaPartition;
 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.path.PatternTreeMap;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
@@ -39,7 +38,6 @@ import org.apache.iotdb.db.exception.metadata.template.TemplateImcompatibeExcept
 import org.apache.iotdb.db.exception.sql.MeasurementNotExistException;
 import org.apache.iotdb.db.exception.sql.SemanticException;
 import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
-import org.apache.iotdb.db.metadata.path.PatternTreeMapFactory;
 import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.mpp.common.MPPQueryContext;
 import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
@@ -54,6 +52,8 @@ import org.apache.iotdb.db.mpp.plan.expression.ExpressionType;
 import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.FillDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.GroupByTimeParameter;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.IntoDeviceMeasurementDescriptor;
+import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.IntoPathDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.parameter.OrderByParameter;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
 import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
@@ -61,7 +61,6 @@ import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
 import org.apache.iotdb.db.mpp.plan.statement.component.FillComponent;
 import org.apache.iotdb.db.mpp.plan.statement.component.GroupByTimeComponent;
 import org.apache.iotdb.db.mpp.plan.statement.component.IntoComponent;
-import org.apache.iotdb.db.mpp.plan.statement.component.IntoItem;
 import org.apache.iotdb.db.mpp.plan.statement.component.Ordering;
 import org.apache.iotdb.db.mpp.plan.statement.component.ResultColumn;
 import org.apache.iotdb.db.mpp.plan.statement.component.SortItem;
@@ -142,16 +141,16 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.TimeZone;
-import java.util.regex.Matcher;
 import java.util.stream.Collectors;
 
 import static com.google.common.base.Preconditions.checkState;
-import static org.apache.iotdb.commons.conf.IoTDBConstant.DOUBLE_COLONS;
-import static org.apache.iotdb.commons.conf.IoTDBConstant.LEVELED_PATH_TEMPLATE_PATTERN;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.ONE_LEVEL_PATH_WILDCARD;
 import static org.apache.iotdb.db.metadata.MetadataConstant.ALL_RESULT_NODES;
 import static org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant.COLUMN_DEVICE;
+import static org.apache.iotdb.db.mpp.plan.analyze.SelectIntoUtils.constructTargetDevice;
+import static org.apache.iotdb.db.mpp.plan.analyze.SelectIntoUtils.constructTargetMeasurement;
+import static org.apache.iotdb.db.mpp.plan.analyze.SelectIntoUtils.constructTargetPath;
 
 /** This visitor is used to analyze each type of Statement and returns the {@link Analysis}. */
 public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext> {
@@ -1000,368 +999,85 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
       QueryStatement queryStatement,
       Set<PartialPath> deviceSet,
       List<Pair<Expression, String>> outputExpressions) {
-    List<Expression> outputColumns =
+    if (!queryStatement.isSelectInto()) {
+      return;
+    }
+
+    List<PartialPath> sourceDevices = new ArrayList<>(deviceSet);
+    List<Expression> sourceColumns =
         outputExpressions.stream()
             .map(Pair::getLeft)
             .collect(Collectors.toCollection(ArrayList::new));
-    boolean isAllRawSeriesQuery = checkIsAllRawSeriesQuery(outputColumns);
 
     IntoComponent intoComponent = queryStatement.getIntoComponent();
-    List<IntoItem> intoItems = intoComponent.getIntoItems();
-
-    List<PartialPath> sourceDevices = new ArrayList<>(deviceSet);
-    List<PartialPath> targetDevices = new ArrayList<>(sourceDevices.size());
-    if (intoComponent.isDeviceExistPlaceholder()) {
-      if (intoItems.size() > 1) {
-        throw new SemanticException("");
-      }
-
-      PartialPath deviceTemplate = intoItems.get(0).getIntoDevice();
-      for (PartialPath sourceDevice : sourceDevices) {
-        targetDevices.add(constructIntoDevice(sourceDevice, deviceTemplate));
-      }
-    } else {
-      if (intoItems.size() != sourceDevices.size()) {
-        throw new SemanticException("");
-      }
-
-      for (IntoItem intoItem : intoItems) {
-        targetDevices.add(intoItem.getIntoDevice());
-      }
-    }
-
-    PatternTreeMap<String, PatternTreeMapFactory.StringSerializer> intoPathPatternTreeMap =
-        PatternTreeMapFactory.getIntoPathPatternTreeMap();
-
-    if (intoComponent.isMeasurementsExistPlaceholder()) {
-      if (!isAllRawSeriesQuery) {
-        throw new SemanticException("");
-      }
-
-      if (intoComponent.isDeviceExistPlaceholder()) {
-        String measurementTemplate = intoItems.get(0).getIntoMeasurements().get(0);
-        for (int i = 0; i < targetDevices.size(); i++) {
-          PartialPath targetDevice = targetDevices.get(i);
-          for (Expression outputColumn : outputColumns) {
-            PartialPath intoPath =
-                targetDevice.concatNode(
-                    constructIntoMeasurement(
-                        sourceDevices.get(i).concatNode(outputColumn.toString()),
-                        measurementTemplate));
-            intoPathPatternTreeMap.append(intoPath, outputColumn.toString());
-            if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
-              throw new SemanticException(
-                  "select into: target paths in into clause should be different.");
-            }
-          }
-        }
-      } else {
-        for (int deviceIndex = 0; deviceIndex < sourceDevices.size(); deviceIndex++) {
-          IntoItem intoItem = intoItems.get(deviceIndex);
-          List<String> intoMeasurements = intoItem.getIntoMeasurements();
-          PartialPath targetDevice = targetDevices.get(deviceIndex);
-
-          if (intoItem.isMeasurementsExistPlaceholder()) {
-            if (intoMeasurements.size() > 1) {
-              throw new SemanticException("");
-            }
-
-            String measurementTemplate = intoMeasurements.get(0);
-            for (Expression outputColumn : outputColumns) {
-              PartialPath intoPath =
-                  targetDevice.concatNode(
-                      constructIntoMeasurement(
-                          sourceDevices.get(deviceIndex).concatNode(outputColumn.toString()),
-                          measurementTemplate));
-              intoPathPatternTreeMap.append(intoPath, outputColumn.toString());
-              if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
-                throw new SemanticException(
-                    "select into: target paths in into clause should be different.");
-              }
-            }
-          } else {
-            if (intoMeasurements.size() != outputColumns.size()) {
-              throw new SemanticException(
-                  "select into: the number of source columns and the number of target paths should be the same.");
-            }
-
-            for (int measurementIndex = 0;
-                measurementIndex < intoMeasurements.size();
-                measurementIndex++) {
-              PartialPath intoPath =
-                  targetDevice.concatNode(intoMeasurements.get(measurementIndex));
-              intoPathPatternTreeMap.append(
-                  intoPath, outputColumns.get(measurementIndex).toString());
-              if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
-                throw new SemanticException(
-                    "select into: target paths in into clause should be different.");
-              }
-            }
-          }
-        }
-      }
-    } else {
-      for (int deviceIndex = 0; deviceIndex < sourceDevices.size(); deviceIndex++) {
-        IntoItem intoItem = intoItems.get(deviceIndex);
-        List<String> intoMeasurements = intoItem.getIntoMeasurements();
-        if (intoMeasurements.size() != outputColumns.size()) {
-          throw new SemanticException(
-              "select into: the number of source columns and the number of target paths should be the same.");
-        }
-
-        PartialPath targetDevice = targetDevices.get(deviceIndex);
-        for (int measurementIndex = 0;
-            measurementIndex < intoMeasurements.size();
-            measurementIndex++) {
-          PartialPath intoPath = targetDevice.concatNode(intoMeasurements.get(measurementIndex));
-          intoPathPatternTreeMap.append(intoPath, outputColumns.get(measurementIndex).toString());
-          if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
-            throw new SemanticException(
-                "select into: target paths in into clause should be different.");
-          }
-        }
-      }
-    }
-
-    if (isAllRawSeriesQuery) {
-      if (intoComponent.isDeviceExistPlaceholder()) {
-        if (intoComponent.isMeasurementsExistPlaceholder()) {
-          String measurementTemplate = intoItems.get(0).getIntoMeasurements().get(0);
-          for (int i = 0; i < targetDevices.size(); i++) {
-            PartialPath targetDevice = targetDevices.get(i);
-            for (Expression outputColumn : outputColumns) {
-              PartialPath intoPath =
-                  targetDevice.concatNode(
-                      constructIntoMeasurement(
-                          sourceDevices.get(i).concatNode(outputColumn.toString()),
-                          measurementTemplate));
-              intoPathPatternTreeMap.append(intoPath, outputColumn.toString());
-              if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
-                throw new SemanticException(
-                    "select into: target paths in into clause should be different.");
-              }
-            }
-          }
+    intoComponent.validate(sourceDevices, sourceColumns);
+
+    IntoDeviceMeasurementDescriptor intoDeviceMeasurementDescriptor =
+        new IntoDeviceMeasurementDescriptor();
+    IntoComponent.IntoDeviceMeasurementIterator intoDeviceMeasurementIterator =
+        intoComponent.getIntoDeviceMeasurementIterator();
+    for (PartialPath sourceDevice : sourceDevices) {
+      PartialPath deviceTemplate = intoDeviceMeasurementIterator.getDeviceTemplate();
+      boolean isAlignedDevice = intoDeviceMeasurementIterator.isAlignedDevice();
+      PartialPath targetDevice = constructTargetDevice(sourceDevice, deviceTemplate);
+      intoDeviceMeasurementDescriptor.specifyTargetDevice(sourceDevice, targetDevice);
+      intoDeviceMeasurementDescriptor.specifyDeviceAlignment(targetDevice, isAlignedDevice);
+
+      for (Expression sourceColumn : sourceColumns) {
+        String measurementTemplate = intoDeviceMeasurementIterator.getMeasurementTemplate();
+        String targetMeasurement;
+        if (sourceColumn instanceof TimeSeriesOperand) {
+          targetMeasurement =
+              constructTargetMeasurement(
+                  sourceDevice.concatNode(sourceColumn.toString()), measurementTemplate);
         } else {
-
+          targetMeasurement = sourceColumn.toString();
         }
-      }
-    } else {
-
-    }
-    analyzeIntoDevices(analysis, intoItems);
-    analysis.setIntoPathPatternTreeMap(intoPathPatternTreeMap);
-  }
-
-  private PartialPath constructIntoDevice(PartialPath sourceDevice, PartialPath deviceTemplate) {
-    String[] sourceNodes = sourceDevice.getNodes();
-    String[] templateNodes = deviceTemplate.getNodes();
-
-    List<String> targetNodes = new ArrayList<>();
-    for (int nodeIndex = 0; nodeIndex < templateNodes.length; nodeIndex++) {
-      String curNode = templateNodes[nodeIndex];
-      if (curNode.equals(DOUBLE_COLONS)) {
-        if (nodeIndex != templateNodes.length - 1) {
-          throw new SemanticException("");
-        }
-        for (; nodeIndex < sourceNodes.length; nodeIndex++) {
-          targetNodes.add(sourceNodes[nodeIndex]);
-        }
-        break;
+        intoDeviceMeasurementDescriptor.specifyTargetMeasurement(
+            targetDevice, sourceColumn.toString(), targetMeasurement);
+        intoDeviceMeasurementIterator.nextMeasurement();
       }
 
-      String resNode = curNode;
-      Matcher matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
-      while (matcher.find()) {
-        String param = matcher.group();
-        int index;
-        try {
-          index = Integer.parseInt(param.substring(2, param.length() - 1).trim());
-        } catch (NumberFormatException e) {
-          throw new SemanticException("select into: the i of ${i} should be an integer.");
-        }
-        if (index < 1 || index >= sourceNodes.length) {
-          throw new SemanticException(
-              "select into: the i of ${i} should be greater than 0 and equal to or less than the length of queried path prefix.");
-        }
-        resNode = matcher.replaceFirst(sourceNodes[index]);
-        matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
-      }
-      targetNodes.add(resNode);
+      intoDeviceMeasurementIterator.nextDevice();
     }
-    return new PartialPath(targetNodes.toArray(new String[0]));
+    analysis.setIntoDeviceMeasurementDescriptor(intoDeviceMeasurementDescriptor);
   }
 
   private void analyzeInto(
       Analysis analysis,
       QueryStatement queryStatement,
       List<Pair<Expression, String>> outputExpressions) {
-    if (queryStatement.getIntoComponent() == null) {
+    if (!queryStatement.isSelectInto()) {
       return;
     }
 
-    List<Expression> outputColumns =
+    List<Expression> sourceColumns =
         outputExpressions.stream()
             .map(Pair::getLeft)
             .collect(Collectors.toCollection(ArrayList::new));
-    boolean isAllRawSeriesQuery = checkIsAllRawSeriesQuery(outputColumns);
 
     IntoComponent intoComponent = queryStatement.getIntoComponent();
-    List<IntoItem> intoItems = intoComponent.getIntoItems();
-
-    List<PartialPath> intoPaths = new ArrayList<>();
-    PatternTreeMap<String, PatternTreeMapFactory.StringSerializer> intoPathPatternTreeMap =
-        PatternTreeMapFactory.getIntoPathPatternTreeMap();
-
-    if (isAllRawSeriesQuery) {
-      List<PartialPath> sourcePaths =
-          outputColumns.stream()
-              .map(expression -> ((TimeSeriesOperand) expression).getPath())
-              .collect(Collectors.toList());
-
-      if (intoComponent.isDeviceExistPlaceholder()) {
-        if (intoComponent.isMeasurementsExistPlaceholder()) {
-          if (intoItems.size() > 1) {
-            throw new SemanticException("");
-          }
-          if (intoItems.get(0).getIntoMeasurements().size() > 1) {
-            throw new SemanticException("");
-          }
-
-          PartialPath deviceTemplate = intoItems.get(0).getIntoDevice();
-          String measurementTemplate = intoItems.get(0).getIntoMeasurements().get(0);
-          for (PartialPath sourcePath : sourcePaths) {
-            intoPaths.add(constructIntoPath(sourcePath, deviceTemplate, measurementTemplate));
-          }
-        } else {
-          int sourcePathIndex = 0;
-          for (IntoItem intoItem : intoItems) {
-            PartialPath deviceTemplate = intoItem.getIntoDevice();
-            List<String> measurementList = intoItem.getIntoMeasurements();
-            for (String measurement : measurementList) {
-              intoPaths.add(
-                  constructIntoPath(sourcePaths.get(sourcePathIndex), deviceTemplate, measurement));
-              sourcePathIndex++;
-            }
-          }
-        }
+    intoComponent.validate(sourceColumns);
+
+    IntoPathDescriptor intoPathDescriptor = new IntoPathDescriptor();
+    IntoComponent.IntoPathIterator intoPathIterator = intoComponent.getIntoPathIterator();
+    for (Expression sourceColumn : sourceColumns) {
+      PartialPath deviceTemplate = intoPathIterator.getDeviceTemplate();
+      String measurementTemplate = intoPathIterator.getMeasurementTemplate();
+      boolean isAlignedDevice = intoPathIterator.isAlignedDevice();
+
+      PartialPath targetPath;
+      if (sourceColumn instanceof TimeSeriesOperand) {
+        PartialPath sourcePath = ((TimeSeriesOperand) sourceColumn).getPath();
+        targetPath = constructTargetPath(sourcePath, deviceTemplate, measurementTemplate);
       } else {
-        if (intoComponent.isMeasurementsExistPlaceholder()) {
-          if (intoItems.size() != outputColumns.size()) {
-            throw new SemanticException(
-                "select into: the number of source columns and the number of target paths should be the same.");
-          }
-
-          for (int i = 0; i < intoItems.size(); i++) {
-            if (intoItems.get(i).getIntoMeasurements().size() != 1) {
-              throw new SemanticException("");
-            }
-
-            PartialPath targetDevice = intoItems.get(i).getIntoDevice();
-            intoPaths.add(
-                targetDevice.concatNode(
-                    constructIntoMeasurement(
-                        sourcePaths.get(i), intoItems.get(0).getIntoMeasurements().get(0))));
-          }
-        } else {
-          intoPaths =
-              intoItems.stream()
-                  .map(IntoItem::getIntoPaths)
-                  .flatMap(List::stream)
-                  .collect(Collectors.toList());
-        }
-      }
-    } else {
-      // disable placeholder
-      for (IntoItem intoItem : intoItems) {
-        if (intoItem.isDeviceExistPlaceholder() || intoItem.isMeasurementsExistPlaceholder()) {
-          throw new SemanticException(
-              "select into: placeholders can only be used in raw timeseries data queries.");
-        }
-      }
-
-      intoPaths =
-          intoItems.stream()
-              .map(IntoItem::getIntoPaths)
-              .flatMap(List::stream)
-              .collect(Collectors.toList());
-    }
-
-    // check quantity consistency
-    if (intoPaths.size() != outputColumns.size()) {
-      throw new SemanticException(
-          "select into: the number of source columns and the number of target paths should be the same.");
-    }
-
-    for (int i = 0; i < intoPaths.size(); i++) {
-      intoPathPatternTreeMap.append(intoPaths.get(i), outputColumns.get(i).toString());
-      if (intoPathPatternTreeMap.getOverlapped(intoPaths.get(i)).size() > 1) {
-        throw new SemanticException(
-            "select into: target paths in into clause should be different.");
-      }
-    }
-
-    analyzeIntoDevices(analysis, intoItems);
-    analysis.setIntoPathPatternTreeMap(intoPathPatternTreeMap);
-  }
-
-  private void analyzeIntoDevices(Analysis analysis, List<IntoItem> intoItems) {
-    Map<PartialPath, Boolean> intoDeviceToAlignedMap = new HashMap<>();
-    for (IntoItem intoItem : intoItems) {
-      PartialPath devicePath = intoItem.getIntoDevice();
-      boolean isAligned = intoItem.isAligned();
-      if (intoDeviceToAlignedMap.containsKey(devicePath)
-          && intoDeviceToAlignedMap.get(devicePath) != isAligned) {
-        throw new SemanticException(
-            String.format(
-                "select into: inconsistent alignment property specified for device '%s'.",
-                devicePath));
-      }
-      intoDeviceToAlignedMap.put(devicePath, isAligned);
-    }
-    analysis.setIntoDeviceToAlignedMap(intoDeviceToAlignedMap);
-  }
-
-  private boolean checkIsAllRawSeriesQuery(List<Expression> expressions) {
-    for (Expression expression : expressions) {
-      if (!(expression instanceof TimeSeriesOperand)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private PartialPath constructIntoPath(
-      PartialPath sourcePath, PartialPath deviceTemplate, String measurementTemplate) {
-    PartialPath targetDevice = constructIntoDevice(sourcePath.getDevicePath(), deviceTemplate);
-    String targetMeasurement = constructIntoMeasurement(sourcePath, measurementTemplate);
-    return targetDevice.concatNode(targetMeasurement);
-  }
-
-  private String constructIntoMeasurement(PartialPath sourcePath, String measurementTemplate) {
-    if (measurementTemplate.equals(DOUBLE_COLONS)) {
-      return sourcePath.getMeasurement();
-    }
-
-    String[] sourceNodes = sourcePath.getNodes();
-    String resNode = measurementTemplate;
-    Matcher matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
-    while (matcher.find()) {
-      String param = matcher.group();
-      int index;
-      try {
-        index = Integer.parseInt(param.substring(2, param.length() - 1).trim());
-      } catch (NumberFormatException e) {
-        throw new SemanticException("select into: the i of ${i} should be an integer.");
-      }
-      if (index < 1 || index >= sourceNodes.length) {
-        throw new SemanticException(
-            "select into: the i of ${i} should be greater than 0 and equal to or less than the length of queried path prefix.");
+        targetPath = deviceTemplate.concatNode(measurementTemplate);
       }
-      resNode = matcher.replaceFirst(sourceNodes[index]);
-      matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
+      intoPathDescriptor.specifyTargetPath(sourceColumn.toString(), targetPath);
+      intoPathDescriptor.specifyDeviceAlignment(targetPath.getDevicePath(), isAlignedDevice);
+      intoPathIterator.next();
     }
-    return resNode;
+    analysis.setIntoPathDescriptor(intoPathDescriptor);
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SelectIntoUtils.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SelectIntoUtils.java
new file mode 100644
index 0000000000..18107823e5
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SelectIntoUtils.java
@@ -0,0 +1,105 @@
+/*
+ * 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.analyze;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.db.mpp.plan.expression.Expression;
+import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+
+import static org.apache.iotdb.commons.conf.IoTDBConstant.DOUBLE_COLONS;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.LEVELED_PATH_TEMPLATE_PATTERN;
+
+public class SelectIntoUtils {
+
+  public static PartialPath constructTargetPath(
+      PartialPath sourcePath, PartialPath deviceTemplate, String measurementTemplate) {
+    PartialPath targetDevice = constructTargetDevice(sourcePath.getDevicePath(), deviceTemplate);
+    String targetMeasurement = constructTargetMeasurement(sourcePath, measurementTemplate);
+    return targetDevice.concatNode(targetMeasurement);
+  }
+
+  public static PartialPath constructTargetDevice(
+      PartialPath sourceDevice, PartialPath deviceTemplate) {
+    String[] sourceNodes = sourceDevice.getNodes();
+    String[] templateNodes = deviceTemplate.getNodes();
+
+    List<String> targetNodes = new ArrayList<>();
+    for (int nodeIndex = 0; nodeIndex < templateNodes.length; nodeIndex++) {
+      String curNode = templateNodes[nodeIndex];
+      if (curNode.equals(DOUBLE_COLONS)) {
+        if (nodeIndex != templateNodes.length - 1) {
+          throw new SemanticException(
+              "select into: placeholder `::` can only be used at the end of the path.");
+        }
+        for (; nodeIndex < sourceNodes.length; nodeIndex++) {
+          targetNodes.add(sourceNodes[nodeIndex]);
+        }
+        break;
+      }
+
+      String resNode = applyLevelPlaceholder(curNode, sourceNodes);
+      targetNodes.add(resNode);
+    }
+    return new PartialPath(targetNodes.toArray(new String[0]));
+  }
+
+  public static String constructTargetMeasurement(
+      PartialPath sourcePath, String measurementTemplate) {
+    if (measurementTemplate.equals(DOUBLE_COLONS)) {
+      return sourcePath.getMeasurement();
+    }
+    return applyLevelPlaceholder(measurementTemplate, sourcePath.getNodes());
+  }
+
+  private static String applyLevelPlaceholder(String templateNode, String[] sourceNodes) {
+    String resNode = templateNode;
+    Matcher matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
+    while (matcher.find()) {
+      String param = matcher.group();
+      int index;
+      try {
+        index = Integer.parseInt(param.substring(2, param.length() - 1).trim());
+      } catch (NumberFormatException e) {
+        throw new SemanticException("select into: the i of ${i} should be an integer.");
+      }
+      if (index < 1 || index >= sourceNodes.length) {
+        throw new SemanticException(
+            "select into: the i of ${i} should be greater than 0 and equal to or less than the length of queried path prefix.");
+      }
+      resNode = matcher.replaceFirst(sourceNodes[index]);
+      matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
+    }
+    return resNode;
+  }
+
+  public static boolean checkIsAllRawSeriesQuery(List<Expression> expressions) {
+    for (Expression expression : expressions) {
+      if (!(expression instanceof TimeSeriesOperand)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoDeviceMeasurementDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoDeviceMeasurementDescriptor.java
new file mode 100644
index 0000000000..c0a0ec120e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoDeviceMeasurementDescriptor.java
@@ -0,0 +1,57 @@
+/*
+ * 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.plan.parameter;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.sql.SemanticException;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.iotdb.db.mpp.plan.statement.component.IntoComponent.DUPLICATE_TARGET_DEVICE_ERROR_MSG;
+import static org.apache.iotdb.db.mpp.plan.statement.component.IntoComponent.DUPLICATE_TARGET_PATH_ERROR_MSG;
+
+public class IntoDeviceMeasurementDescriptor extends IntoPathDescriptor {
+
+  private final Map<PartialPath, PartialPath> targetDeviceToSourceDeviceMap;
+
+  public IntoDeviceMeasurementDescriptor() {
+    super();
+    this.targetDeviceToSourceDeviceMap = new HashMap<>();
+  }
+
+  public void specifyTargetDevice(PartialPath sourceDevice, PartialPath targetDevice) {
+    if (targetDeviceToSourceDeviceMap.containsKey(targetDevice)
+        && !targetDeviceToSourceDeviceMap.get(targetDevice).equals(sourceDevice)) {
+      throw new SemanticException(DUPLICATE_TARGET_DEVICE_ERROR_MSG);
+    }
+    targetDeviceToSourceDeviceMap.put(targetDevice, sourceDevice);
+  }
+
+  public void specifyTargetMeasurement(
+      PartialPath targetDevice, String sourceColumn, String targetMeasurement) {
+    Map<String, String> measurementToSourceColumnMap =
+        targetPathToSourceMap.computeIfAbsent(targetDevice, key -> new HashMap<>());
+    if (measurementToSourceColumnMap.containsKey(targetMeasurement)) {
+      throw new SemanticException(DUPLICATE_TARGET_PATH_ERROR_MSG);
+    }
+    measurementToSourceColumnMap.put(targetMeasurement, sourceColumn);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoPathDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoPathDescriptor.java
new file mode 100644
index 0000000000..d39515dcf0
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoPathDescriptor.java
@@ -0,0 +1,60 @@
+/*
+ * 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.plan.parameter;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.sql.SemanticException;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.iotdb.db.mpp.plan.statement.component.IntoComponent.DUPLICATE_TARGET_PATH_ERROR_MSG;
+
+public class IntoPathDescriptor {
+
+  protected final Map<PartialPath, Map<String, String>> targetPathToSourceMap;
+  protected final Map<PartialPath, Boolean> deviceToAlignedMap;
+
+  public IntoPathDescriptor() {
+    this.targetPathToSourceMap = new HashMap<>();
+    this.deviceToAlignedMap = new HashMap<>();
+  }
+
+  public void specifyTargetPath(String sourceColumn, PartialPath targetPath) {
+    PartialPath targetDevice = targetPath.getDevicePath();
+    String targetMeasurement = targetPath.getMeasurement();
+
+    Map<String, String> measurementToSourceColumnMap =
+        targetPathToSourceMap.computeIfAbsent(targetDevice, key -> new HashMap<>());
+    if (measurementToSourceColumnMap.containsKey(targetMeasurement)) {
+      throw new SemanticException(DUPLICATE_TARGET_PATH_ERROR_MSG);
+    }
+    measurementToSourceColumnMap.put(targetMeasurement, sourceColumn);
+  }
+
+  public void specifyDeviceAlignment(PartialPath targetDevice, boolean isAligned) {
+    if (deviceToAlignedMap.containsKey(targetDevice)
+        && deviceToAlignedMap.get(targetDevice) != isAligned) {
+      throw new SemanticException(
+          "select into: alignment property must be the same for the same device.");
+    }
+    deviceToAlignedMap.put(targetDevice, isAligned);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java
index c5848a3752..223e10602f 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java
@@ -19,23 +19,37 @@
 
 package org.apache.iotdb.db.mpp.plan.statement.component;
 
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.db.mpp.plan.expression.Expression;
 import org.apache.iotdb.db.mpp.plan.statement.StatementNode;
 
 import java.util.List;
 
+import static org.apache.iotdb.db.mpp.plan.analyze.SelectIntoUtils.checkIsAllRawSeriesQuery;
+
 /** This class maintains information of {@code INTO} clause. */
 public class IntoComponent extends StatementNode {
 
+  public static String PLACEHOLDER_MISMATCH_ERROR_MSG =
+      "select into: the correspondence between the placeholder and the raw time series could not be established.";
+  public static String FORBID_PLACEHOLDER_ERROR_MSG =
+      "select into: placeholders can only be used in raw time series data queries.";
+  public static String DEVICE_NUM_MISMATCH_ERROR_MSG =
+      "select into: the number of source devices and the number of target devices should be the same.";
+  public static String PATH_NUM_MISMATCH_ERROR_MSG =
+      "select into: the number of source columns and the number of target paths should be the same.";
+  public static String DUPLICATE_TARGET_DEVICE_ERROR_MSG =
+      "select into: target devices in into clause should be different.";
+  public static String DUPLICATE_TARGET_PATH_ERROR_MSG =
+      "select into: target paths in into clause should be different.";
+
   private final List<IntoItem> intoItems;
 
   public IntoComponent(List<IntoItem> intoItems) {
     this.intoItems = intoItems;
   }
 
-  public List<IntoItem> getIntoItems() {
-    return intoItems;
-  }
-
   public boolean isDeviceExistPlaceholder() {
     for (IntoItem intoItem : intoItems) {
       if (intoItem.isDeviceExistPlaceholder()) {
@@ -53,4 +67,170 @@ public class IntoComponent extends StatementNode {
     }
     return false;
   }
+
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // used in ALIGN BY TIME
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  public void validate(List<Expression> sourceColumns) {
+    boolean isAllRawSeriesQuery = checkIsAllRawSeriesQuery(sourceColumns);
+
+    if (!isAllRawSeriesQuery) {
+      if (isDeviceExistPlaceholder() || isMeasurementsExistPlaceholder()) {
+        throw new SemanticException(FORBID_PLACEHOLDER_ERROR_MSG);
+      }
+    }
+
+    if (isMeasurementsExistPlaceholder()) {
+      for (IntoItem intoItem : intoItems) {
+        if (intoItem.getIntoMeasurements().size() != 1) {
+          throw new SemanticException(PLACEHOLDER_MISMATCH_ERROR_MSG);
+        }
+      }
+
+      if (isDeviceExistPlaceholder()) {
+        if (intoItems.size() != 1) {
+          throw new SemanticException(PLACEHOLDER_MISMATCH_ERROR_MSG);
+        }
+      } else {
+        if (intoItems.size() != sourceColumns.size()) {
+          throw new SemanticException(PATH_NUM_MISMATCH_ERROR_MSG);
+        }
+      }
+    } else {
+      int intoPathsNum =
+          intoItems.stream().mapToInt(item -> item.getIntoMeasurements().size()).sum();
+      if (intoPathsNum != sourceColumns.size()) {
+        throw new SemanticException(PATH_NUM_MISMATCH_ERROR_MSG);
+      }
+    }
+  }
+
+  public IntoPathIterator getIntoPathIterator() {
+    return new IntoPathIterator(
+        intoItems, isDeviceExistPlaceholder(), isMeasurementsExistPlaceholder());
+  }
+
+  public static class IntoPathIterator extends AbstractIntoIterator {
+
+    public IntoPathIterator(
+        List<IntoItem> intoItems,
+        boolean isDeviceExistPlaceholder,
+        boolean isMeasurementsExistPlaceholder) {
+      super(intoItems, isDeviceExistPlaceholder, isMeasurementsExistPlaceholder);
+    }
+
+    public void next() {
+      if (isMeasurementsExistPlaceholder) {
+        if (!isDeviceExistPlaceholder) {
+          deviceIndex++;
+        }
+      } else {
+        measurementIndex++;
+        if (measurementIndex == intoItems.get(deviceIndex).getIntoMeasurements().size()) {
+          deviceIndex++;
+          measurementIndex = 0;
+        }
+      }
+    }
+  }
+
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // used in ALIGN BY DEVICE
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  public void validate(List<PartialPath> sourceDevices, List<Expression> sourceColumns) {
+    boolean isAllRawSeriesQuery = checkIsAllRawSeriesQuery(sourceColumns);
+
+    if (!isAllRawSeriesQuery) {
+      if (isMeasurementsExistPlaceholder()) {
+        throw new SemanticException(FORBID_PLACEHOLDER_ERROR_MSG);
+      }
+    }
+
+    if (isDeviceExistPlaceholder()) {
+      if (intoItems.size() != 1) {
+        throw new SemanticException(PLACEHOLDER_MISMATCH_ERROR_MSG);
+      }
+    } else {
+      if (intoItems.size() != sourceDevices.size()) {
+        throw new SemanticException(DEVICE_NUM_MISMATCH_ERROR_MSG);
+      }
+    }
+
+    for (IntoItem intoItem : intoItems) {
+      List<String> intoMeasurements = intoItem.getIntoMeasurements();
+      if (intoItem.isMeasurementsExistPlaceholder()) {
+        if (intoMeasurements.size() != 1) {
+          throw new SemanticException(PLACEHOLDER_MISMATCH_ERROR_MSG);
+        }
+      } else {
+        if (intoMeasurements.size() != sourceColumns.size()) {
+          throw new SemanticException(PATH_NUM_MISMATCH_ERROR_MSG);
+        }
+      }
+    }
+  }
+
+  public IntoDeviceMeasurementIterator getIntoDeviceMeasurementIterator() {
+    return new IntoDeviceMeasurementIterator(
+        intoItems, isDeviceExistPlaceholder(), isMeasurementsExistPlaceholder());
+  }
+
+  public static class IntoDeviceMeasurementIterator extends AbstractIntoIterator {
+
+    public IntoDeviceMeasurementIterator(
+        List<IntoItem> intoItems,
+        boolean isDeviceExistPlaceholder,
+        boolean isMeasurementsExistPlaceholder) {
+      super(intoItems, isDeviceExistPlaceholder, isMeasurementsExistPlaceholder);
+    }
+
+    public void nextDevice() {
+      if (!isDeviceExistPlaceholder) {
+        deviceIndex++;
+        measurementIndex = 0;
+      }
+    }
+
+    public void nextMeasurement() {
+      if (!intoItems.get(deviceIndex).isMeasurementsExistPlaceholder()) {
+        measurementIndex++;
+      }
+    }
+  }
+
+  public abstract static class AbstractIntoIterator {
+
+    protected final List<IntoItem> intoItems;
+
+    protected final boolean isDeviceExistPlaceholder;
+    protected final boolean isMeasurementsExistPlaceholder;
+
+    protected int deviceIndex;
+    protected int measurementIndex;
+
+    public AbstractIntoIterator(
+        List<IntoItem> intoItems,
+        boolean isDeviceExistPlaceholder,
+        boolean isMeasurementsExistPlaceholder) {
+      this.intoItems = intoItems;
+      this.isDeviceExistPlaceholder = isDeviceExistPlaceholder;
+      this.isMeasurementsExistPlaceholder = isMeasurementsExistPlaceholder;
+      this.deviceIndex = 0;
+      this.measurementIndex = 0;
+    }
+
+    public PartialPath getDeviceTemplate() {
+      return intoItems.get(deviceIndex).getIntoDevice();
+    }
+
+    public String getMeasurementTemplate() {
+      return intoItems.get(deviceIndex).getIntoMeasurements().get(measurementIndex);
+    }
+
+    public boolean isAlignedDevice() {
+      return intoItems.get(deviceIndex).isAligned();
+    }
+  }
 }


[iotdb] 01/04: finish analyzer

Posted by hu...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit fa3290a015b682d3c606c5ef9c0f28e7d7fb5f2f
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Wed Oct 12 17:32:55 2022 +0800

    finish analyzer
---
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  | 288 ++++++++++++++++-----
 .../plan/statement/component/IntoComponent.java    |   9 +
 2 files changed, 228 insertions(+), 69 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
index 2731e84907..f28f1064fb 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
@@ -105,7 +105,6 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTempla
 import org.apache.iotdb.db.mpp.plan.statement.sys.ExplainStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.ShowVersionStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.ShowPipeSinkTypeStatement;
-import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.query.control.SessionManager;
 import org.apache.iotdb.db.utils.FileLoaderUtils;
 import org.apache.iotdb.db.utils.TimePartitionUtils;
@@ -413,15 +412,15 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     // device path patterns in FROM clause
     List<PartialPath> devicePatternList = queryStatement.getFromComponent().getPrefixPaths();
 
-    Set<PartialPath> deviceList = new LinkedHashSet<>();
+    Set<PartialPath> deviceSet = new LinkedHashSet<>();
     for (PartialPath devicePattern : devicePatternList) {
       // get all matched devices
-      deviceList.addAll(
+      deviceSet.addAll(
           schemaTree.getMatchedDevices(devicePattern).stream()
               .map(DeviceSchemaInfo::getDevicePath)
               .collect(Collectors.toList()));
     }
-    return deviceList;
+    return deviceSet;
   }
 
   private List<Pair<Expression, String>> analyzeSelect(
@@ -1010,72 +1009,145 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     IntoComponent intoComponent = queryStatement.getIntoComponent();
     List<IntoItem> intoItems = intoComponent.getIntoItems();
 
-    Map<PartialPath, PartialPath> sourceDeviceToTargetDeviceMap = new HashMap<>();
     List<PartialPath> sourceDevices = new ArrayList<>(deviceSet);
+    List<PartialPath> targetDevices = new ArrayList<>(sourceDevices.size());
     if (intoComponent.isDeviceExistPlaceholder()) {
       if (intoItems.size() > 1) {
         throw new SemanticException("");
       }
 
       PartialPath deviceTemplate = intoItems.get(0).getIntoDevice();
-      for (int i = 0; i < sourceDevices.size(); i++) {
-        PartialPath sourceDevice = sourceDevices.get(i);
-        PartialPath targetDevice = constructIntoDevice(sourceDevice, deviceTemplate);
-        if (sourceDeviceToTargetDeviceMap.containsKey(sourceDevice)
-            && !sourceDeviceToTargetDeviceMap.get(sourceDevice).equals(targetDevice)) {
-          throw new SemanticException("");
-        }
-        sourceDeviceToTargetDeviceMap.put(sourceDevices.get(i), intoItems.get(i).getIntoDevice());
+      for (PartialPath sourceDevice : sourceDevices) {
+        targetDevices.add(constructIntoDevice(sourceDevice, deviceTemplate));
       }
     } else {
       if (intoItems.size() != sourceDevices.size()) {
         throw new SemanticException("");
       }
 
-      for (int i = 0; i < sourceDevices.size(); i++) {
-        PartialPath sourceDevice = sourceDevices.get(i);
-        PartialPath targetDevice = intoItems.get(i).getIntoDevice();
-        if (sourceDeviceToTargetDeviceMap.containsKey(sourceDevice)
-            && !sourceDeviceToTargetDeviceMap.get(sourceDevice).equals(targetDevice)) {
-          throw new SemanticException("");
-        }
-        sourceDeviceToTargetDeviceMap.put(sourceDevices.get(i), intoItems.get(i).getIntoDevice());
+      for (IntoItem intoItem : intoItems) {
+        targetDevices.add(intoItem.getIntoDevice());
       }
     }
 
     PatternTreeMap<String, PatternTreeMapFactory.StringSerializer> intoPathPatternTreeMap =
         PatternTreeMapFactory.getIntoPathPatternTreeMap();
 
-    if (isAllRawSeriesQuery) {
+    if (intoComponent.isMeasurementsExistPlaceholder()) {
+      if (!isAllRawSeriesQuery) {
+        throw new SemanticException("");
+      }
+
+      if (intoComponent.isDeviceExistPlaceholder()) {
+        String measurementTemplate = intoItems.get(0).getIntoMeasurements().get(0);
+        for (int i = 0; i < targetDevices.size(); i++) {
+          PartialPath targetDevice = targetDevices.get(i);
+          for (Expression outputColumn : outputColumns) {
+            PartialPath intoPath =
+                targetDevice.concatNode(
+                    constructIntoMeasurement(
+                        sourceDevices.get(i).concatNode(outputColumn.toString()),
+                        measurementTemplate));
+            intoPathPatternTreeMap.append(intoPath, outputColumn.toString());
+            if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
+              throw new SemanticException(
+                  "select into: target paths in into clause should be different.");
+            }
+          }
+        }
+      } else {
+        for (int deviceIndex = 0; deviceIndex < sourceDevices.size(); deviceIndex++) {
+          IntoItem intoItem = intoItems.get(deviceIndex);
+          List<String> intoMeasurements = intoItem.getIntoMeasurements();
+          PartialPath targetDevice = targetDevices.get(deviceIndex);
+
+          if (intoItem.isMeasurementsExistPlaceholder()) {
+            if (intoMeasurements.size() > 1) {
+              throw new SemanticException("");
+            }
+
+            String measurementTemplate = intoMeasurements.get(0);
+            for (Expression outputColumn : outputColumns) {
+              PartialPath intoPath =
+                  targetDevice.concatNode(
+                      constructIntoMeasurement(
+                          sourceDevices.get(deviceIndex).concatNode(outputColumn.toString()),
+                          measurementTemplate));
+              intoPathPatternTreeMap.append(intoPath, outputColumn.toString());
+              if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
+                throw new SemanticException(
+                    "select into: target paths in into clause should be different.");
+              }
+            }
+          } else {
+            if (intoMeasurements.size() != outputColumns.size()) {
+              throw new SemanticException(
+                  "select into: the number of source columns and the number of target paths should be the same.");
+            }
 
+            for (int measurementIndex = 0;
+                measurementIndex < intoMeasurements.size();
+                measurementIndex++) {
+              PartialPath intoPath =
+                  targetDevice.concatNode(intoMeasurements.get(measurementIndex));
+              intoPathPatternTreeMap.append(
+                  intoPath, outputColumns.get(measurementIndex).toString());
+              if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
+                throw new SemanticException(
+                    "select into: target paths in into clause should be different.");
+              }
+            }
+          }
+        }
+      }
     } else {
-      // disable placeholder
-      for (IntoItem intoItem : intoItems) {
-        if (intoItem.isMeasurementsExistPlaceholder()) {
+      for (int deviceIndex = 0; deviceIndex < sourceDevices.size(); deviceIndex++) {
+        IntoItem intoItem = intoItems.get(deviceIndex);
+        List<String> intoMeasurements = intoItem.getIntoMeasurements();
+        if (intoMeasurements.size() != outputColumns.size()) {
           throw new SemanticException(
-              "select into: placeholders can only be used in raw timeseries data queries.");
+              "select into: the number of source columns and the number of target paths should be the same.");
         }
-      }
 
-      List<PartialPath> intoPaths =
-          intoItems.stream()
-              .map(IntoItem::getIntoPaths)
-              .flatMap(List::stream)
-              .collect(Collectors.toList());
-
-      // check quantity consistency
-      if (intoPaths.size() != outputColumns.size()) {
-        throw new SemanticException(
-            "select into: the number of source columns and the number of target paths should be the same.");
+        PartialPath targetDevice = targetDevices.get(deviceIndex);
+        for (int measurementIndex = 0;
+            measurementIndex < intoMeasurements.size();
+            measurementIndex++) {
+          PartialPath intoPath = targetDevice.concatNode(intoMeasurements.get(measurementIndex));
+          intoPathPatternTreeMap.append(intoPath, outputColumns.get(measurementIndex).toString());
+          if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
+            throw new SemanticException(
+                "select into: target paths in into clause should be different.");
+          }
+        }
       }
+    }
+
+    if (isAllRawSeriesQuery) {
+      if (intoComponent.isDeviceExistPlaceholder()) {
+        if (intoComponent.isMeasurementsExistPlaceholder()) {
+          String measurementTemplate = intoItems.get(0).getIntoMeasurements().get(0);
+          for (int i = 0; i < targetDevices.size(); i++) {
+            PartialPath targetDevice = targetDevices.get(i);
+            for (Expression outputColumn : outputColumns) {
+              PartialPath intoPath =
+                  targetDevice.concatNode(
+                      constructIntoMeasurement(
+                          sourceDevices.get(i).concatNode(outputColumn.toString()),
+                          measurementTemplate));
+              intoPathPatternTreeMap.append(intoPath, outputColumn.toString());
+              if (intoPathPatternTreeMap.getOverlapped(intoPath).size() > 1) {
+                throw new SemanticException(
+                    "select into: target paths in into clause should be different.");
+              }
+            }
+          }
+        } else {
 
-      for (int i = 0; i < intoPaths.size(); i++) {
-        if (intoPathPatternTreeMap.getOverlapped(intoPaths.get(i)).size() > 1) {
-          throw new SemanticException(
-              "select into: target paths in into clause should be different.");
         }
-        intoPathPatternTreeMap.append(intoPaths.get(i), outputColumns.get(i).toString());
       }
+    } else {
+
     }
     analyzeIntoDevices(analysis, intoItems);
     analysis.setIntoPathPatternTreeMap(intoPathPatternTreeMap);
@@ -1092,17 +1164,16 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
         if (nodeIndex != templateNodes.length - 1) {
           throw new SemanticException("");
         }
-        // copy
         for (; nodeIndex < sourceNodes.length; nodeIndex++) {
           targetNodes.add(sourceNodes[nodeIndex]);
         }
         break;
       }
 
-      Matcher m = LEVELED_PATH_TEMPLATE_PATTERN.matcher(curNode);
       String resNode = curNode;
-      while (m.find()) {
-        String param = m.group();
+      Matcher matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
+      while (matcher.find()) {
+        String param = matcher.group();
         int index;
         try {
           index = Integer.parseInt(param.substring(2, param.length() - 1).trim());
@@ -1113,6 +1184,8 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
           throw new SemanticException(
               "select into: the i of ${i} should be greater than 0 and equal to or less than the length of queried path prefix.");
         }
+        resNode = matcher.replaceFirst(sourceNodes[index]);
+        matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
       }
       targetNodes.add(resNode);
     }
@@ -1136,11 +1209,68 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     IntoComponent intoComponent = queryStatement.getIntoComponent();
     List<IntoItem> intoItems = intoComponent.getIntoItems();
 
+    List<PartialPath> intoPaths = new ArrayList<>();
     PatternTreeMap<String, PatternTreeMapFactory.StringSerializer> intoPathPatternTreeMap =
         PatternTreeMapFactory.getIntoPathPatternTreeMap();
 
     if (isAllRawSeriesQuery) {
+      List<PartialPath> sourcePaths =
+          outputColumns.stream()
+              .map(expression -> ((TimeSeriesOperand) expression).getPath())
+              .collect(Collectors.toList());
+
+      if (intoComponent.isDeviceExistPlaceholder()) {
+        if (intoComponent.isMeasurementsExistPlaceholder()) {
+          if (intoItems.size() > 1) {
+            throw new SemanticException("");
+          }
+          if (intoItems.get(0).getIntoMeasurements().size() > 1) {
+            throw new SemanticException("");
+          }
+
+          PartialPath deviceTemplate = intoItems.get(0).getIntoDevice();
+          String measurementTemplate = intoItems.get(0).getIntoMeasurements().get(0);
+          for (PartialPath sourcePath : sourcePaths) {
+            intoPaths.add(constructIntoPath(sourcePath, deviceTemplate, measurementTemplate));
+          }
+        } else {
+          int sourcePathIndex = 0;
+          for (IntoItem intoItem : intoItems) {
+            PartialPath deviceTemplate = intoItem.getIntoDevice();
+            List<String> measurementList = intoItem.getIntoMeasurements();
+            for (String measurement : measurementList) {
+              intoPaths.add(
+                  constructIntoPath(sourcePaths.get(sourcePathIndex), deviceTemplate, measurement));
+              sourcePathIndex++;
+            }
+          }
+        }
+      } else {
+        if (intoComponent.isMeasurementsExistPlaceholder()) {
+          if (intoItems.size() != outputColumns.size()) {
+            throw new SemanticException(
+                "select into: the number of source columns and the number of target paths should be the same.");
+          }
+
+          for (int i = 0; i < intoItems.size(); i++) {
+            if (intoItems.get(i).getIntoMeasurements().size() != 1) {
+              throw new SemanticException("");
+            }
 
+            PartialPath targetDevice = intoItems.get(i).getIntoDevice();
+            intoPaths.add(
+                targetDevice.concatNode(
+                    constructIntoMeasurement(
+                        sourcePaths.get(i), intoItems.get(0).getIntoMeasurements().get(0))));
+          }
+        } else {
+          intoPaths =
+              intoItems.stream()
+                  .map(IntoItem::getIntoPaths)
+                  .flatMap(List::stream)
+                  .collect(Collectors.toList());
+        }
+      }
     } else {
       // disable placeholder
       for (IntoItem intoItem : intoItems) {
@@ -1150,26 +1280,27 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
         }
       }
 
-      List<PartialPath> intoPaths =
+      intoPaths =
           intoItems.stream()
               .map(IntoItem::getIntoPaths)
               .flatMap(List::stream)
               .collect(Collectors.toList());
+    }
 
-      // check quantity consistency
-      if (intoPaths.size() != outputColumns.size()) {
-        throw new SemanticException(
-            "select into: the number of source columns and the number of target paths should be the same.");
-      }
+    // check quantity consistency
+    if (intoPaths.size() != outputColumns.size()) {
+      throw new SemanticException(
+          "select into: the number of source columns and the number of target paths should be the same.");
+    }
 
-      for (int i = 0; i < intoPaths.size(); i++) {
-        intoPathPatternTreeMap.append(intoPaths.get(i), outputColumns.get(i).toString());
-        if (intoPathPatternTreeMap.getOverlapped(intoPaths.get(i)).size() > 1) {
-          throw new SemanticException(
-              "select into: target paths in into clause should be different.");
-        }
+    for (int i = 0; i < intoPaths.size(); i++) {
+      intoPathPatternTreeMap.append(intoPaths.get(i), outputColumns.get(i).toString());
+      if (intoPathPatternTreeMap.getOverlapped(intoPaths.get(i)).size() > 1) {
+        throw new SemanticException(
+            "select into: target paths in into clause should be different.");
       }
     }
+
     analyzeIntoDevices(analysis, intoItems);
     analysis.setIntoPathPatternTreeMap(intoPathPatternTreeMap);
   }
@@ -1194,24 +1325,43 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
   private boolean checkIsAllRawSeriesQuery(List<Expression> expressions) {
     for (Expression expression : expressions) {
       if (!(expression instanceof TimeSeriesOperand)) {
-        return true;
+        return false;
       }
     }
-    return false;
+    return true;
   }
 
   private PartialPath constructIntoPath(
-      Analysis analysis, Expression outputColumn, PartialPath path, boolean isAligned) {
-    if (!path.startWith(SQLConstant.ROOT)) {
-      throw new SemanticException("select into: ");
-    }
-    if (path.containNode(DOUBLE_COLONS)) {
-      throw new SemanticException("select into: ");
+      PartialPath sourcePath, PartialPath deviceTemplate, String measurementTemplate) {
+    PartialPath targetDevice = constructIntoDevice(sourcePath.getDevicePath(), deviceTemplate);
+    String targetMeasurement = constructIntoMeasurement(sourcePath, measurementTemplate);
+    return targetDevice.concatNode(targetMeasurement);
+  }
+
+  private String constructIntoMeasurement(PartialPath sourcePath, String measurementTemplate) {
+    if (measurementTemplate.equals(DOUBLE_COLONS)) {
+      return sourcePath.getMeasurement();
     }
-    if (LEVELED_PATH_TEMPLATE_PATTERN.matcher(path.getFullPath()).find()) {
-      throw new SemanticException("select into: ");
+
+    String[] sourceNodes = sourcePath.getNodes();
+    String resNode = measurementTemplate;
+    Matcher matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
+    while (matcher.find()) {
+      String param = matcher.group();
+      int index;
+      try {
+        index = Integer.parseInt(param.substring(2, param.length() - 1).trim());
+      } catch (NumberFormatException e) {
+        throw new SemanticException("select into: the i of ${i} should be an integer.");
+      }
+      if (index < 1 || index >= sourceNodes.length) {
+        throw new SemanticException(
+            "select into: the i of ${i} should be greater than 0 and equal to or less than the length of queried path prefix.");
+      }
+      resNode = matcher.replaceFirst(sourceNodes[index]);
+      matcher = LEVELED_PATH_TEMPLATE_PATTERN.matcher(resNode);
     }
-    return new MeasurementPath(path, analysis.getType(outputColumn), isAligned);
+    return resNode;
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java
index 1eaa3ff073..c5848a3752 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/IntoComponent.java
@@ -44,4 +44,13 @@ public class IntoComponent extends StatementNode {
     }
     return false;
   }
+
+  public boolean isMeasurementsExistPlaceholder() {
+    for (IntoItem intoItem : intoItems) {
+      if (intoItem.isMeasurementsExistPlaceholder()) {
+        return true;
+      }
+    }
+    return false;
+  }
 }