Return-Path: X-Original-To: apmail-spark-issues-archive@minotaur.apache.org Delivered-To: apmail-spark-issues-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id ADEF617EED for ; Sun, 5 Oct 2014 16:29:34 +0000 (UTC) Received: (qmail 32938 invoked by uid 500); 5 Oct 2014 16:29:34 -0000 Delivered-To: apmail-spark-issues-archive@spark.apache.org Received: (qmail 32905 invoked by uid 500); 5 Oct 2014 16:29:34 -0000 Mailing-List: contact issues-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@spark.apache.org Received: (qmail 32824 invoked by uid 99); 5 Oct 2014 16:29:34 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 05 Oct 2014 16:29:34 +0000 Date: Sun, 5 Oct 2014 16:29:34 +0000 (UTC) From: "Masaru Dobashi (JIRA)" To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (SPARK-3803) ArrayIndexOutOfBoundsException found in executing computePrincipalComponents MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/SPARK-3803?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Masaru Dobashi updated SPARK-3803: ---------------------------------- Description: When I executed computePrincipalComponents method of RowMatrix, I got java.lang.ArrayIndexOutOfBoundsException. {code} 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) {code} The RowMatrix instance was generated from the result of TF-IDF like the following. {code} 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) {code} I think this was because I created HashingTF instance with default numFeatures and Array is used in RowMatrix#computeGramianMatrix method like the following. {code} /** * 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) } {code} 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 does not seems to be fundamental solution. was: When I executed computePrincipalComponents method of RowMatrix, I got java.lang.ArrayIndexOutOfBoundsException. {code} 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) {code} The RowMatrix instance was generated from the result of TF-IDF like the following. {code} 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) {code} I think this was because I created HashingTF instance with default numFeatures and Array is used in RowMatrix#computeGramianMatrix method like the following. {code} /** * 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) } {code} 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 seems to be not fundamental solution. > ArrayIndexOutOfBoundsException found in executing computePrincipalComponents > ---------------------------------------------------------------------------- > > Key: SPARK-3803 > URL: https://issues.apache.org/jira/browse/SPARK-3803 > Project: Spark > Issue Type: Bug > Components: MLlib > Affects Versions: 1.1.0 > Reporter: Masaru Dobashi > > When I executed computePrincipalComponents method of RowMatrix, I got java.lang.ArrayIndexOutOfBoundsException. > {code} > 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) > {code} > The RowMatrix instance was generated from the result of TF-IDF like the following. > {code} > 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) > {code} > I think this was because I created HashingTF instance with default numFeatures and Array is used in RowMatrix#computeGramianMatrix method > like the following. > {code} > /** > * 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) > } > {code} > 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 does not seems to be fundamental solution. -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org For additional commands, e-mail: issues-help@spark.apache.org