Thanks Di Wu for driving this FLIP, the updated FLIP looks good to me, +1 to
start a vote.
Best,
Leonard
> 2024年4月7日 下午12:52,wudi <676366...@qq.com.INVALID> 写道:
>
> Hi,
>
> since there are no more comments for a while, if there are no more comments
> for another day, I will start a vote threa
Hi,
since there are no more comments for a while, if there are no more comments for
another day, I will start a vote thread.
Thanks,
di.wu
> 2024年4月1日 17:52,wudi <676366...@qq.com> 写道:
>
> Hi,
>
> Gentle ping to see if there are any other concerns or things that seems
> missing from the FLI
Hi,
Gentle ping to see if there are any other concerns or things that seems missing
from the FLIP.
Brs
di.wu
> 2024年3月25日 17:52,Feng Jin 写道:
>
> Hi Di
>
> Thank you for your patience and explanation.
>
> If this is a server-side configuration, we currently cannot modify it in
> the client c
Hi Di
Thank you for your patience and explanation.
If this is a server-side configuration, we currently cannot modify it in
the client configuration. If Doris supports client-side configuration in
the future, we can reconsider whether to support it.
I currently have no other questions regarding
Hi, Feng
Yes, if the StreamLoad transaction timeout is very short, you may encounter
this situation.
The timeout for StreamLoad transactions is controlled by the
streaming_label_keep_max_second parameter [1] in FE (Frontend), and the default
value is 12 hours. Currently, it is a global transac
Hi Di
Thanks for your reply.
The timeout I'm referring to here is not the commit timeout, but rather the
timeout for a single streamLoad transaction.
Let's say we have set the transaction timeout for StreamLoad to be 10
minutes. Now, imagine there is a Flink job with two subtasks. Due to
signifi
Hi, Feng,
1. Are you suggesting that when a commit gets stuck, we can interrupt the
commit request using a timeout parameter? Currently, there is no such
parameter. In my understanding, in a two-phase commit, checkpoint must be
enabled, so the commit timeout is essentially the checkpoint timeou
Hi Di,
Thank you for the update, as well as quickly implementing corresponding
capabilities including filter push down and project push down.
Regarding the transaction timeout, I still have some doubts. I would like
to confirm if we can control this timeout parameter in the connector, such
as set
Hi, Feng,
Thank you, that's a great suggestion !
I have already implemented FilterPushDown and removed that parameter on
DorisDynamicTableSource[1], and also updated FLIP.
Regarding the mention of [Doris also aborts transactions], it may not have been
described accurately. It mainly refers to
Hi Di
Thank you for initiating this FLIP, +1 for this.
Regarding the option `doris.filter.query` of doris source table
Can we directly implement the FilterPushDown capability of Flink Source
like Jdbc Source [1] instead of introducing an option?
Regarding two-phase commit,
> At the same time,
Hi,
Thanks for driving this, +1 for the FLIP.
Best,
Ferenc
On Monday, March 11th, 2024 at 15:17, Ahmed Hamdy wrote:
>
>
> Hello,
> Thanks for the proposal, +1 for the FLIP.
>
> Best Regards
> Ahmed Hamdy
>
>
> On Mon, 11 Mar 2024 at 15:12, wudi 676366...@qq.com.invalid wrote:
>
> > Hi
Hello,
Thanks for the proposal, +1 for the FLIP.
Best Regards
Ahmed Hamdy
On Mon, 11 Mar 2024 at 15:12, wudi <676366...@qq.com.invalid> wrote:
> Hi, Leonard
> Thank you for your suggestion.
> I referred to other Connectors[1], modified the naming and types of
> relevant parameters[2], and also
Hi, Leonard
Thank you for your suggestion.
I referred to other Connectors[1], modified the naming and types of relevant
parameters[2], and also updated FLIP.
[1]
https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/connectors/table/overview/
[1]
https://github.com/apache/doris-flink-
Thanks for the explanation Di Wu.
Regards,
Jeyhun
On Thu, Mar 7, 2024 at 7:48 PM Jing Ge wrote:
> Thanks for the clarification. It makes sense to me.
>
> Best regards,
> Jing
>
> On Wed, Mar 6, 2024 at 10:03 AM wudi <676366...@qq.com.invalid> wrote:
>
> > Hi Jing Ge, thanks for your suggestions
Thanks for the clarification. It makes sense to me.
Best regards,
Jing
On Wed, Mar 6, 2024 at 10:03 AM wudi <676366...@qq.com.invalid> wrote:
> Hi Jing Ge, thanks for your suggestions.
>
> 1. Currently, the Flink Doris Connector is compatible with Flink versions
> 1.15-1.18. SupportsCommitter[1]
Thanks wudi for the updating, the FLIP generally looks good to me, I only left
two minor suggestions:
(1) The suffix `.s` in configoption doris.request.query.timeout.s looks strange
to me, could we change all time interval related option value type to Duration ?
(2) Could you check and improve
Hi Jing Ge, thanks for your suggestions.
1. Currently, the Flink Doris Connector is compatible with Flink versions
1.15-1.18. SupportsCommitter[1] seems to be introduced in Flink 1.19, and most
users may not have upgraded their Flink environments to that version yet.
Modifying it now could lead
Hi Di,
Thanks for your proposal. +1 for the contribution. I'd like to know your
thoughts about the following questions:
1. According to your clarification of the exactly-once, thanks for it BTW,
no PreCommitTopology is required. Does it make sense to let DorisSink[1]
implement SupportsCommitter,
Hi, Jeyhun Karimov.
Thanks for your question.
- How to ensure Exactly-Once?
1. When the Checkpoint Barrier arrives, DorisSink will trigger the precommit
api of StreamLoad to complete the persistence of data in Doris (the data will
not be visible at this time), and will also pass this TxnID to th
Thanks, Yanquan, Test Plan has been added at the end of FLIP[1].
[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-399%3A+Flink+Connector+Doris
Brs,
di.wu
> 2024年3月2日 17:13,Yanquan Lv 写道:
>
> Thanks for driving this.
> The content is very detailed, it is recommended to add a section
Hi,
Thanks for the proposal. +1 for the FLIP.
Best,
Hang
Jeyhun Karimov 于2024年3月2日周六 17:53写道:
> Hi,
>
> Thanks for the proposal. +1 for the FLIP.
> I have a few questions:
>
> - How exactly the two (Stream Load's two-phase commit and Flink's two-phase
> commit) combination will ensure the e2e
Hi,
Thanks for the proposal. +1 for the FLIP.
I have a few questions:
- How exactly the two (Stream Load's two-phase commit and Flink's two-phase
commit) combination will ensure the e2e exactly-once semantics?
- The FLIP proposes to combine Doris's batch writing with the primary key
table to ach
Thanks for driving this.
The content is very detailed, it is recommended to add a section on Test
Plan for more completeness.
Di Wu 于2024年1月25日周四 15:40写道:
> Hi all,
>
> Previously, we had some discussions about contributing Flink Doris
> Connector to the Flink community [1]. I want to further pr
Hi all,
Previously, we had some discussions about contributing Flink Doris Connector to
the Flink community [1]. I want to further promote this work. I hope everyone
will help participate in this FLIP discussion and provide more valuable
opinions and suggestions.
Thanks.
[1] https://lists.apa
+1 for this :)
On Thu, Dec 7, 2023 at 6:05 AM wudi <676366...@qq.com.invalid> wrote:
>
>
> Hi all,
>
> As discussed in the previous email [1], about contributing the Flink Doris
> Connector to the Flink community.
>
>
> Apache Doris[2] is a high-performance, real-time analytical database based on
Hi all,
As discussed in the previous email [1], about contributing the Flink Doris
Connector to the Flink community.
Apache Doris[2] is a high-performance, real-time analytical database based on
MPP architecture, for scenarios where Flink is used for data analysis,
processing, or real-time
26 matches
Mail list logo