Re: DataSourceV2 : Transactional Write support

2019-08-05 Thread Matt Cheah
There might be some help from the staging table catalog as well.

 

-Matt Cheah

 

From: Wenchen Fan 
Date: Monday, August 5, 2019 at 7:40 PM
To: Shiv Prashant Sood 
Cc: Ryan Blue , Jungtaek Lim , Spark Dev 
List 
Subject: Re: DataSourceV2 : Transactional Write support

 

I agree with the temp table approach. One idea is: maybe we only need one temp 
table, and each task writes to this temp table. At the end we read the data 
from the temp table and write it to the target table. AFAIK JDBC can handle 
concurrent table writing very well, and it's better than creating thousands of 
temp tables for one write job(assume the input RDD has thousands of partitions).

 

On Tue, Aug 6, 2019 at 7:57 AM Shiv Prashant Sood  
wrote:

Thanks all for the clarification.

 

Regards,

Shiv

 

On Sat, Aug 3, 2019 at 12:49 PM Ryan Blue  wrote:

> What you could try instead is intermediate output: inserting into temporal 
> table in executors, and move inserted records to the final table in driver 
> (must be atomic) 

 

I think that this is the approach that other systems (maybe sqoop?) have taken. 
Insert into independent temporary tables, which can be done quickly. Then for 
the final commit operation, union and insert into the final table. In a lot of 
cases, JDBC databases can do that quickly as well because the data is already 
on disk and just needs to added to the final table.

 

On Fri, Aug 2, 2019 at 7:25 PM Jungtaek Lim  wrote:

I asked similar question for end-to-end exactly-once with Kafka, and you're 
correct distributed transaction is not supported. Introducing distributed 
transaction like "two-phase commit" requires huge change on Spark codebase and 
the feedback was not positive. 

 

What you could try instead is intermediate output: inserting into temporal 
table in executors, and move inserted records to the final table in driver 
(must be atomic).

 

Thanks,

Jungtaek Lim (HeartSaVioR)

 

On Sat, Aug 3, 2019 at 4:56 AM Shiv Prashant Sood  
wrote:

All,

 

I understood that DataSourceV2 supports Transactional write and wanted to  
implement that in JDBC DataSource V2 connector ( PR#25211 [github.com] ). 

 

Don't see how this is feasible for JDBC based connector.  The FW suggest that 
EXECUTOR send a commit message  to DRIVER, and actual commit should only be 
done by DRIVER after receiving all commit confirmations. This will not work for 
JDBC  as commits have to happen on the JDBC Connection which is maintained by 
the EXECUTORS and JDBCConnection  is not serializable that it can be sent to 
the DRIVER. 

 

Am i right in thinking that this cannot be supported for JDBC? My goal is to 
either fully write or roll back the dataframe write operation.

 

Thanks in advance for your help.

 

Regards, 

Shiv


 

-- 

Name : Jungtaek Lim
Blog : http://medium.com/@heartsavior [medium.com]
Twitter : http://twitter.com/heartsavior [twitter.com] 

LinkedIn : http://www.linkedin.com/in/heartsavior [linkedin.com]


 

-- 

Ryan Blue 

Software Engineer

Netflix



smime.p7s
Description: S/MIME cryptographic signature


Re: DataSourceV2 : Transactional Write support

2019-08-05 Thread Wenchen Fan
I agree with the temp table approach. One idea is: maybe we only need one
temp table, and each task writes to this temp table. At the end we read the
data from the temp table and write it to the target table. AFAIK JDBC can
handle concurrent table writing very well, and it's better than creating
thousands of temp tables for one write job(assume the input RDD has
thousands of partitions).

On Tue, Aug 6, 2019 at 7:57 AM Shiv Prashant Sood 
wrote:

> Thanks all for the clarification.
>
> Regards,
> Shiv
>
> On Sat, Aug 3, 2019 at 12:49 PM Ryan Blue 
> wrote:
>
>> > What you could try instead is intermediate output: inserting into
>> temporal table in executors, and move inserted records to the final table
>> in driver (must be atomic)
>>
>> I think that this is the approach that other systems (maybe sqoop?) have
>> taken. Insert into independent temporary tables, which can be done quickly.
>> Then for the final commit operation, union and insert into the final table.
>> In a lot of cases, JDBC databases can do that quickly as well because the
>> data is already on disk and just needs to added to the final table.
>>
>> On Fri, Aug 2, 2019 at 7:25 PM Jungtaek Lim  wrote:
>>
>>> I asked similar question for end-to-end exactly-once with Kafka, and
>>> you're correct distributed transaction is not supported. Introducing
>>> distributed transaction like "two-phase commit" requires huge change on
>>> Spark codebase and the feedback was not positive.
>>>
>>> What you could try instead is intermediate output: inserting into
>>> temporal table in executors, and move inserted records to the final table
>>> in driver (must be atomic).
>>>
>>> Thanks,
>>> Jungtaek Lim (HeartSaVioR)
>>>
>>> On Sat, Aug 3, 2019 at 4:56 AM Shiv Prashant Sood <
>>> shivprash...@gmail.com> wrote:
>>>
 All,

 I understood that DataSourceV2 supports Transactional write and wanted
 to  implement that in JDBC DataSource V2 connector ( PR#25211
  ).

 Don't see how this is feasible for JDBC based connector.  The FW
 suggest that EXECUTOR send a commit message  to DRIVER, and actual
 commit should only be done by DRIVER after receiving all commit
 confirmations. This will not work for JDBC  as commits have to happen on
 the JDBC Connection which is maintained by the EXECUTORS and
 JDBCConnection  is not serializable that it can be sent to the DRIVER.

 Am i right in thinking that this cannot be supported for JDBC? My goal
 is to either fully write or roll back the dataframe write operation.

 Thanks in advance for your help.

 Regards,
 Shiv

>>>
>>>
>>> --
>>> Name : Jungtaek Lim
>>> Blog : http://medium.com/@heartsavior
>>> Twitter : http://twitter.com/heartsavior
>>> LinkedIn : http://www.linkedin.com/in/heartsavior
>>>
>>
>>
>> --
>> Ryan Blue
>> Software Engineer
>> Netflix
>>
>


Re: Ask for ARM CI for spark

2019-08-05 Thread bo zhaobo
Hi shane,
Thanks for your reply. I will wait for you back. ;-)

Thanks,
Best regards
ZhaoBo



[image: Mailtrack]

Sender
notified by
Mailtrack

19/08/06
上午09:06:23

shane knapp  于2019年8月2日周五 下午10:41写道:

> i'm out of town, but will answer some of your questions next week.
>
> On Fri, Aug 2, 2019 at 2:39 AM bo zhaobo 
> wrote:
>
>>
>> Hi Team,
>>
>> Any updates about the CI details? ;-)
>>
>> Also, I will also need your kind help about Spark QA test, could any one
>> can tell us how to trigger that tests? When? How?  So far, I haven't
>> notices how it works.
>>
>> Thanks
>>
>> Best Regards,
>>
>> ZhaoBo
>>
>>
>>
>> [image: Mailtrack]
>> 
>>  Sender
>> notified by
>> Mailtrack
>> 
>>  19/08/02
>> 下午05:37:30
>>
>> bo zhaobo  于2019年7月31日周三 上午11:56写道:
>>
>>> Hi, team.
>>> I want to make the same test on ARM like existing CI does(x86). As
>>> building and testing the whole spark projects will cost too long time, so I
>>> plan to split them to multiple jobs to run for lower time cost. But I
>>> cannot see what the existing CI[1] have done(so many private scripts
>>> called), so could any CI maintainers help/tell us for how to split them and
>>> the details about different CI jobs does? Such as PR title contains [SQL],
>>> [INFRA], [ML], [DOC], [CORE], [PYTHON], [k8s], [DSTREAMS], [MLlib],
>>> [SCHEDULER], [SS],[YARN], [BUIILD] and etc..I found each of them seems run
>>> the different CI job.
>>>
>>> @shane knapp,
>>> Oh, sorry for disturb. I found your email looks like from 'berkeley.edu',
>>> are you the good guy who we are looking for help about this? ;-)
>>> If so, could you give some helps or advices? Thank you.
>>>
>>> Thank you very much,
>>>
>>> Best Regards,
>>>
>>> ZhaoBo
>>>
>>> [1] https://amplab.cs.berkeley.edu/jenkins
>>>
>>>
>>>
>>>
>>> [image: Mailtrack]
>>> 
>>>  Sender
>>> notified by
>>> Mailtrack
>>> 
>>>  19/07/31
>>> 上午11:53:36
>>>
>>> Tianhua huang  于2019年7月29日周一 上午9:38写道:
>>>
 @Sean Owen   Thank you very much. And I saw your
 reply comment in https://issues.apache.org/jira/browse/SPARK-28519, I
 will test with modification and to see whether there are other similar
 tests fail, and will address them together in one pull request.

 On Sat, Jul 27, 2019 at 9:04 PM Sean Owen  wrote:

> Great thanks - we can take this to JIRAs now.
> I think it's worth changing the implementation of atanh if the test
> value just reflects what Spark does, and there's evidence is a little bit
> inaccurate.
> There's an equivalent formula which seems to have better accuracy.
>
> On Fri, Jul 26, 2019 at 10:02 PM Takeshi Yamamuro <
> linguin@gmail.com> wrote:
>
>> Hi, all,
>>
>> FYI:
>> >> @Yuming Wang the results in float8.sql are from PostgreSQL
>> directly?
>> >> Interesting if it also returns the same less accurate result, which
>> >> might suggest it's more to do with underlying OS math libraries.
>> You
>> >> noted that these tests sometimes gave platform-dependent
>> differences
>> >> in the last digit, so wondering if the test value directly reflects
>> >> PostgreSQL or just what we happen to return now.
>>
>> The results in float8.sql.out were recomputed in Spark/JVM.
>> The expected output of the PostgreSQL test is here:
>> https://github.com/postgres/postgres/blob/master/src/test/regress/expected/float8.out#L493
>>
>> As you can see in the file (float8.out), the results other than atanh
>> also are different between Spark/JVM and PostgreSQL.
>> For example, the answers of acosh are:
>> -- PostgreSQL
>>
>> https://github.com/postgres/postgres/blob/master/src/test/regress/expected/float8.out#L487
>> 1.31695789692482
>>
>> -- Spark/JVM
>>
>> https://github.com/apache/spark/blob/master/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out#L523
>> 1.3169578969248166
>>
>> btw, the PostgreSQL implementation for atanh just calls atanh in
>> math.h:
>>
>> https://github.com/postgres/postgres/blob/master/src/backend/utils/adt/float.c#L2606
>>
>> Bests,
>> Takeshi
>>
>>
>
> --
> Shane Knapp
> UC Berkeley EECS Research / RISELab Staff Technical Lead
> https://rise.cs.berkeley.edu
>


Re: DataSourceV2 : Transactional Write support

2019-08-05 Thread Shiv Prashant Sood
Thanks all for the clarification.

Regards,
Shiv

On Sat, Aug 3, 2019 at 12:49 PM Ryan Blue  wrote:

> > What you could try instead is intermediate output: inserting into
> temporal table in executors, and move inserted records to the final table
> in driver (must be atomic)
>
> I think that this is the approach that other systems (maybe sqoop?) have
> taken. Insert into independent temporary tables, which can be done quickly.
> Then for the final commit operation, union and insert into the final table.
> In a lot of cases, JDBC databases can do that quickly as well because the
> data is already on disk and just needs to added to the final table.
>
> On Fri, Aug 2, 2019 at 7:25 PM Jungtaek Lim  wrote:
>
>> I asked similar question for end-to-end exactly-once with Kafka, and
>> you're correct distributed transaction is not supported. Introducing
>> distributed transaction like "two-phase commit" requires huge change on
>> Spark codebase and the feedback was not positive.
>>
>> What you could try instead is intermediate output: inserting into
>> temporal table in executors, and move inserted records to the final table
>> in driver (must be atomic).
>>
>> Thanks,
>> Jungtaek Lim (HeartSaVioR)
>>
>> On Sat, Aug 3, 2019 at 4:56 AM Shiv Prashant Sood 
>> wrote:
>>
>>> All,
>>>
>>> I understood that DataSourceV2 supports Transactional write and wanted
>>> to  implement that in JDBC DataSource V2 connector ( PR#25211
>>>  ).
>>>
>>> Don't see how this is feasible for JDBC based connector.  The FW suggest
>>> that EXECUTOR send a commit message  to DRIVER, and actual commit
>>> should only be done by DRIVER after receiving all commit confirmations.
>>> This will not work for JDBC  as commits have to happen on the JDBC
>>> Connection which is maintained by the EXECUTORS and JDBCConnection  is not
>>> serializable that it can be sent to the DRIVER.
>>>
>>> Am i right in thinking that this cannot be supported for JDBC? My goal
>>> is to either fully write or roll back the dataframe write operation.
>>>
>>> Thanks in advance for your help.
>>>
>>> Regards,
>>> Shiv
>>>
>>
>>
>> --
>> Name : Jungtaek Lim
>> Blog : http://medium.com/@heartsavior
>> Twitter : http://twitter.com/heartsavior
>> LinkedIn : http://www.linkedin.com/in/heartsavior
>>
>
>
> --
> Ryan Blue
> Software Engineer
> Netflix
>


Re: Recognizing non-code contributions

2019-08-05 Thread Nicholas Chammas
On Mon, Aug 5, 2019 at 9:55 AM Sean Owen  wrote:

> On Mon, Aug 5, 2019 at 3:50 AM Myrle Krantz  wrote:
> > So... events coordinators?  I'd still make them committers.  I guess I'm
> still struggling to understand what problem making people VIP's without
> giving them committership is trying to solve.
>
> We may just agree to disagree, which is fine, but I think the argument
> is clear enough: such a person has zero need for the commit bit.
> Turning it around, what are we trying to accomplish by giving said
> person a commit bit? I know people say there's no harm, but I think
> there is at least _some_ downside. We're widening access to change
> software artifacts, the main thing that we put ASF process and checks
> around for liability reasons. I know the point is trust, and said
> person is likely to understand to never use the commit bit, but it
> brings us back to the same place. I don't wish to convince anyone else
> of my stance, though I do find it more logical, just that it's
> reasonable within The Apache Way.
>

+1 to this.


Re: [Discuss] Follow ANSI SQL on table insertion

2019-08-05 Thread Wenchen Fan
Ryan, I agree that it needs a VOTE to decide the most reasonable default
setting. But what shall we do for Spark 3.0 if there is no more progress on
this project anymore(just assume)?

