You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/07/23 01:28:52 UTC

[GitHub] [iotdb] SteveYurongSu opened a new pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

SteveYurongSu opened a new pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614


   https://issues.apache.org/jira/browse/IOTDB-1524


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

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

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



[GitHub] [iotdb] wangchao316 commented on a change in pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
wangchao316 commented on a change in pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#discussion_r676338518



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
##########
@@ -170,7 +175,7 @@ public void serialize(ByteBuffer buffer) {
    *
    * @param buffer
    */
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
+  public void deserialize(ByteBuffer buffer) throws IllegalPathException, IOException {

Review comment:
       GoodJob.
   I have a little question.
   why throw IOException?

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectIntoOperator.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.qp.logical.crud;
+
+import org.apache.iotdb.db.exception.query.LogicalOperatorException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.SelectIntoPlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
+
+import java.util.HashSet;
+import java.util.List;
+
+public class SelectIntoOperator extends Operator {
+
+  private QueryOperator queryOperator;
+
+  private List<PartialPath> intoPaths;
+
+  public SelectIntoOperator() {
+    super(SQLConstant.TOK_SELECT_INTO);
+    operatorType = OperatorType.SELECT_INTO;
+  }
+
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    QueryPlan queryPlan = (QueryPlan) queryOperator.generatePhysicalPlan(generator);
+    if (intoPaths.size() != queryPlan.getPaths().size()) {
+      throw new QueryProcessException(
+          "select into: the number of source paths and the number of target paths should be the same.");
+    }
+    return new SelectIntoPlan(
+        queryPlan, queryOperator.getFromComponent().getPrefixPaths().get(0), intoPaths);
+  }
+
+  public void check() throws LogicalOperatorException {
+    queryOperator.check();
+
+    if (intoPaths.size() > new HashSet<>(intoPaths).size()) {
+      throw new LogicalOperatorException(
+          "select into: target paths in into clause should be different.");
+    }
+
+    if (queryOperator.isAlignByDevice()) {
+      throw new LogicalOperatorException("select into: align by device clauses are not supported.");
+    }
+
+    // disable align
+    if (!queryOperator.isAlignByTime()) {
+      throw new LogicalOperatorException("select into: disable align clauses are not supported.");
+    }
+
+    if (queryOperator instanceof LastQueryOperator) {
+      throw new LogicalOperatorException("select into: last clauses are not supported.");
+    }
+
+    if (queryOperator instanceof AggregationQueryOperator
+        && !(queryOperator instanceof GroupByQueryOperator)) {
+      throw new LogicalOperatorException("select into: aggregation queries are not supported.");
+    }
+
+    if (queryOperator.getSpecialClauseComponent() != null) {
+      SpecialClauseComponent specialClauseComponent = queryOperator.getSpecialClauseComponent();
+      if (specialClauseComponent.hasSlimit()) {
+        throw new LogicalOperatorException("select into: slimit clauses are not supported.");
+      }
+      if (specialClauseComponent.getSeriesOffset() > 0) {
+        throw new LogicalOperatorException("select into: soffset clauses are not supported.");
+      }
+      if (!specialClauseComponent.isAscending()) {
+        throw new LogicalOperatorException(
+            "select into: order by time desc clauses are not supported.");

Review comment:
       add doc in md , "this situation does not supported"




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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41628746/badge)](https://coveralls.io/builds/41628746)
   
   Coverage increased (+0.02%) to 68.096% when pulling **d7385eda5ac4017c810eefaccb253b0e0c37f045 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41602666/badge)](https://coveralls.io/builds/41602666)
   
   Coverage increased (+0.01%) to 68.089% when pulling **40c7d0d1ebadc1f96e672496af573f2a0d433934 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-886320976


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![69.7%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '69.7%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [69.7% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885390932


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [20 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![67.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '67.8%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [67.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41599960/badge)](https://coveralls.io/builds/41599960)
   
   Coverage increased (+0.009%) to 68.088% when pulling **368b8d23112fcca48801fe7b8f3be6b1ee0433e8 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41594441/badge)](https://coveralls.io/builds/41594441)
   
   Coverage increased (+0.01%) to 68.09% when pulling **3a9a27b0409782a1e8930e6d2d0f2bec4f9ee4fb on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885590354


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![67.4%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '67.4%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [67.4% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41601131/badge)](https://coveralls.io/builds/41601131)
   
   Coverage increased (+0.01%) to 68.09% when pulling **368b8d23112fcca48801fe7b8f3be6b1ee0433e8 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885554153


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [19 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![11.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '11.8%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [11.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] HTHou commented on a change in pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#discussion_r676270342



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlanGenerator.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.engine.selectinto;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.BitMap;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/** internallyConstructNewPlan -> collectRowRecord * N -> generateInsertTabletPlan */
+public class InsertTabletPlanGenerator {
+
+  private final String targetDevice;
+  // the index of target path in into clause -> the index of output column of query data set
+  private final List<Integer> targetPathIndexToQueryDataSetIndex;
+  // the index of target path in into clause -> the measurement id of the target path

Review comment:
       in to clause




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

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

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



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#discussion_r677055141



##########
File path: server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
##########
@@ -603,9 +602,15 @@ public TSExecuteStatementResp executeStatement(TSExecuteStatementReq req) {
               physicalPlan,
               req.fetchSize,
               req.timeout,
-              sessionManager.getUsername(req.getSessionId()),
+              req.getSessionId(),
               req.isEnableRedirectQuery())
-          : executeUpdateStatement(physicalPlan, req.getSessionId());
+          : executeUpdateStatement(
+              statement,
+              req.statementId,
+              physicalPlan,
+              req.fetchSize,
+              req.timeout,
+              req.getSessionId());

Review comment:
       I think the name `executeUpdateStatement` is ok.
   
   First, `executeUpdate` is an interface of JDBC, we can keep this name at the entry of the method.
   
   Second, `executeUpdate` may have two tasks to execute, one is` executeSelectIntoStatement`, the other is `executeNonQueryStatement`. Two methods are very different.
   
   
   ![image](https://user-images.githubusercontent.com/30497621/127081040-e08d0ffb-32c8-40ff-9d97-2e5a4bdb5a4e.png)
   




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

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

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885554153


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [19 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![11.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '11.8%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [11.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41602866/badge)](https://coveralls.io/builds/41602866)
   
   Coverage increased (+0.04%) to 68.123% when pulling **40c7d0d1ebadc1f96e672496af573f2a0d433934 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-886227798


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![69.7%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '69.7%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [69.7% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885592472


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![69.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '69.8%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [69.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885590354


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [18 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![67.4%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '67.4%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [67.4% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41593980/badge)](https://coveralls.io/builds/41593980)
   
   Coverage increased (+0.007%) to 68.085% when pulling **4fb0d5bf2ff7b5a1b51ac496c42af5ce916a6ed3 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-886320976


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![69.7%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '69.7%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [69.7% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41632590/badge)](https://coveralls.io/builds/41632590)
   
   Coverage increased (+0.02%) to 68.095% when pulling **3472977ba2fe1799796f248986c3897c6f8dceae on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41599157/badge)](https://coveralls.io/builds/41599157)
   
   Coverage increased (+0.03%) to 68.109% when pulling **07d5a81959495a2a1bbacd551224bb9a59161720 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#discussion_r677052023



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlanGenerator.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.engine.selectinto;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.BitMap;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/** internallyConstructNewPlan -> collectRowRecord * N -> generateInsertTabletPlan */
+public class InsertTabletPlanGenerator {
+
+  private final String targetDevice;
+  // the index of target path in into clause -> the index of output column of query data set
+  private final List<Integer> targetPathIndexToQueryDataSetIndex;
+  // the index of target path in into clause -> the measurement id of the target path

Review comment:
       The variable name has been changed: targetPathIndexToQueryDataSetIndex -> queryDataSetIndexes.
   
   The comments have also been modified.




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

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

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-887461306


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![69.4%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '69.4%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [69.4% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41681401/badge)](https://coveralls.io/builds/41681401)
   
   Coverage increased (+0.01%) to 68.103% when pulling **218f78d9953b53a192cc2e60da457f562d593e12 on select-into** into **943733987d694c17ce286401899ef79d41de1487 on master**.
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41598875/badge)](https://coveralls.io/builds/41598875)
   
   Coverage increased (+0.02%) to 68.094% when pulling **c3eb77d78bab625c07eb84aa70f26ee3098667a2 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41598295/badge)](https://coveralls.io/builds/41598295)
   
   Coverage increased (+0.01%) to 68.09% when pulling **7879e0e1cf80c0570c525b272078ed1b9c8b0cd0 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41668803/badge)](https://coveralls.io/builds/41668803)
   
   Coverage decreased (-0.01%) to 68.075% when pulling **33eb88acd6f7747733adf2834787b8c62fdf2f1c on select-into** into **943733987d694c17ce286401899ef79d41de1487 on master**.
   


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

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

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



[GitHub] [iotdb] coveralls commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41592848/badge)](https://coveralls.io/builds/41592848)
   
   Coverage decreased (-0.02%) to 68.144% when pulling **9067cf98729c6245417327df50783b3f782974c1 on select-into** into **3c6925ce4746e9dfd3b2de1afcf5d6f49223454b on master**.
   


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

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

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



[GitHub] [iotdb] HTHou commented on a change in pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#discussion_r676272437



##########
File path: server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
##########
@@ -603,9 +602,15 @@ public TSExecuteStatementResp executeStatement(TSExecuteStatementReq req) {
               physicalPlan,
               req.fetchSize,
               req.timeout,
-              sessionManager.getUsername(req.getSessionId()),
+              req.getSessionId(),
               req.isEnableRedirectQuery())
-          : executeUpdateStatement(physicalPlan, req.getSessionId());
+          : executeUpdateStatement(
+              statement,
+              req.statementId,
+              physicalPlan,
+              req.fetchSize,
+              req.timeout,
+              req.getSessionId());

Review comment:
       use non query




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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41602713/badge)](https://coveralls.io/builds/41602713)
   
   Coverage increased (+0.02%) to 68.098% when pulling **40c7d0d1ebadc1f96e672496af573f2a0d433934 on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#discussion_r677049396



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
##########
@@ -170,7 +175,7 @@ public void serialize(ByteBuffer buffer) {
    *
    * @param buffer
    */
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
+  public void deserialize(ByteBuffer buffer) throws IllegalPathException, IOException {

Review comment:
       ![image](https://user-images.githubusercontent.com/30497621/127079991-1b0a08ac-dd3e-4058-b834-2a9cd6faca89.png)
   
   IOException may be thrown by `SelectIntoPlan#deserialize`. To be specific, it may be thrown by `queryPlan = (QueryPlan) Factory.create(buffer);`




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

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

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



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#discussion_r677055621



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectIntoOperator.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.qp.logical.crud;
+
+import org.apache.iotdb.db.exception.query.LogicalOperatorException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.SelectIntoPlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
+
+import java.util.HashSet;
+import java.util.List;
+
+public class SelectIntoOperator extends Operator {
+
+  private QueryOperator queryOperator;
+
+  private List<PartialPath> intoPaths;
+
+  public SelectIntoOperator() {
+    super(SQLConstant.TOK_SELECT_INTO);
+    operatorType = OperatorType.SELECT_INTO;
+  }
+
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    QueryPlan queryPlan = (QueryPlan) queryOperator.generatePhysicalPlan(generator);
+    if (intoPaths.size() != queryPlan.getPaths().size()) {
+      throw new QueryProcessException(
+          "select into: the number of source paths and the number of target paths should be the same.");
+    }
+    return new SelectIntoPlan(
+        queryPlan, queryOperator.getFromComponent().getPrefixPaths().get(0), intoPaths);
+  }
+
+  public void check() throws LogicalOperatorException {
+    queryOperator.check();
+
+    if (intoPaths.size() > new HashSet<>(intoPaths).size()) {
+      throw new LogicalOperatorException(
+          "select into: target paths in into clause should be different.");
+    }
+
+    if (queryOperator.isAlignByDevice()) {
+      throw new LogicalOperatorException("select into: align by device clauses are not supported.");
+    }
+
+    // disable align
+    if (!queryOperator.isAlignByTime()) {
+      throw new LogicalOperatorException("select into: disable align clauses are not supported.");
+    }
+
+    if (queryOperator instanceof LastQueryOperator) {
+      throw new LogicalOperatorException("select into: last clauses are not supported.");
+    }
+
+    if (queryOperator instanceof AggregationQueryOperator
+        && !(queryOperator instanceof GroupByQueryOperator)) {
+      throw new LogicalOperatorException("select into: aggregation queries are not supported.");
+    }
+
+    if (queryOperator.getSpecialClauseComponent() != null) {
+      SpecialClauseComponent specialClauseComponent = queryOperator.getSpecialClauseComponent();
+      if (specialClauseComponent.hasSlimit()) {
+        throw new LogicalOperatorException("select into: slimit clauses are not supported.");
+      }
+      if (specialClauseComponent.getSeriesOffset() > 0) {
+        throw new LogicalOperatorException("select into: soffset clauses are not supported.");
+      }
+      if (!specialClauseComponent.isAscending()) {
+        throw new LogicalOperatorException(
+            "select into: order by time desc clauses are not supported.");

Review comment:
       I added:
   
   ** Note that except for the following types of queries, other types of queries (such as `LAST` queries) are not supported.  **
   
   and 
   
   ** Note that except for the following clauses, other query clauses (such as `DESC`, `SOFFSET`, etc.) are not supported.  **
   
   :D




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

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

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-886227798


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![69.7%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '69.7%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [69.7% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] qiaojialin merged pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
qiaojialin merged pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614


   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885592472


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![69.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '69.8%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [69.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885585183


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [19 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![11.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '11.8%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [11.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885390932


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![D](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/D-16px.png 'D')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [2 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [20 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![67.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '67.8%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [67.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-887175672


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![69.3%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '69.3%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [69.3% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] commented on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] commented on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-887175672


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [0 Bugs](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [17 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![69.3%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/60-16px.png '69.3%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [69.3% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] coveralls edited a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885364124


   
   [![Coverage Status](https://coveralls.io/builds/41594542/badge)](https://coveralls.io/builds/41594542)
   
   Coverage decreased (-0.01%) to 68.064% when pulling **3a9a27b0409782a1e8930e6d2d0f2bec4f9ee4fb on select-into** into **72799f33215d5f6291544c354caf58c23a3dd04a on master**.
   


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

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

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



[GitHub] [iotdb] sonarcloud[bot] removed a comment on pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
sonarcloud[bot] removed a comment on pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#issuecomment-885585183


   SonarCloud Quality Gate failed.&nbsp; &nbsp; ![Quality Gate failed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/failed-16px.png 'Quality Gate failed')
   
   [![Bug](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/bug-16px.png 'Bug')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [![C](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/C-16px.png 'C')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG) [1 Bug](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=BUG)  
   [![Vulnerability](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/vulnerability-16px.png 'Vulnerability')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY) [0 Vulnerabilities](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=VULNERABILITY)  
   [![Security Hotspot](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/security_hotspot-16px.png 'Security Hotspot')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT) [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=SECURITY_HOTSPOT)  
   [![Code Smell](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/common/code_smell-16px.png 'Code Smell')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [![A](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/RatingBadge/A-16px.png 'A')](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL) [19 Code Smells](https://sonarcloud.io/project/issues?id=apache_incubator-iotdb&pullRequest=3614&resolved=false&types=CODE_SMELL)
   
   [![11.8%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/CoverageChart/0-16px.png '11.8%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list) [11.8% Coverage](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_coverage&view=list)  
   [![0.0%](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/Duplications/3-16px.png '0.0%')](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list) [0.0% Duplication](https://sonarcloud.io/component_measures?id=apache_incubator-iotdb&pullRequest=3614&metric=new_duplicated_lines_density&view=list)
   
   


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

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

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



[GitHub] [iotdb] qiaojialin commented on a change in pull request #3614: [IOTDB-1524] Support SELECT ... INTO ... clause

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #3614:
URL: https://github.com/apache/iotdb/pull/3614#discussion_r677198261



##########
File path: server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
##########
@@ -1127,41 +1181,47 @@ public TSExecuteStatementResp executeUpdateStatement(TSExecuteStatementReq req)
     }
 
     try {
-      return executeUpdateStatement(req.getStatement(), req.getSessionId());
+      PhysicalPlan physicalPlan =
+          processor.parseSQLToPhysicalPlan(req.statement, sessionManager.getZoneId(req.sessionId));
+      return physicalPlan.isQuery()
+          ? RpcUtils.getTSExecuteStatementResp(
+              TSStatusCode.EXECUTE_STATEMENT_ERROR, "Statement is a query statement.")
+          : executeUpdateStatement(
+              req.statement,
+              req.statementId,
+              physicalPlan,
+              req.fetchSize,
+              req.timeout,
+              req.getSessionId());
+    } catch (InterruptedException e) {
+      LOGGER.error(INFO_INTERRUPT_ERROR, req, e);
+      Thread.currentThread().interrupt();
+      return RpcUtils.getTSExecuteStatementResp(onQueryException(e, "executing update statement"));
     } catch (Exception e) {
       return RpcUtils.getTSExecuteStatementResp(onQueryException(e, "executing update statement"));
     }
   }
 
-  private TSExecuteStatementResp executeUpdateStatement(PhysicalPlan plan, long sessionId) {
-    TSStatus status = checkAuthority(plan, sessionId);
-    if (status != null) {
-      return new TSExecuteStatementResp(status);
-    }
-
-    status = executeNonQueryPlan(plan);
-    TSExecuteStatementResp resp = RpcUtils.getTSExecuteStatementResp(status);
-    long queryId = sessionManager.requestQueryId(false);
-    return resp.setQueryId(queryId);
-  }
-
-  private boolean executeNonQuery(PhysicalPlan plan)
-      throws QueryProcessException, StorageGroupNotSetException, StorageEngineException {
-    if (IoTDBDescriptor.getInstance().getConfig().isReadOnly()) {
-      throw new QueryProcessException(
-          "Current system mode is read-only, does not support non-query operation");
-    }
-    return executor.processNonQuery(plan);
+  private TSExecuteStatementResp executeUpdateStatement(

Review comment:
       add javadoc 

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlanGenerator.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.engine.selectinto;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.BitMap;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/** internallyConstructNewPlan -> collectRowRecord * N -> generateInsertTabletPlan */
+public class InsertTabletPlanGenerator {
+
+  private final String targetDevice;
+  // the index of column in InsertTabletPlan -> the index of output column of query data set
+  private final List<Integer> queryDataSetIndexes;
+  // the index of column in InsertTabletPlan -> the measurement id of the column
+  private final List<String> targetMeasurementIds;
+
+  private final int fetchSize;
+
+  // the following fields are used to construct plan
+  private int rowCount;
+  private long[] times;
+  private Object[] columns;
+  private BitMap[] bitMaps;
+  private TSDataType[] dataTypes;
+
+  private int numberOfInitializedColumns;
+
+  public InsertTabletPlanGenerator(String targetDevice, int fetchSize) {
+    this.targetDevice = targetDevice;
+    queryDataSetIndexes = new ArrayList<>();
+    targetMeasurementIds = new ArrayList<>();
+
+    this.fetchSize = fetchSize;
+  }
+
+  public void collectTargetPathInformation(String targetMeasurementId, int queryDataSetIndex) {
+    targetMeasurementIds.add(targetMeasurementId);
+    queryDataSetIndexes.add(queryDataSetIndex);
+  }
+
+  public void internallyConstructNewPlan() {
+    rowCount = 0;
+    times = new long[fetchSize];

Review comment:
       add a parameter in config file




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

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

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