You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "korlov42 (via GitHub)" <gi...@apache.org> on 2023/06/23 12:13:45 UTC

[GitHub] [ignite-3] korlov42 commented on a diff in pull request #2214: IGNITE-19654: remove redundant planing timeout and use correct one.

korlov42 commented on code in PR #2214:
URL: https://github.com/apache/ignite-3/pull/2214#discussion_r1239624615


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/PlanningContext.java:
##########
@@ -39,23 +39,31 @@
  * Planning context.
  */
 public final class PlanningContext implements Context {
+    /** Parent context. */

Review Comment:
   please don't. Such javadoc's have no value



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java:
##########
@@ -75,9 +74,7 @@ public void pkConstraintConsistencyTest() {
                 .check();
 
         {

Review Comment:
   we don't need this braces anymore 



##########
modules/runner/src/testFixtures/java/org/apache/ignite/internal/sql/util/SqlTestUtils.java:
##########
@@ -32,13 +35,31 @@
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.ignite.internal.sql.engine.type.UuidType;
 import org.apache.ignite.sql.ColumnType;
+import org.apache.ignite.sql.SqlException;
+import org.junit.jupiter.api.function.Executable;
 
 /**
  * Test utils for SQL.
  */
 public class SqlTestUtils {
     private static final ThreadLocalRandom RND = ThreadLocalRandom.current();
 
+
+    /**
+     * <em>Assert</em> that execution of the supplied {@code executable} throws
+     * an {@link SqlException} with expected error code and return the exception.
+     *
+     * @param expectedCode Expected error code of {@link SqlException}
+     * @param executable supplier to execute and check thrown exception.

Review Comment:
   ```suggestion
        * @param executable Supplier to execute and check thrown exception.
   ```



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java:
##########
@@ -120,6 +123,21 @@ public void testSessionExpiration() throws Exception {
         ses2.execute(null, "SELECT 2 + 2").close();
     }
 
+    /** Check correctness of planning timeout. */
+    @Test
+    public void testPlanningTimeout() {
+        IgniteSql sql = igniteSql();
+
+        sql("CREATE TABLE TST1(id INTEGER PRIMARY KEY, val INTEGER)");
+
+        Session ses = sql.sessionBuilder().property(PLANNING_TIMEOUT.name, 1L).build();
+
+        SqlTestUtils.assertSqlExceptionThrows(PLANNING_TIMEOUTED_ERR,
+                () -> ses.execute(null, "SELECT * FROM TST1 t, TST1 t1, TST1 t2"));
+
+        ses.close();

Review Comment:
   what if assertion on the line above fail?



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/PrepareServiceImpl.java:
##########
@@ -143,30 +148,45 @@ public void stop() throws Exception {
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<QueryPlan> prepareAsync(SqlNode sqlNode, BaseQueryContext ctx) {
-        try {
-            assert single(sqlNode);
+    public CompletableFuture<QueryPlan> prepareAsync(SqlNode sqlNode, BaseQueryContext ctx, long plannerTimeout) {
+        CompletableFuture<QueryPlan> result;
+
+        assert single(sqlNode);
+
+        SqlQueryType queryType = Commons.getQueryType(sqlNode);
+        assert queryType != null : "No query type for query: " + sqlNode;
+
+        PlanningContext planningContext = PlanningContext.builder()
+                .parentContext(ctx)
+                .query(ctx.query())
+                .plannerTimeout(plannerTimeout)
+                .build();
 
-            SqlQueryType queryType = Commons.getQueryType(sqlNode);
-            assert queryType != null : "No query type for query: " + sqlNode;
+        result = prepareAsync0(sqlNode, queryType, planningContext);
 
-            PlanningContext planningContext = PlanningContext.builder()
-                    .parentContext(ctx)
-                    .build();
+        return result.exceptionally(ex -> {
+            Throwable th = ExceptionUtils.unwrapCause(ex);
+            if (planningContext.timeouted() && th instanceof RelOptPlanner.CannotPlanException) {
+                LOG.info("Query plan is absent due to planing timeout is reached [query={}]", ctx.query());

Review Comment:
   so, we are going to have 2 lines in the log per each timeout? 



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/BaseQueryContext.java:
##########
@@ -148,35 +148,35 @@ public List<MetadataHandler<?>> handlers(Class<? extends MetadataHandler<?>> hnd
 
     private final QueryCancel cancel;
 
+    private final String query;

Review Comment:
   is it possible to avoid keeping query in the context? 



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/property/PropertiesHelper.java:
##########
@@ -42,6 +43,23 @@ public static Builder newBuilder() {
         return new BuilderImpl();
     }
 
+    /**
+     * Creates new builder and populates it with properties from the given Map. Keys of the map are names of properties, values are values.
+     *
+     * @param props A holder to populate new builder with.
+     * @return A builder containing properties from given holder.
+     */
+    public static Builder newBuilder(Map<String, Object> props) {

Review Comment:
   the only difference between PropertiesHolder and Map<String, Object> is that former ensures type safety. By exposing builder like this, you deprive PropertiesHolder of this advantage.
   
   Besides, this particular code is working only because method equals of class Property is implemented in that way. But there is no guarantee that it won't be changed in the future.



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/PrepareServiceImpl.java:
##########
@@ -143,30 +148,45 @@ public void stop() throws Exception {
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<QueryPlan> prepareAsync(SqlNode sqlNode, BaseQueryContext ctx) {
-        try {
-            assert single(sqlNode);
+    public CompletableFuture<QueryPlan> prepareAsync(SqlNode sqlNode, BaseQueryContext ctx, long plannerTimeout) {
+        CompletableFuture<QueryPlan> result;
+
+        assert single(sqlNode);
+
+        SqlQueryType queryType = Commons.getQueryType(sqlNode);
+        assert queryType != null : "No query type for query: " + sqlNode;
+
+        PlanningContext planningContext = PlanningContext.builder()
+                .parentContext(ctx)
+                .query(ctx.query())
+                .plannerTimeout(plannerTimeout)
+                .build();
 
-            SqlQueryType queryType = Commons.getQueryType(sqlNode);
-            assert queryType != null : "No query type for query: " + sqlNode;
+        result = prepareAsync0(sqlNode, queryType, planningContext);
 
-            PlanningContext planningContext = PlanningContext.builder()
-                    .parentContext(ctx)
-                    .build();
+        return result.exceptionally(ex -> {
+            Throwable th = ExceptionUtils.unwrapCause(ex);
+            if (planningContext.timeouted() && th instanceof RelOptPlanner.CannotPlanException) {
+                LOG.info("Query plan is absent due to planing timeout is reached [query={}]", ctx.query());
+                throw new SqlException(ErrorGroups.Sql.PLANNING_TIMEOUTED_ERR);
+            }
+
+            throw new IgniteException(th);
+                }

Review Comment:
   indentation doesn't look right



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/IgnitePlanner.java:
##########
@@ -560,6 +564,11 @@ public void checkCancel() {
                 long startTs = ctx.startTs();
 
                 if (FastTimestamps.coarseCurrentTimeMillis() - startTs > timeout) {
+                    LOG.info("Planning of a query aborted due to planner timeout threshold is reached [timeout={}, query={}]",

Review Comment:
   I'm not sure we should log it at INFO level



-- 
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: notifications-unsubscribe@ignite.apache.org

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