Re: [DISCUSS] SPIP: FunctionCatalog

2021-02-09 Thread Wenchen Fan
FYI: the Presto UDF API
 also
takes individual parameters instead of the row parameter. I think this
direction at least worth a try so that we can see the performance
difference. It's also mentioned in the design doc as an alternative (Trino).

On Wed, Feb 10, 2021 at 10:18 AM Wenchen Fan  wrote:

> Hi Holden,
>
> As Hyukjin said, following existing designs is not the principle of DS v2
> API design. We should make sure the DS v2 API makes sense. AFAIK we didn't
> fully follow the catalog API design from Hive and I believe Ryan also
> agrees with it.
>
> I think the problem here is we were discussing some very detailed things
> without actual code. I'll implement my idea after the holiday and then we
> can have more effective discussions. We can also do benchmarks and get some
> real numbers.
>
> In the meantime, we can continue to discuss other parts of this proposal,
> and make a prototype if possible. Spark SQL has many active
> contributors/committers and this thread doesn't get much attention yet.
>
> On Wed, Feb 10, 2021 at 6:17 AM Hyukjin Kwon  wrote:
>
>> Just dropping a few lines. I remember that one of the goals in DSv2 is to
>> correct the mistakes we made in the current Spark codes.
>> It would not have much point if we will happen to just follow and mimic
>> what Spark currently does. It might just end up with another copy of Spark
>> APIs, e.g. Expression (internal) APIs. I sincerely would like to avoid this
>> I do believe we have been stuck mainly due to trying to come up with a
>> better design. We already have an ugly picture of the current Spark APIs to
>> draw a better bigger picture.
>>
>>
>> 2021년 2월 10일 (수) 오전 3:28, Holden Karau 님이 작성:
>>
>>> I think this proposal is a good set of trade-offs and has existed in the
>>> community for a long period of time. I especially appreciate how the design
>>> is focused on a minimal useful component, with future optimizations
>>> considered from a point of view of making sure it's flexible, but actual
>>> concrete decisions left for the future once we see how this API is used. I
>>> think if we try and optimize everything right out of the gate, we'll
>>> quickly get stuck (again) and not make any progress.
>>>
>>> On Mon, Feb 8, 2021 at 10:46 AM Ryan Blue  wrote:
>>>
 Hi everyone,

 I'd like to start a discussion for adding a FunctionCatalog interface
 to catalog plugins. This will allow catalogs to expose functions to Spark,
 similar to how the TableCatalog interface allows a catalog to expose
 tables. The proposal doc is available here:
 https://docs.google.com/document/d/1PLBieHIlxZjmoUB0ERF-VozCRJ0xw2j3qKvUNWpWA2U/edit

 Here's a high-level summary of some of the main design choices:
 * Adds the ability to list and load functions, not to create or modify
 them in an external catalog
 * Supports scalar, aggregate, and partial aggregate functions
 * Uses load and bind steps for better error messages and simpler
 implementations
 * Like the DSv2 table read and write APIs, it uses InternalRow to pass
 data
 * Can be extended using mix-in interfaces to add vectorization,
 codegen, and other future features

 There is also a PR with the proposed API:
 https://github.com/apache/spark/pull/24559/files

 Let's discuss the proposal here rather than on that PR, to get better
 visibility. Also, please take the time to read the proposal first. That
 really helps clear up misconceptions.



 --
 Ryan Blue

>>>
>>>
>>> --
>>> Twitter: https://twitter.com/holdenkarau
>>> Books (Learning Spark, High Performance Spark, etc.):
>>> https://amzn.to/2MaRAG9  
>>> YouTube Live Streams: https://www.youtube.com/user/holdenkarau
>>>
>>


Re: [DISCUSS] SPIP: FunctionCatalog

2021-02-09 Thread Wenchen Fan
Hi Holden,

As Hyukjin said, following existing designs is not the principle of DS v2
API design. We should make sure the DS v2 API makes sense. AFAIK we didn't
fully follow the catalog API design from Hive and I believe Ryan also
agrees with it.

I think the problem here is we were discussing some very detailed things
without actual code. I'll implement my idea after the holiday and then we
can have more effective discussions. We can also do benchmarks and get some
real numbers.

In the meantime, we can continue to discuss other parts of this proposal,
and make a prototype if possible. Spark SQL has many active
contributors/committers and this thread doesn't get much attention yet.

On Wed, Feb 10, 2021 at 6:17 AM Hyukjin Kwon  wrote:

> Just dropping a few lines. I remember that one of the goals in DSv2 is to
> correct the mistakes we made in the current Spark codes.
> It would not have much point if we will happen to just follow and mimic
> what Spark currently does. It might just end up with another copy of Spark
> APIs, e.g. Expression (internal) APIs. I sincerely would like to avoid this
> I do believe we have been stuck mainly due to trying to come up with a
> better design. We already have an ugly picture of the current Spark APIs to
> draw a better bigger picture.
>
>
> 2021년 2월 10일 (수) 오전 3:28, Holden Karau 님이 작성:
>
>> I think this proposal is a good set of trade-offs and has existed in the
>> community for a long period of time. I especially appreciate how the design
>> is focused on a minimal useful component, with future optimizations
>> considered from a point of view of making sure it's flexible, but actual
>> concrete decisions left for the future once we see how this API is used. I
>> think if we try and optimize everything right out of the gate, we'll
>> quickly get stuck (again) and not make any progress.
>>
>> On Mon, Feb 8, 2021 at 10:46 AM Ryan Blue  wrote:
>>
>>> Hi everyone,
>>>
>>> I'd like to start a discussion for adding a FunctionCatalog interface to
>>> catalog plugins. This will allow catalogs to expose functions to Spark,
>>> similar to how the TableCatalog interface allows a catalog to expose
>>> tables. The proposal doc is available here:
>>> https://docs.google.com/document/d/1PLBieHIlxZjmoUB0ERF-VozCRJ0xw2j3qKvUNWpWA2U/edit
>>>
>>> Here's a high-level summary of some of the main design choices:
>>> * Adds the ability to list and load functions, not to create or modify
>>> them in an external catalog
>>> * Supports scalar, aggregate, and partial aggregate functions
>>> * Uses load and bind steps for better error messages and simpler
>>> implementations
>>> * Like the DSv2 table read and write APIs, it uses InternalRow to pass
>>> data
>>> * Can be extended using mix-in interfaces to add vectorization, codegen,
>>> and other future features
>>>
>>> There is also a PR with the proposed API:
>>> https://github.com/apache/spark/pull/24559/files
>>>
>>> Let's discuss the proposal here rather than on that PR, to get better
>>> visibility. Also, please take the time to read the proposal first. That
>>> really helps clear up misconceptions.
>>>
>>>
>>>
>>> --
>>> Ryan Blue
>>>
>>
>>
>> --
>> Twitter: https://twitter.com/holdenkarau
>> Books (Learning Spark, High Performance Spark, etc.):
>> https://amzn.to/2MaRAG9  
>> YouTube Live Streams: https://www.youtube.com/user/holdenkarau
>>
>


