Skip to content

IGNITE-20661 JDBC. Support multi statement queries by JDBC#2906

Merged
zstan merged 38 commits intoapache:mainfrom
gridgain:ignite-20661
Dec 29, 2023
Merged

IGNITE-20661 JDBC. Support multi statement queries by JDBC#2906
zstan merged 38 commits intoapache:mainfrom
gridgain:ignite-20661

Conversation

@zstan
Copy link
Contributor

@zstan zstan commented Nov 30, 2023

Thank you for submitting the pull request.

To streamline the review process of the patch and ensure better code quality
we ask both an author and a reviewer to verify the following:

The Review Checklist

  • Formal criteria: TC status, codestyle, mandatory documentation. Also make sure to complete the following:
    - There is a single JIRA ticket related to the pull request.
    - The web-link to the pull request is attached to the JIRA ticket.
    - The JIRA ticket has the Patch Available state.
    - The description of the JIRA ticket explains WHAT was made, WHY and HOW.
    - The pull request title is treated as the final commit message. The following pattern must be used: IGNITE-XXXX Change summary where XXXX - number of JIRA issue.
  • Design: new code conforms with the design principles of the components it is added to.
  • Patch quality: patch cannot be split into smaller pieces, its size must be reasonable.
  • Code quality: code is clean and readable, necessary developer documentation is added if needed.
  • Tests code quality: test set covers positive/negative scenarios, happy/edge cases. Tests are effective in terms of execution time and resources.

Notes

@zstan zstan force-pushed the ignite-20661 branch 3 times, most recently from 866604b to 5923904 Compare December 1, 2023 05:41
@zstan
Copy link
Contributor Author

zstan commented Dec 1, 2023

@korlov42 can u make a review plz ?

JdbcResultSet rs = resSets.get(curRes);

