There is a JIRA and prototype which analyzes the JVM bytecode in the black
box, and convert the closures into catalyst expressions.
https://issues.apache.org/jira/browse/SPARK-14083
This potentially can address the issue discussed here.
Sincerely,
DB Tsai
--
in this case there is no difference in performance. both will do the
operation directly on the internal representation of the data (so the
InternalRow).
also it is worth pointing out that switching back and forth between
Dataset[X] and DataFrame is free.
On Sun, Apr 9, 2017 at 1:28 PM, Shiyuan w
Thank you for the detailed explanation! You point out two reasons why
Dataset is not as efficeint as dataframe:
1). Spark cannot look into lambda and therefore cannot optimize.
2). The type conversion occurs under the hood, eg. from X to internal
row.
Just to check my understanding, some metho
how would you use only relational transformations on dataset?
On Sat, Apr 8, 2017 at 2:15 PM, Shiyuan wrote:
> Hi Spark-users,
> I came across a few sources which mentioned DataFrame can be more
> efficient than Dataset. I can understand this is true because Dataset
> allows functional tran
let me try that again. i left some crap at the bottom of my previous email
as i was editing it. sorry about that. here it goes:
it is because you use Dataset[X] but the actual computations are still done
in Dataset[Row] (so DataFrame). well... the actual computations are done in
RDD[InternalRow] w
it is because you use Dataset[X] but the actual computations are still done
in Dataset[Row] (so DataFrame). well... the actual computations are done in
RDD[InternalRow] with spark's internal types to represent String, Map, Seq,
structs, etc.
so for example if you do:
scala> val x: Dataset[(String,
As far as I am aware in newer Spark versions a DataFrame is the same as
Dataset[Row].
In fact, performance depends on so many factors, so I am not sure such a
comparison makes sense.
> On 8. Apr 2017, at 20:15, Shiyuan wrote:
>
> Hi Spark-users,
> I came across a few sources which mention
Hi Spark-users,
I came across a few sources which mentioned DataFrame can be more
efficient than Dataset. I can understand this is true because Dataset
allows functional transformation which Catalyst cannot look into and hence
cannot optimize well. But can DataFrame be more efficient than Data