Re: [VOTE] Release Spark 3.1.1 (RC2)

2021-02-09 Thread Yikun Jiang
+1, Tested build and basic feature on aarch64(ARM64) environment.

Regards,
Yikun


Yuming Wang  于2021年2月9日周二 下午8:24写道:

> +1. Tested a batch of queries with YARN client mode.
>
> On Tue, Feb 9, 2021 at 2:57 PM 郑瑞峰  wrote:
>
>> +1 (non-binding)
>>
>> Thank you, Hyukjin
>>
>>
>> -- 原始邮件 --
>> *发件人:* "Gengliang Wang" ;
>> *发送时间:* 2021年2月9日(星期二) 中午1:50
>> *收件人:* "Sean Owen";
>> *抄送:* "Hyukjin Kwon";"Yuming Wang"> >;"dev";
>> *主题:* Re: [VOTE] Release Spark 3.1.1 (RC2)
>>
>> +1
>>
>> On Tue, Feb 9, 2021 at 1:39 PM Sean Owen  wrote:
>>
>>> Same result as last time for me, +1. Tested with Java 11.
>>> I fixed the two issues without assignee; one was WontFix though.
>>>
>>> On Mon, Feb 8, 2021 at 7:43 PM Hyukjin Kwon  wrote:
>>>
 Let's set the assignees properly then. Shouldn't be a problem for the
 release.

 On Tue, 9 Feb 2021, 10:40 Yuming Wang,  wrote:

>
> Many tickets do not have correct assignee:
>
> https://issues.apache.org/jira/issues/?jql=project%20%3D%20SPARK%20AND%20status%20in%20(Resolved%2C%20Closed)%20AND%20fixVersion%20in%20(3.1.0%2C%203.1.1)%20AND%20(assignee%20is%20EMPTY%20or%20assignee%20%3D%20apachespark)
>
>
> On Tue, Feb 9, 2021 at 9:05 AM Hyukjin Kwon 
> wrote:
>
>> +1 (binding) from myself too.
>>
>> 2021년 2월 9일 (화) 오전 9:28, Kent Yao 님이 작성:
>>
>>>
>>> +1
>>>
>>> *Kent Yao *
>>> @ Data Science Center, Hangzhou Research Institute, NetEase Corp.
>>> *a spark enthusiast*
>>> *kyuubi is a unified
>>> multi-tenant JDBC interface for large-scale data processing and 
>>> analytics,
>>> built on top of Apache Spark .*
>>> *spark-authorizer A
>>> Spark SQL extension which provides SQL Standard Authorization for 
>>> **Apache
>>> Spark .*
>>> *spark-postgres  A
>>> library for reading data from and transferring data to Postgres / 
>>> Greenplum
>>> with Spark SQL and DataFrames, 10~100x faster.*
>>> *spark-func-extras A
>>> library that brings excellent and useful functions from various modern
>>> database management systems to Apache Spark .*
>>>
>>>
>>>
>>> On 02/9/2021 08:24,Hyukjin Kwon
>>>  wrote:
>>>
>>> Please vote on releasing the following candidate as Apache Spark
>>> version 3.1.1.
>>>
>>> The vote is open until February 15th 5PM PST and passes if a
>>> majority +1 PMC votes are cast, with a minimum of 3 +1 votes.
>>>
>>> Note that it is 7 days this time because it is a holiday season in
>>> several countries including South Korea (where I live), China etc., and 
>>> I
>>> would like to make sure people do not miss it because it is a holiday
>>> season.
>>>
>>> [ ] +1 Release this package as Apache Spark 3.1.1
>>> [ ] -1 Do not release this package because ...
>>>
>>> To learn more about Apache Spark, please see
>>> http://spark.apache.org/
>>>
>>> The tag to be voted on is v3.1.1-rc2 (commit
>>> cf0115ac2d60070399af481b14566f33d22ec45e):
>>> https://github.com/apache/spark/tree/v3.1.1-rc2
>>>
>>> The release files, including signatures, digests, etc. can be found
>>> at:
>>> 
>>> https://dist.apache.org/repos/dist/dev/spark/v3.1.1-rc2-bin/
>>>
>>> Signatures used for Spark RCs can be found in this file:
>>> https://dist.apache.org/repos/dist/dev/spark/KEYS
>>>
>>> The staging repository for this release can be found at:
>>>
>>> https://repository.apache.org/content/repositories/orgapachespark-1365
>>>
>>> The documentation corresponding to this release can be found at:
>>> https://dist.apache.org/repos/dist/dev/spark/v3.1.1-rc2-docs/
>>>
>>> The list of bug fixes going into 3.1.1 can be found at the following
>>> URL:
>>> https://s.apache.org/41kf2
>>>
>>> This release is using the release script of the tag v3.1.1-rc2.
>>>
>>> FAQ
>>>
>>> ===
>>> What happened to 3.1.0?
>>> ===
>>>
>>> There was a technical issue during Apache Spark 3.1.0 preparation,
>>> and it was discussed and decided to skip 3.1.0.
>>> Please see
>>> https://spark.apache.org/news/next-official-release-spark-3.1.1.html for
>>> more details.
>>>
>>> =
>>> How can I help test this release?
>>> =
>>>
>>> If you are a Spark user, you can help us test this release by taking
>>> an existing Spark workload and running on this release candidate,
>>> then
>>> reporting any regressions.

Re: [DISCUSS] SPIP: FunctionCatalog

2021-02-09 Thread Hyukjin Kwon
Just dropping a few lines. I remember that one of the goals in DSv2 is to
correct the mistakes we made in the current Spark codes.
It would not have much point if we will happen to just follow and mimic
what Spark currently does. It might just end up with another copy of Spark
APIs, e.g. Expression (internal) APIs. I sincerely would like to avoid this
I do believe we have been stuck mainly due to trying to come up with a
better design. We already have an ugly picture of the current Spark APIs to
draw a better bigger picture.


2021년 2월 10일 (수) 오전 3:28, Holden Karau 님이 작성:

