Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-11 Thread Wenchen Fan
This vote passes with 4 binding +1 votes, 10 non-binding votes, one +0
vote, and no -1 votes.

Thanks all!

+1 votes (binding):
Wenchen Fan
Herman van Hövell tot Westerflier
Michael Armbrust
Reynold Xin


+1 votes (non-binding):
Xiao Li
Sameer Agarwal
Suresh Thalamati
Ryan Blue
Xingbo Jiang
Dongjoon Hyun
Zhenhua Wang
Noman Khan
vaquar khan
Hemant Bhanawat

+0 votes:
Andrew Ash

On Mon, Sep 11, 2017 at 4:03 PM, Wenchen Fan <cloud0...@gmail.com> wrote:

> yea, join push down (providing the other reader and join conditions) and
> aggregate push down (providing grouping keys and aggregate functions) can
> be added via the current framework in the future.
>
> On Mon, Sep 11, 2017 at 1:54 PM, Hemant Bhanawat <hemant9...@gmail.com>
> wrote:
>
>> +1 (non-binding)
>>
>> I have found the suggestion from Andrew Ash and James about plan push
>> down quite interesting. However, I am not clear about the join push-down
>> support at the data source level. Shouldn't it be the responsibility of the
>> join node to carry out a data source specific join? I mean join node and
>> the data source scan of the two sides can be coalesced into a single node
>> (theoretically). This can be done by providing a Strategy that replaces the
>> join node with a data source specific join node. We are doing it that way
>> for our data sources. I find this more intuitive.
>>
>> BTW, aggregate push-down support is desirable and should be considered as
>> an enhancement going forward.
>>
>> Hemant Bhanawat <https://www.linkedin.com/in/hemant-bhanawat-92a3811>
>> www.snappydata.io
>>
>> On Sun, Sep 10, 2017 at 8:45 PM, vaquar khan <vaquar.k...@gmail.com>
>> wrote:
>>
>>> +1
>>>
>>> Regards,
>>> Vaquar khan
>>>
>>> On Sep 10, 2017 5:18 AM, "Noman Khan" <nomanbp...@live.com> wrote:
>>>
>>>> +1
>>>> --
>>>> *From:* wangzhenhua (G) <wangzhen...@huawei.com>
>>>> *Sent:* Friday, September 8, 2017 2:20:07 AM
>>>> *To:* Dongjoon Hyun; 蒋星博
>>>> *Cc:* Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot
>>>> Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
>>>> *Subject:* 答复: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path
>>>>
>>>>
>>>> +1 (non-binding)  Great to see data source API is going to be improved!
>>>>
>>>>
>>>>
>>>> best regards,
>>>>
>>>> -Zhenhua(Xander)
>>>>
>>>>
>>>>
>>>> *发件人:* Dongjoon Hyun [mailto:dongjoon.h...@gmail.com]
>>>> *发送时间:* 2017年9月8日 4:07
>>>> *收件人:* 蒋星博
>>>> *抄送:* Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot
>>>> Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
>>>> *主题:* Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path
>>>>
>>>>
>>>>
>>>> +1 (non-binding).
>>>>
>>>>
>>>>
>>>> On Thu, Sep 7, 2017 at 12:46 PM, 蒋星博 <jiangxb1...@gmail.com> wrote:
>>>>
>>>> +1
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> Reynold Xin <r...@databricks.com>于2017年9月7日 周四下午12:04写道:
>>>>
>>>> +1 as well
>>>>
>>>>
>>>>
>>>> On Thu, Sep 7, 2017 at 9:12 PM, Michael Armbrust <
>>>> mich...@databricks.com> wrote:
>>>>
>>>> +1
>>>>
>>>>
>>>>
>>>> On Thu, Sep 7, 2017 at 9:32 AM, Ryan Blue <rb...@netflix.com.invalid>
>>>> wrote:
>>>>
>>>> +1 (non-binding)
>>>>
>>>> Thanks for making the updates reflected in the current PR. It would be
>>>> great to see the doc updated before it is finally published though.
>>>>
>>>> Right now it feels like this SPIP is focused more on getting the basics
>>>> right for what many datasources are already doing in API V1 combined with
>>>> other private APIs, vs pushing forward state of the art for performance.
>>>>
>>>> I think that’s the right approach for this SPIP. We can add the support
>>>> you’re talking about later with a more specific plan that doesn’t block
>>>> fixing the problems that this addresses.
>>>>
>>>> ​
>>>>
>>>>
>>>>
>>>> On Thu, Sep 7, 2017 at

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-11 Thread Wenchen Fan
yea, join push down (providing the other reader and join conditions) and
aggregate push down (providing grouping keys and aggregate functions) can
be added via the current framework in the future.

On Mon, Sep 11, 2017 at 1:54 PM, Hemant Bhanawat <hemant9...@gmail.com>
wrote:

> +1 (non-binding)
>
> I have found the suggestion from Andrew Ash and James about plan push down
> quite interesting. However, I am not clear about the join push-down support
> at the data source level. Shouldn't it be the responsibility of the join
> node to carry out a data source specific join? I mean join node and the
> data source scan of the two sides can be coalesced into a single node
> (theoretically). This can be done by providing a Strategy that replaces the
> join node with a data source specific join node. We are doing it that way
> for our data sources. I find this more intuitive.
>
> BTW, aggregate push-down support is desirable and should be considered as
> an enhancement going forward.
>
> Hemant Bhanawat <https://www.linkedin.com/in/hemant-bhanawat-92a3811>
> www.snappydata.io
>
> On Sun, Sep 10, 2017 at 8:45 PM, vaquar khan <vaquar.k...@gmail.com>
> wrote:
>
>> +1
>>
>> Regards,
>> Vaquar khan
>>
>> On Sep 10, 2017 5:18 AM, "Noman Khan" <nomanbp...@live.com> wrote:
>>
>>> +1
>>> --
>>> *From:* wangzhenhua (G) <wangzhen...@huawei.com>
>>> *Sent:* Friday, September 8, 2017 2:20:07 AM
>>> *To:* Dongjoon Hyun; 蒋星博
>>> *Cc:* Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot
>>> Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
>>> *Subject:* 答复: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path
>>>
>>>
>>> +1 (non-binding)  Great to see data source API is going to be improved!
>>>
>>>
>>>
>>> best regards,
>>>
>>> -Zhenhua(Xander)
>>>
>>>
>>>
>>> *发件人:* Dongjoon Hyun [mailto:dongjoon.h...@gmail.com]
>>> *发送时间:* 2017年9月8日 4:07
>>> *收件人:* 蒋星博
>>> *抄送:* Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot
>>> Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
>>> *主题:* Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path
>>>
>>>
>>>
>>> +1 (non-binding).
>>>
>>>
>>>
>>> On Thu, Sep 7, 2017 at 12:46 PM, 蒋星博 <jiangxb1...@gmail.com> wrote:
>>>
>>> +1
>>>
>>>
>>>
>>>
>>>
>>> Reynold Xin <r...@databricks.com>于2017年9月7日 周四下午12:04写道:
>>>
>>> +1 as well
>>>
>>>
>>>
>>> On Thu, Sep 7, 2017 at 9:12 PM, Michael Armbrust <mich...@databricks.com>
>>> wrote:
>>>
>>> +1
>>>
>>>
>>>
>>> On Thu, Sep 7, 2017 at 9:32 AM, Ryan Blue <rb...@netflix.com.invalid>
>>> wrote:
>>>
>>> +1 (non-binding)
>>>
>>> Thanks for making the updates reflected in the current PR. It would be
>>> great to see the doc updated before it is finally published though.
>>>
>>> Right now it feels like this SPIP is focused more on getting the basics
>>> right for what many datasources are already doing in API V1 combined with
>>> other private APIs, vs pushing forward state of the art for performance.
>>>
>>> I think that’s the right approach for this SPIP. We can add the support
>>> you’re talking about later with a more specific plan that doesn’t block
>>> fixing the problems that this addresses.
>>>
>>> ​
>>>
>>>
>>>
>>> On Thu, Sep 7, 2017 at 2:00 AM, Herman van Hövell tot Westerflier <
>>> hvanhov...@databricks.com> wrote:
>>>
>>> +1 (binding)
>>>
>>>
>>>
>>> I personally believe that there is quite a big difference between having
>>> a generic data source interface with a low surface area and pushing down a
>>> significant part of query processing into a datasource. The later has much
>>> wider wider surface area and will require us to stabilize most of the
>>> internal catalyst API's which will be a significant burden on the community
>>> to maintain and has the potential to slow development velocity
>>> significantly. If you want to write such integrations then you should be
>>> prepared to work with catalyst internals and own up to the fact that things
>>> might change across minor vers

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-10 Thread Hemant Bhanawat
+1 (non-binding)

I have found the suggestion from Andrew Ash and James about plan push down
quite interesting. However, I am not clear about the join push-down support
at the data source level. Shouldn't it be the responsibility of the join
node to carry out a data source specific join? I mean join node and the
data source scan of the two sides can be coalesced into a single node
(theoretically). This can be done by providing a Strategy that replaces the
join node with a data source specific join node. We are doing it that way
for our data sources. I find this more intuitive.

BTW, aggregate push-down support is desirable and should be considered as
an enhancement going forward.

Hemant Bhanawat <https://www.linkedin.com/in/hemant-bhanawat-92a3811>
www.snappydata.io

On Sun, Sep 10, 2017 at 8:45 PM, vaquar khan <vaquar.k...@gmail.com> wrote:

> +1
>
> Regards,
> Vaquar khan
>
> On Sep 10, 2017 5:18 AM, "Noman Khan" <nomanbp...@live.com> wrote:
>
>> +1
>> --
>> *From:* wangzhenhua (G) <wangzhen...@huawei.com>
>> *Sent:* Friday, September 8, 2017 2:20:07 AM
>> *To:* Dongjoon Hyun; 蒋星博
>> *Cc:* Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot
>> Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
>> *Subject:* 答复: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path
>>
>>
>> +1 (non-binding)  Great to see data source API is going to be improved!
>>
>>
>>
>> best regards,
>>
>> -Zhenhua(Xander)
>>
>>
>>
>> *发件人:* Dongjoon Hyun [mailto:dongjoon.h...@gmail.com]
>> *发送时间:* 2017年9月8日 4:07
>> *收件人:* 蒋星博
>> *抄送:* Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot
>> Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
>> *主题:* Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path
>>
>>
>>
>> +1 (non-binding).
>>
>>
>>
>> On Thu, Sep 7, 2017 at 12:46 PM, 蒋星博 <jiangxb1...@gmail.com> wrote:
>>
>> +1
>>
>>
>>
>>
>>
>> Reynold Xin <r...@databricks.com>于2017年9月7日 周四下午12:04写道:
>>
>> +1 as well
>>
>>
>>
>> On Thu, Sep 7, 2017 at 9:12 PM, Michael Armbrust <mich...@databricks.com>
>> wrote:
>>
>> +1
>>
>>
>>
>> On Thu, Sep 7, 2017 at 9:32 AM, Ryan Blue <rb...@netflix.com.invalid>
>> wrote:
>>
>> +1 (non-binding)
>>
>> Thanks for making the updates reflected in the current PR. It would be
>> great to see the doc updated before it is finally published though.
>>
>> Right now it feels like this SPIP is focused more on getting the basics
>> right for what many datasources are already doing in API V1 combined with
>> other private APIs, vs pushing forward state of the art for performance.
>>
>> I think that’s the right approach for this SPIP. We can add the support
>> you’re talking about later with a more specific plan that doesn’t block
>> fixing the problems that this addresses.
>>
>> ​
>>
>>
>>
>> On Thu, Sep 7, 2017 at 2:00 AM, Herman van Hövell tot Westerflier <
>> hvanhov...@databricks.com> wrote:
>>
>> +1 (binding)
>>
>>
>>
>> I personally believe that there is quite a big difference between having
>> a generic data source interface with a low surface area and pushing down a
>> significant part of query processing into a datasource. The later has much
>> wider wider surface area and will require us to stabilize most of the
>> internal catalyst API's which will be a significant burden on the community
>> to maintain and has the potential to slow development velocity
>> significantly. If you want to write such integrations then you should be
>> prepared to work with catalyst internals and own up to the fact that things
>> might change across minor versions (and in some cases even maintenance
>> releases). If you are willing to go down that road, then your best bet is
>> to use the already existing spark session extensions which will allow you
>> to write such integrations and can be used as an `escape hatch`.
>>
>>
>>
>>
>>
>> On Thu, Sep 7, 2017 at 10:23 AM, Andrew Ash <and...@andrewash.com> wrote:
>>
>> +0 (non-binding)
>>
>>
>>
>> I think there are benefits to unifying all the Spark-internal datasources
>> into a common public API for sure.  It will serve as a forcing function to
>> ensure that those internal datasources aren't advantaged vs datasources
>> developed externally as plugins to Spa

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-10 Thread vaquar khan
+1

Regards,
Vaquar khan

On Sep 10, 2017 5:18 AM, "Noman Khan" <nomanbp...@live.com> wrote:

