Re: [VOTE] Apache Flink Table Store 0.1.0, release candidate #2

2022-05-07 Thread Jark Wu
Hi all,

I have checked the LICENSE and NOTICE files but found that the uber jar
(flink-table-store-dist)
 doesn't provide an appropriate NOTICE file to list the bundled
dependencies.

I checked the license guideline [1], and it says:
> For Maven artifacts that DO bundle (usually through the
maven-shade-plugin)
> any dependencies the NOTICE file can be found in the
src/main/resources/META-INF
> directory of the respective module. If the module does not have a NOTICE
file yet,
> then add one under src/main/resources/META-INF.
and
> All bundled dependencies must be listed in NOTICE, grouped by license
type,
> using standard maven syntax (groupId:artifactId:version).
> This DOES include ASLv2 dependencies

This DOES include ASLv2 dependencies (for maintainability reasons, as it
simplifies matching of entries in the dependency-tree and NOTICE files)

I'm not sure whether this would block the release or not.
It would be great if @Chesnay Schepler  could help to
check this.

Others look good to me:
- build and compile the source code locally. OK
- checked/verified signatures and hashes, OK
- no missing artifacts in the release staging area. OK
- reviewed the release PR. OK
- ran the quick start in local. looks good

Best,
Jark

[1]: https://cwiki.apache.org/confluence/display/FLINK/Licensing

On Sat, 7 May 2022 at 17:30, Yun Tang  wrote:

> First of all, thanks for Jinsong to drive this. I checked the following:
>
>
>   *   Reviewed the PR of release note.
>   *   Build the package from the scratch: OK
>   *   Check that the README.md file does not have anything unexpected:
> *seems not OK*
>
> I noticed that current README actually does not give correct description
> of how to build and get the expected jar package, and I created a PR [1] to
> resolve this.
>
> [1] https://github.com/apache/flink-table-store/pull/112
>
> Best
> Yun Tang
> 
> From: Yu Li 
> Sent: Saturday, May 7, 2022 17:20
> To: dev 
> Subject: Re: [VOTE] Apache Flink Table Store 0.1.0, release candidate #2
>
> +1 (binding)
>
> - Checked release notes: *OK*
> - Checked sums and signatures: *OK*
> - Checked the jars in the staging repo: *OK*
> - Checked source distribution doesn't include binaries: *OK*
> - Maven clean install from source: *OK*
> - Checked version consistency in pom files: *OK*
> - Went through the quick start: *OK*
>- Minor: the set 'table-store.path' was not fully cleaned up after
> dropping the table and exiting the sql client shell, please check whether
> this is expected behavior.
> - Checked the website updates: *OK*
>   - Note: left some suggestions, please check.
>
> Thanks for driving this release, Jingsong!
>
> Best Regards,
> Yu
>
>
> On Sat, 7 May 2022 at 15:30, Becket Qin  wrote:
>
> > Hi Jingsong,
> >
> > Thanks for driving the very first release of the flink-table-store!
> >
> > +1 (binding) for the RC.
> >
> > I verified the following:
> > - the signature of the artifacts
> > - the source jar does not contain binary files
> > - no accidental distribution of other projects is included in the dist
> jar
> > - went through the quick start
> > - compiled the code locally
> >
> > There were two things that are not blockers but might worth noting:
> > 1. Running the verification requires docker installation. It happens
> that I
> > do not have that installed on my new laptop.
> > 2. In the quick start, after dropping the word_count table, the Flink job
> > that inserts data into the word_count table will keep failover. And this
> > job won't exit after SQL Client exits. It is not a problem of
> > flink-table-store, but it might be non-intuitive for some of the users.
> >
> > I also left some comment in the PR of release announcement.
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
> > On Thu, May 5, 2022 at 5:39 PM Jingsong Li 
> wrote:
> >
> > > Hi, Konstantin and OpenInx,
> > >
> > > I have updated https://github.com/apache/flink-web/pull/531 , you can
> > > take a review~
> > >
> > > Best,
> > > Jingsong
> > >
> > > On Thu, May 5, 2022 at 3:07 PM OpenInx  wrote:
> > > >
> > > > +1 ( non-binding) for the release. I agree with Konstantin that we
> can
> > > add
> > > > more materials about the table-store for the voting.
> > > >
> > > >
> > > >1. Download the source tarball, signature (.asc), and checksum
> > > >(.sha512):   OK
> > > >2. Import gpg keys: download KEYS and run gpg --import
> > > >/path/to/downloaded/KEYS (optional if this hasn’t changed) :  OK
> > > >3. Verify the signature by running: gpg --verify
> > > >flink-table-store-0.1.0-src.tgz.asc:  OK
> > > >4. Verify the checksum by running: shasum -a 256 -c
> > > >flink-table-store-0.1.0-src.tgz.sha512
> > > flink-table-store-0.1.0-src.tgz :  OK
> > > >5. Untar the archive and go into the source directory: tar xzf
> > > >flink-table-store-0.1.0-src.tgz && cd flink-table-store-0.1.0:  OK
> > > >6. Build and test the project: mvn clean package (use Java 8) :
> >  All
> > > >   

Re: [DISCUSS] FLIP-224: Blacklist Mechanism

2022-05-07 Thread Roman Boyko
Hi Lijie!




*a) “Probably storing inside Zookeeper/Configmap might be helpfulhere.”
Can you explain it in detail? I don't fully understand that. In myopinion,
non-active and active are the same, and no special treatment isrequired.*

Sorry this was a misunderstanding from my side. I thought we were talking
about the HA mode (but not about Active and Standalone ResourceManager).
And the original question was - how to handle the blacklisted nodes list at
the moment of leader change? Should we simply forget about them or try to
pre-save that list on the remote storage?

On Sat, 7 May 2022 at 10:51, Yang Wang  wrote:

> Thanks Lijie and ZhuZhu for the explanation.
>
> I just overlooked the "MARK_BLOCKLISTED". For tasks level, it is indeed
> some functionalities the external tools(e.g. kubectl taint) could not
> support.
>
>
> Best,
> Yang
>
> Lijie Wang  于2022年5月6日周五 22:18写道:
>
> > Thanks for your feedback, Jiangang and Martijn.
> >
> > @Jiangang
> >
> >
> > > For auto-detecting, I wonder how to make the strategy and mark a node
> > blocked?
> >
> > In fact, we currently plan to not support auto-detection in this FLIP.
> The
> > part about auto-detection may be continued in a separate FLIP in the
> > future. Some guys have the same concerns as you, and the correctness and
> > necessity of auto-detection may require further discussion in the future.
> >
> > > In session mode, multi jobs can fail on the same bad node and the node
> > should be marked blocked.
> > By design, the blocklist information will be shared among all jobs in a
> > cluster/session. The JM will sync blocklist information with RM.
> >
> > @Martijn
> >
> > > I agree with Yang Wang on this.
> > As Zhu Zhu and I mentioned above, we think the MARK_BLOCKLISTED(Just
> limits
> > the load of the node and does not  kill all the processes on it) is also
> > important, and we think that external systems (*yarn rmadmin or kubectl
> > taint*) cannot support it. So we think it makes sense even only
> *manually*.
> >
> > > I also agree with Chesnay that magical mechanisms are indeed super hard
> > to get right.
> > Yes, as you see, Jiangang(and a few others) have the same concern.
> > However, we currently plan to not support auto-detection in this FLIP,
> and
> > only *manually*. In addition, I'd like to say that the FLIP provides a
> > mechanism to support MARK_BLOCKLISTED and
> > MARK_BLOCKLISTED_AND_EVACUATE_TASKS,
> > the auto-detection may be done by external systems.
> >
> > Best,
> > Lijie
> >
> > Martijn Visser  于2022年5月6日周五 19:04写道:
> >
> > > > If we only support to block nodes manually, then I could not see
> > > the obvious advantages compared with current SRE's approach(via *yarn
> > > rmadmin or kubectl taint*).
> > >
> > > I agree with Yang Wang on this.
> > >
> > > >  To me this sounds yet again like one of those magical mechanisms
> that
> > > will rarely work just right.
> > >
> > > I also agree with Chesnay that magical mechanisms are indeed super hard
> > to
> > > get right.
> > >
> > > Best regards,
> > >
> > > Martijn
> > >
> > > On Fri, 6 May 2022 at 12:03, Jiangang Liu 
> > > wrote:
> > >
> > >> Thanks for the valuable design. The auto-detecting can decrease great
> > work
> > >> for us. We have implemented the similar feature in our inner flink
> > >> version.
> > >> Below is something that I care about:
> > >>
> > >>1. For auto-detecting, I wonder how to make the strategy and mark a
> > >> node
> > >>blocked? Sometimes the blocked node is hard to be detected, for
> > >> example,
> > >>the upper node or the down node will be blocked when network
> > >> unreachable.
> > >>2. I see that the strategy is made in JobMaster side. How about
> > >>implementing the similar logic in resource manager? In session
> mode,
> > >> multi
> > >>jobs can fail on the same bad node and the node should be marked
> > >> blocked.
> > >>If the job makes the strategy, the node may be not marked blocked
> if
> > >> the
> > >>fail times don't exceed the threshold.
> > >>
> > >>
> > >> Zhu Zhu  于2022年5月5日周四 23:35写道:
> > >>
> > >> > Thank you for all your feedback!
> > >> >
> > >> > Besides the answers from Lijie, I'd like to share some of my
> thoughts:
> > >> > 1. Whether to enable automatical blocklist
> > >> > Generally speaking, it is not a goal of FLIP-224.
> > >> > The automatical way should be something built upon the blocklist
> > >> > mechanism and well decoupled. It was designed to be a configurable
> > >> > blocklist strategy, but I think we can further decouple it by
> > >> > introducing a abnormal node detector, as Becket suggested, which
> just
> > >> > uses the blocklist mechanism once bad nodes are detected. However,
> it
> > >> > should be a separate FLIP with further dev discussions and feedback
> > >> > from users. I also agree with Becket that different users have
> > different
> > >> > requirements, and we should listen to them.
> > >> >
> > >> > 2. Is it enough to just take away abnormal nodes 

Re: [DISCUSS] FLIP-222: Support full query lifecycle statements in SQL client

2022-05-07 Thread Jark Wu
Hi Paul,

I think this FLIP has already in a good shape. I just left some additional
thoughts:

*1) the display of savepoint_path*
Could the displayed savepoint_path include the scheme part?
E.g. `hdfs:///flink-savepoints/savepoint-cca7bc-bb1e257f0dab`
IIUC, the scheme part is omitted when it's a local filesystem.
But the behavior would be clearer if including the scheme part in the
design doc.

*2) Please make a decision on multiple options in the FLIP.*
It might give the impression that we will support all the options.

*3) +1 SAVEPOINT and RELEASE SAVEPOINT*
Personally, I also prefer "SAVEPOINT " and "RELEASE SAVEPOINT
"
to "CREATE/DROP SAVEPOINT", as they have been used in mature databases.

*4) +1 SHOW QUERIES*
Btw, the displayed column "address" is a little confusing to me.
At the first glance, I'm not sure what address it is, JM RPC address? JM
REST address? Gateway address?
If this is a link to the job's web UI URL, how about calling it "web_url"
and display in
"http://:" format?
Besides, how about displaying "startTime" or "uptime" as well?

*5) STOP/CANCEL QUERY vs DROP QUERY*
I'm +1 to DROP, because it's more compliant with SQL standard naming, i.e.,
"SHOW/CREATE/DROP".
Separating STOP and CANCEL confuses users a lot what are the differences
between them.
I'm +1 to add the "PURGE" keyword to the DROP QUERY statement, which
indicates to stop query without savepoint.
Note that, PURGE doesn't mean stop with --drain flag. The drain flag will
flush all the registered timers
and windows which could lead to incorrect results when the job is resumed.
I think the drain flag is rarely used
(please correct me if I'm wrong), therefore, I suggest moving this feature
into future work when the needs are clear.

*6) Table API*
I think it makes sense to support the new statements in Table API.
We should try to make the Gateway and CLI simple which just forward
statement to the underlying TableEnvironemnt.
JAR statements are being re-implemented in Table API as well, see
FLIP-214[1].

*7)  and  should be quoted*
All the  and  should be string literal, otherwise
it's hard to parse them.
For example, STOP QUERY ''.

*8) Examples*
Could you add an example that consists of all the statements to show how to
manage the full lifecycle of queries?
Including show queries, create savepoint, remove savepoint, stop query with
a savepoint, and restart query with savepoint.

Best,
Jark

[1]:
https://cwiki.apache.org/confluence/display/FLINK/FLIP-214+Support+Advanced+Function+DDL?src=contextnavpagetreemode


On Fri, 6 May 2022 at 19:13, Martijn Visser 
wrote:

> Hi Paul,
>
> Great that you could find something in the SQL standard! I'll try to read
> the FLIP once more completely next week to see if I have any more concerns.
>
> Best regards,
>
> Martijn
>
> On Fri, 6 May 2022 at 08:21, Paul Lam  wrote:
>
>> I had a look at SQL-2016 that Martijn mentioned, and found that
>> maybe we could follow the transaction savepoint syntax.
>>
>>- SAVEPOINT 
>>- RELEASE SAVEPOINT 
>>
>> These savepoint statements are supported in lots of databases, like
>> Oracle[1], PG[2], MariaDB[3].
>>
>> They’re usually used in the middle of a SQL transaction, so the target
>> would be the current transaction. But if used in Flink SQL session, we
>> need to add a JOB/QUERY id when create a savepoint, thus the syntax
>> would be:
>>
>>- SAVEPOINT  
>>- RELEASE SAVEPOINT 
>>
>> I’m adding it as an alternative in the FLIP.
>>
>> [1]
>> https://docs.oracle.com/cd/B19306_01/server.102/b14200/statements_10001.htm
>> [2] https://www.postgresql.org/docs/current/sql-savepoint.html
>> [3] https://mariadb.com/kb/en/savepoint/
>>
>> Best,
>> Paul Lam
>>
>> 2022年5月4日 16:42,Paul Lam  写道:
>>
>> Hi Shengkai,
>>
>> Thanks a lot for your input!
>>
>> > I just wonder how the users can get the web ui in the application mode.
>> Therefore, it's better we can list the Web UI using the SHOW statement.
>> WDYT?
>>
>> I think it's a valid approach. I'm adding it to the FLIP.
>>
>> > After the investigation, I am fine with the QUERY but the keyword JOB is
>> also okay to me.
>>
>> In addition, CockroachDB has both SHOW QUERIES [1] and SHOW JOBS [2],
>> while the former shows the active running queries and the latter shows
>> the
>> background tasks like schema changes. FYI.
>>
>> WRT the questions:
>>
>> > 1. Could you add some details about the behaviour with the different
>> execution.target, e.g. session, application mode?
>>
>> IMHO, the difference between different `execution.target` is mostly about
>> cluster startup, which has little relation with the proposed statements.
>> These statements rely on the current ClusterClient/JobClient API,
>> which is deployment mode agnostic. Canceling a job in an application
>> cluster is the same as in a session cluster.
>>
>> BTW, application mode is still in the development progress ATM [3].
>>
>> > 2. Considering the SQL Client/Gateway is not limited to submitting the
>> job
>> to the specified cluster, is it able 