> I think this proposal is a good set of trade-offs and has existed in the
> community for a long period of time. I especially appreciate how the design
> is focused on a minimal useful component, with future optimizations
> considered from a point of view of making sure it's flexible, but actual
> concrete decisions left for the future once we see how this API is used. I
> think if we try and optimize everything right out of the gate, we'll
> quickly get stuck (again) and not make any progress.
>
> On Mon, Feb 8, 2021 at 10:46 AM Ryan Blue  wrote:
>
>> Hi everyone,
>>
>> I'd like to start a discussion for adding a FunctionCatalog interface to
>> catalog plugins. This will allow catalogs to expose functions to Spark,
>> similar to how the TableCatalog interface allows a catalog to expose
>> tables. The proposal doc is available here:
>> https://docs.google.com/document/d/1PLBieHIlxZjmoUB0ERF-VozCRJ0xw2j3qKvUNWpWA2U/edit
>>
>> Here's a high-level summary of some of the main design choices:
>> * Adds the ability to list and load functions, not to create or modify
>> them in an external catalog
>> * Supports scalar, aggregate, and partial aggregate functions
>> * Uses load and bind steps for better error messages and simpler
>> implementations
>> * Like the DSv2 table read and write APIs, it uses InternalRow to pass
>> data
>> * Can be extended using mix-in interfaces to add vectorization, codegen,
>> and other future features
>>
>> There is also a PR with the proposed API:
>> https://github.com/apache/spark/pull/24559/files
>>
>> Let's discuss the proposal here rather than on that PR, to get better
>> visibility. Also, please take the time to read the proposal first. That
>> really helps clear up misconceptions.
>>
>>
>>
>> --
>> Ryan Blue
>>
>
>
> --
> Twitter: https://twitter.com/holdenkarau
> Books (Learning Spark, High Performance Spark, etc.):
> https://amzn.to/2MaRAG9  
> YouTube Live Streams: https://www.youtube.com/user/holdenkarau
>


Re: dataFrame.na.fill() fails for column with dot

2021-02-09 Thread Terry Kim
You probably need to update f. name here

as
well, but we can discuss further when you create a JIRA/PR.

Thanks,
Terry

On Tue, Feb 9, 2021 at 9:53 AM Terry Kim  wrote:

> Thanks Amandeep. This seems like a valid bug to me as quoted columns are
> not handled property for na.fill(). I think the better place to fix is in
> DataFrameNaFunctions.scala
> 
>  where
> "f.name" should be quoted.
>
> Could you create a JIRA
> ?
>
> Thanks,
> Terry
>


Re: [DISCUSS] SPIP: FunctionCatalog

2021-02-09 Thread Holden Karau
I think this proposal is a good set of trade-offs and has existed in the
community for a long period of time. I especially appreciate how the design
is focused on a minimal useful component, with future optimizations
considered from a point of view of making sure it's flexible, but actual
concrete decisions left for the future once we see how this API is used. I
think if we try and optimize everything right out of the gate, we'll
quickly get stuck (again) and not make any progress.

On Mon, Feb 8, 2021 at 10:46 AM Ryan Blue  wrote:

> Hi everyone,
>
> I'd like to start a discussion for adding a FunctionCatalog interface to
> catalog plugins. This will allow catalogs to expose functions to Spark,
> similar to how the TableCatalog interface allows a catalog to expose
> tables. The proposal doc is available here:
> https://docs.google.com/document/d/1PLBieHIlxZjmoUB0ERF-VozCRJ0xw2j3qKvUNWpWA2U/edit
>
> Here's a high-level summary of some of the main design choices:
> * Adds the ability to list and load functions, not to create or modify
> them in an external catalog
> * Supports scalar, aggregate, and partial aggregate functions
> * Uses load and bind steps for better error messages and simpler
> implementations
> * Like the DSv2 table read and write APIs, it uses InternalRow to pass data
> * Can be extended using mix-in interfaces to add vectorization, codegen,
> and other future features
>
> There is also a PR with the proposed API:
> https://github.com/apache/spark/pull/24559/files
>
> Let's discuss the proposal here rather than on that PR, to get better
> visibility. Also, please take the time to read the proposal first. That
> really helps clear up misconceptions.
>
>
>
> --
> Ryan Blue
>


-- 
Twitter: https://twitter.com/holdenkarau
Books (Learning Spark, High Performance Spark, etc.):
https://amzn.to/2MaRAG9  
YouTube Live Streams: https://www.youtube.com/user/holdenkarau


Re: [DISCUSS] SPIP: FunctionCatalog

2021-02-09 Thread Ryan Blue
I don’t think that using Invoke really works. The usability is poor if
something goes wrong and it can’t handle varargs or parameterized use cases
well. There isn’t a significant enough performance penalty for passing data
as a row to justify making the API much more difficult and less expressive.
I don’t think that it makes much sense to move forward with the idea.

More replies inline.

On Tue, Feb 9, 2021 at 2:37 AM Wenchen Fan  wrote:

> There’s also a massive performance penalty for the Invoke approach when
> falling back to non-codegen because the function is loaded and invoked each
> time eval is called. It is much cheaper to use a method in an interface.
>
> Can you elaborate? Using the row parameter or individual parameters
> shouldn't change the life cycle of the UDF instance.
>
The eval method looks up the method and invokes it every time using
reflection. That’s quite a bit slower than calling an interface method on
an UDF instance.

> Should they use String or UTF8String? What representations are supported
> and how will Spark detect and produce those representations?
>
> It's the same as InternalRow. We can just copy-paste the document of
> InternalRow to explain the corresponding java type for each data type.
>
My point is that having a single method signature that uses InternalRow and
is inherited from an interface is much easier for users and Spark. If a
user forgets to use UTF8String and writes a method with String instead,
then the UDF wouldn’t work. What then? Does Spark detect that the wrong
type was used? It would need to or else it would be difficult for a UDF
developer to tell what is wrong. And this is a runtime issue so it is
caught late.
-- 
Ryan Blue


Re: dataFrame.na.fill() fails for column with dot

2021-02-09 Thread Terry Kim
Thanks Amandeep. This seems like a valid bug to me as quoted columns are
not handled property for na.fill(). I think the better place to fix is in
DataFrameNaFunctions.scala

where
"f.name" should be quoted.

Could you create a JIRA
?

Thanks,
Terry


Re: Hyperparameter Optimization via Randomization

2021-02-09 Thread Phillip Henry
Hi, Sean.

I've added a comment in the new class to suggest a look at Hyperopt etc if
the user is using Python.

Anyway I've created a pull request:

https://github.com/apache/spark/pull/31535

and all tests, style checks etc pass. Wish me luck :)

And thanks for the support :)

Phillip



On Mon, Feb 8, 2021 at 4:12 PM Sean Owen  wrote:

