ashniku commented on code in PR #6412:
URL: https://github.com/apache/hive/pull/6412#discussion_r3151529054
##########
itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java:
##########
@@ -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", "");
Review Comment:
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.
##########
itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java:
##########
@@ -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(
Review Comment:
You are absolutely right. The assertFalse(...contains("after 0 seconds")) is
logically redundant: if msg.startsWith("Query timed out after 1 seconds") is
true, it is impossible for it to also contain "after 0 seconds". The
assertFalse can never add new information. I'll remove it — the
assertTrue(startsWith(...)) alone is sufficient.
##########
itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java:
##########
@@ -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),
Review Comment:
Agreed, I'll remove the HIVE-28265:
##########
jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java:
##########
@@ -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;
Review Comment:
Fair point. The constant is only used as the initial value of
sessionQueryTimeoutSeconds and in getSessionQueryTimeoutSeconds() Javadoc. I
can inline -1L and drop the constant entirely to simplify.
##########
jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java:
##########
@@ -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() {
Review Comment:
@InvisibleProgrammer I will be waiting for your reply on this.
##########
jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java:
##########
@@ -398,20 +398,95 @@ private TGetOperationStatusResp waitForResultSetStatus()
throws SQLException {
return statusResp;
}
+ /**
+ * HIVE-28265: Prefer server error text (from {@code
TGetOperationStatusResp.errorMessage}) unless
+ * it is empty or the legacy "0 seconds" value; fall back to JDBC {@link
#setQueryTimeout(int)}
+ * or the URL-seeded {@code hive.query.timeout.seconds} on the connection.
+ */
+ private String sqlTimeoutMessageForTimedOutState(String serverMessage) {
+ if (!needsLocalTimeoutMessageForTimedOut(serverMessage)) {
+ return serverMessage;
+ }
+ long effectiveSec = resolveEffectiveTimeoutSecondsForMessage();
+ if (effectiveSec > 0) {
+ return "Query timed out after " + effectiveSec + " seconds";
+ }
+ return "Query timed out";
+ }
+
+ private boolean needsLocalTimeoutMessageForTimedOut(String timeoutMsg) {
+ return StringUtils.isBlank(timeoutMsg)
+ || StringUtils.containsIgnoreCase(timeoutMsg, "after 0 seconds");
+ }
+
+ private long resolveEffectiveTimeoutSecondsForMessage() {
+ if (queryTimeout > 0) {
+ return queryTimeout;
+ }
+ long tracked = connection.getSessionQueryTimeoutSeconds();
+ if (tracked > 0) {
+ return tracked;
+ }
+ return 0L;
+ }
+
+ private SQLException sqlExceptionForCanceledState(TGetOperationStatusResp
statusResp) {
+ final String errMsg = statusResp.getErrorMessage();
+ final String fullErrMsg;
+ if (errMsg == null || errMsg.isEmpty()) {
+ fullErrMsg = QUERY_CANCELLED_MESSAGE;
+ } else {
+ fullErrMsg = QUERY_CANCELLED_MESSAGE + " " + errMsg;
+ }
+ return new SQLException(fullErrMsg, "01000");
+ }
+
+ /**
+ * One GetOperationStatus response: progress update, Thrift status check,
then terminal states.
+ * Extracted to keep {@link #waitForOperationToComplete()} smaller for
static analysis (Sonar).
Review Comment:
Correct. The comment should describe the purpose of the method, not the
tooling reason it was extracted. I'll rewrite it to simply describe what the
method does.
##########
jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java:
##########
@@ -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),
Review Comment:
That phrasing leaks internal PR history into source code and would be
confusing to future readers. I'll rewrite the Javadoc to explain only the what
and why, without referencing the historical alternative that was rejected.
##########
jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java:
##########
@@ -464,8 +509,7 @@ TGetOperationStatusResp waitForOperationToComplete() throws
SQLException {
}
} while (!isOperationComplete);
- // set progress bar to be completed when hive query execution has completed
- if (inPlaceUpdateStream.isPresent()) {
+ if (progressUpdates) {
Review Comment:
@InvisibleProgrammer I will be waiting for your reply on this.
##########
jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java:
##########
@@ -398,20 +398,95 @@ private TGetOperationStatusResp waitForResultSetStatus()
throws SQLException {
return statusResp;
}
+ /**
+ * HIVE-28265: Prefer server error text (from {@code
TGetOperationStatusResp.errorMessage}) unless
Review Comment:
Agreed, I'll remove the HIVE-28265: prefix from that Javadoc as well.
##########
itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java:
##########
@@ -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");
Review Comment:
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.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]