> +1
> --
> *From:* wangzhenhua (G) <wangzhen...@huawei.com>
> *Sent:* Friday, September 8, 2017 2:20:07 AM
> *To:* Dongjoon Hyun; 蒋星博
> *Cc:* Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot
> Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
> *Subject:* 答复: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path
>
>
> +1 (non-binding)  Great to see data source API is going to be improved!
>
>
>
> best regards,
>
> -Zhenhua(Xander)
>
>
>
> *发件人:* Dongjoon Hyun [mailto:dongjoon.h...@gmail.com]
> *发送时间:* 2017年9月8日 4:07
> *收件人:* 蒋星博
> *抄送:* Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot
> Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
> *主题:* Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path
>
>
>
> +1 (non-binding).
>
>
>
> On Thu, Sep 7, 2017 at 12:46 PM, 蒋星博 <jiangxb1...@gmail.com> wrote:
>
> +1
>
>
>
>
>
> Reynold Xin <r...@databricks.com>于2017年9月7日 周四下午12:04写道:
>
> +1 as well
>
>
>
> On Thu, Sep 7, 2017 at 9:12 PM, Michael Armbrust <mich...@databricks.com>
> wrote:
>
> +1
>
>
>
> On Thu, Sep 7, 2017 at 9:32 AM, Ryan Blue <rb...@netflix.com.invalid>
> wrote:
>
> +1 (non-binding)
>
> Thanks for making the updates reflected in the current PR. It would be
> great to see the doc updated before it is finally published though.
>
> Right now it feels like this SPIP is focused more on getting the basics
> right for what many datasources are already doing in API V1 combined with
> other private APIs, vs pushing forward state of the art for performance.
>
> I think that’s the right approach for this SPIP. We can add the support
> you’re talking about later with a more specific plan that doesn’t block
> fixing the problems that this addresses.
>
> ​
>
>
>
> On Thu, Sep 7, 2017 at 2:00 AM, Herman van Hövell tot Westerflier <
> hvanhov...@databricks.com> wrote:
>
> +1 (binding)
>
>
>
> I personally believe that there is quite a big difference between having a
> generic data source interface with a low surface area and pushing down a
> significant part of query processing into a datasource. The later has much
> wider wider surface area and will require us to stabilize most of the
> internal catalyst API's which will be a significant burden on the community
> to maintain and has the potential to slow development velocity
> significantly. If you want to write such integrations then you should be
> prepared to work with catalyst internals and own up to the fact that things
> might change across minor versions (and in some cases even maintenance
> releases). If you are willing to go down that road, then your best bet is
> to use the already existing spark session extensions which will allow you
> to write such integrations and can be used as an `escape hatch`.
>
>
>
>
>
> On Thu, Sep 7, 2017 at 10:23 AM, Andrew Ash <and...@andrewash.com> wrote:
>
> +0 (non-binding)
>
>
>
> I think there are benefits to unifying all the Spark-internal datasources
> into a common public API for sure.  It will serve as a forcing function to
> ensure that those internal datasources aren't advantaged vs datasources
> developed externally as plugins to Spark, and that all Spark features are
> available to all datasources.
>
>
>
> But I also think this read-path proposal avoids the more difficult
> questions around how to continue pushing datasource performance forwards.
> James Baker (my colleague) had a number of questions about advanced
> pushdowns (combined sorting and filtering), and Reynold also noted that
> pushdown of aggregates and joins are desirable on longer timeframes as
> well.  The Spark community saw similar requests, for aggregate pushdown in
> SPARK-12686, join pushdown in SPARK-20259, and arbitrary plan pushdown
> in SPARK-12449.  Clearly a number of people are interested in this kind of
> performance work for datasources.
>
>
>
> To leave enough space for datasource developers to continue experimenting
> with advanced interactions between Spark and their datasources, I'd propose
> we leave some sort of escape valve that enables these datasources to keep
> pushing the boundaries without forking Spark.  Possibly that looks like an
> additional unsupported/unstable interface that pushes down an entire
> (unstable API) logical plan, which is expected to break API on every
> release.   (Spark attempts this full-plan pushdown, and if that fails Spark
> ignores it and continu

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-10 Thread Noman Khan
+1

From: wangzhenhua (G) <wangzhen...@huawei.com>
Sent: Friday, September 8, 2017 2:20:07 AM
To: Dongjoon Hyun; 蒋星博
Cc: Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot 
Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
Subject: 答复: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

+1 (non-binding)  Great to see data source API is going to be improved!

best regards,
-Zhenhua(Xander)

发件人: Dongjoon Hyun [mailto:dongjoon.h...@gmail.com]
发送时间: 2017年9月8日 4:07
收件人: 蒋星博
抄送: Michael Armbrust; Reynold Xin; Andrew Ash; Herman van Hövell tot 
Westerflier; Ryan Blue; Spark dev list; Suresh Thalamati; Wenchen Fan
主题: Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

+1 (non-binding).

On Thu, Sep 7, 2017 at 12:46 PM, 蒋星博 
<jiangxb1...@gmail.com<mailto:jiangxb1...@gmail.com>> wrote:
+1


Reynold Xin <r...@databricks.com<mailto:r...@databricks.com>>于2017年9月7日 
周四下午12:04写道:
+1 as well

On Thu, Sep 7, 2017 at 9:12 PM, Michael Armbrust 
<mich...@databricks.com<mailto:mich...@databricks.com>> wrote:
+1

On Thu, Sep 7, 2017 at 9:32 AM, Ryan Blue 
<rb...@netflix.com.invalid<mailto:rb...@netflix.com.invalid>> wrote:

+1 (non-binding)

Thanks for making the updates reflected in the current PR. It would be great to 
see the doc updated before it is finally published though.

Right now it feels like this SPIP is focused more on getting the basics right 
for what many datasources are already doing in API V1 combined with other 
private APIs, vs pushing forward state of the art for performance.

I think that’s the right approach for this SPIP. We can add the support you’re 
talking about later with a more specific plan that doesn’t block fixing the 
problems that this addresses.
​

On Thu, Sep 7, 2017 at 2:00 AM, Herman van Hövell tot Westerflier 
<hvanhov...@databricks.com<mailto:hvanhov...@databricks.com>> wrote:
+1 (binding)

I personally believe that there is quite a big difference between having a 
generic data source interface with a low surface area and pushing down a 
significant part of query processing into a datasource. The later has much 
wider wider surface area and will require us to stabilize most of the internal 
catalyst API's which will be a significant burden on the community to maintain 
and has the potential to slow development velocity significantly. If you want 
to write such integrations then you should be prepared to work with catalyst 
internals and own up to the fact that things might change across minor versions 
(and in some cases even maintenance releases). If you are willing to go down 
that road, then your best bet is to use the already existing spark session 
extensions which will allow you to write such integrations and can be used as 
an `escape hatch`.


On Thu, Sep 7, 2017 at 10:23 AM, Andrew Ash 
<and...@andrewash.com<mailto:and...@andrewash.com>> wrote:
+0 (non-binding)

I think there are benefits to unifying all the Spark-internal datasources into 
a common public API for sure.  It will serve as a forcing function to ensure 
that those internal datasources aren't advantaged vs datasources developed 
externally as plugins to Spark, and that all Spark features are available to 
all datasources.

But I also think this read-path proposal avoids the more difficult questions 
around how to continue pushing datasource performance forwards.  James Baker 
(my colleague) had a number of questions about advanced pushdowns (combined 
sorting and filtering), and Reynold also noted that pushdown of aggregates and 
joins are desirable on longer timeframes as well.  The Spark community saw 
similar requests, for aggregate pushdown in SPARK-12686, join pushdown in 
SPARK-20259, and arbitrary plan pushdown in SPARK-12449.  Clearly a number of 
people are interested in this kind of performance work for datasources.

To leave enough space for datasource developers to continue experimenting with 
advanced interactions between Spark and their datasources, I'd propose we leave 
some sort of escape valve that enables these datasources to keep pushing the 
boundaries without forking Spark.  Possibly that looks like an additional 
unsupported/unstable interface that pushes down an entire (unstable API) 
logical plan, which is expected to break API on every release.   (Spark 
attempts this full-plan pushdown, and if that fails Spark ignores it and 
continues on with the rest of the V2 API for compatibility).  Or maybe it looks 
like something else that we don't know of yet.  Possibly this falls outside of 
the desired goals for the V2 API and instead should be a separate SPIP.

If we had a plan for this kind of escape valve for advanced datasource 
developers I'd be an unequivocal +1.  Right now it feels like this SPIP is 
focused more on getting the basics right for what many datasources are already 
doing in API V1 combined with other private APIs, vs p

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-07 Thread Dongjoon Hyun
+1 (non-binding).

On Thu, Sep 7, 2017 at 12:46 PM, 蒋星博  wrote:

> +1
>
>
> Reynold Xin 于2017年9月7日 周四下午12:04写道:
>
>> +1 as well
>>
>> On Thu, Sep 7, 2017 at 9:12 PM, Michael Armbrust 
>> wrote:
>>
>>> +1
>>>
>>> On Thu, Sep 7, 2017 at 9:32 AM, Ryan Blue 
>>> wrote:
>>>
 +1 (non-binding)

 Thanks for making the updates reflected in the current PR. It would be
 great to see the doc updated before it is finally published though.

 Right now it feels like this SPIP is focused more on getting the basics
 right for what many datasources are already doing in API V1 combined with
 other private APIs, vs pushing forward state of the art for performance.

 I think that’s the right approach for this SPIP. We can add the support
 you’re talking about later with a more specific plan that doesn’t block
 fixing the problems that this addresses.
 ​

 On Thu, Sep 7, 2017 at 2:00 AM, Herman van Hövell tot Westerflier <
 hvanhov...@databricks.com> wrote:

> +1 (binding)
>
> I personally believe that there is quite a big difference between
> having a generic data source interface with a low surface area and pushing
> down a significant part of query processing into a datasource. The later
> has much wider wider surface area and will require us to stabilize most of
> the internal catalyst API's which will be a significant burden on the
> community to maintain and has the potential to slow development velocity
> significantly. If you want to write such integrations then you should be
> prepared to work with catalyst internals and own up to the fact that 
> things
> might change across minor versions (and in some cases even maintenance
> releases). If you are willing to go down that road, then your best bet is
> to use the already existing spark session extensions which will allow you
> to write such integrations and can be used as an `escape hatch`.
>
>
> On Thu, Sep 7, 2017 at 10:23 AM, Andrew Ash 
> wrote:
>
>> +0 (non-binding)
>>
>> I think there are benefits to unifying all the Spark-internal
>> datasources into a common public API for sure.  It will serve as a 
>> forcing
>> function to ensure that those internal datasources aren't advantaged vs
>> datasources developed externally as plugins to Spark, and that all Spark
>> features are available to all datasources.
>>
>> But I also think this read-path proposal avoids the more difficult
>> questions around how to continue pushing datasource performance forwards.
>> James Baker (my colleague) had a number of questions about advanced
>> pushdowns (combined sorting and filtering), and Reynold also noted that
>> pushdown of aggregates and joins are desirable on longer timeframes as
>> well.  The Spark community saw similar requests, for aggregate pushdown 
>> in
>> SPARK-12686, join pushdown in SPARK-20259, and arbitrary plan pushdown
>> in SPARK-12449.  Clearly a number of people are interested in this kind 
>> of
>> performance work for datasources.
>>
>> To leave enough space for datasource developers to continue
>> experimenting with advanced interactions between Spark and their
>> datasources, I'd propose we leave some sort of escape valve that enables
>> these datasources to keep pushing the boundaries without forking Spark.
>> Possibly that looks like an additional unsupported/unstable interface 
>> that
>> pushes down an entire (unstable API) logical plan, which is expected to
>> break API on every release.   (Spark attempts this full-plan pushdown, 
>> and
>> if that fails Spark ignores it and continues on with the rest of the V2 
>> API
>> for compatibility).  Or maybe it looks like something else that we don't
>> know of yet.  Possibly this falls outside of the desired goals for the V2
>> API and instead should be a separate SPIP.
>>
>> If we had a plan for this kind of escape valve for advanced
>> datasource developers I'd be an unequivocal +1.  Right now it feels like
>> this SPIP is focused more on getting the basics right for what many
>> datasources are already doing in API V1 combined with other private APIs,
>> vs pushing forward state of the art for performance.
>>
>> Andrew
>>
>> On Wed, Sep 6, 2017 at 10:56 PM, Suresh Thalamati <
>> suresh.thalam...@gmail.com> wrote:
>>
>>> +1 (non-binding)
>>>
>>>
>>> On Sep 6, 2017, at 7:29 PM, Wenchen Fan  wrote:
>>>
>>> Hi all,
>>>
>>> In the previous discussion, we decided to split the read and write
>>> path of data source v2 into 2 SPIPs, and I'm sending this email to call 
>>> a
>>> vote for Data Source V2 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-07 Thread 蒋星博
+1


Reynold Xin 于2017年9月7日 周四下午12:04写道:

> +1 as well
>
> On Thu, Sep 7, 2017 at 9:12 PM, Michael Armbrust 
> wrote:
>
>> +1
>>
>> On Thu, Sep 7, 2017 at 9:32 AM, Ryan Blue 
>> wrote:
>>
>>> +1 (non-binding)
>>>
>>> Thanks for making the updates reflected in the current PR. It would be
>>> great to see the doc updated before it is finally published though.
>>>
>>> Right now it feels like this SPIP is focused more on getting the basics
>>> right for what many datasources are already doing in API V1 combined with
>>> other private APIs, vs pushing forward state of the art for performance.
>>>
>>> I think that’s the right approach for this SPIP. We can add the support
>>> you’re talking about later with a more specific plan that doesn’t block
>>> fixing the problems that this addresses.
>>> ​
>>>
>>> On Thu, Sep 7, 2017 at 2:00 AM, Herman van Hövell tot Westerflier <
>>> hvanhov...@databricks.com> wrote:
>>>
 +1 (binding)

 I personally believe that there is quite a big difference between
 having a generic data source interface with a low surface area and pushing
 down a significant part of query processing into a datasource. The later
 has much wider wider surface area and will require us to stabilize most of
 the internal catalyst API's which will be a significant burden on the
 community to maintain and has the potential to slow development velocity
 significantly. If you want to write such integrations then you should be
 prepared to work with catalyst internals and own up to the fact that things
 might change across minor versions (and in some cases even maintenance
 releases). If you are willing to go down that road, then your best bet is
 to use the already existing spark session extensions which will allow you
 to write such integrations and can be used as an `escape hatch`.


 On Thu, Sep 7, 2017 at 10:23 AM, Andrew Ash 
 wrote:

> +0 (non-binding)
>
> I think there are benefits to unifying all the Spark-internal
> datasources into a common public API for sure.  It will serve as a forcing
> function to ensure that those internal datasources aren't advantaged vs
> datasources developed externally as plugins to Spark, and that all Spark
> features are available to all datasources.
>
> But I also think this read-path proposal avoids the more difficult
> questions around how to continue pushing datasource performance forwards.
> James Baker (my colleague) had a number of questions about advanced
> pushdowns (combined sorting and filtering), and Reynold also noted that
> pushdown of aggregates and joins are desirable on longer timeframes as
> well.  The Spark community saw similar requests, for aggregate pushdown in
> SPARK-12686, join pushdown in SPARK-20259, and arbitrary plan pushdown
> in SPARK-12449.  Clearly a number of people are interested in this kind of
> performance work for datasources.
>
> To leave enough space for datasource developers to continue
> experimenting with advanced interactions between Spark and their
> datasources, I'd propose we leave some sort of escape valve that enables
> these datasources to keep pushing the boundaries without forking Spark.
> Possibly that looks like an additional unsupported/unstable interface that
> pushes down an entire (unstable API) logical plan, which is expected to
> break API on every release.   (Spark attempts this full-plan pushdown, and
> if that fails Spark ignores it and continues on with the rest of the V2 
> API
> for compatibility).  Or maybe it looks like something else that we don't
> know of yet.  Possibly this falls outside of the desired goals for the V2
> API and instead should be a separate SPIP.
>
> If we had a plan for this kind of escape valve for advanced datasource
> developers I'd be an unequivocal +1.  Right now it feels like this SPIP is
> focused more on getting the basics right for what many datasources are
> already doing in API V1 combined with other private APIs, vs pushing
> forward state of the art for performance.
>
> Andrew
>
> On Wed, Sep 6, 2017 at 10:56 PM, Suresh Thalamati <
> suresh.thalam...@gmail.com> wrote:
>
>> +1 (non-binding)
>>
>>
>> On Sep 6, 2017, at 7:29 PM, Wenchen Fan  wrote:
>>
>> Hi all,
>>
>> In the previous discussion, we decided to split the read and write
>> path of data source v2 into 2 SPIPs, and I'm sending this email to call a
>> vote for Data Source V2 read path only.
>>
>> The full document of the Data Source API V2 is:
>>
>> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-Z8qU5Frf6WMQZ6jJVM/edit
>>
>> The ready-for-review PR that implements the basic 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-07 Thread Reynold Xin
+1 as well

On Thu, Sep 7, 2017 at 9:12 PM, Michael Armbrust 
wrote:

> +1
>
> On Thu, Sep 7, 2017 at 9:32 AM, Ryan Blue 
> wrote:
>
>> +1 (non-binding)
>>
>> Thanks for making the updates reflected in the current PR. It would be
>> great to see the doc updated before it is finally published though.
>>
>> Right now it feels like this SPIP is focused more on getting the basics
>> right for what many datasources are already doing in API V1 combined with
>> other private APIs, vs pushing forward state of the art for performance.
>>
>> I think that’s the right approach for this SPIP. We can add the support
>> you’re talking about later with a more specific plan that doesn’t block
>> fixing the problems that this addresses.
>> ​
>>
>> On Thu, Sep 7, 2017 at 2:00 AM, Herman van Hövell tot Westerflier <
>> hvanhov...@databricks.com> wrote:
>>
>>> +1 (binding)
>>>
>>> I personally believe that there is quite a big difference between having
>>> a generic data source interface with a low surface area and pushing down a
>>> significant part of query processing into a datasource. The later has much
>>> wider wider surface area and will require us to stabilize most of the
>>> internal catalyst API's which will be a significant burden on the community
>>> to maintain and has the potential to slow development velocity
>>> significantly. If you want to write such integrations then you should be
>>> prepared to work with catalyst internals and own up to the fact that things
>>> might change across minor versions (and in some cases even maintenance
>>> releases). If you are willing to go down that road, then your best bet is
>>> to use the already existing spark session extensions which will allow you
>>> to write such integrations and can be used as an `escape hatch`.
>>>
>>>
>>> On Thu, Sep 7, 2017 at 10:23 AM, Andrew Ash 
>>> wrote:
>>>
 +0 (non-binding)

 I think there are benefits to unifying all the Spark-internal
 datasources into a common public API for sure.  It will serve as a forcing
 function to ensure that those internal datasources aren't advantaged vs
 datasources developed externally as plugins to Spark, and that all Spark
 features are available to all datasources.

 But I also think this read-path proposal avoids the more difficult
 questions around how to continue pushing datasource performance forwards.
 James Baker (my colleague) had a number of questions about advanced
 pushdowns (combined sorting and filtering), and Reynold also noted that
 pushdown of aggregates and joins are desirable on longer timeframes as
 well.  The Spark community saw similar requests, for aggregate pushdown in
 SPARK-12686, join pushdown in SPARK-20259, and arbitrary plan pushdown
 in SPARK-12449.  Clearly a number of people are interested in this kind of
 performance work for datasources.

 To leave enough space for datasource developers to continue
 experimenting with advanced interactions between Spark and their
 datasources, I'd propose we leave some sort of escape valve that enables
 these datasources to keep pushing the boundaries without forking Spark.
 Possibly that looks like an additional unsupported/unstable interface that
 pushes down an entire (unstable API) logical plan, which is expected to
 break API on every release.   (Spark attempts this full-plan pushdown, and
 if that fails Spark ignores it and continues on with the rest of the V2 API
 for compatibility).  Or maybe it looks like something else that we don't
 know of yet.  Possibly this falls outside of the desired goals for the V2
 API and instead should be a separate SPIP.

 If we had a plan for this kind of escape valve for advanced datasource
 developers I'd be an unequivocal +1.  Right now it feels like this SPIP is
 focused more on getting the basics right for what many datasources are
 already doing in API V1 combined with other private APIs, vs pushing
 forward state of the art for performance.

 Andrew

 On Wed, Sep 6, 2017 at 10:56 PM, Suresh Thalamati <
 suresh.thalam...@gmail.com> wrote:

> +1 (non-binding)
>
>
> On Sep 6, 2017, at 7:29 PM, Wenchen Fan  wrote:
>
> Hi all,
>
> In the previous discussion, we decided to split the read and write
> path of data source v2 into 2 SPIPs, and I'm sending this email to call a
> vote for Data Source V2 read path only.
>
> The full document of the Data Source API V2 is:
> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ
> -Z8qU5Frf6WMQZ6jJVM/edit
>
> The ready-for-review PR that implements the basic infrastructure for
> the read path is:
> https://github.com/apache/spark/pull/19136
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-07 Thread Michael Armbrust
+1

On Thu, Sep 7, 2017 at 9:32 AM, Ryan Blue  wrote:

> +1 (non-binding)
>
> Thanks for making the updates reflected in the current PR. It would be
> great to see the doc updated before it is finally published though.
>
> Right now it feels like this SPIP is focused more on getting the basics
> right for what many datasources are already doing in API V1 combined with
> other private APIs, vs pushing forward state of the art for performance.
>
> I think that’s the right approach for this SPIP. We can add the support
> you’re talking about later with a more specific plan that doesn’t block
> fixing the problems that this addresses.
> ​
>
> On Thu, Sep 7, 2017 at 2:00 AM, Herman van Hövell tot Westerflier <
> hvanhov...@databricks.com> wrote:
>
>> +1 (binding)
>>
>> I personally believe that there is quite a big difference between having
>> a generic data source interface with a low surface area and pushing down a
>> significant part of query processing into a datasource. The later has much
>> wider wider surface area and will require us to stabilize most of the
>> internal catalyst API's which will be a significant burden on the community
>> to maintain and has the potential to slow development velocity
>> significantly. If you want to write such integrations then you should be
>> prepared to work with catalyst internals and own up to the fact that things
>> might change across minor versions (and in some cases even maintenance
>> releases). If you are willing to go down that road, then your best bet is
>> to use the already existing spark session extensions which will allow you
>> to write such integrations and can be used as an `escape hatch`.
>>
>>
>> On Thu, Sep 7, 2017 at 10:23 AM, Andrew Ash  wrote:
>>
>>> +0 (non-binding)
>>>
>>> I think there are benefits to unifying all the Spark-internal
>>> datasources into a common public API for sure.  It will serve as a forcing
>>> function to ensure that those internal datasources aren't advantaged vs
>>> datasources developed externally as plugins to Spark, and that all Spark
>>> features are available to all datasources.
>>>
>>> But I also think this read-path proposal avoids the more difficult
>>> questions around how to continue pushing datasource performance forwards.
>>> James Baker (my colleague) had a number of questions about advanced
>>> pushdowns (combined sorting and filtering), and Reynold also noted that
>>> pushdown of aggregates and joins are desirable on longer timeframes as
>>> well.  The Spark community saw similar requests, for aggregate pushdown in
>>> SPARK-12686, join pushdown in SPARK-20259, and arbitrary plan pushdown
>>> in SPARK-12449.  Clearly a number of people are interested in this kind of
>>> performance work for datasources.
>>>
>>> To leave enough space for datasource developers to continue
>>> experimenting with advanced interactions between Spark and their
>>> datasources, I'd propose we leave some sort of escape valve that enables
>>> these datasources to keep pushing the boundaries without forking Spark.
>>> Possibly that looks like an additional unsupported/unstable interface that
>>> pushes down an entire (unstable API) logical plan, which is expected to
>>> break API on every release.   (Spark attempts this full-plan pushdown, and
>>> if that fails Spark ignores it and continues on with the rest of the V2 API
>>> for compatibility).  Or maybe it looks like something else that we don't
>>> know of yet.  Possibly this falls outside of the desired goals for the V2
>>> API and instead should be a separate SPIP.
>>>
>>> If we had a plan for this kind of escape valve for advanced datasource
>>> developers I'd be an unequivocal +1.  Right now it feels like this SPIP is
>>> focused more on getting the basics right for what many datasources are
>>> already doing in API V1 combined with other private APIs, vs pushing
>>> forward state of the art for performance.
>>>
>>> Andrew
>>>
>>> On Wed, Sep 6, 2017 at 10:56 PM, Suresh Thalamati <
>>> suresh.thalam...@gmail.com> wrote:
>>>
 +1 (non-binding)


 On Sep 6, 2017, at 7:29 PM, Wenchen Fan  wrote:

 Hi all,

 In the previous discussion, we decided to split the read and write path
 of data source v2 into 2 SPIPs, and I'm sending this email to call a vote
 for Data Source V2 read path only.

 The full document of the Data Source API V2 is:
 https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ
 -Z8qU5Frf6WMQZ6jJVM/edit

 The ready-for-review PR that implements the basic infrastructure for
 the read path is:
 https://github.com/apache/spark/pull/19136

 The vote will be up for the next 72 hours. Please reply with your vote:

 +1: Yeah, let's go forward and implement the SPIP.
 +0: Don't really care.
 -1: I don't think this is a good idea because of the following
 technical reasons.

 Thanks!



Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-07 Thread Ryan Blue
+1 (non-binding)

Thanks for making the updates reflected in the current PR. It would be
great to see the doc updated before it is finally published though.

Right now it feels like this SPIP is focused more on getting the basics
right for what many datasources are already doing in API V1 combined with
other private APIs, vs pushing forward state of the art for performance.

I think that’s the right approach for this SPIP. We can add the support
you’re talking about later with a more specific plan that doesn’t block
fixing the problems that this addresses.
​

On Thu, Sep 7, 2017 at 2:00 AM, Herman van Hövell tot Westerflier <
hvanhov...@databricks.com> wrote:

> +1 (binding)
>
> I personally believe that there is quite a big difference between having a
> generic data source interface with a low surface area and pushing down a
> significant part of query processing into a datasource. The later has much
> wider wider surface area and will require us to stabilize most of the
> internal catalyst API's which will be a significant burden on the community
> to maintain and has the potential to slow development velocity
> significantly. If you want to write such integrations then you should be
> prepared to work with catalyst internals and own up to the fact that things
> might change across minor versions (and in some cases even maintenance
> releases). If you are willing to go down that road, then your best bet is
> to use the already existing spark session extensions which will allow you
> to write such integrations and can be used as an `escape hatch`.
>
>
> On Thu, Sep 7, 2017 at 10:23 AM, Andrew Ash  wrote:
>
>> +0 (non-binding)
>>
>> I think there are benefits to unifying all the Spark-internal datasources
>> into a common public API for sure.  It will serve as a forcing function to
>> ensure that those internal datasources aren't advantaged vs datasources
>> developed externally as plugins to Spark, and that all Spark features are
>> available to all datasources.
>>
>> But I also think this read-path proposal avoids the more difficult
>> questions around how to continue pushing datasource performance forwards.
>> James Baker (my colleague) had a number of questions about advanced
>> pushdowns (combined sorting and filtering), and Reynold also noted that
>> pushdown of aggregates and joins are desirable on longer timeframes as
>> well.  The Spark community saw similar requests, for aggregate pushdown in
>> SPARK-12686, join pushdown in SPARK-20259, and arbitrary plan pushdown
>> in SPARK-12449.  Clearly a number of people are interested in this kind of
>> performance work for datasources.
>>
>> To leave enough space for datasource developers to continue experimenting
>> with advanced interactions between Spark and their datasources, I'd propose
>> we leave some sort of escape valve that enables these datasources to keep
>> pushing the boundaries without forking Spark.  Possibly that looks like an
>> additional unsupported/unstable interface that pushes down an entire
>> (unstable API) logical plan, which is expected to break API on every
>> release.   (Spark attempts this full-plan pushdown, and if that fails Spark
>> ignores it and continues on with the rest of the V2 API for
>> compatibility).  Or maybe it looks like something else that we don't know
>> of yet.  Possibly this falls outside of the desired goals for the V2 API
>> and instead should be a separate SPIP.
>>
>> If we had a plan for this kind of escape valve for advanced datasource
>> developers I'd be an unequivocal +1.  Right now it feels like this SPIP is
>> focused more on getting the basics right for what many datasources are
>> already doing in API V1 combined with other private APIs, vs pushing
>> forward state of the art for performance.
>>
>> Andrew
>>
>> On Wed, Sep 6, 2017 at 10:56 PM, Suresh Thalamati <
>> suresh.thalam...@gmail.com> wrote:
>>
>>> +1 (non-binding)
>>>
>>>
>>> On Sep 6, 2017, at 7:29 PM, Wenchen Fan  wrote:
>>>
>>> Hi all,
>>>
>>> In the previous discussion, we decided to split the read and write path
>>> of data source v2 into 2 SPIPs, and I'm sending this email to call a vote
>>> for Data Source V2 read path only.
>>>
>>> The full document of the Data Source API V2 is:
>>> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ
>>> -Z8qU5Frf6WMQZ6jJVM/edit
>>>
>>> The ready-for-review PR that implements the basic infrastructure for the
>>> read path is:
>>> https://github.com/apache/spark/pull/19136
>>>
>>> The vote will be up for the next 72 hours. Please reply with your vote:
>>>
>>> +1: Yeah, let's go forward and implement the SPIP.
>>> +0: Don't really care.
>>> -1: I don't think this is a good idea because of the following technical
>>> reasons.
>>>
>>> Thanks!
>>>
>>>
>>>
>>
>
>
> --
>
> Herman van Hövell
>
> Software Engineer
>
> Databricks Inc.
>
> hvanhov...@databricks.com
>
> +31 6 420 590 27
>
> databricks.com
>
> [image: http://databricks.com] 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-07 Thread Herman van Hövell tot Westerflier
+1 (binding)

I personally believe that there is quite a big difference between having a
generic data source interface with a low surface area and pushing down a
significant part of query processing into a datasource. The later has much
wider wider surface area and will require us to stabilize most of the
internal catalyst API's which will be a significant burden on the community
to maintain and has the potential to slow development velocity
significantly. If you want to write such integrations then you should be
prepared to work with catalyst internals and own up to the fact that things
might change across minor versions (and in some cases even maintenance
releases). If you are willing to go down that road, then your best bet is
to use the already existing spark session extensions which will allow you
to write such integrations and can be used as an `escape hatch`.


On Thu, Sep 7, 2017 at 10:23 AM, Andrew Ash  wrote:

> +0 (non-binding)
>
> I think there are benefits to unifying all the Spark-internal datasources
> into a common public API for sure.  It will serve as a forcing function to
> ensure that those internal datasources aren't advantaged vs datasources
> developed externally as plugins to Spark, and that all Spark features are
> available to all datasources.
>
> But I also think this read-path proposal avoids the more difficult
> questions around how to continue pushing datasource performance forwards.
> James Baker (my colleague) had a number of questions about advanced
> pushdowns (combined sorting and filtering), and Reynold also noted that
> pushdown of aggregates and joins are desirable on longer timeframes as
> well.  The Spark community saw similar requests, for aggregate pushdown in
> SPARK-12686, join pushdown in SPARK-20259, and arbitrary plan pushdown
> in SPARK-12449.  Clearly a number of people are interested in this kind of
> performance work for datasources.
>
> To leave enough space for datasource developers to continue experimenting
> with advanced interactions between Spark and their datasources, I'd propose
> we leave some sort of escape valve that enables these datasources to keep
> pushing the boundaries without forking Spark.  Possibly that looks like an
> additional unsupported/unstable interface that pushes down an entire
> (unstable API) logical plan, which is expected to break API on every
> release.   (Spark attempts this full-plan pushdown, and if that fails Spark
> ignores it and continues on with the rest of the V2 API for
> compatibility).  Or maybe it looks like something else that we don't know
> of yet.  Possibly this falls outside of the desired goals for the V2 API
> and instead should be a separate SPIP.
>
> If we had a plan for this kind of escape valve for advanced datasource
> developers I'd be an unequivocal +1.  Right now it feels like this SPIP is
> focused more on getting the basics right for what many datasources are
> already doing in API V1 combined with other private APIs, vs pushing
> forward state of the art for performance.
>
> Andrew
>
> On Wed, Sep 6, 2017 at 10:56 PM, Suresh Thalamati <
> suresh.thalam...@gmail.com> wrote:
>
>> +1 (non-binding)
>>
>>
>> On Sep 6, 2017, at 7:29 PM, Wenchen Fan  wrote:
>>
>> Hi all,
>>
>> In the previous discussion, we decided to split the read and write path
>> of data source v2 into 2 SPIPs, and I'm sending this email to call a vote
>> for Data Source V2 read path only.
>>
>> The full document of the Data Source API V2 is:
>> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ
>> -Z8qU5Frf6WMQZ6jJVM/edit
>>
>> The ready-for-review PR that implements the basic infrastructure for the
>> read path is:
>> https://github.com/apache/spark/pull/19136
>>
>> The vote will be up for the next 72 hours. Please reply with your vote:
>>
>> +1: Yeah, let's go forward and implement the SPIP.
>> +0: Don't really care.
>> -1: I don't think this is a good idea because of the following technical
>> reasons.
>>
>> Thanks!
>>
>>
>>
>


-- 

Herman van Hövell

Software Engineer

Databricks Inc.

hvanhov...@databricks.com

+31 6 420 590 27

databricks.com

[image: http://databricks.com] 



[image: Announcing Databricks Serverless. The first serverless data science
and big data platform. Watch the demo from Spark Summit 2017.]



Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-07 Thread Andrew Ash
+0 (non-binding)

I think there are benefits to unifying all the Spark-internal datasources
into a common public API for sure.  It will serve as a forcing function to
ensure that those internal datasources aren't advantaged vs datasources
developed externally as plugins to Spark, and that all Spark features are
available to all datasources.

But I also think this read-path proposal avoids the more difficult
questions around how to continue pushing datasource performance forwards.
James Baker (my colleague) had a number of questions about advanced
pushdowns (combined sorting and filtering), and Reynold also noted that
pushdown of aggregates and joins are desirable on longer timeframes as
well.  The Spark community saw similar requests, for aggregate pushdown in
SPARK-12686, join pushdown in SPARK-20259, and arbitrary plan pushdown
in SPARK-12449.  Clearly a number of people are interested in this kind of
performance work for datasources.

To leave enough space for datasource developers to continue experimenting
with advanced interactions between Spark and their datasources, I'd propose
we leave some sort of escape valve that enables these datasources to keep
pushing the boundaries without forking Spark.  Possibly that looks like an
additional unsupported/unstable interface that pushes down an entire
(unstable API) logical plan, which is expected to break API on every
release.   (Spark attempts this full-plan pushdown, and if that fails Spark
ignores it and continues on with the rest of the V2 API for
compatibility).  Or maybe it looks like something else that we don't know
of yet.  Possibly this falls outside of the desired goals for the V2 API
and instead should be a separate SPIP.

If we had a plan for this kind of escape valve for advanced datasource
developers I'd be an unequivocal +1.  Right now it feels like this SPIP is
focused more on getting the basics right for what many datasources are
already doing in API V1 combined with other private APIs, vs pushing
forward state of the art for performance.

Andrew

On Wed, Sep 6, 2017 at 10:56 PM, Suresh Thalamati <
suresh.thalam...@gmail.com> wrote:

> +1 (non-binding)
>
>
> On Sep 6, 2017, at 7:29 PM, Wenchen Fan  wrote:
>
> Hi all,
>
> In the previous discussion, we decided to split the read and write path of
> data source v2 into 2 SPIPs, and I'm sending this email to call a vote for
> Data Source V2 read path only.
>
> The full document of the Data Source API V2 is:
> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-
> Z8qU5Frf6WMQZ6jJVM/edit
>
> The ready-for-review PR that implements the basic infrastructure for the
> read path is:
> https://github.com/apache/spark/pull/19136
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following technical
> reasons.
>
> Thanks!
>
>
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-06 Thread Suresh Thalamati
+1 (non-binding)


> On Sep 6, 2017, at 7:29 PM, Wenchen Fan  wrote:
> 
> Hi all,
> 
> In the previous discussion, we decided to split the read and write path of 
> data source v2 into 2 SPIPs, and I'm sending this email to call a vote for 
> Data Source V2 read path only.
> 
> The full document of the Data Source API V2 is:
> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-Z8qU5Frf6WMQZ6jJVM/edit
>  
> 
> 
> The ready-for-review PR that implements the basic infrastructure for the read 
> path is:
> https://github.com/apache/spark/pull/19136 
> 
> 
> The vote will be up for the next 72 hours. Please reply with your vote:
> 
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following technical 
> reasons.
> 
> Thanks!



Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-06 Thread Sameer Agarwal
+1

On Wed, Sep 6, 2017 at 8:53 PM, Xiao Li  wrote:

> +1
>
> Xiao
>
> 2017-09-06 19:37 GMT-07:00 Wenchen Fan :
>
>> adding my own +1 (binding)
>>
>> On Thu, Sep 7, 2017 at 10:29 AM, Wenchen Fan  wrote:
>>
>>> Hi all,
>>>
>>> In the previous discussion, we decided to split the read and write path
>>> of data source v2 into 2 SPIPs, and I'm sending this email to call a vote
>>> for Data Source V2 read path only.
>>>
>>> The full document of the Data Source API V2 is:
>>> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ
>>> -Z8qU5Frf6WMQZ6jJVM/edit
>>>
>>> The ready-for-review PR that implements the basic infrastructure for the
>>> read path is:
>>> https://github.com/apache/spark/pull/19136
>>>
>>> The vote will be up for the next 72 hours. Please reply with your vote:
>>>
>>> +1: Yeah, let's go forward and implement the SPIP.
>>> +0: Don't really care.
>>> -1: I don't think this is a good idea because of the following technical
>>> reasons.
>>>
>>> Thanks!
>>>
>>
>>
>


-- 
Sameer Agarwal
Software Engineer | Databricks Inc.
http://cs.berkeley.edu/~sameerag


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-06 Thread Xiao Li
+1

Xiao

2017-09-06 19:37 GMT-07:00 Wenchen Fan :

> adding my own +1 (binding)
>
> On Thu, Sep 7, 2017 at 10:29 AM, Wenchen Fan  wrote:
>
>> Hi all,
>>
>> In the previous discussion, we decided to split the read and write path
>> of data source v2 into 2 SPIPs, and I'm sending this email to call a vote
>> for Data Source V2 read path only.
>>
>> The full document of the Data Source API V2 is:
>> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ
>> -Z8qU5Frf6WMQZ6jJVM/edit
>>
>> The ready-for-review PR that implements the basic infrastructure for the
>> read path is:
>> https://github.com/apache/spark/pull/19136
>>
>> The vote will be up for the next 72 hours. Please reply with your vote:
>>
>> +1: Yeah, let's go forward and implement the SPIP.
>> +0: Don't really care.
>> -1: I don't think this is a good idea because of the following technical
>> reasons.
>>
>> Thanks!
>>
>
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2 read path

2017-09-06 Thread Wenchen Fan
adding my own +1 (binding)

On Thu, Sep 7, 2017 at 10:29 AM, Wenchen Fan  wrote:

> Hi all,
>
> In the previous discussion, we decided to split the read and write path of
> data source v2 into 2 SPIPs, and I'm sending this email to call a vote for
> Data Source V2 read path only.
>
> The full document of the Data Source API V2 is:
> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-
> Z8qU5Frf6WMQZ6jJVM/edit
>
> The ready-for-review PR that implements the basic infrastructure for the
> read path is:
> https://github.com/apache/spark/pull/19136
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following technical
> reasons.
>
> Thanks!
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-09-06 Thread Wenchen Fan
Hi Ryan,

Yea I agree with you that we should discuss some substantial details during
the vote, and I addressed your comments about schema inference API in my
new PR, please take a look.

I've also called a new vote for the read path, please vote there, thanks!

On Thu, Sep 7, 2017 at 7:55 AM, Ryan Blue  wrote:

> I'm all for keeping this moving and not getting too far into the details
> (like naming), but I think the substantial details should be clarified
> first since they are in the proposal that's being voted on.
>
> I would prefer moving the write side to a separate SPIP, too, since there
> isn't much detail in the proposal and I think we should be more deliberate
> with things like schema evolution.
>
> On Thu, Aug 31, 2017 at 10:33 AM, Wenchen Fan  wrote:
>
>> Hi Ryan,
>>
>> I think for a SPIP, we should not worry too much about details, as we can
>> discuss them during PR review after the vote pass.
>>
>> I think we should focus more on the overall design, like James did. The
>> interface mix-in vs plan push down discussion was great, hope we can get a
>> consensus on this topic soon. The current proposal is, we keep the
>> interface mix-in framework, and add an unstable plan push down trait.
>>
>> For details like interface names, sort push down vs sort propagate, etc.,
>> I think they should not block the vote, as they can be updated/improved
>> within the current interface mix-in framework.
>>
>> About separating read/write proposals, we should definitely send
>> individual PRs for read/write when developing data source v2. I'm also OK
>> with voting on the read side first. The write side is way simpler than the
>> read side, I think it's more important to get agreement on the read side
>> first.
>>
>> BTW, I do appreciate your feedbacks/comments on the prototype, let's keep
>> the discussion there. In the meanwhile, let's have more discussion on the
>> overall framework, and drive this project together.
>>
>> Wenchen
>>
>>
>>
>> On Thu, Aug 31, 2017 at 6:22 AM, Ryan Blue  wrote:
>>
>>> Maybe I'm missing something, but the high-level proposal consists of:
>>> Goals, Non-Goals, and Proposed API. What is there to discuss other than the
>>> details of the API that's being proposed? I think the goals make sense, but
>>> goals alone aren't enough to approve a SPIP.
>>>
>>> On Wed, Aug 30, 2017 at 2:46 PM, Reynold Xin 
>>> wrote:
>>>
 So we seem to be getting into a cycle of discussing more about the
 details of APIs than the high level proposal. The details of APIs are
 important to debate, but those belong more in code reviews.

 One other important thing is that we should avoid API design by
 committee. While it is extremely useful to get feedback, understand the use
 cases, we cannot do API design by incorporating verbatim the union of
 everybody's feedback. API design is largely a tradeoff game. The most
 expressive API would also be harder to use, or sacrifice backward/forward
 compatibility. It is as important to decide what to exclude as what to
 include.

 Unlike the v1 API, the way Wenchen's high level V2 framework is
 proposed makes it very easy to add new features (e.g. clustering
 properties) in the future without breaking any APIs. I'd rather us shipping
 something useful that might not be the most comprehensive set, than
 debating about every single feature we should add and then creating
 something super complicated that has unclear value.



 On Wed, Aug 30, 2017 at 6:37 PM, Ryan Blue  wrote:

> -1 (non-binding)
>
> Sometimes it takes a VOTE thread to get people to actually read and
> comment, so thanks for starting this one… but there’s still discussion
> happening on the prototype API, which it hasn’t been updated. I’d like to
> see the proposal shaped by the ongoing discussion so that we have a 
> better,
> more concrete plan. I think that’s going to produces a better SPIP.
>
> The second reason for -1 is that I think the read- and write-side
> proposals should be separated. The PR
>  currently has “write
> path” listed as a TODO item and most of the discussion I’ve seen is on the
> read side. I think it would be better to separate the read and write APIs
> so we can focus on them individually.
>
> An example of why we should focus on the write path separately is that
> the proposal says this:
>
> Ideally partitioning/bucketing concept should not be exposed in the
> Data Source API V2, because they are just techniques for data skipping and
> pre-partitioning. However, these 2 concepts are already widely used in
> Spark, e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION.
> To be consistent, we need to add partitioning/bucketing to Data 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-09-06 Thread Ryan Blue
I'm all for keeping this moving and not getting too far into the details
(like naming), but I think the substantial details should be clarified
first since they are in the proposal that's being voted on.

I would prefer moving the write side to a separate SPIP, too, since there
isn't much detail in the proposal and I think we should be more deliberate
with things like schema evolution.

On Thu, Aug 31, 2017 at 10:33 AM, Wenchen Fan  wrote:

> Hi Ryan,
>
> I think for a SPIP, we should not worry too much about details, as we can
> discuss them during PR review after the vote pass.
>
> I think we should focus more on the overall design, like James did. The
> interface mix-in vs plan push down discussion was great, hope we can get a
> consensus on this topic soon. The current proposal is, we keep the
> interface mix-in framework, and add an unstable plan push down trait.
>
> For details like interface names, sort push down vs sort propagate, etc.,
> I think they should not block the vote, as they can be updated/improved
> within the current interface mix-in framework.
>
> About separating read/write proposals, we should definitely send
> individual PRs for read/write when developing data source v2. I'm also OK
> with voting on the read side first. The write side is way simpler than the
> read side, I think it's more important to get agreement on the read side
> first.
>
> BTW, I do appreciate your feedbacks/comments on the prototype, let's keep
> the discussion there. In the meanwhile, let's have more discussion on the
> overall framework, and drive this project together.
>
> Wenchen
>
>
>
> On Thu, Aug 31, 2017 at 6:22 AM, Ryan Blue  wrote:
>
>> Maybe I'm missing something, but the high-level proposal consists of:
>> Goals, Non-Goals, and Proposed API. What is there to discuss other than the
>> details of the API that's being proposed? I think the goals make sense, but
>> goals alone aren't enough to approve a SPIP.
>>
>> On Wed, Aug 30, 2017 at 2:46 PM, Reynold Xin  wrote:
>>
>>> So we seem to be getting into a cycle of discussing more about the
>>> details of APIs than the high level proposal. The details of APIs are
>>> important to debate, but those belong more in code reviews.
>>>
>>> One other important thing is that we should avoid API design by
>>> committee. While it is extremely useful to get feedback, understand the use
>>> cases, we cannot do API design by incorporating verbatim the union of
>>> everybody's feedback. API design is largely a tradeoff game. The most
>>> expressive API would also be harder to use, or sacrifice backward/forward
>>> compatibility. It is as important to decide what to exclude as what to
>>> include.
>>>
>>> Unlike the v1 API, the way Wenchen's high level V2 framework is proposed
>>> makes it very easy to add new features (e.g. clustering properties) in the
>>> future without breaking any APIs. I'd rather us shipping something useful
>>> that might not be the most comprehensive set, than debating about every
>>> single feature we should add and then creating something super complicated
>>> that has unclear value.
>>>
>>>
>>>
>>> On Wed, Aug 30, 2017 at 6:37 PM, Ryan Blue  wrote:
>>>
 -1 (non-binding)

 Sometimes it takes a VOTE thread to get people to actually read and
 comment, so thanks for starting this one… but there’s still discussion
 happening on the prototype API, which it hasn’t been updated. I’d like to
 see the proposal shaped by the ongoing discussion so that we have a better,
 more concrete plan. I think that’s going to produces a better SPIP.

 The second reason for -1 is that I think the read- and write-side
 proposals should be separated. The PR
  currently has “write
 path” listed as a TODO item and most of the discussion I’ve seen is on the
 read side. I think it would be better to separate the read and write APIs
 so we can focus on them individually.

 An example of why we should focus on the write path separately is that
 the proposal says this:

 Ideally partitioning/bucketing concept should not be exposed in the
 Data Source API V2, because they are just techniques for data skipping and
 pre-partitioning. However, these 2 concepts are already widely used in
 Spark, e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION.
 To be consistent, we need to add partitioning/bucketing to Data Source V2 .
 . .

 Essentially, the some APIs mix DDL and DML operations. I’d like to
 consider ways to fix that problem instead of carrying the problem forward
 to Data Source V2. We can solve this by adding a high-level API for DDL and
 a better write/insert API that works well with it. Clearly, that discussion
 is independent of the read path, which is why I think separating the two
 proposals would be a win.

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-09-06 Thread Wenchen Fan
Hi all,

I've submitted a PR for a basic data source v2, i.e., only contains
features we already have in data source v1. We can discuss API details like
naming in that PR: https://github.com/apache/spark/pull/19136

In the meanwhile, let's keep this vote open and collecting more feedbacks.

Thanks


On Fri, Sep 1, 2017 at 5:56 PM, Reynold Xin  wrote:

> Why does ordering matter here for sort vs filter? The source should be
> able to handle it in whatever way it wants (which is almost always filter
> beneath sort I'd imagine).
>
> The only ordering that'd matter in the current set of pushdowns is limit -
> it should always mean the root of the pushded tree.
>
>
> On Fri, Sep 1, 2017 at 3:22 AM, Wenchen Fan  wrote:
>
>> > Ideally also getting sort orders _after_ getting filters.
>>
>> Yea we should have a deterministic order when applying various push
>> downs, and I think filter should definitely go before sort. This is one of
>> the details we can discuss during PR review :)
>>
>> On Fri, Sep 1, 2017 at 9:19 AM, James Baker  wrote:
>>
>>> I think that makes sense. I didn't understand backcompat was the primary
>>> driver. I actually don't care right now about aggregations on the
>>> datasource I'm integrating with - I just care about receiving all the
>>> filters (and ideally also the desired sort order) at the same time. I am
>>> mostly fine with anything else; but getting filters at the same time is
>>> important for me, and doesn't seem overly contentious? (e.g. it's
>>> compatible with datasources v1). Ideally also getting sort orders _after_
>>> getting filters.
>>>
>>> That said, an unstable api that gets me the query plan would be
>>> appreciated by plenty I'm sure :) (and would make my implementation more
>>> straightforward - the state management is painful atm).
>>>
>>> James
>>>
>>> On Wed, 30 Aug 2017 at 14:56 Reynold Xin  wrote:
>>>
 Sure that's good to do (and as discussed earlier a good compromise
 might be to expose an interface for the source to decide which part of the
 logical plan they want to accept).

 To me everything is about cost vs benefit.

 In my mind, the biggest issue with the existing data source API is
 backward and forward compatibility. All the data sources written for Spark
 1.x broke in Spark 2.x. And that's one of the biggest value v2 can bring.
 To me it's far more important to have data sources implemented in 2017 to
 be able to work in 2027, in Spark 10.x.

 You are basically arguing for creating a new API that is capable of
 doing arbitrary expression, aggregation, and join pushdowns (you only
 mentioned aggregation so far, but I've talked to enough database people
 that I know once Spark gives them aggregation pushdown, they will come back
 for join pushdown). We can do that using unstable APIs, and creating stable
 APIs would be extremely difficult (still doable, just would take a long
 time to design and implement). As mentioned earlier, it basically involves
 creating a stable representation for all of logical plan, which is a lot of
 work. I think we should still work towards that (for other reasons as
 well), but I'd consider that out of scope for the current one. Otherwise
 we'd not release something probably for the next 2 or 3 years.





 On Wed, Aug 30, 2017 at 11:50 PM, James Baker 
 wrote:

> I guess I was more suggesting that by coding up the powerful mode as
> the API, it becomes easy for someone to layer an easy mode beneath it to
> enable simpler datasources to be integrated (and that simple mode should 
> be
> the out of scope thing).
>
> Taking a small step back here, one of the places where I think I'm
> missing some context is in understanding the target consumers of these
> interfaces. I've done some amount (though likely not enough) of research
> about the places where people have had issues of API surface in the past -
> the concrete tickets I've seen have been based on Cassandra integration
> where you want to indicate clustering, and SAP HANA where they want to 
> push
> down more complicated queries through Spark. This proposal supports the
> former, but the amount of change required to support clustering in the
> current API is not obviously high - whilst the current proposal for V2
> seems to make it very difficult to add support for pushing down plenty of
> aggregations in the future (I've found the question of how to add GROUP BY
> to be pretty tricky to answer for the current proposal).
>
> Googling around for implementations of the current PrunedFilteredScan,
> I basically find a lot of databases, which seems reasonable - SAP HANA,
> ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people
> who've 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-09-01 Thread Reynold Xin
Why does ordering matter here for sort vs filter? The source should be able
to handle it in whatever way it wants (which is almost always filter
beneath sort I'd imagine).

The only ordering that'd matter in the current set of pushdowns is limit -
it should always mean the root of the pushded tree.


On Fri, Sep 1, 2017 at 3:22 AM, Wenchen Fan  wrote:

> > Ideally also getting sort orders _after_ getting filters.
>
> Yea we should have a deterministic order when applying various push downs,
> and I think filter should definitely go before sort. This is one of the
> details we can discuss during PR review :)
>
> On Fri, Sep 1, 2017 at 9:19 AM, James Baker  wrote:
>
>> I think that makes sense. I didn't understand backcompat was the primary
>> driver. I actually don't care right now about aggregations on the
>> datasource I'm integrating with - I just care about receiving all the
>> filters (and ideally also the desired sort order) at the same time. I am
>> mostly fine with anything else; but getting filters at the same time is
>> important for me, and doesn't seem overly contentious? (e.g. it's
>> compatible with datasources v1). Ideally also getting sort orders _after_
>> getting filters.
>>
>> That said, an unstable api that gets me the query plan would be
>> appreciated by plenty I'm sure :) (and would make my implementation more
>> straightforward - the state management is painful atm).
>>
>> James
>>
>> On Wed, 30 Aug 2017 at 14:56 Reynold Xin  wrote:
>>
>>> Sure that's good to do (and as discussed earlier a good compromise might
>>> be to expose an interface for the source to decide which part of the
>>> logical plan they want to accept).
>>>
>>> To me everything is about cost vs benefit.
>>>
>>> In my mind, the biggest issue with the existing data source API is
>>> backward and forward compatibility. All the data sources written for Spark
>>> 1.x broke in Spark 2.x. And that's one of the biggest value v2 can bring.
>>> To me it's far more important to have data sources implemented in 2017 to
>>> be able to work in 2027, in Spark 10.x.
>>>
>>> You are basically arguing for creating a new API that is capable of
>>> doing arbitrary expression, aggregation, and join pushdowns (you only
>>> mentioned aggregation so far, but I've talked to enough database people
>>> that I know once Spark gives them aggregation pushdown, they will come back
>>> for join pushdown). We can do that using unstable APIs, and creating stable
>>> APIs would be extremely difficult (still doable, just would take a long
>>> time to design and implement). As mentioned earlier, it basically involves
>>> creating a stable representation for all of logical plan, which is a lot of
>>> work. I think we should still work towards that (for other reasons as
>>> well), but I'd consider that out of scope for the current one. Otherwise
>>> we'd not release something probably for the next 2 or 3 years.
>>>
>>>
>>>
>>>
>>>
>>> On Wed, Aug 30, 2017 at 11:50 PM, James Baker 
>>> wrote:
>>>
 I guess I was more suggesting that by coding up the powerful mode as
 the API, it becomes easy for someone to layer an easy mode beneath it to
 enable simpler datasources to be integrated (and that simple mode should be
 the out of scope thing).

 Taking a small step back here, one of the places where I think I'm
 missing some context is in understanding the target consumers of these
 interfaces. I've done some amount (though likely not enough) of research
 about the places where people have had issues of API surface in the past -
 the concrete tickets I've seen have been based on Cassandra integration
 where you want to indicate clustering, and SAP HANA where they want to push
 down more complicated queries through Spark. This proposal supports the
 former, but the amount of change required to support clustering in the
 current API is not obviously high - whilst the current proposal for V2
 seems to make it very difficult to add support for pushing down plenty of
 aggregations in the future (I've found the question of how to add GROUP BY
 to be pretty tricky to answer for the current proposal).

 Googling around for implementations of the current PrunedFilteredScan,
 I basically find a lot of databases, which seems reasonable - SAP HANA,
 ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people
 who've used (some of) these connectors and the sticking point has generally
 been that Spark needs to load a lot of data out in order to solve
 aggregations that can be very efficiently pushed down into the datasources.

 So, with this proposal it appears that we're optimising towards making
 it easy to write one-off datasource integrations, with some amount of
 pluggability for people who want to do more complicated things (the most
 interesting being 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-31 Thread Wenchen Fan
> Ideally also getting sort orders _after_ getting filters.

Yea we should have a deterministic order when applying various push downs,
and I think filter should definitely go before sort. This is one of the
details we can discuss during PR review :)