> It seems pretty reasonable to me. If it's a pull request we can code
> review it.
> My only question is just, would it be better to tell people to use
> hyperopt, and how much better is this than implementing randomization on
> the grid.
> But the API change isn't significant so maybe just fine.
>
> On Mon, Feb 8, 2021 at 3:49 AM Phillip Henry 
> wrote:
>
>> Hi, Sean.
>>
>> I don't think sampling from a grid is a good idea as the min/max may lie
>> between grid points. Unconstrained random sampling avoids this problem. To
>> this end, I have an implementation at:
>>
>> https://github.com/apache/spark/compare/master...PhillHenry:master
>>
>> It is unit tested and does not change any already existing code.
>>
>> Totally get what you mean about Hyperopt but this is a pure JVM solution
>> that's fairly straightforward.
>>
>> Is it worth contributing?
>>
>> Thanks,
>>
>> Phillip
>>
>>
>>
>>
>>
>> On Sat, Jan 30, 2021 at 2:00 PM Sean Owen  wrote:
>>
>>> I was thinking ParamGridBuilder would have to change to accommodate a
>>> continuous range of values, and that's not hard, though other code wouldn't
>>> understand that type of value, like the existing simple grid builder.
>>> It's all possible just wondering if simply randomly sampling the grid is
>>> enough. That would be a simpler change, just a new method or argument.
>>>
>>> Yes part of it is that if you really want to search continuous spaces,
>>> hyperopt is probably even better, so how much do you want to put into
>>> Pyspark - something really simple sure.
>>> Not out of the question to do something more complex if it turns out to
>>> also be pretty simple.
>>>
>>> On Sat, Jan 30, 2021 at 4:42 AM Phillip Henry 
>>> wrote:
>>>
 Hi, Sean.

 Perhaps I don't understand. As I see it, ParamGridBuilder builds an
 Array[ParamMap]. What I am proposing is a new class that also builds an
 Array[ParamMap] via its build() method, so there would be no "change in the
 APIs". This new class would, of course, have methods that defined the
 search space (log, linear, etc) over which random values were chosen.

 Now, if this is too trivial to warrant the work and people prefer
 Hyperopt, then so be it. It might be useful for people not using Python but
 they can just roll-their-own, I guess.

 Anyway, looking forward to hearing what you think.

 Regards,

 Phillip



 On Fri, Jan 29, 2021 at 4:18 PM Sean Owen  wrote:

> I think that's a bit orthogonal - right now you can't specify
> continuous spaces. The straightforward thing is to allow random sampling
> from a big grid. You can create a geometric series of values to try, of
> course - 0.001, 0.01, 0.1, etc.
> Yes I get that if you're randomly choosing, you can randomly choose
> from a continuous space of many kinds. I don't know if it helps a lot vs
> the change in APIs (and continuous spaces don't make as much sense for 
> grid
> search)
> Of course it helps a lot if you're doing a smarter search over the
> space, like what hyperopt does. For that, I mean, one can just use
> hyperopt + Spark ML already if desired.
>
> On Fri, Jan 29, 2021 at 9:01 AM Phillip Henry 
> wrote:
>
>> Thanks, Sean! I hope to offer a PR next week.
>>
>> Not sure about a dependency on the grid search, though - but happy to
>> hear your thoughts. I mean, you might want to explore logarithmic space
>> evenly. For example,  something like "please search 1e-7 to 1e-4" leads 
>> to
>> a reasonably random sample being {3e-7, 2e-6, 9e-5}. These are (roughly)
>> evenly spaced in logarithmic space but not in linear space. So, saying 
>> what
>> fraction of a grid search to sample wouldn't make sense (unless the grid
>> was warped, of course).
>>
>> Does that make sense? It might be better for me to just write the
>> code as I don't think it would be very complicated.
>>
>> Happy to hear your thoughts.
>>
>> Phillip
>>
>>
>>
>> On Fri, Jan 29, 2021 at 1:47 PM Sean Owen  wrote:
>>
>>> I don't know of anyone working on that. Yes I think it could be
>>> useful. I think it might be easiest to implement by simply having some
>>> parameter to the grid search process that says what fraction of all
>>> possible combinations you want to randomly test.
>>>
>>> On Fri, Jan 29, 2021 at 5:52 AM Phillip Henry <
>>> londonjava...@gmail.com> wrote:
>>>
 Hi,

 I have no work at the moment so I was wondering if anybody would be
 

Re: [DISCUSS] Add RocksDB StateStore

2021-02-09 Thread Hyukjin Kwon
I mean I am okay with adding it as an external module for the extra
clarification :-)

2021년 2월 9일 (화) 오후 11:10, Hyukjin Kwon 님이 작성:

