-
Notifications
You must be signed in to change notification settings - Fork 4.8k
HIVE-28265: Fix JDBC timeout message for hive.query.timeout.seconds #6412
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
eec11d2
ab425d3
c5399f3
eaae132
aeb5a7c
6c5d8e9
ac0063c
9449a43
2ca54af
32aafd2
8283cd8
d717638
73eebcf
5b110e6
d88a80c
8017eef
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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; | ||
|
|
||
|
|
@@ -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; | ||
|
|
@@ -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; | ||
|
|
||
| /** | ||
|
|
@@ -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); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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? @ayushtkn , what do you think?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thx |
||
| private String jdbcUriString; | ||
| private String host; | ||
| private int port; | ||
|
|
@@ -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 | ||
|
|
@@ -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. | ||
|
|
||
There was a problem hiding this comment.
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?There was a problem hiding this comment.
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.”
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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.