You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/12/29 15:10:50 UTC

[GitHub] [ignite-3] korlov42 commented on a change in pull request #522: IGNITE-15648 Added JDBC integration tests

korlov42 commented on a change in pull request #522:
URL: https://github.com/apache/ignite-3/pull/522#discussion_r776322557



##########
File path: modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcConnection.java
##########
@@ -148,14 +148,19 @@ public JdbcConnection(ConnectionProperties props) {
         long reconnectThrottlingPeriod = connProps.getReconnectThrottlingPeriod();
         int reconnectThrottlingRetries = connProps.getReconnectThrottlingRetries();
 
-        client = ((TcpIgniteClient) IgniteClient
-                .builder()
-                .addresses(addrs)
-                .connectTimeout(netTimeout)
-                .retryLimit(retryLimit)
-                .reconnectThrottlingPeriod(reconnectThrottlingPeriod)
-                .reconnectThrottlingRetries(reconnectThrottlingRetries)
-                .build());
+        try {
+            client = ((TcpIgniteClient) IgniteClient
+                    .builder()
+                    .addresses(addrs)
+                    .connectTimeout(netTimeout)
+                    .retryLimit(retryLimit)
+                    .reconnectThrottlingPeriod(reconnectThrottlingPeriod)
+                    .reconnectThrottlingRetries(reconnectThrottlingRetries)
+                    .build());
+
+        } catch (Exception e) {
+            throw new SQLException("Failed to connect to server", "08001", e);

Review comment:
       what does "08001" mean?

##########
File path: modules/client-common/src/main/java/org/apache/ignite/client/proto/query/event/BatchExecuteResult.java
##########
@@ -26,6 +26,9 @@
  * JDBC batch execute result.
  */
 public class BatchExecuteResult extends Response {
+    /** Empty array instance. */
+    private static final int[] EMPTY_ARRAY = new int[0];

Review comment:
       org.apache.ignite.internal.util.ArrayUtils#INT_EMPTY_ARRAY

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcAbstractStatementSelfTest.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.runner.app.jdbc;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+
+/**
+ * Abstract jdbc test.
+ */
+public abstract class ItJdbcAbstractStatementSelfTest extends AbstractJdbcSelfTest {
+    /** SQL query to populate cache. */

Review comment:
       > to populate cache
   
   did you mean table?

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcAbstractStatementSelfTest.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.runner.app.jdbc;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+
+/**
+ * Abstract jdbc test.
+ */
+public abstract class ItJdbcAbstractStatementSelfTest extends AbstractJdbcSelfTest {
+    /** SQL query to populate cache. */
+    private static final String ITEMS_SQL = "insert into PUBLIC.PERSON(SID, ID, FIRSTNAME, LASTNAME, AGE) values "
+            + "('p1', 1, 'John', 'White', 25), "
+            + "('p2', 2, 'Joe', 'Black', 35), "
+            + "('p3', 3, 'Mike', 'Green', 40)";
+
+    /** SQL query to populate cache. */

Review comment:
       populate with void?

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcAbstractStatementSelfTest.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.runner.app.jdbc;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+
+/**
+ * Abstract jdbc test.
+ */
+public abstract class ItJdbcAbstractStatementSelfTest extends AbstractJdbcSelfTest {
+    /** SQL query to populate cache. */
+    private static final String ITEMS_SQL = "insert into PUBLIC.PERSON(SID, ID, FIRSTNAME, LASTNAME, AGE) values "
+            + "('p1', 1, 'John', 'White', 25), "
+            + "('p2', 2, 'Joe', 'Black', 35), "
+            + "('p3', 3, 'Mike', 'Green', 40)";
+
+    /** SQL query to populate cache. */
+    private static final String DROP_SQL = "DELETE FROM PUBLIC.PERSON;";
+
+    @BeforeEach
+    public void refillTable() throws Exception {
+        Connection connection = DriverManager.getConnection(URL);
+
+        try (Statement s = connection.createStatement()) {
+            s.executeUpdate(DROP_SQL);
+        }
+        try (Statement s = connection.createStatement()) {
+            s.executeUpdate(ITEMS_SQL);
+        }
+    }
+
+    @BeforeAll
+    public static void createTable() throws Exception {
+        Ignite ignite = clusterNodes.get(0);
+
+        TableDefinition personTableDef = SchemaBuilders.tableBuilder("PUBLIC", "PERSON")
+                .columns(
+                        SchemaBuilders.column("ID", ColumnType.INT32).build(),
+                        SchemaBuilders.column("SID", ColumnType.string()).build(),
+                        SchemaBuilders.column("FIRSTNAME", ColumnType.string()).asNullable(false).build(),
+                        SchemaBuilders.column("LASTNAME", ColumnType.string()).asNullable(false).build(),
+                        SchemaBuilders.column("AGE", ColumnType.INT32).asNullable(false).build())
+                .withPrimaryKey("ID")
+                .build();
+
+        ignite.tables().createTable(personTableDef.canonicalName(), (tableChange) -> {

Review comment:
       Have you considered using DDL query? 

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcAbstractStatementSelfTest.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.runner.app.jdbc;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.schema.definition.TableDefinition;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+
+/**
+ * Abstract jdbc test.
+ */
+public abstract class ItJdbcAbstractStatementSelfTest extends AbstractJdbcSelfTest {
+    /** SQL query to populate cache. */
+    private static final String ITEMS_SQL = "insert into PUBLIC.PERSON(SID, ID, FIRSTNAME, LASTNAME, AGE) values "
+            + "('p1', 1, 'John', 'White', 25), "
+            + "('p2', 2, 'Joe', 'Black', 35), "
+            + "('p3', 3, 'Mike', 'Green', 40)";
+
+    /** SQL query to populate cache. */
+    private static final String DROP_SQL = "DELETE FROM PUBLIC.PERSON;";
+
+    @BeforeEach
+    public void refillTable() throws Exception {
+        Connection connection = DriverManager.getConnection(URL);
+
+        try (Statement s = connection.createStatement()) {
+            s.executeUpdate(DROP_SQL);
+        }
+        try (Statement s = connection.createStatement()) {

Review comment:
       why don't just reuse statement? 

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcMetadataSelfTest.java
##########
@@ -144,21 +146,41 @@ public void testDecimalAndDateTypeMetaData() throws Exception {
             assertEquals(2, meta.getColumnCount());
 
             assertEquals("METATEST", meta.getTableName(1).toUpperCase());
-            assertEquals("DECIMAL", meta.getColumnName(1).toUpperCase());
-            assertEquals("DECIMAL", meta.getColumnLabel(1).toUpperCase());
+            assertEquals("DECIMAL_COL", meta.getColumnName(1).toUpperCase());
+            assertEquals("DECIMAL_COL", meta.getColumnLabel(1).toUpperCase());
             assertEquals(DECIMAL, meta.getColumnType(1));
             assertEquals(meta.getColumnTypeName(1), "DECIMAL");
             assertEquals(meta.getColumnClassName(1), "java.math.BigDecimal");
 
             assertEquals("METATEST", meta.getTableName(2).toUpperCase());
-            assertEquals("DATE", meta.getColumnName(2).toUpperCase());
-            assertEquals("DATE", meta.getColumnLabel(2).toUpperCase());
+            assertEquals("DATE_COL", meta.getColumnName(2).toUpperCase());
+            assertEquals("DATE_COL", meta.getColumnLabel(2).toUpperCase());
             assertEquals(DATE, meta.getColumnType(2));
             assertEquals(meta.getColumnTypeName(2), "DATE");
             assertEquals(meta.getColumnClassName(2), "java.sql.Date");
+        } finally {
+            Connection conn = DriverManager.getConnection(URL);
+
+            conn.createStatement().execute("DROP TABLE METATEST;");
         }
     }
 
+    private void createMetaTable() {

Review comment:
       Why not DDL?

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcDeleteStatementSelfTest.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.runner.app.jdbc;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Delete functional statement self test.
+ */
+public class ItJdbcDeleteStatementSelfTest extends ItJdbcAbstractStatementSelfTest {
+    /**
+     * Execute delete query test.
+     *
+     * @throws SQLException If failed.
+     */
+    @Test
+    public void testExecute() throws SQLException {
+        Connection conn = DriverManager.getConnection(URL);
+
+        conn.createStatement().execute("delete from PUBLIC.PERSON where substring(SID, 2, 1)::int % 2 = 0");
+
+        Table person = clusterNodes.get(0).tables().table("PUBLIC.PERSON");

Review comment:
       Why don't just take kvView here? 

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcStatementCancelSelfTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.runner.app.jdbc;
+
+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 static org.junit.jupiter.api.Assertions.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Statement cancel test.
+ */
+@Disabled("IGNITE-16205")
+public class ItJdbcStatementCancelSelfTest extends ItJdbcAbstractStatementSelfTest {
+
+    /** URL. */
+    private static final String URL = "jdbc:ignite:thin://127.0.0.1/";
+
+    /** Connection. */
+    private Connection conn;
+
+    /** Statement. */
+    private Statement stmt;
+
+    /**
+     * Called before execution of every test method in class.
+     *
+     * @throws Exception If failed.
+     */
+    @BeforeEach
+    public void before() throws Exception {
+        conn = DriverManager.getConnection(URL);
+
+        stmt = conn.createStatement();
+
+        assert stmt != null;
+        assert !stmt.isClosed();
+    }
+
+    /**
+     * Called after execution of every test method in class.
+     *
+     * @throws Exception If failed.
+     */
+    @AfterEach
+    public void after() throws Exception {
+        if (stmt != null && !stmt.isClosed()) {
+            stmt.close();
+
+            assert stmt.isClosed();
+        }
+
+        conn.close();
+
+        assert stmt.isClosed();
+        assert conn.isClosed();
+    }
+
+    /**
+     * Trying to cancel stament without query. In given case cancel is noop, so no exception expected.
+     */
+    @Test
+    public void testCancelingStmtWithoutQuery() {
+        try {
+            stmt.cancel();
+        } catch (Exception e) {
+            log.error("Unexpected exception.", e);
+
+            fail("Unexpected exception");
+        }
+    }
+
+    /**
+     * Trying to retrieve result set of a canceled query.
+     * SQLException with message "The query was cancelled while executing." expected.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testResultSetRetrievalInCanceledStatement() throws Exception {
+        stmt.execute("SELECT 1; SELECT 2; SELECT 3;");
+
+        assertNotNull(stmt.getResultSet());
+
+        stmt.cancel();
+
+        assertThrows(SQLException.class, () -> stmt.getResultSet(), "The query was cancelled while executing.");
+    }
+
+    /**
+     * Trying to cancel already cancelled query.
+     * No exceptions exceped.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCancelCanceledQuery() throws Exception {
+        stmt.execute("SELECT 1;");
+
+        assertNotNull(stmt.getResultSet());
+
+        stmt.cancel();
+
+        stmt.cancel();
+
+        assertThrows(SQLException.class, () -> stmt.getResultSet(), "The query was cancelled while executing.");
+    }
+
+    /**
+     * Trying to cancel closed query.
+     * SQLException with message "Statement is closed." expected.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCancelClosedStmt() throws Exception {
+        stmt.close();
+
+        assertThrows(SQLException.class, () -> stmt.cancel(), "Statement is closed.");
+    }
+
+    /**
+     * Trying to call <code>resultSet.next()</code> on a canceled query.
+     * SQLException with message "The query was cancelled while executing." expected.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testResultSetNextAfterCanceling() throws Exception {
+        stmt.setFetchSize(10);
+
+        ResultSet rs = stmt.executeQuery("select * from PUBLIC.PERSON");
+
+        assertTrue(rs.next());
+
+        stmt.cancel();
+
+        assertThrows(SQLException.class, rs::next, "The query was cancelled while executing.");
+    }
+
+    /**
+     * Ensure that it's possible to execute new query on cancelled statement.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCancelAnotherStmt() throws Exception {
+        stmt.setFetchSize(10);
+
+        ResultSet rs = stmt.executeQuery("select * from PUBLIC.PERSON");
+
+        assertTrue(rs.next());
+
+        stmt.cancel();
+
+        ResultSet rs2 = stmt.executeQuery("select * from PUBLIC.PERSON order by ID asc");
+
+        assertTrue(rs2.next(), "The other cursor mustn't be closed");
+    }
+
+    /**
+     * Ensure that stament cancel doesn't effect another statement workflow, created by the same connection.

Review comment:
       *affect

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcStatementCancelSelfTest.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.runner.app.jdbc;
+
+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 static org.junit.jupiter.api.Assertions.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Statement cancel test.
+ */
+@Disabled("IGNITE-16205")
+public class ItJdbcStatementCancelSelfTest extends ItJdbcAbstractStatementSelfTest {
+
+    /** URL. */
+    private static final String URL = "jdbc:ignite:thin://127.0.0.1/";

Review comment:
       please use URL from an AbstractTest

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcSelectAfterAlterTable.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.runner.app.jdbc;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base class for complex SQL tests based on JDBC driver.
+ */
+public class ItJdbcSelectAfterAlterTable extends AbstractJdbcSelfTest {
+    /** JDBC connection. */
+    private Connection conn;
+
+    /** JDBC statement. */
+    private Statement stmt;
+
+    /** {@inheritDoc} */
+    @BeforeEach
+    protected void beforeTest() throws Exception {
+        conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1");
+
+        stmt = conn.createStatement();
+
+        stmt.executeUpdate("CREATE TABLE PUBLIC.PERSON (ID BIGINT, NAME VARCHAR, CITY_ID BIGINT, PRIMARY KEY (ID, CITY_ID))");
+        stmt.executeUpdate("INSERT INTO PUBLIC.PERSON (ID, NAME, CITY_ID) values (1, 'name_1', 11)");
+
+        stmt.executeQuery("select * from PUBLIC.PERSON");

Review comment:
       why do you do this?

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcMetadataPrimaryKeysSelfTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.runner.app.jdbc;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Verifies that primary keys in the metadata are valid.
+ */
+public class ItJdbcMetadataPrimaryKeysSelfTest extends AbstractJdbcSelfTest {
+    /** Url. */
+    private static final String URL = "jdbc:ignite:thin://127.0.0.1";
+
+    /** COLUMN_NAME column index in the metadata table. */
+    private static final int COL_NAME_IDX = 4;
+
+    /** {@inheritDoc} */
+    @AfterEach
+    protected void afterTest() throws Exception {
+        executeUpdate("DROP TABLE IF EXISTS PUBLIC.TEST;");
+    }
+
+    /**
+     * Checks for PK that contains single unwrapped field.
+     */
+    @Test
+    public void testSingleUnwrappedKey() throws Exception {
+        executeUpdate("CREATE TABLE PUBLIC.TEST (ID INT PRIMARY KEY, NAME VARCHAR);");
+
+        checkPkFields("TEST", "ID");
+    }
+
+    /**
+     * Checks for PK that contains single field. Key is forcibly wrapped.
+     */
+    @Test
+    public void testSingleWrappedKey() throws Exception {

Review comment:
       this test doesn't make sense for ignite-3, because there is no such feature as Wrap PK. So one test should be renamed and another deleted

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/AbstractJdbcSelfTest.java
##########
@@ -38,11 +39,14 @@
  */
 public class AbstractJdbcSelfTest {
     /** URL. */
-    protected static final String URL = "jdbc:ignite:thin://127.0.1.1:10800";
+    protected static final String URL = "jdbc:ignite:thin://127.0.0.1:10800";
 
     /** Cluster nodes. */
     protected static final List<Ignite> clusterNodes = new ArrayList<>();
 
+    /** Logger. */
+    protected static IgniteLogger log;

Review comment:
       see no reason to have it static

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcMetadataPrimaryKeysSelfTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.runner.app.jdbc;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Verifies that primary keys in the metadata are valid.
+ */
+public class ItJdbcMetadataPrimaryKeysSelfTest extends AbstractJdbcSelfTest {
+    /** Url. */
+    private static final String URL = "jdbc:ignite:thin://127.0.0.1";

Review comment:
       you already have URL in AbstractJdbcSelfTest

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcMetadataSelfTest.java
##########
@@ -144,21 +146,41 @@ public void testDecimalAndDateTypeMetaData() throws Exception {
             assertEquals(2, meta.getColumnCount());
 
             assertEquals("METATEST", meta.getTableName(1).toUpperCase());
-            assertEquals("DECIMAL", meta.getColumnName(1).toUpperCase());
-            assertEquals("DECIMAL", meta.getColumnLabel(1).toUpperCase());
+            assertEquals("DECIMAL_COL", meta.getColumnName(1).toUpperCase());
+            assertEquals("DECIMAL_COL", meta.getColumnLabel(1).toUpperCase());
             assertEquals(DECIMAL, meta.getColumnType(1));
             assertEquals(meta.getColumnTypeName(1), "DECIMAL");
             assertEquals(meta.getColumnClassName(1), "java.math.BigDecimal");
 
             assertEquals("METATEST", meta.getTableName(2).toUpperCase());
-            assertEquals("DATE", meta.getColumnName(2).toUpperCase());
-            assertEquals("DATE", meta.getColumnLabel(2).toUpperCase());
+            assertEquals("DATE_COL", meta.getColumnName(2).toUpperCase());
+            assertEquals("DATE_COL", meta.getColumnLabel(2).toUpperCase());
             assertEquals(DATE, meta.getColumnType(2));
             assertEquals(meta.getColumnTypeName(2), "DATE");
             assertEquals(meta.getColumnClassName(2), "java.sql.Date");
+        } finally {
+            Connection conn = DriverManager.getConnection(URL);

Review comment:
       I suppose every connection should be closed. The same is valid and for other places

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcStatementSelfTest.java
##########
@@ -172,7 +186,7 @@ public void testExecute() throws Exception {
     }
 
     @Test
-    @Disabled("IGNITE-15108")
+    @Disabled

Review comment:
       could you please provide a link to a related ticket? The same for other "naked" `@Disabled` annotations

##########
File path: modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcErrorsSelfTest.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.runner.app.jdbc;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.sql.BatchUpdateException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test SQLSTATE codes propagation with thin client driver.
+ */
+public class ItJdbcErrorsSelfTest extends ItJdbcErrorsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Connection getConnection() throws SQLException {
+        return DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1");

Review comment:
       please use the URL from an AbstractTest




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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