Re: [Discuss] Creating an Apache Flink slack workspace

2022-05-07 Thread Xintong Song
Thanks all for the valuable feedback.

It seems most people are overall positive about using Slack for dev
discussions, as long as they are properly reflected back to the MLs.
- We definitely need a code of conduct that clearly specifies what people
should / should not do.
- Contributors pinging well-known reviewers /committers, I think that also
happens now on JIRA / Github. Personally, I'd understand a no-reply as a
"soft no". We may consider to also put that in the cod of conduct.

Concerning using Slack for user QAs, it seem the major concern is that, we
may end up repeatedly answering the same questions from different users,
due to lack of capacity for archiving and searching historical
conversations. TBH, I don't have a good solution for the archivability and
searchability. I investigated some tools like Zapier [1], but none of them
seems suitable for us. However, I'd like to share 2 arguments.
- The purpose of Slack is to make the communication more efficient? By
*efficient*, I mean saving time for both question askers and helpers with
instance messages, file transmissions, even voice / video calls, etc.
(Especially for cases where back and forth is needed, as David mentioned.)
It does not mean questions that do not get enough attentions on MLs are now
guaranteed to be answered immediately. We can probably put that into the
code of conduct, and kindly guide users to first search and initiate
questions on MLs.
- I'd also like to share some experience from the Flink China community. We
have 3 DingTalk groups with totally 25k members (might be less, I didn't do
deduplication), posting hundreds of messages daily. What I'm really excited
about is that, there are way more interactions between users & users than
between users & developers. Users are helping each other, sharing
experiences, sending screenshots / log files / documentations and solving
problems together. We the developers seldom get pinged, if not proactively
joined the conversations. The DingTalk groups are way more active compared
to the user-zh@ ML, which I'd attribute to the improvement of interaction
experiences. Admittedly, there are questions being repeatedly asked &
answered, but TBH I don't think that compares to the benefit of a
self-driven user community. I'd really love to see if we can bring such
success to the global English-speaking community.

Concerning StackOverFlow, it definitely worth more attention from the
community. Thanks for the suggestion / reminder, Piotr & David. I think
Slack and StackOverFlow are probably not mutual exclusive.

Thank you~

Xintong Song


[1] https://zapier.com/



On Sat, May 7, 2022 at 9:50 AM Jingsong Li  wrote:

> Most of the open source communities I know have set up their slack
> channels, such as Apache Iceberg [1], Apache Druid [2], etc.
> So I think slack can be worth trying.
>
> David is right, there are some cases that need to communicate back and
> forth, slack communication will be more effective.
>
> But back to the question, ultimately it's about whether there are
> enough core developers willing to invest time in the slack, to
> discuss, to answer questions, to communicate.
> And whether there will be enough time to reply to the mailing list and
> stackoverflow after we put in the slack (which we need to do).
>
> [1] https://iceberg.apache.org/community/#slack
> [2] https://druid.apache.org/community/
>
> On Fri, May 6, 2022 at 10:06 PM David Anderson 
> wrote:
> >
> > I have mixed feelings about this.
> >
> > I have been rather visible on stack overflow, and as a result I get a
> lot of DMs asking for help. I enjoy helping, but want to do it on a
> platform where the responses can be searched and shared.
> >
> > It is currently the case that good questions on stack overflow
> frequently go unanswered because no one with the necessary expertise takes
> the time to respond. If the Flink community has the collective energy to do
> more user outreach, more involvement on stack overflow would be a good
> place to start. Adding slack as another way for users to request help from
> those who are already actively providing support on the existing
> communication channels might just lead to burnout.
> >
> > On the other hand, there are rather rare, but very interesting cases
> where considerable back and forth is needed to figure out what's going on.
> This can happen, for example, when the requirements are unusual, or when a
> difficult to diagnose bug is involved. In these circumstances, something
> like slack is much better suited than email or stack overflow.
> >
> > David
> >
> > On Fri, May 6, 2022 at 3:04 PM Becket Qin  wrote:
> >>
> >> Thanks for the proposal, Xintong.
> >>
> >> While I share the same concerns as those mentioned in the previous
> discussion thread, admittedly there are benefits of having a slack channel
> as a supplementary way to discuss Flink. The fact that this topic is raised
> once a while indicates lasting interests.
> >>
> >> Personally I am open to having 

[jira] [Created] (FLINK-27542) Add end to end tests for Hive to read external table store files

2022-05-07 Thread Caizhi Weng (Jira)
Caizhi Weng created FLINK-27542:
---

 Summary: Add end to end tests for Hive to read external table 
store files
 Key: FLINK-27542
 URL: https://issues.apache.org/jira/browse/FLINK-27542
 Project: Flink
  Issue Type: Sub-task
  Components: Table Store
Affects Versions: table-store-0.2.0
Reporter: Caizhi Weng
 Fix For: table-store-0.2.0


To ensure that jar produced by flink-table-store-hive module can actually work 
in real Hive system we need to add end to end tests.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Re: [VOTE] Apache Flink Table Store 0.1.0, release candidate #2

2022-05-07 Thread Yun Tang
First of all, thanks for Jinsong to drive this. I checked the following:


  *   Reviewed the PR of release note.
  *   Build the package from the scratch: OK
  *   Check that the README.md file does not have anything unexpected: *seems 
not OK*

I noticed that current README actually does not give correct description of how 
to build and get the expected jar package, and I created a PR [1] to resolve 
this.

[1] https://github.com/apache/flink-table-store/pull/112

Best
Yun Tang

From: Yu Li 
Sent: Saturday, May 7, 2022 17:20
To: dev 
Subject: Re: [VOTE] Apache Flink Table Store 0.1.0, release candidate #2

+1 (binding)

- Checked release notes: *OK*
- Checked sums and signatures: *OK*
- Checked the jars in the staging repo: *OK*
- Checked source distribution doesn't include binaries: *OK*
- Maven clean install from source: *OK*
- Checked version consistency in pom files: *OK*
- Went through the quick start: *OK*
   - Minor: the set 'table-store.path' was not fully cleaned up after
dropping the table and exiting the sql client shell, please check whether
this is expected behavior.
- Checked the website updates: *OK*
  - Note: left some suggestions, please check.

Thanks for driving this release, Jingsong!

Best Regards,
Yu


On Sat, 7 May 2022 at 15:30, Becket Qin  wrote:

> Hi Jingsong,
>
> Thanks for driving the very first release of the flink-table-store!
>
> +1 (binding) for the RC.
>
> I verified the following:
> - the signature of the artifacts
> - the source jar does not contain binary files
> - no accidental distribution of other projects is included in the dist jar
> - went through the quick start
> - compiled the code locally
>
> There were two things that are not blockers but might worth noting:
> 1. Running the verification requires docker installation. It happens that I
> do not have that installed on my new laptop.
> 2. In the quick start, after dropping the word_count table, the Flink job
> that inserts data into the word_count table will keep failover. And this
> job won't exit after SQL Client exits. It is not a problem of
> flink-table-store, but it might be non-intuitive for some of the users.
>
> I also left some comment in the PR of release announcement.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Thu, May 5, 2022 at 5:39 PM Jingsong Li  wrote:
>
> > Hi, Konstantin and OpenInx,
> >
> > I have updated https://github.com/apache/flink-web/pull/531 , you can
> > take a review~
> >
> > Best,
> > Jingsong
> >
> > On Thu, May 5, 2022 at 3:07 PM OpenInx  wrote:
> > >
> > > +1 ( non-binding) for the release. I agree with Konstantin that we can
> > add
> > > more materials about the table-store for the voting.
> > >
> > >
> > >1. Download the source tarball, signature (.asc), and checksum
> > >(.sha512):   OK
> > >2. Import gpg keys: download KEYS and run gpg --import
> > >/path/to/downloaded/KEYS (optional if this hasn’t changed) :  OK
> > >3. Verify the signature by running: gpg --verify
> > >flink-table-store-0.1.0-src.tgz.asc:  OK
> > >4. Verify the checksum by running: shasum -a 256 -c
> > >flink-table-store-0.1.0-src.tgz.sha512
> > flink-table-store-0.1.0-src.tgz :  OK
> > >5. Untar the archive and go into the source directory: tar xzf
> > >flink-table-store-0.1.0-src.tgz && cd flink-table-store-0.1.0:  OK
> > >6. Build and test the project: mvn clean package (use Java 8) :
>  All
> > >unit tests passed, except the e2e tests. Seems we will need to set
> up
> > a
> > >docker environment to run those e2e tests successfully.
> > >7. Verify the apache flink access table store:
> > >
> > > ./bin/sql-client.sh -j
> > >
> >
> /Users/openinx/Downloads/flink-table-store-0.1.0/flink-table-store-dist/target/flink-table-store-dist-0.1.0.jar
> > > embedded shell
> > >
> > > SET 'table-store.path' = '/Users/openinx/test/table-store' ;
> > >
> > > SET 'execution.runtime-mode'='batch';
> > >
> > > SET 'sql-client.execution.result-mode' = 'tableau';
> > >
> > > CREATE TABLE MyTable (
> > >
> > > user_id BIGINT,
> > >
> > > item_id BIGINT,
> > >
> > > behavior STRING,
> > >
> > > dt STRING,
> > >
> > > PRIMARY KEY (dt, user_id) NOT ENFORCED
> > >
> > > ) PARTITIONED BY (dt) WITH (
> > >
> > > 'bucket' = '4'
> > >
> > > );
> > >
> > > INSERT INTO MyTable VALUES
> > >
> > > (100, 200, 'buy', '2022-05-04'),
> > >
> > > (101, 201, 'save', '2022-05-04'),
> > >
> > > (101, 201, 'purchase', '2022-05-04');
> > >
> > >
> > > SELECT * FROM MyTable;
> > >
> > > +-+-+--++
> > >
> > > | user_id | item_id | behavior | dt |
> > >
> > > +-+-+--++
> > >
> > > | 100 | 200 | buy | 2022-05-04 |
> > >
> > > | 101 | 201 | purchase | 2022-05-04 |
> > >
> > > +-+-+--++
> > >
> > > 2 rows in set
> > >
> > > On Thu, May 5, 2022 at 2:39 PM Nicholas Jiang <
> nicholasji...@apache.org>
> > > wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > 

Re: Re: [ANNOUNCE] New Flink PMC member: Yang Wang

2022-05-07 Thread Dian Fu
Congrats Yang!

Regards,
Dian

On Sat, May 7, 2022 at 12:51 PM Jacky Lau  wrote:

> Congrats Yang and well Deserved!
>
> Best,
> Jacky Lau
>
> Yun Gao  于2022年5月7日周六 10:44写道:
>
> > Congratulations Yang!
> >
> > Best,
> > Yun Gao
> >
> >
> >
> >  --Original Mail --
> > Sender:David Morávek 
> > Send Date:Sat May 7 01:05:41 2022
> > Recipients:Dev 
> > Subject:Re: [ANNOUNCE] New Flink PMC member: Yang Wang
> > Nice! Congrats Yang, well deserved! ;)
> >
> > On Fri 6. 5. 2022 at 17:53, Peter Huang 
> > wrote:
> >
> > > Congrats, Yang!
> > >
> > >
> > >
> > > Best Regards
> > > Peter Huang
> > >
> > > On Fri, May 6, 2022 at 8:46 AM Yu Li  wrote:
> > >
> > > > Congrats and welcome, Yang!
> > > >
> > > > Best Regards,
> > > > Yu
> > > >
> > > >
> > > > On Fri, 6 May 2022 at 14:48, Paul Lam  wrote:
> > > >
> > > > > Congrats, Yang! Well Deserved!
> > > > >
> > > > > Best,
> > > > > Paul Lam
> > > > >
> > > > > > 2022年5月6日 14:38,Yun Tang  写道:
> > > > > >
> > > > > > Congratulations, Yang!
> > > > > >
> > > > > > Best
> > > > > > Yun Tang
> > > > > > 
> > > > > > From: Jing Ge 
> > > > > > Sent: Friday, May 6, 2022 14:24
> > > > > > To: dev 
> > > > > > Subject: Re: [ANNOUNCE] New Flink PMC member: Yang Wang
> > > > > >
> > > > > > Congrats Yang and well Deserved!
> > > > > >
> > > > > > Best regards,
> > > > > > Jing
> > > > > >
> > > > > > On Fri, May 6, 2022 at 7:38 AM Lincoln Lee <
> lincoln.8...@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > >> Congratulations Yang!
> > > > > >>
> > > > > >> Best,
> > > > > >> Lincoln Lee
> > > > > >>
> > > > > >>
> > > > > >> Őrhidi Mátyás  于2022年5月6日周五 12:46写道:
> > > > > >>
> > > > > >>> Congrats Yang! Well deserved!
> > > > > >>> Best,
> > > > > >>> Matyas
> > > > > >>>
> > > > > >>> On Fri, May 6, 2022 at 5:30 AM huweihua <
> huweihua@gmail.com>
> > > > > wrote:
> > > > > >>>
> > > > >  Congratulations Yang!
> > > > > 
> > > > >  Best,
> > > > >  Weihua
> > > > > 
> > > > > 
> > > > > >>>
> > > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>


Re: [VOTE] Apache Flink Table Store 0.1.0, release candidate #2

2022-05-07 Thread Yu Li
+1 (binding)

- Checked release notes: *OK*
- Checked sums and signatures: *OK*
- Checked the jars in the staging repo: *OK*
- Checked source distribution doesn't include binaries: *OK*
- Maven clean install from source: *OK*
- Checked version consistency in pom files: *OK*
- Went through the quick start: *OK*
   - Minor: the set 'table-store.path' was not fully cleaned up after
dropping the table and exiting the sql client shell, please check whether
this is expected behavior.
- Checked the website updates: *OK*
  - Note: left some suggestions, please check.

Thanks for driving this release, Jingsong!

Best Regards,
Yu


On Sat, 7 May 2022 at 15:30, Becket Qin  wrote:

