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

Josh Mahonin commented on PHOENIX-3192:
---------------------------------------

For the most part, the phoenix-spark code tries not to do to much w.r.t. column 
names, as most of it is offloaded to the PhoenixConfigurationUtil class:
https://github.com/apache/phoenix/blob/master/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala#L81

All I can suggest is to try keep any fixes in the utility classes, if possible, 
so that Pig and the other MR integrations can make use of them as well. Unless 
of course the issue is within phoenix-spark itself, then change whatever you 
think is necessary.

> phoenix-spark dataframe issue with combination of "column family + column 
> name"
> -------------------------------------------------------------------------------
>
>                 Key: PHOENIX-3192
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-3192
>             Project: Phoenix
>          Issue Type: Bug
>            Reporter: Kalyan
>            Assignee: Kalyan
>
> 1. create table with different column families with same column name
> create table tbl_1 (rowkey varchar primary key, cf1.c1 varchar, cf1.c2 
> integer, cf2.c1 double, cf2.c2 boolean, cf3.c1 bigint);
> 2. insert sample data into table
> 3. create dataframe using phoenix table with different column names
> val df1 = sqlContext.phoenixTableAsDataFrame("tbl_1", Array("CF1.C1", 
> "CF2.C2"), conf = configuration )
> df1.show     // this will work
> 4. create dataframe using phoenix table with same column names
> val df2 = sqlContext.phoenixTableAsDataFrame("tbl_1", Array("CF1.C1", 
> "CF2.C1"), conf = configuration )
> df2.show    // this will fail
> 5. reason currently we are not handled the dataframe solution fully (column 
> family + column name). 
> only works with (column name)
> Exception:
> scala> val df2 = sqlContext.phoenixTableAsDataFrame("tbl_1", Array("CF1.C1", 
> "CF2.C1"), conf = configuration )
> df2: org.apache.spark.sql.DataFrame = [C1: string, C1: double]
> scala> df2.show
> 16/08/17 22:16:54 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
> scala.MatchError: 1.5 (of class java.lang.Double)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:295)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:294)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:260)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:250)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$$anonfun$createToCatalystConverter$2.apply(CatalystTypeConverters.scala:401)
>       at 
> org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:492)
>       at 
> org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:492)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>       at scala.collection.Iterator$$anon$10.next(Iterator.scala:312)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>       at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>       at 
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
>       at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
>       at scala.collection.AbstractIterator.to(Iterator.scala:1157)
>       at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
>       at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
>       at 
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
>       at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$5.apply(SparkPlan.scala:212)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$5.apply(SparkPlan.scala:212)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1858)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1858)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       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)
> 16/08/17 22:16:54 WARN TaskSetManager: Lost task 0.0 in stage 0.0 (TID 0, 
> localhost): scala.MatchError: 1.5 (of class java.lang.Double)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:295)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:294)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:260)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:250)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$$anonfun$createToCatalystConverter$2.apply(CatalystTypeConverters.scala:401)
>       at 
> org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:492)
>       at 
> org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:492)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>       at scala.collection.Iterator$$anon$10.next(Iterator.scala:312)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>       at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>       at 
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
>       at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
>       at scala.collection.AbstractIterator.to(Iterator.scala:1157)
>       at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
>       at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
>       at 
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
>       at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$5.apply(SparkPlan.scala:212)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$5.apply(SparkPlan.scala:212)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1858)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1858)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       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)
> 16/08/17 22:16:54 ERROR TaskSetManager: Task 0 in stage 0.0 failed 1 times; 
> aborting job
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in 
> stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 
> (TID 0, localhost): scala.MatchError: 1.5 (of class java.lang.Double)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:295)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:294)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:260)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:250)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$$anonfun$createToCatalystConverter$2.apply(CatalystTypeConverters.scala:401)
>       at 
> org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:492)
>       at 
> org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:492)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>       at scala.collection.Iterator$$anon$10.next(Iterator.scala:312)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>       at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>       at 
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
>       at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
>       at scala.collection.AbstractIterator.to(Iterator.scala:1157)
>       at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
>       at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
>       at 
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
>       at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$5.apply(SparkPlan.scala:212)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$5.apply(SparkPlan.scala:212)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1858)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1858)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       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)
> Driver stacktrace:
>       at 
> org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1431)
>       at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1419)
>       at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1418)
>       at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>       at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
>       at 
> org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1418)
>       at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:799)
>       at 
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:799)
>       at scala.Option.foreach(Option.scala:236)
>       at 
> org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:799)
>       at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1640)
>       at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1599)
>       at 
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1588)
>       at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
>       at 
> org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:620)
>       at org.apache.spark.SparkContext.runJob(SparkContext.scala:1832)
>       at org.apache.spark.SparkContext.runJob(SparkContext.scala:1845)
>       at org.apache.spark.SparkContext.runJob(SparkContext.scala:1858)
>       at 
> org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:212)
>       at 
> org.apache.spark.sql.execution.Limit.executeCollect(basicOperators.scala:165)
>       at 
> org.apache.spark.sql.execution.SparkPlan.executeCollectPublic(SparkPlan.scala:174)
>       at 
> org.apache.spark.sql.DataFrame$$anonfun$org$apache$spark$sql$DataFrame$$execute$1$1.apply(DataFrame.scala:1538)
>       at 
> org.apache.spark.sql.DataFrame$$anonfun$org$apache$spark$sql$DataFrame$$execute$1$1.apply(DataFrame.scala:1538)
>       at 
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:56)
>       at 
> org.apache.spark.sql.DataFrame.withNewExecutionId(DataFrame.scala:2125)
>       at 
> org.apache.spark.sql.DataFrame.org$apache$spark$sql$DataFrame$$execute$1(DataFrame.scala:1537)
>       at 
> org.apache.spark.sql.DataFrame.org$apache$spark$sql$DataFrame$$collect(DataFrame.scala:1544)
>       at 
> org.apache.spark.sql.DataFrame$$anonfun$head$1.apply(DataFrame.scala:1414)
>       at 
> org.apache.spark.sql.DataFrame$$anonfun$head$1.apply(DataFrame.scala:1413)
>       at org.apache.spark.sql.DataFrame.withCallback(DataFrame.scala:2138)
>       at org.apache.spark.sql.DataFrame.head(DataFrame.scala:1413)
>       at org.apache.spark.sql.DataFrame.take(DataFrame.scala:1495)
>       at org.apache.spark.sql.DataFrame.showString(DataFrame.scala:171)
>       at org.apache.spark.sql.DataFrame.show(DataFrame.scala:394)
>       at org.apache.spark.sql.DataFrame.show(DataFrame.scala:355)
>       at org.apache.spark.sql.DataFrame.show(DataFrame.scala:363)
>       at 
> $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:38)
>       at 
> $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:43)
>       at 
> $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:45)
>       at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:47)
>       at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:49)
>       at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:51)
>       at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:53)
>       at $iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:55)
>       at $iwC$$iwC$$iwC$$iwC.<init>(<console>:57)
>       at $iwC$$iwC$$iwC.<init>(<console>:59)
>       at $iwC$$iwC.<init>(<console>:61)
>       at $iwC.<init>(<console>:63)
>       at <init>(<console>:65)
>       at .<init>(<console>:69)
>       at .<clinit>(<console>)
>       at .<init>(<console>:7)
>       at .<clinit>(<console>)
>       at $print(<console>)
>       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:497)
>       at 
> org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)
>       at 
> org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1346)
>       at 
> org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)
>       at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)
>       at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)
>       at 
> org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:857)
>       at 
> org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:902)
>       at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:814)
>       at org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:657)
>       at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:665)
>       at 
> org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:670)
>       at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:997)
>       at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
>       at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
>       at 
> scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
>       at 
> org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$process(SparkILoop.scala:945)
>       at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:1059)
>       at org.apache.spark.repl.Main$.main(Main.scala:31)
>       at org.apache.spark.repl.Main.main(Main.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:497)
>       at 
> org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:731)
>       at 
> org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:181)
>       at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:206)
>       at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:121)
>       at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> Caused by: scala.MatchError: 1.5 (of class java.lang.Double)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:295)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:294)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:260)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toCatalystImpl(CatalystTypeConverters.scala:250)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:102)
>       at 
> org.apache.spark.sql.catalyst.CatalystTypeConverters$$anonfun$createToCatalystConverter$2.apply(CatalystTypeConverters.scala:401)
>       at 
> org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:492)
>       at 
> org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:492)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>       at scala.collection.Iterator$$anon$10.next(Iterator.scala:312)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:727)
>       at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>       at 
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
>       at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
>       at scala.collection.AbstractIterator.to(Iterator.scala:1157)
>       at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
>       at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
>       at 
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
>       at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$5.apply(SparkPlan.scala:212)
>       at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$5.apply(SparkPlan.scala:212)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1858)
>       at 
> org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1858)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       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)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to