On Fri, Sep 1, 2017 at 9:19 AM, James Baker  wrote:

> I think that makes sense. I didn't understand backcompat was the primary
> driver. I actually don't care right now about aggregations on the
> datasource I'm integrating with - I just care about receiving all the
> filters (and ideally also the desired sort order) at the same time. I am
> mostly fine with anything else; but getting filters at the same time is
> important for me, and doesn't seem overly contentious? (e.g. it's
> compatible with datasources v1). Ideally also getting sort orders _after_
> getting filters.
>
> That said, an unstable api that gets me the query plan would be
> appreciated by plenty I'm sure :) (and would make my implementation more
> straightforward - the state management is painful atm).
>
> James
>
> On Wed, 30 Aug 2017 at 14:56 Reynold Xin  wrote:
>
>> Sure that's good to do (and as discussed earlier a good compromise might
>> be to expose an interface for the source to decide which part of the
>> logical plan they want to accept).
>>
>> To me everything is about cost vs benefit.
>>
>> In my mind, the biggest issue with the existing data source API is
>> backward and forward compatibility. All the data sources written for Spark
>> 1.x broke in Spark 2.x. And that's one of the biggest value v2 can bring.
>> To me it's far more important to have data sources implemented in 2017 to
>> be able to work in 2027, in Spark 10.x.
>>
>> You are basically arguing for creating a new API that is capable of doing
>> arbitrary expression, aggregation, and join pushdowns (you only mentioned
>> aggregation so far, but I've talked to enough database people that I know
>> once Spark gives them aggregation pushdown, they will come back for join
>> pushdown). We can do that using unstable APIs, and creating stable APIs
>> would be extremely difficult (still doable, just would take a long time to
>> design and implement). As mentioned earlier, it basically involves creating
>> a stable representation for all of logical plan, which is a lot of work. I
>> think we should still work towards that (for other reasons as well), but
>> I'd consider that out of scope for the current one. Otherwise we'd not
>> release something probably for the next 2 or 3 years.
>>
>>
>>
>>
>>
>> On Wed, Aug 30, 2017 at 11:50 PM, James Baker 
>> wrote:
>>
>>> I guess I was more suggesting that by coding up the powerful mode as the
>>> API, it becomes easy for someone to layer an easy mode beneath it to enable
>>> simpler datasources to be integrated (and that simple mode should be the
>>> out of scope thing).
>>>
>>> Taking a small step back here, one of the places where I think I'm
>>> missing some context is in understanding the target consumers of these
>>> interfaces. I've done some amount (though likely not enough) of research
>>> about the places where people have had issues of API surface in the past -
>>> the concrete tickets I've seen have been based on Cassandra integration
>>> where you want to indicate clustering, and SAP HANA where they want to push
>>> down more complicated queries through Spark. This proposal supports the
>>> former, but the amount of change required to support clustering in the
>>> current API is not obviously high - whilst the current proposal for V2
>>> seems to make it very difficult to add support for pushing down plenty of
>>> aggregations in the future (I've found the question of how to add GROUP BY
>>> to be pretty tricky to answer for the current proposal).
>>>
>>> Googling around for implementations of the current PrunedFilteredScan, I
>>> basically find a lot of databases, which seems reasonable - SAP HANA,
>>> ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people
>>> who've used (some of) these connectors and the sticking point has generally
>>> been that Spark needs to load a lot of data out in order to solve
>>> aggregations that can be very efficiently pushed down into the datasources.
>>>
>>> So, with this proposal it appears that we're optimising towards making
>>> it easy to write one-off datasource integrations, with some amount of
>>> pluggability for people who want to do more complicated things (the most
>>> interesting being bucketing integration). However, my guess is that this
>>> isn't what the current major integrations suffer from; they suffer mostly
>>> from restrictions in what they can push down (which broadly speaking are
>>> not going to go away).
>>>
>>> So the place where I'm confused is that the current integrations can be
>>> made incrementally better as a consequence of this, but the backing data
>>> systems have the features which enable a step change which this API 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-31 Thread James Baker
I think that makes sense. I didn't understand backcompat was the primary 
driver. I actually don't care right now about aggregations on the datasource 
I'm integrating with - I just care about receiving all the filters (and ideally 
also the desired sort order) at the same time. I am mostly fine with anything 
else; but getting filters at the same time is important for me, and doesn't 
seem overly contentious? (e.g. it's compatible with datasources v1). Ideally 
also getting sort orders _after_ getting filters.

That said, an unstable api that gets me the query plan would be appreciated by 
plenty I'm sure :) (and would make my implementation more straightforward - the 
state management is painful atm).

James

On Wed, 30 Aug 2017 at 14:56 Reynold Xin 
> wrote:
Sure that's good to do (and as discussed earlier a good compromise might be to 
expose an interface for the source to decide which part of the logical plan 
they want to accept).

To me everything is about cost vs benefit.

In my mind, the biggest issue with the existing data source API is backward and 
forward compatibility. All the data sources written for Spark 1.x broke in 
Spark 2.x. And that's one of the biggest value v2 can bring. To me it's far 
more important to have data sources implemented in 2017 to be able to work in 
2027, in Spark 10.x.

You are basically arguing for creating a new API that is capable of doing 
arbitrary expression, aggregation, and join pushdowns (you only mentioned 
aggregation so far, but I've talked to enough database people that I know once 
Spark gives them aggregation pushdown, they will come back for join pushdown). 
We can do that using unstable APIs, and creating stable APIs would be extremely 
difficult (still doable, just would take a long time to design and implement). 
As mentioned earlier, it basically involves creating a stable representation 
for all of logical plan, which is a lot of work. I think we should still work 
towards that (for other reasons as well), but I'd consider that out of scope 
for the current one. Otherwise we'd not release something probably for the next 
2 or 3 years.





On Wed, Aug 30, 2017 at 11:50 PM, James Baker 
> wrote:
I guess I was more suggesting that by coding up the powerful mode as the API, 
it becomes easy for someone to layer an easy mode beneath it to enable simpler 
datasources to be integrated (and that simple mode should be the out of scope 
thing).

Taking a small step back here, one of the places where I think I'm missing some 
context is in understanding the target consumers of these interfaces. I've done 
some amount (though likely not enough) of research about the places where 
people have had issues of API surface in the past - the concrete tickets I've 
seen have been based on Cassandra integration where you want to indicate 
clustering, and SAP HANA where they want to push down more complicated queries 
through Spark. This proposal supports the former, but the amount of change 
required to support clustering in the current API is not obviously high - 
whilst the current proposal for V2 seems to make it very difficult to add 
support for pushing down plenty of aggregations in the future (I've found the 
question of how to add GROUP BY to be pretty tricky to answer for the current 
proposal).

Googling around for implementations of the current PrunedFilteredScan, I 
basically find a lot of databases, which seems reasonable - SAP HANA, 
ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people who've 
used (some of) these connectors and the sticking point has generally been that 
Spark needs to load a lot of data out in order to solve aggregations that can 
be very efficiently pushed down into the datasources.

So, with this proposal it appears that we're optimising towards making it easy 
to write one-off datasource integrations, with some amount of pluggability for 
people who want to do more complicated things (the most interesting being 
bucketing integration). However, my guess is that this isn't what the current 
major integrations suffer from; they suffer mostly from restrictions in what 
they can push down (which broadly speaking are not going to go away).

So the place where I'm confused is that the current integrations can be made 
incrementally better as a consequence of this, but the backing data systems 
have the features which enable a step change which this API makes harder to 
achieve in the future. Who are the group of users who benefit the most as a 
consequence of this change, like, who is the target consumer here? My personal 
slant is that it's more important to improve support for other datastores than 
it is to lower the barrier of entry - this is why I've been pushing here.

James

On Wed, 30 Aug 2017 at 09:37 Ryan Blue 
> wrote:

-1 (non-binding)

Sometimes it takes a VOTE 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-31 Thread James Baker
I think that makes sense. I didn't understand backcompat was the primary 
driver. I actually don't care right now about aggregations on the datasource 
I'm integrating with - I just care about receiving all the filters (and ideally 
also the desired sort order) at the same time. I am mostly fine with anything 
else; but getting filters at the same time is important for me, and doesn't 
seem overly contentious? (e.g. it's compatible with datasources v1). Ideally 
also getting sort orders _after_ getting filters.

That said, an unstable api that gets me the query plan would be appreciated by 
plenty I'm sure :) (and would make my implementation more straightforward - the 
state management is painful atm).

James

On Wed, 30 Aug 2017 at 14:56 Reynold Xin 
> wrote:
Sure that's good to do (and as discussed earlier a good compromise might be to 
expose an interface for the source to decide which part of the logical plan 
they want to accept).

To me everything is about cost vs benefit.

In my mind, the biggest issue with the existing data source API is backward and 
forward compatibility. All the data sources written for Spark 1.x broke in 
Spark 2.x. And that's one of the biggest value v2 can bring. To me it's far 
more important to have data sources implemented in 2017 to be able to work in 
2027, in Spark 10.x.

You are basically arguing for creating a new API that is capable of doing 
arbitrary expression, aggregation, and join pushdowns (you only mentioned 
aggregation so far, but I've talked to enough database people that I know once 
Spark gives them aggregation pushdown, they will come back for join pushdown). 
We can do that using unstable APIs, and creating stable APIs would be extremely 
difficult (still doable, just would take a long time to design and implement). 
As mentioned earlier, it basically involves creating a stable representation 
for all of logical plan, which is a lot of work. I think we should still work 
towards that (for other reasons as well), but I'd consider that out of scope 
for the current one. Otherwise we'd not release something probably for the next 
2 or 3 years.





On Wed, Aug 30, 2017 at 11:50 PM, James Baker 
> wrote:
I guess I was more suggesting that by coding up the powerful mode as the API, 
it becomes easy for someone to layer an easy mode beneath it to enable simpler 
datasources to be integrated (and that simple mode should be the out of scope 
thing).

Taking a small step back here, one of the places where I think I'm missing some 
context is in understanding the target consumers of these interfaces. I've done 
some amount (though likely not enough) of research about the places where 
people have had issues of API surface in the past - the concrete tickets I've 
seen have been based on Cassandra integration where you want to indicate 
clustering, and SAP HANA where they want to push down more complicated queries 
through Spark. This proposal supports the former, but the amount of change 
required to support clustering in the current API is not obviously high - 
whilst the current proposal for V2 seems to make it very difficult to add 
support for pushing down plenty of aggregations in the future (I've found the 
question of how to add GROUP BY to be pretty tricky to answer for the current 
proposal).

Googling around for implementations of the current PrunedFilteredScan, I 
basically find a lot of databases, which seems reasonable - SAP HANA, 
ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people who've 
used (some of) these connectors and the sticking point has generally been that 
Spark needs to load a lot of data out in order to solve aggregations that can 
be very efficiently pushed down into the datasources.

So, with this proposal it appears that we're optimising towards making it easy 
to write one-off datasource integrations, with some amount of pluggability for 
people who want to do more complicated things (the most interesting being 
bucketing integration). However, my guess is that this isn't what the current 
major integrations suffer from; they suffer mostly from restrictions in what 
they can push down (which broadly speaking are not going to go away).

So the place where I'm confused is that the current integrations can be made 
incrementally better as a consequence of this, but the backing data systems 
have the features which enable a step change which this API makes harder to 
achieve in the future. Who are the group of users who benefit the most as a 
consequence of this change, like, who is the target consumer here? My personal 
slant is that it's more important to improve support for other datastores than 
it is to lower the barrier of entry - this is why I've been pushing here.

James

On Wed, 30 Aug 2017 at 09:37 Ryan Blue 
> wrote:

-1 (non-binding)

Sometimes it takes a VOTE 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-31 Thread James Baker
I think that makes sense. I didn't understand backcompat was the primary 
driver. I actually don't care right now about aggregations on the datasource 
I'm integrating with - I just care about receiving all the filters (and ideally 
also the desired sort order) at the same time. I am mostly fine with anything 
else; but getting filters at the same time is important for me, and doesn't 
seem overly contentious? (e.g. it's compatible with datasources v1). Ideally 
also getting sort orders _after_ getting filters.

That said, an unstable api that gets me the query plan would be appreciated by 
plenty I'm sure :) (and would make my implementation more straightforward - the 
state management is painful atm).

James

On Wed, 30 Aug 2017 at 14:56 Reynold Xin 
> wrote:
Sure that's good to do (and as discussed earlier a good compromise might be to 
expose an interface for the source to decide which part of the logical plan 
they want to accept).

To me everything is about cost vs benefit.

In my mind, the biggest issue with the existing data source API is backward and 
forward compatibility. All the data sources written for Spark 1.x broke in 
Spark 2.x. And that's one of the biggest value v2 can bring. To me it's far 
more important to have data sources implemented in 2017 to be able to work in 
2027, in Spark 10.x.

You are basically arguing for creating a new API that is capable of doing 
arbitrary expression, aggregation, and join pushdowns (you only mentioned 
aggregation so far, but I've talked to enough database people that I know once 
Spark gives them aggregation pushdown, they will come back for join pushdown). 
We can do that using unstable APIs, and creating stable APIs would be extremely 
difficult (still doable, just would take a long time to design and implement). 
As mentioned earlier, it basically involves creating a stable representation 
for all of logical plan, which is a lot of work. I think we should still work 
towards that (for other reasons as well), but I'd consider that out of scope 
for the current one. Otherwise we'd not release something probably for the next 
2 or 3 years.





On Wed, Aug 30, 2017 at 11:50 PM, James Baker 
> wrote:
I guess I was more suggesting that by coding up the powerful mode as the API, 
it becomes easy for someone to layer an easy mode beneath it to enable simpler 
datasources to be integrated (and that simple mode should be the out of scope 
thing).

Taking a small step back here, one of the places where I think I'm missing some 
context is in understanding the target consumers of these interfaces. I've done 
some amount (though likely not enough) of research about the places where 
people have had issues of API surface in the past - the concrete tickets I've 
seen have been based on Cassandra integration where you want to indicate 
clustering, and SAP HANA where they want to push down more complicated queries 
through Spark. This proposal supports the former, but the amount of change 
required to support clustering in the current API is not obviously high - 
whilst the current proposal for V2 seems to make it very difficult to add 
support for pushing down plenty of aggregations in the future (I've found the 
question of how to add GROUP BY to be pretty tricky to answer for the current 
proposal).

Googling around for implementations of the current PrunedFilteredScan, I 
basically find a lot of databases, which seems reasonable - SAP HANA, 
ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people who've 
used (some of) these connectors and the sticking point has generally been that 
Spark needs to load a lot of data out in order to solve aggregations that can 
be very efficiently pushed down into the datasources.

So, with this proposal it appears that we're optimising towards making it easy 
to write one-off datasource integrations, with some amount of pluggability for 
people who want to do more complicated things (the most interesting being 
bucketing integration). However, my guess is that this isn't what the current 
major integrations suffer from; they suffer mostly from restrictions in what 
they can push down (which broadly speaking are not going to go away).

So the place where I'm confused is that the current integrations can be made 
incrementally better as a consequence of this, but the backing data systems 
have the features which enable a step change which this API makes harder to 
achieve in the future. Who are the group of users who benefit the most as a 
consequence of this change, like, who is the target consumer here? My personal 
slant is that it's more important to improve support for other datastores than 
it is to lower the barrier of entry - this is why I've been pushing here.

James

On Wed, 30 Aug 2017 at 09:37 Ryan Blue 
> wrote:

-1 (non-binding)

Sometimes it takes a VOTE 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-31 Thread Wenchen Fan
Hi Ryan,

I think for a SPIP, we should not worry too much about details, as we can
discuss them during PR review after the vote pass.

I think we should focus more on the overall design, like James did. The
interface mix-in vs plan push down discussion was great, hope we can get a
consensus on this topic soon. The current proposal is, we keep the
interface mix-in framework, and add an unstable plan push down trait.

For details like interface names, sort push down vs sort propagate, etc., I
think they should not block the vote, as they can be updated/improved
within the current interface mix-in framework.

About separating read/write proposals, we should definitely send individual
PRs for read/write when developing data source v2. I'm also OK with voting
on the read side first. The write side is way simpler than the read side, I
think it's more important to get agreement on the read side first.

BTW, I do appreciate your feedbacks/comments on the prototype, let's keep
the discussion there. In the meanwhile, let's have more discussion on the
overall framework, and drive this project together.

Wenchen



On Thu, Aug 31, 2017 at 6:22 AM, Ryan Blue  wrote:

> Maybe I'm missing something, but the high-level proposal consists of:
> Goals, Non-Goals, and Proposed API. What is there to discuss other than the
> details of the API that's being proposed? I think the goals make sense, but
> goals alone aren't enough to approve a SPIP.
>
> On Wed, Aug 30, 2017 at 2:46 PM, Reynold Xin  wrote:
>
>> So we seem to be getting into a cycle of discussing more about the
>> details of APIs than the high level proposal. The details of APIs are
>> important to debate, but those belong more in code reviews.
>>
>> One other important thing is that we should avoid API design by
>> committee. While it is extremely useful to get feedback, understand the use
>> cases, we cannot do API design by incorporating verbatim the union of
>> everybody's feedback. API design is largely a tradeoff game. The most
>> expressive API would also be harder to use, or sacrifice backward/forward
>> compatibility. It is as important to decide what to exclude as what to
>> include.
>>
>> Unlike the v1 API, the way Wenchen's high level V2 framework is proposed
>> makes it very easy to add new features (e.g. clustering properties) in the
>> future without breaking any APIs. I'd rather us shipping something useful
>> that might not be the most comprehensive set, than debating about every
>> single feature we should add and then creating something super complicated
>> that has unclear value.
>>
>>
>>
>> On Wed, Aug 30, 2017 at 6:37 PM, Ryan Blue  wrote:
>>
>>> -1 (non-binding)
>>>
>>> Sometimes it takes a VOTE thread to get people to actually read and
>>> comment, so thanks for starting this one… but there’s still discussion
>>> happening on the prototype API, which it hasn’t been updated. I’d like to
>>> see the proposal shaped by the ongoing discussion so that we have a better,
>>> more concrete plan. I think that’s going to produces a better SPIP.
>>>
>>> The second reason for -1 is that I think the read- and write-side
>>> proposals should be separated. The PR
>>>  currently has “write path”
>>> listed as a TODO item and most of the discussion I’ve seen is on the read
>>> side. I think it would be better to separate the read and write APIs so we
>>> can focus on them individually.
>>>
>>> An example of why we should focus on the write path separately is that
>>> the proposal says this:
>>>
>>> Ideally partitioning/bucketing concept should not be exposed in the Data
>>> Source API V2, because they are just techniques for data skipping and
>>> pre-partitioning. However, these 2 concepts are already widely used in
>>> Spark, e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION.
>>> To be consistent, we need to add partitioning/bucketing to Data Source V2 .
>>> . .
>>>
>>> Essentially, the some APIs mix DDL and DML operations. I’d like to
>>> consider ways to fix that problem instead of carrying the problem forward
>>> to Data Source V2. We can solve this by adding a high-level API for DDL and
>>> a better write/insert API that works well with it. Clearly, that discussion
>>> is independent of the read path, which is why I think separating the two
>>> proposals would be a win.
>>>
>>> rb
>>> ​
>>>
>>> On Wed, Aug 30, 2017 at 4:28 AM, Reynold Xin 
>>> wrote:
>>>
 That might be good to do, but seems like orthogonal to this effort
 itself. It would be a completely different interface.

 On Wed, Aug 30, 2017 at 1:10 PM Wenchen Fan 
 wrote:

> OK I agree with it, how about we add a new interface to push down the
> query plan, based on the current framework? We can mark the
> query-plan-push-down interface as unstable, to save the effort of 
> designing
> a stable 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread Ryan Blue
Maybe I'm missing something, but the high-level proposal consists of:
Goals, Non-Goals, and Proposed API. What is there to discuss other than the
details of the API that's being proposed? I think the goals make sense, but
goals alone aren't enough to approve a SPIP.

On Wed, Aug 30, 2017 at 2:46 PM, Reynold Xin  wrote:

> So we seem to be getting into a cycle of discussing more about the details
> of APIs than the high level proposal. The details of APIs are important to
> debate, but those belong more in code reviews.
>
> One other important thing is that we should avoid API design by committee.
> While it is extremely useful to get feedback, understand the use cases, we
> cannot do API design by incorporating verbatim the union of everybody's
> feedback. API design is largely a tradeoff game. The most expressive API
> would also be harder to use, or sacrifice backward/forward compatibility.
> It is as important to decide what to exclude as what to include.
>
> Unlike the v1 API, the way Wenchen's high level V2 framework is proposed
> makes it very easy to add new features (e.g. clustering properties) in the
> future without breaking any APIs. I'd rather us shipping something useful
> that might not be the most comprehensive set, than debating about every
> single feature we should add and then creating something super complicated
> that has unclear value.
>
>
>
> On Wed, Aug 30, 2017 at 6:37 PM, Ryan Blue  wrote:
>
>> -1 (non-binding)
>>
>> Sometimes it takes a VOTE thread to get people to actually read and
>> comment, so thanks for starting this one… but there’s still discussion
>> happening on the prototype API, which it hasn’t been updated. I’d like to
>> see the proposal shaped by the ongoing discussion so that we have a better,
>> more concrete plan. I think that’s going to produces a better SPIP.
>>
>> The second reason for -1 is that I think the read- and write-side
>> proposals should be separated. The PR
>>  currently has “write path”
>> listed as a TODO item and most of the discussion I’ve seen is on the read
>> side. I think it would be better to separate the read and write APIs so we
>> can focus on them individually.
>>
>> An example of why we should focus on the write path separately is that
>> the proposal says this:
>>
>> Ideally partitioning/bucketing concept should not be exposed in the Data
>> Source API V2, because they are just techniques for data skipping and
>> pre-partitioning. However, these 2 concepts are already widely used in
>> Spark, e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION.
>> To be consistent, we need to add partitioning/bucketing to Data Source V2 .
>> . .
>>
>> Essentially, the some APIs mix DDL and DML operations. I’d like to
>> consider ways to fix that problem instead of carrying the problem forward
>> to Data Source V2. We can solve this by adding a high-level API for DDL and
>> a better write/insert API that works well with it. Clearly, that discussion
>> is independent of the read path, which is why I think separating the two
>> proposals would be a win.
>>
>> rb
>> ​
>>
>> On Wed, Aug 30, 2017 at 4:28 AM, Reynold Xin  wrote:
>>
>>> That might be good to do, but seems like orthogonal to this effort
>>> itself. It would be a completely different interface.
>>>
>>> On Wed, Aug 30, 2017 at 1:10 PM Wenchen Fan  wrote:
>>>
 OK I agree with it, how about we add a new interface to push down the
 query plan, based on the current framework? We can mark the
 query-plan-push-down interface as unstable, to save the effort of designing
 a stable representation of query plan and maintaining forward 
 compatibility.

 On Wed, Aug 30, 2017 at 10:53 AM, James Baker 
 wrote:

> I'll just focus on the one-by-one thing for now - it's the thing that
> blocks me the most.
>
> I think the place where we're most confused here is on the cost of
> determining whether I can push down a filter. For me, in order to work out
> whether I can push down a filter or satisfy a sort, I might have to read
> plenty of data. That said, it's worth me doing this because I can use this
> information to avoid reading >>that much data.
>
> If you give me all the orderings, I will have to read that data many
> times (we stream it to avoid keeping it in memory).
>
> There's also a thing where our typical use cases have many filters
> (20+ is common). So, it's likely not going to work to pass us all the
> combinations. That said, if I can tell you a cost, I know what optimal
> looks like, why can't I just pick that myself?
>
> The current design is friendly to simple datasources, but does not
> have the potential to support this.
>
> So the main problem we have with datasources v1 is that it's
> essentially impossible 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread James Baker
I guess I was more suggesting that by coding up the powerful mode as the API, 
it becomes easy for someone to layer an easy mode beneath it to enable simpler 
datasources to be integrated (and that simple mode should be the out of scope 
thing).

Taking a small step back here, one of the places where I think I'm missing some 
context is in understanding the target consumers of these interfaces. I've done 
some amount (though likely not enough) of research about the places where 
people have had issues of API surface in the past - the concrete tickets I've 
seen have been based on Cassandra integration where you want to indicate 
clustering, and SAP HANA where they want to push down more complicated queries 
through Spark. This proposal supports the former, but the amount of change 
required to support clustering in the current API is not obviously high - 
whilst the current proposal for V2 seems to make it very difficult to add 
support for pushing down plenty of aggregations in the future (I've found the 
question of how to add GROUP BY to be pretty tricky to answer for the current 
proposal).

Googling around for implementations of the current PrunedFilteredScan, I 
basically find a lot of databases, which seems reasonable - SAP HANA, 
ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people who've 
used (some of) these connectors and the sticking point has generally been that 
Spark needs to load a lot of data out in order to solve aggregations that can 
be very efficiently pushed down into the datasources.

So, with this proposal it appears that we're optimising towards making it easy 
to write one-off datasource integrations, with some amount of pluggability for 
people who want to do more complicated things (the most interesting being 
bucketing integration). However, my guess is that this isn't what the current 
major integrations suffer from; they suffer mostly from restrictions in what 
they can push down (which broadly speaking are not going to go away).

So the place where I'm confused is that the current integrations can be made 
incrementally better as a consequence of this, but the backing data systems 
have the features which enable a step change which this API makes harder to 
achieve in the future. Who are the group of users who benefit the most as a 
consequence of this change, like, who is the target consumer here? My personal 
slant is that it's more important to improve support for other datastores than 
it is to lower the barrier of entry - this is why I've been pushing here.

James

On Wed, 30 Aug 2017 at 09:37 Ryan Blue 
> wrote:

-1 (non-binding)

Sometimes it takes a VOTE thread to get people to actually read and comment, so 
thanks for starting this one… but there’s still discussion happening on the 
prototype API, which it hasn’t been updated. I’d like to see the proposal 
shaped by the ongoing discussion so that we have a better, more concrete plan. 
I think that’s going to produces a better SPIP.

The second reason for -1 is that I think the read- and write-side proposals 
should be separated. The PR 
currently has “write path” listed as a TODO item and most of the discussion 
I’ve seen is on the read side. I think it would be better to separate the read 
and write APIs so we can focus on them individually.

An example of why we should focus on the write path separately is that the 
proposal says this:

Ideally partitioning/bucketing concept should not be exposed in the Data Source 
API V2, because they are just techniques for data skipping and 
pre-partitioning. However, these 2 concepts are already widely used in Spark, 
e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION. To be 
consistent, we need to add partitioning/bucketing to Data Source V2 . . .

Essentially, the some APIs mix DDL and DML operations. I’d like to consider 
ways to fix that problem instead of carrying the problem forward to Data Source 
V2. We can solve this by adding a high-level API for DDL and a better 
write/insert API that works well with it. Clearly, that discussion is 
independent of the read path, which is why I think separating the two proposals 
would be a win.

rb

​

On Wed, Aug 30, 2017 at 4:28 AM, Reynold Xin 
> wrote:
That might be good to do, but seems like orthogonal to this effort itself. It 
would be a completely different interface.

On Wed, Aug 30, 2017 at 1:10 PM Wenchen Fan 
> wrote:
OK I agree with it, how about we add a new interface to push down the query 
plan, based on the current framework? We can mark the query-plan-push-down 
interface as unstable, to save the effort of designing a stable representation 
of query plan and maintaining forward compatibility.

On Wed, Aug 30, 2017 at 10:53 AM, James Baker 
> wrote:

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread James Baker
I guess I was more suggesting that by coding up the powerful mode as the API, 
it becomes easy for someone to layer an easy mode beneath it to enable simpler 
datasources to be integrated (and that simple mode should be the out of scope 
thing).

Taking a small step back here, one of the places where I think I'm missing some 
context is in understanding the target consumers of these interfaces. I've done 
some amount (though likely not enough) of research about the places where 
people have had issues of API surface in the past - the concrete tickets I've 
seen have been based on Cassandra integration where you want to indicate 
clustering, and SAP HANA where they want to push down more complicated queries 
through Spark. This proposal supports the former, but the amount of change 
required to support clustering in the current API is not obviously high - 
whilst the current proposal for V2 seems to make it very difficult to add 
support for pushing down plenty of aggregations in the future (I've found the 
question of how to add GROUP BY to be pretty tricky to answer for the current 
proposal).

Googling around for implementations of the current PrunedFilteredScan, I 
basically find a lot of databases, which seems reasonable - SAP HANA, 
ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people who've 
used (some of) these connectors and the sticking point has generally been that 
Spark needs to load a lot of data out in order to solve aggregations that can 
be very efficiently pushed down into the datasources.

So, with this proposal it appears that we're optimising towards making it easy 
to write one-off datasource integrations, with some amount of pluggability for 
people who want to do more complicated things (the most interesting being 
bucketing integration). However, my guess is that this isn't what the current 
major integrations suffer from; they suffer mostly from restrictions in what 
they can push down (which broadly speaking are not going to go away).

So the place where I'm confused is that the current integrations can be made 
incrementally better as a consequence of this, but the backing data systems 
have the features which enable a step change which this API makes harder to 
achieve in the future. Who are the group of users who benefit the most as a 
consequence of this change, like, who is the target consumer here? My personal 
slant is that it's more important to improve support for other datastores than 
it is to lower the barrier of entry - this is why I've been pushing here.

James

On Wed, 30 Aug 2017 at 09:37 Ryan Blue 
> wrote:

-1 (non-binding)

Sometimes it takes a VOTE thread to get people to actually read and comment, so 
thanks for starting this one… but there’s still discussion happening on the 
prototype API, which it hasn’t been updated. I’d like to see the proposal 
shaped by the ongoing discussion so that we have a better, more concrete plan. 
I think that’s going to produces a better SPIP.

The second reason for -1 is that I think the read- and write-side proposals 
should be separated. The PR 
currently has “write path” listed as a TODO item and most of the discussion 
I’ve seen is on the read side. I think it would be better to separate the read 
and write APIs so we can focus on them individually.

An example of why we should focus on the write path separately is that the 
proposal says this:

Ideally partitioning/bucketing concept should not be exposed in the Data Source 
API V2, because they are just techniques for data skipping and 
pre-partitioning. However, these 2 concepts are already widely used in Spark, 
e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION. To be 
consistent, we need to add partitioning/bucketing to Data Source V2 . . .

Essentially, the some APIs mix DDL and DML operations. I’d like to consider 
ways to fix that problem instead of carrying the problem forward to Data Source 
V2. We can solve this by adding a high-level API for DDL and a better 
write/insert API that works well with it. Clearly, that discussion is 
independent of the read path, which is why I think separating the two proposals 
would be a win.

rb

​

On Wed, Aug 30, 2017 at 4:28 AM, Reynold Xin 
> wrote:
That might be good to do, but seems like orthogonal to this effort itself. It 
would be a completely different interface.

On Wed, Aug 30, 2017 at 1:10 PM Wenchen Fan 
> wrote:
OK I agree with it, how about we add a new interface to push down the query 
plan, based on the current framework? We can mark the query-plan-push-down 
interface as unstable, to save the effort of designing a stable representation 
of query plan and maintaining forward compatibility.

On Wed, Aug 30, 2017 at 10:53 AM, James Baker 
> wrote:

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread James Baker
I guess I was more suggesting that by coding up the powerful mode as the API, 
it becomes easy for someone to layer an easy mode beneath it to enable simpler 
datasources to be integrated (and that simple mode should be the out of scope 
thing).

Taking a small step back here, one of the places where I think I'm missing some 
context is in understanding the target consumers of these interfaces. I've done 
some amount (though likely not enough) of research about the places where 
people have had issues of API surface in the past - the concrete tickets I've 
seen have been based on Cassandra integration where you want to indicate 
clustering, and SAP HANA where they want to push down more complicated queries 
through Spark. This proposal supports the former, but the amount of change 
required to support clustering in the current API is not obviously high - 
whilst the current proposal for V2 seems to make it very difficult to add 
support for pushing down plenty of aggregations in the future (I've found the 
question of how to add GROUP BY to be pretty tricky to answer for the current 
proposal).

Googling around for implementations of the current PrunedFilteredScan, I 
basically find a lot of databases, which seems reasonable - SAP HANA, 
ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people who've 
used (some of) these connectors and the sticking point has generally been that 
Spark needs to load a lot of data out in order to solve aggregations that can 
be very efficiently pushed down into the datasources.

So, with this proposal it appears that we're optimising towards making it easy 
to write one-off datasource integrations, with some amount of pluggability for 
people who want to do more complicated things (the most interesting being 
bucketing integration). However, my guess is that this isn't what the current 
major integrations suffer from; they suffer mostly from restrictions in what 
they can push down (which broadly speaking are not going to go away).

So the place where I'm confused is that the current integrations can be made 
incrementally better as a consequence of this, but the backing data systems 
have the features which enable a step change which this API makes harder to 
achieve in the future. Who are the group of users who benefit the most as a 
consequence of this change, like, who is the target consumer here? My personal 
slant is that it's more important to improve support for other datastores than 
it is to lower the barrier of entry - this is why I've been pushing here.

James

On Wed, 30 Aug 2017 at 09:37 Ryan Blue 
> wrote:

-1 (non-binding)

Sometimes it takes a VOTE thread to get people to actually read and comment, so 
thanks for starting this one… but there’s still discussion happening on the 
prototype API, which it hasn’t been updated. I’d like to see the proposal 
shaped by the ongoing discussion so that we have a better, more concrete plan. 
I think that’s going to produces a better SPIP.

The second reason for -1 is that I think the read- and write-side proposals 
should be separated. The PR 
currently has “write path” listed as a TODO item and most of the discussion 
I’ve seen is on the read side. I think it would be better to separate the read 
and write APIs so we can focus on them individually.

An example of why we should focus on the write path separately is that the 
proposal says this:

Ideally partitioning/bucketing concept should not be exposed in the Data Source 
API V2, because they are just techniques for data skipping and 
pre-partitioning. However, these 2 concepts are already widely used in Spark, 
e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION. To be 
consistent, we need to add partitioning/bucketing to Data Source V2 . . .

Essentially, the some APIs mix DDL and DML operations. I’d like to consider 
ways to fix that problem instead of carrying the problem forward to Data Source 
V2. We can solve this by adding a high-level API for DDL and a better 
write/insert API that works well with it. Clearly, that discussion is 
independent of the read path, which is why I think separating the two proposals 
would be a win.

rb

​

On Wed, Aug 30, 2017 at 4:28 AM, Reynold Xin 
> wrote:
That might be good to do, but seems like orthogonal to this effort itself. It 
would be a completely different interface.

On Wed, Aug 30, 2017 at 1:10 PM Wenchen Fan 
> wrote:
OK I agree with it, how about we add a new interface to push down the query 
plan, based on the current framework? We can mark the query-plan-push-down 
interface as unstable, to save the effort of designing a stable representation 
of query plan and maintaining forward compatibility.

On Wed, Aug 30, 2017 at 10:53 AM, James Baker 
> wrote:

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread Reynold Xin
Sure that's good to do (and as discussed earlier a good compromise might be
to expose an interface for the source to decide which part of the logical
plan they want to accept).

To me everything is about cost vs benefit.

In my mind, the biggest issue with the existing data source API is backward
and forward compatibility. All the data sources written for Spark 1.x broke
in Spark 2.x. And that's one of the biggest value v2 can bring. To me it's
far more important to have data sources implemented in 2017 to be able to
work in 2027, in Spark 10.x.

You are basically arguing for creating a new API that is capable of doing
arbitrary expression, aggregation, and join pushdowns (you only mentioned
aggregation so far, but I've talked to enough database people that I know
once Spark gives them aggregation pushdown, they will come back for join
pushdown). We can do that using unstable APIs, and creating stable APIs
would be extremely difficult (still doable, just would take a long time to
design and implement). As mentioned earlier, it basically involves creating
a stable representation for all of logical plan, which is a lot of work. I
think we should still work towards that (for other reasons as well), but
I'd consider that out of scope for the current one. Otherwise we'd not
release something probably for the next 2 or 3 years.





On Wed, Aug 30, 2017 at 11:50 PM, James Baker  wrote:

> I guess I was more suggesting that by coding up the powerful mode as the
> API, it becomes easy for someone to layer an easy mode beneath it to enable
> simpler datasources to be integrated (and that simple mode should be the
> out of scope thing).
>
> Taking a small step back here, one of the places where I think I'm missing
> some context is in understanding the target consumers of these interfaces.
> I've done some amount (though likely not enough) of research about the
> places where people have had issues of API surface in the past - the
> concrete tickets I've seen have been based on Cassandra integration where
> you want to indicate clustering, and SAP HANA where they want to push down
> more complicated queries through Spark. This proposal supports the former,
> but the amount of change required to support clustering in the current API
> is not obviously high - whilst the current proposal for V2 seems to make it
> very difficult to add support for pushing down plenty of aggregations in
> the future (I've found the question of how to add GROUP BY to be pretty
> tricky to answer for the current proposal).
>
> Googling around for implementations of the current PrunedFilteredScan, I
> basically find a lot of databases, which seems reasonable - SAP HANA,
> ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people
> who've used (some of) these connectors and the sticking point has generally
> been that Spark needs to load a lot of data out in order to solve
> aggregations that can be very efficiently pushed down into the datasources.
>
> So, with this proposal it appears that we're optimising towards making it
> easy to write one-off datasource integrations, with some amount of
> pluggability for people who want to do more complicated things (the most
> interesting being bucketing integration). However, my guess is that this
> isn't what the current major integrations suffer from; they suffer mostly
> from restrictions in what they can push down (which broadly speaking are
> not going to go away).
>
> So the place where I'm confused is that the current integrations can be
> made incrementally better as a consequence of this, but the backing data
> systems have the features which enable a step change which this API makes
> harder to achieve in the future. Who are the group of users who benefit the
> most as a consequence of this change, like, who is the target consumer
> here? My personal slant is that it's more important to improve support for
> other datastores than it is to lower the barrier of entry - this is why
> I've been pushing here.
>
> James
>
> On Wed, 30 Aug 2017 at 09:37 Ryan Blue  wrote:
>
>> -1 (non-binding)
>>
>> Sometimes it takes a VOTE thread to get people to actually read and
>> comment, so thanks for starting this one… but there’s still discussion
>> happening on the prototype API, which it hasn’t been updated. I’d like to
>> see the proposal shaped by the ongoing discussion so that we have a better,
>> more concrete plan. I think that’s going to produces a better SPIP.
>>
>> The second reason for -1 is that I think the read- and write-side
>> proposals should be separated. The PR
>>  currently has “write path”
>> listed as a TODO item and most of the discussion I’ve seen is on the read
>> side. I think it would be better to separate the read and write APIs so we
>> can focus on them individually.
>>
>> An example of why we should focus on the write path separately is that
>> the proposal says this:
>>
>> 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread James Baker
I guess I was more suggesting that by coding up the powerful mode as the API, 
it becomes easy for someone to layer an easy mode beneath it to enable simpler 
datasources to be integrated (and that simple mode should be the out of scope 
thing).

Taking a small step back here, one of the places where I think I'm missing some 
context is in understanding the target consumers of these interfaces. I've done 
some amount (though likely not enough) of research about the places where 
people have had issues of API surface in the past - the concrete tickets I've 
seen have been based on Cassandra integration where you want to indicate 
clustering, and SAP HANA where they want to push down more complicated queries 
through Spark. This proposal supports the former, but the amount of change 
required to support clustering in the current API is not obviously high - 
whilst the current proposal for V2 seems to make it very difficult to add 
support for pushing down plenty of aggregations in the future (I've found the 
question of how to add GROUP BY to be pretty tricky to answer for the current 
proposal).

Googling around for implementations of the current PrunedFilteredScan, I 
basically find a lot of databases, which seems reasonable - SAP HANA, 
ElasticSearch, Solr, MongoDB, Apache Phoenix, etc. I've talked to people who've 
used (some of) these connectors and the sticking point has generally been that 
Spark needs to load a lot of data out in order to solve aggregations that can 
be very efficiently pushed down into the datasources.

So, with this proposal it appears that we're optimising towards making it easy 
to write one-off datasource integrations, with some amount of pluggability for 
people who want to do more complicated things (the most interesting being 
bucketing integration). However, my guess is that this isn't what the current 
major integrations suffer from; they suffer mostly from restrictions in what 
they can push down (which broadly speaking are not going to go away).

