Re: [DISCUSS] Set default planner for SQL Client to Blink planner in 1.10 release

2020-01-03 Thread Leonard Xu
+1 for this. 
We bring many SQL/API features and enhance stability in 1.10 release, and 
almost all of them happens in Blink planner.
SQL CLI is the most convenient entrypoint for me, I believe many users will 
have a better experience If we set Blink planner as default planner.

Best,
Leonard

> 在 2020年1月3日,15:16,Terry Wang  写道:
> 
> Since what blink planner can do is a superset of flink planner, big +1 for 
> changing the default planner to Blink planner from my side.
> 
> Best,
> Terry Wang
> 
> 
> 
>> 2020年1月3日 15:00,Jark Wu  写道:
>> 
>> Hi everyone,
>> 
>> In 1.10 release, Flink SQL supports many awesome features and improvements,
>> including:
>> - support watermark statement and computed column in DDL
>> - fully support all data types in Hive
>> - Batch SQL performance improvements (TPC-DS 7x than Hive MR)
>> - support INSERT OVERWRITE and INSERT PARTITION
>> 
>> However, all the features and improvements are only avaiable in Blink
>> planner, not in Old planner.
>> There are also some other features are limited in Blink planner, e.g.
>> Dimension Table Join [1],
>> TopN [2], Deduplicate [3], streaming aggregates optimization [4], and so on.
>> 
>> But Old planner is still the default planner in Table API & SQL. It is
>> frustrating for users to set
>> to blink planner manually when every time start a SQL CLI. And it's
>> surprising to see unsupported
>> exception if they trying out the new features but not switch planner.
>> 
>> SQL CLI is a very important entrypoint for trying out new feautures and
>> prototyping for users.
>> In order to give new planner more exposures, I would like to suggest to set
>> default planner
>> for SQL Client to Blink planner before 1.10 release.
>> 
>> The approach is just changing the default SQL CLI yaml configuration[5]. In
>> this way, the existing
>> environment is still compatible and unaffected.
>> 
>> Changing the default planner for the whole Table API & SQL is another topic
>> and is out of scope of this discussion.
>> 
>> What do you think?
>> 
>> Best,
>> Jark
>> 
>> [1]:
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/joins.html#join-with-a-temporal-table
>> [2]:
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#top-n
>> [3]:
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication
>> [4]:
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/tuning/streaming_aggregation_optimization.html
>> [5]:
>> https://github.com/apache/flink/blob/master/flink-table/flink-sql-client/conf/sql-client-defaults.yaml#L100
> 



Re: [DISCUSS] Set default planner for SQL Client to Blink planner in 1.10 release

2020-01-03 Thread Jingsong Li
Hi Jark,

+1 for default blink planner in SQL-CLI.
I believe this new planner can be put into practice in production.
We've worked hard for nearly a year, but the old planner didn't move on.

And I'd like to cc to u...@flink.apache.org.
If anyone finds that blink planner has any significant defects and has a
larger regression than the old planner, please let us know. We will be very
grateful.

Best,
Jingsong Lee

On Fri, Jan 3, 2020 at 4:14 PM Leonard Xu  wrote:

> +1 for this.
> We bring many SQL/API features and enhance stability in 1.10 release, and
> almost all of them happens in Blink planner.
> SQL CLI is the most convenient entrypoint for me, I believe many users
> will have a better experience If we set Blink planner as default planner.
>
> Best,
> Leonard
>
> > 在 2020年1月3日,15:16,Terry Wang  写道:
> >
> > Since what blink planner can do is a superset of flink planner, big +1
> for changing the default planner to Blink planner from my side.
> >
> > Best,
> > Terry Wang
> >
> >
> >
> >> 2020年1月3日 15:00,Jark Wu  写道:
> >>
> >> Hi everyone,
> >>
> >> In 1.10 release, Flink SQL supports many awesome features and
> improvements,
> >> including:
> >> - support watermark statement and computed column in DDL
> >> - fully support all data types in Hive
> >> - Batch SQL performance improvements (TPC-DS 7x than Hive MR)
> >> - support INSERT OVERWRITE and INSERT PARTITION
> >>
> >> However, all the features and improvements are only avaiable in Blink
> >> planner, not in Old planner.
> >> There are also some other features are limited in Blink planner, e.g.
> >> Dimension Table Join [1],
> >> TopN [2], Deduplicate [3], streaming aggregates optimization [4], and
> so on.
> >>
> >> But Old planner is still the default planner in Table API & SQL. It is
> >> frustrating for users to set
> >> to blink planner manually when every time start a SQL CLI. And it's
> >> surprising to see unsupported
> >> exception if they trying out the new features but not switch planner.
> >>
> >> SQL CLI is a very important entrypoint for trying out new feautures and
> >> prototyping for users.
> >> In order to give new planner more exposures, I would like to suggest to
> set
> >> default planner
> >> for SQL Client to Blink planner before 1.10 release.
> >>
> >> The approach is just changing the default SQL CLI yaml
> configuration[5]. In
> >> this way, the existing
> >> environment is still compatible and unaffected.
> >>
> >> Changing the default planner for the whole Table API & SQL is another
> topic
> >> and is out of scope of this discussion.
> >>
> >> What do you think?
> >>
> >> Best,
> >> Jark
> >>
> >> [1]:
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/joins.html#join-with-a-temporal-table
> >> [2]:
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#top-n
> >> [3]:
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication
> >> [4]:
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/tuning/streaming_aggregation_optimization.html
> >> [5]:
> >>
> https://github.com/apache/flink/blob/master/flink-table/flink-sql-client/conf/sql-client-defaults.yaml#L100
> >
>
>

-- 
Best, Jingsong Lee


[jira] [Created] (FLINK-15467) Should wait for the end of the source thread during the Task cancellation

2020-01-03 Thread ming li (Jira)
ming li created FLINK-15467:
---

 Summary: Should wait for the end of the source thread during the 
Task cancellation
 Key: FLINK-15467
 URL: https://issues.apache.org/jira/browse/FLINK-15467
 Project: Flink
  Issue Type: Bug
  Components: Runtime / Task
