Hi everyone,

I want to jump in the discussion about the "dynamic start offset" problem.
First of all, I share the same concern with Timo and Fabian, that the
"start offset" affects the query semantics, i.e. the query result.
But "hints" is just used for optimization which should affect the result?

I think the "dynamic start offset" is an very important usability problem
which will be faced by many streaming platforms.
I also agree "CREATE TEMPORARY TABLE Temp (LIKE t) WITH
('connector.startup-timestamp-millis' = '1578538374471')" is verbose, what
if we have 10 tables to join?

However, what I want to propose (should be another thread) is a global
configuration to reset start offsets of all the source connectors
in the query session, e.g. "table.sources.start-offset". This is possible
now because `TableSourceFactory.Context` has `getConfiguration`
method to get the session configuration, and use it to create an adapted
TableSource.
Then we can also expose to SQL CLI via SET command, e.g. `SET
'table.sources.start-offset'='earliest';`, which is pretty simple and
straightforward.

This is very similar to KSQL's `SET 'auto.offset.reset'='earliest'` which
is very helpful IMO.

Best,
Jark


On Tue, 10 Mar 2020 at 22:29, Timo Walther <twal...@apache.org> wrote:

> Hi Danny,
>
> compared to the hints, FLIP-110 is fully compliant to the SQL standard.
>
> I don't think that `CREATE TEMPORARY TABLE Temp (LIKE t) WITH (k=v)` is
> too verbose or awkward for the power of basically changing the entire
> connector. Usually, this statement would just precede the query in a
> multiline file. So it can be change "in-place" like the hints you proposed.
>
> Many companies have a well-defined set of tables that should be used. It
> would be dangerous if users can change the path or topic in a hint. The
> catalog/catalog manager should be the entity that controls which tables
> exist and how they can be accessed.
>
>  > what’s the problem there if we user the table hints to support “start
> offset”?
>
> IMHO it violates the meaning of a hint. According to the dictionary, a
> hint is "a statement that expresses indirectly what one prefers not to
> say explicitly". But offsets are a property that are very explicit.
>
> If we go with the hint approach, it should be expressible in the
> TableSourceFactory which properties are supported for hinting. Or do you
> plan to offer those hints in a separate Map<String, String> that cannot
> overwrite existing properties? I think this would be a different story...
>
> Regards,
> Timo
>
>
> On 10.03.20 10:34, Danny Chan wrote:
> > Thanks Timo ~
> >
> > Personally I would say that offset > 0 and start offset = 10 does not
> have the same semantic, so from the SQL aspect, we can not implement a
> “starting offset” hint for query with such a syntax.
> >
> > And the CREATE TABLE LIKE syntax is a DDL which is just verbose for
> defining such dynamic parameters even if it could do that, shall we force
> users to define a temporal table for each query with dynamic params, I
> would say it’s an awkward solution.
> >
> > "Hints should give "hints" but not affect the actual produced result.”
> You mentioned that multiple times and could we give a reason, what’s the
> problem there if we user the table hints to support “start offset” ? From
> my side I saw some benefits for that:
> >
> >
> > • It’s very convent to set up these parameters, the syntax is very much
> like the DDL definition
> > • It’s scope is very clear, right on the table it attathed
> > • It does not affect the table schema, which means in order to specify
> the offset, there is no need to define an offset column which is weird
> actually, offset should never be a column, it’s more like a metadata or a
> start option.
> >
> > So in total, FLIP-110 uses the offset more like a Hive partition prune,
> we can do that if we have an offset column, but most of the case we do not
> define that, so there is actually no conflict or overlap.
> >
> > Best,
> > Danny Chan
> > 在 2020年3月10日 +0800 PM4:28,Timo Walther <twal...@apache.org>,写道:
> >> Hi Danny,
> >>
> >> shouldn't FLIP-110[1] solve most of the problems we have around defining
> >> table properties more dynamically without manual schema work? Also
> >> offset definition is easier with such a syntax. They must not be defined
> >> in catalog but could be temporary tables that extend from the original
> >> table.
> >>
> >> In general, we should aim to keep the syntax concise and don't provide
> >> too many ways of doing the same thing. Hints should give "hints" but not
> >> affect the actual produced result.
> >>
> >> Some connector properties might also change the plan or schema in the
> >> future. E.g. they might also define whether a table source supports
> >> certain push-downs (e.g. predicate push-down).
> >>
> >> Dawid is currently working a draft that might makes it possible to
> >> expose a Kafka offset via the schema such that `SELECT * FROM Topic
> >> WHERE offset > 10` would become possible and could be pushed down. But
> >> this is of course, not planned initially.
> >>
> >> Regards,
> >> Timo
> >>
> >>
> >> [1]
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-110%3A+Support+LIKE+clause+in+CREATE+TABLE
> >>
> >>
> >>
> >> On 10.03.20 08:34, Danny Chan wrote:
> >>> Thanks Wenlong ~
> >>>
> >>> For PROPERTIES Hint Error handling
> >>>
> >>> Actually we have no way to figure out whether a error prone hint is a
> PROPERTIES hint, for example, if use writes a hint like ‘PROPERTIAS’, we do
> not know if this hint is a PROPERTIES hint, what we know is that the hint
> name was not registered in our Flink.
> >>>
> >>> If the user writes the hint name correctly (i.e. PROPERTIES), we did
> can enforce the validation of the hint options though the pluggable
> HintOptionChecker.
> >>>
> >>> For PROPERTIES Hint Option Format
> >>>
> >>> For a key value style hint option, the key can be either a simple
> identifier or a string literal, which means that it’s compatible with our
> DDL syntax. We support simple identifier because many other hints do not
> have the component complex keys like the table properties, and we want to
> unify the parse block.
> >>>
> >>> Best,
> >>> Danny Chan
> >>> 在 2020年3月10日 +0800 PM3:19,wenlong.lwl <wenlong88....@gmail.com>,写道:
> >>>> Hi Danny, thanks for the proposal. +1 for adding table hints, it is
> really
> >>>> a necessary feature for flink sql to integrate with a catalog.
> >>>>
> >>>> For error handling, I think it would be more natural to throw an
> >>>> exception when error table hint provided, because the properties in
> hint
> >>>> will be merged and used to find the table factory which would cause an
> >>>> exception when error properties provided, right? On the other hand,
> unlike
> >>>> other hints which just affect the way to execute the query, the
> property
> >>>> table hint actually affects the result of the query, we should never
> ignore
> >>>> the given property hints.
> >>>>
> >>>> For the format of property hints, currently, in sql client, we accept
> >>>> properties in format of string only in DDL: 'connector.type'='kafka',
> I
> >>>> think the format of properties in hint should be the same as the
> format we
> >>>> defined in ddl. What do you think?
> >>>>
> >>>> Bests,
> >>>> Wenlong Lyu
> >>>>
> >>>> On Tue, 10 Mar 2020 at 14:22, Danny Chan <yuzhao....@gmail.com>
> wrote:
> >>>>
> >>>>> To Weike: About the Error Handing
> >>>>>
> >>>>> To be consistent with other SQL vendors, the default is to log
> warnings
> >>>>> and if there is any error (invalid hint name or options), the hint
> is just
> >>>>> ignored. I have already addressed in the wiki.
> >>>>>
> >>>>> To Timo: About the PROPERTIES Table Hint
> >>>>>
> >>>>> • The properties hints is also optional, user can pass in an option
> to
> >>>>> override the table properties but this does not mean it is required.
> >>>>> • They should not include semantics: does the properties belong to
> >>>>> semantic ? I don't think so, the plan does not change right ? The
> result
> >>>>> set may be affected, but there are already some hints do so, for
> example,
> >>>>> MS-SQL MAXRECURSION and SNAPSHOT hint [1]
> >>>>> • `SELECT * FROM t(k=v, k=v)`: this grammar breaks the SQL standard
> >>>>> compared to the hints way(which is included in comments)
> >>>>> • I actually didn't found any vendors to support such grammar, and
> there
> >>>>> is no way to override table level properties dynamically. For normal
> RDBMS,
> >>>>> I think there are no requests for such dynamic parameters because
> all the
> >>>>> table have the same storage and computation and they are almost all
> batch
> >>>>> tables.
> >>>>> • While Flink as a computation engine has many connectors,
> especially for
> >>>>> some message queue like Kafka, we would have a start_offset which is
> >>>>> different each time we start the query, such parameters can not be
> >>>>> persisted to catalog, because it’s not static, this is actually the
> >>>>> background we propose the table hints to indicate such properties
> >>>>> dynamically.
> >>>>>
> >>>>>
> >>>>> To Jark and Jinsong: I have removed the query hints part and change
> the
> >>>>> title.
> >>>>>
> >>>>> [1]
> >>>>>
> https://docs.microsoft.com/en-us/sql/t-sql/queries/hints-transact-sql-query?view=sql-server-ver15
> >>>>>
> >>>>> Best,
> >>>>> Danny Chan
> >>>>> 在 2020年3月9日 +0800 PM5:46,Timo Walther <twal...@apache.org>,写道:
> >>>>>> Hi Danny,
> >>>>>>
> >>>>>> thanks for the proposal. I agree with Jark and Jingsong. Planner
> hints
> >>>>>> and table hints are orthogonal topics that should be discussed
> >>>>> separately.
> >>>>>>
> >>>>>> I share Jingsong's opinion that we should not use planner hints for
> >>>>>> passing connector properties. Planner hints should be optional at
> any
> >>>>>> time. They should not include semantics but only affect execution
> time.
> >>>>>> Connector properties are an important part of the query itself.
> >>>>>>
> >>>>>> Have you thought about options such as `SELECT * FROM t(k=v, k=v)`?
> How
> >>>>>> are other vendors deal with this problem?
> >>>>>>
> >>>>>> Regards,
> >>>>>> Timo
> >>>>>>
> >>>>>>
> >>>>>> On 09.03.20 10:37, Jingsong Li wrote:
> >>>>>>> Hi Danny, +1 for table hints, thanks for driving.
> >>>>>>>
> >>>>>>> I took a look to FLIP, most of content are talking about query
> hints.
> >>>>> It is
> >>>>>>> hard to discussion and voting. So +1 to split it as Jark said.
> >>>>>>>
> >>>>>>> Another thing is configuration that suitable to config with table
> >>>>> hints:
> >>>>>>> "connector.path" and "connector.topic", Are they really suitable
> for
> >>>>> table
> >>>>>>> hints? Looks weird to me. Because I think these properties are the
> >>>>> core of
> >>>>>>> table.
> >>>>>>>
> >>>>>>> Best,
> >>>>>>> Jingsong Lee
> >>>>>>>
> >>>>>>> On Mon, Mar 9, 2020 at 5:30 PM Jark Wu <imj...@gmail.com> wrote:
> >>>>>>>
> >>>>>>>> Thanks Danny for starting the discussion.
> >>>>>>>> +1 for this feature.
> >>>>>>>>
> >>>>>>>> If we just focus on the table hints not the query hints in this
> >>>>> release,
> >>>>>>>> could you split the FLIP into two FLIPs?
> >>>>>>>> Because it's hard to vote on partial part of a FLIP. You can keep
> >>>>> the table
> >>>>>>>> hints proposal in FLIP-113 and move query hints into another FLIP.
> >>>>>>>> So that we can focuse on the table hints in the FLIP.
> >>>>>>>>
> >>>>>>>> Thanks,
> >>>>>>>> Jark
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Mon, 9 Mar 2020 at 17:14, DONG, Weike <kyled...@connect.hku.hk
> >
> >>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi Danny,
> >>>>>>>>>
> >>>>>>>>> This is a nice feature, +1.
> >>>>>>>>>
> >>>>>>>>> One thing I am interested in but not mentioned in the proposal is
> >>>>> the
> >>>>>>>> error
> >>>>>>>>> handling, as it is quite common for users to write inappropriate
> >>>>> hints in
> >>>>>>>>> SQL code, if illegal or "bad" hints are given, would the system
> >>>>> simply
> >>>>>>>>> ignore them or throw exceptions?
> >>>>>>>>>
> >>>>>>>>> Thanks : )
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Weike
> >>>>>>>>>
> >>>>>>>>> On Mon, Mar 9, 2020 at 5:02 PM Danny Chan <yuzhao....@gmail.com>
> >>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Note:
> >>>>>>>>>> we only plan to support table hints in Flink release 1.11, so
> >>>>> please
> >>>>>>>>> focus
> >>>>>>>>>> mainly on the table hints part and just ignore the planner
> >>>>> hints, sorry
> >>>>>>>>> for
> >>>>>>>>>> that mistake ~
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Danny Chan
> >>>>>>>>>> 在 2020年3月9日 +0800 PM4:36,Danny Chan <yuzhao....@gmail.com>,写道:
> >>>>>>>>>>> Hi, fellows ~
> >>>>>>>>>>>
> >>>>>>>>>>> I would like to propose the supports for SQL hints for our
> >>>>> Flink SQL.
> >>>>>>>>>>>
> >>>>>>>>>>> We would support hints syntax as following:
> >>>>>>>>>>>
> >>>>>>>>>>> select /*+ NO_HASH_JOIN, RESOURCE(mem='128mb',
> >>>>> parallelism='24') */
> >>>>>>>>>>> from
> >>>>>>>>>>> emp /*+ INDEX(idx1, idx2) */
> >>>>>>>>>>> join
> >>>>>>>>>>> dept /*+ PROPERTIES(k1='v1', k2='v2') */
> >>>>>>>>>>> on
> >>>>>>>>>>> emp.deptno = dept.deptno
> >>>>>>>>>>>
> >>>>>>>>>>> Basically we would support both query hints(after the SELECT
> >>>>> keyword)
> >>>>>>>>>> and table hints(after the referenced table name), for 1.11, we
> >>>>> plan to
> >>>>>>>>> only
> >>>>>>>>>> support table hints with a hint probably named PROPERTIES:
> >>>>>>>>>>>
> >>>>>>>>>>> table_name /*+ PROPERTIES(k1='v1', k2='v2') *+/
> >>>>>>>>>>>
> >>>>>>>>>>> I am looking forward to your comments.
> >>>>>>>>>>>
> >>>>>>>>>>> You can access the FLIP here:
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+SQL+and+Planner+Hints
> >>>>>>>>>>>
> >>>>>>>>>>> Best,
> >>>>>>>>>>> Danny Chan
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
> >
>
>

Reply via email to