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

korlov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 146e803bf7 IGNITE-22735 Sql. Avoid double parsing of sql queries 
(#4085)
146e803bf7 is described below

commit 146e803bf73c85ee1000f906e4357432a843e6c3
Author: korlov42 <[email protected]>
AuthorDate: Tue Jul 16 18:54:11 2024 +0300

    IGNITE-22735 Sql. Avoid double parsing of sql queries (#4085)
---
 .../ignite/internal/benchmark/InsertBenchmark.java | 29 +++++++++++++--
 .../engine/prepare/IgniteSqlToRelConvertor.java    | 43 ++++++++++++++++++++++
 .../internal/sql/engine/sql/ParserServiceImpl.java | 19 +++++++++-
 3 files changed, 87 insertions(+), 4 deletions(-)

diff --git 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/InsertBenchmark.java
 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/InsertBenchmark.java
index 289fcb3bcb..82c9043e14 100644
--- 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/InsertBenchmark.java
+++ 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/InsertBenchmark.java
@@ -72,10 +72,18 @@ public class InsertBenchmark extends 
AbstractMultiNodeBenchmark {
      * Benchmark for SQL insert via embedded client.
      */
     @Benchmark
-    public void sqlInsert(SqlState state) {
+    public void sqlPreparedInsert(SqlState state) {
         state.executeQuery();
     }
 
+    /**
+     * Benchmark for SQL insert via embedded client.
+     */
+    @Benchmark
+    public void sqlInlinedInsert(SqlState state) {
+        state.executeInlinedQuery();
+    }
+
     /**
      * Benchmark for SQL multiple rows insert via embedded client.
      */
@@ -144,7 +152,7 @@ public class InsertBenchmark extends 
AbstractMultiNodeBenchmark {
     }
 
     /**
-     * Benchmark state for {@link #sqlInsert(SqlState)} and {@link 
#sqlInsertScript(SqlState)}.
+     * Benchmark state for {@link #sqlPreparedInsert(SqlState)} and {@link 
#sqlInsertScript(SqlState)}.
      *
      * <p>Holds {@link Statement}.
      */
@@ -172,13 +180,19 @@ public class InsertBenchmark extends 
AbstractMultiNodeBenchmark {
             }
         }
 
+        void executeInlinedQuery() {
+            try (ResultSet<?> rs = sql.execute(null, 
createInsertStatement(id++))) {
+                // NO-OP
+            }
+        }
+
         void executeScript() {
             sql.executeScript(statement.query(), id++);
         }
     }
 
     /**
-     * Benchmark state for {@link #sqlInsert(SqlState)} and {@link 
#sqlInsertScript(SqlState)}.
+     * Benchmark state for {@link #sqlPreparedInsert(SqlState)} and {@link 
#sqlInsertScript(SqlState)}.
      *
      * <p>Holds {@link Statement}.
      */
@@ -367,6 +381,15 @@ public class InsertBenchmark extends 
AbstractMultiNodeBenchmark {
         return format(insertQueryTemplate, TABLE_NAME, "ycsb_key", fieldsQ, 
valQ);
     }
 
+    private static String createInsertStatement(int key) {
+        String insertQueryTemplate = "insert into {}({}, {}) values({}, {})";
+
+        String fieldsQ = IntStream.range(1, 11).mapToObj(i -> "field" + 
i).collect(joining(","));
+        String valQ = IntStream.range(1, 11).mapToObj(i -> "'" + FIELD_VAL + 
"'").collect(joining(","));
+
+        return format(insertQueryTemplate, TABLE_NAME, "ycsb_key", fieldsQ, 
key, valQ);
+    }
+
     private static String createMultiInsertStatement() {
         String insertQueryTemplate = "insert into {}({}, {}) values(?, {}), 
(?, {})";
 
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/IgniteSqlToRelConvertor.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/IgniteSqlToRelConvertor.java
index 2350f6504a..d3645bad2b 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/IgniteSqlToRelConvertor.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/IgniteSqlToRelConvertor.java
@@ -50,6 +50,8 @@ import org.apache.calcite.sql.SqlUpdate;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorNamespace;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
@@ -322,4 +324,45 @@ public class IgniteSqlToRelConvertor extends 
SqlToRelConverter {
                 relBuilder.build(), LogicalTableModify.Operation.MERGE,
                 targetColumnNameList, null, false);
     }
+
+    // =========================================================
+    // =                  BEGIN OF COPY-PASTE                  =
+    // vvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvvv
+    // TODO: https://issues.apache.org/jira/browse/IGNITE-22755 remove this 
section
+
+    // Section below is copy-pasted from original SqlToRelConverter.
+    // The only difference is that invocations of requireNonNull()
+    // replaced with similar one but accepting lamda instead of
+    // plain string.
+    @Override
+    protected RelOptTable getTargetTable(SqlNode call) {
+        final SqlValidatorNamespace targetNs = getNamespace(call);
+        SqlValidatorNamespace namespace;
+        if (targetNs.isWrapperFor(SqlValidatorImpl.DmlNamespace.class)) {
+            namespace = targetNs.unwrap(SqlValidatorImpl.DmlNamespace.class);
+        } else {
+            namespace = targetNs.resolve();
+        }
+        RelOptTable table = SqlValidatorUtil.getRelOptTable(namespace, 
catalogReader, null, null);
+        return requireNonNull(table, () -> "no table found for " + call);
+    }
+
+    private <T extends SqlValidatorNamespace> T getNamespace(SqlNode node) {
+        //noinspection unchecked
+        return (T) requireNonNull(
+                getNamespaceOrNull(node),
+                () -> "Namespace is not found for " + node);
+    }
+
+    private <T extends SqlValidatorNamespace> @Nullable T 
getNamespaceOrNull(SqlNode node) {
+        return (@Nullable T) validator().getNamespace(node);
+    }
+
+    private SqlValidator validator() {
+        return requireNonNull(validator, "validator");
+    }
+
+    // ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+    // =                  END OF COPY-PASTE                    =
+    // =========================================================
 }
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/ParserServiceImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/ParserServiceImpl.java
index 98ed99c9cb..f806088007 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/ParserServiceImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/ParserServiceImpl.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.sql.engine.sql;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Supplier;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriterConfig;
@@ -55,12 +56,28 @@ public class ParserServiceImpl implements ParserService {
 
         assert queryType != null : normalizedQuery;
 
+        AtomicReference<SqlNode> holder = new AtomicReference<>(parsedTree);
+
+        @SuppressWarnings("UnnecessaryLocalVariable")
         ParsedResult result = new ParsedResultImpl(
                 queryType,
                 query,
                 normalizedQuery,
                 parsedStatement.dynamicParamsCount(),
-                () -> IgniteSqlParser.parse(query, 
StatementParseResult.MODE).statement()
+                () -> {
+                    // Descendants of SqlNode class are mutable, thus we must 
use every
+                    // syntax node only once to avoid problem. But we already 
parsed the
+                    // query once to get normalized result. An `unparse` 
operation is known
+                    // to be safe, so let's reuse result of parsing for the 
first invocation
+                    // of `parsedTree` method to avoid double-parsing for one 
time queries.
+                    SqlNode ast = holder.getAndSet(null);
+
+                    if (ast != null) {
+                        return ast;
+                    }
+
+                    return IgniteSqlParser.parse(query, 
StatementParseResult.MODE).statement();
+                }
         );
 
         return result;

Reply via email to