[jira] [Commented] (SPARK-1403) Spark on Mesos does not set Thread's context class loader
[ https://issues.apache.org/jira/browse/SPARK-1403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14577552#comment-14577552 ] Kannan Rajah commented on SPARK-1403: - A similar problem has been reported while running Spark on Mesos using MapR distribution. The current thread's context class loader is NULL inside the executor process causing NPE in MapR code. Refer to this discussion. http://answers.mapr.com/questions/163353/spark-from-apache-downloads-site-for-mapr.html#answer-163484 > Spark on Mesos does not set Thread's context class loader > - > > Key: SPARK-1403 > URL: https://issues.apache.org/jira/browse/SPARK-1403 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 1.0.0 > Environment: ubuntu 12.04 on vagrant >Reporter: Bharath Bhushan >Priority: Blocker > Fix For: 1.0.0 > > > I can run spark 0.9.0 on mesos but not spark 1.0.0. This is because the spark > executor on mesos slave throws a java.lang.ClassNotFoundException for > org.apache.spark.serializer.JavaSerializer. > The lengthy discussion is here: > http://apache-spark-user-list.1001560.n3.nabble.com/java-lang-ClassNotFoundException-spark-on-mesos-td3510.html#a3513 -- 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-4048) Enhance and extend hadoop-provided profile
[ https://issues.apache.org/jira/browse/SPARK-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14568012#comment-14568012 ] Kannan Rajah commented on SPARK-4048: - Just for completeness of this discussion, let me point out how the 2 cases are different. Case 1: Spark uses code from Jar A. This is same as curator case. Spark should bundle Jar A. Case 2: Spark uses code from Jar B that internally uses Jar A. This is same as Hadoop API case. Spark can use a B-provided profile and not bundle Jar A and Jar B. Basically expect B and its dependencies to be provided. Case3: Spark uses code from Jar A and also code from Jar B that internally uses Jar A. Spark should bundle Jar A. But this is a problematic case because we could end up with 2 different versions of jar A. But overall I got what you are saying. Based on your comments, this profile has been added to satisfy some specific use case. So I won't file a JIRA. > Enhance and extend hadoop-provided profile > -- > > Key: SPARK-4048 > URL: https://issues.apache.org/jira/browse/SPARK-4048 > Project: Spark > Issue Type: Improvement > Components: Build >Affects Versions: 1.2.0 >Reporter: Marcelo Vanzin >Assignee: Marcelo Vanzin > Fix For: 1.3.0 > > > The hadoop-provided profile is used to not package Hadoop dependencies inside > the Spark assembly. It works, sort of, but it could use some enhancements. A > quick list: > - It doesn't include all things that could be removed from the assembly > - It doesn't work well when you're publishing artifacts based on it > (SPARK-3812 fixes this) > - There are other dependencies that could use similar treatment: Hive, HBase > (for the examples), Flume, Parquet, maybe others I'm missing at the moment. > - Unit tests, more specifically, those that use local-cluster mode, do not > work when the assembly is built with this profile enabled. > - The scripts to launch Spark jobs do not add needed "provided" jars to the > classpath when this profile is enabled, leaving it for people to figure that > out for themselves. > - The examples assembly duplicates a lot of things in the main assembly. > Part of this task is selfish since we build internally with this profile and > we'd like to make it easier for us to merge changes without having to keep > too many patches on top of upstream. But those feel like good improvements to > me, regardless. -- 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-4048) Enhance and extend hadoop-provided profile
[ https://issues.apache.org/jira/browse/SPARK-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14567989#comment-14567989 ] Kannan Rajah commented on SPARK-4048: - Yes, exactly. My question is whether curator jar should be part of the profile. I think it should not be part of the profile because Spark Master is explicitly using code from curator jar. > Enhance and extend hadoop-provided profile > -- > > Key: SPARK-4048 > URL: https://issues.apache.org/jira/browse/SPARK-4048 > Project: Spark > Issue Type: Improvement > Components: Build >Affects Versions: 1.2.0 >Reporter: Marcelo Vanzin >Assignee: Marcelo Vanzin > Fix For: 1.3.0 > > > The hadoop-provided profile is used to not package Hadoop dependencies inside > the Spark assembly. It works, sort of, but it could use some enhancements. A > quick list: > - It doesn't include all things that could be removed from the assembly > - It doesn't work well when you're publishing artifacts based on it > (SPARK-3812 fixes this) > - There are other dependencies that could use similar treatment: Hive, HBase > (for the examples), Flume, Parquet, maybe others I'm missing at the moment. > - Unit tests, more specifically, those that use local-cluster mode, do not > work when the assembly is built with this profile enabled. > - The scripts to launch Spark jobs do not add needed "provided" jars to the > classpath when this profile is enabled, leaving it for people to figure that > out for themselves. > - The examples assembly duplicates a lot of things in the main assembly. > Part of this task is selfish since we build internally with this profile and > we'd like to make it easier for us to merge changes without having to keep > too many patches on top of upstream. But those feel like good improvements to > me, regardless. -- 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-4048) Enhance and extend hadoop-provided profile
[ https://issues.apache.org/jira/browse/SPARK-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14567953#comment-14567953 ] Kannan Rajah commented on SPARK-4048: - I am going to make the curator jar as compile time dependency and test out. Will file JIRA based on that. But I would like to reiterate that Spark Master HA is using a class that belongs to curator jar. So it should make sure that the jar is available without making any assumption that it will be supplied by some other component. I understand that when you have jars coming from several components, it makes it hard to manage the versions. > Enhance and extend hadoop-provided profile > -- > > Key: SPARK-4048 > URL: https://issues.apache.org/jira/browse/SPARK-4048 > Project: Spark > Issue Type: Improvement > Components: Build >Affects Versions: 1.2.0 >Reporter: Marcelo Vanzin >Assignee: Marcelo Vanzin > Fix For: 1.3.0 > > > The hadoop-provided profile is used to not package Hadoop dependencies inside > the Spark assembly. It works, sort of, but it could use some enhancements. A > quick list: > - It doesn't include all things that could be removed from the assembly > - It doesn't work well when you're publishing artifacts based on it > (SPARK-3812 fixes this) > - There are other dependencies that could use similar treatment: Hive, HBase > (for the examples), Flume, Parquet, maybe others I'm missing at the moment. > - Unit tests, more specifically, those that use local-cluster mode, do not > work when the assembly is built with this profile enabled. > - The scripts to launch Spark jobs do not add needed "provided" jars to the > classpath when this profile is enabled, leaving it for people to figure that > out for themselves. > - The examples assembly duplicates a lot of things in the main assembly. > Part of this task is selfish since we build internally with this profile and > we'd like to make it easier for us to merge changes without having to keep > too many patches on top of upstream. But those feel like good improvements to > me, regardless. -- 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-4048) Enhance and extend hadoop-provided profile
[ https://issues.apache.org/jira/browse/SPARK-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14567783#comment-14567783 ] Kannan Rajah commented on SPARK-4048: - Some clarification ... - My understanding of hadoop-provided is that the hadoop and its dependent jars are available as part of the hadoop installation and we can link them to Spark classpath. They need not be bundled inside Spark assembly itself. - In a Hadoop 2.5 installation, there are no curator jars. So even after you add all the jars inside Hadoop installation dir, you will have this problem. This does not happen with Hadoop 2.7 because curator jars are present under its share/hadoop/common/lib dir. - Since curator jar is explicitly being used by Spark class ZooKeeperLeaderElectionAgent (outside the context of Hadoop itself), this jar should not be subject to any profile. Maybe this was done to avoid curator version conflict between hadoop and Spark, but I think this is still a regression when a customer using Hadoop 2.5 upgrades from Spark 1.2 to Spark 1.3 that was built using hadoop-provided profile. > Enhance and extend hadoop-provided profile > -- > > Key: SPARK-4048 > URL: https://issues.apache.org/jira/browse/SPARK-4048 > Project: Spark > Issue Type: Improvement > Components: Build >Affects Versions: 1.2.0 >Reporter: Marcelo Vanzin >Assignee: Marcelo Vanzin > Fix For: 1.3.0 > > > The hadoop-provided profile is used to not package Hadoop dependencies inside > the Spark assembly. It works, sort of, but it could use some enhancements. A > quick list: > - It doesn't include all things that could be removed from the assembly > - It doesn't work well when you're publishing artifacts based on it > (SPARK-3812 fixes this) > - There are other dependencies that could use similar treatment: Hive, HBase > (for the examples), Flume, Parquet, maybe others I'm missing at the moment. > - Unit tests, more specifically, those that use local-cluster mode, do not > work when the assembly is built with this profile enabled. > - The scripts to launch Spark jobs do not add needed "provided" jars to the > classpath when this profile is enabled, leaving it for people to figure that > out for themselves. > - The examples assembly duplicates a lot of things in the main assembly. > Part of this task is selfish since we build internally with this profile and > we'd like to make it easier for us to merge changes without having to keep > too many patches on top of upstream. But those feel like good improvements to > me, regardless. -- 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-4048) Enhance and extend hadoop-provided profile
[ https://issues.apache.org/jira/browse/SPARK-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14567746#comment-14567746 ] Kannan Rajah commented on SPARK-4048: - This fix has caused a regression for Spark built using hadoop-provided mode with Hadoop 2.5. Hadoop 2.5 does not contain curator jars. But spark class ZooKeeperLeaderElectionAgent needs it. This causes spark master to fail. 15/06/01 06:35:24 ERROR ActorSystemImpl: Uncaught fatal error from thread [sparkMaster-akka.actor.default-dispatcher-4] shutting down ActorSystem [sparkMaster] java.lang.NoClassDefFoundError: org/apache/curator/framework/recipes/leader/LeaderLatchListener at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:800) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:449) at java.net.URLClassLoader.access$100(URLClassLoader.java:71) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) at org.apache.spark.deploy.master.ZooKeeperRecoveryModeFactory.createLeaderElectionAgent(RecoveryModeFactory.scala:69) > Enhance and extend hadoop-provided profile > -- > > Key: SPARK-4048 > URL: https://issues.apache.org/jira/browse/SPARK-4048 > Project: Spark > Issue Type: Improvement > Components: Build >Affects Versions: 1.2.0 >Reporter: Marcelo Vanzin >Assignee: Marcelo Vanzin > Fix For: 1.3.0 > > > The hadoop-provided profile is used to not package Hadoop dependencies inside > the Spark assembly. It works, sort of, but it could use some enhancements. A > quick list: > - It doesn't include all things that could be removed from the assembly > - It doesn't work well when you're publishing artifacts based on it > (SPARK-3812 fixes this) > - There are other dependencies that could use similar treatment: Hive, HBase > (for the examples), Flume, Parquet, maybe others I'm missing at the moment. > - Unit tests, more specifically, those that use local-cluster mode, do not > work when the assembly is built with this profile enabled. > - The scripts to launch Spark jobs do not add needed "provided" jars to the > classpath when this profile is enabled, leaving it for people to figure that > out for themselves. > - The examples assembly duplicates a lot of things in the main assembly. > Part of this task is selfish since we build internally with this profile and > we'd like to make it easier for us to merge changes without having to keep > too many patches on top of upstream. But those feel like good improvements to > me, regardless. -- 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-1529) Support DFS based shuffle in addition to Netty shuffle
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kannan Rajah updated SPARK-1529: Description: In some environments, like with MapR, local volumes are accessed through the Hadoop filesystem interface. Shuffle is implemented by writing intermediate data to local disk and serving it to remote node using Netty as a transport mechanism. We want to provide an HDFS based shuffle such that data can be written to HDFS (instead of local disk) and served using HDFS API on the remote nodes. This could involve exposing a file system abstraction to Spark shuffle and have 2 modes of running it. In default mode, it will write to local disk and in the DFS mode, it will write to HDFS. (was: In some environments, like with MapR, local volumes are accessed through the Hadoop filesystem interface. We should allow setting spark.local.dir to a Hadoop filesystem location. ) > Support DFS based shuffle in addition to Netty shuffle > -- > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. Shuffle is implemented by writing intermediate > data to local disk and serving it to remote node using Netty as a transport > mechanism. We want to provide an HDFS based shuffle such that data can be > written to HDFS (instead of local disk) and served using HDFS API on the > remote nodes. This could involve exposing a file system abstraction to Spark > shuffle and have 2 modes of running it. In default mode, it will write to > local disk and in the DFS mode, it will write to HDFS. -- 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-1529) Support DFS based shuffle in addition to Netty shuffle
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kannan Rajah updated SPARK-1529: Summary: Support DFS based shuffle in addition to Netty shuffle (was: Support setting spark.local.dirs to a hadoop FileSystem ) > Support DFS based shuffle in addition to Netty shuffle > -- > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14544563#comment-14544563 ] Kannan Rajah commented on SPARK-1529: - Just wanted to check if folks got a chance to review the changes. If you have any concerns, I will be happy to address them. > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-6511) Publish "hadoop provided" build with instructions for different distros
[ https://issues.apache.org/jira/browse/SPARK-6511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14493592#comment-14493592 ] Kannan Rajah commented on SPARK-6511: - [~pwendell] Just wanted to let you know that we also have a way to add hive and hbase jars to the classpath. This is useful when a setup has multiple versions of hive and hbase installed, but a Spark version will only work with specific version. We have some utility scripts to generate the right classpath entries based on a supported version of hive, hbase. If you think this will be useful in Apache distribution, I can create a JIRA and share the code. At a high level, there are 3 files: - compatibility.version: File that holds supported versions for each ecosystem component. hive_versions=0.13,0.12 hbase_versions=0.98 - compatible_version.sh: Returns the compatible version for a component by looking up compatibilty.version file. The first version that is available on the node is used. - generate_classpath.sh: Uses the above 2 files to generate the classpath. This script is used in spark-env.sh to generate classpath based on hive and hbase. > Publish "hadoop provided" build with instructions for different distros > --- > > Key: SPARK-6511 > URL: https://issues.apache.org/jira/browse/SPARK-6511 > Project: Spark > Issue Type: Improvement > Components: Build >Reporter: Patrick Wendell > > Currently we publish a series of binaries with different Hadoop client jars. > This mostly works, but some users have reported compatibility issues with > different distributions. > One improvement moving forward might be to publish a binary build that simply > asks you to set HADOOP_HOME to pick up the Hadoop client location. That way > it would work across multiple distributions, even if they have subtle > incompatibilities with upstream Hadoop. > I think a first step for this would be to produce such a build for the > community and see how well it works. One potential issue is that our fancy > excludes and dependency re-writing won't work with the simpler "append > Hadoop's classpath to Spark". Also, how we deal with the Hive dependency is > unclear, i.e. should we continue to bundle Spark's Hive (which has some fixes > for dependency conflicts) or do we allow for linking against vanilla Hive at > runtime. -- 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-6511) Publish "hadoop provided" build with instructions for different distros
[ https://issues.apache.org/jira/browse/SPARK-6511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14493585#comment-14493585 ] Kannan Rajah commented on SPARK-6511: - As requested by Patrick, here is an example of what we use in spark-env.sh for MapR distribution. MAPR_HADOOP_CLASSPATH=`hadoop classpath` MAPR_SPARK_CLASSPATH="$MAPR_HADOOP_CLASSPATH:$MAPR_HADOOP_HBASE_VERSION" MAPR_HADOOP_JNI_PATH=`hadoop jnipath` export SPARK_LIBRARY_PATH=$MAPR_HADOOP_JNI_PATH SPARK_SUBMIT_CLASSPATH=$SPARK_SUBMIT_CLASSPATH:$MAPR_SPARK_CLASSPATH SPARK_SUBMIT_LIBRARY_PATH=$SPARK_SUBMIT_LIBRARY_PATH:$MAPR_HADOOP_JNI_PATH export SPARK_SUBMIT_CLASSPATH export SPARK_SUBMIT_LIBRARY_PATH > Publish "hadoop provided" build with instructions for different distros > --- > > Key: SPARK-6511 > URL: https://issues.apache.org/jira/browse/SPARK-6511 > Project: Spark > Issue Type: Improvement > Components: Build >Reporter: Patrick Wendell > > Currently we publish a series of binaries with different Hadoop client jars. > This mostly works, but some users have reported compatibility issues with > different distributions. > One improvement moving forward might be to publish a binary build that simply > asks you to set HADOOP_HOME to pick up the Hadoop client location. That way > it would work across multiple distributions, even if they have subtle > incompatibilities with upstream Hadoop. > I think a first step for this would be to produce such a build for the > community and see how well it works. One potential issue is that our fancy > excludes and dependency re-writing won't work with the simpler "append > Hadoop's classpath to Spark". Also, how we deal with the Hive dependency is > unclear, i.e. should we continue to bundle Spark's Hive (which has some fixes > for dependency conflicts) or do we allow for linking against vanilla Hive at > runtime. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14493251#comment-14493251 ] Kannan Rajah commented on SPARK-1529: - You can use the Compare functionality to see a single page of diffs across commits. Here is the link: https://github.com/rkannan82/spark/compare/4aaf48d46d13129f0f9bdafd771dd80fe568a7dc...rkannan82:7195353a31f7cfb087ec804b597b01fb362bc3f6 A few clarifications. 1. There are 2 reasons for introducing a FileSystem abstraction in Spark instead of directly using Hadoop FileSystem. - There are Spark shuffle specific APIs that needed abstraction. Please take a look at this code: https://github.com/rkannan82/spark/blob/dfs_shuffle/core/src/main/scala/org/apache/spark/storage/FileSystem.scala - For local file system access, we can choose to circumvent using Hadoop's local file system implementation if its not efficient. If you look at LocalFileSystem.scala, for most APIs, it just delegates to the old code of using Spark's disk block manager, etc. In fact, we can just look at this single class and determine if we will hit any performance degradation for the default Apache shuffle code path. https://github.com/rkannan82/spark/blob/dfs_shuffle/core/src/main/scala/org/apache/spark/storage/LocalFileSystem.scala 2. During the write phase, we shuffle to HDFS instead of local file system. While reading back, we don't use the Netty based transport that Apache shuffle uses. Instead we have a new implementation called DFSShuffleClient that reads from HDFS. That is the main difference. https://github.com/rkannan82/spark/blob/dfs_shuffle/network/shuffle/src/main/java/org/apache/spark/network/shuffle/DFSShuffleClient.java > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14491869#comment-14491869 ] Kannan Rajah commented on SPARK-1529: - [~pwendell] The default code path still uses the FileChannel, memory mapping techniques. I just provided an abstraction called FileSystem.scala (not Hadoop's FileSystem.java). LocalFileSystem.scala delegates the call to existing Spark code path that uses FileChannel. I am using Hadoop's RawLocalFileSystem class just to get an InputStream, OutputStream. And this internally also uses FileChannel. Please see RawLocalFileSystem.LocalFSFileInputStream. It is just a wrapper on java.io.FileInputStream. Going back to why I considered this approach. It will allow us to reuse all the logic currently used by SortShuffle code path. We would have to implement pretty much everything that's been done by Spark to do the shuffle on HDFS. We are in the processing of running some performance tests to understand the impact of the change. One of the main things we will be verifying is if there is any performance degradation introduced in the default code path and fix if there is any. Is this acceptable? > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14491834#comment-14491834 ] Kannan Rajah commented on SPARK-1529: - Thanks. FYI, I have pushed few more commits to my repo to handle all the TODOs and bug fixes. So you can track this branch for all the changes: https://github.com/rkannan82/spark/commits/dfs_shuffle > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14484022#comment-14484022 ] Kannan Rajah commented on SPARK-1529: - [~liancheng] Will you be able to take a look at the code and provide some feedback? > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14385648#comment-14385648 ] Kannan Rajah commented on SPARK-1529: - I have pushed the first round of commits to my repo. I would like to get some early feedback on the overall design. https://github.com/rkannan82/spark/commits/dfs_shuffle Commits: https://github.com/rkannan82/spark/commit/ce8b430512b31e932ffdab6e0a2c1a6a1768ffbf https://github.com/rkannan82/spark/commit/8f5415c248c0a9ca5ad3ec9f48f839b24c259813 https://github.com/rkannan82/spark/commit/d9d179ba6c685cc8eb181f442e9bd6ad91cc4290 > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kannan Rajah updated SPARK-1529: Attachment: (was: SparkShuffleUsingHDFS_API.pdf) > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kannan Rajah updated SPARK-1529: Attachment: Spark Shuffle using HDFS.pdf > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Kannan Rajah > Attachments: Spark Shuffle using HDFS.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14361978#comment-14361978 ] Kannan Rajah commented on SPARK-1529: - Can someone assign this bug to me? I am working on a patch. > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Improvement > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > Attachments: SparkShuffleUsingHDFS_API.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-4852) Hive query plan deserialization failure caused by shaded hive-exec jar file when generating golden answers
[ https://issues.apache.org/jira/browse/SPARK-4852?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14356173#comment-14356173 ] Kannan Rajah commented on SPARK-4852: - We are hitting this issue in a production case, not just test. spark-sql UDF queries seem to trigger this code path. We are not using spark's version of hive in this setup. Example: spark-sql> FROM src SELECT nvl(key, ' '), value; 15/03/09 18:28:23 ERROR SparkSQLDriver: Failed in [FROM src SELECT nvl(key, ' '), value] java.lang.NoSuchMethodException: org.apache.hadoop.hive.ql.exec.Utilities.deserializeObjectByKryo(com.esotericsoftware.kryo.Kryo, java.io.InputStream, java.lang.Class) at java.lang.Class.getDeclaredMethod(Class.java:2009) at org.apache.spark.sql.hive.HiveFunctionWrapper.(Shim13.scala:67) org.apache.spark.sql.hive.HiveFunctionRegistry.lookupFunction(hiveUdfs.scala:59) at org.apache.spark.sql.hive.HiveContext$$anon$2.org$apache$spark$sql$catalyst$analysis$OverrideFunctionRegistry$$super$lookupFunction(HiveContext.scala:258) at org.apache.spark.sql.catalyst.analysis.OverrideFunctionRegistry$$anonfun$lookupFunction$2.apply(FunctionRegistry.scala:41) at org.apache.spark.sql.catalyst.analysis.OverrideFunctionRegistry$$anonfun$lookupFunction$2.apply(FunctionRegistry.scala:41) at scala.Option.getOrElse(Option.scala:120) at org.apache.spark.sql.catalyst.analysis.OverrideFunctionRegistry$class.lookupFunction(FunctionRegistry.scala:41) at org.apache.spark.sql.hive.HiveContext$$anon$2.lookupFunction(HiveContext.scala:258) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions$$anonfun$apply$10$$anonfun$applyOrElse$2.applyOrElse(Analyzer.scala:220) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveFunctions$$anonfun$apply$10$$anonfun$applyOrElse$2.applyOrElse(Analyzer.scala:218) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:144) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:162) at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47) at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273) at scala.collection.AbstractIterator.to(Iterator.scala:1157) at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157) at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252) at scala.collection.AbstractIterator.toArray(Iterator.scala:1157) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildrenDown(TreeNode.scala:191) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:147) at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$transformExpressionDown$1(QueryPlan.scala:71) at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$1$$anonfun$apply$1.apply(QueryPlan.scala:85) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$1.apply(QueryPlan.scala:84) at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47) at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273) at scala.collection.AbstractIterator.to(Iterator.scala:1157) at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157) at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252) a
[jira] [Updated] (SPARK-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kannan Rajah updated SPARK-1529: Attachment: (was: MapRShuffleOnSpark.pdf) > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > Attachments: SparkShuffleUsingHDFS_API.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kannan Rajah updated SPARK-1529: Attachment: SparkShuffleUsingHDFS_API.pdf > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > Attachments: SparkShuffleUsingHDFS_API.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14288343#comment-14288343 ] Kannan Rajah edited comment on SPARK-1529 at 1/22/15 10:30 PM: --- Attached: High level proposal of the changes to make shuffle use Distributed File System instead of local file system. was (Author: rkannan82): High level proposal of the changes to make shuffle use Distributed File System instead of local file system. > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > Attachments: MapRShuffleOnSpark.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kannan Rajah updated SPARK-1529: Attachment: MapRShuffleOnSpark.pdf High level proposal of the changes to make shuffle use Distributed File System instead of local file system. > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > Attachments: MapRShuffleOnSpark.pdf > > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14288341#comment-14288341 ] Kannan Rajah commented on SPARK-1529: - [~pwendell] I would like us to consider the option of reusing the write code path of existing shuffle implementation instead of implementing from scratch. This will allow us to take advantage of all the optimizations that are already done and will be done in future. Only the read code path needs to be reimplemented fully as we don't need all the shuffle server logic. There are a handful of shuffle classes that need to use the HDFS abstractions in order to achieve this. I have attached a high level proposal. Let me know your thoughts. Write IndexShufflleBlockManager, SortShuffleWriter, ExternalSorter, BlockObjectWriter. Read BlockStoreShuffleFetcher, HashShuffleReader > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14263928#comment-14263928 ] Kannan Rajah commented on SPARK-1529: - In a MapR distribution, these files need to go into MapR volume and not local disk. This MapR volume is local to the node though, but its part of the distributed file system. This can be achieved in 2 ways: 1. Expose the MapR file system as a NFS mount point. Now, you can use the normal java.io API and data will still get written to MapR volume instead of local disk. 2. Use HDFS API (with underlying MapR implementation) instead of java.io to make the IO go to MapR volume. > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14263603#comment-14263603 ] Kannan Rajah commented on SPARK-1529: - It cannot preclude the use of java.io completely. If there are java.io. APIs that are needed for some use case, then you cannot use the HDFS API. But that is the case normally. The NFS mount based workaround is not as efficient as accessing it through the HDFS interface. Hence the need. > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14263600#comment-14263600 ] Kannan Rajah commented on SPARK-1529: - [~lian cheng] Can you upload this prototype patch so that I can reuse it? What branch was it based off? When I start making new changes, I suppose I can do it against master branch, right? > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14263287#comment-14263287 ] Kannan Rajah edited comment on SPARK-1529 at 1/2/15 10:53 PM: -- [~lian cheng] [~pwendell] I want to work on this JIRA. It's been a while since there has been any update. So can you please share what the current status is? Has there been a consensus on replacing the file API with a HDFS kind of interface and plugging in the right implementation? I will be looking at the code base in the mean time. was (Author: rkannan82): [~lian cheng] [~pwendell]] I want to work on this JIRA. It's been a while since there has been any update. So can you please share what the current status is? Has there been a consensus on replacing the file API with a HDFS kind of interface and plugging in the right implementation? I will be looking at the code base in the mean time. > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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-1529) Support setting spark.local.dirs to a hadoop FileSystem
[ https://issues.apache.org/jira/browse/SPARK-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14263287#comment-14263287 ] Kannan Rajah commented on SPARK-1529: - [~lian cheng] [~pwendell]] I want to work on this JIRA. It's been a while since there has been any update. So can you please share what the current status is? Has there been a consensus on replacing the file API with a HDFS kind of interface and plugging in the right implementation? I will be looking at the code base in the mean time. > Support setting spark.local.dirs to a hadoop FileSystem > > > Key: SPARK-1529 > URL: https://issues.apache.org/jira/browse/SPARK-1529 > Project: Spark > Issue Type: Bug > Components: Spark Core >Reporter: Patrick Wendell >Assignee: Cheng Lian > > In some environments, like with MapR, local volumes are accessed through the > Hadoop filesystem interface. We should allow setting spark.local.dir to a > Hadoop filesystem location. -- 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