Skip to content

HIVE-28265: Fix JDBC timeout message for hive.query.timeout.seconds#6412

Open
ashniku wants to merge 20 commits intoapache:masterfrom
ashniku:HIVE-28265
Open

HIVE-28265: Fix JDBC timeout message for hive.query.timeout.seconds#6412
ashniku wants to merge 20 commits intoapache:masterfrom
ashniku:HIVE-28265

Conversation

@ashniku
Copy link
Copy Markdown
Contributor

@ashniku ashniku commented Apr 6, 2026

What changes were proposed in this pull request?

hive.query.timeout.seconds is enforced correctly, but Beeline/JDBC reported Query timed out after 0 seconds when Statement.setQueryTimeout was not used.

This PR:

  1. SQLOperation (HiveServer2)
    Before cancel(OperationState.TIMEDOUT), set a HiveSQLException whose message is Query timed out after seconds, using the effective operation timeout ( is the same value used to schedule the cancel). GetOperationStatus then exposes the right text via operationException.
    For async execution, do not call setOperationException from the background thread if the operation is already TIMEDOUT, so the timeout message is not overwritten.

  2. HiveStatement (JDBC)
    On TIMEDOUT_STATE, prefer the server errorMessage. If it is missing or clearly wrong (contains after 0 seconds), build the client message from Statement query timeout or from the last SET hive.query.timeout.seconds=... value tracked on HiveConnection. SET is detected with a regex find() so assignments can appear inside a longer script (last match wins).

  3. HiveConnection
    Stores the last parsed hive.query.timeout.seconds from a successful SET for use in the timeout message when needed.

  4. Tests

==> testQueryTimeoutMessageUsesHiveConf: session SET hive.query.timeout.seconds=1s, no setQueryTimeout, slow query via existing SleepMsUDF — expects SQLTimeoutException and message not claiming after 0 seconds, and containing 1.
==> testQueryTimeout: same checks for the existing setQueryTimeout(1) path.

-->

Why are the changes needed?

Does this PR introduce any user-facing change?

How was this patch tested?