> I'm good with this too.
>
> 2021년 2월 9일 (화) 오후 4:16, DB Tsai 님이 작성:
>
>> +1 to add it as an external module so people can test it out and give
>> feedback easier.
>>
>> On Mon, Feb 8, 2021 at 10:22 PM Gabor Somogyi 
>> wrote:
>> >
>> > +1 adding it any way.
>> >
>> > On Mon, 8 Feb 2021, 21:54 Holden Karau,  wrote:
>> >>
>> >> +1 for an external module.
>> >>
>> >> On Mon, Feb 8, 2021 at 11:51 AM Cheng Su 
>> wrote:
>> >>>
>> >>> +1 for (2) adding to external module.
>> >>>
>> >>> I think this feature is useful and popular in practice, and option 2
>> is not conflict with previous concern for dependency.
>> >>>
>> >>>
>> >>>
>> >>> Thanks,
>> >>>
>> >>> Cheng Su
>> >>>
>> >>>
>> >>>
>> >>> From: Dongjoon Hyun 
>> >>> Date: Monday, February 8, 2021 at 10:39 AM
>> >>> To: Jacek Laskowski 
>> >>> Cc: Liang-Chi Hsieh , dev 
>> >>> Subject: Re: [DISCUSS] Add RocksDB StateStore
>> >>>
>> >>>
>> >>>
>> >>> Thank you, Liang-chi and all.
>> >>>
>> >>>
>> >>>
>> >>> +1 for (2) external module design because it can deliver the new
>> feature in a safe way.
>> >>>
>> >>>
>> >>>
>> >>> Bests,
>> >>>
>> >>> Dongjoon
>> >>>
>> >>>
>> >>>
>> >>> On Mon, Feb 8, 2021 at 9:00 AM Jacek Laskowski 
>> wrote:
>> >>>
>> >>> Hi,
>> >>>
>> >>>
>> >>>
>> >>> I'm "okay to add RocksDB StateStore as external module". See no
>> reason not to.
>> >>>
>> >>>
>> >>> Pozdrawiam,
>> >>>
>> >>> Jacek Laskowski
>> >>>
>> >>> 
>> >>>
>> >>> https://about.me/JacekLaskowski
>> >>>
>> >>> "The Internals Of" Online Books
>> >>>
>> >>> Follow me on https://twitter.com/jaceklaskowski
>> >>>
>> >>>
>> >>>
>> >>>
>> >>>
>> >>>
>> >>> On Tue, Feb 2, 2021 at 9:32 AM Liang-Chi Hsieh 
>> wrote:
>> >>>
>> >>> Hi devs,
>> >>>
>> >>> In Spark structured streaming, we need state store for state
>> management for
>> >>> stateful operators such streaming aggregates, joins, etc. We have one
>> and
>> >>> only one state store implementation now. It is in-memory hashmap
>> which was
>> >>> backed up in HDFS complaint file system at the end of every
>> micro-batch.
>> >>>
>> >>> As it basically uses in-memory map to store states, memory
>> consumption is a
>> >>> serious issue and state store size is limited by the size of the
>> executor
>> >>> memory. Moreover, state store using more memory means it may impact
>> the
>> >>> performance of task execution that requires memory too.
>> >>>
>> >>> Internally we see more streaming applications that requires large
>> state in
>> >>> stateful operations. For such requirements, we need a StateStore not
>> rely on
>> >>> memory to store states.
>> >>>
>> >>> This seems to be also true externally as several other major streaming
>> >>> frameworks already use RocksDB for state management. RocksDB is an
>> embedded
>> >>> DB and streaming engines can use it to store state instead of memory
>> >>> storage.
>> >>>
>> >>> So seems to me, it is proven to be good choice for large state usage.
>> But
>> >>> Spark SS still lacks of a built-in state store for the requirement.
>> >>>
>> >>> Previously there was one attempt SPARK-28120 to add RocksDB
>> StateStore into
>> >>> Spark SS. IIUC, it was pushed back due to two concerns: extra code
>> >>> maintenance cost and it introduces RocksDB dependency.
>> >>>
>> >>> For the first concern, as more users require to use the feature, it
>> should
>> >>> be highly used code in SS and more developers will look at it. For
>> second
>> >>> one, we propose (SPARK-34198) to add it as an external module to
>> relieve the
>> >>> dependency concern.
>> >>>
>> >>> Because it was pushed back previously, I'm going to raise this
>> discussion to
>> >>> know what people think about it now, in advance of submitting any
>> code.
>> >>>
>> >>> I think there might be some possible opinions:
>> >>>
>> >>> 1. okay to add RocksDB StateStore into sql core module
>> >>> 2. not okay for 1, but okay to add RocksDB StateStore as external
>> module
>> >>> 3. either 1 or 2 is okay
>> >>> 4. not okay to add RocksDB StateStore, no matter into sql core or as
>> >>> external module
>> >>>
>> >>> Please let us know if you have some thoughts.
>> >>>
>> >>> Thank you.
>> >>>
>> >>> Liang-Chi Hsieh
>> >>>
>> >>>
>> >>>
>> >>>
>> >>> --
>> >>> Sent from: http://apache-spark-developers-list.1001551.n3.nabble.com/
>> >>>
>> >>> -
>> >>> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>> >>
>> >>
>> >>
>> >> --
>> >> Twitter: https://twitter.com/holdenkarau
>> >> Books (Learning Spark, High Performance Spark, etc.):
>> https://amzn.to/2MaRAG9
>> >> YouTube Live Streams: https://www.youtube.com/user/holdenkarau
>>
>>
>>
>> --
>> Sincerely,
>>
>> DB Tsai
>> --
>> Web: https://www.dbtsai.com
>> PGP Key ID: 42E5B25A8F7A82C1
>>
>> 

Re: [DISCUSS] Add RocksDB StateStore

2021-02-09 Thread Hyukjin Kwon
I'm good with this too.

2021년 2월 9일 (화) 오후 4:16, DB Tsai 님이 작성:

> +1 to add it as an external module so people can test it out and give
> feedback easier.
>
> On Mon, Feb 8, 2021 at 10:22 PM Gabor Somogyi 
> wrote:
> >
> > +1 adding it any way.
> >
> > On Mon, 8 Feb 2021, 21:54 Holden Karau,  wrote:
> >>
> >> +1 for an external module.
> >>
> >> On Mon, Feb 8, 2021 at 11:51 AM Cheng Su 
> wrote:
> >>>
> >>> +1 for (2) adding to external module.
> >>>
> >>> I think this feature is useful and popular in practice, and option 2
> is not conflict with previous concern for dependency.
> >>>
> >>>
> >>>
> >>> Thanks,
> >>>
> >>> Cheng Su
> >>>
> >>>
> >>>
> >>> From: Dongjoon Hyun 
> >>> Date: Monday, February 8, 2021 at 10:39 AM
> >>> To: Jacek Laskowski 
> >>> Cc: Liang-Chi Hsieh , dev 
> >>> Subject: Re: [DISCUSS] Add RocksDB StateStore
> >>>
> >>>
> >>>
> >>> Thank you, Liang-chi and all.
> >>>
> >>>
> >>>
> >>> +1 for (2) external module design because it can deliver the new
> feature in a safe way.
> >>>
> >>>
> >>>
> >>> Bests,
> >>>
> >>> Dongjoon
> >>>
> >>>
> >>>
> >>> On Mon, Feb 8, 2021 at 9:00 AM Jacek Laskowski 
> wrote:
> >>>
> >>> Hi,
> >>>
> >>>
> >>>
> >>> I'm "okay to add RocksDB StateStore as external module". See no reason
> not to.
> >>>
> >>>
> >>> Pozdrawiam,
> >>>
> >>> Jacek Laskowski
> >>>
> >>> 
> >>>
> >>> https://about.me/JacekLaskowski
> >>>
> >>> "The Internals Of" Online Books
> >>>
> >>> Follow me on https://twitter.com/jaceklaskowski
> >>>
> >>>
> >>>
> >>>
> >>>
> >>>
> >>> On Tue, Feb 2, 2021 at 9:32 AM Liang-Chi Hsieh 
> wrote:
> >>>
> >>> Hi devs,
> >>>
> >>> In Spark structured streaming, we need state store for state
> management for
> >>> stateful operators such streaming aggregates, joins, etc. We have one
> and
> >>> only one state store implementation now. It is in-memory hashmap which
> was
> >>> backed up in HDFS complaint file system at the end of every
> micro-batch.
> >>>
> >>> As it basically uses in-memory map to store states, memory consumption
> is a
> >>> serious issue and state store size is limited by the size of the
> executor
> >>> memory. Moreover, state store using more memory means it may impact the
> >>> performance of task execution that requires memory too.
> >>>
> >>> Internally we see more streaming applications that requires large
> state in
> >>> stateful operations. For such requirements, we need a StateStore not
> rely on
> >>> memory to store states.
> >>>
> >>> This seems to be also true externally as several other major streaming
> >>> frameworks already use RocksDB for state management. RocksDB is an
> embedded
> >>> DB and streaming engines can use it to store state instead of memory
> >>> storage.
> >>>
> >>> So seems to me, it is proven to be good choice for large state usage.
> But
> >>> Spark SS still lacks of a built-in state store for the requirement.
> >>>
> >>> Previously there was one attempt SPARK-28120 to add RocksDB StateStore
> into
> >>> Spark SS. IIUC, it was pushed back due to two concerns: extra code
> >>> maintenance cost and it introduces RocksDB dependency.
> >>>
> >>> For the first concern, as more users require to use the feature, it
> should
> >>> be highly used code in SS and more developers will look at it. For
> second
> >>> one, we propose (SPARK-34198) to add it as an external module to
> relieve the
> >>> dependency concern.
> >>>
> >>> Because it was pushed back previously, I'm going to raise this
> discussion to
> >>> know what people think about it now, in advance of submitting any code.
> >>>
> >>> I think there might be some possible opinions:
> >>>
> >>> 1. okay to add RocksDB StateStore into sql core module
> >>> 2. not okay for 1, but okay to add RocksDB StateStore as external
> module
> >>> 3. either 1 or 2 is okay
> >>> 4. not okay to add RocksDB StateStore, no matter into sql core or as
> >>> external module
> >>>
> >>> Please let us know if you have some thoughts.
> >>>
> >>> Thank you.
> >>>
> >>> Liang-Chi Hsieh
> >>>
> >>>
> >>>
> >>>
> >>> --
> >>> Sent from: http://apache-spark-developers-list.1001551.n3.nabble.com/
> >>>
> >>> -
> >>> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
> >>
> >>
> >>
> >> --
> >> Twitter: https://twitter.com/holdenkarau
> >> Books (Learning Spark, High Performance Spark, etc.):
> https://amzn.to/2MaRAG9
> >> YouTube Live Streams: https://www.youtube.com/user/holdenkarau
>
>
>
> --
> Sincerely,
>
> DB Tsai
> --
> Web: https://www.dbtsai.com
> PGP Key ID: 42E5B25A8F7A82C1
>
> -
> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>
>