So the place where I'm confused is that the current integrations can be made 
incrementally better as a consequence of this, but the backing data systems 
have the features which enable a step change which this API makes harder to 
achieve in the future. Who are the group of users who benefit the most as a 
consequence of this change, like, who is the target consumer here? My personal 
slant is that it's more important to improve support for other datastores than 
it is to lower the barrier of entry - this is why I've been pushing here.

James

On Wed, 30 Aug 2017 at 09:37 Ryan Blue 
> wrote:

-1 (non-binding)

Sometimes it takes a VOTE thread to get people to actually read and comment, so 
thanks for starting this one… but there’s still discussion happening on the 
prototype API, which it hasn’t been updated. I’d like to see the proposal 
shaped by the ongoing discussion so that we have a better, more concrete plan. 
I think that’s going to produces a better SPIP.

The second reason for -1 is that I think the read- and write-side proposals 
should be separated. The PR 
currently has “write path” listed as a TODO item and most of the discussion 
I’ve seen is on the read side. I think it would be better to separate the read 
and write APIs so we can focus on them individually.

An example of why we should focus on the write path separately is that the 
proposal says this:

Ideally partitioning/bucketing concept should not be exposed in the Data Source 
API V2, because they are just techniques for data skipping and 
pre-partitioning. However, these 2 concepts are already widely used in Spark, 
e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION. To be 
consistent, we need to add partitioning/bucketing to Data Source V2 . . .

Essentially, the some APIs mix DDL and DML operations. I’d like to consider 
ways to fix that problem instead of carrying the problem forward to Data Source 
V2. We can solve this by adding a high-level API for DDL and a better 
write/insert API that works well with it. Clearly, that discussion is 
independent of the read path, which is why I think separating the two proposals 
would be a win.

rb

​

On Wed, Aug 30, 2017 at 4:28 AM, Reynold Xin 
> wrote:
That might be good to do, but seems like orthogonal to this effort itself. It 
would be a completely different interface.

On Wed, Aug 30, 2017 at 1:10 PM Wenchen Fan 
> wrote:
OK I agree with it, how about we add a new interface to push down the query 
plan, based on the current framework? We can mark the query-plan-push-down 
interface as unstable, to save the effort of designing a stable representation 
of query plan and maintaining forward compatibility.

On Wed, Aug 30, 2017 at 10:53 AM, James Baker 
> wrote:

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread Reynold Xin
So we seem to be getting into a cycle of discussing more about the details
of APIs than the high level proposal. The details of APIs are important to
debate, but those belong more in code reviews.

One other important thing is that we should avoid API design by committee.
While it is extremely useful to get feedback, understand the use cases, we
cannot do API design by incorporating verbatim the union of everybody's
feedback. API design is largely a tradeoff game. The most expressive API
would also be harder to use, or sacrifice backward/forward compatibility.
It is as important to decide what to exclude as what to include.

Unlike the v1 API, the way Wenchen's high level V2 framework is proposed
makes it very easy to add new features (e.g. clustering properties) in the
future without breaking any APIs. I'd rather us shipping something useful
that might not be the most comprehensive set, than debating about every
single feature we should add and then creating something super complicated
that has unclear value.



On Wed, Aug 30, 2017 at 6:37 PM, Ryan Blue  wrote:

> -1 (non-binding)
>
> Sometimes it takes a VOTE thread to get people to actually read and
> comment, so thanks for starting this one… but there’s still discussion
> happening on the prototype API, which it hasn’t been updated. I’d like to
> see the proposal shaped by the ongoing discussion so that we have a better,
> more concrete plan. I think that’s going to produces a better SPIP.
>
> The second reason for -1 is that I think the read- and write-side
> proposals should be separated. The PR
>  currently has “write path”
> listed as a TODO item and most of the discussion I’ve seen is on the read
> side. I think it would be better to separate the read and write APIs so we
> can focus on them individually.
>
> An example of why we should focus on the write path separately is that the
> proposal says this:
>
> Ideally partitioning/bucketing concept should not be exposed in the Data
> Source API V2, because they are just techniques for data skipping and
> pre-partitioning. However, these 2 concepts are already widely used in
> Spark, e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION.
> To be consistent, we need to add partitioning/bucketing to Data Source V2 .
> . .
>
> Essentially, the some APIs mix DDL and DML operations. I’d like to
> consider ways to fix that problem instead of carrying the problem forward
> to Data Source V2. We can solve this by adding a high-level API for DDL and
> a better write/insert API that works well with it. Clearly, that discussion
> is independent of the read path, which is why I think separating the two
> proposals would be a win.
>
> rb
> ​
>
> On Wed, Aug 30, 2017 at 4:28 AM, Reynold Xin  wrote:
>
>> That might be good to do, but seems like orthogonal to this effort
>> itself. It would be a completely different interface.
>>
>> On Wed, Aug 30, 2017 at 1:10 PM Wenchen Fan  wrote:
>>
>>> OK I agree with it, how about we add a new interface to push down the
>>> query plan, based on the current framework? We can mark the
>>> query-plan-push-down interface as unstable, to save the effort of designing
>>> a stable representation of query plan and maintaining forward compatibility.
>>>
>>> On Wed, Aug 30, 2017 at 10:53 AM, James Baker 
>>> wrote:
>>>
 I'll just focus on the one-by-one thing for now - it's the thing that
 blocks me the most.

 I think the place where we're most confused here is on the cost of
 determining whether I can push down a filter. For me, in order to work out
 whether I can push down a filter or satisfy a sort, I might have to read
 plenty of data. That said, it's worth me doing this because I can use this
 information to avoid reading >>that much data.

 If you give me all the orderings, I will have to read that data many
 times (we stream it to avoid keeping it in memory).

 There's also a thing where our typical use cases have many filters (20+
 is common). So, it's likely not going to work to pass us all the
 combinations. That said, if I can tell you a cost, I know what optimal
 looks like, why can't I just pick that myself?

 The current design is friendly to simple datasources, but does not have
 the potential to support this.

 So the main problem we have with datasources v1 is that it's
 essentially impossible to leverage a bunch of Spark features - I don't get
 to use bucketing or row batches or all the nice things that I really want
 to use to get decent performance. Provided I can leverage these in a
 moderately supported way which won't break in any given commit, I'll be
 pretty happy with anything that lets me opt out of the restrictions.

 My suggestion here is that if you make a mode which works well for
 complicated use 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread Ryan Blue
-1 (non-binding)

Sometimes it takes a VOTE thread to get people to actually read and
comment, so thanks for starting this one… but there’s still discussion
happening on the prototype API, which it hasn’t been updated. I’d like to
see the proposal shaped by the ongoing discussion so that we have a better,
more concrete plan. I think that’s going to produces a better SPIP.

The second reason for -1 is that I think the read- and write-side proposals
should be separated. The PR 
currently has “write path” listed as a TODO item and most of the discussion
I’ve seen is on the read side. I think it would be better to separate the
read and write APIs so we can focus on them individually.

An example of why we should focus on the write path separately is that the
proposal says this:

Ideally partitioning/bucketing concept should not be exposed in the Data
Source API V2, because they are just techniques for data skipping and
pre-partitioning. However, these 2 concepts are already widely used in
Spark, e.g. DataFrameWriter.partitionBy and DDL syntax like ADD PARTITION.
To be consistent, we need to add partitioning/bucketing to Data Source V2 .
. .

Essentially, the some APIs mix DDL and DML operations. I’d like to consider
ways to fix that problem instead of carrying the problem forward to Data
Source V2. We can solve this by adding a high-level API for DDL and a
better write/insert API that works well with it. Clearly, that discussion
is independent of the read path, which is why I think separating the two
proposals would be a win.

rb
​

On Wed, Aug 30, 2017 at 4:28 AM, Reynold Xin  wrote:

> That might be good to do, but seems like orthogonal to this effort itself.
> It would be a completely different interface.
>
> On Wed, Aug 30, 2017 at 1:10 PM Wenchen Fan  wrote:
>
>> OK I agree with it, how about we add a new interface to push down the
>> query plan, based on the current framework? We can mark the
>> query-plan-push-down interface as unstable, to save the effort of designing
>> a stable representation of query plan and maintaining forward compatibility.
>>
>> On Wed, Aug 30, 2017 at 10:53 AM, James Baker 
>> wrote:
>>
>>> I'll just focus on the one-by-one thing for now - it's the thing that
>>> blocks me the most.
>>>
>>> I think the place where we're most confused here is on the cost of
>>> determining whether I can push down a filter. For me, in order to work out
>>> whether I can push down a filter or satisfy a sort, I might have to read
>>> plenty of data. That said, it's worth me doing this because I can use this
>>> information to avoid reading >>that much data.
>>>
>>> If you give me all the orderings, I will have to read that data many
>>> times (we stream it to avoid keeping it in memory).
>>>
>>> There's also a thing where our typical use cases have many filters (20+
>>> is common). So, it's likely not going to work to pass us all the
>>> combinations. That said, if I can tell you a cost, I know what optimal
>>> looks like, why can't I just pick that myself?
>>>
>>> The current design is friendly to simple datasources, but does not have
>>> the potential to support this.
>>>
>>> So the main problem we have with datasources v1 is that it's essentially
>>> impossible to leverage a bunch of Spark features - I don't get to use
>>> bucketing or row batches or all the nice things that I really want to use
>>> to get decent performance. Provided I can leverage these in a moderately
>>> supported way which won't break in any given commit, I'll be pretty happy
>>> with anything that lets me opt out of the restrictions.
>>>
>>> My suggestion here is that if you make a mode which works well for
>>> complicated use cases, you end up being able to write simple mode in terms
>>> of it very easily. So we could actually provide two APIs, one that lets
>>> people who have more interesting datasources leverage the cool Spark
>>> features, and one that lets people who just want to implement basic
>>> features do that - I'd try to include some kind of layering here. I could
>>> probably sketch out something here if that'd be useful?
>>>
>>> James
>>>
>>> On Tue, 29 Aug 2017 at 18:59 Wenchen Fan  wrote:
>>>
 Hi James,

 Thanks for your feedback! I think your concerns are all valid, but we
 need to make a tradeoff here.

 > Explicitly here, what I'm looking for is a convenient mechanism to
 accept a fully specified set of arguments

 The problem with this approach is: 1) if we wanna add more arguments in
 the future, it's really hard to do without changing the existing interface.
 2) if a user wants to implement a very simple data source, he has to look
 at all the arguments and understand them, which may be a burden for him.
 I don't have a solution to solve these 2 problems, comments are welcome.


 > There are loads of cases 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread Reynold Xin
That might be good to do, but seems like orthogonal to this effort itself.
It would be a completely different interface.

On Wed, Aug 30, 2017 at 1:10 PM Wenchen Fan  wrote:

> OK I agree with it, how about we add a new interface to push down the
> query plan, based on the current framework? We can mark the
> query-plan-push-down interface as unstable, to save the effort of designing
> a stable representation of query plan and maintaining forward compatibility.
>
> On Wed, Aug 30, 2017 at 10:53 AM, James Baker  wrote:
>
>> I'll just focus on the one-by-one thing for now - it's the thing that
>> blocks me the most.
>>
>> I think the place where we're most confused here is on the cost of
>> determining whether I can push down a filter. For me, in order to work out
>> whether I can push down a filter or satisfy a sort, I might have to read
>> plenty of data. That said, it's worth me doing this because I can use this
>> information to avoid reading >>that much data.
>>
>> If you give me all the orderings, I will have to read that data many
>> times (we stream it to avoid keeping it in memory).
>>
>> There's also a thing where our typical use cases have many filters (20+
>> is common). So, it's likely not going to work to pass us all the
>> combinations. That said, if I can tell you a cost, I know what optimal
>> looks like, why can't I just pick that myself?
>>
>> The current design is friendly to simple datasources, but does not have
>> the potential to support this.
>>
>> So the main problem we have with datasources v1 is that it's essentially
>> impossible to leverage a bunch of Spark features - I don't get to use
>> bucketing or row batches or all the nice things that I really want to use
>> to get decent performance. Provided I can leverage these in a moderately
>> supported way which won't break in any given commit, I'll be pretty happy
>> with anything that lets me opt out of the restrictions.
>>
>> My suggestion here is that if you make a mode which works well for
>> complicated use cases, you end up being able to write simple mode in terms
>> of it very easily. So we could actually provide two APIs, one that lets
>> people who have more interesting datasources leverage the cool Spark
>> features, and one that lets people who just want to implement basic
>> features do that - I'd try to include some kind of layering here. I could
>> probably sketch out something here if that'd be useful?
>>
>> James
>>
>> On Tue, 29 Aug 2017 at 18:59 Wenchen Fan  wrote:
>>
>>> Hi James,
>>>
>>> Thanks for your feedback! I think your concerns are all valid, but we
>>> need to make a tradeoff here.
>>>
>>> > Explicitly here, what I'm looking for is a convenient mechanism to
>>> accept a fully specified set of arguments
>>>
>>> The problem with this approach is: 1) if we wanna add more arguments in
>>> the future, it's really hard to do without changing the existing interface.
>>> 2) if a user wants to implement a very simple data source, he has to look
>>> at all the arguments and understand them, which may be a burden for him.
>>> I don't have a solution to solve these 2 problems, comments are welcome.
>>>
>>>
>>> > There are loads of cases like this - you can imagine someone being
>>> able to push down a sort before a filter is applied, but not afterwards.
>>> However, maybe the filter is so selective that it's better to push down the
>>> filter and not handle the sort. I don't get to make this decision, Spark
>>> does (but doesn't have good enough information to do it properly, whilst I
>>> do). I want to be able to choose the parts I push down given knowledge of
>>> my datasource - as defined the APIs don't let me do that, they're strictly
>>> more restrictive than the V1 APIs in this way.
>>>
>>> This is true, the current framework applies push downs one by one,
>>> incrementally. If a data source wanna go back to accept a sort push down
>>> after it accepts a filter push down, it's impossible with the current data
>>> source V2.
>>> Fortunately, we have a solution for this problem. At Spark side,
>>> actually we do have a fully specified set of arguments waiting to be
>>> pushed down, but Spark doesn't know which is the best order to push them
>>> into data source. Spark can try every combination and ask the data source
>>> to report a cost, then Spark can pick the best combination with the lowest
>>> cost. This can also be implemented as a cost report interface, so that
>>> advanced data source can implement it for optimal performance, and simple
>>> data source doesn't need to care about it and keep simple.
>>>
>>>
>>> The current design is very friendly to simple data source, and has the
>>> potential to support complex data source, I prefer the current design over
>>> the plan push down one. What do you think?
>>>
>>>
>>> On Wed, Aug 30, 2017 at 5:53 AM, James Baker 
>>> wrote:
>>>
 Yeah, for sure.

 With the stable 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-30 Thread Wenchen Fan
OK I agree with it, how about we add a new interface to push down the query
plan, based on the current framework? We can mark the query-plan-push-down
interface as unstable, to save the effort of designing a stable
representation of query plan and maintaining forward compatibility.

On Wed, Aug 30, 2017 at 10:53 AM, James Baker  wrote:

> I'll just focus on the one-by-one thing for now - it's the thing that
> blocks me the most.
>
> I think the place where we're most confused here is on the cost of
> determining whether I can push down a filter. For me, in order to work out
> whether I can push down a filter or satisfy a sort, I might have to read
> plenty of data. That said, it's worth me doing this because I can use this
> information to avoid reading >>that much data.
>
> If you give me all the orderings, I will have to read that data many times
> (we stream it to avoid keeping it in memory).
>
> There's also a thing where our typical use cases have many filters (20+ is
> common). So, it's likely not going to work to pass us all the combinations.
> That said, if I can tell you a cost, I know what optimal looks like, why
> can't I just pick that myself?
>
> The current design is friendly to simple datasources, but does not have
> the potential to support this.
>
> So the main problem we have with datasources v1 is that it's essentially
> impossible to leverage a bunch of Spark features - I don't get to use
> bucketing or row batches or all the nice things that I really want to use
> to get decent performance. Provided I can leverage these in a moderately
> supported way which won't break in any given commit, I'll be pretty happy
> with anything that lets me opt out of the restrictions.
>
> My suggestion here is that if you make a mode which works well for
> complicated use cases, you end up being able to write simple mode in terms
> of it very easily. So we could actually provide two APIs, one that lets
> people who have more interesting datasources leverage the cool Spark
> features, and one that lets people who just want to implement basic
> features do that - I'd try to include some kind of layering here. I could
> probably sketch out something here if that'd be useful?
>
> James
>
> On Tue, 29 Aug 2017 at 18:59 Wenchen Fan  wrote:
>
>> Hi James,
>>
>> Thanks for your feedback! I think your concerns are all valid, but we
>> need to make a tradeoff here.
>>
>> > Explicitly here, what I'm looking for is a convenient mechanism to
>> accept a fully specified set of arguments
>>
>> The problem with this approach is: 1) if we wanna add more arguments in
>> the future, it's really hard to do without changing the existing interface.
>> 2) if a user wants to implement a very simple data source, he has to look
>> at all the arguments and understand them, which may be a burden for him.
>> I don't have a solution to solve these 2 problems, comments are welcome.
>>
>>
>> > There are loads of cases like this - you can imagine someone being
>> able to push down a sort before a filter is applied, but not afterwards.
>> However, maybe the filter is so selective that it's better to push down the
>> filter and not handle the sort. I don't get to make this decision, Spark
>> does (but doesn't have good enough information to do it properly, whilst I
>> do). I want to be able to choose the parts I push down given knowledge of
>> my datasource - as defined the APIs don't let me do that, they're strictly
>> more restrictive than the V1 APIs in this way.
>>
>> This is true, the current framework applies push downs one by one,
>> incrementally. If a data source wanna go back to accept a sort push down
>> after it accepts a filter push down, it's impossible with the current data
>> source V2.
>> Fortunately, we have a solution for this problem. At Spark side, actually
>> we do have a fully specified set of arguments waiting to be pushed down,
>> but Spark doesn't know which is the best order to push them into data
>> source. Spark can try every combination and ask the data source to report a
>> cost, then Spark can pick the best combination with the lowest cost. This
>> can also be implemented as a cost report interface, so that advanced data
>> source can implement it for optimal performance, and simple data source
>> doesn't need to care about it and keep simple.
>>
>>
>> The current design is very friendly to simple data source, and has the
>> potential to support complex data source, I prefer the current design over
>> the plan push down one. What do you think?
>>
>>
>> On Wed, Aug 30, 2017 at 5:53 AM, James Baker  wrote:
>>
>>> Yeah, for sure.
>>>
>>> With the stable representation - agree that in the general case this is
>>> pretty intractable, it restricts the modifications that you can do in the
>>> future too much. That said, it shouldn't be as hard if you restrict
>>> yourself to the parts of the plan which are supported by the datasources V2
>>> API 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-29 Thread James Baker
I'll just focus on the one-by-one thing for now - it's the thing that blocks me 
the most.

I think the place where we're most confused here is on the cost of determining 
whether I can push down a filter. For me, in order to work out whether I can 
push down a filter or satisfy a sort, I might have to read plenty of data. That 
said, it's worth me doing this because I can use this information to avoid 
reading >>that much data.

If you give me all the orderings, I will have to read that data many times (we 
stream it to avoid keeping it in memory).

There's also a thing where our typical use cases have many filters (20+ is 
common). So, it's likely not going to work to pass us all the combinations. 
That said, if I can tell you a cost, I know what optimal looks like, why can't 
I just pick that myself?

The current design is friendly to simple datasources, but does not have the 
potential to support this.

So the main problem we have with datasources v1 is that it's essentially 
impossible to leverage a bunch of Spark features - I don't get to use bucketing 
or row batches or all the nice things that I really want to use to get decent 
performance. Provided I can leverage these in a moderately supported way which 
won't break in any given commit, I'll be pretty happy with anything that lets 
me opt out of the restrictions.

My suggestion here is that if you make a mode which works well for complicated 
use cases, you end up being able to write simple mode in terms of it very 
easily. So we could actually provide two APIs, one that lets people who have 
more interesting datasources leverage the cool Spark features, and one that 
lets people who just want to implement basic features do that - I'd try to 
include some kind of layering here. I could probably sketch out something here 
if that'd be useful?

James

On Tue, 29 Aug 2017 at 18:59 Wenchen Fan 
> wrote:
Hi James,

Thanks for your feedback! I think your concerns are all valid, but we need to 
make a tradeoff here.

> Explicitly here, what I'm looking for is a convenient mechanism to accept a 
> fully specified set of arguments

The problem with this approach is: 1) if we wanna add more arguments in the 
future, it's really hard to do without changing the existing interface. 2) if a 
user wants to implement a very simple data source, he has to look at all the 
arguments and understand them, which may be a burden for him.
I don't have a solution to solve these 2 problems, comments are welcome.


> There are loads of cases like this - you can imagine someone being able to 
> push down a sort before a filter is applied, but not afterwards. However, 
> maybe the filter is so selective that it's better to push down the filter and 
> not handle the sort. I don't get to make this decision, Spark does (but 
> doesn't have good enough information to do it properly, whilst I do). I want 
> to be able to choose the parts I push down given knowledge of my datasource - 
> as defined the APIs don't let me do that, they're strictly more restrictive 
> than the V1 APIs in this way.

This is true, the current framework applies push downs one by one, 
incrementally. If a data source wanna go back to accept a sort push down after 
it accepts a filter push down, it's impossible with the current data source V2.
Fortunately, we have a solution for this problem. At Spark side, actually we do 
have a fully specified set of arguments waiting to be pushed down, but Spark 
doesn't know which is the best order to push them into data source. Spark can 
try every combination and ask the data source to report a cost, then Spark can 
pick the best combination with the lowest cost. This can also be implemented as 
a cost report interface, so that advanced data source can implement it for 
optimal performance, and simple data source doesn't need to care about it and 
keep simple.


The current design is very friendly to simple data source, and has the 
potential to support complex data source, I prefer the current design over the 
plan push down one. What do you think?


On Wed, Aug 30, 2017 at 5:53 AM, James Baker 
> wrote:
Yeah, for sure.

With the stable representation - agree that in the general case this is pretty 
intractable, it restricts the modifications that you can do in the future too 
much. That said, it shouldn't be as hard if you restrict yourself to the parts 
of the plan which are supported by the datasources V2 API (which after all, 
need to be translateable properly into the future to support the mixins 
proposed). This should have a pretty small scope in comparison. As long as the 
user can bail out of nodes they don't understand, they should be ok, right?

That said, what would also be fine for us is a place to plug into an unstable 
query plan.

Explicitly here, what I'm looking for is a convenient mechanism to accept a 
fully specified set of arguments (of which I 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-29 Thread James Baker
I'll just focus on the one-by-one thing for now - it's the thing that blocks me 
the most.

I think the place where we're most confused here is on the cost of determining 
whether I can push down a filter. For me, in order to work out whether I can 
push down a filter or satisfy a sort, I might have to read plenty of data. That 
said, it's worth me doing this because I can use this information to avoid 
reading >>that much data.

If you give me all the orderings, I will have to read that data many times (we 
stream it to avoid keeping it in memory).

There's also a thing where our typical use cases have many filters (20+ is 
common). So, it's likely not going to work to pass us all the combinations. 
That said, if I can tell you a cost, I know what optimal looks like, why can't 
I just pick that myself?

The current design is friendly to simple datasources, but does not have the 
potential to support this.

So the main problem we have with datasources v1 is that it's essentially 
impossible to leverage a bunch of Spark features - I don't get to use bucketing 
or row batches or all the nice things that I really want to use to get decent 
performance. Provided I can leverage these in a moderately supported way which 
won't break in any given commit, I'll be pretty happy with anything that lets 
me opt out of the restrictions.

My suggestion here is that if you make a mode which works well for complicated 
use cases, you end up being able to write simple mode in terms of it very 
easily. So we could actually provide two APIs, one that lets people who have 
more interesting datasources leverage the cool Spark features, and one that 
lets people who just want to implement basic features do that - I'd try to 
include some kind of layering here. I could probably sketch out something here 
if that'd be useful?

James

On Tue, 29 Aug 2017 at 18:59 Wenchen Fan 
> wrote:
Hi James,

Thanks for your feedback! I think your concerns are all valid, but we need to 
make a tradeoff here.

> Explicitly here, what I'm looking for is a convenient mechanism to accept a 
> fully specified set of arguments

The problem with this approach is: 1) if we wanna add more arguments in the 
future, it's really hard to do without changing the existing interface. 2) if a 
user wants to implement a very simple data source, he has to look at all the 
arguments and understand them, which may be a burden for him.
I don't have a solution to solve these 2 problems, comments are welcome.


> There are loads of cases like this - you can imagine someone being able to 
> push down a sort before a filter is applied, but not afterwards. However, 
> maybe the filter is so selective that it's better to push down the filter and 
> not handle the sort. I don't get to make this decision, Spark does (but 
> doesn't have good enough information to do it properly, whilst I do). I want 
> to be able to choose the parts I push down given knowledge of my datasource - 
> as defined the APIs don't let me do that, they're strictly more restrictive 
> than the V1 APIs in this way.

This is true, the current framework applies push downs one by one, 
incrementally. If a data source wanna go back to accept a sort push down after 
it accepts a filter push down, it's impossible with the current data source V2.
Fortunately, we have a solution for this problem. At Spark side, actually we do 
have a fully specified set of arguments waiting to be pushed down, but Spark 
doesn't know which is the best order to push them into data source. Spark can 
try every combination and ask the data source to report a cost, then Spark can 
pick the best combination with the lowest cost. This can also be implemented as 
a cost report interface, so that advanced data source can implement it for 
optimal performance, and simple data source doesn't need to care about it and 
keep simple.


The current design is very friendly to simple data source, and has the 
potential to support complex data source, I prefer the current design over the 
plan push down one. What do you think?


On Wed, Aug 30, 2017 at 5:53 AM, James Baker 
> wrote:
Yeah, for sure.

With the stable representation - agree that in the general case this is pretty 
intractable, it restricts the modifications that you can do in the future too 
much. That said, it shouldn't be as hard if you restrict yourself to the parts 
of the plan which are supported by the datasources V2 API (which after all, 
need to be translateable properly into the future to support the mixins 
proposed). This should have a pretty small scope in comparison. As long as the 
user can bail out of nodes they don't understand, they should be ok, right?

That said, what would also be fine for us is a place to plug into an unstable 
query plan.

Explicitly here, what I'm looking for is a convenient mechanism to accept a 
fully specified set of arguments (of which I 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-29 Thread Wenchen Fan
Hi James,

Thanks for your feedback! I think your concerns are all valid, but we need
to make a tradeoff here.

> Explicitly here, what I'm looking for is a convenient mechanism to accept
a fully specified set of arguments

The problem with this approach is: 1) if we wanna add more arguments in the
future, it's really hard to do without changing the existing interface. 2)
if a user wants to implement a very simple data source, he has to look at
all the arguments and understand them, which may be a burden for him.
I don't have a solution to solve these 2 problems, comments are welcome.


> There are loads of cases like this - you can imagine someone being able
to push down a sort before a filter is applied, but not afterwards.
However, maybe the filter is so selective that it's better to push down the
filter and not handle the sort. I don't get to make this decision, Spark
does (but doesn't have good enough information to do it properly, whilst I
do). I want to be able to choose the parts I push down given knowledge of
my datasource - as defined the APIs don't let me do that, they're strictly
more restrictive than the V1 APIs in this way.

This is true, the current framework applies push downs one by one,
incrementally. If a data source wanna go back to accept a sort push down
after it accepts a filter push down, it's impossible with the current data
source V2.
Fortunately, we have a solution for this problem. At Spark side, actually
we do have a fully specified set of arguments waiting to be pushed down,
but Spark doesn't know which is the best order to push them into data
source. Spark can try every combination and ask the data source to report a
cost, then Spark can pick the best combination with the lowest cost. This
can also be implemented as a cost report interface, so that advanced data
source can implement it for optimal performance, and simple data source
doesn't need to care about it and keep simple.


The current design is very friendly to simple data source, and has the
potential to support complex data source, I prefer the current design over
the plan push down one. What do you think?


On Wed, Aug 30, 2017 at 5:53 AM, James Baker  wrote:

> Yeah, for sure.
>
> With the stable representation - agree that in the general case this is
> pretty intractable, it restricts the modifications that you can do in the
> future too much. That said, it shouldn't be as hard if you restrict
> yourself to the parts of the plan which are supported by the datasources V2
> API (which after all, need to be translateable properly into the future to
> support the mixins proposed). This should have a pretty small scope in
> comparison. As long as the user can bail out of nodes they don't
> understand, they should be ok, right?
>
> That said, what would also be fine for us is a place to plug into an
> unstable query plan.
>
> Explicitly here, what I'm looking for is a convenient mechanism to accept
> a fully specified set of arguments (of which I can choose to ignore some),
> and return the information as to which of them I'm ignoring. Taking a query
> plan of sorts is a way of doing this which IMO is intuitive to the user. It
> also provides a convenient location to plug in things like stats. Not at
> all married to the idea of using a query plan here; it just seemed
> convenient.
>
> Regarding the users who just want to be able to pump data into Spark, my
> understanding is that replacing isolated nodes in a query plan is easy.
> That said, our goal here is to be able to push down as much as possible
> into the underlying datastore.
>
> To your second question:
>
> The issue is that if you build up pushdowns incrementally and not all at
> once, you end up having to reject pushdowns and filters that you actually
> can do, which unnecessarily increases overheads.
>
> For example, the dataset
>
> a b c
> 1 2 3
> 1 3 3
> 1 3 4
> 2 1 1
> 2 0 1
>
> can efficiently push down sort(b, c) if I have already applied the filter
> a = 1, but otherwise will force a sort in Spark. On the PR I detail a case
> I see where I can push down two equality filters iff I am given them at the
> same time, whilst not being able to one at a time.
>
> There are loads of cases like this - you can imagine someone being able to
> push down a sort before a filter is applied, but not afterwards. However,
> maybe the filter is so selective that it's better to push down the filter
> and not handle the sort. I don't get to make this decision, Spark does (but
> doesn't have good enough information to do it properly, whilst I do). I
> want to be able to choose the parts I push down given knowledge of my
> datasource - as defined the APIs don't let me do that, they're strictly
> more restrictive than the V1 APIs in this way.
>
> The pattern of not considering things that can be done in bulk bites us in
> other ways. The retrieval methods end up being trickier to implement than
> is necessary because frequently a single operation 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-29 Thread James Baker
Yeah, for sure.

With the stable representation - agree that in the general case this is pretty 
intractable, it restricts the modifications that you can do in the future too 
much. That said, it shouldn't be as hard if you restrict yourself to the parts 
of the plan which are supported by the datasources V2 API (which after all, 
need to be translateable properly into the future to support the mixins 
proposed). This should have a pretty small scope in comparison. As long as the 
user can bail out of nodes they don't understand, they should be ok, right?

That said, what would also be fine for us is a place to plug into an unstable 
query plan.

Explicitly here, what I'm looking for is a convenient mechanism to accept a 
fully specified set of arguments (of which I can choose to ignore some), and 
return the information as to which of them I'm ignoring. Taking a query plan of 
sorts is a way of doing this which IMO is intuitive to the user. It also 
provides a convenient location to plug in things like stats. Not at all married 
to the idea of using a query plan here; it just seemed convenient.

Regarding the users who just want to be able to pump data into Spark, my 
understanding is that replacing isolated nodes in a query plan is easy. That 
said, our goal here is to be able to push down as much as possible into the 
underlying datastore.

To your second question:

The issue is that if you build up pushdowns incrementally and not all at once, 
you end up having to reject pushdowns and filters that you actually can do, 
which unnecessarily increases overheads.

For example, the dataset

a b c
1 2 3
1 3 3
1 3 4
2 1 1
2 0 1

can efficiently push down sort(b, c) if I have already applied the filter a = 
1, but otherwise will force a sort in Spark. On the PR I detail a case I see 
where I can push down two equality filters iff I am given them at the same 
time, whilst not being able to one at a time.

There are loads of cases like this - you can imagine someone being able to push 
down a sort before a filter is applied, but not afterwards. However, maybe the 
filter is so selective that it's better to push down the filter and not handle 
the sort. I don't get to make this decision, Spark does (but doesn't have good 
enough information to do it properly, whilst I do). I want to be able to choose 
the parts I push down given knowledge of my datasource - as defined the APIs 
don't let me do that, they're strictly more restrictive than the V1 APIs in 
this way.

The pattern of not considering things that can be done in bulk bites us in 
other ways. The retrieval methods end up being trickier to implement than is 
necessary because frequently a single operation provides the result of many of 
the getters, but the state is mutable, so you end up with odd caches.

For example, the work I need to do to answer unhandledFilters in V1 is roughly 
the same as the work I need to do to buildScan, so I want to cache it. This 
means that I end up with code that looks like:

public final class CachingFoo implements Foo {
private final Foo delegate;

private List currentFilters = emptyList();
private Supplier barSupplier = newSupplier(currentFilters);

public CachingFoo(Foo delegate) {
this.delegate = delegate;
}

private Supplier newSupplier(List filters) {
return Suppliers.memoize(() -> delegate.computeBar(filters));
}

@Override
public Bar computeBar(List filters) {
if (!filters.equals(currentFilters)) {
currentFilters = filters;
barSupplier = newSupplier(filters);
}

return barSupplier.get();
}
}

which caches the result required in unhandledFilters on the expectation that 
Spark will call buildScan afterwards and get to use the result..

This kind of cache becomes more prominent, but harder to deal with in the new 
APIs. As one example here, the state I will need in order to compute accurate 
column stats internally will likely be a subset of the work required in order 
to get the read tasks, tell you if I can handle filters, etc, so I'll want to 
cache them for reuse. However, the cached information needs to be appropriately 
invalidated when I add a new filter or sort order or limit, and this makes 
implementing the APIs harder and more error-prone.

One thing that'd be great is a defined contract of the order in which Spark 
calls the methods on your datasource (ideally this contract could be implied by 
the way the Java class structure works, but otherwise I can just throw).

James

On Tue, 29 Aug 2017 at 02:56 Reynold Xin 
> wrote:
James,

Thanks for the comment. I think you just pointed out a trade-off between 
expressiveness and API simplicity, compatibility and evolvability. For the max 
expressiveness, we'd want the ability to expose full query plans, and let the 
data source decide which part of the query plan can be pushed down.

The downside to that (full 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-29 Thread James Baker
Yeah, for sure.

With the stable representation - agree that in the general case this is pretty 
intractable, it restricts the modifications that you can do in the future too 
much. That said, it shouldn't be as hard if you restrict yourself to the parts 
of the plan which are supported by the datasources V2 API (which after all, 
need to be translateable properly into the future to support the mixins 
proposed). This should have a pretty small scope in comparison. As long as the 
user can bail out of nodes they don't understand, they should be ok, right?

That said, what would also be fine for us is a place to plug into an unstable 
query plan.

Explicitly here, what I'm looking for is a convenient mechanism to accept a 
fully specified set of arguments (of which I can choose to ignore some), and 
return the information as to which of them I'm ignoring. Taking a query plan of 
sorts is a way of doing this which IMO is intuitive to the user. It also 
provides a convenient location to plug in things like stats. Not at all married 
to the idea of using a query plan here; it just seemed convenient.

Regarding the users who just want to be able to pump data into Spark, my 
understanding is that replacing isolated nodes in a query plan is easy. That 
said, our goal here is to be able to push down as much as possible into the 
underlying datastore.

To your second question:

The issue is that if you build up pushdowns incrementally and not all at once, 
you end up having to reject pushdowns and filters that you actually can do, 
which unnecessarily increases overheads.

For example, the dataset

a b c
1 2 3
1 3 3
1 3 4
2 1 1
2 0 1

can efficiently push down sort(b, c) if I have already applied the filter a = 
1, but otherwise will force a sort in Spark. On the PR I detail a case I see 
where I can push down two equality filters iff I am given them at the same 
time, whilst not being able to one at a time.

There are loads of cases like this - you can imagine someone being able to push 
down a sort before a filter is applied, but not afterwards. However, maybe the 
filter is so selective that it's better to push down the filter and not handle 
the sort. I don't get to make this decision, Spark does (but doesn't have good 
enough information to do it properly, whilst I do). I want to be able to choose 
the parts I push down given knowledge of my datasource - as defined the APIs 
don't let me do that, they're strictly more restrictive than the V1 APIs in 
this way.

The pattern of not considering things that can be done in bulk bites us in 
other ways. The retrieval methods end up being trickier to implement than is 
necessary because frequently a single operation provides the result of many of 
the getters, but the state is mutable, so you end up with odd caches.

For example, the work I need to do to answer unhandledFilters in V1 is roughly 
the same as the work I need to do to buildScan, so I want to cache it. This 
means that I end up with code that looks like:

public final class CachingFoo implements Foo {
private final Foo delegate;

private List currentFilters = emptyList();
private Supplier barSupplier = newSupplier(currentFilters);

public CachingFoo(Foo delegate) {
this.delegate = delegate;
}

private Supplier newSupplier(List filters) {
return Suppliers.memoize(() -> delegate.computeBar(filters));
}

@Override
public Bar computeBar(List filters) {
if (!filters.equals(currentFilters)) {
currentFilters = filters;
barSupplier = newSupplier(filters);
}

return barSupplier.get();
}
}

which caches the result required in unhandledFilters on the expectation that 
Spark will call buildScan afterwards and get to use the result..

This kind of cache becomes more prominent, but harder to deal with in the new 
APIs. As one example here, the state I will need in order to compute accurate 
column stats internally will likely be a subset of the work required in order 
to get the read tasks, tell you if I can handle filters, etc, so I'll want to 
cache them for reuse. However, the cached information needs to be appropriately 
invalidated when I add a new filter or sort order or limit, and this makes 
implementing the APIs harder and more error-prone.

One thing that'd be great is a defined contract of the order in which Spark 
calls the methods on your datasource (ideally this contract could be implied by 
the way the Java class structure works, but otherwise I can just throw).

James

On Tue, 29 Aug 2017 at 02:56 Reynold Xin 
> wrote:
James,

Thanks for the comment. I think you just pointed out a trade-off between 
expressiveness and API simplicity, compatibility and evolvability. For the max 
expressiveness, we'd want the ability to expose full query plans, and let the 
data source decide which part of the query plan can be pushed down.

