Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
16 commits
Select commit Hold shift + click to select a range
eec11d2
HIVE-28265: Fix JDBC timeout message for hive.query.timeout.seconds
ashniku Apr 6, 2026
ab425d3
HIVE-28265: Refactor HiveStatement timeout handling; reset session ti…
ashniku Apr 6, 2026
c5399f3
HIVE-28265: Tighten query timeout message assertions in TestJdbcDriver2
ashniku Apr 7, 2026
eaae132
HIVE-28265: Address JDBC PR review (assertEquals, setter names, docs)
ashniku Apr 7, 2026
aeb5a7c
HIVE-28265: Strip Query ID suffix from JDBC SQLTimeoutException message
ashniku Apr 7, 2026
6c5d8e9
HIVE-28265: Keep full HS2 timeout message; relax JDBC tests for Query…
ashniku Apr 10, 2026
ac0063c
HIVE-28265: Refactor HiveStatement for Sonar (not all Sonar PR issues)
ashniku Apr 10, 2026
9449a43
HIVE-28265: Fix flaky TestPartitionManagement concurrent skip counter
ashniku Apr 10, 2026
2ca54af
HIVE-28265: Remove unused partitionDiscoveryEnabled (Sonar)
ashniku Apr 11, 2026
32aafd2
Revert "HIVE-28265: Remove unused partitionDiscoveryEnabled (Sonar)"
ashniku Apr 11, 2026
8283cd8
HIVE-28265: Checkstyle SKIPPED_ATTEMPTS; fix Beeline/llap precommit t…
ashniku Apr 11, 2026
d717638
HIVE-28265: Remove non-core files from PR (match master)
ashniku Apr 20, 2026
73eebcf
HIVE-28265: Add JDBC URL query timeout test (like testURLWithFetchSize)
ashniku Apr 20, 2026
5b110e6
HIVE-28265: Rename local Connection vars to avoid hiding TestJdbcDriv…
ashniku Apr 20, 2026
d88a80c
HIVE-28265: Seed session query timeout from JDBC URL hive conf
ashniku Apr 21, 2026
8017eef
HIVE-28265: Add test for session timeout message across multiple stat…
ashniku Apr 22, 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,46 @@ 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 #testQueryTimeoutMessageUsesHiveConf()} 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));
assertFalse(
"HIVE-28265: message should not claim 0 seconds: " + msg,
msg.contains("after 0 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 +374,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", "");
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>
* HIVE-28265: {@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 +422,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 +2734,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 +2754,82 @@ public void testQueryTimeout() throws Exception {
stmt.close();
}

/**
* When only {@code hive.query.timeout.seconds} applies (no {@link Statement#setQueryTimeout(int)}),
* the client must still report the real limit in {@link SQLTimeoutException#getMessage()} (before
* HIVE-28265 some paths wrongly showed "after 0 seconds"). Message must begin with
* {@link #QUERY_TIMED_OUT_AFTER_1_SECONDS}; HS2 may append {@code ; Query ID: ...}.
*/
@Test
public void testQueryTimeoutMessageUsesHiveConf() 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");

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 #testQueryTimeoutMessageUsesHiveConf}: 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. Covers the HIVE-28265 multi-statement scenario.
*/
@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
62 changes: 62 additions & 0 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,21 @@
*/
public class HiveConnection implements java.sql.Connection {
private static final Logger LOG = LoggerFactory.getLogger(HiveConnection.class);

/**
* 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;
/**
* Last effective {@code hive.query.timeout.seconds} in seconds: from the JDBC URL
* ({@code ?hive.query.timeout.seconds=...} / {@code hiveconf:} map) at connect time, and/or from a
* client {@code SET} (see {@link org.apache.hive.jdbc.HiveStatement}), or
* {@link #SESSION_QUERY_TIMEOUT_NOT_TRACKED}. A JDBC {@code Connection} may be shared across threads
* with concurrent {@link org.apache.hive.jdbc.HiveStatement}s on one HS2 session; this field uses an
* {@link AtomicLong} so updates remain well-defined (URL first, then last SET wins over prior value).
*/
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

private String jdbcUriString;
private String host;
private int port;
Expand Down Expand Up @@ -190,6 +208,49 @@ public class HiveConnection implements java.sql.Connection {

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

/**
* Sets the effective {@code hive.query.timeout.seconds} (in seconds) after connect (URL) or a
* successful {@code SET hive.query.timeout.seconds=...}. Used for JDBC timeout messages (HIVE-28265).
*/
void setSessionQueryTimeoutSeconds(long seconds) {
sessionQueryTimeoutSeconds.set(seconds);
}

/**
* If the JDBC URL supplied {@code hive.query.timeout.seconds} (query string / {@code hiveconf:} map),
* parse and store it for {@link #getSessionQueryTimeoutSeconds()} so timeout error messages can use it
* without regex-parsing {@code SET} statements. Does not change HS2 behavior (already applied in
* {@link #openSession()}).
*/
private void applySessionQueryTimeoutFromJdbcUrl() {
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 {
HiveConf conf = new HiveConf();
conf.set(ConfVars.HIVE_QUERY_TIMEOUT_SECONDS.varname, raw.trim());
long sec = HiveConf.getTimeVar(conf, ConfVars.HIVE_QUERY_TIMEOUT_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 seconds from the JDBC URL at connect and/or the last client-tracked SET, or
* {@link #SESSION_QUERY_TIMEOUT_NOT_TRACKED} if neither applied
*/
long getSessionQueryTimeoutSeconds() {
return sessionQueryTimeoutSeconds.get();
}

/**
* Get all direct HiveServer2 URLs from a ZooKeeper based HiveServer2 URL
* @param zookeeperBasedHS2Url
Expand Down Expand Up @@ -332,6 +393,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
Loading
Loading