Affects Versions: 1.9.1, 1.9.0
Reporter: ming li


     In the new mailBox model, SourceStreamTask starts a source thread to run 
user methods, and the current execution thread will block on mailbox.takeMail 
(). When a task cancels, the TaskCanceler thread will cancel the task and 
interrupt the execution thread. Therefore, the execution thread of 
SourceStreamTask will throw InterruptedException, then cancel the task again, 
and throw an exception.
{code:java}
//代码占位符
@Override
protected void performDefaultAction(ActionContext context) throws Exception {
   // Against the usual contract of this method, this implementation is not 
step-wise but blocking instead for
   // compatibility reasons with the current source interface (source functions 
run as a loop, not in steps).
   sourceThread.start();

   // We run an alternative mailbox loop that does not involve default actions 
and synchronizes around actions.
   try {
  runAlternativeMailboxLoop();
   } catch (Exception mailboxEx) {
  // We cancel the source function if some runtime exception escaped the 
mailbox.
  if (!isCanceled()) {
 cancelTask();
  }
  throw mailboxEx;
   }

   sourceThread.join();
   if (!isFinished) {
  sourceThread.checkThrowSourceExecutionException();
   }

   context.allActionsCompleted();
}
{code}
   When all tasks of this TaskExecutor are canceled, the blob file will be 
cleaned up. But the real source thread is not finished at this time, which will 
cause a ClassNotFoundException when loading a new class. In this case, the 
source thread may not be able to properly clean up and release resources (such 
as closing child threads, cleaning up local files, etc.). Therefore, I think we 
should mark this task canceled or finished after the execution of the source 
thread is completed.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-15468) INSERT OVERWRITE not supported from SQL CLI

2020-01-03 Thread Rui Li (Jira)
Rui Li created FLINK-15468:
--

 Summary: INSERT OVERWRITE not supported from SQL CLI
 Key: FLINK-15468
 URL: https://issues.apache.org/jira/browse/FLINK-15468
 Project: Flink
  Issue Type: Bug
  Components: Table SQL / Client
Reporter: Rui Li
 Fix For: 1.10.0






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [DISCUSS] Set default planner for SQL Client to Blink planner in 1.10 release

2020-01-03 Thread Jeff Zhang
+1, I have already made blink as the default planner of flink interpreter
in Zeppelin


Jingsong Li  于2020年1月3日周五 下午4:37写道:

> Hi Jark,
>
> +1 for default blink planner in SQL-CLI.
> I believe this new planner can be put into practice in production.
> We've worked hard for nearly a year, but the old planner didn't move on.
>
> And I'd like to cc to u...@flink.apache.org.
> If anyone finds that blink planner has any significant defects and has a
> larger regression than the old planner, please let us know. We will be very
> grateful.
>
> Best,
> Jingsong Lee
>
> On Fri, Jan 3, 2020 at 4:14 PM Leonard Xu  wrote:
>
>> +1 for this.
>> We bring many SQL/API features and enhance stability in 1.10 release, and
>> almost all of them happens in Blink planner.
>> SQL CLI is the most convenient entrypoint for me, I believe many users
>> will have a better experience If we set Blink planner as default planner.
>>
>> Best,
>> Leonard
>>
>> > 在 2020年1月3日,15:16,Terry Wang  写道:
>> >
>> > Since what blink planner can do is a superset of flink planner, big +1
>> for changing the default planner to Blink planner from my side.
>> >
>> > Best,
>> > Terry Wang
>> >
>> >
>> >
>> >> 2020年1月3日 15:00,Jark Wu  写道:
>> >>
>> >> Hi everyone,
>> >>
>> >> In 1.10 release, Flink SQL supports many awesome features and
>> improvements,
>> >> including:
>> >> - support watermark statement and computed column in DDL
>> >> - fully support all data types in Hive
>> >> - Batch SQL performance improvements (TPC-DS 7x than Hive MR)
>> >> - support INSERT OVERWRITE and INSERT PARTITION
>> >>
>> >> However, all the features and improvements are only avaiable in Blink
>> >> planner, not in Old planner.
>> >> There are also some other features are limited in Blink planner, e.g.
>> >> Dimension Table Join [1],
>> >> TopN [2], Deduplicate [3], streaming aggregates optimization [4], and
>> so on.
>> >>
>> >> But Old planner is still the default planner in Table API & SQL. It is
>> >> frustrating for users to set
>> >> to blink planner manually when every time start a SQL CLI. And it's
>> >> surprising to see unsupported
>> >> exception if they trying out the new features but not switch planner.
>> >>
>> >> SQL CLI is a very important entrypoint for trying out new feautures and
>> >> prototyping for users.
>> >> In order to give new planner more exposures, I would like to suggest
>> to set
>> >> default planner
>> >> for SQL Client to Blink planner before 1.10 release.
>> >>
>> >> The approach is just changing the default SQL CLI yaml
>> configuration[5]. In
>> >> this way, the existing
>> >> environment is still compatible and unaffected.
>> >>
>> >> Changing the default planner for the whole Table API & SQL is another
>> topic
>> >> and is out of scope of this discussion.
>> >>
>> >> What do you think?
>> >>
>> >> Best,
>> >> Jark
>> >>
>> >> [1]:
>> >>
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/joins.html#join-with-a-temporal-table
>> >> [2]:
>> >>
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#top-n
>> >> [3]:
>> >>
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication
>> >> [4]:
>> >>
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/tuning/streaming_aggregation_optimization.html
>> >> [5]:
>> >>
>> https://github.com/apache/flink/blob/master/flink-table/flink-sql-client/conf/sql-client-defaults.yaml#L100
>> >
>>
>>
>
> --
> Best, Jingsong Lee
>


-- 
Best Regards

Jeff Zhang


Re: [DISCUSS] Set default planner for SQL Client to Blink planner in 1.10 release

2020-01-03 Thread Timo Walther

Hi Jark,

+1 for making the Blink planner the default planner for the SQL Client.

I think for the Table API, we should give the planner a bit more 
exposure and target the changing of a default planner for 1.11.