> Hi Jingsong,
>
> Thanks for driving the very first release of the flink-table-store!
>
> +1 (binding) for the RC.
>
> I verified the following:
> - the signature of the artifacts
> - the source jar does not contain binary files
> - no accidental distribution of other projects is included in the dist jar
> - went through the quick start
> - compiled the code locally
>
> There were two things that are not blockers but might worth noting:
> 1. Running the verification requires docker installation. It happens that I
> do not have that installed on my new laptop.
> 2. In the quick start, after dropping the word_count table, the Flink job
> that inserts data into the word_count table will keep failover. And this
> job won't exit after SQL Client exits. It is not a problem of
> flink-table-store, but it might be non-intuitive for some of the users.
>
> I also left some comment in the PR of release announcement.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Thu, May 5, 2022 at 5:39 PM Jingsong Li  wrote:
>
> > Hi, Konstantin and OpenInx,
> >
> > I have updated https://github.com/apache/flink-web/pull/531 , you can
> > take a review~
> >
> > Best,
> > Jingsong
> >
> > On Thu, May 5, 2022 at 3:07 PM OpenInx  wrote:
> > >
> > > +1 ( non-binding) for the release. I agree with Konstantin that we can
> > add
> > > more materials about the table-store for the voting.
> > >
> > >
> > >1. Download the source tarball, signature (.asc), and checksum
> > >(.sha512):   OK
> > >2. Import gpg keys: download KEYS and run gpg --import
> > >/path/to/downloaded/KEYS (optional if this hasn’t changed) :  OK
> > >3. Verify the signature by running: gpg --verify
> > >flink-table-store-0.1.0-src.tgz.asc:  OK
> > >4. Verify the checksum by running: shasum -a 256 -c
> > >flink-table-store-0.1.0-src.tgz.sha512
> > flink-table-store-0.1.0-src.tgz :  OK
> > >5. Untar the archive and go into the source directory: tar xzf
> > >flink-table-store-0.1.0-src.tgz && cd flink-table-store-0.1.0:  OK
> > >6. Build and test the project: mvn clean package (use Java 8) :
>  All
> > >unit tests passed, except the e2e tests. Seems we will need to set
> up
> > a
> > >docker environment to run those e2e tests successfully.
> > >7. Verify the apache flink access table store:
> > >
> > > ./bin/sql-client.sh -j
> > >
> >
> /Users/openinx/Downloads/flink-table-store-0.1.0/flink-table-store-dist/target/flink-table-store-dist-0.1.0.jar
> > > embedded shell
> > >
> > > SET 'table-store.path' = '/Users/openinx/test/table-store' ;
> > >
> > > SET 'execution.runtime-mode'='batch';
> > >
> > > SET 'sql-client.execution.result-mode' = 'tableau';
> > >
> > > CREATE TABLE MyTable (
> > >
> > > user_id BIGINT,
> > >
> > > item_id BIGINT,
> > >
> > > behavior STRING,
> > >
> > > dt STRING,
> > >
> > > PRIMARY KEY (dt, user_id) NOT ENFORCED
> > >
> > > ) PARTITIONED BY (dt) WITH (
> > >
> > > 'bucket' = '4'
> > >
> > > );
> > >
> > > INSERT INTO MyTable VALUES
> > >
> > > (100, 200, 'buy', '2022-05-04'),
> > >
> > > (101, 201, 'save', '2022-05-04'),
> > >
> > > (101, 201, 'purchase', '2022-05-04');
> > >
> > >
> > > SELECT * FROM MyTable;
> > >
> > > +-+-+--++
> > >
> > > | user_id | item_id | behavior | dt |
> > >
> > > +-+-+--++
> > >
> > > | 100 | 200 | buy | 2022-05-04 |
> > >
> > > | 101 | 201 | purchase | 2022-05-04 |
> > >
> > > +-+-+--++
> > >
> > > 2 rows in set
> > >
> > > On Thu, May 5, 2022 at 2:39 PM Nicholas Jiang <
> nicholasji...@apache.org>
> > > wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > +1 for the release (non-binding).
> > > >
> > > > - Built and compiled source codes [PASSED]
> > > > - Went through quick start guide [PASSED]
> > > > - Checked README.md [PASSED]
> > > > - Checked that use the table store jar to build query table
> application
> > > > [PASSED]
> > > >
> > > > Best regards,
> > > >
> > > > Nicholas Jiang
> > > >
> > > > On 2022/04/29 02:24:09 Jingsong Li wrote:
> > > > > Hi everyone,
> > > > >
> > > > > Please review and vote on the release candidate #2 for the version
> > 0.1.0
> > > > of
> > > > > Apache Flink Table Store, as follows:
> > > > >
> > > > > [ ] +1, Approve the release
> > > > > [ ] -1, Do 

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

2022-05-07 Thread Dian Fu
Hi Nicholas,

Thanks a lot for bringing up this discussion. If I recall it correctly,
this feature has been requested many times by the users and is among one of
the most requested features in CEP. So big +1 to this feature overall.

Regarding the API, the name `partialWithin` sounds a little weird. Is it
possible to find a name which is more intuitive? Other possible solutions:
- Reuse the existing `Pattern.within` method and change its semantic to the
maximum time interval between patterns. Currently `Pattern.within` is used
to define the maximum time interval between the first event and the last
event. However, the Pattern object represents only one node in a pattern
sequence and so it doesn't make much sense to define the maximum time
interval between the first event and the last event on the Pattern object,
e.g. we could move it to  `PatternStreamBuilder`. However, if we choose
this option, we'd better consider how to keep backward compatibility.
- Introduce a series of methods when appending a new pattern to the
existing one, e.g. `Pattern.followedBy(Pattern group, Time
timeInterval)`. As timeInterval is a property between patterns and so it
makes sense to define this property when appending a new pattern. However,
the drawback is that we need to introduce a series of methods instead of
only one method.

