You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2020/10/14 07:51:57 UTC

[hive] branch master updated: HIVE-24106: Abort polling on the operation state when the current thr… (#1456) (Zhihua Deng reviewed by Zoltan Haindrich)

This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 6392c0e  HIVE-24106: Abort polling on the operation state when the current thr… (#1456) (Zhihua Deng reviewed by Zoltan Haindrich)
6392c0e is described below

commit 6392c0e7f16f0f767aa8c1b70eafa672c0b7865f
Author: dengzh <de...@gmail.com>
AuthorDate: Wed Oct 14 15:51:43 2020 +0800

    HIVE-24106: Abort polling on the operation state when the current thr… (#1456) (Zhihua Deng reviewed by Zoltan Haindrich)
---
 .../java/org/apache/hadoop/hive/ql/ErrorMsg.java   |  1 +
 .../org/apache/hive/jdbc/TestJdbcWithMiniHS2.java  | 29 ++++++++++++++++++++++
 .../java/org/apache/hive/jdbc/HiveStatement.java   |  6 +++++
 3 files changed, 36 insertions(+)

diff --git a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index 92406cd..9b50f75 100644
--- a/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -509,6 +509,7 @@ public enum ErrorMsg {
 
   REPL_FILE_MISSING_FROM_SRC_AND_CM_PATH(20016, "File is missing from both source and cm path."),
   REPL_EXTERNAL_SERVICE_CONNECTION_ERROR(20017, "Failed to connect to {0} service. Error code {1}.",true),
+  CLIENT_POLLING_OPSTATUS_INTERRUPTED(20018, "Interrupted while polling on the operation status", "70100"),
 
   // An exception from runtime that will show the full stack to client
   UNRESOLVED_RT_EXCEPTION(29999, "Runtime Error: {0}", "58004", true),
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
index b80fb58..acbc5fd 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
@@ -47,6 +47,7 @@ import java.util.Set;
 import java.util.Base64;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CancellationException;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -1683,4 +1684,32 @@ public class TestJdbcWithMiniHS2 {
       assertTrue(getDetailedTableDescription(stmt, "emp_mm_table").contains(mndPath));
     }
   }
+
+  @Test
+  public void testInterruptPollingState() throws Exception {
+    ExecutorService pool = Executors.newFixedThreadPool(1);
+    final CountDownLatch latch = new CountDownLatch(1);
+    final Object[] results = new Object[2];
+    results[0] = false;
+    Future future = pool.submit(new Callable<Void>() {
+      @Override
+      public Void call() {
+        try (Statement stmt = conTestDb.createStatement()) {
+          stmt.execute("create temporary function sleepMsUDF as '" + SleepMsUDF.class.getName() + "'");
+          stmt.execute("SELECT sleepMsUDF(1, 10000)");
+          results[0] = true;
+        } catch (Exception e) {
+          results[1] = e;
+        } finally {
+          latch.countDown();
+        }
+        return null;
+      }
+    });
+    Thread.sleep(2000);
+    future.cancel(true);
+    latch.await();
+    assertEquals(false, results[0]);
+    assertEquals("Interrupted while polling on the operation status", ((Exception)results[1]).getMessage());
+  }
 }
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
index cd1c130..0fb8496 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
@@ -59,6 +59,8 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 
+import static org.apache.hadoop.hive.ql.ErrorMsg.CLIENT_POLLING_OPSTATUS_INTERRUPTED;
+
 /**
  * The object used for executing a static SQL statement and returning the
  * results it produces.
@@ -360,6 +362,10 @@ public class HiveStatement implements java.sql.Statement {
     // Poll on the operation status, till the operation is complete
     do {
       try {
+        if (Thread.currentThread().isInterrupted()) {
+          throw new SQLException(CLIENT_POLLING_OPSTATUS_INTERRUPTED.getMsg(),
+              CLIENT_POLLING_OPSTATUS_INTERRUPTED.getSQLState());
+        }
         /**
          * For an async SQLOperation, GetOperationStatus will use the long polling approach It will
          * essentially return after the HIVE_SERVER2_LONG_POLLING_TIMEOUT (a server config) expires