ized View -> modifying a view directly is a little weird.
Thanks,
Jiangjie (Becket) Qin
On Tue, Apr 9, 2024 at 5:46 AM Lincoln Lee wrote:
> Thanks Ron and Timo for your proposal!
>
> Here is my ranking:
>
> 1. Derived table -> extend the persistent semantics of deri
this FLIP will help improve the user experience in
format development (It makes the implementation of FLIP-358[2] much
easier). And it also makes the table-datastream conversion more usable.
Comments are welcome!
Thanks,
Jiangjie (Becket) Qin
[1]
https://cwiki.apache.org/confluence/display/FLINK
+1 (binding)
Thanks,
Jiangjie (Becket) Qin
On Fri, Jan 12, 2024 at 5:58 AM Zhijiang
wrote:
> +1 (binding)
> Best,
> Zhijiang
> --
> From:Kurt Yang
> Send Time:2024年1月12日(星期五) 15:31
> To:dev
> Subject:R
y exposing constructors of SplitFetcherManager as public APIs, and
>> adding a new internal method SplitFetcherManager#getQueue() for
>> SourceReaderBase (well it's a bit hacky I admit but I think exposing
>> methods like poll and notifyAvailable on SplitFetcherManager is even
>>
t;,
where we can add more details to explain the changes listed in the public
interface section.
Thanks,
Jiangjie (Becket) Qin
On Wed, Dec 20, 2023 at 10:07 AM Hongshun Wang
wrote:
> Hi Becket,
>
>
> It has been a long time since we last discussed. Are there any other
> problems wit
.
Cheers,
Jiangjie (Becket) Qin
On Tue, Dec 19, 2023 at 11:00 PM Jiabao Sun
wrote:
> Hi Becket,
>
> I share the same view as you regarding the prefix for this configuration
> option.
>
> For the JDBC connector, I prefer setting 'filter.handling.policy' = 'FOO'
> and throw
have INDEXED_ONLY while
> parquet format may have something else. *
>
I found this is somewhat misleading, because the example here is not a part
of the proposal of this FLIP. It is just an example explaining when a
prefix is needed, which seems orthogonal to the proposal in this FLIP.
Thanks
; connector.*
What do you mean by "native configuration"? From what I understand, the
FLIP does the following:
- introduces a new configuration to the JDBC and MongoDB connector.
- Suggests a convention option name if other connectors are going to add an
option for the same purpose.
Than
Yes, that sounds reasonable to me. We can start with ALWAYS and NEVER, and
add more policies as needed.
Thanks,
Jiangjie (Becket) Qin
On Mon, Dec 18, 2023 at 4:48 PM Jiabao Sun
wrote:
> Thanks Bucket,
>
> The jdbc.filter.handling.policy is good to me as it provides sufficient
> e
ns when a filter cannot be applied to the external
system. But personally I don't see much value in doing this.
Thanks,
Jiangjie (Becket) Qin
On Mon, Dec 18, 2023 at 3:54 PM Jiabao Sun
wrote:
> Hi Becket,
>
> The MySQL connector is currently in the flink-connector-jdbc repository
>
ementation of source. For MySQL, if we are going to introduce a config
to MySql, why not have something like "mysql.filter.handling.policy" with
value of AUTO / NEVER / ALWAYS? Isn't that better than
"ignore.filter.pushdown"?
Thanks,
Jiangjie (Becket) Qin
On Sun, Dec 17, 2023
Hi Peter,
Thanks for updating the patch. The latest patch looks good to me. I've +1ed
on the PR.
Cheers,
Jiangjie (Becket) Qin
On Mon, Dec 11, 2023 at 9:21 PM Péter Váry
wrote:
> Thanks everyone for the lively discussion!
>
> The PR is available which strictly adheres the accepte
the StatefulSink and
TwoPhaseCommittingSink.
Thanks,
Jiangjie (Becket) Qin
On Mon, Dec 4, 2023 at 7:25 PM Gyula Fóra wrote:
> Hi All!
>
> Based on the discussion above, I feel that the most reasonable approach
> from both developers and users perspective at this point is what Becket
> li
for the connector nightly build to depend on the
latest snapshot of the same Flink major version. It helps catching
unexpected breaking changes sooner.
- I'll update the website to reflect the latest API stability policy.
Apologies for the confusion caused by the stale doc.
Thanks,
Jiangjie (Becket
class as PublicEvolving, the constructor is not available to
the users. Only the public and protected methods are considered public API
in this case. Private / package private methods and fields are still
internal.
Thanks,
Jiangjie (Becket) Qin
On Wed, Nov 22, 2023 at 9:46 AM Hongshun Wang
wrote
, but might not be
necessary in this case.
Thanks,
Jiangjie (Becket) Qin
On Tue, Nov 21, 2023 at 10:36 AM Hongshun Wang
wrote:
> Hi Becket,
>
> > Additionally, SplitFetcherTask requires FutureCompletingBlockingQueue as
> a constructor parameter, which is not allowed now.
> Sorry,
FutureCompletingBlockingQueue
> as a
> constructor parameter, which is not allowed
> now.
Are you referring to FetchTask which implements SplitFetcherTask? That
class will remain internal.
Thanks,
Jiangjie (Becket) Qin
On Fri, Nov 17, 2023 at 5:23 PM Hongshun Wang
wrote:
> Hi, Jiangjie(Becket) ,
,
Jiangjie (Becket) Qin
On Thu, Nov 16, 2023 at 8:26 PM Hongshun Wang
wrote:
> Hi Devs,
>
> I have just modified the content of FLIP-389: Annotate
> SingleThreadFetcherManager as PublicEvolving[1].
>
> Now this Flip mainly do two thing:
>
>1. Annotate SingleThreadFetcherM
itable
common config for this. The behavior is very likely source specific.
Thanks,
Jiangjie (Becket) Qin
On Thu, Nov 16, 2023 at 3:41 PM Jiabao Sun
wrote:
> Thanks Becket,
>
> I still believe that adding a configuration at the source level to disable
> filter pushdown is needed. This
meets my bar for the reason stated
above.
Therefore, my suggestion remains the same. Keep the config as a Source
implementation specific configuration.
Thanks,
Jiangjie (Becket) Qin
On Thu, Nov 16, 2023 at 12:36 AM Jiabao Sun
wrote:
> Thanks Becket for the feedback,
>
> Regarding
ific Source implementation.
For the two reasons above, I am -1 to what the FLIP currently proposes.
I think the right way to address the motivation here is still to have a
config like "ignore-pushed-down-filters.enabled" for the specific source
implementation. Please let me know if this s
oks like we can just let
SplitFetcherManager.poll() return a RecordsWithSplitIds.
Thanks,
Jiangjie (Becket) Qin
On Tue, Nov 14, 2023 at 8:11 PM Hongshun Wang
wrote:
> Hi Becket,
> I agree with you and try to modify this Flip[1], which include these
> changes:
>
>1. Mark constructor o
the methods via SplitFetcherManager. I believe
this also makes the code slightly cleaner.
Thanks,
Jiangjie (Becket) Qin
On Fri, Nov 10, 2023 at 12:28 PM Hongshun Wang
wrote:
> @Martijn, I agree with you.
>
>
> I also have two questions at the beginning:
>
>- Why is an Internal
to push back the pushed down filters, even
though this is never desired.
None of the above options looks awkward. I am curious what your solution is
here?
Thanks,
Jiangjie (Becket) Qin
On Tue, Oct 31, 2023 at 3:11 PM Jiabao Sun
wrote:
> Thanks Becket for the further explanation.
>
> Perhaps
be called. There should be no
other config to disable that call. However, Sources can decide how to
behave based on their own configurations after *applyFilters()* is called.
And these configs are specific to those sources, instead of common configs.
Please see the examples I mentioned in the previous ema
the MySql source itself.
As you may see, the above configurations do not disable filter pushdown
itself. They just allow various implementations of filter pushdown. And the
configuration name does not give any illusion that filter pushdown is
disabled.
Thanks,
Jiangjie (Becket) Qin
On Mon, Oct 30
Thanks for digging into the git history, Jark. I agree it makes sense to
deprecate this API in 2.0.
Cheers,
Jiangjie (Becket) Qin
On Fri, Oct 27, 2023 at 5:47 PM Jark Wu wrote:
> Hi Becket,
>
> I checked the history of "
> *table.optimizer.source.predicate-pushdown-enab
t sure in which case this configuration
would be used. Any ideas @Jark Wu ?
Thanks,
Jiangjie (Becket) Qin
On Wed, Oct 25, 2023 at 11:55 PM Jiabao Sun
wrote:
> Thanks Jane for the detailed explanation.
>
> I think that for users, we should respect conventions over configurations.
> C
Hi Martijn,
This FLIP has passed voting[1]. It is a modification on top of the FLIP-95
interface.
Thanks,
Jiangjie (Becket) Qin
[1] https://lists.apache.org/thread/hysv9y1f48gtpr5vx3x40wtjb6cp9ky6
On Wed, Sep 20, 2023 at 9:29 PM Martijn Visser
wrote:
> For clarity purposes, this F
+1 (binding)
Thanks for the FLIP, Archit.
Cheers,
Jiangjie (Becket) Qin
On Thu, Sep 14, 2023 at 10:31 AM Dong Lin wrote:
> Thanks Archit for the FLIP.
>
> +1 (binding)
>
> Regards,
> Dong
>
> On Thu, Sep 14, 2023 at 1:47 AM Archit Goyal >
> wrote:
>
es /
> > classes without annotations as effectively @Internal.
> >
>
> Does it make sense to clearly define that APIs without annotation are
> internal APIs and should be used outside of Flink. And deprecate @Internal?
>
> Best regards,
> Jing
>
> On Mon,
n the future. It is a good practice to create separate interfaces should
be created for the users in this case.
Regarding the API stability promotion, you may want to check the
FLIP-197[1].
Thanks,
Jiangjie (Becket) Qin
[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-197%3A+API+stabilit
NOT be marked as
deprecated. Instead, an immediate refactor should be done to remove the
"deprecated" internal methods / classes / interfaces, and migrate the code
to its successor. Otherwise, technical debts will build up.
Thanks,
Jiangjie (Becket) Qin
On Sat, Sep 9, 2023 at 5:29
hema_A and
AvroSchema_B are compatible. In your case, the Avro schema stored in the
schema registry will be that original Avro schema, i.e. AvroSchema_A.
Thanks,
Jiangjie (Becket) Qin
On Wed, Sep 6, 2023 at 8:32 PM 吴 stephen wrote:
> Hi Becket,
> I notice that a new config will introduce to
still evolve or not, as Deprecated
basically means "public, no more change, to be removed", while
PublicEvolving indicates still evolving. But I guess by intuition users
will just consider the evolving part overridden by the deprecation. Maybe
it is fine to keep both. I'll update the FLIP.
Th
Thanks for pushing the FLIP through.
+1 on the updated FLIP wiki.
Cheers,
Jiangjie (Becket) Qin
On Wed, Sep 6, 2023 at 1:12 PM Venkatakrishnan Sowrirajan
wrote:
> Based on the recent discussions in the thread [DISCUSS] FLIP-356: Support
> Nested Fields Filter Pushdown
&g
have here in
the NestedFieldReferenceExpression.
Thanks,
Jiangjie (Becket) Qin
On Fri, Sep 1, 2023 at 8:12 AM Venkatakrishnan Sowrirajan
wrote:
> Gentle ping on the vote for FLIP-356: Support Nested fields filter pushdown
> <https://www.mail-archive.com/dev@flink.apache.org/msg69289.html>.
>
>
ve the existing code base using it, while introducing a
new method. This is a bad practice adding to technical debts. Instead, a
proper refactor should be done immediately in the same patch to just remove
that private method and migrate all the usage to the new method.
Thanks,
Jiangjie (Becket) Q
class has to be considered as a
ConversionClass of the VARCHAR type, and a StringToEnum converter is
required.
And yes, AvroSchemaUtils should be annotated as @PublicEvolving.
Thanks,
Jiangjie (Becket) Qin
On Thu, Aug 31, 2023 at 5:22 PM Ryan Skraba
wrote:
> Hey -- I have a certain know
Avro use cases out of the box.
Feedbacks are welcome!
Thanks
Jiangjie (Becket) Qin
[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-358%3A+flink-avro+enhancement+and+cleanup
ties.
This helps lower the bar of understanding, reduces the chance of having
loose ends in the API, and will benefit all the participants in the project
over time. I think this is the right way to achieve real API stability.
Otherwise, we may end up chasing our tails to find ways not to change the
even be visible to the cluster. The cluster only knows if a client is
attached or not when a client submits a job.
Thanks,
Jiangjie (Becket) Qin
On Wed, Aug 23, 2023 at 2:35 PM Weihua Hu wrote:
> Hi, Jiangjie
>
> Thanks for the clarification.
>
> My key point is the meaning of the "
ched=true" would be equivalent to
"execution.shutdown-on-attached-exit=false".
2. Current "execution.attached=false" would be equivalent to
"execution.shutdown-on-attached-exit=true" +
"client.attached.after.submission=true".
This will make the same conf
the record being processed is a top level record or nested record.
Thanks,
Jiangjie (Becket) Qin
On Tue, Aug 22, 2023 at 11:43 PM Jark Wu wrote:
> Hi Becket,
>
> I totally agree we should try to have a consistent API for a final state.
> The only concern I have mentioned is
(Becket) Qin
On Tue, Aug 22, 2023 at 2:20 PM Weihua Hu wrote:
> Hi Jiangjie
>
> Sorry for the late reply, I fully agree with the three user sensible
> behaviors you described.
>
> I would like to bring up a point.
>
> Currently, 'execution.attached' is not only
lly, I
prefer the latter if we are designing from scratch. It is clean, consistent
and intuitive. Given the size of Flink, keeping APIs in the same style over
time is important. The migration is also not that complicated.
Thanks,
Jiangjie (Becket) Qin
On Tue, Aug 22, 2023 at 2:23 PM Jark Wu w
prefer this alternative. It takes longer to finish the work,
but the API eventually becomes clean and consistent. But I can live with
the current proposal.
Thanks,
Jiangjie (Becket) Qin
On Sat, Aug 19, 2023 at 12:09 AM Venkatakrishnan Sowrirajan <
vsowr...@asu.edu> wrote:
> Ge
, the execution.attached configuration is only
honored by the client, but not the cluster. Therefore, I think removing it
makes sense.
Thanks,
Jiangjie (Becket) Qin
On Thu, Aug 17, 2023 at 12:31 AM liu ron wrote:
> Hi, Jiangjie
>
> Sorry for late reply. Thank you for such a detailed response. A
the use
cases that the optimization helps are important enough. And this judgement
is somewhat subjective.
Thanks,
Jiangjie (Becket) Qin
On Mon, Aug 14, 2023 at 9:13 PM Jark Wu wrote:
> Hi Becket,
>
> > I kind of think that we can
> restrain the scope to just batch mode, and only f
that it is
a configuration controlling the client side behavior, instead of the
execution of the job.
Thanks,
Jiangjie (Becket) Qin
On Thu, Aug 10, 2023 at 10:34 PM Weihua Hu wrote:
> Hi Allison
>
> Thanks for driving this FLIP. It's a valuable feature for batch jobs.
> This helps keep "Drop Per-Jo
or not, e.g. batch mode, processing time
only jobs, etc.
Thanks,
Jiangjie (Becket) Qin
On Fri, Aug 11, 2023 at 9:46 PM Dong Lin wrote:
> Hi Xintong,
>
> Thanks for the quick reply. I also agree that we should hear from others
> about
> whether this optimization is worthwhile.
>
> P
don't have to address it right away
in the same FLIP, this kind of debt accumulates over time and makes the
project harder to learn and maintain. So, personally I prefer to address
these technical debts as soon as possible.
Thanks,
Jiangjie (Becket) Qin
On Wed, Aug 2, 2023 at 8:19 PM Jark Wu wrote
, DataType
producedDataType)
This will need a FLIP.
Thanks,
Jiangjie (Becket) Qin
On Tue, Aug 1, 2023 at 11:42 PM Venkatakrishnan Sowrirajan
wrote:
> Thanks for the response. Looking forward to your pointers. In the
> meanwhile, let me figure out how we can implement it. Will keep you
Thanks everyone for voting.
We have got 11 approving votes and no disapproving votes. FLIP-321 has now
passed!
The voting details are following:
9 binding votes:
Dong Lin
Xintong Song
Martijn Visser
Stefan Richter
Jing Ge
Matthias Pohl
Zhu Zhu
Jingsong Li
Becket Qin
2 non-binding votes
Hi folks,
I'd like to start the VOTE for FLIP-321[1] which proposes to introduce an
API deprecation process to Flink. The discussion thread for the FLIP can be
found here[2].
The vote will be open until at least July 4, following the consensus voting
process.
Thanks,
Jiangjie (Becket) Qin
[1
anges.
+1 on having toolings to enforce the conventions.
Thanks,
Jiangjie (Becket) Qin
On Wed, Jun 28, 2023 at 5:09 AM Martijn Visser
wrote:
> Hi all,
>
> Thanks for the lively and good discussion. Given the length of the
> discussion, I skimmed through and then did a deep di
that to the
FLIP wiki. This is probably more of a clarification on the existing
convention, rather than a change.
It looks like we are on the same page now for this FLIP. If so, I'll start
a VOTE thread in two days.
Thanks,
Jiangjie (Becket) Qin
On Mon, Jun 26, 2023 at 8:09 PM Xintong Song wrote
speaking, I don't feel this assumption is necessarily true. We
should re-evaluate once we have the new ProcessFunction API in place.
Without the code it is hard to tell for sure. I am actually kind of
optimistic about the maintenance cost.
Thanks,
Jiangjie (Becket) Qin
On Sun, Jun 25, 2023 at 11:30 AM
Thanks much for the input, John, Stefan and Jing.
I think Xingtong has well summarized the pros and cons of the two options.
Let's collect a few more opinions here and we can move forward with the one
more people prefer.
Thanks,
Jiangjie (Becket) Qin
On Wed, Jun 21, 2023 at 3:20 AM Jing Ge
So, if there are concrete examples that you think will block us from
keeping API stability with affordable cost, let's take a look together and
see if that can be improved.
Thanks,
Jiangjie (Becket) Qin
>From what I see,
On Mon, Jun 19, 2023 at 4:45 PM Xintong Song wrote:
> >
&g
nor version bump to remove an API, what do we lose to do
a major version bump instead, so we don't break the common versioning
semantic?
Thanks,
Jiangjie (Becket) Qin
On Mon, Jun 19, 2023 at 3:20 PM Xintong Song wrote:
> >
> > As an end user who only uses Public APIs, if I don't
ore comfortable with the maintenance overhead of deprecated APIs, we can
then have a stronger guarantee for Experimental / PublicEvolving APIs.
Thanks,
Jiangjie (Becket) Qin
On Sun, Jun 18, 2023 at 6:44 AM John Roesler wrote:
> Hi Becket,
>
> Thanks for this FLIP! Having a deprecation proces
ays and a major version bump ensures there is no surprise.
Thanks,
Jiangjie (Becket) Qin
On Fri, Jun 16, 2023 at 10:13 AM Xintong Song wrote:
> Public API is a well defined common concept, and one of its
>> convention is that it only changes with a major version change.
>>
>
&g
again for raising these examples. This is a good discussion, as we
are getting to some root causes of our hesitation about the API stabilities.
Thanks,
Jiangjie (Becket) Qin
On Fri, Jun 16, 2023 at 10:13 AM Xintong Song wrote:
> Public API is a well defined common concept, and one of its
nance overhead for us
as Flink maintainers. When there is a conflict and no way around, having
some trade-off is reasonable. However, in this particular case, there seems
no material benefit of having a stability demotion process while it does
weaken the user experience.
Thanks,
Jiangjie (Becket) Qin
rather bump up the major version again to remove the
deprecated Public API. That seems simpler and does not complicate the well
established versioning semantic conventions.
Thanks,
Jiangjie (Becket) Qin
On Wed, Jun 14, 2023 at 9:27 PM Matthias Pohl
wrote:
> One (made-up) example from the
, and
even fix bugs in the old consumer, so additional maintenance effort is
required. But this allows the users to keep up with Kafka releases which is
extremely rewarding.
Thanks,
Jiangjie (Becket) Qin
On Wed, Jun 14, 2023 at 5:06 PM Matthias Pohl
wrote:
> Thanks for starting this discuss
to bump up
the major version to 3.0 at some point after the migration period has
passed, assuming by then most of the users have migrated away from the
deprecated Public API.
Thanks,
Jiangjie (Becket) Qin
On Wed, Jun 14, 2023 at 4:10 PM Xintong Song wrote:
> Thanks for bringing up this discu
on period in the connectors as
the flink core, if the connectors are upgraded to the latest version of
core promptly.
Thanks,
Jiangjie (Becket) Qin
[1]
https://nightlies.apache.org/flink/flink-docs-master/api/java/deprecated-list.html
On Wed, Jun 14, 2023 at 1:15 AM Jing Ge wrote:
> &
jor version, which
likely introduces non-trivial burdens. If users want new features, they
should upgrade to 2.x.
Thanks,
Jiangjie (Becket) Qin
On Tue, Jun 13, 2023 at 10:24 PM Chesnay Schepler
wrote:
> On 13/06/2023 12:50, Jing Ge wrote:
> > One major issue we have, afaiu, i
d migration to move to the new API.
3. upgrade to later Flink versions in which the code of the deprecated API
is removed.
So, it looks like our story for API stability and compatibility would be
complete with this FLIP.
Thanks,
Jiangjie (Becket) Qin
On Tue, Jun 13, 2023 at 12:30 AM Stefan Ric
deprecated API can be
removed from the source code. So with this FLIP, I'd like to kick off the
discussion about our deprecation process.
https://cwiki.apache.org/confluence/display/FLINK/FLIP-321%3A+Introduce+an+API+deprecation+process
Comments are welcome!
Thanks,
Jiangjie (Becket) Qin
[1]
ht
+1 (binding)
Thanks for driving the FLIP, Archit.
Cheers,
Jiangjie (Becket) Qin
On Tue, Jun 6, 2023 at 4:33 AM Venkatakrishnan Sowrirajan
wrote:
> Thanks for starting the vote on this one, Archit.
>
> +1 (non-binding)
>
> Regards
> Venkata krishnan
>
>
> On Mon, Ju
Thanks for the FLIP, Archit.
The motivation sounds reasonable and it looks like a straightforward
proposal. +1 from me.
Thanks,
Jiangjie (Becket) Qin
On Fri, May 12, 2023 at 1:30 AM Archit Goyal
wrote:
> Hi all,
>
> I am opening this thread to discuss the proposal to support
to be a release manager of the Flink 2.0 release.
Cheers,
Jiangjie (Becket) Qin
On Tue, Apr 25, 2023 at 7:53 PM Leonard Xu wrote:
> Thanks Xintong and Jark for kicking off the great discussion!
>
> The time goes so fast, it is already the 10th anniversary of Flink as an
> Apache project
ensure the output is
committed in a way that aligns with the user semantic.
That said, I agree it might be useful in some cases if users can decided
the checkpoint triggering timing. But that will be a new feature which
needs some careful design.
Thanks,
Jiangjie (Becket) Qin
On Mon, Feb 27,
where every
subtask does that.
Thanks,
Jiangjie (Becket) Qin
On Sat, Jan 14, 2023 at 11:46 AM Yun Tang wrote:
> +1 for this proposal and thanks Qingsheng for driving this.
>
> Considering the interval, we also set the value as 5min, equivalent to the
> default value of metadat
,
Jiangjie (Becket) Qin
On Sat, Jan 14, 2023 at 5:36 AM Konstantin Knauf wrote:
> Hi Becket,
>
> > It is a basic rule of public API that anything exposed by a public
> interface should also be public.
>
> I agree with this in general. Did you get an overview of where w
primitives we want to expose to the users. So instead we just
expose a bunch of internal things and hope users only use the "stable" part
of them.
Thanks,
Jiangjie (Becket) Qin
On Fri, Jan 13, 2023 at 7:28 PM Dawid Wysakowicz
wrote:
> Hi Becket,
>
> May I ask wha
annotation.
Comments are welcome!
Thanks,
Jiangjie (Becket) Qin
[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=240880841
Hooray!! Congratulations to the team!
Cheers,
Jiangjie (Becket) Qin
On Mon, Oct 31, 2022 at 9:57 AM Hang Ruan wrote:
> Congratulations!
>
> Best,
> Hang
>
> Shengkai Fang 于2022年10月31日周一 09:40写道:
>
> > Congratulations!
> >
> > Best,
> > Shengkai
+1 (binding)
Thanks,
Jiangjie (Becket) Qin
On Mon, Oct 31, 2022 at 10:17 AM Jark Wu wrote:
> +1 (binding)
>
> Best,
> Jark
>
> > 2022年10月29日 03:11,Jing Ge 写道:
> >
> > +1 (non-binding)
> >
> > Thanks!
> >
> > Best Regards,
> > J
backwards
compatibility tests on metrics to avoid unintended breaking changes like
this in the future.
Thanks,
Jiangjie (Becket) Qin
On Sun, Oct 9, 2022 at 10:35 AM Qingsheng Ren wrote:
> Hi devs and users,
>
> I’d like to start a discussion about reverting a breaking change about sink
+1
On Mon, Jul 25, 2022 at 9:22 AM Grant L (Grant)
wrote:
> +1
>
> On 2022/07/21 18:27:52 Robert Metzger wrote:
> > +1
> >
> > On Wed, Jul 20, 2022 at 10:48 PM Konstantin Knauf
> wrote:
> >
> > > +1. Thanks!
> > >
> > > Am Mi., 20. Juli 2022 um 16:48 Uhr schrieb Tzu-Li (Gordon) Tai <
> > >
esn't need to invoke the pauseOrResumeSplit()
method and catch the UnsupportedOperation every time. A flag can be set so
it doesn't attempt to pause the split after the first time it sees the
exception.
Thanks,
Jiangjie (Becket) Qin
On Wed, Jul 13, 2022 at 5:11 PM Sebastian Mattheis
wrote:
> H
necessary to
disable the split alignment, as long as the coarse grained alignment can be
a fallback.
Thanks,
Jiangjie (Becket) Qin
On Wed, Jul 13, 2022 at 2:46 PM Sebastian Mattheis
wrote:
> Hi Piotrek,
>
> Sorry I've read it and forgot it when I was ripping out the
> supportsPauseOrResum
binary files.
- verified the pom files and the flink-ml versions are updated.
Thanks,
Jiangjie (Becket) Qin
On Tue, Jul 5, 2022 at 3:58 PM Yun Gao wrote:
> +1 (binding)
>
> - Verified the checksum and signatures.
> - Build from sources without exceptions.
> - Checked the arti
g a method at the moment. And
evolving the SourceReaderContext to use the ReadOnlyRuntimeContext in the
future.
Thanks,
Jiangjie (Becket) Qin
On Tue, Jul 5, 2022 at 8:31 PM Alexander Fedulov
wrote:
> Hi Becket,
>
> I agree with you. We could introduce a *ReadOnlyRuntimeContex
Yes, that sounds reasonable to me. That said, supporting custom events
might still be preferable if that does not complicate the design too much.
It would be good to avoid having a tricky feature availability matrix when
we add new features to the project.
Thanks,
Jiangjie (Becket) Qin
On Mon
, a job has one task whose watermark is always lagging behind,
therefore it causes the other tasks to be throttled. The speculative
execution identified the slow task and decided to run it in another node,
thus unblocking the other subtasks.
Thanks,
Jiangjie (Becket) Qin
On Mon, Jul 4, 2022
Hi Alex,
Yes, that is what I had in mind. We need to add the method
getRuntimeContext() to the SourceReaderContext interface as well.
Thanks,
Jiangjie (Becket) Qin
On Mon, Jul 4, 2022 at 3:01 AM Alexander Fedulov
wrote:
> Hi Becket,
>
> thanks for your input. I like the idea
the global rate untouched on scaling.
Thanks,
Jiangjie (Becket) Qin
On Fri, Jul 1, 2022 at 4:00 AM Alexander Fedulov
wrote:
> Hi all,
>
> getting back to the idea of reusing FlinkConnectorRateLimiter: it is
> designed for the SourceFunction API and has an open() metho
+1.
It looks like we have some decent progress on Flink ML :)
Thanks,
Jiangjie (Becket) Qin
On Fri, Jun 24, 2022 at 8:29 AM Dong Lin wrote:
> Hi Zhipeng and Yun,
>
> Thanks for starting the discussion. +1 for the Flink ML 2.1.0 release.
>
> Cheers,
> Dong
>
> On Thu,
+1 (binding)
Thanks Alex.
Jiangjie (Becket) Qin
On Thu, Jun 16, 2022 at 7:16 PM Lijie Wang wrote:
> +1 (non-binding)
>
> Thanks for driving this.
>
> Best,
> Lijie
>
> Martijn Visser 于2022年6月16日周四 19:07写道:
>
> > +1 (binding)
> >
> >
committee
is actively looking for speakers from the Flink community.
Please don't hesitate to submit a talk [2] if you are interested!
Thanks,
Jiangjie (Becket) Qin
[1] https://2022.currentevent.io/website/39543/
[2] https://sessionize.com/current-2022/
/ connector
developers that needs some attention?
Thanks,
Jiangjie (Becket) Qin
On Fri, Jun 10, 2022 at 3:25 PM David Anderson wrote:
> +1 for deprecating SourceFunction from me as well. And a big THANK YOU to
> Alex Fedulov for bringing forward FLIP-238.
>
> David
>
> On
storage for Flink dynamic tables.
Congratulations and welcome, Jingsong!
Cheers,
Jiangjie (Becket) Qin
(On behalf of the Apache Flink PMC)
nularity.
Anyways, it looks like most people think NDV or its full phrase is a better
name. It probably makes sense to just use either of them.
Thanks,
Jiangjie (Becket) Qin
On Fri, Jun 3, 2022 at 9:45 PM Jark Wu wrote:
> Hi Jing,
>
> I agree with you that "NDV is more SQL-orien
be a good
alternative.
Thanks,
Jiangjie (Becket) Qin
On Thu, Jun 2, 2022 at 12:51 AM Jing Ge wrote:
> Hi Dev,
>
> I am not really sure if it is feasible to start this discussion. According
> to the contribution guidelines, dev ml is the right place to reach
> consensus.
>
> In C
trigger is pulled.
In the above design, partial caching and full caching would be
implementations of the CachingStrategy. And it is OK for users to implement
their own CachingStrategy if they want to.
Thanks,
Jiangjie (Becket) Qin
On Thu, Jun 2, 2022 at 12:14 PM Jark Wu wrote:
> Thank Qingsh
1 - 100 of 442 matches
Mail list logo