Re: More instances = slower Spark job

2017-10-01 Thread Steve Loughran
On 28 Sep 2017, at 15:27, Daniel Siegmann > wrote: Can you kindly explain how Spark uses parallelism for bigger (say 1GB) text file? Does it use InputFormat do create multiple splits and creates 1 partition per split?

Re: More instances = slower Spark job

2017-10-01 Thread Steve Loughran
> On 28 Sep 2017, at 14:45, ayan guha wrote: > > Hi > > Can you kindly explain how Spark uses parallelism for bigger (say 1GB) text > file? Does it use InputFormat do create multiple splits and creates 1 > partition per split? Yes, Input formats give you their splits,

Re: More instances = slower Spark job

2017-10-01 Thread Jeroen Miller
Vadim's "scheduling within an application" approach turned out to be excellent, at least on a single node with the CPU usage reaching about 90%. I directly implemented the code template that Vadim kindly provided: parallel_collection_paths.foreach( path => { val lines =

Re: More instances = slower Spark job

2017-10-01 Thread Gourav Sengupta
Hi Jeroen, I do not believe that I completely agree with the idea that you will be spending more time and memory that way. But if that was also the case why are you not using data frames and UDF? Regards, Gourav On Sun, Oct 1, 2017 at 6:17 PM, Jeroen Miller wrote: >

Re: More instances = slower Spark job

2017-10-01 Thread Jeroen Miller
On Fri, Sep 29, 2017 at 12:20 AM, Gourav Sengupta wrote: > Why are you not using JSON reader of SPARK? Since the filter I want to perform is so simple, I do not want to spend time and memory to deserialise the JSON lines. Jeroen

Re: More instances = slower Spark job

2017-09-29 Thread Vadim Semenov
Hi Jeroen, > However, am I correct in assuming that all the filtering will be then performed on the driver (since the .gz files are not splittable), albeit in several threads? Filtering will not happen on the driver, it'll happen on executors, since `spark.read.json(…).filter(…).write(…)` is a

Re: More instances = slower Spark job

2017-09-29 Thread Gourav Sengupta
I think that the best option is to see whether data frames option of reading JSON files works or not. On Fri, Sep 29, 2017 at 3:53 PM, Alexander Czech < alexander.cz...@googlemail.com> wrote: > Does each gzip file look like this: > > {json1} > {json2} > {json3} > > meaning that each line is a

Re: More instances = slower Spark job

2017-09-29 Thread Alexander Czech
Does each gzip file look like this: {json1} {json2} {json3} meaning that each line is a separate json object? I proccess a similar large file batch and what I do is this: input.txt # each line in input.txt represents a path to a gzip file each containing a json object every line my_rdd =

Re: More instances = slower Spark job

2017-09-28 Thread Gourav Sengupta
I think that Vadim's response makes a lot of sense in terms of utilizing SPARK. Why are you not using JSON reader of SPARK? Your input has to follow a particular JSON style, but then it would be interesting to know whether you have looked into it at all. If you are going to read them only once

Re: More instances = slower Spark job

2017-09-28 Thread Jeroen Miller
On Thu, Sep 28, 2017 at 9:02 PM, Jörn Franke wrote: > It looks to me a little bit strange. First json.gz files are single threaded, > ie each file can only be processed by one thread (so it is good to have many > files of around 128 MB to 512 MB size each). Indeed.

Re: More instances = slower Spark job

2017-09-28 Thread Vadim Semenov
Instead of having one job, you can try processing each file in a separate job, but run multiple jobs in parallel within one SparkContext. Something like this should work for you, it'll submit N jobs from the driver, the jobs will run independently, but executors will dynamically work on different

Re: More instances = slower Spark job

2017-09-28 Thread Jörn Franke
It looks to me a little bit strange. First json.gz files are single threaded, ie each file can only be processed by one thread (so it is good to have many files of around 128 MB to 512 MB size each). Then what you do in the code is already done by the data source. There is no need to read the

Re: More instances = slower Spark job

