From 345faf7a2def1c4de5d94d934e6a0789c72678b5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 1 Jul 2020 15:11:07 +0800 Subject: [PATCH 1/5] [SPARK-32145][SQL] ThriftCLIService.GetOperationStatus should include exception's stack trace to the error message --- .../SparkExecuteStatementOperation.scala | 19 +++------- .../SparkGetCatalogsOperation.scala | 24 ++----------- .../SparkGetColumnsOperation.scala | 19 ++-------- .../SparkGetFunctionsOperation.scala | 18 ++-------- .../SparkGetSchemasOperation.scala | 18 ++-------- .../SparkGetTableTypesOperation.scala | 20 ++--------- .../SparkGetTablesOperation.scala | 25 +++---------- .../SparkGetTypeInfoOperation.scala | 22 ++---------- .../hive/thriftserver/SparkOperation.scala | 18 +++++++++- .../thriftserver/SharedThriftServer.scala | 35 +++++++++++++++++-- .../ThriftServerWithSparkContextSuite.scala | 34 ++++++++++++++++++ .../service/cli/thrift/ThriftCLIService.java | 3 +- 12 files changed, 109 insertions(+), 146 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 1404ece76449..eae5d5d4bcfa 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -18,26 +18,22 @@ package org.apache.spark.sql.hive.thriftserver import java.security.PrivilegedExceptionAction -import java.sql.{Date, Timestamp} -import java.util.{Arrays, Map => JMap, UUID} +import java.util.{Arrays, Map => JMap} import java.util.concurrent.RejectedExecutionException import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.shims.Utils import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLContext} import org.apache.spark.sql.execution.HiveResult.{getTimeFormatters, toHiveString, TimeFormatters} -import org.apache.spark.sql.execution.command.SetCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -315,16 +311,11 @@ private[hive] class SparkExecuteStatementOperation( } else { logError(s"Error executing query with $statementId, currentState $currentState, ", e) setState(OperationState.ERROR) + HiveThriftServer2.eventManager.onStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e)) e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error running query: " + root.toString, root) + case _: HiveSQLException => throw e + case _ => throw new HiveSQLException("Error running query: " + e.toString, e) } } } finally { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index 55070e035b94..01ef78cde895 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -17,17 +17,13 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.UUID - -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType -import org.apache.hive.service.cli.{HiveSQLException, OperationState} +import org.apache.hive.service.cli.OperationState import org.apache.hive.service.cli.operation.GetCatalogsOperation import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own GetCatalogsOperation @@ -62,22 +58,8 @@ private[hive] class SparkGetCatalogsOperation( authorizeMetaGets(HiveOperationType.GET_CATALOGS, null) } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get catalogs operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting catalogs: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index ca8ad5e6ad13..d42732f42668 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -35,7 +35,6 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.hive.thriftserver.ThriftserverShimUtils.toJavaSQLType import org.apache.spark.sql.types.StructType -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own SparkGetColumnsOperation @@ -122,22 +121,8 @@ private[hive] class SparkGetColumnsOperation( } } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get columns operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting columns: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index f5e647bfd4f3..cf5dbae93a36 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -98,22 +98,8 @@ private[hive] class SparkGetFunctionsOperation( } } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get functions operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting functions: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 74220986fcd3..16fd502048e8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -81,22 +81,8 @@ private[hive] class SparkGetSchemasOperation( rowSet.addRow(Array[AnyRef](globalTempViewDb, DEFAULT_HIVE_CATALOG)) } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get schemas operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting schemas: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index 1cf9c3a731af..9e31b8baad78 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetTableTypesOperation @@ -28,7 +27,6 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.catalog.CatalogTableType -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own GetTableTypesOperation @@ -69,22 +67,8 @@ private[hive] class SparkGetTableTypesOperation( rowSet.addRow(Array[AnyRef](tableType)) } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get table types operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting table types: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index a1d21e2d60c6..0d4b9b392f07 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -17,14 +17,12 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.{List => JList, UUID} +import java.util.{List => JList} import java.util.regex.Pattern import scala.collection.JavaConverters._ -import org.apache.commons.lang3.exception.ExceptionUtils -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils +import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObjectUtils} import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetTablesOperation import org.apache.hive.service.cli.session.HiveSession @@ -33,7 +31,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.hive.HiveUtils -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own GetTablesOperation @@ -111,22 +108,8 @@ private[hive] class SparkGetTablesOperation( } } setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get tables operation with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting tables: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index e38139d60df6..c2568ad4ada0 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -19,15 +19,13 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID -import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType -import org.apache.hive.service.cli.{HiveSQLException, OperationState} +import org.apache.hive.service.cli.OperationState import org.apache.hive.service.cli.operation.GetTypeInfoOperation import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext -import org.apache.spark.util.{Utils => SparkUtils} /** * Spark's own GetTypeInfoOperation @@ -87,22 +85,8 @@ private[hive] class SparkGetTypeInfoOperation( rowSet.addRow(rowData) }) setState(OperationState.FINISHED) - } catch { - case e: Throwable => - logError(s"Error executing get type info with $statementId", e) - setState(OperationState.ERROR) - e match { - case hiveException: HiveSQLException => - HiveThriftServer2.eventManager.onStatementError( - statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) - throw hiveException - case _ => - val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.eventManager.onStatementError( - statementId, root.getMessage, SparkUtils.exceptionString(root)) - throw new HiveSQLException("Error getting type info: " + root.toString, root) - } - } + } catch onError() + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index 3da568cfa256..514e31fa77b8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -import org.apache.hive.service.cli.OperationState +import org.apache.hive.service.cli.{HiveSQLException, OperationState} import org.apache.hive.service.cli.operation.Operation import org.apache.spark.SparkContext @@ -93,4 +93,20 @@ private[hive] trait SparkOperation extends Operation with Logging { case t => throw new IllegalArgumentException(s"Unknown table type is found: $t") } + + def state(newState: OperationState): OperationState = { + super.setState(newState) + } + + protected def onError(): PartialFunction[Throwable, Unit] = { + case e: Throwable => + logError(s"Error executing get catalogs operation with $statementId", e) + super.setState(OperationState.ERROR) + HiveThriftServer2.eventManager.onStatementError( + statementId, e.getMessage, Utils.exceptionString(e)) + e match { + case _: HiveSQLException => throw e + case _ => throw new HiveSQLException(s"Error operating $getType ${e.getMessage}", e) + } + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index 3d7933fba17d..b52ef06ae4f7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -24,10 +24,17 @@ import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.util.Try +import java.util import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hive.service.cli.thrift.ThriftCLIService +import org.apache.hive.jdbc.HttpBasicAuthInterceptor +import org.apache.hive.jdbc.Utils.JdbcConnectionParams +import org.apache.hive.service.auth.PlainSaslHelper +import org.apache.hive.service.cli.thrift.{ThriftCLIService, ThriftCLIServiceClient} +import org.apache.http.impl.client.HttpClientBuilder +import org.apache.thrift.protocol.TBinaryProtocol +import org.apache.thrift.transport.{THttpClient, TSocket} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils @@ -76,8 +83,9 @@ trait SharedThriftServer extends SharedSparkSession { s"jdbc:hive2://localhost:$serverPort/" } + protected def user: String = System.getProperty("user.name") + protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = { - val user = System.getProperty("user.name") require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } @@ -91,6 +99,29 @@ trait SharedThriftServer extends SharedSparkSession { } } + protected def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { + require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") + val transport = mode match { + case ServerMode.binary => + val rawTransport = new TSocket("localhost", serverPort) + PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) + case ServerMode.http => + val interceptor = new HttpBasicAuthInterceptor( + user, + "anonymous", + null, null, true, new util.HashMap[String, String]()) + new THttpClient( + s"http://localhost:$serverPort/cliservice", + HttpClientBuilder.create.addInterceptorFirst(interceptor).build()) + } + + val protocol = new TBinaryProtocol(transport) + val client = new ThriftCLIServiceClient(new ThriftserverShimUtils.Client(protocol)) + + transport.open() + try f(client) finally transport.close() + } + private def startThriftServer(attempt: Int): Unit = { logInfo(s"Trying to start HiveThriftServer2: mode=$mode, attempt=$attempt") val sqlContext = spark.newSession().sqlContext diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 1382eb2d79f3..fd3a638c4fa4 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -17,6 +17,10 @@ package org.apache.spark.sql.hive.thriftserver +import java.sql.SQLException + +import org.apache.hive.service.cli.HiveSQLException + trait ThriftServerWithSparkContextSuite extends SharedThriftServer { test("the scratch dir will be deleted during server start but recreated with new operation") { @@ -45,6 +49,36 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { assert(cacheManager.isEmpty) } } + + test("Full stack traces as error message for jdbc or thrift client") { + val sql = "select date_sub(date'2011-11-11', '1.2')" + withCLIServiceClient { client => + val sessionHandle = client.openSession(user, "") + + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] + val e = intercept[HiveSQLException] { + client.executeStatement( + sessionHandle, + sql, + confOverlay) + } + + assert(e.getMessage + .contains("The second argument of 'date_sub' function needs to be an integer.")) + assert(!e.getMessage.contains("" + + "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + } + + withJdbcStatement { statement => + val e = intercept[SQLException] { + statement.executeQuery(sql) + } + assert(e.getMessage + .contains("The second argument of 'date_sub' function needs to be an integer.")) + assert(e.getMessage.contains("" + + "java.lang.NumberFormatException: invalid input syntax for type numeric: 1.2")) + } + } } diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index e46799a1c427..fcf613841722 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -566,7 +566,8 @@ public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) th if (opException != null) { resp.setSqlState(opException.getSQLState()); resp.setErrorCode(opException.getErrorCode()); - resp.setErrorMessage(opException.getMessage()); + resp.setErrorMessage(org.apache.hadoop.util.StringUtils. + stringifyException(opException)); } resp.setStatus(OK_STATUS); } catch (Exception e) { From 7cb0ae8ced60a4a2881267fb3bb6d34e95dabeb6 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 1 Jul 2020 15:20:55 +0800 Subject: [PATCH 2/5] clean --- .../apache/spark/sql/hive/thriftserver/SparkOperation.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index 514e31fa77b8..0acd1b3e9899 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -94,10 +94,6 @@ private[hive] trait SparkOperation extends Operation with Logging { throw new IllegalArgumentException(s"Unknown table type is found: $t") } - def state(newState: OperationState): OperationState = { - super.setState(newState) - } - protected def onError(): PartialFunction[Throwable, Unit] = { case e: Throwable => logError(s"Error executing get catalogs operation with $statementId", e) From 48c2862474c61a0813ccc3d456bd8618e451efad Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 1 Jul 2020 15:35:39 +0800 Subject: [PATCH 3/5] style --- .../spark/sql/hive/thriftserver/SharedThriftServer.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index b52ef06ae4f7..5f1760758552 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -19,17 +19,16 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File import java.sql.{DriverManager, Statement} +import java.util import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.util.Try -import java.util import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.jdbc.HttpBasicAuthInterceptor -import org.apache.hive.jdbc.Utils.JdbcConnectionParams import org.apache.hive.service.auth.PlainSaslHelper import org.apache.hive.service.cli.thrift.{ThriftCLIService, ThriftCLIServiceClient} import org.apache.http.impl.client.HttpClientBuilder From ba0c44ddb558f5b52e23e88f6e6972e5d3192ab4 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 1 Jul 2020 15:50:58 +0800 Subject: [PATCH 4/5] style --- .../org/apache/hive/service/cli/thrift/ThriftCLIService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index fcf613841722..914d6d361259 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -566,8 +566,8 @@ public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) th if (opException != null) { resp.setSqlState(opException.getSQLState()); resp.setErrorCode(opException.getErrorCode()); - resp.setErrorMessage(org.apache.hadoop.util.StringUtils. - stringifyException(opException)); + resp.setErrorMessage(org.apache.hadoop.util.StringUtils + .stringifyException(opException)); } resp.setStatus(OK_STATUS); } catch (Exception e) { From d72074aff830c670685278eb40b08df1237f7c66 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 3 Jul 2020 21:27:27 +0800 Subject: [PATCH 5/5] hive1.2 --- .../org/apache/hive/service/cli/thrift/ThriftCLIService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 783e5795aca7..ff533769b5b8 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -564,7 +564,8 @@ public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) th if (opException != null) { resp.setSqlState(opException.getSQLState()); resp.setErrorCode(opException.getErrorCode()); - resp.setErrorMessage(opException.getMessage()); + resp.setErrorMessage(org.apache.hadoop.util.StringUtils + .stringifyException(opException)); } resp.setStatus(OK_STATUS); } catch (Exception e) {