Hi Deb,

Are you using the master branch or a particular commit? Do you have
negative or out-of-integer-range user or product ids? There is an
issue with ALS' partitioning
(https://spark-project.atlassian.net/browse/SPARK-1281), but I'm not
sure whether that is the reason. Could you try to see whether you can
reproduce the error on a public data set, e.g., movielens? Thanks!

Best,
Xiangrui

On Sat, Apr 5, 2014 at 10:53 PM, Debasish Das <debasish.da...@gmail.com> wrote:
> Hi,
>
> I deployed apache/spark master today and recently there were many ALS
> related checkins and enhancements..
>
> I am running ALS with explicit feedback and I remember most enhancements
> were related to implicit feedback...
>
> With 25 factors my runs were successful but with 50 factors I am getting
> array index out of bound...
>
> Note that I was hitting gc errors before with an older version of spark but
> it seems like the sparse matrix partitioning scheme has changed now...data
> caching looks much balanced now...earlier one node was becoming
> bottleneck...Although I ran with 64g memory per node...
>
> There are around 3M products, 25M users...
>
> Anyone noticed this bug or something similar ?
>
> 14/04/05 23:03:15 WARN TaskSetManager: Loss was due to
> java.lang.ArrayIndexOutOfBoundsException
> java.lang.ArrayIndexOutOfBoundsException: 81029
>     at
> org.apache.spark.mllib.recommendation.ALS$$anonfun$org$apache$spark$mllib$recommendation$ALS$$updateBlock$1$$anonfun$apply$mcVI$sp$1.apply$mcVI$sp(ALS.scala:450)
>     at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:141)
>     at
> org.apache.spark.mllib.recommendation.ALS$$anonfun$org$apache$spark$mllib$recommendation$ALS$$updateBlock$1.apply$mcVI$sp(ALS.scala:446)
>     at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:141)
>     at org.apache.spark.mllib.recommendation.ALS.org
> $apache$spark$mllib$recommendation$ALS$$updateBlock(ALS.scala:445)
>     at
> org.apache.spark.mllib.recommendation.ALS$$anonfun$org$apache$spark$mllib$recommendation$ALS$$updateFeatures$2.apply(ALS.scala:416)
>     at
> org.apache.spark.mllib.recommendation.ALS$$anonfun$org$apache$spark$mllib$recommendation$ALS$$updateFeatures$2.apply(ALS.scala:415)
>     at
> org.apache.spark.rdd.MappedValuesRDD$$anonfun$compute$1.apply(MappedValuesRDD.scala:31)
>     at
> org.apache.spark.rdd.MappedValuesRDD$$anonfun$compute$1.apply(MappedValuesRDD.scala:31)
>     at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>     at
> org.apache.spark.rdd.CoGroupedRDD$$anonfun$compute$4.apply(CoGroupedRDD.scala:149)
>     at
> org.apache.spark.rdd.CoGroupedRDD$$anonfun$compute$4.apply(CoGroupedRDD.scala:147)
>     at
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>     at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
>     at org.apache.spark.rdd.CoGroupedRDD.compute(CoGroupedRDD.scala:147)
>     at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:229)
>     at org.apache.spark.rdd.RDD.iterator(RDD.scala:220)
>     at
> org.apache.spark.rdd.MappedValuesRDD.compute(MappedValuesRDD.scala:31)
>     at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:229)
>     at org.apache.spark.rdd.RDD.iterator(RDD.scala:220)
>     at
> org.apache.spark.rdd.FlatMappedValuesRDD.compute(FlatMappedValuesRDD.scala:31)
>     at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:229)
>     at org.apache.spark.rdd.RDD.iterator(RDD.scala:220)
>     at org.apache.spark.rdd.FlatMappedRDD.compute(FlatMappedRDD.scala:33)
>     at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:229)
>     at org.apache.spark.rdd.RDD.iterator(RDD.scala:220)
>     at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:161)
>     at
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:102)
>     at org.apache.spark.scheduler.Task.run(Task.scala:52)
>     at
> org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$1.apply$mcV$sp(Executor.scala:211)
>     at
> org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:43)
>     at
> org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:42)
>     at java.security.AccessController.doPrivileged(Native Method)
>     at javax.security.auth.Subject.doAs(Subject.java:396)
>     at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
>     at
> org.apache.spark.deploy.SparkHadoopUtil.runAsUser(SparkHadoopUtil.scala:42)
>     at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:176)
>     at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>     at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>     at java.lang.Thread.run(Thread.java:662)
>
> Thanks.
> Deb

Reply via email to