Re: [VOTE] Release Spark 3.1.1 (RC2)

2021-02-09 Thread Yuming Wang
+1. Tested a batch of queries with YARN client mode.

On Tue, Feb 9, 2021 at 2:57 PM 郑瑞峰  wrote:

> +1 (non-binding)
>
> Thank you, Hyukjin
>
>
> -- 原始邮件 --
> *发件人:* "Gengliang Wang" ;
> *发送时间:* 2021年2月9日(星期二) 中午1:50
> *收件人:* "Sean Owen";
> *抄送:* "Hyukjin Kwon";"Yuming Wang" >;"dev";
> *主题:* Re: [VOTE] Release Spark 3.1.1 (RC2)
>
> +1
>
> On Tue, Feb 9, 2021 at 1:39 PM Sean Owen  wrote:
>
>> Same result as last time for me, +1. Tested with Java 11.
>> I fixed the two issues without assignee; one was WontFix though.
>>
>> On Mon, Feb 8, 2021 at 7:43 PM Hyukjin Kwon  wrote:
>>
>>> Let's set the assignees properly then. Shouldn't be a problem for the
>>> release.
>>>
>>> On Tue, 9 Feb 2021, 10:40 Yuming Wang,  wrote:
>>>

 Many tickets do not have correct assignee:

 https://issues.apache.org/jira/issues/?jql=project%20%3D%20SPARK%20AND%20status%20in%20(Resolved%2C%20Closed)%20AND%20fixVersion%20in%20(3.1.0%2C%203.1.1)%20AND%20(assignee%20is%20EMPTY%20or%20assignee%20%3D%20apachespark)


 On Tue, Feb 9, 2021 at 9:05 AM Hyukjin Kwon 
 wrote:

> +1 (binding) from myself too.
>
> 2021년 2월 9일 (화) 오전 9:28, Kent Yao 님이 작성:
>
>>
>> +1
>>
>> *Kent Yao *
>> @ Data Science Center, Hangzhou Research Institute, NetEase Corp.
>> *a spark enthusiast*
>> *kyuubi is a unified
>> multi-tenant JDBC interface for large-scale data processing and 
>> analytics,
>> built on top of Apache Spark .*
>> *spark-authorizer A
>> Spark SQL extension which provides SQL Standard Authorization for 
>> **Apache
>> Spark .*
>> *spark-postgres  A
>> library for reading data from and transferring data to Postgres / 
>> Greenplum
>> with Spark SQL and DataFrames, 10~100x faster.*
>> *spark-func-extras A
>> library that brings excellent and useful functions from various modern
>> database management systems to Apache Spark .*
>>
>>
>>
>> On 02/9/2021 08:24,Hyukjin Kwon
>>  wrote:
>>
>> Please vote on releasing the following candidate as Apache Spark
>> version 3.1.1.
>>
>> The vote is open until February 15th 5PM PST and passes if a majority
>> +1 PMC votes are cast, with a minimum of 3 +1 votes.
>>
>> Note that it is 7 days this time because it is a holiday season in
>> several countries including South Korea (where I live), China etc., and I
>> would like to make sure people do not miss it because it is a holiday
>> season.
>>
>> [ ] +1 Release this package as Apache Spark 3.1.1
>> [ ] -1 Do not release this package because ...
>>
>> To learn more about Apache Spark, please see http://spark.apache.org/
>>
>> The tag to be voted on is v3.1.1-rc2 (commit
>> cf0115ac2d60070399af481b14566f33d22ec45e):
>> https://github.com/apache/spark/tree/v3.1.1-rc2
>>
>> The release files, including signatures, digests, etc. can be found
>> at:
>> 
>> https://dist.apache.org/repos/dist/dev/spark/v3.1.1-rc2-bin/
>>
>> Signatures used for Spark RCs can be found in this file:
>> https://dist.apache.org/repos/dist/dev/spark/KEYS
>>
>> The staging repository for this release can be found at:
>> https://repository.apache.org/content/repositories/orgapachespark-1365
>>
>> The documentation corresponding to this release can be found at:
>> https://dist.apache.org/repos/dist/dev/spark/v3.1.1-rc2-docs/
>>
>> The list of bug fixes going into 3.1.1 can be found at the following
>> URL:
>> https://s.apache.org/41kf2
>>
>> This release is using the release script of the tag v3.1.1-rc2.
>>
>> FAQ
>>
>> ===
>> What happened to 3.1.0?
>> ===
>>
>> There was a technical issue during Apache Spark 3.1.0 preparation,
>> and it was discussed and decided to skip 3.1.0.
>> Please see
>> https://spark.apache.org/news/next-official-release-spark-3.1.1.html for
>> more details.
>>
>> =
>> How can I help test this release?
>> =
>>
>> If you are a Spark user, you can help us test this release by taking
>> an existing Spark workload and running on this release candidate, then
>> reporting any regressions.
>>
>> If you're working in PySpark you can set up a virtual env and install
>> the current RC via "pip install
>> https://dist.apache.org/repos/dist/dev/spark/v3.1.1-rc2-bin/pyspark-3.1.1.tar.gz
>> "
>> and see if anything important breaks.
>> In the 