if (!rs.isQuery()) {
if (rs == null || !rs.isQuery()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if null is considering as legit element of resSets collection, then we need to mark generic type as @Nullable

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this one was not addressed, as far as I can see

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

a few words about comments above:

at the line 407 you've added null checking for element of resSets collection. This implies that now null is considered legit item of the collection. If so, it's better to state in resSets declaration -- List<JdbcResultSet> resSets; --> List<@Nullable JdbcResultSet> resSets; . Now IDEA shows 4 more warning because of this annotation. Two of these warning actually may cause NPE

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

getResultSet() marked as nullable, @nullable JdbcResultSet rs = too, did i miss smth ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i found that resSets has volatile notation seems our Statement implementation is not a thread safe at all.
apeend @nullable, fix nullability checks

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should be fixed

Comment on lines +179 to +182
// a bit hack, instead of calling stmt.close(), gives a chance to catch potential forgiven cursor.
stmt.execute("SELECT 1");
ResultSet rs = stmt.getResultSet();
rs.close();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

to be honest, I don't quite get the purpose of these lines

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the very first execution of stmt.execute alredy registers a first cursor in resourses, thus for correct checking resource leakage without this approach i would need to make one of actions, or call :

  1. stmt.close() - but this concrete implementation can work properly well while sequential .getResultset() can holds a bug.
  2. always get last resultset in tests and forcibly close it.

@zstan zstan requested a review from korlov42 December 7, 2023 11:09
Comment on lines +68 to +161
assertEquals(false, res);
stmt.getResultSet();
stmt.getMoreResults();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
assertEquals(false, res);
stmt.getResultSet();
stmt.getMoreResults();
assertFalse(res);
assertNull(stmt.getResultSet());
assertFalse(stmt.getMoreResults());

🤔

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I understand you, but this case is not for check intermediate statement commands but about correct results after multiple calls, but ok, i fix it, it makes no harm.


/**
* Tests for ddl queries that contain multiply sql statements, separated by ";".
* Tests for queries that contain multiply sql statements, separated by ";".
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

please add some test(s) for autocommit=false (without TX_CONTROL statements) seems this should work fine,

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i think we need to check it after IGNITE-20463

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

seems need to be addressed

@zstan zstan requested a review from xtern December 11, 2023 12:23
* @param req Results request.
* @return Result future.
*/
CompletableFuture<JdbcQuerySingleResult> getMoreResultsAsync(JdbcQueryFetchRequest req);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

usage of JdbcQueryFetchRequest as an input argument to getMoreResults handler kinda misleading. Let's introduce separate class

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

probably we need to rename somehow ? but hold additional equal class just different action seems not so good too, wdyt ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't mind to rename it, but have no good options for name.

btw, what is wrong with new class?


close0(true);

if (!res.hasResults()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what if we got an error from handler? Let's add a test to cover this case

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

append test

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

probably I'm missing something, but I still no sight of error handling. Could you point out to me on particular test that covers failed response?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ItJdbcMultiStatementSelfTest#testSimpleQueryError got an error from handler, or you are talking about something more ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

as for now - we stop further script execution if exception is raised, i.e.

CREATE TABLE test (id INT PRIMARY KEY);
CREATE TABLE test (id INT PRIMARY KEY);
SELECT 1;

we can`t obtain SELECT 1 results, thus it seems enough, or you think that some mock are still necessary here ?

Copy link
Contributor

@xtern xtern Dec 18, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems we need additional check here, something like

                if (res.status() == STATUS_FAILED) {
                    throw createJdbcSqlException(res.err(), res.status());
                }

If the JdbcQueryCursorHandler returns a future with an error, we are not handling it properly here.

(see related comment in JdbcQueryCursorHandlerImpl class)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done, append additional handler test

JdbcResultSet rs = resSets.get(curRes);

if (!rs.isQuery()) {
if (rs == null || !rs.isQuery()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this one was not addressed, as far as I can see

assertFalse(res);
assertNull(stmt.getResultSet());
assertFalse(stmt.getMoreResults());
assertEquals(-1, getResultSetSize());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if you want to check updateCount, please do it explicitly

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

change getResultSetSize javadoc

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fully support aforementioned comments. Why don't check update Count explicitly?

* @param req Results request.
* @return Result future.
*/
CompletableFuture<JdbcQuerySingleResult> getMoreResultsAsync(JdbcQueryFetchRequest req);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't mind to rename it, but have no good options for name.

btw, what is wrong with new class?

*/
public JdbcQuerySingleResult(long cursorId, List<BinaryTupleReader> rowTuples, List<ColumnType> columnTypes, int[] decimalScales,
boolean last) {
boolean last, boolean isQuery, long updateCnt) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we already have contractor for DML statements, so let's use that one

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this particular constructor was used to create result for statement returning result set. With that said, it's better keep it as is (without additional isQuery and updateCnt) to avoid confusion and possible misusage

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done


close0(true);

if (!res.hasResults()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

probably I'm missing something, but I still no sight of error handling. Could you point out to me on particular test that covers failed response?


close0(true);

if (!res.hasResults()) {
Copy link
Contributor

@xtern xtern Dec 18, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems we need additional check here, something like

                if (res.status() == STATUS_FAILED) {
                    throw createJdbcSqlException(res.err(), res.status());
                }

If the JdbcQueryCursorHandler returns a future with an error, we are not handling it properly here.

(see related comment in JdbcQueryCursorHandlerImpl class)

case CLOSE_CURRENT_RESULT:
if (curRes > 0) {
resSets.get(curRes - 1).close0();
resSets.get(curRes - 1).close0(false);
Copy link
Contributor

@xtern xtern Dec 18, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why we are calling close0 with false flag here?

The following example demonstrates resource leakage (or may be I missed something):

    @Test
    public void testX() throws Exception {
        long initial = openCursorsRegistered();
        
        stmt.executeUpdate("CREATE TABLE TESTX(ID INT PRIMARY KEY)");
        stmt.getMoreResults();
        stmt.close();

        assertEquals(0L, initial - openCursorsRegistered());
    }

As I see, the current combination in close0 of isQuery, finished, etc. flags were being used because we were not registering cursors for DML, DDL (and possibly other) queries.

Since we are now registering cursors for this types of queries, I think it is worth reviewing the current conditions in close0, because now they look too complex and leading to errors 😎 .

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is connection context , append below to test and it will pass:

        conn.close();
        assertTrue(waitForCondition(() -> 0L == openCursorsRegistered(), 5000));

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes you are right. I missed this.
But as I see, now - getNextResultSet() already closes resultset with enforce=true, why we need this second call here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

*/
public JdbcQuerySingleResult(long cursorId, List<BinaryTupleReader> rowTuples, List<ColumnType> columnTypes, int[] decimalScales,
boolean last) {
boolean last, boolean isQuery, long updateCnt) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this particular constructor was used to create result for statement returning result set. With that said, it's better keep it as is (without additional isQuery and updateCnt) to avoid confusion and possible misusage

*/
public JdbcQuerySingleResult(boolean hasNext, long updCount) {
hasResults = hasNext;
this.updateCnt = updCount;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if there are no results, why should we provide updateCount?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@Override
public void writeBinary(ClientMessagePacker packer) {
super.writeBinary(packer);
packer.packLong(updateCnt);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if there are no results, we should not pack updateCount

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done


List<ColumnMetadata> columns = cur.metadata().columns();

return buildSingleRequest(batch, columns, cursorId, !batch.hasMore(), queryType);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see that first page response for first statement and next statement are created differently: for first statement only subset of query types goes through buildSingleRequest, while for next statement all types goes through this method. Let's align the creation of result for the first page

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

Comment on lines +125 to +138
CompletableFuture<JdbcQuerySingleResult> fut = IgniteTestUtils.runAsync(() ->
await(cursorHandler.getMoreResultsAsync(new JdbcFetchQueryResultsRequest(1, 100)), 5, TimeUnit.SECONDS)
);

try {
await(fut, 5, TimeUnit.SECONDS);
} catch (Throwable e) {
fail("Unexpected exception is raised.");
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

to be honest, I don't quite understand what you are trying to verify here...

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I verify that JdbcQueryCursorHandlerImpl correctly handles exceptions raised from AsyncSqlCursor, do you think it`s redundant ?

Comment on lines 397 to 398
@Nullable
public ResultSet getResultSet() throws SQLException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
@Nullable
public ResultSet getResultSet() throws SQLException {
public @Nullable ResultSet getResultSet() throws SQLException {

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

JdbcResultSet rs = resSets.get(curRes);

if (!rs.isQuery()) {
if (rs == null || !rs.isQuery()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

a few words about comments above:

at the line 407 you've added null checking for element of resSets collection. This implies that now null is considered legit item of the collection. If so, it's better to state in resSets declaration -- List<JdbcResultSet> resSets; --> List<@Nullable JdbcResultSet> resSets; . Now IDEA shows 4 more warning because of this annotation. Two of these warning actually may cause NPE

throw exceptionally;
}

return nextResultSet != null && nextResultSet.holdResults();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this condition is not quite correct: empty result set still result set, thus true should be returned (try this query SELECT 1; SELECT 1 FROM table(system_range(1, 0)))

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

changed, not here but logic at all


default:
throw new SQLException("Invalid 'current' parameter.");
throw new SQLException("Invalid 'curr' parameter.");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

better to use current because it is how it named in interface

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@Override
public void close() throws SQLException {
close0();
closed = true;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

all resources must be cleaned up as soon as possible, it's not enough to just pretend that result set is closed

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@zstan zstan force-pushed the ignite-20661 branch 2 times, most recently from 98af8e0 to 51ccd71 Compare December 21, 2023 15:13
@zstan
Copy link
Contributor Author

zstan commented Dec 21, 2023

@xtern @korlov42 guys, seems i finish. There is 1 question about test from me, check above. Nullability checks .. idea due to volatile notation highlight all places there this value used no matter exist previous not null check or not.

@zstan zstan force-pushed the ignite-20661 branch 2 times, most recently from dc98aa0 to ded00df Compare December 26, 2023 08:13
Comment on lines +199 to +203
if (cursorId != null) {
packer.packLong(cursorId);
} else {
packer.packNil();
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
if (cursorId != null) {
packer.packLong(cursorId);
} else {
packer.packNil();
}
packer.packLongNullable(cursorId);

}
});

try {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what reason to have the try/catch block?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

look, this test check that no exception is raised from JdbcQueryCursorHandlerImpl#getMoreResultsAsync
comment exception handling in mention method and try to run test.

@zstan zstan merged commit 704b71d into apache:main Dec 29, 2023
@zstan zstan deleted the ignite-20661 branch December 29, 2023 06:48
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants