Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-3803

ArrayIndexOutOfBoundsException found in executing computePrincipalComponents

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.1.0
    • 1.2.0
    • MLlib
    • None

    Description

      When I executed computePrincipalComponents method of RowMatrix, I got java.lang.ArrayIndexOutOfBoundsException.

      14/10/05 20:16:31 INFO DAGScheduler: Failed to run reduce at RDDFunctions.scala:111
      org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 31.0 failed 1 times, most recent failure: Lost task 0.0 in stage 31.0 (TID 611, localhost): java.lang.ArrayIndexOutOfBoundsException: 4878161
              org.apache.spark.mllib.linalg.distributed.RowMatrix$.org$apache$spark$mllib$linalg$distributed$RowMatrix$$dspr(RowMatrix.scala:460)
              org.apache.spark.mllib.linalg.distributed.RowMatrix$$anonfun$3.apply(RowMatrix.scala:114)
              org.apache.spark.mllib.linalg.distributed.RowMatrix$$anonfun$3.apply(RowMatrix.scala:113)
              scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:144)
              scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:144)
              scala.collection.Iterator$class.foreach(Iterator.scala:727)
              scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
              scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:144)
              scala.collection.AbstractIterator.foldLeft(Iterator.scala:1157)
              scala.collection.TraversableOnce$class.aggregate(TraversableOnce.scala:201)
              scala.collection.AbstractIterator.aggregate(Iterator.scala:1157)
              org.apache.spark.mllib.rdd.RDDFunctions$$anonfun$4.apply(RDDFunctions.scala:99)
              org.apache.spark.mllib.rdd.RDDFunctions$$anonfun$4.apply(RDDFunctions.scala:99)
              org.apache.spark.mllib.rdd.RDDFunctions$$anonfun$5.apply(RDDFunctions.scala:100)
              org.apache.spark.mllib.rdd.RDDFunctions$$anonfun$5.apply(RDDFunctions.scala:100)
              org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:596)
              org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:596)
              org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
              org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262)
              org.apache.spark.rdd.RDD.iterator(RDD.scala:229)
              org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
              org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262)
              org.apache.spark.rdd.RDD.iterator(RDD.scala:229)
              org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68)
              org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
              org.apache.spark.scheduler.Task.run(Task.scala:54)
              org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:177)
              java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
              java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
              java.lang.Thread.run(Thread.java:745)
      

      The RowMatrix instance was generated from the result of TF-IDF like the following.

      scala> val hashingTF = new HashingTF()
      scala> val tf = hashingTF.transform(texts)
      scala> import org.apache.spark.mllib.feature.IDF
      scala> tf.cache()
      scala> val idf = new IDF().fit(tf)
      scala> val tfidf: RDD[Vector] = idf.transform(tf)
      
      scala> import org.apache.spark.mllib.linalg.distributed.RowMatrix
      scala> val mat = new RowMatrix(tfidf)
      scala> val pc = mat.computePrincipalComponents(2)
      

      I think this was because I created HashingTF instance with default numFeatures and Array is used in RowMatrix#computeGramianMatrix method
      like the following.

        /**
         * Computes the Gramian matrix `A^T A`.
         */
        def computeGramianMatrix(): Matrix = {
          val n = numCols().toInt
          val nt: Int = n * (n + 1) / 2
      
          // Compute the upper triangular part of the gram matrix.
          val GU = rows.treeAggregate(new BDV[Double](new Array[Double](nt)))(
            seqOp = (U, v) => {
              RowMatrix.dspr(1.0, v, U.data)
              U
            }, combOp = (U1, U2) => U1 += U2)
      
          RowMatrix.triuToFull(n, GU.data)
        }
      

      When the size of Vectors generated by TF-IDF is too large, it makes "nt" to have undesirable value (and undesirable size of Array used in treeAggregate),
      since n * (n + 1) / 2 exceeded Int.MaxValue.

      Is this surmise correct?

      And, of course, I could avoid this situation by creating instance of HashingTF with smaller numFeatures.
      But this may not be fundamental solution.

      Attachments

        Activity

          People

            srowen Sean R. Owen
            dobachi Masaru Dobashi
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: