You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by insperatum <in...@gmail.com> on 2015/03/11 18:00:03 UTC

Scaling problem in RandomForest?

Hi, the Random Forest implementation (1.2.1) is repeatably crashing when I
increase the depth to 20. I generate random synthetic data (36 workers,
1,000,000 examples per worker, 30 features per example) as follows:

    val data = sc.parallelize(1 to 36, 36).mapPartitionsWithIndex((i, _) =>
{
      Array.tabulate(1000000){ _ =>
        new LabeledPoint(Math.random(),
Vectors.dense(Array.fill(30)(math.random)))
      }.toIterator
    }).cache()

...and then train on a Random Forest with 50 trees, to depth 20:

    val strategy = new Strategy(Regression, Variance, 20, maxMemoryInMB =
1000)
    RandomForest.trainRegressor(data, strategy, 50, "sqrt", 1)

...and run on my EC2 cluster (36 slaves, master has 122GB of memory). After
number crunching for a couple of hours, I get the following error:

[sparkDriver-akka.actor.default-dispatcher-3] shutting down ActorSystem
[sparkDriver]
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
	at java.util.Arrays.copyOf(Arrays.java:2271)
	at java.io.ByteArrayOutputStream.grow(ByteArrayOutputStream.java:113)
	at
java.io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:93)
	at java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:140)
	at
java.io.ObjectOutputStream$BlockDataOutputStream.drain(ObjectOutputStream.java:1876)
	at
java.io.ObjectOutputStream$BlockDataOutputStream.setBlockDataMode(ObjectOutputStream.java:1785)
	at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1188)
	at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347)
	at
org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:42)
	at
org.apache.spark.serializer.JavaSerializerInstance.serialize(JavaSerializer.scala:73)
	at
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitMissingTasks(DAGScheduler.scala:834)
	at
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitStage(DAGScheduler.scala:778)
	at
org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$submitStage$4.apply(DAGScheduler.scala:781)
	at
org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$submitStage$4.apply(DAGScheduler.scala:780)
	at scala.collection.immutable.List.foreach(List.scala:318)
	at
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitStage(DAGScheduler.scala:780)
	at
org.apache.spark.scheduler.DAGScheduler.handleJobSubmitted(DAGScheduler.scala:762)
	at
org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1389)
	at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
	at
org.apache.spark.scheduler.DAGSchedulerEventProcessActor.aroundReceive(DAGScheduler.scala:1375)
	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
	at akka.actor.ActorCell.invoke(ActorCell.scala:487)
	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
	at akka.dispatch.Mailbox.run(Mailbox.scala:220)
	at
akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:393)
	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
	at
scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
	at
scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
15/03/11 15:45:51 INFO scheduler.DAGScheduler: Job 92 failed: collectAsMap
at DecisionTree.scala:653, took 46.062487 s



--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Scaling-problem-in-RandomForest-tp22002.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
For additional commands, e-mail: user-help@spark.apache.org


Re: Scaling problem in RandomForest?

Posted by Xiangrui Meng <me...@gmail.com>.
Try increasing the driver memory. We store trees on the driver node.
If maxDepth=20 and numTrees=50, you may need a large driver memory to
store all tree models. You might want to start with a smaller maxDepth
and then increase it and see whether deep trees really help (vs. the
cost). -Xiangrui

On Wed, Mar 11, 2015 at 10:00 AM, insperatum <in...@gmail.com> wrote:
> Hi, the Random Forest implementation (1.2.1) is repeatably crashing when I
> increase the depth to 20. I generate random synthetic data (36 workers,
> 1,000,000 examples per worker, 30 features per example) as follows:
>
>     val data = sc.parallelize(1 to 36, 36).mapPartitionsWithIndex((i, _) =>
> {
>       Array.tabulate(1000000){ _ =>
>         new LabeledPoint(Math.random(),
> Vectors.dense(Array.fill(30)(math.random)))
>       }.toIterator
>     }).cache()
>
> ...and then train on a Random Forest with 50 trees, to depth 20:
>
>     val strategy = new Strategy(Regression, Variance, 20, maxMemoryInMB =
> 1000)
>     RandomForest.trainRegressor(data, strategy, 50, "sqrt", 1)
>
> ...and run on my EC2 cluster (36 slaves, master has 122GB of memory). After
> number crunching for a couple of hours, I get the following error:
>
> [sparkDriver-akka.actor.default-dispatcher-3] shutting down ActorSystem
> [sparkDriver]
> java.lang.OutOfMemoryError: Requested array size exceeds VM limit
>         at java.util.Arrays.copyOf(Arrays.java:2271)
>         at java.io.ByteArrayOutputStream.grow(ByteArrayOutputStream.java:113)
>         at
> java.io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:93)
>         at java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:140)
>         at
> java.io.ObjectOutputStream$BlockDataOutputStream.drain(ObjectOutputStream.java:1876)
>         at
> java.io.ObjectOutputStream$BlockDataOutputStream.setBlockDataMode(ObjectOutputStream.java:1785)
>         at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1188)
>         at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347)
>         at
> org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:42)
>         at
> org.apache.spark.serializer.JavaSerializerInstance.serialize(JavaSerializer.scala:73)
>         at
> org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitMissingTasks(DAGScheduler.scala:834)
>         at
> org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitStage(DAGScheduler.scala:778)
>         at
> org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$submitStage$4.apply(DAGScheduler.scala:781)
>         at
> org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$submitStage$4.apply(DAGScheduler.scala:780)
>         at scala.collection.immutable.List.foreach(List.scala:318)
>         at
> org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitStage(DAGScheduler.scala:780)
>         at
> org.apache.spark.scheduler.DAGScheduler.handleJobSubmitted(DAGScheduler.scala:762)
>         at
> org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1389)
>         at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
>         at
> org.apache.spark.scheduler.DAGSchedulerEventProcessActor.aroundReceive(DAGScheduler.scala:1375)
>         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
>         at akka.actor.ActorCell.invoke(ActorCell.scala:487)
>         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
>         at akka.dispatch.Mailbox.run(Mailbox.scala:220)
>         at
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:393)
>         at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>         at
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>         at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>         at
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> 15/03/11 15:45:51 INFO scheduler.DAGScheduler: Job 92 failed: collectAsMap
> at DecisionTree.scala:653, took 46.062487 s
>
>
>
> --
> View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Scaling-problem-in-RandomForest-tp22002.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
> For additional commands, e-mail: user-help@spark.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
For additional commands, e-mail: user-help@spark.apache.org