Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
21 commits
Select commit Hold shift + click to select a range
c0402ba
HIVE-28265: Fix JDBC timeout message for hive.query.timeout.seconds
ashniku Apr 6, 2026
dd4d51a
HIVE-28265: Refactor HiveStatement timeout handling; reset session ti…
ashniku Apr 6, 2026
d675ed1
HIVE-28265: Tighten query timeout message assertions in TestJdbcDriver2
ashniku Apr 7, 2026
06edfcc
HIVE-28265: Address JDBC PR review (assertEquals, setter names, docs)
ashniku Apr 7, 2026
c8bb6bb
HIVE-28265: Strip Query ID suffix from JDBC SQLTimeoutException message
ashniku Apr 7, 2026
1ab8507
HIVE-28265: Keep full HS2 timeout message; relax JDBC tests for Query…
ashniku Apr 10, 2026
ad53984
HIVE-28265: Refactor HiveStatement for Sonar (not all Sonar PR issues)
ashniku Apr 10, 2026
126310b
HIVE-28265: Checkstyle SKIPPED_ATTEMPTS; fix Beeline/llap precommit t…
ashniku Apr 11, 2026
081ac9c
HIVE-28265: Remove non-core files from PR (match master)
ashniku Apr 20, 2026
3ec5d37
HIVE-28265: Add JDBC URL query timeout test (like testURLWithFetchSize)
ashniku Apr 20, 2026
f9b858f
HIVE-28265: Rename local Connection vars to avoid hiding TestJdbcDriv…
ashniku Apr 20, 2026
7cad7ef
HIVE-28265: Seed session query timeout from JDBC URL hive conf
ashniku Apr 21, 2026
dabb076
HIVE-28265: Add test for session timeout message across multiple stat…
ashniku Apr 22, 2026
c6362df
HIVE-28265: Fix Sonar method-length warning in HiveConnection constru…
ashniku Apr 26, 2026
b1f1d35
HIVE-28265: Trigger CI
ashniku Apr 26, 2026
72dc8bf
HIVE-28265: Remove regex SET-tracking from HiveStatement
ashniku Apr 26, 2026
f7eb960
HIVE-28265: Trigger CI re-run
ashniku Apr 27, 2026
cfefc27
HIVE-28265: Address reviewer comments (Javadoc, constant, test name)
ashniku Apr 28, 2026
cd83d9f
HIVE-28265: Use HYT00 SQL state to select timeout message; restore 01…
ashniku May 4, 2026
e84ee27
HIVE-28265: Use HiveConf.toTime to parse JDBC URL timeout (no full Hi…
ashniku Jun 18, 2026
c3d2ccc
HIVE-28265: Trigger CI re-run
ashniku Jun 19, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import org.apache.hive.service.cli.operation.ClassicTableTypeMapping.ClassicTableTypes;
import org.apache.hive.service.cli.operation.HiveTableTypeMapping;
import org.apache.hive.service.cli.operation.TableTypeMappingFactory.TableTypeMappings;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
Expand Down Expand Up @@ -128,9 +129,43 @@ public class TestJdbcDriver2 {
private static Connection con;
private static final float floatCompareDelta = 0.0001f;

/**
* Required prefix of {@link SQLTimeoutException#getMessage()} for a 1s limit. HS2 may append
* {@code ; Query ID: ...} after the base text from {@code HiveSQLException}.
*/
private static final String QUERY_TIMED_OUT_AFTER_1_SECONDS = "Query timed out after 1 seconds";

@Rule public ExpectedException thrown = ExpectedException.none();
@Rule public final TestName testName = new TestName();

/**
* {@code SET hive.query.timeout.seconds} applies to the whole HS2 session. Tests such as
* {@link #testQueryTimeoutFromSetStatement()} must not leave a short limit on the shared
* {@link #con}, or unrelated tests will see {@link SQLTimeoutException}.
*/
@After
public void resetHiveSessionQueryTimeout() {
try {
if (con == null || con.isClosed()) {
return;
}
try (Statement st = con.createStatement()) {
st.execute("set hive.query.timeout.seconds=0s");
}
} catch (SQLException e) {
LOG.warn("Could not reset hive.query.timeout.seconds after {}", testName.getMethodName(), e);
}
}

private static void assertTimeoutMessageShowsOneSecond(String context, SQLTimeoutException e) {
String msg = e.getMessage();
assertNotNull(context + ": message should not be null", msg);
assertTrue(
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));
}

private static Connection getConnection(String prefix, String postfix) throws SQLException {
Connection con1;
String connString = "jdbc:hive2:///" + prefix + "?" + conf.getOverlayOptionsAsQueryString()
Expand Down Expand Up @@ -336,11 +371,46 @@ private void checkBadUrl(String url) throws SQLException {
* @throws SQLException
*/
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?

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.

The only change to testURLWithFetchSize was a rename: the local variable con was renamed to connectionWithFetchSize to avoid a Sonar HiddenField warning, because the method-level con shadowed the class-level static Connection con. No logic was changed.

Statement stmt = connectionWithFetchSize.createStatement();
assertEquals(stmt.getFetchSize(), 1234);
stmt.close();
con.close();
connectionWithFetchSize.close();
}

/**
* Same idea as {@link #testURLWithFetchSize}: drive session behavior from the JDBC URL instead of
* only {@link Statement#setQueryTimeout(int)} or an explicit {@code SET}. The timeout is supplied
* in the URL query ({@code ?hive_conf_list}) per the driver format
* {@code jdbc:hive2://.../db;sess?hive_conf#hive_var}.
* <p>
* {@link SQLTimeoutException#getMessage()} must reflect the configured limit (1s),
* not {@code after 0 seconds}.
*/
@Test
public void testURLWithHiveQueryTimeoutSeconds() throws Exception {
String udfName = SleepMsUDF.class.getName();
// Postfix appends to the query string after test overlay / lock manager settings.
Connection connectionWithUrlQueryTimeout = getConnection(testDbName, "hive.query.timeout.seconds=1");
try {
Statement stmt1 = connectionWithUrlQueryTimeout.createStatement();
stmt1.execute("create temporary function sleepMsUDF as '" + udfName + "'");
stmt1.close();
Statement stmt = connectionWithUrlQueryTimeout.createStatement();
try {
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) {
assertTimeoutMessageShowsOneSecond("JDBC URL hive.query.timeout.seconds=1 (query string)", e);
} catch (SQLException e) {
fail("Expecting SQLTimeoutException, but got SQLException: " + e);
}
stmt.close();
} finally {
connectionWithUrlQueryTimeout.close();
}
}

@Test
Expand All @@ -349,13 +419,13 @@ public void testURLWithFetchSize() throws SQLException {
* @throws SQLException
*/
public void testCreateTableAsExternal() throws SQLException {
Connection con = getConnection(testDbName + ";hiveCreateAsExternalLegacy=true", "");
Statement stmt = con.createStatement();
Connection connectionWithExternalLegacy = getConnection(testDbName + ";hiveCreateAsExternalLegacy=true", "");
Statement stmt = connectionWithExternalLegacy.createStatement();
ResultSet res = stmt.executeQuery("set hive.create.as.external.legacy");
assertTrue("ResultSet is empty", res.next());
assertEquals("hive.create.as.external.legacy=true", res.getObject(1));
stmt.close();
con.close();
connectionWithExternalLegacy.close();
}

@Test
Expand Down Expand Up @@ -2661,7 +2731,8 @@ public void testQueryTimeout() throws Exception {
+ " t2 on t1.under_col = t2.under_col");
fail("Expecting SQLTimeoutException");
} catch (SQLTimeoutException e) {
assertNotNull(e);
assertTimeoutMessageShowsOneSecond(

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 value of this change?

Let's see, for example the existing test case, testURLWithFetchSize. Is it required having the timeout set to 1 seconds?

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.

@InvisibleProgrammer The old assertNotNull(e) didn’t validate the message. HIVE-28265 fixes cases where the message incorrectly said “after 0 seconds” while the real limit was 1 second. We set the limit to 1s in both tests (setQueryTimeout(1) vs SET hive.query.timeout.seconds=1s) and assert the message starts with Query timed out after 1 seconds and does not contain after 0 seconds. The 1 is the configured timeout, not an arbitrary magic number—if we used 2s, we’d assert 2 seconds. testURLWithFetchSize is a different feature (URL fetchSize); the analogy is only “set config → assert behavior.”

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.

I have commited new one.The new test testURLWithHiveQueryTimeoutSeconds is explicitly a 1 second case:

URL — it opens the connection with
getConnection(testDbName, "hive.query.timeout.seconds=1")
so the effective limit is 1 second.

Assertion — it calls assertTimeoutMessageShowsOneSecond, which requires the message to start with
Query timed out after 1 seconds
(the same constant as testQueryTimeout / testQueryTimeoutMessageUsesHiveConf).

So it’s aligned with the other 1s timeout tests, just with the limit coming from the JDBC URL query instead of setQueryTimeout(1) or SET hive.query.timeout.seconds=1s.

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.

@InvisibleProgrammer could you please check when free and suggest?

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 have doubts about the change itself so currently I'm trying to reproduce the original issue.

The regex way of handling the set hive.query.timeout.seconds statement: in general, set statements are modifying variables that Hive already loads into a HiveConf object. The way of having a regex and checking every single statement makes me suspicious. I don't really get why we should introduce a new way of receiving set commands for one specific command. I more wonder about an other approach, like letting the current value of the setting for the statement/connection or getting a proper exception message at the right way from TGetOperationStatusResp.

On overall, I don't want to accept that the only way to pass a hive config value for a connection (or the effect of this value) is by checking all the statements that the user executed one-by-one.

"JDBC query timeout (1s)", e);
System.err.println(e.toString());
} catch (SQLException e) {
fail("Expecting SQLTimeoutException, but got SQLException: " + e);
Expand All @@ -2680,6 +2751,83 @@ public void testQueryTimeout() throws Exception {
stmt.close();
}

/**
* When the session timeout is configured via a {@code SET hive.query.timeout.seconds=...}
* statement and no {@link Statement#setQueryTimeout(int)} is called, the
* {@link SQLTimeoutException#getMessage()} must still reflect the real limit.
* Message must begin with {@link #QUERY_TIMED_OUT_AFTER_1_SECONDS};
* HS2 may append {@code ; Query ID: ...}.
*/
@Test
public void testQueryTimeoutFromSetStatement() throws Exception {
String udfName = SleepMsUDF.class.getName();
Statement stmt1 = con.createStatement();
stmt1.execute("create temporary function sleepMsUDF as '" + udfName + "'");
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

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.

You are right. The name says "UsesHiveConf" — which implies the timeout is configured through a HiveConf object — but the implementation uses stmt.execute("set hive.query.timeout.seconds=1s"), a SQL SET statement, which is the same mechanism as testQueryTimeoutMessagePersistedAcrossStatements. The name is misleading.

What the test was actually trying to capture: The intended distinction from testQueryTimeout (which uses the JDBC standard stmt.setQueryTimeout(1)) is: timeout set via a session-level Hive config SET, with no setQueryTimeout() call on the statement. "HiveConf" was used loosely to mean "Hive session configuration" rather than "the JDBC timeout API".

The real distinction between the two SET-based tests is the statement lifecycle:

testQueryTimeoutMessageUsesHiveConf: SET and executeQuery run on the same open statement.
testQueryTimeoutMessagePersistedAcrossStatements: SET on stmt2 which is then closed; executeQuery on a brand-new stmt.

Suggestion: Rename testQueryTimeoutMessageUsesHiveConf to something like testQueryTimeoutMessageFromSessionSet (or testQueryTimeoutFromSetStatement) and update its Javadoc to describe the actual scenario — session timeout set via a SET command on the same statement, no setQueryTimeout() call — without the misleading "HiveConf" label. I'm happy to do that rename if you agree.


try {
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) {
assertTimeoutMessageShowsOneSecond(
"Session query timeout (1s)", e);
} catch (SQLException e) {
fail("Expecting SQLTimeoutException, but got SQLException: " + e);
}
stmt.close();
}

/**
* Variant of {@link #testQueryTimeoutFromSetStatement}: the {@code SET} is issued on a separate,
* already-closed statement; the timed-out query runs on a brand-new statement with no
* {@link Statement#setQueryTimeout(int)} call. The tracked session timeout lives on the
* {@link HiveConnection}, so it persists across statement instances and must still drive the
* {@link SQLTimeoutException} message correctly.
*/
@Test
public void testQueryTimeoutMessagePersistedAcrossStatements() throws Exception {
String udfName = SleepMsUDF.class.getName();
Statement stmt1 = con.createStatement();
stmt1.execute("create temporary function sleepMsUDF as '" + udfName + "'");
stmt1.close();

// SET is issued on stmt2, which is then closed – timeout must survive on the connection
Statement stmt2 = con.createStatement();
stmt2.execute("set hive.query.timeout.seconds=1s");
stmt2.close();

// Brand-new statement, no setQueryTimeout() call – relies solely on the tracked session value
Statement stmt = con.createStatement();
System.err.println("Executing query (expecting timeout): ");
try {
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) {
assertTimeoutMessageShowsOneSecond("SET on closed stmt2, executeQuery on new stmt", e);
System.err.println(e.toString());
} catch (SQLException e) {
fail("Expecting SQLTimeoutException, but got SQLException: " + e);
e.printStackTrace();
}

// A fast query must still complete when the per-statement timeout overrides
stmt.setQueryTimeout(5);
try {
stmt.executeQuery("show tables");
} catch (SQLException e) {
fail("Unexpected SQLException: " + e);
e.printStackTrace();
}
stmt.close();
}

/**
* Test the non-null value of the Yarn ATS GUID.
* We spawn 2 threads - one running the query and
Expand Down
86 changes: 74 additions & 12 deletions jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@
import java.util.Optional;
import java.util.Properties;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Stream;

Expand All @@ -82,6 +83,7 @@

import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hive.common.auth.HiveAuthUtils;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
Expand Down Expand Up @@ -155,6 +157,7 @@
import org.apache.thrift.transport.TTransportException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Supplier;

/**
Expand All @@ -163,6 +166,14 @@
*/
public class HiveConnection implements java.sql.Connection {
private static final Logger LOG = LoggerFactory.getLogger(HiveConnection.class);

/**
* Last effective {@code hive.query.timeout.seconds} in seconds, or {@code -1} if not yet set.
* Seeded from the JDBC URL at connect time; a JDBC {@link java.sql.Connection} may be shared
* across threads with concurrent {@link org.apache.hive.jdbc.HiveStatement}s on one HS2 session,
* so this field uses an {@link AtomicLong} to keep updates well-defined.
*/
private final AtomicLong sessionQueryTimeoutSeconds = new AtomicLong(-1L);
private String jdbcUriString;
private String host;
private int port;
Expand Down Expand Up @@ -190,6 +201,46 @@ public class HiveConnection implements java.sql.Connection {

public TCLIService.Iface getClient() { return client; }

/**
* Updates the tracked {@code hive.query.timeout.seconds} value (in seconds) on this connection.
* Called at connect time from the JDBC URL hive-conf map, and may be called again later if needed.
*/
void setSessionQueryTimeoutSeconds(long seconds) {
sessionQueryTimeoutSeconds.set(seconds);
}

/**
* If the JDBC URL supplied {@code hive.query.timeout.seconds} (via the {@code ?hive_conf_list}
* segment), parses and stores the value so that {@link #getSessionQueryTimeoutSeconds()} can
* return it for timeout error messages. This runs once at connect time and does not affect the
* server-side configuration, which is applied separately 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?

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.

@InvisibleProgrammer I will be waiting for your reply on this.

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 didn't find significantly better alternative that doesn't lead more changes: the only other solution I found is to extracting the time handling logging in commons so that it doesn't only accept a configuration object (like now HiveConf.getTimeVar does) but just simply accepts a String so that we wouldn't need a full HiveConf object, just to get that information.
But as it runs only once, at the initialization of the connection, I think it should be fine.

Map<String, String> hiveConfs = connParams.getHiveConfs();
if (hiveConfs == null || hiveConfs.isEmpty()) {
return;
}
String raw = hiveConfs.get(ConfVars.HIVE_QUERY_TIMEOUT_SECONDS.varname);
if (StringUtils.isBlank(raw)) {
return;
}
try {
long sec = HiveConf.toTime(raw.trim(), TimeUnit.SECONDS, TimeUnit.SECONDS);
if (sec > 0) {
setSessionQueryTimeoutSeconds(sec);
}
} catch (Exception e) {
LOG.debug("Could not parse {} from JDBC URL: {}", ConfVars.HIVE_QUERY_TIMEOUT_SECONDS.varname, raw, e);
}
}

/**
* @return the tracked {@code hive.query.timeout.seconds} in seconds, or {@code -1} if not set
*/
long getSessionQueryTimeoutSeconds() {
return sessionQueryTimeoutSeconds.get();
}

/**
* Get all direct HiveServer2 URLs from a ZooKeeper based HiveServer2 URL
* @param zookeeperBasedHS2Url
Expand Down Expand Up @@ -332,6 +383,7 @@ protected HiveConnection(String uri, Properties info,
// hive_conf_list -> hiveConfMap
// hive_var_list -> hiveVarMap
sessConfMap = connParams.getSessionVars();
applySessionQueryTimeoutFromJdbcUrl();
setupLoginTimeout();
if (isKerberosAuthMode()) {
// Ensure UserGroupInformation includes any authorized Kerberos principals.
Expand Down Expand Up @@ -396,18 +448,7 @@ protected HiveConnection(String uri, Properties info,
executeInitSql();
}
} else {
long retryInterval = 1000L;
try {
String strRetries = sessConfMap.get(JdbcConnectionParams.RETRIES);
if (StringUtils.isNotBlank(strRetries)) {
maxRetries = Integer.parseInt(strRetries);
}
String strRetryInterval = sessConfMap.get(JdbcConnectionParams.RETRY_INTERVAL);
if(StringUtils.isNotBlank(strRetryInterval)){
retryInterval = Long.parseLong(strRetryInterval);
}
} catch(NumberFormatException e) { // Ignore the exception
}
long retryInterval = readRetryIntervalMillis();

for (int numRetries = 0;;) {
try {
Expand Down Expand Up @@ -469,6 +510,27 @@ protected HiveConnection(String uri, Properties info,
client = newSynchronizedClient(client);
}

/**
* Reads {@code retries} and {@code retryInterval} from the session configuration, updating
* {@link #maxRetries} as a side-effect, and returns the interval in milliseconds (default 1000).
* Extracted from the constructor to keep its length within Sonar's 150-line limit.
*/
private long readRetryIntervalMillis() {
long retryInterval = 1000L;
try {
String strRetries = sessConfMap.get(JdbcConnectionParams.RETRIES);
if (StringUtils.isNotBlank(strRetries)) {
maxRetries = Integer.parseInt(strRetries);
}
String strRetryInterval = sessConfMap.get(JdbcConnectionParams.RETRY_INTERVAL);
if (StringUtils.isNotBlank(strRetryInterval)) {
retryInterval = Long.parseLong(strRetryInterval);
}
} catch (NumberFormatException e) { // Ignore — bad values are silently skipped
}
return retryInterval;
}

private void executeInitSql() throws SQLException {
if (initFile != null) {
try (Statement st = createStatement()) {
Expand Down
Loading
Loading