Re: [DISCUSS] SPIP: FunctionCatalog

2021-02-09 Thread Wenchen Fan
Hi Ryan,

Sorry if I didn't make it clear. I was referring to implementing UDF using
codegen, not calling the UDF with codegen or not. Calling UDF is Spark's
job and it doesn't matter if the UDF API uses row or individual parameters,
as you said. My point is, it's a bad idea to follow the Expression API to
separate the interpreted and codegen code paths in the UDF API, as it adds
implementation complexity and is error-prone (people need to keep
interpreted and codegen in sync).

You made a good point that the Invoke approach can lead to methods
expansion. It's a common problem in Java if you want to specialize the
generic type, and Scala can mitigate this a bit using syntax sugar (type
specification
).
Varargs is a problem and in that case I agree the row parameter is
better. The current Scala/Java UDF doesn't support varargs and I'm not sure
how common it is. The UDF can take struct type inputs, which kind of
supports varargs as people can do SELECT my_udf(struct(1, 'abc', ...)).

> And, the Invoke approach has a performance penalty when existing rows
could be simply projected using a wrapper.

This only happens in the interpreted code path. When the query falls back
to interpreted execution, it'll be very slow and a little performance
penalty of UDF doesn't really matter.

> There’s also a massive performance penalty for the Invoke approach when
falling back to non-codegen because the function is loaded and invoked each
time eval is called. It is much cheaper to use a method in an interface.

Can you elaborate? Using the row parameter or individual parameters
shouldn't change the life cycle of the UDF instance.

> Should they use String or UTF8String? What representations are supported
and how will Spark detect and produce those representations?

It's the same as InternalRow. We can just copy-paste the document of
InternalRow to explain the corresponding java type for each data type.

On Tue, Feb 9, 2021 at 6:28 AM Ryan Blue  wrote:

> Wenchen,
>
> There are a few issues with the Invoke approach, and I don’t think that
> it is really much better for the additional complexity of the API.
>
> First I think that you’re confusing codegen to call a function with
> codegen to implement a function. The non-goal refers to supporting codegen
> to *implement* a UDF. That’s what could have differences between the
> called version and generated version. But the Invoke option isn’t any
> different in that case because Invoke codegen is only used to call a
> method, and we can codegen int result = udfName(x, y) just like we can
> codegen int result = udfName(row).
>
> The Invoke approach also has a problem with expressiveness. Consider a map
> function that builds a map from its inputs as key/value pairs: map("x", r
> * cos(theta), "y", r * sin(theta)). If this requires a defined Java
> function, then there must be lots of implementations for different numbers
> of pairs, for different types, etc:
>
> public MapData buildMap(String k1, int v1);
> ...
> public MapData buildMap(String k1, long v1);
> ...
> public MapData buildMap(String k1, float v1);
> ...
> public MapData buildMap(String k1, double v1);
> public MapData buildMap(String k1, double v1, String k2, double v2);
> public MapData buildMap(String k1, double v1, String k2, double v2, String 
> k3, double v3);
> ...
>
> Clearly, this and many other use cases would fall back to varargs instead.
> In that case, there is little benefit to using invoke because all of the
> arguments will get collected into an Object[] anyway. That’s basically
> the same thing as using a row object, just without convenience functions
> that return specific types like getString, forcing implementations to
> cast instead. And, the Invoke approach has a performance *penalty* when
> existing rows could be simply projected using a wrapper.
>
> There’s also a massive performance penalty for the Invoke approach when
> falling back to non-codegen because the function is loaded and invoked each
> time eval is called. It is much cheaper to use a method in an interface.
>
> Next, the Invoke approach is much more complicated for implementers to
> use. Should they use String or UTF8String? What representations are
> supported and how will Spark detect and produce those representations? What
> if a function uses both String *and* UTF8String? Will Spark detect this
> for each parameter? Having one or two functions called by Spark is much
> easier to maintain in Spark and avoid a lot of debugging headaches when
> something goes wrong.
>
> On Mon, Feb 8, 2021 at 12:00 PM Wenchen Fan  wrote:
>
> This is a very important feature, thanks for working on it!
>>
>> Spark uses codegen by default, and it's a bit unfortunate to see that
>> codegen support is treated as a non-goal. I think it's better to not ask
>> the UDF implementations to provide two different code paths for interpreted
>> evaluation and 

dataFrame.na.fill() fails for column with dot

2021-02-09 Thread Amandeep Sharma
Hi guys,
Apologies for the long mail.

I am running below code snippet

import org.apache.spark.sql.SparkSession
object ColumnNameWithDot {
 def main(args: Array[String]): Unit = {

 val spark = SparkSession.builder.appName("Simple Application")
 .config("spark.master", "local").getOrCreate()

 spark.sparkContext.setLogLevel("OFF")

 import spark.implicits._
 val df = Seq(("abc", 23), ("def", 44), (null, 9)).toDF("ColWith.Dot",
"Col")
 df.na.fill(Map("`ColWith.Dot`" -> "n/a")).show()

 }
}

and it is failing with error
Exception in thread "main" org.apache.spark.sql.AnalysisException: Cannot
resolve column name "ColWith.Dot" among (ColWith.Dot, Col);

I checked that code-fix were made for the similar issue, found
https://issues.apache.org/jira/browse/SPARK-19473; but none of them fixed
all cases.

I debugged the code below are the observations

   1. In org.apache.spark.sql.DataFrameNaFunctions.fillMap(values:
   Seq[(String, Any)]) the df.resolve(colName) call succeeds, since column
   name is quoted with back tick it resolves the column
   2. val projections = df.schema.fields.map {
   ...
   ...
   }.getOrElse(df.col(f.name)) fails since resolved column name is not
   quoted with backtick

Problem lies in the
org.apache.spark.sql.catalyst.expressions
resolve(nameParts: Seq[String], resolver: Resolver): Option[NamedExpression]

where the comment says we try to resolve it as a column.

