Re: Understanding shuffle file name conflicts

2015-03-24 Thread Saisai Shao
Yes as Josh said, when application is started, Spark will create a unique application-wide folder for related temporary files. And jobs in this application will have a unique shuffle id with unique file names, so shuffle stages within app will not meet name conflicts. Also shuffle files between ap

Re: Understanding shuffle file name conflicts

2015-03-24 Thread Josh Rosen
Which version of Spark are you using? What do you mean when you say that you used a hardcoded location for shuffle files? If you look at the current DiskBlockManager code, it looks like it will create a per-application subdirectory in each of the local root directories. Here's the call to create

Re: Understanding shuffle file name conflicts

2015-03-24 Thread Kannan Rajah
Saisai, This is the not the case when I use spark-submit to run 2 jobs, one after another. The shuffle id remains the same. -- Kannan On Tue, Mar 24, 2015 at 7:35 PM, Saisai Shao wrote: > Hi Kannan, > > As I know the shuffle Id in ShuffleDependency will be increased, so even > if you run the s

Re: hadoop input/output format advanced control

2015-03-24 Thread Patrick Wendell
I see - if you look, in the saving functions we have the option for the user to pass an arbitrary Configuration. https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala#L894 It seems fine to have the same option for the loading functions, if it'

Re: Spark SQL(1.3.0) "import sqlContext.implicits._" seems not work for converting a case class RDD to DataFrame

2015-03-24 Thread Reynold Xin
In particular: http://spark.apache.org/docs/latest/sql-programming-guide.html "Additionally, the implicit conversions now only augment RDDs that are composed of Products (i.e., case classes or tuples) with a method toDF, instead of applying automatically." On Tue, Mar 24, 2015 at 9:07 PM, Ted

Re: Spark SQL(1.3.0) "import sqlContext.implicits._" seems not work for converting a case class RDD to DataFrame

2015-03-24 Thread Ted Yu
Please take a look at: ./sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala ./sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala Cheers On Tue, Mar 24, 2015 at 8:46 PM, Zhiwei Chan wrote: > Hi all, > > I just upgraded spark from 1.2.1 to 1.3.0, and changed the "imp

Spark SQL(1.3.0) "import sqlContext.implicits._" seems not work for converting a case class RDD to DataFrame