fail("Expecting SQLTimeoutException");
} catch (SQLTimeoutException e) {
assertNotNull(e);
assertTrue("Message should reflect JDBC query timeout (1s): " + e.getMessage(),
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Can you show us an example about the whole output that demonstrates the change?

I wonder if it is possible to get any number other than the timeout in the message. Like a timestamp or maybe a query id, host name, etc. Asserting to a single number in a string looks a little bit fragile to me.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Introduced constant QUERY_TIMED_OUT_AFTER_1_SECONDS = "Query timed out after 1 seconds" with Javadoc that this is the full message from HS2 / client (no query id, host, timestamp in that string for these paths).
testQueryTimeout now uses assertEquals, expected value = that constant, with a failure message that repeats the example text.

* {@code N == 1} with flexible whitespace so we do not treat {@code 10} or unrelated digits as {@code 1}.
*/
private static boolean isQueryTimedOutAfterOneSecondMessage(String msg) {
return msg != null && msg.matches("(?is).*timed out after\\s+1\\s+seconds.*");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

We know it is 1 sec. And we don't accept any other output in that case.
In my opinion, regex here can be a little bit overkill.

What about something like:

final String expectedMessage = "Query timed out after 1 seconds";
assertEquals("Message should reflect JDBC query timeout", expectedMesage, message);

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Removed isQueryTimedOutAfterOneSecondMessage (regex helper).
Positive assertions use assertEquals("…", QUERY_TIMED_OUT_AFTER_1_SECONDS, e.getMessage()) in both timeout-related tests.

assertNotNull(e);
assertTrue("Message should reflect JDBC query timeout (1s): " + e.getMessage(),
isQueryTimedOutAfterOneSecondMessage(e.getMessage()));
assertFalse("Message should not claim 0 seconds: " + e.getMessage(),
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Considering the previous assertion, is that assertion possible at all?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

(assertFalse(… "after 0 seconds") after the positive check in testQueryTimeout — “is that even possible?”)

Changes

Removed assertFalse(..., e.getMessage().contains("after 0 seconds")) from testQueryTimeout.

+ " t2 on t1.under_col = t2.under_col");
fail("Expecting SQLTimeoutException");
} catch (SQLTimeoutException e) {
assertNotNull(e);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Is it possible having an exception with null value in a catch block?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

(assertNotNull(e) in catch (SQLTimeoutException e) — can e be null?)

Changes

Removed assertNotNull(e) from both SQLTimeoutException catch blocks in these tests.

assertNotNull(e);
assertTrue("Message should include session timeout (1s): " + e.getMessage(),
isQueryTimedOutAfterOneSecondMessage(e.getMessage()));
assertFalse("Message should not claim 0 seconds (HIVE-28265): " + e.getMessage(),
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

What is the benefits of putting the ticket number into assertions or comments?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Dropped ticket id from assertion messages.
Kept HIVE-28265 and expected message behavior in testQueryTimeoutMessageUsesHiveConf Javadoc (and the constant / assertEquals text describes behavior without the ticket).

assertNotNull(e);
assertTrue("Message should include session timeout (1s): " + e.getMessage(),
isQueryTimedOutAfterOneSecondMessage(e.getMessage()));
assertFalse("Message should not claim 0 seconds (HIVE-28265): " + e.getMessage(),
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Considering the previous assertion, is that assertion possible at all?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Removed assertFalse(..., "after 0 seconds") from testQueryTimeoutMessageUsesHiveConf.

* Sentinel: no {@code SET hive.query.timeout.seconds} has been observed on this connection yet.
*/
static final long SESSION_QUERY_TIMEOUT_NOT_TRACKED = -1L;
private final AtomicLong sessionQueryTimeoutSeconds = new AtomicLong(SESSION_QUERY_TIMEOUT_NOT_TRACKED);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Thinking out loud: I wonder if a connection can have concurrency issue: I mean, you can have multiple individual connections to Hive, but inside a connection itself, can we have multiple hive statements in parallel?
I have no such use case in my mind, but let me ping Ayush about this question.

@ayushtkn , what do you think?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

a single JDBC Connection can be shared across multiple threads, and it is entirely possible to have multiple HiveStatement objects executing concurrently on the same connection (which maps to a single session on the HS2 side).

via Beeline or so maybe not but In Hive Server 2 (HS2), a single JDBC Connection corresponds to a single HS2 Session. You can absolutely execute multiple queries concurrently within the same session by spawning multiple threads on the client side, each using a different HiveStatement created from that single HiveConnection.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Thx

* Records the effective {@code hive.query.timeout.seconds} (in seconds) after a successful
* {@code SET hive.query.timeout.seconds=...} on this connection. Used for JDBC timeout messages.
*/
void recordSessionQueryTimeoutFromSet(long seconds) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Can we keep the getter..setter naming pattern?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

recordSessionQueryTimeoutFromSet(long) → setSessionQueryTimeoutSeconds(long)
getSessionQueryTimeoutSecondsTracked() → getSessionQueryTimeoutSeconds()
HiveStatement updated to call the new names.

ashniku added 17 commits April 26, 2026 13:58
- TestJdbcDriver2: assert exact 'Query timed out after 1 seconds'; drop redundant
  assertNotNull/assertFalse; keep HIVE-28265 context in Javadoc not assert text
- HiveConnection: AtomicLong Javadoc for concurrent statements; rename to
  setSessionQueryTimeoutSeconds / getSessionQueryTimeoutSeconds
- HiveStatement: update call sites

Made-with: Cursor
HiveSQLException appends '; Query ID: ...' to getMessage(). When the
client passes through the server timeout text (non-broken path),
SQLTimeoutException included that suffix and TestJdbcDriver2 exact
assertions failed on CI. Strip the same trailer the server uses before
returning the message.

Made-with: Cursor
… ID suffix

- HiveStatement: stop stripping '; Query ID;' from server timeout text; pass
  through when server message is usable (reverts strip-only follow-up).
- TestJdbcDriver2: assert message starts with expected timeout text and does not
  contain 'after 0 seconds' (HIVE-28265), allowing HS2/HiveSQLException suffix.

Made-with: Cursor
- sqlExceptionForCanceledState: replace ternary with if/else (S3358).
- Extract processOperationStatusResponse from waitForOperationToComplete (S6541).
- Use local progressUpdates flag to simplify nested flow (AvoidNestedBlocks).

Does not address hundreds of repo-wide Sonar findings outside this file.

Made-with: Cursor
Remove skippedAttempts reset on lock acquisition that could wipe concurrent
increments; use AtomicInteger for skipped attempts; reset test counters in
@before to isolate tests.

Made-with: Cursor
…ests

- Rename AtomicInteger counter to SKIPPED_ATTEMPTS (ConstantNameCheck).
- Drop unused partitionDiscoveryEnabled again (revert commit restores history).
- testQueryProgress: accept ELAPSED TIME, Beeline row timing, or Driver Time taken.
- llap_io_cache: use 8MiB RPAD payload to avoid Parquet logging OOM on CI.

Made-with: Cursor
Restore Beeline/llap_io_cache Q test, PartitionManagementTask, and
TestPartitionManagement to upstream master. Core fix remains:
SQLOperation, HiveStatement, HiveConnection, TestJdbcDriver2.

Made-with: Cursor
testURLWithHiveQueryTimeoutSeconds sets hive.query.timeout.seconds via the
URL query string (getConnection postfix), matching the driver doc for
db;sess?hive_conf. Asserts timeout message shows 1s (HIVE-28265).

Made-with: Cursor
…er2.con

Sonar: local name 'con' shadowed static field 'con' (HiddenField).
Made-with: Cursor
Parse hive.query.timeout.seconds from connParams.getHiveConfs() at connect
time using HiveConf.getTimeVar (same semantics as HiveStatement SET path)
so JDBC timeout messages work when the timeout is set via URL only.

Made-with: Cursor
…ements

testQueryTimeoutMessagePersistedAcrossStatements verifies that when SET
hive.query.timeout.seconds is issued on a separate closed statement, the
tracked value on HiveConnection still drives the SQLTimeoutException message
on a subsequent new statement (no setQueryTimeout call).

Made-with: Cursor
…ctor

Extract retry-config parsing from HiveConnection(uri,info,...,initSession)
into readRetryIntervalMillis() so the constructor fits within Sonar's
150-line limit (was 151, now 142).

Made-with: Cursor
Made-with: Cursor
The reviewer (InvisibleProgrammer) correctly pointed out that scanning
every executed SQL string with a regex to detect SET hive.query.timeout.seconds
is the wrong approach. The right source is TGetOperationStatusResp.errorMessage:
SQLOperation already sets "Query timed out after N seconds" with the real value
before cancel(TIMEDOUT), so the client-side regex fallback is unnecessary.

Remove:
- Pattern SET_HIVE_QUERY_TIMEOUT_SECONDS field
- trackSessionQueryTimeoutIfSet(sql) method and its call site
- Unused imports (HiveConf, TimeUnit, Matcher, Pattern)

The timeout message resolution is now:
1. Server errorMessage from TGetOperationStatusResp (primary - correct approach)
2. Statement-level setQueryTimeout() (JDBC standard)
3. URL-seeded hive.query.timeout.seconds from applySessionQueryTimeoutFromJdbcUrl()

All three paths avoid per-statement SQL parsing.

Made-with: Cursor
@ashniku
Copy link
Copy Markdown
Contributor Author

ashniku commented Apr 27, 2026

I wonder, how this solution behaves with a test case when we have a single connection and we execute multiple statements, one-by-one?
Example test case:

  @Test
  public void testQueryTimeout() throws Exception {
    String udfName = SleepMsUDF.class.getName();
    Statement stmt1 = con.createStatement();
    stmt1.execute("create temporary function sleepMsUDF as '" + udfName + "'");
    stmt1.close();
    Statement stmt2 = con.createStatement();
    stmt2.execute("set hive.query.timeout.seconds=1s");
    stmt2.close();
    Statement stmt = con.createStatement();

    System.err.println("Executing query: ");
    try {
      // The test table has 500 rows, so total query time should be ~ 2500ms
      stmt.executeQuery("select sleepMsUDF(t1.under_col, 5) as u0, t1.under_col as u1, "
          + "t2.under_col as u2 from " + tableName + " t1 join " + tableName
          + " t2 on t1.under_col = t2.under_col");
      fail("Expecting SQLTimeoutException");
    } catch (SQLTimeoutException e) {
      assertNotNull(e);
      System.err.println(e.toString());
      assertEquals("Query timed out after 1 seconds", e.getMessage());
    } catch (SQLException e) {
      fail("Expecting SQLTimeoutException, but got SQLException: " + e);
      e.printStackTrace();
    }

    // Test a query where timeout does not kick in. Set it to 5s;
    // show tables should be faster than that
    stmt.setQueryTimeout(5);
    try {
      stmt.executeQuery("show tables");
    } catch (SQLException e) {
      fail("Unexpected SQLException: " + e);
      e.printStackTrace();
    }
    stmt.close();
  }

@ashniku
Copy link
Copy Markdown
Contributor Author

ashniku commented Apr 27, 2026

@InvisibleProgrammer Yes you are right, I was in the wrong direction.

Did you get a chance to reproduce the issue ?

I have changed ,the regex approach was the wrong abstraction. I've removed trackSessionQueryTimeoutIfSet and the Pattern from HiveStatement entirely.

The resolution order is now:

  1. TGetOperationStatusResp.errorMessage (primary) — SQLOperation sets "Query timed out after N seconds" with the actual value before cancel(TIMEDOUT), so the client simply passes this through.

  2. Statement.setQueryTimeout(n) — JDBC standard, no change.
    URL-seeded value — applySessionQueryTimeoutFromJdbcUrl() reads hive.query.timeout.seconds from connParams.getHiveConfs() once at connect time (no SQL scanning at any point).

  3. No SQL string is parsed per-statement on the client side anymore. The multi-statement test (testQueryTimeoutMessagePersistedAcrossStatements) still passes because the server message carries the correct duration.

Could you please check if this approach is the correct one. Could you please ping me @ akpatra@cloudera.com.

Could you please help me suggest the approach so that I can work on it as I am new to opensource project.

@sonarqubecloud
Copy link
Copy Markdown

context + ": should start with " + QUERY_TIMED_OUT_AFTER_1_SECONDS
+ " (HS2 may append ; Query ID: ...); actual=" + msg,
msg.startsWith(QUERY_TIMED_OUT_AFTER_1_SECONDS));
assertFalse(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

What is the intent of this assert? If the message contains after 0 seconds, the previous assertion will fail. Otherwise, we have a new problem, like duplicated output. But this test case is not about testing that one.

public void testURLWithFetchSize() throws SQLException {
Connection con = getConnection(testDbName + ";fetchSize=1234", "");
Statement stmt = con.createStatement();
Connection connectionWithFetchSize = getConnection(testDbName + ";fetchSize=1234", "");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

What is the intention of the changes in this test case?

* in the URL query ({@code ?hive_conf_list}) per the driver format
* {@code jdbc:hive2://.../db;sess?hive_conf#hive_var}.
* <p>
* HIVE-28265: {@link SQLTimeoutException#getMessage()} must reflect the configured limit (1s),
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Please do not add exact ticket numbers into the code.

stmt1.close();

Statement stmt = con.createStatement();
stmt.execute("set hive.query.timeout.seconds=1s");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

The name and the description of the test case suggests it gets the timeout from HiveConf. But it sets the timeout with a set command, like in testQueryTimeoutMessagePersistedAcrossStatements

* Sentinel: no {@code hive.query.timeout.seconds} has been applied from the JDBC URL or a client
* {@code SET} on this connection yet.
*/
static final long SESSION_QUERY_TIMEOUT_NOT_TRACKED = -1L;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Really minor: I don't know if worth introducing a constant just to use it at only one place, the very next effective code line.

}

/**
* If the JDBC URL supplied {@code hive.query.timeout.seconds} (query string / {@code hiveconf:} map),
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I would suggest to rethink this comment. I wonder how a person who didn't see the history of this PR would interpret this:
"without regexparsing..."

* without regex-parsing {@code SET} statements. Does not change HS2 behavior (already applied in
* {@link #openSession()}).
*/
private void applySessionQueryTimeoutFromJdbcUrl() {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I don't think if it is an elegant/performant way of getting a config value from the jdbc connection string:

Creating a HiveConf object just to validate if the setting is a proper time setting looks a little bit overkill. HiveConf is not a small object at all. But also, TimeValidator is not a public class in common so I'm not sure if it can be used here. Maybe making TimeValidator to public can easily solve this issue.

I'm also not sure if we need to handle that case at all. I mean, HIVE-28265 is about improving the timeout message at set expressions. Honestly, I didn't even know that we support setting hive config values in the jdbc connection string. But anyway, if it works, it works.

@ayushtkn , can I ping you to receive your opinion again?

}

/**
* HIVE-28265: Prefer server error text (from {@code TGetOperationStatusResp.errorMessage}) unless
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Please do not add ticket numbers to the source code.


/**
* One GetOperationStatus response: progress update, Thrift status check, then terminal states.
* Extracted to keep {@link #waitForOperationToComplete()} smaller for static analysis (Sonar).
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I don't think if comments like this line adding extra information. Like, what if we switch from Sonar to an other tool? The logic is extracted. Great. It is easier to read the code now. But providing that information maybe too much.


// set progress bar to be completed when hive query execution has completed
if (inPlaceUpdateStream.isPresent()) {
if (progressUpdates) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Honestly, I don't know the answer for this question. But I feel I have to ask this: can the value of inPlaceUpdateStream.isPresent() be changed during the code execution?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants