Ok, I see, thanks for the correction, but this should be optimized.

From: Shixiong Zhu [mailto:[email protected]]
Sent: Tuesday, August 25, 2015 2:08 PM
To: Cheng, Hao
Cc: Jeff Zhang; [email protected]
Subject: Re: DataFrame#show cost 2 Spark Jobs ?

That's two jobs. `SparkPlan.executeTake` will call `runJob` twice in this case.


Best Regards,
Shixiong Zhu

2015-08-25 14:01 GMT+08:00 Cheng, Hao 
<[email protected]<mailto:[email protected]>>:
O, Sorry, I miss reading your reply!

I know the minimum tasks will be 2 for scanning, but Jeff is talking about 2 
jobs, not 2 tasks.

From: Shixiong Zhu [mailto:[email protected]<mailto:[email protected]>]
Sent: Tuesday, August 25, 2015 1:29 PM
To: Cheng, Hao
Cc: Jeff Zhang; [email protected]<mailto:[email protected]>

Subject: Re: DataFrame#show cost 2 Spark Jobs ?

Hao,

I can reproduce it using the master branch. I'm curious why you cannot 
reproduce it. Did you check if the input HadoopRDD did have two partitions? My 
test code is

val df = sqlContext.read.json("examples/src/main/resources/people.json")
df.show()



Best Regards,
Shixiong Zhu

2015-08-25 13:01 GMT+08:00 Cheng, Hao 
<[email protected]<mailto:[email protected]>>:
Hi Jeff, which version are you using? I couldn’t reproduce the 2 spark jobs in 
the `df.show()` with latest code, we did refactor the code for json data source 
recently, not sure you’re running an earlier version of it.

And a known issue is Spark SQL will try to re-list the files every time when 
loading the data for JSON, it’s probably causes longer time for ramp up with 
large number of files/partitions.

From: Jeff Zhang [mailto:[email protected]<mailto:[email protected]>]
Sent: Tuesday, August 25, 2015 8:11 AM
To: Cheng, Hao
Cc: [email protected]<mailto:[email protected]>
Subject: Re: DataFrame#show cost 2 Spark Jobs ?

Hi Cheng,

I know that sqlContext.read will trigger one spark job to infer the schema. 
What I mean is DataFrame#show cost 2 spark jobs. So overall it would cost 3 
jobs.

Here's the command I use:

>> val df = 
>> sqlContext.read.json("file:///Users/hadoop/github/spark/examples/src/main/resources/people.json<file:///\\Users\hadoop\github\spark\examples\src\main\resources\people.json>")
>>         // trigger one spark job to infer schema
>> df.show()            // trigger 2 spark jobs which is weird




On Mon, Aug 24, 2015 at 10:56 PM, Cheng, Hao 
<[email protected]<mailto:[email protected]>> wrote:
The first job is to infer the json schema, and the second one is what you mean 
of the query.
You can provide the schema while loading the json file, like below:

sqlContext.read.schema(xxx).json(“…”)?

Hao
From: Jeff Zhang [mailto:[email protected]<mailto:[email protected]>]
Sent: Monday, August 24, 2015 6:20 PM
To: [email protected]<mailto:[email protected]>
Subject: DataFrame#show cost 2 Spark Jobs ?

It's weird to me that the simple show function will cost 2 spark jobs. 
DataFrame#explain shows it is a very simple operation, not sure why need 2 jobs.

== Parsed Logical Plan ==
Relation[age#0L,name#1] 
JSONRelation[file:/Users/hadoop/github/spark/examples/src/main/resources/people.json]

== Analyzed Logical Plan ==
age: bigint, name: string
Relation[age#0L,name#1] 
JSONRelation[file:/Users/hadoop/github/spark/examples/src/main/resources/people.json]

== Optimized Logical Plan ==
Relation[age#0L,name#1] 
JSONRelation[file:/Users/hadoop/github/spark/examples/src/main/resources/people.json]

== Physical Plan ==
Scan 
JSONRelation[file:/Users/hadoop/github/spark/examples/src/main/resources/people.json][age#0L,name#1]



--
Best Regards

Jeff Zhang



--
Best Regards

Jeff Zhang


Reply via email to