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

Chen Gong commented on SPARK-20851:
-----------------------------------

[~maropu] Thanks for you reply. 

This scenario happened in our own cluster, however, when I tried this scenario 
on my own laptop in which spark is running in standalone mode with one node, no 
this issue at all. So I just list down simplest commands I used in our own 
cluster to reproduce the error, as well as the error log.

*Create a sample dataset*
{noformat}
echo '{"col1":"value", "col2":{"123121":10, "123018231":"rejlsoe"}}' > 
file.json    // A simple example of data
hadoop fs -put file.json <hdfs path>
{noformat}

*Process dataset in pyspark*
{noformat}
df = spark.read.json(<hdfs path>)
df.printSchema()

root
 |-- col1: string (nullable = true)
 |-- col2: struct (nullable = true)
 |    |-- 123018231: string (nullable = true)
 |    |-- 123121: long (nullable = true)

df.createOrReplaceTempView('df')
spark.sql('create table test_tab like df')   // no error at all
{noformat}

Then I could not do any operations on this test_tab
{noformat}
spark.sql('select * from test_tab')
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/home/shopee/spark/python/pyspark/sql/session.py", line 541, in sql
    return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
  File 
"/home/shopee/spark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py", line 
1133, in __call__
  File "/home/shopee/spark/python/pyspark/sql/utils.py", line 63, in deco
    return f(*a, **kw)
  File "/home/shopee/spark/python/lib/py4j-0.10.4-src.zip/py4j/protocol.py", 
line 319, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o45.sql.
: org.apache.spark.SparkException: Cannot recognize hive type string: 
struct<123018231:string,123121:bigint>
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.org$apache$spark$sql$hive$client$HiveClientImpl$$fromHiveColumn(HiveClientImpl.scala:785)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$10$$anonfun$7.apply(HiveClientImpl.scala:365)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$10$$anonfun$7.apply(HiveClientImpl.scala:365)
        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$10.apply(HiveClientImpl.scala:365)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$10.apply(HiveClientImpl.scala:361)
        at scala.Option.map(Option.scala:146)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1.apply(HiveClientImpl.scala:361)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1.apply(HiveClientImpl.scala:359)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:283)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:230)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:229)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:272)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:359)
        at 
org.apache.spark.sql.hive.client.HiveClient$class.getTable(HiveClient.scala:76)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.getTable(HiveClientImpl.scala:78)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable$1.apply(HiveExternalCatalog.scala:110)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable$1.apply(HiveExternalCatalog.scala:110)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:95)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog.org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable(HiveExternalCatalog.scala:109)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$getTable$1.apply(HiveExternalCatalog.scala:601)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$getTable$1.apply(HiveExternalCatalog.scala:601)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:95)
        at 
org.apache.spark.sql.hive.HiveExternalCatalog.getTable(HiveExternalCatalog.scala:600)
        at 
org.apache.spark.sql.hive.HiveMetastoreCatalog.lookupRelation(HiveMetastoreCatalog.scala:106)
        at 
org.apache.spark.sql.hive.HiveSessionCatalog.lookupRelation(HiveSessionCatalog.scala:69)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveRelations$$lookupTableFromCatalog(Analyzer.scala:456)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$8.applyOrElse(Analyzer.scala:478)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$8.applyOrElse(Analyzer.scala:463)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61)
        at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:60)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$1.apply(LogicalPlan.scala:58)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$1.apply(LogicalPlan.scala:58)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:331)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:188)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:329)
        at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:58)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:463)
        at 
org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:453)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:85)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:82)
        at 
scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124)
        at scala.collection.immutable.List.foldLeft(List.scala:84)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:82)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:74)
        at scala.collection.immutable.List.foreach(List.scala:381)
        at 
org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:74)
        at 
org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:64)
        at 
org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:62)
        at 
org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:48)
        at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:63)
        at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:592)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
        at py4j.Gateway.invoke(Gateway.java:280)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at py4j.GatewayConnection.run(GatewayConnection.java:214)
        at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.spark.sql.catalyst.parser.ParseException: 
mismatched input '123018231' expecting '>'(line 1, pos 7)

== SQL ==
struct<123018231:string,123121:bigint>
-------^^^

        at 
org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:197)
        at 
org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:99)
        at 
