[jira] [Commented] (SPARK-13352) BlockFetch does not scale well on large block
[ https://issues.apache.org/jira/browse/SPARK-13352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15236426#comment-15236426 ] Zhang, Liye commented on SPARK-13352: - [~davies], the last result for 500M should be 7.8 seconds, not 7.8 min, right? > BlockFetch does not scale well on large block > - > > Key: SPARK-13352 > URL: https://issues.apache.org/jira/browse/SPARK-13352 > Project: Spark > Issue Type: Bug > Components: Block Manager, Spark Core >Reporter: Davies Liu >Assignee: Zhang, Liye >Priority: Critical > Fix For: 1.6.2, 2.0.0 > > > BlockManager.getRemoteBytes() perform poorly on large block > {code} > test("block manager") { > val N = 500 << 20 > val bm = sc.env.blockManager > val blockId = TaskResultBlockId(0) > val buffer = ByteBuffer.allocate(N) > buffer.limit(N) > bm.putBytes(blockId, buffer, StorageLevel.MEMORY_AND_DISK_SER) > val result = bm.getRemoteBytes(blockId) > assert(result.isDefined) > assert(result.get.limit() === (N)) > } > {code} > Here are runtime for different block sizes: > {code} > 50M3 seconds > 100M 7 seconds > 250M 33 seconds > 500M 2 min > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-13352) BlockFetch does not scale well on large block
[ https://issues.apache.org/jira/browse/SPARK-13352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15234488#comment-15234488 ] Zhang, Liye edited comment on SPARK-13352 at 4/11/16 5:02 AM: -- Hi [~davies], I think this JIRA is related with [SPARK-14242|https://issues.apache.org/jira/browse/SPARK-142242] and [SPARK-14290|https://issues.apache.org/jira/browse/SPARK-14290], can you test with spark master branch again to see if this issue still exists? was (Author: liyezhang556520): Hi [~davies], I think this JIRA is related with [SPARK-14242|https://issues.apache.org/jira/browse/SPARK-142242] and [SPARK-14290|https://issues.apache.org/jira/browse/SPARK-14290], can you test with spark master again to see if this issue still exists? > BlockFetch does not scale well on large block > - > > Key: SPARK-13352 > URL: https://issues.apache.org/jira/browse/SPARK-13352 > Project: Spark > Issue Type: Bug > Components: Block Manager, Spark Core >Reporter: Davies Liu >Priority: Critical > > BlockManager.getRemoteBytes() perform poorly on large block > {code} > test("block manager") { > val N = 500 << 20 > val bm = sc.env.blockManager > val blockId = TaskResultBlockId(0) > val buffer = ByteBuffer.allocate(N) > buffer.limit(N) > bm.putBytes(blockId, buffer, StorageLevel.MEMORY_AND_DISK_SER) > val result = bm.getRemoteBytes(blockId) > assert(result.isDefined) > assert(result.get.limit() === (N)) > } > {code} > Here are runtime for different block sizes: > {code} > 50M3 seconds > 100M 7 seconds > 250M 33 seconds > 500M 2 min > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-13352) BlockFetch does not scale well on large block
[ https://issues.apache.org/jira/browse/SPARK-13352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15234488#comment-15234488 ] Zhang, Liye commented on SPARK-13352: - Hi [~davies], I think this JIRA is related with [SPARK-14242|https://issues.apache.org/jira/browse/SPARK-142242] and [SPARK-14290|https://issues.apache.org/jira/browse/SPARK-14290], can you test with spark master again to see if this issue still exists? > BlockFetch does not scale well on large block > - > > Key: SPARK-13352 > URL: https://issues.apache.org/jira/browse/SPARK-13352 > Project: Spark > Issue Type: Bug > Components: Block Manager, Spark Core >Reporter: Davies Liu >Priority: Critical > > BlockManager.getRemoteBytes() perform poorly on large block > {code} > test("block manager") { > val N = 500 << 20 > val bm = sc.env.blockManager > val blockId = TaskResultBlockId(0) > val buffer = ByteBuffer.allocate(N) > buffer.limit(N) > bm.putBytes(blockId, buffer, StorageLevel.MEMORY_AND_DISK_SER) > val result = bm.getRemoteBytes(blockId) > assert(result.isDefined) > assert(result.get.limit() === (N)) > } > {code} > Here are runtime for different block sizes: > {code} > 50M3 seconds > 100M 7 seconds > 250M 33 seconds > 500M 2 min > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-14290) Fully utilize the network bandwidth for Netty RPC by avoid significant underlying memory copy
[ https://issues.apache.org/jira/browse/SPARK-14290?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-14290: Description: When netty transfer data that is not from *FileRegion*, data will be transfered as *ByteBuf*, If the data is large, there will occur significant performance issue because there is memory copy underlying in *sun.nio.ch.IOUtil.write*, the CPU is 100% used, and network is very low. We can check it by comparing *NIO* and *Netty* for *spark.shuffle.blockTransferService* in spark 1.4. NIO network bandwidth is much better than Netty. How to reproduce: {code} sc.parallelize(Array(1,2,3),3).mapPartitions(a=>Array(new Array[Double](1024 * 1024 * 50)).iterator).reduce((a,b)=> a).length {code} The root cause can referred [here|http://stackoverflow.com/questions/34493320/how-does-buffer-size-affect-nio-channel-performance]. was: When netty transfer data that is not from `FileRegion`, data will be transfered as `ByteBuf`, If the data is large, there will occur significant performance issue because there is memory copy underlying in `sun.nio.ch.IOUtil.write`, the CPU is 100% used, and network is very low. We can check it by comparing `NIO` and `Netty` for`spark.shuffle.blockTransferService` in spark 1.4. NIO network bandwidth is much better than Netty. How to reproduce: {code} sc.parallelize(Array(1,2,3),3).mapPartitions(a=>Array(new Array[Double](1024 * 1024 * 50)).iterator).reduce((a,b)=> a).length {code} The root cause can referred [here|http://stackoverflow.com/questions/34493320/how-does-buffer-size-affect-nio-channel-performance]. > Fully utilize the network bandwidth for Netty RPC by avoid significant > underlying memory copy > - > > Key: SPARK-14290 > URL: https://issues.apache.org/jira/browse/SPARK-14290 > Project: Spark > Issue Type: Improvement > Components: Input/Output, Spark Core >Affects Versions: 1.2.0, 1.3.0, 1.4.0, 1.5.0, 1.6.0, 2.0.0 >Reporter: Zhang, Liye > > When netty transfer data that is not from *FileRegion*, data will be > transfered as *ByteBuf*, If the data is large, there will occur significant > performance issue because there is memory copy underlying in > *sun.nio.ch.IOUtil.write*, the CPU is 100% used, and network is very low. We > can check it by comparing *NIO* and *Netty* for > *spark.shuffle.blockTransferService* in spark 1.4. NIO network bandwidth is > much better than Netty. > How to reproduce: > {code} > sc.parallelize(Array(1,2,3),3).mapPartitions(a=>Array(new Array[Double](1024 > * 1024 * 50)).iterator).reduce((a,b)=> a).length > {code} > The root cause can referred > [here|http://stackoverflow.com/questions/34493320/how-does-buffer-size-affect-nio-channel-performance]. > -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-14290) Fully utilize the network bandwidth for Netty RPC by avoid significant underlying memory copy
Zhang, Liye created SPARK-14290: --- Summary: Fully utilize the network bandwidth for Netty RPC by avoid significant underlying memory copy Key: SPARK-14290 URL: https://issues.apache.org/jira/browse/SPARK-14290 Project: Spark Issue Type: Improvement Components: Input/Output, Spark Core Affects Versions: 1.6.0, 1.5.0, 1.4.0, 1.3.0, 1.2.0, 2.0.0 Reporter: Zhang, Liye When netty transfer data that is not from `FileRegion`, data will be transfered as `ByteBuf`, If the data is large, there will occur significant performance issue because there is memory copy underlying in `sun.nio.ch.IOUtil.write`, the CPU is 100% used, and network is very low. We can check it by comparing `NIO` and `Netty` for`spark.shuffle.blockTransferService` in spark 1.4. NIO network bandwidth is much better than Netty. How to reproduce: {code} sc.parallelize(Array(1,2,3),3).mapPartitions(a=>Array(new Array[Double](1024 * 1024 * 50)).iterator).reduce((a,b)=> a).length {code} The root cause can referred [here|http://stackoverflow.com/questions/34493320/how-does-buffer-size-affect-nio-channel-performance]. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-14242) avoid too many copies in network when a network frame is large
Zhang, Liye created SPARK-14242: --- Summary: avoid too many copies in network when a network frame is large Key: SPARK-14242 URL: https://issues.apache.org/jira/browse/SPARK-14242 Project: Spark Issue Type: Improvement Components: Spark Core Affects Versions: 1.6.1, 1.6.0, 2.0.0 Reporter: Zhang, Liye Priority: Critical when a shuffle block size is huge, say a large array (array size more than 128MB), there will be performance issue for getting remote blocks. This is because network frame size is large, and when we are using a composite buffer, which will consolidate when the components number reaches maximum components number (default is 16) in netty underlying, performance issue will occurs. There will be too many memory copies inside netty's *compositeBuffer*. How to reproduce: {code} sc.parallelize(Array(1,2,3),3).mapPartitions(a=>Array(new Array[Double](1024 * 1024 * 50)).iterator).reduce((a,b)=> a).length {code} In this case, the serialized result size of each task is about 400MB, the result will be transferred to driver as *indirectResult*. We can see after the data transferred to driver, on driver side there will still need a lot of time to process and the 3 CPUs (in this case, parallelism is 3) are fully utilized with system call very high. And this processing time is calculated as result getting time on webUI. Such cases are very common in ML applications, which will return a large array from each executor. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Issue Comment Deleted] (SPARK-12196) Store blocks in different speed storage devices by hierarchy way
[ https://issues.apache.org/jira/browse/SPARK-12196?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-12196: Comment: was deleted (was: I am out of office with limited email access from 12/21/2015 to 12/25/2015. Sorry for slow email response. Any emergency, contact my manager (Cheng, Hao hao.ch...@intel.com). Thanks ) > Store blocks in different speed storage devices by hierarchy way > > > Key: SPARK-12196 > URL: https://issues.apache.org/jira/browse/SPARK-12196 > Project: Spark > Issue Type: New Feature > Components: Spark Core >Reporter: yucai > > *Problem* > Nowadays, users have both SSDs and HDDs. > SSDs have great performance, but capacity is small. HDDs have good capacity, > but x2-x3 lower than SSDs. > How can we get both good? > *Solution* > Our idea is to build hierarchy store: use SSDs as cache and HDDs as backup > storage. > When Spark core allocates blocks for RDD (either shuffle or RDD cache), it > gets blocks from SSDs first, and when SSD’s useable space is less than some > threshold, getting blocks from HDDs. > In our implementation, we actually go further. We support a way to build any > level hierarchy store access all storage medias (NVM, SSD, HDD etc.). > *Performance* > 1. At the best case, our solution performs the same as all SSDs. > 2. At the worst case, like all data are spilled to HDDs, no performance > regression. > 3. Compared with all HDDs, hierarchy store improves more than *_x1.86_* (it > could be higher, CPU reaches bottleneck in our test environment). > 4. Compared with Tachyon, our hierarchy store still *_x1.3_* faster. Because > we support both RDD cache and shuffle and no extra inter process > communication. > *Usage* > 1. Set the priority and threshold for each layer in > spark.storage.hierarchyStore. > {code} > spark.storage.hierarchyStore='nvm 50GB,ssd 80GB' > {code} > It builds a 3 layers hierarchy store: the 1st is "nvm", the 2nd is "sdd", all > the rest form the last layer. > 2. Configure each layer's location, user just needs put the keyword like > "nvm", "ssd", which are specified in step 1, into local dirs, like > spark.local.dir or yarn.nodemanager.local-dirs. > {code} > spark.local.dir=/mnt/nvm1,/mnt/ssd1,/mnt/ssd2,/mnt/ssd3,/mnt/disk1,/mnt/disk2,/mnt/disk3,/mnt/disk4,/mnt/others > {code} > After then, restart your Spark application, it will allocate blocks from nvm > first. > When nvm's usable space is less than 50GB, it starts to allocate from ssd. > When ssd's usable space is less than 80GB, it starts to allocate from the > last layer. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-12196) Store blocks in different speed storage devices by hierarchy way
[ https://issues.apache.org/jira/browse/SPARK-12196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15071354#comment-15071354 ] Zhang, Liye commented on SPARK-12196: - I am out of office with limited email access from 12/21/2015 to 12/25/2015. Sorry for slow email response. Any emergency, contact my manager (Cheng, Hao hao.ch...@intel.com). Thanks > Store blocks in different speed storage devices by hierarchy way > > > Key: SPARK-12196 > URL: https://issues.apache.org/jira/browse/SPARK-12196 > Project: Spark > Issue Type: New Feature > Components: Spark Core >Reporter: yucai > > *Problem* > Nowadays, users have both SSDs and HDDs. > SSDs have great performance, but capacity is small. HDDs have good capacity, > but x2-x3 lower than SSDs. > How can we get both good? > *Solution* > Our idea is to build hierarchy store: use SSDs as cache and HDDs as backup > storage. > When Spark core allocates blocks for RDD (either shuffle or RDD cache), it > gets blocks from SSDs first, and when SSD’s useable space is less than some > threshold, getting blocks from HDDs. > In our implementation, we actually go further. We support a way to build any > level hierarchy store access all storage medias (NVM, SSD, HDD etc.). > *Performance* > 1. At the best case, our solution performs the same as all SSDs. > 2. At the worst case, like all data are spilled to HDDs, no performance > regression. > 3. Compared with all HDDs, hierarchy store improves more than *_x1.86_* (it > could be higher, CPU reaches bottleneck in our test environment). > 4. Compared with Tachyon, our hierarchy store still *_x1.3_* faster. Because > we support both RDD cache and shuffle and no extra inter process > communication. > *Usage* > 1. Set the priority and threshold for each layer in > spark.storage.hierarchyStore. > {code} > spark.storage.hierarchyStore='nvm 50GB,ssd 80GB' > {code} > It builds a 3 layers hierarchy store: the 1st is "nvm", the 2nd is "sdd", all > the rest form the last layer. > 2. Configure each layer's location, user just needs put the keyword like > "nvm", "ssd", which are specified in step 1, into local dirs, like > spark.local.dir or yarn.nodemanager.local-dirs. > {code} > spark.local.dir=/mnt/nvm1,/mnt/ssd1,/mnt/ssd2,/mnt/ssd3,/mnt/disk1,/mnt/disk2,/mnt/disk3,/mnt/disk4,/mnt/others > {code} > After then, restart your Spark application, it will allocate blocks from nvm > first. > When nvm's usable space is less than 50GB, it starts to allocate from ssd. > When ssd's usable space is less than 80GB, it starts to allocate from the > last layer. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Closed] (SPARK-10608) turn off reduce tasks locality as default to avoid bad cases
[ https://issues.apache.org/jira/browse/SPARK-10608?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye closed SPARK-10608. --- Resolution: Later > turn off reduce tasks locality as default to avoid bad cases > > > Key: SPARK-10608 > URL: https://issues.apache.org/jira/browse/SPARK-10608 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Affects Versions: 1.5.0 >Reporter: Zhang, Liye >Priority: Critical > > After [SPARK-2774|https://issues.apache.org/jira/browse/SPARK-2774], which is > aiming to reduce network transform, reduce tasks will have their own locality > other than following the map side locality. This will lead to some bad cases > when there is data skew happens. In some cases, tasks will continue being > distributed on some nodes, and will never be balance distributed. > e.g. If we do not set *spark.scheduler.minRegisteredExecutorsRatio*, then the > input data will only be loaded on part of the nodes, say 4 nodes in 10 nodes. > And this will leading the first batch of the reduce tasks running on the 4 > nodes, and with many pending tasks waiting for distribution. It might be fine > if the tasks runnning for a long time, But if the tasks are running in short > time, for example, less than *spark.locality.wait*, then the locality level > will not get to lower level, and then the following batches of tasks will be > still running on the 4 nodes. Which will ending with all following tasks are > running on the 4 nodes instead of 10 nodes. Even though after several stages > the tasks may evenly distributed, however, the unbalanced tasks distribution > in the beginning will exhaust resources on some nodes first and cause GC more > frequently. Which will lead bad performance. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-9103) Tracking spark's memory usage
[ https://issues.apache.org/jira/browse/SPARK-9103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14988674#comment-14988674 ] Zhang, Liye commented on SPARK-9103: [~pwendell], [~andrewor14] would you please take a look at this JIRA and the design doc? Your comments are appreciated. So that we can move on correctly. > Tracking spark's memory usage > - > > Key: SPARK-9103 > URL: https://issues.apache.org/jira/browse/SPARK-9103 > Project: Spark > Issue Type: Umbrella > Components: Spark Core, Web UI >Reporter: Zhang, Liye > Attachments: Tracking Spark Memory Usage - Phase 1.pdf > > > Currently spark only provides little memory usage information (RDD cache on > webUI) for the executors. User have no idea on what is the memory consumption > when they are running spark applications with a lot of memory used in spark > executors. Especially when they encounter the OOM, it’s really hard to know > what is the cause of the problem. So it would be helpful to give out the > detail memory consumption information for each part of spark, so that user > can clearly have a picture of where the memory is exactly used. > The memory usage info to expose should include but not limited to shuffle, > cache, network, serializer, etc. > User can optionally choose to open this functionality since this is mainly > for debugging and tuning. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-9103) Tracking spark's memory usage
[ https://issues.apache.org/jira/browse/SPARK-9103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-9103: --- Attachment: Tracking Spark Memory Usage - Phase 1.pdf > Tracking spark's memory usage > - > > Key: SPARK-9103 > URL: https://issues.apache.org/jira/browse/SPARK-9103 > Project: Spark > Issue Type: Umbrella > Components: Spark Core, Web UI >Reporter: Zhang, Liye > Attachments: Tracking Spark Memory Usage - Phase 1.pdf > > > Currently spark only provides little memory usage information (RDD cache on > webUI) for the executors. User have no idea on what is the memory consumption > when they are running spark applications with a lot of memory used in spark > executors. Especially when they encounter the OOM, it’s really hard to know > what is the cause of the problem. So it would be helpful to give out the > detail memory consumption information for each part of spark, so that user > can clearly have a picture of where the memory is exactly used. > The memory usage info to expose should include but not limited to shuffle, > cache, network, serializer, etc. > User can optionally choose to open this functionality since this is mainly > for debugging and tuning. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-9103) Tracking spark's memory usage
[ https://issues.apache.org/jira/browse/SPARK-9103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-9103: --- Attachment: (was: Tracking Spark Memory Usage - Phase 1.pdf) > Tracking spark's memory usage > - > > Key: SPARK-9103 > URL: https://issues.apache.org/jira/browse/SPARK-9103 > Project: Spark > Issue Type: Umbrella > Components: Spark Core, Web UI >Reporter: Zhang, Liye > > Currently spark only provides little memory usage information (RDD cache on > webUI) for the executors. User have no idea on what is the memory consumption > when they are running spark applications with a lot of memory used in spark > executors. Especially when they encounter the OOM, it’s really hard to know > what is the cause of the problem. So it would be helpful to give out the > detail memory consumption information for each part of spark, so that user > can clearly have a picture of where the memory is exactly used. > The memory usage info to expose should include but not limited to shuffle, > cache, network, serializer, etc. > User can optionally choose to open this functionality since this is mainly > for debugging and tuning. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-9103) Tracking spark's memory usage
[ https://issues.apache.org/jira/browse/SPARK-9103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14908404#comment-14908404 ] Zhang, Liye commented on SPARK-9103: Hi @Imran Rashid, thanks for reviewing the doc. {quote} 1) Will the proposed design cover SPARK-9111, getting the memory when the executor dies abnormally, (esp when killed by yarn)? It seems to me the answer is "no", which is fine, that can be tackled separately, I just wanted to clarify. {quote} You are right, the answer is "no". This design is for phase 1, we can move it on later to cover [SPARK-9111|https://issues.apache.org/jira/browse/SPARK-9111]. {quote} I see the complexity of having overlapping stages, but I wonder if it could be simplified somewhat. It seems to me you just need to maintain a executorToLatestMetrics: Map[executor, metrics], and then on every stage complete, you just log them all? {quote} Since we want to reduce the number of events to log, I didn't find a way to simplify this for overlapping stages. And in the current implementation, we log all the ExectorMetrics of all the executors when executor complete. I think this can be simplified by only log ExecutorMetrics of executors that is related to the stage instead of all the executors. This will reduce a lot of events to log if there are many stages running on different executors. {quote} but it seems like there is more state & a bit more logging going on {quote} I don't quite understand, what do you mean about "*more state and more logging going on*", can you explain it further? {quote} I don't fully understand why you need to log both "CHB1" and "HB3" in your example. {quote} That is because the "CHB1" is the combined event, and "HB3" is the real event, we have to log "HB3" because there might be no heartbeat received for the stage that after "HB3" (just like stage2 in figure-1 described in the doc). And for that stage, it will use "HB3" instead of "CHB1" because "CHB1" is not the correct event it should refer to. > Tracking spark's memory usage > - > > Key: SPARK-9103 > URL: https://issues.apache.org/jira/browse/SPARK-9103 > Project: Spark > Issue Type: Umbrella > Components: Spark Core, Web UI >Reporter: Zhang, Liye > Attachments: Tracking Spark Memory Usage - Phase 1.pdf > > > Currently spark only provides little memory usage information (RDD cache on > webUI) for the executors. User have no idea on what is the memory consumption > when they are running spark applications with a lot of memory used in spark > executors. Especially when they encounter the OOM, it’s really hard to know > what is the cause of the problem. So it would be helpful to give out the > detail memory consumption information for each part of spark, so that user > can clearly have a picture of where the memory is exactly used. > The memory usage info to expose should include but not limited to shuffle, > cache, network, serializer, etc. > User can optionally choose to open this functionality since this is mainly > for debugging and tuning. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-9103) Tracking spark's memory usage
[ https://issues.apache.org/jira/browse/SPARK-9103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14908404#comment-14908404 ] Zhang, Liye edited comment on SPARK-9103 at 9/25/15 5:56 PM: - Hi [~irashid], thanks for reviewing the doc. {quote} 1) Will the proposed design cover SPARK-9111, getting the memory when the executor dies abnormally, (esp when killed by yarn)? It seems to me the answer is "no", which is fine, that can be tackled separately, I just wanted to clarify. {quote} You are right, the answer is "no". This design is for phase 1, we can move it on later to cover [SPARK-9111|https://issues.apache.org/jira/browse/SPARK-9111]. {quote} I see the complexity of having overlapping stages, but I wonder if it could be simplified somewhat. It seems to me you just need to maintain a executorToLatestMetrics: Map[executor, metrics], and then on every stage complete, you just log them all? {quote} Since we want to reduce the number of events to log, I didn't find a way to simplify this for overlapping stages. And in the current implementation, we log all the ExectorMetrics of all the executors when executor complete. I think this can be simplified by only log ExecutorMetrics of executors that is related to the stage instead of all the executors. This will reduce a lot of events to log if there are many stages running on different executors. {quote} but it seems like there is more state & a bit more logging going on {quote} I don't quite understand, what do you mean about "*more state and more logging going on*", can you explain it further? {quote} I don't fully understand why you need to log both "CHB1" and "HB3" in your example. {quote} That is because the "CHB1" is the combined event, and "HB3" is the real event, we have to log "HB3" because there might be no heartbeat received for the stage that after "HB3" (just like stage2 in figure-1 described in the doc). And for that stage, it will use "HB3" instead of "CHB1" because "CHB1" is not the correct event it should refer to. was (Author: liyezhang556520): Hi @Imran Rashid, thanks for reviewing the doc. {quote} 1) Will the proposed design cover SPARK-9111, getting the memory when the executor dies abnormally, (esp when killed by yarn)? It seems to me the answer is "no", which is fine, that can be tackled separately, I just wanted to clarify. {quote} You are right, the answer is "no". This design is for phase 1, we can move it on later to cover [SPARK-9111|https://issues.apache.org/jira/browse/SPARK-9111]. {quote} I see the complexity of having overlapping stages, but I wonder if it could be simplified somewhat. It seems to me you just need to maintain a executorToLatestMetrics: Map[executor, metrics], and then on every stage complete, you just log them all? {quote} Since we want to reduce the number of events to log, I didn't find a way to simplify this for overlapping stages. And in the current implementation, we log all the ExectorMetrics of all the executors when executor complete. I think this can be simplified by only log ExecutorMetrics of executors that is related to the stage instead of all the executors. This will reduce a lot of events to log if there are many stages running on different executors. {quote} but it seems like there is more state & a bit more logging going on {quote} I don't quite understand, what do you mean about "*more state and more logging going on*", can you explain it further? {quote} I don't fully understand why you need to log both "CHB1" and "HB3" in your example. {quote} That is because the "CHB1" is the combined event, and "HB3" is the real event, we have to log "HB3" because there might be no heartbeat received for the stage that after "HB3" (just like stage2 in figure-1 described in the doc). And for that stage, it will use "HB3" instead of "CHB1" because "CHB1" is not the correct event it should refer to. > Tracking spark's memory usage > - > > Key: SPARK-9103 > URL: https://issues.apache.org/jira/browse/SPARK-9103 > Project: Spark > Issue Type: Umbrella > Components: Spark Core, Web UI >Reporter: Zhang, Liye > Attachments: Tracking Spark Memory Usage - Phase 1.pdf > > > Currently spark only provides little memory usage information (RDD cache on > webUI) for the executors. User have no idea on what is the memory consumption > when they are running spark applications with a lot of memory used in spark > executors. Especially when they encounter the OOM, it’s really hard to know > what is the cause of the problem. So it would be helpful to give out the > detail memory consumption information for each part of spark, so that user > can clearly have a picture of where the memory is exactly used. > The memory usage info to expose should include but not limited to
[jira] [Created] (SPARK-10609) Improve task distribution strategy in taskSetManager
Zhang, Liye created SPARK-10609: --- Summary: Improve task distribution strategy in taskSetManager Key: SPARK-10609 URL: https://issues.apache.org/jira/browse/SPARK-10609 Project: Spark Issue Type: Improvement Components: Spark Core Reporter: Zhang, Liye For some bad cases, the current strategy cannot handle properly especially when reduce tasks locality is enable, We should improve the waiting strategy for pending tasks when there are still cores are free in the cluster. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-10608) turn off reduce tasks locality as default to avoid bad cases
Zhang, Liye created SPARK-10608: --- Summary: turn off reduce tasks locality as default to avoid bad cases Key: SPARK-10608 URL: https://issues.apache.org/jira/browse/SPARK-10608 Project: Spark Issue Type: Improvement Components: Spark Core Affects Versions: 1.5.0 Reporter: Zhang, Liye Priority: Critical After [SPARK-2774|https://issues.apache.org/jira/browse/SPARK-2774], which is aiming to reduce network transform, reduce tasks will have their own locality other than following the map side locality. This will lead to some bad cases when there is data skew happens. In some cases, tasks will continue being distributed on some nodes, and will never be balance distributed. e.g. If we do not set *spark.scheduler.minRegisteredExecutorsRatio*, then the input data will only be loaded on part of the nodes, say 4 nodes in 10 nodes. And this will leading the first batch of the reduce tasks running on the 4 nodes, and with many pending tasks waiting for distribution. It might be fine if the tasks runnning for a long time, But if the tasks are running in short time, for example, less than *spark.locality.wait*, then the locality level will not get to lower level, and then the following batches of tasks will be still running on the 4 nodes. Which will ending with all following tasks are running on the 4 nodes instead of 10 nodes. Even though after several stages the tasks may evenly distributed, however, the unbalanced tasks distribution in the beginning will exhaust resources on some nodes first and cause GC more frequently. Which will lead bad performance. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-9212) update Netty version to 4.0.29.Final for Netty Metrics
[ https://issues.apache.org/jira/browse/SPARK-9212?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-9212: --- Description: In Netty version 4.0.29.Final, metrics for PooledByteBufAllocator is exposed directly, so that no need to get the memory data info in a hack way. (was: In Netty version 4.0.29.Final, metrics for PooledByteBufAllocator is introduced, so that no need to get the memory data in a hack way. ) update Netty version to 4.0.29.Final for Netty Metrics Key: SPARK-9212 URL: https://issues.apache.org/jira/browse/SPARK-9212 Project: Spark Issue Type: Sub-task Components: Shuffle, Spark Core Reporter: Zhang, Liye In Netty version 4.0.29.Final, metrics for PooledByteBufAllocator is exposed directly, so that no need to get the memory data info in a hack way. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-9212) update Netty version to 4.0.29.Final for Netty Metrics
Zhang, Liye created SPARK-9212: -- Summary: update Netty version to 4.0.29.Final for Netty Metrics Key: SPARK-9212 URL: https://issues.apache.org/jira/browse/SPARK-9212 Project: Spark Issue Type: Sub-task Components: Shuffle, Spark Core Reporter: Zhang, Liye In Netty version 4.0.29.Final, metrics for PooledByteBufAllocator is introduced, so that no need to get the memory data in a hack way. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Issue Comment Deleted] (SPARK-9044) Updated RDD name does not reflect under Storage tab
[ https://issues.apache.org/jira/browse/SPARK-9044?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-9044: --- Comment: was deleted (was: Well, I think the component is correct, still it's business of Web UI) Updated RDD name does not reflect under Storage tab - Key: SPARK-9044 URL: https://issues.apache.org/jira/browse/SPARK-9044 Project: Spark Issue Type: Bug Components: Web UI Affects Versions: 1.3.1, 1.4.0 Environment: Mac OSX Reporter: Wenjie Zhang Priority: Minor I was playing the spark-shell in my macbook, here is what I did: scala val textFile = sc.textFile(/Users/jackzhang/Downloads/ProdPart.txt); scala textFile.cache scala textFile.setName(test1) scala textFile.collect scala textFile.name res10: String = test1 After this four commands, I can see the test1 RDD listed in the Storage tab. However, if I continually run following commands, nothing will happen from the Storage tab: scala textFile.setName(test2) scala textFile.cache scala textFile.collect scala textFile.name res10: String = test2 I am expecting the name of the RDD shows in Storage tab should be test2, is this a bug? -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-9044) Updated RDD name does not reflect under Storage tab
[ https://issues.apache.org/jira/browse/SPARK-9044?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14629704#comment-14629704 ] Zhang, Liye commented on SPARK-9044: Well, I think the component is correct, still it's business of Web UI Updated RDD name does not reflect under Storage tab - Key: SPARK-9044 URL: https://issues.apache.org/jira/browse/SPARK-9044 Project: Spark Issue Type: Bug Components: Web UI Affects Versions: 1.3.1, 1.4.0 Environment: Mac OSX Reporter: Wenjie Zhang Priority: Minor I was playing the spark-shell in my macbook, here is what I did: scala val textFile = sc.textFile(/Users/jackzhang/Downloads/ProdPart.txt); scala textFile.cache scala textFile.setName(test1) scala textFile.collect scala textFile.name res10: String = test1 After this four commands, I can see the test1 RDD listed in the Storage tab. However, if I continually run following commands, nothing will happen from the Storage tab: scala textFile.setName(test2) scala textFile.cache scala textFile.collect scala textFile.name res10: String = test2 I am expecting the name of the RDD shows in Storage tab should be test2, is this a bug? -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-9044) Updated RDD name does not reflect under Storage tab
[ https://issues.apache.org/jira/browse/SPARK-9044?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14629703#comment-14629703 ] Zhang, Liye commented on SPARK-9044: Well, I think the component is correct, still it's business of Web UI Updated RDD name does not reflect under Storage tab - Key: SPARK-9044 URL: https://issues.apache.org/jira/browse/SPARK-9044 Project: Spark Issue Type: Bug Components: Web UI Affects Versions: 1.3.1, 1.4.0 Environment: Mac OSX Reporter: Wenjie Zhang Priority: Minor I was playing the spark-shell in my macbook, here is what I did: scala val textFile = sc.textFile(/Users/jackzhang/Downloads/ProdPart.txt); scala textFile.cache scala textFile.setName(test1) scala textFile.collect scala textFile.name res10: String = test1 After this four commands, I can see the test1 RDD listed in the Storage tab. However, if I continually run following commands, nothing will happen from the Storage tab: scala textFile.setName(test2) scala textFile.cache scala textFile.collect scala textFile.name res10: String = test2 I am expecting the name of the RDD shows in Storage tab should be test2, is this a bug? -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-9104) expose network layer memory usage in shuffle part
Zhang, Liye created SPARK-9104: -- Summary: expose network layer memory usage in shuffle part Key: SPARK-9104 URL: https://issues.apache.org/jira/browse/SPARK-9104 Project: Spark Issue Type: Sub-task Components: Spark Core Reporter: Zhang, Liye The default network transportation is netty, and when transfering blocks for shuffle, the network layer will consume a decent size of memory, we shall collect the memory usage of this part and expose it. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-9105) Add an additional WebUi Tab for Memory Usage
Zhang, Liye created SPARK-9105: -- Summary: Add an additional WebUi Tab for Memory Usage Key: SPARK-9105 URL: https://issues.apache.org/jira/browse/SPARK-9105 Project: Spark Issue Type: Sub-task Components: Web UI Reporter: Zhang, Liye Add a spark a WebUI Tab for Memory usage, the Tab should expose memory usage status in different spark components. It should show the summary for each executors and may also the details for each tasks. On this Tab, there may be some duplicated information with Storage Tab, but they are in different showing format, take RDD cache for example, the RDD cached size showed on Storage Tab is indexed with RDD name, while on memory usage Tab, the RDD can be indexed with Executors, or tasks. Also, the two Tabs can share some same Web Pages. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-9103) Tracking spark's memory usage
Zhang, Liye created SPARK-9103: -- Summary: Tracking spark's memory usage Key: SPARK-9103 URL: https://issues.apache.org/jira/browse/SPARK-9103 Project: Spark Issue Type: Umbrella Components: Spark Core, Web UI Reporter: Zhang, Liye Currently spark only provides little memory usage information (RDD cache on webUI) for the executors. User have no idea on what is the memory consumption when they are running spark applications with a lot of memory used in spark executors. Especially when they encounter the OOM, it’s really hard to know what is the cause of the problem. So it would be helpful to give out the detail memory consumption information for each part of spark, so that user can clearly have a picture of where the memory is exactly used. The memory usage info to expose should include but not limited to shuffle, cache, network, serializer, etc. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-9105) Add an additional WebUI Tab for Memory Usage
[ https://issues.apache.org/jira/browse/SPARK-9105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-9105: --- Summary: Add an additional WebUI Tab for Memory Usage (was: Add an additional WebUi Tab for Memory Usage) Add an additional WebUI Tab for Memory Usage Key: SPARK-9105 URL: https://issues.apache.org/jira/browse/SPARK-9105 Project: Spark Issue Type: Sub-task Components: Web UI Reporter: Zhang, Liye Add a spark a WebUI Tab for Memory usage, the Tab should expose memory usage status in different spark components. It should show the summary for each executors and may also the details for each tasks. On this Tab, there may be some duplicated information with Storage Tab, but they are in different showing format, take RDD cache for example, the RDD cached size showed on Storage Tab is indexed with RDD name, while on memory usage Tab, the RDD can be indexed with Executors, or tasks. Also, the two Tabs can share some same Web Pages. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-9106) Log the memory usage info into history server
Zhang, Liye created SPARK-9106: -- Summary: Log the memory usage info into history server Key: SPARK-9106 URL: https://issues.apache.org/jira/browse/SPARK-9106 Project: Spark Issue Type: Sub-task Reporter: Zhang, Liye Save the memory usage info as eventLog, and can be traced from history server. So that user can make an offline analysis. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-9107) Add more information on Web UI for memory Usage
Zhang, Liye created SPARK-9107: -- Summary: Add more information on Web UI for memory Usage Key: SPARK-9107 URL: https://issues.apache.org/jira/browse/SPARK-9107 Project: Spark Issue Type: Sub-task Components: Web UI Reporter: Zhang, Liye In [spark-9104|https://issues.apache.org/jira/browse/SPARK-9104], the memory usage is showed as running time memory usage, which means, we can only see the current status of the memory consumption, just in the same way how the Storage Tab works, it would not store the previous status, and the same situation for history server WebUI. The target for this issue is to show in different finished stages or in different finished jobs, what is the memory usage status when/before the job/stage completes. Also we can give out the maximum and minimum memory size used during each job/stage. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-9103) Tracking spark's memory usage
[ https://issues.apache.org/jira/browse/SPARK-9103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-9103: --- Description: Currently spark only provides little memory usage information (RDD cache on webUI) for the executors. User have no idea on what is the memory consumption when they are running spark applications with a lot of memory used in spark executors. Especially when they encounter the OOM, it’s really hard to know what is the cause of the problem. So it would be helpful to give out the detail memory consumption information for each part of spark, so that user can clearly have a picture of where the memory is exactly used. The memory usage info to expose should include but not limited to shuffle, cache, network, serializer, etc. User can optionally choose to open this functionality since this is mainly for debugging and tuning. was: Currently spark only provides little memory usage information (RDD cache on webUI) for the executors. User have no idea on what is the memory consumption when they are running spark applications with a lot of memory used in spark executors. Especially when they encounter the OOM, it’s really hard to know what is the cause of the problem. So it would be helpful to give out the detail memory consumption information for each part of spark, so that user can clearly have a picture of where the memory is exactly used. The memory usage info to expose should include but not limited to shuffle, cache, network, serializer, etc. Tracking spark's memory usage - Key: SPARK-9103 URL: https://issues.apache.org/jira/browse/SPARK-9103 Project: Spark Issue Type: Umbrella Components: Spark Core, Web UI Reporter: Zhang, Liye Currently spark only provides little memory usage information (RDD cache on webUI) for the executors. User have no idea on what is the memory consumption when they are running spark applications with a lot of memory used in spark executors. Especially when they encounter the OOM, it’s really hard to know what is the cause of the problem. So it would be helpful to give out the detail memory consumption information for each part of spark, so that user can clearly have a picture of where the memory is exactly used. The memory usage info to expose should include but not limited to shuffle, cache, network, serializer, etc. User can optionally choose to open this functionality since this is mainly for debugging and tuning. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-9108) Expose Kryo serializer buffer size
Zhang, Liye created SPARK-9108: -- Summary: Expose Kryo serializer buffer size Key: SPARK-9108 URL: https://issues.apache.org/jira/browse/SPARK-9108 Project: Spark Issue Type: Sub-task Components: Spark Core Reporter: Zhang, Liye Spark serializer may also use a plenty of memory, for Kryo default, the buffer is upto 2GB. It would be better to expose how much memory is used by this part. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-9111) Dumping the memory info when an executor dies abnormally
Zhang, Liye created SPARK-9111: -- Summary: Dumping the memory info when an executor dies abnormally Key: SPARK-9111 URL: https://issues.apache.org/jira/browse/SPARK-9111 Project: Spark Issue Type: Sub-task Components: Spark Core Reporter: Zhang, Liye When an executor is not normally finished, we shall give out it's memory dump info right before the JVM shutting down. So that if the executor is killed because of OOM, we can easily checkout how is the memory used and which part cause the OOM. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-9107) Include memory usage for each job stage
[ https://issues.apache.org/jira/browse/SPARK-9107?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-9107: --- Summary: Include memory usage for each job stage (was: Add more information on Web UI for memory Usage) Include memory usage for each job stage - Key: SPARK-9107 URL: https://issues.apache.org/jira/browse/SPARK-9107 Project: Spark Issue Type: Sub-task Components: Web UI Reporter: Zhang, Liye In [spark-9104|https://issues.apache.org/jira/browse/SPARK-9104], the memory usage is showed as running time memory usage, which means, we can only see the current status of the memory consumption, just in the same way how the Storage Tab works, it would not store the previous status, and the same situation for history server WebUI. The target for this issue is to show in different finished stages or in different finished jobs, what is the memory usage status when/before the job/stage completes. Also we can give out the maximum and minimum memory size used during each job/stage. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-9107) Include memory usage for each job stage
[ https://issues.apache.org/jira/browse/SPARK-9107?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-9107: --- Component/s: Spark Core Include memory usage for each job stage - Key: SPARK-9107 URL: https://issues.apache.org/jira/browse/SPARK-9107 Project: Spark Issue Type: Sub-task Components: Spark Core, Web UI Reporter: Zhang, Liye In [spark-9104|https://issues.apache.org/jira/browse/SPARK-9104], the memory usage is showed as running time memory usage, which means, we can only see the current status of the memory consumption, just in the same way how the Storage Tab works, it would not store the previous status, and the same situation for history server WebUI. The target for this issue is to show in different finished stages or in different finished jobs, what is the memory usage status when/before the job/stage completes. Also we can give out the maximum and minimum memory size used during each job/stage. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-9111) Dumping the memory info when an executor dies abnormally
[ https://issues.apache.org/jira/browse/SPARK-9111?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14630648#comment-14630648 ] Zhang, Liye commented on SPARK-9111: Hi [~srowen], the memory dump mentioned here is about the spark memory usage, which is related with umbrella [SPARK-9103|https://issues.apache.org/jira/browse/SPARK-9103], not the HeapDump, since we want to know what is the memory status for different spark component. It's not easy to get how much memory used for a specific spark component directly from the HeapDump, right? Dumping the memory info when an executor dies abnormally Key: SPARK-9111 URL: https://issues.apache.org/jira/browse/SPARK-9111 Project: Spark Issue Type: Sub-task Components: Spark Core Reporter: Zhang, Liye Priority: Minor When an executor is not normally finished, we shall give out it's memory dump info right before the JVM shutting down. So that if the executor is killed because of OOM, we can easily checkout how is the memory used and which part cause the OOM. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-9044) Updated RDD name does not reflect under Storage tab
[ https://issues.apache.org/jira/browse/SPARK-9044?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14629119#comment-14629119 ] Zhang, Liye commented on SPARK-9044: It doesn't matter where *RDDInfo* belongs to which component. It's just a class contain's RDD infomation and Web UI uses this class. The info showed on Web UI is based on *sparkListener*, and only if there are changes/updates of the RDD status, the updated information will be posted to the listeners. And here, if you just change the RDD name, the CacheManager would not think it need to update anything, and this is where the bug locates. For spark-shell, it is the driver, and it knows the RDD name's change, the spark driver and the Web UI are two separated systems. That's why you can see the update from spark-shell but cannot see it from Web UI. Updated RDD name does not reflect under Storage tab - Key: SPARK-9044 URL: https://issues.apache.org/jira/browse/SPARK-9044 Project: Spark Issue Type: Bug Components: Web UI Affects Versions: 1.3.1, 1.4.0 Environment: Mac OSX Reporter: Wenjie Zhang Priority: Minor I was playing the spark-shell in my macbook, here is what I did: scala val textFile = sc.textFile(/Users/jackzhang/Downloads/ProdPart.txt); scala textFile.cache scala textFile.setName(test1) scala textFile.collect scala textFile.name res10: String = test1 After this four commands, I can see the test1 RDD listed in the Storage tab. However, if I continually run following commands, nothing will happen from the Storage tab: scala textFile.setName(test2) scala textFile.cache scala textFile.collect scala textFile.name res10: String = test2 I am expecting the name of the RDD shows in Storage tab should be test2, is this a bug? -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-9044) Updated RDD name does not reflect under Storage tab
[ https://issues.apache.org/jira/browse/SPARK-9044?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14627472#comment-14627472 ] Zhang, Liye commented on SPARK-9044: It's a bug, since RDD is identified by rdd.id and it is immutable, if we only change the name, the RDD is stay the same (cached), and the blockManager would not regard that there is blocks to update, so the metrics would not get updated. So the RDDInfo will not be updated also. Updated RDD name does not reflect under Storage tab - Key: SPARK-9044 URL: https://issues.apache.org/jira/browse/SPARK-9044 Project: Spark Issue Type: Bug Components: Web UI Affects Versions: 1.3.1, 1.4.0 Environment: Mac OSX Reporter: Wenjie Zhang Priority: Minor I was playing the spark-shell in my macbook, here is what I did: scala val textFile = sc.textFile(/Users/jackzhang/Downloads/ProdPart.txt); scala textFile.cache scala textFile.setName(test1) scala textFile.collect scala textFile.name res10: String = test1 After this four commands, I can see the test1 RDD listed in the Storage tab. However, if I continually run following commands, nothing will happen from the Storage tab: scala textFile.setName(test2) scala textFile.cache scala textFile.collect scala textFile.name res10: String = test2 I am expecting the name of the RDD shows in Storage tab should be test2, is this a bug? -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-9025) Storage tab shows no blocks for cached RDDs
[ https://issues.apache.org/jira/browse/SPARK-9025?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14627487#comment-14627487 ] Zhang, Liye commented on SPARK-9025: I cannot reproduce it either, the result is the same with [~srowen] Storage tab shows no blocks for cached RDDs --- Key: SPARK-9025 URL: https://issues.apache.org/jira/browse/SPARK-9025 Project: Spark Issue Type: Bug Components: Web UI Affects Versions: 1.5.0 Reporter: Andrew Or Simple repro: sc.parallelize(1 to 10).cache().count(), go to storage tab. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6107) event log file ends with .inprogress should be able to display on webUI for standalone mode
[ https://issues.apache.org/jira/browse/SPARK-6107?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14575538#comment-14575538 ] Zhang, Liye commented on SPARK-6107: [~octavian], this is not in 1.3.1, it is fixed in 1.4.0 event log file ends with .inprogress should be able to display on webUI for standalone mode --- Key: SPARK-6107 URL: https://issues.apache.org/jira/browse/SPARK-6107 Project: Spark Issue Type: Bug Components: Web UI Affects Versions: 1.2.1 Reporter: Zhang, Liye Assignee: Zhang, Liye Fix For: 1.4.0 when application is finished running abnormally (Ctrl + c for example), the history event log file is still ends with *.inprogress* suffix. And the application state can not be showed on webUI, User can just see *Application history not foud , Application xxx is still in progress*. User should also can see the status of the abnormal finished applications. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-7865) Hadoop Filesystem for eventlog closed before sparkContext stopped
[ https://issues.apache.org/jira/browse/SPARK-7865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14560266#comment-14560266 ] Zhang, Liye commented on SPARK-7865: Thank [~vanzin]'s reply, I'm just thinking about why Hadoop filesystem closed before Spark JVM stoping. I think the reason can be found in description of PR [#5560|https://github.com/apache/spark/pull/5560]. [~srowen], sorry for opening one more duplicated JIRA, I'll take care of it next time. Hadoop Filesystem for eventlog closed before sparkContext stopped - Key: SPARK-7865 URL: https://issues.apache.org/jira/browse/SPARK-7865 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.4.0 Reporter: Zhang, Liye After [SPARK-3090|https://issues.apache.org/jira/browse/SPARK-3090] (patch [#5969|https://github.com/apache/spark/pull/5696]), SparkContext will be automatically stop if user forget to. While when shutdownhook is called, Eventlog will give out following exception for flushing content: {noformat} 15/05/26 17:40:38 INFO spark.SparkContext: Invoking stop() from shutdown hook 15/05/26 17:40:38 ERROR scheduler.LiveListenerBus: Listener EventLoggingListener threw an exception java.lang.reflect.InvocationTargetException at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:144) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:144) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.EventLoggingListener.logEvent(EventLoggingListener.scala:144) at org.apache.spark.scheduler.EventLoggingListener.onApplicationEnd(EventLoggingListener.scala:188) at org.apache.spark.scheduler.SparkListenerBus$class.onPostEvent(SparkListenerBus.scala:54) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:56) at org.apache.spark.util.AsynchronousListenerBus.postToAll(AsynchronousListenerBus.scala:37) at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1.apply$mcV$sp(AsynchronousListenerBus.scala:79) at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1180) at org.apache.spark.util.AsynchronousListenerBus$$anon$1.run(AsynchronousListenerBus.scala:63) Caused by: java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:323) at org.apache.hadoop.hdfs.DFSClient.access$1200(DFSClient.java:78) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.sync(DFSClient.java:3877) at org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:97) ... 16 more {noformat} And exception for stopping: {noformat} 15/05/26 17:40:39 INFO cluster.SparkDeploySchedulerBackend: Asking each executor to shut down 15/05/26 17:40:39 ERROR util.Utils: Uncaught exception in thread Spark Shutdown Hook java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:323) at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1057) at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:554) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:788) at org.apache.spark.scheduler.EventLoggingListener.stop(EventLoggingListener.scala:209) at org.apache.spark.SparkContext$$anonfun$stop$5.apply(SparkContext.scala:1515) at org.apache.spark.SparkContext$$anonfun$stop$5.apply(SparkContext.scala:1515) at scala.Option.foreach(Option.scala:236) at org.apache.spark.SparkContext.stop(SparkContext.scala:1515) at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:527) at org.apache.spark.util.SparkShutdownHook.run(Utils.scala:2211) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2181) at
[jira] [Created] (SPARK-7865) Hadoop Filesystem for eventlog closed before sparkContext stopped
Zhang, Liye created SPARK-7865: -- Summary: Hadoop Filesystem for eventlog closed before sparkContext stopped Key: SPARK-7865 URL: https://issues.apache.org/jira/browse/SPARK-7865 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.4.0 Reporter: Zhang, Liye After [SPARK-3090|https://issues.apache.org/jira/browse/SPARK-6986] (patch [#5969|https://github.com/apache/spark/pull/5696]), SparkContext will be automatically stop if user forget to. While when shutdownhook is called, Eventlog will give out following exception for flushing content: {noformat} 15/05/26 17:40:38 INFO spark.SparkContext: Invoking stop() from shutdown hook 15/05/26 17:40:38 ERROR scheduler.LiveListenerBus: Listener EventLoggingListener threw an exception java.lang.reflect.InvocationTargetException at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:144) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:144) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.EventLoggingListener.logEvent(EventLoggingListener.scala:144) at org.apache.spark.scheduler.EventLoggingListener.onApplicationEnd(EventLoggingListener.scala:188) at org.apache.spark.scheduler.SparkListenerBus$class.onPostEvent(SparkListenerBus.scala:54) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:56) at org.apache.spark.util.AsynchronousListenerBus.postToAll(AsynchronousListenerBus.scala:37) at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1.apply$mcV$sp(AsynchronousListenerBus.scala:79) at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1180) at org.apache.spark.util.AsynchronousListenerBus$$anon$1.run(AsynchronousListenerBus.scala:63) Caused by: java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:323) at org.apache.hadoop.hdfs.DFSClient.access$1200(DFSClient.java:78) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.sync(DFSClient.java:3877) at org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:97) ... 16 more {noformat} And exception for stopping: {noformat} 15/05/26 17:40:39 INFO cluster.SparkDeploySchedulerBackend: Asking each executor to shut down 15/05/26 17:40:39 ERROR util.Utils: Uncaught exception in thread Spark Shutdown Hook java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:323) at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1057) at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:554) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:788) at org.apache.spark.scheduler.EventLoggingListener.stop(EventLoggingListener.scala:209) at org.apache.spark.SparkContext$$anonfun$stop$5.apply(SparkContext.scala:1515) at org.apache.spark.SparkContext$$anonfun$stop$5.apply(SparkContext.scala:1515) at scala.Option.foreach(Option.scala:236) at org.apache.spark.SparkContext.stop(SparkContext.scala:1515) at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:527) at org.apache.spark.util.SparkShutdownHook.run(Utils.scala:2211) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2181) at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1732) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2181) at scala.util.Try$.apply(Try.scala:161) at org.apache.spark.util.SparkShutdownHookManager.runAll(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anon$6.run(Utils.scala:2163) at
[jira] [Commented] (SPARK-7865) Hadoop Filesystem for eventlog closed before sparkContext stopped
[ https://issues.apache.org/jira/browse/SPARK-7865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14558929#comment-14558929 ] Zhang, Liye commented on SPARK-7865: @[~vanzin], @[~srowen] Hadoop Filesystem for eventlog closed before sparkContext stopped - Key: SPARK-7865 URL: https://issues.apache.org/jira/browse/SPARK-7865 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.4.0 Reporter: Zhang, Liye After [SPARK-3090|https://issues.apache.org/jira/browse/SPARK-3090] (patch [#5969|https://github.com/apache/spark/pull/5696]), SparkContext will be automatically stop if user forget to. While when shutdownhook is called, Eventlog will give out following exception for flushing content: {noformat} 15/05/26 17:40:38 INFO spark.SparkContext: Invoking stop() from shutdown hook 15/05/26 17:40:38 ERROR scheduler.LiveListenerBus: Listener EventLoggingListener threw an exception java.lang.reflect.InvocationTargetException at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:144) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:144) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.EventLoggingListener.logEvent(EventLoggingListener.scala:144) at org.apache.spark.scheduler.EventLoggingListener.onApplicationEnd(EventLoggingListener.scala:188) at org.apache.spark.scheduler.SparkListenerBus$class.onPostEvent(SparkListenerBus.scala:54) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:56) at org.apache.spark.util.AsynchronousListenerBus.postToAll(AsynchronousListenerBus.scala:37) at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1.apply$mcV$sp(AsynchronousListenerBus.scala:79) at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1180) at org.apache.spark.util.AsynchronousListenerBus$$anon$1.run(AsynchronousListenerBus.scala:63) Caused by: java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:323) at org.apache.hadoop.hdfs.DFSClient.access$1200(DFSClient.java:78) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.sync(DFSClient.java:3877) at org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:97) ... 16 more {noformat} And exception for stopping: {noformat} 15/05/26 17:40:39 INFO cluster.SparkDeploySchedulerBackend: Asking each executor to shut down 15/05/26 17:40:39 ERROR util.Utils: Uncaught exception in thread Spark Shutdown Hook java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:323) at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1057) at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:554) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:788) at org.apache.spark.scheduler.EventLoggingListener.stop(EventLoggingListener.scala:209) at org.apache.spark.SparkContext$$anonfun$stop$5.apply(SparkContext.scala:1515) at org.apache.spark.SparkContext$$anonfun$stop$5.apply(SparkContext.scala:1515) at scala.Option.foreach(Option.scala:236) at org.apache.spark.SparkContext.stop(SparkContext.scala:1515) at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:527) at org.apache.spark.util.SparkShutdownHook.run(Utils.scala:2211) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2181) at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1732) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2181)
[jira] [Updated] (SPARK-7865) Hadoop Filesystem for eventlog closed before sparkContext stopped
[ https://issues.apache.org/jira/browse/SPARK-7865?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-7865: --- Description: After [SPARK-3090|https://issues.apache.org/jira/browse/SPARK-3090] (patch [#5969|https://github.com/apache/spark/pull/5696]), SparkContext will be automatically stop if user forget to. While when shutdownhook is called, Eventlog will give out following exception for flushing content: {noformat} 15/05/26 17:40:38 INFO spark.SparkContext: Invoking stop() from shutdown hook 15/05/26 17:40:38 ERROR scheduler.LiveListenerBus: Listener EventLoggingListener threw an exception java.lang.reflect.InvocationTargetException at sun.reflect.GeneratedMethodAccessor20.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:144) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:144) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.EventLoggingListener.logEvent(EventLoggingListener.scala:144) at org.apache.spark.scheduler.EventLoggingListener.onApplicationEnd(EventLoggingListener.scala:188) at org.apache.spark.scheduler.SparkListenerBus$class.onPostEvent(SparkListenerBus.scala:54) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:56) at org.apache.spark.util.AsynchronousListenerBus.postToAll(AsynchronousListenerBus.scala:37) at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1.apply$mcV$sp(AsynchronousListenerBus.scala:79) at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1180) at org.apache.spark.util.AsynchronousListenerBus$$anon$1.run(AsynchronousListenerBus.scala:63) Caused by: java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:323) at org.apache.hadoop.hdfs.DFSClient.access$1200(DFSClient.java:78) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.sync(DFSClient.java:3877) at org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:97) ... 16 more {noformat} And exception for stopping: {noformat} 15/05/26 17:40:39 INFO cluster.SparkDeploySchedulerBackend: Asking each executor to shut down 15/05/26 17:40:39 ERROR util.Utils: Uncaught exception in thread Spark Shutdown Hook java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:323) at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1057) at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:554) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:788) at org.apache.spark.scheduler.EventLoggingListener.stop(EventLoggingListener.scala:209) at org.apache.spark.SparkContext$$anonfun$stop$5.apply(SparkContext.scala:1515) at org.apache.spark.SparkContext$$anonfun$stop$5.apply(SparkContext.scala:1515) at scala.Option.foreach(Option.scala:236) at org.apache.spark.SparkContext.stop(SparkContext.scala:1515) at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:527) at org.apache.spark.util.SparkShutdownHook.run(Utils.scala:2211) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2181) at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1732) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2181) at scala.util.Try$.apply(Try.scala:161) at org.apache.spark.util.SparkShutdownHookManager.runAll(Utils.scala:2181) at org.apache.spark.util.SparkShutdownHookManager$$anon$6.run(Utils.scala:2163) at java.lang.Thread.run(Thread.java:745) {noformat} The Hadoop version is 1.2.1. I'm wondering how the hadoop filesystem closed while spark doesn't explicitly calling the close() API. was: After
[jira] [Created] (SPARK-7854) refine Kryo configuration limits test
Zhang, Liye created SPARK-7854: -- Summary: refine Kryo configuration limits test Key: SPARK-7854 URL: https://issues.apache.org/jira/browse/SPARK-7854 Project: Spark Issue Type: Test Components: Spark Core, Tests Affects Versions: 1.4.0 Reporter: Zhang, Liye Priority: Minor refine the code style, make it more abstraction. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6197) handle json parse exception for eventlog file not finished writing
[ https://issues.apache.org/jira/browse/SPARK-6197?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553600#comment-14553600 ] Zhang, Liye commented on SPARK-6197: Hi [~Xia Hu], This Patch only solve the problem for standalone Mode, for Yarn mode history server, Please refer to patch in [SPARK-6314|https://issues.apache.org/jira/browse/SPARK-6314] handle json parse exception for eventlog file not finished writing --- Key: SPARK-6197 URL: https://issues.apache.org/jira/browse/SPARK-6197 Project: Spark Issue Type: Bug Components: Web UI Affects Versions: 1.3.0 Reporter: Zhang, Liye Assignee: Zhang, Liye Priority: Minor Labels: backport-needed Fix For: 1.4.0 This is a following JIRA for [SPARK-6107|https://issues.apache.org/jira/browse/SPARK-6107]. In [SPARK-6107|https://issues.apache.org/jira/browse/SPARK-6107], webUI can display event log files that with suffix *.inprogress*. However, the eventlog file may be not finished writing for some abnormal cases (e.g. Ctrl+C), In which case, the file maybe truncated in the last line, leading to the line being not in valid Json format. Which will cause Json parse exception when reading the file. For this case, we can just ignore the last line content, since the history for abnormal cases showed on web is only a reference for user, it can demonstrate the past status of the app before terminated abnormally (we can not guarantee the history can show exactly the last moment when app encounter the abnormal situation). -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-7392) Kryo buffer size can not be larger than 2M
Zhang, Liye created SPARK-7392: -- Summary: Kryo buffer size can not be larger than 2M Key: SPARK-7392 URL: https://issues.apache.org/jira/browse/SPARK-7392 Project: Spark Issue Type: Bug Components: Spark Core Reporter: Zhang, Liye Priority: Critical when set *spark.kryoserializer.buffer* larger than 2048k, *IllegalArgumentException* will be thrown. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-7189) History server will always reload the same file even when no log file is updated
[ https://issues.apache.org/jira/browse/SPARK-7189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14520694#comment-14520694 ] Zhang, Liye commented on SPARK-7189: *=* is always need if we use timestamp as one of the criteria. History server will always reload the same file even when no log file is updated Key: SPARK-7189 URL: https://issues.apache.org/jira/browse/SPARK-7189 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhang, Liye Priority: Minor History server will check every log file with it's modification time. It will reload the file if the file's modification time is later or equal to the latest modification time it remembered. So it will reload the same file(s) periodically if the file(s) with the latest modification time even if there is nothing change. This is not necessary. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-7189) History server will always reload the same file even when no log file is updated
[ https://issues.apache.org/jira/browse/SPARK-7189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14520691#comment-14520691 ] Zhang, Liye commented on SPARK-7189: Yes, we can use file size to monitor the file change, that works for *write* operations. And if we introduce file size, we'll need a hashmap to maintain the information, and this hash map can also check whether the file is renamed or not. History server will always reload the same file even when no log file is updated Key: SPARK-7189 URL: https://issues.apache.org/jira/browse/SPARK-7189 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhang, Liye Priority: Minor History server will check every log file with it's modification time. It will reload the file if the file's modification time is later or equal to the latest modification time it remembered. So it will reload the same file(s) periodically if the file(s) with the latest modification time even if there is nothing change. This is not necessary. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-7189) History server will always reload the same file even when no log file is updated
[ https://issues.apache.org/jira/browse/SPARK-7189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14520697#comment-14520697 ] Zhang, Liye commented on SPARK-7189: Just redundant work, and we can leave it. And I marked this issue as *minor* History server will always reload the same file even when no log file is updated Key: SPARK-7189 URL: https://issues.apache.org/jira/browse/SPARK-7189 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhang, Liye Priority: Minor History server will check every log file with it's modification time. It will reload the file if the file's modification time is later or equal to the latest modification time it remembered. So it will reload the same file(s) periodically if the file(s) with the latest modification time even if there is nothing change. This is not necessary. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-7189) History server will always reload the same file even when no log file is updated
[ https://issues.apache.org/jira/browse/SPARK-7189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14517136#comment-14517136 ] Zhang, Liye commented on SPARK-7189: Yes, I think the current solution is a tradeoff, we can not simply changing the = to which will cause other problems. Anyway, I haven't think up any other solution yet, maybe others have some novel/nice ideas. History server will always reload the same file even when no log file is updated Key: SPARK-7189 URL: https://issues.apache.org/jira/browse/SPARK-7189 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhang, Liye Priority: Minor History server will check every log file with it's modification time. It will reload the file if the file's modification time is later or equal to the latest modification time it remembered. So it will reload the same file(s) periodically if the file(s) with the latest modification time even if there is nothing change. This is not necessary. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-7189) History server will always reload the same file even when no log file is updated
[ https://issues.apache.org/jira/browse/SPARK-7189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14518721#comment-14518721 ] Zhang, Liye commented on SPARK-7189: Hi [~vanzin], I think using timestamp is not that precise. This method is very similar with the way using modification time. There will always be situations that several operations finished within very short time (say less than 1 millisecond or even shorter). So timestamp and modification time can not be trusted. The target is to get the status change of the files, including contents change (write operation) and permission change (rename operation). `Inotify` can get the change but it's not available in HDFS before version 2.7. One way to tell the change is to set one flag after each operation and reset the flag after reloading the file. But this will make the code really ugly, a bad option. History server will always reload the same file even when no log file is updated Key: SPARK-7189 URL: https://issues.apache.org/jira/browse/SPARK-7189 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhang, Liye Priority: Minor History server will check every log file with it's modification time. It will reload the file if the file's modification time is later or equal to the latest modification time it remembered. So it will reload the same file(s) periodically if the file(s) with the latest modification time even if there is nothing change. This is not necessary. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-7189) History server will always reload the same file if no log file is updated
Zhang, Liye created SPARK-7189: -- Summary: History server will always reload the same file if no log file is updated Key: SPARK-7189 URL: https://issues.apache.org/jira/browse/SPARK-7189 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhang, Liye Priority: Minor History server will check every log file with it's modification time. It will reload the file if the file's modification time is later or equal to the latest modification time it remembered. So it will reload the same file(s) periodically if the file(s) with the latest modification time even if there is nothing change. This is not necessary. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-7189) History server will always reload the same file even when no log file is updated
[ https://issues.apache.org/jira/browse/SPARK-7189?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-7189: --- Summary: History server will always reload the same file even when no log file is updated (was: History server will always reload the same file if no log file is updated) History server will always reload the same file even when no log file is updated Key: SPARK-7189 URL: https://issues.apache.org/jira/browse/SPARK-7189 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhang, Liye Priority: Minor History server will check every log file with it's modification time. It will reload the file if the file's modification time is later or equal to the latest modification time it remembered. So it will reload the same file(s) periodically if the file(s) with the latest modification time even if there is nothing change. This is not necessary. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-6314) Failed to load application log data from FileStatus
[ https://issues.apache.org/jira/browse/SPARK-6314?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14516436#comment-14516436 ] Zhang, Liye commented on SPARK-6314: Hi [~srowen], this issue can be duplicate with [SPARK-6197|https://issues.apache.org/jira/browse/SPARK-6197], but this issue is not resolved yet. This issue is related to HistoryServer. and [SPARK-6197|https://issues.apache.org/jira/browse/SPARK-6197] only solved the case in eventlog history for the standalone mode. Failed to load application log data from FileStatus --- Key: SPARK-6314 URL: https://issues.apache.org/jira/browse/SPARK-6314 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhichao Zhang There are some errors in history server event-log directory while a job is running: {quote} com.fasterxml.jackson.core.JsonParseException: Unexpected end-of-input: was expecting closing '' for name at com.fasterxml.jackson.core.JsonParser._constructError(JsonParser.java:1419) at com.fasterxml.jackson.core.json.ReaderBasedJsonParser._parseName2(ReaderBasedJsonParser.java:1284) at com.fasterxml.jackson.core.json.ReaderBasedJsonParser._parseName(ReaderBasedJsonParser.java:1268) at com.fasterxml.jackson.core.json.ReaderBasedJsonParser.nextToken(ReaderBasedJsonParser.java:618) at org.json4s.jackson.JValueDeserializer.deserialize(JValueDeserializer.scala:43) at org.json4s.jackson.JValueDeserializer.deserialize(JValueDeserializer.scala:35) at org.json4s.jackson.JValueDeserializer.deserialize(JValueDeserializer.scala:42) at org.json4s.jackson.JValueDeserializer.deserialize(JValueDeserializer.scala:35) at org.json4s.jackson.JValueDeserializer.deserialize(JValueDeserializer.scala:42) at org.json4s.jackson.JValueDeserializer.deserialize(JValueDeserializer.scala:35) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at org.apache.spark.scheduler.ReplayListenerBus.replay(ReplayListenerBus.scala:49) at org.apache.spark.deploy.history.FsHistoryProvider.org$apache$spark$deploy$history$FsHistoryProvider$$replay(FsHistoryProvider.scala:260) at org.apache.spark.deploy.history.FsHistoryProvider$$anonfun$6.apply(FsHistoryProvider.scala:190) at org.apache.spark.deploy.history.FsHistoryProvider$$anonfun$6.apply(FsHistoryProvider.scala:188) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:251) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:251) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:34) at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:251) at scala.collection.AbstractTraversable.flatMap(Traversable.scala:105) at org.apache.spark.deploy.history.FsHistoryProvider.checkForLogs(FsHistoryProvider.scala:188) at org.apache.spark.deploy.history.FsHistoryProvider$$anon$1$$anonfun$run$1.apply$mcV$sp(FsHistoryProvider.scala:94) at org.apache.spark.deploy.history.FsHistoryProvider$$anon$1$$anonfun$run$1.apply(FsHistoryProvider.scala:85) at org.apache.spark.deploy.history.FsHistoryProvider$$anon$1$$anonfun$run$1.apply(FsHistoryProvider.scala:85) at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1617) at org.apache.spark.deploy.history.FsHistoryProvider$$anon$1.run(FsHistoryProvider.scala:84) {quote} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-6676) Add hadoop 2.4+ for profiles in POM.xml
Zhang, Liye created SPARK-6676: -- Summary: Add hadoop 2.4+ for profiles in POM.xml Key: SPARK-6676 URL: https://issues.apache.org/jira/browse/SPARK-6676 Project: Spark Issue Type: Improvement Components: Build, Tests Affects Versions: 1.3.0 Reporter: Zhang, Liye Priority: Minor support *-Phadoop-2.5* and *-Phadoop-2.6* when building and testing Spark -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-6689) MiniYarnCLuster still test failed with hadoop-2.2
[ https://issues.apache.org/jira/browse/SPARK-6689?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-6689: --- Description: when running unit test *YarnClusterSuite* with *hadoop-2.2*, exception will throw because *Timed out waiting for RM to come up*. Some previously related discussion can be traced in [spark-3710|https://issues.apache.org/jira/browse/SPARK-3710] ([PR2682|https://github.com/apache/spark/pull/2682]) and [spark-2778|https://issues.apache.org/jira/browse/SPARK-2778] ([PR2605|https://github.com/apache/spark/pull/2605]). With command *build/sbt -Pyarn -Phadoop-2.2 test-only org.apache.spark.deploy.yarn.YarnClusterSuite*, will get following exceptions: {noformat} [info] Exception encountered when attempting to run a suite with class name: org.apache.spark.deploy.yarn.YarnClusterSuite *** ABORTED *** (15 seconds, 799 milliseconds) [info] java.lang.IllegalStateException: Timed out waiting for RM to come up. [info] at org.apache.spark.deploy.yarn.YarnClusterSuite.beforeAll(YarnClusterSuite.scala:114) [info] at org.scalatest.BeforeAndAfterAll$class.beforeAll(BeforeAndAfterAll.scala:187) [info] at org.apache.spark.deploy.yarn.YarnClusterSuite.beforeAll(YarnClusterSuite.scala:44) [info] at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:253) [info] at org.apache.spark.deploy.yarn.YarnClusterSuite.run(YarnClusterSuite.scala:44) [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:294) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:284) [info] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [info] at java.lang.Thread.run(Thread.java:745) {noformat} And without *-Phadoop-2.2* or replace it with *-Dhadoop.version* (e.g. build/sbt -Pyarn test-only org.apache.spark.deploy.yarn.YarnClusterSuite) more info will come out: {noformat} Exception in thread Thread-7 java.lang.NoClassDefFoundError: org/mortbay/jetty/servlet/Context at org.apache.hadoop.yarn.webapp.WebApps.$for(WebApps.java:309) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.startWepApp(ResourceManager.java:602) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.serviceStart(ResourceManager.java:655) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.yarn.server.MiniYARNCluster$ResourceManagerWrapper$2.run(MiniYARNCluster.java:219) Caused by: java.lang.ClassNotFoundException: org.mortbay.jetty.servlet.Context at java.net.URLClassLoader$1.run(URLClassLoader.java:372) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:360) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) [info] Resolving org.apache.hadoop#hadoop-yarn-server-common;2.2.0 ... Exception in thread Thread-18 java.lang.NoClassDefFoundError: org/mortbay/jetty/servlet/Context at org.apache.hadoop.yarn.webapp.WebApps.$for(WebApps.java:309) at org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer.serviceStart(WebServer.java:62) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.service.CompositeService.serviceStart(CompositeService.java:121) at org.apache.hadoop.yarn.server.nodemanager.NodeManager.serviceStart(NodeManager.java:199) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.yarn.server.MiniYARNCluster$NodeManagerWrapper$1.run(MiniYARNCluster.java:337) Caused by: java.lang.ClassNotFoundException: org.mortbay.jetty.servlet.Context at java.net.URLClassLoader$1.run(URLClassLoader.java:372) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:360) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) [info] Resolving org.apache.hadoop#hadoop-yarn;2.2.0 ... [info] Exception encountered when attempting to run a suite with class name: org.apache.spark.deploy.yarn.YarnClusterSuite ***
[jira] [Created] (SPARK-6689) MiniYarnCLuster still test failed with hadoop-2.2
Zhang, Liye created SPARK-6689: -- Summary: MiniYarnCLuster still test failed with hadoop-2.2 Key: SPARK-6689 URL: https://issues.apache.org/jira/browse/SPARK-6689 Project: Spark Issue Type: Test Components: Tests, YARN Affects Versions: 1.3.0 Reporter: Zhang, Liye when running unit test *YarnClusterSuite* with *hadoop-2.2*, exception will throw because *Timed out waiting for RM to come up*. Some previously related discussion can be traced in [spark-3710|https://issues.apache.org/jira/browse/SPARK-3710] ([PR2682|https://github.com/apache/spark/pull/2682]) and [spark-2778|https://issues.apache.org/jira/browse/SPARK-2778] ([PR2605|https://github.com/apache/spark/pull/2605]). With command *build/sbt -Pyarn -Phadoop-2.2 test-only org.apache.spark.deploy.yarn.YarnClusterSuite*, will get following exceptions: {noformat} [info] Exception encountered when attempting to run a suite with class name: org.apache.spark.deploy.yarn.YarnClusterSuite *** ABORTED *** (15 seconds, 799 milliseconds) [info] java.lang.IllegalStateException: Timed out waiting for RM to come up. [info] at org.apache.spark.deploy.yarn.YarnClusterSuite.beforeAll(YarnClusterSuite.scala:114) [info] at org.scalatest.BeforeAndAfterAll$class.beforeAll(BeforeAndAfterAll.scala:187) [info] at org.apache.spark.deploy.yarn.YarnClusterSuite.beforeAll(YarnClusterSuite.scala:44) [info] at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:253) [info] at org.apache.spark.deploy.yarn.YarnClusterSuite.run(YarnClusterSuite.scala:44) [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:294) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:284) [info] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [info] at java.lang.Thread.run(Thread.java:745) {noformat} And without *-Phadoop-2.2* or replace it with *-Dhadoop.version* (e.g. build/sbt -Pyarn test-only org.apache.spark.deploy.yarn.YarnClusterSuite) more info will come out: {noformat} Exception in thread Thread-7 java.lang.NoClassDefFoundError: org/mortbay/jetty/servlet/Context at org.apache.hadoop.yarn.webapp.WebApps.$for(WebApps.java:309) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.startWepApp(ResourceManager.java:602) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.serviceStart(ResourceManager.java:655) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.yarn.server.MiniYARNCluster$ResourceManagerWrapper$2.run(MiniYARNCluster.java:219) Caused by: java.lang.ClassNotFoundException: org.mortbay.jetty.servlet.Context at java.net.URLClassLoader$1.run(URLClassLoader.java:372) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:360) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) [info] Resolving org.apache.hadoop#hadoop-yarn-server-common;2.2.0 ... Exception in thread Thread-18 java.lang.NoClassDefFoundError: org/mortbay/jetty/servlet/Context at org.apache.hadoop.yarn.webapp.WebApps.$for(WebApps.java:309) at org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer.serviceStart(WebServer.java:62) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.service.CompositeService.serviceStart(CompositeService.java:121) at org.apache.hadoop.yarn.server.nodemanager.NodeManager.serviceStart(NodeManager.java:199) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.yarn.server.MiniYARNCluster$NodeManagerWrapper$1.run(MiniYARNCluster.java:337) Caused by: java.lang.ClassNotFoundException: org.mortbay.jetty.servlet.Context at java.net.URLClassLoader$1.run(URLClassLoader.java:372) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:360) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
[jira] [Commented] (SPARK-6689) MiniYarnCLuster still test failed with hadoop-2.2
[ https://issues.apache.org/jira/browse/SPARK-6689?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14394000#comment-14394000 ] Zhang, Liye commented on SPARK-6689: cc [~vanzin] [~srowen] MiniYarnCLuster still test failed with hadoop-2.2 - Key: SPARK-6689 URL: https://issues.apache.org/jira/browse/SPARK-6689 Project: Spark Issue Type: Test Components: Tests, YARN Affects Versions: 1.3.0 Reporter: Zhang, Liye when running unit test *YarnClusterSuite* with *hadoop-2.2*, exception will throw because *Timed out waiting for RM to come up*. Some previously related discussion can be traced in [spark-3710|https://issues.apache.org/jira/browse/SPARK-3710] ([PR2682|https://github.com/apache/spark/pull/2682]) and [spark-2778|https://issues.apache.org/jira/browse/SPARK-2778] ([PR2605|https://github.com/apache/spark/pull/2605]). With command *build/sbt -Pyarn -Phadoop-2.2 test-only org.apache.spark.deploy.yarn.YarnClusterSuite*, will get following exceptions: {noformat} [info] Exception encountered when attempting to run a suite with class name: org.apache.spark.deploy.yarn.YarnClusterSuite *** ABORTED *** (15 seconds, 799 milliseconds) [info] java.lang.IllegalStateException: Timed out waiting for RM to come up. [info] at org.apache.spark.deploy.yarn.YarnClusterSuite.beforeAll(YarnClusterSuite.scala:114) [info] at org.scalatest.BeforeAndAfterAll$class.beforeAll(BeforeAndAfterAll.scala:187) [info] at org.apache.spark.deploy.yarn.YarnClusterSuite.beforeAll(YarnClusterSuite.scala:44) [info] at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:253) [info] at org.apache.spark.deploy.yarn.YarnClusterSuite.run(YarnClusterSuite.scala:44) [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:294) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:284) [info] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [info] at java.lang.Thread.run(Thread.java:745) {noformat} And without *-Phadoop-2.2* or replace it with *-Dhadoop.version* (e.g. build/sbt -Pyarn test-only org.apache.spark.deploy.yarn.YarnClusterSuite) more info will come out: {noformat} Exception in thread Thread-7 java.lang.NoClassDefFoundError: org/mortbay/jetty/servlet/Context at org.apache.hadoop.yarn.webapp.WebApps.$for(WebApps.java:309) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.startWepApp(ResourceManager.java:602) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.serviceStart(ResourceManager.java:655) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.yarn.server.MiniYARNCluster$ResourceManagerWrapper$2.run(MiniYARNCluster.java:219) Caused by: java.lang.ClassNotFoundException: org.mortbay.jetty.servlet.Context at java.net.URLClassLoader$1.run(URLClassLoader.java:372) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:360) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) [info] Resolving org.apache.hadoop#hadoop-yarn-server-common;2.2.0 ... Exception in thread Thread-18 java.lang.NoClassDefFoundError: org/mortbay/jetty/servlet/Context at org.apache.hadoop.yarn.webapp.WebApps.$for(WebApps.java:309) at org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer.serviceStart(WebServer.java:62) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.service.CompositeService.serviceStart(CompositeService.java:121) at org.apache.hadoop.yarn.server.nodemanager.NodeManager.serviceStart(NodeManager.java:199) at org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) at org.apache.hadoop.yarn.server.MiniYARNCluster$NodeManagerWrapper$1.run(MiniYARNCluster.java:337) Caused by: java.lang.ClassNotFoundException: org.mortbay.jetty.servlet.Context at java.net.URLClassLoader$1.run(URLClassLoader.java:372) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.security.AccessController.doPrivileged(Native Method)
[jira] [Created] (SPARK-6197) handle json parse exception for eventlog file not finished writing
Zhang, Liye created SPARK-6197: -- Summary: handle json parse exception for eventlog file not finished writing Key: SPARK-6197 URL: https://issues.apache.org/jira/browse/SPARK-6197 Project: Spark Issue Type: Bug Components: Spark Core, Web UI Affects Versions: 1.4.0, 1.3.1 Reporter: Zhang, Liye This is a following JIRA for [SPARK-6107|https://issues.apache.org/jira/browse/SPARK-6107]. In [SPARK-6107|https://issues.apache.org/jira/browse/SPARK-6107], webUI can display event log files that with suffix *.inprogress*. However, the eventlog file may be not finished writing for some abnormal cases (e.g. Ctrl+C), In which case, the file maybe truncated in the last line, leading to the line being not in valid Json format. Which will cause Json parse exception when reading the file. For this case, we can just ignore the last line content, since the history for abnormal cases showed on web is only a reference for user, it can demonstrate the past status of the app before terminated abnormally (we can not guarantee the history can show exactly the last moment when app encounter the abnormal situation). -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-6107) event log file ends with .inprogress should be able to display on webUI for standalone mode
Zhang, Liye created SPARK-6107: -- Summary: event log file ends with .inprogress should be able to display on webUI for standalone mode Key: SPARK-6107 URL: https://issues.apache.org/jira/browse/SPARK-6107 Project: Spark Issue Type: Bug Components: Spark Core, Web UI Affects Versions: 1.3.0 Reporter: Zhang, Liye when application is finished running abnormally (Ctrl + c for example), the history event log file is still ends with *.inprogress* suffix. And the application state can not be showed on webUI, User can just see *Application history not foud , Application xxx is still in progress*. User should also can see the status of the abnormal finished applications. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-6078) create event log directory automatically if not exists
[ https://issues.apache.org/jira/browse/SPARK-6078?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-6078: --- Description: when event log directory does not exists, spark just throw IlleagalArgumentException and stop the job. User need manually create directory first. It's better to create the directory automatically if the directory does not exists. (was: when event log directory is not exists, spark just throw IlleagalArgumentException and stop the job. User need manually create directory first. It's better to create the directory automatically if the directory does not exists.) create event log directory automatically if not exists -- Key: SPARK-6078 URL: https://issues.apache.org/jira/browse/SPARK-6078 Project: Spark Issue Type: Improvement Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhang, Liye when event log directory does not exists, spark just throw IlleagalArgumentException and stop the job. User need manually create directory first. It's better to create the directory automatically if the directory does not exists. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-6078) create event log directory automatically if not exists
Zhang, Liye created SPARK-6078: -- Summary: create event log directory automatically if not exists Key: SPARK-6078 URL: https://issues.apache.org/jira/browse/SPARK-6078 Project: Spark Issue Type: Improvement Components: Spark Core Affects Versions: 1.3.0 Reporter: Zhang, Liye when event log directory is not exists, spark just throw IlleagalArgumentException and stop the job. User need manually create directory first. It's better to create the directory automatically if the directory does not exists. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-6036) EventLog process logic has race condition with Akka actor system
Zhang, Liye created SPARK-6036: -- Summary: EventLog process logic has race condition with Akka actor system Key: SPARK-6036 URL: https://issues.apache.org/jira/browse/SPARK-6036 Project: Spark Issue Type: Bug Components: Spark Core, Web UI Affects Versions: 1.3.0 Reporter: Zhang, Liye when application finished, akka actor system will trigger disassociated event, and Master will rebuild SparkUI on web, in which progress will check whether the eventlog files are still in progress. The current logic in SparkContext is first stop the actorsystem, and then stop enentLogListener. This will cause that the enentLogListener has not finished renaming the eventLog dir name (from app-.inprogress to app-) when Spark Master try to access the dir. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-4989) wrong application configuration cause cluster down in standalone mode
Zhang, Liye created SPARK-4989: -- Summary: wrong application configuration cause cluster down in standalone mode Key: SPARK-4989 URL: https://issues.apache.org/jira/browse/SPARK-4989 Project: Spark Issue Type: Bug Components: Deploy, Spark Core Affects Versions: 1.2.0, 1.1.0, 1.0.0 Reporter: Zhang, Liye Priority: Critical when enabling eventlog in standalone mode, if give the wrong configuration, the standalone cluster will down (cause master restart, lose connection with workers). How to reproduce: just give an invalid value to spark.eventLog.dir, for example: *spark.eventLog.dir=hdfs://tmp/logdir1, hdfs://tmp/logdir2*. This will throw illegalArgumentException, which will cause the *Master* restart. And the whole cluster is not available. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4989) wrong application configuration cause cluster down in standalone mode
[ https://issues.apache.org/jira/browse/SPARK-4989?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4989: --- Description: when enabling eventlog in standalone mode, if give the wrong configuration, the standalone cluster will down (cause master restart, lose connection with workers). How to reproduce: just give an invalid value to spark.eventLog.dir, for example: *spark.eventLog.dir=hdfs://tmp/logdir1, hdfs://tmp/logdir2*. This will throw illegalArgumentException, which will cause the *Master* restart. And the whole cluster is not available. This is not acceptable that cluster is crashed by one application's wrong setting. was: when enabling eventlog in standalone mode, if give the wrong configuration, the standalone cluster will down (cause master restart, lose connection with workers). How to reproduce: just give an invalid value to spark.eventLog.dir, for example: *spark.eventLog.dir=hdfs://tmp/logdir1, hdfs://tmp/logdir2*. This will throw illegalArgumentException, which will cause the *Master* restart. And the whole cluster is not available. wrong application configuration cause cluster down in standalone mode - Key: SPARK-4989 URL: https://issues.apache.org/jira/browse/SPARK-4989 Project: Spark Issue Type: Bug Components: Deploy, Spark Core Affects Versions: 1.0.0, 1.1.0, 1.2.0 Reporter: Zhang, Liye Priority: Critical when enabling eventlog in standalone mode, if give the wrong configuration, the standalone cluster will down (cause master restart, lose connection with workers). How to reproduce: just give an invalid value to spark.eventLog.dir, for example: *spark.eventLog.dir=hdfs://tmp/logdir1, hdfs://tmp/logdir2*. This will throw illegalArgumentException, which will cause the *Master* restart. And the whole cluster is not available. This is not acceptable that cluster is crashed by one application's wrong setting. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-4991) Worker should reconnect to Master when Master actor restart
Zhang, Liye created SPARK-4991: -- Summary: Worker should reconnect to Master when Master actor restart Key: SPARK-4991 URL: https://issues.apache.org/jira/browse/SPARK-4991 Project: Spark Issue Type: Improvement Components: Deploy, Spark Core Affects Versions: 1.2.0, 1.1.0, 1.0.0 Reporter: Zhang, Liye This is a following JIRA of [SPARK-4989|https://issues.apache.org/jira/browse/SPARK-4989]. when Master akka actor encounter an exception, the Master will restart (akka actor restart not JVM restart). And all old information are cleared on Master (including workers, applications, etc). However, the workers are not aware of this at all. The state of the cluster is that: the master is on, and all workers are also on, but master is not aware of the exists of workers, and will ignore all worker's heartbeat because all workers are not registered. So that the whole cluster is not available. For some other information about this part, please refer to [SPARK-3736|https://issues.apache.org/jira/browse/SPARK-3736] and [SPARK-4592|https://issues.apache.org/jira/browse/SPARK-4592] -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4989) wrong application configuration cause cluster down in standalone mode
[ https://issues.apache.org/jira/browse/SPARK-4989?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14259938#comment-14259938 ] Zhang, Liye commented on SPARK-4989: An following JIRA is opened for resolving the Cluster resume. Please see [SPARK-4891|https://issues.apache.org/jira/browse/SPARK-4991] wrong application configuration cause cluster down in standalone mode - Key: SPARK-4989 URL: https://issues.apache.org/jira/browse/SPARK-4989 Project: Spark Issue Type: Bug Components: Deploy, Spark Core Affects Versions: 1.0.0, 1.1.0, 1.2.0 Reporter: Zhang, Liye Priority: Critical when enabling eventlog in standalone mode, if give the wrong configuration, the standalone cluster will down (cause master restart, lose connection with workers). How to reproduce: just give an invalid value to spark.eventLog.dir, for example: *spark.eventLog.dir=hdfs://tmp/logdir1, hdfs://tmp/logdir2*. This will throw illegalArgumentException, which will cause the *Master* restart. And the whole cluster is not available. This is not acceptable that cluster is crashed by one application's wrong setting. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-4989) wrong application configuration cause cluster down in standalone mode
[ https://issues.apache.org/jira/browse/SPARK-4989?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14259938#comment-14259938 ] Zhang, Liye edited comment on SPARK-4989 at 12/29/14 2:39 PM: -- A following up JIRA is opened for resolving the Cluster resume. Please see [SPARK-4891|https://issues.apache.org/jira/browse/SPARK-4991] was (Author: liyezhang556520): An following JIRA is opened for resolving the Cluster resume. Please see [SPARK-4891|https://issues.apache.org/jira/browse/SPARK-4991] wrong application configuration cause cluster down in standalone mode - Key: SPARK-4989 URL: https://issues.apache.org/jira/browse/SPARK-4989 Project: Spark Issue Type: Bug Components: Deploy, Spark Core Affects Versions: 1.0.0, 1.1.0, 1.2.0 Reporter: Zhang, Liye Priority: Critical when enabling eventlog in standalone mode, if give the wrong configuration, the standalone cluster will down (cause master restart, lose connection with workers). How to reproduce: just give an invalid value to spark.eventLog.dir, for example: *spark.eventLog.dir=hdfs://tmp/logdir1, hdfs://tmp/logdir2*. This will throw illegalArgumentException, which will cause the *Master* restart. And the whole cluster is not available. This is not acceptable that cluster is crashed by one application's wrong setting. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-4989) wrong application configuration cause cluster down in standalone mode
[ https://issues.apache.org/jira/browse/SPARK-4989?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14259938#comment-14259938 ] Zhang, Liye edited comment on SPARK-4989 at 12/29/14 2:39 PM: -- A following up JIRA is opened for resolving the Cluster's resume. Please see [SPARK-4891|https://issues.apache.org/jira/browse/SPARK-4991] was (Author: liyezhang556520): A following up JIRA is opened for resolving the Cluster resume. Please see [SPARK-4891|https://issues.apache.org/jira/browse/SPARK-4991] wrong application configuration cause cluster down in standalone mode - Key: SPARK-4989 URL: https://issues.apache.org/jira/browse/SPARK-4989 Project: Spark Issue Type: Bug Components: Deploy, Spark Core Affects Versions: 1.0.0, 1.1.0, 1.2.0 Reporter: Zhang, Liye Priority: Critical when enabling eventlog in standalone mode, if give the wrong configuration, the standalone cluster will down (cause master restart, lose connection with workers). How to reproduce: just give an invalid value to spark.eventLog.dir, for example: *spark.eventLog.dir=hdfs://tmp/logdir1, hdfs://tmp/logdir2*. This will throw illegalArgumentException, which will cause the *Master* restart. And the whole cluster is not available. This is not acceptable that cluster is crashed by one application's wrong setting. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4989) wrong application configuration cause cluster down in standalone mode
[ https://issues.apache.org/jira/browse/SPARK-4989?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14260667#comment-14260667 ] Zhang, Liye commented on SPARK-4989: Assume that we set *spark.eventLog.dir=hdfs://host:port/user/username/eventLog0, hdfs://host:port/user/username/eventLog1*, when starting the application, application will fail due to the exception, like following: {code} Exception in thread main java.net.URISyntaxException: Illegal character in path at index 41: hdfs//host:port/user/username/eventLog0, hdfs://host:port/user/username/eventLog1 at java.net.URI$Parser.fail(URI.java:2829) at java.net.URI$Parser.checkChars(URI.java:3002) at java.net.URI$Parser.parseHierarchical(URI.java:3086) at java.net.URI$Parser.parse(URI.java:3034) at java.net.URI.init(URI.java:595) at org.apache.spark.scheduler.EventLoggingListener.init(EventLoggingListener.scala:64) at org.apache.spark.SparkContext.init(SparkContext.scala:357) at org.apache.spark.SparkContext.init(SparkContext.scala:136) at com.intel.benchmark.wordcount.Driver$.main(Driver.scala:44) at com.intel.benchmark.wordcount.Driver.main(Driver.scala) {code} This will cause the *Master* to remove the application and rebuild the sparkUI. In which will also throw the exception to *Master Actor*, which will lead to *Master*'s restart. The exception will be like the following: {code} 14/12/30 09:48:18 ERROR OneForOneStrategy: Illegal character in path at index 41: hdfs://sr148:9000/user/liyezhan/eventLog, hdfs://sr148:9000/user/liyezhan/eventLog1 java.net.URISyntaxException: Illegal character in path at index 41: hdfs//host:port/user/username/eventLog0, hdfs://host:port/user/username/eventLog1 at java.net.URI$Parser.fail(URI.java:2829) at java.net.URI$Parser.checkChars(URI.java:3002) at java.net.URI$Parser.parseHierarchical(URI.java:3086) at java.net.URI$Parser.parse(URI.java:3034) at java.net.URI.init(URI.java:595) at org.apache.spark.util.Utils$.resolveURI(Utils.scala:1561) at org.apache.spark.scheduler.EventLoggingListener$.getLogPath(EventLoggingListener.scala:262) at org.apache.spark.deploy.master.Master$$anonfun$11.apply(Master.scala:724) at org.apache.spark.deploy.master.Master$$anonfun$11.apply(Master.scala:724) at scala.Option.map(Option.scala:145) at org.apache.spark.deploy.master.Master.rebuildSparkUI(Master.scala:724) at org.apache.spark.deploy.master.Master.removeApplication(Master.scala:695) at org.apache.spark.deploy.master.Master.finishApplication(Master.scala:673) at org.apache.spark.deploy.master.Master$$anonfun$receiveWithLogging$1$$anonfun$applyOrElse$29.apply(Master.scala:419) at org.apache.spark.deploy.master.Master$$anonfun$receiveWithLogging$1$$anonfun$applyOrElse$29.apply(Master.scala:419) at scala.Option.foreach(Option.scala:236) at org.apache.spark.deploy.master.Master$$anonfun$receiveWithLogging$1.applyOrElse(Master.scala:419) at scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33) at scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33) at scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25) at org.apache.spark.util.ActorLogReceive$$anon$1.apply(ActorLogReceive.scala:53) at org.apache.spark.util.ActorLogReceive$$anon$1.apply(ActorLogReceive.scala:42) at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:118) at org.apache.spark.util.ActorLogReceive$$anon$1.applyOrElse(ActorLogReceive.scala:42) at akka.actor.Actor$class.aroundReceive(Actor.scala:465) at org.apache.spark.deploy.master.Master.aroundReceive(Master.scala:51) 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) {code} wrong application configuration cause cluster down in standalone mode - Key: SPARK-4989 URL: https://issues.apache.org/jira/browse/SPARK-4989 Project: Spark Issue Type: Bug Components: Deploy, Spark Core Affects Versions: 1.0.0, 1.1.0, 1.2.0 Reporter: Zhang, Liye Priority: Critical when enabling eventlog in standalone mode, if give the wrong
[jira] [Created] (SPARK-4954) Add spark version information in log for standalone mode
Zhang, Liye created SPARK-4954: -- Summary: Add spark version information in log for standalone mode Key: SPARK-4954 URL: https://issues.apache.org/jira/browse/SPARK-4954 Project: Spark Issue Type: Improvement Components: Spark Core Reporter: Zhang, Liye Priority: Minor Fix For: 1.3.0 This is a following JIRA for [SPARK-4870|https://issues.apache.org/jira/browse/SPARK-4870]. The master and worker spark version may be not the same with Driver spark version. That is because spark Jar file might be replaced for new application without restarting the spark cluster. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-4933) eventLog file not found after merging into a single file
Zhang, Liye created SPARK-4933: -- Summary: eventLog file not found after merging into a single file Key: SPARK-4933 URL: https://issues.apache.org/jira/browse/SPARK-4933 Project: Spark Issue Type: Bug Components: Spark Core, Web UI Reporter: Zhang, Liye enent log file not found exception will be thrown after merging eventLog into a single file. Main course is the wrong arguments for getting log file. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4933) eventLog file not found after merging into a single file
[ https://issues.apache.org/jira/browse/SPARK-4933?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4933: --- Description: enent log file not found exception will be thrown after making eventLog into a single file. Main course is the wrong arguments for getting log file. (was: enent log file not found exception will be thrown after merging eventLog into a single file. Main course is the wrong arguments for getting log file.) eventLog file not found after merging into a single file Key: SPARK-4933 URL: https://issues.apache.org/jira/browse/SPARK-4933 Project: Spark Issue Type: Bug Components: Spark Core, Web UI Reporter: Zhang, Liye enent log file not found exception will be thrown after making eventLog into a single file. Main course is the wrong arguments for getting log file. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4933) eventLog file not found after merging into a single file
[ https://issues.apache.org/jira/browse/SPARK-4933?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4933: --- Description: enent log file not found exception will be thrown after making eventLog into a single file. Main cause is the wrong arguments for getting log file. (was: enent log file not found exception will be thrown after making eventLog into a single file. Main course is the wrong arguments for getting log file.) eventLog file not found after merging into a single file Key: SPARK-4933 URL: https://issues.apache.org/jira/browse/SPARK-4933 Project: Spark Issue Type: Bug Components: Spark Core, Web UI Reporter: Zhang, Liye enent log file not found exception will be thrown after making eventLog into a single file. Main cause is the wrong arguments for getting log file. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4933) eventLog file not found after merging into a single file
[ https://issues.apache.org/jira/browse/SPARK-4933?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4933: --- Description: enent log file not found exception will be thrown after making eventLog into a single file. Main cause is the wrong argument for getting log file. (was: enent log file not found exception will be thrown after making eventLog into a single file. Main cause is the wrong arguments for getting log file.) eventLog file not found after merging into a single file Key: SPARK-4933 URL: https://issues.apache.org/jira/browse/SPARK-4933 Project: Spark Issue Type: Bug Components: Spark Core, Web UI Reporter: Zhang, Liye enent log file not found exception will be thrown after making eventLog into a single file. Main cause is the wrong argument for getting log file. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4920) current spark version in UI is not striking
[ https://issues.apache.org/jira/browse/SPARK-4920?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14256545#comment-14256545 ] Zhang, Liye commented on SPARK-4920: Seems standalone mode is not with the version. I agree with [~sowen], it'll be not good looking when the version is the long. Putting the version on footer will be not string but will be flexible for extension. current spark version in UI is not striking --- Key: SPARK-4920 URL: https://issues.apache.org/jira/browse/SPARK-4920 Project: Spark Issue Type: Improvement Components: Web UI Affects Versions: 1.2.0 Reporter: uncleGen Assignee: uncleGen Priority: Minor Fix For: 1.2.1 It is not convenient to see the Spark version. We can keep the same style with Spark website. !https://raw.githubusercontent.com/uncleGen/Tech-Notes/master/spark_version.jpg! -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-4920) current spark version in UI is not striking
[ https://issues.apache.org/jira/browse/SPARK-4920?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14256545#comment-14256545 ] Zhang, Liye edited comment on SPARK-4920 at 12/23/14 3:25 AM: -- Seems standalone mode is not with the version on web UI. I agree with [~sowen], it'll be not good looking when the version is the long. Putting the version on footer will be not striking but will be flexible for extension. was (Author: liyezhang556520): Seems standalone mode is not with the version. I agree with [~sowen], it'll be not good looking when the version is the long. Putting the version on footer will be not string but will be flexible for extension. current spark version in UI is not striking --- Key: SPARK-4920 URL: https://issues.apache.org/jira/browse/SPARK-4920 Project: Spark Issue Type: Improvement Components: Web UI Affects Versions: 1.2.0 Reporter: uncleGen Assignee: uncleGen Priority: Minor Fix For: 1.2.1 It is not convenient to see the Spark version. We can keep the same style with Spark website. !https://raw.githubusercontent.com/uncleGen/Tech-Notes/master/spark_version.jpg! -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4094) checkpoint should still be available after rdd actions
[ https://issues.apache.org/jira/browse/SPARK-4094?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4094: --- Description: rdd.checkpoint() must be called before any actions on this rdd, if there is any other actions before, checkpoint would never succeed. For the following code as example: *rdd = sc.makeRDD(...)* *rdd.collect()* *rdd.checkpoint()* *rdd.count()* This rdd would never be checkpointed. For algorithms that have many iterations would have some problem. Such as graph algorithm, there will have many iterations which will cause the RDD lineage very long. So RDD may need checkpoint after a certain iteration number. And if there is also any action within the iteration loop, the checkpoint() operation will never work for the later iterations after the iteration which calls the action operation. But this would not happen for RDD cache. RDD cache would always make successfully before rdd actions no matter whether there is any actions before cache(). So rdd.checkpoint() should also be with the same behavior with rdd.cache(). was: rdd.checkpoint() must be called before any actions on this rdd, if there is any other actions before, checkpoint would never succeed. For the following code as example: *rdd = sc.makeRDD(...)* *rdd.collect()* *rdd.checkpoint()* *rdd.count()* This rdd would never be checkpointed. For algorithms that have many iterations would have some problem. Such as graph algorithm, there will have many iterations which will cause the RDD lineage very long. So RDD may need checkpoint after a certain iteration number. And if there is also any action within the iteration loop, the checkpoint() operation will never work for the later iterations after the iteration whichs call the action operation. But this would not happen for RDD cache. RDD cache would always make successfully before rdd actions no matter whether there is any actions before cache(). So rdd.checkpoint() should also be with the same behavior with rdd.cache(). checkpoint should still be available after rdd actions -- Key: SPARK-4094 URL: https://issues.apache.org/jira/browse/SPARK-4094 Project: Spark Issue Type: Improvement Components: Spark Core Affects Versions: 1.1.0 Reporter: Zhang, Liye Assignee: Zhang, Liye rdd.checkpoint() must be called before any actions on this rdd, if there is any other actions before, checkpoint would never succeed. For the following code as example: *rdd = sc.makeRDD(...)* *rdd.collect()* *rdd.checkpoint()* *rdd.count()* This rdd would never be checkpointed. For algorithms that have many iterations would have some problem. Such as graph algorithm, there will have many iterations which will cause the RDD lineage very long. So RDD may need checkpoint after a certain iteration number. And if there is also any action within the iteration loop, the checkpoint() operation will never work for the later iterations after the iteration which calls the action operation. But this would not happen for RDD cache. RDD cache would always make successfully before rdd actions no matter whether there is any actions before cache(). So rdd.checkpoint() should also be with the same behavior with rdd.cache(). -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4094) checkpoint should still be available after rdd actions
[ https://issues.apache.org/jira/browse/SPARK-4094?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14255367#comment-14255367 ] Zhang, Liye commented on SPARK-4094: HI [~sowen], I'm also wondering why the original way is restricted. But for the real-world case, especially for graph algorithms that has many iterations, this restricted too much. Hi [~matei], the spark code originally has such restriction. Can you tell if there are other considerations excluding the complexity involved from traversing the whole lineage? And if there are problems allowing checkpoint after rdd actions? checkpoint should still be available after rdd actions -- Key: SPARK-4094 URL: https://issues.apache.org/jira/browse/SPARK-4094 Project: Spark Issue Type: Improvement Components: Spark Core Affects Versions: 1.1.0 Reporter: Zhang, Liye Assignee: Zhang, Liye rdd.checkpoint() must be called before any actions on this rdd, if there is any other actions before, checkpoint would never succeed. For the following code as example: *rdd = sc.makeRDD(...)* *rdd.collect()* *rdd.checkpoint()* *rdd.count()* This rdd would never be checkpointed. For algorithms that have many iterations would have some problem. Such as graph algorithm, there will have many iterations which will cause the RDD lineage very long. So RDD may need checkpoint after a certain iteration number. And if there is also any action within the iteration loop, the checkpoint() operation will never work for the later iterations after the iteration which calls the action operation. But this would not happen for RDD cache. RDD cache would always make successfully before rdd actions no matter whether there is any actions before cache(). So rdd.checkpoint() should also be with the same behavior with rdd.cache(). -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4094) checkpoint should still be available after rdd actions
[ https://issues.apache.org/jira/browse/SPARK-4094?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4094: --- Description: rdd.checkpoint() must be called before any actions on this rdd, if there is any other actions before, checkpoint would never succeed. For the following code as example: *rdd = sc.makeRDD(...)* *rdd.collect()* *rdd.checkpoint()* *rdd.count()* This rdd would never be checkpointed. For algorithms that have many iterations would have some problem. Such as graph algorithm, there will have many iterations which will cause the RDD lineage very long. So RDD may need checkpoint after a certain iteration number. And if there is also any action within the iteration loop, the checkpoint() operation will never work for the later iterations after the iteration whichs call the action operation. But this would not happen for RDD cache. RDD cache would always make successfully before rdd actions no matter whether there is any actions before cache(). So rdd.checkpoint() should also be with the same behavior with rdd.cache(). was: rdd.checkpoint() must be called before any actions on this rdd, if there is any other actions before, checkpoint would never succeed. For the following code as example: *rdd = sc.makeRDD(...)* *rdd.collect()* *rdd.checkpoint()* *rdd.count()* This rdd would never be checkpointed. But this would not happen for RDD cache. RDD cache would always make successfully before rdd actions no matter whether there is any actions before cache(). So rdd.checkpoint() should also be with the same behavior with rdd.cache(). checkpoint should still be available after rdd actions -- Key: SPARK-4094 URL: https://issues.apache.org/jira/browse/SPARK-4094 Project: Spark Issue Type: Improvement Components: Spark Core Affects Versions: 1.1.0 Reporter: Zhang, Liye Assignee: Zhang, Liye rdd.checkpoint() must be called before any actions on this rdd, if there is any other actions before, checkpoint would never succeed. For the following code as example: *rdd = sc.makeRDD(...)* *rdd.collect()* *rdd.checkpoint()* *rdd.count()* This rdd would never be checkpointed. For algorithms that have many iterations would have some problem. Such as graph algorithm, there will have many iterations which will cause the RDD lineage very long. So RDD may need checkpoint after a certain iteration number. And if there is also any action within the iteration loop, the checkpoint() operation will never work for the later iterations after the iteration whichs call the action operation. But this would not happen for RDD cache. RDD cache would always make successfully before rdd actions no matter whether there is any actions before cache(). So rdd.checkpoint() should also be with the same behavior with rdd.cache(). -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-4870) Add version information to driver log
Zhang, Liye created SPARK-4870: -- Summary: Add version information to driver log Key: SPARK-4870 URL: https://issues.apache.org/jira/browse/SPARK-4870 Project: Spark Issue Type: Improvement Components: Spark Core Reporter: Zhang, Liye Driver log doesn't include spark version information, version info is important in testing different spark version. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4740: --- Attachment: repartition test.7z Hi [~rxin], [~adav], I made several tests on HDDs and ramdisk with *repartition(192)* to test shuffle performance for NIO and Netty, with the same dataset as before (400GB). I uploaded archived file *repatition test.7z*, in which there are 6 tests' results: 1, NIO on ramdisk 2, NIO on HDDs 3, Netty on ramdisk with connectionPerPeer set to 1 4, Netty on ramdisk with connectionPerPeer set to 8 5, Netty on HDDs with connectionPerPeer set to 1 6, Netty on HDDs with connectionPerPeer set to 8 P.S. in the attached htmls, unit of IO throughput is requests instead of byte. From the 6 tests, it's very obvious that the reduce performance increases a lot by setting *connectionPerPeer* from 1 to 8. Both with Ramdisk and HDDs. For HDDs, the reduce time of Netty with *connectionPerPeer=8* is about the same with NIO (about 6.7 mins). For Ramdisk, Netty outperforms NIO even with *connectionPerPeer=1*. That is because the memory bandwidth has reaches bound for NIO, it's memory bandwidth bound, which I have confirmed with other tools. That's why the CPU utilization of NIO in reduce phase is only about 50%. While Netty still can get some performance gain by increasing *connectionPerPeer*'s value. This is execpected because NIO need some extra memory copy than Netty. Before these 6 tests, I have monitored the IO with *iostat* for HDDs case. When keeping *connectionPerPeer* as default (=1), Netty's read requests queue size, read requests, await, %util are all smaller than NIO, which means Netty's read parallelism is not well profiled. Till now, we can confirm that Netty doesn't get good read concurrency for small size cluster with many disks (if not set *connetionPerPeer*), but still we can not make a conclusion that Netty can run faster than NIO on HDDs. Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip, repartition test.7z, rxin_patch-on_4_node_cluster_48CoresPerNode(Unbalance).7z When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14242656#comment-14242656 ] Zhang, Liye commented on SPARK-4740: Hi [~adav], I missed there is another patch from [~rxin] set connectionsPerPeer to 1, the result in my last comment is with the default value. I have made another test with connectionsPerPeer set to 2 on 8HDDs, the result is a little better than connectionsPerPeer=1, but still can not compete with NIO. Seems the unbalance of Netty is not introduced by rxin's patch. It exists in the original master branch with HDD. Hi [~rxin], I tested your patch https://github.com/apache/spark/pull/3667 with 8HDDs and with spark.executor.memory=48GB, the result shows this patch doesn't get better performance, the reduce time with patch is longer than without the patch (37mins VS 31 mins). Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip, rxin_patch-on_4_node_cluster_48CoresPerNode(Unbalance).7z When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14241206#comment-14241206 ] Zhang, Liye commented on SPARK-4740: Hi [~adav], [~rxin], I run the test with the latest master branch today, In which rxin's patch is merged. On my 4 nodes 48 cores per node cluster, I set the *spark.local.dir* to one tmpfs (ramdisk) dir, the ramdisk size is 136GB to make sure enough for shuffle (total shuffle write 284GB, total shuffle read 213GB), *spark.executor.memory* is set to 48GB. In this way to eliminate the disk I/O effect. Still with the 400GB data set, the test result shows Netty is better than NIO (reduce time *Netty:24mins* VS *NIO:26mins*). Also, I retested with 8HDDs, remain *spark.executor.memory* with 48GB, set *spark.local.dir* to 8 HDD dirs. The result is about the same as before, that is NIO outperforms Netty (reduce time*Netty:32mins* VS *NIO:25mins*). And in Netty test, unbalance still exists, the best executor finishes 308 tasks, and the worst executor only finished 222 tasks. It seems NIO is not effected with whether it is HDD or ramdisk, while Netty is more sensitive with HDD. Till now, maybe we can limit the problem to the different behavior between Netty and NIO on disk operations. Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip, rxin_patch-on_4_node_cluster_48CoresPerNode(Unbalance).7z When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4740: --- Attachment: rxin_patch-on_4_node_cluster_48CoresPerNode(Unbalance).7z Hi [~rxin], [~adav], I uploaded the debug level log (*rxin_patch-on_4_node_cluster_48CoresPerNode(Unbalance).7z*) for the four node cluster, in the .7z archive, there is a stage page html and a .gz file, logs are in the .gz files, one driver log and four executor logs. The reduce time for this test is 32 mins. One more thing, not all the test with rxin's patch will have serious unbalance problem, sometimes, the finished tasks number are nearly the same. But there will be one node still finishes a lot earlier than others. If then, the reduce will be more faster, say 30 mins, which is much better than without the patch (40mins). However, NIO still outperforms (27 mins). On the 4 node cluster (_48 cores per node, spark.executor.memory=36g, Hadoop1.2.1, 10G NIC_) we can get some performance gain by applying the patch. We also tested on [~jerryshao]'s 6 nodes cluster (_24 cores per node, spark.executor.memory=36g, Hadoop2.4.1, 1G NIC_), the unbalance problems are very serious, some node can only finish about 1/3 tasks of the best one. Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip, rxin_patch-on_4_node_cluster_48CoresPerNode(Unbalance).7z When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14238009#comment-14238009 ] Zhang, Liye edited comment on SPARK-4740 at 12/8/14 4:13 PM: - Hi [~rxin], [~adav], I uploaded the debug level log (*rxin_patch-on_4_node_cluster_48CoresPerNode(Unbalance).7z*) for the four node cluster, in the .7z archive, there is a stage page html and a .gz file, logs are in the .gz files, one driver log and four executor logs. The reduce time for this test is 32 mins. One more thing, not all the test with rxin's patch will have serious unbalance problem, sometimes, the finished tasks number are nearly the same. But there will be one node still finishes a lot earlier than others. If then, the reduce will be more faster, say 30 mins, which is much better than without the patch (40mins). However, NIO still outperforms (27 mins). On the 4 node cluster (_48 cores per node, spark.executor.memory=36g, Hadoop1.2.1, 10G NIC_) we can get some performance gain by applying the patch. We also tested on [~jerryshao]'s 6 nodes cluster (_24 cores per node, spark.executor.memory=36g, Hadoop2.4.1, 1G NIC_), the unbalance problems are very serious, some node can only finish about 1/3 tasks of the best one. Which leading to worse performance with reduce time even longer than that without the patch (*57 mins with patch* VS *50 mins without patch*) was (Author: liyezhang556520): Hi [~rxin], [~adav], I uploaded the debug level log (*rxin_patch-on_4_node_cluster_48CoresPerNode(Unbalance).7z*) for the four node cluster, in the .7z archive, there is a stage page html and a .gz file, logs are in the .gz files, one driver log and four executor logs. The reduce time for this test is 32 mins. One more thing, not all the test with rxin's patch will have serious unbalance problem, sometimes, the finished tasks number are nearly the same. But there will be one node still finishes a lot earlier than others. If then, the reduce will be more faster, say 30 mins, which is much better than without the patch (40mins). However, NIO still outperforms (27 mins). On the 4 node cluster (_48 cores per node, spark.executor.memory=36g, Hadoop1.2.1, 10G NIC_) we can get some performance gain by applying the patch. We also tested on [~jerryshao]'s 6 nodes cluster (_24 cores per node, spark.executor.memory=36g, Hadoop2.4.1, 1G NIC_), the unbalance problems are very serious, some node can only finish about 1/3 tasks of the best one. Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip, rxin_patch-on_4_node_cluster_48CoresPerNode(Unbalance).7z When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14237178#comment-14237178 ] Zhang, Liye commented on SPARK-4740: [~pwendell], I have the same concern first, and I verified that, and replaced the assembly jar files on all nodes, and also restart the cluster, but the problem still exists. One more solid proof is that the better performance is not stick to one machine, maybe one node performs better for this test and another node for other test. Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Comment Edited] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14237178#comment-14237178 ] Zhang, Liye edited comment on SPARK-4740 at 12/7/14 3:32 PM: - [~pwendell], I have the same concern at first. I verified for several times, and replaced the assembly jar files on all nodes, and also restart the cluster, but the problem still exists. One more solid proof is that the better performance node is not stick to one machine, maybe one node performs better for this test and may switch to another node for next test. was (Author: liyezhang556520): [~pwendell], I have the same concern first, and I verified that, and replaced the assembly jar files on all nodes, and also restart the cluster, but the problem still exists. One more solid proof is that the better performance is not stick to one machine, maybe one node performs better for this test and another node for other test. Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14237179#comment-14237179 ] Zhang, Liye commented on SPARK-4740: Hi [~adav], I kept speculation as default (false). Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14236895#comment-14236895 ] Zhang, Liye commented on SPARK-4740: Hi [~rxin], on my 4 node cluster, I just tested default numConnectionsPerPeer, which is 2. After applying the patch, the reduce time reduced from 40 mins to 35 mins. But still longer than Nio. One interesting thing is that one of the four node performs much better than the other nodes. The CPU usage is high, and network throughput is also better than the other 3 nodes. This situation not gonna happen with the current master branch. I'll test with other number of numConnectionsPerPeer later on. Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4740: --- Attachment: (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4740: --- Attachment: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14237015#comment-14237015 ] Zhang, Liye commented on SPARK-4740: Hi [~adav], I attached the jstack info for both the better executor and lower executor FYI. Hi [~rxin], is it possible for you to reproduce the issue? Since [~jerryshao]'s 24 cores machine can see the difference between NIO and Netty, and even with 16 cores per executor Nio can outperform Netty. Because I'm afraid you are not able to access our cluster because of the firewall. Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14237019#comment-14237019 ] Zhang, Liye commented on SPARK-4740: Hi [~adav],[~rxin] for the better executor, in fact, it finished more tasks (320 tasks) in reduce phase, while other 3 executor finishes about 220 tasks (1000 reduce tasks in total). Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Assignee: Reynold Xin Priority: Blocker Attachments: (rxin patch better executor)TestRunner sort-by-key - Thread dump for executor 3_files.zip, (rxin patch normal executor)TestRunner sort-by-key - Thread dump for executor 0 _files.zip, Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4740: --- Attachment: TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Attachments: Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4740: --- Attachment: Spark-perf Test Report 16 Cores per Executor.pdf Hi [~rxin] [~adav], the difference between Netty and NIO is not that much when each executor with 16 cores. Still Nio outperforms Netty, both in network throughput and reduce running time. (In this case, Nio performance seems limited to the CPU computing capacity, it's CPU bound) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Attachments: Spark-perf Test Report 16 Cores per Executor.pdf, Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (Netty-48 Cores per node).zip, TestRunner sort-by-key - Thread dump for executor 1_files (Nio-48 cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4740) Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4740: --- Attachment: Spark-perf Test Report.pdf Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time -- Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Attachments: Spark-perf Test Report.pdf When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14234937#comment-14234937 ] Zhang, Liye commented on SPARK-4740: We found this issue when we make the performance test for [SPARK-2926|https://issues.apache.org/jira/browse/SPARK-2926], since [SPARK-2926|https://issues.apache.org/jira/browse/SPARK-2926] takes less time in reduce phase, so the difference between Netty and Nio is not too much, about 20%. So we tested the master branch, and the difference is more significant, more than 30%. Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time -- Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Attachments: Spark-perf Test Report.pdf When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14234941#comment-14234941 ] Zhang, Liye commented on SPARK-4740: [~adav], I have tested by setting spark.shuffle.io.serverThreads and spark.shuffle.io.clientThreads to 48, the result does not change, Netty takes the same time with 39mins for reduce phase. Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time -- Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Attachments: Spark-perf Test Report.pdf When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4740) Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4740: --- Attachment: TestRunner sort-by-key - Thread dump for executor 1_files (48 Cores per node).zip Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time -- Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Attachments: Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (48 Cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-4740) Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14234983#comment-14234983 ] Zhang, Liye commented on SPARK-4740: [~rxin] I attached the thread dump of one executor (48 cores) in reduce phase, please take a look. I'll try 16 cores later on. Netty's network bandwidth is much lower than NIO in spark-perf and Netty takes longer running time -- Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Attachments: Spark-perf Test Report.pdf, TestRunner sort-by-key - Thread dump for executor 1_files (48 Cores per node).zip When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-4740) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey
[ https://issues.apache.org/jira/browse/SPARK-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Zhang, Liye updated SPARK-4740: --- Attachment: (was: TestRunner sort-by-key - Thread dump for executor 1_files (48 Cores per node).zip) Netty's network throughput is about 1/2 of NIO's in spark-perf sortByKey Key: SPARK-4740 URL: https://issues.apache.org/jira/browse/SPARK-4740 Project: Spark Issue Type: Improvement Components: Shuffle, Spark Core Affects Versions: 1.2.0 Reporter: Zhang, Liye Attachments: Spark-perf Test Report.pdf When testing current spark master (1.3.0-snapshot) with spark-perf (sort-by-key, aggregate-by-key, etc), Netty based shuffle transferService takes much longer time than NIO based shuffle transferService. The network throughput of Netty is only about half of that of NIO. We tested with standalone mode, and the data set we used for test is 20 billion records, and the total size is about 400GB. Spark-perf test is Running on a 4 node cluster with 10G NIC, 48 cpu cores per node and each executor memory is 64GB. The reduce tasks number is set to 1000. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org