Just to clarify:
FLIP wiki:
https://cwiki.apache.org/confluence/display/FLINK/FLIP-63%3A+Rework+table+partition+support
Discussion thread:
http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-63-Rework-table-partition-support-td32770.html
Google Doc:
Thanks Terry for driving this. +1
With these features, SQL-CLI could be really used.
Best,
Jingsong Lee
On Fri, Nov 8, 2019 at 2:20 PM Kurt Young wrote:
> Forgot to vote.. +1 from my side.
>
> Best,
> Kurt
>
>
> On Fri, Nov 8, 2019 at 11:00 AM Kurt Young wrote:
>
> > Hi all,
> >
> > I think
Thanks Kurt and Terry for bringing this to my attention.
Agree each topic of this FLIP touched is super big.
But because it's too big, we can't cover everything in one FLIP. So I think
there are two ways:
1. Decompose this big thing into an independent topic(FLIP), so that each
FLIP can be
Hi Dominik:
I found a problem too that it maybe your root cause.[1]
JobConf in HadoopInputSplit may very big, contains hundreds of
configurations, if it is serialized by every split, that will significantly
reduce performance. Consider thousands of splits, the akka thread of
JobMaster will all on
+1 (non-binding)
Best,
Jingsong Lee
On Tue, Nov 12, 2019 at 9:49 AM vino yang wrote:
> +1 (non-binding)
>
> Best,
> Vino
>
> Xuefu Z 于2019年11月12日周二 上午3:27写道:
>
> > +1 (non-binding)
> >
> > On Mon, Nov 11, 2019 at 9:54 AM Shuyi Chen wrote:
> >
> > > +1 (binding)
> > >
> > > On Sat, Nov 9,
Congratulations to Jark.
Jark has really contributed a lot to the table layer with a long time. Well
deserved.
Best,
Jingsong Lee
On Fri, Nov 8, 2019 at 6:05 PM Yu Li wrote:
> Congratulations Jark! Well deserved!
>
> Best Regards,
> Yu
>
>
> On Fri, 8 Nov 2019 at 17:55, OpenInx wrote:
>
> >
+1 (non-binding)
- verified the signature and checksum
- checked the maven central artifices
- checked "release-9.0-rc1" source code
- verified website pull request listing the new release
Best,
Jingsong Lee
On Thu, Nov 21, 2019 at 5:28 PM Dian Fu wrote:
> +1 (non-binding)
>
> - verified the
+1 thanks dawid for driving this.
Best
Jingsong Lee
Timo Walther :
> +1
>
> Thanks,
> Timo
>
>
> On 23.11.19 00:28, Xuefu Z wrote:
> > +1
> >
> > On Fri, Nov 22, 2019 at 5:31 AM Kurt Young wrote:
> >
> >> +1
> >>
> >> Best,
> >> Kurt
> >>
> >>
> >> On Fri, Nov 22, 2019 at 8:51 PM Dawid
Hi Dawid,
Thanks for you explanation.
You mean that "people who vote for binding +1 need to use apache email
addresses".
Although I can't vote binding +1, I am +1 for this suggestion.
Best,
Jingsong Lee
On Wed, Dec 4, 2019 at 4:04 PM Dawid Wysakowicz
wrote:
> Hi all,
>
> Sorry I think I was
nks,
> Dian
>
> > 在 2019年12月2日,下午3:47,Jingsong Li 写道:
> >
> > Sorry for bothering your voting.
> > Let's discuss in discussion thread.
> >
> > Best,
> > Jingsong Lee
> >
> > On Mon, Dec 2, 2019 at 3:32 PM Jingsong Lee
> wrote:
> &
Hi Piotr,
Huge +1 for N-Ary Stream Operator.
And I love this Golden Shovel award very much!
There are a large number jobs (in production environment) that their
TwoInputOperators that can be chained. We used to only watch the last
ten tasks transmit data through disk and network, which could
Hi Hequn,
+1 (non-binding) Thank you for driving this.
- Verified signatures and checksums
- Maven build from source skip tests (Scala 2.11 and Scala 2.12)
- Start local cluster and web ui is accessible (Scala 2.11 and Scala 2.12)
- Submit WordCount example of both batch and streaming, good
Thanks Chesnay,
+1 to make it official that we no longer actively develop them but user can
still use.
Best,
Jingsong Lee
On Mon, Dec 9, 2019 at 4:47 AM Chesnay Schepler wrote:
> Users can continue to use the 1.9 versions of the 0.8/0.9 connectors
> with future versions of Flink.
>
> We just
Hi Hequn,
+1 (non-binding) Thank you for driving this.
- Verified signatures and checksums
- Maven build from source skip tests (Scala 2.11 and Scala 2.12)
- Start local cluster and web ui is accessible (Scala 2.11 and Scala 2.12)
- Submit WordCount example of both batch and streaming, good
Hi Dev,
After cutting out the branch of 1.10, I tried the following functions of
SQL-CLI and found that it does not support:
- insert overwrite
- PARTITION (partcol1=val1, partcol2=val2 ...)
The SQL pattern is:
INSERT { INTO | OVERWRITE } TABLE tablename1 [PARTITION (partcol1=val1,
partcol2=val2
Hi Leo,
I think run job with external jars is important too.
Have you took a look to PipelineOptions.JARS in configuration?
I think this is a way to set external jars. And SQL-CLI need it too.
Best,
Jingsong Lee
On Wed, Dec 11, 2019 at 9:18 AM 50man wrote:
> Hi everyone,
>
>
> I propose an
Thanks Yu & Gary for your great work.
1.10 is a very important version for batch sql (also streaming sql).
It enables many features to be put into production.
Looking forward to the 1.10 release.
Let's "beat" it by tests.
Best,
Jingsong Lee
On Wed, Dec 11, 2019 at 2:55 PM Wei Zhong wrote:
>
ent implementation violates the accepted
> > design of FLIP-63, which IMO qualifies as a bug. Given that it's a bug
> and
> > has great impact on the usability of our Hive integration, do you think
> we
> > can fix it in 1.10?
> >
> > On Fri, Dec 13, 2
Hi Bowen,
Thanks for driving this.
+1 for this proposal.
Due to our multi version support, users are required to rely on
different dependencies, it does break the "out of box" experience.
Now that the client has changed to go to child first class loader resolve
by default, it puts forward higher
INSERT OVERWRITE and INSERT
> > PARTITION
> > +0 to add yaml dialect conf to SQL-CLI because FLIP-89 is not finished
> > yet, we better do this until FLIP-89 is resolved.
> >
> > Best,
> > Danny Chan
> > 在 2019年12月11日 +0800 PM5:29,Jingsong Li
Thanks Hequn for your driving, 1.8.3 fixed a lot of issues and it is very
useful to users.
Great work!
Best,
Jingsong Lee
On Thu, Dec 12, 2019 at 3:25 PM jincheng sun
wrote:
> Thanks for being the release manager and the great work Hequn :)
> Also thanks to the community making this release
t; > under the dialect limitation, but CREATE PARTITION TABLE is not.
> >
> > So it is indeed a bug which should be fixed in 1.10.
> >
> > Best,
> > Jark
> >
> > On Thu, 12 Dec 2019 at 16:35, Jingsong Li
> wrote:
> >
> >> Hi Jark,
> &g
Hi Dian,
After [1] and [2], in the batch sql world, we will:
- [2] In client/compile side: we use memory weight request memory for
Transformation.
- [1] In runtime side: we use memory fraction to compute memory size and
allocate in StreamOperator.
For your information.
[1]
;
> > Provide an N-Ary stream operator to make everything possible. The upper
> > layer can do anything. These things can be specific optimization, which
> is much
> > more natural than the lower layer.
>
> Do you mean that those Table API/SQL use cases (HashJoin/SortMe
> before `Build2` finished. I don't have a clear solution for now, but
> it's a common pattern we will face.
>
> Best,
> Kurt
>
>
> On Thu, Dec 5, 2019 at 4:37 PM Jingsong Li wrote:
>
> > Hi Piotr,
> >
> > > a) two input operator X -> one input operato
Hi Benchao,
Thanks for your reporting.
As Leonard said, you can create an issue in JIRA. We can go on discussing
in JIRA.
The answer is #1, Blink's behavior ensures that the job runs as much as
possible without interrupting it, so null is returned here to make it
possible to continue running.
In
nything, a vote is required to overturn it. Before that, the original FLIP
> holds.
>
> Thanks,
> Xuefu
>
>
> On Fri, Dec 13, 2019 at 1:20 AM Jingsong Li
> wrote:
>
> > Hi Timo,
> >
> > Thanks for your feedback.
> >
> > The reason of `The DD
Thanks all for explaining.
I misunderstood the original proposal.
-1 to put them in our distributions
+1 to have provide hive uber jars as Seth and Aljoscha advice
Hive is just a connector no matter how important it is.
So I totally agree that we shouldn't put them in our distributions.
We can
Congratulations Zhu Zhu!
Best,
Jingsong Lee
On Mon, Dec 16, 2019 at 10:01 AM Yang Wang wrote:
> Congratulations, Zhu Zhu!
>
> wenlong.lwl 于2019年12月16日周一 上午9:56写道:
>
> > Congratulations, Zhu Zhu!
> >
> > On Mon, 16 Dec 2019 at 09:14, Leonard Xu wrote:
> >
> > > Congratulations, Zhu Zhu ! !
>
is
> >> topic and we apparently missed some test cases on SQL CLI side. Sorry
> for
> >> that.
> >>
> >> From a product perspective, SQL CLI is super critical for Flink-Hive
> >> integration and Flink SQL iteself. INSERT OVERWRITE and PARTI
rrently retrieved records and then the
> caller
> > >> emits them outside of the SourceReader. That way the interface would
> not
> > >> allow to implement an outputting loop where we never hand back control
> > to
> > >> the caller. At the moment, this contract can be e
+1 (non-binding)
Best,
Jingsong Lee
On Mon, Oct 28, 2019 at 2:48 PM Jark Wu wrote:
> Thanks for driving this Danny,
>
> +1 (binding)
>
> Best,
> Jark
>
>
> On Mon, 28 Oct 2019 at 14:26, Danny Chan wrote:
>
> > Hi all,
> >
> > I would like to start the vote for FLIP-70[1] which is discussed
des.
> >
> > Best,
> > Jark
> >
> > [1]:
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Introduction-of-a-Table-API-Java-Expression-DSL-td27787.html
> <
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Int
Hi Peter,
Thanks for your proposal. The first thing I care about most is whether it
can cover the needs of hive.
Hive create function:
CREATE FUNCTION [db_name.]function_name AS class_name
[USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri'] ];
Hive support a list of resources,
Congratulations Becket!
Best,
Jingsong Lee
On Tue, Oct 29, 2019 at 10:18 AM Terry Wang wrote:
> Congratulations, Becket!
>
> Best,
> Terry Wang
>
>
>
> > 2019年10月29日 10:12,OpenInx 写道:
> >
> > Congratulations Becket!
> >
> > On Tue, Oct 29, 2019 at 10:06 AM Zili Chen wrote:
> >
> >>
Hi bowen,
I think the reason is that flink-orc was developed earlier than
flink-formats.
Thanks the responses of everyone, I will create JIRA to do it.
On Wed, Oct 30, 2019 at 10:27 AM Danny Chan wrote:
> +1 to move to flink-format.
>
> Best,
> Danny Chan
> 在 2019年10月29日 +0800
+1 (non-binding)
Best,
Jingsong Lee
On Wed, Oct 23, 2019 at 9:02 PM Yu Li wrote:
> +1 (non-binding)
>
> Best Regards,
> Yu
>
>
> On Wed, 23 Oct 2019 at 16:56, Haibo Sun wrote:
>
> > +1 (non-binding)Best,
> > Haibo
> >
> >
> > At 2019-10-23 09:07:41, "Becket Qin" wrote:
> > >+1 (binding)
> >
+1 (non-binding)
Best,
Jingsong Lee
On Mon, Oct 21, 2019 at 11:36 PM Rong Rong wrote:
> +1 (binding)
>
> Thanks Timo for driving this.
>
> --
> Rong
>
> On Mon, Oct 21, 2019 at 8:19 AM wrote:
>
> > +1 (binding)
> >
> > Best,
> > Xingcan
> >
> > -Original Message-
> > From: jincheng
Hi Bowen:
Thanks for you proposal.
You mean even if there is no catalog, users can completely walk through the
set of temporary objects and work well?
- If it is, I am +1, in memory catalog actually is a temporary catalog, it
can not persist, we can replace it with real temporary objects.
- If
+1 (non-binding)
Best,
Jingsong Lee
On Mon, Nov 18, 2019 at 7:59 PM Becket Qin wrote:
> +1 (binding) on having the test suite.
>
> BTW, it would be good to have a few more details about the performance
> tests. For example:
> 1. How do the testing records look like? The size and key
+1 (non-binding)
Thanks Jark for driving this.
Best,
Jingsong Lee
On Wed, Nov 20, 2019 at 6:25 PM Jark Wu wrote:
> +1 (binding)
>
> Best,
> Jark
>
> On Wed, 20 Nov 2019 at 18:11, Terry Wang wrote:
>
> > +1 (non-binding)
> >
> > Best,
> > Terry Wang
> >
> >
> >
> > > 2019年11月20日 17:47,Dawid
Hi Stephan,
+1 to default have off-heap managed memory.
>From the perspective of batch, In our long-term performance test and online
practice:
- There is no significant difference in performance between heap and
off-heap memory. If it is a heap object, the JVM has many opportunities to
optimize
Hi leo:
Can you share why KafkaSource supports multiple topics?
Why can't we use multiple sources and then union them?
Best,
Jingsong Lee
On Wed, Nov 27, 2019 at 8:19 AM leo wrote:
> Hi,
>
> As Flink does not own the data, so it can not forbid this. It's the
> responsibility of user to
Sorry for bothering your voting.
Let's discuss in discussion thread.
Best,
Jingsong Lee
On Mon, Dec 2, 2019 at 3:32 PM Jingsong Lee wrote:
> Hi Dian:
>
> Thanks for your driving. I have some questions:
>
> - Where should these configurations belong? You have mentioned
> tableApi/SQL, so should
Hi Forward:
Document looks good to me.
I think you can just start doing this.
They all work very independently, so I don't think there's any obvious
blocking.
Best,
Jingsong Lee
On Sat, Nov 30, 2019 at 10:59 AM Forward Xu wrote:
> Hi everyone, It's been a long time since I started this
> [1]:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=120731026
>
> On Wed, 30 Oct 2019 at 11:12, Jingsong Li wrote:
>
> > Hi bowen,
> > I think the reason is that flink-orc was developed earlier than
> > flink-formats.
> >
> > Thanks
Hi all:
We already have the parent model of formats. we have put other
formats(flink-avro, flink-json, flink-parquet, flink-json, flink-csv,
flink-sequence-file) to flink-formats. flink-orc is a format too. So we can
move it to flink-formats.
In theory, there should be no compatibility problem,
+1 for the idea. Thanks Yu for driving this.
Just curious about that can we collect the metrics about Job scheduling and
task launch. the speed of this part is also important.
We can add tests for watch it too.
Look forward to more batch test support.
Best,
Jingsong Lee
On Mon, Nov 4, 2019 at
+1 to disable, It is already introduced by new type system in TimestampType.
I think it is time to update document too.
Best,
Jingsong Lee
On Fri, Nov 22, 2019 at 6:05 PM Kurt Young wrote:
> +1 to disable, we also need to highlight this in 1.10 release notes.
>
> Best,
> Kurt
>
>
> On Fri, Nov
+1 (non-binding) Thanks Dawid for driving this.
Best,
Jingsong Lee
On Thu, Nov 14, 2019 at 6:09 PM Gyula Fóra wrote:
> +1 (binding)
>
> On Thu, Nov 14, 2019 at 11:07 AM Jeff Zhang wrote:
>
> > +1 (non-binding)
> >
> > Kostas Kloudas 于2019年11月14日周四 下午6:04写道:
> >
> > > +1 (binding)
> > >
> > >
+1
Best,
Jingsong Lee
On Thu, Oct 10, 2019 at 3:38 PM Jark Wu wrote:
> +1
>
> Thanks,
> Jark
>
> On Wed, 9 Oct 2019 at 01:03, Xuefu Z wrote:
>
> > +1
> >
> > On Tue, Oct 8, 2019 at 7:00 AM Aljoscha Krettek
> > wrote:
> >
> > > +1
> > >
> > > > On 8. Oct 2019, at 15:35, Timo Walther wrote:
>
Thanks Timo for your pretty nice proposal, big +1 to the FLIP. Left some
minor comments.
A minor concern about flink-planner, precision things maybe cannot be
supported.
Best,
Jingsong Lee
On Tue, Oct 8, 2019 at 5:58 PM zha...@lenovocloud.com <
zha...@lenovocloud.com> wrote:
> unsubscribe
>
>
Hi Stephan,
big +1 for this contribution. It provides another user interface that is
easy to use and popular at this time. these functions, It's hard for users
to write in SQL/TableApi, while using DataStream is too complex. (We've
done some stateFun kind jobs using DataStream before). With
+1 (non-binding)
- Check if checksums files match the corresponding release files
- Check if GPG files match the corresponding release files
- Verify that the source archives do not contains any binaries
- Build the source with Maven to ensure all source files have Apache headers
- Check that all
Thanks Jark for bringing this.
+1 to use a unify name: "Interval Join" before 1.10 is release.
I think maybe "Interval Join" was come from SQL world too in [1].
Another candidate is to use "Range Join", But considering DataStream, I am
OK with "Interval".
[1]
Thanks Yingjie for driving.
It is very useful to have this check list.
I think we can list all problematic third-party libraries.
Including hadoop jar:
org.apache.hadoop.fs.FileSystem.StatisticsDataReferenceCleaner.
Because there are too many libraries with this problem. And our Yarn mode
perJob
Hi all,
I think current design is good.
My understanding is:
For execution mode: bounded mode and continuous mode, It's totally
different. I don't think we have the ability to integrate the two models at
present. It's about scheduling, memory, algorithms, States, etc. we
shouldn't confuse them.
hat’s why I’m not
> saying that we do not want to handle this at some point in the future, but
> at least not in the first version.
>
> Piotrek
>
> > On 5 Dec 2019, at 10:11, Jingsong Li wrote:
> >
> > Kurt mentioned a very interesting thing,
> >
> > If we want
Hi boqi,
Now, you don't need permission now.
For JIRA creation, you can create JIRAs by yourself to report bugs or
improvements.
For JIRA assginment, only committers can assign JIRA to someone, if you
want to contribute some code, you can comment JIRA, and some committers can
help you to assign
Hi Dawid,
Thanks for driving.
- adding $ in scala api looks good to me.
- Just a question, what should be expected to java.lang.Object? literal
object or expression? So the Object is the grammatical sugar of literal?
Best,
Jingsong Lee
On Mon, Feb 10, 2020 at 9:40 PM Timo Walther wrote:
> +1
; - Can we only configure one parameter and calculate another
> > automatically? For example, if we just want to "pipeline", "bundle.size"
> is
> > twice as much as "batch.size", is this work?
> >
> > I don't think this works. These
Thanks all for your discussion.
Hi Dawid,
+1 to apply the logic of parsing a SQL timestamp literal.
I don't fully understand the matrix your list. Should this be the semantics
of SQL cast?
Do you mean this is implicit cast in JSON parser?
I doubt that because these implicit casts are not
; Best,
> Jark
>
> On Wed, 26 Feb 2020 at 22:20, Jingsong Li wrote:
>
>> Thanks all for your discussion.
>>
>> Hi Dawid,
>>
>> +1 to apply the logic of parsing a SQL timestamp literal.
>>
>> I don't fully understand the matrix your list. Should
Thanks for deep investigation.
+1 to disable "Squash and merge" button now.
But I think this is a very serious problem, It affects too many GitHub
workers. Github should deal with it quickly?
Best,
Jingsong Lee
On Thu, Mar 5, 2020 at 7:21 PM Xingbo Huang wrote:
> Hi Jark,
>
> Thanks for
Thanks Dawid for starting this discussion.
I like the "LIKE".
1.For "INHERITS", I think this is a good feature too, yes, ALTER TABLE will
propagate any changes in column data definitions and check constraints down
the inheritance hierarchy. A inherits B, A and B share every things, they
have the
module name can be "flink-connector-hive-1.0"
> rather than "flink-connector-hive-1.2"
>
>
> On Wed, Mar 4, 2020 at 10:20 PM Jingsong Li
> wrote:
>
> > Thanks Bowen for involving.
> >
> > > why you proposed segregating hive versions into t
Thanks Bowen for involving.
> why you proposed segregating hive versions into the 5 ranges above? &
what different Hive features are supported in the 5 ranges?
For only higher client dependencies version support lower hive metastore
versions:
- Hive 1.0.0 - 1.2.2, thrift change is OK, only hive
g the
> lowest Hive version it supports.
>
> What do you think?
>
>
>
> On Wed, Mar 4, 2020 at 11:14 PM Jingsong Li
> wrote:
>
> > Hi Bowen, thanks for your reply.
> >
> > > will there be a base module like "flink-connector-hive-base" which
&g
;>>>> in
> >>>>>> the repository(according to github rules).
> >>>>>>
> >>>>>> If we only left "merge and commits" button, it will
> >>>>>> against requiring
> >> a
> >>&
+1 for this proposal. I have a lot of desired topics in table and batch.
I also second Seth and Stephan 's comment separate this in a clear way.
Have concerns that maybe easy to confuse new users.
If I am a beginner and find a bunch of deep documents, I need to further
distinguish which is
Congratulations Yu, well deserved!
And thanks for your great contribution to the 1.10 release.
Best,
Jingsong Lee
On Thu, Jan 23, 2020 at 6:14 PM Fabian Hueske wrote:
> Congrats Yu!
> Good to have you on board!
>
> Cheers, Fabian
>
> Am Do., 23. Jan. 2020 um 11:13 Uhr schrieb Piotr Nowojski <
Hi Jeff,
For FLINK-15935 [1],
I try to think of it as a non blocker. But it's really an important issue.
The problem is the class loading order. We want to load the class in the
blink-planner.jar, but actually load the class in the flink-planner.jar.
First of all, the order of class loading
r to keep it this way for a while, and see if there're
>> real concerns/complaints from user feedbacks.
>>
>> Please also note the Hive jars are not the only ones needed to integrate
>> with Hive, users have to make sure flink-connector-hive and Hadoop jars
>> are
> you can cover a lot of versions.
>
> Would it make sense to add these to flink-shaded (with proper dependency
> exclusions of unnecessary dependencies) and offer them as a download,
> similar as we offer pre-shaded Hadoop downloads?
>
> Best,
> Stephan
>
>
> On Thu, Fe
Hi,
+1 (non-binding) Thanks for driving this, Gary & Yu.
- Verified signatures and checksums
- Maven build from source skip tests
- Start local cluster and web ui is accessible
- Submit example of both batch and streaming, run well and log no exception.
- Verified pom files point to the
+1 (non-binding)
Official docker support really looks good to me.
Best,
Jingsong Lee
On Fri, Jan 31, 2020 at 12:55 AM aihua li wrote:
> +1 (non-binding)
>
> > 2020年1月30日 下午7:36,Igal Shilman 写道:
> >
> > +1 (non-binding)
> >
> > On Thu, Jan 30, 2020 at 12:18 PM Yu Li wrote:
> >
> >> +1
Hi Zhenghua,
The *getRecordDataType* looks good to me.
But the main problem is how to represent the tuple type in DataType. I
understand that it is necessary to use StructuredType, but at present,
planner does not support StructuredType, so the other way is to support
StructuredType.
Best,
();
/**
* @return Input whether or not it is bounded.
*/
boolean isBounded();
}
}
If there is no objection, I will start a vote thread. (if necessary, I can
also edit a FLIP).
Best,
Jingsong Lee
On Thu, Jan 16, 2020 at 7:56 PM Jingsong Li wrote:
> Thanks Bowen and Timo for involving.
>
.
*/
ReadableConfig getTableConfig();
}
}
Best,
Jingsong Lee
On Tue, Feb 4, 2020 at 1:22 PM Jingsong Li wrote:
> Hi all,
>
> After rethinking and discussion with Kurt, I'd like to remove "isBounded".
> We can delay this is bounded message to TableSink.
> With Ta
+1 (non-binding), thanks for driving.
FLIP-27 is the basis of a lot of follow-up work.
Best,
Jingsong Lee
On Tue, Feb 4, 2020 at 10:26 AM Jark Wu wrote:
> Thanks for driving this Becket!
>
> +1 from my side.
>
> Cheers,
> Jark
>
> On Mon, 3 Feb 2020 at 18:06, Yu Li wrote:
>
> > +1, thanks for
Hi all,
After rethinking and discussion with Kurt, I'd like to remove "isBounded".
We can delay this is bounded message to TableSink.
With TableSink refactor, we need consider "consumeDataStream"
and "consumeBoundedStream".
Best,
Jingsong Lee
On Mon, Feb 3, 2020
Hi all,
I would like to start the vote for the improve of
TableFactory, which is discussed and
reached a consensus in the discussion thread[2].
The vote will be open for at least 72 hours. I'll try to close it
unless there is an objection or not enough votes.
[1]
Another critical issue is FLINK-15858[1].
It is indeed a regression. But we don''t want to block release.
Will try our best to fix it.
[1] https://issues.apache.org/jira/browse/FLINK-15858
Best,
Jingsong Lee
On Tue, Feb 4, 2020 at 9:56 AM Thomas Weise wrote:
> I opened a PR for FLINK-15868
>
Hi all,
For your information, we have document the dependencies detailed
information [1]. I think it's a lot clearer than before, but it's worse
than presto and spark (they avoid or have built-in hive dependency).
I thought about Stephan's suggestion:
- The hive/lib has 200+ jars, but we only
Hi all,
As Jark suggested in VOTE thread.
JIRA created: https://issues.apache.org/jira/browse/FLINK-15912
Best,
Jingsong Lee
On Wed, Feb 5, 2020 at 10:57 AM Jingsong Li wrote:
> Hi Timo,
>
> G ood catch!
>
> I really love the idea 2, a full Flink config looks very good
Hi Dian,
+1 for this, thanks driving.
Documentation looks very good. I can imagine a huge performance improvement
and better integration to other Python libraries.
A few thoughts:
- About data split: "python.fn-execution.arrow.batch.size", can we unify it
with "python.fn-execution.bundle.size"?
HI Kurt,
+1 to remove these methods.
But one concern is that some of the current TableSource/TableSink may not
be ready, such as the JDBCUpsertTableSink, which accepts a JDBCDialect, but
through the TableFactory, there is no way to pass in the JDBCDialect at
present. But I also believe we have
config but might give access to the full
> Flink config
>
> Thanks,
> Timo
>
>
> On 04.02.20 06:27, Jingsong Li wrote:
> > So the interface will be:
> >
> > public interface TableSourceFactory extends TableFactory {
> > ..
> >
> >
Hi all,
Interface updated.
Please re-vote.
Best,
Jingsong Lee
On Tue, Feb 4, 2020 at 1:28 PM Jingsong Li wrote:
> Hi all,
>
> I would like to start the vote for the improve of
> TableFactory, which is discussed and
> reached a consensus in the discussion thread[2].
>
>
Thanks Jincheng,
FLINK-15840 [1] should be a blocker, lead to
"TableEnvironment.from/scan(string path)" cannot be used for all
temporaryTable and catalogTable (not DataStreamTable). Of course, it can be
bypassed by "TableEnvironment.sqlQuery("select * from t")", but "from/scan"
are very important
Thanks for bringing this discussion.
+1 to peforming this big change as early as possible.
You solved my question, why we need "_root_". Yes, I don't like this import
too.
And it is very strange that expressionDsl is in api, but can only work in
api.scala. (Because scala extends
Hi all, thanks for launching this discussion.
About eliminating Google Docs. I agree with Zhijiang, I share my concern
about it.
If the FLIP Driver is a Flink newer or the FLIP is very big and
complicated. His/Her design maybe need change many many things, in this
situation, Google doc is good
Hi Caizhi, thanks for starting this discussion.
There is a FLIP-71 [1] to describe the whole story of view.
Sql-cli now implements a wrong way, and a separate way, which should be
deprecated and unified to TableEnvironment.
> Shall we make it clear and support create table / create temporary
we need a new VOTE for this, I just want to make this
> discussion more publicly.
> What do you think?
>
> Best,
> Jark
>
> On Wed, 5 Feb 2020 at 16:05, Rui Li wrote:
>
> > +1, thanks for the efforts.
> >
> > On Wed, Feb 5, 2020 at 4:00 PM Jingsong Li
Thanks everyone~
It's my pleasure to be part of the community. I hope I can make a better
contribution in future.
Best,
Jingsong Lee
On Fri, Feb 21, 2020 at 2:48 PM Hequn Cheng wrote:
> Congratulations Jingsong! Well deserved.
>
> Best,
> Hequn
>
> On Fri, Feb 21, 2020 at 2:42 PM Yang Wang
t; >> +1 for this.
> >>
> >> I have some minor comments:
> >> - I'm +1 to use $ in both Java and Scala API.
> >> - I'm +1 to use lit(), Spark also provides lit() function to create a
> >> literal value.
> >> - Is it possible to have `i
Hi,
+1 (non-binding) Thanks for driving this, Gary & Yu.
There is an unfriendly error here: "OutOfMemoryError: Direct buffer memory"
in FileChannelBoundedData$FileBufferReader.
It forces our batch users to configure
"taskmanager.memory.task.off-heap.size" in production jobs. And users are
rovided by Arrow, we can gain them
> > > automatically using Arrow.
> > > 3. Good point! As all the classes of Python module is written in Java
> and
> > > it's not suggested to introduce new Scala classes, so I guess it's not
> > easy
> > > to do so right now
w, could you also attach a JIRA issue with the changes described
> in
> > > it,
> > > >>> so that users can find the issue through the mailing list in the
> > > future.
> > > >>>
> > > >>> Best,
> > > >>> Jark
> >
Thank for the great work,
In 1.10, I have modified and reviewed some documents. In that process,
sometimes there is some confusion, how to write is the standard. How to
write is correct to the users.
Docs style now tells me. Learned a lot.
Best,
Jingsong Lee
On Sat, Feb 15, 2020 at 10:00 PM
1 - 100 of 596 matches
Mail list logo