I see, I missed the part that a row is either in positioned mode or nameed
mode.
I can live with this. Thanks.
Best,
Kurt
On Wed, Sep 23, 2020 at 9:07 PM Timo Walther wrote:
> But the examples you mentioned would not be different.
>
> By calling `Row.withNames()`, the row has no definition of
But the examples you mentioned would not be different.
By calling `Row.withNames()`, the row has no definition of position. All
position-based methods would throw an exception.
The hashCode()/equals() would return true for:
> Row row1 = Row.withNames();
> row.setField("a", 1);
> row.setField(
Thanks for the detailed response, 1-5 sounds good to me.
For #6, I just think of another case which would also annoy users. Consider
code like this:
Row row = Row.withNames();
row.setField("a", 1);
row.setField("b", 2);
and for second time, he changes the sequence of setting method calls:
Row r
Hi Kurt,
thanks for your feedback.
1. "moving Schema after DataStream": I don't have a strong opinion here.
One could argue that the API would look similar to a CREATE TABLE
statement: first schema then connector. I updated the FLIP.
2. "will we do some verification?"
Yes, we will definitely
Sorry for being late, I went through the design doc and here are
my comments:
1. A minor one, how about moving Schema after DataStream in all affected
APIs? Such as:
StreamTableEnvironment.fromDataStream(Schema, DataStream): Table
StreamTableEnvironment.createTemporaryView(String, Schema, DataStre
Hi Jark,
thanks for your feedback. I removed `withNamesAndPositions` from the
public API list and added a comment that this is only internal API for
converters and serializers.
I would start a new vote tomorrow if there are no objections.
What do you think?
Regards,
Timo
On 23.09.20 08:55,
Hi Timo,
Sorry for the late reply.
I think it would be great if we can make `withNamesAndPositions` internal
visible. This reduces the complexity of the public API.
It's hard to come up with a perfect solution. So let's move on this FLIP.
I don't have other concerns.
Best,
Jark
On Fri, 18 Sep 20
Hi Jark,
the fieldNames map is not intended for users. I would also be fine to
make it a default scope constructor and access it with some internal
utility class next to the Row class. The fieldNames map must only be
used by serializers and converters. A user has no benefit in using it.
For
Personally I think the fieldNames Map is confusing and not handy.
I just have an idea but not sure what you think.
What about adding a new constructor with List field names, this enables all
name-based setter/getters.
Regarding to List -> Map cost for every record, we can suggest users to
reuse the
Hi everyone,
thanks for all the feedback. I updated the FLIP again on Thursday to
integrate the feedback I got from Jingsong and Jark offline. In
particular I updated the `Improve dealing with Row in DataStream API`
section another time. We introduced static methods for Row that should
make t
Thanks for driving this Timo, +1 for voting ~
Best,
Danny Chan
在 2020年9月10日 +0800 PM3:54,Timo Walther ,写道:
> Thanks everyone for this healthy discussion. I updated the FLIP with the
> outcome. I think the result is one of the last core API refactoring and
> users will be happy to have a consistent
Thanks everyone for this healthy discussion. I updated the FLIP with the
outcome. I think the result is one of the last core API refactoring and
users will be happy to have a consistent changelog support. Thanks for
all the contributions.
If there are no objections, I would continue with a vot
Thanks, i'm fine with that.
Timo Walther 于2020年9月9日周三 下午7:02写道:
> I agree with Jark. It reduces confusion.
>
> The DataStream API doesn't know changelog processing at all. A
> DataStream of Row can be used with both `fromDataStream` and
> `fromChangelogStream`. But only the latter API will inte
I agree with Jark. It reduces confusion.
The DataStream API doesn't know changelog processing at all. A
DataStream of Row can be used with both `fromDataStream` and
`fromChangelogStream`. But only the latter API will interpret it as a
changelog something.
And as I mentioned before, the `toCh
I think it would bring in much confusion by a different API name just because
the DataStream generic type is different.
If there are ChangelogMode that only works for Row, can we have a type check
there ?
Switch to a new API name does not really solve the problem well, people still
need to decl
I prefer to have separate APIs for them as changelog stream requires Row
type.
It would make the API more straightforward and reduce the confusion.
Best,
Jark
On Wed, 9 Sep 2020 at 16:21, Timo Walther wrote:
> I had this in the inital design, but Jark had concerns at least for the
> `toChangelo
I had this in the inital design, but Jark had concerns at least for the
`toChangelogStream(ChangelogMode)` (see earlier discussion).
`fromDataStream(dataStream, schema, changelogMode)` would be possible.
But in this case I would vote for a symmetric API. If we keep
toChangelogStream we should
“But I think the planner needs to
know whether the input is insert-only or not.”
Does fromDataStream(dataStream, schema, changelogMode)
solve your concerns ? People can pass around whatever ChangelogMode they like
as an optional param.
By default: fromDataStream(dataStream, schema), the Changel
The conclusion is that we will drop `fromChangelogStream(ChangelogMode,
DataStream)` but will keep `fromChangelogStream(DataStream)`.
The latter is necessary to have a per-record changeflag.
We could think about merging `fromChangelogStream`/`fromDataStream` and
`toChangelogStream`/`toDataStre
Hi,
I'm +1 to use the naming of from/toDataStream, rather than
from/toInsertStream. So we don't need to deprecate the existing
`fromDataStream`.
I'm +1 to Danny's proposal: fromDataStream(dataStream, Schema) and
toDataStream(table, AbstractDataType)
I think we can also keep the method `createTem
Thanks for the summary Timo ~
I want to clarify a little bit, what is the conclusion about the
fromChangelogStream and toChangelogStream, should we use this name or we use
fromDataStream with an optional ChangelogMode flag ?
Best,
Danny Chan
在 2020年9月8日 +0800 PM8:22,Timo Walther ,写道:
> Hi Danny
Hi Danny,
Your proposed signatures sound good to me.
fromDataStream(dataStream, Schema)
toDataStream(table, AbstractDataType)
They address all my concerns. The API would not be symmetric anymore,
but this is fine with me. Others raised concerns about deprecating
`fromDataStream(dataStream, Ex
Hi, Timo ~
"It is not about changelog mode compatibility, it is about the type
compatibility.”
For fromDataStream(dataStream, Schema), there should not be compatibility
problem or data type inconsistency. We know the logical type from Schema and
physical type from the dataStream itself.
For to
Hi Danny,
"if ChangelogMode.INSERT is the default, existing pipelines should be
compatible"
It is not about changelog mode compatibility, it is about the type
compatibility. The renaming to `toInsertStream` is only to have a mean
of dealing with data type inconsistencies that could break exi
"It is a more conservative approach to introduce that in a
new method rather than changing the existing one under the hood and
potentially break existing pipelines silently”
I like the idea actually, but if ChangelogMode.INSERT is the default, existing
pipelines should be compatible. We can see t
Thanks for the nice summary Dawid. I also see the pain points in this
part of the API. Most of the users just want to add a time attribute.
I'm not sure how much projection features we need to have in a
`fromDataStream`. Users can do column renaming/reordering afterwards in
a `.select()`.
The
Hi all,
A comment from my side on the topic of the current, weird
renaming/naming/reordering when registering a DataStream. It might be
just me, but I find it extremely confusing and I would be really, really
happy if we could simplify it.
I really don't like that the actual behaviour of this met
Thanks Timo ~
“No this is not possible, because T records have no changeflag. Without a
changeflag, a ChangelogMode makes not much sense. “
I agree, but just distinguish the different ChangelogMode with a renamed API
still does not resolve the problem either,
an API change compared to an additio
Hi Timo,
1. "fromDataStream VS fromInsertStream"
In terms of naming, personally, I prefer `fromDataStream`,
`fromChangelogStream`, `toDataStream`, `toChangelogStream` than
`fromInsertStream`, `toInsertStream`.
2. "fromDataStream(DataStream, Expression...) VS
fromInsertStream(DataStream).select()
Hi everyone
thanks for your feedback. It's a lot of content that needs to be
digested. I will update the FLIP shortly to incorporate some of the
feedback already. But let me respond to some topics first:
"not deprecate these API", "the API of the table layer is changing too fast"
I agree tha
> a Row has two modes represented by an internal boolean flag
`hasFieldOrder`
+1 confusion with Dawid that what's the result when index-based setters and
name-based setters are mixed used.
And name-based setters look like append instead of set.
It reminds me of Avro's `GenericRecord`, We should s
Timo, Thanks for the discussion
I have only read the "Conversion of DataStream to Table" part so i would only
put some objections there ~
> StreamTableEnvironment.fromInsertStream(DataStream): Table
At first glance, from the perspective of a user, i'm confused by why we must
dintinguish on the
Jingsong raised a good point. We need to be more careful when deprecating
APIs.
For example, tEnv#createTemporaryView was introduced in release-1.10, users
became familiar with this API in the previous release, but now we want to
deprecate it in the next release.
I also have some concerns about de
Thanks Timo for driving.
My first impression is, can we not deprecate these API?
- StreamTableEnvironment.fromDataStream(DataStream): Table
- StreamTableEnvironment.fromDataStream(DataStream, Expression...):
Table
- StreamTableEnvironment.createTemporaryView(String, DataStream,
Expression...): Uni
Thanks for the healthy discussion Jark and Dawid.
6. "Row#setFieldNames(@Nullable Map fieldNames)"
Yes, I'm concerned about about the per-record performance. A converter
or serializer should prepare an immutable Map instance before (stored in
a member variable) that is simply passed to every n
Hi Timo,
Thanks for the quick response.
5. "StreamStatementSet#attachToStream()"
Joining or using connect() with a different DataStream is a good case.
cc @Godfrey , what do you think about the `attachToStream()` API?
6. "Row#setFieldNames(@Nullable Map fieldNames)"
> We need a Map for constant
Hi all,
I really like the ideas of this FLIP. I think it improves user
experience quite a bit. I wanted to add just two comments:
1. As for the StatementSet I like the approach described in the FLIP for
its simplicity. Moreover the way I see it is that if a user wants to
work with DataStream, the
Hi Jark,
thanks for the detailed review. Let me answer your concerns:
## Conversion of DataStream to Table
1. "We limit the usage of `system_rowtime()/system_proctime` to the leaf
of a QueryOperation tree in the validation phase."
I'm fine with allowing `system_proctime` everywhere in the quer
Hi Timo,
Thanks a lot for the great proposal and sorry for the late reply. This is
an important improvement for DataStream and Table API users.
I have listed my thoughts and questions below ;-)
## Conversion of DataStream to Table
1. "We limit the usage of `system_rowtime()/system_proctime` to
Hi David,
thanks for your feedback. Feedback from someone who interacts with many
users is very valuable. I added an explanation for StatementSets to the
FLIP.
Regarding watermarks and fromInsertStream, actually the
`Schema.watermark("ts", system_watermark())`
is not really necessary in the
Timo, nice to see this.
As someone who expects to use these interfaces, but who doesn't fully
understand the existing Table API, I like what I see. Just a couple of
comments:
The way that watermarks fit into the fromChangelogStream case makes sense
to me, and I'm wondering why watermarks don't co
Hi everyone,
I would like to propose a FLIP that aims to resolve the remaining
shortcomings in the Table API:
https://cwiki.apache.org/confluence/display/FLINK/FLIP-136%3A++Improve+interoperability+between+DataStream+and+Table+API
The Table API has received many new features over the last yea
42 matches
Mail list logo