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

Deenar Toraskar commented on SPARK-15704:
-----------------------------------------

Hi guys

I get a similar error when using complex types in Aggregator. Not sure if this 
is the same issue or something else.

{code:title=Agg.scala|borderStyle=solid}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.TypedColumn
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql.{Encoder,Row}
import sqlContext.implicits._

object CustomSummer extends Aggregator[Valuation, Map[Int, Seq[Double]], 
Seq[Seq[Double]]] with Serializable  {
     def zero: Map[Int, Seq[Double]] = Map()
     def reduce(b: Map[Int, Seq[Double]], a:Valuation): Map[Int, Seq[Double]] = 
{
       val timeInterval: Int = a.timeInterval
       val currentSum: Seq[Double] = b.get(timeInterval).getOrElse(Nil)
       val currentRow: Seq[Double] = a.pvs
       b.updated(timeInterval, sumArray(currentSum, currentRow))
     }     
    def sumArray(a: Seq[Double], b: Seq[Double]): Seq[Double] = Nil
     def merge(b1: Map[Int, Seq[Double]], b2: Map[Int, Seq[Double]]): Map[Int, 
Seq[Double]] = {
        /* merges two maps together ++ replaces any (k,v) from the map on the 
left
        side of ++ (here map1) by (k,v) from the right side map, if (k,_) 
already
        exists in the left side map (here map1), e.g. Map(1->1) ++ Map(1->2) 
results in Map(1->2) */
        b1 ++ b2.map { case (timeInterval, exposures) =>
          timeInterval -> sumArray(exposures, b1.getOrElse(timeInterval, Nil))
        }
     }
     def finish(exposures: Map[Int, Seq[Double]]): Seq[Seq[Double]] = 
      {
        exposures.size match {
          case 0 => null
          case _ => {
            val range = exposures.keySet.max
            // convert map to 2 dimensional array, (timeInterval x Seq[expScn1, 
expScn2, ...]
            (0 to range).map(x => exposures.getOrElse(x, Nil))
          }
        }
      }
  override def bufferEncoder: Encoder[Map[Int,Seq[Double]]] = 
ExpressionEncoder()
  override def outputEncoder: Encoder[Seq[Seq[Double]]] = ExpressionEncoder()
   }

case class Valuation(timeInterval : Int, pvs : Seq[Double])
val valns = sc.parallelize(Seq(Valuation(0, Seq(1.0,2.0,3.0)),
  Valuation(2, Seq(1.0,2.0,3.0)),
  Valuation(1, Seq(1.0,2.0,3.0)),Valuation(2, Seq(1.0,2.0,3.0)),Valuation(0, 
Seq(1.0,2.0,3.0))
  )).toDS

val g_c1 = 
valns.groupByKey(_.timeInterval).agg(CustomSummer.toColumn).show(false)
{code}

I get the following error

{quote}
org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in 
stage 10.0 failed 1 times, most recent failure: Lost task 1.0 in stage 10.0 
(TID 19, localhost): java.lang.IndexOutOfBoundsException: 0
at scala.collection.mutable.ResizableArray$class.apply(ResizableArray.scala:43)
        at scala.collection.mutable.ArrayBuffer.apply(ArrayBuffer.scala:47)
        at scala.collection.mutable.ArrayBuffer.remove(ArrayBuffer.scala:167)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:244)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.withNewChildren(TreeNode.scala:214)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:156)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
        at scala.collection.AbstractTraversable.map(Traversable.scala:105)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
        at scala.collection.AbstractTraversable.map(Traversable.scala:105)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
        at scala.collection.AbstractTraversable.map(Traversable.scala:105)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
        at scala.collection.AbstractTraversable.map(Traversable.scala:105)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.semanticHash(Expression.scala:174)
        at 
org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$Expr.hashCode(EquivalentExpressions.scala:39)
        at scala.runtime.ScalaRunTime$.hash(ScalaRunTime.scala:210)
        at 
scala.collection.mutable.HashTable$HashUtils$class.elemHashCode(HashTable.scala:398)
        at scala.collection.mutable.HashMap.elemHashCode(HashMap.scala:39)
        at 
scala.collection.mutable.HashTable$class.findEntry(HashTable.scala:130)
        at scala.collection.mutable.HashMap.findEntry(HashMap.scala:39)
        at scala.collection.mutable.HashMap.get(HashMap.scala:69)
        at 
org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExpr(EquivalentExpressions.scala:53)
        at 
org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:86)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:661)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:661)
        at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.subexpressionElimination(CodeGenerator.scala:661)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.generateExpressions(CodeGenerator.scala:718)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.create(GenerateMutableProjection.scala:59)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.generate(GenerateMutableProjection.scala:44)
        at 