What do you think about this two step approach?

Regards,
Timo

On 03.01.20 09:37, Jingsong Li wrote:

Hi Jark,

+1 for default blink planner in SQL-CLI.
I believe this new planner can be put into practice in production.
We've worked hard for nearly a year, but the old planner didn't move on.

And I'd like to cc to u...@flink.apache.org.
If anyone finds that blink planner has any significant defects and has a
larger regression than the old planner, please let us know. We will be very
grateful.

Best,
Jingsong Lee

On Fri, Jan 3, 2020 at 4:14 PM Leonard Xu  wrote:


+1 for this.
We bring many SQL/API features and enhance stability in 1.10 release, and
almost all of them happens in Blink planner.
SQL CLI is the most convenient entrypoint for me, I believe many users
will have a better experience If we set Blink planner as default planner.

Best,
Leonard


在 2020年1月3日,15:16,Terry Wang  写道:

Since what blink planner can do is a superset of flink planner, big +1

for changing the default planner to Blink planner from my side.


Best,
Terry Wang




2020年1月3日 15:00,Jark Wu  写道:

Hi everyone,

In 1.10 release, Flink SQL supports many awesome features and

improvements,

including:
- support watermark statement and computed column in DDL
- fully support all data types in Hive
- Batch SQL performance improvements (TPC-DS 7x than Hive MR)
- support INSERT OVERWRITE and INSERT PARTITION

However, all the features and improvements are only avaiable in Blink
planner, not in Old planner.
There are also some other features are limited in Blink planner, e.g.
Dimension Table Join [1],
TopN [2], Deduplicate [3], streaming aggregates optimization [4], and

so on.


But Old planner is still the default planner in Table API & SQL. It is
frustrating for users to set
to blink planner manually when every time start a SQL CLI. And it's
surprising to see unsupported
exception if they trying out the new features but not switch planner.

SQL CLI is a very important entrypoint for trying out new feautures and
prototyping for users.
In order to give new planner more exposures, I would like to suggest to

set

default planner
for SQL Client to Blink planner before 1.10 release.

The approach is just changing the default SQL CLI yaml

configuration[5]. In

this way, the existing
environment is still compatible and unaffected.

Changing the default planner for the whole Table API & SQL is another

topic

and is out of scope of this discussion.

What do you think?

Best,
Jark

[1]:


https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/joins.html#join-with-a-temporal-table

[2]:


https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#top-n

[3]:


https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication

[4]:


https://ci.apache.org/projects/flink/flink-docs-master/dev/table/tuning/streaming_aggregation_optimization.html

[5]:


https://github.com/apache/flink/blob/master/flink-table/flink-sql-client/conf/sql-client-defaults.yaml#L100











Re: [DISCUSS] Set default planner for SQL Client to Blink planner in 1.10 release

2020-01-03 Thread Jark Wu
Hi Timo,

I have the same idea about the two step approach.
+1 to change default planner for Table API in 1.11.

Best,
Jark


On Fri, 3 Jan 2020 at 17:06, Timo Walther  wrote:

> Hi Jark,
>
> +1 for making the Blink planner the default planner for the SQL Client.
>
> I think for the Table API, we should give the planner a bit more
> exposure and target the changing of a default planner for 1.11.
>
> What do you think about this two step approach?
>
> Regards,
> Timo
>
> On 03.01.20 09:37, Jingsong Li wrote:
> > Hi Jark,
> >
> > +1 for default blink planner in SQL-CLI.
> > I believe this new planner can be put into practice in production.
> > We've worked hard for nearly a year, but the old planner didn't move on.
> >
> > And I'd like to cc to u...@flink.apache.org.
> > If anyone finds that blink planner has any significant defects and has a
> > larger regression than the old planner, please let us know. We will be
> very
> > grateful.
> >
> > Best,
> > Jingsong Lee
> >
> > On Fri, Jan 3, 2020 at 4:14 PM Leonard Xu  wrote:
> >
> >> +1 for this.
> >> We bring many SQL/API features and enhance stability in 1.10 release,
> and
> >> almost all of them happens in Blink planner.
> >> SQL CLI is the most convenient entrypoint for me, I believe many users
> >> will have a better experience If we set Blink planner as default
> planner.
> >>
> >> Best,
> >> Leonard
> >>
> >>> 在 2020年1月3日,15:16,Terry Wang  写道:
> >>>
> >>> Since what blink planner can do is a superset of flink planner, big +1
> >> for changing the default planner to Blink planner from my side.
> >>>
> >>> Best,
> >>> Terry Wang
> >>>
> >>>
> >>>
>  2020年1月3日 15:00,Jark Wu  写道:
> 
>  Hi everyone,
> 
>  In 1.10 release, Flink SQL supports many awesome features and
> >> improvements,
>  including:
>  - support watermark statement and computed column in DDL
>  - fully support all data types in Hive
>  - Batch SQL performance improvements (TPC-DS 7x than Hive MR)
>  - support INSERT OVERWRITE and INSERT PARTITION
> 
>  However, all the features and improvements are only avaiable in Blink
>  planner, not in Old planner.
>  There are also some other features are limited in Blink planner, e.g.
>  Dimension Table Join [1],
>  TopN [2], Deduplicate [3], streaming aggregates optimization [4], and
> >> so on.
> 
>  But Old planner is still the default planner in Table API & SQL. It is
>  frustrating for users to set
>  to blink planner manually when every time start a SQL CLI. And it's
>  surprising to see unsupported
>  exception if they trying out the new features but not switch planner.
> 
>  SQL CLI is a very important entrypoint for trying out new feautures
> and
>  prototyping for users.
>  In order to give new planner more exposures, I would like to suggest
> to
> >> set
>  default planner
>  for SQL Client to Blink planner before 1.10 release.
> 
>  The approach is just changing the default SQL CLI yaml
> >> configuration[5]. In
>  this way, the existing
>  environment is still compatible and unaffected.
> 
>  Changing the default planner for the whole Table API & SQL is another
> >> topic
>  and is out of scope of this discussion.
> 
>  What do you think?
> 
>  Best,
>  Jark
> 
>  [1]:
> 
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/joins.html#join-with-a-temporal-table
>  [2]:
> 
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#top-n
>  [3]:
> 
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication
>  [4]:
> 
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/tuning/streaming_aggregation_optimization.html
>  [5]:
> 
> >>
> https://github.com/apache/flink/blob/master/flink-table/flink-sql-client/conf/sql-client-defaults.yaml#L100
> >>>
> >>
> >>
> >
>
>


