[jira] [Created] (FLINK-35221) Support SQL 2011 reserved keywords as identifiers in Flink HiveParser

2024-04-24 Thread Wencong Liu (Jira)
Wencong Liu created FLINK-35221:
---

 Summary: Support SQL 2011 reserved keywords as identifiers in 
Flink HiveParser 
 Key: FLINK-35221
 URL: https://issues.apache.org/jira/browse/FLINK-35221
 Project: Flink
  Issue Type: Improvement
  Components: Connectors / Hive
Affects Versions: 1.20.0
Reporter: Wencong Liu


According to Hive user documentation[1], starting from version 0.13.0, Hive 
prohibits the use of reserved keywords as identifiers. Moreover, versions 2.1.0 
and earlier allow using SQL11 reserved keywords as identifiers by setting 
{{hive.support.sql11.reserved.keywords=false}} in hive-site.xml. This 
compatibility feature facilitates jobs that utilize keywords as identifiers.

HiveParser in Flink, relying on Hive version 2.3.9, lacks the option to treat 
SQL11 reserved keywords as identifiers. This poses a challenge for users 
migrating SQL from Hive 1.x to Flink SQL, as they might encounter scenarios 
where keywords are used as identifiers. Addressing this issue is necessary to 
support such cases.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [ DISCUSS ] FLIP-XXX : [Plugin] Enhancing Flink Failure Management in Kubernetes with Dynamic Termination Log Integration

2024-04-24 Thread Swathi C
Sure Ahmed and Martijn.
Fetching the flink particular job related failure and adding this logic to
termination-log is definitely a sub-task of pluggable enricher as we can
leverage pluggable enricher to achieve this.
But for CRUD level failures, which is mainly used to notify if the job
manager failed might not be using the pluggable enricher. So, let us know
if that needs to be there as a separate FLIP or we can combine that as well
under the pluggable enricher ( by adding another sub task ) ?

Regards,
Swathi C

On Wed, Apr 24, 2024 at 3:46 PM Ahmed Hamdy  wrote:

> Hi,
> I agree with the Martijn, We can reformulate the FLIP to introduce
> termination log as supported pluggable enricher. If you believe the scope
> of work is a subset (Further implementation) we can just add a Jira ticket
> for it. IMO this will also help with implementation taking the existing
> enrichers into reference.
> Best Regards
> Ahmed Hamdy
>
>
> On Tue, 23 Apr 2024 at 15:23, Martijn Visser 
> wrote:
>
> > From a procedural point of view, we shouldn't make FLIPs sub-tasks for
> > existing FLIPs that have been voted/are released. That will only cause
> > confusion down the line. A new FLIP should take existing functionality
> > (like FLIP-304) into account, and propose how to improve on what that
> > original FLIP has introduced or how you're going to leverage what's
> already
> > there.
> >
> > On Tue, Apr 23, 2024 at 11:42 AM ramkrishna vasudevan <
> > ramvasu.fl...@gmail.com> wrote:
> >
> > > Hi Gyula and Ahmed,
> > >
> > > I totally agree that there is an interlap in the final goal that both
> the
> > > FLIPs are achieving here and infact FLIP-304 is more comprehensive for
> > job
> > > failures.
> > >
> > > But as a proposal to move forward can we make Swathi's FLIP/JIRA as a
> sub
> > > task for FLIP-304 and continue with the PR since the main aim is to get
> > the
> > > cluster failure pushed to the termination log for K8s based
> deployments.
> > > And once it is completed we can work to make FLIP-304 to support job
> > > failure propagation to termination log?
> > >
> > > Regards
> > > Ram
> > >
> > > On Thu, Apr 18, 2024 at 10:07 PM Swathi C 
> > > wrote:
> > >
> > > > Hi Gyula and  Ahmed,
> > > >
> > > > Thanks for reviewing this.
> > > >
> > > > @gyula.f...@gmail.com  , currently since our
> aim
> > > as
> > > > part of this FLIP was only to fail the cluster when job manager/flink
> > has
> > > > issues such that the cluster would no longer be usable, hence, we
> > > proposed
> > > > only related to that.
> > > > Your right, that it covers only job main class errors, job manager
> run
> > > time
> > > > failures, if the Job manager wants to write any metadata to any other
> > > > system ( ABFS, S3 , ... )  and the job failures will not be covered.
> > > >
> > > > FLIP-304 is mainly used to provide Failure enrichers for job
> failures.
> > > > Since, this FLIP is mainly for flink Job manager failures, let us
> know
> > if
> > > > we can leverage the goodness of both and try to extend FLIP-304 and
> add
> > > our
> > > > plugin implementation to cover the job level issues ( propagate this
> > info
> > > > to the /dev/termination-log such that, the container status reports
> it
> > > for
> > > > flink on K8S by implementing Failure Enricher interface and
> > > > processFailure() to do this ) and use this FLIP proposal for generic
> > > flink
> > > > cluster (Job manager/cluster ) failures.
> > > >
> > > > Regards,
> > > > Swathi C
> > > >
> > > > On Thu, Apr 18, 2024 at 7:36 PM Ahmed Hamdy 
> > > wrote:
> > > >
> > > > > Hi Swathi!
> > > > > Thanks for the proposal.
> > > > > Could you please elaborate what this FLIP offers more than
> > Flip-304[1]?
> > > > > Flip 304 proposes a Pluggable mechanism for enriching Job failures,
> > If
> > > I
> > > > am
> > > > > not mistaken this proposal looks like a subset of it.
> > > > >
> > > > > 1-
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-304%3A+Pluggable+Failure+Enrichers
> > > > >
> > > > > Best Regards
> > > > > Ahmed Hamdy
> > > > >
> > > > >
> > > > > On Thu, 18 Apr 2024 at 08:23, Gyula Fóra 
> > wrote:
> > > > >
> > > > > > Hi Swathi!
> > > > > >
> > > > > > Thank you for creating this proposal. I really like the general
> > idea
> > > of
> > > > > > increasing the K8s native observability of Flink job errors.
> > > > > >
> > > > > > I took a quick look at your reference PR, the termination log
> > related
> > > > > logic
> > > > > > is contained completely in the ClusterEntrypoint. What type of
> > errors
> > > > > will
> > > > > > this actually cover?
> > > > > >
> > > > > > To me this seems to cover only:
> > > > > >  - Job main class errors (ie startup errors)
> > > > > >  - JobManager failures
> > > > > >
> > > > > > Would regular job errors (that cause only job failover but not JM
> > > > errors)
> > > > > > be reported somehow with this plugin?
> > > > > >
> > > > > > Thanks
> > > > > > Gyula
> > > > > >
> > > > > 

Re: [ DISCUSS ] FLIP-XXX : [Plugin] Enhancing Flink Failure Management in Kubernetes with Dynamic Termination Log Integration

2024-04-24 Thread Ahmed Hamdy
Hi,
I agree with the Martijn, We can reformulate the FLIP to introduce
termination log as supported pluggable enricher. If you believe the scope
of work is a subset (Further implementation) we can just add a Jira ticket
for it. IMO this will also help with implementation taking the existing
enrichers into reference.
Best Regards
Ahmed Hamdy


On Tue, 23 Apr 2024 at 15:23, Martijn Visser 
wrote:

> From a procedural point of view, we shouldn't make FLIPs sub-tasks for
> existing FLIPs that have been voted/are released. That will only cause
> confusion down the line. A new FLIP should take existing functionality
> (like FLIP-304) into account, and propose how to improve on what that
> original FLIP has introduced or how you're going to leverage what's already
> there.
>
> On Tue, Apr 23, 2024 at 11:42 AM ramkrishna vasudevan <
> ramvasu.fl...@gmail.com> wrote:
>
> > Hi Gyula and Ahmed,
> >
> > I totally agree that there is an interlap in the final goal that both the
> > FLIPs are achieving here and infact FLIP-304 is more comprehensive for
> job
> > failures.
> >
> > But as a proposal to move forward can we make Swathi's FLIP/JIRA as a sub
> > task for FLIP-304 and continue with the PR since the main aim is to get
> the
> > cluster failure pushed to the termination log for K8s based deployments.
> > And once it is completed we can work to make FLIP-304 to support job
> > failure propagation to termination log?
> >
> > Regards
> > Ram
> >
> > On Thu, Apr 18, 2024 at 10:07 PM Swathi C 
> > wrote:
> >
> > > Hi Gyula and  Ahmed,
> > >
> > > Thanks for reviewing this.
> > >
> > > @gyula.f...@gmail.com  , currently since our aim
> > as
> > > part of this FLIP was only to fail the cluster when job manager/flink
> has
> > > issues such that the cluster would no longer be usable, hence, we
> > proposed
> > > only related to that.
> > > Your right, that it covers only job main class errors, job manager run
> > time
> > > failures, if the Job manager wants to write any metadata to any other
> > > system ( ABFS, S3 , ... )  and the job failures will not be covered.
> > >
> > > FLIP-304 is mainly used to provide Failure enrichers for job failures.
> > > Since, this FLIP is mainly for flink Job manager failures, let us know
> if
> > > we can leverage the goodness of both and try to extend FLIP-304 and add
> > our
> > > plugin implementation to cover the job level issues ( propagate this
> info
> > > to the /dev/termination-log such that, the container status reports it
> > for
> > > flink on K8S by implementing Failure Enricher interface and
> > > processFailure() to do this ) and use this FLIP proposal for generic
> > flink
> > > cluster (Job manager/cluster ) failures.
> > >
> > > Regards,
> > > Swathi C
> > >
> > > On Thu, Apr 18, 2024 at 7:36 PM Ahmed Hamdy 
> > wrote:
> > >
> > > > Hi Swathi!
> > > > Thanks for the proposal.
> > > > Could you please elaborate what this FLIP offers more than
> Flip-304[1]?
> > > > Flip 304 proposes a Pluggable mechanism for enriching Job failures,
> If
> > I
> > > am
> > > > not mistaken this proposal looks like a subset of it.
> > > >
> > > > 1-
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-304%3A+Pluggable+Failure+Enrichers
> > > >
> > > > Best Regards
> > > > Ahmed Hamdy
> > > >
> > > >
> > > > On Thu, 18 Apr 2024 at 08:23, Gyula Fóra 
> wrote:
> > > >
> > > > > Hi Swathi!
> > > > >
> > > > > Thank you for creating this proposal. I really like the general
> idea
> > of
> > > > > increasing the K8s native observability of Flink job errors.
> > > > >
> > > > > I took a quick look at your reference PR, the termination log
> related
> > > > logic
> > > > > is contained completely in the ClusterEntrypoint. What type of
> errors
> > > > will
> > > > > this actually cover?
> > > > >
> > > > > To me this seems to cover only:
> > > > >  - Job main class errors (ie startup errors)
> > > > >  - JobManager failures
> > > > >
> > > > > Would regular job errors (that cause only job failover but not JM
> > > errors)
> > > > > be reported somehow with this plugin?
> > > > >
> > > > > Thanks
> > > > > Gyula
> > > > >
> > > > > On Tue, Apr 16, 2024 at 8:21 AM Swathi C <
> swathi.c.apa...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi All,
> > > > > >
> > > > > > I would like to start a discussion on FLIP-XXX : [Plugin]
> Enhancing
> > > > Flink
> > > > > > Failure Management in Kubernetes with Dynamic Termination Log
> > > > > Integration.
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://docs.google.com/document/d/1tWR0Fi3w7VQeD_9VUORh8EEOva3q-V0XhymTkNaXHOc/edit?usp=sharing
> > > > > >
> > > > > >
> > > > > > This FLIP proposes an improvement plugin and focuses mainly on
> > Flink
> > > on
> > > > > > K8S but can be used as a generic plugin and add further
> > enhancements.
> > > > > >
> > > > > > Looking forward to everyone's feedback and suggestions. Thank you
> > !!
> > > > > >
> > > > > > Best Regards,
> > > > > > Swathi 

Re: [DISCUSS] FLIP-XXX Support currentFetchEventTimeLag and processingLag metrics

2024-04-24 Thread jialiang tan
Hi lorenzo, thanks for your feedback!


> There can be major discrepancies between the absolute time got by the
> TaskManagers (if clocks are not synchronized via ntp for example), and the
> results of the metric might be quite distant for different TMs.


I don't know much about this. can more experienced contributors help? If it
exists, perhaps `currentEmitEventTimeLag` has the same problem?

Furthermore, comparing the time on the Flink cluster with the event time of
> records might introduce completely inaccurate results.


I think this has been discussed in the FLIP-33 lists thread[2].

2 - I don't think the name `processingLag` represents the processing time
> spent, I would rather see `processingTime` just for the semantics of the
> name itself.


+1, it makes sense to me.

3 - Do you really think the `processingTime` should be a gauge? I
> understand your justification for the fetch lag, but I think the processing
> time should be an histogram. For the inefficiency of this, how about some
> sampling (e.g.: only update the histogram 1 every 1000 events?)


I went back to the discussion of the FLIP-33 lists thread[2]. At first,
`emitEventTimeLag` and `fetchEventTimeLag` were defined as histogram, and
they felt that histogram was too expensive. So they imported
`currentFetchEventTimeLag` and `currentEmitEventTimeLag` instead, and put
`fetchEventTimeLag` and `emitEventTimeLag` in the future work. I think we
need to create another FLIP to discuss it. Now I want to change
`processingTime` to `currentProcessingTime`, still using gauge, and remain
`processingTime` in the future work, WDYT?

4 - At this point, if we have the processing time and number of records, we
> could also add throughput as a metric, so that the user would know how many
> records/second the source is able to produce.


Do you mean `numRecordsInPerSecond` and `numRecordsOutPerSecond`? They were
defined in FLIP-33[1] and used in Flink-1.14.

5 - For the "Kafka Connector" section: can this be generalized for
> connectors in general? Can you provide an example to better understand your
> statement about reflection?


Good idea, I have extended my FLIP.

6 - Does this introduce any UI change for representing the metric?


I'm just adding some new metrics, users can get them from REST API or
collect them into Prometheus, and no change for Flink WEB UI.

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics
[2] https://lists.apache.org/thread/r47zrqto4k8tsc3xvfg392zblfx6dczl

Best,
TanJiaLiang

 于2024年4月24日周三 15:02写道:

> Hello jialiang tan, and thank you for your contribution!
>
> Here are my questions:
>
> 1 - I don't know if exposing this as metric to the user could create some
> harm. There can be major discrepancies between the absolute time got by the
> TaskManagers (if clocks are not synchronized via ntp for example), and the
> results of the metric might be quite distant for different TMs.
> Furthermore, comparing the time on the Flink cluster with the event time of
> records might introduce completely inaccurate results. I think providing
> this should come with many disclaimers to the user. Maybe, more experienced
> contributors can comment on this as well.
>
> 2 - I don't think the name `processingLag` represents the processing time
> spent, I would rather see `processingTime` just for the semantics of the
> name itself.
>
> 3 - Do you really think the `processingTime` should be a gauge? I
> understand your justification for the fetch lag, but I think the processing
> time should be an histogram. For the inefficiency of this, how about some
> sampling (e.g.: only update the histogram 1 every 1000 events?)
>
> 4 - At this point, if we have the processing time and number of records,
> we could also add throughput as a metric, so that the user would know how
> many records/second the source is able to produce.
>
> 5 - For the "Kafka Connector" section: can this be generalized for
> connectors in general? Can you provide an example to better understand your
> statement about reflection?
>
> 6 - Does this introduce any UI change for representing the metric?
>
> Thank you!
> On Apr 22, 2024 at 12:26 +0200, jialiang tan ,
> wrote:
> > Sorry all, it seems bad formatting in my email message, now I send it
> again
> > gently and hope it work.
> >
> > I would like to start a discussion about FLIP-XXX:
> > SupportcurrentFetchEventTimeLag and processingLag metrics [1].
> >
> > The main motivation for this change was that I had some difficulties
> > inimplementing the currentFetchEventTimeLag metrics for KafkaSource [2].
> >
> > So I proposed to let the SourceReaderMetricGroup provide an interface to
> > capturethe FetchTime so that all the FLIP-27 [3] sources can easily
> > implement thecurrentFetchEventTimeLag metrics.
> >
> > In addition, I propose to support the processingLag metric for the
> > FLIP-27sources to measure the current processing latency of the source.
> >
> > See the FLIP [1] and Jira 

[jira] [Created] (FLINK-35229) join An error occurred when the table was empty

2024-04-24 Thread lixu (Jira)
lixu created FLINK-35229:


 Summary: join An error occurred when the table was empty
 Key: FLINK-35229
 URL: https://issues.apache.org/jira/browse/FLINK-35229
 Project: Flink
  Issue Type: Bug
  Components: Table SQL / API
Affects Versions: 1.19.0, 1.17.2, 1.18.0
Reporter: lixu
 Fix For: 1.17.3, 1.19.1, 1.18.1


{code:java}
//代码占位符
StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
env.setRuntimeMode(RuntimeExecutionMode.BATCH).setParallelism(1);
StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(env);

Table ticker = tableEnvironment.fromValues(
DataTypes.ROW(
DataTypes.FIELD("symbol", DataTypes.STRING()),
DataTypes.FIELD("price", DataTypes.BIGINT())
),
row("A", 12L),
row("B", 17L)
);
tableEnvironment.createTemporaryView("ticker_t", ticker);

Table ticker1 = tableEnvironment.fromValues(
DataTypes.ROW(
DataTypes.FIELD("symbol", DataTypes.STRING()),
DataTypes.FIELD("price", DataTypes.BIGINT())
)
);
tableEnvironment.createTemporaryView("ticker_y", ticker1);

Table ticker2 = tableEnvironment.fromValues(
DataTypes.ROW(
DataTypes.FIELD("symbol", DataTypes.STRING()),
DataTypes.FIELD("price", DataTypes.BIGINT())
),
row("A", 12L),
row("B", 17L)
);
tableEnvironment.createTemporaryView("ticker_z", ticker2);

tableEnvironment.sqlQuery("select coalesce(t.symbol, y.symbol, z.symbol) as 
symbol, " +
" t.price as price_t, y.price as price_y,  z.price as price_z " 
+
"from ticker_t t FULL OUTER JOIN ticker_y y ON t.symbol = 
y.symbol " +
"FULL OUTER JOIN ticker_z z ON y.symbol = z.symbol")
.execute().print(); {code}
+++--+--+--+
| op |                         symbol |              price_t |              
price_y |              price_z |
+++--+--+--+
| +I |                              A |                   12 |               
 |                |
| +I |                              B |                   17 |               
 |                |
| +I |                              A |                |               
 |                   12 |
| +I |                              B |                |               
 |                   17 |
+++--+--+--+



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-35230) Split FlinkSqlParserImplTest to reduce the code lines.

2024-04-24 Thread Feng Jin (Jira)
Feng Jin created FLINK-35230:


 Summary: Split FlinkSqlParserImplTest to reduce the code lines.
 Key: FLINK-35230
 URL: https://issues.apache.org/jira/browse/FLINK-35230
 Project: Flink
  Issue Type: Technical Debt
  Components: Table SQL / Planner
Reporter: Feng Jin


With the increasing extension of Calcite syntax, the current 
FlinkSqlParserImplTest has reached nearly 3000 lines of code. 

If it exceeds the current limit, it will result in errors in the code style 
check.

{code:log}
08:33:19.679 [ERROR] 
src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java:[1] 
(sizes) FileLength: File length is 3,166 lines (max allowed is 3,100).
{code}

To facilitate future syntax extensions, I suggest that we split 
FlinkSqlParserImplTest and place the same type of syntax in separate Java tests 
for the convenience of avoiding the continuous growth of the original test 
class.

My current idea is: 
Since *FlinkSqlParserImplTest* currently inherits *SqlParserTest*, and 
*SqlParserTest* itself contains many unit tests, for the convenience of future 
test splits, we should introduce a basic *ParserTestBase* inheriting 
*SqlParserTest*, and disable the original related unit tests in 
*SqlParserTest*. 

This will facilitate writing relevant unit tests more quickly during subsequent 
splitting, without the need to repeatedly execute the unit tests inside 
SqlParserTest.






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread yue ma
hi Yanfei,

Thanks for your feedback and reminders I have updated related information.
In fact, most of them use the default Configrations.

Yanfei Lei  于2024年4月23日周二 12:51写道:

> Hi Yue & Roman,
>
> Thanks for initiating this FLIP and all the efforts for the upgrade.
>
> 8.10.0 introduces some new features, making it possible for Flink to
> implement some new exciting features, and the upgrade also makes
> FRocksDB easier to maintain, +1 for upgrading.
>
> I read the FLIP and have a minor comment, it would be better to add
> some description about the environment/configuration of the nexmark's
> result.
>
> Roman Khachatryan  于2024年4月23日周二 12:07写道:
>
> >
> > Hi,
> >
> > Thanks for writing the proposal and preparing the upgrade.
> >
> > FRocksDB  definitely needs to be kept in sync with the upstream and the
> new
> > APIs are necessary for faster rescaling.
> > We're already using a similar version internally.
> >
> > I reviewed the FLIP and it looks good to me (disclaimer: I took part in
> > some steps of this effort).
> >
> >
> > Regards,
> > Roman
> >
> > On Mon, Apr 22, 2024, 08:11 yue ma  wrote:
> >
> > > Hi Flink devs,
> > >
> > > I would like to start a discussion on FLIP-447: Upgrade FRocksDB from
> > > 6.20.3 to 8.10.0
> > >
> > >
> > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-447%3A+Upgrade+FRocksDB+from+6.20.3++to+8.10.0
> > >
> > > This FLIP proposes upgrading the version of FRocksDB in the Flink
> Project
> > > from 6.20.3 to 8.10.0.
> > > The FLIP mainly introduces the main benefits of upgrading FRocksDB,
> > > including the use of IngestDB which can improve Rescaling performance
> by
> > > more than 10 times in certain scenarios, as well as other potential
> > > optimization points such as async_io, blob db, and tiered storage.The
> > > FLIP also presented test results based on RocksDB 8.10, including
> > > StateBenchmark and Nexmark tests.
> > > Overall, upgrading FRocksDB may result in a small regression of write
> > > performance( which is a very small part of the overall overhead), but
> it
> > > can bring many important performance benefits.
> > > So we hope to upgrade the version of FRocksDB through this FLIP.
> > >
> > > Looking forward to everyone's feedback and suggestions. Thank you!
> > > --
> > > Best regards,
> > > Yue
> > >
>
>
>
> --
> Best,
> Yanfei
>


-- 
Best,
Yue


Re: [DISCUSS] FLIP-448: Introduce Pluggable Workflow Scheduler Interface for Materialized Table

2024-04-24 Thread Feng Jin
Hi Ron

Thank you for initiating this FLIP.

My current questions are as follows:

1. From my current understanding, the workflow handle should not be bound
to the Dynamic Table. Therefore, if the workflow is modified, does it mean
that the scheduling information corresponding to the Dynamic Table will be
lost?

2. Regarding the status information of the workflow, I am wondering if it
is necessary to provide an interface to display the backend scheduling
information? This would make it more convenient to view the execution
status of backend jobs.


Best,
Feng


On Wed, Apr 24, 2024 at 3:24 PM 
wrote:

> Hello Ron Liu! Thank you for your FLIP!
>
> Here are my considerations:
>
> 1.
> About the Operations interfaces, how can they be empty?
> Should not they provide at least a `run` or `execute` method (similar to
> the command pattern)?
> In this way, their implementation can wrap all the implementations details
> of particular schedulers, and the scheduler can simply execute the command.
> In general, I think a simple sequence diagram showcasing the interaction
> between the interfaces would be awesome to better understand the concept.
>
> 2.
> What about the RefreshHandler, I cannot find a definition of its interface
> here.
> Is it out of scope for this FLIP?
>
> 3.
> For the SqlGatewayService arguments:
>
> boolean isPeriodic,
> @Nullable String scheduleTime,
> @Nullable String scheduleTimeFormat,
>
> If it is periodic, where is the period?
> For the scheduleTime and format, why not simply pass an instance of
> LocalDateTime or similar? The gateway should not have the responsibility to
> parse the time.
>
> 4.
> For the REST API:
> wouldn't it be better (more REST) to move the `mt_identifier` to the URL?
> E.g.: v3/materialized_tables//refresh
>
> Thank you!
> On Apr 22, 2024 at 08:42 +0200, Ron Liu , wrote:
> > Hi, Dev
> >
> > I would like to start a discussion about FLIP-448: Introduce Pluggable
> > Workflow Scheduler Interface for Materialized Table.
> >
> > In FLIP-435[1], we proposed Materialized Table, which has two types of
> data
> > refresh modes: Full Refresh & Continuous Refresh Mode. In Full Refresh
> > mode, the Materialized Table relies on a workflow scheduler to perform
> > periodic refresh operation to achieve the desired data freshness.
> >
> > There are numerous open-source workflow schedulers available, with
> popular
> > ones including Airflow and DolphinScheduler. To enable Materialized Table
> > to work with different workflow schedulers, we propose a pluggable
> workflow
> > scheduler interface for Materialized Table in this FLIP.
> >
> > For more details, see FLIP-448 [2]. Looking forward to your feedback.
> >
> > [1] https://lists.apache.org/thread/c1gnn3bvbfs8v1trlf975t327s4rsffs
> > [2]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-448%3A+Introduce+Pluggable+Workflow+Scheduler+Interface+for+Materialized+Table
> >
> > Best,
> > Ron
>


Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread Congxian Qiu
Thanks for driving this,  yue

We also observed significant performance improvements in some cases after
bumped the Rocksdb version, +1 for this work

Best,
Congxian


yue ma  于2024年4月24日周三 19:16写道:

> hi Yanfei,
>
> Thanks for your feedback and reminders I have updated related information.
> In fact, most of them use the default Configrations.
>
> Yanfei Lei  于2024年4月23日周二 12:51写道:
>
> > Hi Yue & Roman,
> >
> > Thanks for initiating this FLIP and all the efforts for the upgrade.
> >
> > 8.10.0 introduces some new features, making it possible for Flink to
> > implement some new exciting features, and the upgrade also makes
> > FRocksDB easier to maintain, +1 for upgrading.
> >
> > I read the FLIP and have a minor comment, it would be better to add
> > some description about the environment/configuration of the nexmark's
> > result.
> >
> > Roman Khachatryan  于2024年4月23日周二 12:07写道:
> >
> > >
> > > Hi,
> > >
> > > Thanks for writing the proposal and preparing the upgrade.
> > >
> > > FRocksDB  definitely needs to be kept in sync with the upstream and the
> > new
> > > APIs are necessary for faster rescaling.
> > > We're already using a similar version internally.
> > >
> > > I reviewed the FLIP and it looks good to me (disclaimer: I took part in
> > > some steps of this effort).
> > >
> > >
> > > Regards,
> > > Roman
> > >
> > > On Mon, Apr 22, 2024, 08:11 yue ma  wrote:
> > >
> > > > Hi Flink devs,
> > > >
> > > > I would like to start a discussion on FLIP-447: Upgrade FRocksDB from
> > > > 6.20.3 to 8.10.0
> > > >
> > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-447%3A+Upgrade+FRocksDB+from+6.20.3++to+8.10.0
> > > >
> > > > This FLIP proposes upgrading the version of FRocksDB in the Flink
> > Project
> > > > from 6.20.3 to 8.10.0.
> > > > The FLIP mainly introduces the main benefits of upgrading FRocksDB,
> > > > including the use of IngestDB which can improve Rescaling performance
> > by
> > > > more than 10 times in certain scenarios, as well as other potential
> > > > optimization points such as async_io, blob db, and tiered storage.The
> > > > FLIP also presented test results based on RocksDB 8.10, including
> > > > StateBenchmark and Nexmark tests.
> > > > Overall, upgrading FRocksDB may result in a small regression of write
> > > > performance( which is a very small part of the overall overhead), but
> > it
> > > > can bring many important performance benefits.
> > > > So we hope to upgrade the version of FRocksDB through this FLIP.
> > > >
> > > > Looking forward to everyone's feedback and suggestions. Thank you!
> > > > --
> > > > Best regards,
> > > > Yue
> > > >
> >
> >
> >
> > --
> > Best,
> > Yanfei
> >
>
>
> --
> Best,
> Yue
>


[jira] [Created] (FLINK-35231) Test relies on listJobsFunction in TestingClusterClient are broken

2024-04-24 Thread Artem Plyusnin (Jira)
Artem Plyusnin created FLINK-35231:
--

 Summary: Test relies on listJobsFunction in TestingClusterClient 
are broken
 Key: FLINK-35231
 URL: https://issues.apache.org/jira/browse/FLINK-35231
 Project: Flink
  Issue Type: Bug
  Components: Kubernetes Operator
Affects Versions: kubernetes-operator-1.9.0
Reporter: Artem Plyusnin


Supplier listJobsFunction is never called in TestingClusterClient therefore 
some assertions are never called.

Affected tests: 
 * FlinkSessionJobObserverTest#testObserveWithEffectiveConfig
 * FlinkDeploymentControllerTest#verifyReconcileWithBadConfig
 * FlinkSessionJobControllerTest#verifyReconcileWithBadConfig

Reason:

When listJobs() requested TestingClusterClient calls 
TestingFlinkClient#getMultipleJobsDetails instead of listJobsFunction



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-35232) Support for retry settings on GCS connector

2024-04-24 Thread Vikas M (Jira)
Vikas M created FLINK-35232:
---

 Summary: Support for retry settings on GCS connector
 Key: FLINK-35232
 URL: https://issues.apache.org/jira/browse/FLINK-35232
 Project: Flink
  Issue Type: Improvement
  Components: Connectors / FileSystem
Affects Versions: 1.15.3, 1.16.2, 1.17.1
Reporter: Vikas M
Assignee: Ravi Singh


The current GCS connector uses the gcs java storage library and bypasses the 
hadoop gcs connector which supports multiple http options. There are situations 
where GCS takes longer to provide a response for a PUT operation than the 
default value.

This change will allow users to customize their connect time and read timeout 
based on their application



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] FLIP-445: Support dynamic parallelism inference for HiveSource

2024-04-24 Thread Xia Sun
Hi everyone,

Thanks for all the feedback!

If there are no more comments, I would like to start the vote thread,
thanks again!

Best,
Xia

Ahmed Hamdy  于2024年4月18日周四 21:31写道:

> Hi Xia,
> I have read through the FLIP and discussion and the new version of the FLIP
> looks better.
> +1 for the proposal.
> Best Regards
> Ahmed Hamdy
>
>
> On Thu, 18 Apr 2024 at 12:21, Ron Liu  wrote:
>
> > Hi, Xia
> >
> > Thanks for updating, looks good to me.
> >
> > Best,
> > Ron
> >
> > Xia Sun  于2024年4月18日周四 19:11写道:
> >
> > > Hi Ron,
> > > Yes, presenting it in a table might be more intuitive. I have already
> > added
> > > the table in the "Public Interfaces | New Config Option" chapter of
> FLIP.
> > > PTAL~
> > >
> > > Ron Liu  于2024年4月18日周四 18:10写道:
> > >
> > > > Hi, Xia
> > > >
> > > > Thanks for your reply.
> > > >
> > > > > That means, in terms
> > > > of priority, `table.exec.hive.infer-source-parallelism` >
> > > > `table.exec.hive.infer-source-parallelism.mode`.
> > > >
> > > > I still have some confusion, if the
> > > > `table.exec.hive.infer-source-parallelism`
> > > > >`table.exec.hive.infer-source-parallelism.mode`, currently
> > > > `table.exec.hive.infer-source-parallelism` default value is true,
> that
> > > > means always static parallelism inference work? Or perhaps after this
> > > FLIP,
> > > > we changed the default behavior of
> > > > `table.exec.hive.infer-source-parallelism` to indicate dynamic
> > > parallelism
> > > > inference when enabled.
> > > > I think you should list the various behaviors of these two options
> that
> > > > coexist in FLIP by a table, only then users can know how the dynamic
> > and
> > > > static parallelism inference work.
> > > >
> > > > Best,
> > > > Ron
> > > >
> > > > Xia Sun  于2024年4月18日周四 16:33写道:
> > > >
> > > > > Hi Ron and Lijie,
> > > > > Thanks for joining the discussion and sharing your suggestions.
> > > > >
> > > > > > the InferMode class should also be introduced in the Public
> > > Interfaces
> > > > > > section!
> > > > >
> > > > >
> > > > > Thanks for the reminder, I have now added the InferMode class to
> the
> > > > Public
> > > > > Interfaces section as well.
> > > > >
> > > > > > `table.exec.hive.infer-source-parallelism.max` is 1024, I checked
> > > > through
> > > > > > the code that the default value is 1000?
> > > > >
> > > > >
> > > > > I have checked and the default value of
> > > > > `table.exec.hive.infer-source-parallelism.max` is indeed 1000. This
> > has
> > > > > been corrected in the FLIP.
> > > > >
> > > > > > how are`table.exec.hive.infer-source-parallelism` and
> > > > > > `table.exec.hive.infer-source-parallelism.mode` compatible?
> > > > >
> > > > >
> > > > > This is indeed a critical point. The current plan is to deprecate
> > > > > `table.exec.hive.infer-source-parallelism` but still utilize it as
> > the
> > > > main
> > > > > switch for enabling automatic parallelism inference. That means, in
> > > terms
> > > > > of priority, `table.exec.hive.infer-source-parallelism` >
> > > > > `table.exec.hive.infer-source-parallelism.mode`. In future
> versions,
> > if
> > > > > `table.exec.hive.infer-source-parallelism` is removed, this logic
> > will
> > > > also
> > > > > need to be revised, leaving only
> > > > > `table.exec.hive.infer-source-parallelism.mode` as the basis for
> > > deciding
> > > > > whether to enable parallelism inference. I have also added this
> > > > description
> > > > > to the FLIP.
> > > > >
> > > > >
> > > > > > In FLIP-367 it is supported to be able to set the Source's
> > > parallelism
> > > > > > individually, if in the future HiveSource also supports this
> > feature,
> > > > > > however, the default value of
> > > > > > `table.exec.hive.infer-source-parallelism.mode` is
> > > `InferMode.DYNAMIC`,
> > > > > at
> > > > > > this point will the parallelism be dynamically derived or will
> the
> > > > > manually
> > > > > > set parallelism take effect, and who has the higher priority?
> > > > >
> > > > >
> > > > > From my understanding, 'manually set parallelism' has the higher
> > > > priority,
> > > > > just like one of the preconditions for the effectiveness of dynamic
> > > > > parallelism inference in the AdaptiveBatchScheduler is that the
> > > vertex's
> > > > > parallelism isn't set. I believe whether it's static inference or
> > > dynamic
> > > > > inference, the manually set parallelism by the user should be
> > > respected.
> > > > >
> > > > > > The `InferMode.NONE` option.
> > > > >
> > > > > Currently, 'adding InferMode.NONE' seems to be the prevailing
> > opinion.
> > > I
> > > > > will add InferMode.NONE as one of the Enum options in InferMode
> > class.
> > > > >
> > > > > Best,
> > > > > Xia
> > > > >
> > > > > Lijie Wang  于2024年4月18日周四 13:50写道:
> > > > >
> > > > > > Thanks for driving the discussion.
> > > > > >
> > > > > > +1 for the proposal and +1 for the `InferMode.NONE` option.
> > > > > >
> > > > > > Best,
> > > > > > Lijie
> > > > > >
> > > > > > Ron liu  于2024年4月18日周四 11:36写道:
> > 

Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread Hangxiang Yu
Hi, Yue.
Very glad to see that IngestDB will be used to improve the rescaling
performance of RocksDB.
And +1 for the upgrade. Thanks for the great work!

On Thu, Apr 25, 2024 at 5:16 AM Martijn Visser 
wrote:

> +1
>
> On Wed, Apr 24, 2024 at 5:31 PM Congxian Qiu 
> wrote:
>
> > Thanks for driving this,  yue
> >
> > We also observed significant performance improvements in some cases after
> > bumped the Rocksdb version, +1 for this work
> >
> > Best,
> > Congxian
> >
> >
> > yue ma  于2024年4月24日周三 19:16写道:
> >
> > > hi Yanfei,
> > >
> > > Thanks for your feedback and reminders I have updated related
> > information.
> > > In fact, most of them use the default Configrations.
> > >
> > > Yanfei Lei  于2024年4月23日周二 12:51写道:
> > >
> > > > Hi Yue & Roman,
> > > >
> > > > Thanks for initiating this FLIP and all the efforts for the upgrade.
> > > >
> > > > 8.10.0 introduces some new features, making it possible for Flink to
> > > > implement some new exciting features, and the upgrade also makes
> > > > FRocksDB easier to maintain, +1 for upgrading.
> > > >
> > > > I read the FLIP and have a minor comment, it would be better to add
> > > > some description about the environment/configuration of the nexmark's
> > > > result.
> > > >
> > > > Roman Khachatryan  于2024年4月23日周二 12:07写道:
> > > >
> > > > >
> > > > > Hi,
> > > > >
> > > > > Thanks for writing the proposal and preparing the upgrade.
> > > > >
> > > > > FRocksDB  definitely needs to be kept in sync with the upstream and
> > the
> > > > new
> > > > > APIs are necessary for faster rescaling.
> > > > > We're already using a similar version internally.
> > > > >
> > > > > I reviewed the FLIP and it looks good to me (disclaimer: I took
> part
> > in
> > > > > some steps of this effort).
> > > > >
> > > > >
> > > > > Regards,
> > > > > Roman
> > > > >
> > > > > On Mon, Apr 22, 2024, 08:11 yue ma  wrote:
> > > > >
> > > > > > Hi Flink devs,
> > > > > >
> > > > > > I would like to start a discussion on FLIP-447: Upgrade FRocksDB
> > from
> > > > > > 6.20.3 to 8.10.0
> > > > > >
> > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-447%3A+Upgrade+FRocksDB+from+6.20.3++to+8.10.0
> > > > > >
> > > > > > This FLIP proposes upgrading the version of FRocksDB in the Flink
> > > > Project
> > > > > > from 6.20.3 to 8.10.0.
> > > > > > The FLIP mainly introduces the main benefits of upgrading
> FRocksDB,
> > > > > > including the use of IngestDB which can improve Rescaling
> > performance
> > > > by
> > > > > > more than 10 times in certain scenarios, as well as other
> potential
> > > > > > optimization points such as async_io, blob db, and tiered
> > storage.The
> > > > > > FLIP also presented test results based on RocksDB 8.10, including
> > > > > > StateBenchmark and Nexmark tests.
> > > > > > Overall, upgrading FRocksDB may result in a small regression of
> > write
> > > > > > performance( which is a very small part of the overall overhead),
> > but
> > > > it
> > > > > > can bring many important performance benefits.
> > > > > > So we hope to upgrade the version of FRocksDB through this FLIP.
> > > > > >
> > > > > > Looking forward to everyone's feedback and suggestions. Thank
> you!
> > > > > > --
> > > > > > Best regards,
> > > > > > Yue
> > > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Best,
> > > > Yanfei
> > > >
> > >
> > >
> > > --
> > > Best,
> > > Yue
> > >
> >
>


-- 
Best,
Hangxiang.


[jira] [Created] (FLINK-35233) HBase lookup result is wrong when lookup cache is enabled

2024-04-24 Thread tanjialiang (Jira)
tanjialiang created FLINK-35233:
---

 Summary: HBase lookup result is wrong when lookup cache is enabled
 Key: FLINK-35233
 URL: https://issues.apache.org/jira/browse/FLINK-35233
 Project: Flink
  Issue Type: Bug
  Components: Connectors / HBase
Affects Versions: hbase-3.0.0
Reporter: tanjialiang


HBase table
||rowkey||name||age||
|1|ben|18|
|2|ken|19|
|3|mark|20|

 
FlinkSQL lookup join with lookup cahce
{code:java}
CREATE TABLE dim_user (
  rowkey STRING,
  info ROW,
  PRIMARY KEY (rowkey) NOT ENFORCED
) WITH (
  'connector' = 'hbase-2.2',
  'zookeeper.quorum' = 'localhost:2181',
  'zookeeper.znode.parent' = '/hbase',
  'table-name' = 'default:test',
  'lookup.cache' = 'PARTIAL',
  'lookup.partial-cache.max-rows' = '1000',
  'lookup.partial-cache.expire-after-write' = '1h'
);

CREATE VIEW user_click AS 
SELECT user_id, proctime() AS proc_time
FROM (
  VALUES('1'), ('2'), ('3'), ('1'), ('2')
) AS t (user_id);

SELECT 
    user_id, 
    info.name, 
    info.age
FROM user_click INNER JOIN dim_user
FOR SYSTEM_TIME AS OF user_click.proc_time
ON dim_user.rowkey = user_click.user_id;{code}
 
Expect Result
||rowkey||name||age||
|1|ben|18|
|2|ken|19|
|3|mark|20|
|1|ben|18|
|2|ken|19|

 

Actual Result
||rowkey||name||age||
|1|ben|18|
|2|ken|19|
|3|mark|20|
|1|mark|20|
|2|mark|20|

 
Wrong result when we lookup user_id 1 and 2 the second time.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread Martijn Visser
+1

On Wed, Apr 24, 2024 at 5:31 PM Congxian Qiu  wrote:

> Thanks for driving this,  yue
>
> We also observed significant performance improvements in some cases after
> bumped the Rocksdb version, +1 for this work
>
> Best,
> Congxian
>
>
> yue ma  于2024年4月24日周三 19:16写道:
>
> > hi Yanfei,
> >
> > Thanks for your feedback and reminders I have updated related
> information.
> > In fact, most of them use the default Configrations.
> >
> > Yanfei Lei  于2024年4月23日周二 12:51写道:
> >
> > > Hi Yue & Roman,
> > >
> > > Thanks for initiating this FLIP and all the efforts for the upgrade.
> > >
> > > 8.10.0 introduces some new features, making it possible for Flink to
> > > implement some new exciting features, and the upgrade also makes
> > > FRocksDB easier to maintain, +1 for upgrading.
> > >
> > > I read the FLIP and have a minor comment, it would be better to add
> > > some description about the environment/configuration of the nexmark's
> > > result.
> > >
> > > Roman Khachatryan  于2024年4月23日周二 12:07写道:
> > >
> > > >
> > > > Hi,
> > > >
> > > > Thanks for writing the proposal and preparing the upgrade.
> > > >
> > > > FRocksDB  definitely needs to be kept in sync with the upstream and
> the
> > > new
> > > > APIs are necessary for faster rescaling.
> > > > We're already using a similar version internally.
> > > >
> > > > I reviewed the FLIP and it looks good to me (disclaimer: I took part
> in
> > > > some steps of this effort).
> > > >
> > > >
> > > > Regards,
> > > > Roman
> > > >
> > > > On Mon, Apr 22, 2024, 08:11 yue ma  wrote:
> > > >
> > > > > Hi Flink devs,
> > > > >
> > > > > I would like to start a discussion on FLIP-447: Upgrade FRocksDB
> from
> > > > > 6.20.3 to 8.10.0
> > > > >
> > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-447%3A+Upgrade+FRocksDB+from+6.20.3++to+8.10.0
> > > > >
> > > > > This FLIP proposes upgrading the version of FRocksDB in the Flink
> > > Project
> > > > > from 6.20.3 to 8.10.0.
> > > > > The FLIP mainly introduces the main benefits of upgrading FRocksDB,
> > > > > including the use of IngestDB which can improve Rescaling
> performance
> > > by
> > > > > more than 10 times in certain scenarios, as well as other potential
> > > > > optimization points such as async_io, blob db, and tiered
> storage.The
> > > > > FLIP also presented test results based on RocksDB 8.10, including
> > > > > StateBenchmark and Nexmark tests.
> > > > > Overall, upgrading FRocksDB may result in a small regression of
> write
> > > > > performance( which is a very small part of the overall overhead),
> but
> > > it
> > > > > can bring many important performance benefits.
> > > > > So we hope to upgrade the version of FRocksDB through this FLIP.
> > > > >
> > > > > Looking forward to everyone's feedback and suggestions. Thank you!
> > > > > --
> > > > > Best regards,
> > > > > Yue
> > > > >
> > >
> > >
> > >
> > > --
> > > Best,
> > > Yanfei
> > >
> >
> >
> > --
> > Best,
> > Yue
> >
>


Re: [DISCUSS] FLIP-445: Support dynamic parallelism inference for HiveSource

2024-04-24 Thread Venkatakrishnan Sowrirajan
Hi Xia,

+1 on introducing dynamic parallelism inference for HiveSource.

Orthogonal to this discussion, curious, how commonly HiveSource is used
these days in the industry given the popularity of table formats/sources
like Iceberg, Hudi and Delta lake?

Thanks
Venkat

On Wed, Apr 24, 2024, 7:41 PM Xia Sun  wrote:

> Hi everyone,
>
> Thanks for all the feedback!
>
> If there are no more comments, I would like to start the vote thread,
> thanks again!
>
> Best,
> Xia
>
> Ahmed Hamdy  于2024年4月18日周四 21:31写道:
>
> > Hi Xia,
> > I have read through the FLIP and discussion and the new version of the
> FLIP
> > looks better.
> > +1 for the proposal.
> > Best Regards
> > Ahmed Hamdy
> >
> >
> > On Thu, 18 Apr 2024 at 12:21, Ron Liu  wrote:
> >
> > > Hi, Xia
> > >
> > > Thanks for updating, looks good to me.
> > >
> > > Best,
> > > Ron
> > >
> > > Xia Sun  于2024年4月18日周四 19:11写道:
> > >
> > > > Hi Ron,
> > > > Yes, presenting it in a table might be more intuitive. I have already
> > > added
> > > > the table in the "Public Interfaces | New Config Option" chapter of
> > FLIP.
> > > > PTAL~
> > > >
> > > > Ron Liu  于2024年4月18日周四 18:10写道:
> > > >
> > > > > Hi, Xia
> > > > >
> > > > > Thanks for your reply.
> > > > >
> > > > > > That means, in terms
> > > > > of priority, `table.exec.hive.infer-source-parallelism` >
> > > > > `table.exec.hive.infer-source-parallelism.mode`.
> > > > >
> > > > > I still have some confusion, if the
> > > > > `table.exec.hive.infer-source-parallelism`
> > > > > >`table.exec.hive.infer-source-parallelism.mode`, currently
> > > > > `table.exec.hive.infer-source-parallelism` default value is true,
> > that
> > > > > means always static parallelism inference work? Or perhaps after
> this
> > > > FLIP,
> > > > > we changed the default behavior of
> > > > > `table.exec.hive.infer-source-parallelism` to indicate dynamic
> > > > parallelism
> > > > > inference when enabled.
> > > > > I think you should list the various behaviors of these two options
> > that
> > > > > coexist in FLIP by a table, only then users can know how the
> dynamic
> > > and
> > > > > static parallelism inference work.
> > > > >
> > > > > Best,
> > > > > Ron
> > > > >
> > > > > Xia Sun  于2024年4月18日周四 16:33写道:
> > > > >
> > > > > > Hi Ron and Lijie,
> > > > > > Thanks for joining the discussion and sharing your suggestions.
> > > > > >
> > > > > > > the InferMode class should also be introduced in the Public
> > > > Interfaces
> > > > > > > section!
> > > > > >
> > > > > >
> > > > > > Thanks for the reminder, I have now added the InferMode class to
> > the
> > > > > Public
> > > > > > Interfaces section as well.
> > > > > >
> > > > > > > `table.exec.hive.infer-source-parallelism.max` is 1024, I
> checked
> > > > > through
> > > > > > > the code that the default value is 1000?
> > > > > >
> > > > > >
> > > > > > I have checked and the default value of
> > > > > > `table.exec.hive.infer-source-parallelism.max` is indeed 1000.
> This
> > > has
> > > > > > been corrected in the FLIP.
> > > > > >
> > > > > > > how are`table.exec.hive.infer-source-parallelism` and
> > > > > > > `table.exec.hive.infer-source-parallelism.mode` compatible?
> > > > > >
> > > > > >
> > > > > > This is indeed a critical point. The current plan is to deprecate
> > > > > > `table.exec.hive.infer-source-parallelism` but still utilize it
> as
> > > the
> > > > > main
> > > > > > switch for enabling automatic parallelism inference. That means,
> in
> > > > terms
> > > > > > of priority, `table.exec.hive.infer-source-parallelism` >
> > > > > > `table.exec.hive.infer-source-parallelism.mode`. In future
> > versions,
> > > if
> > > > > > `table.exec.hive.infer-source-parallelism` is removed, this logic
> > > will
> > > > > also
> > > > > > need to be revised, leaving only
> > > > > > `table.exec.hive.infer-source-parallelism.mode` as the basis for
> > > > deciding
> > > > > > whether to enable parallelism inference. I have also added this
> > > > > description
> > > > > > to the FLIP.
> > > > > >
> > > > > >
> > > > > > > In FLIP-367 it is supported to be able to set the Source's
> > > > parallelism
> > > > > > > individually, if in the future HiveSource also supports this
> > > feature,
> > > > > > > however, the default value of
> > > > > > > `table.exec.hive.infer-source-parallelism.mode` is
> > > > `InferMode.DYNAMIC`,
> > > > > > at
> > > > > > > this point will the parallelism be dynamically derived or will
> > the
> > > > > > manually
> > > > > > > set parallelism take effect, and who has the higher priority?
> > > > > >
> > > > > >
> > > > > > From my understanding, 'manually set parallelism' has the higher
> > > > > priority,
> > > > > > just like one of the preconditions for the effectiveness of
> dynamic
> > > > > > parallelism inference in the AdaptiveBatchScheduler is that the
> > > > vertex's
> > > > > > parallelism isn't set. I believe whether it's static inference or
> > > > dynamic
> > > > > > inference, the manually set parallelism by the user 

Re: [ DISCUSS ] FLIP-XXX : [Plugin] Enhancing Flink Failure Management in Kubernetes with Dynamic Termination Log Integration

2024-04-24 Thread Martijn Visser
I would prefer a separate FLIP

On Wed, Apr 24, 2024 at 3:25 PM Swathi C  wrote:

> Sure Ahmed and Martijn.
> Fetching the flink particular job related failure and adding this logic to
> termination-log is definitely a sub-task of pluggable enricher as we can
> leverage pluggable enricher to achieve this.
> But for CRUD level failures, which is mainly used to notify if the job
> manager failed might not be using the pluggable enricher. So, let us know
> if that needs to be there as a separate FLIP or we can combine that as well
> under the pluggable enricher ( by adding another sub task ) ?
>
> Regards,
> Swathi C
>
> On Wed, Apr 24, 2024 at 3:46 PM Ahmed Hamdy  wrote:
>
> > Hi,
> > I agree with the Martijn, We can reformulate the FLIP to introduce
> > termination log as supported pluggable enricher. If you believe the scope
> > of work is a subset (Further implementation) we can just add a Jira
> ticket
> > for it. IMO this will also help with implementation taking the existing
> > enrichers into reference.
> > Best Regards
> > Ahmed Hamdy
> >
> >
> > On Tue, 23 Apr 2024 at 15:23, Martijn Visser 
> > wrote:
> >
> > > From a procedural point of view, we shouldn't make FLIPs sub-tasks for
> > > existing FLIPs that have been voted/are released. That will only cause
> > > confusion down the line. A new FLIP should take existing functionality
> > > (like FLIP-304) into account, and propose how to improve on what that
> > > original FLIP has introduced or how you're going to leverage what's
> > already
> > > there.
> > >
> > > On Tue, Apr 23, 2024 at 11:42 AM ramkrishna vasudevan <
> > > ramvasu.fl...@gmail.com> wrote:
> > >
> > > > Hi Gyula and Ahmed,
> > > >
> > > > I totally agree that there is an interlap in the final goal that both
> > the
> > > > FLIPs are achieving here and infact FLIP-304 is more comprehensive
> for
> > > job
> > > > failures.
> > > >
> > > > But as a proposal to move forward can we make Swathi's FLIP/JIRA as a
> > sub
> > > > task for FLIP-304 and continue with the PR since the main aim is to
> get
> > > the
> > > > cluster failure pushed to the termination log for K8s based
> > deployments.
> > > > And once it is completed we can work to make FLIP-304 to support job
> > > > failure propagation to termination log?
> > > >
> > > > Regards
> > > > Ram
> > > >
> > > > On Thu, Apr 18, 2024 at 10:07 PM Swathi C  >
> > > > wrote:
> > > >
> > > > > Hi Gyula and  Ahmed,
> > > > >
> > > > > Thanks for reviewing this.
> > > > >
> > > > > @gyula.f...@gmail.com  , currently since our
> > aim
> > > > as
> > > > > part of this FLIP was only to fail the cluster when job
> manager/flink
> > > has
> > > > > issues such that the cluster would no longer be usable, hence, we
> > > > proposed
> > > > > only related to that.
> > > > > Your right, that it covers only job main class errors, job manager
> > run
> > > > time
> > > > > failures, if the Job manager wants to write any metadata to any
> other
> > > > > system ( ABFS, S3 , ... )  and the job failures will not be
> covered.
> > > > >
> > > > > FLIP-304 is mainly used to provide Failure enrichers for job
> > failures.
> > > > > Since, this FLIP is mainly for flink Job manager failures, let us
> > know
> > > if
> > > > > we can leverage the goodness of both and try to extend FLIP-304 and
> > add
> > > > our
> > > > > plugin implementation to cover the job level issues ( propagate
> this
> > > info
> > > > > to the /dev/termination-log such that, the container status reports
> > it
> > > > for
> > > > > flink on K8S by implementing Failure Enricher interface and
> > > > > processFailure() to do this ) and use this FLIP proposal for
> generic
> > > > flink
> > > > > cluster (Job manager/cluster ) failures.
> > > > >
> > > > > Regards,
> > > > > Swathi C
> > > > >
> > > > > On Thu, Apr 18, 2024 at 7:36 PM Ahmed Hamdy 
> > > > wrote:
> > > > >
> > > > > > Hi Swathi!
> > > > > > Thanks for the proposal.
> > > > > > Could you please elaborate what this FLIP offers more than
> > > Flip-304[1]?
> > > > > > Flip 304 proposes a Pluggable mechanism for enriching Job
> failures,
> > > If
> > > > I
> > > > > am
> > > > > > not mistaken this proposal looks like a subset of it.
> > > > > >
> > > > > > 1-
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-304%3A+Pluggable+Failure+Enrichers
> > > > > >
> > > > > > Best Regards
> > > > > > Ahmed Hamdy
> > > > > >
> > > > > >
> > > > > > On Thu, 18 Apr 2024 at 08:23, Gyula Fóra 
> > > wrote:
> > > > > >
> > > > > > > Hi Swathi!
> > > > > > >
> > > > > > > Thank you for creating this proposal. I really like the general
> > > idea
> > > > of
> > > > > > > increasing the K8s native observability of Flink job errors.
> > > > > > >
> > > > > > > I took a quick look at your reference PR, the termination log
> > > related
> > > > > > logic
> > > > > > > is contained completely in the ClusterEntrypoint. What type of
> > > errors
> > > > > > will
> > > > > > > this 

Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread yue ma
Hi lorenzo.affetti,

Thank you for your feedback. I'm sorry, I didn't fully understand your
question.
>From the results of the StateBenchmark, it appears that interfaces with
regression mainly include
(listAdd/listAddAll/mapUpdate/mapRemove/valueAdd/valueUpdate), all of which
call Rocksdb.put. So the conclusion is that it will only affect write
performance.
Of course, I think it is also important and meaningful to investigate the
reasons for write performance regression. I will create tickets to track
the progress. But investigating the cause will not block our upgrade
process this time. I believe that these two can be done in parallel.

Thanks again.

 于2024年4月24日周三 15:41写道:

> Hello yuema!
>
> Thank you for the proposal.
> In light of what is happening to state backends with FLIP-423 and others,
> it definitely makes sense to leverage the full power of latest FRocksDB.
>
> Small nit and question for you:
> do you have any idea how to justify the regression in write perf?
>
> Thank you again, +1 for this.
> On Apr 24, 2024 at 07:58 +0200, Gyula Fóra , wrote:
> > Thank you for driving this effort
> >
> > +1
> >
> > Cheers
> > Gyula
> >
> > On Wed, 24 Apr 2024 at 06:25, Yuan Mei  wrote:
> >
> > > Hey Yue,
> > >
> > > Thanks for all the great efforts significantly improving rescaling and
> > > upgrading rocksdb.
> > >
> > > +1 for this.
> > >
> > > Best
> > > Yuan
> > >
> > > On Wed, Apr 24, 2024 at 10:46 AM Zakelly Lan 
> > > wrote:
> > >
> > > > > Hi Yue,
> > > > >
> > > > > Thanks for this proposal!
> > > > >
> > > > > Given the great improvement we could have, the slight regression
> in write
> > > > > performance is a worthwhile trade-off, particularly as the
> mem-table
> > > > > operations contribute only a minor part to the overall overhead.
> So +1
> > > for
> > > > > this.
> > > > >
> > > > >
> > > > > Best,
> > > > > Zakelly
> > > > >
> > > > > On Tue, Apr 23, 2024 at 12:53 PM Yun Tang 
> wrote:
> > > > >
> > > > > > > Hi Yue,
> > > > > > >
> > > > > > > Thanks for driving this work.
> > > > > > >
> > > > > > > It has been three years since last major upgrade of FRocksDB.
> And it
> > > > > would
> > > > > > > be great improvement of Flink's state-backend with this
> upgrade. +1 for
> > > > > > > this work.
> > > > > > >
> > > > > > >
> > > > > > > Best
> > > > > > > Yun Tang
> > > > > > > 
> > > > > > > From: Yanfei Lei 
> > > > > > > Sent: Tuesday, April 23, 2024 12:50
> > > > > > > To: dev@flink.apache.org 
> > > > > > > Subject: Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3
> to 8.10.0
> > > > > > >
> > > > > > > Hi Yue & Roman,
> > > > > > >
> > > > > > > Thanks for initiating this FLIP and all the efforts for the
> upgrade.
> > > > > > >
> > > > > > > 8.10.0 introduces some new features, making it possible for
> Flink to
> > > > > > > implement some new exciting features, and the upgrade also
> makes
> > > > > > > FRocksDB easier to maintain, +1 for upgrading.
> > > > > > >
> > > > > > > I read the FLIP and have a minor comment, it would be better
> to add
> > > > > > > some description about the environment/configuration of the
> nexmark's
> > > > > > > result.
> > > > > > >
> > > > > > > Roman Khachatryan  于2024年4月23日周二 12:07写道:
> > > > > > >
> > > > > > > > >
> > > > > > > > > Hi,
> > > > > > > > >
> > > > > > > > > Thanks for writing the proposal and preparing the upgrade.
> > > > > > > > >
> > > > > > > > > FRocksDB definitely needs to be kept in sync with the
> upstream and
> > > the
> > > > > > > new
> > > > > > > > > APIs are necessary for faster rescaling.
> > > > > > > > > We're already using a similar version internally.
> > > > > > > > >
> > > > > > > > > I reviewed the FLIP and it looks good to me (disclaimer: I
> took part
> > > in
> > > > > > > > > some steps of this effort).
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Regards,
> > > > > > > > > Roman
> > > > > > > > >
> > > > > > > > > On Mon, Apr 22, 2024, 08:11 yue ma 
> wrote:
> > > > > > > > >
> > > > > > > > > > > Hi Flink devs,
> > > > > > > > > > >
> > > > > > > > > > > I would like to start a discussion on FLIP-447:
> Upgrade FRocksDB
> > > from
> > > > > > > > > > > 6.20.3 to 8.10.0
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-447%3A+Upgrade+FRocksDB+from+6.20.3++to+8.10.0
> > > > > > > > > > >
> > > > > > > > > > > This FLIP proposes upgrading the version of FRocksDB
> in the Flink
> > > > > > > Project
> > > > > > > > > > > from 6.20.3 to 8.10.0.
> > > > > > > > > > > The FLIP mainly introduces the main benefits of
> upgrading FRocksDB,
> > > > > > > > > > > including the use of IngestDB which can improve
> Rescaling
> > > performance
> > > > > > > by
> > > > > > > > > > > more than 10 times in certain scenarios, as well as
> other potential
> > > > > > > > > > > optimization points such as async_io, blob db, and
> tiered
> > > storage.The
> > > > > > > > > > 

[jira] [Created] (FLINK-35234) Fix NullPointerException of org.apache.flink.cdc.common.configuration.ConfigurationUtils#convertToString

2024-04-24 Thread Hongshun Wang (Jira)
Hongshun Wang created FLINK-35234:
-

 Summary: Fix NullPointerException of 
org.apache.flink.cdc.common.configuration.ConfigurationUtils#convertToString
 Key: FLINK-35234
 URL: https://issues.apache.org/jira/browse/FLINK-35234
 Project: Flink
  Issue Type: Bug
  Components: Flink CDC
Affects Versions: cdc-3.1.0
Reporter: Hongshun Wang
 Fix For: cdc-3.2.0


Exception like this:
{code:java}
Caused by: java.lang.NullPointerExceptionat 
org.apache.flink.cdc.common.configuration.ConfigurationUtils.convertToString(ConfigurationUtils.java:133)
 ~[?:?]at 
org.apache.flink.cdc.common.configuration.Configuration.toMap(Configuration.java:138)
 ~[?:?] {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-35228) DynamicKafkaSource does not read re-added topic for the same cluster

2024-04-24 Thread Jira
Ignas Daukšas created FLINK-35228:
-

 Summary: DynamicKafkaSource does not read re-added topic for the 
same cluster
 Key: FLINK-35228
 URL: https://issues.apache.org/jira/browse/FLINK-35228
 Project: Flink
  Issue Type: Bug
  Components: Connectors / Kafka
Affects Versions: kafka-3.1.0
Reporter: Ignas Daukšas


When using DynamicKafkaSource, if topic is removed from the cluster (that has 
more active topics remaining) and then re-added back, consumption from that 
topic won't be happening.

However, if the topic in question is the only topic in that cluster, then 
everything works as expected - consumption restarts once cluster-topic is 
re-added.

Steps to reproduce:
 # Have a DynamicKafkaSource.
 # Have KafkaMetadataService report a single cluster with two topics (A and B) 
for the subscribed stream/streams.
 # Consume some data, topics A and B are consumed as expected.
 # Have KafkaMetadataService remove topic A.
 # Continue consuming data, only topic B consumed as expected.
 # Have KafkaMetadataService re-add topic A.
 # Continue consuming data, however only topic B is actually consumed - this is 
not expected.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [VOTE] FLIP-436: Introduce Catalog-related Syntax

2024-04-24 Thread Yubin Li
Hi everyone,

During the implementation of the "describe catalog" syntax, it was
found that the original output style needed to be improved.
```
desc catalog extended cat2;
+--+-+
| catalog_description_item |
catalog_description_value |
+--+-+
| Name |
 cat2 |
| Type |
generic_in_memory |
|  Comment |
  |
|   Properties | ('default-database','db'),
('type','generic_in_memory') |
+--+-+
4 rows in set
```
After offline discussions with Jane Chan and Jark Wu, we suggest
improving it to the following form:
```
desc catalog extended cat2;
+-+---+
|   info name |info value |
+-+---+
|name |  cat2 |
|type | generic_in_memory |
| comment |   |
| option:default-database |db |
+-+---+
4 rows in set
```

For the following reasons:
1. The title should be consistent with engines such as Databricks for
easy understanding, and it should also be consistent with Flink's own
naming style. Therefore, the title adopts "info name", "info value",
and the key name should be unified in lowercase, so "Name" is replaced
by "name".
Note: Databricks output style [1] as follows:
```
> DESCRIBE CATALOG main;
 info_name info_value
   
 Catalog Name  main
  Comment   Main catalog (auto-created)
Owner metastore-admin-users
 Catalog Type   Regular
```
2. There may be many attributes of the catalog, and it is very poor in
readability when displayed in one line. It should be expanded into
multiple lines, and the key name is prefixed with "option:" to
identify that this is an attribute row. And since `type` is an
important information of the catalog, even if `extended` is not
specified, it should also be displayed, and correspondingly,
"option:type" should be removed to avoid redundancy.

WDYT? Looking forward to your reply!

[1] 
https://learn.microsoft.com/zh-tw/azure/databricks/sql/language-manual/sql-ref-syntax-aux-describe-catalog

Best,
Yubin

On Wed, Mar 20, 2024 at 2:15 PM Benchao Li  wrote:
>
> +1 (binding)
>
> gongzhongqiang  于2024年3月20日周三 11:40写道:
> >
> > +1 (non-binding)
> >
> > Best,
> > Zhongqiang Gong
> >
> > Yubin Li  于2024年3月19日周二 18:03写道:
> >
> > > Hi everyone,
> > >
> > > Thanks for all the feedback, I'd like to start a vote on the FLIP-436:
> > > Introduce Catalog-related Syntax [1]. The discussion thread is here
> > > [2].
> > >
> > > The vote will be open for at least 72 hours unless there is an
> > > objection or insufficient votes.
> > >
> > > [1]
> > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-436%3A+Introduce+Catalog-related+Syntax
> > > [2] https://lists.apache.org/thread/10k1bjb4sngyjwhmfqfky28lyoo7sv0z
> > >
> > > Best regards,
> > > Yubin
> > >
>
>
>
> --
>
> Best,
> Benchao Li


Re: [VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD

2024-04-24 Thread Yuepeng Pan
+1(non-binding)


Best,
Yuepeng Pan 

At 2024-04-24 16:05:07, "Rui Fan" <1996fan...@gmail.com> wrote:
>+1(binding)
>
>Best,
>Rui
>
>On Wed, Apr 24, 2024 at 4:03 PM Mate Czagany  wrote:
>
>> Hi everyone,
>>
>> I'd like to start a vote on the FLIP-446: Kubernetes Operator State
>> Snapshot CRD [1]. The discussion thread is here [2].
>>
>> The vote will be open for at least 72 hours unless there is an objection or
>> insufficient votes.
>>
>> [1]
>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD
>> [2] https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7
>>
>> Regards,
>> Mate
>>


Re: [VOTE] FLIP-436: Introduce Catalog-related Syntax

2024-04-24 Thread Ahmed Hamdy
Hi, +1 (non-binding)
Best Regards
Ahmed Hamdy


On Wed, 24 Apr 2024 at 09:58, Yubin Li  wrote:

> Hi everyone,
>
> During the implementation of the "describe catalog" syntax, it was
> found that the original output style needed to be improved.
> ```
> desc catalog extended cat2;
>
> +--+-+
> | catalog_description_item |
> catalog_description_value |
>
> +--+-+
> | Name |
>  cat2 |
> | Type |
> generic_in_memory |
> |  Comment |
>   |
> |   Properties | ('default-database','db'),
> ('type','generic_in_memory') |
>
> +--+-+
> 4 rows in set
> ```
> After offline discussions with Jane Chan and Jark Wu, we suggest
> improving it to the following form:
> ```
> desc catalog extended cat2;
> +-+---+
> |   info name |info value |
> +-+---+
> |name |  cat2 |
> |type | generic_in_memory |
> | comment |   |
> | option:default-database |db |
> +-+---+
> 4 rows in set
> ```
>
> For the following reasons:
> 1. The title should be consistent with engines such as Databricks for
> easy understanding, and it should also be consistent with Flink's own
> naming style. Therefore, the title adopts "info name", "info value",
> and the key name should be unified in lowercase, so "Name" is replaced
> by "name".
> Note: Databricks output style [1] as follows:
> ```
> > DESCRIBE CATALOG main;
>  info_name info_value
>    
>  Catalog Name  main
>   Comment   Main catalog (auto-created)
> Owner metastore-admin-users
>  Catalog Type   Regular
> ```
> 2. There may be many attributes of the catalog, and it is very poor in
> readability when displayed in one line. It should be expanded into
> multiple lines, and the key name is prefixed with "option:" to
> identify that this is an attribute row. And since `type` is an
> important information of the catalog, even if `extended` is not
> specified, it should also be displayed, and correspondingly,
> "option:type" should be removed to avoid redundancy.
>
> WDYT? Looking forward to your reply!
>
> [1]
> https://learn.microsoft.com/zh-tw/azure/databricks/sql/language-manual/sql-ref-syntax-aux-describe-catalog
>
> Best,
> Yubin
>
> On Wed, Mar 20, 2024 at 2:15 PM Benchao Li  wrote:
> >
> > +1 (binding)
> >
> > gongzhongqiang  于2024年3月20日周三 11:40写道:
> > >
> > > +1 (non-binding)
> > >
> > > Best,
> > > Zhongqiang Gong
> > >
> > > Yubin Li  于2024年3月19日周二 18:03写道:
> > >
> > > > Hi everyone,
> > > >
> > > > Thanks for all the feedback, I'd like to start a vote on the
> FLIP-436:
> > > > Introduce Catalog-related Syntax [1]. The discussion thread is here
> > > > [2].
> > > >
> > > > The vote will be open for at least 72 hours unless there is an
> > > > objection or insufficient votes.
> > > >
> > > > [1]
> > > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-436%3A+Introduce+Catalog-related+Syntax
> > > > [2] https://lists.apache.org/thread/10k1bjb4sngyjwhmfqfky28lyoo7sv0z
> > > >
> > > > Best regards,
> > > > Yubin
> > > >
> >
> >
> >
> > --
> >
> > Best,
> > Benchao Li
>


Re: [VOTE] Release flink-connector-kafka v3.2.0, release candidate #1

2024-04-24 Thread Ahmed Hamdy
Thanks Danny,
+1 (non-binding)

- Verified Checksums and hashes
- Verified Signatures
- Reviewed web PR
- github tag exists
- Build source


Best Regards
Ahmed Hamdy


On Tue, 23 Apr 2024 at 03:47, Muhammet Orazov 
wrote:

> Thanks Danny, +1 (non-binding)
>
> - Checked 512 hash
> - Checked gpg signature
> - Reviewed pr
> - Built the source with JDK 11 & 8
>
> Best,
> Muhammet
>
> On 2024-04-22 13:55, Danny Cranmer wrote:
> > Hi everyone,
> >
> > Please review and vote on release candidate #1 for
> > flink-connector-kafka
> > v3.2.0, as follows:
> > [ ] +1, Approve the release
> > [ ] -1, Do not approve the release (please provide specific comments)
> >
> > This release supports Flink 1.18 and 1.19.
> >
> > The complete staging area is available for your review, which includes:
> > * JIRA release notes [1],
> > * the official Apache source release to be deployed to dist.apache.org
> > [2],
> > which are signed with the key with fingerprint 125FD8DB [3],
> > * all artifacts to be deployed to the Maven Central Repository [4],
> > * source code tag v3.2.0-rc1 [5],
> > * website pull request listing the new release [6].
> > * CI build of the tag [7].
> >
> > The vote will be open for at least 72 hours. It is adopted by majority
> > approval, with at least 3 PMC affirmative votes.
> >
> > Thanks,
> > Danny
> >
> > [1]
> >
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12354209
> > [2]
> >
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-kafka-3.2.0-rc1
> > [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> > [4]
> > https://repository.apache.org/content/repositories/orgapacheflink-1723
> > [5]
> > https://github.com/apache/flink-connector-kafka/releases/tag/v3.2.0-rc1
> > [6] https://github.com/apache/flink-web/pull/738
> > [7] https://github.com/apache/flink-connector-kafka
>


Re: [VOTE] Release flink-connector-cassandra v3.2.0, release candidate #1

2024-04-24 Thread Ahmed Hamdy
Thanks Danny,
+1 (non-binding)

- Verified Checksums and hashes
- Verified Signatures
- Reviewed web PR
- github tag exists
- Build source

Best Regards
Ahmed Hamdy


On Tue, 23 Apr 2024 at 03:57, Muhammet Orazov 
wrote:

> Thanks! +1 (non-binding)
>
> - Checked 512 hash
> - Checked gpg signature
> - Reviewed web pr & release notes
> - Built the source with JDK 11 & 8
>
> Best,
> Muhammet
>
> On 2024-04-22 13:04, Danny Cranmer wrote:
> > Hi everyone,
> >
> > Please review and vote on release candidate #1 for
> > flink-connector-cassandra v3.2.0, as follows:
> > [ ] +1, Approve the release
> > [ ] -1, Do not approve the release (please provide specific comments)
> >
> > This release supports Flink 1.18 and 1.19.
> >
> > The complete staging area is available for your review, which includes:
> > * JIRA release notes [1],
> > * the official Apache source release to be deployed to dist.apache.org
> > [2],
> > which are signed with the key with fingerprint 125FD8DB [3],
> > * all artifacts to be deployed to the Maven Central Repository [4],
> > * source code tag v3.2.0-rc1 [5],
> > * website pull request listing the new release [6].
> > * CI build of the tag [7].
> >
> > The vote will be open for at least 72 hours. It is adopted by majority
> > approval, with at least 3 PMC affirmative votes.
> >
> > Thanks,
> > Danny
> >
> > [1]
> >
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353148
> > [2]
> >
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-cassandra-3.2.0-rc1
> > [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> > [4]
> > https://repository.apache.org/content/repositories/orgapacheflink-1722
> > [5]
> >
> https://github.com/apache/flink-connector-cassandra/releases/tag/v3.2.0-rc1
> > [6] https://github.com/apache/flink-web/pull/737
> > [7]
> >
> https://github.com/apache/flink-connector-cassandra/actions/runs/8784310241
>


Re: [DISCUSS] FLIP-XXX Support currentFetchEventTimeLag and processingLag metrics

2024-04-24 Thread lorenzo . affetti
Hello jialiang tan, and thank you for your contribution!

Here are my questions:

1 - I don't know if exposing this as metric to the user could create some harm. 
There can be major discrepancies between the absolute time got by the 
TaskManagers (if clocks are not synchronized via ntp for example), and the 
results of the metric might be quite distant for different TMs. Furthermore, 
comparing the time on the Flink cluster with the event time of records might 
introduce completely inaccurate results. I think providing this should come 
with many disclaimers to the user. Maybe, more experienced contributors can 
comment on this as well.

2 - I don't think the name `processingLag` represents the processing time 
spent, I would rather see `processingTime` just for the semantics of the name 
itself.

3 - Do you really think the `processingTime` should be a gauge? I understand 
your justification for the fetch lag, but I think the processing time should be 
an histogram. For the inefficiency of this, how about some sampling (e.g.: only 
update the histogram 1 every 1000 events?)

4 - At this point, if we have the processing time and number of records, we 
could also add throughput as a metric, so that the user would know how many 
records/second the source is able to produce.

5 - For the "Kafka Connector" section: can this be generalized for connectors 
in general? Can you provide an example to better understand your statement 
about reflection?

6 - Does this introduce any UI change for representing the metric?

Thank you!
On Apr 22, 2024 at 12:26 +0200, jialiang tan , wrote:
> Sorry all, it seems bad formatting in my email message, now I send it again
> gently and hope it work.
>
> I would like to start a discussion about FLIP-XXX:
> SupportcurrentFetchEventTimeLag and processingLag metrics [1].
>
> The main motivation for this change was that I had some difficulties
> inimplementing the currentFetchEventTimeLag metrics for KafkaSource [2].
>
> So I proposed to let the SourceReaderMetricGroup provide an interface to
> capturethe FetchTime so that all the FLIP-27 [3] sources can easily
> implement thecurrentFetchEventTimeLag metrics.
>
> In addition, I propose to support the processingLag metric for the
> FLIP-27sources to measure the current processing latency of the source.
>
> See the FLIP [1] and Jira [2] for more details.
>
> Looking forward to your comments and opinions!
>
> Thanks,
> TanJiaLiang.
>
> [1]
> https://docs.google.com/document/d/1nPhh1A-v-a7zyQyl1A5-K5DeUqbfxNXdjr2TVBT-QMs/edit?usp=sharing
> [2] https://issues.apache.org/jira/browse/FLINK-33173
> [3]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface
>
> >


Re: [DISCUSS] FLIP-448: Introduce Pluggable Workflow Scheduler Interface for Materialized Table

2024-04-24 Thread lorenzo . affetti
Hello Ron Liu! Thank you for your FLIP!

Here are my considerations:

1.
About the Operations interfaces, how can they be empty?
Should not they provide at least a `run` or `execute` method (similar to the 
command pattern)?
In this way, their implementation can wrap all the implementations details of 
particular schedulers, and the scheduler can simply execute the command.
In general, I think a simple sequence diagram showcasing the interaction 
between the interfaces would be awesome to better understand the concept.

2.
What about the RefreshHandler, I cannot find a definition of its interface here.
Is it out of scope for this FLIP?

3.
For the SqlGatewayService arguments:

            boolean isPeriodic,
            @Nullable String scheduleTime,
            @Nullable String scheduleTimeFormat,

If it is periodic, where is the period?
For the scheduleTime and format, why not simply pass an instance of 
LocalDateTime or similar? The gateway should not have the responsibility to 
parse the time.

4.
For the REST API:
wouldn't it be better (more REST) to move the `mt_identifier` to the URL?
E.g.: v3/materialized_tables//refresh

Thank you!
On Apr 22, 2024 at 08:42 +0200, Ron Liu , wrote:
> Hi, Dev
>
> I would like to start a discussion about FLIP-448: Introduce Pluggable
> Workflow Scheduler Interface for Materialized Table.
>
> In FLIP-435[1], we proposed Materialized Table, which has two types of data
> refresh modes: Full Refresh & Continuous Refresh Mode. In Full Refresh
> mode, the Materialized Table relies on a workflow scheduler to perform
> periodic refresh operation to achieve the desired data freshness.
>
> There are numerous open-source workflow schedulers available, with popular
> ones including Airflow and DolphinScheduler. To enable Materialized Table
> to work with different workflow schedulers, we propose a pluggable workflow
> scheduler interface for Materialized Table in this FLIP.
>
> For more details, see FLIP-448 [2]. Looking forward to your feedback.
>
> [1] https://lists.apache.org/thread/c1gnn3bvbfs8v1trlf975t327s4rsffs
> [2]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-448%3A+Introduce+Pluggable+Workflow+Scheduler+Interface+for+Materialized+Table
>
> Best,
> Ron


[jira] [Created] (FLINK-35222) Adding getJobType for AccessExecutionGraph

2024-04-24 Thread Rui Fan (Jira)
Rui Fan created FLINK-35222:
---

 Summary: Adding getJobType for AccessExecutionGraph
 Key: FLINK-35222
 URL: https://issues.apache.org/jira/browse/FLINK-35222
 Project: Flink
  Issue Type: Sub-task
  Components: Runtime / Web Frontend
Reporter: Rui Fan
Assignee: Rui Fan
 Fix For: 1.20.0


Adding getJobType for AccessExecutionGraph interface, and all implementations 
need to overrite it.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-35224) Show the JobType on Flink WebUI

2024-04-24 Thread Rui Fan (Jira)
Rui Fan created FLINK-35224:
---

 Summary: Show the JobType on Flink WebUI
 Key: FLINK-35224
 URL: https://issues.apache.org/jira/browse/FLINK-35224
 Project: Flink
  Issue Type: Sub-task
  Components: Runtime / Web Frontend
Reporter: Rui Fan
Assignee: Rui Fan
 Fix For: 1.20.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-35225) Remove Execution mode in Flink WebUI

2024-04-24 Thread Rui Fan (Jira)
Rui Fan created FLINK-35225:
---

 Summary: Remove Execution mode in Flink WebUI
 Key: FLINK-35225
 URL: https://issues.apache.org/jira/browse/FLINK-35225
 Project: Flink
  Issue Type: Sub-task
  Components: Runtime / Web Frontend
Reporter: Rui Fan
Assignee: Rui Fan
 Fix For: 1.20.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-35226) Deprecate execution-mode in ExecutionConfigInfo related rest api

2024-04-24 Thread Rui Fan (Jira)
Rui Fan created FLINK-35226:
---

 Summary: Deprecate execution-mode in ExecutionConfigInfo related 
rest api
 Key: FLINK-35226
 URL: https://issues.apache.org/jira/browse/FLINK-35226
 Project: Flink
  Issue Type: Sub-task
  Components: Runtime / Web Frontend
Reporter: Rui Fan
Assignee: Rui Fan
 Fix For: 1.20.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD

2024-04-24 Thread Mate Czagany
Hi everyone,

I'd like to start a vote on the FLIP-446: Kubernetes Operator State
Snapshot CRD [1]. The discussion thread is here [2].

The vote will be open for at least 72 hours unless there is an objection or
insufficient votes.

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD
[2] https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7

Regards,
Mate


Re: [VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD

2024-04-24 Thread Ferenc Csaky
+1 (non-binding), looking forward to this!

Best,
Ferenc




On Wednesday, April 24th, 2024 at 10:03, Mate Czagany  
wrote:

> 
> 
> Hi everyone,
> 
> I'd like to start a vote on the FLIP-446: Kubernetes Operator State
> Snapshot CRD [1]. The discussion thread is here [2].
> 
> The vote will be open for at least 72 hours unless there is an objection or
> insufficient votes.
> 
> [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD
> [2] https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7
> 
> Regards,
> Mate


Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread lorenzo . affetti
Hello yuema!

Thank you for the proposal.
In light of what is happening to state backends with FLIP-423 and others, it 
definitely makes sense to leverage the full power of latest FRocksDB.

Small nit and question for you:
do you have any idea how to justify the regression in write perf?

Thank you again, +1 for this.
On Apr 24, 2024 at 07:58 +0200, Gyula Fóra , wrote:
> Thank you for driving this effort
>
> +1
>
> Cheers
> Gyula
>
> On Wed, 24 Apr 2024 at 06:25, Yuan Mei  wrote:
>
> > Hey Yue,
> >
> > Thanks for all the great efforts significantly improving rescaling and
> > upgrading rocksdb.
> >
> > +1 for this.
> >
> > Best
> > Yuan
> >
> > On Wed, Apr 24, 2024 at 10:46 AM Zakelly Lan 
> > wrote:
> >
> > > > Hi Yue,
> > > >
> > > > Thanks for this proposal!
> > > >
> > > > Given the great improvement we could have, the slight regression in 
> > > > write
> > > > performance is a worthwhile trade-off, particularly as the mem-table
> > > > operations contribute only a minor part to the overall overhead. So +1
> > for
> > > > this.
> > > >
> > > >
> > > > Best,
> > > > Zakelly
> > > >
> > > > On Tue, Apr 23, 2024 at 12:53 PM Yun Tang  wrote:
> > > >
> > > > > > Hi Yue,
> > > > > >
> > > > > > Thanks for driving this work.
> > > > > >
> > > > > > It has been three years since last major upgrade of FRocksDB. And it
> > > > would
> > > > > > be great improvement of Flink's state-backend with this upgrade. +1 
> > > > > > for
> > > > > > this work.
> > > > > >
> > > > > >
> > > > > > Best
> > > > > > Yun Tang
> > > > > > 
> > > > > > From: Yanfei Lei 
> > > > > > Sent: Tuesday, April 23, 2024 12:50
> > > > > > To: dev@flink.apache.org 
> > > > > > Subject: Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 
> > > > > > 8.10.0
> > > > > >
> > > > > > Hi Yue & Roman,
> > > > > >
> > > > > > Thanks for initiating this FLIP and all the efforts for the upgrade.
> > > > > >
> > > > > > 8.10.0 introduces some new features, making it possible for Flink to
> > > > > > implement some new exciting features, and the upgrade also makes
> > > > > > FRocksDB easier to maintain, +1 for upgrading.
> > > > > >
> > > > > > I read the FLIP and have a minor comment, it would be better to add
> > > > > > some description about the environment/configuration of the 
> > > > > > nexmark's
> > > > > > result.
> > > > > >
> > > > > > Roman Khachatryan  于2024年4月23日周二 12:07写道:
> > > > > >
> > > > > > > >
> > > > > > > > Hi,
> > > > > > > >
> > > > > > > > Thanks for writing the proposal and preparing the upgrade.
> > > > > > > >
> > > > > > > > FRocksDB definitely needs to be kept in sync with the upstream 
> > > > > > > > and
> > the
> > > > > > new
> > > > > > > > APIs are necessary for faster rescaling.
> > > > > > > > We're already using a similar version internally.
> > > > > > > >
> > > > > > > > I reviewed the FLIP and it looks good to me (disclaimer: I took 
> > > > > > > > part
> > in
> > > > > > > > some steps of this effort).
> > > > > > > >
> > > > > > > >
> > > > > > > > Regards,
> > > > > > > > Roman
> > > > > > > >
> > > > > > > > On Mon, Apr 22, 2024, 08:11 yue ma  wrote:
> > > > > > > >
> > > > > > > > > > Hi Flink devs,
> > > > > > > > > >
> > > > > > > > > > I would like to start a discussion on FLIP-447: Upgrade 
> > > > > > > > > > FRocksDB
> > from
> > > > > > > > > > 6.20.3 to 8.10.0
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-447%3A+Upgrade+FRocksDB+from+6.20.3++to+8.10.0
> > > > > > > > > >
> > > > > > > > > > This FLIP proposes upgrading the version of FRocksDB in the 
> > > > > > > > > > Flink
> > > > > > Project
> > > > > > > > > > from 6.20.3 to 8.10.0.
> > > > > > > > > > The FLIP mainly introduces the main benefits of upgrading 
> > > > > > > > > > FRocksDB,
> > > > > > > > > > including the use of IngestDB which can improve Rescaling
> > performance
> > > > > > by
> > > > > > > > > > more than 10 times in certain scenarios, as well as other 
> > > > > > > > > > potential
> > > > > > > > > > optimization points such as async_io, blob db, and tiered
> > storage.The
> > > > > > > > > > FLIP also presented test results based on RocksDB 8.10, 
> > > > > > > > > > including
> > > > > > > > > > StateBenchmark and Nexmark tests.
> > > > > > > > > > Overall, upgrading FRocksDB may result in a small 
> > > > > > > > > > regression of
> > write
> > > > > > > > > > performance( which is a very small part of the overall 
> > > > > > > > > > overhead),
> > but
> > > > > > it
> > > > > > > > > > can bring many important performance benefits.
> > > > > > > > > > So we hope to upgrade the version of FRocksDB through this 
> > > > > > > > > > FLIP.
> > > > > > > > > >
> > > > > > > > > > Looking forward to everyone's feedback and suggestions. 
> > > > > > > > > > Thank you!
> > > > > > > > > > --
> > > > > > > > > > Best regards,
> > > > > > > > > > Yue
> > > > > > > > > >
> > > > > >
> > > > 

Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread yue ma
hi Roman,

Thank you very much for your feedback and effort, especially for your help
in releasing Frocksdb products and testing the performance.

Roman Khachatryan  于2024年4月23日周二 12:07写道:

> Hi,
>
> Thanks for writing the proposal and preparing the upgrade.
>
> FRocksDB  definitely needs to be kept in sync with the upstream and the new
> APIs are necessary for faster rescaling.
> We're already using a similar version internally.
>
> I reviewed the FLIP and it looks good to me (disclaimer: I took part in
> some steps of this effort).
>
>
> Regards,
> Roman
>
> On Mon, Apr 22, 2024, 08:11 yue ma  wrote:
>
> > Hi Flink devs,
> >
> > I would like to start a discussion on FLIP-447: Upgrade FRocksDB from
> > 6.20.3 to 8.10.0
> >
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-447%3A+Upgrade+FRocksDB+from+6.20.3++to+8.10.0
> >
> > This FLIP proposes upgrading the version of FRocksDB in the Flink Project
> > from 6.20.3 to 8.10.0.
> > The FLIP mainly introduces the main benefits of upgrading FRocksDB,
> > including the use of IngestDB which can improve Rescaling performance by
> > more than 10 times in certain scenarios, as well as other potential
> > optimization points such as async_io, blob db, and tiered storage.The
> > FLIP also presented test results based on RocksDB 8.10, including
> > StateBenchmark and Nexmark tests.
> > Overall, upgrading FRocksDB may result in a small regression of write
> > performance( which is a very small part of the overall overhead), but it
> > can bring many important performance benefits.
> > So we hope to upgrade the version of FRocksDB through this FLIP.
> >
> > Looking forward to everyone's feedback and suggestions. Thank you!
> > --
> > Best regards,
> > Yue
> >
>


-- 
Best,
Yue


Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread yue ma
Hi , Zakelly Lan

Thank you very much for your feedback and effort, especially for the help
in State Benchmark testing !

Zakelly Lan  于2024年4月24日周三 10:46写道:

> Hi Yue,
>
> Thanks for this proposal!
>
> Given the great improvement we could have, the slight regression in write
> performance is a worthwhile trade-off, particularly as the mem-table
> operations contribute only a minor part to the overall overhead. So +1 for
> this.
>
>
> Best,
> Zakelly
>
> On Tue, Apr 23, 2024 at 12:53 PM Yun Tang  wrote:
>
> > Hi Yue,
> >
> > Thanks for driving this work.
> >
> > It has been three years since last major upgrade of FRocksDB. And it
> would
> > be great improvement of Flink's state-backend with this upgrade. +1 for
> > this work.
> >
> >
> > Best
> > Yun Tang
> > 
> > From: Yanfei Lei 
> > Sent: Tuesday, April 23, 2024 12:50
> > To: dev@flink.apache.org 
> > Subject: Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0
> >
> > Hi Yue & Roman,
> >
> > Thanks for initiating this FLIP and all the efforts for the upgrade.
> >
> > 8.10.0 introduces some new features, making it possible for Flink to
> > implement some new exciting features, and the upgrade also makes
> > FRocksDB easier to maintain, +1 for upgrading.
> >
> > I read the FLIP and have a minor comment, it would be better to add
> > some description about the environment/configuration of the nexmark's
> > result.
> >
> > Roman Khachatryan  于2024年4月23日周二 12:07写道:
> >
> > >
> > > Hi,
> > >
> > > Thanks for writing the proposal and preparing the upgrade.
> > >
> > > FRocksDB  definitely needs to be kept in sync with the upstream and the
> > new
> > > APIs are necessary for faster rescaling.
> > > We're already using a similar version internally.
> > >
> > > I reviewed the FLIP and it looks good to me (disclaimer: I took part in
> > > some steps of this effort).
> > >
> > >
> > > Regards,
> > > Roman
> > >
> > > On Mon, Apr 22, 2024, 08:11 yue ma  wrote:
> > >
> > > > Hi Flink devs,
> > > >
> > > > I would like to start a discussion on FLIP-447: Upgrade FRocksDB from
> > > > 6.20.3 to 8.10.0
> > > >
> > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-447%3A+Upgrade+FRocksDB+from+6.20.3++to+8.10.0
> > > >
> > > > This FLIP proposes upgrading the version of FRocksDB in the Flink
> > Project
> > > > from 6.20.3 to 8.10.0.
> > > > The FLIP mainly introduces the main benefits of upgrading FRocksDB,
> > > > including the use of IngestDB which can improve Rescaling performance
> > by
> > > > more than 10 times in certain scenarios, as well as other potential
> > > > optimization points such as async_io, blob db, and tiered storage.The
> > > > FLIP also presented test results based on RocksDB 8.10, including
> > > > StateBenchmark and Nexmark tests.
> > > > Overall, upgrading FRocksDB may result in a small regression of write
> > > > performance( which is a very small part of the overall overhead), but
> > it
> > > > can bring many important performance benefits.
> > > > So we hope to upgrade the version of FRocksDB through this FLIP.
> > > >
> > > > Looking forward to everyone's feedback and suggestions. Thank you!
> > > > --
> > > > Best regards,
> > > > Yue
> > > >
> >
> >
> >
> > --
> > Best,
> > Yanfei
> >
>


-- 
Best,
Yue


[jira] [Created] (FLINK-35223) Add jobType in JobDetailsInfo related rest api

2024-04-24 Thread Rui Fan (Jira)
Rui Fan created FLINK-35223:
---

 Summary: Add jobType in JobDetailsInfo related rest api
 Key: FLINK-35223
 URL: https://issues.apache.org/jira/browse/FLINK-35223
 Project: Flink
  Issue Type: Sub-task
  Components: Runtime / Web Frontend
Reporter: Rui Fan
Assignee: Rui Fan
 Fix For: 1.20.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (FLINK-35227) Remove execution-mode in ExecutionConfigInfo

2024-04-24 Thread Rui Fan (Jira)
Rui Fan created FLINK-35227:
---

 Summary: Remove execution-mode in ExecutionConfigInfo
 Key: FLINK-35227
 URL: https://issues.apache.org/jira/browse/FLINK-35227
 Project: Flink
  Issue Type: Sub-task
  Components: Runtime / REST
Reporter: Rui Fan
Assignee: Rui Fan
 Fix For: 2.0.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD

2024-04-24 Thread Gyula Fóra
+1 (binding)

Gyula

On Wed, Apr 24, 2024 at 10:07 AM Ferenc Csaky 
wrote:

> +1 (non-binding), looking forward to this!
>
> Best,
> Ferenc
>
>
>
>
> On Wednesday, April 24th, 2024 at 10:03, Mate Czagany 
> wrote:
>
> >
> >
> > Hi everyone,
> >
> > I'd like to start a vote on the FLIP-446: Kubernetes Operator State
> > Snapshot CRD [1]. The discussion thread is here [2].
> >
> > The vote will be open for at least 72 hours unless there is an objection
> or
> > insufficient votes.
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD
> > [2] https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7
> >
> > Regards,
> > Mate
>


Re: [VOTE] FLIP-446: Kubernetes Operator State Snapshot CRD

2024-04-24 Thread Rui Fan
+1(binding)

Best,
Rui

On Wed, Apr 24, 2024 at 4:03 PM Mate Czagany  wrote:

> Hi everyone,
>
> I'd like to start a vote on the FLIP-446: Kubernetes Operator State
> Snapshot CRD [1]. The discussion thread is here [2].
>
> The vote will be open for at least 72 hours unless there is an objection or
> insufficient votes.
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-446%3A+Kubernetes+Operator+State+Snapshot+CRD
> [2] https://lists.apache.org/thread/q5dzjwj0qk34rbg2sczyypfhokxoc3q7
>
> Regards,
> Mate
>