In the master branch, we added SQL support for Data Source V2. This exposes
a serious problem: the table insertion behavior is different between tables
from custom catalogs and from the built-in hive catalog. I don't think this
is acceptable for Spark 3.0.

This leaves us with 2 options (assume no more progress on ANSI mode, strict
mode, etc.):
1. change the table insertion behavior of DS v1 to upcast mode, so that
it's the same with DS v2.
2. change the table insertion behavior of DS v2 to always-cast mode, so
that it's the same with DS v1.

I tried option 1 before, it broke a lot of tests and I believe it will
break many user queries as well. Option 2 is not ideal but it's safe: it's
no worse than the last release.

That's why I disagree with "Finish ANSI SQL mode - but do not make it the
default because it is not safe without an option to enable strict mode.".
Yes it's not safe compared to the ideal solution. But it's safe compared to
the last release. We must consider the risk of not able to finish the
"ideal solution" before Spark 3.0.

The runtime exception mode is still under development. The try_cast or safe
methods are not even planned. The upcast mode has serious backward
compatibility problems. To prepare for the worst, I think we can do this
first: create a flag for the upcast mode, and turn it off by default, for
both data source v1 and v2. Then we can work on the following tasks in
parallel and decide the default behavior later according to the progress:
1. implement the store assignment rule
2. finish the runtime exception mode
3. add try_cast or safe methods

Another option is to ask the PMC to vote for blocking Spark 3.0 if the
"return null behavior" is not fixed. But I don't think it's likely to
happen.

On Tue, Aug 6, 2019 at 12:34 AM Ryan Blue  wrote:

