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,