[ https://issues.apache.org/jira/browse/SPARK-21529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16370100#comment-16370100 ]
Elliot West edited comment on SPARK-21529 at 2/20/18 2:34 PM: -------------------------------------------------------------- Yes [~tomwadeson], but not in a nice way: * We implemented a bunch of UDFs to read union types in Hive (HIVE-15434). * We run a downstream Hive ETL to reformat the data, removing the union type for downstream Spark jobs. * We've prohibited future use of the union type by data producers, because it's so painful, and not supported by most data processing frameworks. was (Author: teabot): Yes [~tomwadeson], but not in a nice way: * We implemented a bunch of UDFs to read unions in Hive (HIVE-15434). * We run a downstream Hive ETL to reformat the data, removing the union type for downstream Spark jobs. * We've prohibited future use of the union type by data producers, because it's so painful, and not supported by most data processing frameworks. > Uniontype not supported when reading from Hive tables. > ------------------------------------------------------ > > Key: SPARK-21529 > URL: https://issues.apache.org/jira/browse/SPARK-21529 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 2.1.0 > Environment: Qubole, DataBricks > Reporter: Elliot West > Priority: Major > Labels: hive, uniontype > > We encounter errors when attempting to read Hive tables whose schema contains > the {{uniontype}}. It appears perhaps that Catalyst > does not support the {{uniontype}} which renders this table unreadable by > Spark (2.1). Although, {{uniontype}} is arguably incomplete in the Hive > query engine, it is fully supported by the storage engine and also the Avro > data format, which we use for these tables. Therefore, I believe it is > a valid, usable type construct that should be supported by Spark. > We've attempted to read the table as follows: > {code} > spark.sql("select * from etl.tbl where acquisition_instant='20170706T133545Z' > limit 5").show > val tblread = spark.read.table("etl.tbl") > {code} > But this always results in the same error message. The pertinent error > messages are as follows (full stack trace below): > {code} > org.apache.spark.SparkException: Cannot recognize hive type string: > uniontype<struct<a:array<uniontype<struct<b: ... > ... > Caused by: org.apache.spark.sql.catalyst.parser.ParseException: > mismatched input '<' expecting > {<EOF>, '('} > (line 1, pos 9) > == SQL == > uniontype<struct<a:array<uniontype<struct<b: ... > ---------^^^ > {code} > h2. Full stack trace > {code} > org.apache.spark.SparkException: Cannot recognize hive type string: > uniontype<struct<a:array<uniontype<struct<b:float,c:float,d:double,e:string,f:string,g:string,h:string,i:string,j:string,k:double,l:double,m:string>>>,n:boolean,o:string,p:bigint,q:string>,struct<r:array<struct<s:string,t:array<uniontype<struct<u:float,v:float,w:double,x:string,y:string,z:string,aa:string,ab:string,ac:string,ad:double,ae:double,af:string>>>>>,ag:boolean,ah:string,ai:bigint,aj:string>> > at > org.apache.spark.sql.hive.client.HiveClientImpl$.fromHiveColumn(HiveClientImpl.scala:800) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11$$anonfun$7.apply(HiveClientImpl.scala:377) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11$$anonfun$7.apply(HiveClientImpl.scala:377) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) > at scala.collection.Iterator$class.foreach(Iterator.scala:893) > at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) > at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) > at scala.collection.AbstractIterable.foreach(Iterable.scala:54) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) > at scala.collection.AbstractTraversable.map(Traversable.scala:104) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11.apply(HiveClientImpl.scala:377) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11.apply(HiveClientImpl.scala:373) > at scala.Option.map(Option.scala:146) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1.apply(HiveClientImpl.scala:373) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1.apply(HiveClientImpl.scala:371) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:290) > at > org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:231) > at > org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:230) > at > org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:273) > at > org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:371) > at > org.apache.spark.sql.hive.client.HiveClient$class.getTable(HiveClient.scala:74) > at > org.apache.spark.sql.hive.client.HiveClientImpl.getTable(HiveClientImpl.scala:79) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable$1.apply(HiveExternalCatalog.scala:118) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable$1.apply(HiveExternalCatalog.scala:118) > at > org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97) > at > org.apache.spark.sql.hive.HiveExternalCatalog.org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable(HiveExternalCatalog.scala:117) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$getTable$1.apply(HiveExternalCatalog.scala:648) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$getTable$1.apply(HiveExternalCatalog.scala:648) > at > org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97) > at > org.apache.spark.sql.hive.HiveExternalCatalog.getTable(HiveExternalCatalog.scala:647) > at > org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupRelation(SessionCatalog.scala:681) > at org.apache.spark.sql.SparkSession.table(SparkSession.scala:622) > at org.apache.spark.sql.SparkSession.table(SparkSession.scala:618) > at org.apache.spark.sql.DataFrameReader.table(DataFrameReader.scala:627) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:43) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:50) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:52) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:54) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:56) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:58) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw.<init>(<console>:60) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw.<init>(<console>:62) > at linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw.<init>(<console>:64) > at linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw.<init>(<console>:66) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$eval$.$print$lzycompute(<console>:7) > at linef5f6809a5e21434ea50b8ad706eb0e8e27.$eval$.$print(<console>:6) > Caused by: org.apache.spark.sql.catalyst.parser.ParseException: > mismatched input '<' expecting > {<EOF>, '('} > (line 1, pos 9) > == SQL == > uniontype<struct<a:array<uniontype<struct<b:float,c:float,d:double,e:string,f:string,g:string,h:string,i:string,j:string,k:double,l:double,m:string>>>,n:boolean,o:string,p:bigint,q:string>,struct<r:array<struct<s:string,t:array<uniontype<struct<u:float,v:float,w:double,x:string,y:string,z:string,aa:string,ab:string,ac:string,ad:double,ae:double,af:string>>>>>,ag:boolean,ah:string,ai:bigint,aj:string>> > ---------^^^ > at > org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:217) > at > org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:114) > at > org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parseDataType(ParseDriver.scala:38) > at > org.apache.spark.sql.hive.client.HiveClientImpl$.fromHiveColumn(HiveClientImpl.scala:797) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11$$anonfun$7.apply(HiveClientImpl.scala:377) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11$$anonfun$7.apply(HiveClientImpl.scala:377) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) > at scala.collection.Iterator$class.foreach(Iterator.scala:893) > at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) > at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) > at scala.collection.AbstractIterable.foreach(Iterable.scala:54) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) > at scala.collection.AbstractTraversable.map(Traversable.scala:104) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11.apply(HiveClientImpl.scala:377) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$11.apply(HiveClientImpl.scala:373) > at scala.Option.map(Option.scala:146) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1.apply(HiveClientImpl.scala:373) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1.apply(HiveClientImpl.scala:371) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:290) > at > org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:231) > at > org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:230) > at > org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:273) > at > org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:371) > at > org.apache.spark.sql.hive.client.HiveClient$class.getTable(HiveClient.scala:74) > at > org.apache.spark.sql.hive.client.HiveClientImpl.getTable(HiveClientImpl.scala:79) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable$1.apply(HiveExternalCatalog.scala:118) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable$1.apply(HiveExternalCatalog.scala:118) > at > org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97) > at > org.apache.spark.sql.hive.HiveExternalCatalog.org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable(HiveExternalCatalog.scala:117) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$getTable$1.apply(HiveExternalCatalog.scala:648) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$getTable$1.apply(HiveExternalCatalog.scala:648) > at > org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97) > at > org.apache.spark.sql.hive.HiveExternalCatalog.getTable(HiveExternalCatalog.scala:647) > at > org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupRelation(SessionCatalog.scala:681) > at org.apache.spark.sql.SparkSession.table(SparkSession.scala:622) > at org.apache.spark.sql.SparkSession.table(SparkSession.scala:618) > at org.apache.spark.sql.DataFrameReader.table(DataFrameReader.scala:627) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:43) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:50) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:52) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:54) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:56) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:58) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw$$iw.<init>(<console>:60) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw$$iw.<init>(<console>:62) > at linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw$$iw.<init>(<console>:64) > at linef5f6809a5e21434ea50b8ad706eb0e8e27.$read$$iw.<init>(<console>:66) > at > linef5f6809a5e21434ea50b8ad706eb0e8e27.$eval$.$print$lzycompute(<console>:7) > at linef5f6809a5e21434ea50b8ad706eb0e8e27.$eval$.$print(<console>:6) > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org