> Wenchen, I don’t think we agree on what “strict mode” would mean. Marco is
> talking about strict mode as an extension of the flag for throwing
> exceptions on overflow for decimal operations. That is not ANSI SQL mode.
>
> Also, we need more than ANSI SQL and runtime failure modes. For the
> motivating problem of validating a write, we need a way to preserve the
> analysis-time failure if types don’t match. That, combined with a runtime
> strict mode, is the option that fails fast and guarantees data isn’t lost.
>
> I agree that adding try_cast or safe methods is a good idea.
>
> So here’s a revised set of steps:
>
>1. Finish ANSI SQL mode - but do not make it the default because it is
>not safe without an option to enable strict mode.
>2. Add strict mode for runtime calculations and turn it on by default
>3. Add a flag to control analysis time vs runtime failures (using
>strict mode or ANSI SQL mode) for v2 writes
>
> The choice of whether runtime or analysis time failures should be the
> default for v2 writes is worth a VOTE on this list. Once we agree on what
> modes and options should be available, we can call a vote to build
> consensus around a reasonable set of defaults, given that there are a lot
> of varying opinions on this thread.
>
> On Mon, Aug 5, 2019 at 12:49 AM Wenchen Fan  wrote:
>
>> I think we need to clarify one thing before further discussion: the
>> proposal is for the next release but not a long term solution.
>>
>> IMO the long term solution should be: completely follow SQL standard
>> (store assignment rule + runtime exception), and provide a variant of
>> functions that can return null instead of runtime exception. For example,
>> the TRY_CAST in SQL server
>> ,
>> or a more general SAFE prefix of functions in Big Query
>> 
>> .
>>
>> This proposal is the first step to move forward to the long term
>> solution: follow the SQL standard store assignment rule. It can help us
>> prevent some table insertion queries that are very likely to fail, at
>> compile time.
>>
>> The following steps in my mind are:
>> * finish the "strict mode"(ANSI SQL mode). It now applies to arithmetic
>> functions, but we need to apply it to more places like cast.
>> * introduce the safe version of functions. The safe version always
>> returns null for invalid input, no matter the strict mode is on or not. We
>> need some additional work to educate users to use the safe version of the
>> functions if they rely on the return null behavior.
>> * turn on the strict mode by default.
>>
>> Hopefully we can finish it soon, in Spark 3.x.
>>
>> Thanks,
>> Wenchen
>>
>> On Sat, Aug 3, 2019 at 7:07 AM Matt Cheah  wrote:
>>
>>> *I agree that having both modes and let the user choose the one he/she
>>> wants is the best option (I don't see big arguments on this 

Re: [Discuss] Follow ANSI SQL on table insertion

2019-08-05 Thread Gengliang Wang
> The choice of whether runtime or analysis time failures should be the
default for v2 writes is worth a VOTE on this list.

The motivation of my proposal is to make the upgrade from 2.4 to 3.0 more
user-friendly and smooth. So I am absolutely +1 on making analysis time
failures as default. But not the way as up-cast, we should consider
following the store assignment rules of PostgreSQL (See
https://github.com/apache/spark/pull/25239/files#diff-df78a74ef92d9b8fb4ac142ff9a62464R463
for
more details).  The rules are improved, and the behavior is within users’
expectation even when queries are broken.

The rules of Up-Cast are quite strict and there is no existing popular DBMS
has the same behavior. Basically Up-Cast doesn’t allow any possible precision
loss,  e.g. “double -> decimal”, “decimal -> double”, “timestamp” -> “
date”, “double” -> long,
 are not allowed. It is possible that many existing ETL jobs failed with
the up-cast rules.
IIUC, there is already -1 on upcast from Wenchen, Hyukjin and me.
So I would suggest two votes

   1. Which should we choose for analysis check?
  1. ANSI SQL store assignment rules (exception null on cast failure,
  which we already did in Spark 2.x)
  2. Up-Cast rules, no any possible precision should be allowed in type
  conversion
   2. Which should be the default behavior for the table insertion?
  1. Analysis time failures
  2. Runtime failures


On Tue, Aug 6, 2019 at 12:34 AM Ryan Blue  wrote:

> Wenchen, I don’t think we agree on what “strict mode” would mean. Marco is
> talking about strict mode as an extension of the flag for throwing
> exceptions on overflow for decimal operations. That is not ANSI SQL mode.
>
> Also, we need more than ANSI SQL and runtime failure modes. For the
> motivating problem of validating a write, we need a way to preserve the
> analysis-time failure if types don’t match. That, combined with a runtime
> strict mode, is the option that fails fast and guarantees data isn’t lost.
>
> I agree that adding try_cast or safe methods is a good idea.
>
> So here’s a revised set of steps:
>
>1. Finish ANSI SQL mode - but do not make it the default because it is
>not safe without an option to enable strict mode.
>2. Add strict mode for runtime calculations and turn it on by default
>3. Add a flag to control analysis time vs runtime failures (using
>strict mode or ANSI SQL mode) for v2 writes
>
> The choice of whether runtime or analysis time failures should be the
> default for v2 writes is worth a VOTE on this list. Once we agree on what
> modes and options should be available, we can call a vote to build
> consensus around a reasonable set of defaults, given that there are a lot
> of varying opinions on this thread.
>
> On Mon, Aug 5, 2019 at 12:49 AM Wenchen Fan  wrote:
>
>> I think we need to clarify one thing before further discussion: the
>> proposal is for the next release but not a long term solution.
>>
>> IMO the long term solution should be: completely follow SQL standard
>> (store assignment rule + runtime exception), and provide a variant of
>> functions that can return null instead of runtime exception. For example,
>> the TRY_CAST in SQL server
>> ,
>> or a more general SAFE prefix of functions in Big Query
>> 
>> .
>>
>> This proposal is the first step to move forward to the long term
>> solution: follow the SQL standard store assignment rule. It can help us
>> prevent some table insertion queries that are very likely to fail, at
>> compile time.
>>
>> The following steps in my mind are:
>> * finish the "strict mode"(ANSI SQL mode). It now applies to arithmetic
>> functions, but we need to apply it to more places like cast.
>> * introduce the safe version of functions. The safe version always
>> returns null for invalid input, no matter the strict mode is on or not. We
>> need some additional work to educate users to use the safe version of the
>> functions if they rely on the return null behavior.
>> * turn on the strict mode by default.
>>
>> Hopefully we can finish it soon, in Spark 3.x.
>>
>> Thanks,
>> Wenchen
>>
>> On Sat, Aug 3, 2019 at 7:07 AM Matt Cheah  wrote:
>>
>>> *I agree that having both modes and let the user choose the one he/she
>>> wants is the best option (I don't see big arguments on this honestly). Once
>>> we have this, I don't see big differences on what is the default. What - I
>>> think - we still have to work on, is to go ahead with the "strict mode"
>>> work and provide a more convenient way for users to switch among the 2
>>> options. I mean: currently we have one flag for throwing exception on
>>> overflow for operations on decimals, one for doing the same for operations
>>> on other data types and probably going ahead we will have more. I think in
>>> the end we will need to 

Re: [Discuss] Follow ANSI SQL on table insertion

2019-08-05 Thread Ryan Blue
Wenchen, I don’t think we agree on what “strict mode” would mean. Marco is
talking about strict mode as an extension of the flag for throwing
exceptions on overflow for decimal operations. That is not ANSI SQL mode.

Also, we need more than ANSI SQL and runtime failure modes. For the
motivating problem of validating a write, we need a way to preserve the
analysis-time failure if types don’t match. That, combined with a runtime
strict mode, is the option that fails fast and guarantees data isn’t lost.

I agree that adding try_cast or safe methods is a good idea.

So here’s a revised set of steps:

   1. Finish ANSI SQL mode - but do not make it the default because it is
   not safe without an option to enable strict mode.
   2. Add strict mode for runtime calculations and turn it on by default
   3. Add a flag to control analysis time vs runtime failures (using strict
   mode or ANSI SQL mode) for v2 writes

The choice of whether runtime or analysis time failures should be the
default for v2 writes is worth a VOTE on this list. Once we agree on what
modes and options should be available, we can call a vote to build
consensus around a reasonable set of defaults, given that there are a lot
of varying opinions on this thread.

On Mon, Aug 5, 2019 at 12:49 AM Wenchen Fan  wrote:

> I think we need to clarify one thing before further discussion: the
> proposal is for the next release but not a long term solution.
>
> IMO the long term solution should be: completely follow SQL standard
> (store assignment rule + runtime exception), and provide a variant of
> functions that can return null instead of runtime exception. For example,
> the TRY_CAST in SQL server
> ,
> or a more general SAFE prefix of functions in Big Query
> 
> .
>
> This proposal is the first step to move forward to the long term solution:
> follow the SQL standard store assignment rule. It can help us prevent some
> table insertion queries that are very likely to fail, at compile time.
>
> The following steps in my mind are:
> * finish the "strict mode"(ANSI SQL mode). It now applies to arithmetic
> functions, but we need to apply it to more places like cast.
> * introduce the safe version of functions. The safe version always returns
> null for invalid input, no matter the strict mode is on or not. We need
> some additional work to educate users to use the safe version of the
> functions if they rely on the return null behavior.
> * turn on the strict mode by default.
>
> Hopefully we can finish it soon, in Spark 3.x.
>
> Thanks,
> Wenchen
>
> On Sat, Aug 3, 2019 at 7:07 AM Matt Cheah  wrote:
>
>> *I agree that having both modes and let the user choose the one he/she
>> wants is the best option (I don't see big arguments on this honestly). Once
>> we have this, I don't see big differences on what is the default. What - I
>> think - we still have to work on, is to go ahead with the "strict mode"
>> work and provide a more convenient way for users to switch among the 2
>> options. I mean: currently we have one flag for throwing exception on
>> overflow for operations on decimals, one for doing the same for operations
>> on other data types and probably going ahead we will have more. I think in
>> the end we will need to collect them all under an "umbrella" flag which
>> lets the user simply switch between strict and non-strict mode. I also
>> think that we will need to document this very well and give it particular
>> attention in our docs, maybe with a dedicated section, in order to provide
>> enough visibility on it to end users.*
>>
>>
>>
>> I’m +1 on adding a strict mode flag this way, but I’m undecided on
>> whether or not we want a separate flag for each of the arithmetic overflow
>> situations that could produce invalid results. My intuition is yes, because
>> different users have different levels of tolerance for different kinds of
>> errors. I’d expect these sorts of configurations to be set up at an
>> infrastructure level, e.g. to maintain consistent standards throughout a
>> whole organization.
>>
>>
>>
>> *From: *Gengliang Wang 
>> *Date: *Thursday, August 1, 2019 at 3:07 AM
>> *To: *Marco Gaido 
>> *Cc: *Wenchen Fan , Hyukjin Kwon <
>> gurwls...@gmail.com>, Russell Spitzer , Ryan
>> Blue , Reynold Xin , Matt Cheah <
>> mch...@palantir.com>, Takeshi Yamamuro , Spark
>> dev list 
>> *Subject: *Re: [Discuss] Follow ANSI SQL on table insertion
>>
>>
>>
>> Hi all,
>>
>>
>>
>> Let me explain a little bit on the proposal.
>>
>> By default, we follow the store assignment rules in table insertion. On
>> invalid casting, the result is null. It's better than the behavior in Spark
>> 2.x while keeping backward-compatibility. It is
>>
>> If users can't torrent the silently corrupting, they can enable the new
>> mode which throws runtime exceptions.
>>
>> The 

Re: Recognizing non-code contributions

2019-08-05 Thread Sean Owen
On Mon, Aug 5, 2019 at 3:50 AM Myrle Krantz  wrote:
> So... events coordinators?  I'd still make them committers.  I guess I'm 
> still struggling to understand what problem making people VIP's without 
> giving them committership is trying to solve.

We may just agree to disagree, which is fine, but I think the argument
is clear enough: such a person has zero need for the commit bit.
Turning it around, what are we trying to accomplish by giving said
person a commit bit? I know people say there's no harm, but I think
there is at least _some_ downside. We're widening access to change
software artifacts, the main thing that we put ASF process and checks
around for liability reasons. I know the point is trust, and said
person is likely to understand to never use the commit bit, but it
brings us back to the same place. I don't wish to convince anyone else
of my stance, though I do find it more logical, just that it's
reasonable within The Apache Way.


> It also just occurred to me this morning: There are actually other privileges 
> which go along with the "commit-bit" other than the ability to commit at will 
> to the project's repos: people who are committers get an Apache e-mail 
> address, and they get discounted entry to ApacheCon.  People who are 
> committers also get added to our committers mailing list, and are thus a 
> little easier to integrate into our foundation-wide efforts.
>
> To apply this to the example above, the Apache e-mail address can make it a 
> tad easier for an event coordinator to conduct official business for a 
> project.

Great points. Again if I'm making it up? a "VIP" should get an Apache
email address and discounts. Sure, why not put them on a committers@
list too for visibility.


> But that's not what I'm arguing.  I'm arguing that you can safely give commit 
> access to anyone for whom you're reasonably certain they will do no harm.  
> And your certainty of that can actually be much higher with non-coding 
> committers.  So if someone suggests a non-coding contributor for committer, 
> it should be fairly easy to say "yes".  Especially if you're on a project 
> like Spark where PMC ⊂ committers.

(Just to again be clear, we aren't talking only about 'non-coding'
committers. If it's shorthand for 'not contributing to docs/code in
the repo', we're on the same page.)
The case that started this is a corner case. The more interesting case
is, in fact, a docs-only contributor. That hasn't quite come up -- we
had a case of build- and config-only committer though. It's worth
keeping these arguments in mind for this more ambiguous hypothetical.
No objections in theory to making said person a committer, but in
practice it may be a harder case, possibly unnecessarily hard.

> For solving the early and easy recognition problem, I've heard about people 
> just saying "thank you" or making a "contributor of the month" honor.  That 
> kind of recognition doesn't necessarily have to be in the form of a status.

Sure, we do some of that on PRs, but probably need to do more. There
are some regular contributors doing good work, and I hope they feel
recognized by the fact they get more attention because of their track
record, but being explicit goes a long way.

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



unsubscribe

2019-08-05 Thread Duan,Bing
unsubscribe.

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



Re: Recognizing non-code contributions

2019-08-05 Thread Myrle Krantz
On Sun, Aug 4, 2019 at 10:06 PM Sean Owen  wrote:

> Oops, I also failed to copy dev@
>
> On Sun, Aug 4, 2019 at 3:06 PM Sean Owen  wrote:
> >
> > On Sun, Aug 4, 2019 at 1:54 PM Myrle Krantz  wrote:
> > >> No, I think the position here was docs-only contributors _could_ be
> > >> committers. The "only coders can be committers" idea was I think a
> > >> misreading and accidental strawman from the members@ thread.
> > >
> > > Can you be specific?  This status us supposed to serve as a form of
> recognition, right?  For who or what contributions?
> >
> > This "VIP" status is purely theoretical, but if I were making it up?
> > it would be for anyone, and any 'valuable' contribution to the
> > community of any kind.
> > Here I'm merely saying that nobody here suggested docs-only
> > contributors could never be committers. That seemed to be what the
> > members@ thread was taking on, as if the argument was that docs-only
> > contributors are at best these "VIPs". I want to make sure this isn't
> > predicated on a misunderstanding.
>

So... events coordinators?  I'd still make them committers.  I guess I'm
still struggling to understand what problem making people VIP's without
giving them committership is trying to solve.

It also just occurred to me this morning: There are actually other
privileges which go along with the "commit-bit" other than the ability to
commit at will to the project's repos: people who are committers get an
Apache e-mail address, and they get discounted entry to ApacheCon.  People
who are committers also get added to our committers mailing list, and are
thus a little easier to integrate into our foundation-wide efforts.

To apply this to the example above, the Apache e-mail address can make it a
tad easier for an event coordinator to conduct official business for a
project.

> > We have a saying at Apache: merit doesn't expire.  This is especially
> important for volunteer-based project work.  People's availability varies.
> If the barrier to re-entry is low, you're more likely to get contributions
> from these people.  Contributions that can have a particularly high value
> because they are informed by a longer history of the project.
> >
> > I know, I'm also here at Apache :) I guess I'm just showing we have
> > already taken a broad view of merit, consistent with Apache, when
> > there are even reasonable arguments about defining committer merit
> > more narrowly. This responds to the general notion I hear, that the
> > PMC here is somehow out of line and needs reeducation.
>

I don't think the PMC is "out of line".  I advocate for ease of access to
other projects too.  (For example:
https://twitter.com/myrleKrantz/status/1110475989941932038).  I do it
because I think it will be helpful to the projects.  But it absolutely is
true that every project decides for themselves what obstacles to access
they want to put in to place.  As long as they are being equitable while
doing that, the projects are within their rights.


> > > But the real question which I feel needs an answer here is: what's the
> harm you fear?  With modern source control, surely any harm is reversible?
> If you're worried about people doing things outside their "abilities", then
> maybe the real question you need to ask when choosing committers is just:
> do they know to ask for help first in fragile parts of the code?  You may
> find that someone who is doesn't see themselves as a coder is *less* likely
> to break code than other committers.
> >
> > This is a vital point. The argument is, what are you afraid of? how
> > much harm can a committer do? The logical extreme is to give everyone
> > commit access, which nobody appears to advocate. But where's the line?
> > What 'merit' are we talking about when we say people earn merit and
> > that's rewarded with committership? You said it: good judgment above
> > all. We have a committer that only run builds and config and doesn't
> > deal with code at all, and we agree that's good.
>

There are some people who will advocate that everyone should receive commit
access.  SVN is an example of this, and they had great success with that
approach.

But that's not what I'm arguing.  I'm arguing that you can safely give
commit access to anyone for whom you're reasonably certain they will do no
harm.  And your certainty of that can actually be much higher with
non-coding committers.  So if someone suggests a non-coding contributor for
committer, it should be fairly easy to say "yes".  Especially if you're on
a project like Spark where PMC ⊂ committers.


> > That is I'm not sure we (speaking broadly for Spark) disagree
> > substantively. Again the premise here was: somebody who hasn't and
> > doesn't want to touch anything about the project or ASF. I'm surprised
> > if there's an argument for making said person a committer; I thought
> > this was the easiest of cases. Still, I am quite sure ASF projects
> > have done this (and I disagreed). It's not crazy. But it's at least as
> > 

Re: [Discuss] Follow ANSI SQL on table insertion

2019-08-05 Thread Wenchen Fan
I think we need to clarify one thing before further discussion: the
proposal is for the next release but not a long term solution.

IMO the long term solution should be: completely follow SQL standard (store
assignment rule + runtime exception), and provide a variant of functions
that can return null instead of runtime exception. For example, the TRY_CAST
in SQL server
,
or a more general SAFE prefix of functions in Big Query

.

This proposal is the first step to move forward to the long term solution:
follow the SQL standard store assignment rule. It can help us prevent some
table insertion queries that are very likely to fail, at compile time.

The following steps in my mind are:
* finish the "strict mode"(ANSI SQL mode). It now applies to arithmetic
functions, but we need to apply it to more places like cast.
* introduce the safe version of functions. The safe version always returns
null for invalid input, no matter the strict mode is on or not. We need
some additional work to educate users to use the safe version of the
functions if they rely on the return null behavior.
* turn on the strict mode by default.

Hopefully we can finish it soon, in Spark 3.x.

Thanks,
Wenchen

On Sat, Aug 3, 2019 at 7:07 AM Matt Cheah  wrote:

> *I agree that having both modes and let the user choose the one he/she
> wants is the best option (I don't see big arguments on this honestly). Once
> we have this, I don't see big differences on what is the default. What - I
> think - we still have to work on, is to go ahead with the "strict mode"
> work and provide a more convenient way for users to switch among the 2
> options. I mean: currently we have one flag for throwing exception on
> overflow for operations on decimals, one for doing the same for operations
> on other data types and probably going ahead we will have more. I think in
> the end we will need to collect them all under an "umbrella" flag which
> lets the user simply switch between strict and non-strict mode. I also
> think that we will need to document this very well and give it particular
> attention in our docs, maybe with a dedicated section, in order to provide
> enough visibility on it to end users.*
>
>
>
> I’m +1 on adding a strict mode flag this way, but I’m undecided on whether
> or not we want a separate flag for each of the arithmetic overflow
> situations that could produce invalid results. My intuition is yes, because
> different users have different levels of tolerance for different kinds of
> errors. I’d expect these sorts of configurations to be set up at an
> infrastructure level, e.g. to maintain consistent standards throughout a
> whole organization.
>
>
>
> *From: *Gengliang Wang 
> *Date: *Thursday, August 1, 2019 at 3:07 AM
> *To: *Marco Gaido 
> *Cc: *Wenchen Fan , Hyukjin Kwon ,
> Russell Spitzer , Ryan Blue ,
> Reynold Xin , Matt Cheah ,
> Takeshi Yamamuro , Spark dev list <
> dev@spark.apache.org>
> *Subject: *Re: [Discuss] Follow ANSI SQL on table insertion
>
>
>
> Hi all,
>
>
>
> Let me explain a little bit on the proposal.
>
> By default, we follow the store assignment rules in table insertion. On
> invalid casting, the result is null. It's better than the behavior in Spark
> 2.x while keeping backward-compatibility. It is
>
> If users can't torrent the silently corrupting, they can enable the new
> mode which throws runtime exceptions.
>
> The proposal itself is quite complete. It satisfies different users to
> some degree.
>
>
>
> It is hard to avoid null in data processing anyway. For example,
>
> > select 2147483647 + 1
>
> 2147483647 is the max value of Int. And the result data type of pulsing
> two integers are supposed to be Integer type. Since the value of
> (2147483647 + 1) can't fit into Int, I think Spark return null or throw
> runtime exceptions in such case. (Someone can argue that we can always
> convert the result as wider types, but that's another topic about
> performance and DBMS behaviors)
>
>
>
> So, give a table t with an Int column, *checking data type with Up-Cast
> can't avoid possible null values in the following SQL*, as the result
> data type of (int_column_a + int_column_b) is int type.
>
> >  insert into t select int_column_a + int_column_b from tbl_a, tbl_b;
>
>
>
> Furthermore, if Spark uses Up-Cast and a user's existing ETL job failed
> because of that, what should he/she do then? I think he/she will try adding
> "cast" to queries first. Maybe a project for unifying data schema over all
> data sources has to be done later on if he/she has enough resource. The
> upgrade can be painful because of the strict rules of Up-Cast, while the
> user scenario might be able to tolerate converting Double to Decimal, or
> Timestamp to Date.
>
>
>
>
>
> Gengliang
>
>
>
> On Thu, Aug 1, 2019 at 4:55 PM Marco Gaido  wrote:
>
> Hi all,