2017-09-28 Thread Jeroen Miller
More details on what I want to achieve. Maybe someone can suggest a course of action. My processing is extremely simple: reading .json.gz text files, filtering each line according a regex, and saving the surviving lines in a similarly named .gz file. Unfortunately changing the data format is

Re: More instances = slower Spark job

2017-09-28 Thread Daniel Siegmann
On Thu, Sep 28, 2017 at 7:23 AM, Gourav Sengupta wrote: > > I will be very surprised if someone tells me that a 1 GB CSV text file is > automatically split and read by multiple executors in SPARK. It does not > matter whether it stays in HDFS, S3 or any other system. >

Re: More instances = slower Spark job

2017-09-28 Thread Daniel Siegmann
> Can you kindly explain how Spark uses parallelism for bigger (say 1GB) > text file? Does it use InputFormat do create multiple splits and creates 1 > partition per split? Also, in case of S3 or NFS, how does the input split > work? I understand for HDFS files are already pre-split so Spark can

Re: More instances = slower Spark job

2017-09-28 Thread Gourav Sengupta
Hi, I will be very surprised if someone tells me that a 1 GB CSV text file is automatically split and read by multiple executors in SPARK. It does not matter whether it stays in HDFS, S3 or any other system. Now if someone tells me that in case I have a smaller CSV file of 100MB size and that

Re: More instances = slower Spark job

2017-09-28 Thread ayan guha
Hi Can you kindly explain how Spark uses parallelism for bigger (say 1GB) text file? Does it use InputFormat do create multiple splits and creates 1 partition per split? Also, in case of S3 or NFS, how does the input split work? I understand for HDFS files are already pre-split so Spark can use

Re: More instances = slower Spark job

2017-09-28 Thread Daniel Siegmann
> no matter what you do and how many nodes you start, in case you have a > single text file, it will not use parallelism. > This is not true, unless the file is small or is gzipped (gzipped files cannot be split).

RE: More instances = slower Spark job

2017-09-28 Thread JG Perrin
To: Tejeshwar J1 <tejeshwar...@globallogic.com.invalid> Cc: Jeroen Miller <bluedasya...@gmail.com>; user@spark.apache.org Subject: Re: More instances = slower Spark job Also check if the compression algorithm you use is splittable? Thanks, Sonal Nube Technologies<http://www.nubetech

Re: More instances = slower Spark job

2017-09-28 Thread Gourav Sengupta
Hi, no matter what you do and how many nodes you start, in case you have a single text file, it will not use parallelism. Therefore there are options of transferring the textfile to parquet, and other formats, or just splitting the text file itself into several individual files. Please do let

Re: More instances = slower Spark job

2017-09-28 Thread Sonal Goyal
yo* > serialization. > > > > > > Thanks, > > Tejeshwar > > > > > > *From:* Jeroen Miller [mailto:bluedasya...@gmail.com] > *Sent:* Thursday, September 28, 2017 2:11 PM > *To:* user@spark.apache.org > *Subject:* More instances = slower Spark job >

Re: More instances = slower Spark job

2017-09-28 Thread Steve Loughran
On 28 Sep 2017, at 09:41, Jeroen Miller > wrote: Hello, I am experiencing a disappointing performance issue with my Spark jobs as I scale up the number of instances. The task is trivial: I am loading large (compressed) text files from S3,

RE: More instances = slower Spark job

2017-09-28 Thread Tejeshwar J1
, Tejeshwar *From:* Jeroen Miller [mailto:bluedasya...@gmail.com] *Sent:* Thursday, September 28, 2017 2:11 PM *To:* user@spark.apache.org *Subject:* More instances = slower Spark job Hello, I am experiencing a disappointing performance issue with my Spark jobs as I scale up the number

More instances = slower Spark job

2017-09-28 Thread Jeroen Miller
Hello, I am experiencing a disappointing performance issue with my Spark jobs as I scale up the number of instances. The task is trivial: I am loading large (compressed) text files from S3, filtering out lines that do not match a regex, counting the numbers of remaining lines and saving the