diff --git a/modules/runner/build.gradle b/modules/runner/build.gradle index 1eeb875f3ea..dd7e153966e 100644 --- a/modules/runner/build.gradle +++ b/modules/runner/build.gradle @@ -116,7 +116,6 @@ dependencies { testImplementation libs.auto.service.annotations integrationTestAnnotationProcessor project(':ignite-configuration-annotation-processor') - integrationTestAnnotationProcessor libs.auto.service integrationTestAnnotationProcessor libs.jmh.annotation.processor integrationTestImplementation project(':ignite-distribution-zones') integrationTestImplementation project(':ignite-affinity') @@ -178,10 +177,11 @@ dependencies { exclude group: 'org.ow2.asm', module: 'asm' } integrationTestImplementation libs.typesafe.config - integrationTestImplementation libs.auto.service.annotations integrationTestImplementation libs.netty.common integrationTestImplementation libs.netty.handler + testFixturesAnnotationProcessor libs.auto.service + testFixturesImplementation project(':ignite-core') testFixturesImplementation project(':ignite-api') testFixturesImplementation project(':ignite-raft-api') @@ -195,7 +195,7 @@ dependencies { testFixturesImplementation testFixtures(project(':ignite-network')) testFixturesImplementation libs.jetbrains.annotations testFixturesImplementation libs.hamcrest.core - testFixturesImplementation libs.auto.service + testFixturesImplementation libs.auto.service.annotations } tasks.register("runnerPlatformTest", JavaExec) { diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItRaftCommandLeftInLogUntilRestartTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItRaftCommandLeftInLogUntilRestartTest.java index 2252bf3ef2e..fa51f638fa6 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItRaftCommandLeftInLogUntilRestartTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItRaftCommandLeftInLogUntilRestartTest.java @@ -33,6 +33,7 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.stream.IntStream; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.app.IgniteImpl; import org.apache.ignite.internal.lang.IgniteBiTuple; import org.apache.ignite.internal.lang.IgniteStringFormatter; @@ -44,7 +45,6 @@ import org.apache.ignite.internal.schema.marshaller.TupleMarshallerException; import org.apache.ignite.internal.schema.marshaller.TupleMarshallerImpl; import org.apache.ignite.internal.schema.row.Row; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.table.TableImpl; import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.internal.thread.NamedThreadFactory; @@ -73,7 +73,7 @@ public class ItRaftCommandLeftInLogUntilRestartTest extends ClusterPerClassInteg }; @Override - protected int nodes() { + protected int initialNodes() { return 2; } @@ -116,7 +116,7 @@ public void testUpdateAllCommand() throws Exception { public void testCommitCommandKeyValueView() throws Exception { restartClusterWithNotAppliedCommands( tx -> { - var kvView = CLUSTER_NODES.get(0).tables().table(DEFAULT_TABLE_NAME).keyValueView(); + var kvView = CLUSTER.aliveNode().tables().table(DEFAULT_TABLE_NAME).keyValueView(); for (var row : dataSet) { kvView.put(tx, Tuple.create().set("ID", row[0]), Tuple.create().set("NAME", row[1]).set("SALARY", row[2])); @@ -139,7 +139,7 @@ public void testUpdateCommandKeyValueView() throws Exception { tx -> { }, tx -> { - var kvView = CLUSTER_NODES.get(0).tables().table(DEFAULT_TABLE_NAME).keyValueView(); + var kvView = CLUSTER.aliveNode().tables().table(DEFAULT_TABLE_NAME).keyValueView(); for (var row : dataSet) { kvView.put(tx, Tuple.create().set("ID", row[0]), Tuple.create().set("NAME", row[1]).set("SALARY", row[2])); @@ -162,8 +162,8 @@ private void restartClusterWithNotAppliedCommands( Consumer afterBlock, Consumer checkAction ) throws Exception { - var node0 = (IgniteImpl) CLUSTER_NODES.get(0); - var node1 = (IgniteImpl) CLUSTER_NODES.get(1); + var node0 = CLUSTER.node(0); + var node1 = CLUSTER.node(1); AtomicReference> leaderAndGroupRef = new AtomicReference<>(); @@ -204,14 +204,13 @@ private void restartClusterWithNotAppliedCommands( tx.rollback(); } - stopNodes(); + CLUSTER.stopNode(0); + CLUSTER.stopNode(1); log.info("Restart the cluster"); - startCluster(); - - var node0Started = (IgniteImpl) CLUSTER_NODES.get(0); - var node1Started = (IgniteImpl) CLUSTER_NODES.get(1); + var node0Started = CLUSTER.startNode(0); + var node1Started = CLUSTER.startNode(1); var ignite = isNode0Leader ? node1Started : node0Started; diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/BaseSqlIntegrationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/BaseSqlIntegrationTest.java new file mode 100644 index 00000000000..8aa9f7fa97a --- /dev/null +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/BaseSqlIntegrationTest.java @@ -0,0 +1,342 @@ +/* + * 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.sql; + +import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertAll; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; +import org.apache.ignite.Ignite; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; +import org.apache.ignite.internal.app.IgniteImpl; +import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor; +import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor; +import org.apache.ignite.internal.raft.Peer; +import org.apache.ignite.internal.raft.service.RaftGroupService; +import org.apache.ignite.internal.sql.engine.SqlQueryProcessor; +import org.apache.ignite.internal.sql.engine.util.InjectQueryCheckerFactory; +import org.apache.ignite.internal.sql.engine.util.QueryChecker; +import org.apache.ignite.internal.sql.engine.util.QueryCheckerExtension; +import org.apache.ignite.internal.sql.engine.util.QueryCheckerFactory; +import org.apache.ignite.internal.storage.index.IndexStorage; +import org.apache.ignite.internal.storage.index.StorageIndexDescriptor; +import org.apache.ignite.internal.systemview.SystemViewManagerImpl; +import org.apache.ignite.internal.table.InternalTable; +import org.apache.ignite.internal.table.TableImpl; +import org.apache.ignite.internal.table.TableTestUtils; +import org.apache.ignite.internal.tx.TxManager; +import org.apache.ignite.sql.ColumnMetadata; +import org.apache.ignite.sql.IgniteSql; +import org.apache.ignite.table.Table; +import org.apache.ignite.tx.IgniteTransactions; +import org.apache.ignite.tx.Transaction; +import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.extension.ExtendWith; + +/** + * Base class for SQL integration tests. + */ +@ExtendWith(QueryCheckerExtension.class) +public class BaseSqlIntegrationTest extends ClusterPerClassIntegrationTest { + @InjectQueryCheckerFactory + protected static QueryCheckerFactory queryCheckerFactory; + + /** + * Executes the query and validates any asserts passed to the builder. + * + * @param qry Query to execute. + * @return Instance of QueryChecker. + */ + protected static QueryChecker assertQuery(String qry) { + return assertQuery(null, qry); + } + + /** + * Executes the query with the given transaction and validates any asserts passed to the builder. + * + * @param tx Transaction. + * @param qry Query to execute. + * @return Instance of QueryChecker. + */ + protected static QueryChecker assertQuery(Transaction tx, String qry) { + IgniteImpl node = CLUSTER.aliveNode(); + + return queryCheckerFactory.create(node.queryEngine(), node.transactions(), tx, qry); + } + + /** + * Used for join checks, disables other join rules for executing exact join algo. + * + * @param qry Query for check. + * @param joinType Type of join algo. + * @param rules Additional rules need to be disabled. + */ + protected static QueryChecker assertQuery(String qry, JoinType joinType, String... rules) { + return assertQuery(qry) + .disableRules(joinType.disabledRules) + .disableRules(rules); + } + + /** + * Used for query with aggregates checks, disables other aggregate rules for executing exact agregate algo. + * + * @param qry Query for check. + * @param aggregateType Type of aggregate algo. + * @param rules Additional rules need to be disabled. + */ + protected static QueryChecker assertQuery(String qry, AggregateType aggregateType, String... rules) { + return assertQuery(qry) + .disableRules(aggregateType.disabledRules) + .disableRules(rules); + } + + /** + * Join type. + */ + protected enum JoinType { + NESTED_LOOP( + "CorrelatedNestedLoopJoin", + "JoinCommuteRule", + "MergeJoinConverter" + ), + + MERGE( + "CorrelatedNestedLoopJoin", + "JoinCommuteRule", + "NestedLoopJoinConverter" + ), + + CORRELATED( + "MergeJoinConverter", + "JoinCommuteRule", + "NestedLoopJoinConverter" + ); + + private final String[] disabledRules; + + JoinType(String... disabledRules) { + this.disabledRules = disabledRules; + } + } + + /** + * Aggregate type. + */ + protected enum AggregateType { + SORT( + "ColocatedHashAggregateConverterRule", + "ColocatedSortAggregateConverterRule", + "MapReduceHashAggregateConverterRule" + ), + + HASH( + "ColocatedHashAggregateConverterRule", + "ColocatedSortAggregateConverterRule", + "MapReduceSortAggregateConverterRule" + ); + + private final String[] disabledRules; + + AggregateType(String... disabledRules) { + this.disabledRules = disabledRules; + } + } + + protected static void createAndPopulateTable() { + createTable(DEFAULT_TABLE_NAME, 1, 8); + + int idx = 0; + + insertData("person", List.of("ID", "NAME", "SALARY"), new Object[][]{ + {idx++, "Igor", 10d}, + {idx++, null, 15d}, + {idx++, "Ilya", 15d}, + {idx++, "Roma", 10d}, + {idx, "Roma", 10d} + }); + } + + protected static void checkMetadata(ColumnMetadata expectedMeta, ColumnMetadata actualMeta) { + assertAll("Missmatch:\n expected = " + expectedMeta + ",\n actual = " + actualMeta, + () -> assertEquals(expectedMeta.name(), actualMeta.name(), "name"), + () -> assertEquals(expectedMeta.nullable(), actualMeta.nullable(), "nullable"), + () -> assertSame(expectedMeta.type(), actualMeta.type(), "type"), + () -> assertEquals(expectedMeta.precision(), actualMeta.precision(), "precision"), + () -> assertEquals(expectedMeta.scale(), actualMeta.scale(), "scale"), + () -> assertSame(expectedMeta.valueClass(), actualMeta.valueClass(), "value class"), + () -> { + if (expectedMeta.origin() == null) { + assertNull(actualMeta.origin(), "origin"); + + return; + } + + assertNotNull(actualMeta.origin(), "origin"); + assertEquals(expectedMeta.origin().schemaName(), actualMeta.origin().schemaName(), " origin schema"); + assertEquals(expectedMeta.origin().tableName(), actualMeta.origin().tableName(), " origin table"); + assertEquals(expectedMeta.origin().columnName(), actualMeta.origin().columnName(), " origin column"); + } + ); + } + + /** + * Returns transaction manager for first cluster node. + */ + protected IgniteTransactions igniteTx() { + return CLUSTER.aliveNode().transactions(); + } + + /** + * Gets the SQL API. + * + * @return SQL API. + */ + protected IgniteSql igniteSql() { + return CLUSTER.aliveNode().sql(); + } + + /** + * Returns internal {@code SqlQueryProcessor} for first cluster node. + */ + protected SqlQueryProcessor queryProcessor() { + return (SqlQueryProcessor) CLUSTER.aliveNode().queryEngine(); + } + + /** + * Returns internal {@code TxManager} for first cluster node. + */ + protected TxManager txManager() { + return CLUSTER.aliveNode().txManager(); + } + + protected static Table table(String canonicalName) { + return CLUSTER.aliveNode().tables().table(canonicalName); + } + + /** + * Returns internal {@code SystemViewManager} for first cluster node. + */ + protected SystemViewManagerImpl systemViewManager() { + return (SystemViewManagerImpl) CLUSTER.aliveNode().systemViewManager(); + } + + /** + * Waits for the index to be built on all nodes. + * + * @param tableName Table name. + * @param indexName Index name. + * @return Nodes on which the partition index was built. + * @throws Exception If failed. + */ + protected static Map> waitForIndexBuild(String tableName, String indexName) throws Exception { + Map> partitionIdToNodes = new HashMap<>(); + + CLUSTER.runningNodes().forEach(clusterNode -> { + try { + TableImpl tableImpl = getTableImpl(clusterNode, tableName); + + assertNotNull(tableImpl, clusterNode.name() + " : " + tableName); + + InternalTable internalTable = tableImpl.internalTable(); + + assertTrue( + waitForCondition(() -> getIndexDescriptor(clusterNode, indexName) != null, 10, TimeUnit.SECONDS.toMillis(10)), + String.format("node=%s, tableName=%s, indexName=%s", clusterNode.name(), tableName, indexName) + ); + + for (int partitionId = 0; partitionId < internalTable.partitions(); partitionId++) { + RaftGroupService raftGroupService = internalTable.partitionRaftGroupService(partitionId); + + Stream allPeers = Stream.concat(Stream.of(raftGroupService.leader()), raftGroupService.peers().stream()); + + // Let's check if there is a node in the partition assignments. + if (allPeers.map(Peer::consistentId).noneMatch(clusterNode.name()::equals)) { + continue; + } + + CatalogTableDescriptor tableDescriptor = getTableDescriptor(clusterNode, tableName); + CatalogIndexDescriptor indexDescriptor = getIndexDescriptor(clusterNode, indexName); + + IndexStorage index = internalTable.storage().getOrCreateIndex( + partitionId, + StorageIndexDescriptor.create(tableDescriptor, indexDescriptor) + ); + + assertTrue(waitForCondition(() -> index.getNextRowIdToBuild() == null, 10, TimeUnit.SECONDS.toMillis(10))); + + partitionIdToNodes.computeIfAbsent(partitionId, p -> new ArrayList<>()).add(clusterNode); + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + return partitionIdToNodes; + } + + /** + * Returns table descriptor of the given table at the given node, or {@code null} if no such table exists. + * + * @param node Node. + * @param tableName Table name. + */ + private static @Nullable CatalogTableDescriptor getTableDescriptor(Ignite node, String tableName) { + IgniteImpl nodeImpl = (IgniteImpl) node; + + return TableTestUtils.getTable(nodeImpl.catalogManager(), tableName, nodeImpl.clock().nowLong()); + } + + /** + * Returns the table by name, {@code null} if absent. + * + * @param node Node. + * @param tableName Table name. + */ + protected static @Nullable TableImpl getTableImpl(Ignite node, String tableName) { + CompletableFuture tableFuture = node.tables().tableAsync(tableName); + + assertThat(tableFuture, willSucceedFast()); + + return (TableImpl) tableFuture.join(); + } + + /** + * Returns table index descriptor of the given index at the given node, or {@code null} if no such index exists. + * + * @param node Node. + * @param indexName Index name. + */ + protected static @Nullable CatalogIndexDescriptor getIndexDescriptor(Ignite node, String indexName) { + IgniteImpl nodeImpl = (IgniteImpl) node; + + return nodeImpl.catalogManager().index(indexName, nodeImpl.clock().nowLong()); + } +} diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java index 368b21c76dc..347b9c9e968 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit; import org.apache.calcite.schema.SchemaPlus; import org.apache.ignite.Ignite; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.QueryCancelledException; import org.apache.ignite.internal.sql.engine.SqlQueryProcessor; import org.apache.ignite.internal.sql.engine.schema.IgniteTable; @@ -53,9 +53,9 @@ import org.junit.jupiter.api.Test; /** Test common SQL API. */ -public class ItCommonApiTest extends ClusterPerClassIntegrationTest { +public class ItCommonApiTest extends BaseSqlIntegrationTest { @Override - protected int nodes() { + protected int initialNodes() { return 1; } @@ -106,7 +106,7 @@ public void checkTimestampOperations() { String kvTblName = "tbl_all_columns_sql"; String keyCol = "KEY"; - Ignite node = CLUSTER_NODES.get(0); + Ignite node = CLUSTER.aliveNode(); // TODO: https://issues.apache.org/jira/browse/IGNITE-19162 Trim all less than millisecond information from timestamp //String tsStr = "2023-03-29T08:22:33.005007Z"; @@ -161,7 +161,7 @@ public void testTxStateChangedOnErroneousOp() { SqlSchemaManager oldManager = (SqlSchemaManager) IgniteTestUtils.getFieldValue(queryProcessor(), SqlQueryProcessor.class, "sqlSchemaManager"); - Transaction tx = CLUSTER_NODES.get(0).transactions().begin(); + Transaction tx = CLUSTER.aliveNode().transactions().begin(); try { sql(tx, "INSERT INTO PUBLIC.TEST VALUES(1, 1)"); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java index b5188c88a07..65dd25b2da0 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java @@ -36,8 +36,8 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.ignite.internal.catalog.commands.CatalogUtils; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.api.ColumnMetadataImpl.ColumnOriginImpl; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.sql.engine.QueryCancelledException; import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.internal.tx.TxManager; @@ -73,12 +73,12 @@ * By default, any SQL API test should be added to the base class and use special provided methods to interact * with the API in a API-type-independent manner. For any API-specific test, should be used the appropriate subclass. */ -public abstract class ItSqlApiBaseTest extends ClusterPerClassIntegrationTest { +public abstract class ItSqlApiBaseTest extends BaseSqlIntegrationTest { protected static final int ROW_COUNT = 16; @AfterEach public void dropTables() { - for (Table t : CLUSTER_NODES.get(0).tables().tables()) { + for (Table t : CLUSTER.aliveNode().tables().tables()) { sql("DROP TABLE " + t.name()); } } @@ -592,7 +592,7 @@ public void select() { public void batch() { sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); - IgniteSql sql = CLUSTER_NODES.get(0).sql(); + IgniteSql sql = CLUSTER.aliveNode().sql(); Session ses = sql.sessionBuilder().defaultPageSize(ROW_COUNT / 2).build(); BatchedArguments args = BatchedArguments.of(0, 0); @@ -629,7 +629,7 @@ public void batchIncomplete() { sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); - IgniteSql sql = CLUSTER_NODES.get(0).sql(); + IgniteSql sql = CLUSTER.aliveNode().sql(); Session ses = sql.sessionBuilder().defaultPageSize(ROW_COUNT / 2).build(); BatchedArguments args = BatchedArguments.of(0, 0); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientAsynchronousApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientAsynchronousApiTest.java index 425a76f56f5..b04024f862c 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientAsynchronousApiTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientAsynchronousApiTest.java @@ -19,6 +19,7 @@ import static org.apache.ignite.internal.runner.app.client.ItAbstractThinClientTest.getClientAddresses; +import java.util.List; import org.apache.ignite.client.IgniteClient; import org.apache.ignite.sql.IgniteSql; import org.apache.ignite.tx.IgniteTransactions; @@ -34,7 +35,7 @@ public class ItSqlClientAsynchronousApiTest extends ItSqlAsynchronousApiTest { @BeforeAll public void startClient() { - client = IgniteClient.builder().addresses(getClientAddresses(CLUSTER_NODES).get(0)).build(); + client = IgniteClient.builder().addresses(getClientAddresses(List.of(CLUSTER.aliveNode())).get(0)).build(); } @AfterAll diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientMetricsTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientMetricsTest.java index 0c7b32fdf4e..5e3669ea2dd 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientMetricsTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientMetricsTest.java @@ -28,7 +28,7 @@ import java.util.concurrent.TimeUnit; import org.apache.ignite.internal.metrics.MetricManager; import org.apache.ignite.internal.metrics.MetricSet; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.metrics.SqlClientMetricSource; import org.apache.ignite.lang.IgniteException; import org.apache.ignite.sql.IgniteSql; @@ -42,7 +42,7 @@ import org.junit.jupiter.api.function.Executable; /** Test SQL client metrics. */ -public class ItSqlClientMetricsTest extends ClusterPerClassIntegrationTest { +public class ItSqlClientMetricsTest extends BaseSqlIntegrationTest { private MetricManager metricManager; private IgniteSql sql; private MetricSet clientMetricSet; @@ -68,7 +68,7 @@ void afterEach() throws Exception { } @Override - protected int nodes() { + protected int initialNodes() { return 1; } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientSynchronousApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientSynchronousApiTest.java index 5bb9e4d9aee..045c7455352 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientSynchronousApiTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlClientSynchronousApiTest.java @@ -19,6 +19,7 @@ import static org.apache.ignite.internal.runner.app.client.ItAbstractThinClientTest.getClientAddresses; +import java.util.List; import org.apache.ignite.client.IgniteClient; import org.apache.ignite.sql.IgniteSql; import org.apache.ignite.tx.IgniteTransactions; @@ -36,7 +37,7 @@ public class ItSqlClientSynchronousApiTest extends ItSqlSynchronousApiTest { @BeforeAll public void startClient() { - client = IgniteClient.builder().addresses(getClientAddresses(CLUSTER_NODES).get(0)).build(); + client = IgniteClient.builder().addresses(getClientAddresses(List.of(CLUSTER.aliveNode())).get(0)).build(); } @AfterAll diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ClusterPerClassIntegrationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ClusterPerClassIntegrationTest.java deleted file mode 100644 index bafa888fc65..00000000000 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ClusterPerClassIntegrationTest.java +++ /dev/null @@ -1,587 +0,0 @@ -/* - * 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.sql.engine; - -import static java.util.stream.Collectors.toList; -import static org.apache.ignite.internal.table.TableTestUtils.getTable; -import static org.apache.ignite.internal.testframework.IgniteTestUtils.await; -import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName; -import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition; -import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; -import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.jupiter.api.Assertions.assertAll; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertSame; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgnitionManager; -import org.apache.ignite.InitParameters; -import org.apache.ignite.internal.IgniteIntegrationTest; -import org.apache.ignite.internal.app.IgniteImpl; -import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor; -import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor; -import org.apache.ignite.internal.lang.IgniteStringFormatter; -import org.apache.ignite.internal.logger.IgniteLogger; -import org.apache.ignite.internal.logger.Loggers; -import org.apache.ignite.internal.raft.Peer; -import org.apache.ignite.internal.raft.service.RaftGroupService; -import org.apache.ignite.internal.sql.engine.util.InjectQueryCheckerFactory; -import org.apache.ignite.internal.sql.engine.util.QueryChecker; -import org.apache.ignite.internal.sql.engine.util.QueryCheckerExtension; -import org.apache.ignite.internal.sql.engine.util.QueryCheckerFactory; -import org.apache.ignite.internal.sql.engine.util.SqlTestUtils; -import org.apache.ignite.internal.storage.index.IndexStorage; -import org.apache.ignite.internal.storage.index.StorageIndexDescriptor; -import org.apache.ignite.internal.systemview.SystemViewManagerImpl; -import org.apache.ignite.internal.table.InternalTable; -import org.apache.ignite.internal.table.TableImpl; -import org.apache.ignite.internal.testframework.TestIgnitionManager; -import org.apache.ignite.internal.testframework.WorkDirectory; -import org.apache.ignite.internal.tx.TxManager; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.sql.ColumnMetadata; -import org.apache.ignite.sql.IgniteSql; -import org.apache.ignite.sql.Session; -import org.apache.ignite.table.RecordView; -import org.apache.ignite.table.Table; -import org.apache.ignite.table.Tuple; -import org.apache.ignite.tx.IgniteTransactions; -import org.apache.ignite.tx.Transaction; -import org.jetbrains.annotations.Nullable; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.TestInfo; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.extension.ExtendWith; - -/** - * Abstract basic integration test that starts a cluster once for all the tests it runs. - */ -@ExtendWith(QueryCheckerExtension.class) -@TestInstance(TestInstance.Lifecycle.PER_CLASS) -public abstract class ClusterPerClassIntegrationTest extends IgniteIntegrationTest { - private static final IgniteLogger LOG = Loggers.forClass(ClusterPerClassIntegrationTest.class); - - /** Test default table name. */ - protected static final String DEFAULT_TABLE_NAME = "person"; - - /** Base port number. */ - private static final int BASE_PORT = 3344; - - /** Base client port number. */ - private static final int BASE_CLIENT_PORT = 10800; - - /** Base rest port number. */ - protected static final int BASE_REST_PORT = 10300; - - /** Nodes bootstrap configuration pattern. */ - private static final String NODE_BOOTSTRAP_CFG = "{\n" - + " \"network\": {\n" - + " \"port\":{},\n" - + " \"nodeFinder\":{\n" - + " \"netClusterNodes\": [ {} ]\n" - + " }\n" - + " },\n" - + " clientConnector: { port:{} },\n" - + " rest.port: {}\n" - + "}"; - - /** Cluster nodes. */ - protected static final List CLUSTER_NODES = new ArrayList<>(); - - /** Work directory. */ - @WorkDirectory - private static Path WORK_DIR; - - /** Information object that is initialised on the test startup. */ - private TestInfo testInfo; - - /** - * Before all. - * - * @param testInfo Test information object. - */ - @BeforeAll - void beforeAll(TestInfo testInfo) { - LOG.info("Start beforeAll()"); - - this.testInfo = testInfo; - - startCluster(); - - LOG.info("End beforeAll()"); - } - - @InjectQueryCheckerFactory - protected static QueryCheckerFactory queryCheckerFactory; - - /** - * Starts and initializes a test cluster. - */ - protected void startCluster() { - String connectNodeAddr = "\"localhost:" + BASE_PORT + '\"'; - - List> futures = new ArrayList<>(); - - for (int i = 0; i < nodes(); i++) { - String nodeName = testNodeName(testInfo, i); - - String config = IgniteStringFormatter.format( - NODE_BOOTSTRAP_CFG, - BASE_PORT + i, connectNodeAddr, BASE_CLIENT_PORT + i, BASE_REST_PORT + i - ); - - futures.add(TestIgnitionManager.start(nodeName, config, WORK_DIR.resolve(nodeName))); - } - - String metaStorageNodeName = testNodeName(testInfo, 0); - - InitParameters initParameters = InitParameters.builder() - .destinationNodeName(metaStorageNodeName) - .metaStorageNodeNames(List.of(metaStorageNodeName)) - .clusterName("cluster") - .build(); - TestIgnitionManager.init(initParameters); - - for (CompletableFuture future : futures) { - assertThat(future, willCompleteSuccessfully()); - - CLUSTER_NODES.add(await(future)); - } - } - - /** - * Get a count of nodes in the Ignite cluster. - * - * @return Count of nodes. - */ - protected int nodes() { - return 3; - } - - /** - * After all. - */ - @AfterAll - void afterAll() throws Exception { - LOG.info("Start afterAll()"); - - stopNodes(); - - LOG.info("End afterAll()"); - } - - /** - * Stops all started nodes. - */ - protected void stopNodes() throws Exception { - List closeables = IntStream.range(0, nodes()) - .mapToObj(i -> testNodeName(testInfo, i)) - .map(nodeName -> (AutoCloseable) () -> IgnitionManager.stop(nodeName)) - .collect(toList()); - - IgniteUtils.closeAll(closeables); - - CLUSTER_NODES.clear(); - } - - /** Drops all visible tables. */ - protected void dropAllTables() { - for (Table t : CLUSTER_NODES.get(0).tables().tables()) { - sql("DROP TABLE " + t.name()); - } - } - - /** - * Appends indexes. - * - * @param node Execution cluster node. - * @param idxs Map with index representation. - * @param tblCanonicalName Canonical table name to create index in. - */ - protected static void addIndexes(Ignite node, Map> idxs, String tblCanonicalName) { - try (Session ses = node.sql().createSession()) { - for (Map.Entry> idx : idxs.entrySet()) { - ses.execute(null, String.format("CREATE INDEX %s ON %s (%s)", idx.getKey(), tblCanonicalName, - String.join(",", idx.getValue()))); - } - } - } - - /** - * Returns table index descriptor of the given index at the given node, or {@code null} if no such index exists. - * - * @param node Node. - * @param indexName Index name. - */ - public static @Nullable CatalogIndexDescriptor getIndexDescriptor(Ignite node, String indexName) { - IgniteImpl nodeImpl = (IgniteImpl) node; - - return nodeImpl.catalogManager().index(indexName, nodeImpl.clock().nowLong()); - } - - /** - * Executes the query and validates any asserts passed to the builder. - * - * @param qry Query to execute. - * @return Instance of QueryChecker. - */ - protected static QueryChecker assertQuery(String qry) { - return assertQuery(null, qry); - } - - /** - * Executes the query with the given transaction and validates any asserts passed to the builder. - * - * @param tx Transaction. - * @param qry Query to execute. - * @return Instance of QueryChecker. - */ - protected static QueryChecker assertQuery(Transaction tx, String qry) { - IgniteImpl node = (IgniteImpl) CLUSTER_NODES.get(0); - - return queryCheckerFactory.create(node.queryEngine(), node.transactions(), tx, qry); - } - - /** - * Used for join checks, disables other join rules for executing exact join algo. - * - * @param qry Query for check. - * @param joinType Type of join algo. - * @param rules Additional rules need to be disabled. - */ - static QueryChecker assertQuery(String qry, JoinType joinType, String... rules) { - return assertQuery(qry) - .disableRules(joinType.disabledRules) - .disableRules(rules); - } - - /** - * Used for query with aggregates checks, disables other aggregate rules for executing exact agregate algo. - * - * @param qry Query for check. - * @param aggregateType Type of aggregate algo. - * @param rules Additional rules need to be disabled. - */ - static QueryChecker assertQuery(String qry, AggregateType aggregateType, String... rules) { - return assertQuery(qry) - .disableRules(aggregateType.disabledRules) - .disableRules(rules); - } - - /** - * Creates a table. - * - * @param name Table name. - * @param replicas Replica factor. - * @param partitions Partitions count. - */ - protected static Table createTable(String name, int replicas, int partitions) { - sql(IgniteStringFormatter.format("CREATE ZONE IF NOT EXISTS {} WITH REPLICAS={}, PARTITIONS={};", - "ZONE_" + name.toUpperCase(), replicas, partitions)); - sql(IgniteStringFormatter.format("CREATE TABLE IF NOT EXISTS {} (id INT PRIMARY KEY, name VARCHAR, salary DOUBLE) " - + "WITH PRIMARY_ZONE='{}'", name, "ZONE_" + name.toUpperCase())); - - return CLUSTER_NODES.get(0).tables().table(name); - } - - enum JoinType { - NESTED_LOOP( - "CorrelatedNestedLoopJoin", - "JoinCommuteRule", - "MergeJoinConverter" - ), - - MERGE( - "CorrelatedNestedLoopJoin", - "JoinCommuteRule", - "NestedLoopJoinConverter" - ), - - CORRELATED( - "MergeJoinConverter", - "JoinCommuteRule", - "NestedLoopJoinConverter" - ); - - private final String[] disabledRules; - - JoinType(String... disabledRules) { - this.disabledRules = disabledRules; - } - } - - enum AggregateType { - SORT( - "ColocatedHashAggregateConverterRule", - "ColocatedSortAggregateConverterRule", - "MapReduceHashAggregateConverterRule" - ), - - HASH( - "ColocatedHashAggregateConverterRule", - "ColocatedSortAggregateConverterRule", - "MapReduceSortAggregateConverterRule" - ); - - private final String[] disabledRules; - - AggregateType(String... disabledRules) { - this.disabledRules = disabledRules; - } - } - - protected static void createAndPopulateTable() { - createTable(DEFAULT_TABLE_NAME, 1, 8); - - int idx = 0; - - insertData("person", List.of("ID", "NAME", "SALARY"), new Object[][]{ - {idx++, "Igor", 10d}, - {idx++, null, 15d}, - {idx++, "Ilya", 15d}, - {idx++, "Roma", 10d}, - {idx, "Roma", 10d} - }); - } - - protected static Table table(String canonicalName) { - return CLUSTER_NODES.get(0).tables().table(canonicalName); - } - - protected static void insertData(String tblName, List columnNames, Object[]... tuples) { - Transaction tx = CLUSTER_NODES.get(0).transactions().begin(); - - insertDataInTransaction(tx, tblName, columnNames, tuples); - - tx.commit(); - } - - protected static void insertDataInTransaction(Transaction tx, String tblName, List columnNames, Object[][] tuples) { - String insertStmt = "INSERT INTO " + tblName + "(" + String.join(", ", columnNames) + ")" - + " VALUES (" + ", ?".repeat(columnNames.size()).substring(2) + ")"; - - for (Object[] args : tuples) { - sql(tx, insertStmt, args); - } - } - - protected static void checkData(Table table, String[] columnNames, Object[]... tuples) { - RecordView view = table.recordView(); - - for (Object[] tuple : tuples) { - assert tuple != null && tuple.length == columnNames.length; - - Object id = tuple[0]; - - assert id != null : "Primary key cannot be null"; - - Tuple row = view.get(null, Tuple.create().set(columnNames[0], id)); - - assertNotNull(row); - - for (int i = 0; i < columnNames.length; i++) { - assertEquals(tuple[i], row.value(columnNames[i])); - } - } - } - - protected static List> sql(String sql, Object... args) { - return sql(null, sql, args); - } - - protected static List> sql(@Nullable Transaction tx, String sql, Object... args) { - Ignite ignite = CLUSTER_NODES.get(0); - - return SqlTestUtils.sql(ignite, tx, sql, args); - } - - protected static void checkMetadata(ColumnMetadata expectedMeta, ColumnMetadata actualMeta) { - assertAll("Missmatch:\n expected = " + expectedMeta + ",\n actual = " + actualMeta, - () -> assertEquals(expectedMeta.name(), actualMeta.name(), "name"), - () -> assertEquals(expectedMeta.nullable(), actualMeta.nullable(), "nullable"), - () -> assertSame(expectedMeta.type(), actualMeta.type(), "type"), - () -> assertEquals(expectedMeta.precision(), actualMeta.precision(), "precision"), - () -> assertEquals(expectedMeta.scale(), actualMeta.scale(), "scale"), - () -> assertSame(expectedMeta.valueClass(), actualMeta.valueClass(), "value class"), - () -> { - if (expectedMeta.origin() == null) { - assertNull(actualMeta.origin(), "origin"); - - return; - } - - assertNotNull(actualMeta.origin(), "origin"); - assertEquals(expectedMeta.origin().schemaName(), actualMeta.origin().schemaName(), " origin schema"); - assertEquals(expectedMeta.origin().tableName(), actualMeta.origin().tableName(), " origin table"); - assertEquals(expectedMeta.origin().columnName(), actualMeta.origin().columnName(), " origin column"); - } - ); - } - - /** - * Waits for the index to be built on all nodes. - * - * @param tableName Table name. - * @param indexName Index name. - * @return Nodes on which the partition index was built. - * @throws Exception If failed. - */ - protected static Map> waitForIndexBuild(String tableName, String indexName) throws Exception { - Map> partitionIdToNodes = new HashMap<>(); - - for (Ignite clusterNode : CLUSTER_NODES) { - TableImpl tableImpl = getTableImpl(clusterNode, tableName); - - assertNotNull(tableImpl, clusterNode.name() + " : " + tableName); - - InternalTable internalTable = tableImpl.internalTable(); - - assertTrue( - waitForCondition(() -> getIndexDescriptor(clusterNode, indexName) != null, 10, TimeUnit.SECONDS.toMillis(10)), - String.format("node=%s, tableName=%s, indexName=%s", clusterNode.name(), tableName, indexName) - ); - - for (int partitionId = 0; partitionId < internalTable.partitions(); partitionId++) { - RaftGroupService raftGroupService = internalTable.partitionRaftGroupService(partitionId); - - Stream allPeers = Stream.concat(Stream.of(raftGroupService.leader()), raftGroupService.peers().stream()); - - // Let's check if there is a node in the partition assignments. - if (allPeers.map(Peer::consistentId).noneMatch(clusterNode.name()::equals)) { - continue; - } - - CatalogTableDescriptor tableDescriptor = getTableDescriptor(clusterNode, tableName); - CatalogIndexDescriptor indexDescriptor = getIndexDescriptor(clusterNode, indexName); - - IndexStorage index = internalTable.storage().getOrCreateIndex( - partitionId, - StorageIndexDescriptor.create(tableDescriptor, indexDescriptor) - ); - - assertTrue(waitForCondition(() -> index.getNextRowIdToBuild() == null, 10, TimeUnit.SECONDS.toMillis(10))); - - partitionIdToNodes.computeIfAbsent(partitionId, p -> new ArrayList<>()).add(clusterNode); - } - } - - return partitionIdToNodes; - } - - /** - * Returns internal {@code SqlQueryProcessor} for first cluster node. - */ - protected SqlQueryProcessor queryProcessor() { - return (SqlQueryProcessor) ((IgniteImpl) CLUSTER_NODES.get(0)).queryEngine(); - } - - /** - * Returns internal {@code SystemViewManager} for first cluster node. - */ - protected SystemViewManagerImpl systemViewManager() { - return (SystemViewManagerImpl) ((IgniteImpl) CLUSTER_NODES.get(0)).systemViewManager(); - } - - /** - * Returns internal {@code TxManager} for first cluster node. - */ - protected TxManager txManager() { - return ((IgniteImpl) CLUSTER_NODES.get(0)).txManager(); - } - - /** - * Returns transaction manager for first cluster node. - */ - protected IgniteTransactions igniteTx() { - return CLUSTER_NODES.get(0).transactions(); - } - - /** - * Gets the SQL API. - * - * @return SQL API. - */ - protected IgniteSql igniteSql() { - return CLUSTER_NODES.get(0).sql(); - } - - /** - * Looks up a node by a consistent ID, {@code null} if absent. - * - * @param consistentId Node consistent ID. - */ - static @Nullable IgniteImpl findByConsistentId(String consistentId) { - return CLUSTER_NODES.stream() - .map(IgniteImpl.class::cast) - .filter(ignite -> consistentId.equals(ignite.node().name())) - .findFirst() - .orElse(null); - } - - /** - * Returns the table by name, {@code null} if absent. - * - * @param node Node. - * @param tableName Table name. - */ - static @Nullable TableImpl getTableImpl(Ignite node, String tableName) { - CompletableFuture
tableFuture = node.tables().tableAsync(tableName); - - assertThat(tableFuture, willSucceedFast()); - - return (TableImpl) tableFuture.join(); - } - - /** - * Returns the index ID from the catalog, {@code null} if there is no index. - * - * @param node Node. - * @param indexName Index name. - */ - static @Nullable Integer indexId(Ignite node, String indexName) { - CatalogIndexDescriptor indexDescriptor = getIndexDescriptor(node, indexName); - - return indexDescriptor == null ? null : indexDescriptor.id(); - } - - /** - * Returns table descriptor of the given table at the given node, or {@code null} if no such table exists. - * - * @param node Node. - * @param tableName Table name. - */ - private static @Nullable CatalogTableDescriptor getTableDescriptor(Ignite node, String tableName) { - IgniteImpl nodeImpl = (IgniteImpl) node; - - return getTable(nodeImpl.catalogManager(), tableName, nodeImpl.clock().nowLong()); - } -} diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItAggregatesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItAggregatesTest.java index e6b5f41a95a..fb4df325f13 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItAggregatesTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItAggregatesTest.java @@ -26,6 +26,7 @@ import java.util.Locale; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.hint.IgniteHint; import org.apache.ignite.internal.sql.engine.util.HintUtils; import org.apache.ignite.internal.sql.engine.util.QueryChecker; @@ -41,7 +42,7 @@ /** * Group of tests to verify aggregation functions. */ -public class ItAggregatesTest extends ClusterPerClassIntegrationTest { +public class ItAggregatesTest extends BaseSqlIntegrationTest { private static final String[] DISABLED_RULES = {"MapReduceHashAggregateConverterRule", "MapReduceSortAggregateConverterRule", "ColocatedHashAggregateConverterRule", "ColocatedSortAggregateConverterRule"}; diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItBuildIndexTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItBuildIndexTest.java index bb0da64ae23..6040d427a0d 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItBuildIndexTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItBuildIndexTest.java @@ -22,12 +22,10 @@ import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast; -import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -41,14 +39,17 @@ import java.util.stream.Stream; import org.apache.ignite.Ignite; import org.apache.ignite.internal.app.IgniteImpl; +import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor; import org.apache.ignite.internal.lang.IgniteStringFormatter; import org.apache.ignite.internal.raft.Command; import org.apache.ignite.internal.raft.Peer; import org.apache.ignite.internal.raft.service.RaftGroupService; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.table.TableImpl; import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand; import org.apache.ignite.network.NetworkMessage; import org.apache.ignite.raft.jraft.rpc.ActionRequest; +import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Disabled; @@ -57,7 +58,7 @@ import org.junit.jupiter.params.provider.MethodSource; /** Integration test of index building. */ -public class ItBuildIndexTest extends ClusterPerClassIntegrationTest { +public class ItBuildIndexTest extends BaseSqlIntegrationTest { private static final String ZONE_NAME = "ZONE_TABLE"; private static final String TABLE_NAME = "TEST_TABLE"; @@ -69,9 +70,7 @@ void tearDown() { sql("DROP TABLE IF EXISTS " + TABLE_NAME); sql("DROP ZONE IF EXISTS " + ZONE_NAME); - CLUSTER_NODES.stream() - .map(IgniteImpl.class::cast) - .forEach(IgniteImpl::stopDroppingMessages); + CLUSTER.runningNodes().forEach(IgniteImpl::stopDroppingMessages); } @ParameterizedTest(name = "replicas : {0}") @@ -120,19 +119,19 @@ void testChangePrimaryReplicaOnMiddleBuildIndex() throws Exception { assertThat(sendBuildIndexCommandFuture, willSucceedFast()); // Let's make sure that the indexes are eventually built. - checkIndexBuild(1, nodes(), INDEX_NAME); + checkIndexBuild(1, initialNodes(), INDEX_NAME); } /** * Prepares an index build for a primary replica change. *
    - *
  • Creates a table (replicas = {@link #nodes()}, partitions = 1) and populates it;
  • + *
  • Creates a table (replicas = {@link #initialNodes()}, partitions = 1) and populates it;
  • *
  • Creates an index;
  • *
  • Drop send {@link BuildIndexCommand} from the primary replica.
  • *
*/ private void prepareBuildIndexToChangePrimaryReplica() throws Exception { - int nodes = nodes(); + int nodes = initialNodes(); assertThat(nodes, greaterThanOrEqualTo(2)); createAndPopulateTable(nodes, 1); @@ -193,9 +192,8 @@ private static void createIndex(String indexName) throws Exception { * @param indexName Name of an index to wait for. */ private static void waitForIndex(String indexName) throws InterruptedException { - assertFalse(nullOrEmpty(CLUSTER_NODES)); assertTrue(waitForCondition( - () -> CLUSTER_NODES.stream().map(node -> getIndexDescriptor(node, indexName)).allMatch(Objects::nonNull), + () -> CLUSTER.runningNodes().map(node -> getIndexDescriptor(node, indexName)).allMatch(Objects::nonNull), 10_000) ); } @@ -213,7 +211,7 @@ private static RaftGroupService getRaftClient(Ignite node, int partitionId) { * @param partitionId Partition ID. */ private static List collectPeers(int partitionId) { - RaftGroupService raftGroupService = getRaftClient(CLUSTER_NODES.get(0), partitionId); + RaftGroupService raftGroupService = getRaftClient(CLUSTER.aliveNode(), partitionId); List peers = raftGroupService.peers(); assertNotNull(peers); @@ -272,4 +270,16 @@ private static void checkIndexBuild(int partitions, int replicas, String indexNa ); } } + + /** + * Returns the index ID from the catalog, {@code null} if there is no index. + * + * @param node Node. + * @param indexName Index name. + */ + private static @Nullable Integer indexId(Ignite node, String indexName) { + CatalogIndexDescriptor indexDescriptor = getIndexDescriptor(node, indexName); + + return indexDescriptor == null ? null : indexDescriptor.id(); + } } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCorrelatesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCorrelatesTest.java index cf87810c6c7..82ecf995c4b 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCorrelatesTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCorrelatesTest.java @@ -19,11 +19,12 @@ import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsSubPlan; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; /** Tests for correlated queries. */ -public class ItCorrelatesTest extends ClusterPerClassIntegrationTest { +public class ItCorrelatesTest extends BaseSqlIntegrationTest { private static final String DISABLED_JOIN_RULES = " /*+ DISABLE_RULE('MergeJoinConverter', 'NestedLoopJoinConverter') */ "; @AfterEach @@ -37,7 +38,7 @@ public void testCorrelatesAssignedBeforeAccess() { sql("create table test_tbl(k INTEGER primary key, v INTEGER)"); //TODO: IGNITE-16323 When the issue is not fixed the invocation required for update metadata. - CLUSTER_NODES.get(0).tables().tables(); + CLUSTER.aliveNode().tables().tables(); sql("INSERT INTO test_tbl VALUES (1, 1)"); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java index f46465ea174..bf3beea7821 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java @@ -27,6 +27,7 @@ import java.util.List; import org.apache.ignite.internal.app.IgniteImpl; import org.apache.ignite.internal.schema.Column; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.table.TableImpl; import org.apache.ignite.lang.ErrorGroups.Sql; import org.junit.jupiter.api.AfterEach; @@ -35,7 +36,7 @@ /** * Integration test for CREATE TABLE DDL command. */ -public class ItCreateTableDdlTest extends ClusterPerClassIntegrationTest { +public class ItCreateTableDdlTest extends BaseSqlIntegrationTest { @AfterEach public void dropTables() { dropAllTables(); @@ -188,7 +189,7 @@ public void testDropAndAddColumns() { public void checkSchemaUpdatedWithEqAlterColumn() { sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); - IgniteImpl node = (IgniteImpl) CLUSTER_NODES.get(0); + IgniteImpl node = (IgniteImpl) CLUSTER.aliveNode(); int tableVersionBefore = getTableStrict(node.catalogManager(), "TEST", node.clock().nowLong()).tableVersion(); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java index 1c05f2cc3ad..c2aef2ac70d 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDataTypesTest.java @@ -35,6 +35,7 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeUtil; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.util.Commons; import org.apache.ignite.internal.sql.engine.util.QueryChecker; import org.apache.ignite.lang.ErrorGroups.Sql; @@ -49,7 +50,7 @@ /** * Test SQL data types. */ -public class ItDataTypesTest extends ClusterPerClassIntegrationTest { +public class ItDataTypesTest extends BaseSqlIntegrationTest { private static final String NUMERIC_OVERFLOW_ERROR = "Numeric field overflow"; @@ -60,7 +61,7 @@ public class ItDataTypesTest extends ClusterPerClassIntegrationTest { */ @AfterEach public void dropTables() { - var igniteTables = CLUSTER_NODES.get(0).tables(); + var igniteTables = CLUSTER.aliveNode().tables(); for (var table : igniteTables.tables()) { sql("DROP TABLE " + table.name()); @@ -573,7 +574,7 @@ public String toString() { } @Override - protected int nodes() { + protected int initialNodes() { return 1; } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java index 3d7f594174f..67cc93dbb31 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.exec.rel.AbstractNode; import org.apache.ignite.internal.testframework.WithSystemProperty; import org.apache.ignite.lang.ErrorGroups.Sql; @@ -42,10 +43,10 @@ /** * Various DML tests. */ -public class ItDmlTest extends ClusterPerClassIntegrationTest { +public class ItDmlTest extends BaseSqlIntegrationTest { @Override - protected int nodes() { + protected int initialNodes() { return 3; } @@ -140,7 +141,7 @@ public void rangeReadAndExclusiveInsert() { log.info("Data was loaded."); - Transaction tx = CLUSTER_NODES.get(0).transactions().begin(); + Transaction tx = CLUSTER.aliveNode().transactions().begin(); sql(tx, "SELECT * FROM test WHERE val <= 1 ORDER BY val"); @@ -412,14 +413,14 @@ public void testMergeKeysConflict() { public void scanExecutedWithinGivenTransaction() { sql("CREATE TABLE test (id int primary key, val int)"); - Transaction tx = CLUSTER_NODES.get(0).transactions().begin(); + Transaction tx = CLUSTER.aliveNode().transactions().begin(); sql(tx, "INSERT INTO test VALUES (0, 0)"); // just inserted row should be visible within the same transaction assertEquals(1, sql(tx, "select * from test").size()); - Transaction anotherTx = CLUSTER_NODES.get(0).transactions().begin(); + Transaction anotherTx = CLUSTER.aliveNode().transactions().begin(); // just inserted row should not be visible until related transaction is committed assertEquals(0, sql(anotherTx, "select * from test").size()); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDynamicParameterTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDynamicParameterTest.java index 0e462940cd3..00d301b6e9f 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDynamicParameterTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDynamicParameterTest.java @@ -28,6 +28,7 @@ import java.util.stream.Stream; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory; import org.apache.ignite.internal.sql.engine.util.Commons; import org.apache.ignite.internal.sql.engine.util.MetadataMatcher; @@ -44,7 +45,7 @@ import org.junit.jupiter.params.provider.MethodSource; /** Dynamic parameters checks. */ -public class ItDynamicParameterTest extends ClusterPerClassIntegrationTest { +public class ItDynamicParameterTest extends BaseSqlIntegrationTest { @BeforeEach public void createTable() { @@ -278,7 +279,7 @@ private static Stream varcharCasts() { } @Override - protected int nodes() { + protected int initialNodes() { return 1; } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItFunctionsTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItFunctionsTest.java index 997f6274f8b..f816b8111b5 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItFunctionsTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItFunctionsTest.java @@ -37,6 +37,7 @@ import java.util.function.Function; import java.util.stream.Stream; import org.apache.calcite.sql.validate.SqlValidatorException; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.util.MetadataMatcher; import org.apache.ignite.lang.ErrorGroups.Sql; import org.apache.ignite.lang.IgniteException; @@ -49,7 +50,7 @@ /** * Test Ignite SQL functions. */ -public class ItFunctionsTest extends ClusterPerClassIntegrationTest { +public class ItFunctionsTest extends BaseSqlIntegrationTest { private static final Object[] NULL_RESULT = { null }; @Test diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItHashSpoolIntegrationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItHashSpoolIntegrationTest.java index cef1e09ad1f..66f892454a8 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItHashSpoolIntegrationTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItHashSpoolIntegrationTest.java @@ -17,12 +17,13 @@ package org.apache.ignite.internal.sql.engine; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.junit.jupiter.api.Test; /** * Hash spool test. */ -public class ItHashSpoolIntegrationTest extends ClusterPerClassIntegrationTest { +public class ItHashSpoolIntegrationTest extends BaseSqlIntegrationTest { @Test public void testNullsInSearchRow() { sql("CREATE TABLE t(pk varchar default gen_random_uuid PRIMARY KEY, i1 INTEGER, i2 INTEGER)"); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItHashSpoolTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItHashSpoolTest.java index 58823a6daab..ef7a97c82fa 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItHashSpoolTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItHashSpoolTest.java @@ -17,8 +17,10 @@ package org.apache.ignite.internal.sql.engine; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.logger.IgniteLogger; import org.apache.ignite.internal.logger.Loggers; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.util.QueryChecker; import org.apache.ignite.table.Table; import org.junit.jupiter.api.AfterEach; @@ -28,7 +30,7 @@ /** * Hash spool test. */ -public class ItHashSpoolTest extends ClusterPerClassIntegrationTest { +public class ItHashSpoolTest extends BaseSqlIntegrationTest { private static final IgniteLogger LOG = Loggers.forClass(ClusterPerClassIntegrationTest.class); /** @@ -40,7 +42,7 @@ protected void cleanUp() { LOG.info("Start cleanUp()"); } - for (Table table : CLUSTER_NODES.get(0).tables().tables()) { + for (Table table : CLUSTER.aliveNode().tables().tables()) { sql("DROP TABLE " + table.name()); } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItImplicitCastsTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItImplicitCastsTest.java index 596a1662104..5ad336bea83 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItImplicitCastsTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItImplicitCastsTest.java @@ -28,6 +28,7 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeUtil; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.type.IgniteCustomType; import org.apache.ignite.internal.sql.engine.type.IgniteCustomTypeCoercionRules; import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory; @@ -45,7 +46,7 @@ /** * Implicit casts are added where it is necessary to do so. */ -public class ItImplicitCastsTest extends ClusterPerClassIntegrationTest { +public class ItImplicitCastsTest extends BaseSqlIntegrationTest { @BeforeEach @AfterEach @@ -196,7 +197,7 @@ private static Stream columnPairs() { } private static void initData(ColumnPair columnPair) { - Transaction tx = CLUSTER_NODES.get(0).transactions().begin(); + Transaction tx = CLUSTER.aliveNode().transactions().begin(); sql(tx, format("INSERT INTO T11 VALUES(1, CAST({} AS {}))", columnPair.lhsLiteral(1), columnPair.lhs)); sql(tx, format("INSERT INTO T11 VALUES(2, CAST({} AS {}))", columnPair.lhsLiteral(3), columnPair.lhs)); sql(tx, format("INSERT INTO T12 VALUES(1, CAST({} AS {}))", columnPair.lhsLiteral(2), columnPair.rhs)); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexDdlTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexDdlTest.java index e32dcc95132..76900ff3aed 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexDdlTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexDdlTest.java @@ -20,6 +20,7 @@ import static org.apache.ignite.internal.catalog.CatalogService.DEFAULT_SCHEMA_NAME; import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.catalog.IndexExistsValidationException; import org.apache.ignite.internal.catalog.IndexNotFoundValidationException; import org.junit.jupiter.api.AfterEach; diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java index 945281f2185..93233f96020 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.stream.Stream; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.logger.IgniteLogger; import org.apache.ignite.internal.logger.Loggers; import org.apache.ignite.table.Table; @@ -46,7 +47,7 @@ protected void cleanUp() { LOG.info("Start cleanUp()"); } - for (Table table : CLUSTER_NODES.get(0).tables().tables()) { + for (Table table : CLUSTER.aliveNode().tables().tables()) { sql("DROP TABLE " + table.name()); sql("DROP ZONE " + "ZONE_" + table.name().toUpperCase()); } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIntervalTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIntervalTest.java index d41c0702b65..02b35a83aec 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIntervalTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIntervalTest.java @@ -28,14 +28,15 @@ import java.time.LocalTime; import java.time.Period; import org.apache.ignite.internal.lang.IgniteInternalException; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.lang.IgniteException; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; /** Interval coverage tests. */ -public class ItIntervalTest extends ClusterPerClassIntegrationTest { +public class ItIntervalTest extends BaseSqlIntegrationTest { @Override - protected int nodes() { + protected int initialNodes() { return 1; } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItJoinTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItJoinTest.java index f00a3e0d52f..7ea2a07aac4 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItJoinTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItJoinTest.java @@ -20,6 +20,7 @@ import java.util.Arrays; import java.util.List; import java.util.stream.Stream; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.util.QueryChecker; import org.apache.ignite.internal.testframework.WithSystemProperty; import org.junit.jupiter.api.Assumptions; @@ -32,7 +33,7 @@ /** * Check JOIN on basic cases. */ -public class ItJoinTest extends ClusterPerClassIntegrationTest { +public class ItJoinTest extends BaseSqlIntegrationTest { @BeforeAll public static void beforeTestsStarted() { sql("CREATE TABLE t1 (id INT PRIMARY KEY, c1 INT NOT NULL, c2 INT, c3 INT)"); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItLimitOffsetTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItLimitOffsetTest.java index 11a7a1eb14e..111988b14d6 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItLimitOffsetTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItLimitOffsetTest.java @@ -23,6 +23,7 @@ import java.math.BigDecimal; import java.util.Arrays; import java.util.List; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.util.Commons; import org.apache.ignite.lang.ErrorGroups.Sql; import org.apache.ignite.sql.Session; @@ -33,7 +34,7 @@ /** * Check LIMIT and\or OFFSET commands. */ -public class ItLimitOffsetTest extends ClusterPerClassIntegrationTest { +public class ItLimitOffsetTest extends BaseSqlIntegrationTest { @BeforeEach void beforeEach() { sql("CREATE TABLE test (pk INT PRIMARY KEY, col0 INT)"); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMetadataTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMetadataTest.java index ff626f43544..3ea29eb6bcb 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMetadataTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMetadataTest.java @@ -24,6 +24,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.util.MetadataMatcher; import org.apache.ignite.sql.ColumnType; import org.apache.ignite.sql.ResultSet; @@ -35,7 +36,7 @@ /** * Group of tests to verify the query metadata returned alongside the query result. */ -public class ItMetadataTest extends ClusterPerClassIntegrationTest { +public class ItMetadataTest extends BaseSqlIntegrationTest { /** * Before all. */ @@ -45,7 +46,7 @@ static void initTestData() { } @Override - protected int nodes() { + protected int initialNodes() { return 1; } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMixedQueriesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMixedQueriesTest.java index 1759e4026ae..d19ac7673f5 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMixedQueriesTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMixedQueriesTest.java @@ -26,13 +26,14 @@ import java.util.Arrays; import java.util.List; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; /** * Group of tests that still has not been sorted out. It’s better to avoid extending this class with new tests. */ -public class ItMixedQueriesTest extends ClusterPerClassIntegrationTest { +public class ItMixedQueriesTest extends BaseSqlIntegrationTest { /** * Before all. */ diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMultistatementTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMultistatementTest.java index fadefebd9ed..f838a74be85 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMultistatementTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMultistatementTest.java @@ -28,6 +28,7 @@ import java.util.concurrent.CompletionException; import java.util.stream.Stream; import org.apache.ignite.Ignite; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.sql.IgniteSql; import org.apache.ignite.sql.Session; import org.apache.ignite.sql.SqlException; @@ -49,7 +50,7 @@ public class ItMultistatementTest extends ClusterPerClassIntegrationTest { @ParameterizedTest @MethodSource("txControlCalls") public void testTxControlStatementsAreNotAllowedWithSingleStatementMethods(String stmtSql, ExecMethod execMethod) { - Ignite ignite = CLUSTER_NODES.get(0); + Ignite ignite = CLUSTER.aliveNode(); IgniteSql igniteSql = ignite.sql(); try (Session session = igniteSql.createSession()) { diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItOrToUnionRuleTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItOrToUnionRuleTest.java index ab685f75582..4e82af3e08b 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItOrToUnionRuleTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItOrToUnionRuleTest.java @@ -24,6 +24,7 @@ import static org.hamcrest.CoreMatchers.not; import java.util.List; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -37,7 +38,7 @@ *

SELECT * FROM products WHERE category = 'Photo' UNION ALL SELECT * FROM products WHERE subcategory ='Camera Media' AND LNNVL(category, * 'Photo'); */ -public class ItOrToUnionRuleTest extends ClusterPerClassIntegrationTest { +public class ItOrToUnionRuleTest extends BaseSqlIntegrationTest { public static final String IDX_SUBCAT_ID = "IDX_SUBCAT_ID"; public static final String IDX_SUBCATEGORY = "IDX_SUBCATEGORY"; diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItPkOnlyTableCrossApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItPkOnlyTableCrossApiTest.java index d772d5cf65b..34e10a1dfc9 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItPkOnlyTableCrossApiTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItPkOnlyTableCrossApiTest.java @@ -33,6 +33,7 @@ import java.util.Collections; import java.util.List; import java.util.function.Consumer; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.lang.IgniteException; import org.apache.ignite.lang.NullableValue; import org.apache.ignite.lang.UnexpectedNullValueException; @@ -52,12 +53,12 @@ * Tests basic operations using a different API * on a table where all columns belong to the primary key. */ -public class ItPkOnlyTableCrossApiTest extends ClusterPerClassIntegrationTest { +public class ItPkOnlyTableCrossApiTest extends BaseSqlIntegrationTest { /** Storage engine types. */ private static final String[] ENGINES = {"aipersist", "aimem", "rocksdb"}; @Override - protected int nodes() { + protected int initialNodes() { return 1; } @@ -365,11 +366,11 @@ private TestEnvironment(String engine, boolean readOnlyTx) { } private Table table() { - return CLUSTER_NODES.get(0).tables().table(tableName(engine)); + return CLUSTER.aliveNode().tables().table(tableName(engine)); } private void runInTransaction(List> writeOps, Consumer readOp) { - IgniteTransactions transactions = CLUSTER_NODES.get(0).transactions(); + IgniteTransactions transactions = CLUSTER.aliveNode().transactions(); for (Consumer writeOp : writeOps) { transactions.runInTransaction(writeOp, new TransactionOptions().readOnly(false)); @@ -379,7 +380,7 @@ private void runInTransaction(List> writeOps, Consumer writeOp, Consumer readOp) { - IgniteTransactions transactions = CLUSTER_NODES.get(0).transactions(); + IgniteTransactions transactions = CLUSTER.aliveNode().transactions(); if (readOnlyTx) { // Start a separate transaction for the write operation. diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItProjectScanMergeRuleTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItProjectScanMergeRuleTest.java index e017bd04e4e..0e5388ecb2c 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItProjectScanMergeRuleTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItProjectScanMergeRuleTest.java @@ -24,6 +24,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import java.util.List; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.lang.IgniteException; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -33,7 +34,7 @@ * with only useful columns and. For example for tables: T1(f12, f12, f13) and T2(f21, f22, f23) sql execution: SELECT t1.f11, t2.f21 FROM * T1 t1 INNER JOIN T2 t2 on t1.f11 = t2.f22" need to eleminate all unused coluns and take into account only: f11, f21 and f22 cols. */ -public class ItProjectScanMergeRuleTest extends ClusterPerClassIntegrationTest { +public class ItProjectScanMergeRuleTest extends BaseSqlIntegrationTest { public static final String IDX_CAT_ID = "IDX_CAT_ID"; /** diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java index 46914a7b1e0..fcb50d165f1 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java @@ -28,6 +28,7 @@ import java.time.LocalDate; import java.util.List; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.util.QueryChecker; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; @@ -36,7 +37,7 @@ /** * Basic index tests. */ -public class ItSecondaryIndexTest extends ClusterPerClassIntegrationTest { +public class ItSecondaryIndexTest extends BaseSqlIntegrationTest { private static final String PK_SORTED_IDX = "PK_SORTED_IDX"; private static final String DEPID_IDX = "DEPID_IDX"; diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSetOpTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSetOpTest.java index f1592795f60..4ede763a3d2 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSetOpTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSetOpTest.java @@ -25,6 +25,7 @@ import java.util.function.Predicate; import java.util.stream.Stream; import java.util.stream.StreamSupport; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.hint.IgniteHint; import org.apache.ignite.internal.sql.engine.util.HintUtils; import org.apache.ignite.internal.sql.engine.util.QueryChecker; @@ -39,7 +40,7 @@ /** * Integration test for set op (EXCEPT, INTERSECT). */ -public class ItSetOpTest extends ClusterPerClassIntegrationTest { +public class ItSetOpTest extends BaseSqlIntegrationTest { /** * Before all. */ diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSqlOperatorsTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSqlOperatorsTest.java index 9189220143c..d1887178348 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSqlOperatorsTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSqlOperatorsTest.java @@ -23,6 +23,7 @@ import java.time.Period; import java.util.List; import java.util.Map; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.sql.fun.IgniteSqlOperatorTable; import org.apache.ignite.internal.sql.engine.util.QueryChecker; import org.junit.jupiter.api.BeforeAll; @@ -36,10 +37,10 @@ * * @see IgniteSqlOperatorTable */ -public class ItSqlOperatorsTest extends ClusterPerClassIntegrationTest { +public class ItSqlOperatorsTest extends BaseSqlIntegrationTest { /** {@inheritDoc} */ @Override - protected int nodes() { + protected int initialNodes() { return 1; } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSystemViewsTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSystemViewsTest.java index 31d8d22faf0..81456242458 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSystemViewsTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSystemViewsTest.java @@ -21,6 +21,7 @@ import static org.apache.ignite.internal.sql.engine.ItSystemViewsTest.KnownSystemView.SYSTEM_VIEWS; import static org.apache.ignite.internal.sql.engine.ItSystemViewsTest.KnownSystemView.SYSTEM_VIEW_COLUMNS; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.util.MetadataMatcher; import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.sql.ColumnType; @@ -33,7 +34,7 @@ /** * End-to-end tests to verify system views. */ -public class ItSystemViewsTest extends ClusterPerClassIntegrationTest { +public class ItSystemViewsTest extends BaseSqlIntegrationTest { enum KnownSystemView { SYSTEM_VIEWS("SYSTEM", "SYSTEM_VIEWS"), SYSTEM_VIEW_COLUMNS("SYSTEM", "SYSTEM_VIEW_COLUMNS"), @@ -52,9 +53,9 @@ String canonicalName() { } } - @BeforeAll @Override - void beforeAll(TestInfo testInfo) { + @BeforeAll + protected void beforeAll(TestInfo testInfo) { super.beforeAll(testInfo); IgniteTestUtils.await(systemViewManager().completeRegistration()); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItZoneDdlTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItZoneDdlTest.java index 8589f1b05dd..40f8769046d 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItZoneDdlTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItZoneDdlTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.sql.engine; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.distributionzones.DistributionZoneAlreadyExistsException; import org.apache.ignite.internal.distributionzones.DistributionZoneNotFoundException; import org.apache.ignite.internal.testframework.IgniteTestUtils; diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/tests/BaseDataTypeTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/tests/BaseDataTypeTest.java index 93563e2ae2f..ff401274ce4 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/tests/BaseDataTypeTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/tests/BaseDataTypeTest.java @@ -26,7 +26,7 @@ import java.util.stream.Stream; import org.apache.calcite.sql.SqlKind; import org.apache.ignite.internal.app.IgniteImpl; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.engine.type.IgniteCustomTypeSpec; import org.apache.ignite.internal.sql.engine.util.Commons; import org.apache.ignite.internal.sql.engine.util.NativeTypeWrapper; @@ -66,7 +66,7 @@ * * @param A storage type of a data type. */ -public abstract class BaseDataTypeTest> extends ClusterPerClassIntegrationTest { +public abstract class BaseDataTypeTest> extends BaseSqlIntegrationTest { protected DataTypeTestSpec testTypeSpec; @@ -112,7 +112,7 @@ public void cleanTables() { protected final QueryChecker checkQuery(String query) { QueryTemplate queryTemplate = createQueryTemplate(query); - IgniteImpl node = (IgniteImpl) CLUSTER_NODES.get(0); + IgniteImpl node = (IgniteImpl) CLUSTER.aliveNode(); return queryCheckerFactory.create(node.queryEngine(), node.transactions(), this::validateMetadata, queryTemplate); } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/uuid/ItUuidQueryTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/uuid/ItUuidQueryTest.java index 1ff3310e2e8..c58164b68cb 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/uuid/ItUuidQueryTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/uuid/ItUuidQueryTest.java @@ -36,7 +36,7 @@ */ public class ItUuidQueryTest extends BaseQueryDataTypeTest { @Override - protected int nodes() { + protected int initialNodes() { return 1; } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/varbinary/ItVarBinaryExpressionTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/varbinary/ItVarBinaryExpressionTest.java index bbc77d2b7db..25b5275d689 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/varbinary/ItVarBinaryExpressionTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/datatypes/varbinary/ItVarBinaryExpressionTest.java @@ -33,7 +33,7 @@ */ public class ItVarBinaryExpressionTest extends BaseExpressionDataTypeTest { @Override - protected int nodes() { + protected int initialNodes() { return 1; } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/streamer/ItAbstractDataStreamerTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/streamer/ItAbstractDataStreamerTest.java index 575361106d3..8539432443e 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/streamer/ItAbstractDataStreamerTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/streamer/ItAbstractDataStreamerTest.java @@ -33,7 +33,7 @@ import java.util.concurrent.SubmissionPublisher; import java.util.concurrent.TimeUnit; import org.apache.ignite.Ignite; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.sql.Session; import org.apache.ignite.table.DataStreamerOptions; import org.apache.ignite.table.KeyValueView; diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/streamer/ItServerDataStreamerTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/streamer/ItServerDataStreamerTest.java index 8592cc66c58..1b4bb56c341 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/streamer/ItServerDataStreamerTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/streamer/ItServerDataStreamerTest.java @@ -25,6 +25,6 @@ public class ItServerDataStreamerTest extends ItAbstractDataStreamerTest { @Override Ignite ignite() { - return CLUSTER_NODES.get(0); + return CLUSTER.aliveNode(); } } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItPublicApiColocationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItPublicApiColocationTest.java index 107c65342c9..88d2d43b847 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItPublicApiColocationTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItPublicApiColocationTest.java @@ -39,9 +39,9 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.schema.BinaryRow; import org.apache.ignite.internal.schema.SchemaRegistry; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.testframework.WorkDirectoryExtension; import org.apache.ignite.internal.type.NativeTypeSpec; import org.apache.ignite.table.Table; @@ -73,7 +73,7 @@ public class ItPublicApiColocationTest extends ClusterPerClassIntegrationTest { @AfterEach public void dropTables() { - for (Table t : CLUSTER_NODES.get(0).tables().tables()) { + for (Table t : CLUSTER.aliveNode().tables().tables()) { sql("DROP TABLE " + t.name()); } } @@ -94,9 +94,9 @@ public void colocationOneColumn(NativeTypeSpec type) throws Exception { sql("insert into test1 values(?, ?, ?)", i, generateValueByType(i, type), 0); } - int parts = ((TableImpl) CLUSTER_NODES.get(0).tables().table("test0")).internalTable().partitions(); - TableImpl tbl0 = (TableImpl) CLUSTER_NODES.get(0).tables().table("test0"); - TableImpl tbl1 = (TableImpl) CLUSTER_NODES.get(0).tables().table("test1"); + int parts = ((TableImpl) CLUSTER.aliveNode().tables().table("test0")).internalTable().partitions(); + TableImpl tbl0 = (TableImpl) CLUSTER.aliveNode().tables().table("test0"); + TableImpl tbl1 = (TableImpl) CLUSTER.aliveNode().tables().table("test1"); for (int i = 0; i < parts; ++i) { List r0 = getAll(tbl0, i); @@ -138,9 +138,9 @@ public void colocationTwoColumns(NativeTypeSpec t0, NativeTypeSpec t1) throws Ex sql("insert into test1 values(?, ?, ?, ?)", i, generateValueByType(i, t0), generateValueByType(i, t1), 0); } - int parts = ((TableImpl) CLUSTER_NODES.get(0).tables().table("test0")).internalTable().partitions(); - TableImpl tbl0 = (TableImpl) CLUSTER_NODES.get(0).tables().table("test0"); - TableImpl tbl1 = (TableImpl) CLUSTER_NODES.get(0).tables().table("test1"); + int parts = ((TableImpl) CLUSTER.aliveNode().tables().table("test0")).internalTable().partitions(); + TableImpl tbl0 = (TableImpl) CLUSTER.aliveNode().tables().table("test0"); + TableImpl tbl1 = (TableImpl) CLUSTER.aliveNode().tables().table("test1"); Function tupleColocationExtract = (t) -> { Tuple ret = Tuple.create(); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItReadOnlyTransactionTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItReadOnlyTransactionTest.java index ce3e9d33590..ef26fdf8f8f 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItReadOnlyTransactionTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItReadOnlyTransactionTest.java @@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import org.apache.ignite.Ignite; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.app.IgniteImpl; import org.apache.ignite.internal.hlc.HybridClock; import org.apache.ignite.internal.hlc.HybridTimestamp; @@ -38,7 +39,6 @@ import org.apache.ignite.internal.schema.SchemaDescriptor; import org.apache.ignite.internal.schema.row.Row; import org.apache.ignite.internal.schema.row.RowAssembler; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.network.ClusterNode; import org.apache.ignite.tx.Transaction; @@ -61,11 +61,11 @@ public class ItReadOnlyTransactionTest extends ClusterPerClassIntegrationTest { @BeforeEach public void beforeEach() { sql(IgniteStringFormatter.format("CREATE ZONE IF NOT EXISTS {} WITH REPLICAS={}, PARTITIONS={};", - ZONE_NAME, nodes(), 10)); + ZONE_NAME, initialNodes(), 10)); sql(IgniteStringFormatter.format("CREATE TABLE {}(id INT PRIMARY KEY, val VARCHAR) WITH PRIMARY_ZONE='{}'", TABLE_NAME, ZONE_NAME)); - Ignite ignite = CLUSTER_NODES.get(0); + Ignite ignite = CLUSTER.aliveNode(); ignite.transactions().runInTransaction(tx -> { for (int i = 0; i < 100; i++) { @@ -87,12 +87,12 @@ public void afterEach() { @Test public void testFutureRead() throws Exception { - for (int i = 0; i < nodes(); i++) { - Ignite ignite = CLUSTER_NODES.get(i); + for (int i = 0; i < initialNodes(); i++) { + IgniteImpl ignite = CLUSTER.node(i); InternalTable internalTable = ((TableImpl) ignite.tables().table(TABLE_NAME)).internalTable(); SchemaDescriptor schema = ((TableImpl) ignite.tables().table(TABLE_NAME)).schemaView().lastKnownSchema(); - HybridClock clock = ((IgniteImpl) ignite).clock(); + HybridClock clock = ignite.clock(); Collection nodes = ignite.clusterNodes(); @@ -130,19 +130,19 @@ public void testFutureRead() throws Exception { } assertTrue(IgniteTestUtils.waitForCondition( - () -> checkData(null, id -> id < 100 ? ("str " + id) : ("new str " + id)) == 100 + nodes(), + () -> checkData(null, id -> id < 100 ? ("str " + id) : ("new str " + id)) == 100 + initialNodes(), 10_000 )); } @Test public void testPastRead() throws Exception { - for (int i = 0; i < nodes(); i++) { - Ignite ignite = CLUSTER_NODES.get(i); + for (int i = 0; i < initialNodes(); i++) { + IgniteImpl ignite = CLUSTER.node(i); InternalTable internalTable = ((TableImpl) ignite.tables().table(TABLE_NAME)).internalTable(); SchemaDescriptor schema = ((TableImpl) ignite.tables().table(TABLE_NAME)).schemaView().lastKnownSchema(); - HybridClock clock = ((IgniteImpl) ignite).clock(); + HybridClock clock = ignite.clock(); Collection nodes = ignite.clusterNodes(); @@ -173,7 +173,7 @@ public void testPastRead() throws Exception { } } - assertEquals(100 - nodes(), checkData(null, id -> "str " + id)); + assertEquals(100 - initialNodes(), checkData(null, id -> "str " + id)); } private static Row createRow(SchemaDescriptor schema, int id) { diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java index 2e889aa01a5..eb581f00059 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java @@ -61,7 +61,7 @@ import org.apache.ignite.internal.schema.SchemaDescriptor; import org.apache.ignite.internal.schema.row.Row; import org.apache.ignite.internal.schema.row.RowAssembler; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.internal.testframework.IgniteTestUtils.RunnableX; import org.apache.ignite.internal.tx.InternalTransaction; @@ -84,7 +84,7 @@ /** * Tests to check a scan internal command. */ -public class ItTableScanTest extends ClusterPerClassIntegrationTest { +public class ItTableScanTest extends BaseSqlIntegrationTest { /** Table name. */ private static final String TABLE_NAME = "test"; @@ -354,7 +354,7 @@ public void testGetAndDeleteDuringPureTableScan() throws Exception { * @throws Exception If failed. */ public void pureTableScan(Function> txOperationAction) throws Exception { - InternalTransaction tx = (InternalTransaction) CLUSTER_NODES.get(0).transactions().begin(); + InternalTransaction tx = (InternalTransaction) CLUSTER.aliveNode().transactions().begin(); log.info("Old transaction [id={}]", tx.id()); @@ -646,7 +646,7 @@ public void testMvScan(boolean readOnly) throws Exception { List assignments = internalTable.assignments(); // Any node from assignments will do it. - ClusterNode node0 = CLUSTER_NODES.get(0).clusterNodes().stream().filter(clusterNode -> { + ClusterNode node0 = CLUSTER.aliveNode().clusterNodes().stream().filter(clusterNode -> { return assignments.contains(clusterNode.name()); }).findFirst().orElseThrow(); @@ -742,7 +742,7 @@ private static void clearData(TableImpl table) { * Gets an index id. */ private static int getSortedIndexId() { - CatalogManager catalogManager = ((IgniteImpl) CLUSTER_NODES.get(0)).catalogManager(); + CatalogManager catalogManager = ((IgniteImpl) CLUSTER.aliveNode()).catalogManager(); int catalogVersion = catalogManager.latestCatalogVersion(); @@ -805,7 +805,7 @@ private static TableImpl getOrCreateTable() { sql("CREATE INDEX IF NOT EXISTS " + SORTED_IDX + " ON " + TABLE_NAME + " USING TREE (valInt)"); - return (TableImpl) CLUSTER_NODES.get(0).tables().table(TABLE_NAME); + return (TableImpl) CLUSTER.aliveNode().tables().table(TABLE_NAME); } /** @@ -872,7 +872,7 @@ private static Row createKeyRow(int id) { * @return Transaction. */ private InternalTransaction startTxWithEnlistedPartition(int partId, boolean readOnly) { - Ignite ignite = CLUSTER_NODES.get(0); + Ignite ignite = CLUSTER.aliveNode(); InternalTransaction tx = (InternalTransaction) ignite.transactions().begin(new TransactionOptions().readOnly(readOnly)); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItThinClientColocationTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItThinClientColocationTest.java index 9f444b4f4ce..8be8a69bf98 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItThinClientColocationTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItThinClientColocationTest.java @@ -27,6 +27,7 @@ import java.util.stream.Stream; import org.apache.ignite.client.IgniteClient; import org.apache.ignite.client.handler.requests.table.ClientTableCommon; +import org.apache.ignite.internal.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.catalog.commands.CatalogUtils; import org.apache.ignite.internal.client.table.ClientColumn; import org.apache.ignite.internal.client.table.ClientSchema; @@ -37,7 +38,6 @@ import org.apache.ignite.internal.schema.marshaller.TupleMarshallerException; import org.apache.ignite.internal.schema.marshaller.TupleMarshallerImpl; import org.apache.ignite.internal.schema.row.Row; -import org.apache.ignite.internal.sql.engine.ClusterPerClassIntegrationTest; import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.internal.type.NativeType; import org.apache.ignite.internal.type.NativeTypes; @@ -82,7 +82,7 @@ public void testCustomColocationColumnOrder(boolean reverseColocationOrder) thro sql("create table " + tableName + "(id integer, id0 bigint, id1 varchar, v INTEGER, " + "primary key(id, id0, id1)) colocate by " + (reverseColocationOrder ? "(id1, id0)" : "(id0, id1)")); - Table serverTable = CLUSTER_NODES.get(0).tables().table(tableName); + Table serverTable = CLUSTER.aliveNode().tables().table(tableName); RecordBinaryViewImpl serverView = (RecordBinaryViewImpl) serverTable.recordView(); TupleMarshaller marsh = serverView.marshaller(1); diff --git a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java new file mode 100644 index 00000000000..b217804b337 --- /dev/null +++ b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java @@ -0,0 +1,167 @@ +/* + * 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; + +import java.nio.file.Path; +import java.util.List; +import java.util.Objects; +import org.apache.ignite.internal.app.IgniteImpl; +import org.apache.ignite.internal.lang.IgniteStringFormatter; +import org.apache.ignite.internal.sql.engine.util.SqlTestUtils; +import org.apache.ignite.internal.testframework.WorkDirectory; +import org.apache.ignite.table.Table; +import org.apache.ignite.tx.Transaction; +import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.api.TestInstance; + +/** + * Abstract basic integration test that starts a cluster once for all the tests it runs. + */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class ClusterPerClassIntegrationTest extends IgniteIntegrationTest { + /** Test default table name. */ + protected static final String DEFAULT_TABLE_NAME = "person"; + + /** Nodes bootstrap configuration pattern. */ + private static final String NODE_BOOTSTRAP_CFG_TEMPLATE = "{\n" + + " network: {\n" + + " port: {},\n" + + " nodeFinder: {\n" + + " netClusterNodes: [ {} ]\n" + + " }\n" + + " },\n" + + " clientConnector: { port:{} },\n" + + " rest.port: {}\n" + + "}"; + + /** Cluster nodes. */ + protected static Cluster CLUSTER; + + /** Work directory. */ + @WorkDirectory + private static Path WORK_DIR; + + /** + * Before all. + * + * @param testInfo Test information object. + */ + @BeforeAll + protected void beforeAll(TestInfo testInfo) { + CLUSTER = new Cluster(testInfo, WORK_DIR, getNodeBootstrapConfigTemplate()); + + if (initialNodes() > 0) { + CLUSTER.startAndInit(initialNodes(), cmgMetastoreNodes()); + } + } + + /** + * Get a count of nodes in the Ignite cluster. + * + * @return Count of nodes. + */ + protected int initialNodes() { + return 3; + } + + protected int[] cmgMetastoreNodes() { + return new int[] { 0 }; + } + + /** + * Returns node bootstrap config template. + * + * @return Node bootstrap config template. + */ + protected String getNodeBootstrapConfigTemplate() { + return NODE_BOOTSTRAP_CFG_TEMPLATE; + } + + /** + * After all. + */ + @AfterAll + void afterAll() throws Exception { + CLUSTER.shutdown(); + } + + /** Drops all visible tables. */ + protected static void dropAllTables() { + for (Table t : CLUSTER.aliveNode().tables().tables()) { + sql("DROP TABLE " + t.name()); + } + } + + /** + * Creates a table. + * + * @param name Table name. + * @param replicas Replica factor. + * @param partitions Partitions count. + */ + protected static Table createTable(String name, int replicas, int partitions) { + sql(IgniteStringFormatter.format("CREATE ZONE IF NOT EXISTS {} WITH REPLICAS={}, PARTITIONS={};", + "ZONE_" + name.toUpperCase(), replicas, partitions)); + sql(IgniteStringFormatter.format("CREATE TABLE IF NOT EXISTS {} (id INT PRIMARY KEY, name VARCHAR, salary DOUBLE) " + + "WITH PRIMARY_ZONE='{}'", name, "ZONE_" + name.toUpperCase())); + + return CLUSTER.node(0).tables().table(name); + } + + protected static void insertData(String tblName, List columnNames, Object[]... tuples) { + Transaction tx = CLUSTER.node(0).transactions().begin(); + + insertDataInTransaction(tx, tblName, columnNames, tuples); + + tx.commit(); + } + + protected static void insertDataInTransaction(Transaction tx, String tblName, List columnNames, Object[]... tuples) { + String insertStmt = "INSERT INTO " + tblName + "(" + String.join(", ", columnNames) + ")" + + " VALUES (" + ", ?".repeat(columnNames.size()).substring(2) + ")"; + + for (Object[] args : tuples) { + sql(tx, insertStmt, args); + } + } + + protected static List> sql(String sql, Object... args) { + return sql(null, sql, args); + } + + protected static List> sql(@Nullable Transaction tx, String sql, Object... args) { + return SqlTestUtils.sql(CLUSTER.node(0), tx, sql, args); + } + + /** + * Looks up a node by a consistent ID, {@code null} if absent. + * + * @param consistentId Node consistent ID. + */ + protected static @Nullable IgniteImpl findByConsistentId(String consistentId) { + return CLUSTER.runningNodes() + .filter(Objects::nonNull) + .map(IgniteImpl.class::cast) + .filter(ignite -> consistentId.equals(ignite.name())) + .findFirst() + .orElse(null); + } +}