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

tledkov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 39793a7  IGNITE-9182 propagate lazy flag to synthetic SELECT used by 
first step of DML queries (closes #8473)
39793a7 is described below

commit 39793a70becd153d7773ae2796c9646a636d1ece
Author: tledkov <tled...@gridgain.com>
AuthorDate: Thu Nov 19 16:32:36 2020 +0300

    IGNITE-9182 propagate lazy flag to synthetic SELECT used by first step of 
DML queries (closes #8473)
---
 .../processors/query/h2/IgniteH2Indexing.java      |  19 +-
 .../processors/query/h2/dml/DmlAstUtils.java       |  66 +++++
 .../processors/query/h2/dml/UpdatePlan.java        |  17 +-
 .../processors/query/h2/dml/UpdatePlanBuilder.java |   9 +-
 .../processors/query/h2/sql/GridSqlSelect.java     |  26 ++
 .../internal/processors/query/LazyOnDmlTest.java   | 328 +++++++++++++++++++++
 .../IgniteBinaryCacheQueryTestSuite2.java          |   3 +
 7 files changed, 460 insertions(+), 8 deletions(-)

diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 0ded892..1154289 100644
--- 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -1600,6 +1600,7 @@ public class IgniteH2Indexing implements 
GridQueryIndexing {
         fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
         fldsQry.setPageSize(pageSize);
         fldsQry.setLocal(true);
+        fldsQry.setLazy(U.isFlagSet(flags, GridH2QueryRequest.FLAG_LAZY));
 
         boolean loc = true;
 
@@ -1643,7 +1644,8 @@ public class IgniteH2Indexing implements 
GridQueryIndexing {
             .setEnforceJoinOrder(fldsQry.isEnforceJoinOrder())
             .setLocal(fldsQry.isLocal())
             .setPageSize(fldsQry.getPageSize())
-            .setTimeout(fldsQry.getTimeout(), TimeUnit.MILLISECONDS);
+            .setTimeout(fldsQry.getTimeout(), TimeUnit.MILLISECONDS)
+            .setLazy(fldsQry.isLazy());
 
         QueryCursorImpl<List<?>> cur;
 
@@ -2881,7 +2883,12 @@ public class IgniteH2Indexing implements 
GridQueryIndexing {
             .setEnforceJoinOrder(qryDesc.enforceJoinOrder())
             .setLocal(qryDesc.local())
             .setPageSize(qryParams.pageSize())
-            .setTimeout(qryParams.timeout(), TimeUnit.MILLISECONDS);
+            .setTimeout(qryParams.timeout(), TimeUnit.MILLISECONDS)
+            // On no MVCC mode we cannot use lazy mode when UPDATE query 
contains updated columns
+            // in WHERE condition because it may be cause of update one entry 
several times
+            // (when index for such columns is selected for scan):
+            // e.g. : UPDATE test SET val = val + 1 WHERE val >= ?
+            .setLazy(qryParams.lazy() && plan.canSelectBeLazy());
 
         Iterable<List<?>> cur;
 
@@ -3014,7 +3021,10 @@ public class IgniteH2Indexing implements 
GridQueryIndexing {
                         .setEnforceJoinOrder(qryDesc.enforceJoinOrder())
                         .setLocal(qryDesc.local())
                         .setPageSize(qryParams.pageSize())
-                        .setTimeout((int)timeout, TimeUnit.MILLISECONDS);
+                        .setTimeout((int)timeout, TimeUnit.MILLISECONDS)
+                        // In MVCC mode we can use lazy mode always (when is 
set up) without dependency on
+                        // updated columns and WHERE condition.
+                        .setLazy(qryParams.lazy());
 
                     FieldsQueryCursor<List<?>> cur = executeSelectForDml(
                         qryDesc.schemaName(),
@@ -3055,6 +3065,9 @@ public class IgniteH2Indexing implements 
GridQueryIndexing {
             if (distributedPlan.isReplicatedOnly())
                 flags |= GridH2QueryRequest.FLAG_REPLICATED;
 
+            if (qryParams.lazy())
+                flags |= GridH2QueryRequest.FLAG_LAZY;
+
             flags = GridH2QueryRequest.setDataPageScanEnabled(flags,
                 qryParams.dataPageScanEnabled());
 
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
index e679ff5..ee9daa4 100644
--- 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
@@ -18,9 +18,11 @@
 package org.apache.ignite.internal.processors.query.h2.dml;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
@@ -54,6 +56,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.h2.command.Parser;
 import org.h2.expression.Expression;
+import org.h2.index.Index;
 import org.h2.table.Column;
 import org.h2.table.Table;
 import org.h2.util.IntArray;
@@ -349,6 +352,7 @@ public final class DmlAstUtils {
 
         for (GridSqlColumn c : update.cols()) {
             String newColName = Parser.quoteIdentifier("_upd_" + 
c.columnName());
+
             // We have to use aliases to cover cases when the user
             // wants to update _val field directly (if it's a literal)
             GridSqlAlias alias = new GridSqlAlias(newColName, 
elementOrDefault(update.set().get(c.columnName()), c), true);
@@ -358,6 +362,23 @@ public final class DmlAstUtils {
 
         GridSqlElement where = update.where();
 
+        // On no MVCC mode we cannot use lazy mode when UPDATE query contains 
index with updated columns
+        // and that index may be chosen to scan by WHERE condition
+        // because in this case any rows update may be updated several times.
+        // e.g. in the cases below we cannot use lazy mode:
+        //
+        // 1. CREATE INDEX idx on test(val)
+        //    UPDATE test SET val = val + 1 WHERE val >= ?
+        //
+        // 2. CREATE INDEX idx on test(val0, val1)
+        //    UPDATE test SET val1 = val1 + 1 WHERE val0 >= ?
+        mapQry.canBeLazy(!isIndexWithUpdateColumnsMayBeUsed(
+            gridTbl,
+            update.cols().stream()
+                .map(GridSqlColumn::column)
+                .collect(Collectors.toSet()),
+            extractColumns(gridTbl, where)));
+
         mapQry.where(where);
         mapQry.limit(update.limit());
 
@@ -365,6 +386,51 @@ public final class DmlAstUtils {
     }
 
     /**
+     * @return Set columns of the specified table that are used in expression.
+     */
+    private static Set<Column> extractColumns(GridH2Table tbl, GridSqlAst 
expr) {
+        if (expr == null)
+            return Collections.emptySet();
+
+        if (expr instanceof GridSqlColumn && 
((GridSqlColumn)expr).column().getTable().equals(tbl))
+            return Collections.singleton(((GridSqlColumn)expr).column());
+
+        HashSet<Column> set = new HashSet<>();
+
+        for (int i = 0; i < expr.size(); ++i)
+            set.addAll(extractColumns(tbl, expr.child(i)));
+
+        return set;
+    }
+
+    /**
+     * @return {@code true} if the index contains update columns may be 
potentially used for scan.
+     */
+    private static boolean isIndexWithUpdateColumnsMayBeUsed(
+        GridH2Table tbl,
+        Set<Column> updateCols,
+        Set<Column> whereCols) {
+        if (F.isEmpty(whereCols))
+            return false;
+
+        if (updateCols.size() == 1 && whereCols.size() == 1
+            && 
tbl.rowDescriptor().isValueColumn(F.first(updateCols).getColumnId())
+            && 
tbl.rowDescriptor().isValueColumn(F.first(whereCols).getColumnId()))
+            return true;
+
+        for (Index idx : tbl.getIndexes()) {
+            if (idx.equals(tbl.getPrimaryKey()) || 
whereCols.contains(idx.getColumns()[0])) {
+                for (Column idxCol : idx.getColumns()) {
+                    if (updateCols.contains(idxCol))
+                        return true;
+                }
+            }
+        }
+
+        return false;
+    }
+
+    /**
      * Do what we can to compute default value for this column (mimics H2 
behavior).
      * @see Table#getDefaultValue
      * @see Column#validateConvertUpdateSequence
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
index 66c4bb2..aeaea29 100644
--- 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlan.java
@@ -93,6 +93,9 @@ public final class UpdatePlan {
     /** Additional info for distributed update. */
     private final DmlDistributedPlanInfo distributed;
 
+    /** Additional info for distributed update. */
+    private final boolean canSelectBeLazy;
+
     /**
      * Constructor.
      *
@@ -125,7 +128,8 @@ public final class UpdatePlan {
         List<List<DmlArgument>> rows,
         int rowsNum,
         @Nullable FastUpdate fastUpdate,
-        @Nullable DmlDistributedPlanInfo distributed
+        @Nullable DmlDistributedPlanInfo distributed,
+        boolean canSelectBeLazy
     ) {
         this.colNames = colNames;
         this.colTypes = colTypes;
@@ -145,6 +149,7 @@ public final class UpdatePlan {
         this.isLocSubqry = isLocSubqry;
         this.fastUpdate = fastUpdate;
         this.distributed = distributed;
+        this.canSelectBeLazy = canSelectBeLazy;
     }
 
     /**
@@ -177,7 +182,8 @@ public final class UpdatePlan {
             null,
             0,
             fastUpdate,
-            distributed
+            distributed,
+            true
         );
     }
 
@@ -600,6 +606,13 @@ public final class UpdatePlan {
     }
 
     /**
+     * @return {@code true} is the SELECT query may be executed in lazy mode.
+     */
+    public boolean canSelectBeLazy() {
+        return canSelectBeLazy;
+    }
+
+    /**
      * Abstract iterator.
      */
     private abstract static class AbstractIterator extends 
GridCloseableIteratorAdapterEx<Object>
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index 8cc41b2..fd9496c 100644
--- 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -311,7 +311,8 @@ public final class UpdatePlanBuilder {
             rows,
             rowsNum,
             null,
-            distributed
+            distributed,
+            false
         );
     }
 
@@ -472,7 +473,8 @@ public final class UpdatePlanBuilder {
                     null,
                     0,
                     null,
-                    distributed
+                    distributed,
+                    sel.canBeLazy()
                 );
             }
             else {
@@ -590,7 +592,8 @@ public final class UpdatePlanBuilder {
             null,
             0,
             null,
-            null
+            null,
+            true
         );
     }
 
diff --git 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
index 6ecf002..93c1b8e 100644
--- 
a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
+++ 
b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
@@ -61,6 +61,13 @@ public class GridSqlSelect extends GridSqlQuery {
     /** */
     private boolean isForUpdate;
 
+    /** Used only for SELECT based on UPDATE.
+     * It cannot be lazy when updated columns are used in the conditions.
+     * In this case index based on these columns may be chosen to scan and 
some rows may be updated
+     * more than once time.
+     */
+    private boolean canBeLazy;
+
     /**
      * @param colIdx Column index as for {@link #column(int)}.
      * @return Child index for {@link #child(int)}.
@@ -437,4 +444,23 @@ public class GridSqlSelect extends GridSqlQuery {
 
         return copy;
     }
+
+    /**
+     * @param canBeLazy see {@link #canBeLazy()}.
+     */
+    public void canBeLazy(boolean canBeLazy) {
+        this.canBeLazy = canBeLazy;
+    }
+
+    /**
+     * Used only for SELECT based on UPDATE.
+     * It cannot be lazy when updated columns are used in the conditions.
+     * In this case index based on these columns may be chosen to scan and 
some rows may be updated
+     * more than once time.
+     *
+     * @return {@code true} is lazy flag is applicable.
+     */
+    public boolean canBeLazy() {
+        return canBeLazy;
+    }
 }
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyOnDmlTest.java
 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyOnDmlTest.java
new file mode 100644
index 0000000..bed2178
--- /dev/null
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyOnDmlTest.java
@@ -0,0 +1,328 @@
+/*
+ * 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.ignite.internal.processors.query;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import 
org.apache.ignite.internal.processors.cache.index.AbstractIndexingCommonTest;
+import org.apache.ignite.internal.processors.query.h2.H2PooledConnection;
+import org.apache.ignite.internal.processors.query.h2.H2QueryInfo;
+import org.apache.ignite.internal.processors.query.h2.H2Utils;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Tests for lazy mode for DML queries.
+ */
+@RunWith(Parameterized.class)
+public class LazyOnDmlTest extends AbstractIndexingCommonTest {
+    /** Keys count. */
+    private static final int KEY_CNT = 3_000;
+
+    /** */
+    @Parameterized.Parameter
+    public CacheAtomicityMode atomicityMode;
+
+    /** */
+    @Parameterized.Parameter(1)
+    public CacheMode cacheMode;
+
+    /**
+     * @return Test parameters.
+     */
+    @Parameterized.Parameters(name = "atomicityMode={0}, cacheMode={1}")
+    public static Collection parameters() {
+        Set<Object[]> paramsSet = new LinkedHashSet<>();
+
+        Object[] paramTemplate = new Object[2];
+
+        for (CacheAtomicityMode atomicityMode : CacheAtomicityMode.values()) {
+            paramTemplate = Arrays.copyOf(paramTemplate, paramTemplate.length);
+
+            paramTemplate[0] = atomicityMode;
+
+            for (CacheMode cacheMode : new CacheMode[] {CacheMode.PARTITIONED, 
CacheMode.REPLICATED}) {
+                Object[] params = Arrays.copyOf(paramTemplate, 
paramTemplate.length);
+
+                params[1] = cacheMode;
+
+                paramsSet.add(params);
+            }
+        }
+
+        return paramsSet;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        GridQueryProcessor.idxCls = CheckLazyIndexing.class;
+
+        startGrids(3);
+   }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        IgniteCache<Long, Long> c = grid(0).createCache(new 
CacheConfiguration<Long, Long>()
+            .setName("test")
+            .setSqlSchema("TEST")
+            .setAtomicityMode(atomicityMode)
+            .setCacheMode(cacheMode)
+            .setQueryEntities(Collections.singleton(new 
QueryEntity(Long.class.getName(), "testVal")
+                .setTableName("test")
+                .addQueryField("id", Long.class.getName(), null)
+                .addQueryField("val0", Long.class.getName(), null)
+                .addQueryField("val1", Long.class.getName(), null)
+                .addQueryField("val2", Long.class.getName(), null)
+                .setKeyFieldName("id")
+                .setIndexes(Collections.singletonList(
+                    new QueryIndex(Arrays.asList("val0", "val1"), 
QueryIndexType.SORTED)
+                ))
+            ))
+            .setBackups(1)
+            .setAffinity(new RendezvousAffinityFunction(false, 10)));
+
+        try (IgniteDataStreamer streamer = grid(0).dataStreamer("test")) {
+            for (long i = 0; i < KEY_CNT; ++i) {
+                BinaryObjectBuilder bob = grid(0).binary().builder("testVal");
+
+                bob.setField("val0", i);
+                bob.setField("val1", i);
+                bob.setField("val2", i);
+
+                streamer.addData(i, bob.build());
+            }
+        }
+
+        sql("CREATE TABLE table1 (id INT PRIMARY KEY, col0 INT, col1 VARCHAR 
(100))");
+
+        sql("INSERT INTO table1 (id, col0, col1) " +
+            "SELECT 1, 11, 'FIRST' " +
+            "UNION ALL " +
+            "SELECT 11,12, 'SECOND' " +
+            "UNION ALL " +
+            "SELECT 21, 13, 'THIRD' " +
+            "UNION ALL " +
+            "SELECT 31, 14, 'FOURTH'");
+
+        sql("CREATE TABLE  table2 (id INT PRIMARY KEY, col0 INT, col1 VARCHAR 
(100))");
+
+        sql("INSERT INTO table2 (id, col0, col1) " +
+            "SELECT 1, 21, 'TWO-ONE' " +
+            "UNION ALL " +
+            "SELECT 11, 22, 'TWO-TWO' " +
+            "UNION ALL " +
+            "SELECT 21, 23, 'TWO-THREE' " +
+            "UNION ALL " +
+            "SELECT 31, 24, 'TWO-FOUR'");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        for (String cache : grid(0).cacheNames())
+            grid(0).cache(cache).destroy();
+
+        super.afterTest();
+    }
+
+    /**
+     */
+    @Test
+    public void testUpdateNotLazy() throws Exception {
+        checkUpdateNotLazy("UPDATE test SET val0 = val0 + 1 WHERE val0 >= 0");
+        checkUpdateNotLazy("UPDATE test SET val1 = val1 + 1 WHERE val0 >= 0");
+    }
+
+    /**
+     */
+    public void checkUpdateNotLazy(String sql) throws Exception {
+        try (AutoCloseable checker = CheckLazyIndexing.checkLazy(atomicityMode 
== CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT)) {
+            List<List<?>> res = sql(sql).getAll();
+
+            // Check that all rows updates only ones.
+            assertEquals((long)KEY_CNT, res.get(0).get(0));
+        }
+    }
+
+    /**
+     */
+    @Test
+    public void testUpdateLazy() throws Exception {
+        checkUpdateLazy("UPDATE test SET val0 = val0 + 1");
+        checkUpdateLazy("UPDATE test SET val2 = val2 + 1 WHERE val2 >= 0");
+        checkUpdateLazy("UPDATE test SET val0 = val0 + 1 WHERE val1 >= 0");
+    }
+
+    /**
+     */
+    public void checkUpdateLazy(String sql) throws Exception {
+        try (AutoCloseable checker = CheckLazyIndexing.checkLazy(true)) {
+            List<List<?>> res = sql(sql).getAll();
+
+            // Check that all rows updates only ones.
+            assertEquals((long)KEY_CNT, res.get(0).get(0));
+        }
+    }
+
+    /**
+     */
+    @Test
+    public void testDeleteWithoutReduce() throws Exception {
+        try (AutoCloseable checker = CheckLazyIndexing.checkLazy(true)) {
+            List<List<?>> res = sql("DELETE FROM test WHERE val0 >= 
0").getAll();
+
+            assertEquals((long)KEY_CNT, res.get(0).get(0));
+        }
+    }
+
+    /**
+     */
+    @Test
+    public void testUpdateFromSubqueryLazy() throws Exception {
+        try (AutoCloseable checker = CheckLazyIndexing.checkLazy(true)) {
+            List<List<?>> res;
+
+            res = sql("UPDATE table1 " +
+                "SET (col0, col1) = " +
+                "   (SELECT table2.col0, table2.col1 FROM table2 WHERE 
table2.id = table1.id)" +
+                "WHERE table1.id in (21, 31)").getAll();
+
+            assertEquals(2L, res.get(0).get(0));
+
+            res = sql("UPDATE table1 " +
+                "SET (col0, col1) = " +
+                "   (SELECT table2.col0, table2.col1 FROM table2 WHERE 
table2.id = table1.id) " +
+                "WHERE exists (select * from table2 where table2.id = 
table1.id) " +
+                "AND table1.id in (21, 31)").getAll();
+
+            assertEquals(2L, res.get(0).get(0));
+        }
+    }
+
+    /**
+     */
+    @Test
+    public void testUpdateValueField() throws Exception {
+        sql("CREATE TABLE TEST2 (id INT PRIMARY KEY, val INT) " +
+            "WITH\"WRAP_VALUE=false\"");
+
+        sql("INSERT INTO TEST2 VALUES (0, 0), (1, 1), (2, 2)");
+
+        try (AutoCloseable checker = CheckLazyIndexing.checkLazy(false)) {
+            // 'val' field is the alias for _val. There is index for _val.
+            List<List<?>> res = sql("UPDATE TEST2 SET _val = _val + 1 WHERE 
val >=0").getAll();
+
+            assertEquals(3L, res.get(0).get(0));
+        }
+    }
+
+    /**
+     * @param sql SQL query.
+     * @param args Query parameters.
+     * @return Results cursor.
+     */
+    private FieldsQueryCursor<List<?>> sql(String sql, Object... args) {
+        return sql(grid(0), sql, args);
+    }
+
+    /**
+     * @param ign Node.
+     * @param sql SQL query.
+     * @param args Query parameters.
+     * @return Results cursor.
+     */
+    private FieldsQueryCursor<List<?>> sql(IgniteEx ign, String sql, Object... 
args) {
+        return ign.context().query().querySqlFields(new SqlFieldsQuery(sql)
+            .setLazy(true)
+            .setSchema("TEST")
+            .setPageSize(1)
+            .setArgs(args), false);
+    }
+
+    /** */
+    private static class CheckLazyIndexing extends IgniteH2Indexing {
+        /** */
+        private static Boolean expectedLazy;
+
+        /** */
+        private static int qryCnt;
+
+        /** {@inheritDoc} */
+        @Override public ResultSet executeSqlQueryWithTimer(PreparedStatement 
stmt, H2PooledConnection conn, String sql,
+            int timeoutMillis, @Nullable GridQueryCancel cancel, Boolean 
dataPageScanEnabled,
+            H2QueryInfo qryInfo) throws IgniteCheckedException {
+            if (expectedLazy != null) {
+                assertEquals(
+                    "Unexpected lazy flag [sql=" + sql + ']',
+                    (boolean)expectedLazy,
+                    H2Utils.session(conn.connection()).isLazyQueryExecution()
+                );
+            }
+
+            qryCnt++;
+
+            return super.executeSqlQueryWithTimer(stmt, conn, sql, 
timeoutMillis, cancel, dataPageScanEnabled, qryInfo);
+        }
+
+        /** */
+        public static AutoCloseable checkLazy(boolean expLazy) {
+            expectedLazy = expLazy;
+
+            return () -> {
+                assertTrue("Lazy checker doesn't work properly", 
CheckLazyIndexing.qryCnt > 0);
+
+                expectedLazy = null;
+                qryCnt = 0;
+            };
+        }
+    }
+}
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
 
b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
index 2022fce..6676e47 100644
--- 
a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
@@ -63,6 +63,7 @@ import 
org.apache.ignite.internal.processors.query.IgniteCacheGroupsSqlDistribut
 import 
org.apache.ignite.internal.processors.query.IgniteCacheGroupsSqlSegmentedIndexMultiNodeSelfTest;
 import 
org.apache.ignite.internal.processors.query.IgniteCacheGroupsSqlSegmentedIndexSelfTest;
 import 
org.apache.ignite.internal.processors.query.IgniteSqlCreateTableTemplateTest;
+import org.apache.ignite.internal.processors.query.LazyOnDmlTest;
 import org.apache.ignite.internal.processors.query.LocalQueryLazyTest;
 import org.apache.ignite.internal.processors.query.LongRunningQueryTest;
 import 
org.apache.ignite.internal.processors.query.SqlIndexConsistencyAfterInterruptAtomicCacheOperationTest;
@@ -92,6 +93,8 @@ import org.junit.runners.Suite;
  */
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
+    LazyOnDmlTest.class,
+
     DefaultQueryTimeoutTestSuite.class,
 
     CreateIndexOnInvalidDataTypeTest.class,

Reply via email to