Re: Execution stalls in LogisticRegressionWithSGD
We have maven-enforcer-plugin defined in the pom. I don't know why it didn't work for you. Could you try rebuild with maven2 and confirm that there is no error message? If that is the case, please create a JIRA for it. Thanks! -Xiangrui On Wed, Jul 9, 2014 at 3:53 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Xiangrui, Thanks for all the help in resolving this issue. The cause turned out to bethe build environment rather than runtime configuration. The build process had picked up maven2 while building spark. Using binaries that were rebuilt using m3, the entire processing went through fine. While I'm aware that the build instruction page specifies m3 as the min requirement, declaratively preventing accidental m2 usage (e.g. through something like the maven enforcer plugin?) might help other developers avoid such issues. -Bharath On Mon, Jul 7, 2014 at 9:43 PM, Xiangrui Meng men...@gmail.com wrote: It seems to me a setup issue. I just tested news20.binary (1355191 features) on a 2-node EC2 cluster and it worked well. I added one line to conf/spark-env.sh: export SPARK_JAVA_OPTS= -Dspark.akka.frameSize=20 and launched spark-shell with --driver-memory 20g. Could you re-try with an EC2 setup? If it still doesn't work, please attach all your code and logs. Best, Xiangrui On Sun, Jul 6, 2014 at 1:35 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi Xiangrui, 1) Yes, I used the same build (compiled locally from source) to the host that has (master, slave1) and the second host with slave2. 2) The execution was successful when run in local mode with reduced number of partitions. Does this imply issues communicating/coordinating across processes (i.e. driver, master and workers)? Thanks, Bharath On Sun, Jul 6, 2014 at 11:37 AM, Xiangrui Meng men...@gmail.com wrote: Hi Bharath, 1) Did you sync the spark jar and conf to the worker nodes after build? 2) Since the dataset is not large, could you try local mode first using `spark-summit --driver-memory 12g --master local[*]`? 3) Try to use less number of partitions, say 5. If the problem is still there, please attach the full master/worker log files. Best, Xiangrui On Fri, Jul 4, 2014 at 12:16 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Xiangrui, Leaving the frameSize unspecified led to an error message (and failure) stating that the task size (~11M) was larger. I hence set it to an arbitrarily large value ( I realize 500 was unrealistic unnecessary in this case). I've now set the size to 20M and repeated the runs. The earlier runs were on an uncached RDD. Caching the RDD (and setting spark.storage.memoryFraction=0.5) resulted in marginal speed up of execution, but the end result remained the same. The cached RDD size is as follows: RDD NameStorage LevelCached Partitions Fraction CachedSize in MemorySize in TachyonSize on Disk 1084 Memory Deserialized 1x Replicated 80 100% 165.9 MB 0.0 B 0.0 B The corresponding master logs were: 14/07/04 06:29:34 INFO Master: Removing executor app-20140704062238-0033/1 because it is EXITED 14/07/04 06:29:34 INFO Master: Launching executor app-20140704062238-0033/2 on worker worker-20140630124441-slave1-40182 14/07/04 06:29:34 INFO Master: Removing executor app-20140704062238-0033/0 because it is EXITED 14/07/04 06:29:34 INFO Master: Launching executor app-20140704062238-0033/3 on worker worker-20140630102913-slave2-44735 14/07/04 06:29:37 INFO Master: Removing executor app-20140704062238-0033/2 because it is EXITED 14/07/04 06:29:37 INFO Master: Launching executor app-20140704062238-0033/4 on worker worker-20140630124441-slave1-40182 14/07/04 06:29:37 INFO Master: Removing executor app-20140704062238-0033/3 because it is EXITED 14/07/04 06:29:37 INFO Master: Launching executor app-20140704062238-0033/5 on worker worker-20140630102913-slave2-44735 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 INFO Master: Removing app app-20140704062238-0033 14/07/04 06:29:39 INFO LocalActorRef: Message [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from Actor[akka://sparkMaster/deadLetters] to Actor[akka://sparkMaster/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkMaster%4010.3.1.135%3A33061-123#1986674260] was not delivered. [39] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'. 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 INFO Master:
Re: Execution stalls in LogisticRegressionWithSGD
It seems to me a setup issue. I just tested news20.binary (1355191 features) on a 2-node EC2 cluster and it worked well. I added one line to conf/spark-env.sh: export SPARK_JAVA_OPTS= -Dspark.akka.frameSize=20 and launched spark-shell with --driver-memory 20g. Could you re-try with an EC2 setup? If it still doesn't work, please attach all your code and logs. Best, Xiangrui On Sun, Jul 6, 2014 at 1:35 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi Xiangrui, 1) Yes, I used the same build (compiled locally from source) to the host that has (master, slave1) and the second host with slave2. 2) The execution was successful when run in local mode with reduced number of partitions. Does this imply issues communicating/coordinating across processes (i.e. driver, master and workers)? Thanks, Bharath On Sun, Jul 6, 2014 at 11:37 AM, Xiangrui Meng men...@gmail.com wrote: Hi Bharath, 1) Did you sync the spark jar and conf to the worker nodes after build? 2) Since the dataset is not large, could you try local mode first using `spark-summit --driver-memory 12g --master local[*]`? 3) Try to use less number of partitions, say 5. If the problem is still there, please attach the full master/worker log files. Best, Xiangrui On Fri, Jul 4, 2014 at 12:16 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Xiangrui, Leaving the frameSize unspecified led to an error message (and failure) stating that the task size (~11M) was larger. I hence set it to an arbitrarily large value ( I realize 500 was unrealistic unnecessary in this case). I've now set the size to 20M and repeated the runs. The earlier runs were on an uncached RDD. Caching the RDD (and setting spark.storage.memoryFraction=0.5) resulted in marginal speed up of execution, but the end result remained the same. The cached RDD size is as follows: RDD NameStorage LevelCached Partitions Fraction CachedSize in MemorySize in TachyonSize on Disk 1084 Memory Deserialized 1x Replicated 80 100% 165.9 MB 0.0 B 0.0 B The corresponding master logs were: 14/07/04 06:29:34 INFO Master: Removing executor app-20140704062238-0033/1 because it is EXITED 14/07/04 06:29:34 INFO Master: Launching executor app-20140704062238-0033/2 on worker worker-20140630124441-slave1-40182 14/07/04 06:29:34 INFO Master: Removing executor app-20140704062238-0033/0 because it is EXITED 14/07/04 06:29:34 INFO Master: Launching executor app-20140704062238-0033/3 on worker worker-20140630102913-slave2-44735 14/07/04 06:29:37 INFO Master: Removing executor app-20140704062238-0033/2 because it is EXITED 14/07/04 06:29:37 INFO Master: Launching executor app-20140704062238-0033/4 on worker worker-20140630124441-slave1-40182 14/07/04 06:29:37 INFO Master: Removing executor app-20140704062238-0033/3 because it is EXITED 14/07/04 06:29:37 INFO Master: Launching executor app-20140704062238-0033/5 on worker worker-20140630102913-slave2-44735 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 INFO Master: Removing app app-20140704062238-0033 14/07/04 06:29:39 INFO LocalActorRef: Message [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from Actor[akka://sparkMaster/deadLetters] to Actor[akka://sparkMaster/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkMaster%4010.3.1.135%3A33061-123#1986674260] was not delivered. [39] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'. 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 ERROR EndpointWriter: AssociationError [akka.tcp://sparkMaster@master:7077] - [akka.tcp://spark@slave2:45172]: Error [Association failed with [akka.tcp://spark@slave2:45172]] [ akka.remote.EndpointAssociationException: Association failed with [akka.tcp://spark@slave2:45172] Caused by: akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2: Connection refused: slave2/10.3.1.135:45172 ] 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 ERROR EndpointWriter: AssociationError [akka.tcp://sparkMaster@master:7077] - [akka.tcp://spark@slave2:45172]: Error [Association failed with [akka.tcp://spark@slave2:45172]] [ akka.remote.EndpointAssociationException: Association failed with [akka.tcp://spark@slave2:45172] Caused by: akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2: Connection refused: slave2/10.3.1.135:45172 ] 14/07/04 06:29:39 ERROR EndpointWriter:
Re: Execution stalls in LogisticRegressionWithSGD
Hi Bharath, 1) Did you sync the spark jar and conf to the worker nodes after build? 2) Since the dataset is not large, could you try local mode first using `spark-summit --driver-memory 12g --master local[*]`? 3) Try to use less number of partitions, say 5. If the problem is still there, please attach the full master/worker log files. Best, Xiangrui On Fri, Jul 4, 2014 at 12:16 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Xiangrui, Leaving the frameSize unspecified led to an error message (and failure) stating that the task size (~11M) was larger. I hence set it to an arbitrarily large value ( I realize 500 was unrealistic unnecessary in this case). I've now set the size to 20M and repeated the runs. The earlier runs were on an uncached RDD. Caching the RDD (and setting spark.storage.memoryFraction=0.5) resulted in marginal speed up of execution, but the end result remained the same. The cached RDD size is as follows: RDD NameStorage LevelCached Partitions Fraction CachedSize in MemorySize in TachyonSize on Disk 1084 Memory Deserialized 1x Replicated 80 100% 165.9 MB 0.0 B 0.0 B The corresponding master logs were: 14/07/04 06:29:34 INFO Master: Removing executor app-20140704062238-0033/1 because it is EXITED 14/07/04 06:29:34 INFO Master: Launching executor app-20140704062238-0033/2 on worker worker-20140630124441-slave1-40182 14/07/04 06:29:34 INFO Master: Removing executor app-20140704062238-0033/0 because it is EXITED 14/07/04 06:29:34 INFO Master: Launching executor app-20140704062238-0033/3 on worker worker-20140630102913-slave2-44735 14/07/04 06:29:37 INFO Master: Removing executor app-20140704062238-0033/2 because it is EXITED 14/07/04 06:29:37 INFO Master: Launching executor app-20140704062238-0033/4 on worker worker-20140630124441-slave1-40182 14/07/04 06:29:37 INFO Master: Removing executor app-20140704062238-0033/3 because it is EXITED 14/07/04 06:29:37 INFO Master: Launching executor app-20140704062238-0033/5 on worker worker-20140630102913-slave2-44735 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 INFO Master: Removing app app-20140704062238-0033 14/07/04 06:29:39 INFO LocalActorRef: Message [akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying] from Actor[akka://sparkMaster/deadLetters] to Actor[akka://sparkMaster/system/transports/akkaprotocolmanager.tcp0/akkaProtocol-tcp%3A%2F%2FsparkMaster%4010.3.1.135%3A33061-123#1986674260] was not delivered. [39] dead letters encountered. This logging can be turned off or adjusted with configuration settings 'akka.log-dead-letters' and 'akka.log-dead-letters-during-shutdown'. 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 ERROR EndpointWriter: AssociationError [akka.tcp://sparkMaster@master:7077] - [akka.tcp://spark@slave2:45172]: Error [Association failed with [akka.tcp://spark@slave2:45172]] [ akka.remote.EndpointAssociationException: Association failed with [akka.tcp://spark@slave2:45172] Caused by: akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2: Connection refused: slave2/10.3.1.135:45172 ] 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:39 ERROR EndpointWriter: AssociationError [akka.tcp://sparkMaster@master:7077] - [akka.tcp://spark@slave2:45172]: Error [Association failed with [akka.tcp://spark@slave2:45172]] [ akka.remote.EndpointAssociationException: Association failed with [akka.tcp://spark@slave2:45172] Caused by: akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2: Connection refused: slave2/10.3.1.135:45172 ] 14/07/04 06:29:39 ERROR EndpointWriter: AssociationError [akka.tcp://sparkMaster@master:7077] - [akka.tcp://spark@slave2:45172]: Error [Association failed with [akka.tcp://spark@slave2:45172]] [ akka.remote.EndpointAssociationException: Association failed with [akka.tcp://spark@slave2:45172] Caused by: akka.remote.transport.netty.NettyTransport$$anonfun$associate$1$$anon$2: Connection refused: slave2/10.3.1.135:45172 ] 14/07/04 06:29:39 INFO Master: akka.tcp://spark@slave2:45172 got disassociated, removing it. 14/07/04 06:29:40 WARN Master: Got status update for unknown executor app-20140704062238-0033/5 14/07/04 06:29:40 WARN Master: Got status update for unknown executor app-20140704062238-0033/4 Coincidentally, after the initial executor failed, each following executor that was re-spawned failed with the following logs: (e.g the following was from slave1:~/spark-1.0.1-rc1/work/app-20140704062238-0033/2/stderr) log4j:WARN No appenders could be found for logger (org.apache.hadoop.conf.Configuration).
Re: Execution stalls in LogisticRegressionWithSGD
Hi Xiangrui, The issue with aggergating/counting over large feature vectors (as part of LogisticRegressionWithSGD) continues to exist, but now in another form: while the execution doesn't freeze (due to SPARK-1112), it now fails at the second or third gradient descent iteration consistently with an error level log message, but no stacktrace. I'm running against 1.0.1-rc1, and have tried setting spark.akka.frameSize as high as 500. When the execution fails, each of the two executors log the following message (corresponding to aggregate at GradientDescent.scala:178) : 14/07/02 14:09:09 INFO BlockFetcherIterator$BasicBlockFetcherIterator: maxBytesInFlight: 50331648, targetRequestSize: 10066329 14/07/02 14:09:09 INFO BlockFetcherIterator$BasicBlockFetcherIterator: Getting 2 non-empty blocks out of 2 blocks 14/07/02 14:09:09 INFO BlockFetcherIterator$BasicBlockFetcherIterator: Started 1 remote fetches in 0 ms 14/07/02 14:09:11 INFO Executor: Serialized size of result for 737 is 5959086 14/07/02 14:09:11 INFO Executor: Sending result for 737 directly to driver 14/07/02 14:09:11 INFO Executor: Finished task ID 737 14/07/02 14:09:18 ERROR CoarseGrainedExecutorBackend: Driver Disassociated [akka.tcp://sparkExecutor@(slave1,slave2):51941] - [akka.tcp://spark@master:59487] disassociated! Shutting down. There is no separate stacktrace on the driver side. Each input record is of the form p1, p2, (p1,p2) where p1, p2 (p1,p2) are categorical features with large cardinality, and X is the double label with a continuous value. The categorical variables are converted to binary variables which results in a feature vector of size 741092 (composed of all unique categories across p1, p2 and (p1,p2)). Thus, the labeled point for input record is a sparse vector of size 741092 with only 3 variables set in the record. The total number of records is 683233 after aggregating the input data on (p1, p2). When attempting to train on the unaggregated records (1337907 in number spread across 455 files), the execution fails at count, GradientDescent.scala:161 with the following log (Snipped lines corresponding to other input files) 14/07/02 16:02:03 INFO HadoopRDD: Input split: file:~/part-r-00012:2834590+2834590 14/07/02 16:02:03 INFO HadoopRDD: Input split: file:~/part-r-5:0+2845559 14/07/02 16:02:03 INFO HadoopRDD: Input split: file:~/part-r-5:2845559+2845560 14/07/02 16:02:03 INFO Executor: Serialized size of result for 726 is 615 14/07/02 16:02:03 INFO Executor: Sending result for 726 directly to driver 14/07/02 16:02:03 INFO Executor: Finished task ID 726 14/07/02 16:02:12 ERROR CoarseGrainedExecutorBackend: Driver Disassociated [akka.tcp://sparkExecutor@slave1:48423] - [akka.tcp://spark@master:55792] disassociated! Shutting down. A count() attempted on the input RDD before beginning training has the following metrics: MetricMin25thMedian75th Max Result serialization time0 ms0 ms0 ms0 ms0 ms Duration33 s33 s35 s35 s35 s Time spent fetching task results0 ms0 ms0 ms0 ms0 ms Scheduler delay0.1 s0.1 s0.3 s0.3 s0.3 s Aggregated Metrics by Executor ID Address Task Time Total Failed Succeeded Shuffle Read Shuffle Write Shuf Spill (Mem) Shuf Spill (Disk) 0 CANNOT FIND ADDRESS 34 s 1 0 1 0.0 B 0.0 B 0.0 B 0.0 B 1 CANNOT FIND ADDRESS 36 s 1 0 1 0.0 B 0.0 B 0.0 B 0.0 B Tasks Task IndexTask IDStatusLocality LevelExecutorLaunch TimeDurationGC TimeResult Ser TimeErrors 0 726 SUCCESS PROCESS_LOCAL slave1 2014/07/02 16:01:28 35 s 0.1 s 1 727 SUCCESS PROCESS_LOCAL slave2 2014/07/02 16:01:28 33 s 99 ms Any pointers / diagnosis please? On Thu, Jun 19, 2014 at 10:03 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Thanks. I'll await the fix to re-run my test. On Thu, Jun 19, 2014 at 8:28 AM, Xiangrui Meng men...@gmail.com wrote: Hi Bharath, This is related to SPARK-1112, which we already found the root cause. I will let you know when this is fixed. Best, Xiangrui On Tue, Jun 17, 2014 at 7:37 PM, Bharath Ravi Kumar reachb...@gmail.com wrote: Couple more points: 1)The inexplicable stalling of execution with large feature sets appears similar to that reported with the news-20 dataset: http://mail-archives.apache.org/mod_mbox/spark-user/201406.mbox/%3c53a03542.1010...@gmail.com%3E 2) The NPE trying to call mapToPair convert an RDDLong, Long, Integer, Integer into a JavaPairRDDTuple2Long,Long, Tuple2Integer,Integer is unrelated to mllib. Thanks, Bharath On Wed, Jun 18, 2014 at 7:14 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi Xiangrui , I'm using
Re: Execution stalls in LogisticRegressionWithSGD
Hi Bharath, This is related to SPARK-1112, which we already found the root cause. I will let you know when this is fixed. Best, Xiangrui On Tue, Jun 17, 2014 at 7:37 PM, Bharath Ravi Kumar reachb...@gmail.com wrote: Couple more points: 1)The inexplicable stalling of execution with large feature sets appears similar to that reported with the news-20 dataset: http://mail-archives.apache.org/mod_mbox/spark-user/201406.mbox/%3c53a03542.1010...@gmail.com%3E 2) The NPE trying to call mapToPair convert an RDDLong, Long, Integer, Integer into a JavaPairRDDTuple2Long,Long, Tuple2Integer,Integer is unrelated to mllib. Thanks, Bharath On Wed, Jun 18, 2014 at 7:14 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi Xiangrui , I'm using 1.0.0. Thanks, Bharath On 18-Jun-2014 1:43 am, Xiangrui Meng men...@gmail.com wrote: Hi Bharath, Thanks for posting the details! Which Spark version are you using? Best, Xiangrui On Tue, Jun 17, 2014 at 6:48 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi, (Apologies for the long mail, but it's necessary to provide sufficient details considering the number of issues faced.) I'm running into issues testing LogisticRegressionWithSGD a two node cluster (each node with 24 cores and 16G available to slaves out of 24G on the system). Here's a description of the application: The model is being trained based on categorical features x, y, and (x,y). The categorical features are mapped to binary features by converting each distinct value in the category enum into a binary feature by itself (i.e presence of that value in a record implies corresponding feature = 1, else feature = 0. So, there'd be as many distinct features as enum values) . The training vector is laid out as [x1,x2...xn,y1,y2yn,(x1,y1),(x2,y2)...(xn,yn)]. Each record in the training data has only one combination (Xk,Yk) and a label appearing in the record. Thus, the corresponding labeledpoint sparse vector would only have 3 values Xk, Yk, (Xk,Yk) set for a record. The total length of the vector (though parse) would be nearly 614000. The number of records is about 1.33 million. The records have been coalesced into 20 partitions across two nodes. The input data has not been cached. (NOTE: I do realize the records features may seem large for a two node setup, but given the memory cpu, and the fact that I'm willing to give up some turnaround time, I don't see why tasks should inexplicably fail) Additional parameters include: spark.executor.memory = 14G spark.default.parallelism = 1 spark.cores.max=20 spark.storage.memoryFraction=0.8 //No cache space required (Trying to set spark.akka.frameSize to a larger number, say, 20 didn't help either) The model training was initialized as : new LogisticRegressionWithSGD(1, maxIterations, 0.0, 0.05) However, after 4 iterations of gradient descent, the entire execution appeared to stall inexplicably. The corresponding executor details and details of the stalled stage (number 14) are as follows: MetricMin25th Median75th Max Result serialization time12 ms13 ms14 ms16 ms18 ms Duration4 s4 s5 s5 s 5 s Time spent fetching task 0 ms0 ms0 ms0 ms0 ms results Scheduler delay6 s6 s6 s6 s 12 s Stage Id 14 aggregate at GradientDescent.scala:178 Task IndexTask IDStatusLocality Level Executor Launch TimeDurationGC Result Ser TimeErrors Time 0 600 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 1 601 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 2 602 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 3 603 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 4 604 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 5 605 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 6 606 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 14 ms 7 607 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 8 608 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 9 609 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 14 ms 10 610 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 11 611 SUCCESS PROCESS_LOCAL
Re: Execution stalls in LogisticRegressionWithSGD
Thanks. I'll await the fix to re-run my test. On Thu, Jun 19, 2014 at 8:28 AM, Xiangrui Meng men...@gmail.com wrote: Hi Bharath, This is related to SPARK-1112, which we already found the root cause. I will let you know when this is fixed. Best, Xiangrui On Tue, Jun 17, 2014 at 7:37 PM, Bharath Ravi Kumar reachb...@gmail.com wrote: Couple more points: 1)The inexplicable stalling of execution with large feature sets appears similar to that reported with the news-20 dataset: http://mail-archives.apache.org/mod_mbox/spark-user/201406.mbox/%3c53a03542.1010...@gmail.com%3E 2) The NPE trying to call mapToPair convert an RDDLong, Long, Integer, Integer into a JavaPairRDDTuple2Long,Long, Tuple2Integer,Integer is unrelated to mllib. Thanks, Bharath On Wed, Jun 18, 2014 at 7:14 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi Xiangrui , I'm using 1.0.0. Thanks, Bharath On 18-Jun-2014 1:43 am, Xiangrui Meng men...@gmail.com wrote: Hi Bharath, Thanks for posting the details! Which Spark version are you using? Best, Xiangrui On Tue, Jun 17, 2014 at 6:48 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi, (Apologies for the long mail, but it's necessary to provide sufficient details considering the number of issues faced.) I'm running into issues testing LogisticRegressionWithSGD a two node cluster (each node with 24 cores and 16G available to slaves out of 24G on the system). Here's a description of the application: The model is being trained based on categorical features x, y, and (x,y). The categorical features are mapped to binary features by converting each distinct value in the category enum into a binary feature by itself (i.e presence of that value in a record implies corresponding feature = 1, else feature = 0. So, there'd be as many distinct features as enum values) . The training vector is laid out as [x1,x2...xn,y1,y2yn,(x1,y1),(x2,y2)...(xn,yn)]. Each record in the training data has only one combination (Xk,Yk) and a label appearing in the record. Thus, the corresponding labeledpoint sparse vector would only have 3 values Xk, Yk, (Xk,Yk) set for a record. The total length of the vector (though parse) would be nearly 614000. The number of records is about 1.33 million. The records have been coalesced into 20 partitions across two nodes. The input data has not been cached. (NOTE: I do realize the records features may seem large for a two node setup, but given the memory cpu, and the fact that I'm willing to give up some turnaround time, I don't see why tasks should inexplicably fail) Additional parameters include: spark.executor.memory = 14G spark.default.parallelism = 1 spark.cores.max=20 spark.storage.memoryFraction=0.8 //No cache space required (Trying to set spark.akka.frameSize to a larger number, say, 20 didn't help either) The model training was initialized as : new LogisticRegressionWithSGD(1, maxIterations, 0.0, 0.05) However, after 4 iterations of gradient descent, the entire execution appeared to stall inexplicably. The corresponding executor details and details of the stalled stage (number 14) are as follows: MetricMin25th Median75th Max Result serialization time12 ms13 ms14 ms16 ms18 ms Duration4 s4 s5 s5 s 5 s Time spent fetching task 0 ms0 ms0 ms0 ms0 ms results Scheduler delay6 s6 s6 s6 s 12 s Stage Id 14 aggregate at GradientDescent.scala:178 Task IndexTask IDStatusLocality Level Executor Launch TimeDurationGC Result Ser Time Errors Time 0 600 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 1 601 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 2 602 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 3 603 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 4 604 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 5 605 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 6 606 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 14 ms 7 607 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 8 608 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 9 609 SUCCESS
Execution stalls in LogisticRegressionWithSGD
Hi, (Apologies for the long mail, but it's necessary to provide sufficient details considering the number of issues faced.) I'm running into issues testing LogisticRegressionWithSGD a two node cluster (each node with 24 cores and 16G available to slaves out of 24G on the system). Here's a description of the application: The model is being trained based on categorical features x, y, and (x,y). The categorical features are mapped to binary features by converting each distinct value in the category enum into a binary feature by itself (i.e presence of that value in a record implies corresponding feature = 1, else feature = 0. So, there'd be as many distinct features as enum values) . The training vector is laid out as [x1,x2...xn,y1,y2yn,(x1,y1),(x2,y2)...(xn,yn)]. Each record in the training data has only one combination (Xk,Yk) and a label appearing in the record. Thus, the corresponding labeledpoint sparse vector would only have 3 values Xk, Yk, (Xk,Yk) set for a record. The total length of the vector (though parse) would be nearly 614000. The number of records is about 1.33 million. The records have been coalesced into 20 partitions across two nodes. The input data has not been cached. (NOTE: I do realize the records features may seem large for a two node setup, but given the memory cpu, and the fact that I'm willing to give up some turnaround time, I don't see why tasks should inexplicably fail) Additional parameters include: spark.executor.memory = 14G spark.default.parallelism = 1 spark.cores.max=20 spark.storage.memoryFraction=0.8 //No cache space required (Trying to set spark.akka.frameSize to a larger number, say, 20 didn't help either) The model training was initialized as : new LogisticRegressionWithSGD(1, maxIterations, 0.0, 0.05) However, after 4 iterations of gradient descent, the entire execution appeared to stall inexplicably. The corresponding executor details and details of the stalled stage (number 14) are as follows: MetricMin25th Median75th Max Result serialization time12 ms13 ms14 ms16 ms18 ms Duration4 s4 s5 s5 s5 s Time spent fetching task 0 ms0 ms0 ms0 ms0 ms results Scheduler delay6 s6 s6 s6 s 12 s Stage Id 14 aggregate at GradientDescent.scala:178 Task IndexTask IDStatusLocality Level Executor Launch TimeDurationGC Result Ser TimeErrors Time 0 600 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 1 601 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 2 602 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 3 603 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 4 604 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 5 605 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 6 606 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 14 ms 7 607 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 8 608 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 9 609 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 14 ms 10 610 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 11 611 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 13 ms 12 612 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 18 ms 13 613 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 13 ms 14 614 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 14 ms 15 615 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 12 ms 16 616 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 17 617 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 18 ms 18 618 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 16 ms 19 619 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 18 ms Executor stats: RDD BlocksMemory UsedDisk UsedActive TasksFailed Tasks Complete TasksTotal TasksTask Time
Re: Execution stalls in LogisticRegressionWithSGD
Hi Bharath, Thanks for posting the details! Which Spark version are you using? Best, Xiangrui On Tue, Jun 17, 2014 at 6:48 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi, (Apologies for the long mail, but it's necessary to provide sufficient details considering the number of issues faced.) I'm running into issues testing LogisticRegressionWithSGD a two node cluster (each node with 24 cores and 16G available to slaves out of 24G on the system). Here's a description of the application: The model is being trained based on categorical features x, y, and (x,y). The categorical features are mapped to binary features by converting each distinct value in the category enum into a binary feature by itself (i.e presence of that value in a record implies corresponding feature = 1, else feature = 0. So, there'd be as many distinct features as enum values) . The training vector is laid out as [x1,x2...xn,y1,y2yn,(x1,y1),(x2,y2)...(xn,yn)]. Each record in the training data has only one combination (Xk,Yk) and a label appearing in the record. Thus, the corresponding labeledpoint sparse vector would only have 3 values Xk, Yk, (Xk,Yk) set for a record. The total length of the vector (though parse) would be nearly 614000. The number of records is about 1.33 million. The records have been coalesced into 20 partitions across two nodes. The input data has not been cached. (NOTE: I do realize the records features may seem large for a two node setup, but given the memory cpu, and the fact that I'm willing to give up some turnaround time, I don't see why tasks should inexplicably fail) Additional parameters include: spark.executor.memory = 14G spark.default.parallelism = 1 spark.cores.max=20 spark.storage.memoryFraction=0.8 //No cache space required (Trying to set spark.akka.frameSize to a larger number, say, 20 didn't help either) The model training was initialized as : new LogisticRegressionWithSGD(1, maxIterations, 0.0, 0.05) However, after 4 iterations of gradient descent, the entire execution appeared to stall inexplicably. The corresponding executor details and details of the stalled stage (number 14) are as follows: MetricMin25th Median75th Max Result serialization time12 ms13 ms14 ms16 ms18 ms Duration4 s4 s5 s5 s5 s Time spent fetching task 0 ms0 ms0 ms0 ms0 ms results Scheduler delay6 s6 s6 s6 s 12 s Stage Id 14 aggregate at GradientDescent.scala:178 Task IndexTask IDStatusLocality Level Executor Launch TimeDurationGC Result Ser TimeErrors Time 0 600 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 1 601 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 2 602 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 3 603 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 4 604 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 5 605 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 6 606 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 14 ms 7 607 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 8 608 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 9 609 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 14 ms 10 610 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 11 611 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 13 ms 12 612 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 18 ms 13 613 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 13 ms 14 614 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 14 ms 15 615 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 12 ms 16 616 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 17 617 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 18 ms 18 618 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 16 ms 19 619 SUCCESS PROCESS_LOCAL
Re: Execution stalls in LogisticRegressionWithSGD
Hi Xiangrui , I'm using 1.0.0. Thanks, Bharath On 18-Jun-2014 1:43 am, Xiangrui Meng men...@gmail.com wrote: Hi Bharath, Thanks for posting the details! Which Spark version are you using? Best, Xiangrui On Tue, Jun 17, 2014 at 6:48 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi, (Apologies for the long mail, but it's necessary to provide sufficient details considering the number of issues faced.) I'm running into issues testing LogisticRegressionWithSGD a two node cluster (each node with 24 cores and 16G available to slaves out of 24G on the system). Here's a description of the application: The model is being trained based on categorical features x, y, and (x,y). The categorical features are mapped to binary features by converting each distinct value in the category enum into a binary feature by itself (i.e presence of that value in a record implies corresponding feature = 1, else feature = 0. So, there'd be as many distinct features as enum values) . The training vector is laid out as [x1,x2...xn,y1,y2yn,(x1,y1),(x2,y2)...(xn,yn)]. Each record in the training data has only one combination (Xk,Yk) and a label appearing in the record. Thus, the corresponding labeledpoint sparse vector would only have 3 values Xk, Yk, (Xk,Yk) set for a record. The total length of the vector (though parse) would be nearly 614000. The number of records is about 1.33 million. The records have been coalesced into 20 partitions across two nodes. The input data has not been cached. (NOTE: I do realize the records features may seem large for a two node setup, but given the memory cpu, and the fact that I'm willing to give up some turnaround time, I don't see why tasks should inexplicably fail) Additional parameters include: spark.executor.memory = 14G spark.default.parallelism = 1 spark.cores.max=20 spark.storage.memoryFraction=0.8 //No cache space required (Trying to set spark.akka.frameSize to a larger number, say, 20 didn't help either) The model training was initialized as : new LogisticRegressionWithSGD(1, maxIterations, 0.0, 0.05) However, after 4 iterations of gradient descent, the entire execution appeared to stall inexplicably. The corresponding executor details and details of the stalled stage (number 14) are as follows: MetricMin25th Median75th Max Result serialization time12 ms13 ms14 ms16 ms18 ms Duration4 s4 s5 s5 s 5 s Time spent fetching task 0 ms0 ms0 ms0 ms0 ms results Scheduler delay6 s6 s6 s6 s 12 s Stage Id 14 aggregate at GradientDescent.scala:178 Task IndexTask IDStatusLocality Level Executor Launch TimeDurationGC Result Ser TimeErrors Time 0 600 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 1 601 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 2 602 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 3 603 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 4 604 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 5 605 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 6 606 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 14 ms 7 607 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 8 608 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 9 609 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 14 ms 10 610 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 11 611 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 13 ms 12 612 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 18 ms 13 613 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 13 ms 14 614 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 14 ms 15 615 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 12 ms 16 616 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 17 617 SUCCESS PROCESS_LOCAL
Re: Execution stalls in LogisticRegressionWithSGD
Couple more points: 1)The inexplicable stalling of execution with large feature sets appears similar to that reported with the news-20 dataset: http://mail-archives.apache.org/mod_mbox/spark-user/201406.mbox/%3c53a03542.1010...@gmail.com%3E 2) The NPE trying to call mapToPair convert an RDDLong, Long, Integer, Integer into a JavaPairRDDTuple2Long,Long, Tuple2Integer,Integer is unrelated to mllib. Thanks, Bharath On Wed, Jun 18, 2014 at 7:14 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi Xiangrui , I'm using 1.0.0. Thanks, Bharath On 18-Jun-2014 1:43 am, Xiangrui Meng men...@gmail.com wrote: Hi Bharath, Thanks for posting the details! Which Spark version are you using? Best, Xiangrui On Tue, Jun 17, 2014 at 6:48 AM, Bharath Ravi Kumar reachb...@gmail.com wrote: Hi, (Apologies for the long mail, but it's necessary to provide sufficient details considering the number of issues faced.) I'm running into issues testing LogisticRegressionWithSGD a two node cluster (each node with 24 cores and 16G available to slaves out of 24G on the system). Here's a description of the application: The model is being trained based on categorical features x, y, and (x,y). The categorical features are mapped to binary features by converting each distinct value in the category enum into a binary feature by itself (i.e presence of that value in a record implies corresponding feature = 1, else feature = 0. So, there'd be as many distinct features as enum values) . The training vector is laid out as [x1,x2...xn,y1,y2yn,(x1,y1),(x2,y2)...(xn,yn)]. Each record in the training data has only one combination (Xk,Yk) and a label appearing in the record. Thus, the corresponding labeledpoint sparse vector would only have 3 values Xk, Yk, (Xk,Yk) set for a record. The total length of the vector (though parse) would be nearly 614000. The number of records is about 1.33 million. The records have been coalesced into 20 partitions across two nodes. The input data has not been cached. (NOTE: I do realize the records features may seem large for a two node setup, but given the memory cpu, and the fact that I'm willing to give up some turnaround time, I don't see why tasks should inexplicably fail) Additional parameters include: spark.executor.memory = 14G spark.default.parallelism = 1 spark.cores.max=20 spark.storage.memoryFraction=0.8 //No cache space required (Trying to set spark.akka.frameSize to a larger number, say, 20 didn't help either) The model training was initialized as : new LogisticRegressionWithSGD(1, maxIterations, 0.0, 0.05) However, after 4 iterations of gradient descent, the entire execution appeared to stall inexplicably. The corresponding executor details and details of the stalled stage (number 14) are as follows: MetricMin25th Median75th Max Result serialization time12 ms13 ms14 ms16 ms18 ms Duration4 s4 s5 s5 s 5 s Time spent fetching task 0 ms0 ms0 ms0 ms0 ms results Scheduler delay6 s6 s6 s6 s 12 s Stage Id 14 aggregate at GradientDescent.scala:178 Task IndexTask IDStatusLocality Level Executor Launch TimeDurationGC Result Ser TimeErrors Time 0 600 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 1 601 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 2 602 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 3 603 RUNNING PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 4 604 RUNNING PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 1.1 h 5 605 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 6 606 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 14 ms 7 607 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 2 s 12 ms 8 608 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 9 609 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 14 ms 10 610 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 15 ms 11 611 SUCCESS PROCESS_LOCAL casual.dataone.foo.bar.com 2014/06/17 10:32:27 4 s 1 s 13 ms 12 612 SUCCESS PROCESS_LOCAL serious.dataone.foo.bar.com 2014/06/17 10:32:27 5 s 1 s 18 ms 13 613 SUCCESS PROCESS_LOCAL