+1 with a separate repo and +1 with the flink-storage name
On Fri, Jan 7, 2022 at 8:40 AM Jingsong Li wrote:
> Hi everyone,
>
> Vote for create a separate sub project for FLIP-188 thread is here:
> https://lists.apache.org/thread/wzzhr27cvrh6w107bn464m1m1ycfll1z
>
> Best,
> Jingsong
>
>
> On Fri
Hi everyone,
Vote for create a separate sub project for FLIP-188 thread is here:
https://lists.apache.org/thread/wzzhr27cvrh6w107bn464m1m1ycfll1z
Best,
Jingsong
On Fri, Jan 7, 2022 at 3:30 PM Jingsong Li wrote:
>
> Hi Timo,
>
> I think we can consider exposing to DataStream users in the future
Hi Timo,
I think we can consider exposing to DataStream users in the future, if
the API definition is clear after.
I am fine with `flink-table-store` too.
But I tend to prefer shorter and clearer name:
`flink-store`.
I think I can create a separate thread to vote.
Looking forward to your thought
+1 for a separate repository. And also +1 for finding a good name.
`flink-warehouse` would be definitely a good marketing name but I agree
that we should not start marketing for code bases. Are we planning to
make this storage also available to DataStream API users? If not, I
would also vote f
Thanks Till for your suggestions.
Personally, I like flink-warehouse, this is what we want to convey to
the user, but it indicates a bit too much scope.
How about just calling it flink-store?
Simply to convey an impression: this is flink's store project,
providing a built-in store for the flink c
Hi Jingsong,
I think that developing flink-dynamic-storage as a separate sub project is
a very good idea since it allows us to move a lot faster and decouple
releases from Flink. Hence big +1.
Do we want to name it flink-dynamic-storage or shall we use a more
descriptive name? dynamic-storage sou
Hi Jingsong,
That sounds promising! +1 from my side to continue development under
flink-dynamic-storage as a Flink subproject. I think having a more in-depth
interface will benefit everyone.
Best regards,
Martijn
On Tue, 28 Dec 2021 at 04:23, Jingsong Li wrote:
> Hi all,
>
> After some experi
Hi all,
After some experimentation, we felt no problem putting the dynamic
storage outside of flink, and it also allowed us to design the
interface in more depth.
What do you think? If there is no problem, I am asking for PMC's help
here: we want to propose flink-dynamic-storage as a flink subpro
Hi Stephan,
Thanks for your reply.
Data never expires automatically.
If there is a need for data retention, the user can choose one of the
following options:
- In the SQL for querying the managed table, users filter the data by themselves
- Define the time partition, and users can delete the exp
Thanks for digging into this.
Regarding this query:
INSERT INTO the_table
SELECT window_end, COUNT(*)
FROM (TUMBLE(TABLE interactions, DESCRIPTOR(ts), INTERVAL '5' MINUTES))
GROUP BY window_end
HAVING now() - window_end <= INTERVAL '14' DAYS;
I am not sure I understand what the conclusion
Hi Stephan,
The discussion with you made me think more deeply about some things.
(3) Log Scan Startup mode.
> As I understand it, that means, though, that any managed table that uses
> another managed table as a source will automatically always use the "full
> scan" mode (snapshot + log subscr
Hi Jingsong!
Thank you for all the explanations. To follow up on the points:
(1) Log implementation
Good to hear you are looking to make this extensible.
(2) change tracking
Understood, makes sense (used for re-processing).
(3) Log Scan Startup mode.
Your explanation makes sense.
As I un
Hi Timo,
> It would be great if we can add not only `Receive any type of changelog` but
> also `Receive any type of datatype`.
Nice, I think we can.
> Please clarify whether the compact DDL is a synchronous or asynchrounous
> operation in the API? So far all DDL was synchrounous. And only DML
Hi Stephan,
###
about PK:
> (6) Different consistency guarantees with PK and without PK
> (7) Eventual Consistency Mode vs. Faster Checkpoints
Let's clarify one thing first:
For sink with PK, Flink SQL cannot generate complete changes (without
UPDATE_BEFORE).
- For d
Hi, Stephan and Timo, thanks very much for your replies. I try to
reply to you one by one, split into multiple emails.
First, according to my understanding, from the perspective of database
(mysql) and storage, think about what streaming and batch are:
- Batch is the snapshot, it contains the full
Hi everyone,
sorry for the delay in joining this thread. I went through the FLIP and
have some comments (maybe overlapping with Stephan's comments, which I
haven't read yet):
a. > More importantly, in order to solve the cognitive bar...
It would be great if we can add not only `Receive any t
Hi all!
Thank you for the writeup of this feature. I like the general direction a
lot.
There are some open questions and confusing details still, which I think we
need to clarify first to make this feature really good.
Below are questions/suggestions on the FLIP:
Best,
Stephan
===
Hi Eron,
There is a POC LogStore abstraction: [1].
However, our current focus is not on the abstract log store, because
it is a very complex system. We can't clarify all requirements and
abstractions at the beginning, such as whether to use log store as the
WAL of file store. File store and log s
Jingsong, regarding the LogStore abstraction, I understand that you want to
retain some flexibility as the implementation evolves. It makes sense that
the abstract interfaces would be @Internal for now. Would you kindly
ensure the minimal extensibility is in place, so that the Pulsar dev
communit
Hi all,
I have started the voting thread [1]. Please cast your vote there or
ask additional questions here.
[1] https://lists.apache.org/thread/v3fzx0p6n2jogn86sptzr30kr3yw37sq
Best,
Jingsong
On Mon, Nov 1, 2021 at 5:41 PM Jingsong Li wrote:
>
> Hi Till,
>
> Thanks for your suggestion.
>
> At
Hi Till,
Thanks for your suggestion.
At present, we do not want users to use other storage implementations,
which will undoubtedly require us to propose interfaces and APIs with
compatibility guarantee, which will complicate our design. And some
designs are constantly changing, we will constantly
Hi Kurt,
Thanks a lot for the detailed explanation. I do see that implementing this
feature outside of Flink will be a bigger effort because we probably have
to think more about the exact interfaces and contracts. On the other hand,
I can also imagine that users might want to use different storage
Hi Till,
We have discussed the possibility of putting this FLIP into another
repository offline
with Stephan and Timo. This looks similar with another under going effort
which trying
to put all connectors outside the Flink core repository.
>From the motivation and scope of this FLIP, it's quite d
Hi Jingsong,
Thanks for creating this FLIP. I don't have a lot to add because I am not
very familiar with the SQL components. While reading the FLIP I was
wondering what would we need in Flink to build something like the BDT
feature outside of Flink as a kind of extension? Would something like thi
Hi all,
I updated FLIP based on your feedback:
1. Introduce interfaces: GenericCatalog, ManagedTableFactory,
TableDescriptor.forManaged
2. Introduce log.scan.startup.mode (default initial) to Hybrid source.
3. Add description to miss dropped table.
Best,
Jingsong
On Mon, Oct 25, 2021 at 3:39
Hi Ingo,
Really appreciate your feedback.
#1. The reason why we insist on using no "connector" option is that we
want to bring the following design to users:
- With the "connector" option, it is a mapping, unmanaged table.
- Without the "connector" option, it is a managed table. It may be an
Iceb
Hi Jingsong,
thanks again for the answers. I think requiring catalogs to implement an
interface to support BDTs is something we'll need (though personally I
still prefer explicit DDL here over the "no connector option" approach).
What about more edge cases like
CREATE TABLE T (f0 INT);
ALTER TAB
Hi Ingo,
I thought again.
I'll try to sort out the current catalog behaviors.
Actually, we can divide catalogs into three categories:
1. ExternalCatalog: it can only read or create a single table kind
which connects to external storage. TableFactory is provided by
Catalog, which can have nothing
Hi JingSong,
thank you for the answers!
> BDT only can be dropped by Flink SQL DDL now.
Maybe I'm misunderstanding, but that's only true from the Flink side. What
I meant is that a table could disappear from a catalog entirely outside of
Flink. As a simple example, consider a catalog which repre
Hi Ingo and wenlong,
Thanks for your feedback. Very good questions!
(Built-in Dynamic Table is simplified as BDT)
First, let's look at the following questions:
1. Does BDT want a separate catalog or can it be placed in all
catalogs (such as InMemoryCatalog and HiveCatalog)?
- BDT wants the lat
Hi Jingsong, thanks for the proposal, providing a built-in storage solution
for users will make flink SQL much more easier to use in production.
I have some questions which may be missed in the FLIP, but may be important
IMO:
1. Is it possible to read historical data from the file store first and
Hi Jingsong,
thank you for writing up the proposal. The benefits such a mechanism will
bring will be very valuable! I haven't yet looked into this in detail, but
one question came to my mind immediately:
The DDL for these tables seems to rely on there not being a 'connector'
option. However, cata
32 matches
Mail list logo