org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parseDataType(ParseDriver.scala:37)
        at 
org.apache.spark.sql.hive.client.HiveClientImpl.org$apache$spark$sql$hive$client$HiveClientImpl$$fromHiveColumn(HiveClientImpl.scala:782)
        ... 71 more

{noformat}

Thank you again for any clue.

> Drop spark table failed if a column name is a numeric string
> ------------------------------------------------------------
>
>                 Key: SPARK-20851
>                 URL: https://issues.apache.org/jira/browse/SPARK-20851
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.1.0
>         Environment: linux redhat
>            Reporter: Chen Gong
>
> I tried to read a json file to a spark dataframe
> {noformat}
> df = spark.read.json('path.json')
> df.write.parquet('dataframe', compression='snappy')
> {noformat}
> However, there are some columns' names are numeric strings, such as 
> "989238883". Then I created spark sql table by using this
> {noformat}
> create table if not exists `a` using org.apache.spark.sql.parquet options 
> (path 'dataframe');  // It works well
> {noformat}
> But after created table, any operations, like select, drop table on this 
> table will raise the same exceptions below
> {noformat}
> org.apache.spark.SparkException: Cannot recognize hive type string: 
> array<struct<X:string,added_tags:array<string>,assignee_id:bigint,attachments:array<struct<content_type:string,content_url:string,file_name:string,height:bigint,id:bigint,inline:boolean,mapped_content_url:string,size:bigint,thumbnails:array<struct<content_type:string,content_url:string,file_name:string,height:bigint,id:bigint,inline:boolean,mapped_content_url:string,size:bigint,url:string,width:bigint>>,url:string,width:bigint>>,audit_id:bigint,author_id:bigint,body:string,brand_id:string,created_at:string,custom_ticket_fields:struct<49244727:string,51588527:string,51591767:string,51950848:string,51950868:string,51950888:string,51950928:string,52359587:string,55276747:string,56958227:string,57080067:string,57080667:string,57107727:string,57112447:string,57113207:string,57411128:string,57424648:string,57442588:string,62382188:string,74862088:string,74871788:string>,event_type:string,group_id:bigint,html_body:string,id:bigint,is_public:string,locale_id:string,organization_id:string,plain_body:string,previous_value:string,priority:string,public:boolean,rel:string,removed_tags:array<string>,requester_id:bigint,satisfaction_probability:string,satisfaction_score:string,sla_policy:string,status:string,tags:array<string>,ticket_form_id:string,type:string,via:string,via_reference_id:bigint>>
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl.org$apache$spark$sql$hive$client$HiveClientImpl$$fromHiveColumn(HiveClientImpl.scala:785)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$10$$anonfun$7.apply(HiveClientImpl.scala:365)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$10$$anonfun$7.apply(HiveClientImpl.scala:365)
>       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$10.apply(HiveClientImpl.scala:365)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$10.apply(HiveClientImpl.scala:361)
>       at scala.Option.map(Option.scala:146)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1.apply(HiveClientImpl.scala:361)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1.apply(HiveClientImpl.scala:359)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:283)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:230)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:229)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:272)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:359)
>       at 
> org.apache.spark.sql.hive.client.HiveClient$class.getTable(HiveClient.scala:76)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl.getTable(HiveClientImpl.scala:78)
>       at 
> org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable$1.apply(HiveExternalCatalog.scala:110)
>       at 
> org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable$1.apply(HiveExternalCatalog.scala:110)
>       at 
> org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:95)
>       at 
> org.apache.spark.sql.hive.HiveExternalCatalog.org$apache$spark$sql$hive$HiveExternalCatalog$$getRawTable(HiveExternalCatalog.scala:109)
>       at 
> org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$getTable$1.apply(HiveExternalCatalog.scala:601)
>       at 
> org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$getTable$1.apply(HiveExternalCatalog.scala:601)
>       at 
> org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:95)
>       at 
> org.apache.spark.sql.hive.HiveExternalCatalog.getTable(HiveExternalCatalog.scala:600)
>       at 
> org.apache.spark.sql.catalyst.catalog.SessionCatalog.getTableMetadata(SessionCatalog.scala:290)
>       at 
> org.apache.spark.sql.execution.command.DropTableCommand.run(ddl.scala:191)
>       at 
> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58)
>       at 
> org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56)
>       at 
> org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135)
>       at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>       at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132)
>       at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113)
>       at 
> org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:87)
>       at 
> org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:87)
>       at org.apache.spark.sql.Dataset.<init>(Dataset.scala:185)
>       at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:64)
>       at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:592)
>       at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:699)
>       at 
> org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:62)
>       at 
> org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:335)
>       at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376)
>       at 
> org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:247)
>       at 
> org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:738)
>       at 
> org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:187)
>       at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:212)
>       at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:126)
>       at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> Caused by: org.apache.spark.sql.catalyst.parser.ParseException: 
> mismatched input '49244727' expecting '>'(line 1, pos 540)
> == SQL ==
> array<struct<X:string,added_tags:array<string>,assignee_id:bigint,attachments:array<struct<content_type:string,content_url:string,file_name:string,height:bigint,id:bigint,inline:boolean,mapped_content_url:string,size:bigint,thumbnails:array<struct<content_type:string,content_url:string,file_name:string,height:bigint,id:bigint,inline:boolean,mapped_content_url:string,size:bigint,url:string,width:bigint>>,url:string,width:bigint>>,audit_id:bigint,author_id:bigint,body:string,brand_id:string,created_at:string,custom_ticket_fields:struct<49244727:string,51588527:string,51591767:string,51950848:string,51950868:string,51950888:string,51950928:string,52359587:string,55276747:string,56958227:string,57080067:string,57080667:string,57107727:string,57112447:string,57113207:string,57411128:string,57424648:string,57442588:string,62382188:string,74862088:string,74871788:string>,event_type:string,group_id:bigint,html_body:string,id:bigint,is_public:string,locale_id:string,organization_id:string,plain_body:string,previous_value:string,priority:string,public:boolean,rel:string,removed_tags:array<string>,requester_id:bigint,satisfaction_probability:string,satisfaction_score:string,sla_policy:string,status:string,tags:array<string>,ticket_form_id:string,type:string,via:string,via_reference_id:bigint>>
> ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------^^^
>       at 
> org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:197)
>       at 
> org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:99)
>       at 
> org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parseDataType(ParseDriver.scala:37)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl.org$apache$spark$sql$hive$client$HiveClientImpl$$fromHiveColumn(HiveClientImpl.scala:782)
>       ... 61 more
> org.apache.spark.SparkException: Cannot recognize hive type string: 
> array<struct<X:string,added_tags:array<string>,assignee_id:bigint,attachments:array<struct<content_type:string,content_url:string,file_name:string,height:bigint,id:bigint,inline:boolean,mapped_content_url:string,size:bigint,thumbnails:array<struct<content_type:string,content_url:string,file_name:string,height:bigint,id:bigint,inline:boolean,mapped_content_url:string,size:bigint,url:string,width:bigint>>,url:string,width:bigint>>,audit_id:bigint,author_id:bigint,body:string,brand_id:string,created_at:string,custom_ticket_fields:struct<49244727:string,51588527:string,51591767:string,51950848:string,51950868:string,51950888:string,51950928:string,52359587:string,55276747:string,56958227:string,57080067:string,57080667:string,57107727:string,57112447:string,57113207:string,57411128:string,57424648:string,57442588:string,62382188:string,74862088:string,74871788:string>,event_type:string,group_id:bigint,html_body:string,id:bigint,is_public:string,locale_id:string,organization_id:string,plain_body:string,previous_value:string,priority:string,public:boolean,rel:string,removed_tags:array<string>,requester_id:bigint,satisfaction_probability:string,satisfaction_score:string,sla_policy:string,status:string,tags:array<string>,ticket_form_id:string,type:string,via:string,via_reference_id:bigint>>
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl.org$apache$spark$sql$hive$client$HiveClientImpl$$fromHiveColumn(HiveClientImpl.scala:785)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$10$$anonfun$7.apply(HiveClientImpl.scala:365)
>       at 
> org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getTableOption$1$$anonfun$apply$10$$anonfun$7.apply(HiveClientImpl.scala:365)
>       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)
> {noformat}
> Everything was fine after I deleted these columns in json file and redo all 
> steps above. Any suggestion about this problem, appreciated.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

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

Reply via email to