2015-03-24 Thread Zhiwei Chan
Hi all, I just upgraded spark from 1.2.1 to 1.3.0, and changed the "import sqlContext.createSchemaRDD" to "import sqlContext.implicits._" in my code. (I scan the programming guide and it seems this is the only change I need to do). But it come to an error when run compile as following: >>> [ERRO

Re: Understanding shuffle file name conflicts

2015-03-24 Thread Saisai Shao
Hi Kannan, As I know the shuffle Id in ShuffleDependency will be increased, so even if you run the same job twice, the shuffle dependency as well as shuffle id is different, so the shuffle file name which is combined by (shuffleId+mapId+reduceId) will be changed, so there's no name conflict even i

RE: Using CUDA within Spark / boosting linear algebra

2015-03-24 Thread Ulanov, Alexander
Hi, I am trying to use nvblas with netlib-java from Spark. nvblas functions should replace current blas functions calls after executing LD_PRELOAD as suggested in http://docs.nvidia.com/cuda/nvblas/#Usage without any changes to netlib-java. It seems to work for simple Java example, but I cannot

Re: Experience using binary packages on various Hadoop distros

2015-03-24 Thread Patrick Wendell
We can probably better explain that if you are not using HDFS or YARN, you can download any binary. However, my question was about if the existing binaries do not work well with newer Hadoop versions, which I heard some people suggest but I'm looking for more specific issues. On Tue, Mar 24, 2015

Re: Experience using binary packages on various Hadoop distros

2015-03-24 Thread Jey Kottalam
Could we gracefully fallback to an in-tree Hadoop binary (e.g. 1.0.4) in that case? I think many new Spark users are confused about why Spark has anything to do with Hadoop, e.g. I could see myself being confused when the download page asks me to select a "package type". I know that what I want is

Re: hadoop input/output format advanced control

2015-03-24 Thread Koert Kuipers
the (compression) codec parameter that is now part of many saveAs... methods came from a similar need. see SPARK-763 hadoop has many options like this. you either going to have to allow many more of these optional arguments to all the methods that r

Re: hadoop input/output format advanced control

2015-03-24 Thread Koert Kuipers
i would like to use objectFile with some tweaks to the hadoop conf. currently there is no way to do that, except recreating objectFile myself. and some of the code objectFile uses i have no access to, since its private to spark. On Tue, Mar 24, 2015 at 2:59 PM, Patrick Wendell wrote: > Yeah - t

Re: Experience using binary packages on various Hadoop distros

2015-03-24 Thread Matei Zaharia
Just a note, one challenge with the BYOH version might be that users who download that can't run in local mode without also having Hadoop. But if we describe it correctly then hopefully it's okay. Matei > On Mar 24, 2015, at 3:05 PM, Patrick Wendell wrote: > > Hey All, > > For a while we've

Re: Spark-thriftserver Issue

2015-03-24 Thread Zhan Zhang
You can try to set it in spark-env.sh. # - SPARK_LOG_DIR Where log files are stored. (Default: ${SPARK_HOME}/logs) # - SPARK_PID_DIR Where the pid file is stored. (Default: /tmp) Thanks. Zhan Zhang On Mar 24, 2015, at 12:10 PM, Anubhav Agarwal mailto:anubha...@gmail.com>> wrote:

Re: Spark-thriftserver Issue

2015-03-24 Thread Anubhav Agarwal
Zhan specifying port fixed the port issue. Is it possible to specify the log directory while starting the spark thriftserver? Still getting this error even through the folder exists and everyone has permission to use that directory. drwxr-xr-x 2 root root 4096 Mar 24 19:04 spark-

Experience using binary packages on various Hadoop distros

2015-03-24 Thread Patrick Wendell
Hey All, For a while we've published binary packages with different Hadoop client's pre-bundled. We currently have three interfaces to a Hadoop cluster (a) the HDFS client (b) the YARN client (c) the Hive client. Because (a) and (b) are supposed to be backwards compatible interfaces. My working a

Re: hadoop input/output format advanced control

2015-03-24 Thread Patrick Wendell
Yeah - to Nick's point, I think the way to do this is to pass in a custom conf when you create a Hadoop RDD (that's AFAIK why the conf field is there). Is there anything you can't do with that feature? On Tue, Mar 24, 2015 at 11:50 AM, Nick Pentreath wrote: > Imran, on your point to read multiple

Re: Any guidance on when to back port and how far?

2015-03-24 Thread Patrick Wendell
My philosophy has been basically what you suggested, Sean. One thing you didn't mention though is if a bug fix seems complicated, I will think very hard before back-porting it. This is because "fixes" can introduce their own new bugs, in some cases worse than the original issue. It's really bad to

Re: hadoop input/output format advanced control

2015-03-24 Thread Nick Pentreath
Imran, on your point to read multiple files together in a partition, is it not simpler to use the approach of copy Hadoop conf and set per-RDD settings for min split to control the input size per partition, together with something like CombineFileInputFormat? On Tue, Mar 24, 2015 at 5:28 PM, Imran

Re: Any guidance on when to back port and how far?

2015-03-24 Thread Michael Armbrust
Two other criteria that I use when deciding what to backport: - Is it a regression from a previous minor release? I'm much more likely to backport fixes in this case, as I'd love for most people to stay up to date. - How scary is the change? I think the primary goal is stability of the maintena

Understanding shuffle file name conflicts

