Re: Spark speed performance

2014-11-02 Thread jan.zikes

Thank you, I would expect it to work as you write, but I am probably 
experiencing it working other way. But now it seems that Spark is generally 
trying to fit everything to RAM. I run Spark on YARN and I have wraped this to 
another question: 
http://apache-spark-user-list.1001560.n3.nabble.com/Spark-on-Yarn-probably-trying-to-load-all-the-data-to-RAM-td17908.html
__


coalesce() is a streaming operation if used without the second parameter, it 
does not put all the data in RAM. If used with the second parameter (shuffle = 
true), then it performs a shuffle, but still does not put all the data in RAM.
On Sat, Nov 1, 2014 at 12:09 PM, jan.zi...@centrum.cz jan.zi...@centrum.cz 
wrote:
Now I am getting to problems using:

distData = sc.textFile(sys.argv[2]).coalesce(10)
 
The problem is that it seems that Spark is trying to put all the data to RAM 
first and then perform coalesce. Do you know if there is something that would 
do coalesce on fly with for example fixed size of the partition? Do you think 
that something like this is possible? Unfortunately I am not able to find 
anything like this in the Spark documentation.

Thank you in advance for any advices or suggestions.

Best regards,
Jan 
__



Thank you very much lot of very small json files was exactly the speed 
performance problem, using coalesce makes my Spark program to run on single 
node only twice slower (even with starting Spark) than single node Python 
program, which is acceptable.

Jan 
__

Because the overhead between JVM and Python, single task will be
slower than your local Python scripts, but it's very easy to scale to
many CPUs.

Even one CPUs, it's not common that PySpark was 100 times slower. You
have many small files, each file will be processed by a task, which
will have about 100ms overhead (scheduled and executed), but the small
file can be processed in your single thread Python script in less than
1ms.

You could pack your json files into larger ones, or you could try to
merge the small tasks into larger one by coalesce(N), such as:

distData = sc.textFile(sys.argv[2]).coalesce(10)  # which will have 10
partitons (tasks)

Davies

On Sat, Oct 18, 2014 at 12:07 PM,  jan.zi...@centrum.cz 
jan.zi...@centrum.cz wrote:

Hi,

I have program that I have for single computer (in Python) exection and also
implemented the same for Spark. This program basically only reads .json from
which it takes one field and saves it back. Using Spark my program runs
aproximately 100 times slower on 1 master and 1 slave. So I would like to
ask where possibly might be the problem?

My Spark program looks like:



sc = SparkContext(appName=Json data preprocessor)

distData = sc.textFile(sys.argv[2])

json_extractor = JsonExtractor(sys.argv[1])

cleanedData = distData.flatMap(json_extractor.extract_json)

cleanedData.saveAsTextFile(sys.argv[3])

JsonExtractor only selects the data from field that is given by sys.argv[1].



My data are basically many small one json files, where is one json per line.

I have tried both, reading and writing the data from/to Amazon S3, local
disc on all the machines.

I would like to ask if there is something that I am missing or if Spark is
supposed to be so slow in comparison with the local non parallelized single
node program.



Thank you in advance for any suggestions or hints.



-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org 
user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org 
user-h...@spark.apache.org


-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org 
user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org 
user-h...@spark.apache.org


-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org 
user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org 
user-h...@spark.apache.org

-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org

Re: Spark speed performance

2014-11-01 Thread jan.zikes

Now I am getting to problems using:

distData = sc.textFile(sys.argv[2]).coalesce(10)
 
The problem is that it seems that Spark is trying to put all the data to RAM 
first and then perform coalesce. Do you know if there is something that would 
do coalesce on fly with for example fixed size of the partition? Do you think 
that something like this is possible? Unfortunately I am not able to find 
anything like this in the Spark documentation.

Thank you in advance for any advices or suggestions.

Best regards,
Jan 
__



Thank you very much lot of very small json files was exactly the speed 
performance problem, using coalesce makes my Spark program to run on single 
node only twice slower (even with starting Spark) than single node Python 
program, which is acceptable.

Jan 
__

Because the overhead between JVM and Python, single task will be
slower than your local Python scripts, but it's very easy to scale to
many CPUs.

Even one CPUs, it's not common that PySpark was 100 times slower. You
have many small files, each file will be processed by a task, which
will have about 100ms overhead (scheduled and executed), but the small
file can be processed in your single thread Python script in less than
1ms.

You could pack your json files into larger ones, or you could try to
merge the small tasks into larger one by coalesce(N), such as:

distData = sc.textFile(sys.argv[2]).coalesce(10)  # which will have 10
partitons (tasks)

Davies

On Sat, Oct 18, 2014 at 12:07 PM,  jan.zi...@centrum.cz wrote:

Hi,

I have program that I have for single computer (in Python) exection and also
implemented the same for Spark. This program basically only reads .json from
which it takes one field and saves it back. Using Spark my program runs
aproximately 100 times slower on 1 master and 1 slave. So I would like to
ask where possibly might be the problem?

My Spark program looks like:



sc = SparkContext(appName=Json data preprocessor)

distData = sc.textFile(sys.argv[2])

json_extractor = JsonExtractor(sys.argv[1])

cleanedData = distData.flatMap(json_extractor.extract_json)

cleanedData.saveAsTextFile(sys.argv[3])

JsonExtractor only selects the data from field that is given by sys.argv[1].



My data are basically many small one json files, where is one json per line.

I have tried both, reading and writing the data from/to Amazon S3, local
disc on all the machines.

I would like to ask if there is something that I am missing or if Spark is
supposed to be so slow in comparison with the local non parallelized single
node program.



Thank you in advance for any suggestions or hints.



-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org


-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org


-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org

Re: Spark speed performance

2014-11-01 Thread Aaron Davidson
coalesce() is a streaming operation if used without the second parameter,
it does not put all the data in RAM. If used with the second parameter
(shuffle = true), then it performs a shuffle, but still does not put all
the data in RAM.

On Sat, Nov 1, 2014 at 12:09 PM, jan.zi...@centrum.cz wrote:

 Now I am getting to problems using:

 distData = sc.textFile(sys.argv[2]).coalesce(10)



 The problem is that it seems that Spark is trying to put all the data to
 RAM first and then perform coalesce. Do you know if there is something
 that would do coalesce on fly with for example fixed size of the partition?
 Do you think that something like this is possible? Unfortunately I am not
 able to find anything like this in the Spark documentation.

 Thank you in advance for any advices or suggestions.

 Best regards,
 Jan

 __


 Thank you very much lot of very small json files was exactly the speed
 performance problem, using coalesce makes my Spark program to run on single
 node only twice slower (even with starting Spark) than single node Python
 program, which is acceptable.

 Jan
 __

 Because the overhead between JVM and Python, single task will be
 slower than your local Python scripts, but it's very easy to scale to
 many CPUs.

 Even one CPUs, it's not common that PySpark was 100 times slower. You
 have many small files, each file will be processed by a task, which
 will have about 100ms overhead (scheduled and executed), but the small
 file can be processed in your single thread Python script in less than
 1ms.

 You could pack your json files into larger ones, or you could try to
 merge the small tasks into larger one by coalesce(N), such as:

 distData = sc.textFile(sys.argv[2]).coalesce(10)  # which will have 10
 partitons (tasks)

 Davies

 On Sat, Oct 18, 2014 at 12:07 PM,  jan.zi...@centrum.cz wrote:
  Hi,
 
  I have program that I have for single computer (in Python) exection and
 also
  implemented the same for Spark. This program basically only reads .json
 from
  which it takes one field and saves it back. Using Spark my program runs
  aproximately 100 times slower on 1 master and 1 slave. So I would like to
  ask where possibly might be the problem?
 
  My Spark program looks like:
 
 
 
  sc = SparkContext(appName=Json data preprocessor)
 
  distData = sc.textFile(sys.argv[2])
 
  json_extractor = JsonExtractor(sys.argv[1])
 
  cleanedData = distData.flatMap(json_extractor.extract_json)
 
  cleanedData.saveAsTextFile(sys.argv[3])
 
  JsonExtractor only selects the data from field that is given by
 sys.argv[1].
 
 
 
  My data are basically many small one json files, where is one json per
 line.
 
  I have tried both, reading and writing the data from/to Amazon S3, local
  disc on all the machines.
 
  I would like to ask if there is something that I am missing or if Spark
 is
  supposed to be so slow in comparison with the local non parallelized
 single
  node program.
 
 
 
  Thank you in advance for any suggestions or hints.
 
 
 
  -
  To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
  For additional commands, e-mail: user-h...@spark.apache.org

 -
 To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
 For additional commands, e-mail: user-h...@spark.apache.org


 -
 To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
 For additional commands, e-mail: user-h...@spark.apache.org



Re: Spark speed performance

2014-10-19 Thread jan.zikes
Thank you very much lot of very small json files was exactly the speed 
performance problem, using coalesce makes my Spark program to run on single 
node only twice slower (even with starting Spark) than single node Python 
program, which is acceptable.

Jan 
__

Because the overhead between JVM and Python, single task will be
slower than your local Python scripts, but it's very easy to scale to
many CPUs.

Even one CPUs, it's not common that PySpark was 100 times slower. You
have many small files, each file will be processed by a task, which
will have about 100ms overhead (scheduled and executed), but the small
file can be processed in your single thread Python script in less than
1ms.

You could pack your json files into larger ones, or you could try to
merge the small tasks into larger one by coalesce(N), such as:

distData = sc.textFile(sys.argv[2]).coalesce(10)  # which will have 10
partitons (tasks)

Davies

On Sat, Oct 18, 2014 at 12:07 PM,  jan.zi...@centrum.cz wrote:
 Hi,

 I have program that I have for single computer (in Python) exection and also
 implemented the same for Spark. This program basically only reads .json from
 which it takes one field and saves it back. Using Spark my program runs
 aproximately 100 times slower on 1 master and 1 slave. So I would like to
 ask where possibly might be the problem?

 My Spark program looks like:



 sc = SparkContext(appName=Json data preprocessor)

 distData = sc.textFile(sys.argv[2])

 json_extractor = JsonExtractor(sys.argv[1])

 cleanedData = distData.flatMap(json_extractor.extract_json)

 cleanedData.saveAsTextFile(sys.argv[3])

 JsonExtractor only selects the data from field that is given by sys.argv[1].



 My data are basically many small one json files, where is one json per line.

 I have tried both, reading and writing the data from/to Amazon S3, local
 disc on all the machines.

 I would like to ask if there is something that I am missing or if Spark is
 supposed to be so slow in comparison with the local non parallelized single
 node program.



 Thank you in advance for any suggestions or hints.



 -
 To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
 For additional commands, e-mail: user-h...@spark.apache.org

