You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/04/16 10:39:06 UTC

[GitHub] [pinot] xiangfu0 opened a new pull request, #8557: Adding DML definition and parse SQL InsertFile

xiangfu0 opened a new pull request, #8557:
URL: https://github.com/apache/pinot/pull/8557

   Adding DML definition and parse SQL InsertFile


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1108103361

   > I wonder if `LOAD DATA` makes more sense, as `INSERT` often follows with a list of concrete values.
   > 
   > Here is the reference from MySQL: https://dev.mysql.com/doc/refman/8.0/en/load-data.html https://dev.mysql.com/doc/refman/8.0/en/insert.html
   
   Talked with @walterddr offline and feel we can keep the current syntax for now. We can support MySQL syntax later.


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #8557:
URL: https://github.com/apache/pinot/pull/8557#discussion_r862272026


##########
pinot-core/src/main/java/org/apache/pinot/core/query/executor/sql/SqlQueryExecutor.java:
##########
@@ -0,0 +1,125 @@
+/**
+ * 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.pinot.core.query.executor.sql;
+
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.minion.MinionClient;
+import org.apache.pinot.common.response.BrokerResponse;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.helix.LeadControllerUtils;
+import org.apache.pinot.spi.config.task.AdhocTaskConfig;
+import org.apache.pinot.sql.parsers.SqlNodeAndOptions;
+import org.apache.pinot.sql.parsers.dml.DataManipulationStatement;
+import org.apache.pinot.sql.parsers.dml.DataManipulationStatementParser;
+
+
+/**
+ * SqlQueryExecutor executes all SQL queries including DQL, DML, DCL, DDL.
+ *
+ */
+public class SqlQueryExecutor {
+  private final String _controllerUrl;
+  private final HelixManager _helixManager;
+
+  /**
+   * Fetch the lead controller from helix, HA is not guaranteed.
+   * @param helixManager is used to query leader controller from helix.
+   */
+  public SqlQueryExecutor(@Nonnull HelixManager helixManager) {
+    _helixManager = helixManager;
+    _controllerUrl = null;
+  }
+
+  /**
+   * Recommended to provide the controller vip or service name for access.
+   * @param controllerUrl controller service name for sending minion task requests
+   */
+  public SqlQueryExecutor(@Nonnull String controllerUrl) {
+    _controllerUrl = controllerUrl;
+    _helixManager = null;
+  }
+
+  private static String getControllerBaseUrl(HelixManager helixManager) {
+    String instanceHostPort = LeadControllerUtils.getHelixClusterLeader(helixManager);
+    if (instanceHostPort == null) {
+      throw new RuntimeException("Unable to locate the leader pinot controller, please retry later...");
+    }
+    int index = instanceHostPort.lastIndexOf('_');
+    if (index < 1) {

Review Comment:
   (minor)
   ```suggestion
       if (index < 0) {
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/query/executor/sql/SqlQueryExecutor.java:
##########
@@ -0,0 +1,125 @@
+/**
+ * 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.pinot.core.query.executor.sql;
+
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.minion.MinionClient;
+import org.apache.pinot.common.response.BrokerResponse;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.helix.LeadControllerUtils;
+import org.apache.pinot.spi.config.task.AdhocTaskConfig;
+import org.apache.pinot.sql.parsers.SqlNodeAndOptions;
+import org.apache.pinot.sql.parsers.dml.DataManipulationStatement;
+import org.apache.pinot.sql.parsers.dml.DataManipulationStatementParser;
+
+
+/**
+ * SqlQueryExecutor executes all SQL queries including DQL, DML, DCL, DDL.
+ *
+ */
+public class SqlQueryExecutor {
+  private final String _controllerUrl;
+  private final HelixManager _helixManager;
+
+  /**
+   * Fetch the lead controller from helix, HA is not guaranteed.
+   * @param helixManager is used to query leader controller from helix.
+   */
+  public SqlQueryExecutor(@Nonnull HelixManager helixManager) {

Review Comment:
   (minor) Don't annotate `@Nonnull`, we assume everything is non-null, and only annotate `@Nullable`



##########
pinot-core/src/main/java/org/apache/pinot/core/query/executor/sql/SqlQueryExecutor.java:
##########
@@ -0,0 +1,125 @@
+/**
+ * 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.pinot.core.query.executor.sql;
+
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.minion.MinionClient;
+import org.apache.pinot.common.response.BrokerResponse;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.helix.LeadControllerUtils;
+import org.apache.pinot.spi.config.task.AdhocTaskConfig;
+import org.apache.pinot.sql.parsers.SqlNodeAndOptions;
+import org.apache.pinot.sql.parsers.dml.DataManipulationStatement;
+import org.apache.pinot.sql.parsers.dml.DataManipulationStatementParser;
+
+
+/**
+ * SqlQueryExecutor executes all SQL queries including DQL, DML, DCL, DDL.
+ *
+ */
+public class SqlQueryExecutor {
+  private final String _controllerUrl;
+  private final HelixManager _helixManager;
+
+  /**
+   * Fetch the lead controller from helix, HA is not guaranteed.
+   * @param helixManager is used to query leader controller from helix.
+   */
+  public SqlQueryExecutor(@Nonnull HelixManager helixManager) {
+    _helixManager = helixManager;
+    _controllerUrl = null;
+  }
+
+  /**
+   * Recommended to provide the controller vip or service name for access.
+   * @param controllerUrl controller service name for sending minion task requests
+   */
+  public SqlQueryExecutor(@Nonnull String controllerUrl) {

Review Comment:
   Same here



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1104262034

   > I wonder if `LOAD DATA` makes more sense, as `INSERT` often follows with a list of concrete values.
   > 
   > Here is the reference from MySQL: https://dev.mysql.com/doc/refman/8.0/en/load-data.html https://dev.mysql.com/doc/refman/8.0/en/insert.html
   
   Agreed, using `LOAD DATA` makes more sense. 
   
   I was trying to be on par with other technologies (https://clickhouse.com/docs/en/sql-reference/statements/insert-into/#inserting-data-from-a-file). But `LOAD DATA` should be the right standard.
   
   cc: @walterddr 
   


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on a diff in pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on code in PR #8557:
URL: https://github.com/apache/pinot/pull/8557#discussion_r853314724


##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotClientRequest.java:
##########
@@ -197,6 +206,29 @@ public void processSqlQueryPost(String query, @Suspended AsyncResponse asyncResp
     }
   }
 
+  private BrokerResponse executeSqlQuery(ObjectNode sqlRequestJson, HttpRequesterIdentity httpRequesterIdentity)
+      throws Exception {
+    SqlNodeAndOptions sqlNodeAndOptions;
+    try {
+      sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sqlRequestJson.get(Request.SQL).asText());
+    } catch (Exception e) {
+      return new BrokerResponseNative(QueryException.getException(QueryException.PQL_PARSING_ERROR, e));
+    }
+    PinotSqlType sqlType = CalciteSqlParser.extractSqlType(sqlNodeAndOptions.getSqlNode());
+    switch (sqlType) {
+      case DQL:
+        return _requestHandler.handleRequest(sqlRequestJson, httpRequesterIdentity, new RequestStatistics());

Review Comment:
   yes. If we want to reuse the SqlNode, it requires refactoring to add it into requestHandler's interface, which I don't want to change  for now.



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] amrishlal commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
amrishlal commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1103301917

   Looks good, but would be useful to have a few unit and / or integration tests. Also, `AccessControl.hasAccess(...)` methods are currently being called in BaseBrokerREquestHandler:
   
   ```
   Line 189:
       // First-stage access control to prevent unauthenticated requests from using up resources. Secondary table-level
       // check comes later.
       boolean hasAccess = _accessControlFactory.create().hasAccess(requesterIdentity);
   
   Line 295:
       // Second-stage table-level access control
       boolean hasTableAccess = _accessControlFactory.create().hasAccess(requesterIdentity, serverBrokerRequest);
   ```
   
   I am wondering if something similar can be done for all DML/DDL statements as well to avoid either accidental or malicious usecases where a user who normally runs SELECT statements ends up running an INSERT statement with valid S3 credential to insert bad data into a Pinot table (?).


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #8557:
URL: https://github.com/apache/pinot/pull/8557#discussion_r851673992


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java:
##########
@@ -135,20 +142,39 @@ public String handlePostSql(String requestJsonStr, @Context HttpHeaders httpHead
         queryOptions = requestJson.get("queryOptions").asText();
       }
       LOGGER.debug("Trace: {}, Running query: {}", traceEnabled, sqlQuery);
-      return getQueryResponse(sqlQuery, traceEnabled, queryOptions, httpHeaders, CommonConstants.Broker.Request.SQL);
+      return executeSqlQuery(httpHeaders, sqlQuery, traceEnabled, queryOptions);
     } catch (Exception e) {
       LOGGER.error("Caught exception while processing post request", e);
       return QueryException.getException(QueryException.INTERNAL_ERROR, e).toString();
     }
   }
 
+  private String executeSqlQuery(@Context HttpHeaders httpHeaders, String sqlQuery,
+      String traceEnabled, String queryOptions)
+      throws Exception {
+    SqlNodeAndOptions sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery);
+    PinotSqlType sqlType = CalciteSqlParser.extractSqlType(sqlNodeAndOptions.getSqlNode());
+    switch (sqlType) {
+      case DQL:
+        return getQueryResponse(sqlQuery, traceEnabled, queryOptions, httpHeaders, CommonConstants.Broker.Request.SQL);
+      case DML:
+        Map<String, String> headers =
+            httpHeaders.getRequestHeaders().entrySet().stream().filter(entry -> !entry.getValue().isEmpty())
+                .map(entry -> Pair.of(entry.getKey(), entry.getValue().get(0)))
+                .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+        return _sqlQueryExecutor.executeDMLStatement(sqlNodeAndOptions, headers).toJsonString();

Review Comment:
   Since this is executed on controller, we can skip sending the request again



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java:
##########
@@ -135,20 +142,39 @@ public String handlePostSql(String requestJsonStr, @Context HttpHeaders httpHead
         queryOptions = requestJson.get("queryOptions").asText();
       }
       LOGGER.debug("Trace: {}, Running query: {}", traceEnabled, sqlQuery);
-      return getQueryResponse(sqlQuery, traceEnabled, queryOptions, httpHeaders, CommonConstants.Broker.Request.SQL);
+      return executeSqlQuery(httpHeaders, sqlQuery, traceEnabled, queryOptions);
     } catch (Exception e) {
       LOGGER.error("Caught exception while processing post request", e);
       return QueryException.getException(QueryException.INTERNAL_ERROR, e).toString();
     }
   }
 
+  private String executeSqlQuery(@Context HttpHeaders httpHeaders, String sqlQuery,
+      String traceEnabled, String queryOptions)
+      throws Exception {
+    SqlNodeAndOptions sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery);
+    PinotSqlType sqlType = CalciteSqlParser.extractSqlType(sqlNodeAndOptions.getSqlNode());
+    switch (sqlType) {
+      case DQL:
+        return getQueryResponse(sqlQuery, traceEnabled, queryOptions, httpHeaders, CommonConstants.Broker.Request.SQL);

Review Comment:
   Since we already compiled the query, no need to compile again in `getQueryResponse()`



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on a diff in pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on code in PR #8557:
URL: https://github.com/apache/pinot/pull/8557#discussion_r853315366


##########
pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java:
##########
@@ -107,6 +108,40 @@ private static String removeTerminatingSemicolon(String sql) {
     return sql;
   }
 
+  public static SqlNodeAndOptions compileToSqlNodeAndOptions(String sql)
+      throws Exception {
+    // Remove the comments from the query
+    sql = removeComments(sql);
+
+    // Remove the terminating semicolon from the query
+    sql = removeTerminatingSemicolon(sql);
+
+    // Extract OPTION statements from sql as Calcite Parser doesn't parse it.
+    List<String> options = extractOptionsFromSql(sql);

Review Comment:
   Yeah, since we already have the OPTIONS handler and be able to parse it out, then I don't complain. 
   Eventually, this should all go to Parser.



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1100640294

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8557?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8557](https://codecov.io/gh/apache/pinot/pull/8557?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6163d00) into [master](https://codecov.io/gh/apache/pinot/commit/2704d887324014817097e609b1e7d794a898ea99?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2704d88) will **decrease** coverage by `15.37%`.
   > The diff coverage is `0.00%`.
   
   > :exclamation: Current head 6163d00 differs from pull request most recent head 15bf90b. Consider uploading reports for the commit 15bf90b to get more accurate results
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8557       +/-   ##
   =============================================
   - Coverage     29.52%   14.15%   -15.38%     
   - Complexity        0       84       +84     
   =============================================
     Files          1674     1646       -28     
     Lines         87872    86395     -1477     
     Branches      13313    13148      -165     
   =============================================
   - Hits          25942    12225    -13717     
   - Misses        59551    73249    +13698     
   + Partials       2379      921     -1458     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests2 | `14.15% <0.00%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8557?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...rg/apache/pinot/common/minion/MinionTaskUtils.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01pbmlvblRhc2tVdGlscy5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...org/apache/pinot/sql/parsers/CalciteSqlParser.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvcGFyc2Vycy9DYWxjaXRlU3FsUGFyc2VyLmphdmE=) | `0.00% <0.00%> (-68.03%)` | :arrow_down: |
   | [...ava/org/apache/pinot/sql/parsers/PinotSqlType.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvcGFyc2Vycy9QaW5vdFNxbFR5cGUuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/pinot/sql/parsers/SqlNodeAndOptions.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvcGFyc2Vycy9TcWxOb2RlQW5kT3B0aW9ucy5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...l/parsers/dml/DataManipulationStatementParser.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvcGFyc2Vycy9kbWwvRGF0YU1hbmlwdWxhdGlvblN0YXRlbWVudFBhcnNlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...g/apache/pinot/sql/parsers/dml/InsertIntoFile.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcWwvcGFyc2Vycy9kbWwvSW5zZXJ0SW50b0ZpbGUuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...t/controller/api/resources/PinotQueryResource.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90UXVlcnlSZXNvdXJjZS5qYXZh) | `0.00% <0.00%> (-50.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/core/routing/RoutingTable.java](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9yb3V0aW5nL1JvdXRpbmdUYWJsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [817 more](https://codecov.io/gh/apache/pinot/pull/8557/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8557?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8557?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [2704d88...15bf90b](https://codecov.io/gh/apache/pinot/pull/8557?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on a diff in pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
walterddr commented on code in PR #8557:
URL: https://github.com/apache/pinot/pull/8557#discussion_r853170025


##########
pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java:
##########
@@ -107,6 +108,40 @@ private static String removeTerminatingSemicolon(String sql) {
     return sql;
   }
 
+  public static SqlNodeAndOptions compileToSqlNodeAndOptions(String sql)
+      throws Exception {
+    // Remove the comments from the query
+    sql = removeComments(sql);
+
+    // Remove the terminating semicolon from the query
+    sql = removeTerminatingSemicolon(sql);
+
+    // Extract OPTION statements from sql as Calcite Parser doesn't parse it.
+    List<String> options = extractOptionsFromSql(sql);

Review Comment:
   this can be cleaned up by extending parser.jj to allow `OPTION` keyword but that can be done later. 



##########
pinot-broker/src/main/java/org/apache/pinot/broker/api/resources/PinotClientRequest.java:
##########
@@ -197,6 +206,29 @@ public void processSqlQueryPost(String query, @Suspended AsyncResponse asyncResp
     }
   }
 
+  private BrokerResponse executeSqlQuery(ObjectNode sqlRequestJson, HttpRequesterIdentity httpRequesterIdentity)
+      throws Exception {
+    SqlNodeAndOptions sqlNodeAndOptions;
+    try {
+      sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sqlRequestJson.get(Request.SQL).asText());
+    } catch (Exception e) {
+      return new BrokerResponseNative(QueryException.getException(QueryException.PQL_PARSING_ERROR, e));
+    }
+    PinotSqlType sqlType = CalciteSqlParser.extractSqlType(sqlNodeAndOptions.getSqlNode());
+    switch (sqlType) {
+      case DQL:
+        return _requestHandler.handleRequest(sqlRequestJson, httpRequesterIdentity, new RequestStatistics());

Review Comment:
   this means all parsing done so far is not used, instead sending the stuff to broker and parse it again?



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
walterddr commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1104432567

   I think `INSERT INTO` makes more sense for several reasons
   1. there's no ANSI standard for loading data from static objects; it is a DML specific to a database system.
   2. `INSERT INTO FROM xx` supports default SqlInsert syntactic extension from `INSERT` statement which is a type of extensible statement similar to `CREATE xxx` is extensible statement as DDL; I don't think there's a statement extension from `LOAD xxx` keyword. 
   3. `INSERT INTO (SELECT * FROM MYTABLE)` is actually a valid syntax in many databases


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] jackjlli commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
jackjlli commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1104185868

   I wonder if `LOAD DATA` makes more sense, as `INSERT` often follows with a list of concrete values.
   
   Here is the reference from MySQL:
   https://dev.mysql.com/doc/refman/8.0/en/load-data.html
   https://dev.mysql.com/doc/refman/8.0/en/insert.html


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1112592876

   > > > > > High level question: should we reuse the current SQL endpoint (HTTP GET), or introduce a new one. How do other systems handle it?
   > > > > 
   > > > > 
   > > > > For most of the DB, there would be just only one endpoint for executing all statements. So I feel we should just extend the existing SQL endpoint.
   > > > 
   > > > 
   > > > Agree that we should use one single endpoint to handle all statements. My concern is that the single SQL endpoint should probably be a `POST` request instead of the current `GET` request. I'd suggest adding a new `POST` endpoint for all statements, and keep the current `GET` endpoint which can only be used to query data
   > > 
   > > 
   > > GET endpoint is not used much, so far I think only pinot-admin Query command is still using GET api, all the other endpoint are using POST.
   > > For endpoint perspective, do you mean we should create a new POST endpoint, say `/sql/v2` for all SQL semantics and keep existing endpoint just for DQL?
   > 
   > Oh, just realize we already have the POST endpoint. What I meant is to make POST (reusing the existing one) to accept all SQL semantics, and keep GET only accepting DQL. We may add a check in the GET endpoint to prevent user using GET to change the table.
   
   done, throws exception when it's not DQL for GET api.


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1111289197

   > > > > High level question: should we reuse the current SQL endpoint (HTTP GET), or introduce a new one. How do other systems handle it?
   > > > 
   > > > 
   > > > For most of the DB, there would be just only one endpoint for executing all statements. So I feel we should just extend the existing SQL endpoint.
   > > 
   > > 
   > > Agree that we should use one single endpoint to handle all statements. My concern is that the single SQL endpoint should probably be a `POST` request instead of the current `GET` request. I'd suggest adding a new `POST` endpoint for all statements, and keep the current `GET` endpoint which can only be used to query data
   > 
   > GET endpoint is not used much, so far I think only pinot-admin Query command is still using GET api, all the other endpoint are using POST.
   > 
   > For endpoint perspective, do you mean we should create a new POST endpoint, say `/sql/v2` for all SQL semantics and keep existing endpoint just for DQL?
   
   Oh, just realize we already have the POST endpoint. What I meant is to make POST (reusing the existing one) to accept all SQL semantics, and keep GET only accepting DQL. We may add a check in the GET endpoint to prevent user using GET to change the table.


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1110230751

   > > > High level question: should we reuse the current SQL endpoint (HTTP GET), or introduce a new one. How do other systems handle it?
   > > 
   > > 
   > > For most of the DB, there would be just only one endpoint for executing all statements. So I feel we should just extend the existing SQL endpoint.
   > 
   > Agree that we should use one single endpoint to handle all statements. My concern is that the single SQL endpoint should probably be a `POST` request instead of the current `GET` request. I'd suggest adding a new `POST` endpoint for all statements, and keep the current `GET` endpoint which can only be used to query data
   
   GET endpoint is not used much, so far I think only pinot-admin Query command is still using GET api, all the other endpoint are using POST.
   
   For endpoint perspective, do you mean we should create a new POST endpoint, say `/sql/v2` for all SQL semantics and keep existing endpoint just for DQL?


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on a diff in pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on code in PR #8557:
URL: https://github.com/apache/pinot/pull/8557#discussion_r862282415


##########
pinot-core/src/main/java/org/apache/pinot/core/query/executor/sql/SqlQueryExecutor.java:
##########
@@ -0,0 +1,125 @@
+/**
+ * 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.pinot.core.query.executor.sql;
+
+import com.google.common.collect.ImmutableList;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.minion.MinionClient;
+import org.apache.pinot.common.response.BrokerResponse;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.helix.LeadControllerUtils;
+import org.apache.pinot.spi.config.task.AdhocTaskConfig;
+import org.apache.pinot.sql.parsers.SqlNodeAndOptions;
+import org.apache.pinot.sql.parsers.dml.DataManipulationStatement;
+import org.apache.pinot.sql.parsers.dml.DataManipulationStatementParser;
+
+
+/**
+ * SqlQueryExecutor executes all SQL queries including DQL, DML, DCL, DDL.
+ *
+ */
+public class SqlQueryExecutor {
+  private final String _controllerUrl;
+  private final HelixManager _helixManager;
+
+  /**
+   * Fetch the lead controller from helix, HA is not guaranteed.
+   * @param helixManager is used to query leader controller from helix.
+   */
+  public SqlQueryExecutor(@Nonnull HelixManager helixManager) {
+    _helixManager = helixManager;
+    _controllerUrl = null;
+  }
+
+  /**
+   * Recommended to provide the controller vip or service name for access.
+   * @param controllerUrl controller service name for sending minion task requests
+   */
+  public SqlQueryExecutor(@Nonnull String controllerUrl) {
+    _controllerUrl = controllerUrl;
+    _helixManager = null;
+  }
+
+  private static String getControllerBaseUrl(HelixManager helixManager) {
+    String instanceHostPort = LeadControllerUtils.getHelixClusterLeader(helixManager);
+    if (instanceHostPort == null) {
+      throw new RuntimeException("Unable to locate the leader pinot controller, please retry later...");
+    }
+    int index = instanceHostPort.lastIndexOf('_');
+    if (index < 1) {

Review Comment:
   substring will throw exception if start >= end



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1104264182

   > I wonder if `LOAD DATA` makes more sense, as `INSERT` often follows with a list of concrete values.
   > 
   > Here is the reference from MySQL: https://dev.mysql.com/doc/refman/8.0/en/load-data.html https://dev.mysql.com/doc/refman/8.0/en/insert.html
   
   Will update PR to change to `LOAD DATA` semantics.  


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 merged pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 merged PR #8557:
URL: https://github.com/apache/pinot/pull/8557


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
walterddr commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1104437034

   to add to it. `LOAD FILE` seems to be a MySQL only standard. 
   for example PostgresDB: 
   - `LOAD` keyword is used to load sharelib https://www.postgresql.org/docs/current/sql-load.html
   - `INSERT` + `COPY` seems to be used for loading files into table: https://www.postgresql.org/docs/current/sql-insert.html , https://www.postgresql.org/docs/14/sql-copy.html
   and if we view the `COPY` syntax it is very similar to the proposed syntax: 
   ```
   COPY table_name [ ( column_name [, ...] ) ]
       FROM { 'filename' | PROGRAM 'command' | STDIN }
       [ [ WITH ] ( option [, ...] ) ]
       [ WHERE condition ]
   ```
   where the `FROM { 'filename' }` part represents an external table from a 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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on a diff in pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on code in PR #8557:
URL: https://github.com/apache/pinot/pull/8557#discussion_r851684604


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java:
##########
@@ -135,20 +142,39 @@ public String handlePostSql(String requestJsonStr, @Context HttpHeaders httpHead
         queryOptions = requestJson.get("queryOptions").asText();
       }
       LOGGER.debug("Trace: {}, Running query: {}", traceEnabled, sqlQuery);
-      return getQueryResponse(sqlQuery, traceEnabled, queryOptions, httpHeaders, CommonConstants.Broker.Request.SQL);
+      return executeSqlQuery(httpHeaders, sqlQuery, traceEnabled, queryOptions);
     } catch (Exception e) {
       LOGGER.error("Caught exception while processing post request", e);
       return QueryException.getException(QueryException.INTERNAL_ERROR, e).toString();
     }
   }
 
+  private String executeSqlQuery(@Context HttpHeaders httpHeaders, String sqlQuery,
+      String traceEnabled, String queryOptions)
+      throws Exception {
+    SqlNodeAndOptions sqlNodeAndOptions = CalciteSqlParser.compileToSqlNodeAndOptions(sqlQuery);
+    PinotSqlType sqlType = CalciteSqlParser.extractSqlType(sqlNodeAndOptions.getSqlNode());
+    switch (sqlType) {
+      case DQL:
+        return getQueryResponse(sqlQuery, traceEnabled, queryOptions, httpHeaders, CommonConstants.Broker.Request.SQL);
+      case DML:
+        Map<String, String> headers =
+            httpHeaders.getRequestHeaders().entrySet().stream().filter(entry -> !entry.getValue().isEmpty())
+                .map(entry -> Pair.of(entry.getKey(), entry.getValue().get(0)))
+                .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+        return _sqlQueryExecutor.executeDMLStatement(sqlNodeAndOptions, headers).toJsonString();

Review Comment:
   DML, DDL statements are not latency-sensitive statements, so it might be ok to have an extra route but keep the code path clean.



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1100775824

   > High level question: should we reuse the current SQL endpoint (HTTP GET), or introduce a new one. How do other systems handle it?
   
   For most of the DB, there would be just only one endpoint for executing all statements. So I feel we should just extend the existing SQL endpoint.
   


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] xiangfu0 commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
xiangfu0 commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1102901663

   > looks good. although I am not sure it is good to encode aws key/token in plain text. any chance we can do it differently?
   
   Users will definitely type it in plain text and send it over using the query console. We should internal mask those, similar to current recurring minion tasks.
   


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang commented on pull request #8557: Adding DML definition and parse SQL InsertFile

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on PR #8557:
URL: https://github.com/apache/pinot/pull/8557#issuecomment-1110216143

   > > High level question: should we reuse the current SQL endpoint (HTTP GET), or introduce a new one. How do other systems handle it?
   > 
   > For most of the DB, there would be just only one endpoint for executing all statements. So I feel we should just extend the existing SQL endpoint.
   
   Agree that we should use one single endpoint to handle all statements. My concern is that the single SQL endpoint should probably be a `POST` request instead of the current `GET` request. I'd suggest adding a new `POST` endpoint for all statements, and keep the current `GET` endpoint which can only be used to query data


-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org