org.apache.spark.sql.execution.SparkPlan.newMutableProjection(SparkPlan.scala:369)
        at 
org.apache.spark.sql.execution.aggregate.SortAggregateExec$$anonfun$doExecute$1$$anonfun$3$$anonfun$4.apply(SortAggregateExec.scala:93)
        at 
org.apache.spark.sql.execution.aggregate.SortAggregateExec$$anonfun$doExecute$1$$anonfun$3$$anonfun$4.apply(SortAggregateExec.scala:92)
        at 
org.apache.spark.sql.execution.aggregate.AggregationIterator.generateProcessRow(AggregationIterator.scala:178)
        at 
org.apache.spark.sql.execution.aggregate.AggregationIterator.<init>(AggregationIterator.scala:197)
        at 
org.apache.spark.sql.execution.aggregate.SortBasedAggregationIterator.<init>(SortBasedAggregationIterator.scala:29)
        at 
org.apache.spark.sql.execution.aggregate.SortAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortAggregateExec.scala:84)
        at 
org.apache.spark.sql.execution.aggregate.SortAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortAggregateExec.scala:75)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:780)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:780)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47)
        at org.apache.spark.scheduler.Task.run(Task.scala:85)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
        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:1450)
        at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1438)
        at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1437)
        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:1437)
        at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
        at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)
        at scala.Option.foreach(Option.scala:236)
        at 
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:811)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1659)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1618)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1607)
        at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
        at 
org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:632)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:1872)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:1885)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:1898)
        at 
org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:347)
        at 
org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:39)
        at 
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2176)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57)
        at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2525)
        at 
org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2175)
        at 
org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:2182)
        at 
org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1918)
        at 
org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1917)
        at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2555)
        at org.apache.spark.sql.Dataset.head(Dataset.scala:1917)
        at org.apache.spark.sql.Dataset.take(Dataset.scala:2132)
        at org.apache.spark.sql.Dataset.showString(Dataset.scala:239)
        at org.apache.spark.sql.Dataset.show(Dataset.scala:526)
        at org.apache.spark.sql.Dataset.show(Dataset.scala:506)
Caused by: java.lang.IndexOutOfBoundsException: 0
        at 
scala.collection.mutable.ResizableArray$class.apply(ResizableArray.scala:43)
        at scala.collection.mutable.ArrayBuffer.apply(ArrayBuffer.scala:47)
        at scala.collection.mutable.ArrayBuffer.remove(ArrayBuffer.scala:167)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:244)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179)
        at 
org.apache.spark.sql.catalyst.trees.TreeNode.withNewChildren(TreeNode.scala:214)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:156)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
        at scala.collection.AbstractTraversable.map(Traversable.scala:105)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
        at scala.collection.AbstractTraversable.map(Traversable.scala:105)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
        at scala.collection.AbstractTraversable.map(Traversable.scala:105)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:155)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
        at scala.collection.AbstractTraversable.map(Traversable.scala:105)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:155)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:154)
        at 
org.apache.spark.sql.catalyst.expressions.Expression.semanticHash(Expression.scala:174)
        at 
org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$Expr.hashCode(EquivalentExpressions.scala:39)
        at scala.runtime.ScalaRunTime$.hash(ScalaRunTime.scala:210)
        at 
scala.collection.mutable.HashTable$HashUtils$class.elemHashCode(HashTable.scala:398)
        at scala.collection.mutable.HashMap.elemHashCode(HashMap.scala:39)
        at 
scala.collection.mutable.HashTable$class.findEntry(HashTable.scala:130)
        at scala.collection.mutable.HashMap.findEntry(HashMap.scala:39)
        at scala.collection.mutable.HashMap.get(HashMap.scala:69)
        at 
org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExpr(EquivalentExpressions.scala:53)
        at 
org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:86)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:661)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:661)
        at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.subexpressionElimination(CodeGenerator.scala:661)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.generateExpressions(CodeGenerator.scala:718)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.create(GenerateMutableProjection.scala:59)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.generate(GenerateMutableProjection.scala:44)
        at 
org.apache.spark.sql.execution.SparkPlan.newMutableProjection(SparkPlan.scala:369)
        at 
org.apache.spark.sql.execution.aggregate.SortAggregateExec$$anonfun$doExecute$1$$anonfun$3$$anonfun$4.apply(SortAggregateExec.scala:93)
        at 
org.apache.spark.sql.execution.aggregate.SortAggregateExec$$anonfun$doExecute$1$$anonfun$3$$anonfun$4.apply(SortAggregateExec.scala:92)
        at 