We need also to make the semantic clear in a few corner cases, e.g.
- What's the semantic of `A.followedBy(B).times(3).partialWithin(1 min)`?
Doesn't it mean that all three B events should occur in 1 minute or only
the first B event should occur in 1 minute?
- What's the semantic of
`A.followedBy(GroupPattern.begin("B").followedBy("C")).partialWithin(1
min)``? Doesn't it mean that B and C should occur after A in 1 minute?

Besides, this FLIP only describes how the newly introduced API will be
used, however, it lacks details about how you will implement it. It doesn't
need to be very detailed, however, you should describe the basic ideas
behind it, e.g. how will you support A.notFollowedBy(B).partialWithin(1
min)? It could make sure that you have considered it thoroughly and also
makes others confident that this feature could be implemented in a clean
way.

Regards,
Dian



On Fri, May 6, 2022 at 7:32 PM yue ma  wrote:

> hi Nicholas,
>
> Thanks for bringing this discussion, we also think it's a useful feature.
> Some fine-grained timeout pattern matching  can be implemented in CEP which
> makes Flink CEP more powerful
>
> Nicholas  于2022年5月5日周四 14:28写道:
>
> > Hi everyone,
> >
> >
> >
> >
> > Pattern#withIn interface in CEP defines the maximum time interval in
> which
> > a matching pattern has to be completed in order to be considered valid,
> > which interval corresponds to the maximum time gap between first and the
> > last event. The interval representing the maximum time gap between events
> > is required to define in the scenario like purchasing good within a
> maximum
> > of 5 minutes after browsing.
> >
> >
> >
> >
> > I would like to start a discussion about FLIP-228[1], in which within
> > between events is proposed in Pattern to support the definition of the
> > maximum time interval in which a completed partial matching pattern is
> > considered valid, which interval represents the maximum time gap between
> > events for partial matching Pattern.
> >
> >
> >
> >
> > Hence we propose the Pattern#partialWithin interface to define the
> maximum
> > time interval in which a completed partial matching pattern is considered
> > valid. Please take a look at the FLIP page [1] to get more details. Any
> > feedback about the FLIP-228 would be appreciated!
> >
> >
> >
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-228%3A+Support+Within+between+events+in+CEP+Pattern
> >
> >
> >
> >
> > Best regards,
> >
> > Nicholas Jiang
>


[jira] [Created] (FLINK-27541) Support distribute by For FlinkSql

2022-05-07 Thread Aiden (Jira)
Aiden created FLINK-27541:
-

 Summary: Support distribute by For FlinkSql
 Key: FLINK-27541
 URL: https://issues.apache.org/jira/browse/FLINK-27541
 Project: Flink
  Issue Type: Improvement
  Components: Table SQL / Planner
Reporter: Aiden


Now we cann't add a shuffle-operation in a sql-job.
Sometimes , for example,  I have a kafka-source(three partitions) with 
parallelism three. And then I have a lookup-join function, I want process the 
data distribute by id so that the data can split into thre parallelism evenly 
(The source maybe slant seriously).  In DataStream API i can do it with 
keyby(),  but it's so sad that i can  do nothing when i use a sql;



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Re: [DISCUSS] FLIP-91: Support SQL Client Gateway

2022-05-07 Thread Jingsong Li
Hi Shengkai, thanks for your reply.

> REST API is the user interface. The REST API transforms the request to the
invocation of the SQLGatewayService that is the one doing the work. We
split the Gateway into the SQLGatewayService and Endpoint(REST API) and its
benefit is that all the Endpoints share the same SQLGatewayService.

Finally, I got the point of SQLGatewayService, It is just for
`Pluggable Endpoint Discovery`.
I suggest you reorganize the FLIP, what Pluggable Endpoint Discovery
is, and how users to add new Endpoint, before introducing
SQLGatewayService.

Then I have some doubts about the name SQLGatewayService, I think
Service is seriously misleading, it is just an implementation class
and not a Service.

What about just `SQLGateway`?

Best,
Jingsong

On Sat, May 7, 2022 at 4:03 PM Shengkai Fang  wrote:
>
> Hi Martijn.
>
> It seems we have reached consensus to support the Gateway inside the Flink
> code base.
>
> Hi, Echo.
>
> Thanks for your interest.
>
> > whether flink-sql-gateway should be retained in the Flink project.
>
> I think the discussion above is clear. It is the essential tool to provide
> out-of-box experience for users.
>
> > For stream processing, what's the point of getting the result? Is it just
> for debugging and how to unify with batch processing
>
> At the client side, when the OperationStaus is FINISHED, the client is able
> to fetch the results from the Gateway. It is unified with the batch
> processing now.
>
> > For batch processing, does the gateway need to cache all fetch results?
>
> No. In the Gateway, we will only buffer partial data and wait for the
> client to consume. If the client takes away the buffered data, the Gateway
> will clear the buffer and notify the fetcher thread starts to work until
> the buffer is full again. The mechanism behind is much like the
> producer-consumer model[1].
>
> [1] https://en.wikipedia.org/wiki/Producer%E2%80%93consumer_problem
>
> > Whether executing query and fetch results should be synchronous or
> asynchronous?
>
> Do you mean the executeStatement response should also contain the result? I
> don't think using the asynchronous API will cause performance regression.
> In most cases, the network latency is about 100ms or lower. You can ping
> www.baidu.com or www.google.com to test the latency.
>
> > When executing a query in flink-sql-client, I often find error logs of
> FlinkJobNotFoundException. Should this be optimized?
>
> It's related to the current client implementation. I think you can open a
> jira ticket, add more details and we can discuss the problem in the ticket.
> In the FLIP-91, the Gateway can store the log per Operation. It may solve
> your problems.
>
>
> Best,
> Shengkai


Re: [DISCUSS] FLIP-91: Support SQL Client Gateway

2022-05-07 Thread Shengkai Fang
Hi Martijn.

It seems we have reached consensus to support the Gateway inside the Flink
code base.

Hi, Echo.

Thanks for your interest.

> whether flink-sql-gateway should be retained in the Flink project.

I think the discussion above is clear. It is the essential tool to provide
out-of-box experience for users.

> For stream processing, what's the point of getting the result? Is it just
for debugging and how to unify with batch processing

At the client side, when the OperationStaus is FINISHED, the client is able
to fetch the results from the Gateway. It is unified with the batch
processing now.

> For batch processing, does the gateway need to cache all fetch results?

No. In the Gateway, we will only buffer partial data and wait for the
client to consume. If the client takes away the buffered data, the Gateway
will clear the buffer and notify the fetcher thread starts to work until
the buffer is full again. The mechanism behind is much like the
producer-consumer model[1].

[1] https://en.wikipedia.org/wiki/Producer%E2%80%93consumer_problem

> Whether executing query and fetch results should be synchronous or
asynchronous?

Do you mean the executeStatement response should also contain the result? I
don't think using the asynchronous API will cause performance regression.
In most cases, the network latency is about 100ms or lower. You can ping
www.baidu.com or www.google.com to test the latency.

> When executing a query in flink-sql-client, I often find error logs of
FlinkJobNotFoundException. Should this be optimized?

It's related to the current client implementation. I think you can open a
jira ticket, add more details and we can discuss the problem in the ticket.
In the FLIP-91, the Gateway can store the log per Operation. It may solve
your problems.


Best,
Shengkai


RE: Re: How to support C#/dotNet ?

2022-05-07 Thread Bruce Tian
 I mean former purpose , I want to offer API for C# on Apache Flink   same as 
[4].  

On 2022/05/07 02:40:33 Yun Tang wrote:
> Hi Tian,
> 
> I am not sure whether you want to get such SDK or just want to refer some 
> other existing repos to try to implement your own.
> For the former purpose, you can find several non-official projects in 
> github[1], which has not been tested widely.
> If you just want to refer to some other existing repos, I could share some 
> experience here. Several years ago, I participated in an open-source project 
> Mobius[2], which offers API for C# on Apache Spark.
> The kernel idea is to introduce adapter from C# to call java [3]. Though this 
> project has been deprecated, you can refer to a still active project [4] for 
> more information to see how to support C# API in Flink.
> 
> 
> [1] https://github.com/HEF-Sharp/HEF.Flink
> [2] https://github.com/microsoft/Mobius
> [3] https://github.com/microsoft/Mobius/tree/master/csharp/Adapter
> [4] https://github.com/dotnet/spark
> 
> Best
> Yun Tang
> 
> 
> From: Cristian Constantinescu 
> Sent: Saturday, May 7, 2022 9:58
> To: dev@flink.apache.org 
> Subject: Re: How to support C#/dotNet ?
> 
> Not exactly what you asked for, but... Have a look at the Apache Beam
> project.
> 
> Their goal is quite literally any language on any runner (including Flink).
> 
> They recently released an initial version in JavaScript "for fun" and it
> took about two weeks to develop. To give you an idea of the effort it takes.
> 
> On Fri., May 6, 2022, 21:53 Bruce Tian,  wrote:
> 
> > Hi folks
> >
> > Flink is a very  awesome project ! ,I want to develop C# SDK which is
> > similar Java  python SDK . Is there  any protocol between flink with client
> > SDK ?
> >
> >
> 

RE: Re: How to support C#/dotNet ?

2022-05-07 Thread Bruce Tian
> Have a look at the Apache Beam
I will look that,thanks

On 2022/05/07 01:58:07 Cristian Constantinescu wrote:
> Not exactly what you asked for, but... Have a look at the Apache Beam
> project.
> 
> Their goal is quite literally any language on any runner (including Flink).
> 
> They recently released an initial version in JavaScript "for fun" and it
> took about two weeks to develop. To give you an idea of the effort it takes.
> 
> On Fri., May 6, 2022, 21:53 Bruce Tian,  wrote:
> 
> > Hi folks
> >
> > Flink is a very  awesome project ! ,I want to develop C# SDK which is
> > similar Java  python SDK . Is there  any protocol between flink with client
> > SDK ?
> >
> >
> 

Re: Edit Permissions for Flink Connector Template

2022-05-07 Thread Xintong Song
Thanks for the info, Martijn.

Jeremy, the permission is granted. Please have a try.

Thank you~

Xintong Song



On Sat, May 7, 2022 at 2:22 PM Martijn Visser 
wrote:

> Hi all,
>
> For context, I've had an offline discussion with Jeremy on what's needed to
> propose a new Flink connector. That's why there is a need to create a FLIP.
> His Confluence user name is jeremyber
>
> Best regards,
>
> Martijn
>
> Op za 7 mei 2022 om 04:18 schreef Xintong Song 
>
> > Hi Jeremy,
> >
> > Could you add a link to the previous discussion?
> >
> > And you would need to first create an account at
> > https://cwiki.apache.org/confluence .
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Fri, May 6, 2022 at 9:26 PM Ber, Jeremy 
> > wrote:
> >
> > > Hello,
> > >
> > > I require Confluence Edit Permissions in order to create a Flink
> > Connector
> > > Template page as discussed via e-mail.
> > >
> > > Jeremy
> > >
> >
> --
> ---
> Martijn Visser
> https://twitter.com/MartijnVisser82
> https://github.com/MartijnVisser
>


[jira] [Created] (FLINK-27540) Let FileStoreSource accept pre-planned manifest entries

2022-05-07 Thread Jane Chan (Jira)
Jane Chan created FLINK-27540:
-

 Summary: Let FileStoreSource accept pre-planned manifest entries
 Key: FLINK-27540
 URL: https://issues.apache.org/jira/browse/FLINK-27540
 Project: Flink
  Issue Type: Sub-task
  Components: Table Store
Affects Versions: table-store-0.2.0
Reporter: Jane Chan
 Fix For: table-store-0.2.0


When manual compaction is triggered, the manifest entries are collected at the 
planning phase already(to accelerate the compaction). The source does not need 
to scan and plan again during the runtime.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Created] (FLINK-27539) support consuming update and delete changes In Windowing TVFs

2022-05-07 Thread hjw (Jira)
hjw created FLINK-27539:
---

 Summary: support consuming update and delete changes In Windowing 
TVFs
 Key: FLINK-27539
 URL: https://issues.apache.org/jira/browse/FLINK-27539
 Project: Flink
  Issue Type: Improvement
  Components: Table SQL / API
Affects Versions: 1.15.0
Reporter: hjw


custom_kafka is a cdc table

sql:
{code:java}

select DATE_FORMAT(window_end,'-MM-dd') as date_str,sum(money) as total,name
from TABLE(CUMULATE(TABLE custom_kafka,descriptor(createtime),interval '1' 
MINUTES,interval '1' DAY ))
where status='1'
group by name,window_start,window_end;
{code}

Error

org.apache.flink.table.api.TableException: StreamPhysicalWindowAggregate 
doesn't support consuming update and delete changes which is produced by node 
TableSourceScan(table=[[default_catalog, default_database,custom_kafka]], 
fields=[name, money, status,createtime,operation_ts])

But I found Group Window Aggregation is works when use cdc table

select DATE_FORMAT(TUMBLE_END(createtime,interval '10' MINUTES),'-MM-dd') 
as date_str,sum(money) as total,name
from custom_kafka
where status='1'
group by name,TUMBLE(createtime,interval '10' MINUTES)



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Re: [VOTE] Apache Flink Table Store 0.1.0, release candidate #2

2022-05-07 Thread Becket Qin
Hi Jingsong,

Thanks for driving the very first release of the flink-table-store!

+1 (binding) for the RC.

I verified the following:
- the signature of the artifacts
- the source jar does not contain binary files
- no accidental distribution of other projects is included in the dist jar
- went through the quick start
- compiled the code locally

There were two things that are not blockers but might worth noting:
1. Running the verification requires docker installation. It happens that I
do not have that installed on my new laptop.
2. In the quick start, after dropping the word_count table, the Flink job
that inserts data into the word_count table will keep failover. And this
job won't exit after SQL Client exits. It is not a problem of
flink-table-store, but it might be non-intuitive for some of the users.

I also left some comment in the PR of release announcement.

Thanks,

Jiangjie (Becket) Qin

On Thu, May 5, 2022 at 5:39 PM Jingsong Li  wrote:

> Hi, Konstantin and OpenInx,
>
> I have updated https://github.com/apache/flink-web/pull/531 , you can
> take a review~
>
> Best,
> Jingsong
>
> On Thu, May 5, 2022 at 3:07 PM OpenInx  wrote:
> >
> > +1 ( non-binding) for the release. I agree with Konstantin that we can
> add
> > more materials about the table-store for the voting.
> >
> >
> >1. Download the source tarball, signature (.asc), and checksum
> >(.sha512):   OK
> >2. Import gpg keys: download KEYS and run gpg --import
> >/path/to/downloaded/KEYS (optional if this hasn’t changed) :  OK
> >3. Verify the signature by running: gpg --verify
> >flink-table-store-0.1.0-src.tgz.asc:  OK
> >4. Verify the checksum by running: shasum -a 256 -c
> >flink-table-store-0.1.0-src.tgz.sha512
> flink-table-store-0.1.0-src.tgz :  OK
> >5. Untar the archive and go into the source directory: tar xzf
> >flink-table-store-0.1.0-src.tgz && cd flink-table-store-0.1.0:  OK
> >6. Build and test the project: mvn clean package (use Java 8) :   All
> >unit tests passed, except the e2e tests. Seems we will need to set up
> a
> >docker environment to run those e2e tests successfully.
> >7. Verify the apache flink access table store:
> >
> > ./bin/sql-client.sh -j
> >
> /Users/openinx/Downloads/flink-table-store-0.1.0/flink-table-store-dist/target/flink-table-store-dist-0.1.0.jar
> > embedded shell
> >
> > SET 'table-store.path' = '/Users/openinx/test/table-store' ;
> >
> > SET 'execution.runtime-mode'='batch';
> >
> > SET 'sql-client.execution.result-mode' = 'tableau';
> >
> > CREATE TABLE MyTable (
> >
> > user_id BIGINT,
> >
> > item_id BIGINT,
> >
> > behavior STRING,
> >
> > dt STRING,
> >
> > PRIMARY KEY (dt, user_id) NOT ENFORCED
> >
> > ) PARTITIONED BY (dt) WITH (
> >
> > 'bucket' = '4'
> >
> > );
> >
> > INSERT INTO MyTable VALUES
> >
> > (100, 200, 'buy', '2022-05-04'),
> >
> > (101, 201, 'save', '2022-05-04'),
> >
> > (101, 201, 'purchase', '2022-05-04');
> >
> >
> > SELECT * FROM MyTable;
> >
> > +-+-+--++
> >
> > | user_id | item_id | behavior | dt |
> >
> > +-+-+--++
> >
> > | 100 | 200 | buy | 2022-05-04 |
> >
> > | 101 | 201 | purchase | 2022-05-04 |
> >
> > +-+-+--++
> >
> > 2 rows in set
> >
> > On Thu, May 5, 2022 at 2:39 PM Nicholas Jiang 
> > wrote:
> >
> > > Hi everyone,
> > >
> > > +1 for the release (non-binding).
> > >
> > > - Built and compiled source codes [PASSED]
> > > - Went through quick start guide [PASSED]
> > > - Checked README.md [PASSED]
> > > - Checked that use the table store jar to build query table application
> > > [PASSED]
> > >
> > > Best regards,
> > >
> > > Nicholas Jiang
> > >
> > > On 2022/04/29 02:24:09 Jingsong Li wrote:
> > > > Hi everyone,
> > > >
> > > > Please review and vote on the release candidate #2 for the version
> 0.1.0
> > > of
> > > > Apache Flink Table Store, as follows:
> > > >
> > > > [ ] +1, Approve the release
> > > > [ ] -1, Do not approve the release (please provide specific comments)
> > > >
> > > > **Release Overview**
> > > >
> > > > As an overview, the release consists of the following:
> > > > a) Table Store canonical source distribution, to be deployed to the
> > > > release repository at dist.apache.org
> > > > b) Maven artifacts to be deployed to the Maven Central Repository
> > > >
> > > > **Staging Areas to Review**
> > > >
> > > > The staging areas containing the above mentioned artifacts are as
> > > follows,
> > > > for your review:
> > > > * All artifacts for a) and b) can be found in the corresponding dev
> > > > repository at dist.apache.org [2]
> > > > * All artifacts for c) can be found at the Apache Nexus Repository
> [3]
> > > > * Pre Bundled Binaries Jar can work fine with quick start [4][5]
> > > >
> > > > All artifacts are signed with the key
> > > > 2C2B6A653B07086B65E4369F7C76245E0A318150 [6]
> > > >
> > > > Other links for your review:
> > > > * JIRA release notes [7]
> > > 

Re: [DISCUSS] DockerHub repository maintainers

2022-05-07 Thread Márton Balassi
Hi team,

I volunteer for the flink-kubernetes-operator repo.

On Fri, May 6, 2022 at 1:42 PM Xintong Song  wrote:

> @Till,
>
> Thanks for volunteering.
>
> @Konstantin,
>
> From my experience, the effort that requires DockerHub access in the main
> project release process is quite limited. I helped Yun Gao on releasing the
> 1.15.0 images, and what I did was just check out the `flink-docker` repo
> and run the release script, that's it. If all the sub-projects are as easy
> as the main project, then it's probably ok that only a small group of
> people have access. Concerning the redundancy, if a maintainer from a
> sub-project is temporarily unreachable, I believe the other maintainers
> would be glad to help.
>
> It would of course be good to have more seats. I just haven't come up with
> good reasons to persuade the INFRA folks. What's your suggestions?
>
>
> Thank you~
>
> Xintong Song
>
>
>
> On Fri, May 6, 2022 at 6:38 PM Konstantin Knauf  wrote:
>
> > Hi Xintong,
> >
> > it is a pity that we can only have 5 maintainers. Every (patch) release
> of
> > flink, flink-statefun, the flink-kubernetes-operator requires a
> maintainer
> > to publish the image then, if I am not mistaken. As its mostly different
> > groups managing the sub-projects, this is quite the bottleneck. If we
> give
> > one seat to flink-statefun maintainers, one to the
> > flink-kubernetes-operator maintainers, this leaves three seats for Apache
> > Flink core, and there is no redundancy for the other projects. When I
> > managed the last two patch releases, the DockerHub access was also the
> > biggest hurdle. Maybe we can talk to the INFRA people again. We can
> > certainly reduce it, but 5 is very little.
> >
> > Cheers,
> >
> > Konstantin
> >
> >
> >
> >
> >
> >
> > Am Fr., 6. Mai 2022 um 09:00 Uhr schrieb Till Rohrmann <
> > trohrm...@apache.org
> > >:
> >
> > > Hi everyone,
> > >
> > > thanks for starting this discussion Xintong. I would volunteer as a
> > > maintainer of the flink-statefun Docker repository if you need one.
> > >
> > > Cheers,
> > > Till
> > >
> > > On Fri, May 6, 2022 at 6:22 AM Xintong Song 
> > wrote:
> > >
> > >> It seems to me we at least don't have a consensus on dropping the use
> of
> > >> apache namespace, which means we need to decide on a list of
> maintainers
> > >> anyway. So maybe we can get the discussion back to the maintainers. We
> > may
> > >> continue the official-image vs. apache-namespace in a separate thread
> if
> > >> necessary.
> > >>
> > >> As mentioned previously, we need to reduce the number of maintainers
> > from
> > >> 20 to 5, as required by INFRA. Jingsong and I would like to volunteer
> > as 2
> > >> of the 5, and we would like to learn who else wants to join us. Of
> > course
> > >> the list of maintainers can be modified later.
> > >>
> > >> *This also means the current maintainers may be removed from the
> list.*
> > >> Please let us know if you still need that privilege. CC-ed all the
> > current
> > >> maintainers for attention.
> > >>
> > >> Thank you~
> > >>
> > >> Xintong Song
> > >>
> > >>
> > >>
> > >> On Wed, May 4, 2022 at 3:14 PM Chesnay Schepler 
> > >> wrote:
> > >>
> > >> > One advantage is that the images are periodically rebuilt to get
> > >> > security fixes.
> > >> >
> > >> > The operator is a different story anyway because it is AFAIK only
> > >> > supposed to be used via docker
> > >> > (i.e., no standalone mode), which alleviates concerns about keeping
> > the
> > >> > logic within the image
> > >> > to a minimum (which bit us in the past on the flink side).
> > >> >
> > >> > On 03/05/2022 16:09, Yang Wang wrote:
> > >> > > The flink-kubernetes-operator project is only published
> > >> > > via apache/flink-kubernetes-operator on docker hub and github
> > >> packages.
> > >> > > We do not find the obvious advantages by using docker hub official
> > >> > images.
> > >> > >
> > >> > > Best,
> > >> > > Yang
> > >> > >
> > >> > > Xintong Song  于2022年4月28日周四 19:27写道:
> > >> > >
> > >> > >> I agree with you that doing QA for the image after the release
> has
> > >> been
> > >> > >> finalized doesn't feel right. IIUR, that is mostly because
> official
> > >> > image
> > >> > >> PR needs 1) the binary release being deployed and propagated and
> 2)
> > >> the
> > >> > >> corresponding git commit being specified. I'm not completely sure
> > >> about
> > >> > >> this. Maybe we can improve the process by investigating more
> about
> > >> the
> > >> > >> feasibility of pre-verifying an official image PR before
> finalizing
> > >> the
> > >> > >> release. It's definitely a good thing to do if possible.
> > >> > >>
> > >> > >> I also agree that QA from DockerHub folks is valuable to us.
> > >> > >>
> > >> > >> I'm not against publishing official-images, and I'm not against
> > >> working
> > >> > >> closely with the DockerHub folks to improve the process of
> > delivering
> > >> > the
> > >> > >> official image. However, I don't think these should become
> reasons
> 

Re: Edit Permissions for Flink Connector Template

2022-05-07 Thread Martijn Visser
Hi all,

For context, I've had an offline discussion with Jeremy on what's needed to
propose a new Flink connector. That's why there is a need to create a FLIP.
His Confluence user name is jeremyber

Best regards,

Martijn

Op za 7 mei 2022 om 04:18 schreef Xintong Song 

> Hi Jeremy,
>
> Could you add a link to the previous discussion?
>
> And you would need to first create an account at
> https://cwiki.apache.org/confluence .
>
> Thank you~
>
> Xintong Song
>
>
>
> On Fri, May 6, 2022 at 9:26 PM Ber, Jeremy 
> wrote:
>
> > Hello,
> >
> > I require Confluence Edit Permissions in order to create a Flink
> Connector
> > Template page as discussed via e-mail.
> >
> > Jeremy
> >
>
-- 
---
Martijn Visser
https://twitter.com/MartijnVisser82
https://github.com/MartijnVisser