// If none of attributes match database.table.column pattern or
// `table.column` pattern, we try to resolve it as a column.
val (candidates, nestedFields) = matches match {
case (Seq(), _) =>
val name = nameParts.head
val attributes = collectMatches(name,
direct.get(name.toLowerCase(Locale.ROOT)))
(attributes, nameParts.tail)
case _ => matches
}

should be changed to

// If none of attributes match database.table.column pattern or
// `table.column` pattern, we try to resolve it as a column.
val (candidates, nestedFields) = matches match {
case (Seq(), _) =>
val name = nameParts.mkString(".")
val attributes = collectMatches(name,
direct.get(name.toLowerCase(Locale.ROOT)))
(attributes, Seq.empty)
case _ => matches
}
git diff is as below

-  val name = nameParts.head
+  val name = nameParts.mkString(".")
   val attributes = collectMatches(name,
direct.get(name.toLowerCase(Locale.ROOT)))
-  (attributes, nameParts.tail)
+  (attributes, Seq.empty)

I tested this change, there is no longer need to use backtick with columns
having dot in the name.
Can this change be merged?

Regards,
Amandeep


Re: [VOTE] Release Spark 3.1.1 (RC2)

2021-02-09 Thread Jungtaek Lim
+1 (non-binding)

* verified signatures
* built custom distribution with enabling kubernetes & hadoop-cloud profile
* built custom docker image from dist
* ran applications "rate to kafka" & "kafka to kafka" on k8s cluster (local
k3s)

Thanks for driving the release!

Jungtaek Lim (HeartSaVioR)

On Tue, Feb 9, 2021 at 4:41 PM Cheng Su  wrote:

> +1 for this release candidate.
>
>
>
> Thanks,
>
> Cheng Su
>
>
>
> *From: *郑瑞峰 
> *Date: *Monday, February 8, 2021 at 10:58 PM
> *To: *Gengliang Wang , Sean Owen 
> *Cc: *gurwls223 , Yuming Wang ,
> dev 
> *Subject: *回复: [VOTE] Release Spark 3.1.1 (RC2)
>
>
>
> +1 (non-binding)
>
>
>
> Thank you, Hyukjin
>
>
>
>
>
> -- 原始邮件 --
>
> *发件人**:* "Gengliang Wang" ;
>
> *发送时间**:* 2021年2月9日(星期二) 中午1:50
>
> *收件人**:* "Sean Owen";
>
> *抄送**:* "Hyukjin Kwon";"Yuming Wang" >;"dev";
>
> *主题**:* Re: [VOTE] Release Spark 3.1.1 (RC2)
>
>
>
> +1
>
>
>
> On Tue, Feb 9, 2021 at 1:39 PM Sean Owen  wrote:
>
> Same result as last time for me, +1. Tested with Java 11.
>
> I fixed the two issues without assignee; one was WontFix though.
>
>
>
> On Mon, Feb 8, 2021 at 7:43 PM Hyukjin Kwon  wrote:
>
> Let's set the assignees properly then. Shouldn't be a problem for the
> release.
>
>
>
> On Tue, 9 Feb 2021, 10:40 Yuming Wang,  wrote:
>
>
>
> Many tickets do not have correct assignee:
>
>
> https://issues.apache.org/jira/issues/?jql=project%20%3D%20SPARK%20AND%20status%20in%20(Resolved%2C%20Closed)%20AND%20fixVersion%20in%20(3.1.0%2C%203.1.1)%20AND%20(assignee%20is%20EMPTY%20or%20assignee%20%3D%20apachespark)
> 
>
>
>
>
>
> On Tue, Feb 9, 2021 at 9:05 AM Hyukjin Kwon  wrote:
>
> +1 (binding) from myself too.
>
>
>
> 2021년 2월 9일 (화) 오전 9:28, Kent Yao 님이 작성:
>
>
>
> +1
>
>
>
> *Kent Yao *
>
> @ Data Science Center, Hangzhou Research Institute, NetEase Corp.
>
> *a spark enthusiast*
>
> *kyuubi **is a unified
> multi-tenant JDBC interface for large-scale data processing and analytics,
> built on top of **Apache Spark **.*
> *spark-authorizer **A Spark
> SQL extension which provides SQL Standard Authorization for **Apache
> Spark **.*
> *spark-postgres  **A library
> for reading data from and transferring data to Postgres / Greenplum with
> Spark SQL and DataFrames, 10~100x faster.*
> *spark-func-extras **A
> library that brings excellent and useful functions from various modern
> database management systems to **Apache Spark *
> *.*
>
>
>
>
>
> On 02/9/2021 08:24,Hyukjin Kwon 
> wrote:
>
> Please vote on releasing the following candidate as Apache Spark version
> 3.1.1.
>
>
>
> The vote is open until February 15th 5PM PST and passes if a majority +1
> PMC votes are cast, with a minimum of 3 +1 votes.
>
>
>
> Note that it is 7 days this time because it is a holiday season in several
> countries including South Korea (where I live), China etc., and I would
> like to make sure people do not miss it because it is a holiday season.
>
>
>
> [ ] +1 Release this package as Apache Spark 3.1.1
>
> [ ] -1 Do not release this package because ...
>
>
>
> To learn more about Apache Spark, please see http://spark.apache.org/
>
>
>
> The tag to be voted on is v3.1.1-rc2 (commit
> cf0115ac2d60070399af481b14566f33d22ec45e):
>
> https://github.com/apache/spark/tree/v3.1.1-rc2
>
>
>
> The release files, including signatures, digests, etc. can be found at:
>
> https://dist.apache.org/repos/dist/dev/spark/v3.1.1-rc2-bin/
>
> Signatures used for Spark RCs can be found in this file:
>
> https://dist.apache.org/repos/dist/dev/spark/KEYS
>
>
>
> The staging repository for this release can be found at:
>
> https://repository.apache.org/content/repositories/orgapachespark-1365
>
>
>
> The documentation corresponding to this release can be found at:
>
> https://dist.apache.org/repos/dist/dev/spark/v3.1.1-rc2-docs/
>
>
>
> The list of bug fixes going into 3.1.1 can be found at the following URL:
>
> https://s.apache.org/41kf2
>
>
>
> This release is using the release script of the tag v3.1.1-rc2.
>
>
>
> FAQ
>
>
>
> ===
> What happened to 3.1.0?
>
> ===
>
> There was a technical issue during Apache Spark 3.1.0 preparation, and it
> was discussed and decided to skip 3.1.0.
> Please see
> https://spark.apache.org/news/next-official-release-spark-3.1.1.html for
> more details.
>
> =
>
> How can I help test this release?
>
> =
>
>
>
> If you are a Spark user, you can help us test this release by taking
>
> an existing Spark workload and running on this