-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org

-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org



Spark speed performance

2014-10-18 Thread jan.zikes

Hi,

I have program that I have for single computer (in Python) exection and also 
implemented the same for Spark. This program basically only reads .json from 
which it takes one field and saves it back. Using Spark my program runs 
aproximately 100 times slower on 1 master and 1 slave. So I would like to ask 
where possibly might be the problem?

My Spark program looks like:
 
sc = SparkContext(appName=Json data preprocessor)
distData = sc.textFile(sys.argv[2])
json_extractor = JsonExtractor(sys.argv[1])
cleanedData = distData.flatMap(json_extractor.extract_json) 
cleanedData.saveAsTextFile(sys.argv[3])

JsonExtractor only selects the data from field that is given by sys.argv[1].
 
My data are basically many small one json files, where is one json per line.

I have tried both, reading and writing the data from/to Amazon S3, local disc 
on all the machines.

I would like to ask if there is something that I am missing or if Spark is 
supposed to be so slow in comparison with the local non parallelized single 
node program. 
 
Thank you in advance for any suggestions or hints.


-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org

Re: Spark speed performance

2014-10-18 Thread Evan Sparks
How many files do you have and how big is each JSON object?

Spark works better with a few big files vs many smaller ones. So you could try 
cat'ing your files together and rerunning the same experiment. 

- Evan


 On Oct 18, 2014, at 12:07 PM, jan.zi...@centrum.cz jan.zi...@centrum.cz 
 wrote:
 
 Hi,
 
 I have program that I have for single computer (in Python) exection and also 
 implemented the same for Spark. This program basically only reads .json from 
 which it takes one field and saves it back. Using Spark my program runs 
 aproximately 100 times slower on 1 master and 1 slave. So I would like to ask 
 where possibly might be the problem?
 
 My Spark program looks like:
  
 sc = SparkContext(appName=Json data preprocessor)
 distData = sc.textFile(sys.argv[2])
 json_extractor = JsonExtractor(sys.argv[1])
 cleanedData = distData.flatMap(json_extractor.extract_json) 
 cleanedData.saveAsTextFile(sys.argv[3])
 
 JsonExtractor only selects the data from field that is given by sys.argv[1].
  
 My data are basically many small one json files, where is one json per line.
 
 I have tried both, reading and writing the data from/to Amazon S3, local disc 
 on all the machines.
 
 I would like to ask if there is something that I am missing or if Spark is 
 supposed to be so slow in comparison with the local non parallelized single 
 node program. 
  
 Thank you in advance for any suggestions or hints.
 
 -
 To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
 For additional commands, e-mail: user-h...@spark.apache.org


Re: Spark speed performance

2014-10-18 Thread Davies Liu
How many CPUs on the slave?

Because the overhead between JVM and Python, single task will be
slower than your local Python scripts, but it's very easy to scale to
many CPUs.

Even one CPUs, it's not common that PySpark was 100 times slower. You
have many small files, each file will be processed by a task, which
will have about 100ms overhead (scheduled and executed), but the small
file can be processed in your single thread Python script in less than
1ms.

You could pack your json files into larger ones, or you could try to
merge the small tasks into larger one by coalesce(N), such as:

distData = sc.textFile(sys.argv[2]).coalesce(10)  # which will have 10
partitons (tasks)

Davies

On Sat, Oct 18, 2014 at 12:07 PM,  jan.zi...@centrum.cz wrote:
 Hi,

 I have program that I have for single computer (in Python) exection and also
 implemented the same for Spark. This program basically only reads .json from
 which it takes one field and saves it back. Using Spark my program runs
 aproximately 100 times slower on 1 master and 1 slave. So I would like to
 ask where possibly might be the problem?

 My Spark program looks like:



 sc = SparkContext(appName=Json data preprocessor)

 distData = sc.textFile(sys.argv[2])

 json_extractor = JsonExtractor(sys.argv[1])

 cleanedData = distData.flatMap(json_extractor.extract_json)

 cleanedData.saveAsTextFile(sys.argv[3])

 JsonExtractor only selects the data from field that is given by sys.argv[1].



 My data are basically many small one json files, where is one json per line.

 I have tried both, reading and writing the data from/to Amazon S3, local
 disc on all the machines.

 I would like to ask if there is something that I am missing or if Spark is
 supposed to be so slow in comparison with the local non parallelized single
 node program.



 Thank you in advance for any suggestions or hints.



 -
 To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
 For additional commands, e-mail: user-h...@spark.apache.org

-
To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
For additional commands, e-mail: user-h...@spark.apache.org