You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Ken Tore Tallakstad (JIRA)" <ji...@apache.org> on 2018/01/10 13:16:00 UTC

[jira] [Commented] (SPARK-21396) Spark Hive Thriftserver doesn't return UDT field

    [ https://issues.apache.org/jira/browse/SPARK-21396?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16320201#comment-16320201 ] 

Ken Tore Tallakstad commented on SPARK-21396:
---------------------------------------------

Does the bug originate from this function inside
[https://github.com/apache/spark/blob/master/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala] ?

```
def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) {
    dataTypes(ordinal) match {
      case StringType =>
        to += from.getString(ordinal)
      case IntegerType =>
        to += from.getInt(ordinal)
      case BooleanType =>
        to += from.getBoolean(ordinal)
      case DoubleType =>
        to += from.getDouble(ordinal)
      case FloatType =>
        to += from.getFloat(ordinal)
      case DecimalType() =>
        to += from.getDecimal(ordinal)
      case LongType =>
        to += from.getLong(ordinal)
      case ByteType =>
        to += from.getByte(ordinal)
      case ShortType =>
        to += from.getShort(ordinal)
      case DateType =>
        to += from.getAs[Date](ordinal)
      case TimestampType =>
        to += from.getAs[Timestamp](ordinal)
      case BinaryType =>
        to += from.getAs[Array[Byte]](ordinal)
      case _: ArrayType | _: StructType | _: MapType =>
        val hiveString = HiveUtils.toHiveString((from.get(ordinal), dataTypes(ordinal)))
        to += hiveString
    }
  }
```

A UDT will not match any of these right??

> Spark Hive Thriftserver doesn't return UDT field
> ------------------------------------------------
>
>                 Key: SPARK-21396
>                 URL: https://issues.apache.org/jira/browse/SPARK-21396
>             Project: Spark
>          Issue Type: Improvement
>          Components: SQL
>    Affects Versions: 2.1.1
>            Reporter: Haopu Wang
>              Labels: Hive, ThriftServer2, user-defined-type
>
> I want to query a table with a MLLib Vector field and get below exception.
> Can Spark Hive Thriftserver be enhanced to return UDT field?
> ======
> 2017-07-13 13:14:25,435 WARN  [org.apache.hive.service.cli.thrift.ThriftCLIService] (HiveServer2-Handler-Pool: Thread-18537;) Error fetching results: 
> java.lang.RuntimeException: scala.MatchError: org.apache.spark.ml.linalg.VectorUDT@3bfc3ba7 (of class org.apache.spark.ml.linalg.VectorUDT)
> 	at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:83)
> 	at org.apache.hive.service.cli.session.HiveSessionProxy.access$000(HiveSessionProxy.java:36)
> 	at org.apache.hive.service.cli.session.HiveSessionProxy$1.run(HiveSessionProxy.java:63)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:422)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1556)
> 	at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:59)
> 	at com.sun.proxy.$Proxy29.fetchResults(Unknown Source)
> 	at org.apache.hive.service.cli.CLIService.fetchResults(CLIService.java:454)
> 	at org.apache.hive.service.cli.thrift.ThriftCLIService.FetchResults(ThriftCLIService.java:621)
> 	at org.apache.hive.service.cli.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1553)
> 	at org.apache.hive.service.cli.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1538)
> 	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
> 	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
> 	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:56)
> 	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: scala.MatchError: org.apache.spark.ml.linalg.VectorUDT@3bfc3ba7 (of class org.apache.spark.ml.linalg.VectorUDT)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.addNonNullColumnValue(SparkExecuteStatementOperation.scala:80)
> 	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.getNextRowSet(SparkExecuteStatementOperation.scala:144)
> 	at org.apache.hive.service.cli.operation.OperationManager.getOperationNextRowSet(OperationManager.java:220)
> 	at org.apache.hive.service.cli.session.HiveSessionImpl.fetchResults(HiveSessionImpl.java:685)
> 	at sun.reflect.GeneratedMethodAccessor54.invoke(Unknown Source)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:498)
> 	at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:78)
> 	... 18 more



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org