2015-03-24 Thread Kannan Rajah
I am working on SPARK-1529. I ran into an issue with my change, where the same shuffle file was being reused across 2 jobs. Please note this only happens when I use a hard coded location to use for shuffle files, say "/tmp". It does not happen with normal code path that uses DiskBlockManager to pic

Re: Spark Executor resources

2015-03-24 Thread Zoltán Zvara
I'm trying to log Tasks to understand physical plan and to visualize which RDD's which partition is currently computed from which creation site along with other information. I want to charge the TaskRunner to do this before actually invoking runTask() on Task and again just before giving the Task t

Re: Spark Executor resources

2015-03-24 Thread Sandy Ryza
That's correct. What's the reason this information is needed? -Sandy On Tue, Mar 24, 2015 at 11:41 AM, Zoltán Zvara wrote: > Thank you for your response! > > I guess the (Spark)AM, who gives the container leash to the NM (along with > the executor JAR and command to run) must know how many CPU

Re: Spark Executor resources

2015-03-24 Thread Zoltán Zvara
Thank you for your response! I guess the (Spark)AM, who gives the container leash to the NM (along with the executor JAR and command to run) must know how many CPU or RAM that container capped, isolated at. There must be a resource vector along the encrypted container leash if I'm right that descr

Re: Spark Executor resources

2015-03-24 Thread Sandy Ryza
Hi Zoltan, If running on YARN, the YARN NodeManager starts executors. I don't think there's a 100% precise way for the Spark executor way to know how many resources are allotted to it. It can come close by looking at the Spark configuration options used to request it (spark.executor.memory and s

Re: hadoop input/output format advanced control

2015-03-24 Thread Imran Rashid
I think this would be a great addition, I totally agree that you need to be able to set these at a finer context than just the SparkContext. Just to play devil's advocate, though -- the alternative is for you just subclass HadoopRDD yourself, or make a totally new RDD, and then you could expose wh

Re: Optimize the first map reduce of DStream

2015-03-24 Thread Zoltán Zvara
​AFAIK Spark Streaming can not work in a way like this. Transformations are made on DStreams, where DStreams are basically hold (time, allocatedBlocksForBatch) pairs.​ Allocated blocks are allocated by the JobGenerator, unallocated blocks (infos) are collected by ReceivedBlockTracker. In Spark Stre

Re: Optimize the first map reduce of DStream

2015-03-24 Thread Bin Wang
I'm not looking for limit the block size. Here is another example. Say we want to count the lines from the stream in one hour. In a normal program, we may write it like this: int sum = 0 while (line = getFromStream()) { store(line) // store the line into storage instead of memory. sum++ }

Any guidance on when to back port and how far?

2015-03-24 Thread Sean Owen
So far, my rule of thumb has been: - Don't back-port new features or improvements in general, only bug fixes - Don't back-port minor bug fixes - Back-port bug fixes that seem important enough to not wait for the next minor release - Back-port site doc changes to the release most likely to go out n

Re: Optimize the first map reduce of DStream

2015-03-24 Thread Zoltán Zvara
There is a BlockGenerator on each worker node next to the ReceiverSupervisorImpl, which generates Blocks out of an ArrayBuffer in each interval (block_interval). These Blocks are passed to ReceiverSupervisorImpl, which throws these blocks to into the BlockManager for storage. BlockInfos are passed

Re: Optimize the first map reduce of DStream

2015-03-24 Thread Arush Kharbanda
The block size is configurable and that way I think you can reduce the block interval, to keep the block in memory only for the limiter interval? Is that what you are looking for? On Tue, Mar 24, 2015 at 1:38 PM, Bin Wang wrote: > Hi, > > I'm learning Spark and I find there could be some optimiz

Optimize the first map reduce of DStream

2015-03-24 Thread Bin Wang
Hi, I'm learning Spark and I find there could be some optimize for the current streaming implementation. Correct me if I'm wrong. The current streaming implementation put the data of one batch into memory (as RDD). But it seems not necessary. For example, if I want to count the lines which conta