Re: [DISCUSS] FLIP-90: Support SQL 2016-2017 JSON functions in Flink SQL

2020-01-03 Thread Timo Walther

Hi,

sorry for jumping into the discussion so late. I had a quick look at the 
FLIP. It looks very nice and detailed. I have one question that I could 
not find in the FLIP itself. Maybe it is hidden in the long discussion 
thread.


What are the return types of all functions? Do we introduce new types 
with this FLIP? Also the RAW types should be avoided. Do all functions 
return STRING/BOOLEAN?


Thanks,
Timo


On 31.12.19 09:39, Hequn Cheng wrote:

Thanks a lot for the update. +1 to start a vote.

On Tue, Dec 31, 2019 at 2:38 PM Forward Xu  wrote:


Hi Jark, Hequn,

I have updated the documentation.

Best,

Forward

Forward Xu  于2019年12月29日周日 下午4:01写道:


Hi Jark, Hequn,

Thank you very much, Introducing new `TableSymbol`s sounds like a good
idea. +1 for the proposal.

I think this idea is good, I will add this in the documentation.


Best, Forward

Hequn Cheng  于2019年12月29日周日 下午3:41写道:


Hi Jark,

Introducing new `TableSymbol`s sounds like a good idea. +1 for the
proposal.
@ForwardXu what do you think? Would be great if the document can be
updated
accordingly.

Best, Hequn


On Fri, Dec 27, 2019 at 4:14 PM Jark Wu  wrote:


Thanks for looking into the design Hequn. I agree it would be great to
have a full story design.

For the ON ERROR and ON EMPTY clause in Table API, some initial
thoughts in my mind is that
we can introduce some new `TableSymbol`s as the second parameter of

json

function, e.g. `JsonErrorStrategy`.

For example,

JSON_VALUE(v, 'lax $.b' ERROR ON ERROR)
== is equal to Table API ==>
v.jsonValue("lax $.b", JsonErrorStrategy.ERROR)

Best,
Jark


On Thu, 26 Dec 2019 at 23:37, Hequn Cheng 

wrote:



Hi Jark & ForwardXu,

The design doc looks very nice! Only some minor feedback from my

side.


As calcite has already implemented the JSON functions, I would

suppose

the semantics and implementation are right for SQL.

For TableAPI, I think the most important is to keep align with the
SQL(which has also been mentioned by Jark in the previous

discussion).

Have

an equivalent feature set for all APIs and maintain it otherwise

confusion

increases especially when more and more functions are added. The

document

has documented how to support TableAPI. I think this is very good!

And

it

would be better to also include ON ERROR or ON EMPTY for Table API.

We

can

implement these features step by step, but maybe we should design all

these

once for all to avoid API changes later. Meanwhile, these features

are

also

commonly required by users.

Would be great to also have your opinions!

Best,
Hequn


On Mon, Dec 23, 2019 at 10:15 AM Jark Wu  wrote:


Hi Forward,

Thanks for creating the FLIP. +1 to start a vote.

  @Hequn Cheng  @Kurt Young 

,

could you help to review the design doc too?

Best,
Jark


On Mon, 23 Dec 2019 at 10:10, tison  wrote:


modified:





https://lists.apache.org/x/thread.html/b3c0265cc2b660fe11ce550b84a831a7606de12908ff7ff0959a4794@%3Cdev.flink.apache.org%3E
















Re: [DISCUSS] FLIP-90: Support SQL 2016-2017 JSON functions in Flink SQL

2020-01-03 Thread Jark Wu
Hi Timo,

That's a good point.
We didn't introduce any new types. We will use the function definition
defined by Calcite[1].
So all the functions return STRING/BOOLEAN.

Hi Forward,
I think we may need an additional column to describe the return type of
each function.

Best,
Jark

[1]:
https://github.com/apache/calcite/blob/master/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java

On Fri, 3 Jan 2020 at 17:30, Timo Walther  wrote:

> Hi,
>
> sorry for jumping into the discussion so late. I had a quick look at the
> FLIP. It looks very nice and detailed. I have one question that I could
> not find in the FLIP itself. Maybe it is hidden in the long discussion
> thread.
>
> What are the return types of all functions? Do we introduce new types
> with this FLIP? Also the RAW types should be avoided. Do all functions
> return STRING/BOOLEAN?
>
> Thanks,
> Timo
>
>
> On 31.12.19 09:39, Hequn Cheng wrote:
> > Thanks a lot for the update. +1 to start a vote.
> >
> > On Tue, Dec 31, 2019 at 2:38 PM Forward Xu 
> wrote:
> >
> >> Hi Jark, Hequn,
> >>
> >> I have updated the documentation.
> >>
> >> Best,
> >>
> >> Forward
> >>
> >> Forward Xu  于2019年12月29日周日 下午4:01写道:
> >>
> >>> Hi Jark, Hequn,
> >>>
> >>> Thank you very much, Introducing new `TableSymbol`s sounds like a good
> >>> idea. +1 for the proposal.
> >>>
> >>> I think this idea is good, I will add this in the documentation.
> >>>
> >>>
> >>> Best, Forward
> >>>
> >>> Hequn Cheng  于2019年12月29日周日 下午3:41写道:
> >>>
>  Hi Jark,
> 
>  Introducing new `TableSymbol`s sounds like a good idea. +1 for the
>  proposal.
>  @ForwardXu what do you think? Would be great if the document can be
>  updated
>  accordingly.
> 
>  Best, Hequn
> 
> 
>  On Fri, Dec 27, 2019 at 4:14 PM Jark Wu  wrote:
> 
> > Thanks for looking into the design Hequn. I agree it would be great
> to
> > have a full story design.
> >
> > For the ON ERROR and ON EMPTY clause in Table API, some initial
> > thoughts in my mind is that
> > we can introduce some new `TableSymbol`s as the second parameter of
> >> json
> > function, e.g. `JsonErrorStrategy`.
> >
> > For example,
> >
> > JSON_VALUE(v, 'lax $.b' ERROR ON ERROR)
> > == is equal to Table API ==>
> > v.jsonValue("lax $.b", JsonErrorStrategy.ERROR)
> >
> > Best,
> > Jark
> >
> >
> > On Thu, 26 Dec 2019 at 23:37, Hequn Cheng 
> >> wrote:
> >
> >> Hi Jark & ForwardXu,
> >>
> >> The design doc looks very nice! Only some minor feedback from my
> >> side.
> >>
> >> As calcite has already implemented the JSON functions, I would
> >> suppose
> >> the semantics and implementation are right for SQL.
> >>
> >> For TableAPI, I think the most important is to keep align with the
> >> SQL(which has also been mentioned by Jark in the previous
> >> discussion).
>  Have
> >> an equivalent feature set for all APIs and maintain it otherwise
>  confusion
> >> increases especially when more and more functions are added. The
>  document
> >> has documented how to support TableAPI. I think this is very good!
> >> And
>  it
> >> would be better to also include ON ERROR or ON EMPTY for Table API.
> >> We
>  can
> >> implement these features step by step, but maybe we should design
> all
>  these
> >> once for all to avoid API changes later. Meanwhile, these features
> >> are
>  also
> >> commonly required by users.
> >>
> >> Would be great to also have your opinions!
> >>
> >> Best,
> >> Hequn
> >>
> >>
> >> On Mon, Dec 23, 2019 at 10:15 AM Jark Wu  wrote:
> >>
> >>> Hi Forward,
> >>>
> >>> Thanks for creating the FLIP. +1 to start a vote.
> >>>
> >>>   @Hequn Cheng  @Kurt Young <
> ykt...@gmail.com>
> >> ,
> >>> could you help to review the design doc too?
> >>>
> >>> Best,
> >>> Jark
> >>>
> >>>
> >>> On Mon, 23 Dec 2019 at 10:10, tison  wrote:
> >>>
>  modified:
> 
> 
> 
> >>
> https://lists.apache.org/x/thread.html/b3c0265cc2b660fe11ce550b84a831a7606de12908ff7ff0959a4794@%3Cdev.flink.apache.org%3E
> 
> >>>
> 
> >>>
> >>
> >
>
>


[jira] [Created] (FLINK-15469) UpsertStreamTableSink should support new type system

2020-01-03 Thread Zhenghua Gao (Jira)
Zhenghua Gao created FLINK-15469:


 Summary: UpsertStreamTableSink should support new type system
 Key: FLINK-15469
 URL: https://issues.apache.org/jira/browse/FLINK-15469
 Project: Flink
  Issue Type: New Feature
  Components: Table SQL / API
Affects Versions: 1.10.0
Reporter: Zhenghua Gao
 Fix For: 1.11.0


Currently *UpsertStreamTableSink* can only returns TypeInformation of the 
requested record, which can't support types with precision and scale, e.g. 
TIMESTAMP(p), DECIMAL(p,s).

A proposal is deprecating the *getRecordType* API and adding a 
*getRecordDataType* API instead to return the data type of the requested record.
{code:java}
/**
 * Returns the requested record type.
 * 
 * @Deprecated This method will be removed in future versions. It's recommended 
to use {@link #getRecordDataType()} instead.
 */
@Deprecated
TypeInformation getRecordType();

/*
 * Returns the requested record data type.
 */
DataType getRecordDataType();

{code}
 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [ANNOUNCE] Progress of Apache Flink 1.10 #3

2020-01-03 Thread Congxian Qiu
Thanks for the great work, and look forward to the first RC.

Best,
Congxian


Jingsong Li  于2020年1月3日周五 下午12:58写道:

> Thanks Yu and Gary,
>
> Table and hive layer have resolved all blockers and critical issues. (38
> blockers and 33 critical issues)
>
> Look forward to the first RC.
>
> Best,
> Jingsong Lee
>
> On Fri, Jan 3, 2020 at 11:16 AM Danny Chan  wrote:
>
> > Thanks for the great work ~
> >
> > Best,
> > Danny Chan
> > 在 2020年1月2日 +0800 PM8:48,Yu Li ,写道:
> > > Hi community,
> > >
> > > It has been three weeks since we cut the release-1.10 branch and kicked
> > off
> > > the release testing phase [1]. Thanks to all the efforts of the
> > community,
> > > 46 blocker and 11 critical issues [2] have been resolved during this
> > > period, and now we are left with only 12 blocker issues [3].
> > >
> > > If everything goes smoothly, we could possibly resolve all blockers and
> > > produce the first RC in the 3rd week of Jan. Please be prepared for
> > testing
> > > the RC and let us know if there are any questions.
> > >
> > > Best,
> > > Gary and Yu
> > >
> > > [1] https://s.apache.org/1okuv
> > > [2] https://s.apache.org/gbd0r
> > > [3] https://s.apache.org/euzaj
> >
>
>
> --
> Best, Jingsong Lee
>


Re: [ANNOUNCE] Progress of Apache Flink 1.10 #3

2020-01-03 Thread vino yang
Hi Yu,

Thanks for your great work.

Best,
Vino

Congxian Qiu  于2020年1月3日周五 下午6:18写道:

> Thanks for the great work, and look forward to the first RC.
>
> Best,
> Congxian
>
>
> Jingsong Li  于2020年1月3日周五 下午12:58写道:
>
> > Thanks Yu and Gary,
> >
> > Table and hive layer have resolved all blockers and critical issues. (38
> > blockers and 33 critical issues)
> >
> > Look forward to the first RC.
> >
> > Best,
> > Jingsong Lee
> >
> > On Fri, Jan 3, 2020 at 11:16 AM Danny Chan  wrote:
> >
> > > Thanks for the great work ~
> > >
> > > Best,
> > > Danny Chan
> > > 在 2020年1月2日 +0800 PM8:48,Yu Li ,写道:
> > > > Hi community,
> > > >
> > > > It has been three weeks since we cut the release-1.10 branch and
> kicked
> > > off
> > > > the release testing phase [1]. Thanks to all the efforts of the
> > > community,
> > > > 46 blocker and 11 critical issues [2] have been resolved during this
> > > > period, and now we are left with only 12 blocker issues [3].
> > > >
> > > > If everything goes smoothly, we could possibly resolve all blockers
> and
> > > > produce the first RC in the 3rd week of Jan. Please be prepared for
> > > testing
> > > > the RC and let us know if there are any questions.
> > > >
> > > > Best,
> > > > Gary and Yu
> > > >
> > > > [1] https://s.apache.org/1okuv
> > > > [2] https://s.apache.org/gbd0r
> > > > [3] https://s.apache.org/euzaj
> > >
> >
> >
> > --
> > Best, Jingsong Lee
> >
>


Re: [DISCUSS] Set default planner for SQL Client to Blink planner in 1.10 release

2020-01-03 Thread Jark Wu
Hi Benoît,

Thanks for the reminder. I will look into the issue and hopefully we can
target it into 1.9.2 and 1.10.

Cheers,
Jark

On Fri, 3 Jan 2020 at 18:21, Benoît Paris <
benoit.pa...@centraliens-lille.org> wrote:

> >  If anyone finds that blink planner has any significant defects and has
> a larger regression than the old planner, please let us know.
>
> Overall, the Blink-exclusive features are must (TopN, deduplicate,
> LAST_VALUE, plan reuse, etc)! But all use cases of the Legacy planner in
> production are not covered:
> An edge case of Temporal Table Functions does not allow computed Tables
> (as opposed to TableSources) to be used on the query side in Blink (
> https://issues.apache.org/jira/browse/FLINK-14200)
>
> Cheers
> Ben
>
>
> On Fri, Jan 3, 2020 at 10:00 AM Jeff Zhang  wrote:
>
>> +1, I have already made blink as the default planner of flink interpreter
>> in Zeppelin
>>
>>
>> Jingsong Li  于2020年1月3日周五 下午4:37写道:
>>
>>> Hi Jark,
>>>
>>> +1 for default blink planner in SQL-CLI.
>>> I believe this new planner can be put into practice in production.
>>> We've worked hard for nearly a year, but the old planner didn't move on.
>>>
>>> And I'd like to cc to u...@flink.apache.org.
>>> If anyone finds that blink planner has any significant defects and has a
>>> larger regression than the old planner, please let us know. We will be very
>>> grateful.
>>>
>>> Best,
>>> Jingsong Lee
>>>
>>> On Fri, Jan 3, 2020 at 4:14 PM Leonard Xu  wrote:
>>>
 +1 for this.
 We bring many SQL/API features and enhance stability in 1.10 release,
 and almost all of them happens in Blink planner.
 SQL CLI is the most convenient entrypoint for me, I believe many users
 will have a better experience If we set Blink planner as default planner.

 Best,
 Leonard

 > 在 2020年1月3日,15:16,Terry Wang  写道:
 >
 > Since what blink planner can do is a superset of flink planner, big
 +1 for changing the default planner to Blink planner from my side.
 >
 > Best,
 > Terry Wang
 >
 >
 >
 >> 2020年1月3日 15:00,Jark Wu  写道:
 >>
 >> Hi everyone,
 >>
 >> In 1.10 release, Flink SQL supports many awesome features and
 improvements,
 >> including:
 >> - support watermark statement and computed column in DDL
 >> - fully support all data types in Hive
 >> - Batch SQL performance improvements (TPC-DS 7x than Hive MR)
 >> - support INSERT OVERWRITE and INSERT PARTITION
 >>
 >> However, all the features and improvements are only avaiable in Blink
 >> planner, not in Old planner.
 >> There are also some other features are limited in Blink planner, e.g.
 >> Dimension Table Join [1],
 >> TopN [2], Deduplicate [3], streaming aggregates optimization [4],
 and so on.
 >>
 >> But Old planner is still the default planner in Table API & SQL. It
 is
 >> frustrating for users to set
 >> to blink planner manually when every time start a SQL CLI. And it's
 >> surprising to see unsupported
 >> exception if they trying out the new features but not switch planner.
 >>
 >> SQL CLI is a very important entrypoint for trying out new feautures
 and
 >> prototyping for users.
 >> In order to give new planner more exposures, I would like to suggest
 to set
 >> default planner
 >> for SQL Client to Blink planner before 1.10 release.
 >>
 >> The approach is just changing the default SQL CLI yaml
 configuration[5]. In
 >> this way, the existing
 >> environment is still compatible and unaffected.
 >>
 >> Changing the default planner for the whole Table API & SQL is
 another topic
 >> and is out of scope of this discussion.
 >>
 >> What do you think?
 >>
 >> Best,
 >> Jark
 >>
 >> [1]:
 >>
 https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/joins.html#join-with-a-temporal-table
 >> [2]:
 >>
 https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#top-n
 >> [3]:
 >>
 https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication
 >> [4]:
 >>
 https://ci.apache.org/projects/flink/flink-docs-master/dev/table/tuning/streaming_aggregation_optimization.html
 >> [5]:
 >>
 https://github.com/apache/flink/blob/master/flink-table/flink-sql-client/conf/sql-client-defaults.yaml#L100
 >


>>>
>>> --
>>> Best, Jingsong Lee
>>>
>>
>>
>> --
>> Best Regards
>>
>> Jeff Zhang
>>
>
>
> --
> Benoît Paris
> Ingénieur Machine Learning Explicable
> Tél : +33 6 60 74 23 00
> http://benoit.paris
> http://explicable.ml
>


[jira] [Created] (FLINK-15470) Remove YARN magic properties file

2020-01-03 Thread Yang Wang (Jira)
Yang Wang created FLINK-15470:
-

 Summary: Remove YARN magic properties file
 Key: FLINK-15470
 URL: https://issues.apache.org/jira/browse/FLINK-15470
 Project: Flink
  Issue Type: Task
  Components: Command Line Client, Deployment / YARN
Reporter: Yang Wang


Currently, when we start a session, flink client will write a yarn properties 
under \{java.io.tmpdir}/.yarn-properties-\{user-name}. So the user could use a 
simple command {{./bin/flink run ./examples/batch/WordCount.jar}} to submit a 
job to the session.

 

Even if the session cluster has finished or kill by {{yarn application -kill}}, 
it will still submit a job to the non-existing session. Some users has run into 
this problem in the ML. So i suggest to remove the properties file and always 
use {{-yid application_id}} to submit job an existing session.

[https://lists.apache.org/thread.html/b0e0e388a16d1891d1de651bec027a3820130f28aaaeb843440cb679%40%3Cuser.flink.apache.org%3E]

 

We also have some discussion under 
[FLINK-15179|https://issues.apache.org/jira/browse/FLINK-15179]. The cli 
options will be deprecated in the future and we hope {{FlinkYarnSessionCli}} 
could use {{ExecutorCli}}. So we will try to clean up the code, i think the 
magic properties file is the case.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-15471) HA e2e check for empty .out files isn't working

2020-01-03 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-15471:


 Summary: HA e2e check for empty .out files isn't working
 Key: FLINK-15471
 URL: https://issues.apache.org/jira/browse/FLINK-15471
 Project: Flink
  Issue Type: Improvement
  Components: Test Infrastructure
Affects Versions: 1.9.0
Reporter: Chesnay Schepler
Assignee: Chesnay Schepler
 Fix For: 1.9.2, 1.10.0


{{common_ha.sh#verify_logs:}}
{code}
if ! check_logs_for_non_empty_out_files; then
echo "FAILURE: Alerts found at the general purpose job."
EXIT_CODE=1
fi
{code}

Since check_logs_for_non_empty_out_files does only set EXIT_CODE without 
modifying the return value the check will never fail.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-15472) Support SQL Client Gateway

2020-01-03 Thread godfrey he (Jira)
godfrey he created FLINK-15472:
--

 Summary: Support SQL Client Gateway 
 Key: FLINK-15472
 URL: https://issues.apache.org/jira/browse/FLINK-15472
 Project: Flink
  Issue Type: New Feature
  Components: Table SQL / Client
Reporter: godfrey he


FLIP-91: 
https://cwiki.apache.org/confluence/display/FLINK/FLIP-91%3A+Support+SQL+Client+Gateway
design document: 
https://docs.google.com/document/d/1T7--664rts4t_4gjRPw937S9ln9Plf1yghNQ9IiHQtQ



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-15473) Add support for Linux on ppc64le to MemoryArchitecture

2020-01-03 Thread Ronald O. Edmark (Jira)
Ronald O. Edmark created FLINK-15473:


 Summary: Add support for Linux on ppc64le to MemoryArchitecture
 Key: FLINK-15473
 URL: https://issues.apache.org/jira/browse/FLINK-15473
 Project: Flink
  Issue Type: Improvement
 Environment: Red Hat, Ubuntu Linux on PowerPC.
Reporter: Ronald O. Edmark


Similar to issue FLINK-13449

Please add "ppc64le" to MemoryArchitecture.java

final List names64bit = Arrays.asList("amd64", "x86_64", "aarch64", 
"ppc64le");

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [DISCUSS] FLIP-90: Support SQL 2016-2017 JSON functions in Flink SQL

2020-01-03 Thread Forward Xu
Hi Timo, Jack,
Well, I added additional column to describe the return type of each
function and
updated the google doc.

Best,
Forward

Jark Wu  于2020年1月3日周五 下午5:48写道:

> Hi Timo,
>
> That's a good point.
> We didn't introduce any new types. We will use the function definition
> defined by Calcite[1].
> So all the functions return STRING/BOOLEAN.
>
> Hi Forward,
> I think we may need an additional column to describe the return type of
> each function.
>
> Best,
> Jark
>
> [1]:
>
> https://github.com/apache/calcite/blob/master/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java
>
> On Fri, 3 Jan 2020 at 17:30, Timo Walther  wrote:
>
> > Hi,
> >
> > sorry for jumping into the discussion so late. I had a quick look at the
> > FLIP. It looks very nice and detailed. I have one question that I could
> > not find in the FLIP itself. Maybe it is hidden in the long discussion
> > thread.
> >
> > What are the return types of all functions? Do we introduce new types
> > with this FLIP? Also the RAW types should be avoided. Do all functions
> > return STRING/BOOLEAN?
> >
> > Thanks,
> > Timo
> >
> >
> > On 31.12.19 09:39, Hequn Cheng wrote:
> > > Thanks a lot for the update. +1 to start a vote.
> > >
> > > On Tue, Dec 31, 2019 at 2:38 PM Forward Xu 
> > wrote:
> > >
> > >> Hi Jark, Hequn,
> > >>
> > >> I have updated the documentation.
> > >>
> > >> Best,
> > >>
> > >> Forward
> > >>
> > >> Forward Xu  于2019年12月29日周日 下午4:01写道:
> > >>
> > >>> Hi Jark, Hequn,
> > >>>
> > >>> Thank you very much, Introducing new `TableSymbol`s sounds like a
> good
> > >>> idea. +1 for the proposal.
> > >>>
> > >>> I think this idea is good, I will add this in the documentation.
> > >>>
> > >>>
> > >>> Best, Forward
> > >>>
> > >>> Hequn Cheng  于2019年12月29日周日 下午3:41写道:
> > >>>
> >  Hi Jark,
> > 
> >  Introducing new `TableSymbol`s sounds like a good idea. +1 for the
> >  proposal.
> >  @ForwardXu what do you think? Would be great if the document can be
> >  updated
> >  accordingly.
> > 
> >  Best, Hequn
> > 
> > 
> >  On Fri, Dec 27, 2019 at 4:14 PM Jark Wu  wrote:
> > 
> > > Thanks for looking into the design Hequn. I agree it would be great
> > to
> > > have a full story design.
> > >
> > > For the ON ERROR and ON EMPTY clause in Table API, some initial
> > > thoughts in my mind is that
> > > we can introduce some new `TableSymbol`s as the second parameter of
> > >> json
> > > function, e.g. `JsonErrorStrategy`.
> > >
> > > For example,
> > >
> > > JSON_VALUE(v, 'lax $.b' ERROR ON ERROR)
> > > == is equal to Table API ==>
> > > v.jsonValue("lax $.b", JsonErrorStrategy.ERROR)
> > >
> > > Best,
> > > Jark
> > >
> > >
> > > On Thu, 26 Dec 2019 at 23:37, Hequn Cheng 
> > >> wrote:
> > >
> > >> Hi Jark & ForwardXu,
> > >>
> > >> The design doc looks very nice! Only some minor feedback from my
> > >> side.
> > >>
> > >> As calcite has already implemented the JSON functions, I would
> > >> suppose
> > >> the semantics and implementation are right for SQL.
> > >>
> > >> For TableAPI, I think the most important is to keep align with the
> > >> SQL(which has also been mentioned by Jark in the previous
> > >> discussion).
> >  Have
> > >> an equivalent feature set for all APIs and maintain it otherwise
> >  confusion
> > >> increases especially when more and more functions are added. The
> >  document
> > >> has documented how to support TableAPI. I think this is very good!
> > >> And
> >  it
> > >> would be better to also include ON ERROR or ON EMPTY for Table
> API.
> > >> We
> >  can
> > >> implement these features step by step, but maybe we should design
> > all
> >  these
> > >> once for all to avoid API changes later. Meanwhile, these features
> > >> are
> >  also
> > >> commonly required by users.
> > >>
> > >> Would be great to also have your opinions!
> > >>
> > >> Best,
> > >> Hequn
> > >>
> > >>
> > >> On Mon, Dec 23, 2019 at 10:15 AM Jark Wu 
> wrote:
> > >>
> > >>> Hi Forward,
> > >>>
> > >>> Thanks for creating the FLIP. +1 to start a vote.
> > >>>
> > >>>   @Hequn Cheng  @Kurt Young <
> > ykt...@gmail.com>
> > >> ,
> > >>> could you help to review the design doc too?
> > >>>
> > >>> Best,
> > >>> Jark
> > >>>
> > >>>
> > >>> On Mon, 23 Dec 2019 at 10:10, tison 
> wrote:
> > >>>
> >  modified:
> > 
> > 
> > 
> > >>
> >
> https://lists.apache.org/x/thread.html/b3c0265cc2b660fe11ce550b84a831a7606de12908ff7ff0959a4794@%3Cdev.flink.apache.org%3E
> > 
> > >>>
> > 
> > >>>
> > >>
> > >
> >
> >
>


[jira] [Created] (FLINK-15474) In TypeSerializerUpgradeTestBase, create serializer snapshots "on demand"

2020-01-03 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-15474:


 Summary: In TypeSerializerUpgradeTestBase, create serializer 
snapshots "on demand"
 Key: FLINK-15474
 URL: https://issues.apache.org/jira/browse/FLINK-15474
 Project: Flink
  Issue Type: Bug
  Components: API / Type Serialization System, Tests
Affects Versions: 1.9.0, 1.10.0
Reporter: Aljoscha Krettek
Assignee: Aljoscha Krettek
 Fix For: 1.10.0


Currently, we store binary snapshots in the repository for all the different 
serializer upgrade test configurations (see linked POC for which snapshots are 
there for just the POJO serializer). This is hard to maintain, because someone 
has to go back and generate snapshtos from previous Flink versions and add them 
to the repo when updating the tests for a new Flink version. It's also 
problematic from a repository perspective because we keep piling up binary 
snapshots.

Instead, we can create a snapshot "on demand" from a previous Flink version by 
using a classloader that has the previous Flink jar.

I created a POC which demonstrated the approach: 
[https://github.com/aljoscha/flink/tree/jit-serializer-test-base]. The 
advantage is that we don't need binary snapshots in the repo anymore, updating 
the tests to a newer Flink version should be as easy as adding a new migration 
version and Flink download url. The downside is that the test now downloads 
Flink releases (in the PoC this is done using the caching infra introduced for 
e2e tests), which is costly and also re-generates the snapshots for every test, 
which is also costly. The test time (minus downloading) goes up from about 300 
ms to roughly 6 seconds. That's not something I would call a unit test. We 
could call these "integration tests" (or even e2e tests) and only run them 
nightly. Side note, we don't have test coverage for serializer upgrades from 
1.8 and 1.9 currently, so even doing it nightly would be an improvement.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-15475) Add isOutputTypeUsed() API to Transformation

2020-01-03 Thread Rong Rong (Jira)
Rong Rong created FLINK-15475:
-

 Summary: Add isOutputTypeUsed() API to Transformation
 Key: FLINK-15475
 URL: https://issues.apache.org/jira/browse/FLINK-15475
 Project: Flink
  Issue Type: Improvement
  Components: API / Core, API / DataSet, API / DataStream
Reporter: Rong Rong
Assignee: Rong Rong


Currently there's no way to "peek" into a Transformation and see if OutputType 
has been used or not. The only way is to invoke the {{setOutputType}} API and 
wrap around it with a try-catch block. 

It would be nice if we have a `isTypeUsed()` or `isOutputTypeUsed()` API to 
check whether a particular transformation has a definitive output type set / 
used or not.





--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-15476) Update StreamingFileSink documentation -- bulk encoded writer now supports customized checkpoint policy

2020-01-03 Thread Ying Xu (Jira)
Ying Xu created FLINK-15476:
---

 Summary: Update StreamingFileSink documentation -- bulk encoded 
writer now supports customized checkpoint policy
 Key: FLINK-15476
 URL: https://issues.apache.org/jira/browse/FLINK-15476
 Project: Flink
  Issue Type: Improvement
Reporter: Ying Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)