org.apache.spark.sql.execution.aggregate.AggregationIterator.generateProcessRow(AggregationIterator.scala:178)
        at 
org.apache.spark.sql.execution.aggregate.AggregationIterator.<init>(AggregationIterator.scala:197)
        at 
org.apache.spark.sql.execution.aggregate.SortBasedAggregationIterator.<init>(SortBasedAggregationIterator.scala:29)
        at 
org.apache.spark.sql.execution.aggregate.SortAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortAggregateExec.scala:84)
        at 
org.apache.spark.sql.execution.aggregate.SortAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortAggregateExec.scala:75)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:780)
        at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:780)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47)
        at org.apache.spark.scheduler.Task.run(Task.scala:85)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
        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)
{quote}

> TungstenAggregate crashes 
> --------------------------
>
>                 Key: SPARK-15704
>                 URL: https://issues.apache.org/jira/browse/SPARK-15704
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.0.0
>            Reporter: Hiroshi Inoue
>            Assignee: Hiroshi Inoue
>             Fix For: 2.0.0
>
>
> When I run DatasetBenchmark, the JVM crashes while executing "Dataset complex 
> Aggregator" test case due to IndexOutOfBoundsException.
> The error happens in TungstenAggregate; the mappings between bufferSerializer 
> and bufferDeserializer are broken due to unresolved attribute.
> {quote}
> 16/06/02 01:41:05 ERROR Executor: Exception in task 0.0 in stage 67.0 (TID 
> 232)
> java.lang.IndexOutOfBoundsException: -1
>       at 
> scala.collection.LinearSeqOptimized$class.apply(LinearSeqOptimized.scala:65)
>       at scala.collection.immutable.List.apply(List.scala:84)
>       at 
> org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate$RichAttribute.right(interfaces.scala:389)
>       at 
> org.apache.spark.sql.execution.aggregate.TypedAggregateExpression$$anonfun$3.applyOrElse(TypedAggregateExpression.scala:110)
>       at 
> org.apache.spark.sql.execution.aggregate.TypedAggregateExpression$$anonfun$3.applyOrElse(TypedAggregateExpression.scala:109)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:265)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:265)
>       at 
> org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:68)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:264)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:270)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:270)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:307)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:893)
>       at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
>       at 
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
>       at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
>       at scala.collection.AbstractIterator.to(Iterator.scala:1336)
>       at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
>       at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1336)
>       at 
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
>       at scala.collection.AbstractIterator.toArray(Iterator.scala:1336)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:356)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:270)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:270)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:270)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5$$anonfun$apply$11.apply(TreeNode.scala:336)
>       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.immutable.List.foreach(List.scala:381)
>       at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
>       at scala.collection.immutable.List.map(List.scala:285)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:334)
>       at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:893)
>       at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
>       at 
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
>       at 
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
>       at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
>       at scala.collection.AbstractIterator.to(Iterator.scala:1336)
>       at 
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
>       at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1336)
>       at 
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
>       at scala.collection.AbstractIterator.toArray(Iterator.scala:1336)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:356)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:270)
>       at 
> org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:254)
>       at 
> org.apache.spark.sql.execution.aggregate.TypedAggregateExpression.mergeExpressions$lzycompute(TypedAggregateExpression.scala:109)
>       at 
> org.apache.spark.sql.execution.aggregate.TypedAggregateExpression.mergeExpressions(TypedAggregateExpression.scala:105)
>       at 
> org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$7.apply(AggregationIterator.scala:162)
>       at 
> org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$7.apply(AggregationIterator.scala:158)
>       at 
> scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
>       at 
> scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
>       at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>       at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
>       at 
> scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
>       at scala.collection.AbstractTraversable.flatMap(Traversable.scala:104)
>       at 
> org.apache.spark.sql.execution.aggregate.AggregationIterator.generateProcessRow(AggregationIterator.scala:158)
>       at 
> org.apache.spark.sql.execution.aggregate.AggregationIterator.<init>(AggregationIterator.scala:197)
>       at 
> org.apache.spark.sql.execution.aggregate.SortBasedAggregationIterator.<init>(SortBasedAggregationIterator.scala:39)
>       at 
> org.apache.spark.sql.execution.aggregate.SortBasedAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortBasedAggregateExec.scala:80)
>       at 
> org.apache.spark.sql.execution.aggregate.SortBasedAggregateExec$$anonfun$doExecute$1$$anonfun$3.apply(SortBasedAggregateExec.scala:71)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:775)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:775)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:282)
>       at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
>       at org.apache.spark.scheduler.Task.run(Task.scala:85)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
>       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)
> {quote}



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

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

Reply via email to