dataframe operations are expressed as transformations on columns, basically
on locations inside the row objects. this specificity can be exploited by
catalyst to optimize these operations. since catalyst knows exactly what
positions in the row object you modified or not at any point and often also
what operation you did on them it can reason about these and do
optimizations like re-ordering of operations, compiling operations, and
running operations on serialized/internal formats.

when you use case classes and lamba operations not as much information is
available and the operation cannot be performed on the internal
representation. so conversions and/or deserializations are necessary.

On Tue, Feb 19, 2019 at 12:59 AM Lunagariya, Dhaval <
dhaval.lunagar...@citi.com> wrote:

> It does for dataframe also. Please try example.
>
>
>
> df1 = spark.range(2, 10000000, 2)
>
> df2 = spark.range(2, 10000000, 4)
>
> step1 = df1.repartition(5)
>
> step12 = df2.repartition(6)
>
> step2 = step1.selectExpr("id * 5 as id")
>
> step3 = step2.join(step12, ["id"])
>
> step4 = step3.selectExpr("sum(id)")
>
> step4.collect()
>
>
>
> step4._jdf.queryExecution().debug().codegen()
>
>
>
> You will see the generated code.
>
>
>
> Regards,
>
> Dhaval
>
>
>
> *From:* [External] Akhilanand <akhilanand...@gmail.com>
> *Sent:* Tuesday, February 19, 2019 10:29 AM
> *To:* Koert Kuipers <ko...@tresata.com>
> *Cc:* user <user@spark.apache.org>
> *Subject:* Re: Difference between dataset and dataframe
>
>
>
> Thanks for the reply. But can you please tell why dataframes are
> performant than datasets? Any specifics would be helpful.
>
>
>
> Also, could you comment on the tungsten code gen part of my question?
>
>
> On Feb 18, 2019, at 10:47 PM, Koert Kuipers <ko...@tresata.com> wrote:
>
> in the api DataFrame is just Dataset[Row]. so this makes you think Dataset
> is the generic api. interestingly enough under the hood everything is
> really Dataset[Row], so DataFrame is really the "native" language for spark
> sql, not Dataset.
>
>
>
> i find DataFrame to be significantly more performant. in general if you
> use Dataset you miss out on some optimizations. also Encoders are not very
> pleasant to work with.
>
>
>
> On Mon, Feb 18, 2019 at 9:09 PM Akhilanand <akhilanand...@gmail.com>
> wrote:
>
>
> Hello,
>
> I have been recently exploring about dataset and dataframes. I would
> really appreciate if someone could answer these questions:
>
> 1) Is there any difference in terms performance when we use datasets over
> dataframes? Is it significant to choose 1 over other. I do realise there
> would be some overhead due case classes but how significant is that? Are
> there any other implications.
>
> 2) Is the Tungsten code generation done only for datasets or is there any
> internal process to generate bytecode for dataframes as well? Since its
> related to jvm , I think its just for datasets but I couldn’t find anything
> that tells it specifically. If its just for datasets , does that mean we
> miss out on the project tungsten optimisation for dataframes?
>
>
>
> Regards,
> Akhilanand BV
>
> ---------------------------------------------------------------------
> To unsubscribe e-mail: user-unsubscr...@spark.apache.org
>
>

Reply via email to