lowka commented on code in PR #2613: URL: https://github.com/apache/ignite-3/pull/2613#discussion_r1348484039
########## modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java: ########## @@ -0,0 +1,860 @@ +/* + * 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.api; + +import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsIndexScan; +import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsTableScan; +import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.asStream; +import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.assertThrowsSqlException; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.ignite.internal.catalog.commands.CatalogUtils; +import org.apache.ignite.internal.client.sql.ClientSql; +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; +import org.apache.ignite.lang.ColumnAlreadyExistsException; +import org.apache.ignite.lang.ColumnNotFoundException; +import org.apache.ignite.lang.ErrorGroups; +import org.apache.ignite.lang.ErrorGroups.Catalog; +import org.apache.ignite.lang.ErrorGroups.Index; +import org.apache.ignite.lang.ErrorGroups.Sql; +import org.apache.ignite.lang.ErrorGroups.Transactions; +import org.apache.ignite.lang.IgniteException; +import org.apache.ignite.lang.IndexAlreadyExistsException; +import org.apache.ignite.lang.IndexNotFoundException; +import org.apache.ignite.lang.TableAlreadyExistsException; +import org.apache.ignite.lang.TableNotFoundException; +import org.apache.ignite.sql.BatchedArguments; +import org.apache.ignite.sql.ColumnMetadata; +import org.apache.ignite.sql.ColumnType; +import org.apache.ignite.sql.CursorClosedException; +import org.apache.ignite.sql.IgniteSql; +import org.apache.ignite.sql.NoRowSetExpectedException; +import org.apache.ignite.sql.ResultSet; +import org.apache.ignite.sql.ResultSetMetadata; +import org.apache.ignite.sql.Session; +import org.apache.ignite.sql.SqlBatchException; +import org.apache.ignite.sql.SqlException; +import org.apache.ignite.sql.SqlRow; +import org.apache.ignite.table.Table; +import org.apache.ignite.tx.Transaction; +import org.apache.ignite.tx.TransactionOptions; +import org.hamcrest.Matcher; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +/** + * Tests for SQL API. + * Tests will be run through synchronous, asynchronous API and client entry points. + * 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 { + protected static final int ROW_COUNT = 16; + + @AfterEach + public void dropTables() { + for (Table t : CLUSTER_NODES.get(0).tables().tables()) { + sql("DROP TABLE " + t.name()); + } + } + + @Test + @Disabled("https://issues.apache.org/jira/browse/IGNITE-20096") + public void ddl() throws Exception { + IgniteSql sql = igniteSql(); + Session ses = sql.createSession(); + + // CREATE TABLE + checkDdl(true, ses, "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + checkError( + TableAlreadyExistsException.class, + ErrorGroups.Table.TABLE_ALREADY_EXISTS_ERR, + "Table already exists [name=\"PUBLIC\".\"TEST\"]", + ses, + "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)" + ); + checkSqlError( + ErrorGroups.Table.TABLE_DEFINITION_ERR, + "Can't create table with duplicate columns: ID, VAL, VAL", + ses, + "CREATE TABLE TEST1(ID INT PRIMARY KEY, VAL INT, VAL INT)" + ); + checkDdl(false, ses, "CREATE TABLE IF NOT EXISTS TEST(ID INT PRIMARY KEY, VAL VARCHAR)"); + + // ADD COLUMN + checkDdl(true, ses, "ALTER TABLE TEST ADD COLUMN VAL1 VARCHAR"); + checkError( + TableNotFoundException.class, + ErrorGroups.Table.TABLE_NOT_FOUND_ERR, + "The table does not exist [name=\"PUBLIC\".\"NOT_EXISTS_TABLE\"]", + ses, + "ALTER TABLE NOT_EXISTS_TABLE ADD COLUMN VAL1 VARCHAR" + ); + checkDdl(false, ses, "ALTER TABLE IF EXISTS NOT_EXISTS_TABLE ADD COLUMN VAL1 VARCHAR"); + checkError( + ColumnAlreadyExistsException.class, + ErrorGroups.Table.COLUMN_ALREADY_EXISTS_ERR, + "Column already exists [name=\"VAL1\"]", + ses, + "ALTER TABLE TEST ADD COLUMN VAL1 INT" + ); + + // CREATE INDEX + checkDdl(true, ses, "CREATE INDEX TEST_IDX ON TEST(VAL0)"); + checkError( + IndexAlreadyExistsException.class, + Index.INDEX_ALREADY_EXISTS_ERR, + "Index already exists [name=\"PUBLIC\".\"TEST_IDX\"]", + ses, + "CREATE INDEX TEST_IDX ON TEST(VAL1)" + ); + checkDdl(false, ses, "CREATE INDEX IF NOT EXISTS TEST_IDX ON TEST(VAL1)"); + + // TODO: IGNITE-19150 We are waiting for schema synchronization to avoid races to create and destroy indexes + waitForIndexBuild("TEST", "TEST_IDX"); + + checkDdl(true, ses, "DROP INDEX TESt_iDX"); + checkDdl(true, ses, "CREATE INDEX TEST_IDX1 ON TEST(VAL0)"); + checkDdl(true, ses, "CREATE INDEX TEST_IDX2 ON TEST(VAL0)"); + checkDdl(true, ses, "CREATE INDEX TEST_IDX3 ON TEST(ID, VAL0, VAL1)"); + checkSqlError( + Index.INVALID_INDEX_DEFINITION_ERR, + "Can't create index on duplicate columns: VAL0, VAL0", + ses, + "CREATE INDEX TEST_IDX4 ON TEST(VAL0, VAL0)" + ); + + checkSqlError( + Sql.STMT_VALIDATION_ERR, + "Can`t delete column(s). Column VAL1 is used by indexes [TEST_IDX3].", + ses, + "ALTER TABLE TEST DROP COLUMN val1" + ); + + SqlException ex = checkSqlError( + Sql.STMT_VALIDATION_ERR, + "Can`t delete column(s).", + ses, + "ALTER TABLE TEST DROP COLUMN (val0, val1)" + ); + + String msg = ex.getMessage(); + String explainMsg = "Unexpected error message: " + msg; + + assertTrue(msg.contains("Column VAL0 is used by indexes ["), explainMsg); + assertTrue(msg.contains("TEST_IDX1") && msg.contains("TEST_IDX2") && msg.contains("TEST_IDX3"), explainMsg); + assertTrue(msg.contains("Column VAL1 is used by indexes [TEST_IDX3]"), explainMsg); + + checkSqlError( + Sql.STMT_VALIDATION_ERR, + "Can`t delete column, belongs to primary key: [name=ID]", + ses, + "ALTER TABLE TEST DROP COLUMN id" + ); + + // TODO: IGNITE-19150 We are waiting for schema synchronization to avoid races to create and destroy indexes + waitForIndexBuild("TEST", "TEST_IDX3"); + checkDdl(true, ses, "DROP INDEX TESt_iDX3"); + + // DROP COLUMNS + checkDdl(true, ses, "ALTER TABLE TEST DROP COLUMN VAL1"); + checkError( + TableNotFoundException.class, + ErrorGroups.Table.TABLE_NOT_FOUND_ERR, + "The table does not exist [name=\"PUBLIC\".\"NOT_EXISTS_TABLE\"]", + ses, + "ALTER TABLE NOT_EXISTS_TABLE DROP COLUMN VAL1" + ); + checkDdl(false, ses, "ALTER TABLE IF EXISTS NOT_EXISTS_TABLE DROP COLUMN VAL1"); + checkError( + ColumnNotFoundException.class, + ErrorGroups.Table.COLUMN_NOT_FOUND_ERR, + "Column does not exist [tableName=\"PUBLIC\".\"TEST\", columnName=\"VAL1\"]", + ses, + "ALTER TABLE TEST DROP COLUMN VAL1" + ); + + // DROP TABLE + checkDdl(false, ses, "DROP TABLE IF EXISTS NOT_EXISTS_TABLE"); + + checkDdl(true, ses, "DROP TABLE TEST"); + checkError( + TableNotFoundException.class, + ErrorGroups.Table.TABLE_NOT_FOUND_ERR, + "The table does not exist [name=\"PUBLIC\".\"TEST\"]", + ses, + "DROP TABLE TEST" + ); + + checkDdl(false, ses, "DROP INDEX IF EXISTS TEST_IDX"); + + checkError( + IndexNotFoundException.class, + Index.INDEX_NOT_FOUND_ERR, + "Index does not exist [name=\"PUBLIC\".\"TEST_IDX\"]", ses, + "DROP INDEX TEST_IDX" + ); + } + + /** Check all transactions are processed correctly even with case of sql Exception raised. */ + @Test + public void implicitTransactionsStates() { + IgniteSql sql = igniteSql(); + + if (sql instanceof ClientSql) { + return; + } + + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + + Session ses = sql.createSession(); + + TxManager txManager = txManager(); + + for (int i = 0; i < ROW_COUNT; ++i) { + assertThrowsSqlException( + Catalog.VALIDATION_ERR, + "Table with name 'PUBLIC.TEST' already exists", + () -> execute(ses, "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)") + ); + } + + // No new transactions through ddl. + assertEquals(0, txManager.pending()); + } + + /** Check correctness of implicit and explicit transactions. */ + @Test + public void checkTransactionsWithDml() { + IgniteSql sql = igniteSql(); + + if (sql instanceof ClientSql) { + return; + } + + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + + Session ses = sql.createSession(); + + TxManager txManagerInternal = txManager(); + + int txPrevCnt = txManagerInternal.finished(); + + for (int i = 0; i < ROW_COUNT; ++i) { + checkDml(1, ses, "INSERT INTO TEST VALUES (?, ?)", i, i); + } + + // Outer tx with further commit. + Transaction outerTx = igniteTx().begin(); + + for (int i = ROW_COUNT; i < 2 * ROW_COUNT; ++i) { + checkDml(1, outerTx, ses, "INSERT INTO TEST VALUES (?, ?)", i, i); + } + + commit(outerTx); + + // Outdated tx. + Transaction outerTx0 = outerTx; + //ToDo: IGNITE-20387 , here should be used assertThrowsSqlException method with code and message `"Transaction is already finished" + IgniteException e = assertThrows(IgniteException.class, + () -> checkDml(1, outerTx0, ses, "INSERT INTO TEST VALUES (?, ?)", ROW_COUNT, Integer.MAX_VALUE)); + assertEquals(Transactions.TX_FAILED_READ_WRITE_OPERATION_ERR, e.code()); + + assertThrowsSqlException( + Sql.CONSTRAINT_VIOLATION_ERR, + "PK unique constraint is violated", + () -> checkDml(1, ses, "INSERT INTO TEST VALUES (?, ?)", ROW_COUNT, Integer.MAX_VALUE)); + + ResultSet<SqlRow> rs = executeForRead(ses, "SELECT VAL0 FROM TEST ORDER BY VAL0"); + + assertEquals(2 * ROW_COUNT, asStream(rs).count()); + + rs.close(); + + outerTx = igniteTx().begin(); + + rs = executeForRead(ses, outerTx, "SELECT VAL0 FROM TEST ORDER BY VAL0"); + + assertEquals(2 * ROW_COUNT, asStream(rs).count()); + + rs.close(); + + outerTx.commit(); + + checkDml(2 * ROW_COUNT, ses, "UPDATE TEST SET VAL0 = VAL0 + ?", 1); + + checkDml(2 * ROW_COUNT, ses, "DELETE FROM TEST WHERE VAL0 >= 0"); + + assertEquals(ROW_COUNT + 1 + 1 + 1 + 1 + 1 + 1, txManagerInternal.finished() - txPrevCnt); + + assertEquals(0, txManagerInternal.pending()); + } + + /** Check correctness of explicit transaction rollback. */ + @Test + public void checkExplicitTxRollback() { + IgniteSql sql = igniteSql(); + + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + + Session ses = sql.createSession(); + + // Outer tx with further commit. + Transaction outerTx = igniteTx().begin(); + + for (int i = 0; i < ROW_COUNT; ++i) { + checkDml(1, outerTx, ses, "INSERT INTO TEST VALUES (?, ?)", i, i); + } + + rollback(outerTx); + + ResultSet<SqlRow> rs = executeForRead(ses, "SELECT VAL0 FROM TEST ORDER BY VAL0"); + + asStream(rs); + assertEquals(0, asStream(rs).count()); + + rs.close(); + } + + /** Check correctness of rw and ro transactions for table scan. */ + @Test + public void checkMixedTransactionsForTable() { + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + + Matcher<String> planMatcher = containsTableScan("PUBLIC", "TEST"); + + checkMixedTransactions(planMatcher); + } + + + /** Check correctness of rw and ro transactions for index scan. */ + @Test + public void checkMixedTransactionsForIndex() throws Exception { + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + sql("CREATE INDEX TEST_IDX ON TEST(VAL0)"); + + Matcher<String> planMatcher = containsIndexScan("PUBLIC", "TEST", "TEST_IDX"); + + checkMixedTransactions(planMatcher); + } + + private void checkMixedTransactions(Matcher<String> planMatcher) { + IgniteSql sql = igniteSql(); + + if (sql instanceof ClientSql) { Review Comment: I think it is better to move such guards into a test case body directly, so we know that the test is not expected to run for sql clients. ########## modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java: ########## @@ -0,0 +1,860 @@ +/* + * 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.api; + +import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsIndexScan; +import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsTableScan; +import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.asStream; +import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.assertThrowsSqlException; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.ignite.internal.catalog.commands.CatalogUtils; +import org.apache.ignite.internal.client.sql.ClientSql; +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; +import org.apache.ignite.lang.ColumnAlreadyExistsException; +import org.apache.ignite.lang.ColumnNotFoundException; +import org.apache.ignite.lang.ErrorGroups; +import org.apache.ignite.lang.ErrorGroups.Catalog; +import org.apache.ignite.lang.ErrorGroups.Index; +import org.apache.ignite.lang.ErrorGroups.Sql; +import org.apache.ignite.lang.ErrorGroups.Transactions; +import org.apache.ignite.lang.IgniteException; +import org.apache.ignite.lang.IndexAlreadyExistsException; +import org.apache.ignite.lang.IndexNotFoundException; +import org.apache.ignite.lang.TableAlreadyExistsException; +import org.apache.ignite.lang.TableNotFoundException; +import org.apache.ignite.sql.BatchedArguments; +import org.apache.ignite.sql.ColumnMetadata; +import org.apache.ignite.sql.ColumnType; +import org.apache.ignite.sql.CursorClosedException; +import org.apache.ignite.sql.IgniteSql; +import org.apache.ignite.sql.NoRowSetExpectedException; +import org.apache.ignite.sql.ResultSet; +import org.apache.ignite.sql.ResultSetMetadata; +import org.apache.ignite.sql.Session; +import org.apache.ignite.sql.SqlBatchException; +import org.apache.ignite.sql.SqlException; +import org.apache.ignite.sql.SqlRow; +import org.apache.ignite.table.Table; +import org.apache.ignite.tx.Transaction; +import org.apache.ignite.tx.TransactionOptions; +import org.hamcrest.Matcher; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +/** + * Tests for SQL API. + * Tests will be run through synchronous, asynchronous API and client entry points. + * 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 { + protected static final int ROW_COUNT = 16; + + @AfterEach + public void dropTables() { + for (Table t : CLUSTER_NODES.get(0).tables().tables()) { + sql("DROP TABLE " + t.name()); + } + } + + @Test + @Disabled("https://issues.apache.org/jira/browse/IGNITE-20096") + public void ddl() throws Exception { + IgniteSql sql = igniteSql(); + Session ses = sql.createSession(); + + // CREATE TABLE + checkDdl(true, ses, "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + checkError( + TableAlreadyExistsException.class, + ErrorGroups.Table.TABLE_ALREADY_EXISTS_ERR, + "Table already exists [name=\"PUBLIC\".\"TEST\"]", + ses, + "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)" + ); + checkSqlError( + ErrorGroups.Table.TABLE_DEFINITION_ERR, + "Can't create table with duplicate columns: ID, VAL, VAL", + ses, + "CREATE TABLE TEST1(ID INT PRIMARY KEY, VAL INT, VAL INT)" + ); + checkDdl(false, ses, "CREATE TABLE IF NOT EXISTS TEST(ID INT PRIMARY KEY, VAL VARCHAR)"); + + // ADD COLUMN + checkDdl(true, ses, "ALTER TABLE TEST ADD COLUMN VAL1 VARCHAR"); + checkError( + TableNotFoundException.class, + ErrorGroups.Table.TABLE_NOT_FOUND_ERR, + "The table does not exist [name=\"PUBLIC\".\"NOT_EXISTS_TABLE\"]", + ses, + "ALTER TABLE NOT_EXISTS_TABLE ADD COLUMN VAL1 VARCHAR" + ); + checkDdl(false, ses, "ALTER TABLE IF EXISTS NOT_EXISTS_TABLE ADD COLUMN VAL1 VARCHAR"); + checkError( + ColumnAlreadyExistsException.class, + ErrorGroups.Table.COLUMN_ALREADY_EXISTS_ERR, + "Column already exists [name=\"VAL1\"]", + ses, + "ALTER TABLE TEST ADD COLUMN VAL1 INT" + ); + + // CREATE INDEX + checkDdl(true, ses, "CREATE INDEX TEST_IDX ON TEST(VAL0)"); + checkError( + IndexAlreadyExistsException.class, + Index.INDEX_ALREADY_EXISTS_ERR, + "Index already exists [name=\"PUBLIC\".\"TEST_IDX\"]", + ses, + "CREATE INDEX TEST_IDX ON TEST(VAL1)" + ); + checkDdl(false, ses, "CREATE INDEX IF NOT EXISTS TEST_IDX ON TEST(VAL1)"); + + // TODO: IGNITE-19150 We are waiting for schema synchronization to avoid races to create and destroy indexes + waitForIndexBuild("TEST", "TEST_IDX"); + + checkDdl(true, ses, "DROP INDEX TESt_iDX"); + checkDdl(true, ses, "CREATE INDEX TEST_IDX1 ON TEST(VAL0)"); + checkDdl(true, ses, "CREATE INDEX TEST_IDX2 ON TEST(VAL0)"); + checkDdl(true, ses, "CREATE INDEX TEST_IDX3 ON TEST(ID, VAL0, VAL1)"); + checkSqlError( + Index.INVALID_INDEX_DEFINITION_ERR, + "Can't create index on duplicate columns: VAL0, VAL0", + ses, + "CREATE INDEX TEST_IDX4 ON TEST(VAL0, VAL0)" + ); + + checkSqlError( + Sql.STMT_VALIDATION_ERR, + "Can`t delete column(s). Column VAL1 is used by indexes [TEST_IDX3].", + ses, + "ALTER TABLE TEST DROP COLUMN val1" + ); + + SqlException ex = checkSqlError( + Sql.STMT_VALIDATION_ERR, + "Can`t delete column(s).", + ses, + "ALTER TABLE TEST DROP COLUMN (val0, val1)" + ); + + String msg = ex.getMessage(); + String explainMsg = "Unexpected error message: " + msg; + + assertTrue(msg.contains("Column VAL0 is used by indexes ["), explainMsg); + assertTrue(msg.contains("TEST_IDX1") && msg.contains("TEST_IDX2") && msg.contains("TEST_IDX3"), explainMsg); + assertTrue(msg.contains("Column VAL1 is used by indexes [TEST_IDX3]"), explainMsg); + + checkSqlError( + Sql.STMT_VALIDATION_ERR, + "Can`t delete column, belongs to primary key: [name=ID]", + ses, + "ALTER TABLE TEST DROP COLUMN id" + ); + + // TODO: IGNITE-19150 We are waiting for schema synchronization to avoid races to create and destroy indexes + waitForIndexBuild("TEST", "TEST_IDX3"); + checkDdl(true, ses, "DROP INDEX TESt_iDX3"); + + // DROP COLUMNS + checkDdl(true, ses, "ALTER TABLE TEST DROP COLUMN VAL1"); + checkError( + TableNotFoundException.class, + ErrorGroups.Table.TABLE_NOT_FOUND_ERR, + "The table does not exist [name=\"PUBLIC\".\"NOT_EXISTS_TABLE\"]", + ses, + "ALTER TABLE NOT_EXISTS_TABLE DROP COLUMN VAL1" + ); + checkDdl(false, ses, "ALTER TABLE IF EXISTS NOT_EXISTS_TABLE DROP COLUMN VAL1"); + checkError( + ColumnNotFoundException.class, + ErrorGroups.Table.COLUMN_NOT_FOUND_ERR, + "Column does not exist [tableName=\"PUBLIC\".\"TEST\", columnName=\"VAL1\"]", + ses, + "ALTER TABLE TEST DROP COLUMN VAL1" + ); + + // DROP TABLE + checkDdl(false, ses, "DROP TABLE IF EXISTS NOT_EXISTS_TABLE"); + + checkDdl(true, ses, "DROP TABLE TEST"); + checkError( + TableNotFoundException.class, + ErrorGroups.Table.TABLE_NOT_FOUND_ERR, + "The table does not exist [name=\"PUBLIC\".\"TEST\"]", + ses, + "DROP TABLE TEST" + ); + + checkDdl(false, ses, "DROP INDEX IF EXISTS TEST_IDX"); + + checkError( + IndexNotFoundException.class, + Index.INDEX_NOT_FOUND_ERR, + "Index does not exist [name=\"PUBLIC\".\"TEST_IDX\"]", ses, + "DROP INDEX TEST_IDX" + ); + } + + /** Check all transactions are processed correctly even with case of sql Exception raised. */ + @Test + public void implicitTransactionsStates() { + IgniteSql sql = igniteSql(); + + if (sql instanceof ClientSql) { + return; + } + + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + + Session ses = sql.createSession(); + + TxManager txManager = txManager(); + + for (int i = 0; i < ROW_COUNT; ++i) { + assertThrowsSqlException( + Catalog.VALIDATION_ERR, + "Table with name 'PUBLIC.TEST' already exists", + () -> execute(ses, "CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)") + ); + } + + // No new transactions through ddl. + assertEquals(0, txManager.pending()); + } + + /** Check correctness of implicit and explicit transactions. */ + @Test + public void checkTransactionsWithDml() { + IgniteSql sql = igniteSql(); + + if (sql instanceof ClientSql) { + return; + } + + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + + Session ses = sql.createSession(); + + TxManager txManagerInternal = txManager(); + + int txPrevCnt = txManagerInternal.finished(); + + for (int i = 0; i < ROW_COUNT; ++i) { + checkDml(1, ses, "INSERT INTO TEST VALUES (?, ?)", i, i); + } + + // Outer tx with further commit. + Transaction outerTx = igniteTx().begin(); + + for (int i = ROW_COUNT; i < 2 * ROW_COUNT; ++i) { + checkDml(1, outerTx, ses, "INSERT INTO TEST VALUES (?, ?)", i, i); + } + + commit(outerTx); + + // Outdated tx. + Transaction outerTx0 = outerTx; + //ToDo: IGNITE-20387 , here should be used assertThrowsSqlException method with code and message `"Transaction is already finished" + IgniteException e = assertThrows(IgniteException.class, + () -> checkDml(1, outerTx0, ses, "INSERT INTO TEST VALUES (?, ?)", ROW_COUNT, Integer.MAX_VALUE)); + assertEquals(Transactions.TX_FAILED_READ_WRITE_OPERATION_ERR, e.code()); + + assertThrowsSqlException( + Sql.CONSTRAINT_VIOLATION_ERR, + "PK unique constraint is violated", + () -> checkDml(1, ses, "INSERT INTO TEST VALUES (?, ?)", ROW_COUNT, Integer.MAX_VALUE)); + + ResultSet<SqlRow> rs = executeForRead(ses, "SELECT VAL0 FROM TEST ORDER BY VAL0"); + + assertEquals(2 * ROW_COUNT, asStream(rs).count()); + + rs.close(); + + outerTx = igniteTx().begin(); + + rs = executeForRead(ses, outerTx, "SELECT VAL0 FROM TEST ORDER BY VAL0"); + + assertEquals(2 * ROW_COUNT, asStream(rs).count()); + + rs.close(); + + outerTx.commit(); + + checkDml(2 * ROW_COUNT, ses, "UPDATE TEST SET VAL0 = VAL0 + ?", 1); + + checkDml(2 * ROW_COUNT, ses, "DELETE FROM TEST WHERE VAL0 >= 0"); + + assertEquals(ROW_COUNT + 1 + 1 + 1 + 1 + 1 + 1, txManagerInternal.finished() - txPrevCnt); + + assertEquals(0, txManagerInternal.pending()); + } + + /** Check correctness of explicit transaction rollback. */ + @Test + public void checkExplicitTxRollback() { + IgniteSql sql = igniteSql(); + + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + + Session ses = sql.createSession(); + + // Outer tx with further commit. + Transaction outerTx = igniteTx().begin(); + + for (int i = 0; i < ROW_COUNT; ++i) { + checkDml(1, outerTx, ses, "INSERT INTO TEST VALUES (?, ?)", i, i); + } + + rollback(outerTx); + + ResultSet<SqlRow> rs = executeForRead(ses, "SELECT VAL0 FROM TEST ORDER BY VAL0"); + + asStream(rs); + assertEquals(0, asStream(rs).count()); + + rs.close(); + } + + /** Check correctness of rw and ro transactions for table scan. */ + @Test + public void checkMixedTransactionsForTable() { + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + + Matcher<String> planMatcher = containsTableScan("PUBLIC", "TEST"); + + checkMixedTransactions(planMatcher); + } + + + /** Check correctness of rw and ro transactions for index scan. */ + @Test + public void checkMixedTransactionsForIndex() throws Exception { + sql("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL0 INT)"); + sql("CREATE INDEX TEST_IDX ON TEST(VAL0)"); + + Matcher<String> planMatcher = containsIndexScan("PUBLIC", "TEST", "TEST_IDX"); + + checkMixedTransactions(planMatcher); + } + + private void checkMixedTransactions(Matcher<String> planMatcher) { + IgniteSql sql = igniteSql(); + + if (sql instanceof ClientSql) { Review Comment: I think it is better to move such guards into a test case body directly, so we know that a test case is not expected to run for sql clients. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