The downside to that (full 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-29 Thread Reynold Xin
James,

Thanks for the comment. I think you just pointed out a trade-off between
expressiveness and API simplicity, compatibility and evolvability. For the
max expressiveness, we'd want the ability to expose full query plans, and
let the data source decide which part of the query plan can be pushed down.

The downside to that (full query plan push down) are:

1. It is extremely difficult to design a stable representation for logical
/ physical plan. It is doable, but we'd be the first to do it. I'm not sure
of any mainstream databases being able to do that in the past. The design
of that API itself, to make sure we have a good story for backward and
forward compatibility, would probably take months if not years. It might
still be good to do, or offer an experimental trait without compatibility
guarantee that uses the current Catalyst internal logical plan.

2. Most data source developers simply want a way to offer some data,
without any pushdown. Having to understand query plans is a burden rather
than a gift.


Re: your point about the proposed v2 being worse than v1 for your use case.

Can you say more? You used the argument that in v2 there are more support
for broader pushdown and as a result it is harder to implement. That's how
it is supposed to be. If a data source simply implements one of the trait,
it'd be logically identical to v1. I don't see why it would be worse or
better, other than v2 provides much stronger forward compatibility
guarantees than v1.


On Tue, Aug 29, 2017 at 4:54 AM, James Baker  wrote:

> Copying from the code review comments I just submitted on the draft API (
> https://github.com/cloud-fan/spark/pull/10#pullrequestreview-59088745):
>
> Context here is that I've spent some time implementing a Spark datasource
> and have had some issues with the current API which are made worse in V2.
>
> The general conclusion I’ve come to here is that this is very hard to
> actually implement (in a similar but more aggressive way than DataSource
> V1, because of the extra methods and dimensions we get in V2).
>
> In DataSources V1 PrunedFilteredScan, the issue is that you are passed in
> the filters with the buildScan method, and then passed in again with the
> unhandledFilters method.
>
> However, the filters that you can’t handle might be data dependent, which
> the current API does not handle well. Suppose I can handle filter A some of
> the time, and filter B some of the time. If I’m passed in both, then either
> A and B are unhandled, or A, or B, or neither. The work I have to do to
> work this out is essentially the same as I have to do while actually
> generating my RDD (essentially I have to generate my partitions), so I end
> up doing some weird caching work.
>
> This V2 API proposal has the same issues, but perhaps moreso. In
> PrunedFilteredScan, there is essentially one degree of freedom for pruning
> (filters), so you just have to implement caching between unhandledFilters
> and buildScan. However, here we have many degrees of freedom; sorts,
> individual filters, clustering, sampling, maybe aggregations eventually -
> and these operations are not all commutative, and computing my support
> one-by-one can easily end up being more expensive than computing all in one
> go.
>
> For some trivial examples:
>
> - After filtering, I might be sorted, whilst before filtering I might not
> be.
>
> - Filtering with certain filters might affect my ability to push down
> others.
>
> - Filtering with aggregations (as mooted) might not be possible to push
> down.
>
> And with the API as currently mooted, I need to be able to go back and
> change my results because they might change later.
>
> Really what would be good here is to pass all of the filters and sorts etc
> all at once, and then I return the parts I can’t handle.
>
> I’d prefer in general that this be implemented by passing some kind of
> query plan to the datasource which enables this kind of replacement.
> Explicitly don’t want to give the whole query plan - that sounds painful -
> would prefer we push down only the parts of the query plan we deem to be
> stable. With the mix-in approach, I don’t think we can guarantee the
> properties we want without a two-phase thing - I’d really love to be able
> to just define a straightforward union type which is our supported pushdown
> stuff, and then the user can transform and return it.
>
> I think this ends up being a more elegant API for consumers, and also far
> more intuitive.
>
> James
>
> On Mon, 28 Aug 2017 at 18:00 蒋星博  wrote:
>
>> +1 (Non-binding)
>>
>> Xiao Li 于2017年8月28日 周一下午5:38写道:
>>
>>> +1
>>>
>>> 2017-08-28 12:45 GMT-07:00 Cody Koeninger :
>>>
 Just wanted to point out that because the jira isn't labeled SPIP, it
 won't have shown up linked from

 http://spark.apache.org/improvement-proposals.html

 On Mon, Aug 28, 2017 at 2:20 PM, Wenchen Fan 

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-28 Thread James Baker
Copying from the code review comments I just submitted on the draft API 
(https://github.com/cloud-fan/spark/pull/10#pullrequestreview-59088745):

Context here is that I've spent some time implementing a Spark datasource and 
have had some issues with the current API which are made worse in V2.

The general conclusion I’ve come to here is that this is very hard to actually 
implement (in a similar but more aggressive way than DataSource V1, because of 
the extra methods and dimensions we get in V2).

In DataSources V1 PrunedFilteredScan, the issue is that you are passed in the 
filters with the buildScan method, and then passed in again with the 
unhandledFilters method.

However, the filters that you can’t handle might be data dependent, which the 
current API does not handle well. Suppose I can handle filter A some of the 
time, and filter B some of the time. If I’m passed in both, then either A and B 
are unhandled, or A, or B, or neither. The work I have to do to work this out 
is essentially the same as I have to do while actually generating my RDD 
(essentially I have to generate my partitions), so I end up doing some weird 
caching work.

This V2 API proposal has the same issues, but perhaps moreso. In 
PrunedFilteredScan, there is essentially one degree of freedom for pruning 
(filters), so you just have to implement caching between unhandledFilters and 
buildScan. However, here we have many degrees of freedom; sorts, individual 
filters, clustering, sampling, maybe aggregations eventually - and these 
operations are not all commutative, and computing my support one-by-one can 
easily end up being more expensive than computing all in one go.

For some trivial examples:

- After filtering, I might be sorted, whilst before filtering I might not be.

- Filtering with certain filters might affect my ability to push down others.

- Filtering with aggregations (as mooted) might not be possible to push down.

And with the API as currently mooted, I need to be able to go back and change 
my results because they might change later.

Really what would be good here is to pass all of the filters and sorts etc all 
at once, and then I return the parts I can’t handle.

I’d prefer in general that this be implemented by passing some kind of query 
plan to the datasource which enables this kind of replacement. Explicitly don’t 
want to give the whole query plan - that sounds painful - would prefer we push 
down only the parts of the query plan we deem to be stable. With the mix-in 
approach, I don’t think we can guarantee the properties we want without a 
two-phase thing - I’d really love to be able to just define a straightforward 
union type which is our supported pushdown stuff, and then the user can 
transform and return it.

I think this ends up being a more elegant API for consumers, and also far more 
intuitive.

James

On Mon, 28 Aug 2017 at 18:00 蒋星博 
> wrote:
+1 (Non-binding)

Xiao Li >于2017年8月28日 
周一下午5:38写道:
+1

2017-08-28 12:45 GMT-07:00 Cody Koeninger 
>:
Just wanted to point out that because the jira isn't labeled SPIP, it
won't have shown up linked from

http://spark.apache.org/improvement-proposals.html

On Mon, Aug 28, 2017 at 2:20 PM, Wenchen Fan 
> wrote:
> Hi all,
>
> It has been almost 2 weeks since I proposed the data source V2 for
> discussion, and we already got some feedbacks on the JIRA ticket and the
> prototype PR, so I'd like to call for a vote.
>
> The full document of the Data Source API V2 is:
> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-Z8qU5Frf6WMQZ6jJVM/edit
>
> Note that, this vote should focus on high-level design/framework, not
> specified APIs, as we can always change/improve specified APIs during
> development.
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following technical
> reasons.
>
> Thanks!

-
To unsubscribe e-mail: 
dev-unsubscr...@spark.apache.org




Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-28 Thread James Baker
Copying from the code review comments I just submitted on the draft API 
(https://github.com/cloud-fan/spark/pull/10#pullrequestreview-59088745):

Context here is that I've spent some time implementing a Spark datasource and 
have had some issues with the current API which are made worse in V2.

The general conclusion I’ve come to here is that this is very hard to actually 
implement (in a similar but more aggressive way than DataSource V1, because of 
the extra methods and dimensions we get in V2).

In DataSources V1 PrunedFilteredScan, the issue is that you are passed in the 
filters with the buildScan method, and then passed in again with the 
unhandledFilters method.

However, the filters that you can’t handle might be data dependent, which the 
current API does not handle well. Suppose I can handle filter A some of the 
time, and filter B some of the time. If I’m passed in both, then either A and B 
are unhandled, or A, or B, or neither. The work I have to do to work this out 
is essentially the same as I have to do while actually generating my RDD 
(essentially I have to generate my partitions), so I end up doing some weird 
caching work.

This V2 API proposal has the same issues, but perhaps moreso. In 
PrunedFilteredScan, there is essentially one degree of freedom for pruning 
(filters), so you just have to implement caching between unhandledFilters and 
buildScan. However, here we have many degrees of freedom; sorts, individual 
filters, clustering, sampling, maybe aggregations eventually - and these 
operations are not all commutative, and computing my support one-by-one can 
easily end up being more expensive than computing all in one go.

For some trivial examples:

- After filtering, I might be sorted, whilst before filtering I might not be.

- Filtering with certain filters might affect my ability to push down others.

- Filtering with aggregations (as mooted) might not be possible to push down.

And with the API as currently mooted, I need to be able to go back and change 
my results because they might change later.

Really what would be good here is to pass all of the filters and sorts etc all 
at once, and then I return the parts I can’t handle.

I’d prefer in general that this be implemented by passing some kind of query 
plan to the datasource which enables this kind of replacement. Explicitly don’t 
want to give the whole query plan - that sounds painful - would prefer we push 
down only the parts of the query plan we deem to be stable. With the mix-in 
approach, I don’t think we can guarantee the properties we want without a 
two-phase thing - I’d really love to be able to just define a straightforward 
union type which is our supported pushdown stuff, and then the user can 
transform and return it.

I think this ends up being a more elegant API for consumers, and also far more 
intuitive.

James

On Mon, 28 Aug 2017 at 18:00 蒋星博 
> wrote:
+1 (Non-binding)

Xiao Li >于2017年8月28日 
周一下午5:38写道:
+1

2017-08-28 12:45 GMT-07:00 Cody Koeninger 
>:
Just wanted to point out that because the jira isn't labeled SPIP, it
won't have shown up linked from

http://spark.apache.org/improvement-proposals.html

On Mon, Aug 28, 2017 at 2:20 PM, Wenchen Fan 
> wrote:
> Hi all,
>
> It has been almost 2 weeks since I proposed the data source V2 for
> discussion, and we already got some feedbacks on the JIRA ticket and the
> prototype PR, so I'd like to call for a vote.
>
> The full document of the Data Source API V2 is:
> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-Z8qU5Frf6WMQZ6jJVM/edit
>
> Note that, this vote should focus on high-level design/framework, not
> specified APIs, as we can always change/improve specified APIs during
> development.
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following technical
> reasons.
>
> Thanks!

-
To unsubscribe e-mail: 
dev-unsubscr...@spark.apache.org




Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-28 Thread James Baker
Copying from the code review comments I just submitted on the draft API 
(https://github.com/cloud-fan/spark/pull/10#pullrequestreview-59088745):

Context here is that I've spent some time implementing a Spark datasource and 
have had some issues with the current API which are made worse in V2.

The general conclusion I’ve come to here is that this is very hard to actually 
implement (in a similar but more aggressive way than DataSource V1, because of 
the extra methods and dimensions we get in V2).

In DataSources V1 PrunedFilteredScan, the issue is that you are passed in the 
filters with the buildScan method, and then passed in again with the 
unhandledFilters method.

However, the filters that you can’t handle might be data dependent, which the 
current API does not handle well. Suppose I can handle filter A some of the 
time, and filter B some of the time. If I’m passed in both, then either A and B 
are unhandled, or A, or B, or neither. The work I have to do to work this out 
is essentially the same as I have to do while actually generating my RDD 
(essentially I have to generate my partitions), so I end up doing some weird 
caching work.

This V2 API proposal has the same issues, but perhaps moreso. In 
PrunedFilteredScan, there is essentially one degree of freedom for pruning 
(filters), so you just have to implement caching between unhandledFilters and 
buildScan. However, here we have many degrees of freedom; sorts, individual 
filters, clustering, sampling, maybe aggregations eventually - and these 
operations are not all commutative, and computing my support one-by-one can 
easily end up being more expensive than computing all in one go.

For some trivial examples:

- After filtering, I might be sorted, whilst before filtering I might not be.

- Filtering with certain filters might affect my ability to push down others.

- Filtering with aggregations (as mooted) might not be possible to push down.

And with the API as currently mooted, I need to be able to go back and change 
my results because they might change later.

Really what would be good here is to pass all of the filters and sorts etc all 
at once, and then I return the parts I can’t handle.

I’d prefer in general that this be implemented by passing some kind of query 
plan to the datasource which enables this kind of replacement. Explicitly don’t 
want to give the whole query plan - that sounds painful - would prefer we push 
down only the parts of the query plan we deem to be stable. With the mix-in 
approach, I don’t think we can guarantee the properties we want without a 
two-phase thing - I’d really love to be able to just define a straightforward 
union type which is our supported pushdown stuff, and then the user can 
transform and return it.

I think this ends up being a more elegant API for consumers, and also far more 
intuitive.

James

On Mon, 28 Aug 2017 at 18:00 蒋星博 
> wrote:
+1 (Non-binding)

Xiao Li >于2017年8月28日 
周一下午5:38写道:
+1

2017-08-28 12:45 GMT-07:00 Cody Koeninger 
>:
Just wanted to point out that because the jira isn't labeled SPIP, it
won't have shown up linked from

http://spark.apache.org/improvement-proposals.html

On Mon, Aug 28, 2017 at 2:20 PM, Wenchen Fan 
> wrote:
> Hi all,
>
> It has been almost 2 weeks since I proposed the data source V2 for
> discussion, and we already got some feedbacks on the JIRA ticket and the
> prototype PR, so I'd like to call for a vote.
>
> The full document of the Data Source API V2 is:
> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-Z8qU5Frf6WMQZ6jJVM/edit
>
> Note that, this vote should focus on high-level design/framework, not
> specified APIs, as we can always change/improve specified APIs during
> development.
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following technical
> reasons.
>
> Thanks!

-
To unsubscribe e-mail: 
dev-unsubscr...@spark.apache.org




Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-28 Thread 蒋星博
+1 (Non-binding)

Xiao Li 于2017年8月28日 周一下午5:38写道:

> +1
>
> 2017-08-28 12:45 GMT-07:00 Cody Koeninger :
>
>> Just wanted to point out that because the jira isn't labeled SPIP, it
>> won't have shown up linked from
>>
>> http://spark.apache.org/improvement-proposals.html
>>
>> On Mon, Aug 28, 2017 at 2:20 PM, Wenchen Fan  wrote:
>> > Hi all,
>> >
>> > It has been almost 2 weeks since I proposed the data source V2 for
>> > discussion, and we already got some feedbacks on the JIRA ticket and the
>> > prototype PR, so I'd like to call for a vote.
>> >
>> > The full document of the Data Source API V2 is:
>> >
>> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-Z8qU5Frf6WMQZ6jJVM/edit
>> >
>> > Note that, this vote should focus on high-level design/framework, not
>> > specified APIs, as we can always change/improve specified APIs during
>> > development.
>> >
>> > The vote will be up for the next 72 hours. Please reply with your vote:
>> >
>> > +1: Yeah, let's go forward and implement the SPIP.
>> > +0: Don't really care.
>> > -1: I don't think this is a good idea because of the following technical
>> > reasons.
>> >
>> > Thanks!
>>
>> -
>> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>>
>>
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-28 Thread Xiao Li
+1

2017-08-28 12:45 GMT-07:00 Cody Koeninger :

> Just wanted to point out that because the jira isn't labeled SPIP, it
> won't have shown up linked from
>
> http://spark.apache.org/improvement-proposals.html
>
> On Mon, Aug 28, 2017 at 2:20 PM, Wenchen Fan  wrote:
> > Hi all,
> >
> > It has been almost 2 weeks since I proposed the data source V2 for
> > discussion, and we already got some feedbacks on the JIRA ticket and the
> > prototype PR, so I'd like to call for a vote.
> >
> > The full document of the Data Source API V2 is:
> > https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-
> Z8qU5Frf6WMQZ6jJVM/edit
> >
> > Note that, this vote should focus on high-level design/framework, not
> > specified APIs, as we can always change/improve specified APIs during
> > development.
> >
> > The vote will be up for the next 72 hours. Please reply with your vote:
> >
> > +1: Yeah, let's go forward and implement the SPIP.
> > +0: Don't really care.
> > -1: I don't think this is a good idea because of the following technical
> > reasons.
> >
> > Thanks!
>
> -
> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-28 Thread Cody Koeninger
Just wanted to point out that because the jira isn't labeled SPIP, it
won't have shown up linked from

http://spark.apache.org/improvement-proposals.html

On Mon, Aug 28, 2017 at 2:20 PM, Wenchen Fan  wrote:
> Hi all,
>
> It has been almost 2 weeks since I proposed the data source V2 for
> discussion, and we already got some feedbacks on the JIRA ticket and the
> prototype PR, so I'd like to call for a vote.
>
> The full document of the Data Source API V2 is:
> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-Z8qU5Frf6WMQZ6jJVM/edit
>
> Note that, this vote should focus on high-level design/framework, not
> specified APIs, as we can always change/improve specified APIs during
> development.
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following technical
> reasons.
>
> Thanks!

-
To unsubscribe e-mail: dev-unsubscr...@spark.apache.org



Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-28 Thread Russell Spitzer
+1 (Non-binding)

The clustering approach covers most of my requirements on saving some
shuffles. We kind of left the "should the user be allowed to provide a full
partitioner" discussion on the table. I understand that would require
exposing a lot of internals so this is perhaps a good compromise.

On Mon, Aug 28, 2017 at 12:20 PM Wenchen Fan  wrote:

> Hi all,
>
> It has been almost 2 weeks since I proposed the data source V2 for
> discussion, and we already got some feedbacks on the JIRA ticket and the
> prototype PR, so I'd like to call for a vote.
>
> The full document of the Data Source API V2 is:
>
> https://docs.google.com/document/d/1n_vUVbF4KD3gxTmkNEon5qdQ-Z8qU5Frf6WMQZ6jJVM/edit
>
> Note that, this vote should focus on high-level design/framework, not
> specified APIs, as we can always change/improve specified APIs during
> development.
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following technical
> reasons.
>
> Thanks!
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-17 Thread Mark Hamstra
Points 2, 3 and 4 of the Project Plan in that document (i.e. "port existing
data sources using internal APIs to use the proposed public Data Source V2
API") have my full support. Really, I'd like to see that dog-fooding effort
completed and lesson learned from it fully digested before we remove any
unstable annotations from the new API. It's okay to get a proposal out
there so that we can talk about it and start implementing and using it
internally, followed by external use under the unstable annotations, but I
don't want to see a premature vote on a final form of a new public API.

On Thu, Aug 17, 2017 at 8:55 AM, Reynold Xin  wrote:

> Yea I don't think it's a good idea to upload a doc and then call for a
> vote immediately. People need time to digest ...
>
>
> On Thu, Aug 17, 2017 at 6:22 AM, Wenchen Fan  wrote:
>
>> Sorry let's remove the VOTE tag as I just wanna bring this up for
>> discussion.
>>
>> I'll restart the voting process after we have enough discussion on the
>> JIRA ticket or here in this email thread.
>>
>> On Thu, Aug 17, 2017 at 9:12 PM, Russell Spitzer <
>> russell.spit...@gmail.com> wrote:
>>
>>> -1, I don't think there has really been any discussion of this api
>>> change yet or at least it hasn't occurred on the jira ticket
>>>
>>> On Thu, Aug 17, 2017 at 8:05 AM Wenchen Fan  wrote:
>>>
 adding my own +1 (binding)

 On Thu, Aug 17, 2017 at 9:02 PM, Wenchen Fan 
 wrote:

> Hi all,
>
> Following the SPIP process, I'm putting this SPIP up for a vote.
>
> The current data source API doesn't work well because of some
> limitations like: no partitioning/bucketing support, no columnar read, 
> hard
> to support more operator push down, etc.
>
> I'm proposing a Data Source API V2 to address these problems, please
> read the full document at https://issues.apache.org/jira
> /secure/attachment/12882332/SPIP%20Data%20Source%20API%20V2.pdf
>
> Since this SPIP is mostly about APIs, I also created a prototype and
> put java docs on these interfaces, so that it's easier to review these
> interfaces and discuss: https://github.com/cl
> oud-fan/spark/pull/10/files
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following
> technical reasons.
>
> Thanks!
>


>>
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-17 Thread Reynold Xin
Yea I don't think it's a good idea to upload a doc and then call for a vote
immediately. People need time to digest ...


On Thu, Aug 17, 2017 at 6:22 AM, Wenchen Fan  wrote:

> Sorry let's remove the VOTE tag as I just wanna bring this up for
> discussion.
>
> I'll restart the voting process after we have enough discussion on the
> JIRA ticket or here in this email thread.
>
> On Thu, Aug 17, 2017 at 9:12 PM, Russell Spitzer <
> russell.spit...@gmail.com> wrote:
>
>> -1, I don't think there has really been any discussion of this api change
>> yet or at least it hasn't occurred on the jira ticket
>>
>> On Thu, Aug 17, 2017 at 8:05 AM Wenchen Fan  wrote:
>>
>>> adding my own +1 (binding)
>>>
>>> On Thu, Aug 17, 2017 at 9:02 PM, Wenchen Fan 
>>> wrote:
>>>
 Hi all,

 Following the SPIP process, I'm putting this SPIP up for a vote.

 The current data source API doesn't work well because of some
 limitations like: no partitioning/bucketing support, no columnar read, hard
 to support more operator push down, etc.

 I'm proposing a Data Source API V2 to address these problems, please
 read the full document at https://issues.apache.org/jira
 /secure/attachment/12882332/SPIP%20Data%20Source%20API%20V2.pdf

 Since this SPIP is mostly about APIs, I also created a prototype and
 put java docs on these interfaces, so that it's easier to review these
 interfaces and discuss: https://github.com/cl
 oud-fan/spark/pull/10/files

 The vote will be up for the next 72 hours. Please reply with your vote:

 +1: Yeah, let's go forward and implement the SPIP.
 +0: Don't really care.
 -1: I don't think this is a good idea because of the following
 technical reasons.

 Thanks!

>>>
>>>
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-17 Thread Wenchen Fan
Sorry let's remove the VOTE tag as I just wanna bring this up for
discussion.

I'll restart the voting process after we have enough discussion on the JIRA
ticket or here in this email thread.

On Thu, Aug 17, 2017 at 9:12 PM, Russell Spitzer 
wrote:

> -1, I don't think there has really been any discussion of this api change
> yet or at least it hasn't occurred on the jira ticket
>
> On Thu, Aug 17, 2017 at 8:05 AM Wenchen Fan  wrote:
>
>> adding my own +1 (binding)
>>
>> On Thu, Aug 17, 2017 at 9:02 PM, Wenchen Fan  wrote:
>>
>>> Hi all,
>>>
>>> Following the SPIP process, I'm putting this SPIP up for a vote.
>>>
>>> The current data source API doesn't work well because of some
>>> limitations like: no partitioning/bucketing support, no columnar read, hard
>>> to support more operator push down, etc.
>>>
>>> I'm proposing a Data Source API V2 to address these problems, please
>>> read the full document at https://issues.apache.org/
>>> jira/secure/attachment/12882332/SPIP%20Data%20Source%20API%20V2.pdf
>>>
>>> Since this SPIP is mostly about APIs, I also created a prototype and put
>>> java docs on these interfaces, so that it's easier to review these
>>> interfaces and discuss: https://github.com/cloud-fan/spark/pull/10/files
>>>
>>> The vote will be up for the next 72 hours. Please reply with your vote:
>>>
>>> +1: Yeah, let's go forward and implement the SPIP.
>>> +0: Don't really care.
>>> -1: I don't think this is a good idea because of the following
>>> technical reasons.
>>>
>>> Thanks!
>>>
>>
>>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-17 Thread Russell Spitzer
-1, I don't think there has really been any discussion of this api change
yet or at least it hasn't occurred on the jira ticket

On Thu, Aug 17, 2017 at 8:05 AM Wenchen Fan  wrote:

> adding my own +1 (binding)
>
> On Thu, Aug 17, 2017 at 9:02 PM, Wenchen Fan  wrote:
>
>> Hi all,
>>
>> Following the SPIP process, I'm putting this SPIP up for a vote.
>>
>> The current data source API doesn't work well because of some limitations
>> like: no partitioning/bucketing support, no columnar read, hard to support
>> more operator push down, etc.
>>
>> I'm proposing a Data Source API V2 to address these problems, please read
>> the full document at
>> https://issues.apache.org/jira/secure/attachment/12882332/SPIP%20Data%20Source%20API%20V2.pdf
>>
>> Since this SPIP is mostly about APIs, I also created a prototype and put
>> java docs on these interfaces, so that it's easier to review these
>> interfaces and discuss: https://github.com/cloud-fan/spark/pull/10/files
>>
>> The vote will be up for the next 72 hours. Please reply with your vote:
>>
>> +1: Yeah, let's go forward and implement the SPIP.
>> +0: Don't really care.
>> -1: I don't think this is a good idea because of the following
>> technical reasons.
>>
>> Thanks!
>>
>
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-17 Thread 蒋星博
+1 (non-binding)

Wenchen Fan 于2017年8月17日 周四下午9:05写道:

> adding my own +1 (binding)
>
> On Thu, Aug 17, 2017 at 9:02 PM, Wenchen Fan  wrote:
>
>> Hi all,
>>
>> Following the SPIP process, I'm putting this SPIP up for a vote.
>>
>> The current data source API doesn't work well because of some limitations
>> like: no partitioning/bucketing support, no columnar read, hard to support
>> more operator push down, etc.
>>
>> I'm proposing a Data Source API V2 to address these problems, please read
>> the full document at
>> https://issues.apache.org/jira/secure/attachment/12882332/SPIP%20Data%20Source%20API%20V2.pdf
>>
>> Since this SPIP is mostly about APIs, I also created a prototype and put
>> java docs on these interfaces, so that it's easier to review these
>> interfaces and discuss: https://github.com/cloud-fan/spark/pull/10/files
>>
>> The vote will be up for the next 72 hours. Please reply with your vote:
>>
>> +1: Yeah, let's go forward and implement the SPIP.
>> +0: Don't really care.
>> -1: I don't think this is a good idea because of the following
>> technical reasons.
>>
>> Thanks!
>>
>
>


Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-17 Thread Wenchen Fan
adding my own +1 (binding)

On Thu, Aug 17, 2017 at 9:02 PM, Wenchen Fan  wrote:

> Hi all,
>
> Following the SPIP process, I'm putting this SPIP up for a vote.
>
> The current data source API doesn't work well because of some limitations
> like: no partitioning/bucketing support, no columnar read, hard to support
> more operator push down, etc.
>
> I'm proposing a Data Source API V2 to address these problems, please read
> the full document at https://issues.apache.org/jira/secure/attachment/
> 12882332/SPIP%20Data%20Source%20API%20V2.pdf
>
> Since this SPIP is mostly about APIs, I also created a prototype and put
> java docs on these interfaces, so that it's easier to review these
> interfaces and discuss: https://github.com/cloud-fan/spark/pull/10/files
>
> The vote will be up for the next 72 hours. Please reply with your vote:
>
> +1: Yeah, let's go forward and implement the SPIP.
> +0: Don't really care.
> -1: I don't think this is a good idea because of the following
> technical reasons.
>
> Thanks!
>