You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Danny Chan <yu...@gmail.com> on 2020/03/09 08:36:21 UTC

[DISCUSS]FLIP-113: Support SQL and planner hints

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

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Fabian Hueske <fh...@gmail.com>.
Hi,

RE error handling:
I don't think it is a good idea to simply log invalid hints. This makes it
very hard for systems that integrate Flink to consume these errors.
I'm not saying we should throw an exception. We could also execute a query
with invalid hints but should have a mechanism to provide information about
invalid hints to systems that integrates Flink.

RE changing query semantics with hints:
I agree with Timo on this one, especially if queries with invalid hints are
executed.
Otherwise, it will be hard for users to validate that the result is what
they expected it to be.
If hints only affect execution strategies, users can be sure that the query
result is correct even if the execution plan was not fixed as hinted.

Best,
Fabian


Am Di., 10. März 2020 um 10:34 Uhr schrieb Danny Chan <yuzhao.cyz@gmail.com
>:

> 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 <tw...@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 <we...@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 <yu...@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 <tw...@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 <im...@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 <
> kyledong@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.cyz@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.cyz@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
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <da...@apache.org>.
Thanks Timo and Fabian ~

compared to the hints, FLIP-110 is fully compliant to the SQL standard.


I don't think so, here is the syntax of CREATE TABLE in SQL-2011 IWD
9075-2:201?(E) 11.3 <table definition>:

<table definition> ::=
  CREATE [ <table scope> ] TABLE <table name> <table contents source>
      [ WITH <system versioning clause> ]
      [ ON COMMIT <table commit action> ROWS ]
<table contents source> ::=
    <table element list>
  | <typed table clause>
  | <as subquery clause>

<table element list> ::=
  <left paren> <table element> [ { <comma> <table element> }... ] <right
paren>
<table element> ::=
    <column definition>
  | <table period definition>
  | <table constraint definition>
  | <like clause>

<like clause> ::=
  LIKE <table name> [ <like options> ]
<like options> ::=
  <like option>...
<like option> ::=
    <identity option>
  | <column default option>
  | <generation option>
<identity option> ::=
    INCLUDING IDENTITY
  | EXCLUDING IDENTITY
<column default option> ::=
    INCLUDING DEFAULTS
  | EXCLUDING DEFAULTS
<generation option> ::=
    INCLUDING GENERATED
  | EXCLUDING GENERATED

So, according to the standard SQL, we can have syntax as "CREATE TABLE t2
like t1 [like options]";
MySQL supports "CREATE TABLE new_tbl LIKE orig_tbl" but there is no WITH
options[1]. Basically the syntax "CREATE TABLE t like with (k1=v1, k2=v2)"
syntax has is totally not compatible with the SQL standard.

I agree with Fabian that the connector properties is important to the query
execution, we can change the error handling to throws if:

   - The hints was not recognized for Flink
   - The hint options are in invalid format

Both of these are pluggable.

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


We can limit the hint options of PROPERTIES to only support those we want
to support, if you think that is necessary, there is no need to distinguish
in TableSourceFactory for these attributes. And like you said, this is
totally about properties validation which is another story, we have this
problem even if we use the DDL way.

[1] https://dev.mysql.com/doc/refman/5.7/en/create-table-like.html

Timo Walther <tw...@apache.org> 于2020年3月10日周二 下午10:29写道:

> 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 <tw...@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 <we...@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 <yu...@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 <tw...@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 <im...@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 <kyledong@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 <yu...@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 <yu...@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
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
> >
>
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Kurt Young <yk...@gmail.com>.
+1, LGTM

Best,
Kurt


On Wed, Apr 8, 2020 at 10:27 AM Jark Wu <im...@gmail.com> wrote:

> Thanks for the summary Danny. +1 to the new proposal.
>
> I have a minor concern about the global configuration
> `table.optimizer.dynamic-table-options.enabled`, does it belong to
> optimizer?
> From my point of view, it is just an API to set table options and uses
> Calcite in the implementation.
> I'm also thinking about what's the name of other configurations, e.g
> time-zone, code-gen length, state ttl.
> Should they prefix with "optimizer" or "exec" or something else or nothing?
>
> Best,
> Jark
>
> On Tue, 7 Apr 2020 at 23:17, Timo Walther <tw...@apache.org> wrote:
>
> > Thanks for the update Danny. +1 from my side.
> >
> > Regards,
> > Timo
> >
> >
> > On 07.04.20 13:25, Danny Chan wrote:
> > > Hi, every ~
> > >
> > > It seems that we all agree to drop the idea for white/black list for
> each
> > > connector, and have a global config option to default disable this
> > feature.
> > >
> > > I have also discussed with Timo and Jark about the interface
> > > TableSourceTable.Context.getExecutionOptions and finally we decide to
> > > introduce a new interface CatalogTable#copy(Map<String, String>) to
> > support
> > > re-generate the table with new table options.
> > >
> > > So let me summarize the current design broadly again:
> > >
> > >     - Use the syntax /*+ OPTIONS('k1' = 'v1', 'k2' = 'v2') to describe
> > the
> > >     dynamic table options
> > >     - There is no constraint on which option key can be used in the
> > OPTIONS,
> > >     that means, any option key is allowed, the factory would to the
> > validation
> > >     work finally
> > >     - Introduce method CatalogTable#copy, we use this method to
> > regenerate a
> > >     new CatalogTable to find a table factory and creates table
> > source/sink
> > >     - There is a global config option to default disable this feature
> (if
> > >     user uses OPTIONS, an exception throws to tell open the option)
> > >
> > > I have updated the WIKI
> > > <
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
> > >,
> > > look forward to your suggestions ~
> > >
> > > Jark Wu <im...@gmail.com> 于2020年4月7日周二 上午11:24写道:
> > >
> > >> I'm fine to disable this feature by default and avoid
> > >> whitelisting/blacklisting. This simplifies a lot of things.
> > >>
> > >> Regarding to TableSourceFactory#Context#getExecutionOptions, do we
> > really
> > >> need this interface?
> > >> Should the connector factory be aware of the properties is merged with
> > >> hints or not?
> > >> What's the problem if we always get properties from
> > >> `CatalogTable#getProperties`?
> > >>
> > >> Best,
> > >> Jark
> > >>
> > >> On Tue, 7 Apr 2020 at 10:39, Kurt Young <yk...@gmail.com> wrote:
> > >>
> > >>> Sounds like a reasonable compromise, disabling this feature by
> default
> > >> is a
> > >>> way to protect
> > >>> the vulnerability, and we can simplify the design quite a lot. We can
> > >>> gather some users'
> > >>> feedback to see whether further protections are necessary in the
> > future.
> > >>>
> > >>> Best,
> > >>> Kurt
> > >>>
> > >>>
> > >>> On Mon, Apr 6, 2020 at 11:49 PM Timo Walther <tw...@apache.org>
> > wrote:
> > >>>
> > >>>> I agree with Aljoscha. The length of this thread shows that this is
> > >>>> highly controversal. I think nobody really likes this feature 100%
> but
> > >>>> we could not find a better solution. I would consider it as a
> > >>>> nice-to-have improvement during a notebook/debugging session.
> > >>>>
> > >>>> I would accept avoiding whitelisting/blacklisting if the feature is
> > >>>> disabled by default. And we make the merged properties available in
> a
> > >>>> separate TableSourceFactory#Context#getExecutionOptions as Danny
> > >>> proposed.
> > >>>>
> > >>>> What do you think?
> > >>>>
> > >>>> Thanks,
> > >>>> Timo
> > >>>>
> > >>>>
> > >>>> On 06.04.20 09:59, Aljoscha Krettek wrote:
> > >>>>> The reason I'm saying it should be disabled by default is that this
> > >>> uses
> > >>>>> hint syntax, and hints should really not change query semantics.
> > >>>>>
> > >>>>> I'm quite strongly against hints that change query semantics, but
> if
> > >> we
> > >>>>> disable this by default I would be (reluctantly) OK with the
> feature.
> > >>>>> Companies that create deployments or set up the SQL environment for
> > >>>>> users can enable the feature if they want.
> > >>>>>
> > >>>>> But yes, I also agree that we don't need whitelisting/blacklisting,
> > >>>>> which makes this a lot easier to do.
> > >>>>>
> > >>>>> Best,
> > >>>>> Aljoscha
> > >>>>>
> > >>>>> On 06.04.20 04:27, Danny Chan wrote:
> > >>>>>> Hi, everyone ~
> > >>>>>>
> > >>>>>> @Aljoscha @Timo
> > >>>>>>
> > >>>>>>> I think we're designing ourselves into ever more complicated
> > >> corners
> > >>>>>> here
> > >>>>>>
> > >>>>>> I kindly agree that, personally didn't see strong reasons why we
> > >>>>>> should limit on each connector properties:
> > >>>>>>
> > >>>>>> • we can define any table options for CREATE TABLE, why we treat
> the
> > >>>>>> dynamic options differently, we never consider any security
> problems
> > >>>>>> when create table, we should not either for dynamic table options
> > >>>>>> • If we do not have whitelist properties or blacklist properties,
> > >> the
> > >>>>>> table source creation work would be much easier, just used the
> > >> merged
> > >>>>>> options. There is no need to modify each connector to decide which
> > >>>>>> options could be overridden and how we merge them(the merge work
> is
> > >>>>>> redundant).
> > >>>>>> • @Timo, how about we support another interface
> > >>>>>> `TableSourceFactory#Context.getExecutionOptions`, we always use
> this
> > >>>>>> interface to get the options to create our table source. There is
> no
> > >>>>>> need to copy the catalog table itselt, we just need to generate
> our
> > >>>>>> Context correctly.
> > >>>>>> • @Aljoscha I agree to have a global config option, but I disagree
> > >> to
> > >>>>>> default disable it, a global default config would break the user
> > >>>>>> experience too much, especially when user want to modify the
> options
> > >>>>>> in a ad-hoc way.
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> I suggest to remove `TableSourceFactory#supportedHintOptions` or
> > >>>>>> `TableSourceFactory#forbiddenHintOptions` based on the fact that
> we
> > >>>>>> does not have black/white list for CREATE TABLE at all at lease
> for
> > >>>>>> current codebase.
> > >>>>>>
> > >>>>>>
> > >>>>>> @Timo (i have replied offline but allows to represent it here
> again)
> > >>>>>>
> > >>>>>> The `TableSourceFactory#supportedHintOptions` doesn't work well
> for
> > >> 3
> > >>>>>> reasons compared to `TableSourceFactory#forbiddenHintOptions`:
> > >>>>>> 1. For key with wildcard, like connector.property.* , use a
> > >> blacklist
> > >>>>>> make us have the ability to disable some of the keys under that,
> > >> i.e.
> > >>>>>> connector.property.key1 , a whitelist can only match with prefix
> > >>>>>>
> > >>>>>> 2. We want the connectors to have the ability to disable format
> type
> > >>>>>> switch format.type but allows all the other properties, e.g.
> > >> format.*
> > >>>>>> without format.type(let's call it SET_B), if we use the whitelist,
> > >> we
> > >>>>>> have to enumerate all the specific format keys start with format
> > >>>>>> (SET_B), but with the old connector factories, we have no idea
> what
> > >>>>>> specific format keys it supports(there is either a format.* or
> > >>> nothing).
> > >>>>>>
> > >>>>>> 3. Except the cases for 1 and 2, for normal keys(no wildcard), the
> > >>>>>> blacklist and whitelist has the same expressiveness, use blacklist
> > >>>>>> makes the code not too verbose to enumerate all the duplicate keys
> > >>>>>> with #supportedKeys .(Not very strong reason, but i think as a
> > >>>>>> connector developer, it makes sense)
> > >>>>>>
> > >>>>>> Best,
> > >>>>>> Danny Chan
> > >>>>>> 在 2020年4月3日 +0800 PM11:28,Timo Walther <tw...@apache.org>,写道:
> > >>>>>>> Hi everyone,
> > >>>>>>>
> > >>>>>>> @Aljoscha: I disagree with your approach because a `Catalog` can
> > >>> return
> > >>>>>>> a custom factory that is not using any properties. The hinting
> must
> > >>> be
> > >>>>>>> transparent to a factory. We should NOT modify the metadata
> > >>>>>>> `CatalogTable` at any point in time after the catalog.
> > >>>>>>>
> > >>>>>>> @Danny, @Jingsong: How about we stick to the original design that
> > >> we
> > >>>>>>> wanted to vote on but use:
> > >>>>>>>
> > >>>>>>> Set<String> supportedHintProperties()
> > >>>>>>>
> > >>>>>>> This fits better to the old factory design. And for the new
> FLIP-95
> > >>>>>>> factories we will use `ConfigOption` and provide good utilities
> for
> > >>>>>>> merging with hints etc.
> > >>>>>>>
> > >>>>>>> We can allow `"format.*"` in `supportedHintProperties()` to allow
> > >>>>>>> hinting in formats.
> > >>>>>>>
> > >>>>>>> What do you think?
> > >>>>>>>
> > >>>>>>> Regards,
> > >>>>>>> Timo
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On 02.04.20 16:24, Aljoscha Krettek wrote:
> > >>>>>>>> I think we're designing ourselves into ever more complicated
> > >> corners
> > >>>>>>>> here. Maybe we need to take a step back and reconsider. What
> would
> > >>> you
> > >>>>>>>> think about this (somewhat) simpler proposal:
> > >>>>>>>>
> > >>>>>>>> - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or
> > >>>>>>>> CONNECTOR_PROPERTIES, depending on what naming we want to have
> for
> > >>>> this
> > >>>>>>>> in the future. This will simply overwrite all connector
> > >> properties,
> > >>>> the
> > >>>>>>>> table factories don't know about hints but simply work with the
> > >>>>>>>> properties that they are given
> > >>>>>>>>
> > >>>>>>>> - this special hint is disabled by default and can be activated
> > >>> with a
> > >>>>>>>> global option "foo.bazzle.connector-hints" (or something like
> > >> this)
> > >>>>>>>> which has a warning that describes that this can change query
> > >>>> semantics
> > >>>>>>>> etc.
> > >>>>>>>>
> > >>>>>>>> That's it. This makes connector implementations a lot easier
> while
> > >>>>>>>> still
> > >>>>>>>> allowing inline configuration.
> > >>>>>>>>
> > >>>>>>>> I still don't like using hint syntax at all for this, because I
> > >>>>>>>> strongly
> > >>>>>>>> maintain that hints should not change query syntax. In general
> > >> using
> > >>>>>>>> hints should be kept to a minimum because they usually point to
> > >>>>>>>> shortcomings in the system.
> > >>>>>>>>
> > >>>>>>>> Best,
> > >>>>>>>> Aljoscha
> > >>>>>>>>
> > >>>>>>>> On 02.04.20 06:06, Jingsong Li wrote:
> > >>>>>>>>> Hi Dawid,
> > >>>>>>>>>
> > >>>>>>>>>> When a factory is instantiated it has access to the
> > >> CatalogTable,
> > >>>>>>>>> therefore it has access to all the original properties. In turn
> > >> it
> > >>>>>>>>> knows
> > >>>>>>>>> the original format and can call
> > >>>> FormatFactory#supportedHintOptions().
> > >>>>>>>>>
> > >>>>>>>>> Factory can only get CatalogTable when creating source or sink,
> > >>>>>>>>> right? IIUC, TableFactory may be stateless too.
> > >>>>>>>>> When invoking SourceFactory#supportedHintOptions(), it can not
> > >>>>>>>>> get CatalogTable, so it is impossible to create FormatFactory?
> > >>>>>>>>>
> > >>>>>>>>> Best,
> > >>>>>>>>> Jingsong Lee
> > >>>>>>>>>
> > >>>>>>>>> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <
> yuzhao.cyz@gmail.com
> > >>>
> > >>>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> Hi, Dawid, thanks so much for the detail suggestions ~
> > >>>>>>>>>>
> > >>>>>>>>>> 1. Regarding the motivation:
> > >>>>>>>>>>
> > >>>>>>>>>> I agree it's not a good suggested way based on the fact that
> we
> > >>> have
> > >>>>>>>>>> better solution, but i think we can support override that as
> > >> long
> > >>>>>>>>>> as it
> > >>>>>>>>>> exists as one of the the table options. I would remove if from
> > >> the
> > >>>>>>>>>> motication part.
> > >>>>>>>>>>
> > >>>>>>>>>> 2. The options passes around during sql-to-rel conversion,
> right
> > >>>>>>>>>> after we
> > >>>>>>>>>> generate the RelOptTable (CatalogSourceTable#toRel in Flink),
> > >> this
> > >>>> is
> > >>>>>>>>>> indeed a push way method at least in the RelOptTable layer,
> then
> > >>>>>>>>>> we hand
> > >>>>>>>>>> over the options to TableSourceFactory with our own context,
> > >> which
> > >>>> is
> > >>>>>>>>>> fine
> > >>>>>>>>>> becuase TableSourceFactory#Context is the contact to pass
> around
> > >>>>>>>>>> these
> > >>>>>>>>>> table-about variables.
> > >>>>>>>>>>
> > >>>>>>>>>> 3. "We should not end up with an extreme example where we can
> > >>>>>>>>>> change the
> > >>>>>>>>>> connector type", i totally agree that, and i have listed the
> > >>>>>>>>>> "connector.type" as forbidden attribute in the WIKI. As for
> the
> > >>>>>>>>>> format, i
> > >>>>>>>>>> think the connector itself can/should control whether to
> > >> override
> > >>>> the
> > >>>>>>>>>> "format.type", that is one of the reason i change the
> > >>>>>>>>>> TableSourceFactory#supportedHintOpitons to
> > >>>>>>>>>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we
> can
> > >>>>>>>>>> limit the
> > >>>>>>>>>> format keys we want conveniently.
> > >>>>>>>>>>
> > >>>>>>>>>> 4. SQL Hints syntax.
> > >>>>>>>>>>
> > >>>>>>>>>>> I think the k and v in the hint_item should be QUOTED_STRING
> > >> (not
> > >>>>>>>>>>> sure
> > >>>>>>>>>> if it is equivalent to string_literal).
> > >>>>>>>>>>
> > >>>>>>>>>> I disagree, we at least should keep sync with our DDL: use the
> > >>>> string
> > >>>>>>>>>> literal as the key. We did also support the simple identifier
> > >>>> because
> > >>>>>>>>>> this
> > >>>>>>>>>> is the common hint syntax from Calcite, it does not hurt
> > >> anything
> > >>>> for
> > >>>>>>>>>> the
> > >>>>>>>>>> OPTIONS hint, the unsupported keys would validate fails.(If
> you
> > >>>> think
> > >>>>>>>>>> that
> > >>>>>>>>>> may cause some confuse, i can make the syntax pluggable for
> each
> > >>>>>>>>>> hint in
> > >>>>>>>>>> CALCITE 1.23)
> > >>>>>>>>>>
> > >>>>>>>>>> We only supports OPTIONS hint in the FLIP, and i have changed
> > >> the
> > >>>>>>>>>> title to
> > >>>>>>>>>> "Supports dynamic table options", would make it more clear in
> > >> the
> > >>>>>>>>>> WIKI.
> > >>>>>>>>>>
> > >>>>>>>>>> 5. Yes, we also have this concerns from our offline
> discussion,
> > >>>>>>>>>> that is
> > >>>>>>>>>> one of the reason, why i change the
> > >>>>>>>>>> TableSourceFactory#supportedHintOpitons
> > >>>>>>>>>> to TableSourceFactory#forbiddenHintOpitons, i do not choose
> > >>>>>>>>>> Set<String>
> > >>>>>>>>>> supportedHintOptionKeys() with wildcard for 2 reasons:
> > >>>>>>>>>>
> > >>>>>>>>>>      - The wildcard is still not descriptive, we can still not
> > >>>>>>>>>> forbidden one
> > >>>>>>>>>> of the properties among the wildcard properties, we can not
> > >> enable
> > >>>> or
> > >>>>>>>>>> disable them totally
> > >>>>>>>>>>      - ConfigOption is our new structure for keys, and it does
> > >> not
> > >>>>>>>>>> support
> > >>>>>>>>>> wildcard yet.
> > >>>>>>>>>>
> > >>>>>>>>>> Best,
> > >>>>>>>>>> Danny Chan
> > >>>>>>>>>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz
> > >>>>>>>>>> <dw...@apache.org>,写道:
> > >>>>>>>>>>> Hi,
> > >>>>>>>>>>> Few comments from my side:
> > >>>>>>>>>>> 1. Regarding the motivation:
> > >>>>>>>>>>> I think the example with changing the update-mode is not a
> good
> > >>>>>>>>>>> one. In
> > >>>>>>>>>> the long term this should be done with EMIT CHANGELOG
> (discussed
> > >>> in
> > >>>>>>>>>> FLIP-105).
> > >>>>>>>>>>> Nitpicking: I would mention that it is rather for
> > >>> debugging/ad-hoc
> > >>>>>>>>>> solution. I think this should not be a recommended way for
> > >>>> production
> > >>>>>>>>>> use
> > >>>>>>>>>> cases as it bypasses the Catalog, which should be the source
> of
> > >>>>>>>>>> truth.
> > >>>>>>>>>>> 2. I could not understand how the additional options will be
> > >>>>>>>>>>> passed to
> > >>>>>>>>>> the TableSourceFactory. Could you elaborate a bit more on
> that?
> > >> I
> > >>>> see
> > >>>>>>>>>> there
> > >>>>>>>>>> is a Context interface that gives the options. But cannot
> find a
> > >>> way
> > >>>>>>>>>> to get
> > >>>>>>>>>> the context itself in the factory. Moreover I think it would
> > >> make
> > >>>>>>>>>> more
> > >>>>>>>>>> sense to have rather a push based approach here. Something
> like
> > >>>>>>>>>> applyOptions(ReadableConfig) method.
> > >>>>>>>>>>> 3. As for the concerns Jingsong raised in the voting thread.
> I
> > >>>>>>>>>>> think it
> > >>>>>>>>>> is not a big problem, but I agree this should be also
> > >> described. I
> > >>>>>>>>>> disagree
> > >>>>>>>>>> with "Connector don't know format information in TableFactory
> > >>> before
> > >>>>>>>>>> obtains real properties, so it can not list any format
> > >>>>>>>>>> `supportedHintOptions`".
> > >>>>>>>>>>> When a factory is instantiated it has access to the
> > >> CatalogTable,
> > >>>>>>>>>> therefore it has access to all the original properties. In
> turn
> > >> it
> > >>>>>>>>>> knows
> > >>>>>>>>>> the original format and can call
> > >>>>>>>>>> FormatFactory#supportedHintOptions().
> > >>>>>>>>>>> The only case when this would not work would be if we allow
> > >>>> changing
> > >>>>>>>>>>> the
> > >>>>>>>>>> format of the Table (e.g. from avro to parquet), which does
> not
> > >>>> sound
> > >>>>>>>>>> like
> > >>>>>>>>>> a good idea to me. I think this feature should not end up as a
> > >> way
> > >>>> to
> > >>>>>>>>>> declare a whole table inline in a SQL query, but should rather
> > >> be
> > >>> a
> > >>>>>>>>>> simple
> > >>>>>>>>>> way for debugging queries. We should not end up with an
> extreme
> > >>>>>>>>>> example
> > >>>>>>>>>> where we do:
> > >>>>>>>>>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka',
> ...,
> > >>>>>>>>>> 'format.type' = 'json', ....) */
> > >>>>>>>>>>> 4. SQL Hints syntax.
> > >>>>>>>>>>> I think the k and v in the hint_item should be QUOTED_STRING
> > >> (not
> > >>>>>>>>>>> sure
> > >>>>>>>>>> if it is equivalent to string_literal). I think we should not
> > >> use
> > >>>>>>>>>> simple_identifier because this implies that we cannot use e.g.
> > >> any
> > >>>>>>>>>> SQL
> > >>>>>>>>>> keywords. Anyway it has nothing to do with identifiers. If I
> am
> > >>> not
> > >>>>>>>>>> mistaken it is also how the options in the CREATE statement
> are
> > >>>>>>>>>> implemented.
> > >>>>>>>>>>> What is the purpose of the remaining hint_item:
> > >>> hint_name(hint_opt
> > >>>> [
> > >>>>>>>>>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I
> got a
> > >>>>>>>>>> feeling it
> > >>>>>>>>>> does also suggests to support the whole Apache Calcite hint
> > >> system
> > >>>>>>>>>> without
> > >>>>>>>>>> specifying that explicitly. Is the intention of the FLIP to
> > >>> support
> > >>>>>>>>>> choosing e.g. JOIN strategies through hints already? If it is
> so
> > >>> it
> > >>>>>>>>>> should
> > >>>>>>>>>> be mentioned in the FLIP, imo.
> > >>>>>>>>>>> 5. I think something does not work around the
> > >>>>>>>>>>> supportedHintOptions and
> > >>>>>>>>>> wildcards. How do you want to represent a wildcard key as a
> > >>>>>>>>>> ConfigOption? I
> > >>>>>>>>>> am not sure about that, just a though, maybe it make sense to
> > >> have
> > >>>>>>>>>> rather
> > >>>>>>>>>> Set<String> supportedHintOptionKeys()?
> > >>>>>>>>>>> Best,
> > >>>>>>>>>>> Dawid
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>
> > >>>>
> > >>>
> > >>
> > >
> >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <da...@apache.org>.
Hi everyone ~

Thanks for the feedback, i would start a new vote again if there are no new
input objections after 24 hours ~

Best,
Danny

Jark Wu <im...@gmail.com> 于2020年4月8日周三 下午7:19写道:

> `table.dynamic-table-options.enabled` and `TableConfigOptions` sounds good
> to me.
>
> Best,
> Jark
>
> On Wed, 8 Apr 2020 at 18:59, Danny Chan <yu...@gmail.com> wrote:
>
> > `table.dynamic-table-options.enabled` seems fine to me, I would make a
> new
> > `TableConfigOptions` class and put the config option there ~
> >
> > What do you think about the new class to put ?
> >
> > Best,
> > Danny Chan
> > 在 2020年4月8日 +0800 PM5:33,dev@flink.apache.org,写道:
> > >
> > > `table.dynamic-table-options.enabled`
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jark Wu <im...@gmail.com>.
`table.dynamic-table-options.enabled` and `TableConfigOptions` sounds good
to me.

Best,
Jark

On Wed, 8 Apr 2020 at 18:59, Danny Chan <yu...@gmail.com> wrote:

> `table.dynamic-table-options.enabled` seems fine to me, I would make a new
> `TableConfigOptions` class and put the config option there ~
>
> What do you think about the new class to put ?
>
> Best,
> Danny Chan
> 在 2020年4月8日 +0800 PM5:33,dev@flink.apache.org,写道:
> >
> > `table.dynamic-table-options.enabled`
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
`table.dynamic-table-options.enabled` seems fine to me, I would make a new `TableConfigOptions` class and put the config option there ~

What do you think about the new class to put ?

Best,
Danny Chan
在 2020年4月8日 +0800 PM5:33,dev@flink.apache.org,写道:
>
> `table.dynamic-table-options.enabled`

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Aljoscha Krettek <al...@apache.org>.
On 08.04.20 04:27, Jark Wu wrote:

> I have a minor concern about the global configuration
> `table.optimizer.dynamic-table-options.enabled`, does it belong to
> optimizer?
>  From my point of view, it is just an API to set table options and uses
> Calcite in the implementation.
> I'm also thinking about what's the name of other configurations, e.g
> time-zone, code-gen length, state ttl.
> Should they prefix with "optimizer" or "exec" or something else or nothing?

I agree that this is probably not the right option name. Could we just 
have `table.dynamic-table-options.enabled`?

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jark Wu <im...@gmail.com>.
Thanks for the summary Danny. +1 to the new proposal.

I have a minor concern about the global configuration
`table.optimizer.dynamic-table-options.enabled`, does it belong to
optimizer?
From my point of view, it is just an API to set table options and uses
Calcite in the implementation.
I'm also thinking about what's the name of other configurations, e.g
time-zone, code-gen length, state ttl.
Should they prefix with "optimizer" or "exec" or something else or nothing?

Best,
Jark

On Tue, 7 Apr 2020 at 23:17, Timo Walther <tw...@apache.org> wrote:

> Thanks for the update Danny. +1 from my side.
>
> Regards,
> Timo
>
>
> On 07.04.20 13:25, Danny Chan wrote:
> > Hi, every ~
> >
> > It seems that we all agree to drop the idea for white/black list for each
> > connector, and have a global config option to default disable this
> feature.
> >
> > I have also discussed with Timo and Jark about the interface
> > TableSourceTable.Context.getExecutionOptions and finally we decide to
> > introduce a new interface CatalogTable#copy(Map<String, String>) to
> support
> > re-generate the table with new table options.
> >
> > So let me summarize the current design broadly again:
> >
> >     - Use the syntax /*+ OPTIONS('k1' = 'v1', 'k2' = 'v2') to describe
> the
> >     dynamic table options
> >     - There is no constraint on which option key can be used in the
> OPTIONS,
> >     that means, any option key is allowed, the factory would to the
> validation
> >     work finally
> >     - Introduce method CatalogTable#copy, we use this method to
> regenerate a
> >     new CatalogTable to find a table factory and creates table
> source/sink
> >     - There is a global config option to default disable this feature (if
> >     user uses OPTIONS, an exception throws to tell open the option)
> >
> > I have updated the WIKI
> > <
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
> >,
> > look forward to your suggestions ~
> >
> > Jark Wu <im...@gmail.com> 于2020年4月7日周二 上午11:24写道:
> >
> >> I'm fine to disable this feature by default and avoid
> >> whitelisting/blacklisting. This simplifies a lot of things.
> >>
> >> Regarding to TableSourceFactory#Context#getExecutionOptions, do we
> really
> >> need this interface?
> >> Should the connector factory be aware of the properties is merged with
> >> hints or not?
> >> What's the problem if we always get properties from
> >> `CatalogTable#getProperties`?
> >>
> >> Best,
> >> Jark
> >>
> >> On Tue, 7 Apr 2020 at 10:39, Kurt Young <yk...@gmail.com> wrote:
> >>
> >>> Sounds like a reasonable compromise, disabling this feature by default
> >> is a
> >>> way to protect
> >>> the vulnerability, and we can simplify the design quite a lot. We can
> >>> gather some users'
> >>> feedback to see whether further protections are necessary in the
> future.
> >>>
> >>> Best,
> >>> Kurt
> >>>
> >>>
> >>> On Mon, Apr 6, 2020 at 11:49 PM Timo Walther <tw...@apache.org>
> wrote:
> >>>
> >>>> I agree with Aljoscha. The length of this thread shows that this is
> >>>> highly controversal. I think nobody really likes this feature 100% but
> >>>> we could not find a better solution. I would consider it as a
> >>>> nice-to-have improvement during a notebook/debugging session.
> >>>>
> >>>> I would accept avoiding whitelisting/blacklisting if the feature is
> >>>> disabled by default. And we make the merged properties available in a
> >>>> separate TableSourceFactory#Context#getExecutionOptions as Danny
> >>> proposed.
> >>>>
> >>>> What do you think?
> >>>>
> >>>> Thanks,
> >>>> Timo
> >>>>
> >>>>
> >>>> On 06.04.20 09:59, Aljoscha Krettek wrote:
> >>>>> The reason I'm saying it should be disabled by default is that this
> >>> uses
> >>>>> hint syntax, and hints should really not change query semantics.
> >>>>>
> >>>>> I'm quite strongly against hints that change query semantics, but if
> >> we
> >>>>> disable this by default I would be (reluctantly) OK with the feature.
> >>>>> Companies that create deployments or set up the SQL environment for
> >>>>> users can enable the feature if they want.
> >>>>>
> >>>>> But yes, I also agree that we don't need whitelisting/blacklisting,
> >>>>> which makes this a lot easier to do.
> >>>>>
> >>>>> Best,
> >>>>> Aljoscha
> >>>>>
> >>>>> On 06.04.20 04:27, Danny Chan wrote:
> >>>>>> Hi, everyone ~
> >>>>>>
> >>>>>> @Aljoscha @Timo
> >>>>>>
> >>>>>>> I think we're designing ourselves into ever more complicated
> >> corners
> >>>>>> here
> >>>>>>
> >>>>>> I kindly agree that, personally didn't see strong reasons why we
> >>>>>> should limit on each connector properties:
> >>>>>>
> >>>>>> • we can define any table options for CREATE TABLE, why we treat the
> >>>>>> dynamic options differently, we never consider any security problems
> >>>>>> when create table, we should not either for dynamic table options
> >>>>>> • If we do not have whitelist properties or blacklist properties,
> >> the
> >>>>>> table source creation work would be much easier, just used the
> >> merged
> >>>>>> options. There is no need to modify each connector to decide which
> >>>>>> options could be overridden and how we merge them(the merge work is
> >>>>>> redundant).
> >>>>>> • @Timo, how about we support another interface
> >>>>>> `TableSourceFactory#Context.getExecutionOptions`, we always use this
> >>>>>> interface to get the options to create our table source. There is no
> >>>>>> need to copy the catalog table itselt, we just need to generate our
> >>>>>> Context correctly.
> >>>>>> • @Aljoscha I agree to have a global config option, but I disagree
> >> to
> >>>>>> default disable it, a global default config would break the user
> >>>>>> experience too much, especially when user want to modify the options
> >>>>>> in a ad-hoc way.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> I suggest to remove `TableSourceFactory#supportedHintOptions` or
> >>>>>> `TableSourceFactory#forbiddenHintOptions` based on the fact that we
> >>>>>> does not have black/white list for CREATE TABLE at all at lease for
> >>>>>> current codebase.
> >>>>>>
> >>>>>>
> >>>>>> @Timo (i have replied offline but allows to represent it here again)
> >>>>>>
> >>>>>> The `TableSourceFactory#supportedHintOptions` doesn't work well for
> >> 3
> >>>>>> reasons compared to `TableSourceFactory#forbiddenHintOptions`:
> >>>>>> 1. For key with wildcard, like connector.property.* , use a
> >> blacklist
> >>>>>> make us have the ability to disable some of the keys under that,
> >> i.e.
> >>>>>> connector.property.key1 , a whitelist can only match with prefix
> >>>>>>
> >>>>>> 2. We want the connectors to have the ability to disable format type
> >>>>>> switch format.type but allows all the other properties, e.g.
> >> format.*
> >>>>>> without format.type(let's call it SET_B), if we use the whitelist,
> >> we
> >>>>>> have to enumerate all the specific format keys start with format
> >>>>>> (SET_B), but with the old connector factories, we have no idea what
> >>>>>> specific format keys it supports(there is either a format.* or
> >>> nothing).
> >>>>>>
> >>>>>> 3. Except the cases for 1 and 2, for normal keys(no wildcard), the
> >>>>>> blacklist and whitelist has the same expressiveness, use blacklist
> >>>>>> makes the code not too verbose to enumerate all the duplicate keys
> >>>>>> with #supportedKeys .(Not very strong reason, but i think as a
> >>>>>> connector developer, it makes sense)
> >>>>>>
> >>>>>> Best,
> >>>>>> Danny Chan
> >>>>>> 在 2020年4月3日 +0800 PM11:28,Timo Walther <tw...@apache.org>,写道:
> >>>>>>> Hi everyone,
> >>>>>>>
> >>>>>>> @Aljoscha: I disagree with your approach because a `Catalog` can
> >>> return
> >>>>>>> a custom factory that is not using any properties. The hinting must
> >>> be
> >>>>>>> transparent to a factory. We should NOT modify the metadata
> >>>>>>> `CatalogTable` at any point in time after the catalog.
> >>>>>>>
> >>>>>>> @Danny, @Jingsong: How about we stick to the original design that
> >> we
> >>>>>>> wanted to vote on but use:
> >>>>>>>
> >>>>>>> Set<String> supportedHintProperties()
> >>>>>>>
> >>>>>>> This fits better to the old factory design. And for the new FLIP-95
> >>>>>>> factories we will use `ConfigOption` and provide good utilities for
> >>>>>>> merging with hints etc.
> >>>>>>>
> >>>>>>> We can allow `"format.*"` in `supportedHintProperties()` to allow
> >>>>>>> hinting in formats.
> >>>>>>>
> >>>>>>> What do you think?
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Timo
> >>>>>>>
> >>>>>>>
> >>>>>>> On 02.04.20 16:24, Aljoscha Krettek wrote:
> >>>>>>>> I think we're designing ourselves into ever more complicated
> >> corners
> >>>>>>>> here. Maybe we need to take a step back and reconsider. What would
> >>> you
> >>>>>>>> think about this (somewhat) simpler proposal:
> >>>>>>>>
> >>>>>>>> - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or
> >>>>>>>> CONNECTOR_PROPERTIES, depending on what naming we want to have for
> >>>> this
> >>>>>>>> in the future. This will simply overwrite all connector
> >> properties,
> >>>> the
> >>>>>>>> table factories don't know about hints but simply work with the
> >>>>>>>> properties that they are given
> >>>>>>>>
> >>>>>>>> - this special hint is disabled by default and can be activated
> >>> with a
> >>>>>>>> global option "foo.bazzle.connector-hints" (or something like
> >> this)
> >>>>>>>> which has a warning that describes that this can change query
> >>>> semantics
> >>>>>>>> etc.
> >>>>>>>>
> >>>>>>>> That's it. This makes connector implementations a lot easier while
> >>>>>>>> still
> >>>>>>>> allowing inline configuration.
> >>>>>>>>
> >>>>>>>> I still don't like using hint syntax at all for this, because I
> >>>>>>>> strongly
> >>>>>>>> maintain that hints should not change query syntax. In general
> >> using
> >>>>>>>> hints should be kept to a minimum because they usually point to
> >>>>>>>> shortcomings in the system.
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Aljoscha
> >>>>>>>>
> >>>>>>>> On 02.04.20 06:06, Jingsong Li wrote:
> >>>>>>>>> Hi Dawid,
> >>>>>>>>>
> >>>>>>>>>> When a factory is instantiated it has access to the
> >> CatalogTable,
> >>>>>>>>> therefore it has access to all the original properties. In turn
> >> it
> >>>>>>>>> knows
> >>>>>>>>> the original format and can call
> >>>> FormatFactory#supportedHintOptions().
> >>>>>>>>>
> >>>>>>>>> Factory can only get CatalogTable when creating source or sink,
> >>>>>>>>> right? IIUC, TableFactory may be stateless too.
> >>>>>>>>> When invoking SourceFactory#supportedHintOptions(), it can not
> >>>>>>>>> get CatalogTable, so it is impossible to create FormatFactory?
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Jingsong Lee
> >>>>>>>>>
> >>>>>>>>> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yuzhao.cyz@gmail.com
> >>>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi, Dawid, thanks so much for the detail suggestions ~
> >>>>>>>>>>
> >>>>>>>>>> 1. Regarding the motivation:
> >>>>>>>>>>
> >>>>>>>>>> I agree it's not a good suggested way based on the fact that we
> >>> have
> >>>>>>>>>> better solution, but i think we can support override that as
> >> long
> >>>>>>>>>> as it
> >>>>>>>>>> exists as one of the the table options. I would remove if from
> >> the
> >>>>>>>>>> motication part.
> >>>>>>>>>>
> >>>>>>>>>> 2. The options passes around during sql-to-rel conversion, right
> >>>>>>>>>> after we
> >>>>>>>>>> generate the RelOptTable (CatalogSourceTable#toRel in Flink),
> >> this
> >>>> is
> >>>>>>>>>> indeed a push way method at least in the RelOptTable layer, then
> >>>>>>>>>> we hand
> >>>>>>>>>> over the options to TableSourceFactory with our own context,
> >> which
> >>>> is
> >>>>>>>>>> fine
> >>>>>>>>>> becuase TableSourceFactory#Context is the contact to pass around
> >>>>>>>>>> these
> >>>>>>>>>> table-about variables.
> >>>>>>>>>>
> >>>>>>>>>> 3. "We should not end up with an extreme example where we can
> >>>>>>>>>> change the
> >>>>>>>>>> connector type", i totally agree that, and i have listed the
> >>>>>>>>>> "connector.type" as forbidden attribute in the WIKI. As for the
> >>>>>>>>>> format, i
> >>>>>>>>>> think the connector itself can/should control whether to
> >> override
> >>>> the
> >>>>>>>>>> "format.type", that is one of the reason i change the
> >>>>>>>>>> TableSourceFactory#supportedHintOpitons to
> >>>>>>>>>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can
> >>>>>>>>>> limit the
> >>>>>>>>>> format keys we want conveniently.
> >>>>>>>>>>
> >>>>>>>>>> 4. SQL Hints syntax.
> >>>>>>>>>>
> >>>>>>>>>>> I think the k and v in the hint_item should be QUOTED_STRING
> >> (not
> >>>>>>>>>>> sure
> >>>>>>>>>> if it is equivalent to string_literal).
> >>>>>>>>>>
> >>>>>>>>>> I disagree, we at least should keep sync with our DDL: use the
> >>>> string
> >>>>>>>>>> literal as the key. We did also support the simple identifier
> >>>> because
> >>>>>>>>>> this
> >>>>>>>>>> is the common hint syntax from Calcite, it does not hurt
> >> anything
> >>>> for
> >>>>>>>>>> the
> >>>>>>>>>> OPTIONS hint, the unsupported keys would validate fails.(If you
> >>>> think
> >>>>>>>>>> that
> >>>>>>>>>> may cause some confuse, i can make the syntax pluggable for each
> >>>>>>>>>> hint in
> >>>>>>>>>> CALCITE 1.23)
> >>>>>>>>>>
> >>>>>>>>>> We only supports OPTIONS hint in the FLIP, and i have changed
> >> the
> >>>>>>>>>> title to
> >>>>>>>>>> "Supports dynamic table options", would make it more clear in
> >> the
> >>>>>>>>>> WIKI.
> >>>>>>>>>>
> >>>>>>>>>> 5. Yes, we also have this concerns from our offline discussion,
> >>>>>>>>>> that is
> >>>>>>>>>> one of the reason, why i change the
> >>>>>>>>>> TableSourceFactory#supportedHintOpitons
> >>>>>>>>>> to TableSourceFactory#forbiddenHintOpitons, i do not choose
> >>>>>>>>>> Set<String>
> >>>>>>>>>> supportedHintOptionKeys() with wildcard for 2 reasons:
> >>>>>>>>>>
> >>>>>>>>>>      - The wildcard is still not descriptive, we can still not
> >>>>>>>>>> forbidden one
> >>>>>>>>>> of the properties among the wildcard properties, we can not
> >> enable
> >>>> or
> >>>>>>>>>> disable them totally
> >>>>>>>>>>      - ConfigOption is our new structure for keys, and it does
> >> not
> >>>>>>>>>> support
> >>>>>>>>>> wildcard yet.
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Danny Chan
> >>>>>>>>>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz
> >>>>>>>>>> <dw...@apache.org>,写道:
> >>>>>>>>>>> Hi,
> >>>>>>>>>>> Few comments from my side:
> >>>>>>>>>>> 1. Regarding the motivation:
> >>>>>>>>>>> I think the example with changing the update-mode is not a good
> >>>>>>>>>>> one. In
> >>>>>>>>>> the long term this should be done with EMIT CHANGELOG (discussed
> >>> in
> >>>>>>>>>> FLIP-105).
> >>>>>>>>>>> Nitpicking: I would mention that it is rather for
> >>> debugging/ad-hoc
> >>>>>>>>>> solution. I think this should not be a recommended way for
> >>>> production
> >>>>>>>>>> use
> >>>>>>>>>> cases as it bypasses the Catalog, which should be the source of
> >>>>>>>>>> truth.
> >>>>>>>>>>> 2. I could not understand how the additional options will be
> >>>>>>>>>>> passed to
> >>>>>>>>>> the TableSourceFactory. Could you elaborate a bit more on that?
> >> I
> >>>> see
> >>>>>>>>>> there
> >>>>>>>>>> is a Context interface that gives the options. But cannot find a
> >>> way
> >>>>>>>>>> to get
> >>>>>>>>>> the context itself in the factory. Moreover I think it would
> >> make
> >>>>>>>>>> more
> >>>>>>>>>> sense to have rather a push based approach here. Something like
> >>>>>>>>>> applyOptions(ReadableConfig) method.
> >>>>>>>>>>> 3. As for the concerns Jingsong raised in the voting thread. I
> >>>>>>>>>>> think it
> >>>>>>>>>> is not a big problem, but I agree this should be also
> >> described. I
> >>>>>>>>>> disagree
> >>>>>>>>>> with "Connector don't know format information in TableFactory
> >>> before
> >>>>>>>>>> obtains real properties, so it can not list any format
> >>>>>>>>>> `supportedHintOptions`".
> >>>>>>>>>>> When a factory is instantiated it has access to the
> >> CatalogTable,
> >>>>>>>>>> therefore it has access to all the original properties. In turn
> >> it
> >>>>>>>>>> knows
> >>>>>>>>>> the original format and can call
> >>>>>>>>>> FormatFactory#supportedHintOptions().
> >>>>>>>>>>> The only case when this would not work would be if we allow
> >>>> changing
> >>>>>>>>>>> the
> >>>>>>>>>> format of the Table (e.g. from avro to parquet), which does not
> >>>> sound
> >>>>>>>>>> like
> >>>>>>>>>> a good idea to me. I think this feature should not end up as a
> >> way
> >>>> to
> >>>>>>>>>> declare a whole table inline in a SQL query, but should rather
> >> be
> >>> a
> >>>>>>>>>> simple
> >>>>>>>>>> way for debugging queries. We should not end up with an extreme
> >>>>>>>>>> example
> >>>>>>>>>> where we do:
> >>>>>>>>>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
> >>>>>>>>>> 'format.type' = 'json', ....) */
> >>>>>>>>>>> 4. SQL Hints syntax.
> >>>>>>>>>>> I think the k and v in the hint_item should be QUOTED_STRING
> >> (not
> >>>>>>>>>>> sure
> >>>>>>>>>> if it is equivalent to string_literal). I think we should not
> >> use
> >>>>>>>>>> simple_identifier because this implies that we cannot use e.g.
> >> any
> >>>>>>>>>> SQL
> >>>>>>>>>> keywords. Anyway it has nothing to do with identifiers. If I am
> >>> not
> >>>>>>>>>> mistaken it is also how the options in the CREATE statement are
> >>>>>>>>>> implemented.
> >>>>>>>>>>> What is the purpose of the remaining hint_item:
> >>> hint_name(hint_opt
> >>>> [
> >>>>>>>>>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a
> >>>>>>>>>> feeling it
> >>>>>>>>>> does also suggests to support the whole Apache Calcite hint
> >> system
> >>>>>>>>>> without
> >>>>>>>>>> specifying that explicitly. Is the intention of the FLIP to
> >>> support
> >>>>>>>>>> choosing e.g. JOIN strategies through hints already? If it is so
> >>> it
> >>>>>>>>>> should
> >>>>>>>>>> be mentioned in the FLIP, imo.
> >>>>>>>>>>> 5. I think something does not work around the
> >>>>>>>>>>> supportedHintOptions and
> >>>>>>>>>> wildcards. How do you want to represent a wildcard key as a
> >>>>>>>>>> ConfigOption? I
> >>>>>>>>>> am not sure about that, just a though, maybe it make sense to
> >> have
> >>>>>>>>>> rather
> >>>>>>>>>> Set<String> supportedHintOptionKeys()?
> >>>>>>>>>>> Best,
> >>>>>>>>>>> Dawid
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>>>
> >>>
> >>
> >
>
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@apache.org>.
Thanks for the update Danny. +1 from my side.

Regards,
Timo


On 07.04.20 13:25, Danny Chan wrote:
> Hi, every ~
> 
> It seems that we all agree to drop the idea for white/black list for each
> connector, and have a global config option to default disable this feature.
> 
> I have also discussed with Timo and Jark about the interface
> TableSourceTable.Context.getExecutionOptions and finally we decide to
> introduce a new interface CatalogTable#copy(Map<String, String>) to support
> re-generate the table with new table options.
> 
> So let me summarize the current design broadly again:
> 
>     - Use the syntax /*+ OPTIONS('k1' = 'v1', 'k2' = 'v2') to describe the
>     dynamic table options
>     - There is no constraint on which option key can be used in the OPTIONS,
>     that means, any option key is allowed, the factory would to the validation
>     work finally
>     - Introduce method CatalogTable#copy, we use this method to regenerate a
>     new CatalogTable to find a table factory and creates table source/sink
>     - There is a global config option to default disable this feature (if
>     user uses OPTIONS, an exception throws to tell open the option)
> 
> I have updated the WIKI
> <https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL>,
> look forward to your suggestions ~
> 
> Jark Wu <im...@gmail.com> 于2020年4月7日周二 上午11:24写道:
> 
>> I'm fine to disable this feature by default and avoid
>> whitelisting/blacklisting. This simplifies a lot of things.
>>
>> Regarding to TableSourceFactory#Context#getExecutionOptions, do we really
>> need this interface?
>> Should the connector factory be aware of the properties is merged with
>> hints or not?
>> What's the problem if we always get properties from
>> `CatalogTable#getProperties`?
>>
>> Best,
>> Jark
>>
>> On Tue, 7 Apr 2020 at 10:39, Kurt Young <yk...@gmail.com> wrote:
>>
>>> Sounds like a reasonable compromise, disabling this feature by default
>> is a
>>> way to protect
>>> the vulnerability, and we can simplify the design quite a lot. We can
>>> gather some users'
>>> feedback to see whether further protections are necessary in the future.
>>>
>>> Best,
>>> Kurt
>>>
>>>
>>> On Mon, Apr 6, 2020 at 11:49 PM Timo Walther <tw...@apache.org> wrote:
>>>
>>>> I agree with Aljoscha. The length of this thread shows that this is
>>>> highly controversal. I think nobody really likes this feature 100% but
>>>> we could not find a better solution. I would consider it as a
>>>> nice-to-have improvement during a notebook/debugging session.
>>>>
>>>> I would accept avoiding whitelisting/blacklisting if the feature is
>>>> disabled by default. And we make the merged properties available in a
>>>> separate TableSourceFactory#Context#getExecutionOptions as Danny
>>> proposed.
>>>>
>>>> What do you think?
>>>>
>>>> Thanks,
>>>> Timo
>>>>
>>>>
>>>> On 06.04.20 09:59, Aljoscha Krettek wrote:
>>>>> The reason I'm saying it should be disabled by default is that this
>>> uses
>>>>> hint syntax, and hints should really not change query semantics.
>>>>>
>>>>> I'm quite strongly against hints that change query semantics, but if
>> we
>>>>> disable this by default I would be (reluctantly) OK with the feature.
>>>>> Companies that create deployments or set up the SQL environment for
>>>>> users can enable the feature if they want.
>>>>>
>>>>> But yes, I also agree that we don't need whitelisting/blacklisting,
>>>>> which makes this a lot easier to do.
>>>>>
>>>>> Best,
>>>>> Aljoscha
>>>>>
>>>>> On 06.04.20 04:27, Danny Chan wrote:
>>>>>> Hi, everyone ~
>>>>>>
>>>>>> @Aljoscha @Timo
>>>>>>
>>>>>>> I think we're designing ourselves into ever more complicated
>> corners
>>>>>> here
>>>>>>
>>>>>> I kindly agree that, personally didn't see strong reasons why we
>>>>>> should limit on each connector properties:
>>>>>>
>>>>>> • we can define any table options for CREATE TABLE, why we treat the
>>>>>> dynamic options differently, we never consider any security problems
>>>>>> when create table, we should not either for dynamic table options
>>>>>> • If we do not have whitelist properties or blacklist properties,
>> the
>>>>>> table source creation work would be much easier, just used the
>> merged
>>>>>> options. There is no need to modify each connector to decide which
>>>>>> options could be overridden and how we merge them(the merge work is
>>>>>> redundant).
>>>>>> • @Timo, how about we support another interface
>>>>>> `TableSourceFactory#Context.getExecutionOptions`, we always use this
>>>>>> interface to get the options to create our table source. There is no
>>>>>> need to copy the catalog table itselt, we just need to generate our
>>>>>> Context correctly.
>>>>>> • @Aljoscha I agree to have a global config option, but I disagree
>> to
>>>>>> default disable it, a global default config would break the user
>>>>>> experience too much, especially when user want to modify the options
>>>>>> in a ad-hoc way.
>>>>>>
>>>>>>
>>>>>>
>>>>>> I suggest to remove `TableSourceFactory#supportedHintOptions` or
>>>>>> `TableSourceFactory#forbiddenHintOptions` based on the fact that we
>>>>>> does not have black/white list for CREATE TABLE at all at lease for
>>>>>> current codebase.
>>>>>>
>>>>>>
>>>>>> @Timo (i have replied offline but allows to represent it here again)
>>>>>>
>>>>>> The `TableSourceFactory#supportedHintOptions` doesn't work well for
>> 3
>>>>>> reasons compared to `TableSourceFactory#forbiddenHintOptions`:
>>>>>> 1. For key with wildcard, like connector.property.* , use a
>> blacklist
>>>>>> make us have the ability to disable some of the keys under that,
>> i.e.
>>>>>> connector.property.key1 , a whitelist can only match with prefix
>>>>>>
>>>>>> 2. We want the connectors to have the ability to disable format type
>>>>>> switch format.type but allows all the other properties, e.g.
>> format.*
>>>>>> without format.type(let's call it SET_B), if we use the whitelist,
>> we
>>>>>> have to enumerate all the specific format keys start with format
>>>>>> (SET_B), but with the old connector factories, we have no idea what
>>>>>> specific format keys it supports(there is either a format.* or
>>> nothing).
>>>>>>
>>>>>> 3. Except the cases for 1 and 2, for normal keys(no wildcard), the
>>>>>> blacklist and whitelist has the same expressiveness, use blacklist
>>>>>> makes the code not too verbose to enumerate all the duplicate keys
>>>>>> with #supportedKeys .(Not very strong reason, but i think as a
>>>>>> connector developer, it makes sense)
>>>>>>
>>>>>> Best,
>>>>>> Danny Chan
>>>>>> 在 2020年4月3日 +0800 PM11:28,Timo Walther <tw...@apache.org>,写道:
>>>>>>> Hi everyone,
>>>>>>>
>>>>>>> @Aljoscha: I disagree with your approach because a `Catalog` can
>>> return
>>>>>>> a custom factory that is not using any properties. The hinting must
>>> be
>>>>>>> transparent to a factory. We should NOT modify the metadata
>>>>>>> `CatalogTable` at any point in time after the catalog.
>>>>>>>
>>>>>>> @Danny, @Jingsong: How about we stick to the original design that
>> we
>>>>>>> wanted to vote on but use:
>>>>>>>
>>>>>>> Set<String> supportedHintProperties()
>>>>>>>
>>>>>>> This fits better to the old factory design. And for the new FLIP-95
>>>>>>> factories we will use `ConfigOption` and provide good utilities for
>>>>>>> merging with hints etc.
>>>>>>>
>>>>>>> We can allow `"format.*"` in `supportedHintProperties()` to allow
>>>>>>> hinting in formats.
>>>>>>>
>>>>>>> What do you think?
>>>>>>>
>>>>>>> Regards,
>>>>>>> Timo
>>>>>>>
>>>>>>>
>>>>>>> On 02.04.20 16:24, Aljoscha Krettek wrote:
>>>>>>>> I think we're designing ourselves into ever more complicated
>> corners
>>>>>>>> here. Maybe we need to take a step back and reconsider. What would
>>> you
>>>>>>>> think about this (somewhat) simpler proposal:
>>>>>>>>
>>>>>>>> - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or
>>>>>>>> CONNECTOR_PROPERTIES, depending on what naming we want to have for
>>>> this
>>>>>>>> in the future. This will simply overwrite all connector
>> properties,
>>>> the
>>>>>>>> table factories don't know about hints but simply work with the
>>>>>>>> properties that they are given
>>>>>>>>
>>>>>>>> - this special hint is disabled by default and can be activated
>>> with a
>>>>>>>> global option "foo.bazzle.connector-hints" (or something like
>> this)
>>>>>>>> which has a warning that describes that this can change query
>>>> semantics
>>>>>>>> etc.
>>>>>>>>
>>>>>>>> That's it. This makes connector implementations a lot easier while
>>>>>>>> still
>>>>>>>> allowing inline configuration.
>>>>>>>>
>>>>>>>> I still don't like using hint syntax at all for this, because I
>>>>>>>> strongly
>>>>>>>> maintain that hints should not change query syntax. In general
>> using
>>>>>>>> hints should be kept to a minimum because they usually point to
>>>>>>>> shortcomings in the system.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Aljoscha
>>>>>>>>
>>>>>>>> On 02.04.20 06:06, Jingsong Li wrote:
>>>>>>>>> Hi Dawid,
>>>>>>>>>
>>>>>>>>>> When a factory is instantiated it has access to the
>> CatalogTable,
>>>>>>>>> therefore it has access to all the original properties. In turn
>> it
>>>>>>>>> knows
>>>>>>>>> the original format and can call
>>>> FormatFactory#supportedHintOptions().
>>>>>>>>>
>>>>>>>>> Factory can only get CatalogTable when creating source or sink,
>>>>>>>>> right? IIUC, TableFactory may be stateless too.
>>>>>>>>> When invoking SourceFactory#supportedHintOptions(), it can not
>>>>>>>>> get CatalogTable, so it is impossible to create FormatFactory?
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Jingsong Lee
>>>>>>>>>
>>>>>>>>> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yuzhao.cyz@gmail.com
>>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi, Dawid, thanks so much for the detail suggestions ~
>>>>>>>>>>
>>>>>>>>>> 1. Regarding the motivation:
>>>>>>>>>>
>>>>>>>>>> I agree it's not a good suggested way based on the fact that we
>>> have
>>>>>>>>>> better solution, but i think we can support override that as
>> long
>>>>>>>>>> as it
>>>>>>>>>> exists as one of the the table options. I would remove if from
>> the
>>>>>>>>>> motication part.
>>>>>>>>>>
>>>>>>>>>> 2. The options passes around during sql-to-rel conversion, right
>>>>>>>>>> after we
>>>>>>>>>> generate the RelOptTable (CatalogSourceTable#toRel in Flink),
>> this
>>>> is
>>>>>>>>>> indeed a push way method at least in the RelOptTable layer, then
>>>>>>>>>> we hand
>>>>>>>>>> over the options to TableSourceFactory with our own context,
>> which
>>>> is
>>>>>>>>>> fine
>>>>>>>>>> becuase TableSourceFactory#Context is the contact to pass around
>>>>>>>>>> these
>>>>>>>>>> table-about variables.
>>>>>>>>>>
>>>>>>>>>> 3. "We should not end up with an extreme example where we can
>>>>>>>>>> change the
>>>>>>>>>> connector type", i totally agree that, and i have listed the
>>>>>>>>>> "connector.type" as forbidden attribute in the WIKI. As for the
>>>>>>>>>> format, i
>>>>>>>>>> think the connector itself can/should control whether to
>> override
>>>> the
>>>>>>>>>> "format.type", that is one of the reason i change the
>>>>>>>>>> TableSourceFactory#supportedHintOpitons to
>>>>>>>>>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can
>>>>>>>>>> limit the
>>>>>>>>>> format keys we want conveniently.
>>>>>>>>>>
>>>>>>>>>> 4. SQL Hints syntax.
>>>>>>>>>>
>>>>>>>>>>> I think the k and v in the hint_item should be QUOTED_STRING
>> (not
>>>>>>>>>>> sure
>>>>>>>>>> if it is equivalent to string_literal).
>>>>>>>>>>
>>>>>>>>>> I disagree, we at least should keep sync with our DDL: use the
>>>> string
>>>>>>>>>> literal as the key. We did also support the simple identifier
>>>> because
>>>>>>>>>> this
>>>>>>>>>> is the common hint syntax from Calcite, it does not hurt
>> anything
>>>> for
>>>>>>>>>> the
>>>>>>>>>> OPTIONS hint, the unsupported keys would validate fails.(If you
>>>> think
>>>>>>>>>> that
>>>>>>>>>> may cause some confuse, i can make the syntax pluggable for each
>>>>>>>>>> hint in
>>>>>>>>>> CALCITE 1.23)
>>>>>>>>>>
>>>>>>>>>> We only supports OPTIONS hint in the FLIP, and i have changed
>> the
>>>>>>>>>> title to
>>>>>>>>>> "Supports dynamic table options", would make it more clear in
>> the
>>>>>>>>>> WIKI.
>>>>>>>>>>
>>>>>>>>>> 5. Yes, we also have this concerns from our offline discussion,
>>>>>>>>>> that is
>>>>>>>>>> one of the reason, why i change the
>>>>>>>>>> TableSourceFactory#supportedHintOpitons
>>>>>>>>>> to TableSourceFactory#forbiddenHintOpitons, i do not choose
>>>>>>>>>> Set<String>
>>>>>>>>>> supportedHintOptionKeys() with wildcard for 2 reasons:
>>>>>>>>>>
>>>>>>>>>>      - The wildcard is still not descriptive, we can still not
>>>>>>>>>> forbidden one
>>>>>>>>>> of the properties among the wildcard properties, we can not
>> enable
>>>> or
>>>>>>>>>> disable them totally
>>>>>>>>>>      - ConfigOption is our new structure for keys, and it does
>> not
>>>>>>>>>> support
>>>>>>>>>> wildcard yet.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Danny Chan
>>>>>>>>>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz
>>>>>>>>>> <dw...@apache.org>,写道:
>>>>>>>>>>> Hi,
>>>>>>>>>>> Few comments from my side:
>>>>>>>>>>> 1. Regarding the motivation:
>>>>>>>>>>> I think the example with changing the update-mode is not a good
>>>>>>>>>>> one. In
>>>>>>>>>> the long term this should be done with EMIT CHANGELOG (discussed
>>> in
>>>>>>>>>> FLIP-105).
>>>>>>>>>>> Nitpicking: I would mention that it is rather for
>>> debugging/ad-hoc
>>>>>>>>>> solution. I think this should not be a recommended way for
>>>> production
>>>>>>>>>> use
>>>>>>>>>> cases as it bypasses the Catalog, which should be the source of
>>>>>>>>>> truth.
>>>>>>>>>>> 2. I could not understand how the additional options will be
>>>>>>>>>>> passed to
>>>>>>>>>> the TableSourceFactory. Could you elaborate a bit more on that?
>> I
>>>> see
>>>>>>>>>> there
>>>>>>>>>> is a Context interface that gives the options. But cannot find a
>>> way
>>>>>>>>>> to get
>>>>>>>>>> the context itself in the factory. Moreover I think it would
>> make
>>>>>>>>>> more
>>>>>>>>>> sense to have rather a push based approach here. Something like
>>>>>>>>>> applyOptions(ReadableConfig) method.
>>>>>>>>>>> 3. As for the concerns Jingsong raised in the voting thread. I
>>>>>>>>>>> think it
>>>>>>>>>> is not a big problem, but I agree this should be also
>> described. I
>>>>>>>>>> disagree
>>>>>>>>>> with "Connector don't know format information in TableFactory
>>> before
>>>>>>>>>> obtains real properties, so it can not list any format
>>>>>>>>>> `supportedHintOptions`".
>>>>>>>>>>> When a factory is instantiated it has access to the
>> CatalogTable,
>>>>>>>>>> therefore it has access to all the original properties. In turn
>> it
>>>>>>>>>> knows
>>>>>>>>>> the original format and can call
>>>>>>>>>> FormatFactory#supportedHintOptions().
>>>>>>>>>>> The only case when this would not work would be if we allow
>>>> changing
>>>>>>>>>>> the
>>>>>>>>>> format of the Table (e.g. from avro to parquet), which does not
>>>> sound
>>>>>>>>>> like
>>>>>>>>>> a good idea to me. I think this feature should not end up as a
>> way
>>>> to
>>>>>>>>>> declare a whole table inline in a SQL query, but should rather
>> be
>>> a
>>>>>>>>>> simple
>>>>>>>>>> way for debugging queries. We should not end up with an extreme
>>>>>>>>>> example
>>>>>>>>>> where we do:
>>>>>>>>>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
>>>>>>>>>> 'format.type' = 'json', ....) */
>>>>>>>>>>> 4. SQL Hints syntax.
>>>>>>>>>>> I think the k and v in the hint_item should be QUOTED_STRING
>> (not
>>>>>>>>>>> sure
>>>>>>>>>> if it is equivalent to string_literal). I think we should not
>> use
>>>>>>>>>> simple_identifier because this implies that we cannot use e.g.
>> any
>>>>>>>>>> SQL
>>>>>>>>>> keywords. Anyway it has nothing to do with identifiers. If I am
>>> not
>>>>>>>>>> mistaken it is also how the options in the CREATE statement are
>>>>>>>>>> implemented.
>>>>>>>>>>> What is the purpose of the remaining hint_item:
>>> hint_name(hint_opt
>>>> [
>>>>>>>>>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a
>>>>>>>>>> feeling it
>>>>>>>>>> does also suggests to support the whole Apache Calcite hint
>> system
>>>>>>>>>> without
>>>>>>>>>> specifying that explicitly. Is the intention of the FLIP to
>>> support
>>>>>>>>>> choosing e.g. JOIN strategies through hints already? If it is so
>>> it
>>>>>>>>>> should
>>>>>>>>>> be mentioned in the FLIP, imo.
>>>>>>>>>>> 5. I think something does not work around the
>>>>>>>>>>> supportedHintOptions and
>>>>>>>>>> wildcards. How do you want to represent a wildcard key as a
>>>>>>>>>> ConfigOption? I
>>>>>>>>>> am not sure about that, just a though, maybe it make sense to
>> have
>>>>>>>>>> rather
>>>>>>>>>> Set<String> supportedHintOptionKeys()?
>>>>>>>>>>> Best,
>>>>>>>>>>> Dawid
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>>>
>>>
>>
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <da...@apache.org>.
Hi, every ~

It seems that we all agree to drop the idea for white/black list for each
connector, and have a global config option to default disable this feature.

I have also discussed with Timo and Jark about the interface
TableSourceTable.Context.getExecutionOptions and finally we decide to
introduce a new interface CatalogTable#copy(Map<String, String>) to support
re-generate the table with new table options.

So let me summarize the current design broadly again:

   - Use the syntax /*+ OPTIONS('k1' = 'v1', 'k2' = 'v2') to describe the
   dynamic table options
   - There is no constraint on which option key can be used in the OPTIONS,
   that means, any option key is allowed, the factory would to the validation
   work finally
   - Introduce method CatalogTable#copy, we use this method to regenerate a
   new CatalogTable to find a table factory and creates table source/sink
   - There is a global config option to default disable this feature (if
   user uses OPTIONS, an exception throws to tell open the option)

I have updated the WIKI
<https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL>,
look forward to your suggestions ~

Jark Wu <im...@gmail.com> 于2020年4月7日周二 上午11:24写道:

> I'm fine to disable this feature by default and avoid
> whitelisting/blacklisting. This simplifies a lot of things.
>
> Regarding to TableSourceFactory#Context#getExecutionOptions, do we really
> need this interface?
> Should the connector factory be aware of the properties is merged with
> hints or not?
> What's the problem if we always get properties from
> `CatalogTable#getProperties`?
>
> Best,
> Jark
>
> On Tue, 7 Apr 2020 at 10:39, Kurt Young <yk...@gmail.com> wrote:
>
> > Sounds like a reasonable compromise, disabling this feature by default
> is a
> > way to protect
> > the vulnerability, and we can simplify the design quite a lot. We can
> > gather some users'
> > feedback to see whether further protections are necessary in the future.
> >
> > Best,
> > Kurt
> >
> >
> > On Mon, Apr 6, 2020 at 11:49 PM Timo Walther <tw...@apache.org> wrote:
> >
> > > I agree with Aljoscha. The length of this thread shows that this is
> > > highly controversal. I think nobody really likes this feature 100% but
> > > we could not find a better solution. I would consider it as a
> > > nice-to-have improvement during a notebook/debugging session.
> > >
> > > I would accept avoiding whitelisting/blacklisting if the feature is
> > > disabled by default. And we make the merged properties available in a
> > > separate TableSourceFactory#Context#getExecutionOptions as Danny
> > proposed.
> > >
> > > What do you think?
> > >
> > > Thanks,
> > > Timo
> > >
> > >
> > > On 06.04.20 09:59, Aljoscha Krettek wrote:
> > > > The reason I'm saying it should be disabled by default is that this
> > uses
> > > > hint syntax, and hints should really not change query semantics.
> > > >
> > > > I'm quite strongly against hints that change query semantics, but if
> we
> > > > disable this by default I would be (reluctantly) OK with the feature.
> > > > Companies that create deployments or set up the SQL environment for
> > > > users can enable the feature if they want.
> > > >
> > > > But yes, I also agree that we don't need whitelisting/blacklisting,
> > > > which makes this a lot easier to do.
> > > >
> > > > Best,
> > > > Aljoscha
> > > >
> > > > On 06.04.20 04:27, Danny Chan wrote:
> > > >> Hi, everyone ~
> > > >>
> > > >> @Aljoscha @Timo
> > > >>
> > > >>> I think we're designing ourselves into ever more complicated
> corners
> > > >> here
> > > >>
> > > >> I kindly agree that, personally didn't see strong reasons why we
> > > >> should limit on each connector properties:
> > > >>
> > > >> • we can define any table options for CREATE TABLE, why we treat the
> > > >> dynamic options differently, we never consider any security problems
> > > >> when create table, we should not either for dynamic table options
> > > >> • If we do not have whitelist properties or blacklist properties,
> the
> > > >> table source creation work would be much easier, just used the
> merged
> > > >> options. There is no need to modify each connector to decide which
> > > >> options could be overridden and how we merge them(the merge work is
> > > >> redundant).
> > > >> • @Timo, how about we support another interface
> > > >> `TableSourceFactory#Context.getExecutionOptions`, we always use this
> > > >> interface to get the options to create our table source. There is no
> > > >> need to copy the catalog table itselt, we just need to generate our
> > > >> Context correctly.
> > > >> • @Aljoscha I agree to have a global config option, but I disagree
> to
> > > >> default disable it, a global default config would break the user
> > > >> experience too much, especially when user want to modify the options
> > > >> in a ad-hoc way.
> > > >>
> > > >>
> > > >>
> > > >> I suggest to remove `TableSourceFactory#supportedHintOptions` or
> > > >> `TableSourceFactory#forbiddenHintOptions` based on the fact that we
> > > >> does not have black/white list for CREATE TABLE at all at lease for
> > > >> current codebase.
> > > >>
> > > >>
> > > >> @Timo (i have replied offline but allows to represent it here again)
> > > >>
> > > >> The `TableSourceFactory#supportedHintOptions` doesn't work well for
> 3
> > > >> reasons compared to `TableSourceFactory#forbiddenHintOptions`:
> > > >> 1. For key with wildcard, like connector.property.* , use a
> blacklist
> > > >> make us have the ability to disable some of the keys under that,
> i.e.
> > > >> connector.property.key1 , a whitelist can only match with prefix
> > > >>
> > > >> 2. We want the connectors to have the ability to disable format type
> > > >> switch format.type but allows all the other properties, e.g.
> format.*
> > > >> without format.type(let's call it SET_B), if we use the whitelist,
> we
> > > >> have to enumerate all the specific format keys start with format
> > > >> (SET_B), but with the old connector factories, we have no idea what
> > > >> specific format keys it supports(there is either a format.* or
> > nothing).
> > > >>
> > > >> 3. Except the cases for 1 and 2, for normal keys(no wildcard), the
> > > >> blacklist and whitelist has the same expressiveness, use blacklist
> > > >> makes the code not too verbose to enumerate all the duplicate keys
> > > >> with #supportedKeys .(Not very strong reason, but i think as a
> > > >> connector developer, it makes sense)
> > > >>
> > > >> Best,
> > > >> Danny Chan
> > > >> 在 2020年4月3日 +0800 PM11:28,Timo Walther <tw...@apache.org>,写道:
> > > >>> Hi everyone,
> > > >>>
> > > >>> @Aljoscha: I disagree with your approach because a `Catalog` can
> > return
> > > >>> a custom factory that is not using any properties. The hinting must
> > be
> > > >>> transparent to a factory. We should NOT modify the metadata
> > > >>> `CatalogTable` at any point in time after the catalog.
> > > >>>
> > > >>> @Danny, @Jingsong: How about we stick to the original design that
> we
> > > >>> wanted to vote on but use:
> > > >>>
> > > >>> Set<String> supportedHintProperties()
> > > >>>
> > > >>> This fits better to the old factory design. And for the new FLIP-95
> > > >>> factories we will use `ConfigOption` and provide good utilities for
> > > >>> merging with hints etc.
> > > >>>
> > > >>> We can allow `"format.*"` in `supportedHintProperties()` to allow
> > > >>> hinting in formats.
> > > >>>
> > > >>> What do you think?
> > > >>>
> > > >>> Regards,
> > > >>> Timo
> > > >>>
> > > >>>
> > > >>> On 02.04.20 16:24, Aljoscha Krettek wrote:
> > > >>>> I think we're designing ourselves into ever more complicated
> corners
> > > >>>> here. Maybe we need to take a step back and reconsider. What would
> > you
> > > >>>> think about this (somewhat) simpler proposal:
> > > >>>>
> > > >>>> - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or
> > > >>>> CONNECTOR_PROPERTIES, depending on what naming we want to have for
> > > this
> > > >>>> in the future. This will simply overwrite all connector
> properties,
> > > the
> > > >>>> table factories don't know about hints but simply work with the
> > > >>>> properties that they are given
> > > >>>>
> > > >>>> - this special hint is disabled by default and can be activated
> > with a
> > > >>>> global option "foo.bazzle.connector-hints" (or something like
> this)
> > > >>>> which has a warning that describes that this can change query
> > > semantics
> > > >>>> etc.
> > > >>>>
> > > >>>> That's it. This makes connector implementations a lot easier while
> > > >>>> still
> > > >>>> allowing inline configuration.
> > > >>>>
> > > >>>> I still don't like using hint syntax at all for this, because I
> > > >>>> strongly
> > > >>>> maintain that hints should not change query syntax. In general
> using
> > > >>>> hints should be kept to a minimum because they usually point to
> > > >>>> shortcomings in the system.
> > > >>>>
> > > >>>> Best,
> > > >>>> Aljoscha
> > > >>>>
> > > >>>> On 02.04.20 06:06, Jingsong Li wrote:
> > > >>>>> Hi Dawid,
> > > >>>>>
> > > >>>>>> When a factory is instantiated it has access to the
> CatalogTable,
> > > >>>>> therefore it has access to all the original properties. In turn
> it
> > > >>>>> knows
> > > >>>>> the original format and can call
> > > FormatFactory#supportedHintOptions().
> > > >>>>>
> > > >>>>> Factory can only get CatalogTable when creating source or sink,
> > > >>>>> right? IIUC, TableFactory may be stateless too.
> > > >>>>> When invoking SourceFactory#supportedHintOptions(), it can not
> > > >>>>> get CatalogTable, so it is impossible to create FormatFactory?
> > > >>>>>
> > > >>>>> Best,
> > > >>>>> Jingsong Lee
> > > >>>>>
> > > >>>>> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yuzhao.cyz@gmail.com
> >
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>> Hi, Dawid, thanks so much for the detail suggestions ~
> > > >>>>>>
> > > >>>>>> 1. Regarding the motivation:
> > > >>>>>>
> > > >>>>>> I agree it's not a good suggested way based on the fact that we
> > have
> > > >>>>>> better solution, but i think we can support override that as
> long
> > > >>>>>> as it
> > > >>>>>> exists as one of the the table options. I would remove if from
> the
> > > >>>>>> motication part.
> > > >>>>>>
> > > >>>>>> 2. The options passes around during sql-to-rel conversion, right
> > > >>>>>> after we
> > > >>>>>> generate the RelOptTable (CatalogSourceTable#toRel in Flink),
> this
> > > is
> > > >>>>>> indeed a push way method at least in the RelOptTable layer, then
> > > >>>>>> we hand
> > > >>>>>> over the options to TableSourceFactory with our own context,
> which
> > > is
> > > >>>>>> fine
> > > >>>>>> becuase TableSourceFactory#Context is the contact to pass around
> > > >>>>>> these
> > > >>>>>> table-about variables.
> > > >>>>>>
> > > >>>>>> 3. "We should not end up with an extreme example where we can
> > > >>>>>> change the
> > > >>>>>> connector type", i totally agree that, and i have listed the
> > > >>>>>> "connector.type" as forbidden attribute in the WIKI. As for the
> > > >>>>>> format, i
> > > >>>>>> think the connector itself can/should control whether to
> override
> > > the
> > > >>>>>> "format.type", that is one of the reason i change the
> > > >>>>>> TableSourceFactory#supportedHintOpitons to
> > > >>>>>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can
> > > >>>>>> limit the
> > > >>>>>> format keys we want conveniently.
> > > >>>>>>
> > > >>>>>> 4. SQL Hints syntax.
> > > >>>>>>
> > > >>>>>>> I think the k and v in the hint_item should be QUOTED_STRING
> (not
> > > >>>>>>> sure
> > > >>>>>> if it is equivalent to string_literal).
> > > >>>>>>
> > > >>>>>> I disagree, we at least should keep sync with our DDL: use the
> > > string
> > > >>>>>> literal as the key. We did also support the simple identifier
> > > because
> > > >>>>>> this
> > > >>>>>> is the common hint syntax from Calcite, it does not hurt
> anything
> > > for
> > > >>>>>> the
> > > >>>>>> OPTIONS hint, the unsupported keys would validate fails.(If you
> > > think
> > > >>>>>> that
> > > >>>>>> may cause some confuse, i can make the syntax pluggable for each
> > > >>>>>> hint in
> > > >>>>>> CALCITE 1.23)
> > > >>>>>>
> > > >>>>>> We only supports OPTIONS hint in the FLIP, and i have changed
> the
> > > >>>>>> title to
> > > >>>>>> "Supports dynamic table options", would make it more clear in
> the
> > > >>>>>> WIKI.
> > > >>>>>>
> > > >>>>>> 5. Yes, we also have this concerns from our offline discussion,
> > > >>>>>> that is
> > > >>>>>> one of the reason, why i change the
> > > >>>>>> TableSourceFactory#supportedHintOpitons
> > > >>>>>> to TableSourceFactory#forbiddenHintOpitons, i do not choose
> > > >>>>>> Set<String>
> > > >>>>>> supportedHintOptionKeys() with wildcard for 2 reasons:
> > > >>>>>>
> > > >>>>>>     - The wildcard is still not descriptive, we can still not
> > > >>>>>> forbidden one
> > > >>>>>> of the properties among the wildcard properties, we can not
> enable
> > > or
> > > >>>>>> disable them totally
> > > >>>>>>     - ConfigOption is our new structure for keys, and it does
> not
> > > >>>>>> support
> > > >>>>>> wildcard yet.
> > > >>>>>>
> > > >>>>>> Best,
> > > >>>>>> Danny Chan
> > > >>>>>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz
> > > >>>>>> <dw...@apache.org>,写道:
> > > >>>>>>> Hi,
> > > >>>>>>> Few comments from my side:
> > > >>>>>>> 1. Regarding the motivation:
> > > >>>>>>> I think the example with changing the update-mode is not a good
> > > >>>>>>> one. In
> > > >>>>>> the long term this should be done with EMIT CHANGELOG (discussed
> > in
> > > >>>>>> FLIP-105).
> > > >>>>>>> Nitpicking: I would mention that it is rather for
> > debugging/ad-hoc
> > > >>>>>> solution. I think this should not be a recommended way for
> > > production
> > > >>>>>> use
> > > >>>>>> cases as it bypasses the Catalog, which should be the source of
> > > >>>>>> truth.
> > > >>>>>>> 2. I could not understand how the additional options will be
> > > >>>>>>> passed to
> > > >>>>>> the TableSourceFactory. Could you elaborate a bit more on that?
> I
> > > see
> > > >>>>>> there
> > > >>>>>> is a Context interface that gives the options. But cannot find a
> > way
> > > >>>>>> to get
> > > >>>>>> the context itself in the factory. Moreover I think it would
> make
> > > >>>>>> more
> > > >>>>>> sense to have rather a push based approach here. Something like
> > > >>>>>> applyOptions(ReadableConfig) method.
> > > >>>>>>> 3. As for the concerns Jingsong raised in the voting thread. I
> > > >>>>>>> think it
> > > >>>>>> is not a big problem, but I agree this should be also
> described. I
> > > >>>>>> disagree
> > > >>>>>> with "Connector don't know format information in TableFactory
> > before
> > > >>>>>> obtains real properties, so it can not list any format
> > > >>>>>> `supportedHintOptions`".
> > > >>>>>>> When a factory is instantiated it has access to the
> CatalogTable,
> > > >>>>>> therefore it has access to all the original properties. In turn
> it
> > > >>>>>> knows
> > > >>>>>> the original format and can call
> > > >>>>>> FormatFactory#supportedHintOptions().
> > > >>>>>>> The only case when this would not work would be if we allow
> > > changing
> > > >>>>>>> the
> > > >>>>>> format of the Table (e.g. from avro to parquet), which does not
> > > sound
> > > >>>>>> like
> > > >>>>>> a good idea to me. I think this feature should not end up as a
> way
> > > to
> > > >>>>>> declare a whole table inline in a SQL query, but should rather
> be
> > a
> > > >>>>>> simple
> > > >>>>>> way for debugging queries. We should not end up with an extreme
> > > >>>>>> example
> > > >>>>>> where we do:
> > > >>>>>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
> > > >>>>>> 'format.type' = 'json', ....) */
> > > >>>>>>> 4. SQL Hints syntax.
> > > >>>>>>> I think the k and v in the hint_item should be QUOTED_STRING
> (not
> > > >>>>>>> sure
> > > >>>>>> if it is equivalent to string_literal). I think we should not
> use
> > > >>>>>> simple_identifier because this implies that we cannot use e.g.
> any
> > > >>>>>> SQL
> > > >>>>>> keywords. Anyway it has nothing to do with identifiers. If I am
> > not
> > > >>>>>> mistaken it is also how the options in the CREATE statement are
> > > >>>>>> implemented.
> > > >>>>>>> What is the purpose of the remaining hint_item:
> > hint_name(hint_opt
> > > [
> > > >>>>>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a
> > > >>>>>> feeling it
> > > >>>>>> does also suggests to support the whole Apache Calcite hint
> system
> > > >>>>>> without
> > > >>>>>> specifying that explicitly. Is the intention of the FLIP to
> > support
> > > >>>>>> choosing e.g. JOIN strategies through hints already? If it is so
> > it
> > > >>>>>> should
> > > >>>>>> be mentioned in the FLIP, imo.
> > > >>>>>>> 5. I think something does not work around the
> > > >>>>>>> supportedHintOptions and
> > > >>>>>> wildcards. How do you want to represent a wildcard key as a
> > > >>>>>> ConfigOption? I
> > > >>>>>> am not sure about that, just a though, maybe it make sense to
> have
> > > >>>>>> rather
> > > >>>>>> Set<String> supportedHintOptionKeys()?
> > > >>>>>>> Best,
> > > >>>>>>> Dawid
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>
> > > >>
> > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jark Wu <im...@gmail.com>.
I'm fine to disable this feature by default and avoid
whitelisting/blacklisting. This simplifies a lot of things.

Regarding to TableSourceFactory#Context#getExecutionOptions, do we really
need this interface?
Should the connector factory be aware of the properties is merged with
hints or not?
What's the problem if we always get properties from
`CatalogTable#getProperties`?

Best,
Jark

On Tue, 7 Apr 2020 at 10:39, Kurt Young <yk...@gmail.com> wrote:

> Sounds like a reasonable compromise, disabling this feature by default is a
> way to protect
> the vulnerability, and we can simplify the design quite a lot. We can
> gather some users'
> feedback to see whether further protections are necessary in the future.
>
> Best,
> Kurt
>
>
> On Mon, Apr 6, 2020 at 11:49 PM Timo Walther <tw...@apache.org> wrote:
>
> > I agree with Aljoscha. The length of this thread shows that this is
> > highly controversal. I think nobody really likes this feature 100% but
> > we could not find a better solution. I would consider it as a
> > nice-to-have improvement during a notebook/debugging session.
> >
> > I would accept avoiding whitelisting/blacklisting if the feature is
> > disabled by default. And we make the merged properties available in a
> > separate TableSourceFactory#Context#getExecutionOptions as Danny
> proposed.
> >
> > What do you think?
> >
> > Thanks,
> > Timo
> >
> >
> > On 06.04.20 09:59, Aljoscha Krettek wrote:
> > > The reason I'm saying it should be disabled by default is that this
> uses
> > > hint syntax, and hints should really not change query semantics.
> > >
> > > I'm quite strongly against hints that change query semantics, but if we
> > > disable this by default I would be (reluctantly) OK with the feature.
> > > Companies that create deployments or set up the SQL environment for
> > > users can enable the feature if they want.
> > >
> > > But yes, I also agree that we don't need whitelisting/blacklisting,
> > > which makes this a lot easier to do.
> > >
> > > Best,
> > > Aljoscha
> > >
> > > On 06.04.20 04:27, Danny Chan wrote:
> > >> Hi, everyone ~
> > >>
> > >> @Aljoscha @Timo
> > >>
> > >>> I think we're designing ourselves into ever more complicated corners
> > >> here
> > >>
> > >> I kindly agree that, personally didn't see strong reasons why we
> > >> should limit on each connector properties:
> > >>
> > >> • we can define any table options for CREATE TABLE, why we treat the
> > >> dynamic options differently, we never consider any security problems
> > >> when create table, we should not either for dynamic table options
> > >> • If we do not have whitelist properties or blacklist properties, the
> > >> table source creation work would be much easier, just used the merged
> > >> options. There is no need to modify each connector to decide which
> > >> options could be overridden and how we merge them(the merge work is
> > >> redundant).
> > >> • @Timo, how about we support another interface
> > >> `TableSourceFactory#Context.getExecutionOptions`, we always use this
> > >> interface to get the options to create our table source. There is no
> > >> need to copy the catalog table itselt, we just need to generate our
> > >> Context correctly.
> > >> • @Aljoscha I agree to have a global config option, but I disagree to
> > >> default disable it, a global default config would break the user
> > >> experience too much, especially when user want to modify the options
> > >> in a ad-hoc way.
> > >>
> > >>
> > >>
> > >> I suggest to remove `TableSourceFactory#supportedHintOptions` or
> > >> `TableSourceFactory#forbiddenHintOptions` based on the fact that we
> > >> does not have black/white list for CREATE TABLE at all at lease for
> > >> current codebase.
> > >>
> > >>
> > >> @Timo (i have replied offline but allows to represent it here again)
> > >>
> > >> The `TableSourceFactory#supportedHintOptions` doesn't work well for 3
> > >> reasons compared to `TableSourceFactory#forbiddenHintOptions`:
> > >> 1. For key with wildcard, like connector.property.* , use a blacklist
> > >> make us have the ability to disable some of the keys under that, i.e.
> > >> connector.property.key1 , a whitelist can only match with prefix
> > >>
> > >> 2. We want the connectors to have the ability to disable format type
> > >> switch format.type but allows all the other properties, e.g. format.*
> > >> without format.type(let's call it SET_B), if we use the whitelist, we
> > >> have to enumerate all the specific format keys start with format
> > >> (SET_B), but with the old connector factories, we have no idea what
> > >> specific format keys it supports(there is either a format.* or
> nothing).
> > >>
> > >> 3. Except the cases for 1 and 2, for normal keys(no wildcard), the
> > >> blacklist and whitelist has the same expressiveness, use blacklist
> > >> makes the code not too verbose to enumerate all the duplicate keys
> > >> with #supportedKeys .(Not very strong reason, but i think as a
> > >> connector developer, it makes sense)
> > >>
> > >> Best,
> > >> Danny Chan
> > >> 在 2020年4月3日 +0800 PM11:28,Timo Walther <tw...@apache.org>,写道:
> > >>> Hi everyone,
> > >>>
> > >>> @Aljoscha: I disagree with your approach because a `Catalog` can
> return
> > >>> a custom factory that is not using any properties. The hinting must
> be
> > >>> transparent to a factory. We should NOT modify the metadata
> > >>> `CatalogTable` at any point in time after the catalog.
> > >>>
> > >>> @Danny, @Jingsong: How about we stick to the original design that we
> > >>> wanted to vote on but use:
> > >>>
> > >>> Set<String> supportedHintProperties()
> > >>>
> > >>> This fits better to the old factory design. And for the new FLIP-95
> > >>> factories we will use `ConfigOption` and provide good utilities for
> > >>> merging with hints etc.
> > >>>
> > >>> We can allow `"format.*"` in `supportedHintProperties()` to allow
> > >>> hinting in formats.
> > >>>
> > >>> What do you think?
> > >>>
> > >>> Regards,
> > >>> Timo
> > >>>
> > >>>
> > >>> On 02.04.20 16:24, Aljoscha Krettek wrote:
> > >>>> I think we're designing ourselves into ever more complicated corners
> > >>>> here. Maybe we need to take a step back and reconsider. What would
> you
> > >>>> think about this (somewhat) simpler proposal:
> > >>>>
> > >>>> - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or
> > >>>> CONNECTOR_PROPERTIES, depending on what naming we want to have for
> > this
> > >>>> in the future. This will simply overwrite all connector properties,
> > the
> > >>>> table factories don't know about hints but simply work with the
> > >>>> properties that they are given
> > >>>>
> > >>>> - this special hint is disabled by default and can be activated
> with a
> > >>>> global option "foo.bazzle.connector-hints" (or something like this)
> > >>>> which has a warning that describes that this can change query
> > semantics
> > >>>> etc.
> > >>>>
> > >>>> That's it. This makes connector implementations a lot easier while
> > >>>> still
> > >>>> allowing inline configuration.
> > >>>>
> > >>>> I still don't like using hint syntax at all for this, because I
> > >>>> strongly
> > >>>> maintain that hints should not change query syntax. In general using
> > >>>> hints should be kept to a minimum because they usually point to
> > >>>> shortcomings in the system.
> > >>>>
> > >>>> Best,
> > >>>> Aljoscha
> > >>>>
> > >>>> On 02.04.20 06:06, Jingsong Li wrote:
> > >>>>> Hi Dawid,
> > >>>>>
> > >>>>>> When a factory is instantiated it has access to the CatalogTable,
> > >>>>> therefore it has access to all the original properties. In turn it
> > >>>>> knows
> > >>>>> the original format and can call
> > FormatFactory#supportedHintOptions().
> > >>>>>
> > >>>>> Factory can only get CatalogTable when creating source or sink,
> > >>>>> right? IIUC, TableFactory may be stateless too.
> > >>>>> When invoking SourceFactory#supportedHintOptions(), it can not
> > >>>>> get CatalogTable, so it is impossible to create FormatFactory?
> > >>>>>
> > >>>>> Best,
> > >>>>> Jingsong Lee
> > >>>>>
> > >>>>> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yu...@gmail.com>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> Hi, Dawid, thanks so much for the detail suggestions ~
> > >>>>>>
> > >>>>>> 1. Regarding the motivation:
> > >>>>>>
> > >>>>>> I agree it's not a good suggested way based on the fact that we
> have
> > >>>>>> better solution, but i think we can support override that as long
> > >>>>>> as it
> > >>>>>> exists as one of the the table options. I would remove if from the
> > >>>>>> motication part.
> > >>>>>>
> > >>>>>> 2. The options passes around during sql-to-rel conversion, right
> > >>>>>> after we
> > >>>>>> generate the RelOptTable (CatalogSourceTable#toRel in Flink), this
> > is
> > >>>>>> indeed a push way method at least in the RelOptTable layer, then
> > >>>>>> we hand
> > >>>>>> over the options to TableSourceFactory with our own context, which
> > is
> > >>>>>> fine
> > >>>>>> becuase TableSourceFactory#Context is the contact to pass around
> > >>>>>> these
> > >>>>>> table-about variables.
> > >>>>>>
> > >>>>>> 3. "We should not end up with an extreme example where we can
> > >>>>>> change the
> > >>>>>> connector type", i totally agree that, and i have listed the
> > >>>>>> "connector.type" as forbidden attribute in the WIKI. As for the
> > >>>>>> format, i
> > >>>>>> think the connector itself can/should control whether to override
> > the
> > >>>>>> "format.type", that is one of the reason i change the
> > >>>>>> TableSourceFactory#supportedHintOpitons to
> > >>>>>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can
> > >>>>>> limit the
> > >>>>>> format keys we want conveniently.
> > >>>>>>
> > >>>>>> 4. SQL Hints syntax.
> > >>>>>>
> > >>>>>>> I think the k and v in the hint_item should be QUOTED_STRING (not
> > >>>>>>> sure
> > >>>>>> if it is equivalent to string_literal).
> > >>>>>>
> > >>>>>> I disagree, we at least should keep sync with our DDL: use the
> > string
> > >>>>>> literal as the key. We did also support the simple identifier
> > because
> > >>>>>> this
> > >>>>>> is the common hint syntax from Calcite, it does not hurt anything
> > for
> > >>>>>> the
> > >>>>>> OPTIONS hint, the unsupported keys would validate fails.(If you
> > think
> > >>>>>> that
> > >>>>>> may cause some confuse, i can make the syntax pluggable for each
> > >>>>>> hint in
> > >>>>>> CALCITE 1.23)
> > >>>>>>
> > >>>>>> We only supports OPTIONS hint in the FLIP, and i have changed the
> > >>>>>> title to
> > >>>>>> "Supports dynamic table options", would make it more clear in the
> > >>>>>> WIKI.
> > >>>>>>
> > >>>>>> 5. Yes, we also have this concerns from our offline discussion,
> > >>>>>> that is
> > >>>>>> one of the reason, why i change the
> > >>>>>> TableSourceFactory#supportedHintOpitons
> > >>>>>> to TableSourceFactory#forbiddenHintOpitons, i do not choose
> > >>>>>> Set<String>
> > >>>>>> supportedHintOptionKeys() with wildcard for 2 reasons:
> > >>>>>>
> > >>>>>>     - The wildcard is still not descriptive, we can still not
> > >>>>>> forbidden one
> > >>>>>> of the properties among the wildcard properties, we can not enable
> > or
> > >>>>>> disable them totally
> > >>>>>>     - ConfigOption is our new structure for keys, and it does not
> > >>>>>> support
> > >>>>>> wildcard yet.
> > >>>>>>
> > >>>>>> Best,
> > >>>>>> Danny Chan
> > >>>>>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz
> > >>>>>> <dw...@apache.org>,写道:
> > >>>>>>> Hi,
> > >>>>>>> Few comments from my side:
> > >>>>>>> 1. Regarding the motivation:
> > >>>>>>> I think the example with changing the update-mode is not a good
> > >>>>>>> one. In
> > >>>>>> the long term this should be done with EMIT CHANGELOG (discussed
> in
> > >>>>>> FLIP-105).
> > >>>>>>> Nitpicking: I would mention that it is rather for
> debugging/ad-hoc
> > >>>>>> solution. I think this should not be a recommended way for
> > production
> > >>>>>> use
> > >>>>>> cases as it bypasses the Catalog, which should be the source of
> > >>>>>> truth.
> > >>>>>>> 2. I could not understand how the additional options will be
> > >>>>>>> passed to
> > >>>>>> the TableSourceFactory. Could you elaborate a bit more on that? I
> > see
> > >>>>>> there
> > >>>>>> is a Context interface that gives the options. But cannot find a
> way
> > >>>>>> to get
> > >>>>>> the context itself in the factory. Moreover I think it would make
> > >>>>>> more
> > >>>>>> sense to have rather a push based approach here. Something like
> > >>>>>> applyOptions(ReadableConfig) method.
> > >>>>>>> 3. As for the concerns Jingsong raised in the voting thread. I
> > >>>>>>> think it
> > >>>>>> is not a big problem, but I agree this should be also described. I
> > >>>>>> disagree
> > >>>>>> with "Connector don't know format information in TableFactory
> before
> > >>>>>> obtains real properties, so it can not list any format
> > >>>>>> `supportedHintOptions`".
> > >>>>>>> When a factory is instantiated it has access to the CatalogTable,
> > >>>>>> therefore it has access to all the original properties. In turn it
> > >>>>>> knows
> > >>>>>> the original format and can call
> > >>>>>> FormatFactory#supportedHintOptions().
> > >>>>>>> The only case when this would not work would be if we allow
> > changing
> > >>>>>>> the
> > >>>>>> format of the Table (e.g. from avro to parquet), which does not
> > sound
> > >>>>>> like
> > >>>>>> a good idea to me. I think this feature should not end up as a way
> > to
> > >>>>>> declare a whole table inline in a SQL query, but should rather be
> a
> > >>>>>> simple
> > >>>>>> way for debugging queries. We should not end up with an extreme
> > >>>>>> example
> > >>>>>> where we do:
> > >>>>>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
> > >>>>>> 'format.type' = 'json', ....) */
> > >>>>>>> 4. SQL Hints syntax.
> > >>>>>>> I think the k and v in the hint_item should be QUOTED_STRING (not
> > >>>>>>> sure
> > >>>>>> if it is equivalent to string_literal). I think we should not use
> > >>>>>> simple_identifier because this implies that we cannot use e.g. any
> > >>>>>> SQL
> > >>>>>> keywords. Anyway it has nothing to do with identifiers. If I am
> not
> > >>>>>> mistaken it is also how the options in the CREATE statement are
> > >>>>>> implemented.
> > >>>>>>> What is the purpose of the remaining hint_item:
> hint_name(hint_opt
> > [
> > >>>>>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a
> > >>>>>> feeling it
> > >>>>>> does also suggests to support the whole Apache Calcite hint system
> > >>>>>> without
> > >>>>>> specifying that explicitly. Is the intention of the FLIP to
> support
> > >>>>>> choosing e.g. JOIN strategies through hints already? If it is so
> it
> > >>>>>> should
> > >>>>>> be mentioned in the FLIP, imo.
> > >>>>>>> 5. I think something does not work around the
> > >>>>>>> supportedHintOptions and
> > >>>>>> wildcards. How do you want to represent a wildcard key as a
> > >>>>>> ConfigOption? I
> > >>>>>> am not sure about that, just a though, maybe it make sense to have
> > >>>>>> rather
> > >>>>>> Set<String> supportedHintOptionKeys()?
> > >>>>>>> Best,
> > >>>>>>> Dawid
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>
> > >>
> >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Kurt Young <yk...@gmail.com>.
Sounds like a reasonable compromise, disabling this feature by default is a
way to protect
the vulnerability, and we can simplify the design quite a lot. We can
gather some users'
feedback to see whether further protections are necessary in the future.

Best,
Kurt


On Mon, Apr 6, 2020 at 11:49 PM Timo Walther <tw...@apache.org> wrote:

> I agree with Aljoscha. The length of this thread shows that this is
> highly controversal. I think nobody really likes this feature 100% but
> we could not find a better solution. I would consider it as a
> nice-to-have improvement during a notebook/debugging session.
>
> I would accept avoiding whitelisting/blacklisting if the feature is
> disabled by default. And we make the merged properties available in a
> separate TableSourceFactory#Context#getExecutionOptions as Danny proposed.
>
> What do you think?
>
> Thanks,
> Timo
>
>
> On 06.04.20 09:59, Aljoscha Krettek wrote:
> > The reason I'm saying it should be disabled by default is that this uses
> > hint syntax, and hints should really not change query semantics.
> >
> > I'm quite strongly against hints that change query semantics, but if we
> > disable this by default I would be (reluctantly) OK with the feature.
> > Companies that create deployments or set up the SQL environment for
> > users can enable the feature if they want.
> >
> > But yes, I also agree that we don't need whitelisting/blacklisting,
> > which makes this a lot easier to do.
> >
> > Best,
> > Aljoscha
> >
> > On 06.04.20 04:27, Danny Chan wrote:
> >> Hi, everyone ~
> >>
> >> @Aljoscha @Timo
> >>
> >>> I think we're designing ourselves into ever more complicated corners
> >> here
> >>
> >> I kindly agree that, personally didn't see strong reasons why we
> >> should limit on each connector properties:
> >>
> >> • we can define any table options for CREATE TABLE, why we treat the
> >> dynamic options differently, we never consider any security problems
> >> when create table, we should not either for dynamic table options
> >> • If we do not have whitelist properties or blacklist properties, the
> >> table source creation work would be much easier, just used the merged
> >> options. There is no need to modify each connector to decide which
> >> options could be overridden and how we merge them(the merge work is
> >> redundant).
> >> • @Timo, how about we support another interface
> >> `TableSourceFactory#Context.getExecutionOptions`, we always use this
> >> interface to get the options to create our table source. There is no
> >> need to copy the catalog table itselt, we just need to generate our
> >> Context correctly.
> >> • @Aljoscha I agree to have a global config option, but I disagree to
> >> default disable it, a global default config would break the user
> >> experience too much, especially when user want to modify the options
> >> in a ad-hoc way.
> >>
> >>
> >>
> >> I suggest to remove `TableSourceFactory#supportedHintOptions` or
> >> `TableSourceFactory#forbiddenHintOptions` based on the fact that we
> >> does not have black/white list for CREATE TABLE at all at lease for
> >> current codebase.
> >>
> >>
> >> @Timo (i have replied offline but allows to represent it here again)
> >>
> >> The `TableSourceFactory#supportedHintOptions` doesn't work well for 3
> >> reasons compared to `TableSourceFactory#forbiddenHintOptions`:
> >> 1. For key with wildcard, like connector.property.* , use a blacklist
> >> make us have the ability to disable some of the keys under that, i.e.
> >> connector.property.key1 , a whitelist can only match with prefix
> >>
> >> 2. We want the connectors to have the ability to disable format type
> >> switch format.type but allows all the other properties, e.g. format.*
> >> without format.type(let's call it SET_B), if we use the whitelist, we
> >> have to enumerate all the specific format keys start with format
> >> (SET_B), but with the old connector factories, we have no idea what
> >> specific format keys it supports(there is either a format.* or nothing).
> >>
> >> 3. Except the cases for 1 and 2, for normal keys(no wildcard), the
> >> blacklist and whitelist has the same expressiveness, use blacklist
> >> makes the code not too verbose to enumerate all the duplicate keys
> >> with #supportedKeys .(Not very strong reason, but i think as a
> >> connector developer, it makes sense)
> >>
> >> Best,
> >> Danny Chan
> >> 在 2020年4月3日 +0800 PM11:28,Timo Walther <tw...@apache.org>,写道:
> >>> Hi everyone,
> >>>
> >>> @Aljoscha: I disagree with your approach because a `Catalog` can return
> >>> a custom factory that is not using any properties. The hinting must be
> >>> transparent to a factory. We should NOT modify the metadata
> >>> `CatalogTable` at any point in time after the catalog.
> >>>
> >>> @Danny, @Jingsong: How about we stick to the original design that we
> >>> wanted to vote on but use:
> >>>
> >>> Set<String> supportedHintProperties()
> >>>
> >>> This fits better to the old factory design. And for the new FLIP-95
> >>> factories we will use `ConfigOption` and provide good utilities for
> >>> merging with hints etc.
> >>>
> >>> We can allow `"format.*"` in `supportedHintProperties()` to allow
> >>> hinting in formats.
> >>>
> >>> What do you think?
> >>>
> >>> Regards,
> >>> Timo
> >>>
> >>>
> >>> On 02.04.20 16:24, Aljoscha Krettek wrote:
> >>>> I think we're designing ourselves into ever more complicated corners
> >>>> here. Maybe we need to take a step back and reconsider. What would you
> >>>> think about this (somewhat) simpler proposal:
> >>>>
> >>>> - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or
> >>>> CONNECTOR_PROPERTIES, depending on what naming we want to have for
> this
> >>>> in the future. This will simply overwrite all connector properties,
> the
> >>>> table factories don't know about hints but simply work with the
> >>>> properties that they are given
> >>>>
> >>>> - this special hint is disabled by default and can be activated with a
> >>>> global option "foo.bazzle.connector-hints" (or something like this)
> >>>> which has a warning that describes that this can change query
> semantics
> >>>> etc.
> >>>>
> >>>> That's it. This makes connector implementations a lot easier while
> >>>> still
> >>>> allowing inline configuration.
> >>>>
> >>>> I still don't like using hint syntax at all for this, because I
> >>>> strongly
> >>>> maintain that hints should not change query syntax. In general using
> >>>> hints should be kept to a minimum because they usually point to
> >>>> shortcomings in the system.
> >>>>
> >>>> Best,
> >>>> Aljoscha
> >>>>
> >>>> On 02.04.20 06:06, Jingsong Li wrote:
> >>>>> Hi Dawid,
> >>>>>
> >>>>>> When a factory is instantiated it has access to the CatalogTable,
> >>>>> therefore it has access to all the original properties. In turn it
> >>>>> knows
> >>>>> the original format and can call
> FormatFactory#supportedHintOptions().
> >>>>>
> >>>>> Factory can only get CatalogTable when creating source or sink,
> >>>>> right? IIUC, TableFactory may be stateless too.
> >>>>> When invoking SourceFactory#supportedHintOptions(), it can not
> >>>>> get CatalogTable, so it is impossible to create FormatFactory?
> >>>>>
> >>>>> Best,
> >>>>> Jingsong Lee
> >>>>>
> >>>>> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yu...@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi, Dawid, thanks so much for the detail suggestions ~
> >>>>>>
> >>>>>> 1. Regarding the motivation:
> >>>>>>
> >>>>>> I agree it's not a good suggested way based on the fact that we have
> >>>>>> better solution, but i think we can support override that as long
> >>>>>> as it
> >>>>>> exists as one of the the table options. I would remove if from the
> >>>>>> motication part.
> >>>>>>
> >>>>>> 2. The options passes around during sql-to-rel conversion, right
> >>>>>> after we
> >>>>>> generate the RelOptTable (CatalogSourceTable#toRel in Flink), this
> is
> >>>>>> indeed a push way method at least in the RelOptTable layer, then
> >>>>>> we hand
> >>>>>> over the options to TableSourceFactory with our own context, which
> is
> >>>>>> fine
> >>>>>> becuase TableSourceFactory#Context is the contact to pass around
> >>>>>> these
> >>>>>> table-about variables.
> >>>>>>
> >>>>>> 3. "We should not end up with an extreme example where we can
> >>>>>> change the
> >>>>>> connector type", i totally agree that, and i have listed the
> >>>>>> "connector.type" as forbidden attribute in the WIKI. As for the
> >>>>>> format, i
> >>>>>> think the connector itself can/should control whether to override
> the
> >>>>>> "format.type", that is one of the reason i change the
> >>>>>> TableSourceFactory#supportedHintOpitons to
> >>>>>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can
> >>>>>> limit the
> >>>>>> format keys we want conveniently.
> >>>>>>
> >>>>>> 4. SQL Hints syntax.
> >>>>>>
> >>>>>>> I think the k and v in the hint_item should be QUOTED_STRING (not
> >>>>>>> sure
> >>>>>> if it is equivalent to string_literal).
> >>>>>>
> >>>>>> I disagree, we at least should keep sync with our DDL: use the
> string
> >>>>>> literal as the key. We did also support the simple identifier
> because
> >>>>>> this
> >>>>>> is the common hint syntax from Calcite, it does not hurt anything
> for
> >>>>>> the
> >>>>>> OPTIONS hint, the unsupported keys would validate fails.(If you
> think
> >>>>>> that
> >>>>>> may cause some confuse, i can make the syntax pluggable for each
> >>>>>> hint in
> >>>>>> CALCITE 1.23)
> >>>>>>
> >>>>>> We only supports OPTIONS hint in the FLIP, and i have changed the
> >>>>>> title to
> >>>>>> "Supports dynamic table options", would make it more clear in the
> >>>>>> WIKI.
> >>>>>>
> >>>>>> 5. Yes, we also have this concerns from our offline discussion,
> >>>>>> that is
> >>>>>> one of the reason, why i change the
> >>>>>> TableSourceFactory#supportedHintOpitons
> >>>>>> to TableSourceFactory#forbiddenHintOpitons, i do not choose
> >>>>>> Set<String>
> >>>>>> supportedHintOptionKeys() with wildcard for 2 reasons:
> >>>>>>
> >>>>>>     - The wildcard is still not descriptive, we can still not
> >>>>>> forbidden one
> >>>>>> of the properties among the wildcard properties, we can not enable
> or
> >>>>>> disable them totally
> >>>>>>     - ConfigOption is our new structure for keys, and it does not
> >>>>>> support
> >>>>>> wildcard yet.
> >>>>>>
> >>>>>> Best,
> >>>>>> Danny Chan
> >>>>>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz
> >>>>>> <dw...@apache.org>,写道:
> >>>>>>> Hi,
> >>>>>>> Few comments from my side:
> >>>>>>> 1. Regarding the motivation:
> >>>>>>> I think the example with changing the update-mode is not a good
> >>>>>>> one. In
> >>>>>> the long term this should be done with EMIT CHANGELOG (discussed in
> >>>>>> FLIP-105).
> >>>>>>> Nitpicking: I would mention that it is rather for debugging/ad-hoc
> >>>>>> solution. I think this should not be a recommended way for
> production
> >>>>>> use
> >>>>>> cases as it bypasses the Catalog, which should be the source of
> >>>>>> truth.
> >>>>>>> 2. I could not understand how the additional options will be
> >>>>>>> passed to
> >>>>>> the TableSourceFactory. Could you elaborate a bit more on that? I
> see
> >>>>>> there
> >>>>>> is a Context interface that gives the options. But cannot find a way
> >>>>>> to get
> >>>>>> the context itself in the factory. Moreover I think it would make
> >>>>>> more
> >>>>>> sense to have rather a push based approach here. Something like
> >>>>>> applyOptions(ReadableConfig) method.
> >>>>>>> 3. As for the concerns Jingsong raised in the voting thread. I
> >>>>>>> think it
> >>>>>> is not a big problem, but I agree this should be also described. I
> >>>>>> disagree
> >>>>>> with "Connector don't know format information in TableFactory before
> >>>>>> obtains real properties, so it can not list any format
> >>>>>> `supportedHintOptions`".
> >>>>>>> When a factory is instantiated it has access to the CatalogTable,
> >>>>>> therefore it has access to all the original properties. In turn it
> >>>>>> knows
> >>>>>> the original format and can call
> >>>>>> FormatFactory#supportedHintOptions().
> >>>>>>> The only case when this would not work would be if we allow
> changing
> >>>>>>> the
> >>>>>> format of the Table (e.g. from avro to parquet), which does not
> sound
> >>>>>> like
> >>>>>> a good idea to me. I think this feature should not end up as a way
> to
> >>>>>> declare a whole table inline in a SQL query, but should rather be a
> >>>>>> simple
> >>>>>> way for debugging queries. We should not end up with an extreme
> >>>>>> example
> >>>>>> where we do:
> >>>>>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
> >>>>>> 'format.type' = 'json', ....) */
> >>>>>>> 4. SQL Hints syntax.
> >>>>>>> I think the k and v in the hint_item should be QUOTED_STRING (not
> >>>>>>> sure
> >>>>>> if it is equivalent to string_literal). I think we should not use
> >>>>>> simple_identifier because this implies that we cannot use e.g. any
> >>>>>> SQL
> >>>>>> keywords. Anyway it has nothing to do with identifiers. If I am not
> >>>>>> mistaken it is also how the options in the CREATE statement are
> >>>>>> implemented.
> >>>>>>> What is the purpose of the remaining hint_item: hint_name(hint_opt
> [
> >>>>>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a
> >>>>>> feeling it
> >>>>>> does also suggests to support the whole Apache Calcite hint system
> >>>>>> without
> >>>>>> specifying that explicitly. Is the intention of the FLIP to support
> >>>>>> choosing e.g. JOIN strategies through hints already? If it is so it
> >>>>>> should
> >>>>>> be mentioned in the FLIP, imo.
> >>>>>>> 5. I think something does not work around the
> >>>>>>> supportedHintOptions and
> >>>>>> wildcards. How do you want to represent a wildcard key as a
> >>>>>> ConfigOption? I
> >>>>>> am not sure about that, just a though, maybe it make sense to have
> >>>>>> rather
> >>>>>> Set<String> supportedHintOptionKeys()?
> >>>>>>> Best,
> >>>>>>> Dawid
> >>>>>>
> >>>>>
> >>>>>
> >>>
> >>
>
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@apache.org>.
I agree with Aljoscha. The length of this thread shows that this is 
highly controversal. I think nobody really likes this feature 100% but 
we could not find a better solution. I would consider it as a 
nice-to-have improvement during a notebook/debugging session.

I would accept avoiding whitelisting/blacklisting if the feature is 
disabled by default. And we make the merged properties available in a 
separate TableSourceFactory#Context#getExecutionOptions as Danny proposed.

What do you think?

Thanks,
Timo


On 06.04.20 09:59, Aljoscha Krettek wrote:
> The reason I'm saying it should be disabled by default is that this uses 
> hint syntax, and hints should really not change query semantics.
> 
> I'm quite strongly against hints that change query semantics, but if we 
> disable this by default I would be (reluctantly) OK with the feature. 
> Companies that create deployments or set up the SQL environment for 
> users can enable the feature if they want.
> 
> But yes, I also agree that we don't need whitelisting/blacklisting, 
> which makes this a lot easier to do.
> 
> Best,
> Aljoscha
> 
> On 06.04.20 04:27, Danny Chan wrote:
>> Hi, everyone ~
>>
>> @Aljoscha @Timo
>>
>>> I think we're designing ourselves into ever more complicated corners
>> here
>>
>> I kindly agree that, personally didn't see strong reasons why we 
>> should limit on each connector properties:
>>
>> • we can define any table options for CREATE TABLE, why we treat the 
>> dynamic options differently, we never consider any security problems 
>> when create table, we should not either for dynamic table options
>> • If we do not have whitelist properties or blacklist properties, the 
>> table source creation work would be much easier, just used the merged 
>> options. There is no need to modify each connector to decide which 
>> options could be overridden and how we merge them(the merge work is 
>> redundant).
>> • @Timo, how about we support another interface 
>> `TableSourceFactory#Context.getExecutionOptions`, we always use this 
>> interface to get the options to create our table source. There is no 
>> need to copy the catalog table itselt, we just need to generate our 
>> Context correctly.
>> • @Aljoscha I agree to have a global config option, but I disagree to 
>> default disable it, a global default config would break the user 
>> experience too much, especially when user want to modify the options 
>> in a ad-hoc way.
>>
>>
>>
>> I suggest to remove `TableSourceFactory#supportedHintOptions` or 
>> `TableSourceFactory#forbiddenHintOptions` based on the fact that we 
>> does not have black/white list for CREATE TABLE at all at lease for 
>> current codebase.
>>
>>
>> @Timo (i have replied offline but allows to represent it here again)
>>
>> The `TableSourceFactory#supportedHintOptions` doesn't work well for 3 
>> reasons compared to `TableSourceFactory#forbiddenHintOptions`:
>> 1. For key with wildcard, like connector.property.* , use a blacklist 
>> make us have the ability to disable some of the keys under that, i.e. 
>> connector.property.key1 , a whitelist can only match with prefix
>>
>> 2. We want the connectors to have the ability to disable format type 
>> switch format.type but allows all the other properties, e.g. format.* 
>> without format.type(let's call it SET_B), if we use the whitelist, we 
>> have to enumerate all the specific format keys start with format 
>> (SET_B), but with the old connector factories, we have no idea what 
>> specific format keys it supports(there is either a format.* or nothing).
>>
>> 3. Except the cases for 1 and 2, for normal keys(no wildcard), the 
>> blacklist and whitelist has the same expressiveness, use blacklist 
>> makes the code not too verbose to enumerate all the duplicate keys 
>> with #supportedKeys .(Not very strong reason, but i think as a 
>> connector developer, it makes sense)
>>
>> Best,
>> Danny Chan
>> 在 2020年4月3日 +0800 PM11:28,Timo Walther <tw...@apache.org>,写道:
>>> Hi everyone,
>>>
>>> @Aljoscha: I disagree with your approach because a `Catalog` can return
>>> a custom factory that is not using any properties. The hinting must be
>>> transparent to a factory. We should NOT modify the metadata
>>> `CatalogTable` at any point in time after the catalog.
>>>
>>> @Danny, @Jingsong: How about we stick to the original design that we
>>> wanted to vote on but use:
>>>
>>> Set<String> supportedHintProperties()
>>>
>>> This fits better to the old factory design. And for the new FLIP-95
>>> factories we will use `ConfigOption` and provide good utilities for
>>> merging with hints etc.
>>>
>>> We can allow `"format.*"` in `supportedHintProperties()` to allow
>>> hinting in formats.
>>>
>>> What do you think?
>>>
>>> Regards,
>>> Timo
>>>
>>>
>>> On 02.04.20 16:24, Aljoscha Krettek wrote:
>>>> I think we're designing ourselves into ever more complicated corners
>>>> here. Maybe we need to take a step back and reconsider. What would you
>>>> think about this (somewhat) simpler proposal:
>>>>
>>>> - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or
>>>> CONNECTOR_PROPERTIES, depending on what naming we want to have for this
>>>> in the future. This will simply overwrite all connector properties, the
>>>> table factories don't know about hints but simply work with the
>>>> properties that they are given
>>>>
>>>> - this special hint is disabled by default and can be activated with a
>>>> global option "foo.bazzle.connector-hints" (or something like this)
>>>> which has a warning that describes that this can change query semantics
>>>> etc.
>>>>
>>>> That's it. This makes connector implementations a lot easier while 
>>>> still
>>>> allowing inline configuration.
>>>>
>>>> I still don't like using hint syntax at all for this, because I 
>>>> strongly
>>>> maintain that hints should not change query syntax. In general using
>>>> hints should be kept to a minimum because they usually point to
>>>> shortcomings in the system.
>>>>
>>>> Best,
>>>> Aljoscha
>>>>
>>>> On 02.04.20 06:06, Jingsong Li wrote:
>>>>> Hi Dawid,
>>>>>
>>>>>> When a factory is instantiated it has access to the CatalogTable,
>>>>> therefore it has access to all the original properties. In turn it 
>>>>> knows
>>>>> the original format and can call FormatFactory#supportedHintOptions().
>>>>>
>>>>> Factory can only get CatalogTable when creating source or sink,
>>>>> right? IIUC, TableFactory may be stateless too.
>>>>> When invoking SourceFactory#supportedHintOptions(), it can not
>>>>> get CatalogTable, so it is impossible to create FormatFactory?
>>>>>
>>>>> Best,
>>>>> Jingsong Lee
>>>>>
>>>>> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yu...@gmail.com> 
>>>>> wrote:
>>>>>
>>>>>> Hi, Dawid, thanks so much for the detail suggestions ~
>>>>>>
>>>>>> 1. Regarding the motivation:
>>>>>>
>>>>>> I agree it's not a good suggested way based on the fact that we have
>>>>>> better solution, but i think we can support override that as long 
>>>>>> as it
>>>>>> exists as one of the the table options. I would remove if from the
>>>>>> motication part.
>>>>>>
>>>>>> 2. The options passes around during sql-to-rel conversion, right
>>>>>> after we
>>>>>> generate the RelOptTable (CatalogSourceTable#toRel in Flink), this is
>>>>>> indeed a push way method at least in the RelOptTable layer, then 
>>>>>> we hand
>>>>>> over the options to TableSourceFactory with our own context, which is
>>>>>> fine
>>>>>> becuase TableSourceFactory#Context is the contact to pass around 
>>>>>> these
>>>>>> table-about variables.
>>>>>>
>>>>>> 3. "We should not end up with an extreme example where we can 
>>>>>> change the
>>>>>> connector type", i totally agree that, and i have listed the
>>>>>> "connector.type" as forbidden attribute in the WIKI. As for the
>>>>>> format, i
>>>>>> think the connector itself can/should control whether to override the
>>>>>> "format.type", that is one of the reason i change the
>>>>>> TableSourceFactory#supportedHintOpitons to
>>>>>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can
>>>>>> limit the
>>>>>> format keys we want conveniently.
>>>>>>
>>>>>> 4. SQL Hints syntax.
>>>>>>
>>>>>>> I think the k and v in the hint_item should be QUOTED_STRING (not 
>>>>>>> sure
>>>>>> if it is equivalent to string_literal).
>>>>>>
>>>>>> I disagree, we at least should keep sync with our DDL: use the string
>>>>>> literal as the key. We did also support the simple identifier because
>>>>>> this
>>>>>> is the common hint syntax from Calcite, it does not hurt anything for
>>>>>> the
>>>>>> OPTIONS hint, the unsupported keys would validate fails.(If you think
>>>>>> that
>>>>>> may cause some confuse, i can make the syntax pluggable for each 
>>>>>> hint in
>>>>>> CALCITE 1.23)
>>>>>>
>>>>>> We only supports OPTIONS hint in the FLIP, and i have changed the
>>>>>> title to
>>>>>> "Supports dynamic table options", would make it more clear in the 
>>>>>> WIKI.
>>>>>>
>>>>>> 5. Yes, we also have this concerns from our offline discussion, 
>>>>>> that is
>>>>>> one of the reason, why i change the
>>>>>> TableSourceFactory#supportedHintOpitons
>>>>>> to TableSourceFactory#forbiddenHintOpitons, i do not choose 
>>>>>> Set<String>
>>>>>> supportedHintOptionKeys() with wildcard for 2 reasons:
>>>>>>
>>>>>>     - The wildcard is still not descriptive, we can still not
>>>>>> forbidden one
>>>>>> of the properties among the wildcard properties, we can not enable or
>>>>>> disable them totally
>>>>>>     - ConfigOption is our new structure for keys, and it does not 
>>>>>> support
>>>>>> wildcard yet.
>>>>>>
>>>>>> Best,
>>>>>> Danny Chan
>>>>>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz
>>>>>> <dw...@apache.org>,写道:
>>>>>>> Hi,
>>>>>>> Few comments from my side:
>>>>>>> 1. Regarding the motivation:
>>>>>>> I think the example with changing the update-mode is not a good 
>>>>>>> one. In
>>>>>> the long term this should be done with EMIT CHANGELOG (discussed in
>>>>>> FLIP-105).
>>>>>>> Nitpicking: I would mention that it is rather for debugging/ad-hoc
>>>>>> solution. I think this should not be a recommended way for production
>>>>>> use
>>>>>> cases as it bypasses the Catalog, which should be the source of 
>>>>>> truth.
>>>>>>> 2. I could not understand how the additional options will be 
>>>>>>> passed to
>>>>>> the TableSourceFactory. Could you elaborate a bit more on that? I see
>>>>>> there
>>>>>> is a Context interface that gives the options. But cannot find a way
>>>>>> to get
>>>>>> the context itself in the factory. Moreover I think it would make 
>>>>>> more
>>>>>> sense to have rather a push based approach here. Something like
>>>>>> applyOptions(ReadableConfig) method.
>>>>>>> 3. As for the concerns Jingsong raised in the voting thread. I 
>>>>>>> think it
>>>>>> is not a big problem, but I agree this should be also described. I
>>>>>> disagree
>>>>>> with "Connector don't know format information in TableFactory before
>>>>>> obtains real properties, so it can not list any format
>>>>>> `supportedHintOptions`".
>>>>>>> When a factory is instantiated it has access to the CatalogTable,
>>>>>> therefore it has access to all the original properties. In turn it 
>>>>>> knows
>>>>>> the original format and can call 
>>>>>> FormatFactory#supportedHintOptions().
>>>>>>> The only case when this would not work would be if we allow changing
>>>>>>> the
>>>>>> format of the Table (e.g. from avro to parquet), which does not sound
>>>>>> like
>>>>>> a good idea to me. I think this feature should not end up as a way to
>>>>>> declare a whole table inline in a SQL query, but should rather be a
>>>>>> simple
>>>>>> way for debugging queries. We should not end up with an extreme 
>>>>>> example
>>>>>> where we do:
>>>>>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
>>>>>> 'format.type' = 'json', ....) */
>>>>>>> 4. SQL Hints syntax.
>>>>>>> I think the k and v in the hint_item should be QUOTED_STRING (not 
>>>>>>> sure
>>>>>> if it is equivalent to string_literal). I think we should not use
>>>>>> simple_identifier because this implies that we cannot use e.g. any 
>>>>>> SQL
>>>>>> keywords. Anyway it has nothing to do with identifiers. If I am not
>>>>>> mistaken it is also how the options in the CREATE statement are
>>>>>> implemented.
>>>>>>> What is the purpose of the remaining hint_item: hint_name(hint_opt [
>>>>>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a
>>>>>> feeling it
>>>>>> does also suggests to support the whole Apache Calcite hint system
>>>>>> without
>>>>>> specifying that explicitly. Is the intention of the FLIP to support
>>>>>> choosing e.g. JOIN strategies through hints already? If it is so it
>>>>>> should
>>>>>> be mentioned in the FLIP, imo.
>>>>>>> 5. I think something does not work around the 
>>>>>>> supportedHintOptions and
>>>>>> wildcards. How do you want to represent a wildcard key as a
>>>>>> ConfigOption? I
>>>>>> am not sure about that, just a though, maybe it make sense to have
>>>>>> rather
>>>>>> Set<String> supportedHintOptionKeys()?
>>>>>>> Best,
>>>>>>> Dawid
>>>>>>
>>>>>
>>>>>
>>>
>>


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Aljoscha Krettek <al...@apache.org>.
The reason I'm saying it should be disabled by default is that this uses 
hint syntax, and hints should really not change query semantics.

I'm quite strongly against hints that change query semantics, but if we 
disable this by default I would be (reluctantly) OK with the feature. 
Companies that create deployments or set up the SQL environment for 
users can enable the feature if they want.

But yes, I also agree that we don't need whitelisting/blacklisting, 
which makes this a lot easier to do.

Best,
Aljoscha

On 06.04.20 04:27, Danny Chan wrote:
> Hi, everyone ~
> 
> @Aljoscha @Timo
> 
>> I think we're designing ourselves into ever more complicated corners
> here
> 
> I kindly agree that, personally didn't see strong reasons why we should limit on each connector properties:
> 
> • we can define any table options for CREATE TABLE, why we treat the dynamic options differently, we never consider any security problems when create table, we should not either for dynamic table options
> • If we do not have whitelist properties or blacklist properties, the table source creation work would be much easier, just used the merged options. There is no need to modify each connector to decide which options could be overridden and how we merge them(the merge work is redundant).
> • @Timo, how about we support another interface `TableSourceFactory#Context.getExecutionOptions`, we always use this interface to get the options to create our table source. There is no need to copy the catalog table itselt, we just need to generate our Context correctly.
> • @Aljoscha I agree to have a global config option, but I disagree to default disable it, a global default config would break the user experience too much, especially when user want to modify the options in a ad-hoc way.
> 
> 
> 
> I suggest to remove `TableSourceFactory#supportedHintOptions` or `TableSourceFactory#forbiddenHintOptions` based on the fact that we does not have black/white list for CREATE TABLE at all at lease for current codebase.
> 
> 
> @Timo (i have replied offline but allows to represent it here again)
> 
> The `TableSourceFactory#supportedHintOptions` doesn't work well for 3 reasons compared to `TableSourceFactory#forbiddenHintOptions`:
> 1. For key with wildcard, like connector.property.* , use a blacklist make us have the ability to disable some of the keys under that, i.e. connector.property.key1 , a whitelist can only match with prefix
> 
> 2. We want the connectors to have the ability to disable format type switch format.type but allows all the other properties, e.g. format.* without format.type(let's call it SET_B), if we use the whitelist, we have to enumerate all the specific format keys start with format (SET_B), but with the old connector factories, we have no idea what specific format keys it supports(there is either a format.* or nothing).
> 
> 3. Except the cases for 1 and 2, for normal keys(no wildcard), the blacklist and whitelist has the same expressiveness, use blacklist makes the code not too verbose to enumerate all the duplicate keys with #supportedKeys .(Not very strong reason, but i think as a connector developer, it makes sense)
> 
> Best,
> Danny Chan
> 在 2020年4月3日 +0800 PM11:28,Timo Walther <tw...@apache.org>,写道:
>> Hi everyone,
>>
>> @Aljoscha: I disagree with your approach because a `Catalog` can return
>> a custom factory that is not using any properties. The hinting must be
>> transparent to a factory. We should NOT modify the metadata
>> `CatalogTable` at any point in time after the catalog.
>>
>> @Danny, @Jingsong: How about we stick to the original design that we
>> wanted to vote on but use:
>>
>> Set<String> supportedHintProperties()
>>
>> This fits better to the old factory design. And for the new FLIP-95
>> factories we will use `ConfigOption` and provide good utilities for
>> merging with hints etc.
>>
>> We can allow `"format.*"` in `supportedHintProperties()` to allow
>> hinting in formats.
>>
>> What do you think?
>>
>> Regards,
>> Timo
>>
>>
>> On 02.04.20 16:24, Aljoscha Krettek wrote:
>>> I think we're designing ourselves into ever more complicated corners
>>> here. Maybe we need to take a step back and reconsider. What would you
>>> think about this (somewhat) simpler proposal:
>>>
>>> - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or
>>> CONNECTOR_PROPERTIES, depending on what naming we want to have for this
>>> in the future. This will simply overwrite all connector properties, the
>>> table factories don't know about hints but simply work with the
>>> properties that they are given
>>>
>>> - this special hint is disabled by default and can be activated with a
>>> global option "foo.bazzle.connector-hints" (or something like this)
>>> which has a warning that describes that this can change query semantics
>>> etc.
>>>
>>> That's it. This makes connector implementations a lot easier while still
>>> allowing inline configuration.
>>>
>>> I still don't like using hint syntax at all for this, because I strongly
>>> maintain that hints should not change query syntax. In general using
>>> hints should be kept to a minimum because they usually point to
>>> shortcomings in the system.
>>>
>>> Best,
>>> Aljoscha
>>>
>>> On 02.04.20 06:06, Jingsong Li wrote:
>>>> Hi Dawid,
>>>>
>>>>> When a factory is instantiated it has access to the CatalogTable,
>>>> therefore it has access to all the original properties. In turn it knows
>>>> the original format and can call FormatFactory#supportedHintOptions().
>>>>
>>>> Factory can only get CatalogTable when creating source or sink,
>>>> right? IIUC, TableFactory may be stateless too.
>>>> When invoking SourceFactory#supportedHintOptions(), it can not
>>>> get CatalogTable, so it is impossible to create FormatFactory?
>>>>
>>>> Best,
>>>> Jingsong Lee
>>>>
>>>> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yu...@gmail.com> wrote:
>>>>
>>>>> Hi, Dawid, thanks so much for the detail suggestions ~
>>>>>
>>>>> 1. Regarding the motivation:
>>>>>
>>>>> I agree it's not a good suggested way based on the fact that we have
>>>>> better solution, but i think we can support override that as long as it
>>>>> exists as one of the the table options. I would remove if from the
>>>>> motication part.
>>>>>
>>>>> 2. The options passes around during sql-to-rel conversion, right
>>>>> after we
>>>>> generate the RelOptTable (CatalogSourceTable#toRel in Flink), this is
>>>>> indeed a push way method at least in the RelOptTable layer, then we hand
>>>>> over the options to TableSourceFactory with our own context, which is
>>>>> fine
>>>>> becuase TableSourceFactory#Context is the contact to pass around these
>>>>> table-about variables.
>>>>>
>>>>> 3. "We should not end up with an extreme example where we can change the
>>>>> connector type", i totally agree that, and i have listed the
>>>>> "connector.type" as forbidden attribute in the WIKI. As for the
>>>>> format, i
>>>>> think the connector itself can/should control whether to override the
>>>>> "format.type", that is one of the reason i change the
>>>>> TableSourceFactory#supportedHintOpitons to
>>>>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can
>>>>> limit the
>>>>> format keys we want conveniently.
>>>>>
>>>>> 4. SQL Hints syntax.
>>>>>
>>>>>> I think the k and v in the hint_item should be QUOTED_STRING (not sure
>>>>> if it is equivalent to string_literal).
>>>>>
>>>>> I disagree, we at least should keep sync with our DDL: use the string
>>>>> literal as the key. We did also support the simple identifier because
>>>>> this
>>>>> is the common hint syntax from Calcite, it does not hurt anything for
>>>>> the
>>>>> OPTIONS hint, the unsupported keys would validate fails.(If you think
>>>>> that
>>>>> may cause some confuse, i can make the syntax pluggable for each hint in
>>>>> CALCITE 1.23)
>>>>>
>>>>> We only supports OPTIONS hint in the FLIP, and i have changed the
>>>>> title to
>>>>> "Supports dynamic table options", would make it more clear in the WIKI.
>>>>>
>>>>> 5. Yes, we also have this concerns from our offline discussion, that is
>>>>> one of the reason, why i change the
>>>>> TableSourceFactory#supportedHintOpitons
>>>>> to TableSourceFactory#forbiddenHintOpitons, i do not choose Set<String>
>>>>> supportedHintOptionKeys() with wildcard for 2 reasons:
>>>>>
>>>>>     - The wildcard is still not descriptive, we can still not
>>>>> forbidden one
>>>>> of the properties among the wildcard properties, we can not enable or
>>>>> disable them totally
>>>>>     - ConfigOption is our new structure for keys, and it does not support
>>>>> wildcard yet.
>>>>>
>>>>> Best,
>>>>> Danny Chan
>>>>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz
>>>>> <dw...@apache.org>,写道:
>>>>>> Hi,
>>>>>> Few comments from my side:
>>>>>> 1. Regarding the motivation:
>>>>>> I think the example with changing the update-mode is not a good one. In
>>>>> the long term this should be done with EMIT CHANGELOG (discussed in
>>>>> FLIP-105).
>>>>>> Nitpicking: I would mention that it is rather for debugging/ad-hoc
>>>>> solution. I think this should not be a recommended way for production
>>>>> use
>>>>> cases as it bypasses the Catalog, which should be the source of truth.
>>>>>> 2. I could not understand how the additional options will be passed to
>>>>> the TableSourceFactory. Could you elaborate a bit more on that? I see
>>>>> there
>>>>> is a Context interface that gives the options. But cannot find a way
>>>>> to get
>>>>> the context itself in the factory. Moreover I think it would make more
>>>>> sense to have rather a push based approach here. Something like
>>>>> applyOptions(ReadableConfig) method.
>>>>>> 3. As for the concerns Jingsong raised in the voting thread. I think it
>>>>> is not a big problem, but I agree this should be also described. I
>>>>> disagree
>>>>> with "Connector don't know format information in TableFactory before
>>>>> obtains real properties, so it can not list any format
>>>>> `supportedHintOptions`".
>>>>>> When a factory is instantiated it has access to the CatalogTable,
>>>>> therefore it has access to all the original properties. In turn it knows
>>>>> the original format and can call FormatFactory#supportedHintOptions().
>>>>>> The only case when this would not work would be if we allow changing
>>>>>> the
>>>>> format of the Table (e.g. from avro to parquet), which does not sound
>>>>> like
>>>>> a good idea to me. I think this feature should not end up as a way to
>>>>> declare a whole table inline in a SQL query, but should rather be a
>>>>> simple
>>>>> way for debugging queries. We should not end up with an extreme example
>>>>> where we do:
>>>>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
>>>>> 'format.type' = 'json', ....) */
>>>>>> 4. SQL Hints syntax.
>>>>>> I think the k and v in the hint_item should be QUOTED_STRING (not sure
>>>>> if it is equivalent to string_literal). I think we should not use
>>>>> simple_identifier because this implies that we cannot use e.g. any SQL
>>>>> keywords. Anyway it has nothing to do with identifiers. If I am not
>>>>> mistaken it is also how the options in the CREATE statement are
>>>>> implemented.
>>>>>> What is the purpose of the remaining hint_item: hint_name(hint_opt [
>>>>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a
>>>>> feeling it
>>>>> does also suggests to support the whole Apache Calcite hint system
>>>>> without
>>>>> specifying that explicitly. Is the intention of the FLIP to support
>>>>> choosing e.g. JOIN strategies through hints already? If it is so it
>>>>> should
>>>>> be mentioned in the FLIP, imo.
>>>>>> 5. I think something does not work around the supportedHintOptions and
>>>>> wildcards. How do you want to represent a wildcard key as a
>>>>> ConfigOption? I
>>>>> am not sure about that, just a though, maybe it make sense to have
>>>>> rather
>>>>> Set<String> supportedHintOptionKeys()?
>>>>>> Best,
>>>>>> Dawid
>>>>>
>>>>
>>>>
>>
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
Hi, everyone ~

@Aljoscha @Timo

> I think we're designing ourselves into ever more complicated corners
here

I kindly agree that, personally didn't see strong reasons why we should limit on each connector properties:

• we can define any table options for CREATE TABLE, why we treat the dynamic options differently, we never consider any security problems when create table, we should not either for dynamic table options
• If we do not have whitelist properties or blacklist properties, the table source creation work would be much easier, just used the merged options. There is no need to modify each connector to decide which options could be overridden and how we merge them(the merge work is redundant).
• @Timo, how about we support another interface `TableSourceFactory#Context.getExecutionOptions`, we always use this interface to get the options to create our table source. There is no need to copy the catalog table itselt, we just need to generate our Context correctly.
• @Aljoscha I agree to have a global config option, but I disagree to default disable it, a global default config would break the user experience too much, especially when user want to modify the options in a ad-hoc way.



I suggest to remove `TableSourceFactory#supportedHintOptions` or `TableSourceFactory#forbiddenHintOptions` based on the fact that we does not have black/white list for CREATE TABLE at all at lease for current codebase.


@Timo (i have replied offline but allows to represent it here again)

The `TableSourceFactory#supportedHintOptions` doesn't work well for 3 reasons compared to `TableSourceFactory#forbiddenHintOptions`:
1. For key with wildcard, like connector.property.* , use a blacklist make us have the ability to disable some of the keys under that, i.e. connector.property.key1 , a whitelist can only match with prefix

2. We want the connectors to have the ability to disable format type switch format.type but allows all the other properties, e.g. format.* without format.type(let's call it SET_B), if we use the whitelist, we have to enumerate all the specific format keys start with format (SET_B), but with the old connector factories, we have no idea what specific format keys it supports(there is either a format.* or nothing).

3. Except the cases for 1 and 2, for normal keys(no wildcard), the blacklist and whitelist has the same expressiveness, use blacklist makes the code not too verbose to enumerate all the duplicate keys with #supportedKeys .(Not very strong reason, but i think as a connector developer, it makes sense)

Best,
Danny Chan
在 2020年4月3日 +0800 PM11:28,Timo Walther <tw...@apache.org>,写道:
> Hi everyone,
>
> @Aljoscha: I disagree with your approach because a `Catalog` can return
> a custom factory that is not using any properties. The hinting must be
> transparent to a factory. We should NOT modify the metadata
> `CatalogTable` at any point in time after the catalog.
>
> @Danny, @Jingsong: How about we stick to the original design that we
> wanted to vote on but use:
>
> Set<String> supportedHintProperties()
>
> This fits better to the old factory design. And for the new FLIP-95
> factories we will use `ConfigOption` and provide good utilities for
> merging with hints etc.
>
> We can allow `"format.*"` in `supportedHintProperties()` to allow
> hinting in formats.
>
> What do you think?
>
> Regards,
> Timo
>
>
> On 02.04.20 16:24, Aljoscha Krettek wrote:
> > I think we're designing ourselves into ever more complicated corners
> > here. Maybe we need to take a step back and reconsider. What would you
> > think about this (somewhat) simpler proposal:
> >
> > - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or
> > CONNECTOR_PROPERTIES, depending on what naming we want to have for this
> > in the future. This will simply overwrite all connector properties, the
> > table factories don't know about hints but simply work with the
> > properties that they are given
> >
> > - this special hint is disabled by default and can be activated with a
> > global option "foo.bazzle.connector-hints" (or something like this)
> > which has a warning that describes that this can change query semantics
> > etc.
> >
> > That's it. This makes connector implementations a lot easier while still
> > allowing inline configuration.
> >
> > I still don't like using hint syntax at all for this, because I strongly
> > maintain that hints should not change query syntax. In general using
> > hints should be kept to a minimum because they usually point to
> > shortcomings in the system.
> >
> > Best,
> > Aljoscha
> >
> > On 02.04.20 06:06, Jingsong Li wrote:
> > > Hi Dawid,
> > >
> > > > When a factory is instantiated it has access to the CatalogTable,
> > > therefore it has access to all the original properties. In turn it knows
> > > the original format and can call FormatFactory#supportedHintOptions().
> > >
> > > Factory can only get CatalogTable when creating source or sink,
> > > right? IIUC, TableFactory may be stateless too.
> > > When invoking SourceFactory#supportedHintOptions(), it can not
> > > get CatalogTable, so it is impossible to create FormatFactory?
> > >
> > > Best,
> > > Jingsong Lee
> > >
> > > On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yu...@gmail.com> wrote:
> > >
> > > > Hi, Dawid, thanks so much for the detail suggestions ~
> > > >
> > > > 1. Regarding the motivation:
> > > >
> > > > I agree it's not a good suggested way based on the fact that we have
> > > > better solution, but i think we can support override that as long as it
> > > > exists as one of the the table options. I would remove if from the
> > > > motication part.
> > > >
> > > > 2. The options passes around during sql-to-rel conversion, right
> > > > after we
> > > > generate the RelOptTable (CatalogSourceTable#toRel in Flink), this is
> > > > indeed a push way method at least in the RelOptTable layer, then we hand
> > > > over the options to TableSourceFactory with our own context, which is
> > > > fine
> > > > becuase TableSourceFactory#Context is the contact to pass around these
> > > > table-about variables.
> > > >
> > > > 3. "We should not end up with an extreme example where we can change the
> > > > connector type", i totally agree that, and i have listed the
> > > > "connector.type" as forbidden attribute in the WIKI. As for the
> > > > format, i
> > > > think the connector itself can/should control whether to override the
> > > > "format.type", that is one of the reason i change the
> > > > TableSourceFactory#supportedHintOpitons to
> > > > TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can
> > > > limit the
> > > > format keys we want conveniently.
> > > >
> > > > 4. SQL Hints syntax.
> > > >
> > > > > I think the k and v in the hint_item should be QUOTED_STRING (not sure
> > > > if it is equivalent to string_literal).
> > > >
> > > > I disagree, we at least should keep sync with our DDL: use the string
> > > > literal as the key. We did also support the simple identifier because
> > > > this
> > > > is the common hint syntax from Calcite, it does not hurt anything for
> > > > the
> > > > OPTIONS hint, the unsupported keys would validate fails.(If you think
> > > > that
> > > > may cause some confuse, i can make the syntax pluggable for each hint in
> > > > CALCITE 1.23)
> > > >
> > > > We only supports OPTIONS hint in the FLIP, and i have changed the
> > > > title to
> > > > "Supports dynamic table options", would make it more clear in the WIKI.
> > > >
> > > > 5. Yes, we also have this concerns from our offline discussion, that is
> > > > one of the reason, why i change the
> > > > TableSourceFactory#supportedHintOpitons
> > > > to TableSourceFactory#forbiddenHintOpitons, i do not choose Set<String>
> > > > supportedHintOptionKeys() with wildcard for 2 reasons:
> > > >
> > > >    - The wildcard is still not descriptive, we can still not
> > > > forbidden one
> > > > of the properties among the wildcard properties, we can not enable or
> > > > disable them totally
> > > >    - ConfigOption is our new structure for keys, and it does not support
> > > > wildcard yet.
> > > >
> > > > Best,
> > > > Danny Chan
> > > > 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz
> > > > <dw...@apache.org>,写道:
> > > > > Hi,
> > > > > Few comments from my side:
> > > > > 1. Regarding the motivation:
> > > > > I think the example with changing the update-mode is not a good one. In
> > > > the long term this should be done with EMIT CHANGELOG (discussed in
> > > > FLIP-105).
> > > > > Nitpicking: I would mention that it is rather for debugging/ad-hoc
> > > > solution. I think this should not be a recommended way for production
> > > > use
> > > > cases as it bypasses the Catalog, which should be the source of truth.
> > > > > 2. I could not understand how the additional options will be passed to
> > > > the TableSourceFactory. Could you elaborate a bit more on that? I see
> > > > there
> > > > is a Context interface that gives the options. But cannot find a way
> > > > to get
> > > > the context itself in the factory. Moreover I think it would make more
> > > > sense to have rather a push based approach here. Something like
> > > > applyOptions(ReadableConfig) method.
> > > > > 3. As for the concerns Jingsong raised in the voting thread. I think it
> > > > is not a big problem, but I agree this should be also described. I
> > > > disagree
> > > > with "Connector don't know format information in TableFactory before
> > > > obtains real properties, so it can not list any format
> > > > `supportedHintOptions`".
> > > > > When a factory is instantiated it has access to the CatalogTable,
> > > > therefore it has access to all the original properties. In turn it knows
> > > > the original format and can call FormatFactory#supportedHintOptions().
> > > > > The only case when this would not work would be if we allow changing
> > > > > the
> > > > format of the Table (e.g. from avro to parquet), which does not sound
> > > > like
> > > > a good idea to me. I think this feature should not end up as a way to
> > > > declare a whole table inline in a SQL query, but should rather be a
> > > > simple
> > > > way for debugging queries. We should not end up with an extreme example
> > > > where we do:
> > > > > SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
> > > > 'format.type' = 'json', ....) */
> > > > > 4. SQL Hints syntax.
> > > > > I think the k and v in the hint_item should be QUOTED_STRING (not sure
> > > > if it is equivalent to string_literal). I think we should not use
> > > > simple_identifier because this implies that we cannot use e.g. any SQL
> > > > keywords. Anyway it has nothing to do with identifiers. If I am not
> > > > mistaken it is also how the options in the CREATE statement are
> > > > implemented.
> > > > > What is the purpose of the remaining hint_item: hint_name(hint_opt [
> > > > ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a
> > > > feeling it
> > > > does also suggests to support the whole Apache Calcite hint system
> > > > without
> > > > specifying that explicitly. Is the intention of the FLIP to support
> > > > choosing e.g. JOIN strategies through hints already? If it is so it
> > > > should
> > > > be mentioned in the FLIP, imo.
> > > > > 5. I think something does not work around the supportedHintOptions and
> > > > wildcards. How do you want to represent a wildcard key as a
> > > > ConfigOption? I
> > > > am not sure about that, just a though, maybe it make sense to have
> > > > rather
> > > > Set<String> supportedHintOptionKeys()?
> > > > > Best,
> > > > > Dawid
> > > >
> > >
> > >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@apache.org>.
Hi everyone,

@Aljoscha: I disagree with your approach because a `Catalog` can return 
a custom factory that is not using any properties. The hinting must be 
transparent to a factory. We should NOT modify the metadata 
`CatalogTable` at any point in time after the catalog.

@Danny, @Jingsong: How about we stick to the original design that we 
wanted to vote on but use:

Set<String> supportedHintProperties()

This fits better to the old factory design. And for the new FLIP-95 
factories we will use `ConfigOption` and provide good utilities for 
merging with hints etc.

We can allow `"format.*"` in `supportedHintProperties()` to allow 
hinting in formats.

What do you think?

Regards,
Timo


On 02.04.20 16:24, Aljoscha Krettek wrote:
> I think we're designing ourselves into ever more complicated corners 
> here. Maybe we need to take a step back and reconsider. What would you 
> think about this (somewhat) simpler proposal:
> 
> - introduce a hint called CONNECTOR_OPTIONS(k=v,...). or 
> CONNECTOR_PROPERTIES, depending on what naming we want to have for this 
> in the future. This will simply overwrite all connector properties, the 
> table factories don't know about hints but simply work with the 
> properties that they are given
> 
> - this special hint is disabled by default and can be activated with a 
> global option "foo.bazzle.connector-hints" (or something like this) 
> which has a warning that describes that this can change query semantics 
> etc.
> 
> That's it. This makes connector implementations a lot easier while still 
> allowing inline configuration.
> 
> I still don't like using hint syntax at all for this, because I strongly 
> maintain that hints should not change query syntax. In general using 
> hints should be kept to a minimum because they usually point to 
> shortcomings in the system.
> 
> Best,
> Aljoscha
> 
> On 02.04.20 06:06, Jingsong Li wrote:
>> Hi Dawid,
>>
>>> When a factory is instantiated it has access to the CatalogTable,
>> therefore it has access to all the original properties. In turn it knows
>> the original format and can call FormatFactory#supportedHintOptions().
>>
>> Factory can only get CatalogTable when creating source or sink,
>> right? IIUC, TableFactory may be stateless too.
>> When invoking SourceFactory#supportedHintOptions(), it can not
>> get CatalogTable, so it is impossible to create FormatFactory?
>>
>> Best,
>> Jingsong Lee
>>
>> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yu...@gmail.com> wrote:
>>
>>> Hi, Dawid, thanks so much for the detail suggestions ~
>>>
>>> 1. Regarding the motivation:
>>>
>>> I agree it's not a good suggested way based on the fact that we have
>>> better solution, but i think we can support override that as long as it
>>> exists as one of the the table options. I would remove if from the
>>> motication part.
>>>
>>> 2. The options passes around during sql-to-rel conversion, right 
>>> after we
>>> generate the RelOptTable (CatalogSourceTable#toRel in Flink), this is
>>> indeed a push way method at least in the RelOptTable layer, then we hand
>>> over the options to TableSourceFactory with our own context, which is 
>>> fine
>>> becuase TableSourceFactory#Context is the contact to pass around these
>>> table-about variables.
>>>
>>> 3. "We should not end up with an extreme example where we can change the
>>> connector type", i totally agree that, and i have listed the
>>> "connector.type" as forbidden attribute in the WIKI. As for the 
>>> format, i
>>> think the connector itself can/should control whether to override the
>>> "format.type", that is one of the reason i change the
>>> TableSourceFactory#supportedHintOpitons to
>>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can 
>>> limit the
>>> format keys we want conveniently.
>>>
>>> 4. SQL Hints syntax.
>>>
>>>> I think the k and v in the hint_item should be QUOTED_STRING (not sure
>>> if it is equivalent to string_literal).
>>>
>>> I disagree, we at least should keep sync with our DDL: use the string
>>> literal as the key. We did also support the simple identifier because 
>>> this
>>> is the common hint syntax from Calcite, it does not hurt anything for 
>>> the
>>> OPTIONS hint, the unsupported keys would validate fails.(If you think 
>>> that
>>> may cause some confuse, i can make the syntax pluggable for each hint in
>>> CALCITE 1.23)
>>>
>>> We only supports OPTIONS hint in the FLIP, and i have changed the 
>>> title to
>>> "Supports dynamic table options", would make it more clear in the WIKI.
>>>
>>> 5. Yes, we also have this concerns from our offline discussion, that is
>>> one of the reason, why i change the 
>>> TableSourceFactory#supportedHintOpitons
>>> to TableSourceFactory#forbiddenHintOpitons, i do not choose Set<String>
>>> supportedHintOptionKeys() with wildcard for 2 reasons:
>>>
>>>    - The wildcard is still not descriptive, we can still not 
>>> forbidden one
>>> of the properties among the wildcard properties, we can not enable or
>>> disable them totally
>>>    - ConfigOption is our new structure for keys, and it does not support
>>> wildcard yet.
>>>
>>> Best,
>>> Danny Chan
>>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz 
>>> <dw...@apache.org>,写道:
>>>> Hi,
>>>> Few comments from my side:
>>>> 1. Regarding the motivation:
>>>> I think the example with changing the update-mode is not a good one. In
>>> the long term this should be done with EMIT CHANGELOG (discussed in
>>> FLIP-105).
>>>> Nitpicking: I would mention that it is rather for debugging/ad-hoc
>>> solution. I think this should not be a recommended way for production 
>>> use
>>> cases as it bypasses the Catalog, which should be the source of truth.
>>>> 2. I could not understand how the additional options will be passed to
>>> the TableSourceFactory. Could you elaborate a bit more on that? I see 
>>> there
>>> is a Context interface that gives the options. But cannot find a way 
>>> to get
>>> the context itself in the factory. Moreover I think it would make more
>>> sense to have rather a push based approach here. Something like
>>> applyOptions(ReadableConfig) method.
>>>> 3. As for the concerns Jingsong raised in the voting thread. I think it
>>> is not a big problem, but I agree this should be also described. I 
>>> disagree
>>> with "Connector don't know format information in TableFactory before
>>> obtains real properties, so it can not list any format
>>> `supportedHintOptions`".
>>>> When a factory is instantiated it has access to the CatalogTable,
>>> therefore it has access to all the original properties. In turn it knows
>>> the original format and can call FormatFactory#supportedHintOptions().
>>>> The only case when this would not work would be if we allow changing 
>>>> the
>>> format of the Table (e.g. from avro to parquet), which does not sound 
>>> like
>>> a good idea to me. I think this feature should not end up as a way to
>>> declare a whole table inline in a SQL query, but should rather be a 
>>> simple
>>> way for debugging queries. We should not end up with an extreme example
>>> where we do:
>>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
>>> 'format.type' = 'json', ....) */
>>>> 4. SQL Hints syntax.
>>>> I think the k and v in the hint_item should be QUOTED_STRING (not sure
>>> if it is equivalent to string_literal). I think we should not use
>>> simple_identifier because this implies that we cannot use e.g. any SQL
>>> keywords. Anyway it has nothing to do with identifiers. If I am not
>>> mistaken it is also how the options in the CREATE statement are 
>>> implemented.
>>>> What is the purpose of the remaining hint_item: hint_name(hint_opt [
>>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a 
>>> feeling it
>>> does also suggests to support the whole Apache Calcite hint system 
>>> without
>>> specifying that explicitly. Is the intention of the FLIP to support
>>> choosing e.g. JOIN strategies through hints already? If it is so it 
>>> should
>>> be mentioned in the FLIP, imo.
>>>> 5. I think something does not work around the supportedHintOptions and
>>> wildcards. How do you want to represent a wildcard key as a 
>>> ConfigOption? I
>>> am not sure about that, just a though, maybe it make sense to have 
>>> rather
>>> Set<String> supportedHintOptionKeys()?
>>>> Best,
>>>> Dawid
>>>
>>
>>


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Aljoscha Krettek <al...@apache.org>.
I think we're designing ourselves into ever more complicated corners 
here. Maybe we need to take a step back and reconsider. What would you 
think about this (somewhat) simpler proposal:

- introduce a hint called CONNECTOR_OPTIONS(k=v,...). or 
CONNECTOR_PROPERTIES, depending on what naming we want to have for this 
in the future. This will simply overwrite all connector properties, the 
table factories don't know about hints but simply work with the 
properties that they are given

- this special hint is disabled by default and can be activated with a 
global option "foo.bazzle.connector-hints" (or something like this) 
which has a warning that describes that this can change query semantics etc.

That's it. This makes connector implementations a lot easier while still 
allowing inline configuration.

I still don't like using hint syntax at all for this, because I strongly 
maintain that hints should not change query syntax. In general using 
hints should be kept to a minimum because they usually point to 
shortcomings in the system.

Best,
Aljoscha

On 02.04.20 06:06, Jingsong Li wrote:
> Hi Dawid,
> 
>> When a factory is instantiated it has access to the CatalogTable,
> therefore it has access to all the original properties. In turn it knows
> the original format and can call FormatFactory#supportedHintOptions().
> 
> Factory can only get CatalogTable when creating source or sink,
> right? IIUC, TableFactory may be stateless too.
> When invoking SourceFactory#supportedHintOptions(), it can not
> get CatalogTable, so it is impossible to create FormatFactory?
> 
> Best,
> Jingsong Lee
> 
> On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yu...@gmail.com> wrote:
> 
>> Hi, Dawid, thanks so much for the detail suggestions ~
>>
>> 1. Regarding the motivation:
>>
>> I agree it's not a good suggested way based on the fact that we have
>> better solution, but i think we can support override that as long as it
>> exists as one of the the table options. I would remove if from the
>> motication part.
>>
>> 2. The options passes around during sql-to-rel conversion, right after we
>> generate the RelOptTable (CatalogSourceTable#toRel in Flink), this is
>> indeed a push way method at least in the RelOptTable layer, then we hand
>> over the options to TableSourceFactory with our own context, which is fine
>> becuase TableSourceFactory#Context is the contact to pass around these
>> table-about variables.
>>
>> 3. "We should not end up with an extreme example where we can change the
>> connector type", i totally agree that, and i have listed the
>> "connector.type" as forbidden attribute in the WIKI. As for the format, i
>> think the connector itself can/should control whether to override the
>> "format.type", that is one of the reason i change the
>> TableSourceFactory#supportedHintOpitons to
>> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can limit the
>> format keys we want conveniently.
>>
>> 4. SQL Hints syntax.
>>
>>> I think the k and v in the hint_item should be QUOTED_STRING (not sure
>> if it is equivalent to string_literal).
>>
>> I disagree, we at least should keep sync with our DDL: use the string
>> literal as the key. We did also support the simple identifier because this
>> is the common hint syntax from Calcite, it does not hurt anything for the
>> OPTIONS hint, the unsupported keys would validate fails.(If you think that
>> may cause some confuse, i can make the syntax pluggable for each hint in
>> CALCITE 1.23)
>>
>> We only supports OPTIONS hint in the FLIP, and i have changed the title to
>> "Supports dynamic table options", would make it more clear in the WIKI.
>>
>> 5. Yes, we also have this concerns from our offline discussion, that is
>> one of the reason, why i change the TableSourceFactory#supportedHintOpitons
>> to TableSourceFactory#forbiddenHintOpitons, i do not choose Set<String>
>> supportedHintOptionKeys() with wildcard for 2 reasons:
>>
>>    - The wildcard is still not descriptive, we can still not forbidden one
>> of the properties among the wildcard properties, we can not enable or
>> disable them totally
>>    - ConfigOption is our new structure for keys, and it does not support
>> wildcard yet.
>>
>> Best,
>> Danny Chan
>> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz <dw...@apache.org>,写道:
>>> Hi,
>>> Few comments from my side:
>>> 1. Regarding the motivation:
>>> I think the example with changing the update-mode is not a good one. In
>> the long term this should be done with EMIT CHANGELOG (discussed in
>> FLIP-105).
>>> Nitpicking: I would mention that it is rather for debugging/ad-hoc
>> solution. I think this should not be a recommended way for production use
>> cases as it bypasses the Catalog, which should be the source of truth.
>>> 2. I could not understand how the additional options will be passed to
>> the TableSourceFactory. Could you elaborate a bit more on that? I see there
>> is a Context interface that gives the options. But cannot find a way to get
>> the context itself in the factory. Moreover I think it would make more
>> sense to have rather a push based approach here. Something like
>> applyOptions(ReadableConfig) method.
>>> 3. As for the concerns Jingsong raised in the voting thread. I think it
>> is not a big problem, but I agree this should be also described. I disagree
>> with "Connector don't know format information in TableFactory before
>> obtains real properties, so it can not list any format
>> `supportedHintOptions`".
>>> When a factory is instantiated it has access to the CatalogTable,
>> therefore it has access to all the original properties. In turn it knows
>> the original format and can call FormatFactory#supportedHintOptions().
>>> The only case when this would not work would be if we allow changing the
>> format of the Table (e.g. from avro to parquet), which does not sound like
>> a good idea to me. I think this feature should not end up as a way to
>> declare a whole table inline in a SQL query, but should rather be a simple
>> way for debugging queries. We should not end up with an extreme example
>> where we do:
>>> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
>> 'format.type' = 'json', ....) */
>>> 4. SQL Hints syntax.
>>> I think the k and v in the hint_item should be QUOTED_STRING (not sure
>> if it is equivalent to string_literal). I think we should not use
>> simple_identifier because this implies that we cannot use e.g. any SQL
>> keywords. Anyway it has nothing to do with identifiers. If I am not
>> mistaken it is also how the options in the CREATE statement are implemented.
>>> What is the purpose of the remaining hint_item: hint_name(hint_opt [
>> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a feeling it
>> does also suggests to support the whole Apache Calcite hint system without
>> specifying that explicitly. Is the intention of the FLIP to support
>> choosing e.g. JOIN strategies through hints already? If it is so it should
>> be mentioned in the FLIP, imo.
>>> 5. I think something does not work around the supportedHintOptions and
>> wildcards. How do you want to represent a wildcard key as a ConfigOption? I
>> am not sure about that, just a though, maybe it make sense to have rather
>> Set<String> supportedHintOptionKeys()?
>>> Best,
>>> Dawid
>>
> 
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jingsong Li <ji...@gmail.com>.
Hi Dawid,

> When a factory is instantiated it has access to the CatalogTable,
therefore it has access to all the original properties. In turn it knows
the original format and can call FormatFactory#supportedHintOptions().

Factory can only get CatalogTable when creating source or sink,
right? IIUC, TableFactory may be stateless too.
When invoking SourceFactory#supportedHintOptions(), it can not
get CatalogTable, so it is impossible to create FormatFactory?

Best,
Jingsong Lee

On Wed, Apr 1, 2020 at 10:05 PM Danny Chan <yu...@gmail.com> wrote:

> Hi, Dawid, thanks so much for the detail suggestions ~
>
> 1. Regarding the motivation:
>
> I agree it's not a good suggested way based on the fact that we have
> better solution, but i think we can support override that as long as it
> exists as one of the the table options. I would remove if from the
> motication part.
>
> 2. The options passes around during sql-to-rel conversion, right after we
> generate the RelOptTable (CatalogSourceTable#toRel in Flink), this is
> indeed a push way method at least in the RelOptTable layer, then we hand
> over the options to TableSourceFactory with our own context, which is fine
> becuase TableSourceFactory#Context is the contact to pass around these
> table-about variables.
>
> 3. "We should not end up with an extreme example where we can change the
> connector type", i totally agree that, and i have listed the
> "connector.type" as forbidden attribute in the WIKI. As for the format, i
> think the connector itself can/should control whether to override the
> "format.type", that is one of the reason i change the
> TableSourceFactory#supportedHintOpitons to
> TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can limit the
> format keys we want conveniently.
>
> 4. SQL Hints syntax.
>
> > I think the k and v in the hint_item should be QUOTED_STRING (not sure
> if it is equivalent to string_literal).
>
> I disagree, we at least should keep sync with our DDL: use the string
> literal as the key. We did also support the simple identifier because this
> is the common hint syntax from Calcite, it does not hurt anything for the
> OPTIONS hint, the unsupported keys would validate fails.(If you think that
> may cause some confuse, i can make the syntax pluggable for each hint in
> CALCITE 1.23)
>
> We only supports OPTIONS hint in the FLIP, and i have changed the title to
> "Supports dynamic table options", would make it more clear in the WIKI.
>
> 5. Yes, we also have this concerns from our offline discussion, that is
> one of the reason, why i change the TableSourceFactory#supportedHintOpitons
> to TableSourceFactory#forbiddenHintOpitons, i do not choose Set<String>
> supportedHintOptionKeys() with wildcard for 2 reasons:
>
>   - The wildcard is still not descriptive, we can still not forbidden one
> of the properties among the wildcard properties, we can not enable or
> disable them totally
>   - ConfigOption is our new structure for keys, and it does not support
> wildcard yet.
>
> Best,
> Danny Chan
> 在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz <dw...@apache.org>,写道:
> > Hi,
> > Few comments from my side:
> > 1. Regarding the motivation:
> > I think the example with changing the update-mode is not a good one. In
> the long term this should be done with EMIT CHANGELOG (discussed in
> FLIP-105).
> > Nitpicking: I would mention that it is rather for debugging/ad-hoc
> solution. I think this should not be a recommended way for production use
> cases as it bypasses the Catalog, which should be the source of truth.
> > 2. I could not understand how the additional options will be passed to
> the TableSourceFactory. Could you elaborate a bit more on that? I see there
> is a Context interface that gives the options. But cannot find a way to get
> the context itself in the factory. Moreover I think it would make more
> sense to have rather a push based approach here. Something like
> applyOptions(ReadableConfig) method.
> > 3. As for the concerns Jingsong raised in the voting thread. I think it
> is not a big problem, but I agree this should be also described. I disagree
> with "Connector don't know format information in TableFactory before
> obtains real properties, so it can not list any format
> `supportedHintOptions`".
> > When a factory is instantiated it has access to the CatalogTable,
> therefore it has access to all the original properties. In turn it knows
> the original format and can call FormatFactory#supportedHintOptions().
> > The only case when this would not work would be if we allow changing the
> format of the Table (e.g. from avro to parquet), which does not sound like
> a good idea to me. I think this feature should not end up as a way to
> declare a whole table inline in a SQL query, but should rather be a simple
> way for debugging queries. We should not end up with an extreme example
> where we do:
> > SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
> 'format.type' = 'json', ....) */
> > 4. SQL Hints syntax.
> > I think the k and v in the hint_item should be QUOTED_STRING (not sure
> if it is equivalent to string_literal). I think we should not use
> simple_identifier because this implies that we cannot use e.g. any SQL
> keywords. Anyway it has nothing to do with identifiers. If I am not
> mistaken it is also how the options in the CREATE statement are implemented.
> > What is the purpose of the remaining hint_item: hint_name(hint_opt [
> ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a feeling it
> does also suggests to support the whole Apache Calcite hint system without
> specifying that explicitly. Is the intention of the FLIP to support
> choosing e.g. JOIN strategies through hints already? If it is so it should
> be mentioned in the FLIP, imo.
> > 5. I think something does not work around the supportedHintOptions and
> wildcards. How do you want to represent a wildcard key as a ConfigOption? I
> am not sure about that, just a though, maybe it make sense to have rather
> Set<String> supportedHintOptionKeys()?
> > Best,
> > Dawid
>


-- 
Best, Jingsong Lee

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
Hi, Dawid, thanks so much for the detail suggestions ~

1. Regarding the motivation:

I agree it's not a good suggested way based on the fact that we have better solution, but i think we can support override that as long as it exists as one of the the table options. I would remove if from the motication part.

2. The options passes around during sql-to-rel conversion, right after we generate the RelOptTable (CatalogSourceTable#toRel in Flink), this is indeed a push way method at least in the RelOptTable layer, then we hand over the options to TableSourceFactory with our own context, which is fine becuase TableSourceFactory#Context is the contact to pass around these table-about variables.

3. "We should not end up with an extreme example where we can change the connector type", i totally agree that, and i have listed the "connector.type" as forbidden attribute in the WIKI. As for the format, i think the connector itself can/should control whether to override the "format.type", that is one of the reason i change the TableSourceFactory#supportedHintOpitons to
TableSourceFactory#forbiddenHintOpitons, use a blacklist, we can limit the format keys we want conveniently.

4. SQL Hints syntax.

> I think the k and v in the hint_item should be QUOTED_STRING (not sure if it is equivalent to string_literal).

I disagree, we at least should keep sync with our DDL: use the string literal as the key. We did also support the simple identifier because this is the common hint syntax from Calcite, it does not hurt anything for the OPTIONS hint, the unsupported keys would validate fails.(If you think that may cause some confuse, i can make the syntax pluggable for each hint in CALCITE 1.23)

We only supports OPTIONS hint in the FLIP, and i have changed the title to "Supports dynamic table options", would make it more clear in the WIKI.

5. Yes, we also have this concerns from our offline discussion, that is one of the reason, why i change the TableSourceFactory#supportedHintOpitons to TableSourceFactory#forbiddenHintOpitons, i do not choose Set<String> supportedHintOptionKeys() with wildcard for 2 reasons:

  - The wildcard is still not descriptive, we can still not forbidden one of the properties among the wildcard properties, we can not enable or disable them totally
  - ConfigOption is our new structure for keys, and it does not support wildcard yet.

Best,
Danny Chan
在 2020年4月1日 +0800 PM9:12,Dawid Wysakowicz <dw...@apache.org>,写道:
> Hi,
> Few comments from my side:
> 1. Regarding the motivation:
> I think the example with changing the update-mode is not a good one. In the long term this should be done with EMIT CHANGELOG (discussed in FLIP-105).
> Nitpicking: I would mention that it is rather for debugging/ad-hoc solution. I think this should not be a recommended way for production use cases as it bypasses the Catalog, which should be the source of truth.
> 2. I could not understand how the additional options will be passed to the TableSourceFactory. Could you elaborate a bit more on that? I see there is a Context interface that gives the options. But cannot find a way to get the context itself in the factory. Moreover I think it would make more sense to have rather a push based approach here. Something like applyOptions(ReadableConfig) method.
> 3. As for the concerns Jingsong raised in the voting thread. I think it is not a big problem, but I agree this should be also described. I disagree with "Connector don't know format information in TableFactory before obtains real properties, so it can not list any format `supportedHintOptions`".
> When a factory is instantiated it has access to the CatalogTable, therefore it has access to all the original properties. In turn it knows the original format and can call FormatFactory#supportedHintOptions().
> The only case when this would not work would be if we allow changing the format of the Table (e.g. from avro to parquet), which does not sound like a good idea to me. I think this feature should not end up as a way to declare a whole table inline in a SQL query, but should rather be a simple way for debugging queries. We should not end up with an extreme example where we do:
> SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ..., 'format.type' = 'json', ....) */
> 4. SQL Hints syntax.
> I think the k and v in the hint_item should be QUOTED_STRING (not sure if it is equivalent to string_literal). I think we should not use simple_identifier because this implies that we cannot use e.g. any SQL keywords. Anyway it has nothing to do with identifiers. If I am not mistaken it is also how the options in the CREATE statement are implemented.
> What is the purpose of the remaining hint_item: hint_name(hint_opt [ ,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a feeling it does also suggests to support the whole Apache Calcite hint system without specifying that explicitly. Is the intention of the FLIP to support choosing e.g. JOIN strategies through hints already? If it is so it should be mentioned in the FLIP, imo.
> 5. I think something does not work around the supportedHintOptions and wildcards. How do you want to represent a wildcard key as a ConfigOption? I am not sure about that, just a though, maybe it make sense to have rather Set<String> supportedHintOptionKeys()?
> Best,
> Dawid

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Dawid Wysakowicz <dw...@apache.org>.
Hi,

Few comments from my side:

1. Regarding the motivation:

I think the example with changing the update-mode is not a good one. In
the long term this should be done with EMIT CHANGELOG (discussed in
FLIP-105).

Nitpicking: I would mention that it is rather for debugging/ad-hoc
solution. I think this should not be a recommended way for production
use cases as it bypasses the Catalog, which should be the source of truth.

2. I could not understand how the additional options will be passed to
the TableSourceFactory. Could you elaborate a bit more on that? I see
there is a Context interface that gives the options. But cannot find a
way to get the context itself in the factory. Moreover I think it would
make more sense to have rather a push based approach here. Something
like applyOptions(ReadableConfig) method.

3. As for the concerns Jingsong raised in the voting thread. I think it
is not a big problem, but I agree this should be also described. I
disagree with "Connector don't know format information in TableFactory
before obtains real properties, so it can not list any format
`supportedHintOptions`".

When a factory is instantiated it has access to the CatalogTable,
therefore it has access to all the original properties. In turn it knows
the original format and can call FormatFactory#supportedHintOptions().

The only case when this would not work would be if we allow changing the
format of the Table (e.g. from avro to parquet), which does not sound
like a good idea to me. I think this feature should not end up as a way
to declare a whole table inline in a SQL query, but should rather be a
simple way for debugging queries. We should not end up with an extreme
example where we do:

|SELECT * FROM myTable /* OPTIONS('connector.type'='kafka', ...,
'format.type' = 'json', ....) */|

4. SQL Hints syntax.

I think the k and v in the hint_item should be QUOTED_STRING (not sure
if it is equivalent to string_literal). I think we should not use
simple_identifier because this implies that we cannot use e.g. any SQL
keywords. Anyway it has nothing to do with identifiers. If I am not
mistaken it is also how the options in the CREATE statement are implemented.

What is the purpose of the remaining hint_item: hint_name(hint_opt [
,hint_opt ]*)? It is not discussed in the FLIP. Moreover I got a feeling
it does also suggests to support the whole Apache Calcite hint system
without specifying that explicitly. Is the intention of the FLIP to
support choosing e.g. JOIN strategies through hints already? If it is so
it should be mentioned in the FLIP, imo.

5. I think something does not work around the supportedHintOptions and
wildcards. How do you want to represent a wildcard key as a
ConfigOption? I am not sure about that, just a though, maybe it make
sense to have rather Set<String> supportedHintOptionKeys()?

Best,

Dawid

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <da...@apache.org>.
Sorry, i meant white-list ~

Danny Chan <da...@apache.org> 于2020年3月27日周五 下午12:40写道:

> Thanks everyone for the feedback ~
>
> - For the global config option belongs to `ExecutionConfigOptions` or
> `OptimizerConfigOptions`, i have to strong objections, switch
> to `OptimizerConfigOptions` is okey to me and i have updated the WIKI
> - For use while-list or black-list, i have opinion with Timo, so black-list
>
> I would fire a Vote if there are no other objections soon, thanks ~
>
> Timo Walther <tw...@apache.org> 于2020年3月26日周四 下午6:31写道:
>
>> Hi everyone,
>>
>> it is not only about security concerns. Hint options should be
>> well-defined. We had a couple of people that were concerned about
>> changing the semantics with a concept that is called "hint". These
>> options are more like "debugging options" while someone is developing a
>> connector or using a notebook to quickly produce some rows.
>>
>> The final pipeline should use a temporary table instead. I suggest to
>> use a whitelist and force people to think about what should be exposed
>> as a hint. By default, no option should be exposed. It is better to be
>> conservative here.
>>
>> Regards,
>> Timo
>>
>>
>> On 26.03.20 10:31, Danny Chan wrote:
>> > Thanks Kurt for the suggestion ~
>> >
>> > In my opinion:
>> > - There is no need for TableFormatFactory#supportedHintOptions because
>> all
>> > the format options can be configured dynamically, they have no security
>> > issues
>> > - Dynamic table options is not an optimization, it is more like an
>> > execution behavior from my side
>> >
>> > Kurt Young <yk...@gmail.com> 于2020年3月26日周四 下午4:47写道:
>> >
>> >> Hi Danny,
>> >>
>> >> Thanks for the updates. I have 2 comments regarding to latest document:
>> >>
>> >> 1) I think we also need `*supportedHintOptions*` for
>> >> `*TableFormatFactory*`
>> >> 2) IMO "dynamic-table-options.enabled" should belong to `
>> >> *OptimizerConfigOptions*`
>> >>
>> >> Best,
>> >> Kurt
>> >>
>> >>
>> >> On Thu, Mar 26, 2020 at 4:40 PM Timo Walther <tw...@apache.org>
>> wrote:
>> >>
>> >>> Thanks for the update Danny. +1 for this proposal.
>> >>>
>> >>> Regards,
>> >>> Timo
>> >>>
>> >>> On 26.03.20 04:51, Danny Chan wrote:
>> >>>> Thanks everyone who engaged in this discussion ~
>> >>>>
>> >>>> Our goal is "Supports Dynamic Table Options for Flink SQL". After an
>> >>>> offline discussion with Kurt, Timo and Dawid, we have made the final
>> >>>> conclusion, here is the summary:
>> >>>>
>> >>>>
>> >>>>      - Use comment style syntax to specify the dynamic table options:
>> >> "/*+
>> >>>>      *OPTIONS*(k1='v1', k2='v2') */"
>> >>>>      - Have constraint on the options keys: the options that may
>> bring
>> >> in
>> >>>>      security problems should not be allowed, i.e. Kafka connector
>> >>> zookeeper
>> >>>>      endpoint URL and topic name
>> >>>>      - Use white-list to control the allowed options for each
>> connector,
>> >>>>      which is more safe for future extention
>> >>>>      - We allow to enable/disable this feature globally
>> >>>>      - Implement based on the current code base first, and when
>> FLIP-95
>> >> is
>> >>>>      checked in, implement this feature based on new interface
>> >>>>
>> >>>> Any suggestions are appreciated ~
>> >>>>
>> >>>> [1]
>> >>>>
>> >>>
>> >>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
>> >>>>
>> >>>> Best,
>> >>>> Danny Chan
>> >>>>
>> >>>> Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:
>> >>>>
>> >>>>> Hi everyone,
>> >>>>>
>> >>>>> Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid
>> >> it
>> >>>>> doesn't solve the problems but increases some development and
>> learning
>> >>>>> burdens.
>> >>>>>
>> >>>>> # increase development and learning burden
>> >>>>>
>> >>>>> According to the discussion so far, we want to support overriding a
>> >>> subset
>> >>>>> of options in hints which doesn't affect semantics.
>> >>>>> With the `supportedHintOptions`, it's up to the connector developers
>> >> to
>> >>>>> decide which options will not affect semantics, and to be hint
>> >> options.
>> >>>>> However, the question is how to distinguish whether an option will
>> >>> *affect
>> >>>>> semantics*? What happens if an option will affect semantics but
>> >>> provided as
>> >>>>> hint options?
>> >>>>>   From my point of view, it's not easy to distinguish. For example,
>> the
>> >>>>> "format.ignore-parse-error" can be a very useful dynamic option but
>> >> that
>> >>>>> will affect semantic, because the result is different (null vs
>> >>> exception).
>> >>>>> Another example, the "connector.lookup.cache.*" options are also
>> very
>> >>>>> useful to tune jobs, however, it will also affect the job results. I
>> >> can
>> >>>>> come up many more useful options but may affect semantics.
>> >>>>>
>> >>>>> I can see that the community will under endless discussion around
>> "can
>> >>> this
>> >>>>> option to be a hint option?",  "wether this option will affect
>> >>> semantics?".
>> >>>>> You can also find that we already have different opinions on
>> >>>>> "ignore-parse-error". Those discussion is a waste of time! That's
>> not
>> >>> what
>> >>>>> users want!
>> >>>>> The problem is user need this, this, this options and HOW to expose
>> >>> them?
>> >>>>> We should focus on that.
>> >>>>>
>> >>>>> Then there could be two endings in the future:
>> >>>>> 1) compromise on the usability, we drop the rule that hints don't
>> >> affect
>> >>>>> semantics, allow all the useful options in the hints list.
>> >>>>> 2) stick on the rule, users will find this is a stumbling feature
>> >> which
>> >>>>> doesn't solve their problems.
>> >>>>>       And they will be surprised why this option can't be set, but
>> the
>> >>> other
>> >>>>> could. *semantic* is hard to be understood by users.
>> >>>>>
>> >>>>> # doesn't solve the problems
>> >>>>>
>> >>>>> I think the purpose of this FLIP is to allow users to quickly
>> override
>> >>> some
>> >>>>> connectors' properties to tune their jobs.
>> >>>>> However, `supportedHintOptions` is off track. It only allows a
>> subset
>> >>>>> options and for the users it's not *clear* which subset is allowed.
>> >>>>>
>> >>>>> Besides, I'm not sure `supportedHintOptions` can work well for all
>> >>> cases.
>> >>>>> How could you support kafka properties (`connector.properties.*`) as
>> >>> hint
>> >>>>> options? Some kafka properties may affect semantics
>> >> (bootstrap.servers),
>> >>>>> some may not (max.poll.records). Besides, I think it's not possible
>> to
>> >>> list
>> >>>>> all the possible kafka properties [1].
>> >>>>>
>> >>>>> In summary, IMO, `supportedHintOptions`
>> >>>>> (1) it increase the complexity to develop a connector
>> >>>>> (2) it confuses users which options can be used in hint, which are
>> >> not,
>> >>>>> they have to check the docs again and again.
>> >>>>> (3) it doesn't solve the problems which we want to solve by this
>> FLIP.
>> >>>>>
>> >>>>> I think we should avoid introducing some partial solutions.
>> Otherwise,
>> >>> we
>> >>>>> will be stuck in a loop that introduce new API -> deprecate API ->
>> >>>>> introduce new API....
>> >>>>>
>> >>>>> I personally in favor of an explicit WITH syntax after the table as
>> a
>> >>> part
>> >>>>> of the query which is mentioned by Kurt before, e.g. SELECT * from T
>> >>>>> WITH('key' = 'value') .
>> >>>>> It allows users to dynamically set options which can affect
>> semantics.
>> >>> It
>> >>>>> will be very flexible to solve users' problems so far.
>> >>>>>
>> >>>>> Best,
>> >>>>> Jark
>> >>>>>
>> >>>>> [1]: https://kafka.apache.org/documentation/#consumerconfigs
>> >>>>>
>> >>>>> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com>
>> >> wrote:
>> >>>>>
>> >>>>>> My POC is here for the hints options merge [1].
>> >>>>>>
>> >>>>>> Personally, I have no strong objections for splitting hints with
>> the
>> >>>>>> CatalogTable, the only cons is a more complex implementation but
>> the
>> >>>>>> concept is more clear, and I have updated the WIKI.
>> >>>>>>
>> >>>>>> I think it would be nice if we can support the format “ignore-parse
>> >>>>> error”
>> >>>>>> option key, the CSV source already has a key [2] and we can use
>> that
>> >> in
>> >>>>> the
>> >>>>>> supportedHIntOptions, for the common CSV and JSON formats, we cal
>> >> also
>> >>>>> give
>> >>>>>> a support. This is the only kind of key in formats that “do not
>> >> change
>> >>>>> the
>> >>>>>> semantics” (somehow), what do you think about this ~
>> >>>>>>
>> >>>>>> [1]
>> >>>>>>
>> >>>>>
>> >>>
>> >>
>> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
>> >>>>>> [2]
>> >>>>>>
>> >>>>>
>> >>>
>> >>
>> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
>> >>>>>>
>> >>>>>> Best,
>> >>>>>> Danny Chan
>> >>>>>> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
>> >>>>>>> Hi everyone,
>> >>>>>>>
>> >>>>>>> +1 to Kurt's suggestion. Let's just have it in source and sink
>> >>>>> factories
>> >>>>>>> for now. We can still move this method up in the future.
>> Currently,
>> >> I
>> >>>>>>> don't see a need for catalogs or formats. Because how would you
>> >> target
>> >>>>> a
>> >>>>>>> format in the query?
>> >>>>>>>
>> >>>>>>> @Danny: Can you send a link to your PoC? I'm very skeptical about
>> >>>>>>> creating a new CatalogTable in planner. Actually CatalogTable
>> should
>> >>> be
>> >>>>>>> immutable between Catalog and Factory. Because a catalog can
>> return
>> >>> its
>> >>>>>>> own factory and fully control the instantiation. Depending on the
>> >>>>>>> implementation, that means it can be possible that the catalog has
>> >>>>>>> encoded more information in a concrete subclass implementing the
>> >>>>>>> interface. I vote for separating the concerns of catalog
>> information
>> >>>>> and
>> >>>>>>> hints in the factory explicitly.
>> >>>>>>>
>> >>>>>>> Regards,
>> >>>>>>> Timo
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> On 18.03.20 05:41, Jingsong Li wrote:
>> >>>>>>>> Hi,
>> >>>>>>>>
>> >>>>>>>> I am thinking we can provide hints to *table* related instances.
>> >>>>>>>> - TableFormatFactory: of cause we need hints support, there are
>> >> many
>> >>>>>> format
>> >>>>>>>> options in DDL too.
>> >>>>>>>> - catalog and module: I don't know, maybe in future we can
>> provide
>> >>>>> some
>> >>>>>>>> hints for them.
>> >>>>>>>>
>> >>>>>>>> Best,
>> >>>>>>>> Jingsong Lee
>> >>>>>>>>
>> >>>>>>>> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <
>> yuzhao.cyz@gmail.com>
>> >>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>> Yes, I think we should move the `supportedHintOptions` from
>> >>>>>> TableFactory
>> >>>>>>>>> to TableSourceFactory, and we also need to add the interface to
>> >>>>>>>>> TableSinkFactory though because sink target table may also have
>> >>>>> hints
>> >>>>>>>>> attached.
>> >>>>>>>>>
>> >>>>>>>>> Best,
>> >>>>>>>>> Danny Chan
>> >>>>>>>>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
>> >>>>>>>>>> Have one question for adding `supportedHintOptions` method to
>> >>>>>>>>>> `TableFactory`. It seems
>> >>>>>>>>>> `TableFactory` is a base factory interface for all *table
>> module*
>> >>>>>> related
>> >>>>>>>>>> instances, such as
>> >>>>>>>>>> catalog, module, format and so on. It's not created only for
>> >>>>>> *table*. Is
>> >>>>>>>>> it
>> >>>>>>>>>> possible to move it
>> >>>>>>>>>> to `TableSourceFactory`?
>> >>>>>>>>>>
>> >>>>>>>>>> Best,
>> >>>>>>>>>> Kurt
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
>> >>>>> yuzhao.cyz@gmail.com>
>> >>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>> Thanks Timo ~
>> >>>>>>>>>>>
>> >>>>>>>>>>> For the naming itself, I also think the PROPERTIES is not that
>> >>>>>>>>> concise, so
>> >>>>>>>>>>> +1 for OPTIONS (I had thought about that, but there are many
>> >>>>>> codes in
>> >>>>>>>>>>> current Flink called it properties, i.e. the
>> >>>>>> DescriptorProperties,
>> >>>>>>>>>>> #getSupportedProperties), let’s use OPTIONS if this is our new
>> >>>>>>>>> preference.
>> >>>>>>>>>>>
>> >>>>>>>>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
>> >>>>>>>>> ConfigOption
>> >>>>>>>>>>> can take more info. AFAIK, Spark also call their table options
>> >>>>>> instead
>> >>>>>>>>> of
>> >>>>>>>>>>> properties. [1]
>> >>>>>>>>>>>
>> >>>>>>>>>>> In my local POC, I did create a new CatalogTable, and it works
>> >>>>>> for
>> >>>>>>>>> current
>> >>>>>>>>>>> connectors well, all the DDL tables would finally yield a
>> >>>>>> CatalogTable
>> >>>>>>>>>>> instance and we can apply the options to that(in the
>> >>>>>> CatalogSourceTable
>> >>>>>>>>>>> when we generating the TableSource), the pros is that we do
>> not
>> >>>>>> need to
>> >>>>>>>>>>> modify the codes of connectors itself. If we split the options
>> >>>>>> from
>> >>>>>>>>>>> CatalogTable, we may need to add some additional logic in each
>> >>>>>>>>> connector
>> >>>>>>>>>>> factories in order to merge these properties (and the logic
>> are
>> >>>>>> almost
>> >>>>>>>>> the
>> >>>>>>>>>>> same), what do you think about this?
>> >>>>>>>>>>>
>> >>>>>>>>>>> [1]
>> >>>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>
>> >>>>>
>> >>>
>> >>
>> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
>> >>>>>>>>>>>
>> >>>>>>>>>>> Best,
>> >>>>>>>>>>> Danny Chan
>> >>>>>>>>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
>> >>>>>> ,写道:
>> >>>>>>>>>>>> Hi Danny,
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> thanks for updating the FLIP. I think your current design is
>> >>>>>>>>> sufficient
>> >>>>>>>>>>>> to separate hints from result-related properties.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> One remark to the naming itself: I would vote for calling the
>> >>>>>> hints
>> >>>>>>>>>>>> around table scan `OPTIONS('k'='v')`. We used the term
>> >>>>>> "properties"
>> >>>>>>>>> in
>> >>>>>>>>>>>> the past but since we want to unify the Flink configuration
>> >>>>>>>>> experience,
>> >>>>>>>>>>>> we should use consistent naming and classes around
>> >>>>>> `ConfigOptions`.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> It would be nice to use `Set<ConfigOption>
>> >>>>>> supportedHintOptions();`
>> >>>>>>>>> to
>> >>>>>>>>>>>> start using config options instead of pure string properties.
>> >>>>>> This
>> >>>>>>>>> will
>> >>>>>>>>>>>> also allow us to generate documentation in the future around
>> >>>>>>>>> supported
>> >>>>>>>>>>>> data types, ranges, etc. for options. At some point we would
>> >>>>>> also
>> >>>>>>>>> like
>> >>>>>>>>>>>> to drop `DescriptorProperties` class. "Options" is also used
>> >>>>>> in the
>> >>>>>>>>>>>> documentation [1] and in the SQL/MED standard [2].
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Furthermore, I would still vote for separating CatalogTable
>> >>>>>> and hint
>> >>>>>>>>>>>> options. Otherwise the planner would need to create a new
>> >>>>>>>>> CatalogTable
>> >>>>>>>>>>>> instance which might not always be easy. We should offer them
>> >>>>>> via:
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>
>> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
>> >>>>>>>>>>>> ReadableConfig
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> What do you think?
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Regards,
>> >>>>>>>>>>>> Timo
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> [1]
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>
>> >>>>>
>> >>>
>> >>
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
>> >>>>>>>>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
>> >>>>>>>>>>>>> @Danny sounds good.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> Maybe it is worth listing all the classes of problems that
>> >>>>>> you
>> >>>>>>>>> want to
>> >>>>>>>>>>>>> address and then look at each class and see if hints are a
>> >>>>>> good
>> >>>>>>>>> default
>> >>>>>>>>>>>>> solution or a good optional way of simplifying things?
>> >>>>>>>>>>>>> The discussion has grown a lot and it is starting to be
>> >>>>> hard
>> >>>>>> to
>> >>>>>>>>>>> distinguish
>> >>>>>>>>>>>>> the parts where everyone agrees from the parts were there
>> >>>>> are
>> >>>>>>>>> concerns.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
>> >>>>>> danny0405@apache.org>
>> >>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>> Thanks Stephan ~
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> We can remove the support for properties that may change
>> >>>>>> the
>> >>>>>>>>>>> semantics of
>> >>>>>>>>>>>>>> query if you think that is a trouble.
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> How about we support the /*+ properties() */ hint only
>> >>>>> for
>> >>>>>> those
>> >>>>>>>>>>> optimize
>> >>>>>>>>>>>>>> parameters, such as the fetch size of source or something
>> >>>>>> like
>> >>>>>>>>> that,
>> >>>>>>>>>>> does
>> >>>>>>>>>>>>>> that make sense?
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> I think Bowen has actually put it very well.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> (1) Hints that change semantics looks like trouble
>> >>>>>> waiting to
>> >>>>>>>>>>> happen. For
>> >>>>>>>>>>>>>>> example Kafka offset handling should be in filters. The
>> >>>>>> Kafka
>> >>>>>>>>>>> source
>> >>>>>>>>>>>>>> should
>> >>>>>>>>>>>>>>> support predicate pushdown.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> (2) Hints should not be a workaround for current
>> >>>>>> shortcomings.
>> >>>>>>>>> A
>> >>>>>>>>>>> lot of
>> >>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>> suggested above sounds exactly like that. Working
>> >>>>> around
>> >>>>>>>>>>> catalog/DDL
>> >>>>>>>>>>>>>>> shortcomings, missing exposure of metadata (offsets),
>> >>>>>> missing
>> >>>>>>>>>>> predicate
>> >>>>>>>>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as
>> >>>>> a
>> >>>>>> quick
>> >>>>>>>>> fix
>> >>>>>>>>>>> for
>> >>>>>>>>>>>>>>> these issues, rather than fixing the root causes, will
>> >>>>>> much
>> >>>>>>>>> likely
>> >>>>>>>>>>> bite
>> >>>>>>>>>>>>>> us
>> >>>>>>>>>>>>>>> back badly in the future.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>> Stephan
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
>> >>>>>> ykt836@gmail.com>
>> >>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> It seems this FLIP's name is somewhat misleading.
>> >>>>> From
>> >>>>>> my
>> >>>>>>>>>>>>>> understanding,
>> >>>>>>>>>>>>>>>> this FLIP is trying to
>> >>>>>>>>>>>>>>>> address the dynamic parameter issue, and table hints
>> >>>>>> is the
>> >>>>>>>>> way
>> >>>>>>>>>>> we wan
>> >>>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>> choose. I think we should
>> >>>>>>>>>>>>>>>> be focus on "what's the right way to solve dynamic
>> >>>>>> property"
>> >>>>>>>>>>> instead of
>> >>>>>>>>>>>>>>>> discussing "whether table
>> >>>>>>>>>>>>>>>> hints can affect query semantics".
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> For now, there are two proposed ways to achieve
>> >>>>> dynamic
>> >>>>>>>>> property:
>> >>>>>>>>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with
>> >>>>>> (xxx)
>> >>>>>>>>>>>>>>>> 2. use custom "from t with (xxx)" syntax
>> >>>>>>>>>>>>>>>> 3. "Borrow" the table hints to have a special
>> >>>>>> PROPERTIES
>> >>>>>>>>> hint.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> The first one didn't break anything, but the only
>> >>>>>> problem i
>> >>>>>>>>> see
>> >>>>>>>>>>> is a
>> >>>>>>>>>>>>>>> little
>> >>>>>>>>>>>>>>>> more verbose than the table hint
>> >>>>>>>>>>>>>>>> approach. I can imagine when someone using SQL CLI to
>> >>>>>> have a
>> >>>>>>>>> sql
>> >>>>>>>>>>>>>>>> experience, it's quite often that
>> >>>>>>>>>>>>>>>> he will modify the table property, some use cases i
>> >>>>> can
>> >>>>>>>>> think of:
>> >>>>>>>>>>>>>>>> 1. the source contains some corrupted data, i want to
>> >>>>>> turn
>> >>>>>>>>> on the
>> >>>>>>>>>>>>>>>> "ignore-error" flag for certain formats.
>> >>>>>>>>>>>>>>>> 2. I have a kafka table and want to see some sample
>> >>>>>> data
>> >>>>>>>>> from the
>> >>>>>>>>>>>>>>>> beginning, so i change the offset
>> >>>>>>>>>>>>>>>> to "earliest", and then I want to observe the latest
>> >>>>>> data
>> >>>>>>>>> which
>> >>>>>>>>>>> keeps
>> >>>>>>>>>>>>>>>> coming in. I would write another query
>> >>>>>>>>>>>>>>>> to select from the latest table.
>> >>>>>>>>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly
>> >>>>> then
>> >>>>>> i can
>> >>>>>>>>>>> observe
>> >>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>> data from database side.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> Most of such use cases are quite ad-hoc. If every
>> >>>>> time
>> >>>>>> I
>> >>>>>>>>> want to
>> >>>>>>>>>>> have a
>> >>>>>>>>>>>>>>>> different experience, i need to create
>> >>>>>>>>>>>>>>>> a temporary table and then also modify my query, it
>> >>>>>> doesn't
>> >>>>>>>>> feel
>> >>>>>>>>>>>>>> smooth.
>> >>>>>>>>>>>>>>>> Embed such dynamic property into
>> >>>>>>>>>>>>>>>> query would have better user experience.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is
>> >>>>>> breaking
>> >>>>>>>>> SQL
>> >>>>>>>>>>>>>>> compliant,
>> >>>>>>>>>>>>>>>> and for #3, it only breaks some
>> >>>>>>>>>>>>>>>> unwritten rules, but we can have an explanation on
>> >>>>>> that. And
>> >>>>>>>>> I
>> >>>>>>>>>>> really
>> >>>>>>>>>>>>>>> doubt
>> >>>>>>>>>>>>>>>> whether user would complain about
>> >>>>>>>>>>>>>>>> this when they actually have flexible and good
>> >>>>>> experience
>> >>>>>>>>> using
>> >>>>>>>>>>> this.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>>> Kurt
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
>> >>>>>>>>> yuzhao.cyz@gmail.com
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Thanks Aljoscha ~
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> I agree for most of the query hints, they are
>> >>>>>> optional as
>> >>>>>>>>> an
>> >>>>>>>>>>>>>> optimizer
>> >>>>>>>>>>>>>>>>> instruction, especially for the traditional RDBMS.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> But, just like BenChao said, Flink as a computation
>> >>>>>> engine
>> >>>>>>>>> has
>> >>>>>>>>>>> many
>> >>>>>>>>>>>>>>>>> different kind of data sources, thus, dynamic
>> >>>>>> parameters
>> >>>>>>>>> like
>> >>>>>>>>>>>>>>>> start_offest
>> >>>>>>>>>>>>>>>>> can only bind to each table scope, we can not set a
>> >>>>>> session
>> >>>>>>>>>>> config
>> >>>>>>>>>>>>>> like
>> >>>>>>>>>>>>>>>>> KSQL because they are all about Kafka:
>> >>>>>>>>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Thus the most flexible way to set up these dynamic
>> >>>>>> params
>> >>>>>>>>> is
>> >>>>>>>>>>> to bind
>> >>>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>>> the table scope in the query when we want to
>> >>>>> override
>> >>>>>>>>>>> something, so
>> >>>>>>>>>>>>>> we
>> >>>>>>>>>>>>>>>> have
>> >>>>>>>>>>>>>>>>> these solutions above (with pros and cons from my
>> >>>>>> side):
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Pros:
>> >>>>>>>>>>>>>>>>> - Easy to add
>> >>>>>>>>>>>>>>>>> - Parameters are part of the main query
>> >>>>>>>>>>>>>>>>> Cons:
>> >>>>>>>>>>>>>>>>> - Not SQL compliant
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */
>> >>>>>> (from
>> >>>>>>>>> me)
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Pros:
>> >>>>>>>>>>>>>>>>> - Easy to add
>> >>>>>>>>>>>>>>>>> - SQL compliant because it is nested in the
>> >>>>> comments
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Cons:
>> >>>>>>>>>>>>>>>>> - Parameters are not part of the main query
>> >>>>>>>>>>>>>>>>> - Cryptic syntax for new users
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> The biggest problem for hints way may be the “if
>> >>>>>> hints
>> >>>>>>>>> must be
>> >>>>>>>>>>>>>>> optional”,
>> >>>>>>>>>>>>>>>>> actually we have though about 1 for a while but
>> >>>>>> aborted
>> >>>>>>>>>>> because it
>> >>>>>>>>>>>>>>> breaks
>> >>>>>>>>>>>>>>>>> the SQL standard too much. And we replace it with
>> >>>>> 2,
>> >>>>>>>>> because
>> >>>>>>>>>>> the
>> >>>>>>>>>>>>>> hints
>> >>>>>>>>>>>>>>>>> syntax do not break SQL standard(nested in
>> >>>>> comments).
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint
>> >>>>>> that
>> >>>>>>>>> allows
>> >>>>>>>>>>>>>> override
>> >>>>>>>>>>>>>>>>> some properties of table dynamically, it does not
>> >>>>>> break
>> >>>>>>>>>>> anything, at
>> >>>>>>>>>>>>>>>> lease
>> >>>>>>>>>>>>>>>>> for current Flink use cases.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Planner hints are optional just because they are
>> >>>>>> naturally
>> >>>>>>>>>>> enforcers
>> >>>>>>>>>>>>>> of
>> >>>>>>>>>>>>>>>>> the planner, most of them aim to instruct the
>> >>>>>> optimizer,
>> >>>>>>>>> but,
>> >>>>>>>>>>> the
>> >>>>>>>>>>>>>> table
>> >>>>>>>>>>>>>>>>> hints is a little different, table hints can
>> >>>>> specify
>> >>>>>> the
>> >>>>>>>>> table
>> >>>>>>>>>>> meta
>> >>>>>>>>>>>>>>> like
>> >>>>>>>>>>>>>>>>> index column, and it is very convenient to specify
>> >>>>>> table
>> >>>>>>>>>>> properties.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */
>> >>>>>> table
>> >>>>>>>>> hint,
>> >>>>>>>>>>> we
>> >>>>>>>>>>>>>> can
>> >>>>>>>>>>>>>>>>> call it table dynamic parameters.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>>>> Danny Chan
>> >>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
>> >>>>>>>>>>> aljoscha@apache.org>,写道:
>> >>>>>>>>>>>>>>>>>> Hi,
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> I don't understand this discussion. Hints, as I
>> >>>>>>>>> understand
>> >>>>>>>>>>> them,
>> >>>>>>>>>>>>>>> should
>> >>>>>>>>>>>>>>>>>> work like this:
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> - hints are *optional* advice for the optimizer
>> >>>>> to
>> >>>>>> try
>> >>>>>>>>> and
>> >>>>>>>>>>> help it
>> >>>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>>>> find a good execution strategy
>> >>>>>>>>>>>>>>>>>> - hints should not change query semantics, i.e.
>> >>>>>> they
>> >>>>>>>>> should
>> >>>>>>>>>>> not
>> >>>>>>>>>>>>>>> change
>> >>>>>>>>>>>>>>>>>> connector properties executing a query with
>> >>>>> taking
>> >>>>>> into
>> >>>>>>>>>>> account the
>> >>>>>>>>>>>>>>>>>> hints *must* produce the same result as executing
>> >>>>>> the
>> >>>>>>>>> query
>> >>>>>>>>>>> without
>> >>>>>>>>>>>>>>>>>> taking into account the hints
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>   From these simple requirements you can derive a
>> >>>>>> solution
>> >>>>>>>>>>> that makes
>> >>>>>>>>>>>>>>>>>> sense. I don't have a strong preference for the
>> >>>>>> syntax
>> >>>>>>>>> but we
>> >>>>>>>>>>>>>> should
>> >>>>>>>>>>>>>>>>>> strive to be in line with prior work.
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>>>>> Aljoscha
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
>> >>>>>>>>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> I agree with Kurt that option2 is too
>> >>>>>> complicated to
>> >>>>>>>>> use
>> >>>>>>>>>>> because:
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> • As a Kafka topic consumer, the user must
>> >>>>>> define both
>> >>>>>>>>> the
>> >>>>>>>>>>>>>> virtual
>> >>>>>>>>>>>>>>>>> column for start offset and he must apply a special
>> >>>>>> filter
>> >>>>>>>>>>> predicate
>> >>>>>>>>>>>>>>>> after
>> >>>>>>>>>>>>>>>>> each query
>> >>>>>>>>>>>>>>>>>>> • And for the internal implementation, the
>> >>>>>> metadata
>> >>>>>>>>> column
>> >>>>>>>>>>> push
>> >>>>>>>>>>>>>>> down
>> >>>>>>>>>>>>>>>>> is another hard topic, each kind of message queue
>> >>>>>> may have
>> >>>>>>>>> its
>> >>>>>>>>>>> offset
>> >>>>>>>>>>>>>>>>> attribute, we need to consider the expression type
>> >>>>>> for
>> >>>>>>>>>>> different
>> >>>>>>>>>>>>>> kind;
>> >>>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>> source also need to recognize the constant column
>> >>>>> as
>> >>>>>> a
>> >>>>>>>>> config
>> >>>>>>>>>>>>>>>> option(which
>> >>>>>>>>>>>>>>>>> is weird because usually what we pushed down is a
>> >>>>>> table
>> >>>>>>>>> column)
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> For option 1 and option3, I think there is no
>> >>>>>>>>> difference,
>> >>>>>>>>>>> option1
>> >>>>>>>>>>>>>>> is
>> >>>>>>>>>>>>>>>>> also a hint syntax which is introduced in Sybase
>> >>>>> and
>> >>>>>>>>>>> referenced then
>> >>>>>>>>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
>> >>>>>>>>>>> ambitiousness.
>> >>>>>>>>>>>>>>>> Personally
>> >>>>>>>>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword
>> >>>>>> for
>> >>>>>>>>> these
>> >>>>>>>>>>> reasons:
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> • We do not break the standard SQL, the hints
>> >>>>> are
>> >>>>>>>>> nested
>> >>>>>>>>>>> in SQL
>> >>>>>>>>>>>>>>>>> comments
>> >>>>>>>>>>>>>>>>>>> • We do not need to introduce additional WITH
>> >>>>>> keyword
>> >>>>>>>>>>> which may
>> >>>>>>>>>>>>>>>> appear
>> >>>>>>>>>>>>>>>>> in a query if we use that because a table can be
>> >>>>>>>>> referenced in
>> >>>>>>>>>>> all
>> >>>>>>>>>>>>>>> kinds
>> >>>>>>>>>>>>>>>> of
>> >>>>>>>>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
>> >>>>>> make
>> >>>>>>>>> our
>> >>>>>>>>>>> sql
>> >>>>>>>>>>>>>> query
>> >>>>>>>>>>>>>>>>> break too much of the SQL from standard
>> >>>>>>>>>>>>>>>>>>> • We would have uniform syntax for hints as
>> >>>>> query
>> >>>>>>>>> hint, one
>> >>>>>>>>>>>>>> syntax
>> >>>>>>>>>>>>>>>>> fits all and more easy to use
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> And here is the reason why we choose a uniform
>> >>>>>> Oracle
>> >>>>>>>>>>> style query
>> >>>>>>>>>>>>>>>>> hint syntax which is addressed by Julian Hyde when
>> >>>>> we
>> >>>>>>>>> design
>> >>>>>>>>>>> the
>> >>>>>>>>>>>>>> syntax
>> >>>>>>>>>>>>>>>>> from the Calcite community:
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for
>> >>>>>> table
>> >>>>>>>>> hints.
>> >>>>>>>>>>> It
>> >>>>>>>>>>>>>> adds a
>> >>>>>>>>>>>>>>>>> new use of the WITH keyword that is unrelated to
>> >>>>> the
>> >>>>>> use of
>> >>>>>>>>>>> WITH for
>> >>>>>>>>>>>>>>>>> common-table expressions.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> A historical note. Microsoft SQL Server
>> >>>>>> inherited its
>> >>>>>>>>> hint
>> >>>>>>>>>>> syntax
>> >>>>>>>>>>>>>>>> from
>> >>>>>>>>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
>> >>>>>>>>>>> Programming”[1], page
>> >>>>>>>>>>>>>>>> 632,
>> >>>>>>>>>>>>>>>>> “Optimizer hints”. The book was written in 1999,
>> >>>>> and
>> >>>>>> covers
>> >>>>>>>>>>> Microsoft
>> >>>>>>>>>>>>>>> SQL
>> >>>>>>>>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
>> >>>>>> but the
>> >>>>>>>>>>> syntax very
>> >>>>>>>>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft
>> >>>>> SQL
>> >>>>>>>>> Server was
>> >>>>>>>>>>>>>> forked
>> >>>>>>>>>>>>>>> in
>> >>>>>>>>>>>>>>>>> 1993.)
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Microsoft later added the WITH keyword to make
>> >>>>>> it less
>> >>>>>>>>>>> ambiguous,
>> >>>>>>>>>>>>>>> and
>> >>>>>>>>>>>>>>>>> has now deprecated the syntax that does not use
>> >>>>> WITH.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> They are forced to keep the syntax for
>> >>>>> backwards
>> >>>>>>>>>>> compatibility
>> >>>>>>>>>>>>>> but
>> >>>>>>>>>>>>>>>>> that doesn’t mean that we should shoulder their
>> >>>>>> burden.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> I think formatted comments are the right
>> >>>>>> container for
>> >>>>>>>>>>> hints
>> >>>>>>>>>>>>>>> because
>> >>>>>>>>>>>>>>>>> it allows us to change the hint syntax without
>> >>>>>> changing
>> >>>>>>>>> the SQL
>> >>>>>>>>>>>>>> parser,
>> >>>>>>>>>>>>>>>> and
>> >>>>>>>>>>>>>>>>> makes clear that we are at liberty to ignore hints
>> >>>>>>>>> entirely.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Julian
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
>> >>>>>>>>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>>>>>> Danny Chan
>> >>>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
>> >>>>>>>>> twalthr@apache.org
>> >>>>>>>>>>>> ,写道:
>> >>>>>>>>>>>>>>>>>>>> Hi Danny,
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> it is true that our DDL is not standard
>> >>>>>> compliant by
>> >>>>>>>>>>> using the
>> >>>>>>>>>>>>>>> WITH
>> >>>>>>>>>>>>>>>>>>>> clause. Nevertheless, we aim for not
>> >>>>> diverging
>> >>>>>> too
>> >>>>>>>>> much
>> >>>>>>>>>>> and the
>> >>>>>>>>>>>>>>>> LIKE
>> >>>>>>>>>>>>>>>>>>>> clause is an example of that. It will solve
>> >>>>>> things
>> >>>>>>>>> like
>> >>>>>>>>>>>>>>> overwriting
>> >>>>>>>>>>>>>>>>>>>> WATERMARKs, add additional/modifying
>> >>>>>> properties and
>> >>>>>>>>>>> inherit
>> >>>>>>>>>>>>>>> schema.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3
>> >>>>>> types
>> >>>>>>>>>>> definition
>> >>>>>>>>>>>>>>>>> together.
>> >>>>>>>>>>>>>>>>>>>> We are not the first ones that try to solve
>> >>>>>> this.
>> >>>>>>>>> There
>> >>>>>>>>>>> is also
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>> SQL
>> >>>>>>>>>>>>>>>>>>>> MED standard [1] that tried to tackle this
>> >>>>>> problem. I
>> >>>>>>>>>>> think it
>> >>>>>>>>>>>>>>> was
>> >>>>>>>>>>>>>>>>> not
>> >>>>>>>>>>>>>>>>>>>> considered when designing the current DDL.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
>> >>>>>>>>> offsets. I
>> >>>>>>>>>>> will
>> >>>>>>>>>>>>>>> give
>> >>>>>>>>>>>>>>>>> some
>> >>>>>>>>>>>>>>>>>>>> examples and look forward to feedback here:
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part
>> >>>>>> of the
>> >>>>>>>>>>> query
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Pros:
>> >>>>>>>>>>>>>>>>>>>> - Easy to add
>> >>>>>>>>>>>>>>>>>>>> - Parameters are part of the main query
>> >>>>>>>>>>>>>>>>>>>> - No complicated hinting syntax
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Cons:
>> >>>>>>>>>>>>>>>>>>>> - Not SQL compliant
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> *Option 2* Use metadata in query
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
>> >>>>>>>>>>>>>>>> SYSTEM_METADATA('offset'))`
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset >
>> >>>>> TIMESTAMP
>> >>>>>>>>>>> '2012-12-12
>> >>>>>>>>>>>>>>>>> 12:34:22'`
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Pros:
>> >>>>>>>>>>>>>>>>>>>> - SQL compliant in the query
>> >>>>>>>>>>>>>>>>>>>> - Access of metadata in the DDL which is
>> >>>>>> required
>> >>>>>>>>> anyway
>> >>>>>>>>>>>>>>>>>>>> - Regular pushdown rules apply
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Cons:
>> >>>>>>>>>>>>>>>>>>>> - Users need to add an additional comlumn in
>> >>>>>> the DDL
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> *Option 3*: Use hints for properties
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> `
>> >>>>>>>>>>>>>>>>>>>> SELECT *
>> >>>>>>>>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
>> >>>>>>>>>>>>>>>>>>>> `
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Pros:
>> >>>>>>>>>>>>>>>>>>>> - Easy to add
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Cons:
>> >>>>>>>>>>>>>>>>>>>> - Parameters are not part of the main query
>> >>>>>>>>>>>>>>>>>>>> - Cryptic syntax for new users
>> >>>>>>>>>>>>>>>>>>>> - Not standard compliant.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> If we go with this option, I would suggest to
>> >>>>>> make it
>> >>>>>>>>>>> available
>> >>>>>>>>>>>>>>> in
>> >>>>>>>>>>>>>>>> a
>> >>>>>>>>>>>>>>>>>>>> separate map and don't mix it with statically
>> >>>>>> defined
>> >>>>>>>>>>>>>> properties.
>> >>>>>>>>>>>>>>>>> Such
>> >>>>>>>>>>>>>>>>>>>> that the factory can decide which properties
>> >>>>>> have the
>> >>>>>>>>>>> right to
>> >>>>>>>>>>>>>> be
>> >>>>>>>>>>>>>>>>>>>> overwritten by the hints:
>> >>>>>>>>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
>> >>>>>>>>>>> ReadableConfig
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Regards,
>> >>>>>>>>>>>>>>>>>>>> Timo
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Currently I see 3 options as a
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
>> >>>>>>>>>>>>>>>>>>>>> Thanks Bowen ~
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> I agree we should somehow categorize our
>> >>>>>> connector
>> >>>>>>>>>>>>>> parameters.
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> For type1, I’m already preparing a solution
>> >>>>>> like
>> >>>>>>>>> the
>> >>>>>>>>>>>>>> Confluent
>> >>>>>>>>>>>>>>>>> schema registry + Avro schema inference thing, so
>> >>>>>> this may
>> >>>>>>>>> not
>> >>>>>>>>>>> be a
>> >>>>>>>>>>>>>>>> problem
>> >>>>>>>>>>>>>>>>> in the near future.
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> For type3, I have some questions:
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset >
>> >>>>> 12pm
>> >>>>>>>>> yesterday”
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> Where does the offset column come from, a
>> >>>>>> virtual
>> >>>>>>>>>>> column from
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>> table schema, you said that
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> They change
>> >>>>>>>>>>>>>>>>>>>>> almost every time a query starts and have
>> >>>>>> nothing
>> >>>>>>>>> to
>> >>>>>>>>>>> do with
>> >>>>>>>>>>>>>>>>> metadata, thus
>> >>>>>>>>>>>>>>>>>>>>> should not be part of table definition/DDL
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> But why you can reference it in the query,
>> >>>>>> I’m
>> >>>>>>>>>>> confused for
>> >>>>>>>>>>>>>>> that,
>> >>>>>>>>>>>>>>>>> can you elaborate a little ?
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>>>>>>>> Danny Chan
>> >>>>>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
>> >>>>>>>>>>> bowenli86@gmail.com
>> >>>>>>>>>>>>>>> ,写道:
>> >>>>>>>>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> The root cause of too much manual work is
>> >>>>>> Flink
>> >>>>>>>>> DDL
>> >>>>>>>>>>> has
>> >>>>>>>>>>>>>>> mixed 3
>> >>>>>>>>>>>>>>>>> types of
>> >>>>>>>>>>>>>>>>>>>>>> params together and doesn't handle each
>> >>>>> of
>> >>>>>> them
>> >>>>>>>>> very
>> >>>>>>>>>>> well.
>> >>>>>>>>>>>>>>>> Below
>> >>>>>>>>>>>>>>>>> are how I
>> >>>>>>>>>>>>>>>>>>>>>> categorize them and corresponding
>> >>>>>> solutions in my
>> >>>>>>>>>>> mind:
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
>> >>>>>>>>> external
>> >>>>>>>>>>>>>>>> endpoint/url,
>> >>>>>>>>>>>>>>>>>>>>>> username/pwd, schemas, formats.
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> Such metadata are mostly already
>> >>>>>> accessible in
>> >>>>>>>>>>> external
>> >>>>>>>>>>>>>>> system
>> >>>>>>>>>>>>>>>>> as long as
>> >>>>>>>>>>>>>>>>>>>>>> endpoints and credentials are provided.
>> >>>>>> Flink can
>> >>>>>>>>>>> get it
>> >>>>>>>>>>>>>> thru
>> >>>>>>>>>>>>>>>>> catalogs, but
>> >>>>>>>>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus
>> >>>>>> Flink
>> >>>>>>>>> just
>> >>>>>>>>>>> hasn't
>> >>>>>>>>>>>>>>>> been
>> >>>>>>>>>>>>>>>>> able to
>> >>>>>>>>>>>>>>>>>>>>>> leverage that. So the solution should be
>> >>>>>> building
>> >>>>>>>>>>> more
>> >>>>>>>>>>>>>>>> catalogs.
>> >>>>>>>>>>>>>>>>> Such
>> >>>>>>>>>>>>>>>>>>>>>> params should be part of a Flink table
>> >>>>>>>>>>> DDL/definition, and
>> >>>>>>>>>>>>>>> not
>> >>>>>>>>>>>>>>>>> overridable
>> >>>>>>>>>>>>>>>>>>>>>> in any means.
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc
>> >>>>>> connector's
>> >>>>>>>>>>> fetch size,
>> >>>>>>>>>>>>>>>>> elasticsearch
>> >>>>>>>>>>>>>>>>>>>>>> connector's bulk flush size.
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> Such params don't affect query results,
>> >>>>> but
>> >>>>>>>>> affect
>> >>>>>>>>>>> how
>> >>>>>>>>>>>>>>> results
>> >>>>>>>>>>>>>>>>> are produced
>> >>>>>>>>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) -
>> >>>>> they
>> >>>>>> are
>> >>>>>>>>>>> essentially
>> >>>>>>>>>>>>>>>>> execution and
>> >>>>>>>>>>>>>>>>>>>>>> implementation details. They change often
>> >>>>>> in
>> >>>>>>>>>>> exploration or
>> >>>>>>>>>>>>>>>>> development
>> >>>>>>>>>>>>>>>>>>>>>> stages, but not quite frequently in
>> >>>>>> well-defined
>> >>>>>>>>>>>>>> long-running
>> >>>>>>>>>>>>>>>>> pipelines.
>> >>>>>>>>>>>>>>>>>>>>>> They should always have default values
>> >>>>> and
>> >>>>>> can be
>> >>>>>>>>>>> missing
>> >>>>>>>>>>>>>> in
>> >>>>>>>>>>>>>>>>> query. They
>> >>>>>>>>>>>>>>>>>>>>>> can be part of a table DDL/definition,
>> >>>>> but
>> >>>>>> should
>> >>>>>>>>>>> also be
>> >>>>>>>>>>>>>>>>> replaceable in a
>> >>>>>>>>>>>>>>>>>>>>>> query - *this is what table "hints" in
>> >>>>>> FLIP-113
>> >>>>>>>>>>> should
>> >>>>>>>>>>>>>>> cover*.
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka
>> >>>>>> connector's
>> >>>>>>>>>>> start
>> >>>>>>>>>>>>>>> offset.
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> Such params affect query results - the
>> >>>>>> semantics.
>> >>>>>>>>>>> They'd
>> >>>>>>>>>>>>>>> better
>> >>>>>>>>>>>>>>>>> be as
>> >>>>>>>>>>>>>>>>>>>>>> filter conditions in WHERE clause that
>> >>>>> can
>> >>>>>> be
>> >>>>>>>>> pushed
>> >>>>>>>>>>> down.
>> >>>>>>>>>>>>>>> They
>> >>>>>>>>>>>>>>>>> change
>> >>>>>>>>>>>>>>>>>>>>>> almost every time a query starts and have
>> >>>>>>>>> nothing to
>> >>>>>>>>>>> do
>> >>>>>>>>>>>>>> with
>> >>>>>>>>>>>>>>>>> metadata, thus
>> >>>>>>>>>>>>>>>>>>>>>> should not be part of table
>> >>>>>> definition/DDL, nor
>> >>>>>>>>> be
>> >>>>>>>>>>>>>> persisted
>> >>>>>>>>>>>>>>> in
>> >>>>>>>>>>>>>>>>> catalogs.
>> >>>>>>>>>>>>>>>>>>>>>> If they will, users should create views
>> >>>>> to
>> >>>>>> keep
>> >>>>>>>>> such
>> >>>>>>>>>>> params
>> >>>>>>>>>>>>>>>>> around (note
>> >>>>>>>>>>>>>>>>>>>>>> this is different from variable
>> >>>>>> substitution).
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we
>> >>>>>> get these
>> >>>>>>>>>>> params
>> >>>>>>>>>>>>>>> right,
>> >>>>>>>>>>>>>>>>> here're the
>> >>>>>>>>>>>>>>>>>>>>>> steps users need to do to develop and run
>> >>>>>> a Flink
>> >>>>>>>>>>> job:
>> >>>>>>>>>>>>>>>>>>>>>> - configure a Flink
>> >>>>>> ConfluentSchemaRegistry with
>> >>>>>>>>> url,
>> >>>>>>>>>>>>>>> username,
>> >>>>>>>>>>>>>>>>> and password
>> >>>>>>>>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset
>> >>>>>>> 12pm
>> >>>>>>>>>>> yesterday"
>> >>>>>>>>>>>>>>>>> (simplified
>> >>>>>>>>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink
>> >>>>> automatically
>> >>>>>>>>> retrieves
>> >>>>>>>>>>> all
>> >>>>>>>>>>>>>>>>> metadata of
>> >>>>>>>>>>>>>>>>>>>>>> schema, file format, etc and start the
>> >>>>> job
>> >>>>>>>>>>>>>>>>>>>>>> - users want to make the job read Kafka
>> >>>>>> topic
>> >>>>>>>>>>> faster, so it
>> >>>>>>>>>>>>>>>> goes
>> >>>>>>>>>>>>>>>>> as "SELECT
>> >>>>>>>>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/
>> >>>>>> WHERE
>> >>>>>>>>>>> offset >
>> >>>>>>>>>>>>>> 12pm
>> >>>>>>>>>>>>>>>>> yesterday"
>> >>>>>>>>>>>>>>>>>>>>>> - done and satisfied, users submit it to
>> >>>>>>>>> production
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with
>> >>>>> (k1=v1,
>> >>>>>>>>> k2=v2),
>> >>>>>>>>>>> I think
>> >>>>>>>>>>>>>>>> it's
>> >>>>>>>>>>>>>>>>> a
>> >>>>>>>>>>>>>>>>>>>>>> nice-to-have feature, but not a
>> >>>>>> strategically
>> >>>>>>>>>>> critical,
>> >>>>>>>>>>>>>>>>> long-term solution,
>> >>>>>>>>>>>>>>>>>>>>>> because
>> >>>>>>>>>>>>>>>>>>>>>> 1) It may seem promising at the current
>> >>>>>> stage to
>> >>>>>>>>>>> solve the
>> >>>>>>>>>>>>>>>>>>>>>> too-much-manual-work problem, but that's
>> >>>>>> only
>> >>>>>>>>>>> because Flink
>> >>>>>>>>>>>>>>>>> hasn't
>> >>>>>>>>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3
>> >>>>>> types
>> >>>>>>>>> of
>> >>>>>>>>>>> params
>> >>>>>>>>>>>>>>> above
>> >>>>>>>>>>>>>>>>> properly.
>> >>>>>>>>>>>>>>>>>>>>>> Once we get the params types right, the
>> >>>>>> LIKE
>> >>>>>>>>> syntax
>> >>>>>>>>>>> won't
>> >>>>>>>>>>>>>> be
>> >>>>>>>>>>>>>>>> that
>> >>>>>>>>>>>>>>>>>>>>>> important, and will be just an easier way
>> >>>>>> to
>> >>>>>>>>> create
>> >>>>>>>>>>> tables
>> >>>>>>>>>>>>>>>>> without retyping
>> >>>>>>>>>>>>>>>>>>>>>> long fields like username and pwd.
>> >>>>>>>>>>>>>>>>>>>>>> 2) Note that only some rare type of
>> >>>>>> catalog can
>> >>>>>>>>>>> store k-v
>> >>>>>>>>>>>>>>>>> property pair, so
>> >>>>>>>>>>>>>>>>>>>>>> table created this way often cannot be
>> >>>>>>>>> persisted. In
>> >>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>> foreseeable
>> >>>>>>>>>>>>>>>>>>>>>> future, such catalog will only be
>> >>>>>> HiveCatalog,
>> >>>>>>>>> and
>> >>>>>>>>>>> not
>> >>>>>>>>>>>>>>> everyone
>> >>>>>>>>>>>>>>>>> has a Hive
>> >>>>>>>>>>>>>>>>>>>>>> metastore. To be honest, without
>> >>>>>> persistence,
>> >>>>>>>>>>> recreating
>> >>>>>>>>>>>>>>> tables
>> >>>>>>>>>>>>>>>>> every time
>> >>>>>>>>>>>>>>>>>>>>>> this way is still a lot of keyboard
>> >>>>> typing.
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> Cheers,
>> >>>>>>>>>>>>>>>>>>>>>> Bowen
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt
>> >>>>> Young
>> >>>>>> <
>> >>>>>>>>>>>>>> ykt836@gmail.com
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>> If a specific connector want to have
>> >>>>> such
>> >>>>>>>>>>> parameter and
>> >>>>>>>>>>>>>>> read
>> >>>>>>>>>>>>>>>>> if out of
>> >>>>>>>>>>>>>>>>>>>>>>> configuration, then that's fine.
>> >>>>>>>>>>>>>>>>>>>>>>> If we are talking about a configuration
>> >>>>>> for all
>> >>>>>>>>>>> kinds of
>> >>>>>>>>>>>>>>>>> sources, I would
>> >>>>>>>>>>>>>>>>>>>>>>> be super careful about that.
>> >>>>>>>>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases,
>> >>>>>> but it
>> >>>>>>>>>>> will also
>> >>>>>>>>>>>>>>> make
>> >>>>>>>>>>>>>>>>> the left 20%
>> >>>>>>>>>>>>>>>>>>>>>>> feels weird.
>> >>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>>>>>>>>>> Kurt
>> >>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark
>> >>>>> Wu
>> >>>>>> <
>> >>>>>>>>>>>>>> imjark@gmail.com
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>> Hi Kurt,
>> >>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
>> >>>>>>>>>>>>>>>>>>>>>>>> I'm not saying to use the global
>> >>>>>>>>> configuration to
>> >>>>>>>>>>>>>>> override
>> >>>>>>>>>>>>>>>>> connector
>> >>>>>>>>>>>>>>>>>>>>>>>> properties by the planner.
>> >>>>>>>>>>>>>>>>>>>>>>>> But the connector should take this
>> >>>>>>>>> configuration
>> >>>>>>>>>>> and
>> >>>>>>>>>>>>>>>>> translate into their
>> >>>>>>>>>>>>>>>>>>>>>>>> client API.
>> >>>>>>>>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues
>> >>>>>> support
>> >>>>>>>>>>> eariliest
>> >>>>>>>>>>>>>>> and
>> >>>>>>>>>>>>>>>>> latest and a
>> >>>>>>>>>>>>>>>>>>>>>>>> timestamp value as start point.
>> >>>>>>>>>>>>>>>>>>>>>>>> So we can support 3 options for this
>> >>>>>>>>>>> configuration:
>> >>>>>>>>>>>>>>>>> "eariliest", "latest"
>> >>>>>>>>>>>>>>>>>>>>>>>> and a timestamp string value.
>> >>>>>>>>>>>>>>>>>>>>>>>> Of course, this can't solve 100%
>> >>>>>> cases, but I
>> >>>>>>>>>>> guess can
>> >>>>>>>>>>>>>>>>> sovle 80% or 90%
>> >>>>>>>>>>>>>>>>>>>>>>>> cases.
>> >>>>>>>>>>>>>>>>>>>>>>>> And the remaining cases can be
>> >>>>>> resolved by
>> >>>>>>>>> LIKE
>> >>>>>>>>>>> syntax
>> >>>>>>>>>>>>>>>> which
>> >>>>>>>>>>>>>>>>> I guess is
>> >>>>>>>>>>>>>>>>>>>>>>> not
>> >>>>>>>>>>>>>>>>>>>>>>>> very common cases.
>> >>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>>>>>>>>>>> Jark
>> >>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt
>> >>>>>> Young <
>> >>>>>>>>>>>>>>> ykt836@gmail.com
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>> Good to have such lovely
>> >>>>>> discussions. I
>> >>>>>>>>> also
>> >>>>>>>>>>> want to
>> >>>>>>>>>>>>>>>> share
>> >>>>>>>>>>>>>>>>> some of my
>> >>>>>>>>>>>>>>>>>>>>>>>>> opinions.
>> >>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I
>> >>>>>> also
>> >>>>>>>>> think
>> >>>>>>>>>>> ignore
>> >>>>>>>>>>>>>>>>> invalid hints would
>> >>>>>>>>>>>>>>>>>>>>>>>> be
>> >>>>>>>>>>>>>>>>>>>>>>>>> dangerous, maybe
>> >>>>>>>>>>>>>>>>>>>>>>>>> the simplest solution is just throw
>> >>>>>> an
>> >>>>>>>>>>> exception.
>> >>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>> #2 Regarding to property
>> >>>>>> replacement: I
>> >>>>>>>>> don't
>> >>>>>>>>>>> think
>> >>>>>>>>>>>>>> we
>> >>>>>>>>>>>>>>>>> should
>> >>>>>>>>>>>>>>>>>>>>>>> constraint
>> >>>>>>>>>>>>>>>>>>>>>>>>> ourself to
>> >>>>>>>>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
>> >>>>>>>>> forbidden
>> >>>>>>>>>>> it
>> >>>>>>>>>>>>>>>> modifying
>> >>>>>>>>>>>>>>>>> any
>> >>>>>>>>>>>>>>>>>>>>>>> properties
>> >>>>>>>>>>>>>>>>>>>>>>>>> which can effect
>> >>>>>>>>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is
>> >>>>>> one of
>> >>>>>>>>> the
>> >>>>>>>>>>> table
>> >>>>>>>>>>>>>>>> hints,
>> >>>>>>>>>>>>>>>>> and a
>> >>>>>>>>>>>>>>>>>>>>>>> powerful
>> >>>>>>>>>>>>>>>>>>>>>>>>> one. It can
>> >>>>>>>>>>>>>>>>>>>>>>>>> modify properties located in DDL's
>> >>>>>> WITH
>> >>>>>>>>> block.
>> >>>>>>>>>>> But I
>> >>>>>>>>>>>>>>> also
>> >>>>>>>>>>>>>>>>> see the harm
>> >>>>>>>>>>>>>>>>>>>>>>>> that
>> >>>>>>>>>>>>>>>>>>>>>>>>> if we make it
>> >>>>>>>>>>>>>>>>>>>>>>>>> too flexible like change the kafka
>> >>>>>> topic
>> >>>>>>>>> name
>> >>>>>>>>>>> with a
>> >>>>>>>>>>>>>>>> hint.
>> >>>>>>>>>>>>>>>>> Such use
>> >>>>>>>>>>>>>>>>>>>>>>> case
>> >>>>>>>>>>>>>>>>>>>>>>>> is
>> >>>>>>>>>>>>>>>>>>>>>>>>> not common and
>> >>>>>>>>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I
>> >>>>> would
>> >>>>>>>>> propose
>> >>>>>>>>>>> we have
>> >>>>>>>>>>>>>> a
>> >>>>>>>>>>>>>>>> map
>> >>>>>>>>>>>>>>>>> of hintable
>> >>>>>>>>>>>>>>>>>>>>>>>>> properties for each
>> >>>>>>>>>>>>>>>>>>>>>>>>> connector, and should validate all
>> >>>>>> passed
>> >>>>>>>>> in
>> >>>>>>>>>>>>>> properties
>> >>>>>>>>>>>>>>>>> are actually
>> >>>>>>>>>>>>>>>>>>>>>>>>> hintable. And combining with
>> >>>>>>>>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
>> >>>>>>>>> exception
>> >>>>>>>>>>> once
>> >>>>>>>>>>>>>>>> received
>> >>>>>>>>>>>>>>>>> invalid
>> >>>>>>>>>>>>>>>>>>>>>>>>> property.
>> >>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm
>> >>>>>> not sure
>> >>>>>>>>>>> it's
>> >>>>>>>>>>>>>>>> feasible.
>> >>>>>>>>>>>>>>>>> Different
>> >>>>>>>>>>>>>>>>>>>>>>>>> connectors will have totally
>> >>>>>>>>>>>>>>>>>>>>>>>>> different properties to represent
>> >>>>>> offset,
>> >>>>>>>>> some
>> >>>>>>>>>>> might
>> >>>>>>>>>>>>>> be
>> >>>>>>>>>>>>>>>>> timestamps,
>> >>>>>>>>>>>>>>>>>>>>>>> some
>> >>>>>>>>>>>>>>>>>>>>>>>>> might be string literals
>> >>>>>>>>>>>>>>>>>>>>>>>>> like "earliest", and others might
>> >>>>> be
>> >>>>>> just
>> >>>>>>>>>>> integers.
>> >>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>>>>>>>>>>>> Kurt
>> >>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM
>> >>>>>> Jark Wu <
>> >>>>>>>>>>>>>>>> imjark@gmail.com>
>> >>>>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>> 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 <
>> >>>>>>>>>>>>>>>>> twalthr@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 <
>> >>>>>>>>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
>> >>>>>>>>>>>>>>>>> twalthr@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
>> >>>>>>>>>>>>>>>> <
>> >>>>>>>>>>>>>>>>> imjark@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 <
>> >>>>>>>>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>
>> >>>
>> >>
>> >
>>
>>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <da...@apache.org>.
Thanks everyone for the feedback ~

- For the global config option belongs to `ExecutionConfigOptions` or
`OptimizerConfigOptions`, i have to strong objections, switch
to `OptimizerConfigOptions` is okey to me and i have updated the WIKI
- For use while-list or black-list, i have opinion with Timo, so black-list

I would fire a Vote if there are no other objections soon, thanks ~

Timo Walther <tw...@apache.org> 于2020年3月26日周四 下午6:31写道:

> Hi everyone,
>
> it is not only about security concerns. Hint options should be
> well-defined. We had a couple of people that were concerned about
> changing the semantics with a concept that is called "hint". These
> options are more like "debugging options" while someone is developing a
> connector or using a notebook to quickly produce some rows.
>
> The final pipeline should use a temporary table instead. I suggest to
> use a whitelist and force people to think about what should be exposed
> as a hint. By default, no option should be exposed. It is better to be
> conservative here.
>
> Regards,
> Timo
>
>
> On 26.03.20 10:31, Danny Chan wrote:
> > Thanks Kurt for the suggestion ~
> >
> > In my opinion:
> > - There is no need for TableFormatFactory#supportedHintOptions because
> all
> > the format options can be configured dynamically, they have no security
> > issues
> > - Dynamic table options is not an optimization, it is more like an
> > execution behavior from my side
> >
> > Kurt Young <yk...@gmail.com> 于2020年3月26日周四 下午4:47写道:
> >
> >> Hi Danny,
> >>
> >> Thanks for the updates. I have 2 comments regarding to latest document:
> >>
> >> 1) I think we also need `*supportedHintOptions*` for
> >> `*TableFormatFactory*`
> >> 2) IMO "dynamic-table-options.enabled" should belong to `
> >> *OptimizerConfigOptions*`
> >>
> >> Best,
> >> Kurt
> >>
> >>
> >> On Thu, Mar 26, 2020 at 4:40 PM Timo Walther <tw...@apache.org>
> wrote:
> >>
> >>> Thanks for the update Danny. +1 for this proposal.
> >>>
> >>> Regards,
> >>> Timo
> >>>
> >>> On 26.03.20 04:51, Danny Chan wrote:
> >>>> Thanks everyone who engaged in this discussion ~
> >>>>
> >>>> Our goal is "Supports Dynamic Table Options for Flink SQL". After an
> >>>> offline discussion with Kurt, Timo and Dawid, we have made the final
> >>>> conclusion, here is the summary:
> >>>>
> >>>>
> >>>>      - Use comment style syntax to specify the dynamic table options:
> >> "/*+
> >>>>      *OPTIONS*(k1='v1', k2='v2') */"
> >>>>      - Have constraint on the options keys: the options that may bring
> >> in
> >>>>      security problems should not be allowed, i.e. Kafka connector
> >>> zookeeper
> >>>>      endpoint URL and topic name
> >>>>      - Use white-list to control the allowed options for each
> connector,
> >>>>      which is more safe for future extention
> >>>>      - We allow to enable/disable this feature globally
> >>>>      - Implement based on the current code base first, and when
> FLIP-95
> >> is
> >>>>      checked in, implement this feature based on new interface
> >>>>
> >>>> Any suggestions are appreciated ~
> >>>>
> >>>> [1]
> >>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
> >>>>
> >>>> Best,
> >>>> Danny Chan
> >>>>
> >>>> Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:
> >>>>
> >>>>> Hi everyone,
> >>>>>
> >>>>> Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid
> >> it
> >>>>> doesn't solve the problems but increases some development and
> learning
> >>>>> burdens.
> >>>>>
> >>>>> # increase development and learning burden
> >>>>>
> >>>>> According to the discussion so far, we want to support overriding a
> >>> subset
> >>>>> of options in hints which doesn't affect semantics.
> >>>>> With the `supportedHintOptions`, it's up to the connector developers
> >> to
> >>>>> decide which options will not affect semantics, and to be hint
> >> options.
> >>>>> However, the question is how to distinguish whether an option will
> >>> *affect
> >>>>> semantics*? What happens if an option will affect semantics but
> >>> provided as
> >>>>> hint options?
> >>>>>   From my point of view, it's not easy to distinguish. For example,
> the
> >>>>> "format.ignore-parse-error" can be a very useful dynamic option but
> >> that
> >>>>> will affect semantic, because the result is different (null vs
> >>> exception).
> >>>>> Another example, the "connector.lookup.cache.*" options are also very
> >>>>> useful to tune jobs, however, it will also affect the job results. I
> >> can
> >>>>> come up many more useful options but may affect semantics.
> >>>>>
> >>>>> I can see that the community will under endless discussion around
> "can
> >>> this
> >>>>> option to be a hint option?",  "wether this option will affect
> >>> semantics?".
> >>>>> You can also find that we already have different opinions on
> >>>>> "ignore-parse-error". Those discussion is a waste of time! That's not
> >>> what
> >>>>> users want!
> >>>>> The problem is user need this, this, this options and HOW to expose
> >>> them?
> >>>>> We should focus on that.
> >>>>>
> >>>>> Then there could be two endings in the future:
> >>>>> 1) compromise on the usability, we drop the rule that hints don't
> >> affect
> >>>>> semantics, allow all the useful options in the hints list.
> >>>>> 2) stick on the rule, users will find this is a stumbling feature
> >> which
> >>>>> doesn't solve their problems.
> >>>>>       And they will be surprised why this option can't be set, but
> the
> >>> other
> >>>>> could. *semantic* is hard to be understood by users.
> >>>>>
> >>>>> # doesn't solve the problems
> >>>>>
> >>>>> I think the purpose of this FLIP is to allow users to quickly
> override
> >>> some
> >>>>> connectors' properties to tune their jobs.
> >>>>> However, `supportedHintOptions` is off track. It only allows a subset
> >>>>> options and for the users it's not *clear* which subset is allowed.
> >>>>>
> >>>>> Besides, I'm not sure `supportedHintOptions` can work well for all
> >>> cases.
> >>>>> How could you support kafka properties (`connector.properties.*`) as
> >>> hint
> >>>>> options? Some kafka properties may affect semantics
> >> (bootstrap.servers),
> >>>>> some may not (max.poll.records). Besides, I think it's not possible
> to
> >>> list
> >>>>> all the possible kafka properties [1].
> >>>>>
> >>>>> In summary, IMO, `supportedHintOptions`
> >>>>> (1) it increase the complexity to develop a connector
> >>>>> (2) it confuses users which options can be used in hint, which are
> >> not,
> >>>>> they have to check the docs again and again.
> >>>>> (3) it doesn't solve the problems which we want to solve by this
> FLIP.
> >>>>>
> >>>>> I think we should avoid introducing some partial solutions.
> Otherwise,
> >>> we
> >>>>> will be stuck in a loop that introduce new API -> deprecate API ->
> >>>>> introduce new API....
> >>>>>
> >>>>> I personally in favor of an explicit WITH syntax after the table as a
> >>> part
> >>>>> of the query which is mentioned by Kurt before, e.g. SELECT * from T
> >>>>> WITH('key' = 'value') .
> >>>>> It allows users to dynamically set options which can affect
> semantics.
> >>> It
> >>>>> will be very flexible to solve users' problems so far.
> >>>>>
> >>>>> Best,
> >>>>> Jark
> >>>>>
> >>>>> [1]: https://kafka.apache.org/documentation/#consumerconfigs
> >>>>>
> >>>>> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com>
> >> wrote:
> >>>>>
> >>>>>> My POC is here for the hints options merge [1].
> >>>>>>
> >>>>>> Personally, I have no strong objections for splitting hints with the
> >>>>>> CatalogTable, the only cons is a more complex implementation but the
> >>>>>> concept is more clear, and I have updated the WIKI.
> >>>>>>
> >>>>>> I think it would be nice if we can support the format “ignore-parse
> >>>>> error”
> >>>>>> option key, the CSV source already has a key [2] and we can use that
> >> in
> >>>>> the
> >>>>>> supportedHIntOptions, for the common CSV and JSON formats, we cal
> >> also
> >>>>> give
> >>>>>> a support. This is the only kind of key in formats that “do not
> >> change
> >>>>> the
> >>>>>> semantics” (somehow), what do you think about this ~
> >>>>>>
> >>>>>> [1]
> >>>>>>
> >>>>>
> >>>
> >>
> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
> >>>>>> [2]
> >>>>>>
> >>>>>
> >>>
> >>
> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
> >>>>>>
> >>>>>> Best,
> >>>>>> Danny Chan
> >>>>>> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
> >>>>>>> Hi everyone,
> >>>>>>>
> >>>>>>> +1 to Kurt's suggestion. Let's just have it in source and sink
> >>>>> factories
> >>>>>>> for now. We can still move this method up in the future. Currently,
> >> I
> >>>>>>> don't see a need for catalogs or formats. Because how would you
> >> target
> >>>>> a
> >>>>>>> format in the query?
> >>>>>>>
> >>>>>>> @Danny: Can you send a link to your PoC? I'm very skeptical about
> >>>>>>> creating a new CatalogTable in planner. Actually CatalogTable
> should
> >>> be
> >>>>>>> immutable between Catalog and Factory. Because a catalog can return
> >>> its
> >>>>>>> own factory and fully control the instantiation. Depending on the
> >>>>>>> implementation, that means it can be possible that the catalog has
> >>>>>>> encoded more information in a concrete subclass implementing the
> >>>>>>> interface. I vote for separating the concerns of catalog
> information
> >>>>> and
> >>>>>>> hints in the factory explicitly.
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Timo
> >>>>>>>
> >>>>>>>
> >>>>>>> On 18.03.20 05:41, Jingsong Li wrote:
> >>>>>>>> Hi,
> >>>>>>>>
> >>>>>>>> I am thinking we can provide hints to *table* related instances.
> >>>>>>>> - TableFormatFactory: of cause we need hints support, there are
> >> many
> >>>>>> format
> >>>>>>>> options in DDL too.
> >>>>>>>> - catalog and module: I don't know, maybe in future we can provide
> >>>>> some
> >>>>>>>> hints for them.
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Jingsong Lee
> >>>>>>>>
> >>>>>>>> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yuzhao.cyz@gmail.com
> >
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Yes, I think we should move the `supportedHintOptions` from
> >>>>>> TableFactory
> >>>>>>>>> to TableSourceFactory, and we also need to add the interface to
> >>>>>>>>> TableSinkFactory though because sink target table may also have
> >>>>> hints
> >>>>>>>>> attached.
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Danny Chan
> >>>>>>>>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> >>>>>>>>>> Have one question for adding `supportedHintOptions` method to
> >>>>>>>>>> `TableFactory`. It seems
> >>>>>>>>>> `TableFactory` is a base factory interface for all *table
> module*
> >>>>>> related
> >>>>>>>>>> instances, such as
> >>>>>>>>>> catalog, module, format and so on. It's not created only for
> >>>>>> *table*. Is
> >>>>>>>>> it
> >>>>>>>>>> possible to move it
> >>>>>>>>>> to `TableSourceFactory`?
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Kurt
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
> >>>>> yuzhao.cyz@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Thanks Timo ~
> >>>>>>>>>>>
> >>>>>>>>>>> For the naming itself, I also think the PROPERTIES is not that
> >>>>>>>>> concise, so
> >>>>>>>>>>> +1 for OPTIONS (I had thought about that, but there are many
> >>>>>> codes in
> >>>>>>>>>>> current Flink called it properties, i.e. the
> >>>>>> DescriptorProperties,
> >>>>>>>>>>> #getSupportedProperties), let’s use OPTIONS if this is our new
> >>>>>>>>> preference.
> >>>>>>>>>>>
> >>>>>>>>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
> >>>>>>>>> ConfigOption
> >>>>>>>>>>> can take more info. AFAIK, Spark also call their table options
> >>>>>> instead
> >>>>>>>>> of
> >>>>>>>>>>> properties. [1]
> >>>>>>>>>>>
> >>>>>>>>>>> In my local POC, I did create a new CatalogTable, and it works
> >>>>>> for
> >>>>>>>>> current
> >>>>>>>>>>> connectors well, all the DDL tables would finally yield a
> >>>>>> CatalogTable
> >>>>>>>>>>> instance and we can apply the options to that(in the
> >>>>>> CatalogSourceTable
> >>>>>>>>>>> when we generating the TableSource), the pros is that we do not
> >>>>>> need to
> >>>>>>>>>>> modify the codes of connectors itself. If we split the options
> >>>>>> from
> >>>>>>>>>>> CatalogTable, we may need to add some additional logic in each
> >>>>>>>>> connector
> >>>>>>>>>>> factories in order to merge these properties (and the logic are
> >>>>>> almost
> >>>>>>>>> the
> >>>>>>>>>>> same), what do you think about this?
> >>>>>>>>>>>
> >>>>>>>>>>> [1]
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> >>>>>>>>>>>
> >>>>>>>>>>> Best,
> >>>>>>>>>>> Danny Chan
> >>>>>>>>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
> >>>>>> ,写道:
> >>>>>>>>>>>> Hi Danny,
> >>>>>>>>>>>>
> >>>>>>>>>>>> thanks for updating the FLIP. I think your current design is
> >>>>>>>>> sufficient
> >>>>>>>>>>>> to separate hints from result-related properties.
> >>>>>>>>>>>>
> >>>>>>>>>>>> One remark to the naming itself: I would vote for calling the
> >>>>>> hints
> >>>>>>>>>>>> around table scan `OPTIONS('k'='v')`. We used the term
> >>>>>> "properties"
> >>>>>>>>> in
> >>>>>>>>>>>> the past but since we want to unify the Flink configuration
> >>>>>>>>> experience,
> >>>>>>>>>>>> we should use consistent naming and classes around
> >>>>>> `ConfigOptions`.
> >>>>>>>>>>>>
> >>>>>>>>>>>> It would be nice to use `Set<ConfigOption>
> >>>>>> supportedHintOptions();`
> >>>>>>>>> to
> >>>>>>>>>>>> start using config options instead of pure string properties.
> >>>>>> This
> >>>>>>>>> will
> >>>>>>>>>>>> also allow us to generate documentation in the future around
> >>>>>>>>> supported
> >>>>>>>>>>>> data types, ranges, etc. for options. At some point we would
> >>>>>> also
> >>>>>>>>> like
> >>>>>>>>>>>> to drop `DescriptorProperties` class. "Options" is also used
> >>>>>> in the
> >>>>>>>>>>>> documentation [1] and in the SQL/MED standard [2].
> >>>>>>>>>>>>
> >>>>>>>>>>>> Furthermore, I would still vote for separating CatalogTable
> >>>>>> and hint
> >>>>>>>>>>>> options. Otherwise the planner would need to create a new
> >>>>>>>>> CatalogTable
> >>>>>>>>>>>> instance which might not always be easy. We should offer them
> >>>>>> via:
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>
> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> >>>>>>>>>>>> ReadableConfig
> >>>>>>>>>>>>
> >>>>>>>>>>>> What do you think?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regards,
> >>>>>>>>>>>> Timo
> >>>>>>>>>>>>
> >>>>>>>>>>>> [1]
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> >>>>>>>>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
> >>>>>>>>>>>>> @Danny sounds good.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Maybe it is worth listing all the classes of problems that
> >>>>>> you
> >>>>>>>>> want to
> >>>>>>>>>>>>> address and then look at each class and see if hints are a
> >>>>>> good
> >>>>>>>>> default
> >>>>>>>>>>>>> solution or a good optional way of simplifying things?
> >>>>>>>>>>>>> The discussion has grown a lot and it is starting to be
> >>>>> hard
> >>>>>> to
> >>>>>>>>>>> distinguish
> >>>>>>>>>>>>> the parts where everyone agrees from the parts were there
> >>>>> are
> >>>>>>>>> concerns.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
> >>>>>> danny0405@apache.org>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks Stephan ~
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> We can remove the support for properties that may change
> >>>>>> the
> >>>>>>>>>>> semantics of
> >>>>>>>>>>>>>> query if you think that is a trouble.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> How about we support the /*+ properties() */ hint only
> >>>>> for
> >>>>>> those
> >>>>>>>>>>> optimize
> >>>>>>>>>>>>>> parameters, such as the fetch size of source or something
> >>>>>> like
> >>>>>>>>> that,
> >>>>>>>>>>> does
> >>>>>>>>>>>>>> that make sense?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I think Bowen has actually put it very well.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> (1) Hints that change semantics looks like trouble
> >>>>>> waiting to
> >>>>>>>>>>> happen. For
> >>>>>>>>>>>>>>> example Kafka offset handling should be in filters. The
> >>>>>> Kafka
> >>>>>>>>>>> source
> >>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>> support predicate pushdown.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> (2) Hints should not be a workaround for current
> >>>>>> shortcomings.
> >>>>>>>>> A
> >>>>>>>>>>> lot of
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> suggested above sounds exactly like that. Working
> >>>>> around
> >>>>>>>>>>> catalog/DDL
> >>>>>>>>>>>>>>> shortcomings, missing exposure of metadata (offsets),
> >>>>>> missing
> >>>>>>>>>>> predicate
> >>>>>>>>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as
> >>>>> a
> >>>>>> quick
> >>>>>>>>> fix
> >>>>>>>>>>> for
> >>>>>>>>>>>>>>> these issues, rather than fixing the root causes, will
> >>>>>> much
> >>>>>>>>> likely
> >>>>>>>>>>> bite
> >>>>>>>>>>>>>> us
> >>>>>>>>>>>>>>> back badly in the future.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>> Stephan
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
> >>>>>> ykt836@gmail.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> It seems this FLIP's name is somewhat misleading.
> >>>>> From
> >>>>>> my
> >>>>>>>>>>>>>> understanding,
> >>>>>>>>>>>>>>>> this FLIP is trying to
> >>>>>>>>>>>>>>>> address the dynamic parameter issue, and table hints
> >>>>>> is the
> >>>>>>>>> way
> >>>>>>>>>>> we wan
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> choose. I think we should
> >>>>>>>>>>>>>>>> be focus on "what's the right way to solve dynamic
> >>>>>> property"
> >>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>> discussing "whether table
> >>>>>>>>>>>>>>>> hints can affect query semantics".
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> For now, there are two proposed ways to achieve
> >>>>> dynamic
> >>>>>>>>> property:
> >>>>>>>>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with
> >>>>>> (xxx)
> >>>>>>>>>>>>>>>> 2. use custom "from t with (xxx)" syntax
> >>>>>>>>>>>>>>>> 3. "Borrow" the table hints to have a special
> >>>>>> PROPERTIES
> >>>>>>>>> hint.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> The first one didn't break anything, but the only
> >>>>>> problem i
> >>>>>>>>> see
> >>>>>>>>>>> is a
> >>>>>>>>>>>>>>> little
> >>>>>>>>>>>>>>>> more verbose than the table hint
> >>>>>>>>>>>>>>>> approach. I can imagine when someone using SQL CLI to
> >>>>>> have a
> >>>>>>>>> sql
> >>>>>>>>>>>>>>>> experience, it's quite often that
> >>>>>>>>>>>>>>>> he will modify the table property, some use cases i
> >>>>> can
> >>>>>>>>> think of:
> >>>>>>>>>>>>>>>> 1. the source contains some corrupted data, i want to
> >>>>>> turn
> >>>>>>>>> on the
> >>>>>>>>>>>>>>>> "ignore-error" flag for certain formats.
> >>>>>>>>>>>>>>>> 2. I have a kafka table and want to see some sample
> >>>>>> data
> >>>>>>>>> from the
> >>>>>>>>>>>>>>>> beginning, so i change the offset
> >>>>>>>>>>>>>>>> to "earliest", and then I want to observe the latest
> >>>>>> data
> >>>>>>>>> which
> >>>>>>>>>>> keeps
> >>>>>>>>>>>>>>>> coming in. I would write another query
> >>>>>>>>>>>>>>>> to select from the latest table.
> >>>>>>>>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly
> >>>>> then
> >>>>>> i can
> >>>>>>>>>>> observe
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> data from database side.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Most of such use cases are quite ad-hoc. If every
> >>>>> time
> >>>>>> I
> >>>>>>>>> want to
> >>>>>>>>>>> have a
> >>>>>>>>>>>>>>>> different experience, i need to create
> >>>>>>>>>>>>>>>> a temporary table and then also modify my query, it
> >>>>>> doesn't
> >>>>>>>>> feel
> >>>>>>>>>>>>>> smooth.
> >>>>>>>>>>>>>>>> Embed such dynamic property into
> >>>>>>>>>>>>>>>> query would have better user experience.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is
> >>>>>> breaking
> >>>>>>>>> SQL
> >>>>>>>>>>>>>>> compliant,
> >>>>>>>>>>>>>>>> and for #3, it only breaks some
> >>>>>>>>>>>>>>>> unwritten rules, but we can have an explanation on
> >>>>>> that. And
> >>>>>>>>> I
> >>>>>>>>>>> really
> >>>>>>>>>>>>>>> doubt
> >>>>>>>>>>>>>>>> whether user would complain about
> >>>>>>>>>>>>>>>> this when they actually have flexible and good
> >>>>>> experience
> >>>>>>>>> using
> >>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>> Kurt
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> >>>>>>>>> yuzhao.cyz@gmail.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks Aljoscha ~
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I agree for most of the query hints, they are
> >>>>>> optional as
> >>>>>>>>> an
> >>>>>>>>>>>>>> optimizer
> >>>>>>>>>>>>>>>>> instruction, especially for the traditional RDBMS.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> But, just like BenChao said, Flink as a computation
> >>>>>> engine
> >>>>>>>>> has
> >>>>>>>>>>> many
> >>>>>>>>>>>>>>>>> different kind of data sources, thus, dynamic
> >>>>>> parameters
> >>>>>>>>> like
> >>>>>>>>>>>>>>>> start_offest
> >>>>>>>>>>>>>>>>> can only bind to each table scope, we can not set a
> >>>>>> session
> >>>>>>>>>>> config
> >>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>> KSQL because they are all about Kafka:
> >>>>>>>>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thus the most flexible way to set up these dynamic
> >>>>>> params
> >>>>>>>>> is
> >>>>>>>>>>> to bind
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> the table scope in the query when we want to
> >>>>> override
> >>>>>>>>>>> something, so
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>> these solutions above (with pros and cons from my
> >>>>>> side):
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>>>>>> - Parameters are part of the main query
> >>>>>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>>>>> - Not SQL compliant
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */
> >>>>>> (from
> >>>>>>>>> me)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>>>>>> - SQL compliant because it is nested in the
> >>>>> comments
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>>>>> - Parameters are not part of the main query
> >>>>>>>>>>>>>>>>> - Cryptic syntax for new users
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> The biggest problem for hints way may be the “if
> >>>>>> hints
> >>>>>>>>> must be
> >>>>>>>>>>>>>>> optional”,
> >>>>>>>>>>>>>>>>> actually we have though about 1 for a while but
> >>>>>> aborted
> >>>>>>>>>>> because it
> >>>>>>>>>>>>>>> breaks
> >>>>>>>>>>>>>>>>> the SQL standard too much. And we replace it with
> >>>>> 2,
> >>>>>>>>> because
> >>>>>>>>>>> the
> >>>>>>>>>>>>>> hints
> >>>>>>>>>>>>>>>>> syntax do not break SQL standard(nested in
> >>>>> comments).
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint
> >>>>>> that
> >>>>>>>>> allows
> >>>>>>>>>>>>>> override
> >>>>>>>>>>>>>>>>> some properties of table dynamically, it does not
> >>>>>> break
> >>>>>>>>>>> anything, at
> >>>>>>>>>>>>>>>> lease
> >>>>>>>>>>>>>>>>> for current Flink use cases.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Planner hints are optional just because they are
> >>>>>> naturally
> >>>>>>>>>>> enforcers
> >>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> the planner, most of them aim to instruct the
> >>>>>> optimizer,
> >>>>>>>>> but,
> >>>>>>>>>>> the
> >>>>>>>>>>>>>> table
> >>>>>>>>>>>>>>>>> hints is a little different, table hints can
> >>>>> specify
> >>>>>> the
> >>>>>>>>> table
> >>>>>>>>>>> meta
> >>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>> index column, and it is very convenient to specify
> >>>>>> table
> >>>>>>>>>>> properties.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */
> >>>>>> table
> >>>>>>>>> hint,
> >>>>>>>>>>> we
> >>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>> call it table dynamic parameters.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>> Danny Chan
> >>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> >>>>>>>>>>> aljoscha@apache.org>,写道:
> >>>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I don't understand this discussion. Hints, as I
> >>>>>>>>> understand
> >>>>>>>>>>> them,
> >>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>> work like this:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> - hints are *optional* advice for the optimizer
> >>>>> to
> >>>>>> try
> >>>>>>>>> and
> >>>>>>>>>>> help it
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> find a good execution strategy
> >>>>>>>>>>>>>>>>>> - hints should not change query semantics, i.e.
> >>>>>> they
> >>>>>>>>> should
> >>>>>>>>>>> not
> >>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>> connector properties executing a query with
> >>>>> taking
> >>>>>> into
> >>>>>>>>>>> account the
> >>>>>>>>>>>>>>>>>> hints *must* produce the same result as executing
> >>>>>> the
> >>>>>>>>> query
> >>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>> taking into account the hints
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>   From these simple requirements you can derive a
> >>>>>> solution
> >>>>>>>>>>> that makes
> >>>>>>>>>>>>>>>>>> sense. I don't have a strong preference for the
> >>>>>> syntax
> >>>>>>>>> but we
> >>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>> strive to be in line with prior work.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>> Aljoscha
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
> >>>>>>>>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I agree with Kurt that option2 is too
> >>>>>> complicated to
> >>>>>>>>> use
> >>>>>>>>>>> because:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> • As a Kafka topic consumer, the user must
> >>>>>> define both
> >>>>>>>>> the
> >>>>>>>>>>>>>> virtual
> >>>>>>>>>>>>>>>>> column for start offset and he must apply a special
> >>>>>> filter
> >>>>>>>>>>> predicate
> >>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>> each query
> >>>>>>>>>>>>>>>>>>> • And for the internal implementation, the
> >>>>>> metadata
> >>>>>>>>> column
> >>>>>>>>>>> push
> >>>>>>>>>>>>>>> down
> >>>>>>>>>>>>>>>>> is another hard topic, each kind of message queue
> >>>>>> may have
> >>>>>>>>> its
> >>>>>>>>>>> offset
> >>>>>>>>>>>>>>>>> attribute, we need to consider the expression type
> >>>>>> for
> >>>>>>>>>>> different
> >>>>>>>>>>>>>> kind;
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> source also need to recognize the constant column
> >>>>> as
> >>>>>> a
> >>>>>>>>> config
> >>>>>>>>>>>>>>>> option(which
> >>>>>>>>>>>>>>>>> is weird because usually what we pushed down is a
> >>>>>> table
> >>>>>>>>> column)
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> For option 1 and option3, I think there is no
> >>>>>>>>> difference,
> >>>>>>>>>>> option1
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> also a hint syntax which is introduced in Sybase
> >>>>> and
> >>>>>>>>>>> referenced then
> >>>>>>>>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
> >>>>>>>>>>> ambitiousness.
> >>>>>>>>>>>>>>>> Personally
> >>>>>>>>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword
> >>>>>> for
> >>>>>>>>> these
> >>>>>>>>>>> reasons:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> • We do not break the standard SQL, the hints
> >>>>> are
> >>>>>>>>> nested
> >>>>>>>>>>> in SQL
> >>>>>>>>>>>>>>>>> comments
> >>>>>>>>>>>>>>>>>>> • We do not need to introduce additional WITH
> >>>>>> keyword
> >>>>>>>>>>> which may
> >>>>>>>>>>>>>>>> appear
> >>>>>>>>>>>>>>>>> in a query if we use that because a table can be
> >>>>>>>>> referenced in
> >>>>>>>>>>> all
> >>>>>>>>>>>>>>> kinds
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
> >>>>>> make
> >>>>>>>>> our
> >>>>>>>>>>> sql
> >>>>>>>>>>>>>> query
> >>>>>>>>>>>>>>>>> break too much of the SQL from standard
> >>>>>>>>>>>>>>>>>>> • We would have uniform syntax for hints as
> >>>>> query
> >>>>>>>>> hint, one
> >>>>>>>>>>>>>> syntax
> >>>>>>>>>>>>>>>>> fits all and more easy to use
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> And here is the reason why we choose a uniform
> >>>>>> Oracle
> >>>>>>>>>>> style query
> >>>>>>>>>>>>>>>>> hint syntax which is addressed by Julian Hyde when
> >>>>> we
> >>>>>>>>> design
> >>>>>>>>>>> the
> >>>>>>>>>>>>>> syntax
> >>>>>>>>>>>>>>>>> from the Calcite community:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for
> >>>>>> table
> >>>>>>>>> hints.
> >>>>>>>>>>> It
> >>>>>>>>>>>>>> adds a
> >>>>>>>>>>>>>>>>> new use of the WITH keyword that is unrelated to
> >>>>> the
> >>>>>> use of
> >>>>>>>>>>> WITH for
> >>>>>>>>>>>>>>>>> common-table expressions.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> A historical note. Microsoft SQL Server
> >>>>>> inherited its
> >>>>>>>>> hint
> >>>>>>>>>>> syntax
> >>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
> >>>>>>>>>>> Programming”[1], page
> >>>>>>>>>>>>>>>> 632,
> >>>>>>>>>>>>>>>>> “Optimizer hints”. The book was written in 1999,
> >>>>> and
> >>>>>> covers
> >>>>>>>>>>> Microsoft
> >>>>>>>>>>>>>>> SQL
> >>>>>>>>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
> >>>>>> but the
> >>>>>>>>>>> syntax very
> >>>>>>>>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft
> >>>>> SQL
> >>>>>>>>> Server was
> >>>>>>>>>>>>>> forked
> >>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> 1993.)
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Microsoft later added the WITH keyword to make
> >>>>>> it less
> >>>>>>>>>>> ambiguous,
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> has now deprecated the syntax that does not use
> >>>>> WITH.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> They are forced to keep the syntax for
> >>>>> backwards
> >>>>>>>>>>> compatibility
> >>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>> that doesn’t mean that we should shoulder their
> >>>>>> burden.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I think formatted comments are the right
> >>>>>> container for
> >>>>>>>>>>> hints
> >>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>> it allows us to change the hint syntax without
> >>>>>> changing
> >>>>>>>>> the SQL
> >>>>>>>>>>>>>> parser,
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> makes clear that we are at liberty to ignore hints
> >>>>>>>>> entirely.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Julian
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
> >>>>>>>>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>> Danny Chan
> >>>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> >>>>>>>>> twalthr@apache.org
> >>>>>>>>>>>> ,写道:
> >>>>>>>>>>>>>>>>>>>> Hi Danny,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> it is true that our DDL is not standard
> >>>>>> compliant by
> >>>>>>>>>>> using the
> >>>>>>>>>>>>>>> WITH
> >>>>>>>>>>>>>>>>>>>> clause. Nevertheless, we aim for not
> >>>>> diverging
> >>>>>> too
> >>>>>>>>> much
> >>>>>>>>>>> and the
> >>>>>>>>>>>>>>>> LIKE
> >>>>>>>>>>>>>>>>>>>> clause is an example of that. It will solve
> >>>>>> things
> >>>>>>>>> like
> >>>>>>>>>>>>>>> overwriting
> >>>>>>>>>>>>>>>>>>>> WATERMARKs, add additional/modifying
> >>>>>> properties and
> >>>>>>>>>>> inherit
> >>>>>>>>>>>>>>> schema.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3
> >>>>>> types
> >>>>>>>>>>> definition
> >>>>>>>>>>>>>>>>> together.
> >>>>>>>>>>>>>>>>>>>> We are not the first ones that try to solve
> >>>>>> this.
> >>>>>>>>> There
> >>>>>>>>>>> is also
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> SQL
> >>>>>>>>>>>>>>>>>>>> MED standard [1] that tried to tackle this
> >>>>>> problem. I
> >>>>>>>>>>> think it
> >>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>> considered when designing the current DDL.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
> >>>>>>>>> offsets. I
> >>>>>>>>>>> will
> >>>>>>>>>>>>>>> give
> >>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>> examples and look forward to feedback here:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part
> >>>>>> of the
> >>>>>>>>>>> query
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>>>>>>>>> - Parameters are part of the main query
> >>>>>>>>>>>>>>>>>>>> - No complicated hinting syntax
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>>>>>>>> - Not SQL compliant
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> *Option 2* Use metadata in query
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
> >>>>>>>>>>>>>>>> SYSTEM_METADATA('offset'))`
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset >
> >>>>> TIMESTAMP
> >>>>>>>>>>> '2012-12-12
> >>>>>>>>>>>>>>>>> 12:34:22'`
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>>>>>>>> - SQL compliant in the query
> >>>>>>>>>>>>>>>>>>>> - Access of metadata in the DDL which is
> >>>>>> required
> >>>>>>>>> anyway
> >>>>>>>>>>>>>>>>>>>> - Regular pushdown rules apply
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>>>>>>>> - Users need to add an additional comlumn in
> >>>>>> the DDL
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> *Option 3*: Use hints for properties
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> `
> >>>>>>>>>>>>>>>>>>>> SELECT *
> >>>>>>>>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
> >>>>>>>>>>>>>>>>>>>> `
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>>>>>>>> - Parameters are not part of the main query
> >>>>>>>>>>>>>>>>>>>> - Cryptic syntax for new users
> >>>>>>>>>>>>>>>>>>>> - Not standard compliant.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> If we go with this option, I would suggest to
> >>>>>> make it
> >>>>>>>>>>> available
> >>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> separate map and don't mix it with statically
> >>>>>> defined
> >>>>>>>>>>>>>> properties.
> >>>>>>>>>>>>>>>>> Such
> >>>>>>>>>>>>>>>>>>>> that the factory can decide which properties
> >>>>>> have the
> >>>>>>>>>>> right to
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> overwritten by the hints:
> >>>>>>>>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
> >>>>>>>>>>> ReadableConfig
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>> Timo
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Currently I see 3 options as a
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
> >>>>>>>>>>>>>>>>>>>>> Thanks Bowen ~
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I agree we should somehow categorize our
> >>>>>> connector
> >>>>>>>>>>>>>> parameters.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> For type1, I’m already preparing a solution
> >>>>>> like
> >>>>>>>>> the
> >>>>>>>>>>>>>> Confluent
> >>>>>>>>>>>>>>>>> schema registry + Avro schema inference thing, so
> >>>>>> this may
> >>>>>>>>> not
> >>>>>>>>>>> be a
> >>>>>>>>>>>>>>>> problem
> >>>>>>>>>>>>>>>>> in the near future.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> For type3, I have some questions:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset >
> >>>>> 12pm
> >>>>>>>>> yesterday”
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Where does the offset column come from, a
> >>>>>> virtual
> >>>>>>>>>>> column from
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> table schema, you said that
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> They change
> >>>>>>>>>>>>>>>>>>>>> almost every time a query starts and have
> >>>>>> nothing
> >>>>>>>>> to
> >>>>>>>>>>> do with
> >>>>>>>>>>>>>>>>> metadata, thus
> >>>>>>>>>>>>>>>>>>>>> should not be part of table definition/DDL
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> But why you can reference it in the query,
> >>>>>> I’m
> >>>>>>>>>>> confused for
> >>>>>>>>>>>>>>> that,
> >>>>>>>>>>>>>>>>> can you elaborate a little ?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>> Danny Chan
> >>>>>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> >>>>>>>>>>> bowenli86@gmail.com
> >>>>>>>>>>>>>>> ,写道:
> >>>>>>>>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> The root cause of too much manual work is
> >>>>>> Flink
> >>>>>>>>> DDL
> >>>>>>>>>>> has
> >>>>>>>>>>>>>>> mixed 3
> >>>>>>>>>>>>>>>>> types of
> >>>>>>>>>>>>>>>>>>>>>> params together and doesn't handle each
> >>>>> of
> >>>>>> them
> >>>>>>>>> very
> >>>>>>>>>>> well.
> >>>>>>>>>>>>>>>> Below
> >>>>>>>>>>>>>>>>> are how I
> >>>>>>>>>>>>>>>>>>>>>> categorize them and corresponding
> >>>>>> solutions in my
> >>>>>>>>>>> mind:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
> >>>>>>>>> external
> >>>>>>>>>>>>>>>> endpoint/url,
> >>>>>>>>>>>>>>>>>>>>>> username/pwd, schemas, formats.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Such metadata are mostly already
> >>>>>> accessible in
> >>>>>>>>>>> external
> >>>>>>>>>>>>>>> system
> >>>>>>>>>>>>>>>>> as long as
> >>>>>>>>>>>>>>>>>>>>>> endpoints and credentials are provided.
> >>>>>> Flink can
> >>>>>>>>>>> get it
> >>>>>>>>>>>>>> thru
> >>>>>>>>>>>>>>>>> catalogs, but
> >>>>>>>>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus
> >>>>>> Flink
> >>>>>>>>> just
> >>>>>>>>>>> hasn't
> >>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>>>>>>>>>> leverage that. So the solution should be
> >>>>>> building
> >>>>>>>>>>> more
> >>>>>>>>>>>>>>>> catalogs.
> >>>>>>>>>>>>>>>>> Such
> >>>>>>>>>>>>>>>>>>>>>> params should be part of a Flink table
> >>>>>>>>>>> DDL/definition, and
> >>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>> overridable
> >>>>>>>>>>>>>>>>>>>>>> in any means.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc
> >>>>>> connector's
> >>>>>>>>>>> fetch size,
> >>>>>>>>>>>>>>>>> elasticsearch
> >>>>>>>>>>>>>>>>>>>>>> connector's bulk flush size.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Such params don't affect query results,
> >>>>> but
> >>>>>>>>> affect
> >>>>>>>>>>> how
> >>>>>>>>>>>>>>> results
> >>>>>>>>>>>>>>>>> are produced
> >>>>>>>>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) -
> >>>>> they
> >>>>>> are
> >>>>>>>>>>> essentially
> >>>>>>>>>>>>>>>>> execution and
> >>>>>>>>>>>>>>>>>>>>>> implementation details. They change often
> >>>>>> in
> >>>>>>>>>>> exploration or
> >>>>>>>>>>>>>>>>> development
> >>>>>>>>>>>>>>>>>>>>>> stages, but not quite frequently in
> >>>>>> well-defined
> >>>>>>>>>>>>>> long-running
> >>>>>>>>>>>>>>>>> pipelines.
> >>>>>>>>>>>>>>>>>>>>>> They should always have default values
> >>>>> and
> >>>>>> can be
> >>>>>>>>>>> missing
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> query. They
> >>>>>>>>>>>>>>>>>>>>>> can be part of a table DDL/definition,
> >>>>> but
> >>>>>> should
> >>>>>>>>>>> also be
> >>>>>>>>>>>>>>>>> replaceable in a
> >>>>>>>>>>>>>>>>>>>>>> query - *this is what table "hints" in
> >>>>>> FLIP-113
> >>>>>>>>>>> should
> >>>>>>>>>>>>>>> cover*.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka
> >>>>>> connector's
> >>>>>>>>>>> start
> >>>>>>>>>>>>>>> offset.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Such params affect query results - the
> >>>>>> semantics.
> >>>>>>>>>>> They'd
> >>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>> be as
> >>>>>>>>>>>>>>>>>>>>>> filter conditions in WHERE clause that
> >>>>> can
> >>>>>> be
> >>>>>>>>> pushed
> >>>>>>>>>>> down.
> >>>>>>>>>>>>>>> They
> >>>>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>>>>> almost every time a query starts and have
> >>>>>>>>> nothing to
> >>>>>>>>>>> do
> >>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>> metadata, thus
> >>>>>>>>>>>>>>>>>>>>>> should not be part of table
> >>>>>> definition/DDL, nor
> >>>>>>>>> be
> >>>>>>>>>>>>>> persisted
> >>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> catalogs.
> >>>>>>>>>>>>>>>>>>>>>> If they will, users should create views
> >>>>> to
> >>>>>> keep
> >>>>>>>>> such
> >>>>>>>>>>> params
> >>>>>>>>>>>>>>>>> around (note
> >>>>>>>>>>>>>>>>>>>>>> this is different from variable
> >>>>>> substitution).
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we
> >>>>>> get these
> >>>>>>>>>>> params
> >>>>>>>>>>>>>>> right,
> >>>>>>>>>>>>>>>>> here're the
> >>>>>>>>>>>>>>>>>>>>>> steps users need to do to develop and run
> >>>>>> a Flink
> >>>>>>>>>>> job:
> >>>>>>>>>>>>>>>>>>>>>> - configure a Flink
> >>>>>> ConfluentSchemaRegistry with
> >>>>>>>>> url,
> >>>>>>>>>>>>>>> username,
> >>>>>>>>>>>>>>>>> and password
> >>>>>>>>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset
> >>>>>>> 12pm
> >>>>>>>>>>> yesterday"
> >>>>>>>>>>>>>>>>> (simplified
> >>>>>>>>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink
> >>>>> automatically
> >>>>>>>>> retrieves
> >>>>>>>>>>> all
> >>>>>>>>>>>>>>>>> metadata of
> >>>>>>>>>>>>>>>>>>>>>> schema, file format, etc and start the
> >>>>> job
> >>>>>>>>>>>>>>>>>>>>>> - users want to make the job read Kafka
> >>>>>> topic
> >>>>>>>>>>> faster, so it
> >>>>>>>>>>>>>>>> goes
> >>>>>>>>>>>>>>>>> as "SELECT
> >>>>>>>>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/
> >>>>>> WHERE
> >>>>>>>>>>> offset >
> >>>>>>>>>>>>>> 12pm
> >>>>>>>>>>>>>>>>> yesterday"
> >>>>>>>>>>>>>>>>>>>>>> - done and satisfied, users submit it to
> >>>>>>>>> production
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with
> >>>>> (k1=v1,
> >>>>>>>>> k2=v2),
> >>>>>>>>>>> I think
> >>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>> nice-to-have feature, but not a
> >>>>>> strategically
> >>>>>>>>>>> critical,
> >>>>>>>>>>>>>>>>> long-term solution,
> >>>>>>>>>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>>>>>>> 1) It may seem promising at the current
> >>>>>> stage to
> >>>>>>>>>>> solve the
> >>>>>>>>>>>>>>>>>>>>>> too-much-manual-work problem, but that's
> >>>>>> only
> >>>>>>>>>>> because Flink
> >>>>>>>>>>>>>>>>> hasn't
> >>>>>>>>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3
> >>>>>> types
> >>>>>>>>> of
> >>>>>>>>>>> params
> >>>>>>>>>>>>>>> above
> >>>>>>>>>>>>>>>>> properly.
> >>>>>>>>>>>>>>>>>>>>>> Once we get the params types right, the
> >>>>>> LIKE
> >>>>>>>>> syntax
> >>>>>>>>>>> won't
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> important, and will be just an easier way
> >>>>>> to
> >>>>>>>>> create
> >>>>>>>>>>> tables
> >>>>>>>>>>>>>>>>> without retyping
> >>>>>>>>>>>>>>>>>>>>>> long fields like username and pwd.
> >>>>>>>>>>>>>>>>>>>>>> 2) Note that only some rare type of
> >>>>>> catalog can
> >>>>>>>>>>> store k-v
> >>>>>>>>>>>>>>>>> property pair, so
> >>>>>>>>>>>>>>>>>>>>>> table created this way often cannot be
> >>>>>>>>> persisted. In
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> foreseeable
> >>>>>>>>>>>>>>>>>>>>>> future, such catalog will only be
> >>>>>> HiveCatalog,
> >>>>>>>>> and
> >>>>>>>>>>> not
> >>>>>>>>>>>>>>> everyone
> >>>>>>>>>>>>>>>>> has a Hive
> >>>>>>>>>>>>>>>>>>>>>> metastore. To be honest, without
> >>>>>> persistence,
> >>>>>>>>>>> recreating
> >>>>>>>>>>>>>>> tables
> >>>>>>>>>>>>>>>>> every time
> >>>>>>>>>>>>>>>>>>>>>> this way is still a lot of keyboard
> >>>>> typing.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>> Bowen
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt
> >>>>> Young
> >>>>>> <
> >>>>>>>>>>>>>> ykt836@gmail.com
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> If a specific connector want to have
> >>>>> such
> >>>>>>>>>>> parameter and
> >>>>>>>>>>>>>>> read
> >>>>>>>>>>>>>>>>> if out of
> >>>>>>>>>>>>>>>>>>>>>>> configuration, then that's fine.
> >>>>>>>>>>>>>>>>>>>>>>> If we are talking about a configuration
> >>>>>> for all
> >>>>>>>>>>> kinds of
> >>>>>>>>>>>>>>>>> sources, I would
> >>>>>>>>>>>>>>>>>>>>>>> be super careful about that.
> >>>>>>>>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases,
> >>>>>> but it
> >>>>>>>>>>> will also
> >>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>> the left 20%
> >>>>>>>>>>>>>>>>>>>>>>> feels weird.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>> Kurt
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark
> >>>>> Wu
> >>>>>> <
> >>>>>>>>>>>>>> imjark@gmail.com
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Hi Kurt,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
> >>>>>>>>>>>>>>>>>>>>>>>> I'm not saying to use the global
> >>>>>>>>> configuration to
> >>>>>>>>>>>>>>> override
> >>>>>>>>>>>>>>>>> connector
> >>>>>>>>>>>>>>>>>>>>>>>> properties by the planner.
> >>>>>>>>>>>>>>>>>>>>>>>> But the connector should take this
> >>>>>>>>> configuration
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> translate into their
> >>>>>>>>>>>>>>>>>>>>>>>> client API.
> >>>>>>>>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues
> >>>>>> support
> >>>>>>>>>>> eariliest
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> latest and a
> >>>>>>>>>>>>>>>>>>>>>>>> timestamp value as start point.
> >>>>>>>>>>>>>>>>>>>>>>>> So we can support 3 options for this
> >>>>>>>>>>> configuration:
> >>>>>>>>>>>>>>>>> "eariliest", "latest"
> >>>>>>>>>>>>>>>>>>>>>>>> and a timestamp string value.
> >>>>>>>>>>>>>>>>>>>>>>>> Of course, this can't solve 100%
> >>>>>> cases, but I
> >>>>>>>>>>> guess can
> >>>>>>>>>>>>>>>>> sovle 80% or 90%
> >>>>>>>>>>>>>>>>>>>>>>>> cases.
> >>>>>>>>>>>>>>>>>>>>>>>> And the remaining cases can be
> >>>>>> resolved by
> >>>>>>>>> LIKE
> >>>>>>>>>>> syntax
> >>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>> I guess is
> >>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>> very common cases.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>>> Jark
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt
> >>>>>> Young <
> >>>>>>>>>>>>>>> ykt836@gmail.com
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Good to have such lovely
> >>>>>> discussions. I
> >>>>>>>>> also
> >>>>>>>>>>> want to
> >>>>>>>>>>>>>>>> share
> >>>>>>>>>>>>>>>>> some of my
> >>>>>>>>>>>>>>>>>>>>>>>>> opinions.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I
> >>>>>> also
> >>>>>>>>> think
> >>>>>>>>>>> ignore
> >>>>>>>>>>>>>>>>> invalid hints would
> >>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>> dangerous, maybe
> >>>>>>>>>>>>>>>>>>>>>>>>> the simplest solution is just throw
> >>>>>> an
> >>>>>>>>>>> exception.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> #2 Regarding to property
> >>>>>> replacement: I
> >>>>>>>>> don't
> >>>>>>>>>>> think
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>> constraint
> >>>>>>>>>>>>>>>>>>>>>>>>> ourself to
> >>>>>>>>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
> >>>>>>>>> forbidden
> >>>>>>>>>>> it
> >>>>>>>>>>>>>>>> modifying
> >>>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>>>>> properties
> >>>>>>>>>>>>>>>>>>>>>>>>> which can effect
> >>>>>>>>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is
> >>>>>> one of
> >>>>>>>>> the
> >>>>>>>>>>> table
> >>>>>>>>>>>>>>>> hints,
> >>>>>>>>>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>>>>>>> powerful
> >>>>>>>>>>>>>>>>>>>>>>>>> one. It can
> >>>>>>>>>>>>>>>>>>>>>>>>> modify properties located in DDL's
> >>>>>> WITH
> >>>>>>>>> block.
> >>>>>>>>>>> But I
> >>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>> see the harm
> >>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>> if we make it
> >>>>>>>>>>>>>>>>>>>>>>>>> too flexible like change the kafka
> >>>>>> topic
> >>>>>>>>> name
> >>>>>>>>>>> with a
> >>>>>>>>>>>>>>>> hint.
> >>>>>>>>>>>>>>>>> Such use
> >>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>> not common and
> >>>>>>>>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I
> >>>>> would
> >>>>>>>>> propose
> >>>>>>>>>>> we have
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> map
> >>>>>>>>>>>>>>>>> of hintable
> >>>>>>>>>>>>>>>>>>>>>>>>> properties for each
> >>>>>>>>>>>>>>>>>>>>>>>>> connector, and should validate all
> >>>>>> passed
> >>>>>>>>> in
> >>>>>>>>>>>>>> properties
> >>>>>>>>>>>>>>>>> are actually
> >>>>>>>>>>>>>>>>>>>>>>>>> hintable. And combining with
> >>>>>>>>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
> >>>>>>>>> exception
> >>>>>>>>>>> once
> >>>>>>>>>>>>>>>> received
> >>>>>>>>>>>>>>>>> invalid
> >>>>>>>>>>>>>>>>>>>>>>>>> property.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm
> >>>>>> not sure
> >>>>>>>>>>> it's
> >>>>>>>>>>>>>>>> feasible.
> >>>>>>>>>>>>>>>>> Different
> >>>>>>>>>>>>>>>>>>>>>>>>> connectors will have totally
> >>>>>>>>>>>>>>>>>>>>>>>>> different properties to represent
> >>>>>> offset,
> >>>>>>>>> some
> >>>>>>>>>>> might
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>> timestamps,
> >>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>> might be string literals
> >>>>>>>>>>>>>>>>>>>>>>>>> like "earliest", and others might
> >>>>> be
> >>>>>> just
> >>>>>>>>>>> integers.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>>>> Kurt
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM
> >>>>>> Jark Wu <
> >>>>>>>>>>>>>>>> imjark@gmail.com>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>>>>> twalthr@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 <
> >>>>>>>>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
> >>>>>>>>>>>>>>>>> twalthr@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
> >>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>> imjark@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 <
> >>>>>>>>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>>
> >>
> >
>
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@apache.org>.
Hi everyone,

it is not only about security concerns. Hint options should be 
well-defined. We had a couple of people that were concerned about 
changing the semantics with a concept that is called "hint". These 
options are more like "debugging options" while someone is developing a 
connector or using a notebook to quickly produce some rows.

The final pipeline should use a temporary table instead. I suggest to 
use a whitelist and force people to think about what should be exposed 
as a hint. By default, no option should be exposed. It is better to be 
conservative here.

Regards,
Timo


On 26.03.20 10:31, Danny Chan wrote:
> Thanks Kurt for the suggestion ~
> 
> In my opinion:
> - There is no need for TableFormatFactory#supportedHintOptions because all
> the format options can be configured dynamically, they have no security
> issues
> - Dynamic table options is not an optimization, it is more like an
> execution behavior from my side
> 
> Kurt Young <yk...@gmail.com> 于2020年3月26日周四 下午4:47写道:
> 
>> Hi Danny,
>>
>> Thanks for the updates. I have 2 comments regarding to latest document:
>>
>> 1) I think we also need `*supportedHintOptions*` for
>> `*TableFormatFactory*`
>> 2) IMO "dynamic-table-options.enabled" should belong to `
>> *OptimizerConfigOptions*`
>>
>> Best,
>> Kurt
>>
>>
>> On Thu, Mar 26, 2020 at 4:40 PM Timo Walther <tw...@apache.org> wrote:
>>
>>> Thanks for the update Danny. +1 for this proposal.
>>>
>>> Regards,
>>> Timo
>>>
>>> On 26.03.20 04:51, Danny Chan wrote:
>>>> Thanks everyone who engaged in this discussion ~
>>>>
>>>> Our goal is "Supports Dynamic Table Options for Flink SQL". After an
>>>> offline discussion with Kurt, Timo and Dawid, we have made the final
>>>> conclusion, here is the summary:
>>>>
>>>>
>>>>      - Use comment style syntax to specify the dynamic table options:
>> "/*+
>>>>      *OPTIONS*(k1='v1', k2='v2') */"
>>>>      - Have constraint on the options keys: the options that may bring
>> in
>>>>      security problems should not be allowed, i.e. Kafka connector
>>> zookeeper
>>>>      endpoint URL and topic name
>>>>      - Use white-list to control the allowed options for each connector,
>>>>      which is more safe for future extention
>>>>      - We allow to enable/disable this feature globally
>>>>      - Implement based on the current code base first, and when FLIP-95
>> is
>>>>      checked in, implement this feature based on new interface
>>>>
>>>> Any suggestions are appreciated ~
>>>>
>>>> [1]
>>>>
>>>
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
>>>>
>>>> Best,
>>>> Danny Chan
>>>>
>>>> Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:
>>>>
>>>>> Hi everyone,
>>>>>
>>>>> Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid
>> it
>>>>> doesn't solve the problems but increases some development and learning
>>>>> burdens.
>>>>>
>>>>> # increase development and learning burden
>>>>>
>>>>> According to the discussion so far, we want to support overriding a
>>> subset
>>>>> of options in hints which doesn't affect semantics.
>>>>> With the `supportedHintOptions`, it's up to the connector developers
>> to
>>>>> decide which options will not affect semantics, and to be hint
>> options.
>>>>> However, the question is how to distinguish whether an option will
>>> *affect
>>>>> semantics*? What happens if an option will affect semantics but
>>> provided as
>>>>> hint options?
>>>>>   From my point of view, it's not easy to distinguish. For example, the
>>>>> "format.ignore-parse-error" can be a very useful dynamic option but
>> that
>>>>> will affect semantic, because the result is different (null vs
>>> exception).
>>>>> Another example, the "connector.lookup.cache.*" options are also very
>>>>> useful to tune jobs, however, it will also affect the job results. I
>> can
>>>>> come up many more useful options but may affect semantics.
>>>>>
>>>>> I can see that the community will under endless discussion around "can
>>> this
>>>>> option to be a hint option?",  "wether this option will affect
>>> semantics?".
>>>>> You can also find that we already have different opinions on
>>>>> "ignore-parse-error". Those discussion is a waste of time! That's not
>>> what
>>>>> users want!
>>>>> The problem is user need this, this, this options and HOW to expose
>>> them?
>>>>> We should focus on that.
>>>>>
>>>>> Then there could be two endings in the future:
>>>>> 1) compromise on the usability, we drop the rule that hints don't
>> affect
>>>>> semantics, allow all the useful options in the hints list.
>>>>> 2) stick on the rule, users will find this is a stumbling feature
>> which
>>>>> doesn't solve their problems.
>>>>>       And they will be surprised why this option can't be set, but the
>>> other
>>>>> could. *semantic* is hard to be understood by users.
>>>>>
>>>>> # doesn't solve the problems
>>>>>
>>>>> I think the purpose of this FLIP is to allow users to quickly override
>>> some
>>>>> connectors' properties to tune their jobs.
>>>>> However, `supportedHintOptions` is off track. It only allows a subset
>>>>> options and for the users it's not *clear* which subset is allowed.
>>>>>
>>>>> Besides, I'm not sure `supportedHintOptions` can work well for all
>>> cases.
>>>>> How could you support kafka properties (`connector.properties.*`) as
>>> hint
>>>>> options? Some kafka properties may affect semantics
>> (bootstrap.servers),
>>>>> some may not (max.poll.records). Besides, I think it's not possible to
>>> list
>>>>> all the possible kafka properties [1].
>>>>>
>>>>> In summary, IMO, `supportedHintOptions`
>>>>> (1) it increase the complexity to develop a connector
>>>>> (2) it confuses users which options can be used in hint, which are
>> not,
>>>>> they have to check the docs again and again.
>>>>> (3) it doesn't solve the problems which we want to solve by this FLIP.
>>>>>
>>>>> I think we should avoid introducing some partial solutions. Otherwise,
>>> we
>>>>> will be stuck in a loop that introduce new API -> deprecate API ->
>>>>> introduce new API....
>>>>>
>>>>> I personally in favor of an explicit WITH syntax after the table as a
>>> part
>>>>> of the query which is mentioned by Kurt before, e.g. SELECT * from T
>>>>> WITH('key' = 'value') .
>>>>> It allows users to dynamically set options which can affect semantics.
>>> It
>>>>> will be very flexible to solve users' problems so far.
>>>>>
>>>>> Best,
>>>>> Jark
>>>>>
>>>>> [1]: https://kafka.apache.org/documentation/#consumerconfigs
>>>>>
>>>>> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com>
>> wrote:
>>>>>
>>>>>> My POC is here for the hints options merge [1].
>>>>>>
>>>>>> Personally, I have no strong objections for splitting hints with the
>>>>>> CatalogTable, the only cons is a more complex implementation but the
>>>>>> concept is more clear, and I have updated the WIKI.
>>>>>>
>>>>>> I think it would be nice if we can support the format “ignore-parse
>>>>> error”
>>>>>> option key, the CSV source already has a key [2] and we can use that
>> in
>>>>> the
>>>>>> supportedHIntOptions, for the common CSV and JSON formats, we cal
>> also
>>>>> give
>>>>>> a support. This is the only kind of key in formats that “do not
>> change
>>>>> the
>>>>>> semantics” (somehow), what do you think about this ~
>>>>>>
>>>>>> [1]
>>>>>>
>>>>>
>>>
>> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
>>>>>> [2]
>>>>>>
>>>>>
>>>
>> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
>>>>>>
>>>>>> Best,
>>>>>> Danny Chan
>>>>>> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
>>>>>>> Hi everyone,
>>>>>>>
>>>>>>> +1 to Kurt's suggestion. Let's just have it in source and sink
>>>>> factories
>>>>>>> for now. We can still move this method up in the future. Currently,
>> I
>>>>>>> don't see a need for catalogs or formats. Because how would you
>> target
>>>>> a
>>>>>>> format in the query?
>>>>>>>
>>>>>>> @Danny: Can you send a link to your PoC? I'm very skeptical about
>>>>>>> creating a new CatalogTable in planner. Actually CatalogTable should
>>> be
>>>>>>> immutable between Catalog and Factory. Because a catalog can return
>>> its
>>>>>>> own factory and fully control the instantiation. Depending on the
>>>>>>> implementation, that means it can be possible that the catalog has
>>>>>>> encoded more information in a concrete subclass implementing the
>>>>>>> interface. I vote for separating the concerns of catalog information
>>>>> and
>>>>>>> hints in the factory explicitly.
>>>>>>>
>>>>>>> Regards,
>>>>>>> Timo
>>>>>>>
>>>>>>>
>>>>>>> On 18.03.20 05:41, Jingsong Li wrote:
>>>>>>>> Hi,
>>>>>>>>
>>>>>>>> I am thinking we can provide hints to *table* related instances.
>>>>>>>> - TableFormatFactory: of cause we need hints support, there are
>> many
>>>>>> format
>>>>>>>> options in DDL too.
>>>>>>>> - catalog and module: I don't know, maybe in future we can provide
>>>>> some
>>>>>>>> hints for them.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Jingsong Lee
>>>>>>>>
>>>>>>>> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com>
>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Yes, I think we should move the `supportedHintOptions` from
>>>>>> TableFactory
>>>>>>>>> to TableSourceFactory, and we also need to add the interface to
>>>>>>>>> TableSinkFactory though because sink target table may also have
>>>>> hints
>>>>>>>>> attached.
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Danny Chan
>>>>>>>>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
>>>>>>>>>> Have one question for adding `supportedHintOptions` method to
>>>>>>>>>> `TableFactory`. It seems
>>>>>>>>>> `TableFactory` is a base factory interface for all *table module*
>>>>>> related
>>>>>>>>>> instances, such as
>>>>>>>>>> catalog, module, format and so on. It's not created only for
>>>>>> *table*. Is
>>>>>>>>> it
>>>>>>>>>> possible to move it
>>>>>>>>>> to `TableSourceFactory`?
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Kurt
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
>>>>> yuzhao.cyz@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Thanks Timo ~
>>>>>>>>>>>
>>>>>>>>>>> For the naming itself, I also think the PROPERTIES is not that
>>>>>>>>> concise, so
>>>>>>>>>>> +1 for OPTIONS (I had thought about that, but there are many
>>>>>> codes in
>>>>>>>>>>> current Flink called it properties, i.e. the
>>>>>> DescriptorProperties,
>>>>>>>>>>> #getSupportedProperties), let’s use OPTIONS if this is our new
>>>>>>>>> preference.
>>>>>>>>>>>
>>>>>>>>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
>>>>>>>>> ConfigOption
>>>>>>>>>>> can take more info. AFAIK, Spark also call their table options
>>>>>> instead
>>>>>>>>> of
>>>>>>>>>>> properties. [1]
>>>>>>>>>>>
>>>>>>>>>>> In my local POC, I did create a new CatalogTable, and it works
>>>>>> for
>>>>>>>>> current
>>>>>>>>>>> connectors well, all the DDL tables would finally yield a
>>>>>> CatalogTable
>>>>>>>>>>> instance and we can apply the options to that(in the
>>>>>> CatalogSourceTable
>>>>>>>>>>> when we generating the TableSource), the pros is that we do not
>>>>>> need to
>>>>>>>>>>> modify the codes of connectors itself. If we split the options
>>>>>> from
>>>>>>>>>>> CatalogTable, we may need to add some additional logic in each
>>>>>>>>> connector
>>>>>>>>>>> factories in order to merge these properties (and the logic are
>>>>>> almost
>>>>>>>>> the
>>>>>>>>>>> same), what do you think about this?
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>>
>>>>>>>>>
>>>>>>
>>>>>
>>>
>> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> Danny Chan
>>>>>>>>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
>>>>>> ,写道:
>>>>>>>>>>>> Hi Danny,
>>>>>>>>>>>>
>>>>>>>>>>>> thanks for updating the FLIP. I think your current design is
>>>>>>>>> sufficient
>>>>>>>>>>>> to separate hints from result-related properties.
>>>>>>>>>>>>
>>>>>>>>>>>> One remark to the naming itself: I would vote for calling the
>>>>>> hints
>>>>>>>>>>>> around table scan `OPTIONS('k'='v')`. We used the term
>>>>>> "properties"
>>>>>>>>> in
>>>>>>>>>>>> the past but since we want to unify the Flink configuration
>>>>>>>>> experience,
>>>>>>>>>>>> we should use consistent naming and classes around
>>>>>> `ConfigOptions`.
>>>>>>>>>>>>
>>>>>>>>>>>> It would be nice to use `Set<ConfigOption>
>>>>>> supportedHintOptions();`
>>>>>>>>> to
>>>>>>>>>>>> start using config options instead of pure string properties.
>>>>>> This
>>>>>>>>> will
>>>>>>>>>>>> also allow us to generate documentation in the future around
>>>>>>>>> supported
>>>>>>>>>>>> data types, ranges, etc. for options. At some point we would
>>>>>> also
>>>>>>>>> like
>>>>>>>>>>>> to drop `DescriptorProperties` class. "Options" is also used
>>>>>> in the
>>>>>>>>>>>> documentation [1] and in the SQL/MED standard [2].
>>>>>>>>>>>>
>>>>>>>>>>>> Furthermore, I would still vote for separating CatalogTable
>>>>>> and hint
>>>>>>>>>>>> options. Otherwise the planner would need to create a new
>>>>>>>>> CatalogTable
>>>>>>>>>>>> instance which might not always be easy. We should offer them
>>>>>> via:
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
>>>>>>>>>>>> ReadableConfig
>>>>>>>>>>>>
>>>>>>>>>>>> What do you think?
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>> Timo
>>>>>>>>>>>>
>>>>>>>>>>>> [1]
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>
>>>>>
>>>
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
>>>>>>>>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
>>>>>>>>>>>>> @Danny sounds good.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Maybe it is worth listing all the classes of problems that
>>>>>> you
>>>>>>>>> want to
>>>>>>>>>>>>> address and then look at each class and see if hints are a
>>>>>> good
>>>>>>>>> default
>>>>>>>>>>>>> solution or a good optional way of simplifying things?
>>>>>>>>>>>>> The discussion has grown a lot and it is starting to be
>>>>> hard
>>>>>> to
>>>>>>>>>>> distinguish
>>>>>>>>>>>>> the parts where everyone agrees from the parts were there
>>>>> are
>>>>>>>>> concerns.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
>>>>>> danny0405@apache.org>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks Stephan ~
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> We can remove the support for properties that may change
>>>>>> the
>>>>>>>>>>> semantics of
>>>>>>>>>>>>>> query if you think that is a trouble.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> How about we support the /*+ properties() */ hint only
>>>>> for
>>>>>> those
>>>>>>>>>>> optimize
>>>>>>>>>>>>>> parameters, such as the fetch size of source or something
>>>>>> like
>>>>>>>>> that,
>>>>>>>>>>> does
>>>>>>>>>>>>>> that make sense?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think Bowen has actually put it very well.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (1) Hints that change semantics looks like trouble
>>>>>> waiting to
>>>>>>>>>>> happen. For
>>>>>>>>>>>>>>> example Kafka offset handling should be in filters. The
>>>>>> Kafka
>>>>>>>>>>> source
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>> support predicate pushdown.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (2) Hints should not be a workaround for current
>>>>>> shortcomings.
>>>>>>>>> A
>>>>>>>>>>> lot of
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> suggested above sounds exactly like that. Working
>>>>> around
>>>>>>>>>>> catalog/DDL
>>>>>>>>>>>>>>> shortcomings, missing exposure of metadata (offsets),
>>>>>> missing
>>>>>>>>>>> predicate
>>>>>>>>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as
>>>>> a
>>>>>> quick
>>>>>>>>> fix
>>>>>>>>>>> for
>>>>>>>>>>>>>>> these issues, rather than fixing the root causes, will
>>>>>> much
>>>>>>>>> likely
>>>>>>>>>>> bite
>>>>>>>>>>>>>> us
>>>>>>>>>>>>>>> back badly in the future.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>> Stephan
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
>>>>>> ykt836@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It seems this FLIP's name is somewhat misleading.
>>>>> From
>>>>>> my
>>>>>>>>>>>>>> understanding,
>>>>>>>>>>>>>>>> this FLIP is trying to
>>>>>>>>>>>>>>>> address the dynamic parameter issue, and table hints
>>>>>> is the
>>>>>>>>> way
>>>>>>>>>>> we wan
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> choose. I think we should
>>>>>>>>>>>>>>>> be focus on "what's the right way to solve dynamic
>>>>>> property"
>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>> discussing "whether table
>>>>>>>>>>>>>>>> hints can affect query semantics".
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For now, there are two proposed ways to achieve
>>>>> dynamic
>>>>>>>>> property:
>>>>>>>>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with
>>>>>> (xxx)
>>>>>>>>>>>>>>>> 2. use custom "from t with (xxx)" syntax
>>>>>>>>>>>>>>>> 3. "Borrow" the table hints to have a special
>>>>>> PROPERTIES
>>>>>>>>> hint.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The first one didn't break anything, but the only
>>>>>> problem i
>>>>>>>>> see
>>>>>>>>>>> is a
>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>> more verbose than the table hint
>>>>>>>>>>>>>>>> approach. I can imagine when someone using SQL CLI to
>>>>>> have a
>>>>>>>>> sql
>>>>>>>>>>>>>>>> experience, it's quite often that
>>>>>>>>>>>>>>>> he will modify the table property, some use cases i
>>>>> can
>>>>>>>>> think of:
>>>>>>>>>>>>>>>> 1. the source contains some corrupted data, i want to
>>>>>> turn
>>>>>>>>> on the
>>>>>>>>>>>>>>>> "ignore-error" flag for certain formats.
>>>>>>>>>>>>>>>> 2. I have a kafka table and want to see some sample
>>>>>> data
>>>>>>>>> from the
>>>>>>>>>>>>>>>> beginning, so i change the offset
>>>>>>>>>>>>>>>> to "earliest", and then I want to observe the latest
>>>>>> data
>>>>>>>>> which
>>>>>>>>>>> keeps
>>>>>>>>>>>>>>>> coming in. I would write another query
>>>>>>>>>>>>>>>> to select from the latest table.
>>>>>>>>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly
>>>>> then
>>>>>> i can
>>>>>>>>>>> observe
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> data from database side.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Most of such use cases are quite ad-hoc. If every
>>>>> time
>>>>>> I
>>>>>>>>> want to
>>>>>>>>>>> have a
>>>>>>>>>>>>>>>> different experience, i need to create
>>>>>>>>>>>>>>>> a temporary table and then also modify my query, it
>>>>>> doesn't
>>>>>>>>> feel
>>>>>>>>>>>>>> smooth.
>>>>>>>>>>>>>>>> Embed such dynamic property into
>>>>>>>>>>>>>>>> query would have better user experience.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is
>>>>>> breaking
>>>>>>>>> SQL
>>>>>>>>>>>>>>> compliant,
>>>>>>>>>>>>>>>> and for #3, it only breaks some
>>>>>>>>>>>>>>>> unwritten rules, but we can have an explanation on
>>>>>> that. And
>>>>>>>>> I
>>>>>>>>>>> really
>>>>>>>>>>>>>>> doubt
>>>>>>>>>>>>>>>> whether user would complain about
>>>>>>>>>>>>>>>> this when they actually have flexible and good
>>>>>> experience
>>>>>>>>> using
>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> Kurt
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
>>>>>>>>> yuzhao.cyz@gmail.com
>>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks Aljoscha ~
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I agree for most of the query hints, they are
>>>>>> optional as
>>>>>>>>> an
>>>>>>>>>>>>>> optimizer
>>>>>>>>>>>>>>>>> instruction, especially for the traditional RDBMS.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> But, just like BenChao said, Flink as a computation
>>>>>> engine
>>>>>>>>> has
>>>>>>>>>>> many
>>>>>>>>>>>>>>>>> different kind of data sources, thus, dynamic
>>>>>> parameters
>>>>>>>>> like
>>>>>>>>>>>>>>>> start_offest
>>>>>>>>>>>>>>>>> can only bind to each table scope, we can not set a
>>>>>> session
>>>>>>>>>>> config
>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>> KSQL because they are all about Kafka:
>>>>>>>>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thus the most flexible way to set up these dynamic
>>>>>> params
>>>>>>>>> is
>>>>>>>>>>> to bind
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> the table scope in the query when we want to
>>>>> override
>>>>>>>>>>> something, so
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>> these solutions above (with pros and cons from my
>>>>>> side):
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>>>>>> - Parameters are part of the main query
>>>>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>>>>> - Not SQL compliant
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */
>>>>>> (from
>>>>>>>>> me)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>>>>>> - SQL compliant because it is nested in the
>>>>> comments
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>>>>> - Parameters are not part of the main query
>>>>>>>>>>>>>>>>> - Cryptic syntax for new users
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The biggest problem for hints way may be the “if
>>>>>> hints
>>>>>>>>> must be
>>>>>>>>>>>>>>> optional”,
>>>>>>>>>>>>>>>>> actually we have though about 1 for a while but
>>>>>> aborted
>>>>>>>>>>> because it
>>>>>>>>>>>>>>> breaks
>>>>>>>>>>>>>>>>> the SQL standard too much. And we replace it with
>>>>> 2,
>>>>>>>>> because
>>>>>>>>>>> the
>>>>>>>>>>>>>> hints
>>>>>>>>>>>>>>>>> syntax do not break SQL standard(nested in
>>>>> comments).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint
>>>>>> that
>>>>>>>>> allows
>>>>>>>>>>>>>> override
>>>>>>>>>>>>>>>>> some properties of table dynamically, it does not
>>>>>> break
>>>>>>>>>>> anything, at
>>>>>>>>>>>>>>>> lease
>>>>>>>>>>>>>>>>> for current Flink use cases.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Planner hints are optional just because they are
>>>>>> naturally
>>>>>>>>>>> enforcers
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> the planner, most of them aim to instruct the
>>>>>> optimizer,
>>>>>>>>> but,
>>>>>>>>>>> the
>>>>>>>>>>>>>> table
>>>>>>>>>>>>>>>>> hints is a little different, table hints can
>>>>> specify
>>>>>> the
>>>>>>>>> table
>>>>>>>>>>> meta
>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>> index column, and it is very convenient to specify
>>>>>> table
>>>>>>>>>>> properties.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */
>>>>>> table
>>>>>>>>> hint,
>>>>>>>>>>> we
>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>> call it table dynamic parameters.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>> Danny Chan
>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
>>>>>>>>>>> aljoscha@apache.org>,写道:
>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I don't understand this discussion. Hints, as I
>>>>>>>>> understand
>>>>>>>>>>> them,
>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>> work like this:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> - hints are *optional* advice for the optimizer
>>>>> to
>>>>>> try
>>>>>>>>> and
>>>>>>>>>>> help it
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> find a good execution strategy
>>>>>>>>>>>>>>>>>> - hints should not change query semantics, i.e.
>>>>>> they
>>>>>>>>> should
>>>>>>>>>>> not
>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>> connector properties executing a query with
>>>>> taking
>>>>>> into
>>>>>>>>>>> account the
>>>>>>>>>>>>>>>>>> hints *must* produce the same result as executing
>>>>>> the
>>>>>>>>> query
>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>> taking into account the hints
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>   From these simple requirements you can derive a
>>>>>> solution
>>>>>>>>>>> that makes
>>>>>>>>>>>>>>>>>> sense. I don't have a strong preference for the
>>>>>> syntax
>>>>>>>>> but we
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>> strive to be in line with prior work.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> Aljoscha
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
>>>>>>>>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I agree with Kurt that option2 is too
>>>>>> complicated to
>>>>>>>>> use
>>>>>>>>>>> because:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> • As a Kafka topic consumer, the user must
>>>>>> define both
>>>>>>>>> the
>>>>>>>>>>>>>> virtual
>>>>>>>>>>>>>>>>> column for start offset and he must apply a special
>>>>>> filter
>>>>>>>>>>> predicate
>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>> each query
>>>>>>>>>>>>>>>>>>> • And for the internal implementation, the
>>>>>> metadata
>>>>>>>>> column
>>>>>>>>>>> push
>>>>>>>>>>>>>>> down
>>>>>>>>>>>>>>>>> is another hard topic, each kind of message queue
>>>>>> may have
>>>>>>>>> its
>>>>>>>>>>> offset
>>>>>>>>>>>>>>>>> attribute, we need to consider the expression type
>>>>>> for
>>>>>>>>>>> different
>>>>>>>>>>>>>> kind;
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> source also need to recognize the constant column
>>>>> as
>>>>>> a
>>>>>>>>> config
>>>>>>>>>>>>>>>> option(which
>>>>>>>>>>>>>>>>> is weird because usually what we pushed down is a
>>>>>> table
>>>>>>>>> column)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> For option 1 and option3, I think there is no
>>>>>>>>> difference,
>>>>>>>>>>> option1
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> also a hint syntax which is introduced in Sybase
>>>>> and
>>>>>>>>>>> referenced then
>>>>>>>>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
>>>>>>>>>>> ambitiousness.
>>>>>>>>>>>>>>>> Personally
>>>>>>>>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword
>>>>>> for
>>>>>>>>> these
>>>>>>>>>>> reasons:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> • We do not break the standard SQL, the hints
>>>>> are
>>>>>>>>> nested
>>>>>>>>>>> in SQL
>>>>>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>>>>> • We do not need to introduce additional WITH
>>>>>> keyword
>>>>>>>>>>> which may
>>>>>>>>>>>>>>>> appear
>>>>>>>>>>>>>>>>> in a query if we use that because a table can be
>>>>>>>>> referenced in
>>>>>>>>>>> all
>>>>>>>>>>>>>>> kinds
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
>>>>>> make
>>>>>>>>> our
>>>>>>>>>>> sql
>>>>>>>>>>>>>> query
>>>>>>>>>>>>>>>>> break too much of the SQL from standard
>>>>>>>>>>>>>>>>>>> • We would have uniform syntax for hints as
>>>>> query
>>>>>>>>> hint, one
>>>>>>>>>>>>>> syntax
>>>>>>>>>>>>>>>>> fits all and more easy to use
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> And here is the reason why we choose a uniform
>>>>>> Oracle
>>>>>>>>>>> style query
>>>>>>>>>>>>>>>>> hint syntax which is addressed by Julian Hyde when
>>>>> we
>>>>>>>>> design
>>>>>>>>>>> the
>>>>>>>>>>>>>> syntax
>>>>>>>>>>>>>>>>> from the Calcite community:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for
>>>>>> table
>>>>>>>>> hints.
>>>>>>>>>>> It
>>>>>>>>>>>>>> adds a
>>>>>>>>>>>>>>>>> new use of the WITH keyword that is unrelated to
>>>>> the
>>>>>> use of
>>>>>>>>>>> WITH for
>>>>>>>>>>>>>>>>> common-table expressions.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> A historical note. Microsoft SQL Server
>>>>>> inherited its
>>>>>>>>> hint
>>>>>>>>>>> syntax
>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
>>>>>>>>>>> Programming”[1], page
>>>>>>>>>>>>>>>> 632,
>>>>>>>>>>>>>>>>> “Optimizer hints”. The book was written in 1999,
>>>>> and
>>>>>> covers
>>>>>>>>>>> Microsoft
>>>>>>>>>>>>>>> SQL
>>>>>>>>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
>>>>>> but the
>>>>>>>>>>> syntax very
>>>>>>>>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft
>>>>> SQL
>>>>>>>>> Server was
>>>>>>>>>>>>>> forked
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> 1993.)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Microsoft later added the WITH keyword to make
>>>>>> it less
>>>>>>>>>>> ambiguous,
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> has now deprecated the syntax that does not use
>>>>> WITH.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> They are forced to keep the syntax for
>>>>> backwards
>>>>>>>>>>> compatibility
>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>> that doesn’t mean that we should shoulder their
>>>>>> burden.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I think formatted comments are the right
>>>>>> container for
>>>>>>>>>>> hints
>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>> it allows us to change the hint syntax without
>>>>>> changing
>>>>>>>>> the SQL
>>>>>>>>>>>>>> parser,
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> makes clear that we are at liberty to ignore hints
>>>>>>>>> entirely.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Julian
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
>>>>>>>>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>> Danny Chan
>>>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
>>>>>>>>> twalthr@apache.org
>>>>>>>>>>>> ,写道:
>>>>>>>>>>>>>>>>>>>> Hi Danny,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> it is true that our DDL is not standard
>>>>>> compliant by
>>>>>>>>>>> using the
>>>>>>>>>>>>>>> WITH
>>>>>>>>>>>>>>>>>>>> clause. Nevertheless, we aim for not
>>>>> diverging
>>>>>> too
>>>>>>>>> much
>>>>>>>>>>> and the
>>>>>>>>>>>>>>>> LIKE
>>>>>>>>>>>>>>>>>>>> clause is an example of that. It will solve
>>>>>> things
>>>>>>>>> like
>>>>>>>>>>>>>>> overwriting
>>>>>>>>>>>>>>>>>>>> WATERMARKs, add additional/modifying
>>>>>> properties and
>>>>>>>>>>> inherit
>>>>>>>>>>>>>>> schema.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3
>>>>>> types
>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>> together.
>>>>>>>>>>>>>>>>>>>> We are not the first ones that try to solve
>>>>>> this.
>>>>>>>>> There
>>>>>>>>>>> is also
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> SQL
>>>>>>>>>>>>>>>>>>>> MED standard [1] that tried to tackle this
>>>>>> problem. I
>>>>>>>>>>> think it
>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> considered when designing the current DDL.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
>>>>>>>>> offsets. I
>>>>>>>>>>> will
>>>>>>>>>>>>>>> give
>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>> examples and look forward to feedback here:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part
>>>>>> of the
>>>>>>>>>>> query
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>>>>>>>>> - Parameters are part of the main query
>>>>>>>>>>>>>>>>>>>> - No complicated hinting syntax
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>>>>>>>> - Not SQL compliant
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> *Option 2* Use metadata in query
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
>>>>>>>>>>>>>>>> SYSTEM_METADATA('offset'))`
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset >
>>>>> TIMESTAMP
>>>>>>>>>>> '2012-12-12
>>>>>>>>>>>>>>>>> 12:34:22'`
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>>>>>>>> - SQL compliant in the query
>>>>>>>>>>>>>>>>>>>> - Access of metadata in the DDL which is
>>>>>> required
>>>>>>>>> anyway
>>>>>>>>>>>>>>>>>>>> - Regular pushdown rules apply
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>>>>>>>> - Users need to add an additional comlumn in
>>>>>> the DDL
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> *Option 3*: Use hints for properties
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> `
>>>>>>>>>>>>>>>>>>>> SELECT *
>>>>>>>>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
>>>>>>>>>>>>>>>>>>>> `
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>>>>>>>> - Parameters are not part of the main query
>>>>>>>>>>>>>>>>>>>> - Cryptic syntax for new users
>>>>>>>>>>>>>>>>>>>> - Not standard compliant.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> If we go with this option, I would suggest to
>>>>>> make it
>>>>>>>>>>> available
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> separate map and don't mix it with statically
>>>>>> defined
>>>>>>>>>>>>>> properties.
>>>>>>>>>>>>>>>>> Such
>>>>>>>>>>>>>>>>>>>> that the factory can decide which properties
>>>>>> have the
>>>>>>>>>>> right to
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> overwritten by the hints:
>>>>>>>>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
>>>>>>>>>>> ReadableConfig
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>> Timo
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Currently I see 3 options as a
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
>>>>>>>>>>>>>>>>>>>>> Thanks Bowen ~
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I agree we should somehow categorize our
>>>>>> connector
>>>>>>>>>>>>>> parameters.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> For type1, I’m already preparing a solution
>>>>>> like
>>>>>>>>> the
>>>>>>>>>>>>>> Confluent
>>>>>>>>>>>>>>>>> schema registry + Avro schema inference thing, so
>>>>>> this may
>>>>>>>>> not
>>>>>>>>>>> be a
>>>>>>>>>>>>>>>> problem
>>>>>>>>>>>>>>>>> in the near future.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> For type3, I have some questions:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset >
>>>>> 12pm
>>>>>>>>> yesterday”
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Where does the offset column come from, a
>>>>>> virtual
>>>>>>>>>>> column from
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> table schema, you said that
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> They change
>>>>>>>>>>>>>>>>>>>>> almost every time a query starts and have
>>>>>> nothing
>>>>>>>>> to
>>>>>>>>>>> do with
>>>>>>>>>>>>>>>>> metadata, thus
>>>>>>>>>>>>>>>>>>>>> should not be part of table definition/DDL
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> But why you can reference it in the query,
>>>>>> I’m
>>>>>>>>>>> confused for
>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>> can you elaborate a little ?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>> Danny Chan
>>>>>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
>>>>>>>>>>> bowenli86@gmail.com
>>>>>>>>>>>>>>> ,写道:
>>>>>>>>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> The root cause of too much manual work is
>>>>>> Flink
>>>>>>>>> DDL
>>>>>>>>>>> has
>>>>>>>>>>>>>>> mixed 3
>>>>>>>>>>>>>>>>> types of
>>>>>>>>>>>>>>>>>>>>>> params together and doesn't handle each
>>>>> of
>>>>>> them
>>>>>>>>> very
>>>>>>>>>>> well.
>>>>>>>>>>>>>>>> Below
>>>>>>>>>>>>>>>>> are how I
>>>>>>>>>>>>>>>>>>>>>> categorize them and corresponding
>>>>>> solutions in my
>>>>>>>>>>> mind:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
>>>>>>>>> external
>>>>>>>>>>>>>>>> endpoint/url,
>>>>>>>>>>>>>>>>>>>>>> username/pwd, schemas, formats.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Such metadata are mostly already
>>>>>> accessible in
>>>>>>>>>>> external
>>>>>>>>>>>>>>> system
>>>>>>>>>>>>>>>>> as long as
>>>>>>>>>>>>>>>>>>>>>> endpoints and credentials are provided.
>>>>>> Flink can
>>>>>>>>>>> get it
>>>>>>>>>>>>>> thru
>>>>>>>>>>>>>>>>> catalogs, but
>>>>>>>>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus
>>>>>> Flink
>>>>>>>>> just
>>>>>>>>>>> hasn't
>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>> leverage that. So the solution should be
>>>>>> building
>>>>>>>>>>> more
>>>>>>>>>>>>>>>> catalogs.
>>>>>>>>>>>>>>>>> Such
>>>>>>>>>>>>>>>>>>>>>> params should be part of a Flink table
>>>>>>>>>>> DDL/definition, and
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>> overridable
>>>>>>>>>>>>>>>>>>>>>> in any means.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc
>>>>>> connector's
>>>>>>>>>>> fetch size,
>>>>>>>>>>>>>>>>> elasticsearch
>>>>>>>>>>>>>>>>>>>>>> connector's bulk flush size.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Such params don't affect query results,
>>>>> but
>>>>>>>>> affect
>>>>>>>>>>> how
>>>>>>>>>>>>>>> results
>>>>>>>>>>>>>>>>> are produced
>>>>>>>>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) -
>>>>> they
>>>>>> are
>>>>>>>>>>> essentially
>>>>>>>>>>>>>>>>> execution and
>>>>>>>>>>>>>>>>>>>>>> implementation details. They change often
>>>>>> in
>>>>>>>>>>> exploration or
>>>>>>>>>>>>>>>>> development
>>>>>>>>>>>>>>>>>>>>>> stages, but not quite frequently in
>>>>>> well-defined
>>>>>>>>>>>>>> long-running
>>>>>>>>>>>>>>>>> pipelines.
>>>>>>>>>>>>>>>>>>>>>> They should always have default values
>>>>> and
>>>>>> can be
>>>>>>>>>>> missing
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> query. They
>>>>>>>>>>>>>>>>>>>>>> can be part of a table DDL/definition,
>>>>> but
>>>>>> should
>>>>>>>>>>> also be
>>>>>>>>>>>>>>>>> replaceable in a
>>>>>>>>>>>>>>>>>>>>>> query - *this is what table "hints" in
>>>>>> FLIP-113
>>>>>>>>>>> should
>>>>>>>>>>>>>>> cover*.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka
>>>>>> connector's
>>>>>>>>>>> start
>>>>>>>>>>>>>>> offset.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Such params affect query results - the
>>>>>> semantics.
>>>>>>>>>>> They'd
>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>> be as
>>>>>>>>>>>>>>>>>>>>>> filter conditions in WHERE clause that
>>>>> can
>>>>>> be
>>>>>>>>> pushed
>>>>>>>>>>> down.
>>>>>>>>>>>>>>> They
>>>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>>>>> almost every time a query starts and have
>>>>>>>>> nothing to
>>>>>>>>>>> do
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>> metadata, thus
>>>>>>>>>>>>>>>>>>>>>> should not be part of table
>>>>>> definition/DDL, nor
>>>>>>>>> be
>>>>>>>>>>>>>> persisted
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> catalogs.
>>>>>>>>>>>>>>>>>>>>>> If they will, users should create views
>>>>> to
>>>>>> keep
>>>>>>>>> such
>>>>>>>>>>> params
>>>>>>>>>>>>>>>>> around (note
>>>>>>>>>>>>>>>>>>>>>> this is different from variable
>>>>>> substitution).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we
>>>>>> get these
>>>>>>>>>>> params
>>>>>>>>>>>>>>> right,
>>>>>>>>>>>>>>>>> here're the
>>>>>>>>>>>>>>>>>>>>>> steps users need to do to develop and run
>>>>>> a Flink
>>>>>>>>>>> job:
>>>>>>>>>>>>>>>>>>>>>> - configure a Flink
>>>>>> ConfluentSchemaRegistry with
>>>>>>>>> url,
>>>>>>>>>>>>>>> username,
>>>>>>>>>>>>>>>>> and password
>>>>>>>>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset
>>>>>>> 12pm
>>>>>>>>>>> yesterday"
>>>>>>>>>>>>>>>>> (simplified
>>>>>>>>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink
>>>>> automatically
>>>>>>>>> retrieves
>>>>>>>>>>> all
>>>>>>>>>>>>>>>>> metadata of
>>>>>>>>>>>>>>>>>>>>>> schema, file format, etc and start the
>>>>> job
>>>>>>>>>>>>>>>>>>>>>> - users want to make the job read Kafka
>>>>>> topic
>>>>>>>>>>> faster, so it
>>>>>>>>>>>>>>>> goes
>>>>>>>>>>>>>>>>> as "SELECT
>>>>>>>>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/
>>>>>> WHERE
>>>>>>>>>>> offset >
>>>>>>>>>>>>>> 12pm
>>>>>>>>>>>>>>>>> yesterday"
>>>>>>>>>>>>>>>>>>>>>> - done and satisfied, users submit it to
>>>>>>>>> production
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with
>>>>> (k1=v1,
>>>>>>>>> k2=v2),
>>>>>>>>>>> I think
>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> nice-to-have feature, but not a
>>>>>> strategically
>>>>>>>>>>> critical,
>>>>>>>>>>>>>>>>> long-term solution,
>>>>>>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>>>>> 1) It may seem promising at the current
>>>>>> stage to
>>>>>>>>>>> solve the
>>>>>>>>>>>>>>>>>>>>>> too-much-manual-work problem, but that's
>>>>>> only
>>>>>>>>>>> because Flink
>>>>>>>>>>>>>>>>> hasn't
>>>>>>>>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3
>>>>>> types
>>>>>>>>> of
>>>>>>>>>>> params
>>>>>>>>>>>>>>> above
>>>>>>>>>>>>>>>>> properly.
>>>>>>>>>>>>>>>>>>>>>> Once we get the params types right, the
>>>>>> LIKE
>>>>>>>>> syntax
>>>>>>>>>>> won't
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> important, and will be just an easier way
>>>>>> to
>>>>>>>>> create
>>>>>>>>>>> tables
>>>>>>>>>>>>>>>>> without retyping
>>>>>>>>>>>>>>>>>>>>>> long fields like username and pwd.
>>>>>>>>>>>>>>>>>>>>>> 2) Note that only some rare type of
>>>>>> catalog can
>>>>>>>>>>> store k-v
>>>>>>>>>>>>>>>>> property pair, so
>>>>>>>>>>>>>>>>>>>>>> table created this way often cannot be
>>>>>>>>> persisted. In
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> foreseeable
>>>>>>>>>>>>>>>>>>>>>> future, such catalog will only be
>>>>>> HiveCatalog,
>>>>>>>>> and
>>>>>>>>>>> not
>>>>>>>>>>>>>>> everyone
>>>>>>>>>>>>>>>>> has a Hive
>>>>>>>>>>>>>>>>>>>>>> metastore. To be honest, without
>>>>>> persistence,
>>>>>>>>>>> recreating
>>>>>>>>>>>>>>> tables
>>>>>>>>>>>>>>>>> every time
>>>>>>>>>>>>>>>>>>>>>> this way is still a lot of keyboard
>>>>> typing.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>> Bowen
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt
>>>>> Young
>>>>>> <
>>>>>>>>>>>>>> ykt836@gmail.com
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If a specific connector want to have
>>>>> such
>>>>>>>>>>> parameter and
>>>>>>>>>>>>>>> read
>>>>>>>>>>>>>>>>> if out of
>>>>>>>>>>>>>>>>>>>>>>> configuration, then that's fine.
>>>>>>>>>>>>>>>>>>>>>>> If we are talking about a configuration
>>>>>> for all
>>>>>>>>>>> kinds of
>>>>>>>>>>>>>>>>> sources, I would
>>>>>>>>>>>>>>>>>>>>>>> be super careful about that.
>>>>>>>>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases,
>>>>>> but it
>>>>>>>>>>> will also
>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>> the left 20%
>>>>>>>>>>>>>>>>>>>>>>> feels weird.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>> Kurt
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark
>>>>> Wu
>>>>>> <
>>>>>>>>>>>>>> imjark@gmail.com
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi Kurt,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
>>>>>>>>>>>>>>>>>>>>>>>> I'm not saying to use the global
>>>>>>>>> configuration to
>>>>>>>>>>>>>>> override
>>>>>>>>>>>>>>>>> connector
>>>>>>>>>>>>>>>>>>>>>>>> properties by the planner.
>>>>>>>>>>>>>>>>>>>>>>>> But the connector should take this
>>>>>>>>> configuration
>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> translate into their
>>>>>>>>>>>>>>>>>>>>>>>> client API.
>>>>>>>>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues
>>>>>> support
>>>>>>>>>>> eariliest
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> latest and a
>>>>>>>>>>>>>>>>>>>>>>>> timestamp value as start point.
>>>>>>>>>>>>>>>>>>>>>>>> So we can support 3 options for this
>>>>>>>>>>> configuration:
>>>>>>>>>>>>>>>>> "eariliest", "latest"
>>>>>>>>>>>>>>>>>>>>>>>> and a timestamp string value.
>>>>>>>>>>>>>>>>>>>>>>>> Of course, this can't solve 100%
>>>>>> cases, but I
>>>>>>>>>>> guess can
>>>>>>>>>>>>>>>>> sovle 80% or 90%
>>>>>>>>>>>>>>>>>>>>>>>> cases.
>>>>>>>>>>>>>>>>>>>>>>>> And the remaining cases can be
>>>>>> resolved by
>>>>>>>>> LIKE
>>>>>>>>>>> syntax
>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>> I guess is
>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> very common cases.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>> Jark
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt
>>>>>> Young <
>>>>>>>>>>>>>>> ykt836@gmail.com
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Good to have such lovely
>>>>>> discussions. I
>>>>>>>>> also
>>>>>>>>>>> want to
>>>>>>>>>>>>>>>> share
>>>>>>>>>>>>>>>>> some of my
>>>>>>>>>>>>>>>>>>>>>>>>> opinions.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I
>>>>>> also
>>>>>>>>> think
>>>>>>>>>>> ignore
>>>>>>>>>>>>>>>>> invalid hints would
>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>> dangerous, maybe
>>>>>>>>>>>>>>>>>>>>>>>>> the simplest solution is just throw
>>>>>> an
>>>>>>>>>>> exception.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> #2 Regarding to property
>>>>>> replacement: I
>>>>>>>>> don't
>>>>>>>>>>> think
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>> constraint
>>>>>>>>>>>>>>>>>>>>>>>>> ourself to
>>>>>>>>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
>>>>>>>>> forbidden
>>>>>>>>>>> it
>>>>>>>>>>>>>>>> modifying
>>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>>>>> properties
>>>>>>>>>>>>>>>>>>>>>>>>> which can effect
>>>>>>>>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is
>>>>>> one of
>>>>>>>>> the
>>>>>>>>>>> table
>>>>>>>>>>>>>>>> hints,
>>>>>>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>>>>> powerful
>>>>>>>>>>>>>>>>>>>>>>>>> one. It can
>>>>>>>>>>>>>>>>>>>>>>>>> modify properties located in DDL's
>>>>>> WITH
>>>>>>>>> block.
>>>>>>>>>>> But I
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>> see the harm
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>> if we make it
>>>>>>>>>>>>>>>>>>>>>>>>> too flexible like change the kafka
>>>>>> topic
>>>>>>>>> name
>>>>>>>>>>> with a
>>>>>>>>>>>>>>>> hint.
>>>>>>>>>>>>>>>>> Such use
>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>> not common and
>>>>>>>>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I
>>>>> would
>>>>>>>>> propose
>>>>>>>>>>> we have
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>> of hintable
>>>>>>>>>>>>>>>>>>>>>>>>> properties for each
>>>>>>>>>>>>>>>>>>>>>>>>> connector, and should validate all
>>>>>> passed
>>>>>>>>> in
>>>>>>>>>>>>>> properties
>>>>>>>>>>>>>>>>> are actually
>>>>>>>>>>>>>>>>>>>>>>>>> hintable. And combining with
>>>>>>>>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
>>>>>>>>> exception
>>>>>>>>>>> once
>>>>>>>>>>>>>>>> received
>>>>>>>>>>>>>>>>> invalid
>>>>>>>>>>>>>>>>>>>>>>>>> property.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm
>>>>>> not sure
>>>>>>>>>>> it's
>>>>>>>>>>>>>>>> feasible.
>>>>>>>>>>>>>>>>> Different
>>>>>>>>>>>>>>>>>>>>>>>>> connectors will have totally
>>>>>>>>>>>>>>>>>>>>>>>>> different properties to represent
>>>>>> offset,
>>>>>>>>> some
>>>>>>>>>>> might
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>> timestamps,
>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>> might be string literals
>>>>>>>>>>>>>>>>>>>>>>>>> like "earliest", and others might
>>>>> be
>>>>>> just
>>>>>>>>>>> integers.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>> Kurt
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM
>>>>>> Jark Wu <
>>>>>>>>>>>>>>>> imjark@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>> twalthr@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 <
>>>>>>>>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
>>>>>>>>>>>>>>>>> twalthr@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
>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>> imjark@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 <
>>>>>>>>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>>
>>
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Kurt Young <yk...@gmail.com>.
Sorry maybe I didn't make myself clear. I think some format property is
very suitable to
be hinted, like "ignore errors during parsing".  Maybe we should have a
dedicated
Hintable interface, and have `supportedHintOptions` method inside. All
factories supports
hint could implement from it.

Best,
Kurt


On Wed, Mar 18, 2020 at 9:10 PM Timo Walther <tw...@apache.org> wrote:

> Hi everyone,
>
> +1 to Kurt's suggestion. Let's just have it in source and sink factories
> for now. We can still move this method up in the future. Currently, I
> don't see a need for catalogs or formats. Because how would you target a
> format in the query?
>
> @Danny: Can you send a link to your PoC? I'm very skeptical about
> creating a new CatalogTable in planner. Actually CatalogTable should be
> immutable between Catalog and Factory. Because a catalog can return its
> own factory and fully control the instantiation. Depending on the
> implementation, that means it can be possible that the catalog has
> encoded more information in a concrete subclass implementing the
> interface. I vote for separating the concerns of catalog information and
> hints in the factory explicitly.
>
> Regards,
> Timo
>
>
> On 18.03.20 05:41, Jingsong Li wrote:
> > Hi,
> >
> > I am thinking we can provide hints to *table* related instances.
> > - TableFormatFactory: of cause we need hints support, there are many
> format
> > options in DDL too.
> > - catalog and module: I don't know, maybe in future we can provide some
> > hints for them.
> >
> > Best,
> > Jingsong Lee
> >
> > On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com>
> wrote:
> >
> >> Yes, I think we should move the `supportedHintOptions` from TableFactory
> >> to TableSourceFactory, and we also need to add the interface to
> >> TableSinkFactory though because sink target table may also have hints
> >> attached.
> >>
> >> Best,
> >> Danny Chan
> >> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> >>> Have one question for adding `supportedHintOptions` method to
> >>> `TableFactory`. It seems
> >>> `TableFactory` is a base factory interface for all *table module*
> related
> >>> instances, such as
> >>> catalog, module, format and so on. It's not created only for *table*.
> Is
> >> it
> >>> possible to move it
> >>> to `TableSourceFactory`?
> >>>
> >>> Best,
> >>> Kurt
> >>>
> >>>
> >>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <yu...@gmail.com>
> >> wrote:
> >>>
> >>>> Thanks Timo ~
> >>>>
> >>>> For the naming itself, I also think the PROPERTIES is not that
> >> concise, so
> >>>> +1 for OPTIONS (I had thought about that, but there are many codes in
> >>>> current Flink called it properties, i.e. the DescriptorProperties,
> >>>> #getSupportedProperties), let’s use OPTIONS if this is our new
> >> preference.
> >>>>
> >>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
> >> ConfigOption
> >>>> can take more info. AFAIK, Spark also call their table options instead
> >> of
> >>>> properties. [1]
> >>>>
> >>>> In my local POC, I did create a new CatalogTable, and it works for
> >> current
> >>>> connectors well, all the DDL tables would finally yield a CatalogTable
> >>>> instance and we can apply the options to that(in the
> CatalogSourceTable
> >>>> when we generating the TableSource), the pros is that we do not need
> to
> >>>> modify the codes of connectors itself. If we split the options from
> >>>> CatalogTable, we may need to add some additional logic in each
> >> connector
> >>>> factories in order to merge these properties (and the logic are almost
> >> the
> >>>> same), what do you think about this?
> >>>>
> >>>> [1]
> >>>>
> >>
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> >>>>
> >>>> Best,
> >>>> Danny Chan
> >>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <tw...@apache.org>,写道:
> >>>>> Hi Danny,
> >>>>>
> >>>>> thanks for updating the FLIP. I think your current design is
> >> sufficient
> >>>>> to separate hints from result-related properties.
> >>>>>
> >>>>> One remark to the naming itself: I would vote for calling the hints
> >>>>> around table scan `OPTIONS('k'='v')`. We used the term "properties"
> >> in
> >>>>> the past but since we want to unify the Flink configuration
> >> experience,
> >>>>> we should use consistent naming and classes around `ConfigOptions`.
> >>>>>
> >>>>> It would be nice to use `Set<ConfigOption> supportedHintOptions();`
> >> to
> >>>>> start using config options instead of pure string properties. This
> >> will
> >>>>> also allow us to generate documentation in the future around
> >> supported
> >>>>> data types, ranges, etc. for options. At some point we would also
> >> like
> >>>>> to drop `DescriptorProperties` class. "Options" is also used in the
> >>>>> documentation [1] and in the SQL/MED standard [2].
> >>>>>
> >>>>> Furthermore, I would still vote for separating CatalogTable and hint
> >>>>> options. Otherwise the planner would need to create a new
> >> CatalogTable
> >>>>> instance which might not always be easy. We should offer them via:
> >>>>>
> >>>>> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> >>>>> ReadableConfig
> >>>>>
> >>>>> What do you think?
> >>>>>
> >>>>> Regards,
> >>>>> Timo
> >>>>>
> >>>>> [1]
> >>>>>
> >>>>
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> >>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
> >>>>>
> >>>>>
> >>>>> On 12.03.20 15:06, Stephan Ewen wrote:
> >>>>>> @Danny sounds good.
> >>>>>>
> >>>>>> Maybe it is worth listing all the classes of problems that you
> >> want to
> >>>>>> address and then look at each class and see if hints are a good
> >> default
> >>>>>> solution or a good optional way of simplifying things?
> >>>>>> The discussion has grown a lot and it is starting to be hard to
> >>>> distinguish
> >>>>>> the parts where everyone agrees from the parts were there are
> >> concerns.
> >>>>>>
> >>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <da...@apache.org>
> >>>> wrote:
> >>>>>>
> >>>>>>> Thanks Stephan ~
> >>>>>>>
> >>>>>>> We can remove the support for properties that may change the
> >>>> semantics of
> >>>>>>> query if you think that is a trouble.
> >>>>>>>
> >>>>>>> How about we support the /*+ properties() */ hint only for those
> >>>> optimize
> >>>>>>> parameters, such as the fetch size of source or something like
> >> that,
> >>>> does
> >>>>>>> that make sense?
> >>>>>>>
> >>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> >>>>>>>
> >>>>>>>> I think Bowen has actually put it very well.
> >>>>>>>>
> >>>>>>>> (1) Hints that change semantics looks like trouble waiting to
> >>>> happen. For
> >>>>>>>> example Kafka offset handling should be in filters. The Kafka
> >>>> source
> >>>>>>> should
> >>>>>>>> support predicate pushdown.
> >>>>>>>>
> >>>>>>>> (2) Hints should not be a workaround for current shortcomings.
> >> A
> >>>> lot of
> >>>>>>> the
> >>>>>>>> suggested above sounds exactly like that. Working around
> >>>> catalog/DDL
> >>>>>>>> shortcomings, missing exposure of metadata (offsets), missing
> >>>> predicate
> >>>>>>>> pushdown in Kafka. Abusing a feature like hints now as a quick
> >> fix
> >>>> for
> >>>>>>>> these issues, rather than fixing the root causes, will much
> >> likely
> >>>> bite
> >>>>>>> us
> >>>>>>>> back badly in the future.
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Stephan
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com>
> >>>> wrote:
> >>>>>>>>
> >>>>>>>>> It seems this FLIP's name is somewhat misleading. From my
> >>>>>>> understanding,
> >>>>>>>>> this FLIP is trying to
> >>>>>>>>> address the dynamic parameter issue, and table hints is the
> >> way
> >>>> we wan
> >>>>>>> to
> >>>>>>>>> choose. I think we should
> >>>>>>>>> be focus on "what's the right way to solve dynamic property"
> >>>> instead of
> >>>>>>>>> discussing "whether table
> >>>>>>>>> hints can affect query semantics".
> >>>>>>>>>
> >>>>>>>>> For now, there are two proposed ways to achieve dynamic
> >> property:
> >>>>>>>>> 1. FLIP-110: create temporary table xx like xx with (xxx)
> >>>>>>>>> 2. use custom "from t with (xxx)" syntax
> >>>>>>>>> 3. "Borrow" the table hints to have a special PROPERTIES
> >> hint.
> >>>>>>>>>
> >>>>>>>>> The first one didn't break anything, but the only problem i
> >> see
> >>>> is a
> >>>>>>>> little
> >>>>>>>>> more verbose than the table hint
> >>>>>>>>> approach. I can imagine when someone using SQL CLI to have a
> >> sql
> >>>>>>>>> experience, it's quite often that
> >>>>>>>>> he will modify the table property, some use cases i can
> >> think of:
> >>>>>>>>> 1. the source contains some corrupted data, i want to turn
> >> on the
> >>>>>>>>> "ignore-error" flag for certain formats.
> >>>>>>>>> 2. I have a kafka table and want to see some sample data
> >> from the
> >>>>>>>>> beginning, so i change the offset
> >>>>>>>>> to "earliest", and then I want to observe the latest data
> >> which
> >>>> keeps
> >>>>>>>>> coming in. I would write another query
> >>>>>>>>> to select from the latest table.
> >>>>>>>>> 3. I want to my jdbc sink flush data more eagerly then i can
> >>>> observe
> >>>>>>> the
> >>>>>>>>> data from database side.
> >>>>>>>>>
> >>>>>>>>> Most of such use cases are quite ad-hoc. If every time I
> >> want to
> >>>> have a
> >>>>>>>>> different experience, i need to create
> >>>>>>>>> a temporary table and then also modify my query, it doesn't
> >> feel
> >>>>>>> smooth.
> >>>>>>>>> Embed such dynamic property into
> >>>>>>>>> query would have better user experience.
> >>>>>>>>>
> >>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is breaking
> >> SQL
> >>>>>>>> compliant,
> >>>>>>>>> and for #3, it only breaks some
> >>>>>>>>> unwritten rules, but we can have an explanation on that. And
> >> I
> >>>> really
> >>>>>>>> doubt
> >>>>>>>>> whether user would complain about
> >>>>>>>>> this when they actually have flexible and good experience
> >> using
> >>>> this.
> >>>>>>>>>
> >>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Kurt
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> >> yuzhao.cyz@gmail.com
> >>>>>
> >>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Thanks Aljoscha ~
> >>>>>>>>>>
> >>>>>>>>>> I agree for most of the query hints, they are optional as
> >> an
> >>>>>>> optimizer
> >>>>>>>>>> instruction, especially for the traditional RDBMS.
> >>>>>>>>>>
> >>>>>>>>>> But, just like BenChao said, Flink as a computation engine
> >> has
> >>>> many
> >>>>>>>>>> different kind of data sources, thus, dynamic parameters
> >> like
> >>>>>>>>> start_offest
> >>>>>>>>>> can only bind to each table scope, we can not set a session
> >>>> config
> >>>>>>> like
> >>>>>>>>>> KSQL because they are all about Kafka:
> >>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
> >>>>>>>>>>
> >>>>>>>>>> Thus the most flexible way to set up these dynamic params
> >> is
> >>>> to bind
> >>>>>>> to
> >>>>>>>>>> the table scope in the query when we want to override
> >>>> something, so
> >>>>>>> we
> >>>>>>>>> have
> >>>>>>>>>> these solutions above (with pros and cons from my side):
> >>>>>>>>>>
> >>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
> >>>>>>>>>>
> >>>>>>>>>> Pros:
> >>>>>>>>>> - Easy to add
> >>>>>>>>>> - Parameters are part of the main query
> >>>>>>>>>> Cons:
> >>>>>>>>>> - Not SQL compliant
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from
> >> me)
> >>>>>>>>>>
> >>>>>>>>>> Pros:
> >>>>>>>>>> - Easy to add
> >>>>>>>>>> - SQL compliant because it is nested in the comments
> >>>>>>>>>>
> >>>>>>>>>> Cons:
> >>>>>>>>>> - Parameters are not part of the main query
> >>>>>>>>>> - Cryptic syntax for new users
> >>>>>>>>>>
> >>>>>>>>>> The biggest problem for hints way may be the “if hints
> >> must be
> >>>>>>>> optional”,
> >>>>>>>>>> actually we have though about 1 for a while but aborted
> >>>> because it
> >>>>>>>> breaks
> >>>>>>>>>> the SQL standard too much. And we replace it with 2,
> >> because
> >>>> the
> >>>>>>> hints
> >>>>>>>>>> syntax do not break SQL standard(nested in comments).
> >>>>>>>>>>
> >>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint that
> >> allows
> >>>>>>> override
> >>>>>>>>>> some properties of table dynamically, it does not break
> >>>> anything, at
> >>>>>>>>> lease
> >>>>>>>>>> for current Flink use cases.
> >>>>>>>>>>
> >>>>>>>>>> Planner hints are optional just because they are naturally
> >>>> enforcers
> >>>>>>> of
> >>>>>>>>>> the planner, most of them aim to instruct the optimizer,
> >> but,
> >>>> the
> >>>>>>> table
> >>>>>>>>>> hints is a little different, table hints can specify the
> >> table
> >>>> meta
> >>>>>>>> like
> >>>>>>>>>> index column, and it is very convenient to specify table
> >>>> properties.
> >>>>>>>>>>
> >>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */ table
> >> hint,
> >>>> we
> >>>>>>> can
> >>>>>>>>>> call it table dynamic parameters.
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Danny Chan
> >>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> >>>> aljoscha@apache.org>,写道:
> >>>>>>>>>>> Hi,
> >>>>>>>>>>>
> >>>>>>>>>>> I don't understand this discussion. Hints, as I
> >> understand
> >>>> them,
> >>>>>>>> should
> >>>>>>>>>>> work like this:
> >>>>>>>>>>>
> >>>>>>>>>>> - hints are *optional* advice for the optimizer to try
> >> and
> >>>> help it
> >>>>>>> to
> >>>>>>>>>>> find a good execution strategy
> >>>>>>>>>>> - hints should not change query semantics, i.e. they
> >> should
> >>>> not
> >>>>>>>> change
> >>>>>>>>>>> connector properties executing a query with taking into
> >>>> account the
> >>>>>>>>>>> hints *must* produce the same result as executing the
> >> query
> >>>> without
> >>>>>>>>>>> taking into account the hints
> >>>>>>>>>>>
> >>>>>>>>>>>  From these simple requirements you can derive a solution
> >>>> that makes
> >>>>>>>>>>> sense. I don't have a strong preference for the syntax
> >> but we
> >>>>>>> should
> >>>>>>>>>>> strive to be in line with prior work.
> >>>>>>>>>>>
> >>>>>>>>>>> Best,
> >>>>>>>>>>> Aljoscha
> >>>>>>>>>>>
> >>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
> >>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
> >>>>>>>>>>>>
> >>>>>>>>>>>> I agree with Kurt that option2 is too complicated to
> >> use
> >>>> because:
> >>>>>>>>>>>>
> >>>>>>>>>>>> • As a Kafka topic consumer, the user must define both
> >> the
> >>>>>>> virtual
> >>>>>>>>>> column for start offset and he must apply a special filter
> >>>> predicate
> >>>>>>>>> after
> >>>>>>>>>> each query
> >>>>>>>>>>>> • And for the internal implementation, the metadata
> >> column
> >>>> push
> >>>>>>>> down
> >>>>>>>>>> is another hard topic, each kind of message queue may have
> >> its
> >>>> offset
> >>>>>>>>>> attribute, we need to consider the expression type for
> >>>> different
> >>>>>>> kind;
> >>>>>>>>> the
> >>>>>>>>>> source also need to recognize the constant column as a
> >> config
> >>>>>>>>> option(which
> >>>>>>>>>> is weird because usually what we pushed down is a table
> >> column)
> >>>>>>>>>>>>
> >>>>>>>>>>>> For option 1 and option3, I think there is no
> >> difference,
> >>>> option1
> >>>>>>>> is
> >>>>>>>>>> also a hint syntax which is introduced in Sybase and
> >>>> referenced then
> >>>>>>>>>> deprecated by MS-SQL in 199X years because of the
> >>>> ambitiousness.
> >>>>>>>>> Personally
> >>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword for
> >> these
> >>>> reasons:
> >>>>>>>>>>>>
> >>>>>>>>>>>> • We do not break the standard SQL, the hints are
> >> nested
> >>>> in SQL
> >>>>>>>>>> comments
> >>>>>>>>>>>> • We do not need to introduce additional WITH keyword
> >>>> which may
> >>>>>>>>> appear
> >>>>>>>>>> in a query if we use that because a table can be
> >> referenced in
> >>>> all
> >>>>>>>> kinds
> >>>>>>>>> of
> >>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make
> >> our
> >>>> sql
> >>>>>>> query
> >>>>>>>>>> break too much of the SQL from standard
> >>>>>>>>>>>> • We would have uniform syntax for hints as query
> >> hint, one
> >>>>>>> syntax
> >>>>>>>>>> fits all and more easy to use
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> And here is the reason why we choose a uniform Oracle
> >>>> style query
> >>>>>>>>>> hint syntax which is addressed by Julian Hyde when we
> >> design
> >>>> the
> >>>>>>> syntax
> >>>>>>>>>> from the Calcite community:
> >>>>>>>>>>>>
> >>>>>>>>>>>> I don’t much like the MSSQL-style syntax for table
> >> hints.
> >>>> It
> >>>>>>> adds a
> >>>>>>>>>> new use of the WITH keyword that is unrelated to the use of
> >>>> WITH for
> >>>>>>>>>> common-table expressions.
> >>>>>>>>>>>>
> >>>>>>>>>>>> A historical note. Microsoft SQL Server inherited its
> >> hint
> >>>> syntax
> >>>>>>>>> from
> >>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
> >>>> Programming”[1], page
> >>>>>>>>> 632,
> >>>>>>>>>> “Optimizer hints”. The book was written in 1999, and covers
> >>>> Microsoft
> >>>>>>>> SQL
> >>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the
> >>>> syntax very
> >>>>>>>>>> likely predates Sybase 4.3, from which Microsoft SQL
> >> Server was
> >>>>>>> forked
> >>>>>>>> in
> >>>>>>>>>> 1993.)
> >>>>>>>>>>>>
> >>>>>>>>>>>> Microsoft later added the WITH keyword to make it less
> >>>> ambiguous,
> >>>>>>>> and
> >>>>>>>>>> has now deprecated the syntax that does not use WITH.
> >>>>>>>>>>>>
> >>>>>>>>>>>> They are forced to keep the syntax for backwards
> >>>> compatibility
> >>>>>>> but
> >>>>>>>>>> that doesn’t mean that we should shoulder their burden.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I think formatted comments are the right container for
> >>>> hints
> >>>>>>>> because
> >>>>>>>>>> it allows us to change the hint syntax without changing
> >> the SQL
> >>>>>>> parser,
> >>>>>>>>> and
> >>>>>>>>>> makes clear that we are at liberty to ignore hints
> >> entirely.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Julian
> >>>>>>>>>>>>
> >>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
> >>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Best,
> >>>>>>>>>>>> Danny Chan
> >>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> >> twalthr@apache.org
> >>>>> ,写道:
> >>>>>>>>>>>>> Hi Danny,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> it is true that our DDL is not standard compliant by
> >>>> using the
> >>>>>>>> WITH
> >>>>>>>>>>>>> clause. Nevertheless, we aim for not diverging too
> >> much
> >>>> and the
> >>>>>>>>> LIKE
> >>>>>>>>>>>>> clause is an example of that. It will solve things
> >> like
> >>>>>>>> overwriting
> >>>>>>>>>>>>> WATERMARKs, add additional/modifying properties and
> >>>> inherit
> >>>>>>>> schema.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3 types
> >>>> definition
> >>>>>>>>>> together.
> >>>>>>>>>>>>> We are not the first ones that try to solve this.
> >> There
> >>>> is also
> >>>>>>>> the
> >>>>>>>>>> SQL
> >>>>>>>>>>>>> MED standard [1] that tried to tackle this problem. I
> >>>> think it
> >>>>>>>> was
> >>>>>>>>>> not
> >>>>>>>>>>>>> considered when designing the current DDL.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
> >> offsets. I
> >>>> will
> >>>>>>>> give
> >>>>>>>>>> some
> >>>>>>>>>>>>> examples and look forward to feedback here:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part of the
> >>>> query
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>> - Parameters are part of the main query
> >>>>>>>>>>>>> - No complicated hinting syntax
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>> - Not SQL compliant
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> *Option 2* Use metadata in query
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
> >>>>>>>>> SYSTEM_METADATA('offset'))`
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset > TIMESTAMP
> >>>> '2012-12-12
> >>>>>>>>>> 12:34:22'`
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>> - SQL compliant in the query
> >>>>>>>>>>>>> - Access of metadata in the DDL which is required
> >> anyway
> >>>>>>>>>>>>> - Regular pushdown rules apply
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>> - Users need to add an additional comlumn in the DDL
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> *Option 3*: Use hints for properties
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> `
> >>>>>>>>>>>>> SELECT *
> >>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
> >>>>>>>>>>>>> `
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>> - Parameters are not part of the main query
> >>>>>>>>>>>>> - Cryptic syntax for new users
> >>>>>>>>>>>>> - Not standard compliant.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> If we go with this option, I would suggest to make it
> >>>> available
> >>>>>>>> in
> >>>>>>>>> a
> >>>>>>>>>>>>> separate map and don't mix it with statically defined
> >>>>>>> properties.
> >>>>>>>>>> Such
> >>>>>>>>>>>>> that the factory can decide which properties have the
> >>>> right to
> >>>>>>> be
> >>>>>>>>>>>>> overwritten by the hints:
> >>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
> >>>> ReadableConfig
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>> Timo
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Currently I see 3 options as a
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
> >>>>>>>>>>>>>> Thanks Bowen ~
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I agree we should somehow categorize our connector
> >>>>>>> parameters.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> For type1, I’m already preparing a solution like
> >> the
> >>>>>>> Confluent
> >>>>>>>>>> schema registry + Avro schema inference thing, so this may
> >> not
> >>>> be a
> >>>>>>>>> problem
> >>>>>>>>>> in the near future.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> For type3, I have some questions:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset > 12pm
> >> yesterday”
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Where does the offset column come from, a virtual
> >>>> column from
> >>>>>>>> the
> >>>>>>>>>> table schema, you said that
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> They change
> >>>>>>>>>>>>>> almost every time a query starts and have nothing
> >> to
> >>>> do with
> >>>>>>>>>> metadata, thus
> >>>>>>>>>>>>>> should not be part of table definition/DDL
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> But why you can reference it in the query, I’m
> >>>> confused for
> >>>>>>>> that,
> >>>>>>>>>> can you elaborate a little ?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>> Danny Chan
> >>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> >>>> bowenli86@gmail.com
> >>>>>>>> ,写道:
> >>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The root cause of too much manual work is Flink
> >> DDL
> >>>> has
> >>>>>>>> mixed 3
> >>>>>>>>>> types of
> >>>>>>>>>>>>>>> params together and doesn't handle each of them
> >> very
> >>>> well.
> >>>>>>>>> Below
> >>>>>>>>>> are how I
> >>>>>>>>>>>>>>> categorize them and corresponding solutions in my
> >>>> mind:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - type 1: Metadata of external data, like
> >> external
> >>>>>>>>> endpoint/url,
> >>>>>>>>>>>>>>> username/pwd, schemas, formats.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Such metadata are mostly already accessible in
> >>>> external
> >>>>>>>> system
> >>>>>>>>>> as long as
> >>>>>>>>>>>>>>> endpoints and credentials are provided. Flink can
> >>>> get it
> >>>>>>> thru
> >>>>>>>>>> catalogs, but
> >>>>>>>>>>>>>>> we haven't had many catalogs yet and thus Flink
> >> just
> >>>> hasn't
> >>>>>>>>> been
> >>>>>>>>>> able to
> >>>>>>>>>>>>>>> leverage that. So the solution should be building
> >>>> more
> >>>>>>>>> catalogs.
> >>>>>>>>>> Such
> >>>>>>>>>>>>>>> params should be part of a Flink table
> >>>> DDL/definition, and
> >>>>>>>> not
> >>>>>>>>>> overridable
> >>>>>>>>>>>>>>> in any means.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc connector's
> >>>> fetch size,
> >>>>>>>>>> elasticsearch
> >>>>>>>>>>>>>>> connector's bulk flush size.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Such params don't affect query results, but
> >> affect
> >>>> how
> >>>>>>>> results
> >>>>>>>>>> are produced
> >>>>>>>>>>>>>>> (eg. fast or slow, aka performance) - they are
> >>>> essentially
> >>>>>>>>>> execution and
> >>>>>>>>>>>>>>> implementation details. They change often in
> >>>> exploration or
> >>>>>>>>>> development
> >>>>>>>>>>>>>>> stages, but not quite frequently in well-defined
> >>>>>>> long-running
> >>>>>>>>>> pipelines.
> >>>>>>>>>>>>>>> They should always have default values and can be
> >>>> missing
> >>>>>>> in
> >>>>>>>>>> query. They
> >>>>>>>>>>>>>>> can be part of a table DDL/definition, but should
> >>>> also be
> >>>>>>>>>> replaceable in a
> >>>>>>>>>>>>>>> query - *this is what table "hints" in FLIP-113
> >>>> should
> >>>>>>>> cover*.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - type 3: Semantic params, like kafka connector's
> >>>> start
> >>>>>>>> offset.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Such params affect query results - the semantics.
> >>>> They'd
> >>>>>>>> better
> >>>>>>>>>> be as
> >>>>>>>>>>>>>>> filter conditions in WHERE clause that can be
> >> pushed
> >>>> down.
> >>>>>>>> They
> >>>>>>>>>> change
> >>>>>>>>>>>>>>> almost every time a query starts and have
> >> nothing to
> >>>> do
> >>>>>>> with
> >>>>>>>>>> metadata, thus
> >>>>>>>>>>>>>>> should not be part of table definition/DDL, nor
> >> be
> >>>>>>> persisted
> >>>>>>>> in
> >>>>>>>>>> catalogs.
> >>>>>>>>>>>>>>> If they will, users should create views to keep
> >> such
> >>>> params
> >>>>>>>>>> around (note
> >>>>>>>>>>>>>>> this is different from variable substitution).
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we get these
> >>>> params
> >>>>>>>> right,
> >>>>>>>>>> here're the
> >>>>>>>>>>>>>>> steps users need to do to develop and run a Flink
> >>>> job:
> >>>>>>>>>>>>>>> - configure a Flink ConfluentSchemaRegistry with
> >> url,
> >>>>>>>> username,
> >>>>>>>>>> and password
> >>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset > 12pm
> >>>> yesterday"
> >>>>>>>>>> (simplified
> >>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink automatically
> >> retrieves
> >>>> all
> >>>>>>>>>> metadata of
> >>>>>>>>>>>>>>> schema, file format, etc and start the job
> >>>>>>>>>>>>>>> - users want to make the job read Kafka topic
> >>>> faster, so it
> >>>>>>>>> goes
> >>>>>>>>>> as "SELECT
> >>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/ WHERE
> >>>> offset >
> >>>>>>> 12pm
> >>>>>>>>>> yesterday"
> >>>>>>>>>>>>>>> - done and satisfied, users submit it to
> >> production
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with (k1=v1,
> >> k2=v2),
> >>>> I think
> >>>>>>>>> it's
> >>>>>>>>>> a
> >>>>>>>>>>>>>>> nice-to-have feature, but not a strategically
> >>>> critical,
> >>>>>>>>>> long-term solution,
> >>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>> 1) It may seem promising at the current stage to
> >>>> solve the
> >>>>>>>>>>>>>>> too-much-manual-work problem, but that's only
> >>>> because Flink
> >>>>>>>>>> hasn't
> >>>>>>>>>>>>>>> leveraged catalogs well and handled the 3 types
> >> of
> >>>> params
> >>>>>>>> above
> >>>>>>>>>> properly.
> >>>>>>>>>>>>>>> Once we get the params types right, the LIKE
> >> syntax
> >>>> won't
> >>>>>>> be
> >>>>>>>>> that
> >>>>>>>>>>>>>>> important, and will be just an easier way to
> >> create
> >>>> tables
> >>>>>>>>>> without retyping
> >>>>>>>>>>>>>>> long fields like username and pwd.
> >>>>>>>>>>>>>>> 2) Note that only some rare type of catalog can
> >>>> store k-v
> >>>>>>>>>> property pair, so
> >>>>>>>>>>>>>>> table created this way often cannot be
> >> persisted. In
> >>>> the
> >>>>>>>>>> foreseeable
> >>>>>>>>>>>>>>> future, such catalog will only be HiveCatalog,
> >> and
> >>>> not
> >>>>>>>> everyone
> >>>>>>>>>> has a Hive
> >>>>>>>>>>>>>>> metastore. To be honest, without persistence,
> >>>> recreating
> >>>>>>>> tables
> >>>>>>>>>> every time
> >>>>>>>>>>>>>>> this way is still a lot of keyboard typing.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>> Bowen
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <
> >>>>>>> ykt836@gmail.com
> >>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> If a specific connector want to have such
> >>>> parameter and
> >>>>>>>> read
> >>>>>>>>>> if out of
> >>>>>>>>>>>>>>>> configuration, then that's fine.
> >>>>>>>>>>>>>>>> If we are talking about a configuration for all
> >>>> kinds of
> >>>>>>>>>> sources, I would
> >>>>>>>>>>>>>>>> be super careful about that.
> >>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases, but it
> >>>> will also
> >>>>>>>> make
> >>>>>>>>>> the left 20%
> >>>>>>>>>>>>>>>> feels weird.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>> Kurt
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <
> >>>>>>> imjark@gmail.com
> >>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Kurt,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> #3 Regarding to global offset:
> >>>>>>>>>>>>>>>>> I'm not saying to use the global
> >> configuration to
> >>>>>>>> override
> >>>>>>>>>> connector
> >>>>>>>>>>>>>>>>> properties by the planner.
> >>>>>>>>>>>>>>>>> But the connector should take this
> >> configuration
> >>>> and
> >>>>>>>>>> translate into their
> >>>>>>>>>>>>>>>>> client API.
> >>>>>>>>>>>>>>>>> AFAIK, almost all the message queues support
> >>>> eariliest
> >>>>>>>> and
> >>>>>>>>>> latest and a
> >>>>>>>>>>>>>>>>> timestamp value as start point.
> >>>>>>>>>>>>>>>>> So we can support 3 options for this
> >>>> configuration:
> >>>>>>>>>> "eariliest", "latest"
> >>>>>>>>>>>>>>>>> and a timestamp string value.
> >>>>>>>>>>>>>>>>> Of course, this can't solve 100% cases, but I
> >>>> guess can
> >>>>>>>>>> sovle 80% or 90%
> >>>>>>>>>>>>>>>>> cases.
> >>>>>>>>>>>>>>>>> And the remaining cases can be resolved by
> >> LIKE
> >>>> syntax
> >>>>>>>>> which
> >>>>>>>>>> I guess is
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>> very common cases.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>> Jark
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt Young <
> >>>>>>>> ykt836@gmail.com
> >>>>>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Good to have such lovely discussions. I
> >> also
> >>>> want to
> >>>>>>>>> share
> >>>>>>>>>> some of my
> >>>>>>>>>>>>>>>>>> opinions.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I also
> >> think
> >>>> ignore
> >>>>>>>>>> invalid hints would
> >>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> dangerous, maybe
> >>>>>>>>>>>>>>>>>> the simplest solution is just throw an
> >>>> exception.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> #2 Regarding to property replacement: I
> >> don't
> >>>> think
> >>>>>>> we
> >>>>>>>>>> should
> >>>>>>>>>>>>>>>> constraint
> >>>>>>>>>>>>>>>>>> ourself to
> >>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
> >> forbidden
> >>>> it
> >>>>>>>>> modifying
> >>>>>>>>>> any
> >>>>>>>>>>>>>>>> properties
> >>>>>>>>>>>>>>>>>> which can effect
> >>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is one of
> >> the
> >>>> table
> >>>>>>>>> hints,
> >>>>>>>>>> and a
> >>>>>>>>>>>>>>>> powerful
> >>>>>>>>>>>>>>>>>> one. It can
> >>>>>>>>>>>>>>>>>> modify properties located in DDL's WITH
> >> block.
> >>>> But I
> >>>>>>>> also
> >>>>>>>>>> see the harm
> >>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> if we make it
> >>>>>>>>>>>>>>>>>> too flexible like change the kafka topic
> >> name
> >>>> with a
> >>>>>>>>> hint.
> >>>>>>>>>> Such use
> >>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> not common and
> >>>>>>>>>>>>>>>>>> sounds very dangerous to me. I would
> >> propose
> >>>> we have
> >>>>>>> a
> >>>>>>>>> map
> >>>>>>>>>> of hintable
> >>>>>>>>>>>>>>>>>> properties for each
> >>>>>>>>>>>>>>>>>> connector, and should validate all passed
> >> in
> >>>>>>> properties
> >>>>>>>>>> are actually
> >>>>>>>>>>>>>>>>>> hintable. And combining with
> >>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
> >> exception
> >>>> once
> >>>>>>>>> received
> >>>>>>>>>> invalid
> >>>>>>>>>>>>>>>>>> property.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm not sure
> >>>> it's
> >>>>>>>>> feasible.
> >>>>>>>>>> Different
> >>>>>>>>>>>>>>>>>> connectors will have totally
> >>>>>>>>>>>>>>>>>> different properties to represent offset,
> >> some
> >>>> might
> >>>>>>> be
> >>>>>>>>>> timestamps,
> >>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>> might be string literals
> >>>>>>>>>>>>>>>>>> like "earliest", and others might be just
> >>>> integers.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>> Kurt
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
> >>>>>>>>> imjark@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>> twalthr@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 <
> >>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
> >>>>>>>>>> twalthr@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
> >>>>>>>>> <
> >>>>>>>>>> imjark@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 <
> >>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>
> >
> >
>
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <da...@apache.org>.
Thanks Kurt for the suggestion ~

In my opinion:
- There is no need for TableFormatFactory#supportedHintOptions because all
the format options can be configured dynamically, they have no security
issues
- Dynamic table options is not an optimization, it is more like an
execution behavior from my side

Kurt Young <yk...@gmail.com> 于2020年3月26日周四 下午4:47写道:

> Hi Danny,
>
> Thanks for the updates. I have 2 comments regarding to latest document:
>
> 1) I think we also need `*supportedHintOptions*` for
> `*TableFormatFactory*`
> 2) IMO "dynamic-table-options.enabled" should belong to `
> *OptimizerConfigOptions*`
>
> Best,
> Kurt
>
>
> On Thu, Mar 26, 2020 at 4:40 PM Timo Walther <tw...@apache.org> wrote:
>
> > Thanks for the update Danny. +1 for this proposal.
> >
> > Regards,
> > Timo
> >
> > On 26.03.20 04:51, Danny Chan wrote:
> > > Thanks everyone who engaged in this discussion ~
> > >
> > > Our goal is "Supports Dynamic Table Options for Flink SQL". After an
> > > offline discussion with Kurt, Timo and Dawid, we have made the final
> > > conclusion, here is the summary:
> > >
> > >
> > >     - Use comment style syntax to specify the dynamic table options:
> "/*+
> > >     *OPTIONS*(k1='v1', k2='v2') */"
> > >     - Have constraint on the options keys: the options that may bring
> in
> > >     security problems should not be allowed, i.e. Kafka connector
> > zookeeper
> > >     endpoint URL and topic name
> > >     - Use white-list to control the allowed options for each connector,
> > >     which is more safe for future extention
> > >     - We allow to enable/disable this feature globally
> > >     - Implement based on the current code base first, and when FLIP-95
> is
> > >     checked in, implement this feature based on new interface
> > >
> > > Any suggestions are appreciated ~
> > >
> > > [1]
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
> > >
> > > Best,
> > > Danny Chan
> > >
> > > Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:
> > >
> > >> Hi everyone,
> > >>
> > >> Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid
> it
> > >> doesn't solve the problems but increases some development and learning
> > >> burdens.
> > >>
> > >> # increase development and learning burden
> > >>
> > >> According to the discussion so far, we want to support overriding a
> > subset
> > >> of options in hints which doesn't affect semantics.
> > >> With the `supportedHintOptions`, it's up to the connector developers
> to
> > >> decide which options will not affect semantics, and to be hint
> options.
> > >> However, the question is how to distinguish whether an option will
> > *affect
> > >> semantics*? What happens if an option will affect semantics but
> > provided as
> > >> hint options?
> > >>  From my point of view, it's not easy to distinguish. For example, the
> > >> "format.ignore-parse-error" can be a very useful dynamic option but
> that
> > >> will affect semantic, because the result is different (null vs
> > exception).
> > >> Another example, the "connector.lookup.cache.*" options are also very
> > >> useful to tune jobs, however, it will also affect the job results. I
> can
> > >> come up many more useful options but may affect semantics.
> > >>
> > >> I can see that the community will under endless discussion around "can
> > this
> > >> option to be a hint option?",  "wether this option will affect
> > semantics?".
> > >> You can also find that we already have different opinions on
> > >> "ignore-parse-error". Those discussion is a waste of time! That's not
> > what
> > >> users want!
> > >> The problem is user need this, this, this options and HOW to expose
> > them?
> > >> We should focus on that.
> > >>
> > >> Then there could be two endings in the future:
> > >> 1) compromise on the usability, we drop the rule that hints don't
> affect
> > >> semantics, allow all the useful options in the hints list.
> > >> 2) stick on the rule, users will find this is a stumbling feature
> which
> > >> doesn't solve their problems.
> > >>      And they will be surprised why this option can't be set, but the
> > other
> > >> could. *semantic* is hard to be understood by users.
> > >>
> > >> # doesn't solve the problems
> > >>
> > >> I think the purpose of this FLIP is to allow users to quickly override
> > some
> > >> connectors' properties to tune their jobs.
> > >> However, `supportedHintOptions` is off track. It only allows a subset
> > >> options and for the users it's not *clear* which subset is allowed.
> > >>
> > >> Besides, I'm not sure `supportedHintOptions` can work well for all
> > cases.
> > >> How could you support kafka properties (`connector.properties.*`) as
> > hint
> > >> options? Some kafka properties may affect semantics
> (bootstrap.servers),
> > >> some may not (max.poll.records). Besides, I think it's not possible to
> > list
> > >> all the possible kafka properties [1].
> > >>
> > >> In summary, IMO, `supportedHintOptions`
> > >> (1) it increase the complexity to develop a connector
> > >> (2) it confuses users which options can be used in hint, which are
> not,
> > >> they have to check the docs again and again.
> > >> (3) it doesn't solve the problems which we want to solve by this FLIP.
> > >>
> > >> I think we should avoid introducing some partial solutions. Otherwise,
> > we
> > >> will be stuck in a loop that introduce new API -> deprecate API ->
> > >> introduce new API....
> > >>
> > >> I personally in favor of an explicit WITH syntax after the table as a
> > part
> > >> of the query which is mentioned by Kurt before, e.g. SELECT * from T
> > >> WITH('key' = 'value') .
> > >> It allows users to dynamically set options which can affect semantics.
> > It
> > >> will be very flexible to solve users' problems so far.
> > >>
> > >> Best,
> > >> Jark
> > >>
> > >> [1]: https://kafka.apache.org/documentation/#consumerconfigs
> > >>
> > >> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com>
> wrote:
> > >>
> > >>> My POC is here for the hints options merge [1].
> > >>>
> > >>> Personally, I have no strong objections for splitting hints with the
> > >>> CatalogTable, the only cons is a more complex implementation but the
> > >>> concept is more clear, and I have updated the WIKI.
> > >>>
> > >>> I think it would be nice if we can support the format “ignore-parse
> > >> error”
> > >>> option key, the CSV source already has a key [2] and we can use that
> in
> > >> the
> > >>> supportedHIntOptions, for the common CSV and JSON formats, we cal
> also
> > >> give
> > >>> a support. This is the only kind of key in formats that “do not
> change
> > >> the
> > >>> semantics” (somehow), what do you think about this ~
> > >>>
> > >>> [1]
> > >>>
> > >>
> >
> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
> > >>> [2]
> > >>>
> > >>
> >
> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
> > >>>
> > >>> Best,
> > >>> Danny Chan
> > >>> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
> > >>>> Hi everyone,
> > >>>>
> > >>>> +1 to Kurt's suggestion. Let's just have it in source and sink
> > >> factories
> > >>>> for now. We can still move this method up in the future. Currently,
> I
> > >>>> don't see a need for catalogs or formats. Because how would you
> target
> > >> a
> > >>>> format in the query?
> > >>>>
> > >>>> @Danny: Can you send a link to your PoC? I'm very skeptical about
> > >>>> creating a new CatalogTable in planner. Actually CatalogTable should
> > be
> > >>>> immutable between Catalog and Factory. Because a catalog can return
> > its
> > >>>> own factory and fully control the instantiation. Depending on the
> > >>>> implementation, that means it can be possible that the catalog has
> > >>>> encoded more information in a concrete subclass implementing the
> > >>>> interface. I vote for separating the concerns of catalog information
> > >> and
> > >>>> hints in the factory explicitly.
> > >>>>
> > >>>> Regards,
> > >>>> Timo
> > >>>>
> > >>>>
> > >>>> On 18.03.20 05:41, Jingsong Li wrote:
> > >>>>> Hi,
> > >>>>>
> > >>>>> I am thinking we can provide hints to *table* related instances.
> > >>>>> - TableFormatFactory: of cause we need hints support, there are
> many
> > >>> format
> > >>>>> options in DDL too.
> > >>>>> - catalog and module: I don't know, maybe in future we can provide
> > >> some
> > >>>>> hints for them.
> > >>>>>
> > >>>>> Best,
> > >>>>> Jingsong Lee
> > >>>>>
> > >>>>> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com>
> > >>> wrote:
> > >>>>>
> > >>>>>> Yes, I think we should move the `supportedHintOptions` from
> > >>> TableFactory
> > >>>>>> to TableSourceFactory, and we also need to add the interface to
> > >>>>>> TableSinkFactory though because sink target table may also have
> > >> hints
> > >>>>>> attached.
> > >>>>>>
> > >>>>>> Best,
> > >>>>>> Danny Chan
> > >>>>>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> > >>>>>>> Have one question for adding `supportedHintOptions` method to
> > >>>>>>> `TableFactory`. It seems
> > >>>>>>> `TableFactory` is a base factory interface for all *table module*
> > >>> related
> > >>>>>>> instances, such as
> > >>>>>>> catalog, module, format and so on. It's not created only for
> > >>> *table*. Is
> > >>>>>> it
> > >>>>>>> possible to move it
> > >>>>>>> to `TableSourceFactory`?
> > >>>>>>>
> > >>>>>>> Best,
> > >>>>>>> Kurt
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
> > >> yuzhao.cyz@gmail.com>
> > >>>>>> wrote:
> > >>>>>>>
> > >>>>>>>> Thanks Timo ~
> > >>>>>>>>
> > >>>>>>>> For the naming itself, I also think the PROPERTIES is not that
> > >>>>>> concise, so
> > >>>>>>>> +1 for OPTIONS (I had thought about that, but there are many
> > >>> codes in
> > >>>>>>>> current Flink called it properties, i.e. the
> > >>> DescriptorProperties,
> > >>>>>>>> #getSupportedProperties), let’s use OPTIONS if this is our new
> > >>>>>> preference.
> > >>>>>>>>
> > >>>>>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
> > >>>>>> ConfigOption
> > >>>>>>>> can take more info. AFAIK, Spark also call their table options
> > >>> instead
> > >>>>>> of
> > >>>>>>>> properties. [1]
> > >>>>>>>>
> > >>>>>>>> In my local POC, I did create a new CatalogTable, and it works
> > >>> for
> > >>>>>> current
> > >>>>>>>> connectors well, all the DDL tables would finally yield a
> > >>> CatalogTable
> > >>>>>>>> instance and we can apply the options to that(in the
> > >>> CatalogSourceTable
> > >>>>>>>> when we generating the TableSource), the pros is that we do not
> > >>> need to
> > >>>>>>>> modify the codes of connectors itself. If we split the options
> > >>> from
> > >>>>>>>> CatalogTable, we may need to add some additional logic in each
> > >>>>>> connector
> > >>>>>>>> factories in order to merge these properties (and the logic are
> > >>> almost
> > >>>>>> the
> > >>>>>>>> same), what do you think about this?
> > >>>>>>>>
> > >>>>>>>> [1]
> > >>>>>>>>
> > >>>>>>
> > >>>
> > >>
> >
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> > >>>>>>>>
> > >>>>>>>> Best,
> > >>>>>>>> Danny Chan
> > >>>>>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
> > >>> ,写道:
> > >>>>>>>>> Hi Danny,
> > >>>>>>>>>
> > >>>>>>>>> thanks for updating the FLIP. I think your current design is
> > >>>>>> sufficient
> > >>>>>>>>> to separate hints from result-related properties.
> > >>>>>>>>>
> > >>>>>>>>> One remark to the naming itself: I would vote for calling the
> > >>> hints
> > >>>>>>>>> around table scan `OPTIONS('k'='v')`. We used the term
> > >>> "properties"
> > >>>>>> in
> > >>>>>>>>> the past but since we want to unify the Flink configuration
> > >>>>>> experience,
> > >>>>>>>>> we should use consistent naming and classes around
> > >>> `ConfigOptions`.
> > >>>>>>>>>
> > >>>>>>>>> It would be nice to use `Set<ConfigOption>
> > >>> supportedHintOptions();`
> > >>>>>> to
> > >>>>>>>>> start using config options instead of pure string properties.
> > >>> This
> > >>>>>> will
> > >>>>>>>>> also allow us to generate documentation in the future around
> > >>>>>> supported
> > >>>>>>>>> data types, ranges, etc. for options. At some point we would
> > >>> also
> > >>>>>> like
> > >>>>>>>>> to drop `DescriptorProperties` class. "Options" is also used
> > >>> in the
> > >>>>>>>>> documentation [1] and in the SQL/MED standard [2].
> > >>>>>>>>>
> > >>>>>>>>> Furthermore, I would still vote for separating CatalogTable
> > >>> and hint
> > >>>>>>>>> options. Otherwise the planner would need to create a new
> > >>>>>> CatalogTable
> > >>>>>>>>> instance which might not always be easy. We should offer them
> > >>> via:
> > >>>>>>>>>
> > >>>>>>>>>
> > >>> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > >>>>>>>>> ReadableConfig
> > >>>>>>>>>
> > >>>>>>>>> What do you think?
> > >>>>>>>>>
> > >>>>>>>>> Regards,
> > >>>>>>>>> Timo
> > >>>>>>>>>
> > >>>>>>>>> [1]
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>
> > >>
> >
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > >>>>>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
> > >>>>>>>>>> @Danny sounds good.
> > >>>>>>>>>>
> > >>>>>>>>>> Maybe it is worth listing all the classes of problems that
> > >>> you
> > >>>>>> want to
> > >>>>>>>>>> address and then look at each class and see if hints are a
> > >>> good
> > >>>>>> default
> > >>>>>>>>>> solution or a good optional way of simplifying things?
> > >>>>>>>>>> The discussion has grown a lot and it is starting to be
> > >> hard
> > >>> to
> > >>>>>>>> distinguish
> > >>>>>>>>>> the parts where everyone agrees from the parts were there
> > >> are
> > >>>>>> concerns.
> > >>>>>>>>>>
> > >>>>>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
> > >>> danny0405@apache.org>
> > >>>>>>>> wrote:
> > >>>>>>>>>>
> > >>>>>>>>>>> Thanks Stephan ~
> > >>>>>>>>>>>
> > >>>>>>>>>>> We can remove the support for properties that may change
> > >>> the
> > >>>>>>>> semantics of
> > >>>>>>>>>>> query if you think that is a trouble.
> > >>>>>>>>>>>
> > >>>>>>>>>>> How about we support the /*+ properties() */ hint only
> > >> for
> > >>> those
> > >>>>>>>> optimize
> > >>>>>>>>>>> parameters, such as the fetch size of source or something
> > >>> like
> > >>>>>> that,
> > >>>>>>>> does
> > >>>>>>>>>>> that make sense?
> > >>>>>>>>>>>
> > >>>>>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> I think Bowen has actually put it very well.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> (1) Hints that change semantics looks like trouble
> > >>> waiting to
> > >>>>>>>> happen. For
> > >>>>>>>>>>>> example Kafka offset handling should be in filters. The
> > >>> Kafka
> > >>>>>>>> source
> > >>>>>>>>>>> should
> > >>>>>>>>>>>> support predicate pushdown.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> (2) Hints should not be a workaround for current
> > >>> shortcomings.
> > >>>>>> A
> > >>>>>>>> lot of
> > >>>>>>>>>>> the
> > >>>>>>>>>>>> suggested above sounds exactly like that. Working
> > >> around
> > >>>>>>>> catalog/DDL
> > >>>>>>>>>>>> shortcomings, missing exposure of metadata (offsets),
> > >>> missing
> > >>>>>>>> predicate
> > >>>>>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as
> > >> a
> > >>> quick
> > >>>>>> fix
> > >>>>>>>> for
> > >>>>>>>>>>>> these issues, rather than fixing the root causes, will
> > >>> much
> > >>>>>> likely
> > >>>>>>>> bite
> > >>>>>>>>>>> us
> > >>>>>>>>>>>> back badly in the future.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Best,
> > >>>>>>>>>>>> Stephan
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
> > >>> ykt836@gmail.com>
> > >>>>>>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>> It seems this FLIP's name is somewhat misleading.
> > >> From
> > >>> my
> > >>>>>>>>>>> understanding,
> > >>>>>>>>>>>>> this FLIP is trying to
> > >>>>>>>>>>>>> address the dynamic parameter issue, and table hints
> > >>> is the
> > >>>>>> way
> > >>>>>>>> we wan
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>> choose. I think we should
> > >>>>>>>>>>>>> be focus on "what's the right way to solve dynamic
> > >>> property"
> > >>>>>>>> instead of
> > >>>>>>>>>>>>> discussing "whether table
> > >>>>>>>>>>>>> hints can affect query semantics".
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> For now, there are two proposed ways to achieve
> > >> dynamic
> > >>>>>> property:
> > >>>>>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with
> > >>> (xxx)
> > >>>>>>>>>>>>> 2. use custom "from t with (xxx)" syntax
> > >>>>>>>>>>>>> 3. "Borrow" the table hints to have a special
> > >>> PROPERTIES
> > >>>>>> hint.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> The first one didn't break anything, but the only
> > >>> problem i
> > >>>>>> see
> > >>>>>>>> is a
> > >>>>>>>>>>>> little
> > >>>>>>>>>>>>> more verbose than the table hint
> > >>>>>>>>>>>>> approach. I can imagine when someone using SQL CLI to
> > >>> have a
> > >>>>>> sql
> > >>>>>>>>>>>>> experience, it's quite often that
> > >>>>>>>>>>>>> he will modify the table property, some use cases i
> > >> can
> > >>>>>> think of:
> > >>>>>>>>>>>>> 1. the source contains some corrupted data, i want to
> > >>> turn
> > >>>>>> on the
> > >>>>>>>>>>>>> "ignore-error" flag for certain formats.
> > >>>>>>>>>>>>> 2. I have a kafka table and want to see some sample
> > >>> data
> > >>>>>> from the
> > >>>>>>>>>>>>> beginning, so i change the offset
> > >>>>>>>>>>>>> to "earliest", and then I want to observe the latest
> > >>> data
> > >>>>>> which
> > >>>>>>>> keeps
> > >>>>>>>>>>>>> coming in. I would write another query
> > >>>>>>>>>>>>> to select from the latest table.
> > >>>>>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly
> > >> then
> > >>> i can
> > >>>>>>>> observe
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>> data from database side.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Most of such use cases are quite ad-hoc. If every
> > >> time
> > >>> I
> > >>>>>> want to
> > >>>>>>>> have a
> > >>>>>>>>>>>>> different experience, i need to create
> > >>>>>>>>>>>>> a temporary table and then also modify my query, it
> > >>> doesn't
> > >>>>>> feel
> > >>>>>>>>>>> smooth.
> > >>>>>>>>>>>>> Embed such dynamic property into
> > >>>>>>>>>>>>> query would have better user experience.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is
> > >>> breaking
> > >>>>>> SQL
> > >>>>>>>>>>>> compliant,
> > >>>>>>>>>>>>> and for #3, it only breaks some
> > >>>>>>>>>>>>> unwritten rules, but we can have an explanation on
> > >>> that. And
> > >>>>>> I
> > >>>>>>>> really
> > >>>>>>>>>>>> doubt
> > >>>>>>>>>>>>> whether user would complain about
> > >>>>>>>>>>>>> this when they actually have flexible and good
> > >>> experience
> > >>>>>> using
> > >>>>>>>> this.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>> Kurt
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> > >>>>>> yuzhao.cyz@gmail.com
> > >>>>>>>>>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thanks Aljoscha ~
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> I agree for most of the query hints, they are
> > >>> optional as
> > >>>>>> an
> > >>>>>>>>>>> optimizer
> > >>>>>>>>>>>>>> instruction, especially for the traditional RDBMS.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> But, just like BenChao said, Flink as a computation
> > >>> engine
> > >>>>>> has
> > >>>>>>>> many
> > >>>>>>>>>>>>>> different kind of data sources, thus, dynamic
> > >>> parameters
> > >>>>>> like
> > >>>>>>>>>>>>> start_offest
> > >>>>>>>>>>>>>> can only bind to each table scope, we can not set a
> > >>> session
> > >>>>>>>> config
> > >>>>>>>>>>> like
> > >>>>>>>>>>>>>> KSQL because they are all about Kafka:
> > >>>>>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thus the most flexible way to set up these dynamic
> > >>> params
> > >>>>>> is
> > >>>>>>>> to bind
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>>> the table scope in the query when we want to
> > >> override
> > >>>>>>>> something, so
> > >>>>>>>>>>> we
> > >>>>>>>>>>>>> have
> > >>>>>>>>>>>>>> these solutions above (with pros and cons from my
> > >>> side):
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>> - Easy to add
> > >>>>>>>>>>>>>> - Parameters are part of the main query
> > >>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>> - Not SQL compliant
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */
> > >>> (from
> > >>>>>> me)
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>> - Easy to add
> > >>>>>>>>>>>>>> - SQL compliant because it is nested in the
> > >> comments
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>> - Parameters are not part of the main query
> > >>>>>>>>>>>>>> - Cryptic syntax for new users
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> The biggest problem for hints way may be the “if
> > >>> hints
> > >>>>>> must be
> > >>>>>>>>>>>> optional”,
> > >>>>>>>>>>>>>> actually we have though about 1 for a while but
> > >>> aborted
> > >>>>>>>> because it
> > >>>>>>>>>>>> breaks
> > >>>>>>>>>>>>>> the SQL standard too much. And we replace it with
> > >> 2,
> > >>>>>> because
> > >>>>>>>> the
> > >>>>>>>>>>> hints
> > >>>>>>>>>>>>>> syntax do not break SQL standard(nested in
> > >> comments).
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint
> > >>> that
> > >>>>>> allows
> > >>>>>>>>>>> override
> > >>>>>>>>>>>>>> some properties of table dynamically, it does not
> > >>> break
> > >>>>>>>> anything, at
> > >>>>>>>>>>>>> lease
> > >>>>>>>>>>>>>> for current Flink use cases.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Planner hints are optional just because they are
> > >>> naturally
> > >>>>>>>> enforcers
> > >>>>>>>>>>> of
> > >>>>>>>>>>>>>> the planner, most of them aim to instruct the
> > >>> optimizer,
> > >>>>>> but,
> > >>>>>>>> the
> > >>>>>>>>>>> table
> > >>>>>>>>>>>>>> hints is a little different, table hints can
> > >> specify
> > >>> the
> > >>>>>> table
> > >>>>>>>> meta
> > >>>>>>>>>>>> like
> > >>>>>>>>>>>>>> index column, and it is very convenient to specify
> > >>> table
> > >>>>>>>> properties.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */
> > >>> table
> > >>>>>> hint,
> > >>>>>>>> we
> > >>>>>>>>>>> can
> > >>>>>>>>>>>>>> call it table dynamic parameters.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>> Danny Chan
> > >>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> > >>>>>>>> aljoscha@apache.org>,写道:
> > >>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I don't understand this discussion. Hints, as I
> > >>>>>> understand
> > >>>>>>>> them,
> > >>>>>>>>>>>> should
> > >>>>>>>>>>>>>>> work like this:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> - hints are *optional* advice for the optimizer
> > >> to
> > >>> try
> > >>>>>> and
> > >>>>>>>> help it
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>>>> find a good execution strategy
> > >>>>>>>>>>>>>>> - hints should not change query semantics, i.e.
> > >>> they
> > >>>>>> should
> > >>>>>>>> not
> > >>>>>>>>>>>> change
> > >>>>>>>>>>>>>>> connector properties executing a query with
> > >> taking
> > >>> into
> > >>>>>>>> account the
> > >>>>>>>>>>>>>>> hints *must* produce the same result as executing
> > >>> the
> > >>>>>> query
> > >>>>>>>> without
> > >>>>>>>>>>>>>>> taking into account the hints
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>  From these simple requirements you can derive a
> > >>> solution
> > >>>>>>>> that makes
> > >>>>>>>>>>>>>>> sense. I don't have a strong preference for the
> > >>> syntax
> > >>>>>> but we
> > >>>>>>>>>>> should
> > >>>>>>>>>>>>>>> strive to be in line with prior work.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>> Aljoscha
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
> > >>>>>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I agree with Kurt that option2 is too
> > >>> complicated to
> > >>>>>> use
> > >>>>>>>> because:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> • As a Kafka topic consumer, the user must
> > >>> define both
> > >>>>>> the
> > >>>>>>>>>>> virtual
> > >>>>>>>>>>>>>> column for start offset and he must apply a special
> > >>> filter
> > >>>>>>>> predicate
> > >>>>>>>>>>>>> after
> > >>>>>>>>>>>>>> each query
> > >>>>>>>>>>>>>>>> • And for the internal implementation, the
> > >>> metadata
> > >>>>>> column
> > >>>>>>>> push
> > >>>>>>>>>>>> down
> > >>>>>>>>>>>>>> is another hard topic, each kind of message queue
> > >>> may have
> > >>>>>> its
> > >>>>>>>> offset
> > >>>>>>>>>>>>>> attribute, we need to consider the expression type
> > >>> for
> > >>>>>>>> different
> > >>>>>>>>>>> kind;
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>> source also need to recognize the constant column
> > >> as
> > >>> a
> > >>>>>> config
> > >>>>>>>>>>>>> option(which
> > >>>>>>>>>>>>>> is weird because usually what we pushed down is a
> > >>> table
> > >>>>>> column)
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> For option 1 and option3, I think there is no
> > >>>>>> difference,
> > >>>>>>>> option1
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>> also a hint syntax which is introduced in Sybase
> > >> and
> > >>>>>>>> referenced then
> > >>>>>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
> > >>>>>>>> ambitiousness.
> > >>>>>>>>>>>>> Personally
> > >>>>>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword
> > >>> for
> > >>>>>> these
> > >>>>>>>> reasons:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> • We do not break the standard SQL, the hints
> > >> are
> > >>>>>> nested
> > >>>>>>>> in SQL
> > >>>>>>>>>>>>>> comments
> > >>>>>>>>>>>>>>>> • We do not need to introduce additional WITH
> > >>> keyword
> > >>>>>>>> which may
> > >>>>>>>>>>>>> appear
> > >>>>>>>>>>>>>> in a query if we use that because a table can be
> > >>>>>> referenced in
> > >>>>>>>> all
> > >>>>>>>>>>>> kinds
> > >>>>>>>>>>>>> of
> > >>>>>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
> > >>> make
> > >>>>>> our
> > >>>>>>>> sql
> > >>>>>>>>>>> query
> > >>>>>>>>>>>>>> break too much of the SQL from standard
> > >>>>>>>>>>>>>>>> • We would have uniform syntax for hints as
> > >> query
> > >>>>>> hint, one
> > >>>>>>>>>>> syntax
> > >>>>>>>>>>>>>> fits all and more easy to use
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> And here is the reason why we choose a uniform
> > >>> Oracle
> > >>>>>>>> style query
> > >>>>>>>>>>>>>> hint syntax which is addressed by Julian Hyde when
> > >> we
> > >>>>>> design
> > >>>>>>>> the
> > >>>>>>>>>>> syntax
> > >>>>>>>>>>>>>> from the Calcite community:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for
> > >>> table
> > >>>>>> hints.
> > >>>>>>>> It
> > >>>>>>>>>>> adds a
> > >>>>>>>>>>>>>> new use of the WITH keyword that is unrelated to
> > >> the
> > >>> use of
> > >>>>>>>> WITH for
> > >>>>>>>>>>>>>> common-table expressions.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> A historical note. Microsoft SQL Server
> > >>> inherited its
> > >>>>>> hint
> > >>>>>>>> syntax
> > >>>>>>>>>>>>> from
> > >>>>>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
> > >>>>>>>> Programming”[1], page
> > >>>>>>>>>>>>> 632,
> > >>>>>>>>>>>>>> “Optimizer hints”. The book was written in 1999,
> > >> and
> > >>> covers
> > >>>>>>>> Microsoft
> > >>>>>>>>>>>> SQL
> > >>>>>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
> > >>> but the
> > >>>>>>>> syntax very
> > >>>>>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft
> > >> SQL
> > >>>>>> Server was
> > >>>>>>>>>>> forked
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>>> 1993.)
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Microsoft later added the WITH keyword to make
> > >>> it less
> > >>>>>>>> ambiguous,
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>> has now deprecated the syntax that does not use
> > >> WITH.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> They are forced to keep the syntax for
> > >> backwards
> > >>>>>>>> compatibility
> > >>>>>>>>>>> but
> > >>>>>>>>>>>>>> that doesn’t mean that we should shoulder their
> > >>> burden.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I think formatted comments are the right
> > >>> container for
> > >>>>>>>> hints
> > >>>>>>>>>>>> because
> > >>>>>>>>>>>>>> it allows us to change the hint syntax without
> > >>> changing
> > >>>>>> the SQL
> > >>>>>>>>>>> parser,
> > >>>>>>>>>>>>> and
> > >>>>>>>>>>>>>> makes clear that we are at liberty to ignore hints
> > >>>>>> entirely.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Julian
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
> > >>>>>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>> Danny Chan
> > >>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> > >>>>>> twalthr@apache.org
> > >>>>>>>>> ,写道:
> > >>>>>>>>>>>>>>>>> Hi Danny,
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> it is true that our DDL is not standard
> > >>> compliant by
> > >>>>>>>> using the
> > >>>>>>>>>>>> WITH
> > >>>>>>>>>>>>>>>>> clause. Nevertheless, we aim for not
> > >> diverging
> > >>> too
> > >>>>>> much
> > >>>>>>>> and the
> > >>>>>>>>>>>>> LIKE
> > >>>>>>>>>>>>>>>>> clause is an example of that. It will solve
> > >>> things
> > >>>>>> like
> > >>>>>>>>>>>> overwriting
> > >>>>>>>>>>>>>>>>> WATERMARKs, add additional/modifying
> > >>> properties and
> > >>>>>>>> inherit
> > >>>>>>>>>>>> schema.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3
> > >>> types
> > >>>>>>>> definition
> > >>>>>>>>>>>>>> together.
> > >>>>>>>>>>>>>>>>> We are not the first ones that try to solve
> > >>> this.
> > >>>>>> There
> > >>>>>>>> is also
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>> SQL
> > >>>>>>>>>>>>>>>>> MED standard [1] that tried to tackle this
> > >>> problem. I
> > >>>>>>>> think it
> > >>>>>>>>>>>> was
> > >>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>> considered when designing the current DDL.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
> > >>>>>> offsets. I
> > >>>>>>>> will
> > >>>>>>>>>>>> give
> > >>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>> examples and look forward to feedback here:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part
> > >>> of the
> > >>>>>>>> query
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>>>>> - Easy to add
> > >>>>>>>>>>>>>>>>> - Parameters are part of the main query
> > >>>>>>>>>>>>>>>>> - No complicated hinting syntax
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>>>>> - Not SQL compliant
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> *Option 2* Use metadata in query
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
> > >>>>>>>>>>>>> SYSTEM_METADATA('offset'))`
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset >
> > >> TIMESTAMP
> > >>>>>>>> '2012-12-12
> > >>>>>>>>>>>>>> 12:34:22'`
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>>>>> - SQL compliant in the query
> > >>>>>>>>>>>>>>>>> - Access of metadata in the DDL which is
> > >>> required
> > >>>>>> anyway
> > >>>>>>>>>>>>>>>>> - Regular pushdown rules apply
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>>>>> - Users need to add an additional comlumn in
> > >>> the DDL
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> *Option 3*: Use hints for properties
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> `
> > >>>>>>>>>>>>>>>>> SELECT *
> > >>>>>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
> > >>>>>>>>>>>>>>>>> `
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>>>>> - Easy to add
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>>>>> - Parameters are not part of the main query
> > >>>>>>>>>>>>>>>>> - Cryptic syntax for new users
> > >>>>>>>>>>>>>>>>> - Not standard compliant.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> If we go with this option, I would suggest to
> > >>> make it
> > >>>>>>>> available
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>> separate map and don't mix it with statically
> > >>> defined
> > >>>>>>>>>>> properties.
> > >>>>>>>>>>>>>> Such
> > >>>>>>>>>>>>>>>>> that the factory can decide which properties
> > >>> have the
> > >>>>>>>> right to
> > >>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>> overwritten by the hints:
> > >>>>>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
> > >>>>>>>> ReadableConfig
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>> Timo
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Currently I see 3 options as a
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
> > >>>>>>>>>>>>>>>>>> Thanks Bowen ~
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I agree we should somehow categorize our
> > >>> connector
> > >>>>>>>>>>> parameters.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> For type1, I’m already preparing a solution
> > >>> like
> > >>>>>> the
> > >>>>>>>>>>> Confluent
> > >>>>>>>>>>>>>> schema registry + Avro schema inference thing, so
> > >>> this may
> > >>>>>> not
> > >>>>>>>> be a
> > >>>>>>>>>>>>> problem
> > >>>>>>>>>>>>>> in the near future.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> For type3, I have some questions:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset >
> > >> 12pm
> > >>>>>> yesterday”
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Where does the offset column come from, a
> > >>> virtual
> > >>>>>>>> column from
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>> table schema, you said that
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> They change
> > >>>>>>>>>>>>>>>>>> almost every time a query starts and have
> > >>> nothing
> > >>>>>> to
> > >>>>>>>> do with
> > >>>>>>>>>>>>>> metadata, thus
> > >>>>>>>>>>>>>>>>>> should not be part of table definition/DDL
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> But why you can reference it in the query,
> > >>> I’m
> > >>>>>>>> confused for
> > >>>>>>>>>>>> that,
> > >>>>>>>>>>>>>> can you elaborate a little ?
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>> Danny Chan
> > >>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> > >>>>>>>> bowenli86@gmail.com
> > >>>>>>>>>>>> ,写道:
> > >>>>>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> The root cause of too much manual work is
> > >>> Flink
> > >>>>>> DDL
> > >>>>>>>> has
> > >>>>>>>>>>>> mixed 3
> > >>>>>>>>>>>>>> types of
> > >>>>>>>>>>>>>>>>>>> params together and doesn't handle each
> > >> of
> > >>> them
> > >>>>>> very
> > >>>>>>>> well.
> > >>>>>>>>>>>>> Below
> > >>>>>>>>>>>>>> are how I
> > >>>>>>>>>>>>>>>>>>> categorize them and corresponding
> > >>> solutions in my
> > >>>>>>>> mind:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
> > >>>>>> external
> > >>>>>>>>>>>>> endpoint/url,
> > >>>>>>>>>>>>>>>>>>> username/pwd, schemas, formats.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Such metadata are mostly already
> > >>> accessible in
> > >>>>>>>> external
> > >>>>>>>>>>>> system
> > >>>>>>>>>>>>>> as long as
> > >>>>>>>>>>>>>>>>>>> endpoints and credentials are provided.
> > >>> Flink can
> > >>>>>>>> get it
> > >>>>>>>>>>> thru
> > >>>>>>>>>>>>>> catalogs, but
> > >>>>>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus
> > >>> Flink
> > >>>>>> just
> > >>>>>>>> hasn't
> > >>>>>>>>>>>>> been
> > >>>>>>>>>>>>>> able to
> > >>>>>>>>>>>>>>>>>>> leverage that. So the solution should be
> > >>> building
> > >>>>>>>> more
> > >>>>>>>>>>>>> catalogs.
> > >>>>>>>>>>>>>> Such
> > >>>>>>>>>>>>>>>>>>> params should be part of a Flink table
> > >>>>>>>> DDL/definition, and
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>> overridable
> > >>>>>>>>>>>>>>>>>>> in any means.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc
> > >>> connector's
> > >>>>>>>> fetch size,
> > >>>>>>>>>>>>>> elasticsearch
> > >>>>>>>>>>>>>>>>>>> connector's bulk flush size.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Such params don't affect query results,
> > >> but
> > >>>>>> affect
> > >>>>>>>> how
> > >>>>>>>>>>>> results
> > >>>>>>>>>>>>>> are produced
> > >>>>>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) -
> > >> they
> > >>> are
> > >>>>>>>> essentially
> > >>>>>>>>>>>>>> execution and
> > >>>>>>>>>>>>>>>>>>> implementation details. They change often
> > >>> in
> > >>>>>>>> exploration or
> > >>>>>>>>>>>>>> development
> > >>>>>>>>>>>>>>>>>>> stages, but not quite frequently in
> > >>> well-defined
> > >>>>>>>>>>> long-running
> > >>>>>>>>>>>>>> pipelines.
> > >>>>>>>>>>>>>>>>>>> They should always have default values
> > >> and
> > >>> can be
> > >>>>>>>> missing
> > >>>>>>>>>>> in
> > >>>>>>>>>>>>>> query. They
> > >>>>>>>>>>>>>>>>>>> can be part of a table DDL/definition,
> > >> but
> > >>> should
> > >>>>>>>> also be
> > >>>>>>>>>>>>>> replaceable in a
> > >>>>>>>>>>>>>>>>>>> query - *this is what table "hints" in
> > >>> FLIP-113
> > >>>>>>>> should
> > >>>>>>>>>>>> cover*.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka
> > >>> connector's
> > >>>>>>>> start
> > >>>>>>>>>>>> offset.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Such params affect query results - the
> > >>> semantics.
> > >>>>>>>> They'd
> > >>>>>>>>>>>> better
> > >>>>>>>>>>>>>> be as
> > >>>>>>>>>>>>>>>>>>> filter conditions in WHERE clause that
> > >> can
> > >>> be
> > >>>>>> pushed
> > >>>>>>>> down.
> > >>>>>>>>>>>> They
> > >>>>>>>>>>>>>> change
> > >>>>>>>>>>>>>>>>>>> almost every time a query starts and have
> > >>>>>> nothing to
> > >>>>>>>> do
> > >>>>>>>>>>> with
> > >>>>>>>>>>>>>> metadata, thus
> > >>>>>>>>>>>>>>>>>>> should not be part of table
> > >>> definition/DDL, nor
> > >>>>>> be
> > >>>>>>>>>>> persisted
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>>> catalogs.
> > >>>>>>>>>>>>>>>>>>> If they will, users should create views
> > >> to
> > >>> keep
> > >>>>>> such
> > >>>>>>>> params
> > >>>>>>>>>>>>>> around (note
> > >>>>>>>>>>>>>>>>>>> this is different from variable
> > >>> substitution).
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we
> > >>> get these
> > >>>>>>>> params
> > >>>>>>>>>>>> right,
> > >>>>>>>>>>>>>> here're the
> > >>>>>>>>>>>>>>>>>>> steps users need to do to develop and run
> > >>> a Flink
> > >>>>>>>> job:
> > >>>>>>>>>>>>>>>>>>> - configure a Flink
> > >>> ConfluentSchemaRegistry with
> > >>>>>> url,
> > >>>>>>>>>>>> username,
> > >>>>>>>>>>>>>> and password
> > >>>>>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset
> > >>>> 12pm
> > >>>>>>>> yesterday"
> > >>>>>>>>>>>>>> (simplified
> > >>>>>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink
> > >> automatically
> > >>>>>> retrieves
> > >>>>>>>> all
> > >>>>>>>>>>>>>> metadata of
> > >>>>>>>>>>>>>>>>>>> schema, file format, etc and start the
> > >> job
> > >>>>>>>>>>>>>>>>>>> - users want to make the job read Kafka
> > >>> topic
> > >>>>>>>> faster, so it
> > >>>>>>>>>>>>> goes
> > >>>>>>>>>>>>>> as "SELECT
> > >>>>>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/
> > >>> WHERE
> > >>>>>>>> offset >
> > >>>>>>>>>>> 12pm
> > >>>>>>>>>>>>>> yesterday"
> > >>>>>>>>>>>>>>>>>>> - done and satisfied, users submit it to
> > >>>>>> production
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with
> > >> (k1=v1,
> > >>>>>> k2=v2),
> > >>>>>>>> I think
> > >>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>> nice-to-have feature, but not a
> > >>> strategically
> > >>>>>>>> critical,
> > >>>>>>>>>>>>>> long-term solution,
> > >>>>>>>>>>>>>>>>>>> because
> > >>>>>>>>>>>>>>>>>>> 1) It may seem promising at the current
> > >>> stage to
> > >>>>>>>> solve the
> > >>>>>>>>>>>>>>>>>>> too-much-manual-work problem, but that's
> > >>> only
> > >>>>>>>> because Flink
> > >>>>>>>>>>>>>> hasn't
> > >>>>>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3
> > >>> types
> > >>>>>> of
> > >>>>>>>> params
> > >>>>>>>>>>>> above
> > >>>>>>>>>>>>>> properly.
> > >>>>>>>>>>>>>>>>>>> Once we get the params types right, the
> > >>> LIKE
> > >>>>>> syntax
> > >>>>>>>> won't
> > >>>>>>>>>>> be
> > >>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> important, and will be just an easier way
> > >>> to
> > >>>>>> create
> > >>>>>>>> tables
> > >>>>>>>>>>>>>> without retyping
> > >>>>>>>>>>>>>>>>>>> long fields like username and pwd.
> > >>>>>>>>>>>>>>>>>>> 2) Note that only some rare type of
> > >>> catalog can
> > >>>>>>>> store k-v
> > >>>>>>>>>>>>>> property pair, so
> > >>>>>>>>>>>>>>>>>>> table created this way often cannot be
> > >>>>>> persisted. In
> > >>>>>>>> the
> > >>>>>>>>>>>>>> foreseeable
> > >>>>>>>>>>>>>>>>>>> future, such catalog will only be
> > >>> HiveCatalog,
> > >>>>>> and
> > >>>>>>>> not
> > >>>>>>>>>>>> everyone
> > >>>>>>>>>>>>>> has a Hive
> > >>>>>>>>>>>>>>>>>>> metastore. To be honest, without
> > >>> persistence,
> > >>>>>>>> recreating
> > >>>>>>>>>>>> tables
> > >>>>>>>>>>>>>> every time
> > >>>>>>>>>>>>>>>>>>> this way is still a lot of keyboard
> > >> typing.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>> Bowen
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt
> > >> Young
> > >>> <
> > >>>>>>>>>>> ykt836@gmail.com
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> If a specific connector want to have
> > >> such
> > >>>>>>>> parameter and
> > >>>>>>>>>>>> read
> > >>>>>>>>>>>>>> if out of
> > >>>>>>>>>>>>>>>>>>>> configuration, then that's fine.
> > >>>>>>>>>>>>>>>>>>>> If we are talking about a configuration
> > >>> for all
> > >>>>>>>> kinds of
> > >>>>>>>>>>>>>> sources, I would
> > >>>>>>>>>>>>>>>>>>>> be super careful about that.
> > >>>>>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases,
> > >>> but it
> > >>>>>>>> will also
> > >>>>>>>>>>>> make
> > >>>>>>>>>>>>>> the left 20%
> > >>>>>>>>>>>>>>>>>>>> feels weird.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>> Kurt
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark
> > >> Wu
> > >>> <
> > >>>>>>>>>>> imjark@gmail.com
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Hi Kurt,
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
> > >>>>>>>>>>>>>>>>>>>>> I'm not saying to use the global
> > >>>>>> configuration to
> > >>>>>>>>>>>> override
> > >>>>>>>>>>>>>> connector
> > >>>>>>>>>>>>>>>>>>>>> properties by the planner.
> > >>>>>>>>>>>>>>>>>>>>> But the connector should take this
> > >>>>>> configuration
> > >>>>>>>> and
> > >>>>>>>>>>>>>> translate into their
> > >>>>>>>>>>>>>>>>>>>>> client API.
> > >>>>>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues
> > >>> support
> > >>>>>>>> eariliest
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>> latest and a
> > >>>>>>>>>>>>>>>>>>>>> timestamp value as start point.
> > >>>>>>>>>>>>>>>>>>>>> So we can support 3 options for this
> > >>>>>>>> configuration:
> > >>>>>>>>>>>>>> "eariliest", "latest"
> > >>>>>>>>>>>>>>>>>>>>> and a timestamp string value.
> > >>>>>>>>>>>>>>>>>>>>> Of course, this can't solve 100%
> > >>> cases, but I
> > >>>>>>>> guess can
> > >>>>>>>>>>>>>> sovle 80% or 90%
> > >>>>>>>>>>>>>>>>>>>>> cases.
> > >>>>>>>>>>>>>>>>>>>>> And the remaining cases can be
> > >>> resolved by
> > >>>>>> LIKE
> > >>>>>>>> syntax
> > >>>>>>>>>>>>> which
> > >>>>>>>>>>>>>> I guess is
> > >>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>> very common cases.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>> Jark
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt
> > >>> Young <
> > >>>>>>>>>>>> ykt836@gmail.com
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Good to have such lovely
> > >>> discussions. I
> > >>>>>> also
> > >>>>>>>> want to
> > >>>>>>>>>>>>> share
> > >>>>>>>>>>>>>> some of my
> > >>>>>>>>>>>>>>>>>>>>>> opinions.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I
> > >>> also
> > >>>>>> think
> > >>>>>>>> ignore
> > >>>>>>>>>>>>>> invalid hints would
> > >>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>> dangerous, maybe
> > >>>>>>>>>>>>>>>>>>>>>> the simplest solution is just throw
> > >>> an
> > >>>>>>>> exception.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> #2 Regarding to property
> > >>> replacement: I
> > >>>>>> don't
> > >>>>>>>> think
> > >>>>>>>>>>> we
> > >>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>> constraint
> > >>>>>>>>>>>>>>>>>>>>>> ourself to
> > >>>>>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
> > >>>>>> forbidden
> > >>>>>>>> it
> > >>>>>>>>>>>>> modifying
> > >>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>> properties
> > >>>>>>>>>>>>>>>>>>>>>> which can effect
> > >>>>>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is
> > >>> one of
> > >>>>>> the
> > >>>>>>>> table
> > >>>>>>>>>>>>> hints,
> > >>>>>>>>>>>>>> and a
> > >>>>>>>>>>>>>>>>>>>> powerful
> > >>>>>>>>>>>>>>>>>>>>>> one. It can
> > >>>>>>>>>>>>>>>>>>>>>> modify properties located in DDL's
> > >>> WITH
> > >>>>>> block.
> > >>>>>>>> But I
> > >>>>>>>>>>>> also
> > >>>>>>>>>>>>>> see the harm
> > >>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>> if we make it
> > >>>>>>>>>>>>>>>>>>>>>> too flexible like change the kafka
> > >>> topic
> > >>>>>> name
> > >>>>>>>> with a
> > >>>>>>>>>>>>> hint.
> > >>>>>>>>>>>>>> Such use
> > >>>>>>>>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>> not common and
> > >>>>>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I
> > >> would
> > >>>>>> propose
> > >>>>>>>> we have
> > >>>>>>>>>>> a
> > >>>>>>>>>>>>> map
> > >>>>>>>>>>>>>> of hintable
> > >>>>>>>>>>>>>>>>>>>>>> properties for each
> > >>>>>>>>>>>>>>>>>>>>>> connector, and should validate all
> > >>> passed
> > >>>>>> in
> > >>>>>>>>>>> properties
> > >>>>>>>>>>>>>> are actually
> > >>>>>>>>>>>>>>>>>>>>>> hintable. And combining with
> > >>>>>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
> > >>>>>> exception
> > >>>>>>>> once
> > >>>>>>>>>>>>> received
> > >>>>>>>>>>>>>> invalid
> > >>>>>>>>>>>>>>>>>>>>>> property.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm
> > >>> not sure
> > >>>>>>>> it's
> > >>>>>>>>>>>>> feasible.
> > >>>>>>>>>>>>>> Different
> > >>>>>>>>>>>>>>>>>>>>>> connectors will have totally
> > >>>>>>>>>>>>>>>>>>>>>> different properties to represent
> > >>> offset,
> > >>>>>> some
> > >>>>>>>> might
> > >>>>>>>>>>> be
> > >>>>>>>>>>>>>> timestamps,
> > >>>>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>> might be string literals
> > >>>>>>>>>>>>>>>>>>>>>> like "earliest", and others might
> > >> be
> > >>> just
> > >>>>>>>> integers.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>>> Kurt
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM
> > >>> Jark Wu <
> > >>>>>>>>>>>>> imjark@gmail.com>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 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 <
> > >>>>>>>>>>>>>> twalthr@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 <
> > >>>>>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
> > >>>>>>>>>>>>>> twalthr@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
> > >>>>>>>>>>>>> <
> > >>>>>>>>>>>>>> imjark@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 <
> > >>>>>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >
> >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jingsong Li <ji...@gmail.com>.
Thanks Danny for update.

+1 to "dynamic-table-options.enabled" should belong to
`*OptimizerConfigOptions*`.
Hint looks like optimizer in my opinion. Actually optimizer affect
execution, but it is optimizer, not directly related to execution.

+1 to `unsupportedHintOptions`, we already list all options in
`requiredContext` and `supportedProperties`, list again in the hints
is inconvenient.

Best,
Jingsong Lee

On Thu, Mar 26, 2020 at 5:36 PM Danny Chan <da...@apache.org> wrote:

> Thanks Jark for the feedback ~
>
> I actually have a discussion offline with Timo and we think the black-list
> options has implicit rick with the growing new table options, a black-list
> there means all the new introduced options are default to be configurable
> dynamically, if the user forget to add it into the black-list, that would
> be a risk, what do you think about this @Timo ?
>
> Jark Wu <im...@gmail.com> 于2020年3月26日周四 下午5:29写道:
>
> > Hi Danny,
> >
> > Regarding to `supportedHintOptions()` interface, I suggest to use the
> > inverted version, `unsupportedHintOptions()`.
> > Because I think the disallowed list is much smaller.
> > In addition, it's hard to list all the properties under
> > `connector.properties.*`.
> > But we know `connector.properties.bootstrap.servers` and
> > `connector.properties.zookeeper.connect` are the only security options.
> >
> > Best,
> > Jark
> >
> > On Thu, 26 Mar 2020 at 16:47, Kurt Young <yk...@gmail.com> wrote:
> >
> > > Hi Danny,
> > >
> > > Thanks for the updates. I have 2 comments regarding to latest document:
> > >
> > > 1) I think we also need `*supportedHintOptions*` for
> > > `*TableFormatFactory*`
> > > 2) IMO "dynamic-table-options.enabled" should belong to `
> > > *OptimizerConfigOptions*`
> > >
> > > Best,
> > > Kurt
> > >
> > >
> > > On Thu, Mar 26, 2020 at 4:40 PM Timo Walther <tw...@apache.org>
> wrote:
> > >
> > > > Thanks for the update Danny. +1 for this proposal.
> > > >
> > > > Regards,
> > > > Timo
> > > >
> > > > On 26.03.20 04:51, Danny Chan wrote:
> > > > > Thanks everyone who engaged in this discussion ~
> > > > >
> > > > > Our goal is "Supports Dynamic Table Options for Flink SQL". After
> an
> > > > > offline discussion with Kurt, Timo and Dawid, we have made the
> final
> > > > > conclusion, here is the summary:
> > > > >
> > > > >
> > > > >     - Use comment style syntax to specify the dynamic table
> options:
> > > "/*+
> > > > >     *OPTIONS*(k1='v1', k2='v2') */"
> > > > >     - Have constraint on the options keys: the options that may
> bring
> > > in
> > > > >     security problems should not be allowed, i.e. Kafka connector
> > > > zookeeper
> > > > >     endpoint URL and topic name
> > > > >     - Use white-list to control the allowed options for each
> > connector,
> > > > >     which is more safe for future extention
> > > > >     - We allow to enable/disable this feature globally
> > > > >     - Implement based on the current code base first, and when
> > FLIP-95
> > > is
> > > > >     checked in, implement this feature based on new interface
> > > > >
> > > > > Any suggestions are appreciated ~
> > > > >
> > > > > [1]
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
> > > > >
> > > > > Best,
> > > > > Danny Chan
> > > > >
> > > > > Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:
> > > > >
> > > > >> Hi everyone,
> > > > >>
> > > > >> Sorry, but I'm not sure about the `supportedHintOptions`. I'm
> afraid
> > > it
> > > > >> doesn't solve the problems but increases some development and
> > learning
> > > > >> burdens.
> > > > >>
> > > > >> # increase development and learning burden
> > > > >>
> > > > >> According to the discussion so far, we want to support overriding
> a
> > > > subset
> > > > >> of options in hints which doesn't affect semantics.
> > > > >> With the `supportedHintOptions`, it's up to the connector
> developers
> > > to
> > > > >> decide which options will not affect semantics, and to be hint
> > > options.
> > > > >> However, the question is how to distinguish whether an option will
> > > > *affect
> > > > >> semantics*? What happens if an option will affect semantics but
> > > > provided as
> > > > >> hint options?
> > > > >>  From my point of view, it's not easy to distinguish. For example,
> > the
> > > > >> "format.ignore-parse-error" can be a very useful dynamic option
> but
> > > that
> > > > >> will affect semantic, because the result is different (null vs
> > > > exception).
> > > > >> Another example, the "connector.lookup.cache.*" options are also
> > very
> > > > >> useful to tune jobs, however, it will also affect the job
> results. I
> > > can
> > > > >> come up many more useful options but may affect semantics.
> > > > >>
> > > > >> I can see that the community will under endless discussion around
> > "can
> > > > this
> > > > >> option to be a hint option?",  "wether this option will affect
> > > > semantics?".
> > > > >> You can also find that we already have different opinions on
> > > > >> "ignore-parse-error". Those discussion is a waste of time! That's
> > not
> > > > what
> > > > >> users want!
> > > > >> The problem is user need this, this, this options and HOW to
> expose
> > > > them?
> > > > >> We should focus on that.
> > > > >>
> > > > >> Then there could be two endings in the future:
> > > > >> 1) compromise on the usability, we drop the rule that hints don't
> > > affect
> > > > >> semantics, allow all the useful options in the hints list.
> > > > >> 2) stick on the rule, users will find this is a stumbling feature
> > > which
> > > > >> doesn't solve their problems.
> > > > >>      And they will be surprised why this option can't be set, but
> > the
> > > > other
> > > > >> could. *semantic* is hard to be understood by users.
> > > > >>
> > > > >> # doesn't solve the problems
> > > > >>
> > > > >> I think the purpose of this FLIP is to allow users to quickly
> > override
> > > > some
> > > > >> connectors' properties to tune their jobs.
> > > > >> However, `supportedHintOptions` is off track. It only allows a
> > subset
> > > > >> options and for the users it's not *clear* which subset is
> allowed.
> > > > >>
> > > > >> Besides, I'm not sure `supportedHintOptions` can work well for all
> > > > cases.
> > > > >> How could you support kafka properties (`connector.properties.*`)
> as
> > > > hint
> > > > >> options? Some kafka properties may affect semantics
> > > (bootstrap.servers),
> > > > >> some may not (max.poll.records). Besides, I think it's not
> possible
> > to
> > > > list
> > > > >> all the possible kafka properties [1].
> > > > >>
> > > > >> In summary, IMO, `supportedHintOptions`
> > > > >> (1) it increase the complexity to develop a connector
> > > > >> (2) it confuses users which options can be used in hint, which are
> > > not,
> > > > >> they have to check the docs again and again.
> > > > >> (3) it doesn't solve the problems which we want to solve by this
> > FLIP.
> > > > >>
> > > > >> I think we should avoid introducing some partial solutions.
> > Otherwise,
> > > > we
> > > > >> will be stuck in a loop that introduce new API -> deprecate API ->
> > > > >> introduce new API....
> > > > >>
> > > > >> I personally in favor of an explicit WITH syntax after the table
> as
> > a
> > > > part
> > > > >> of the query which is mentioned by Kurt before, e.g. SELECT *
> from T
> > > > >> WITH('key' = 'value') .
> > > > >> It allows users to dynamically set options which can affect
> > semantics.
> > > > It
> > > > >> will be very flexible to solve users' problems so far.
> > > > >>
> > > > >> Best,
> > > > >> Jark
> > > > >>
> > > > >> [1]: https://kafka.apache.org/documentation/#consumerconfigs
> > > > >>
> > > > >> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com>
> > > wrote:
> > > > >>
> > > > >>> My POC is here for the hints options merge [1].
> > > > >>>
> > > > >>> Personally, I have no strong objections for splitting hints with
> > the
> > > > >>> CatalogTable, the only cons is a more complex implementation but
> > the
> > > > >>> concept is more clear, and I have updated the WIKI.
> > > > >>>
> > > > >>> I think it would be nice if we can support the format
> “ignore-parse
> > > > >> error”
> > > > >>> option key, the CSV source already has a key [2] and we can use
> > that
> > > in
> > > > >> the
> > > > >>> supportedHIntOptions, for the common CSV and JSON formats, we cal
> > > also
> > > > >> give
> > > > >>> a support. This is the only kind of key in formats that “do not
> > > change
> > > > >> the
> > > > >>> semantics” (somehow), what do you think about this ~
> > > > >>>
> > > > >>> [1]
> > > > >>>
> > > > >>
> > > >
> > >
> >
> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
> > > > >>> [2]
> > > > >>>
> > > > >>
> > > >
> > >
> >
> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
> > > > >>>
> > > > >>> Best,
> > > > >>> Danny Chan
> > > > >>> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
> > > > >>>> Hi everyone,
> > > > >>>>
> > > > >>>> +1 to Kurt's suggestion. Let's just have it in source and sink
> > > > >> factories
> > > > >>>> for now. We can still move this method up in the future.
> > Currently,
> > > I
> > > > >>>> don't see a need for catalogs or formats. Because how would you
> > > target
> > > > >> a
> > > > >>>> format in the query?
> > > > >>>>
> > > > >>>> @Danny: Can you send a link to your PoC? I'm very skeptical
> about
> > > > >>>> creating a new CatalogTable in planner. Actually CatalogTable
> > should
> > > > be
> > > > >>>> immutable between Catalog and Factory. Because a catalog can
> > return
> > > > its
> > > > >>>> own factory and fully control the instantiation. Depending on
> the
> > > > >>>> implementation, that means it can be possible that the catalog
> has
> > > > >>>> encoded more information in a concrete subclass implementing the
> > > > >>>> interface. I vote for separating the concerns of catalog
> > information
> > > > >> and
> > > > >>>> hints in the factory explicitly.
> > > > >>>>
> > > > >>>> Regards,
> > > > >>>> Timo
> > > > >>>>
> > > > >>>>
> > > > >>>> On 18.03.20 05:41, Jingsong Li wrote:
> > > > >>>>> Hi,
> > > > >>>>>
> > > > >>>>> I am thinking we can provide hints to *table* related
> instances.
> > > > >>>>> - TableFormatFactory: of cause we need hints support, there are
> > > many
> > > > >>> format
> > > > >>>>> options in DDL too.
> > > > >>>>> - catalog and module: I don't know, maybe in future we can
> > provide
> > > > >> some
> > > > >>>>> hints for them.
> > > > >>>>>
> > > > >>>>> Best,
> > > > >>>>> Jingsong Lee
> > > > >>>>>
> > > > >>>>> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <
> > yuzhao.cyz@gmail.com>
> > > > >>> wrote:
> > > > >>>>>
> > > > >>>>>> Yes, I think we should move the `supportedHintOptions` from
> > > > >>> TableFactory
> > > > >>>>>> to TableSourceFactory, and we also need to add the interface
> to
> > > > >>>>>> TableSinkFactory though because sink target table may also
> have
> > > > >> hints
> > > > >>>>>> attached.
> > > > >>>>>>
> > > > >>>>>> Best,
> > > > >>>>>> Danny Chan
> > > > >>>>>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> > > > >>>>>>> Have one question for adding `supportedHintOptions` method to
> > > > >>>>>>> `TableFactory`. It seems
> > > > >>>>>>> `TableFactory` is a base factory interface for all *table
> > module*
> > > > >>> related
> > > > >>>>>>> instances, such as
> > > > >>>>>>> catalog, module, format and so on. It's not created only for
> > > > >>> *table*. Is
> > > > >>>>>> it
> > > > >>>>>>> possible to move it
> > > > >>>>>>> to `TableSourceFactory`?
> > > > >>>>>>>
> > > > >>>>>>> Best,
> > > > >>>>>>> Kurt
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
> > > > >> yuzhao.cyz@gmail.com>
> > > > >>>>>> wrote:
> > > > >>>>>>>
> > > > >>>>>>>> Thanks Timo ~
> > > > >>>>>>>>
> > > > >>>>>>>> For the naming itself, I also think the PROPERTIES is not
> that
> > > > >>>>>> concise, so
> > > > >>>>>>>> +1 for OPTIONS (I had thought about that, but there are many
> > > > >>> codes in
> > > > >>>>>>>> current Flink called it properties, i.e. the
> > > > >>> DescriptorProperties,
> > > > >>>>>>>> #getSupportedProperties), let’s use OPTIONS if this is our
> new
> > > > >>>>>> preference.
> > > > >>>>>>>>
> > > > >>>>>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
> > > > >>>>>> ConfigOption
> > > > >>>>>>>> can take more info. AFAIK, Spark also call their table
> options
> > > > >>> instead
> > > > >>>>>> of
> > > > >>>>>>>> properties. [1]
> > > > >>>>>>>>
> > > > >>>>>>>> In my local POC, I did create a new CatalogTable, and it
> works
> > > > >>> for
> > > > >>>>>> current
> > > > >>>>>>>> connectors well, all the DDL tables would finally yield a
> > > > >>> CatalogTable
> > > > >>>>>>>> instance and we can apply the options to that(in the
> > > > >>> CatalogSourceTable
> > > > >>>>>>>> when we generating the TableSource), the pros is that we do
> > not
> > > > >>> need to
> > > > >>>>>>>> modify the codes of connectors itself. If we split the
> options
> > > > >>> from
> > > > >>>>>>>> CatalogTable, we may need to add some additional logic in
> each
> > > > >>>>>> connector
> > > > >>>>>>>> factories in order to merge these properties (and the logic
> > are
> > > > >>> almost
> > > > >>>>>> the
> > > > >>>>>>>> same), what do you think about this?
> > > > >>>>>>>>
> > > > >>>>>>>> [1]
> > > > >>>>>>>>
> > > > >>>>>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> > > > >>>>>>>>
> > > > >>>>>>>> Best,
> > > > >>>>>>>> Danny Chan
> > > > >>>>>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
> > > > >>> ,写道:
> > > > >>>>>>>>> Hi Danny,
> > > > >>>>>>>>>
> > > > >>>>>>>>> thanks for updating the FLIP. I think your current design
> is
> > > > >>>>>> sufficient
> > > > >>>>>>>>> to separate hints from result-related properties.
> > > > >>>>>>>>>
> > > > >>>>>>>>> One remark to the naming itself: I would vote for calling
> the
> > > > >>> hints
> > > > >>>>>>>>> around table scan `OPTIONS('k'='v')`. We used the term
> > > > >>> "properties"
> > > > >>>>>> in
> > > > >>>>>>>>> the past but since we want to unify the Flink configuration
> > > > >>>>>> experience,
> > > > >>>>>>>>> we should use consistent naming and classes around
> > > > >>> `ConfigOptions`.
> > > > >>>>>>>>>
> > > > >>>>>>>>> It would be nice to use `Set<ConfigOption>
> > > > >>> supportedHintOptions();`
> > > > >>>>>> to
> > > > >>>>>>>>> start using config options instead of pure string
> properties.
> > > > >>> This
> > > > >>>>>> will
> > > > >>>>>>>>> also allow us to generate documentation in the future
> around
> > > > >>>>>> supported
> > > > >>>>>>>>> data types, ranges, etc. for options. At some point we
> would
> > > > >>> also
> > > > >>>>>> like
> > > > >>>>>>>>> to drop `DescriptorProperties` class. "Options" is also
> used
> > > > >>> in the
> > > > >>>>>>>>> documentation [1] and in the SQL/MED standard [2].
> > > > >>>>>>>>>
> > > > >>>>>>>>> Furthermore, I would still vote for separating CatalogTable
> > > > >>> and hint
> > > > >>>>>>>>> options. Otherwise the planner would need to create a new
> > > > >>>>>> CatalogTable
> > > > >>>>>>>>> instance which might not always be easy. We should offer
> them
> > > > >>> via:
> > > > >>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>
> > org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > > > >>>>>>>>> ReadableConfig
> > > > >>>>>>>>>
> > > > >>>>>>>>> What do you think?
> > > > >>>>>>>>>
> > > > >>>>>>>>> Regards,
> > > > >>>>>>>>> Timo
> > > > >>>>>>>>>
> > > > >>>>>>>>> [1]
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > > > >>>>>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
> > > > >>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
> > > > >>>>>>>>>> @Danny sounds good.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Maybe it is worth listing all the classes of problems that
> > > > >>> you
> > > > >>>>>> want to
> > > > >>>>>>>>>> address and then look at each class and see if hints are a
> > > > >>> good
> > > > >>>>>> default
> > > > >>>>>>>>>> solution or a good optional way of simplifying things?
> > > > >>>>>>>>>> The discussion has grown a lot and it is starting to be
> > > > >> hard
> > > > >>> to
> > > > >>>>>>>> distinguish
> > > > >>>>>>>>>> the parts where everyone agrees from the parts were there
> > > > >> are
> > > > >>>>>> concerns.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
> > > > >>> danny0405@apache.org>
> > > > >>>>>>>> wrote:
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>> Thanks Stephan ~
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> We can remove the support for properties that may change
> > > > >>> the
> > > > >>>>>>>> semantics of
> > > > >>>>>>>>>>> query if you think that is a trouble.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> How about we support the /*+ properties() */ hint only
> > > > >> for
> > > > >>> those
> > > > >>>>>>>> optimize
> > > > >>>>>>>>>>> parameters, such as the fetch size of source or something
> > > > >>> like
> > > > >>>>>> that,
> > > > >>>>>>>> does
> > > > >>>>>>>>>>> that make sense?
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>> I think Bowen has actually put it very well.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> (1) Hints that change semantics looks like trouble
> > > > >>> waiting to
> > > > >>>>>>>> happen. For
> > > > >>>>>>>>>>>> example Kafka offset handling should be in filters. The
> > > > >>> Kafka
> > > > >>>>>>>> source
> > > > >>>>>>>>>>> should
> > > > >>>>>>>>>>>> support predicate pushdown.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> (2) Hints should not be a workaround for current
> > > > >>> shortcomings.
> > > > >>>>>> A
> > > > >>>>>>>> lot of
> > > > >>>>>>>>>>> the
> > > > >>>>>>>>>>>> suggested above sounds exactly like that. Working
> > > > >> around
> > > > >>>>>>>> catalog/DDL
> > > > >>>>>>>>>>>> shortcomings, missing exposure of metadata (offsets),
> > > > >>> missing
> > > > >>>>>>>> predicate
> > > > >>>>>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as
> > > > >> a
> > > > >>> quick
> > > > >>>>>> fix
> > > > >>>>>>>> for
> > > > >>>>>>>>>>>> these issues, rather than fixing the root causes, will
> > > > >>> much
> > > > >>>>>> likely
> > > > >>>>>>>> bite
> > > > >>>>>>>>>>> us
> > > > >>>>>>>>>>>> back badly in the future.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>> Stephan
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
> > > > >>> ykt836@gmail.com>
> > > > >>>>>>>> wrote:
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>> It seems this FLIP's name is somewhat misleading.
> > > > >> From
> > > > >>> my
> > > > >>>>>>>>>>> understanding,
> > > > >>>>>>>>>>>>> this FLIP is trying to
> > > > >>>>>>>>>>>>> address the dynamic parameter issue, and table hints
> > > > >>> is the
> > > > >>>>>> way
> > > > >>>>>>>> we wan
> > > > >>>>>>>>>>> to
> > > > >>>>>>>>>>>>> choose. I think we should
> > > > >>>>>>>>>>>>> be focus on "what's the right way to solve dynamic
> > > > >>> property"
> > > > >>>>>>>> instead of
> > > > >>>>>>>>>>>>> discussing "whether table
> > > > >>>>>>>>>>>>> hints can affect query semantics".
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> For now, there are two proposed ways to achieve
> > > > >> dynamic
> > > > >>>>>> property:
> > > > >>>>>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with
> > > > >>> (xxx)
> > > > >>>>>>>>>>>>> 2. use custom "from t with (xxx)" syntax
> > > > >>>>>>>>>>>>> 3. "Borrow" the table hints to have a special
> > > > >>> PROPERTIES
> > > > >>>>>> hint.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> The first one didn't break anything, but the only
> > > > >>> problem i
> > > > >>>>>> see
> > > > >>>>>>>> is a
> > > > >>>>>>>>>>>> little
> > > > >>>>>>>>>>>>> more verbose than the table hint
> > > > >>>>>>>>>>>>> approach. I can imagine when someone using SQL CLI to
> > > > >>> have a
> > > > >>>>>> sql
> > > > >>>>>>>>>>>>> experience, it's quite often that
> > > > >>>>>>>>>>>>> he will modify the table property, some use cases i
> > > > >> can
> > > > >>>>>> think of:
> > > > >>>>>>>>>>>>> 1. the source contains some corrupted data, i want to
> > > > >>> turn
> > > > >>>>>> on the
> > > > >>>>>>>>>>>>> "ignore-error" flag for certain formats.
> > > > >>>>>>>>>>>>> 2. I have a kafka table and want to see some sample
> > > > >>> data
> > > > >>>>>> from the
> > > > >>>>>>>>>>>>> beginning, so i change the offset
> > > > >>>>>>>>>>>>> to "earliest", and then I want to observe the latest
> > > > >>> data
> > > > >>>>>> which
> > > > >>>>>>>> keeps
> > > > >>>>>>>>>>>>> coming in. I would write another query
> > > > >>>>>>>>>>>>> to select from the latest table.
> > > > >>>>>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly
> > > > >> then
> > > > >>> i can
> > > > >>>>>>>> observe
> > > > >>>>>>>>>>> the
> > > > >>>>>>>>>>>>> data from database side.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Most of such use cases are quite ad-hoc. If every
> > > > >> time
> > > > >>> I
> > > > >>>>>> want to
> > > > >>>>>>>> have a
> > > > >>>>>>>>>>>>> different experience, i need to create
> > > > >>>>>>>>>>>>> a temporary table and then also modify my query, it
> > > > >>> doesn't
> > > > >>>>>> feel
> > > > >>>>>>>>>>> smooth.
> > > > >>>>>>>>>>>>> Embed such dynamic property into
> > > > >>>>>>>>>>>>> query would have better user experience.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is
> > > > >>> breaking
> > > > >>>>>> SQL
> > > > >>>>>>>>>>>> compliant,
> > > > >>>>>>>>>>>>> and for #3, it only breaks some
> > > > >>>>>>>>>>>>> unwritten rules, but we can have an explanation on
> > > > >>> that. And
> > > > >>>>>> I
> > > > >>>>>>>> really
> > > > >>>>>>>>>>>> doubt
> > > > >>>>>>>>>>>>> whether user would complain about
> > > > >>>>>>>>>>>>> this when they actually have flexible and good
> > > > >>> experience
> > > > >>>>>> using
> > > > >>>>>>>> this.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>> Kurt
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> > > > >>>>>> yuzhao.cyz@gmail.com
> > > > >>>>>>>>>
> > > > >>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Thanks Aljoscha ~
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> I agree for most of the query hints, they are
> > > > >>> optional as
> > > > >>>>>> an
> > > > >>>>>>>>>>> optimizer
> > > > >>>>>>>>>>>>>> instruction, especially for the traditional RDBMS.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> But, just like BenChao said, Flink as a computation
> > > > >>> engine
> > > > >>>>>> has
> > > > >>>>>>>> many
> > > > >>>>>>>>>>>>>> different kind of data sources, thus, dynamic
> > > > >>> parameters
> > > > >>>>>> like
> > > > >>>>>>>>>>>>> start_offest
> > > > >>>>>>>>>>>>>> can only bind to each table scope, we can not set a
> > > > >>> session
> > > > >>>>>>>> config
> > > > >>>>>>>>>>> like
> > > > >>>>>>>>>>>>>> KSQL because they are all about Kafka:
> > > > >>>>>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Thus the most flexible way to set up these dynamic
> > > > >>> params
> > > > >>>>>> is
> > > > >>>>>>>> to bind
> > > > >>>>>>>>>>> to
> > > > >>>>>>>>>>>>>> the table scope in the query when we want to
> > > > >> override
> > > > >>>>>>>> something, so
> > > > >>>>>>>>>>> we
> > > > >>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>> these solutions above (with pros and cons from my
> > > > >>> side):
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Pros:
> > > > >>>>>>>>>>>>>> - Easy to add
> > > > >>>>>>>>>>>>>> - Parameters are part of the main query
> > > > >>>>>>>>>>>>>> Cons:
> > > > >>>>>>>>>>>>>> - Not SQL compliant
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */
> > > > >>> (from
> > > > >>>>>> me)
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Pros:
> > > > >>>>>>>>>>>>>> - Easy to add
> > > > >>>>>>>>>>>>>> - SQL compliant because it is nested in the
> > > > >> comments
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Cons:
> > > > >>>>>>>>>>>>>> - Parameters are not part of the main query
> > > > >>>>>>>>>>>>>> - Cryptic syntax for new users
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> The biggest problem for hints way may be the “if
> > > > >>> hints
> > > > >>>>>> must be
> > > > >>>>>>>>>>>> optional”,
> > > > >>>>>>>>>>>>>> actually we have though about 1 for a while but
> > > > >>> aborted
> > > > >>>>>>>> because it
> > > > >>>>>>>>>>>> breaks
> > > > >>>>>>>>>>>>>> the SQL standard too much. And we replace it with
> > > > >> 2,
> > > > >>>>>> because
> > > > >>>>>>>> the
> > > > >>>>>>>>>>> hints
> > > > >>>>>>>>>>>>>> syntax do not break SQL standard(nested in
> > > > >> comments).
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint
> > > > >>> that
> > > > >>>>>> allows
> > > > >>>>>>>>>>> override
> > > > >>>>>>>>>>>>>> some properties of table dynamically, it does not
> > > > >>> break
> > > > >>>>>>>> anything, at
> > > > >>>>>>>>>>>>> lease
> > > > >>>>>>>>>>>>>> for current Flink use cases.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Planner hints are optional just because they are
> > > > >>> naturally
> > > > >>>>>>>> enforcers
> > > > >>>>>>>>>>> of
> > > > >>>>>>>>>>>>>> the planner, most of them aim to instruct the
> > > > >>> optimizer,
> > > > >>>>>> but,
> > > > >>>>>>>> the
> > > > >>>>>>>>>>> table
> > > > >>>>>>>>>>>>>> hints is a little different, table hints can
> > > > >> specify
> > > > >>> the
> > > > >>>>>> table
> > > > >>>>>>>> meta
> > > > >>>>>>>>>>>> like
> > > > >>>>>>>>>>>>>> index column, and it is very convenient to specify
> > > > >>> table
> > > > >>>>>>>> properties.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */
> > > > >>> table
> > > > >>>>>> hint,
> > > > >>>>>>>> we
> > > > >>>>>>>>>>> can
> > > > >>>>>>>>>>>>>> call it table dynamic parameters.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>> Danny Chan
> > > > >>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> > > > >>>>>>>> aljoscha@apache.org>,写道:
> > > > >>>>>>>>>>>>>>> Hi,
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> I don't understand this discussion. Hints, as I
> > > > >>>>>> understand
> > > > >>>>>>>> them,
> > > > >>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>> work like this:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> - hints are *optional* advice for the optimizer
> > > > >> to
> > > > >>> try
> > > > >>>>>> and
> > > > >>>>>>>> help it
> > > > >>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>> find a good execution strategy
> > > > >>>>>>>>>>>>>>> - hints should not change query semantics, i.e.
> > > > >>> they
> > > > >>>>>> should
> > > > >>>>>>>> not
> > > > >>>>>>>>>>>> change
> > > > >>>>>>>>>>>>>>> connector properties executing a query with
> > > > >> taking
> > > > >>> into
> > > > >>>>>>>> account the
> > > > >>>>>>>>>>>>>>> hints *must* produce the same result as executing
> > > > >>> the
> > > > >>>>>> query
> > > > >>>>>>>> without
> > > > >>>>>>>>>>>>>>> taking into account the hints
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>  From these simple requirements you can derive a
> > > > >>> solution
> > > > >>>>>>>> that makes
> > > > >>>>>>>>>>>>>>> sense. I don't have a strong preference for the
> > > > >>> syntax
> > > > >>>>>> but we
> > > > >>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>> strive to be in line with prior work.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>> Aljoscha
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
> > > > >>>>>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> I agree with Kurt that option2 is too
> > > > >>> complicated to
> > > > >>>>>> use
> > > > >>>>>>>> because:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> • As a Kafka topic consumer, the user must
> > > > >>> define both
> > > > >>>>>> the
> > > > >>>>>>>>>>> virtual
> > > > >>>>>>>>>>>>>> column for start offset and he must apply a special
> > > > >>> filter
> > > > >>>>>>>> predicate
> > > > >>>>>>>>>>>>> after
> > > > >>>>>>>>>>>>>> each query
> > > > >>>>>>>>>>>>>>>> • And for the internal implementation, the
> > > > >>> metadata
> > > > >>>>>> column
> > > > >>>>>>>> push
> > > > >>>>>>>>>>>> down
> > > > >>>>>>>>>>>>>> is another hard topic, each kind of message queue
> > > > >>> may have
> > > > >>>>>> its
> > > > >>>>>>>> offset
> > > > >>>>>>>>>>>>>> attribute, we need to consider the expression type
> > > > >>> for
> > > > >>>>>>>> different
> > > > >>>>>>>>>>> kind;
> > > > >>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>> source also need to recognize the constant column
> > > > >> as
> > > > >>> a
> > > > >>>>>> config
> > > > >>>>>>>>>>>>> option(which
> > > > >>>>>>>>>>>>>> is weird because usually what we pushed down is a
> > > > >>> table
> > > > >>>>>> column)
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> For option 1 and option3, I think there is no
> > > > >>>>>> difference,
> > > > >>>>>>>> option1
> > > > >>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>> also a hint syntax which is introduced in Sybase
> > > > >> and
> > > > >>>>>>>> referenced then
> > > > >>>>>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
> > > > >>>>>>>> ambitiousness.
> > > > >>>>>>>>>>>>> Personally
> > > > >>>>>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword
> > > > >>> for
> > > > >>>>>> these
> > > > >>>>>>>> reasons:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> • We do not break the standard SQL, the hints
> > > > >> are
> > > > >>>>>> nested
> > > > >>>>>>>> in SQL
> > > > >>>>>>>>>>>>>> comments
> > > > >>>>>>>>>>>>>>>> • We do not need to introduce additional WITH
> > > > >>> keyword
> > > > >>>>>>>> which may
> > > > >>>>>>>>>>>>> appear
> > > > >>>>>>>>>>>>>> in a query if we use that because a table can be
> > > > >>>>>> referenced in
> > > > >>>>>>>> all
> > > > >>>>>>>>>>>> kinds
> > > > >>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
> > > > >>> make
> > > > >>>>>> our
> > > > >>>>>>>> sql
> > > > >>>>>>>>>>> query
> > > > >>>>>>>>>>>>>> break too much of the SQL from standard
> > > > >>>>>>>>>>>>>>>> • We would have uniform syntax for hints as
> > > > >> query
> > > > >>>>>> hint, one
> > > > >>>>>>>>>>> syntax
> > > > >>>>>>>>>>>>>> fits all and more easy to use
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> And here is the reason why we choose a uniform
> > > > >>> Oracle
> > > > >>>>>>>> style query
> > > > >>>>>>>>>>>>>> hint syntax which is addressed by Julian Hyde when
> > > > >> we
> > > > >>>>>> design
> > > > >>>>>>>> the
> > > > >>>>>>>>>>> syntax
> > > > >>>>>>>>>>>>>> from the Calcite community:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for
> > > > >>> table
> > > > >>>>>> hints.
> > > > >>>>>>>> It
> > > > >>>>>>>>>>> adds a
> > > > >>>>>>>>>>>>>> new use of the WITH keyword that is unrelated to
> > > > >> the
> > > > >>> use of
> > > > >>>>>>>> WITH for
> > > > >>>>>>>>>>>>>> common-table expressions.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> A historical note. Microsoft SQL Server
> > > > >>> inherited its
> > > > >>>>>> hint
> > > > >>>>>>>> syntax
> > > > >>>>>>>>>>>>> from
> > > > >>>>>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
> > > > >>>>>>>> Programming”[1], page
> > > > >>>>>>>>>>>>> 632,
> > > > >>>>>>>>>>>>>> “Optimizer hints”. The book was written in 1999,
> > > > >> and
> > > > >>> covers
> > > > >>>>>>>> Microsoft
> > > > >>>>>>>>>>>> SQL
> > > > >>>>>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
> > > > >>> but the
> > > > >>>>>>>> syntax very
> > > > >>>>>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft
> > > > >> SQL
> > > > >>>>>> Server was
> > > > >>>>>>>>>>> forked
> > > > >>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>> 1993.)
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Microsoft later added the WITH keyword to make
> > > > >>> it less
> > > > >>>>>>>> ambiguous,
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>> has now deprecated the syntax that does not use
> > > > >> WITH.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> They are forced to keep the syntax for
> > > > >> backwards
> > > > >>>>>>>> compatibility
> > > > >>>>>>>>>>> but
> > > > >>>>>>>>>>>>>> that doesn’t mean that we should shoulder their
> > > > >>> burden.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> I think formatted comments are the right
> > > > >>> container for
> > > > >>>>>>>> hints
> > > > >>>>>>>>>>>> because
> > > > >>>>>>>>>>>>>> it allows us to change the hint syntax without
> > > > >>> changing
> > > > >>>>>> the SQL
> > > > >>>>>>>>>>> parser,
> > > > >>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>> makes clear that we are at liberty to ignore hints
> > > > >>>>>> entirely.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Julian
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
> > > > >>>>>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>> Danny Chan
> > > > >>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> > > > >>>>>> twalthr@apache.org
> > > > >>>>>>>>> ,写道:
> > > > >>>>>>>>>>>>>>>>> Hi Danny,
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> it is true that our DDL is not standard
> > > > >>> compliant by
> > > > >>>>>>>> using the
> > > > >>>>>>>>>>>> WITH
> > > > >>>>>>>>>>>>>>>>> clause. Nevertheless, we aim for not
> > > > >> diverging
> > > > >>> too
> > > > >>>>>> much
> > > > >>>>>>>> and the
> > > > >>>>>>>>>>>>> LIKE
> > > > >>>>>>>>>>>>>>>>> clause is an example of that. It will solve
> > > > >>> things
> > > > >>>>>> like
> > > > >>>>>>>>>>>> overwriting
> > > > >>>>>>>>>>>>>>>>> WATERMARKs, add additional/modifying
> > > > >>> properties and
> > > > >>>>>>>> inherit
> > > > >>>>>>>>>>>> schema.
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3
> > > > >>> types
> > > > >>>>>>>> definition
> > > > >>>>>>>>>>>>>> together.
> > > > >>>>>>>>>>>>>>>>> We are not the first ones that try to solve
> > > > >>> this.
> > > > >>>>>> There
> > > > >>>>>>>> is also
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>> SQL
> > > > >>>>>>>>>>>>>>>>> MED standard [1] that tried to tackle this
> > > > >>> problem. I
> > > > >>>>>>>> think it
> > > > >>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>> considered when designing the current DDL.
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
> > > > >>>>>> offsets. I
> > > > >>>>>>>> will
> > > > >>>>>>>>>>>> give
> > > > >>>>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>> examples and look forward to feedback here:
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part
> > > > >>> of the
> > > > >>>>>>>> query
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Pros:
> > > > >>>>>>>>>>>>>>>>> - Easy to add
> > > > >>>>>>>>>>>>>>>>> - Parameters are part of the main query
> > > > >>>>>>>>>>>>>>>>> - No complicated hinting syntax
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Cons:
> > > > >>>>>>>>>>>>>>>>> - Not SQL compliant
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> *Option 2* Use metadata in query
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
> > > > >>>>>>>>>>>>> SYSTEM_METADATA('offset'))`
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset >
> > > > >> TIMESTAMP
> > > > >>>>>>>> '2012-12-12
> > > > >>>>>>>>>>>>>> 12:34:22'`
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Pros:
> > > > >>>>>>>>>>>>>>>>> - SQL compliant in the query
> > > > >>>>>>>>>>>>>>>>> - Access of metadata in the DDL which is
> > > > >>> required
> > > > >>>>>> anyway
> > > > >>>>>>>>>>>>>>>>> - Regular pushdown rules apply
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Cons:
> > > > >>>>>>>>>>>>>>>>> - Users need to add an additional comlumn in
> > > > >>> the DDL
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> *Option 3*: Use hints for properties
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> `
> > > > >>>>>>>>>>>>>>>>> SELECT *
> > > > >>>>>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > >>>>>>>>>>>>>>>>> `
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Pros:
> > > > >>>>>>>>>>>>>>>>> - Easy to add
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Cons:
> > > > >>>>>>>>>>>>>>>>> - Parameters are not part of the main query
> > > > >>>>>>>>>>>>>>>>> - Cryptic syntax for new users
> > > > >>>>>>>>>>>>>>>>> - Not standard compliant.
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> If we go with this option, I would suggest to
> > > > >>> make it
> > > > >>>>>>>> available
> > > > >>>>>>>>>>>> in
> > > > >>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>> separate map and don't mix it with statically
> > > > >>> defined
> > > > >>>>>>>>>>> properties.
> > > > >>>>>>>>>>>>>> Such
> > > > >>>>>>>>>>>>>>>>> that the factory can decide which properties
> > > > >>> have the
> > > > >>>>>>>> right to
> > > > >>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>> overwritten by the hints:
> > > > >>>>>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
> > > > >>>>>>>> ReadableConfig
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>> Timo
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> Currently I see 3 options as a
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
> > > > >>>>>>>>>>>>>>>>>> Thanks Bowen ~
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> I agree we should somehow categorize our
> > > > >>> connector
> > > > >>>>>>>>>>> parameters.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> For type1, I’m already preparing a solution
> > > > >>> like
> > > > >>>>>> the
> > > > >>>>>>>>>>> Confluent
> > > > >>>>>>>>>>>>>> schema registry + Avro schema inference thing, so
> > > > >>> this may
> > > > >>>>>> not
> > > > >>>>>>>> be a
> > > > >>>>>>>>>>>>> problem
> > > > >>>>>>>>>>>>>> in the near future.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> For type3, I have some questions:
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset >
> > > > >> 12pm
> > > > >>>>>> yesterday”
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> Where does the offset column come from, a
> > > > >>> virtual
> > > > >>>>>>>> column from
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>> table schema, you said that
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> They change
> > > > >>>>>>>>>>>>>>>>>> almost every time a query starts and have
> > > > >>> nothing
> > > > >>>>>> to
> > > > >>>>>>>> do with
> > > > >>>>>>>>>>>>>> metadata, thus
> > > > >>>>>>>>>>>>>>>>>> should not be part of table definition/DDL
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> But why you can reference it in the query,
> > > > >>> I’m
> > > > >>>>>>>> confused for
> > > > >>>>>>>>>>>> that,
> > > > >>>>>>>>>>>>>> can you elaborate a little ?
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>> Danny Chan
> > > > >>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> > > > >>>>>>>> bowenli86@gmail.com
> > > > >>>>>>>>>>>> ,写道:
> > > > >>>>>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> The root cause of too much manual work is
> > > > >>> Flink
> > > > >>>>>> DDL
> > > > >>>>>>>> has
> > > > >>>>>>>>>>>> mixed 3
> > > > >>>>>>>>>>>>>> types of
> > > > >>>>>>>>>>>>>>>>>>> params together and doesn't handle each
> > > > >> of
> > > > >>> them
> > > > >>>>>> very
> > > > >>>>>>>> well.
> > > > >>>>>>>>>>>>> Below
> > > > >>>>>>>>>>>>>> are how I
> > > > >>>>>>>>>>>>>>>>>>> categorize them and corresponding
> > > > >>> solutions in my
> > > > >>>>>>>> mind:
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
> > > > >>>>>> external
> > > > >>>>>>>>>>>>> endpoint/url,
> > > > >>>>>>>>>>>>>>>>>>> username/pwd, schemas, formats.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Such metadata are mostly already
> > > > >>> accessible in
> > > > >>>>>>>> external
> > > > >>>>>>>>>>>> system
> > > > >>>>>>>>>>>>>> as long as
> > > > >>>>>>>>>>>>>>>>>>> endpoints and credentials are provided.
> > > > >>> Flink can
> > > > >>>>>>>> get it
> > > > >>>>>>>>>>> thru
> > > > >>>>>>>>>>>>>> catalogs, but
> > > > >>>>>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus
> > > > >>> Flink
> > > > >>>>>> just
> > > > >>>>>>>> hasn't
> > > > >>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>> able to
> > > > >>>>>>>>>>>>>>>>>>> leverage that. So the solution should be
> > > > >>> building
> > > > >>>>>>>> more
> > > > >>>>>>>>>>>>> catalogs.
> > > > >>>>>>>>>>>>>> Such
> > > > >>>>>>>>>>>>>>>>>>> params should be part of a Flink table
> > > > >>>>>>>> DDL/definition, and
> > > > >>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>> overridable
> > > > >>>>>>>>>>>>>>>>>>> in any means.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc
> > > > >>> connector's
> > > > >>>>>>>> fetch size,
> > > > >>>>>>>>>>>>>> elasticsearch
> > > > >>>>>>>>>>>>>>>>>>> connector's bulk flush size.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Such params don't affect query results,
> > > > >> but
> > > > >>>>>> affect
> > > > >>>>>>>> how
> > > > >>>>>>>>>>>> results
> > > > >>>>>>>>>>>>>> are produced
> > > > >>>>>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) -
> > > > >> they
> > > > >>> are
> > > > >>>>>>>> essentially
> > > > >>>>>>>>>>>>>> execution and
> > > > >>>>>>>>>>>>>>>>>>> implementation details. They change often
> > > > >>> in
> > > > >>>>>>>> exploration or
> > > > >>>>>>>>>>>>>> development
> > > > >>>>>>>>>>>>>>>>>>> stages, but not quite frequently in
> > > > >>> well-defined
> > > > >>>>>>>>>>> long-running
> > > > >>>>>>>>>>>>>> pipelines.
> > > > >>>>>>>>>>>>>>>>>>> They should always have default values
> > > > >> and
> > > > >>> can be
> > > > >>>>>>>> missing
> > > > >>>>>>>>>>> in
> > > > >>>>>>>>>>>>>> query. They
> > > > >>>>>>>>>>>>>>>>>>> can be part of a table DDL/definition,
> > > > >> but
> > > > >>> should
> > > > >>>>>>>> also be
> > > > >>>>>>>>>>>>>> replaceable in a
> > > > >>>>>>>>>>>>>>>>>>> query - *this is what table "hints" in
> > > > >>> FLIP-113
> > > > >>>>>>>> should
> > > > >>>>>>>>>>>> cover*.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka
> > > > >>> connector's
> > > > >>>>>>>> start
> > > > >>>>>>>>>>>> offset.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Such params affect query results - the
> > > > >>> semantics.
> > > > >>>>>>>> They'd
> > > > >>>>>>>>>>>> better
> > > > >>>>>>>>>>>>>> be as
> > > > >>>>>>>>>>>>>>>>>>> filter conditions in WHERE clause that
> > > > >> can
> > > > >>> be
> > > > >>>>>> pushed
> > > > >>>>>>>> down.
> > > > >>>>>>>>>>>> They
> > > > >>>>>>>>>>>>>> change
> > > > >>>>>>>>>>>>>>>>>>> almost every time a query starts and have
> > > > >>>>>> nothing to
> > > > >>>>>>>> do
> > > > >>>>>>>>>>> with
> > > > >>>>>>>>>>>>>> metadata, thus
> > > > >>>>>>>>>>>>>>>>>>> should not be part of table
> > > > >>> definition/DDL, nor
> > > > >>>>>> be
> > > > >>>>>>>>>>> persisted
> > > > >>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>> catalogs.
> > > > >>>>>>>>>>>>>>>>>>> If they will, users should create views
> > > > >> to
> > > > >>> keep
> > > > >>>>>> such
> > > > >>>>>>>> params
> > > > >>>>>>>>>>>>>> around (note
> > > > >>>>>>>>>>>>>>>>>>> this is different from variable
> > > > >>> substitution).
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we
> > > > >>> get these
> > > > >>>>>>>> params
> > > > >>>>>>>>>>>> right,
> > > > >>>>>>>>>>>>>> here're the
> > > > >>>>>>>>>>>>>>>>>>> steps users need to do to develop and run
> > > > >>> a Flink
> > > > >>>>>>>> job:
> > > > >>>>>>>>>>>>>>>>>>> - configure a Flink
> > > > >>> ConfluentSchemaRegistry with
> > > > >>>>>> url,
> > > > >>>>>>>>>>>> username,
> > > > >>>>>>>>>>>>>> and password
> > > > >>>>>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset
> > > > >>>> 12pm
> > > > >>>>>>>> yesterday"
> > > > >>>>>>>>>>>>>> (simplified
> > > > >>>>>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink
> > > > >> automatically
> > > > >>>>>> retrieves
> > > > >>>>>>>> all
> > > > >>>>>>>>>>>>>> metadata of
> > > > >>>>>>>>>>>>>>>>>>> schema, file format, etc and start the
> > > > >> job
> > > > >>>>>>>>>>>>>>>>>>> - users want to make the job read Kafka
> > > > >>> topic
> > > > >>>>>>>> faster, so it
> > > > >>>>>>>>>>>>> goes
> > > > >>>>>>>>>>>>>> as "SELECT
> > > > >>>>>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/
> > > > >>> WHERE
> > > > >>>>>>>> offset >
> > > > >>>>>>>>>>> 12pm
> > > > >>>>>>>>>>>>>> yesterday"
> > > > >>>>>>>>>>>>>>>>>>> - done and satisfied, users submit it to
> > > > >>>>>> production
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with
> > > > >> (k1=v1,
> > > > >>>>>> k2=v2),
> > > > >>>>>>>> I think
> > > > >>>>>>>>>>>>> it's
> > > > >>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>> nice-to-have feature, but not a
> > > > >>> strategically
> > > > >>>>>>>> critical,
> > > > >>>>>>>>>>>>>> long-term solution,
> > > > >>>>>>>>>>>>>>>>>>> because
> > > > >>>>>>>>>>>>>>>>>>> 1) It may seem promising at the current
> > > > >>> stage to
> > > > >>>>>>>> solve the
> > > > >>>>>>>>>>>>>>>>>>> too-much-manual-work problem, but that's
> > > > >>> only
> > > > >>>>>>>> because Flink
> > > > >>>>>>>>>>>>>> hasn't
> > > > >>>>>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3
> > > > >>> types
> > > > >>>>>> of
> > > > >>>>>>>> params
> > > > >>>>>>>>>>>> above
> > > > >>>>>>>>>>>>>> properly.
> > > > >>>>>>>>>>>>>>>>>>> Once we get the params types right, the
> > > > >>> LIKE
> > > > >>>>>> syntax
> > > > >>>>>>>> won't
> > > > >>>>>>>>>>> be
> > > > >>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>> important, and will be just an easier way
> > > > >>> to
> > > > >>>>>> create
> > > > >>>>>>>> tables
> > > > >>>>>>>>>>>>>> without retyping
> > > > >>>>>>>>>>>>>>>>>>> long fields like username and pwd.
> > > > >>>>>>>>>>>>>>>>>>> 2) Note that only some rare type of
> > > > >>> catalog can
> > > > >>>>>>>> store k-v
> > > > >>>>>>>>>>>>>> property pair, so
> > > > >>>>>>>>>>>>>>>>>>> table created this way often cannot be
> > > > >>>>>> persisted. In
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>> foreseeable
> > > > >>>>>>>>>>>>>>>>>>> future, such catalog will only be
> > > > >>> HiveCatalog,
> > > > >>>>>> and
> > > > >>>>>>>> not
> > > > >>>>>>>>>>>> everyone
> > > > >>>>>>>>>>>>>> has a Hive
> > > > >>>>>>>>>>>>>>>>>>> metastore. To be honest, without
> > > > >>> persistence,
> > > > >>>>>>>> recreating
> > > > >>>>>>>>>>>> tables
> > > > >>>>>>>>>>>>>> every time
> > > > >>>>>>>>>>>>>>>>>>> this way is still a lot of keyboard
> > > > >> typing.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Cheers,
> > > > >>>>>>>>>>>>>>>>>>> Bowen
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt
> > > > >> Young
> > > > >>> <
> > > > >>>>>>>>>>> ykt836@gmail.com
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> If a specific connector want to have
> > > > >> such
> > > > >>>>>>>> parameter and
> > > > >>>>>>>>>>>> read
> > > > >>>>>>>>>>>>>> if out of
> > > > >>>>>>>>>>>>>>>>>>>> configuration, then that's fine.
> > > > >>>>>>>>>>>>>>>>>>>> If we are talking about a configuration
> > > > >>> for all
> > > > >>>>>>>> kinds of
> > > > >>>>>>>>>>>>>> sources, I would
> > > > >>>>>>>>>>>>>>>>>>>> be super careful about that.
> > > > >>>>>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases,
> > > > >>> but it
> > > > >>>>>>>> will also
> > > > >>>>>>>>>>>> make
> > > > >>>>>>>>>>>>>> the left 20%
> > > > >>>>>>>>>>>>>>>>>>>> feels weird.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>> Kurt
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark
> > > > >> Wu
> > > > >>> <
> > > > >>>>>>>>>>> imjark@gmail.com
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Hi Kurt,
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
> > > > >>>>>>>>>>>>>>>>>>>>> I'm not saying to use the global
> > > > >>>>>> configuration to
> > > > >>>>>>>>>>>> override
> > > > >>>>>>>>>>>>>> connector
> > > > >>>>>>>>>>>>>>>>>>>>> properties by the planner.
> > > > >>>>>>>>>>>>>>>>>>>>> But the connector should take this
> > > > >>>>>> configuration
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>>>> translate into their
> > > > >>>>>>>>>>>>>>>>>>>>> client API.
> > > > >>>>>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues
> > > > >>> support
> > > > >>>>>>>> eariliest
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>> latest and a
> > > > >>>>>>>>>>>>>>>>>>>>> timestamp value as start point.
> > > > >>>>>>>>>>>>>>>>>>>>> So we can support 3 options for this
> > > > >>>>>>>> configuration:
> > > > >>>>>>>>>>>>>> "eariliest", "latest"
> > > > >>>>>>>>>>>>>>>>>>>>> and a timestamp string value.
> > > > >>>>>>>>>>>>>>>>>>>>> Of course, this can't solve 100%
> > > > >>> cases, but I
> > > > >>>>>>>> guess can
> > > > >>>>>>>>>>>>>> sovle 80% or 90%
> > > > >>>>>>>>>>>>>>>>>>>>> cases.
> > > > >>>>>>>>>>>>>>>>>>>>> And the remaining cases can be
> > > > >>> resolved by
> > > > >>>>>> LIKE
> > > > >>>>>>>> syntax
> > > > >>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>> I guess is
> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>> very common cases.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>>> Jark
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt
> > > > >>> Young <
> > > > >>>>>>>>>>>> ykt836@gmail.com
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Good to have such lovely
> > > > >>> discussions. I
> > > > >>>>>> also
> > > > >>>>>>>> want to
> > > > >>>>>>>>>>>>> share
> > > > >>>>>>>>>>>>>> some of my
> > > > >>>>>>>>>>>>>>>>>>>>>> opinions.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I
> > > > >>> also
> > > > >>>>>> think
> > > > >>>>>>>> ignore
> > > > >>>>>>>>>>>>>> invalid hints would
> > > > >>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>> dangerous, maybe
> > > > >>>>>>>>>>>>>>>>>>>>>> the simplest solution is just throw
> > > > >>> an
> > > > >>>>>>>> exception.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> #2 Regarding to property
> > > > >>> replacement: I
> > > > >>>>>> don't
> > > > >>>>>>>> think
> > > > >>>>>>>>>>> we
> > > > >>>>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>>>>>>> constraint
> > > > >>>>>>>>>>>>>>>>>>>>>> ourself to
> > > > >>>>>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
> > > > >>>>>> forbidden
> > > > >>>>>>>> it
> > > > >>>>>>>>>>>>> modifying
> > > > >>>>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>>>> properties
> > > > >>>>>>>>>>>>>>>>>>>>>> which can effect
> > > > >>>>>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is
> > > > >>> one of
> > > > >>>>>> the
> > > > >>>>>>>> table
> > > > >>>>>>>>>>>>> hints,
> > > > >>>>>>>>>>>>>> and a
> > > > >>>>>>>>>>>>>>>>>>>> powerful
> > > > >>>>>>>>>>>>>>>>>>>>>> one. It can
> > > > >>>>>>>>>>>>>>>>>>>>>> modify properties located in DDL's
> > > > >>> WITH
> > > > >>>>>> block.
> > > > >>>>>>>> But I
> > > > >>>>>>>>>>>> also
> > > > >>>>>>>>>>>>>> see the harm
> > > > >>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>> if we make it
> > > > >>>>>>>>>>>>>>>>>>>>>> too flexible like change the kafka
> > > > >>> topic
> > > > >>>>>> name
> > > > >>>>>>>> with a
> > > > >>>>>>>>>>>>> hint.
> > > > >>>>>>>>>>>>>> Such use
> > > > >>>>>>>>>>>>>>>>>>>> case
> > > > >>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>> not common and
> > > > >>>>>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I
> > > > >> would
> > > > >>>>>> propose
> > > > >>>>>>>> we have
> > > > >>>>>>>>>>> a
> > > > >>>>>>>>>>>>> map
> > > > >>>>>>>>>>>>>> of hintable
> > > > >>>>>>>>>>>>>>>>>>>>>> properties for each
> > > > >>>>>>>>>>>>>>>>>>>>>> connector, and should validate all
> > > > >>> passed
> > > > >>>>>> in
> > > > >>>>>>>>>>> properties
> > > > >>>>>>>>>>>>>> are actually
> > > > >>>>>>>>>>>>>>>>>>>>>> hintable. And combining with
> > > > >>>>>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
> > > > >>>>>> exception
> > > > >>>>>>>> once
> > > > >>>>>>>>>>>>> received
> > > > >>>>>>>>>>>>>> invalid
> > > > >>>>>>>>>>>>>>>>>>>>>> property.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm
> > > > >>> not sure
> > > > >>>>>>>> it's
> > > > >>>>>>>>>>>>> feasible.
> > > > >>>>>>>>>>>>>> Different
> > > > >>>>>>>>>>>>>>>>>>>>>> connectors will have totally
> > > > >>>>>>>>>>>>>>>>>>>>>> different properties to represent
> > > > >>> offset,
> > > > >>>>>> some
> > > > >>>>>>>> might
> > > > >>>>>>>>>>> be
> > > > >>>>>>>>>>>>>> timestamps,
> > > > >>>>>>>>>>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>>>>>>> might be string literals
> > > > >>>>>>>>>>>>>>>>>>>>>> like "earliest", and others might
> > > > >> be
> > > > >>> just
> > > > >>>>>>>> integers.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>>>> Kurt
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM
> > > > >>> Jark Wu <
> > > > >>>>>>>>>>>>> imjark@gmail.com>
> > > > >>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 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 <
> > > > >>>>>>>>>>>>>> twalthr@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 <
> > > > >>>>>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
> > > > >>>>>>>>>>>>>> twalthr@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
> > > > >>>>>>>>>>>>> <
> > > > >>>>>>>>>>>>>> imjark@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 <
> > > > >>>>>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>>
> > > > >>
> > > > >
> > > >
> > > >
> > >
> >
>


-- 
Best, Jingsong Lee

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <da...@apache.org>.
Thanks Jark for the feedback ~

I actually have a discussion offline with Timo and we think the black-list
options has implicit rick with the growing new table options, a black-list
there means all the new introduced options are default to be configurable
dynamically, if the user forget to add it into the black-list, that would
be a risk, what do you think about this @Timo ?

Jark Wu <im...@gmail.com> 于2020年3月26日周四 下午5:29写道:

> Hi Danny,
>
> Regarding to `supportedHintOptions()` interface, I suggest to use the
> inverted version, `unsupportedHintOptions()`.
> Because I think the disallowed list is much smaller.
> In addition, it's hard to list all the properties under
> `connector.properties.*`.
> But we know `connector.properties.bootstrap.servers` and
> `connector.properties.zookeeper.connect` are the only security options.
>
> Best,
> Jark
>
> On Thu, 26 Mar 2020 at 16:47, Kurt Young <yk...@gmail.com> wrote:
>
> > Hi Danny,
> >
> > Thanks for the updates. I have 2 comments regarding to latest document:
> >
> > 1) I think we also need `*supportedHintOptions*` for
> > `*TableFormatFactory*`
> > 2) IMO "dynamic-table-options.enabled" should belong to `
> > *OptimizerConfigOptions*`
> >
> > Best,
> > Kurt
> >
> >
> > On Thu, Mar 26, 2020 at 4:40 PM Timo Walther <tw...@apache.org> wrote:
> >
> > > Thanks for the update Danny. +1 for this proposal.
> > >
> > > Regards,
> > > Timo
> > >
> > > On 26.03.20 04:51, Danny Chan wrote:
> > > > Thanks everyone who engaged in this discussion ~
> > > >
> > > > Our goal is "Supports Dynamic Table Options for Flink SQL". After an
> > > > offline discussion with Kurt, Timo and Dawid, we have made the final
> > > > conclusion, here is the summary:
> > > >
> > > >
> > > >     - Use comment style syntax to specify the dynamic table options:
> > "/*+
> > > >     *OPTIONS*(k1='v1', k2='v2') */"
> > > >     - Have constraint on the options keys: the options that may bring
> > in
> > > >     security problems should not be allowed, i.e. Kafka connector
> > > zookeeper
> > > >     endpoint URL and topic name
> > > >     - Use white-list to control the allowed options for each
> connector,
> > > >     which is more safe for future extention
> > > >     - We allow to enable/disable this feature globally
> > > >     - Implement based on the current code base first, and when
> FLIP-95
> > is
> > > >     checked in, implement this feature based on new interface
> > > >
> > > > Any suggestions are appreciated ~
> > > >
> > > > [1]
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
> > > >
> > > > Best,
> > > > Danny Chan
> > > >
> > > > Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:
> > > >
> > > >> Hi everyone,
> > > >>
> > > >> Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid
> > it
> > > >> doesn't solve the problems but increases some development and
> learning
> > > >> burdens.
> > > >>
> > > >> # increase development and learning burden
> > > >>
> > > >> According to the discussion so far, we want to support overriding a
> > > subset
> > > >> of options in hints which doesn't affect semantics.
> > > >> With the `supportedHintOptions`, it's up to the connector developers
> > to
> > > >> decide which options will not affect semantics, and to be hint
> > options.
> > > >> However, the question is how to distinguish whether an option will
> > > *affect
> > > >> semantics*? What happens if an option will affect semantics but
> > > provided as
> > > >> hint options?
> > > >>  From my point of view, it's not easy to distinguish. For example,
> the
> > > >> "format.ignore-parse-error" can be a very useful dynamic option but
> > that
> > > >> will affect semantic, because the result is different (null vs
> > > exception).
> > > >> Another example, the "connector.lookup.cache.*" options are also
> very
> > > >> useful to tune jobs, however, it will also affect the job results. I
> > can
> > > >> come up many more useful options but may affect semantics.
> > > >>
> > > >> I can see that the community will under endless discussion around
> "can
> > > this
> > > >> option to be a hint option?",  "wether this option will affect
> > > semantics?".
> > > >> You can also find that we already have different opinions on
> > > >> "ignore-parse-error". Those discussion is a waste of time! That's
> not
> > > what
> > > >> users want!
> > > >> The problem is user need this, this, this options and HOW to expose
> > > them?
> > > >> We should focus on that.
> > > >>
> > > >> Then there could be two endings in the future:
> > > >> 1) compromise on the usability, we drop the rule that hints don't
> > affect
> > > >> semantics, allow all the useful options in the hints list.
> > > >> 2) stick on the rule, users will find this is a stumbling feature
> > which
> > > >> doesn't solve their problems.
> > > >>      And they will be surprised why this option can't be set, but
> the
> > > other
> > > >> could. *semantic* is hard to be understood by users.
> > > >>
> > > >> # doesn't solve the problems
> > > >>
> > > >> I think the purpose of this FLIP is to allow users to quickly
> override
> > > some
> > > >> connectors' properties to tune their jobs.
> > > >> However, `supportedHintOptions` is off track. It only allows a
> subset
> > > >> options and for the users it's not *clear* which subset is allowed.
> > > >>
> > > >> Besides, I'm not sure `supportedHintOptions` can work well for all
> > > cases.
> > > >> How could you support kafka properties (`connector.properties.*`) as
> > > hint
> > > >> options? Some kafka properties may affect semantics
> > (bootstrap.servers),
> > > >> some may not (max.poll.records). Besides, I think it's not possible
> to
> > > list
> > > >> all the possible kafka properties [1].
> > > >>
> > > >> In summary, IMO, `supportedHintOptions`
> > > >> (1) it increase the complexity to develop a connector
> > > >> (2) it confuses users which options can be used in hint, which are
> > not,
> > > >> they have to check the docs again and again.
> > > >> (3) it doesn't solve the problems which we want to solve by this
> FLIP.
> > > >>
> > > >> I think we should avoid introducing some partial solutions.
> Otherwise,
> > > we
> > > >> will be stuck in a loop that introduce new API -> deprecate API ->
> > > >> introduce new API....
> > > >>
> > > >> I personally in favor of an explicit WITH syntax after the table as
> a
> > > part
> > > >> of the query which is mentioned by Kurt before, e.g. SELECT * from T
> > > >> WITH('key' = 'value') .
> > > >> It allows users to dynamically set options which can affect
> semantics.
> > > It
> > > >> will be very flexible to solve users' problems so far.
> > > >>
> > > >> Best,
> > > >> Jark
> > > >>
> > > >> [1]: https://kafka.apache.org/documentation/#consumerconfigs
> > > >>
> > > >> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com>
> > wrote:
> > > >>
> > > >>> My POC is here for the hints options merge [1].
> > > >>>
> > > >>> Personally, I have no strong objections for splitting hints with
> the
> > > >>> CatalogTable, the only cons is a more complex implementation but
> the
> > > >>> concept is more clear, and I have updated the WIKI.
> > > >>>
> > > >>> I think it would be nice if we can support the format “ignore-parse
> > > >> error”
> > > >>> option key, the CSV source already has a key [2] and we can use
> that
> > in
> > > >> the
> > > >>> supportedHIntOptions, for the common CSV and JSON formats, we cal
> > also
> > > >> give
> > > >>> a support. This is the only kind of key in formats that “do not
> > change
> > > >> the
> > > >>> semantics” (somehow), what do you think about this ~
> > > >>>
> > > >>> [1]
> > > >>>
> > > >>
> > >
> >
> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
> > > >>> [2]
> > > >>>
> > > >>
> > >
> >
> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
> > > >>>
> > > >>> Best,
> > > >>> Danny Chan
> > > >>> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
> > > >>>> Hi everyone,
> > > >>>>
> > > >>>> +1 to Kurt's suggestion. Let's just have it in source and sink
> > > >> factories
> > > >>>> for now. We can still move this method up in the future.
> Currently,
> > I
> > > >>>> don't see a need for catalogs or formats. Because how would you
> > target
> > > >> a
> > > >>>> format in the query?
> > > >>>>
> > > >>>> @Danny: Can you send a link to your PoC? I'm very skeptical about
> > > >>>> creating a new CatalogTable in planner. Actually CatalogTable
> should
> > > be
> > > >>>> immutable between Catalog and Factory. Because a catalog can
> return
> > > its
> > > >>>> own factory and fully control the instantiation. Depending on the
> > > >>>> implementation, that means it can be possible that the catalog has
> > > >>>> encoded more information in a concrete subclass implementing the
> > > >>>> interface. I vote for separating the concerns of catalog
> information
> > > >> and
> > > >>>> hints in the factory explicitly.
> > > >>>>
> > > >>>> Regards,
> > > >>>> Timo
> > > >>>>
> > > >>>>
> > > >>>> On 18.03.20 05:41, Jingsong Li wrote:
> > > >>>>> Hi,
> > > >>>>>
> > > >>>>> I am thinking we can provide hints to *table* related instances.
> > > >>>>> - TableFormatFactory: of cause we need hints support, there are
> > many
> > > >>> format
> > > >>>>> options in DDL too.
> > > >>>>> - catalog and module: I don't know, maybe in future we can
> provide
> > > >> some
> > > >>>>> hints for them.
> > > >>>>>
> > > >>>>> Best,
> > > >>>>> Jingsong Lee
> > > >>>>>
> > > >>>>> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <
> yuzhao.cyz@gmail.com>
> > > >>> wrote:
> > > >>>>>
> > > >>>>>> Yes, I think we should move the `supportedHintOptions` from
> > > >>> TableFactory
> > > >>>>>> to TableSourceFactory, and we also need to add the interface to
> > > >>>>>> TableSinkFactory though because sink target table may also have
> > > >> hints
> > > >>>>>> attached.
> > > >>>>>>
> > > >>>>>> Best,
> > > >>>>>> Danny Chan
> > > >>>>>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> > > >>>>>>> Have one question for adding `supportedHintOptions` method to
> > > >>>>>>> `TableFactory`. It seems
> > > >>>>>>> `TableFactory` is a base factory interface for all *table
> module*
> > > >>> related
> > > >>>>>>> instances, such as
> > > >>>>>>> catalog, module, format and so on. It's not created only for
> > > >>> *table*. Is
> > > >>>>>> it
> > > >>>>>>> possible to move it
> > > >>>>>>> to `TableSourceFactory`?
> > > >>>>>>>
> > > >>>>>>> Best,
> > > >>>>>>> Kurt
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
> > > >> yuzhao.cyz@gmail.com>
> > > >>>>>> wrote:
> > > >>>>>>>
> > > >>>>>>>> Thanks Timo ~
> > > >>>>>>>>
> > > >>>>>>>> For the naming itself, I also think the PROPERTIES is not that
> > > >>>>>> concise, so
> > > >>>>>>>> +1 for OPTIONS (I had thought about that, but there are many
> > > >>> codes in
> > > >>>>>>>> current Flink called it properties, i.e. the
> > > >>> DescriptorProperties,
> > > >>>>>>>> #getSupportedProperties), let’s use OPTIONS if this is our new
> > > >>>>>> preference.
> > > >>>>>>>>
> > > >>>>>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
> > > >>>>>> ConfigOption
> > > >>>>>>>> can take more info. AFAIK, Spark also call their table options
> > > >>> instead
> > > >>>>>> of
> > > >>>>>>>> properties. [1]
> > > >>>>>>>>
> > > >>>>>>>> In my local POC, I did create a new CatalogTable, and it works
> > > >>> for
> > > >>>>>> current
> > > >>>>>>>> connectors well, all the DDL tables would finally yield a
> > > >>> CatalogTable
> > > >>>>>>>> instance and we can apply the options to that(in the
> > > >>> CatalogSourceTable
> > > >>>>>>>> when we generating the TableSource), the pros is that we do
> not
> > > >>> need to
> > > >>>>>>>> modify the codes of connectors itself. If we split the options
> > > >>> from
> > > >>>>>>>> CatalogTable, we may need to add some additional logic in each
> > > >>>>>> connector
> > > >>>>>>>> factories in order to merge these properties (and the logic
> are
> > > >>> almost
> > > >>>>>> the
> > > >>>>>>>> same), what do you think about this?
> > > >>>>>>>>
> > > >>>>>>>> [1]
> > > >>>>>>>>
> > > >>>>>>
> > > >>>
> > > >>
> > >
> >
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> > > >>>>>>>>
> > > >>>>>>>> Best,
> > > >>>>>>>> Danny Chan
> > > >>>>>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
> > > >>> ,写道:
> > > >>>>>>>>> Hi Danny,
> > > >>>>>>>>>
> > > >>>>>>>>> thanks for updating the FLIP. I think your current design is
> > > >>>>>> sufficient
> > > >>>>>>>>> to separate hints from result-related properties.
> > > >>>>>>>>>
> > > >>>>>>>>> One remark to the naming itself: I would vote for calling the
> > > >>> hints
> > > >>>>>>>>> around table scan `OPTIONS('k'='v')`. We used the term
> > > >>> "properties"
> > > >>>>>> in
> > > >>>>>>>>> the past but since we want to unify the Flink configuration
> > > >>>>>> experience,
> > > >>>>>>>>> we should use consistent naming and classes around
> > > >>> `ConfigOptions`.
> > > >>>>>>>>>
> > > >>>>>>>>> It would be nice to use `Set<ConfigOption>
> > > >>> supportedHintOptions();`
> > > >>>>>> to
> > > >>>>>>>>> start using config options instead of pure string properties.
> > > >>> This
> > > >>>>>> will
> > > >>>>>>>>> also allow us to generate documentation in the future around
> > > >>>>>> supported
> > > >>>>>>>>> data types, ranges, etc. for options. At some point we would
> > > >>> also
> > > >>>>>> like
> > > >>>>>>>>> to drop `DescriptorProperties` class. "Options" is also used
> > > >>> in the
> > > >>>>>>>>> documentation [1] and in the SQL/MED standard [2].
> > > >>>>>>>>>
> > > >>>>>>>>> Furthermore, I would still vote for separating CatalogTable
> > > >>> and hint
> > > >>>>>>>>> options. Otherwise the planner would need to create a new
> > > >>>>>> CatalogTable
> > > >>>>>>>>> instance which might not always be easy. We should offer them
> > > >>> via:
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>
> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > > >>>>>>>>> ReadableConfig
> > > >>>>>>>>>
> > > >>>>>>>>> What do you think?
> > > >>>>>>>>>
> > > >>>>>>>>> Regards,
> > > >>>>>>>>> Timo
> > > >>>>>>>>>
> > > >>>>>>>>> [1]
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>
> > > >>>
> > > >>
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > > >>>>>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
> > > >>>>>>>>>> @Danny sounds good.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Maybe it is worth listing all the classes of problems that
> > > >>> you
> > > >>>>>> want to
> > > >>>>>>>>>> address and then look at each class and see if hints are a
> > > >>> good
> > > >>>>>> default
> > > >>>>>>>>>> solution or a good optional way of simplifying things?
> > > >>>>>>>>>> The discussion has grown a lot and it is starting to be
> > > >> hard
> > > >>> to
> > > >>>>>>>> distinguish
> > > >>>>>>>>>> the parts where everyone agrees from the parts were there
> > > >> are
> > > >>>>>> concerns.
> > > >>>>>>>>>>
> > > >>>>>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
> > > >>> danny0405@apache.org>
> > > >>>>>>>> wrote:
> > > >>>>>>>>>>
> > > >>>>>>>>>>> Thanks Stephan ~
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> We can remove the support for properties that may change
> > > >>> the
> > > >>>>>>>> semantics of
> > > >>>>>>>>>>> query if you think that is a trouble.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> How about we support the /*+ properties() */ hint only
> > > >> for
> > > >>> those
> > > >>>>>>>> optimize
> > > >>>>>>>>>>> parameters, such as the fetch size of source or something
> > > >>> like
> > > >>>>>> that,
> > > >>>>>>>> does
> > > >>>>>>>>>>> that make sense?
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> I think Bowen has actually put it very well.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> (1) Hints that change semantics looks like trouble
> > > >>> waiting to
> > > >>>>>>>> happen. For
> > > >>>>>>>>>>>> example Kafka offset handling should be in filters. The
> > > >>> Kafka
> > > >>>>>>>> source
> > > >>>>>>>>>>> should
> > > >>>>>>>>>>>> support predicate pushdown.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> (2) Hints should not be a workaround for current
> > > >>> shortcomings.
> > > >>>>>> A
> > > >>>>>>>> lot of
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>> suggested above sounds exactly like that. Working
> > > >> around
> > > >>>>>>>> catalog/DDL
> > > >>>>>>>>>>>> shortcomings, missing exposure of metadata (offsets),
> > > >>> missing
> > > >>>>>>>> predicate
> > > >>>>>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as
> > > >> a
> > > >>> quick
> > > >>>>>> fix
> > > >>>>>>>> for
> > > >>>>>>>>>>>> these issues, rather than fixing the root causes, will
> > > >>> much
> > > >>>>>> likely
> > > >>>>>>>> bite
> > > >>>>>>>>>>> us
> > > >>>>>>>>>>>> back badly in the future.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Best,
> > > >>>>>>>>>>>> Stephan
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
> > > >>> ykt836@gmail.com>
> > > >>>>>>>> wrote:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>> It seems this FLIP's name is somewhat misleading.
> > > >> From
> > > >>> my
> > > >>>>>>>>>>> understanding,
> > > >>>>>>>>>>>>> this FLIP is trying to
> > > >>>>>>>>>>>>> address the dynamic parameter issue, and table hints
> > > >>> is the
> > > >>>>>> way
> > > >>>>>>>> we wan
> > > >>>>>>>>>>> to
> > > >>>>>>>>>>>>> choose. I think we should
> > > >>>>>>>>>>>>> be focus on "what's the right way to solve dynamic
> > > >>> property"
> > > >>>>>>>> instead of
> > > >>>>>>>>>>>>> discussing "whether table
> > > >>>>>>>>>>>>> hints can affect query semantics".
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> For now, there are two proposed ways to achieve
> > > >> dynamic
> > > >>>>>> property:
> > > >>>>>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with
> > > >>> (xxx)
> > > >>>>>>>>>>>>> 2. use custom "from t with (xxx)" syntax
> > > >>>>>>>>>>>>> 3. "Borrow" the table hints to have a special
> > > >>> PROPERTIES
> > > >>>>>> hint.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> The first one didn't break anything, but the only
> > > >>> problem i
> > > >>>>>> see
> > > >>>>>>>> is a
> > > >>>>>>>>>>>> little
> > > >>>>>>>>>>>>> more verbose than the table hint
> > > >>>>>>>>>>>>> approach. I can imagine when someone using SQL CLI to
> > > >>> have a
> > > >>>>>> sql
> > > >>>>>>>>>>>>> experience, it's quite often that
> > > >>>>>>>>>>>>> he will modify the table property, some use cases i
> > > >> can
> > > >>>>>> think of:
> > > >>>>>>>>>>>>> 1. the source contains some corrupted data, i want to
> > > >>> turn
> > > >>>>>> on the
> > > >>>>>>>>>>>>> "ignore-error" flag for certain formats.
> > > >>>>>>>>>>>>> 2. I have a kafka table and want to see some sample
> > > >>> data
> > > >>>>>> from the
> > > >>>>>>>>>>>>> beginning, so i change the offset
> > > >>>>>>>>>>>>> to "earliest", and then I want to observe the latest
> > > >>> data
> > > >>>>>> which
> > > >>>>>>>> keeps
> > > >>>>>>>>>>>>> coming in. I would write another query
> > > >>>>>>>>>>>>> to select from the latest table.
> > > >>>>>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly
> > > >> then
> > > >>> i can
> > > >>>>>>>> observe
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>>> data from database side.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Most of such use cases are quite ad-hoc. If every
> > > >> time
> > > >>> I
> > > >>>>>> want to
> > > >>>>>>>> have a
> > > >>>>>>>>>>>>> different experience, i need to create
> > > >>>>>>>>>>>>> a temporary table and then also modify my query, it
> > > >>> doesn't
> > > >>>>>> feel
> > > >>>>>>>>>>> smooth.
> > > >>>>>>>>>>>>> Embed such dynamic property into
> > > >>>>>>>>>>>>> query would have better user experience.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is
> > > >>> breaking
> > > >>>>>> SQL
> > > >>>>>>>>>>>> compliant,
> > > >>>>>>>>>>>>> and for #3, it only breaks some
> > > >>>>>>>>>>>>> unwritten rules, but we can have an explanation on
> > > >>> that. And
> > > >>>>>> I
> > > >>>>>>>> really
> > > >>>>>>>>>>>> doubt
> > > >>>>>>>>>>>>> whether user would complain about
> > > >>>>>>>>>>>>> this when they actually have flexible and good
> > > >>> experience
> > > >>>>>> using
> > > >>>>>>>> this.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>> Kurt
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> > > >>>>>> yuzhao.cyz@gmail.com
> > > >>>>>>>>>
> > > >>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Thanks Aljoscha ~
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> I agree for most of the query hints, they are
> > > >>> optional as
> > > >>>>>> an
> > > >>>>>>>>>>> optimizer
> > > >>>>>>>>>>>>>> instruction, especially for the traditional RDBMS.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> But, just like BenChao said, Flink as a computation
> > > >>> engine
> > > >>>>>> has
> > > >>>>>>>> many
> > > >>>>>>>>>>>>>> different kind of data sources, thus, dynamic
> > > >>> parameters
> > > >>>>>> like
> > > >>>>>>>>>>>>> start_offest
> > > >>>>>>>>>>>>>> can only bind to each table scope, we can not set a
> > > >>> session
> > > >>>>>>>> config
> > > >>>>>>>>>>> like
> > > >>>>>>>>>>>>>> KSQL because they are all about Kafka:
> > > >>>>>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Thus the most flexible way to set up these dynamic
> > > >>> params
> > > >>>>>> is
> > > >>>>>>>> to bind
> > > >>>>>>>>>>> to
> > > >>>>>>>>>>>>>> the table scope in the query when we want to
> > > >> override
> > > >>>>>>>> something, so
> > > >>>>>>>>>>> we
> > > >>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>> these solutions above (with pros and cons from my
> > > >>> side):
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Pros:
> > > >>>>>>>>>>>>>> - Easy to add
> > > >>>>>>>>>>>>>> - Parameters are part of the main query
> > > >>>>>>>>>>>>>> Cons:
> > > >>>>>>>>>>>>>> - Not SQL compliant
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */
> > > >>> (from
> > > >>>>>> me)
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Pros:
> > > >>>>>>>>>>>>>> - Easy to add
> > > >>>>>>>>>>>>>> - SQL compliant because it is nested in the
> > > >> comments
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Cons:
> > > >>>>>>>>>>>>>> - Parameters are not part of the main query
> > > >>>>>>>>>>>>>> - Cryptic syntax for new users
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> The biggest problem for hints way may be the “if
> > > >>> hints
> > > >>>>>> must be
> > > >>>>>>>>>>>> optional”,
> > > >>>>>>>>>>>>>> actually we have though about 1 for a while but
> > > >>> aborted
> > > >>>>>>>> because it
> > > >>>>>>>>>>>> breaks
> > > >>>>>>>>>>>>>> the SQL standard too much. And we replace it with
> > > >> 2,
> > > >>>>>> because
> > > >>>>>>>> the
> > > >>>>>>>>>>> hints
> > > >>>>>>>>>>>>>> syntax do not break SQL standard(nested in
> > > >> comments).
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint
> > > >>> that
> > > >>>>>> allows
> > > >>>>>>>>>>> override
> > > >>>>>>>>>>>>>> some properties of table dynamically, it does not
> > > >>> break
> > > >>>>>>>> anything, at
> > > >>>>>>>>>>>>> lease
> > > >>>>>>>>>>>>>> for current Flink use cases.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Planner hints are optional just because they are
> > > >>> naturally
> > > >>>>>>>> enforcers
> > > >>>>>>>>>>> of
> > > >>>>>>>>>>>>>> the planner, most of them aim to instruct the
> > > >>> optimizer,
> > > >>>>>> but,
> > > >>>>>>>> the
> > > >>>>>>>>>>> table
> > > >>>>>>>>>>>>>> hints is a little different, table hints can
> > > >> specify
> > > >>> the
> > > >>>>>> table
> > > >>>>>>>> meta
> > > >>>>>>>>>>>> like
> > > >>>>>>>>>>>>>> index column, and it is very convenient to specify
> > > >>> table
> > > >>>>>>>> properties.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */
> > > >>> table
> > > >>>>>> hint,
> > > >>>>>>>> we
> > > >>>>>>>>>>> can
> > > >>>>>>>>>>>>>> call it table dynamic parameters.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>> Danny Chan
> > > >>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> > > >>>>>>>> aljoscha@apache.org>,写道:
> > > >>>>>>>>>>>>>>> Hi,
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> I don't understand this discussion. Hints, as I
> > > >>>>>> understand
> > > >>>>>>>> them,
> > > >>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>> work like this:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> - hints are *optional* advice for the optimizer
> > > >> to
> > > >>> try
> > > >>>>>> and
> > > >>>>>>>> help it
> > > >>>>>>>>>>> to
> > > >>>>>>>>>>>>>>> find a good execution strategy
> > > >>>>>>>>>>>>>>> - hints should not change query semantics, i.e.
> > > >>> they
> > > >>>>>> should
> > > >>>>>>>> not
> > > >>>>>>>>>>>> change
> > > >>>>>>>>>>>>>>> connector properties executing a query with
> > > >> taking
> > > >>> into
> > > >>>>>>>> account the
> > > >>>>>>>>>>>>>>> hints *must* produce the same result as executing
> > > >>> the
> > > >>>>>> query
> > > >>>>>>>> without
> > > >>>>>>>>>>>>>>> taking into account the hints
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>  From these simple requirements you can derive a
> > > >>> solution
> > > >>>>>>>> that makes
> > > >>>>>>>>>>>>>>> sense. I don't have a strong preference for the
> > > >>> syntax
> > > >>>>>> but we
> > > >>>>>>>>>>> should
> > > >>>>>>>>>>>>>>> strive to be in line with prior work.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>> Aljoscha
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
> > > >>>>>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I agree with Kurt that option2 is too
> > > >>> complicated to
> > > >>>>>> use
> > > >>>>>>>> because:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> • As a Kafka topic consumer, the user must
> > > >>> define both
> > > >>>>>> the
> > > >>>>>>>>>>> virtual
> > > >>>>>>>>>>>>>> column for start offset and he must apply a special
> > > >>> filter
> > > >>>>>>>> predicate
> > > >>>>>>>>>>>>> after
> > > >>>>>>>>>>>>>> each query
> > > >>>>>>>>>>>>>>>> • And for the internal implementation, the
> > > >>> metadata
> > > >>>>>> column
> > > >>>>>>>> push
> > > >>>>>>>>>>>> down
> > > >>>>>>>>>>>>>> is another hard topic, each kind of message queue
> > > >>> may have
> > > >>>>>> its
> > > >>>>>>>> offset
> > > >>>>>>>>>>>>>> attribute, we need to consider the expression type
> > > >>> for
> > > >>>>>>>> different
> > > >>>>>>>>>>> kind;
> > > >>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>> source also need to recognize the constant column
> > > >> as
> > > >>> a
> > > >>>>>> config
> > > >>>>>>>>>>>>> option(which
> > > >>>>>>>>>>>>>> is weird because usually what we pushed down is a
> > > >>> table
> > > >>>>>> column)
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> For option 1 and option3, I think there is no
> > > >>>>>> difference,
> > > >>>>>>>> option1
> > > >>>>>>>>>>>> is
> > > >>>>>>>>>>>>>> also a hint syntax which is introduced in Sybase
> > > >> and
> > > >>>>>>>> referenced then
> > > >>>>>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
> > > >>>>>>>> ambitiousness.
> > > >>>>>>>>>>>>> Personally
> > > >>>>>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword
> > > >>> for
> > > >>>>>> these
> > > >>>>>>>> reasons:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> • We do not break the standard SQL, the hints
> > > >> are
> > > >>>>>> nested
> > > >>>>>>>> in SQL
> > > >>>>>>>>>>>>>> comments
> > > >>>>>>>>>>>>>>>> • We do not need to introduce additional WITH
> > > >>> keyword
> > > >>>>>>>> which may
> > > >>>>>>>>>>>>> appear
> > > >>>>>>>>>>>>>> in a query if we use that because a table can be
> > > >>>>>> referenced in
> > > >>>>>>>> all
> > > >>>>>>>>>>>> kinds
> > > >>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
> > > >>> make
> > > >>>>>> our
> > > >>>>>>>> sql
> > > >>>>>>>>>>> query
> > > >>>>>>>>>>>>>> break too much of the SQL from standard
> > > >>>>>>>>>>>>>>>> • We would have uniform syntax for hints as
> > > >> query
> > > >>>>>> hint, one
> > > >>>>>>>>>>> syntax
> > > >>>>>>>>>>>>>> fits all and more easy to use
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> And here is the reason why we choose a uniform
> > > >>> Oracle
> > > >>>>>>>> style query
> > > >>>>>>>>>>>>>> hint syntax which is addressed by Julian Hyde when
> > > >> we
> > > >>>>>> design
> > > >>>>>>>> the
> > > >>>>>>>>>>> syntax
> > > >>>>>>>>>>>>>> from the Calcite community:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for
> > > >>> table
> > > >>>>>> hints.
> > > >>>>>>>> It
> > > >>>>>>>>>>> adds a
> > > >>>>>>>>>>>>>> new use of the WITH keyword that is unrelated to
> > > >> the
> > > >>> use of
> > > >>>>>>>> WITH for
> > > >>>>>>>>>>>>>> common-table expressions.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> A historical note. Microsoft SQL Server
> > > >>> inherited its
> > > >>>>>> hint
> > > >>>>>>>> syntax
> > > >>>>>>>>>>>>> from
> > > >>>>>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
> > > >>>>>>>> Programming”[1], page
> > > >>>>>>>>>>>>> 632,
> > > >>>>>>>>>>>>>> “Optimizer hints”. The book was written in 1999,
> > > >> and
> > > >>> covers
> > > >>>>>>>> Microsoft
> > > >>>>>>>>>>>> SQL
> > > >>>>>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
> > > >>> but the
> > > >>>>>>>> syntax very
> > > >>>>>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft
> > > >> SQL
> > > >>>>>> Server was
> > > >>>>>>>>>>> forked
> > > >>>>>>>>>>>> in
> > > >>>>>>>>>>>>>> 1993.)
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Microsoft later added the WITH keyword to make
> > > >>> it less
> > > >>>>>>>> ambiguous,
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>> has now deprecated the syntax that does not use
> > > >> WITH.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> They are forced to keep the syntax for
> > > >> backwards
> > > >>>>>>>> compatibility
> > > >>>>>>>>>>> but
> > > >>>>>>>>>>>>>> that doesn’t mean that we should shoulder their
> > > >>> burden.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I think formatted comments are the right
> > > >>> container for
> > > >>>>>>>> hints
> > > >>>>>>>>>>>> because
> > > >>>>>>>>>>>>>> it allows us to change the hint syntax without
> > > >>> changing
> > > >>>>>> the SQL
> > > >>>>>>>>>>> parser,
> > > >>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>> makes clear that we are at liberty to ignore hints
> > > >>>>>> entirely.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Julian
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
> > > >>>>>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>> Danny Chan
> > > >>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> > > >>>>>> twalthr@apache.org
> > > >>>>>>>>> ,写道:
> > > >>>>>>>>>>>>>>>>> Hi Danny,
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> it is true that our DDL is not standard
> > > >>> compliant by
> > > >>>>>>>> using the
> > > >>>>>>>>>>>> WITH
> > > >>>>>>>>>>>>>>>>> clause. Nevertheless, we aim for not
> > > >> diverging
> > > >>> too
> > > >>>>>> much
> > > >>>>>>>> and the
> > > >>>>>>>>>>>>> LIKE
> > > >>>>>>>>>>>>>>>>> clause is an example of that. It will solve
> > > >>> things
> > > >>>>>> like
> > > >>>>>>>>>>>> overwriting
> > > >>>>>>>>>>>>>>>>> WATERMARKs, add additional/modifying
> > > >>> properties and
> > > >>>>>>>> inherit
> > > >>>>>>>>>>>> schema.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3
> > > >>> types
> > > >>>>>>>> definition
> > > >>>>>>>>>>>>>> together.
> > > >>>>>>>>>>>>>>>>> We are not the first ones that try to solve
> > > >>> this.
> > > >>>>>> There
> > > >>>>>>>> is also
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>> SQL
> > > >>>>>>>>>>>>>>>>> MED standard [1] that tried to tackle this
> > > >>> problem. I
> > > >>>>>>>> think it
> > > >>>>>>>>>>>> was
> > > >>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>> considered when designing the current DDL.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
> > > >>>>>> offsets. I
> > > >>>>>>>> will
> > > >>>>>>>>>>>> give
> > > >>>>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>> examples and look forward to feedback here:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part
> > > >>> of the
> > > >>>>>>>> query
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Pros:
> > > >>>>>>>>>>>>>>>>> - Easy to add
> > > >>>>>>>>>>>>>>>>> - Parameters are part of the main query
> > > >>>>>>>>>>>>>>>>> - No complicated hinting syntax
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Cons:
> > > >>>>>>>>>>>>>>>>> - Not SQL compliant
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> *Option 2* Use metadata in query
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
> > > >>>>>>>>>>>>> SYSTEM_METADATA('offset'))`
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset >
> > > >> TIMESTAMP
> > > >>>>>>>> '2012-12-12
> > > >>>>>>>>>>>>>> 12:34:22'`
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Pros:
> > > >>>>>>>>>>>>>>>>> - SQL compliant in the query
> > > >>>>>>>>>>>>>>>>> - Access of metadata in the DDL which is
> > > >>> required
> > > >>>>>> anyway
> > > >>>>>>>>>>>>>>>>> - Regular pushdown rules apply
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Cons:
> > > >>>>>>>>>>>>>>>>> - Users need to add an additional comlumn in
> > > >>> the DDL
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> *Option 3*: Use hints for properties
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> `
> > > >>>>>>>>>>>>>>>>> SELECT *
> > > >>>>>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > >>>>>>>>>>>>>>>>> `
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Pros:
> > > >>>>>>>>>>>>>>>>> - Easy to add
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Cons:
> > > >>>>>>>>>>>>>>>>> - Parameters are not part of the main query
> > > >>>>>>>>>>>>>>>>> - Cryptic syntax for new users
> > > >>>>>>>>>>>>>>>>> - Not standard compliant.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> If we go with this option, I would suggest to
> > > >>> make it
> > > >>>>>>>> available
> > > >>>>>>>>>>>> in
> > > >>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>> separate map and don't mix it with statically
> > > >>> defined
> > > >>>>>>>>>>> properties.
> > > >>>>>>>>>>>>>> Such
> > > >>>>>>>>>>>>>>>>> that the factory can decide which properties
> > > >>> have the
> > > >>>>>>>> right to
> > > >>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>> overwritten by the hints:
> > > >>>>>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
> > > >>>>>>>> ReadableConfig
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>> Timo
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Currently I see 3 options as a
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
> > > >>>>>>>>>>>>>>>>>> Thanks Bowen ~
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I agree we should somehow categorize our
> > > >>> connector
> > > >>>>>>>>>>> parameters.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> For type1, I’m already preparing a solution
> > > >>> like
> > > >>>>>> the
> > > >>>>>>>>>>> Confluent
> > > >>>>>>>>>>>>>> schema registry + Avro schema inference thing, so
> > > >>> this may
> > > >>>>>> not
> > > >>>>>>>> be a
> > > >>>>>>>>>>>>> problem
> > > >>>>>>>>>>>>>> in the near future.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> For type3, I have some questions:
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset >
> > > >> 12pm
> > > >>>>>> yesterday”
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Where does the offset column come from, a
> > > >>> virtual
> > > >>>>>>>> column from
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>> table schema, you said that
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> They change
> > > >>>>>>>>>>>>>>>>>> almost every time a query starts and have
> > > >>> nothing
> > > >>>>>> to
> > > >>>>>>>> do with
> > > >>>>>>>>>>>>>> metadata, thus
> > > >>>>>>>>>>>>>>>>>> should not be part of table definition/DDL
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> But why you can reference it in the query,
> > > >>> I’m
> > > >>>>>>>> confused for
> > > >>>>>>>>>>>> that,
> > > >>>>>>>>>>>>>> can you elaborate a little ?
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>> Danny Chan
> > > >>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> > > >>>>>>>> bowenli86@gmail.com
> > > >>>>>>>>>>>> ,写道:
> > > >>>>>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> The root cause of too much manual work is
> > > >>> Flink
> > > >>>>>> DDL
> > > >>>>>>>> has
> > > >>>>>>>>>>>> mixed 3
> > > >>>>>>>>>>>>>> types of
> > > >>>>>>>>>>>>>>>>>>> params together and doesn't handle each
> > > >> of
> > > >>> them
> > > >>>>>> very
> > > >>>>>>>> well.
> > > >>>>>>>>>>>>> Below
> > > >>>>>>>>>>>>>> are how I
> > > >>>>>>>>>>>>>>>>>>> categorize them and corresponding
> > > >>> solutions in my
> > > >>>>>>>> mind:
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
> > > >>>>>> external
> > > >>>>>>>>>>>>> endpoint/url,
> > > >>>>>>>>>>>>>>>>>>> username/pwd, schemas, formats.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Such metadata are mostly already
> > > >>> accessible in
> > > >>>>>>>> external
> > > >>>>>>>>>>>> system
> > > >>>>>>>>>>>>>> as long as
> > > >>>>>>>>>>>>>>>>>>> endpoints and credentials are provided.
> > > >>> Flink can
> > > >>>>>>>> get it
> > > >>>>>>>>>>> thru
> > > >>>>>>>>>>>>>> catalogs, but
> > > >>>>>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus
> > > >>> Flink
> > > >>>>>> just
> > > >>>>>>>> hasn't
> > > >>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>> able to
> > > >>>>>>>>>>>>>>>>>>> leverage that. So the solution should be
> > > >>> building
> > > >>>>>>>> more
> > > >>>>>>>>>>>>> catalogs.
> > > >>>>>>>>>>>>>> Such
> > > >>>>>>>>>>>>>>>>>>> params should be part of a Flink table
> > > >>>>>>>> DDL/definition, and
> > > >>>>>>>>>>>> not
> > > >>>>>>>>>>>>>> overridable
> > > >>>>>>>>>>>>>>>>>>> in any means.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc
> > > >>> connector's
> > > >>>>>>>> fetch size,
> > > >>>>>>>>>>>>>> elasticsearch
> > > >>>>>>>>>>>>>>>>>>> connector's bulk flush size.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Such params don't affect query results,
> > > >> but
> > > >>>>>> affect
> > > >>>>>>>> how
> > > >>>>>>>>>>>> results
> > > >>>>>>>>>>>>>> are produced
> > > >>>>>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) -
> > > >> they
> > > >>> are
> > > >>>>>>>> essentially
> > > >>>>>>>>>>>>>> execution and
> > > >>>>>>>>>>>>>>>>>>> implementation details. They change often
> > > >>> in
> > > >>>>>>>> exploration or
> > > >>>>>>>>>>>>>> development
> > > >>>>>>>>>>>>>>>>>>> stages, but not quite frequently in
> > > >>> well-defined
> > > >>>>>>>>>>> long-running
> > > >>>>>>>>>>>>>> pipelines.
> > > >>>>>>>>>>>>>>>>>>> They should always have default values
> > > >> and
> > > >>> can be
> > > >>>>>>>> missing
> > > >>>>>>>>>>> in
> > > >>>>>>>>>>>>>> query. They
> > > >>>>>>>>>>>>>>>>>>> can be part of a table DDL/definition,
> > > >> but
> > > >>> should
> > > >>>>>>>> also be
> > > >>>>>>>>>>>>>> replaceable in a
> > > >>>>>>>>>>>>>>>>>>> query - *this is what table "hints" in
> > > >>> FLIP-113
> > > >>>>>>>> should
> > > >>>>>>>>>>>> cover*.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka
> > > >>> connector's
> > > >>>>>>>> start
> > > >>>>>>>>>>>> offset.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Such params affect query results - the
> > > >>> semantics.
> > > >>>>>>>> They'd
> > > >>>>>>>>>>>> better
> > > >>>>>>>>>>>>>> be as
> > > >>>>>>>>>>>>>>>>>>> filter conditions in WHERE clause that
> > > >> can
> > > >>> be
> > > >>>>>> pushed
> > > >>>>>>>> down.
> > > >>>>>>>>>>>> They
> > > >>>>>>>>>>>>>> change
> > > >>>>>>>>>>>>>>>>>>> almost every time a query starts and have
> > > >>>>>> nothing to
> > > >>>>>>>> do
> > > >>>>>>>>>>> with
> > > >>>>>>>>>>>>>> metadata, thus
> > > >>>>>>>>>>>>>>>>>>> should not be part of table
> > > >>> definition/DDL, nor
> > > >>>>>> be
> > > >>>>>>>>>>> persisted
> > > >>>>>>>>>>>> in
> > > >>>>>>>>>>>>>> catalogs.
> > > >>>>>>>>>>>>>>>>>>> If they will, users should create views
> > > >> to
> > > >>> keep
> > > >>>>>> such
> > > >>>>>>>> params
> > > >>>>>>>>>>>>>> around (note
> > > >>>>>>>>>>>>>>>>>>> this is different from variable
> > > >>> substitution).
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we
> > > >>> get these
> > > >>>>>>>> params
> > > >>>>>>>>>>>> right,
> > > >>>>>>>>>>>>>> here're the
> > > >>>>>>>>>>>>>>>>>>> steps users need to do to develop and run
> > > >>> a Flink
> > > >>>>>>>> job:
> > > >>>>>>>>>>>>>>>>>>> - configure a Flink
> > > >>> ConfluentSchemaRegistry with
> > > >>>>>> url,
> > > >>>>>>>>>>>> username,
> > > >>>>>>>>>>>>>> and password
> > > >>>>>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset
> > > >>>> 12pm
> > > >>>>>>>> yesterday"
> > > >>>>>>>>>>>>>> (simplified
> > > >>>>>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink
> > > >> automatically
> > > >>>>>> retrieves
> > > >>>>>>>> all
> > > >>>>>>>>>>>>>> metadata of
> > > >>>>>>>>>>>>>>>>>>> schema, file format, etc and start the
> > > >> job
> > > >>>>>>>>>>>>>>>>>>> - users want to make the job read Kafka
> > > >>> topic
> > > >>>>>>>> faster, so it
> > > >>>>>>>>>>>>> goes
> > > >>>>>>>>>>>>>> as "SELECT
> > > >>>>>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/
> > > >>> WHERE
> > > >>>>>>>> offset >
> > > >>>>>>>>>>> 12pm
> > > >>>>>>>>>>>>>> yesterday"
> > > >>>>>>>>>>>>>>>>>>> - done and satisfied, users submit it to
> > > >>>>>> production
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with
> > > >> (k1=v1,
> > > >>>>>> k2=v2),
> > > >>>>>>>> I think
> > > >>>>>>>>>>>>> it's
> > > >>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>> nice-to-have feature, but not a
> > > >>> strategically
> > > >>>>>>>> critical,
> > > >>>>>>>>>>>>>> long-term solution,
> > > >>>>>>>>>>>>>>>>>>> because
> > > >>>>>>>>>>>>>>>>>>> 1) It may seem promising at the current
> > > >>> stage to
> > > >>>>>>>> solve the
> > > >>>>>>>>>>>>>>>>>>> too-much-manual-work problem, but that's
> > > >>> only
> > > >>>>>>>> because Flink
> > > >>>>>>>>>>>>>> hasn't
> > > >>>>>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3
> > > >>> types
> > > >>>>>> of
> > > >>>>>>>> params
> > > >>>>>>>>>>>> above
> > > >>>>>>>>>>>>>> properly.
> > > >>>>>>>>>>>>>>>>>>> Once we get the params types right, the
> > > >>> LIKE
> > > >>>>>> syntax
> > > >>>>>>>> won't
> > > >>>>>>>>>>> be
> > > >>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>> important, and will be just an easier way
> > > >>> to
> > > >>>>>> create
> > > >>>>>>>> tables
> > > >>>>>>>>>>>>>> without retyping
> > > >>>>>>>>>>>>>>>>>>> long fields like username and pwd.
> > > >>>>>>>>>>>>>>>>>>> 2) Note that only some rare type of
> > > >>> catalog can
> > > >>>>>>>> store k-v
> > > >>>>>>>>>>>>>> property pair, so
> > > >>>>>>>>>>>>>>>>>>> table created this way often cannot be
> > > >>>>>> persisted. In
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>> foreseeable
> > > >>>>>>>>>>>>>>>>>>> future, such catalog will only be
> > > >>> HiveCatalog,
> > > >>>>>> and
> > > >>>>>>>> not
> > > >>>>>>>>>>>> everyone
> > > >>>>>>>>>>>>>> has a Hive
> > > >>>>>>>>>>>>>>>>>>> metastore. To be honest, without
> > > >>> persistence,
> > > >>>>>>>> recreating
> > > >>>>>>>>>>>> tables
> > > >>>>>>>>>>>>>> every time
> > > >>>>>>>>>>>>>>>>>>> this way is still a lot of keyboard
> > > >> typing.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Cheers,
> > > >>>>>>>>>>>>>>>>>>> Bowen
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt
> > > >> Young
> > > >>> <
> > > >>>>>>>>>>> ykt836@gmail.com
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> If a specific connector want to have
> > > >> such
> > > >>>>>>>> parameter and
> > > >>>>>>>>>>>> read
> > > >>>>>>>>>>>>>> if out of
> > > >>>>>>>>>>>>>>>>>>>> configuration, then that's fine.
> > > >>>>>>>>>>>>>>>>>>>> If we are talking about a configuration
> > > >>> for all
> > > >>>>>>>> kinds of
> > > >>>>>>>>>>>>>> sources, I would
> > > >>>>>>>>>>>>>>>>>>>> be super careful about that.
> > > >>>>>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases,
> > > >>> but it
> > > >>>>>>>> will also
> > > >>>>>>>>>>>> make
> > > >>>>>>>>>>>>>> the left 20%
> > > >>>>>>>>>>>>>>>>>>>> feels weird.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>> Kurt
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark
> > > >> Wu
> > > >>> <
> > > >>>>>>>>>>> imjark@gmail.com
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Hi Kurt,
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
> > > >>>>>>>>>>>>>>>>>>>>> I'm not saying to use the global
> > > >>>>>> configuration to
> > > >>>>>>>>>>>> override
> > > >>>>>>>>>>>>>> connector
> > > >>>>>>>>>>>>>>>>>>>>> properties by the planner.
> > > >>>>>>>>>>>>>>>>>>>>> But the connector should take this
> > > >>>>>> configuration
> > > >>>>>>>> and
> > > >>>>>>>>>>>>>> translate into their
> > > >>>>>>>>>>>>>>>>>>>>> client API.
> > > >>>>>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues
> > > >>> support
> > > >>>>>>>> eariliest
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>> latest and a
> > > >>>>>>>>>>>>>>>>>>>>> timestamp value as start point.
> > > >>>>>>>>>>>>>>>>>>>>> So we can support 3 options for this
> > > >>>>>>>> configuration:
> > > >>>>>>>>>>>>>> "eariliest", "latest"
> > > >>>>>>>>>>>>>>>>>>>>> and a timestamp string value.
> > > >>>>>>>>>>>>>>>>>>>>> Of course, this can't solve 100%
> > > >>> cases, but I
> > > >>>>>>>> guess can
> > > >>>>>>>>>>>>>> sovle 80% or 90%
> > > >>>>>>>>>>>>>>>>>>>>> cases.
> > > >>>>>>>>>>>>>>>>>>>>> And the remaining cases can be
> > > >>> resolved by
> > > >>>>>> LIKE
> > > >>>>>>>> syntax
> > > >>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>> I guess is
> > > >>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>> very common cases.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>>> Jark
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt
> > > >>> Young <
> > > >>>>>>>>>>>> ykt836@gmail.com
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Good to have such lovely
> > > >>> discussions. I
> > > >>>>>> also
> > > >>>>>>>> want to
> > > >>>>>>>>>>>>> share
> > > >>>>>>>>>>>>>> some of my
> > > >>>>>>>>>>>>>>>>>>>>>> opinions.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I
> > > >>> also
> > > >>>>>> think
> > > >>>>>>>> ignore
> > > >>>>>>>>>>>>>> invalid hints would
> > > >>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>> dangerous, maybe
> > > >>>>>>>>>>>>>>>>>>>>>> the simplest solution is just throw
> > > >>> an
> > > >>>>>>>> exception.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> #2 Regarding to property
> > > >>> replacement: I
> > > >>>>>> don't
> > > >>>>>>>> think
> > > >>>>>>>>>>> we
> > > >>>>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>>>>>>> constraint
> > > >>>>>>>>>>>>>>>>>>>>>> ourself to
> > > >>>>>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
> > > >>>>>> forbidden
> > > >>>>>>>> it
> > > >>>>>>>>>>>>> modifying
> > > >>>>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>>>>> properties
> > > >>>>>>>>>>>>>>>>>>>>>> which can effect
> > > >>>>>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is
> > > >>> one of
> > > >>>>>> the
> > > >>>>>>>> table
> > > >>>>>>>>>>>>> hints,
> > > >>>>>>>>>>>>>> and a
> > > >>>>>>>>>>>>>>>>>>>> powerful
> > > >>>>>>>>>>>>>>>>>>>>>> one. It can
> > > >>>>>>>>>>>>>>>>>>>>>> modify properties located in DDL's
> > > >>> WITH
> > > >>>>>> block.
> > > >>>>>>>> But I
> > > >>>>>>>>>>>> also
> > > >>>>>>>>>>>>>> see the harm
> > > >>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>> if we make it
> > > >>>>>>>>>>>>>>>>>>>>>> too flexible like change the kafka
> > > >>> topic
> > > >>>>>> name
> > > >>>>>>>> with a
> > > >>>>>>>>>>>>> hint.
> > > >>>>>>>>>>>>>> Such use
> > > >>>>>>>>>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>> not common and
> > > >>>>>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I
> > > >> would
> > > >>>>>> propose
> > > >>>>>>>> we have
> > > >>>>>>>>>>> a
> > > >>>>>>>>>>>>> map
> > > >>>>>>>>>>>>>> of hintable
> > > >>>>>>>>>>>>>>>>>>>>>> properties for each
> > > >>>>>>>>>>>>>>>>>>>>>> connector, and should validate all
> > > >>> passed
> > > >>>>>> in
> > > >>>>>>>>>>> properties
> > > >>>>>>>>>>>>>> are actually
> > > >>>>>>>>>>>>>>>>>>>>>> hintable. And combining with
> > > >>>>>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
> > > >>>>>> exception
> > > >>>>>>>> once
> > > >>>>>>>>>>>>> received
> > > >>>>>>>>>>>>>> invalid
> > > >>>>>>>>>>>>>>>>>>>>>> property.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm
> > > >>> not sure
> > > >>>>>>>> it's
> > > >>>>>>>>>>>>> feasible.
> > > >>>>>>>>>>>>>> Different
> > > >>>>>>>>>>>>>>>>>>>>>> connectors will have totally
> > > >>>>>>>>>>>>>>>>>>>>>> different properties to represent
> > > >>> offset,
> > > >>>>>> some
> > > >>>>>>>> might
> > > >>>>>>>>>>> be
> > > >>>>>>>>>>>>>> timestamps,
> > > >>>>>>>>>>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>>>>>>> might be string literals
> > > >>>>>>>>>>>>>>>>>>>>>> like "earliest", and others might
> > > >> be
> > > >>> just
> > > >>>>>>>> integers.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>>>> Kurt
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM
> > > >>> Jark Wu <
> > > >>>>>>>>>>>>> imjark@gmail.com>
> > > >>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 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 <
> > > >>>>>>>>>>>>>> twalthr@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 <
> > > >>>>>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
> > > >>>>>>>>>>>>>> twalthr@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
> > > >>>>>>>>>>>>> <
> > > >>>>>>>>>>>>>> imjark@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 <
> > > >>>>>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>
> > > >>
> > > >
> > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jark Wu <im...@gmail.com>.
Hi Danny,

Regarding to `supportedHintOptions()` interface, I suggest to use the
inverted version, `unsupportedHintOptions()`.
Because I think the disallowed list is much smaller.
In addition, it's hard to list all the properties under
`connector.properties.*`.
But we know `connector.properties.bootstrap.servers` and
`connector.properties.zookeeper.connect` are the only security options.

Best,
Jark

On Thu, 26 Mar 2020 at 16:47, Kurt Young <yk...@gmail.com> wrote:

> Hi Danny,
>
> Thanks for the updates. I have 2 comments regarding to latest document:
>
> 1) I think we also need `*supportedHintOptions*` for
> `*TableFormatFactory*`
> 2) IMO "dynamic-table-options.enabled" should belong to `
> *OptimizerConfigOptions*`
>
> Best,
> Kurt
>
>
> On Thu, Mar 26, 2020 at 4:40 PM Timo Walther <tw...@apache.org> wrote:
>
> > Thanks for the update Danny. +1 for this proposal.
> >
> > Regards,
> > Timo
> >
> > On 26.03.20 04:51, Danny Chan wrote:
> > > Thanks everyone who engaged in this discussion ~
> > >
> > > Our goal is "Supports Dynamic Table Options for Flink SQL". After an
> > > offline discussion with Kurt, Timo and Dawid, we have made the final
> > > conclusion, here is the summary:
> > >
> > >
> > >     - Use comment style syntax to specify the dynamic table options:
> "/*+
> > >     *OPTIONS*(k1='v1', k2='v2') */"
> > >     - Have constraint on the options keys: the options that may bring
> in
> > >     security problems should not be allowed, i.e. Kafka connector
> > zookeeper
> > >     endpoint URL and topic name
> > >     - Use white-list to control the allowed options for each connector,
> > >     which is more safe for future extention
> > >     - We allow to enable/disable this feature globally
> > >     - Implement based on the current code base first, and when FLIP-95
> is
> > >     checked in, implement this feature based on new interface
> > >
> > > Any suggestions are appreciated ~
> > >
> > > [1]
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
> > >
> > > Best,
> > > Danny Chan
> > >
> > > Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:
> > >
> > >> Hi everyone,
> > >>
> > >> Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid
> it
> > >> doesn't solve the problems but increases some development and learning
> > >> burdens.
> > >>
> > >> # increase development and learning burden
> > >>
> > >> According to the discussion so far, we want to support overriding a
> > subset
> > >> of options in hints which doesn't affect semantics.
> > >> With the `supportedHintOptions`, it's up to the connector developers
> to
> > >> decide which options will not affect semantics, and to be hint
> options.
> > >> However, the question is how to distinguish whether an option will
> > *affect
> > >> semantics*? What happens if an option will affect semantics but
> > provided as
> > >> hint options?
> > >>  From my point of view, it's not easy to distinguish. For example, the
> > >> "format.ignore-parse-error" can be a very useful dynamic option but
> that
> > >> will affect semantic, because the result is different (null vs
> > exception).
> > >> Another example, the "connector.lookup.cache.*" options are also very
> > >> useful to tune jobs, however, it will also affect the job results. I
> can
> > >> come up many more useful options but may affect semantics.
> > >>
> > >> I can see that the community will under endless discussion around "can
> > this
> > >> option to be a hint option?",  "wether this option will affect
> > semantics?".
> > >> You can also find that we already have different opinions on
> > >> "ignore-parse-error". Those discussion is a waste of time! That's not
> > what
> > >> users want!
> > >> The problem is user need this, this, this options and HOW to expose
> > them?
> > >> We should focus on that.
> > >>
> > >> Then there could be two endings in the future:
> > >> 1) compromise on the usability, we drop the rule that hints don't
> affect
> > >> semantics, allow all the useful options in the hints list.
> > >> 2) stick on the rule, users will find this is a stumbling feature
> which
> > >> doesn't solve their problems.
> > >>      And they will be surprised why this option can't be set, but the
> > other
> > >> could. *semantic* is hard to be understood by users.
> > >>
> > >> # doesn't solve the problems
> > >>
> > >> I think the purpose of this FLIP is to allow users to quickly override
> > some
> > >> connectors' properties to tune their jobs.
> > >> However, `supportedHintOptions` is off track. It only allows a subset
> > >> options and for the users it's not *clear* which subset is allowed.
> > >>
> > >> Besides, I'm not sure `supportedHintOptions` can work well for all
> > cases.
> > >> How could you support kafka properties (`connector.properties.*`) as
> > hint
> > >> options? Some kafka properties may affect semantics
> (bootstrap.servers),
> > >> some may not (max.poll.records). Besides, I think it's not possible to
> > list
> > >> all the possible kafka properties [1].
> > >>
> > >> In summary, IMO, `supportedHintOptions`
> > >> (1) it increase the complexity to develop a connector
> > >> (2) it confuses users which options can be used in hint, which are
> not,
> > >> they have to check the docs again and again.
> > >> (3) it doesn't solve the problems which we want to solve by this FLIP.
> > >>
> > >> I think we should avoid introducing some partial solutions. Otherwise,
> > we
> > >> will be stuck in a loop that introduce new API -> deprecate API ->
> > >> introduce new API....
> > >>
> > >> I personally in favor of an explicit WITH syntax after the table as a
> > part
> > >> of the query which is mentioned by Kurt before, e.g. SELECT * from T
> > >> WITH('key' = 'value') .
> > >> It allows users to dynamically set options which can affect semantics.
> > It
> > >> will be very flexible to solve users' problems so far.
> > >>
> > >> Best,
> > >> Jark
> > >>
> > >> [1]: https://kafka.apache.org/documentation/#consumerconfigs
> > >>
> > >> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com>
> wrote:
> > >>
> > >>> My POC is here for the hints options merge [1].
> > >>>
> > >>> Personally, I have no strong objections for splitting hints with the
> > >>> CatalogTable, the only cons is a more complex implementation but the
> > >>> concept is more clear, and I have updated the WIKI.
> > >>>
> > >>> I think it would be nice if we can support the format “ignore-parse
> > >> error”
> > >>> option key, the CSV source already has a key [2] and we can use that
> in
> > >> the
> > >>> supportedHIntOptions, for the common CSV and JSON formats, we cal
> also
> > >> give
> > >>> a support. This is the only kind of key in formats that “do not
> change
> > >> the
> > >>> semantics” (somehow), what do you think about this ~
> > >>>
> > >>> [1]
> > >>>
> > >>
> >
> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
> > >>> [2]
> > >>>
> > >>
> >
> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
> > >>>
> > >>> Best,
> > >>> Danny Chan
> > >>> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
> > >>>> Hi everyone,
> > >>>>
> > >>>> +1 to Kurt's suggestion. Let's just have it in source and sink
> > >> factories
> > >>>> for now. We can still move this method up in the future. Currently,
> I
> > >>>> don't see a need for catalogs or formats. Because how would you
> target
> > >> a
> > >>>> format in the query?
> > >>>>
> > >>>> @Danny: Can you send a link to your PoC? I'm very skeptical about
> > >>>> creating a new CatalogTable in planner. Actually CatalogTable should
> > be
> > >>>> immutable between Catalog and Factory. Because a catalog can return
> > its
> > >>>> own factory and fully control the instantiation. Depending on the
> > >>>> implementation, that means it can be possible that the catalog has
> > >>>> encoded more information in a concrete subclass implementing the
> > >>>> interface. I vote for separating the concerns of catalog information
> > >> and
> > >>>> hints in the factory explicitly.
> > >>>>
> > >>>> Regards,
> > >>>> Timo
> > >>>>
> > >>>>
> > >>>> On 18.03.20 05:41, Jingsong Li wrote:
> > >>>>> Hi,
> > >>>>>
> > >>>>> I am thinking we can provide hints to *table* related instances.
> > >>>>> - TableFormatFactory: of cause we need hints support, there are
> many
> > >>> format
> > >>>>> options in DDL too.
> > >>>>> - catalog and module: I don't know, maybe in future we can provide
> > >> some
> > >>>>> hints for them.
> > >>>>>
> > >>>>> Best,
> > >>>>> Jingsong Lee
> > >>>>>
> > >>>>> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com>
> > >>> wrote:
> > >>>>>
> > >>>>>> Yes, I think we should move the `supportedHintOptions` from
> > >>> TableFactory
> > >>>>>> to TableSourceFactory, and we also need to add the interface to
> > >>>>>> TableSinkFactory though because sink target table may also have
> > >> hints
> > >>>>>> attached.
> > >>>>>>
> > >>>>>> Best,
> > >>>>>> Danny Chan
> > >>>>>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> > >>>>>>> Have one question for adding `supportedHintOptions` method to
> > >>>>>>> `TableFactory`. It seems
> > >>>>>>> `TableFactory` is a base factory interface for all *table module*
> > >>> related
> > >>>>>>> instances, such as
> > >>>>>>> catalog, module, format and so on. It's not created only for
> > >>> *table*. Is
> > >>>>>> it
> > >>>>>>> possible to move it
> > >>>>>>> to `TableSourceFactory`?
> > >>>>>>>
> > >>>>>>> Best,
> > >>>>>>> Kurt
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
> > >> yuzhao.cyz@gmail.com>
> > >>>>>> wrote:
> > >>>>>>>
> > >>>>>>>> Thanks Timo ~
> > >>>>>>>>
> > >>>>>>>> For the naming itself, I also think the PROPERTIES is not that
> > >>>>>> concise, so
> > >>>>>>>> +1 for OPTIONS (I had thought about that, but there are many
> > >>> codes in
> > >>>>>>>> current Flink called it properties, i.e. the
> > >>> DescriptorProperties,
> > >>>>>>>> #getSupportedProperties), let’s use OPTIONS if this is our new
> > >>>>>> preference.
> > >>>>>>>>
> > >>>>>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
> > >>>>>> ConfigOption
> > >>>>>>>> can take more info. AFAIK, Spark also call their table options
> > >>> instead
> > >>>>>> of
> > >>>>>>>> properties. [1]
> > >>>>>>>>
> > >>>>>>>> In my local POC, I did create a new CatalogTable, and it works
> > >>> for
> > >>>>>> current
> > >>>>>>>> connectors well, all the DDL tables would finally yield a
> > >>> CatalogTable
> > >>>>>>>> instance and we can apply the options to that(in the
> > >>> CatalogSourceTable
> > >>>>>>>> when we generating the TableSource), the pros is that we do not
> > >>> need to
> > >>>>>>>> modify the codes of connectors itself. If we split the options
> > >>> from
> > >>>>>>>> CatalogTable, we may need to add some additional logic in each
> > >>>>>> connector
> > >>>>>>>> factories in order to merge these properties (and the logic are
> > >>> almost
> > >>>>>> the
> > >>>>>>>> same), what do you think about this?
> > >>>>>>>>
> > >>>>>>>> [1]
> > >>>>>>>>
> > >>>>>>
> > >>>
> > >>
> >
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> > >>>>>>>>
> > >>>>>>>> Best,
> > >>>>>>>> Danny Chan
> > >>>>>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
> > >>> ,写道:
> > >>>>>>>>> Hi Danny,
> > >>>>>>>>>
> > >>>>>>>>> thanks for updating the FLIP. I think your current design is
> > >>>>>> sufficient
> > >>>>>>>>> to separate hints from result-related properties.
> > >>>>>>>>>
> > >>>>>>>>> One remark to the naming itself: I would vote for calling the
> > >>> hints
> > >>>>>>>>> around table scan `OPTIONS('k'='v')`. We used the term
> > >>> "properties"
> > >>>>>> in
> > >>>>>>>>> the past but since we want to unify the Flink configuration
> > >>>>>> experience,
> > >>>>>>>>> we should use consistent naming and classes around
> > >>> `ConfigOptions`.
> > >>>>>>>>>
> > >>>>>>>>> It would be nice to use `Set<ConfigOption>
> > >>> supportedHintOptions();`
> > >>>>>> to
> > >>>>>>>>> start using config options instead of pure string properties.
> > >>> This
> > >>>>>> will
> > >>>>>>>>> also allow us to generate documentation in the future around
> > >>>>>> supported
> > >>>>>>>>> data types, ranges, etc. for options. At some point we would
> > >>> also
> > >>>>>> like
> > >>>>>>>>> to drop `DescriptorProperties` class. "Options" is also used
> > >>> in the
> > >>>>>>>>> documentation [1] and in the SQL/MED standard [2].
> > >>>>>>>>>
> > >>>>>>>>> Furthermore, I would still vote for separating CatalogTable
> > >>> and hint
> > >>>>>>>>> options. Otherwise the planner would need to create a new
> > >>>>>> CatalogTable
> > >>>>>>>>> instance which might not always be easy. We should offer them
> > >>> via:
> > >>>>>>>>>
> > >>>>>>>>>
> > >>> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > >>>>>>>>> ReadableConfig
> > >>>>>>>>>
> > >>>>>>>>> What do you think?
> > >>>>>>>>>
> > >>>>>>>>> Regards,
> > >>>>>>>>> Timo
> > >>>>>>>>>
> > >>>>>>>>> [1]
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>
> > >>
> >
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > >>>>>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
> > >>>>>>>>>> @Danny sounds good.
> > >>>>>>>>>>
> > >>>>>>>>>> Maybe it is worth listing all the classes of problems that
> > >>> you
> > >>>>>> want to
> > >>>>>>>>>> address and then look at each class and see if hints are a
> > >>> good
> > >>>>>> default
> > >>>>>>>>>> solution or a good optional way of simplifying things?
> > >>>>>>>>>> The discussion has grown a lot and it is starting to be
> > >> hard
> > >>> to
> > >>>>>>>> distinguish
> > >>>>>>>>>> the parts where everyone agrees from the parts were there
> > >> are
> > >>>>>> concerns.
> > >>>>>>>>>>
> > >>>>>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
> > >>> danny0405@apache.org>
> > >>>>>>>> wrote:
> > >>>>>>>>>>
> > >>>>>>>>>>> Thanks Stephan ~
> > >>>>>>>>>>>
> > >>>>>>>>>>> We can remove the support for properties that may change
> > >>> the
> > >>>>>>>> semantics of
> > >>>>>>>>>>> query if you think that is a trouble.
> > >>>>>>>>>>>
> > >>>>>>>>>>> How about we support the /*+ properties() */ hint only
> > >> for
> > >>> those
> > >>>>>>>> optimize
> > >>>>>>>>>>> parameters, such as the fetch size of source or something
> > >>> like
> > >>>>>> that,
> > >>>>>>>> does
> > >>>>>>>>>>> that make sense?
> > >>>>>>>>>>>
> > >>>>>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> I think Bowen has actually put it very well.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> (1) Hints that change semantics looks like trouble
> > >>> waiting to
> > >>>>>>>> happen. For
> > >>>>>>>>>>>> example Kafka offset handling should be in filters. The
> > >>> Kafka
> > >>>>>>>> source
> > >>>>>>>>>>> should
> > >>>>>>>>>>>> support predicate pushdown.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> (2) Hints should not be a workaround for current
> > >>> shortcomings.
> > >>>>>> A
> > >>>>>>>> lot of
> > >>>>>>>>>>> the
> > >>>>>>>>>>>> suggested above sounds exactly like that. Working
> > >> around
> > >>>>>>>> catalog/DDL
> > >>>>>>>>>>>> shortcomings, missing exposure of metadata (offsets),
> > >>> missing
> > >>>>>>>> predicate
> > >>>>>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as
> > >> a
> > >>> quick
> > >>>>>> fix
> > >>>>>>>> for
> > >>>>>>>>>>>> these issues, rather than fixing the root causes, will
> > >>> much
> > >>>>>> likely
> > >>>>>>>> bite
> > >>>>>>>>>>> us
> > >>>>>>>>>>>> back badly in the future.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Best,
> > >>>>>>>>>>>> Stephan
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
> > >>> ykt836@gmail.com>
> > >>>>>>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>> It seems this FLIP's name is somewhat misleading.
> > >> From
> > >>> my
> > >>>>>>>>>>> understanding,
> > >>>>>>>>>>>>> this FLIP is trying to
> > >>>>>>>>>>>>> address the dynamic parameter issue, and table hints
> > >>> is the
> > >>>>>> way
> > >>>>>>>> we wan
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>> choose. I think we should
> > >>>>>>>>>>>>> be focus on "what's the right way to solve dynamic
> > >>> property"
> > >>>>>>>> instead of
> > >>>>>>>>>>>>> discussing "whether table
> > >>>>>>>>>>>>> hints can affect query semantics".
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> For now, there are two proposed ways to achieve
> > >> dynamic
> > >>>>>> property:
> > >>>>>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with
> > >>> (xxx)
> > >>>>>>>>>>>>> 2. use custom "from t with (xxx)" syntax
> > >>>>>>>>>>>>> 3. "Borrow" the table hints to have a special
> > >>> PROPERTIES
> > >>>>>> hint.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> The first one didn't break anything, but the only
> > >>> problem i
> > >>>>>> see
> > >>>>>>>> is a
> > >>>>>>>>>>>> little
> > >>>>>>>>>>>>> more verbose than the table hint
> > >>>>>>>>>>>>> approach. I can imagine when someone using SQL CLI to
> > >>> have a
> > >>>>>> sql
> > >>>>>>>>>>>>> experience, it's quite often that
> > >>>>>>>>>>>>> he will modify the table property, some use cases i
> > >> can
> > >>>>>> think of:
> > >>>>>>>>>>>>> 1. the source contains some corrupted data, i want to
> > >>> turn
> > >>>>>> on the
> > >>>>>>>>>>>>> "ignore-error" flag for certain formats.
> > >>>>>>>>>>>>> 2. I have a kafka table and want to see some sample
> > >>> data
> > >>>>>> from the
> > >>>>>>>>>>>>> beginning, so i change the offset
> > >>>>>>>>>>>>> to "earliest", and then I want to observe the latest
> > >>> data
> > >>>>>> which
> > >>>>>>>> keeps
> > >>>>>>>>>>>>> coming in. I would write another query
> > >>>>>>>>>>>>> to select from the latest table.
> > >>>>>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly
> > >> then
> > >>> i can
> > >>>>>>>> observe
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>> data from database side.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Most of such use cases are quite ad-hoc. If every
> > >> time
> > >>> I
> > >>>>>> want to
> > >>>>>>>> have a
> > >>>>>>>>>>>>> different experience, i need to create
> > >>>>>>>>>>>>> a temporary table and then also modify my query, it
> > >>> doesn't
> > >>>>>> feel
> > >>>>>>>>>>> smooth.
> > >>>>>>>>>>>>> Embed such dynamic property into
> > >>>>>>>>>>>>> query would have better user experience.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is
> > >>> breaking
> > >>>>>> SQL
> > >>>>>>>>>>>> compliant,
> > >>>>>>>>>>>>> and for #3, it only breaks some
> > >>>>>>>>>>>>> unwritten rules, but we can have an explanation on
> > >>> that. And
> > >>>>>> I
> > >>>>>>>> really
> > >>>>>>>>>>>> doubt
> > >>>>>>>>>>>>> whether user would complain about
> > >>>>>>>>>>>>> this when they actually have flexible and good
> > >>> experience
> > >>>>>> using
> > >>>>>>>> this.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>> Kurt
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> > >>>>>> yuzhao.cyz@gmail.com
> > >>>>>>>>>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thanks Aljoscha ~
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> I agree for most of the query hints, they are
> > >>> optional as
> > >>>>>> an
> > >>>>>>>>>>> optimizer
> > >>>>>>>>>>>>>> instruction, especially for the traditional RDBMS.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> But, just like BenChao said, Flink as a computation
> > >>> engine
> > >>>>>> has
> > >>>>>>>> many
> > >>>>>>>>>>>>>> different kind of data sources, thus, dynamic
> > >>> parameters
> > >>>>>> like
> > >>>>>>>>>>>>> start_offest
> > >>>>>>>>>>>>>> can only bind to each table scope, we can not set a
> > >>> session
> > >>>>>>>> config
> > >>>>>>>>>>> like
> > >>>>>>>>>>>>>> KSQL because they are all about Kafka:
> > >>>>>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thus the most flexible way to set up these dynamic
> > >>> params
> > >>>>>> is
> > >>>>>>>> to bind
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>>> the table scope in the query when we want to
> > >> override
> > >>>>>>>> something, so
> > >>>>>>>>>>> we
> > >>>>>>>>>>>>> have
> > >>>>>>>>>>>>>> these solutions above (with pros and cons from my
> > >>> side):
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>> - Easy to add
> > >>>>>>>>>>>>>> - Parameters are part of the main query
> > >>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>> - Not SQL compliant
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */
> > >>> (from
> > >>>>>> me)
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>> - Easy to add
> > >>>>>>>>>>>>>> - SQL compliant because it is nested in the
> > >> comments
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>> - Parameters are not part of the main query
> > >>>>>>>>>>>>>> - Cryptic syntax for new users
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> The biggest problem for hints way may be the “if
> > >>> hints
> > >>>>>> must be
> > >>>>>>>>>>>> optional”,
> > >>>>>>>>>>>>>> actually we have though about 1 for a while but
> > >>> aborted
> > >>>>>>>> because it
> > >>>>>>>>>>>> breaks
> > >>>>>>>>>>>>>> the SQL standard too much. And we replace it with
> > >> 2,
> > >>>>>> because
> > >>>>>>>> the
> > >>>>>>>>>>> hints
> > >>>>>>>>>>>>>> syntax do not break SQL standard(nested in
> > >> comments).
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint
> > >>> that
> > >>>>>> allows
> > >>>>>>>>>>> override
> > >>>>>>>>>>>>>> some properties of table dynamically, it does not
> > >>> break
> > >>>>>>>> anything, at
> > >>>>>>>>>>>>> lease
> > >>>>>>>>>>>>>> for current Flink use cases.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Planner hints are optional just because they are
> > >>> naturally
> > >>>>>>>> enforcers
> > >>>>>>>>>>> of
> > >>>>>>>>>>>>>> the planner, most of them aim to instruct the
> > >>> optimizer,
> > >>>>>> but,
> > >>>>>>>> the
> > >>>>>>>>>>> table
> > >>>>>>>>>>>>>> hints is a little different, table hints can
> > >> specify
> > >>> the
> > >>>>>> table
> > >>>>>>>> meta
> > >>>>>>>>>>>> like
> > >>>>>>>>>>>>>> index column, and it is very convenient to specify
> > >>> table
> > >>>>>>>> properties.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */
> > >>> table
> > >>>>>> hint,
> > >>>>>>>> we
> > >>>>>>>>>>> can
> > >>>>>>>>>>>>>> call it table dynamic parameters.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>> Danny Chan
> > >>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> > >>>>>>>> aljoscha@apache.org>,写道:
> > >>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I don't understand this discussion. Hints, as I
> > >>>>>> understand
> > >>>>>>>> them,
> > >>>>>>>>>>>> should
> > >>>>>>>>>>>>>>> work like this:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> - hints are *optional* advice for the optimizer
> > >> to
> > >>> try
> > >>>>>> and
> > >>>>>>>> help it
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>>>> find a good execution strategy
> > >>>>>>>>>>>>>>> - hints should not change query semantics, i.e.
> > >>> they
> > >>>>>> should
> > >>>>>>>> not
> > >>>>>>>>>>>> change
> > >>>>>>>>>>>>>>> connector properties executing a query with
> > >> taking
> > >>> into
> > >>>>>>>> account the
> > >>>>>>>>>>>>>>> hints *must* produce the same result as executing
> > >>> the
> > >>>>>> query
> > >>>>>>>> without
> > >>>>>>>>>>>>>>> taking into account the hints
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>  From these simple requirements you can derive a
> > >>> solution
> > >>>>>>>> that makes
> > >>>>>>>>>>>>>>> sense. I don't have a strong preference for the
> > >>> syntax
> > >>>>>> but we
> > >>>>>>>>>>> should
> > >>>>>>>>>>>>>>> strive to be in line with prior work.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>> Aljoscha
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
> > >>>>>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I agree with Kurt that option2 is too
> > >>> complicated to
> > >>>>>> use
> > >>>>>>>> because:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> • As a Kafka topic consumer, the user must
> > >>> define both
> > >>>>>> the
> > >>>>>>>>>>> virtual
> > >>>>>>>>>>>>>> column for start offset and he must apply a special
> > >>> filter
> > >>>>>>>> predicate
> > >>>>>>>>>>>>> after
> > >>>>>>>>>>>>>> each query
> > >>>>>>>>>>>>>>>> • And for the internal implementation, the
> > >>> metadata
> > >>>>>> column
> > >>>>>>>> push
> > >>>>>>>>>>>> down
> > >>>>>>>>>>>>>> is another hard topic, each kind of message queue
> > >>> may have
> > >>>>>> its
> > >>>>>>>> offset
> > >>>>>>>>>>>>>> attribute, we need to consider the expression type
> > >>> for
> > >>>>>>>> different
> > >>>>>>>>>>> kind;
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>> source also need to recognize the constant column
> > >> as
> > >>> a
> > >>>>>> config
> > >>>>>>>>>>>>> option(which
> > >>>>>>>>>>>>>> is weird because usually what we pushed down is a
> > >>> table
> > >>>>>> column)
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> For option 1 and option3, I think there is no
> > >>>>>> difference,
> > >>>>>>>> option1
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>> also a hint syntax which is introduced in Sybase
> > >> and
> > >>>>>>>> referenced then
> > >>>>>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
> > >>>>>>>> ambitiousness.
> > >>>>>>>>>>>>> Personally
> > >>>>>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword
> > >>> for
> > >>>>>> these
> > >>>>>>>> reasons:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> • We do not break the standard SQL, the hints
> > >> are
> > >>>>>> nested
> > >>>>>>>> in SQL
> > >>>>>>>>>>>>>> comments
> > >>>>>>>>>>>>>>>> • We do not need to introduce additional WITH
> > >>> keyword
> > >>>>>>>> which may
> > >>>>>>>>>>>>> appear
> > >>>>>>>>>>>>>> in a query if we use that because a table can be
> > >>>>>> referenced in
> > >>>>>>>> all
> > >>>>>>>>>>>> kinds
> > >>>>>>>>>>>>> of
> > >>>>>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
> > >>> make
> > >>>>>> our
> > >>>>>>>> sql
> > >>>>>>>>>>> query
> > >>>>>>>>>>>>>> break too much of the SQL from standard
> > >>>>>>>>>>>>>>>> • We would have uniform syntax for hints as
> > >> query
> > >>>>>> hint, one
> > >>>>>>>>>>> syntax
> > >>>>>>>>>>>>>> fits all and more easy to use
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> And here is the reason why we choose a uniform
> > >>> Oracle
> > >>>>>>>> style query
> > >>>>>>>>>>>>>> hint syntax which is addressed by Julian Hyde when
> > >> we
> > >>>>>> design
> > >>>>>>>> the
> > >>>>>>>>>>> syntax
> > >>>>>>>>>>>>>> from the Calcite community:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for
> > >>> table
> > >>>>>> hints.
> > >>>>>>>> It
> > >>>>>>>>>>> adds a
> > >>>>>>>>>>>>>> new use of the WITH keyword that is unrelated to
> > >> the
> > >>> use of
> > >>>>>>>> WITH for
> > >>>>>>>>>>>>>> common-table expressions.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> A historical note. Microsoft SQL Server
> > >>> inherited its
> > >>>>>> hint
> > >>>>>>>> syntax
> > >>>>>>>>>>>>> from
> > >>>>>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
> > >>>>>>>> Programming”[1], page
> > >>>>>>>>>>>>> 632,
> > >>>>>>>>>>>>>> “Optimizer hints”. The book was written in 1999,
> > >> and
> > >>> covers
> > >>>>>>>> Microsoft
> > >>>>>>>>>>>> SQL
> > >>>>>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
> > >>> but the
> > >>>>>>>> syntax very
> > >>>>>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft
> > >> SQL
> > >>>>>> Server was
> > >>>>>>>>>>> forked
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>>> 1993.)
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Microsoft later added the WITH keyword to make
> > >>> it less
> > >>>>>>>> ambiguous,
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>> has now deprecated the syntax that does not use
> > >> WITH.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> They are forced to keep the syntax for
> > >> backwards
> > >>>>>>>> compatibility
> > >>>>>>>>>>> but
> > >>>>>>>>>>>>>> that doesn’t mean that we should shoulder their
> > >>> burden.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I think formatted comments are the right
> > >>> container for
> > >>>>>>>> hints
> > >>>>>>>>>>>> because
> > >>>>>>>>>>>>>> it allows us to change the hint syntax without
> > >>> changing
> > >>>>>> the SQL
> > >>>>>>>>>>> parser,
> > >>>>>>>>>>>>> and
> > >>>>>>>>>>>>>> makes clear that we are at liberty to ignore hints
> > >>>>>> entirely.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Julian
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
> > >>>>>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>> Danny Chan
> > >>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> > >>>>>> twalthr@apache.org
> > >>>>>>>>> ,写道:
> > >>>>>>>>>>>>>>>>> Hi Danny,
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> it is true that our DDL is not standard
> > >>> compliant by
> > >>>>>>>> using the
> > >>>>>>>>>>>> WITH
> > >>>>>>>>>>>>>>>>> clause. Nevertheless, we aim for not
> > >> diverging
> > >>> too
> > >>>>>> much
> > >>>>>>>> and the
> > >>>>>>>>>>>>> LIKE
> > >>>>>>>>>>>>>>>>> clause is an example of that. It will solve
> > >>> things
> > >>>>>> like
> > >>>>>>>>>>>> overwriting
> > >>>>>>>>>>>>>>>>> WATERMARKs, add additional/modifying
> > >>> properties and
> > >>>>>>>> inherit
> > >>>>>>>>>>>> schema.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3
> > >>> types
> > >>>>>>>> definition
> > >>>>>>>>>>>>>> together.
> > >>>>>>>>>>>>>>>>> We are not the first ones that try to solve
> > >>> this.
> > >>>>>> There
> > >>>>>>>> is also
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>> SQL
> > >>>>>>>>>>>>>>>>> MED standard [1] that tried to tackle this
> > >>> problem. I
> > >>>>>>>> think it
> > >>>>>>>>>>>> was
> > >>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>> considered when designing the current DDL.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
> > >>>>>> offsets. I
> > >>>>>>>> will
> > >>>>>>>>>>>> give
> > >>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>> examples and look forward to feedback here:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part
> > >>> of the
> > >>>>>>>> query
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>>>>> - Easy to add
> > >>>>>>>>>>>>>>>>> - Parameters are part of the main query
> > >>>>>>>>>>>>>>>>> - No complicated hinting syntax
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>>>>> - Not SQL compliant
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> *Option 2* Use metadata in query
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
> > >>>>>>>>>>>>> SYSTEM_METADATA('offset'))`
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset >
> > >> TIMESTAMP
> > >>>>>>>> '2012-12-12
> > >>>>>>>>>>>>>> 12:34:22'`
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>>>>> - SQL compliant in the query
> > >>>>>>>>>>>>>>>>> - Access of metadata in the DDL which is
> > >>> required
> > >>>>>> anyway
> > >>>>>>>>>>>>>>>>> - Regular pushdown rules apply
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>>>>> - Users need to add an additional comlumn in
> > >>> the DDL
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> *Option 3*: Use hints for properties
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> `
> > >>>>>>>>>>>>>>>>> SELECT *
> > >>>>>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
> > >>>>>>>>>>>>>>>>> `
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Pros:
> > >>>>>>>>>>>>>>>>> - Easy to add
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Cons:
> > >>>>>>>>>>>>>>>>> - Parameters are not part of the main query
> > >>>>>>>>>>>>>>>>> - Cryptic syntax for new users
> > >>>>>>>>>>>>>>>>> - Not standard compliant.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> If we go with this option, I would suggest to
> > >>> make it
> > >>>>>>>> available
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>> separate map and don't mix it with statically
> > >>> defined
> > >>>>>>>>>>> properties.
> > >>>>>>>>>>>>>> Such
> > >>>>>>>>>>>>>>>>> that the factory can decide which properties
> > >>> have the
> > >>>>>>>> right to
> > >>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>> overwritten by the hints:
> > >>>>>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
> > >>>>>>>> ReadableConfig
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>> Timo
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Currently I see 3 options as a
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
> > >>>>>>>>>>>>>>>>>> Thanks Bowen ~
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I agree we should somehow categorize our
> > >>> connector
> > >>>>>>>>>>> parameters.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> For type1, I’m already preparing a solution
> > >>> like
> > >>>>>> the
> > >>>>>>>>>>> Confluent
> > >>>>>>>>>>>>>> schema registry + Avro schema inference thing, so
> > >>> this may
> > >>>>>> not
> > >>>>>>>> be a
> > >>>>>>>>>>>>> problem
> > >>>>>>>>>>>>>> in the near future.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> For type3, I have some questions:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset >
> > >> 12pm
> > >>>>>> yesterday”
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Where does the offset column come from, a
> > >>> virtual
> > >>>>>>>> column from
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>> table schema, you said that
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> They change
> > >>>>>>>>>>>>>>>>>> almost every time a query starts and have
> > >>> nothing
> > >>>>>> to
> > >>>>>>>> do with
> > >>>>>>>>>>>>>> metadata, thus
> > >>>>>>>>>>>>>>>>>> should not be part of table definition/DDL
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> But why you can reference it in the query,
> > >>> I’m
> > >>>>>>>> confused for
> > >>>>>>>>>>>> that,
> > >>>>>>>>>>>>>> can you elaborate a little ?
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>> Danny Chan
> > >>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> > >>>>>>>> bowenli86@gmail.com
> > >>>>>>>>>>>> ,写道:
> > >>>>>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> The root cause of too much manual work is
> > >>> Flink
> > >>>>>> DDL
> > >>>>>>>> has
> > >>>>>>>>>>>> mixed 3
> > >>>>>>>>>>>>>> types of
> > >>>>>>>>>>>>>>>>>>> params together and doesn't handle each
> > >> of
> > >>> them
> > >>>>>> very
> > >>>>>>>> well.
> > >>>>>>>>>>>>> Below
> > >>>>>>>>>>>>>> are how I
> > >>>>>>>>>>>>>>>>>>> categorize them and corresponding
> > >>> solutions in my
> > >>>>>>>> mind:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
> > >>>>>> external
> > >>>>>>>>>>>>> endpoint/url,
> > >>>>>>>>>>>>>>>>>>> username/pwd, schemas, formats.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Such metadata are mostly already
> > >>> accessible in
> > >>>>>>>> external
> > >>>>>>>>>>>> system
> > >>>>>>>>>>>>>> as long as
> > >>>>>>>>>>>>>>>>>>> endpoints and credentials are provided.
> > >>> Flink can
> > >>>>>>>> get it
> > >>>>>>>>>>> thru
> > >>>>>>>>>>>>>> catalogs, but
> > >>>>>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus
> > >>> Flink
> > >>>>>> just
> > >>>>>>>> hasn't
> > >>>>>>>>>>>>> been
> > >>>>>>>>>>>>>> able to
> > >>>>>>>>>>>>>>>>>>> leverage that. So the solution should be
> > >>> building
> > >>>>>>>> more
> > >>>>>>>>>>>>> catalogs.
> > >>>>>>>>>>>>>> Such
> > >>>>>>>>>>>>>>>>>>> params should be part of a Flink table
> > >>>>>>>> DDL/definition, and
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>> overridable
> > >>>>>>>>>>>>>>>>>>> in any means.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc
> > >>> connector's
> > >>>>>>>> fetch size,
> > >>>>>>>>>>>>>> elasticsearch
> > >>>>>>>>>>>>>>>>>>> connector's bulk flush size.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Such params don't affect query results,
> > >> but
> > >>>>>> affect
> > >>>>>>>> how
> > >>>>>>>>>>>> results
> > >>>>>>>>>>>>>> are produced
> > >>>>>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) -
> > >> they
> > >>> are
> > >>>>>>>> essentially
> > >>>>>>>>>>>>>> execution and
> > >>>>>>>>>>>>>>>>>>> implementation details. They change often
> > >>> in
> > >>>>>>>> exploration or
> > >>>>>>>>>>>>>> development
> > >>>>>>>>>>>>>>>>>>> stages, but not quite frequently in
> > >>> well-defined
> > >>>>>>>>>>> long-running
> > >>>>>>>>>>>>>> pipelines.
> > >>>>>>>>>>>>>>>>>>> They should always have default values
> > >> and
> > >>> can be
> > >>>>>>>> missing
> > >>>>>>>>>>> in
> > >>>>>>>>>>>>>> query. They
> > >>>>>>>>>>>>>>>>>>> can be part of a table DDL/definition,
> > >> but
> > >>> should
> > >>>>>>>> also be
> > >>>>>>>>>>>>>> replaceable in a
> > >>>>>>>>>>>>>>>>>>> query - *this is what table "hints" in
> > >>> FLIP-113
> > >>>>>>>> should
> > >>>>>>>>>>>> cover*.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka
> > >>> connector's
> > >>>>>>>> start
> > >>>>>>>>>>>> offset.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Such params affect query results - the
> > >>> semantics.
> > >>>>>>>> They'd
> > >>>>>>>>>>>> better
> > >>>>>>>>>>>>>> be as
> > >>>>>>>>>>>>>>>>>>> filter conditions in WHERE clause that
> > >> can
> > >>> be
> > >>>>>> pushed
> > >>>>>>>> down.
> > >>>>>>>>>>>> They
> > >>>>>>>>>>>>>> change
> > >>>>>>>>>>>>>>>>>>> almost every time a query starts and have
> > >>>>>> nothing to
> > >>>>>>>> do
> > >>>>>>>>>>> with
> > >>>>>>>>>>>>>> metadata, thus
> > >>>>>>>>>>>>>>>>>>> should not be part of table
> > >>> definition/DDL, nor
> > >>>>>> be
> > >>>>>>>>>>> persisted
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>>> catalogs.
> > >>>>>>>>>>>>>>>>>>> If they will, users should create views
> > >> to
> > >>> keep
> > >>>>>> such
> > >>>>>>>> params
> > >>>>>>>>>>>>>> around (note
> > >>>>>>>>>>>>>>>>>>> this is different from variable
> > >>> substitution).
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we
> > >>> get these
> > >>>>>>>> params
> > >>>>>>>>>>>> right,
> > >>>>>>>>>>>>>> here're the
> > >>>>>>>>>>>>>>>>>>> steps users need to do to develop and run
> > >>> a Flink
> > >>>>>>>> job:
> > >>>>>>>>>>>>>>>>>>> - configure a Flink
> > >>> ConfluentSchemaRegistry with
> > >>>>>> url,
> > >>>>>>>>>>>> username,
> > >>>>>>>>>>>>>> and password
> > >>>>>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset
> > >>>> 12pm
> > >>>>>>>> yesterday"
> > >>>>>>>>>>>>>> (simplified
> > >>>>>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink
> > >> automatically
> > >>>>>> retrieves
> > >>>>>>>> all
> > >>>>>>>>>>>>>> metadata of
> > >>>>>>>>>>>>>>>>>>> schema, file format, etc and start the
> > >> job
> > >>>>>>>>>>>>>>>>>>> - users want to make the job read Kafka
> > >>> topic
> > >>>>>>>> faster, so it
> > >>>>>>>>>>>>> goes
> > >>>>>>>>>>>>>> as "SELECT
> > >>>>>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/
> > >>> WHERE
> > >>>>>>>> offset >
> > >>>>>>>>>>> 12pm
> > >>>>>>>>>>>>>> yesterday"
> > >>>>>>>>>>>>>>>>>>> - done and satisfied, users submit it to
> > >>>>>> production
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with
> > >> (k1=v1,
> > >>>>>> k2=v2),
> > >>>>>>>> I think
> > >>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>> nice-to-have feature, but not a
> > >>> strategically
> > >>>>>>>> critical,
> > >>>>>>>>>>>>>> long-term solution,
> > >>>>>>>>>>>>>>>>>>> because
> > >>>>>>>>>>>>>>>>>>> 1) It may seem promising at the current
> > >>> stage to
> > >>>>>>>> solve the
> > >>>>>>>>>>>>>>>>>>> too-much-manual-work problem, but that's
> > >>> only
> > >>>>>>>> because Flink
> > >>>>>>>>>>>>>> hasn't
> > >>>>>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3
> > >>> types
> > >>>>>> of
> > >>>>>>>> params
> > >>>>>>>>>>>> above
> > >>>>>>>>>>>>>> properly.
> > >>>>>>>>>>>>>>>>>>> Once we get the params types right, the
> > >>> LIKE
> > >>>>>> syntax
> > >>>>>>>> won't
> > >>>>>>>>>>> be
> > >>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> important, and will be just an easier way
> > >>> to
> > >>>>>> create
> > >>>>>>>> tables
> > >>>>>>>>>>>>>> without retyping
> > >>>>>>>>>>>>>>>>>>> long fields like username and pwd.
> > >>>>>>>>>>>>>>>>>>> 2) Note that only some rare type of
> > >>> catalog can
> > >>>>>>>> store k-v
> > >>>>>>>>>>>>>> property pair, so
> > >>>>>>>>>>>>>>>>>>> table created this way often cannot be
> > >>>>>> persisted. In
> > >>>>>>>> the
> > >>>>>>>>>>>>>> foreseeable
> > >>>>>>>>>>>>>>>>>>> future, such catalog will only be
> > >>> HiveCatalog,
> > >>>>>> and
> > >>>>>>>> not
> > >>>>>>>>>>>> everyone
> > >>>>>>>>>>>>>> has a Hive
> > >>>>>>>>>>>>>>>>>>> metastore. To be honest, without
> > >>> persistence,
> > >>>>>>>> recreating
> > >>>>>>>>>>>> tables
> > >>>>>>>>>>>>>> every time
> > >>>>>>>>>>>>>>>>>>> this way is still a lot of keyboard
> > >> typing.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>> Bowen
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt
> > >> Young
> > >>> <
> > >>>>>>>>>>> ykt836@gmail.com
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> If a specific connector want to have
> > >> such
> > >>>>>>>> parameter and
> > >>>>>>>>>>>> read
> > >>>>>>>>>>>>>> if out of
> > >>>>>>>>>>>>>>>>>>>> configuration, then that's fine.
> > >>>>>>>>>>>>>>>>>>>> If we are talking about a configuration
> > >>> for all
> > >>>>>>>> kinds of
> > >>>>>>>>>>>>>> sources, I would
> > >>>>>>>>>>>>>>>>>>>> be super careful about that.
> > >>>>>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases,
> > >>> but it
> > >>>>>>>> will also
> > >>>>>>>>>>>> make
> > >>>>>>>>>>>>>> the left 20%
> > >>>>>>>>>>>>>>>>>>>> feels weird.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>> Kurt
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark
> > >> Wu
> > >>> <
> > >>>>>>>>>>> imjark@gmail.com
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Hi Kurt,
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
> > >>>>>>>>>>>>>>>>>>>>> I'm not saying to use the global
> > >>>>>> configuration to
> > >>>>>>>>>>>> override
> > >>>>>>>>>>>>>> connector
> > >>>>>>>>>>>>>>>>>>>>> properties by the planner.
> > >>>>>>>>>>>>>>>>>>>>> But the connector should take this
> > >>>>>> configuration
> > >>>>>>>> and
> > >>>>>>>>>>>>>> translate into their
> > >>>>>>>>>>>>>>>>>>>>> client API.
> > >>>>>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues
> > >>> support
> > >>>>>>>> eariliest
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>> latest and a
> > >>>>>>>>>>>>>>>>>>>>> timestamp value as start point.
> > >>>>>>>>>>>>>>>>>>>>> So we can support 3 options for this
> > >>>>>>>> configuration:
> > >>>>>>>>>>>>>> "eariliest", "latest"
> > >>>>>>>>>>>>>>>>>>>>> and a timestamp string value.
> > >>>>>>>>>>>>>>>>>>>>> Of course, this can't solve 100%
> > >>> cases, but I
> > >>>>>>>> guess can
> > >>>>>>>>>>>>>> sovle 80% or 90%
> > >>>>>>>>>>>>>>>>>>>>> cases.
> > >>>>>>>>>>>>>>>>>>>>> And the remaining cases can be
> > >>> resolved by
> > >>>>>> LIKE
> > >>>>>>>> syntax
> > >>>>>>>>>>>>> which
> > >>>>>>>>>>>>>> I guess is
> > >>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>> very common cases.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>> Jark
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt
> > >>> Young <
> > >>>>>>>>>>>> ykt836@gmail.com
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Good to have such lovely
> > >>> discussions. I
> > >>>>>> also
> > >>>>>>>> want to
> > >>>>>>>>>>>>> share
> > >>>>>>>>>>>>>> some of my
> > >>>>>>>>>>>>>>>>>>>>>> opinions.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I
> > >>> also
> > >>>>>> think
> > >>>>>>>> ignore
> > >>>>>>>>>>>>>> invalid hints would
> > >>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>> dangerous, maybe
> > >>>>>>>>>>>>>>>>>>>>>> the simplest solution is just throw
> > >>> an
> > >>>>>>>> exception.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> #2 Regarding to property
> > >>> replacement: I
> > >>>>>> don't
> > >>>>>>>> think
> > >>>>>>>>>>> we
> > >>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>> constraint
> > >>>>>>>>>>>>>>>>>>>>>> ourself to
> > >>>>>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
> > >>>>>> forbidden
> > >>>>>>>> it
> > >>>>>>>>>>>>> modifying
> > >>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>> properties
> > >>>>>>>>>>>>>>>>>>>>>> which can effect
> > >>>>>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is
> > >>> one of
> > >>>>>> the
> > >>>>>>>> table
> > >>>>>>>>>>>>> hints,
> > >>>>>>>>>>>>>> and a
> > >>>>>>>>>>>>>>>>>>>> powerful
> > >>>>>>>>>>>>>>>>>>>>>> one. It can
> > >>>>>>>>>>>>>>>>>>>>>> modify properties located in DDL's
> > >>> WITH
> > >>>>>> block.
> > >>>>>>>> But I
> > >>>>>>>>>>>> also
> > >>>>>>>>>>>>>> see the harm
> > >>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>> if we make it
> > >>>>>>>>>>>>>>>>>>>>>> too flexible like change the kafka
> > >>> topic
> > >>>>>> name
> > >>>>>>>> with a
> > >>>>>>>>>>>>> hint.
> > >>>>>>>>>>>>>> Such use
> > >>>>>>>>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>> not common and
> > >>>>>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I
> > >> would
> > >>>>>> propose
> > >>>>>>>> we have
> > >>>>>>>>>>> a
> > >>>>>>>>>>>>> map
> > >>>>>>>>>>>>>> of hintable
> > >>>>>>>>>>>>>>>>>>>>>> properties for each
> > >>>>>>>>>>>>>>>>>>>>>> connector, and should validate all
> > >>> passed
> > >>>>>> in
> > >>>>>>>>>>> properties
> > >>>>>>>>>>>>>> are actually
> > >>>>>>>>>>>>>>>>>>>>>> hintable. And combining with
> > >>>>>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
> > >>>>>> exception
> > >>>>>>>> once
> > >>>>>>>>>>>>> received
> > >>>>>>>>>>>>>> invalid
> > >>>>>>>>>>>>>>>>>>>>>> property.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm
> > >>> not sure
> > >>>>>>>> it's
> > >>>>>>>>>>>>> feasible.
> > >>>>>>>>>>>>>> Different
> > >>>>>>>>>>>>>>>>>>>>>> connectors will have totally
> > >>>>>>>>>>>>>>>>>>>>>> different properties to represent
> > >>> offset,
> > >>>>>> some
> > >>>>>>>> might
> > >>>>>>>>>>> be
> > >>>>>>>>>>>>>> timestamps,
> > >>>>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>> might be string literals
> > >>>>>>>>>>>>>>>>>>>>>> like "earliest", and others might
> > >> be
> > >>> just
> > >>>>>>>> integers.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>>> Kurt
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM
> > >>> Jark Wu <
> > >>>>>>>>>>>>> imjark@gmail.com>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 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 <
> > >>>>>>>>>>>>>> twalthr@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 <
> > >>>>>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
> > >>>>>>>>>>>>>> twalthr@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
> > >>>>>>>>>>>>> <
> > >>>>>>>>>>>>>> imjark@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 <
> > >>>>>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >
> >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Kurt Young <yk...@gmail.com>.
Hi Danny,

Thanks for the updates. I have 2 comments regarding to latest document:

1) I think we also need `*supportedHintOptions*` for  `*TableFormatFactory*`
2) IMO "dynamic-table-options.enabled" should belong to `
*OptimizerConfigOptions*`

Best,
Kurt


On Thu, Mar 26, 2020 at 4:40 PM Timo Walther <tw...@apache.org> wrote:

> Thanks for the update Danny. +1 for this proposal.
>
> Regards,
> Timo
>
> On 26.03.20 04:51, Danny Chan wrote:
> > Thanks everyone who engaged in this discussion ~
> >
> > Our goal is "Supports Dynamic Table Options for Flink SQL". After an
> > offline discussion with Kurt, Timo and Dawid, we have made the final
> > conclusion, here is the summary:
> >
> >
> >     - Use comment style syntax to specify the dynamic table options: "/*+
> >     *OPTIONS*(k1='v1', k2='v2') */"
> >     - Have constraint on the options keys: the options that may bring in
> >     security problems should not be allowed, i.e. Kafka connector
> zookeeper
> >     endpoint URL and topic name
> >     - Use white-list to control the allowed options for each connector,
> >     which is more safe for future extention
> >     - We allow to enable/disable this feature globally
> >     - Implement based on the current code base first, and when FLIP-95 is
> >     checked in, implement this feature based on new interface
> >
> > Any suggestions are appreciated ~
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
> >
> > Best,
> > Danny Chan
> >
> > Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:
> >
> >> Hi everyone,
> >>
> >> Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid it
> >> doesn't solve the problems but increases some development and learning
> >> burdens.
> >>
> >> # increase development and learning burden
> >>
> >> According to the discussion so far, we want to support overriding a
> subset
> >> of options in hints which doesn't affect semantics.
> >> With the `supportedHintOptions`, it's up to the connector developers to
> >> decide which options will not affect semantics, and to be hint options.
> >> However, the question is how to distinguish whether an option will
> *affect
> >> semantics*? What happens if an option will affect semantics but
> provided as
> >> hint options?
> >>  From my point of view, it's not easy to distinguish. For example, the
> >> "format.ignore-parse-error" can be a very useful dynamic option but that
> >> will affect semantic, because the result is different (null vs
> exception).
> >> Another example, the "connector.lookup.cache.*" options are also very
> >> useful to tune jobs, however, it will also affect the job results. I can
> >> come up many more useful options but may affect semantics.
> >>
> >> I can see that the community will under endless discussion around "can
> this
> >> option to be a hint option?",  "wether this option will affect
> semantics?".
> >> You can also find that we already have different opinions on
> >> "ignore-parse-error". Those discussion is a waste of time! That's not
> what
> >> users want!
> >> The problem is user need this, this, this options and HOW to expose
> them?
> >> We should focus on that.
> >>
> >> Then there could be two endings in the future:
> >> 1) compromise on the usability, we drop the rule that hints don't affect
> >> semantics, allow all the useful options in the hints list.
> >> 2) stick on the rule, users will find this is a stumbling feature which
> >> doesn't solve their problems.
> >>      And they will be surprised why this option can't be set, but the
> other
> >> could. *semantic* is hard to be understood by users.
> >>
> >> # doesn't solve the problems
> >>
> >> I think the purpose of this FLIP is to allow users to quickly override
> some
> >> connectors' properties to tune their jobs.
> >> However, `supportedHintOptions` is off track. It only allows a subset
> >> options and for the users it's not *clear* which subset is allowed.
> >>
> >> Besides, I'm not sure `supportedHintOptions` can work well for all
> cases.
> >> How could you support kafka properties (`connector.properties.*`) as
> hint
> >> options? Some kafka properties may affect semantics (bootstrap.servers),
> >> some may not (max.poll.records). Besides, I think it's not possible to
> list
> >> all the possible kafka properties [1].
> >>
> >> In summary, IMO, `supportedHintOptions`
> >> (1) it increase the complexity to develop a connector
> >> (2) it confuses users which options can be used in hint, which are not,
> >> they have to check the docs again and again.
> >> (3) it doesn't solve the problems which we want to solve by this FLIP.
> >>
> >> I think we should avoid introducing some partial solutions. Otherwise,
> we
> >> will be stuck in a loop that introduce new API -> deprecate API ->
> >> introduce new API....
> >>
> >> I personally in favor of an explicit WITH syntax after the table as a
> part
> >> of the query which is mentioned by Kurt before, e.g. SELECT * from T
> >> WITH('key' = 'value') .
> >> It allows users to dynamically set options which can affect semantics.
> It
> >> will be very flexible to solve users' problems so far.
> >>
> >> Best,
> >> Jark
> >>
> >> [1]: https://kafka.apache.org/documentation/#consumerconfigs
> >>
> >> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com> wrote:
> >>
> >>> My POC is here for the hints options merge [1].
> >>>
> >>> Personally, I have no strong objections for splitting hints with the
> >>> CatalogTable, the only cons is a more complex implementation but the
> >>> concept is more clear, and I have updated the WIKI.
> >>>
> >>> I think it would be nice if we can support the format “ignore-parse
> >> error”
> >>> option key, the CSV source already has a key [2] and we can use that in
> >> the
> >>> supportedHIntOptions, for the common CSV and JSON formats, we cal also
> >> give
> >>> a support. This is the only kind of key in formats that “do not change
> >> the
> >>> semantics” (somehow), what do you think about this ~
> >>>
> >>> [1]
> >>>
> >>
> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
> >>> [2]
> >>>
> >>
> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
> >>>
> >>> Best,
> >>> Danny Chan
> >>> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
> >>>> Hi everyone,
> >>>>
> >>>> +1 to Kurt's suggestion. Let's just have it in source and sink
> >> factories
> >>>> for now. We can still move this method up in the future. Currently, I
> >>>> don't see a need for catalogs or formats. Because how would you target
> >> a
> >>>> format in the query?
> >>>>
> >>>> @Danny: Can you send a link to your PoC? I'm very skeptical about
> >>>> creating a new CatalogTable in planner. Actually CatalogTable should
> be
> >>>> immutable between Catalog and Factory. Because a catalog can return
> its
> >>>> own factory and fully control the instantiation. Depending on the
> >>>> implementation, that means it can be possible that the catalog has
> >>>> encoded more information in a concrete subclass implementing the
> >>>> interface. I vote for separating the concerns of catalog information
> >> and
> >>>> hints in the factory explicitly.
> >>>>
> >>>> Regards,
> >>>> Timo
> >>>>
> >>>>
> >>>> On 18.03.20 05:41, Jingsong Li wrote:
> >>>>> Hi,
> >>>>>
> >>>>> I am thinking we can provide hints to *table* related instances.
> >>>>> - TableFormatFactory: of cause we need hints support, there are many
> >>> format
> >>>>> options in DDL too.
> >>>>> - catalog and module: I don't know, maybe in future we can provide
> >> some
> >>>>> hints for them.
> >>>>>
> >>>>> Best,
> >>>>> Jingsong Lee
> >>>>>
> >>>>> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com>
> >>> wrote:
> >>>>>
> >>>>>> Yes, I think we should move the `supportedHintOptions` from
> >>> TableFactory
> >>>>>> to TableSourceFactory, and we also need to add the interface to
> >>>>>> TableSinkFactory though because sink target table may also have
> >> hints
> >>>>>> attached.
> >>>>>>
> >>>>>> Best,
> >>>>>> Danny Chan
> >>>>>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> >>>>>>> Have one question for adding `supportedHintOptions` method to
> >>>>>>> `TableFactory`. It seems
> >>>>>>> `TableFactory` is a base factory interface for all *table module*
> >>> related
> >>>>>>> instances, such as
> >>>>>>> catalog, module, format and so on. It's not created only for
> >>> *table*. Is
> >>>>>> it
> >>>>>>> possible to move it
> >>>>>>> to `TableSourceFactory`?
> >>>>>>>
> >>>>>>> Best,
> >>>>>>> Kurt
> >>>>>>>
> >>>>>>>
> >>>>>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
> >> yuzhao.cyz@gmail.com>
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> Thanks Timo ~
> >>>>>>>>
> >>>>>>>> For the naming itself, I also think the PROPERTIES is not that
> >>>>>> concise, so
> >>>>>>>> +1 for OPTIONS (I had thought about that, but there are many
> >>> codes in
> >>>>>>>> current Flink called it properties, i.e. the
> >>> DescriptorProperties,
> >>>>>>>> #getSupportedProperties), let’s use OPTIONS if this is our new
> >>>>>> preference.
> >>>>>>>>
> >>>>>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
> >>>>>> ConfigOption
> >>>>>>>> can take more info. AFAIK, Spark also call their table options
> >>> instead
> >>>>>> of
> >>>>>>>> properties. [1]
> >>>>>>>>
> >>>>>>>> In my local POC, I did create a new CatalogTable, and it works
> >>> for
> >>>>>> current
> >>>>>>>> connectors well, all the DDL tables would finally yield a
> >>> CatalogTable
> >>>>>>>> instance and we can apply the options to that(in the
> >>> CatalogSourceTable
> >>>>>>>> when we generating the TableSource), the pros is that we do not
> >>> need to
> >>>>>>>> modify the codes of connectors itself. If we split the options
> >>> from
> >>>>>>>> CatalogTable, we may need to add some additional logic in each
> >>>>>> connector
> >>>>>>>> factories in order to merge these properties (and the logic are
> >>> almost
> >>>>>> the
> >>>>>>>> same), what do you think about this?
> >>>>>>>>
> >>>>>>>> [1]
> >>>>>>>>
> >>>>>>
> >>>
> >>
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Danny Chan
> >>>>>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
> >>> ,写道:
> >>>>>>>>> Hi Danny,
> >>>>>>>>>
> >>>>>>>>> thanks for updating the FLIP. I think your current design is
> >>>>>> sufficient
> >>>>>>>>> to separate hints from result-related properties.
> >>>>>>>>>
> >>>>>>>>> One remark to the naming itself: I would vote for calling the
> >>> hints
> >>>>>>>>> around table scan `OPTIONS('k'='v')`. We used the term
> >>> "properties"
> >>>>>> in
> >>>>>>>>> the past but since we want to unify the Flink configuration
> >>>>>> experience,
> >>>>>>>>> we should use consistent naming and classes around
> >>> `ConfigOptions`.
> >>>>>>>>>
> >>>>>>>>> It would be nice to use `Set<ConfigOption>
> >>> supportedHintOptions();`
> >>>>>> to
> >>>>>>>>> start using config options instead of pure string properties.
> >>> This
> >>>>>> will
> >>>>>>>>> also allow us to generate documentation in the future around
> >>>>>> supported
> >>>>>>>>> data types, ranges, etc. for options. At some point we would
> >>> also
> >>>>>> like
> >>>>>>>>> to drop `DescriptorProperties` class. "Options" is also used
> >>> in the
> >>>>>>>>> documentation [1] and in the SQL/MED standard [2].
> >>>>>>>>>
> >>>>>>>>> Furthermore, I would still vote for separating CatalogTable
> >>> and hint
> >>>>>>>>> options. Otherwise the planner would need to create a new
> >>>>>> CatalogTable
> >>>>>>>>> instance which might not always be easy. We should offer them
> >>> via:
> >>>>>>>>>
> >>>>>>>>>
> >>> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> >>>>>>>>> ReadableConfig
> >>>>>>>>>
> >>>>>>>>> What do you think?
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>> Timo
> >>>>>>>>>
> >>>>>>>>> [1]
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>
> >>
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> >>>>>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
> >>>>>>>>>> @Danny sounds good.
> >>>>>>>>>>
> >>>>>>>>>> Maybe it is worth listing all the classes of problems that
> >>> you
> >>>>>> want to
> >>>>>>>>>> address and then look at each class and see if hints are a
> >>> good
> >>>>>> default
> >>>>>>>>>> solution or a good optional way of simplifying things?
> >>>>>>>>>> The discussion has grown a lot and it is starting to be
> >> hard
> >>> to
> >>>>>>>> distinguish
> >>>>>>>>>> the parts where everyone agrees from the parts were there
> >> are
> >>>>>> concerns.
> >>>>>>>>>>
> >>>>>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
> >>> danny0405@apache.org>
> >>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Thanks Stephan ~
> >>>>>>>>>>>
> >>>>>>>>>>> We can remove the support for properties that may change
> >>> the
> >>>>>>>> semantics of
> >>>>>>>>>>> query if you think that is a trouble.
> >>>>>>>>>>>
> >>>>>>>>>>> How about we support the /*+ properties() */ hint only
> >> for
> >>> those
> >>>>>>>> optimize
> >>>>>>>>>>> parameters, such as the fetch size of source or something
> >>> like
> >>>>>> that,
> >>>>>>>> does
> >>>>>>>>>>> that make sense?
> >>>>>>>>>>>
> >>>>>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> >>>>>>>>>>>
> >>>>>>>>>>>> I think Bowen has actually put it very well.
> >>>>>>>>>>>>
> >>>>>>>>>>>> (1) Hints that change semantics looks like trouble
> >>> waiting to
> >>>>>>>> happen. For
> >>>>>>>>>>>> example Kafka offset handling should be in filters. The
> >>> Kafka
> >>>>>>>> source
> >>>>>>>>>>> should
> >>>>>>>>>>>> support predicate pushdown.
> >>>>>>>>>>>>
> >>>>>>>>>>>> (2) Hints should not be a workaround for current
> >>> shortcomings.
> >>>>>> A
> >>>>>>>> lot of
> >>>>>>>>>>> the
> >>>>>>>>>>>> suggested above sounds exactly like that. Working
> >> around
> >>>>>>>> catalog/DDL
> >>>>>>>>>>>> shortcomings, missing exposure of metadata (offsets),
> >>> missing
> >>>>>>>> predicate
> >>>>>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as
> >> a
> >>> quick
> >>>>>> fix
> >>>>>>>> for
> >>>>>>>>>>>> these issues, rather than fixing the root causes, will
> >>> much
> >>>>>> likely
> >>>>>>>> bite
> >>>>>>>>>>> us
> >>>>>>>>>>>> back badly in the future.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Best,
> >>>>>>>>>>>> Stephan
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
> >>> ykt836@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> It seems this FLIP's name is somewhat misleading.
> >> From
> >>> my
> >>>>>>>>>>> understanding,
> >>>>>>>>>>>>> this FLIP is trying to
> >>>>>>>>>>>>> address the dynamic parameter issue, and table hints
> >>> is the
> >>>>>> way
> >>>>>>>> we wan
> >>>>>>>>>>> to
> >>>>>>>>>>>>> choose. I think we should
> >>>>>>>>>>>>> be focus on "what's the right way to solve dynamic
> >>> property"
> >>>>>>>> instead of
> >>>>>>>>>>>>> discussing "whether table
> >>>>>>>>>>>>> hints can affect query semantics".
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> For now, there are two proposed ways to achieve
> >> dynamic
> >>>>>> property:
> >>>>>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with
> >>> (xxx)
> >>>>>>>>>>>>> 2. use custom "from t with (xxx)" syntax
> >>>>>>>>>>>>> 3. "Borrow" the table hints to have a special
> >>> PROPERTIES
> >>>>>> hint.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> The first one didn't break anything, but the only
> >>> problem i
> >>>>>> see
> >>>>>>>> is a
> >>>>>>>>>>>> little
> >>>>>>>>>>>>> more verbose than the table hint
> >>>>>>>>>>>>> approach. I can imagine when someone using SQL CLI to
> >>> have a
> >>>>>> sql
> >>>>>>>>>>>>> experience, it's quite often that
> >>>>>>>>>>>>> he will modify the table property, some use cases i
> >> can
> >>>>>> think of:
> >>>>>>>>>>>>> 1. the source contains some corrupted data, i want to
> >>> turn
> >>>>>> on the
> >>>>>>>>>>>>> "ignore-error" flag for certain formats.
> >>>>>>>>>>>>> 2. I have a kafka table and want to see some sample
> >>> data
> >>>>>> from the
> >>>>>>>>>>>>> beginning, so i change the offset
> >>>>>>>>>>>>> to "earliest", and then I want to observe the latest
> >>> data
> >>>>>> which
> >>>>>>>> keeps
> >>>>>>>>>>>>> coming in. I would write another query
> >>>>>>>>>>>>> to select from the latest table.
> >>>>>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly
> >> then
> >>> i can
> >>>>>>>> observe
> >>>>>>>>>>> the
> >>>>>>>>>>>>> data from database side.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Most of such use cases are quite ad-hoc. If every
> >> time
> >>> I
> >>>>>> want to
> >>>>>>>> have a
> >>>>>>>>>>>>> different experience, i need to create
> >>>>>>>>>>>>> a temporary table and then also modify my query, it
> >>> doesn't
> >>>>>> feel
> >>>>>>>>>>> smooth.
> >>>>>>>>>>>>> Embed such dynamic property into
> >>>>>>>>>>>>> query would have better user experience.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is
> >>> breaking
> >>>>>> SQL
> >>>>>>>>>>>> compliant,
> >>>>>>>>>>>>> and for #3, it only breaks some
> >>>>>>>>>>>>> unwritten rules, but we can have an explanation on
> >>> that. And
> >>>>>> I
> >>>>>>>> really
> >>>>>>>>>>>> doubt
> >>>>>>>>>>>>> whether user would complain about
> >>>>>>>>>>>>> this when they actually have flexible and good
> >>> experience
> >>>>>> using
> >>>>>>>> this.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Best,
> >>>>>>>>>>>>> Kurt
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> >>>>>> yuzhao.cyz@gmail.com
> >>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks Aljoscha ~
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I agree for most of the query hints, they are
> >>> optional as
> >>>>>> an
> >>>>>>>>>>> optimizer
> >>>>>>>>>>>>>> instruction, especially for the traditional RDBMS.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> But, just like BenChao said, Flink as a computation
> >>> engine
> >>>>>> has
> >>>>>>>> many
> >>>>>>>>>>>>>> different kind of data sources, thus, dynamic
> >>> parameters
> >>>>>> like
> >>>>>>>>>>>>> start_offest
> >>>>>>>>>>>>>> can only bind to each table scope, we can not set a
> >>> session
> >>>>>>>> config
> >>>>>>>>>>> like
> >>>>>>>>>>>>>> KSQL because they are all about Kafka:
> >>>>>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thus the most flexible way to set up these dynamic
> >>> params
> >>>>>> is
> >>>>>>>> to bind
> >>>>>>>>>>> to
> >>>>>>>>>>>>>> the table scope in the query when we want to
> >> override
> >>>>>>>> something, so
> >>>>>>>>>>> we
> >>>>>>>>>>>>> have
> >>>>>>>>>>>>>> these solutions above (with pros and cons from my
> >>> side):
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>>> - Parameters are part of the main query
> >>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>> - Not SQL compliant
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */
> >>> (from
> >>>>>> me)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>>> - SQL compliant because it is nested in the
> >> comments
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>> - Parameters are not part of the main query
> >>>>>>>>>>>>>> - Cryptic syntax for new users
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> The biggest problem for hints way may be the “if
> >>> hints
> >>>>>> must be
> >>>>>>>>>>>> optional”,
> >>>>>>>>>>>>>> actually we have though about 1 for a while but
> >>> aborted
> >>>>>>>> because it
> >>>>>>>>>>>> breaks
> >>>>>>>>>>>>>> the SQL standard too much. And we replace it with
> >> 2,
> >>>>>> because
> >>>>>>>> the
> >>>>>>>>>>> hints
> >>>>>>>>>>>>>> syntax do not break SQL standard(nested in
> >> comments).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint
> >>> that
> >>>>>> allows
> >>>>>>>>>>> override
> >>>>>>>>>>>>>> some properties of table dynamically, it does not
> >>> break
> >>>>>>>> anything, at
> >>>>>>>>>>>>> lease
> >>>>>>>>>>>>>> for current Flink use cases.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Planner hints are optional just because they are
> >>> naturally
> >>>>>>>> enforcers
> >>>>>>>>>>> of
> >>>>>>>>>>>>>> the planner, most of them aim to instruct the
> >>> optimizer,
> >>>>>> but,
> >>>>>>>> the
> >>>>>>>>>>> table
> >>>>>>>>>>>>>> hints is a little different, table hints can
> >> specify
> >>> the
> >>>>>> table
> >>>>>>>> meta
> >>>>>>>>>>>> like
> >>>>>>>>>>>>>> index column, and it is very convenient to specify
> >>> table
> >>>>>>>> properties.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */
> >>> table
> >>>>>> hint,
> >>>>>>>> we
> >>>>>>>>>>> can
> >>>>>>>>>>>>>> call it table dynamic parameters.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>> Danny Chan
> >>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> >>>>>>>> aljoscha@apache.org>,写道:
> >>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I don't understand this discussion. Hints, as I
> >>>>>> understand
> >>>>>>>> them,
> >>>>>>>>>>>> should
> >>>>>>>>>>>>>>> work like this:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> - hints are *optional* advice for the optimizer
> >> to
> >>> try
> >>>>>> and
> >>>>>>>> help it
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>> find a good execution strategy
> >>>>>>>>>>>>>>> - hints should not change query semantics, i.e.
> >>> they
> >>>>>> should
> >>>>>>>> not
> >>>>>>>>>>>> change
> >>>>>>>>>>>>>>> connector properties executing a query with
> >> taking
> >>> into
> >>>>>>>> account the
> >>>>>>>>>>>>>>> hints *must* produce the same result as executing
> >>> the
> >>>>>> query
> >>>>>>>> without
> >>>>>>>>>>>>>>> taking into account the hints
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>  From these simple requirements you can derive a
> >>> solution
> >>>>>>>> that makes
> >>>>>>>>>>>>>>> sense. I don't have a strong preference for the
> >>> syntax
> >>>>>> but we
> >>>>>>>>>>> should
> >>>>>>>>>>>>>>> strive to be in line with prior work.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>> Aljoscha
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
> >>>>>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I agree with Kurt that option2 is too
> >>> complicated to
> >>>>>> use
> >>>>>>>> because:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> • As a Kafka topic consumer, the user must
> >>> define both
> >>>>>> the
> >>>>>>>>>>> virtual
> >>>>>>>>>>>>>> column for start offset and he must apply a special
> >>> filter
> >>>>>>>> predicate
> >>>>>>>>>>>>> after
> >>>>>>>>>>>>>> each query
> >>>>>>>>>>>>>>>> • And for the internal implementation, the
> >>> metadata
> >>>>>> column
> >>>>>>>> push
> >>>>>>>>>>>> down
> >>>>>>>>>>>>>> is another hard topic, each kind of message queue
> >>> may have
> >>>>>> its
> >>>>>>>> offset
> >>>>>>>>>>>>>> attribute, we need to consider the expression type
> >>> for
> >>>>>>>> different
> >>>>>>>>>>> kind;
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>> source also need to recognize the constant column
> >> as
> >>> a
> >>>>>> config
> >>>>>>>>>>>>> option(which
> >>>>>>>>>>>>>> is weird because usually what we pushed down is a
> >>> table
> >>>>>> column)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> For option 1 and option3, I think there is no
> >>>>>> difference,
> >>>>>>>> option1
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>> also a hint syntax which is introduced in Sybase
> >> and
> >>>>>>>> referenced then
> >>>>>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
> >>>>>>>> ambitiousness.
> >>>>>>>>>>>>> Personally
> >>>>>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword
> >>> for
> >>>>>> these
> >>>>>>>> reasons:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> • We do not break the standard SQL, the hints
> >> are
> >>>>>> nested
> >>>>>>>> in SQL
> >>>>>>>>>>>>>> comments
> >>>>>>>>>>>>>>>> • We do not need to introduce additional WITH
> >>> keyword
> >>>>>>>> which may
> >>>>>>>>>>>>> appear
> >>>>>>>>>>>>>> in a query if we use that because a table can be
> >>>>>> referenced in
> >>>>>>>> all
> >>>>>>>>>>>> kinds
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
> >>> make
> >>>>>> our
> >>>>>>>> sql
> >>>>>>>>>>> query
> >>>>>>>>>>>>>> break too much of the SQL from standard
> >>>>>>>>>>>>>>>> • We would have uniform syntax for hints as
> >> query
> >>>>>> hint, one
> >>>>>>>>>>> syntax
> >>>>>>>>>>>>>> fits all and more easy to use
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> And here is the reason why we choose a uniform
> >>> Oracle
> >>>>>>>> style query
> >>>>>>>>>>>>>> hint syntax which is addressed by Julian Hyde when
> >> we
> >>>>>> design
> >>>>>>>> the
> >>>>>>>>>>> syntax
> >>>>>>>>>>>>>> from the Calcite community:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for
> >>> table
> >>>>>> hints.
> >>>>>>>> It
> >>>>>>>>>>> adds a
> >>>>>>>>>>>>>> new use of the WITH keyword that is unrelated to
> >> the
> >>> use of
> >>>>>>>> WITH for
> >>>>>>>>>>>>>> common-table expressions.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> A historical note. Microsoft SQL Server
> >>> inherited its
> >>>>>> hint
> >>>>>>>> syntax
> >>>>>>>>>>>>> from
> >>>>>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
> >>>>>>>> Programming”[1], page
> >>>>>>>>>>>>> 632,
> >>>>>>>>>>>>>> “Optimizer hints”. The book was written in 1999,
> >> and
> >>> covers
> >>>>>>>> Microsoft
> >>>>>>>>>>>> SQL
> >>>>>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
> >>> but the
> >>>>>>>> syntax very
> >>>>>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft
> >> SQL
> >>>>>> Server was
> >>>>>>>>>>> forked
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>> 1993.)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Microsoft later added the WITH keyword to make
> >>> it less
> >>>>>>>> ambiguous,
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>> has now deprecated the syntax that does not use
> >> WITH.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> They are forced to keep the syntax for
> >> backwards
> >>>>>>>> compatibility
> >>>>>>>>>>> but
> >>>>>>>>>>>>>> that doesn’t mean that we should shoulder their
> >>> burden.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I think formatted comments are the right
> >>> container for
> >>>>>>>> hints
> >>>>>>>>>>>> because
> >>>>>>>>>>>>>> it allows us to change the hint syntax without
> >>> changing
> >>>>>> the SQL
> >>>>>>>>>>> parser,
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>> makes clear that we are at liberty to ignore hints
> >>>>>> entirely.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Julian
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
> >>>>>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>> Danny Chan
> >>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> >>>>>> twalthr@apache.org
> >>>>>>>>> ,写道:
> >>>>>>>>>>>>>>>>> Hi Danny,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> it is true that our DDL is not standard
> >>> compliant by
> >>>>>>>> using the
> >>>>>>>>>>>> WITH
> >>>>>>>>>>>>>>>>> clause. Nevertheless, we aim for not
> >> diverging
> >>> too
> >>>>>> much
> >>>>>>>> and the
> >>>>>>>>>>>>> LIKE
> >>>>>>>>>>>>>>>>> clause is an example of that. It will solve
> >>> things
> >>>>>> like
> >>>>>>>>>>>> overwriting
> >>>>>>>>>>>>>>>>> WATERMARKs, add additional/modifying
> >>> properties and
> >>>>>>>> inherit
> >>>>>>>>>>>> schema.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3
> >>> types
> >>>>>>>> definition
> >>>>>>>>>>>>>> together.
> >>>>>>>>>>>>>>>>> We are not the first ones that try to solve
> >>> this.
> >>>>>> There
> >>>>>>>> is also
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> SQL
> >>>>>>>>>>>>>>>>> MED standard [1] that tried to tackle this
> >>> problem. I
> >>>>>>>> think it
> >>>>>>>>>>>> was
> >>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>> considered when designing the current DDL.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
> >>>>>> offsets. I
> >>>>>>>> will
> >>>>>>>>>>>> give
> >>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>> examples and look forward to feedback here:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part
> >>> of the
> >>>>>>>> query
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>>>>>> - Parameters are part of the main query
> >>>>>>>>>>>>>>>>> - No complicated hinting syntax
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>>>>> - Not SQL compliant
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> *Option 2* Use metadata in query
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
> >>>>>>>>>>>>> SYSTEM_METADATA('offset'))`
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset >
> >> TIMESTAMP
> >>>>>>>> '2012-12-12
> >>>>>>>>>>>>>> 12:34:22'`
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>>>>> - SQL compliant in the query
> >>>>>>>>>>>>>>>>> - Access of metadata in the DDL which is
> >>> required
> >>>>>> anyway
> >>>>>>>>>>>>>>>>> - Regular pushdown rules apply
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>>>>> - Users need to add an additional comlumn in
> >>> the DDL
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> *Option 3*: Use hints for properties
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> `
> >>>>>>>>>>>>>>>>> SELECT *
> >>>>>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
> >>>>>>>>>>>>>>>>> `
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Pros:
> >>>>>>>>>>>>>>>>> - Easy to add
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cons:
> >>>>>>>>>>>>>>>>> - Parameters are not part of the main query
> >>>>>>>>>>>>>>>>> - Cryptic syntax for new users
> >>>>>>>>>>>>>>>>> - Not standard compliant.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> If we go with this option, I would suggest to
> >>> make it
> >>>>>>>> available
> >>>>>>>>>>>> in
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> separate map and don't mix it with statically
> >>> defined
> >>>>>>>>>>> properties.
> >>>>>>>>>>>>>> Such
> >>>>>>>>>>>>>>>>> that the factory can decide which properties
> >>> have the
> >>>>>>>> right to
> >>>>>>>>>>> be
> >>>>>>>>>>>>>>>>> overwritten by the hints:
> >>>>>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
> >>>>>>>> ReadableConfig
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>> Timo
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Currently I see 3 options as a
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
> >>>>>>>>>>>>>>>>>> Thanks Bowen ~
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I agree we should somehow categorize our
> >>> connector
> >>>>>>>>>>> parameters.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> For type1, I’m already preparing a solution
> >>> like
> >>>>>> the
> >>>>>>>>>>> Confluent
> >>>>>>>>>>>>>> schema registry + Avro schema inference thing, so
> >>> this may
> >>>>>> not
> >>>>>>>> be a
> >>>>>>>>>>>>> problem
> >>>>>>>>>>>>>> in the near future.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> For type3, I have some questions:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset >
> >> 12pm
> >>>>>> yesterday”
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Where does the offset column come from, a
> >>> virtual
> >>>>>>>> column from
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> table schema, you said that
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> They change
> >>>>>>>>>>>>>>>>>> almost every time a query starts and have
> >>> nothing
> >>>>>> to
> >>>>>>>> do with
> >>>>>>>>>>>>>> metadata, thus
> >>>>>>>>>>>>>>>>>> should not be part of table definition/DDL
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> But why you can reference it in the query,
> >>> I’m
> >>>>>>>> confused for
> >>>>>>>>>>>> that,
> >>>>>>>>>>>>>> can you elaborate a little ?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>> Danny Chan
> >>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> >>>>>>>> bowenli86@gmail.com
> >>>>>>>>>>>> ,写道:
> >>>>>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> The root cause of too much manual work is
> >>> Flink
> >>>>>> DDL
> >>>>>>>> has
> >>>>>>>>>>>> mixed 3
> >>>>>>>>>>>>>> types of
> >>>>>>>>>>>>>>>>>>> params together and doesn't handle each
> >> of
> >>> them
> >>>>>> very
> >>>>>>>> well.
> >>>>>>>>>>>>> Below
> >>>>>>>>>>>>>> are how I
> >>>>>>>>>>>>>>>>>>> categorize them and corresponding
> >>> solutions in my
> >>>>>>>> mind:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
> >>>>>> external
> >>>>>>>>>>>>> endpoint/url,
> >>>>>>>>>>>>>>>>>>> username/pwd, schemas, formats.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Such metadata are mostly already
> >>> accessible in
> >>>>>>>> external
> >>>>>>>>>>>> system
> >>>>>>>>>>>>>> as long as
> >>>>>>>>>>>>>>>>>>> endpoints and credentials are provided.
> >>> Flink can
> >>>>>>>> get it
> >>>>>>>>>>> thru
> >>>>>>>>>>>>>> catalogs, but
> >>>>>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus
> >>> Flink
> >>>>>> just
> >>>>>>>> hasn't
> >>>>>>>>>>>>> been
> >>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>>>>>>> leverage that. So the solution should be
> >>> building
> >>>>>>>> more
> >>>>>>>>>>>>> catalogs.
> >>>>>>>>>>>>>> Such
> >>>>>>>>>>>>>>>>>>> params should be part of a Flink table
> >>>>>>>> DDL/definition, and
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>> overridable
> >>>>>>>>>>>>>>>>>>> in any means.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc
> >>> connector's
> >>>>>>>> fetch size,
> >>>>>>>>>>>>>> elasticsearch
> >>>>>>>>>>>>>>>>>>> connector's bulk flush size.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Such params don't affect query results,
> >> but
> >>>>>> affect
> >>>>>>>> how
> >>>>>>>>>>>> results
> >>>>>>>>>>>>>> are produced
> >>>>>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) -
> >> they
> >>> are
> >>>>>>>> essentially
> >>>>>>>>>>>>>> execution and
> >>>>>>>>>>>>>>>>>>> implementation details. They change often
> >>> in
> >>>>>>>> exploration or
> >>>>>>>>>>>>>> development
> >>>>>>>>>>>>>>>>>>> stages, but not quite frequently in
> >>> well-defined
> >>>>>>>>>>> long-running
> >>>>>>>>>>>>>> pipelines.
> >>>>>>>>>>>>>>>>>>> They should always have default values
> >> and
> >>> can be
> >>>>>>>> missing
> >>>>>>>>>>> in
> >>>>>>>>>>>>>> query. They
> >>>>>>>>>>>>>>>>>>> can be part of a table DDL/definition,
> >> but
> >>> should
> >>>>>>>> also be
> >>>>>>>>>>>>>> replaceable in a
> >>>>>>>>>>>>>>>>>>> query - *this is what table "hints" in
> >>> FLIP-113
> >>>>>>>> should
> >>>>>>>>>>>> cover*.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka
> >>> connector's
> >>>>>>>> start
> >>>>>>>>>>>> offset.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Such params affect query results - the
> >>> semantics.
> >>>>>>>> They'd
> >>>>>>>>>>>> better
> >>>>>>>>>>>>>> be as
> >>>>>>>>>>>>>>>>>>> filter conditions in WHERE clause that
> >> can
> >>> be
> >>>>>> pushed
> >>>>>>>> down.
> >>>>>>>>>>>> They
> >>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>> almost every time a query starts and have
> >>>>>> nothing to
> >>>>>>>> do
> >>>>>>>>>>> with
> >>>>>>>>>>>>>> metadata, thus
> >>>>>>>>>>>>>>>>>>> should not be part of table
> >>> definition/DDL, nor
> >>>>>> be
> >>>>>>>>>>> persisted
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>> catalogs.
> >>>>>>>>>>>>>>>>>>> If they will, users should create views
> >> to
> >>> keep
> >>>>>> such
> >>>>>>>> params
> >>>>>>>>>>>>>> around (note
> >>>>>>>>>>>>>>>>>>> this is different from variable
> >>> substitution).
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we
> >>> get these
> >>>>>>>> params
> >>>>>>>>>>>> right,
> >>>>>>>>>>>>>> here're the
> >>>>>>>>>>>>>>>>>>> steps users need to do to develop and run
> >>> a Flink
> >>>>>>>> job:
> >>>>>>>>>>>>>>>>>>> - configure a Flink
> >>> ConfluentSchemaRegistry with
> >>>>>> url,
> >>>>>>>>>>>> username,
> >>>>>>>>>>>>>> and password
> >>>>>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset
> >>>> 12pm
> >>>>>>>> yesterday"
> >>>>>>>>>>>>>> (simplified
> >>>>>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink
> >> automatically
> >>>>>> retrieves
> >>>>>>>> all
> >>>>>>>>>>>>>> metadata of
> >>>>>>>>>>>>>>>>>>> schema, file format, etc and start the
> >> job
> >>>>>>>>>>>>>>>>>>> - users want to make the job read Kafka
> >>> topic
> >>>>>>>> faster, so it
> >>>>>>>>>>>>> goes
> >>>>>>>>>>>>>> as "SELECT
> >>>>>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/
> >>> WHERE
> >>>>>>>> offset >
> >>>>>>>>>>> 12pm
> >>>>>>>>>>>>>> yesterday"
> >>>>>>>>>>>>>>>>>>> - done and satisfied, users submit it to
> >>>>>> production
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with
> >> (k1=v1,
> >>>>>> k2=v2),
> >>>>>>>> I think
> >>>>>>>>>>>>> it's
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> nice-to-have feature, but not a
> >>> strategically
> >>>>>>>> critical,
> >>>>>>>>>>>>>> long-term solution,
> >>>>>>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>>>> 1) It may seem promising at the current
> >>> stage to
> >>>>>>>> solve the
> >>>>>>>>>>>>>>>>>>> too-much-manual-work problem, but that's
> >>> only
> >>>>>>>> because Flink
> >>>>>>>>>>>>>> hasn't
> >>>>>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3
> >>> types
> >>>>>> of
> >>>>>>>> params
> >>>>>>>>>>>> above
> >>>>>>>>>>>>>> properly.
> >>>>>>>>>>>>>>>>>>> Once we get the params types right, the
> >>> LIKE
> >>>>>> syntax
> >>>>>>>> won't
> >>>>>>>>>>> be
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> important, and will be just an easier way
> >>> to
> >>>>>> create
> >>>>>>>> tables
> >>>>>>>>>>>>>> without retyping
> >>>>>>>>>>>>>>>>>>> long fields like username and pwd.
> >>>>>>>>>>>>>>>>>>> 2) Note that only some rare type of
> >>> catalog can
> >>>>>>>> store k-v
> >>>>>>>>>>>>>> property pair, so
> >>>>>>>>>>>>>>>>>>> table created this way often cannot be
> >>>>>> persisted. In
> >>>>>>>> the
> >>>>>>>>>>>>>> foreseeable
> >>>>>>>>>>>>>>>>>>> future, such catalog will only be
> >>> HiveCatalog,
> >>>>>> and
> >>>>>>>> not
> >>>>>>>>>>>> everyone
> >>>>>>>>>>>>>> has a Hive
> >>>>>>>>>>>>>>>>>>> metastore. To be honest, without
> >>> persistence,
> >>>>>>>> recreating
> >>>>>>>>>>>> tables
> >>>>>>>>>>>>>> every time
> >>>>>>>>>>>>>>>>>>> this way is still a lot of keyboard
> >> typing.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>> Bowen
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt
> >> Young
> >>> <
> >>>>>>>>>>> ykt836@gmail.com
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> If a specific connector want to have
> >> such
> >>>>>>>> parameter and
> >>>>>>>>>>>> read
> >>>>>>>>>>>>>> if out of
> >>>>>>>>>>>>>>>>>>>> configuration, then that's fine.
> >>>>>>>>>>>>>>>>>>>> If we are talking about a configuration
> >>> for all
> >>>>>>>> kinds of
> >>>>>>>>>>>>>> sources, I would
> >>>>>>>>>>>>>>>>>>>> be super careful about that.
> >>>>>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases,
> >>> but it
> >>>>>>>> will also
> >>>>>>>>>>>> make
> >>>>>>>>>>>>>> the left 20%
> >>>>>>>>>>>>>>>>>>>> feels weird.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>> Kurt
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark
> >> Wu
> >>> <
> >>>>>>>>>>> imjark@gmail.com
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi Kurt,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
> >>>>>>>>>>>>>>>>>>>>> I'm not saying to use the global
> >>>>>> configuration to
> >>>>>>>>>>>> override
> >>>>>>>>>>>>>> connector
> >>>>>>>>>>>>>>>>>>>>> properties by the planner.
> >>>>>>>>>>>>>>>>>>>>> But the connector should take this
> >>>>>> configuration
> >>>>>>>> and
> >>>>>>>>>>>>>> translate into their
> >>>>>>>>>>>>>>>>>>>>> client API.
> >>>>>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues
> >>> support
> >>>>>>>> eariliest
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>> latest and a
> >>>>>>>>>>>>>>>>>>>>> timestamp value as start point.
> >>>>>>>>>>>>>>>>>>>>> So we can support 3 options for this
> >>>>>>>> configuration:
> >>>>>>>>>>>>>> "eariliest", "latest"
> >>>>>>>>>>>>>>>>>>>>> and a timestamp string value.
> >>>>>>>>>>>>>>>>>>>>> Of course, this can't solve 100%
> >>> cases, but I
> >>>>>>>> guess can
> >>>>>>>>>>>>>> sovle 80% or 90%
> >>>>>>>>>>>>>>>>>>>>> cases.
> >>>>>>>>>>>>>>>>>>>>> And the remaining cases can be
> >>> resolved by
> >>>>>> LIKE
> >>>>>>>> syntax
> >>>>>>>>>>>>> which
> >>>>>>>>>>>>>> I guess is
> >>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>> very common cases.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>> Jark
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt
> >>> Young <
> >>>>>>>>>>>> ykt836@gmail.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Good to have such lovely
> >>> discussions. I
> >>>>>> also
> >>>>>>>> want to
> >>>>>>>>>>>>> share
> >>>>>>>>>>>>>> some of my
> >>>>>>>>>>>>>>>>>>>>>> opinions.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I
> >>> also
> >>>>>> think
> >>>>>>>> ignore
> >>>>>>>>>>>>>> invalid hints would
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>> dangerous, maybe
> >>>>>>>>>>>>>>>>>>>>>> the simplest solution is just throw
> >>> an
> >>>>>>>> exception.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> #2 Regarding to property
> >>> replacement: I
> >>>>>> don't
> >>>>>>>> think
> >>>>>>>>>>> we
> >>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>> constraint
> >>>>>>>>>>>>>>>>>>>>>> ourself to
> >>>>>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
> >>>>>> forbidden
> >>>>>>>> it
> >>>>>>>>>>>>> modifying
> >>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>> properties
> >>>>>>>>>>>>>>>>>>>>>> which can effect
> >>>>>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is
> >>> one of
> >>>>>> the
> >>>>>>>> table
> >>>>>>>>>>>>> hints,
> >>>>>>>>>>>>>> and a
> >>>>>>>>>>>>>>>>>>>> powerful
> >>>>>>>>>>>>>>>>>>>>>> one. It can
> >>>>>>>>>>>>>>>>>>>>>> modify properties located in DDL's
> >>> WITH
> >>>>>> block.
> >>>>>>>> But I
> >>>>>>>>>>>> also
> >>>>>>>>>>>>>> see the harm
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> if we make it
> >>>>>>>>>>>>>>>>>>>>>> too flexible like change the kafka
> >>> topic
> >>>>>> name
> >>>>>>>> with a
> >>>>>>>>>>>>> hint.
> >>>>>>>>>>>>>> Such use
> >>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> not common and
> >>>>>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I
> >> would
> >>>>>> propose
> >>>>>>>> we have
> >>>>>>>>>>> a
> >>>>>>>>>>>>> map
> >>>>>>>>>>>>>> of hintable
> >>>>>>>>>>>>>>>>>>>>>> properties for each
> >>>>>>>>>>>>>>>>>>>>>> connector, and should validate all
> >>> passed
> >>>>>> in
> >>>>>>>>>>> properties
> >>>>>>>>>>>>>> are actually
> >>>>>>>>>>>>>>>>>>>>>> hintable. And combining with
> >>>>>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
> >>>>>> exception
> >>>>>>>> once
> >>>>>>>>>>>>> received
> >>>>>>>>>>>>>> invalid
> >>>>>>>>>>>>>>>>>>>>>> property.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm
> >>> not sure
> >>>>>>>> it's
> >>>>>>>>>>>>> feasible.
> >>>>>>>>>>>>>> Different
> >>>>>>>>>>>>>>>>>>>>>> connectors will have totally
> >>>>>>>>>>>>>>>>>>>>>> different properties to represent
> >>> offset,
> >>>>>> some
> >>>>>>>> might
> >>>>>>>>>>> be
> >>>>>>>>>>>>>> timestamps,
> >>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>> might be string literals
> >>>>>>>>>>>>>>>>>>>>>> like "earliest", and others might
> >> be
> >>> just
> >>>>>>>> integers.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>> Kurt
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM
> >>> Jark Wu <
> >>>>>>>>>>>>> imjark@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>> twalthr@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 <
> >>>>>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
> >>>>>>>>>>>>>> twalthr@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
> >>>>>>>>>>>>> <
> >>>>>>>>>>>>>> imjark@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 <
> >>>>>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@apache.org>.
Thanks for the update Danny. +1 for this proposal.

Regards,
Timo

On 26.03.20 04:51, Danny Chan wrote:
> Thanks everyone who engaged in this discussion ~
> 
> Our goal is "Supports Dynamic Table Options for Flink SQL". After an
> offline discussion with Kurt, Timo and Dawid, we have made the final
> conclusion, here is the summary:
> 
> 
>     - Use comment style syntax to specify the dynamic table options: "/*+
>     *OPTIONS*(k1='v1', k2='v2') */"
>     - Have constraint on the options keys: the options that may bring in
>     security problems should not be allowed, i.e. Kafka connector zookeeper
>     endpoint URL and topic name
>     - Use white-list to control the allowed options for each connector,
>     which is more safe for future extention
>     - We allow to enable/disable this feature globally
>     - Implement based on the current code base first, and when FLIP-95 is
>     checked in, implement this feature based on new interface
> 
> Any suggestions are appreciated ~
> 
> [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL
> 
> Best,
> Danny Chan
> 
> Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:
> 
>> Hi everyone,
>>
>> Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid it
>> doesn't solve the problems but increases some development and learning
>> burdens.
>>
>> # increase development and learning burden
>>
>> According to the discussion so far, we want to support overriding a subset
>> of options in hints which doesn't affect semantics.
>> With the `supportedHintOptions`, it's up to the connector developers to
>> decide which options will not affect semantics, and to be hint options.
>> However, the question is how to distinguish whether an option will *affect
>> semantics*? What happens if an option will affect semantics but provided as
>> hint options?
>>  From my point of view, it's not easy to distinguish. For example, the
>> "format.ignore-parse-error" can be a very useful dynamic option but that
>> will affect semantic, because the result is different (null vs exception).
>> Another example, the "connector.lookup.cache.*" options are also very
>> useful to tune jobs, however, it will also affect the job results. I can
>> come up many more useful options but may affect semantics.
>>
>> I can see that the community will under endless discussion around "can this
>> option to be a hint option?",  "wether this option will affect semantics?".
>> You can also find that we already have different opinions on
>> "ignore-parse-error". Those discussion is a waste of time! That's not what
>> users want!
>> The problem is user need this, this, this options and HOW to expose them?
>> We should focus on that.
>>
>> Then there could be two endings in the future:
>> 1) compromise on the usability, we drop the rule that hints don't affect
>> semantics, allow all the useful options in the hints list.
>> 2) stick on the rule, users will find this is a stumbling feature which
>> doesn't solve their problems.
>>      And they will be surprised why this option can't be set, but the other
>> could. *semantic* is hard to be understood by users.
>>
>> # doesn't solve the problems
>>
>> I think the purpose of this FLIP is to allow users to quickly override some
>> connectors' properties to tune their jobs.
>> However, `supportedHintOptions` is off track. It only allows a subset
>> options and for the users it's not *clear* which subset is allowed.
>>
>> Besides, I'm not sure `supportedHintOptions` can work well for all cases.
>> How could you support kafka properties (`connector.properties.*`) as hint
>> options? Some kafka properties may affect semantics (bootstrap.servers),
>> some may not (max.poll.records). Besides, I think it's not possible to list
>> all the possible kafka properties [1].
>>
>> In summary, IMO, `supportedHintOptions`
>> (1) it increase the complexity to develop a connector
>> (2) it confuses users which options can be used in hint, which are not,
>> they have to check the docs again and again.
>> (3) it doesn't solve the problems which we want to solve by this FLIP.
>>
>> I think we should avoid introducing some partial solutions. Otherwise, we
>> will be stuck in a loop that introduce new API -> deprecate API ->
>> introduce new API....
>>
>> I personally in favor of an explicit WITH syntax after the table as a part
>> of the query which is mentioned by Kurt before, e.g. SELECT * from T
>> WITH('key' = 'value') .
>> It allows users to dynamically set options which can affect semantics. It
>> will be very flexible to solve users' problems so far.
>>
>> Best,
>> Jark
>>
>> [1]: https://kafka.apache.org/documentation/#consumerconfigs
>>
>> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com> wrote:
>>
>>> My POC is here for the hints options merge [1].
>>>
>>> Personally, I have no strong objections for splitting hints with the
>>> CatalogTable, the only cons is a more complex implementation but the
>>> concept is more clear, and I have updated the WIKI.
>>>
>>> I think it would be nice if we can support the format “ignore-parse
>> error”
>>> option key, the CSV source already has a key [2] and we can use that in
>> the
>>> supportedHIntOptions, for the common CSV and JSON formats, we cal also
>> give
>>> a support. This is the only kind of key in formats that “do not change
>> the
>>> semantics” (somehow), what do you think about this ~
>>>
>>> [1]
>>>
>> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
>>> [2]
>>>
>> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
>>>
>>> Best,
>>> Danny Chan
>>> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
>>>> Hi everyone,
>>>>
>>>> +1 to Kurt's suggestion. Let's just have it in source and sink
>> factories
>>>> for now. We can still move this method up in the future. Currently, I
>>>> don't see a need for catalogs or formats. Because how would you target
>> a
>>>> format in the query?
>>>>
>>>> @Danny: Can you send a link to your PoC? I'm very skeptical about
>>>> creating a new CatalogTable in planner. Actually CatalogTable should be
>>>> immutable between Catalog and Factory. Because a catalog can return its
>>>> own factory and fully control the instantiation. Depending on the
>>>> implementation, that means it can be possible that the catalog has
>>>> encoded more information in a concrete subclass implementing the
>>>> interface. I vote for separating the concerns of catalog information
>> and
>>>> hints in the factory explicitly.
>>>>
>>>> Regards,
>>>> Timo
>>>>
>>>>
>>>> On 18.03.20 05:41, Jingsong Li wrote:
>>>>> Hi,
>>>>>
>>>>> I am thinking we can provide hints to *table* related instances.
>>>>> - TableFormatFactory: of cause we need hints support, there are many
>>> format
>>>>> options in DDL too.
>>>>> - catalog and module: I don't know, maybe in future we can provide
>> some
>>>>> hints for them.
>>>>>
>>>>> Best,
>>>>> Jingsong Lee
>>>>>
>>>>> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com>
>>> wrote:
>>>>>
>>>>>> Yes, I think we should move the `supportedHintOptions` from
>>> TableFactory
>>>>>> to TableSourceFactory, and we also need to add the interface to
>>>>>> TableSinkFactory though because sink target table may also have
>> hints
>>>>>> attached.
>>>>>>
>>>>>> Best,
>>>>>> Danny Chan
>>>>>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
>>>>>>> Have one question for adding `supportedHintOptions` method to
>>>>>>> `TableFactory`. It seems
>>>>>>> `TableFactory` is a base factory interface for all *table module*
>>> related
>>>>>>> instances, such as
>>>>>>> catalog, module, format and so on. It's not created only for
>>> *table*. Is
>>>>>> it
>>>>>>> possible to move it
>>>>>>> to `TableSourceFactory`?
>>>>>>>
>>>>>>> Best,
>>>>>>> Kurt
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
>> yuzhao.cyz@gmail.com>
>>>>>> wrote:
>>>>>>>
>>>>>>>> Thanks Timo ~
>>>>>>>>
>>>>>>>> For the naming itself, I also think the PROPERTIES is not that
>>>>>> concise, so
>>>>>>>> +1 for OPTIONS (I had thought about that, but there are many
>>> codes in
>>>>>>>> current Flink called it properties, i.e. the
>>> DescriptorProperties,
>>>>>>>> #getSupportedProperties), let’s use OPTIONS if this is our new
>>>>>> preference.
>>>>>>>>
>>>>>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
>>>>>> ConfigOption
>>>>>>>> can take more info. AFAIK, Spark also call their table options
>>> instead
>>>>>> of
>>>>>>>> properties. [1]
>>>>>>>>
>>>>>>>> In my local POC, I did create a new CatalogTable, and it works
>>> for
>>>>>> current
>>>>>>>> connectors well, all the DDL tables would finally yield a
>>> CatalogTable
>>>>>>>> instance and we can apply the options to that(in the
>>> CatalogSourceTable
>>>>>>>> when we generating the TableSource), the pros is that we do not
>>> need to
>>>>>>>> modify the codes of connectors itself. If we split the options
>>> from
>>>>>>>> CatalogTable, we may need to add some additional logic in each
>>>>>> connector
>>>>>>>> factories in order to merge these properties (and the logic are
>>> almost
>>>>>> the
>>>>>>>> same), what do you think about this?
>>>>>>>>
>>>>>>>> [1]
>>>>>>>>
>>>>>>
>>>
>> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Danny Chan
>>>>>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
>>> ,写道:
>>>>>>>>> Hi Danny,
>>>>>>>>>
>>>>>>>>> thanks for updating the FLIP. I think your current design is
>>>>>> sufficient
>>>>>>>>> to separate hints from result-related properties.
>>>>>>>>>
>>>>>>>>> One remark to the naming itself: I would vote for calling the
>>> hints
>>>>>>>>> around table scan `OPTIONS('k'='v')`. We used the term
>>> "properties"
>>>>>> in
>>>>>>>>> the past but since we want to unify the Flink configuration
>>>>>> experience,
>>>>>>>>> we should use consistent naming and classes around
>>> `ConfigOptions`.
>>>>>>>>>
>>>>>>>>> It would be nice to use `Set<ConfigOption>
>>> supportedHintOptions();`
>>>>>> to
>>>>>>>>> start using config options instead of pure string properties.
>>> This
>>>>>> will
>>>>>>>>> also allow us to generate documentation in the future around
>>>>>> supported
>>>>>>>>> data types, ranges, etc. for options. At some point we would
>>> also
>>>>>> like
>>>>>>>>> to drop `DescriptorProperties` class. "Options" is also used
>>> in the
>>>>>>>>> documentation [1] and in the SQL/MED standard [2].
>>>>>>>>>
>>>>>>>>> Furthermore, I would still vote for separating CatalogTable
>>> and hint
>>>>>>>>> options. Otherwise the planner would need to create a new
>>>>>> CatalogTable
>>>>>>>>> instance which might not always be easy. We should offer them
>>> via:
>>>>>>>>>
>>>>>>>>>
>>> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
>>>>>>>>> ReadableConfig
>>>>>>>>>
>>>>>>>>> What do you think?
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>> Timo
>>>>>>>>>
>>>>>>>>> [1]
>>>>>>>>>
>>>>>>>>
>>>>>>
>>>
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
>>>>>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
>>>>>>>>>> @Danny sounds good.
>>>>>>>>>>
>>>>>>>>>> Maybe it is worth listing all the classes of problems that
>>> you
>>>>>> want to
>>>>>>>>>> address and then look at each class and see if hints are a
>>> good
>>>>>> default
>>>>>>>>>> solution or a good optional way of simplifying things?
>>>>>>>>>> The discussion has grown a lot and it is starting to be
>> hard
>>> to
>>>>>>>> distinguish
>>>>>>>>>> the parts where everyone agrees from the parts were there
>> are
>>>>>> concerns.
>>>>>>>>>>
>>>>>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
>>> danny0405@apache.org>
>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Thanks Stephan ~
>>>>>>>>>>>
>>>>>>>>>>> We can remove the support for properties that may change
>>> the
>>>>>>>> semantics of
>>>>>>>>>>> query if you think that is a trouble.
>>>>>>>>>>>
>>>>>>>>>>> How about we support the /*+ properties() */ hint only
>> for
>>> those
>>>>>>>> optimize
>>>>>>>>>>> parameters, such as the fetch size of source or something
>>> like
>>>>>> that,
>>>>>>>> does
>>>>>>>>>>> that make sense?
>>>>>>>>>>>
>>>>>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
>>>>>>>>>>>
>>>>>>>>>>>> I think Bowen has actually put it very well.
>>>>>>>>>>>>
>>>>>>>>>>>> (1) Hints that change semantics looks like trouble
>>> waiting to
>>>>>>>> happen. For
>>>>>>>>>>>> example Kafka offset handling should be in filters. The
>>> Kafka
>>>>>>>> source
>>>>>>>>>>> should
>>>>>>>>>>>> support predicate pushdown.
>>>>>>>>>>>>
>>>>>>>>>>>> (2) Hints should not be a workaround for current
>>> shortcomings.
>>>>>> A
>>>>>>>> lot of
>>>>>>>>>>> the
>>>>>>>>>>>> suggested above sounds exactly like that. Working
>> around
>>>>>>>> catalog/DDL
>>>>>>>>>>>> shortcomings, missing exposure of metadata (offsets),
>>> missing
>>>>>>>> predicate
>>>>>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as
>> a
>>> quick
>>>>>> fix
>>>>>>>> for
>>>>>>>>>>>> these issues, rather than fixing the root causes, will
>>> much
>>>>>> likely
>>>>>>>> bite
>>>>>>>>>>> us
>>>>>>>>>>>> back badly in the future.
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> Stephan
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
>>> ykt836@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> It seems this FLIP's name is somewhat misleading.
>> From
>>> my
>>>>>>>>>>> understanding,
>>>>>>>>>>>>> this FLIP is trying to
>>>>>>>>>>>>> address the dynamic parameter issue, and table hints
>>> is the
>>>>>> way
>>>>>>>> we wan
>>>>>>>>>>> to
>>>>>>>>>>>>> choose. I think we should
>>>>>>>>>>>>> be focus on "what's the right way to solve dynamic
>>> property"
>>>>>>>> instead of
>>>>>>>>>>>>> discussing "whether table
>>>>>>>>>>>>> hints can affect query semantics".
>>>>>>>>>>>>>
>>>>>>>>>>>>> For now, there are two proposed ways to achieve
>> dynamic
>>>>>> property:
>>>>>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with
>>> (xxx)
>>>>>>>>>>>>> 2. use custom "from t with (xxx)" syntax
>>>>>>>>>>>>> 3. "Borrow" the table hints to have a special
>>> PROPERTIES
>>>>>> hint.
>>>>>>>>>>>>>
>>>>>>>>>>>>> The first one didn't break anything, but the only
>>> problem i
>>>>>> see
>>>>>>>> is a
>>>>>>>>>>>> little
>>>>>>>>>>>>> more verbose than the table hint
>>>>>>>>>>>>> approach. I can imagine when someone using SQL CLI to
>>> have a
>>>>>> sql
>>>>>>>>>>>>> experience, it's quite often that
>>>>>>>>>>>>> he will modify the table property, some use cases i
>> can
>>>>>> think of:
>>>>>>>>>>>>> 1. the source contains some corrupted data, i want to
>>> turn
>>>>>> on the
>>>>>>>>>>>>> "ignore-error" flag for certain formats.
>>>>>>>>>>>>> 2. I have a kafka table and want to see some sample
>>> data
>>>>>> from the
>>>>>>>>>>>>> beginning, so i change the offset
>>>>>>>>>>>>> to "earliest", and then I want to observe the latest
>>> data
>>>>>> which
>>>>>>>> keeps
>>>>>>>>>>>>> coming in. I would write another query
>>>>>>>>>>>>> to select from the latest table.
>>>>>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly
>> then
>>> i can
>>>>>>>> observe
>>>>>>>>>>> the
>>>>>>>>>>>>> data from database side.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Most of such use cases are quite ad-hoc. If every
>> time
>>> I
>>>>>> want to
>>>>>>>> have a
>>>>>>>>>>>>> different experience, i need to create
>>>>>>>>>>>>> a temporary table and then also modify my query, it
>>> doesn't
>>>>>> feel
>>>>>>>>>>> smooth.
>>>>>>>>>>>>> Embed such dynamic property into
>>>>>>>>>>>>> query would have better user experience.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is
>>> breaking
>>>>>> SQL
>>>>>>>>>>>> compliant,
>>>>>>>>>>>>> and for #3, it only breaks some
>>>>>>>>>>>>> unwritten rules, but we can have an explanation on
>>> that. And
>>>>>> I
>>>>>>>> really
>>>>>>>>>>>> doubt
>>>>>>>>>>>>> whether user would complain about
>>>>>>>>>>>>> this when they actually have flexible and good
>>> experience
>>>>>> using
>>>>>>>> this.
>>>>>>>>>>>>>
>>>>>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> Kurt
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
>>>>>> yuzhao.cyz@gmail.com
>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks Aljoscha ~
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I agree for most of the query hints, they are
>>> optional as
>>>>>> an
>>>>>>>>>>> optimizer
>>>>>>>>>>>>>> instruction, especially for the traditional RDBMS.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> But, just like BenChao said, Flink as a computation
>>> engine
>>>>>> has
>>>>>>>> many
>>>>>>>>>>>>>> different kind of data sources, thus, dynamic
>>> parameters
>>>>>> like
>>>>>>>>>>>>> start_offest
>>>>>>>>>>>>>> can only bind to each table scope, we can not set a
>>> session
>>>>>>>> config
>>>>>>>>>>> like
>>>>>>>>>>>>>> KSQL because they are all about Kafka:
>>>>>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thus the most flexible way to set up these dynamic
>>> params
>>>>>> is
>>>>>>>> to bind
>>>>>>>>>>> to
>>>>>>>>>>>>>> the table scope in the query when we want to
>> override
>>>>>>>> something, so
>>>>>>>>>>> we
>>>>>>>>>>>>> have
>>>>>>>>>>>>>> these solutions above (with pros and cons from my
>>> side):
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>>> - Parameters are part of the main query
>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>> - Not SQL compliant
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */
>>> (from
>>>>>> me)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>>> - SQL compliant because it is nested in the
>> comments
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>> - Parameters are not part of the main query
>>>>>>>>>>>>>> - Cryptic syntax for new users
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The biggest problem for hints way may be the “if
>>> hints
>>>>>> must be
>>>>>>>>>>>> optional”,
>>>>>>>>>>>>>> actually we have though about 1 for a while but
>>> aborted
>>>>>>>> because it
>>>>>>>>>>>> breaks
>>>>>>>>>>>>>> the SQL standard too much. And we replace it with
>> 2,
>>>>>> because
>>>>>>>> the
>>>>>>>>>>> hints
>>>>>>>>>>>>>> syntax do not break SQL standard(nested in
>> comments).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint
>>> that
>>>>>> allows
>>>>>>>>>>> override
>>>>>>>>>>>>>> some properties of table dynamically, it does not
>>> break
>>>>>>>> anything, at
>>>>>>>>>>>>> lease
>>>>>>>>>>>>>> for current Flink use cases.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Planner hints are optional just because they are
>>> naturally
>>>>>>>> enforcers
>>>>>>>>>>> of
>>>>>>>>>>>>>> the planner, most of them aim to instruct the
>>> optimizer,
>>>>>> but,
>>>>>>>> the
>>>>>>>>>>> table
>>>>>>>>>>>>>> hints is a little different, table hints can
>> specify
>>> the
>>>>>> table
>>>>>>>> meta
>>>>>>>>>>>> like
>>>>>>>>>>>>>> index column, and it is very convenient to specify
>>> table
>>>>>>>> properties.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */
>>> table
>>>>>> hint,
>>>>>>>> we
>>>>>>>>>>> can
>>>>>>>>>>>>>> call it table dynamic parameters.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>> Danny Chan
>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
>>>>>>>> aljoscha@apache.org>,写道:
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I don't understand this discussion. Hints, as I
>>>>>> understand
>>>>>>>> them,
>>>>>>>>>>>> should
>>>>>>>>>>>>>>> work like this:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> - hints are *optional* advice for the optimizer
>> to
>>> try
>>>>>> and
>>>>>>>> help it
>>>>>>>>>>> to
>>>>>>>>>>>>>>> find a good execution strategy
>>>>>>>>>>>>>>> - hints should not change query semantics, i.e.
>>> they
>>>>>> should
>>>>>>>> not
>>>>>>>>>>>> change
>>>>>>>>>>>>>>> connector properties executing a query with
>> taking
>>> into
>>>>>>>> account the
>>>>>>>>>>>>>>> hints *must* produce the same result as executing
>>> the
>>>>>> query
>>>>>>>> without
>>>>>>>>>>>>>>> taking into account the hints
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>  From these simple requirements you can derive a
>>> solution
>>>>>>>> that makes
>>>>>>>>>>>>>>> sense. I don't have a strong preference for the
>>> syntax
>>>>>> but we
>>>>>>>>>>> should
>>>>>>>>>>>>>>> strive to be in line with prior work.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>> Aljoscha
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
>>>>>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I agree with Kurt that option2 is too
>>> complicated to
>>>>>> use
>>>>>>>> because:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> • As a Kafka topic consumer, the user must
>>> define both
>>>>>> the
>>>>>>>>>>> virtual
>>>>>>>>>>>>>> column for start offset and he must apply a special
>>> filter
>>>>>>>> predicate
>>>>>>>>>>>>> after
>>>>>>>>>>>>>> each query
>>>>>>>>>>>>>>>> • And for the internal implementation, the
>>> metadata
>>>>>> column
>>>>>>>> push
>>>>>>>>>>>> down
>>>>>>>>>>>>>> is another hard topic, each kind of message queue
>>> may have
>>>>>> its
>>>>>>>> offset
>>>>>>>>>>>>>> attribute, we need to consider the expression type
>>> for
>>>>>>>> different
>>>>>>>>>>> kind;
>>>>>>>>>>>>> the
>>>>>>>>>>>>>> source also need to recognize the constant column
>> as
>>> a
>>>>>> config
>>>>>>>>>>>>> option(which
>>>>>>>>>>>>>> is weird because usually what we pushed down is a
>>> table
>>>>>> column)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For option 1 and option3, I think there is no
>>>>>> difference,
>>>>>>>> option1
>>>>>>>>>>>> is
>>>>>>>>>>>>>> also a hint syntax which is introduced in Sybase
>> and
>>>>>>>> referenced then
>>>>>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
>>>>>>>> ambitiousness.
>>>>>>>>>>>>> Personally
>>>>>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword
>>> for
>>>>>> these
>>>>>>>> reasons:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> • We do not break the standard SQL, the hints
>> are
>>>>>> nested
>>>>>>>> in SQL
>>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>> • We do not need to introduce additional WITH
>>> keyword
>>>>>>>> which may
>>>>>>>>>>>>> appear
>>>>>>>>>>>>>> in a query if we use that because a table can be
>>>>>> referenced in
>>>>>>>> all
>>>>>>>>>>>> kinds
>>>>>>>>>>>>> of
>>>>>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
>>> make
>>>>>> our
>>>>>>>> sql
>>>>>>>>>>> query
>>>>>>>>>>>>>> break too much of the SQL from standard
>>>>>>>>>>>>>>>> • We would have uniform syntax for hints as
>> query
>>>>>> hint, one
>>>>>>>>>>> syntax
>>>>>>>>>>>>>> fits all and more easy to use
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> And here is the reason why we choose a uniform
>>> Oracle
>>>>>>>> style query
>>>>>>>>>>>>>> hint syntax which is addressed by Julian Hyde when
>> we
>>>>>> design
>>>>>>>> the
>>>>>>>>>>> syntax
>>>>>>>>>>>>>> from the Calcite community:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for
>>> table
>>>>>> hints.
>>>>>>>> It
>>>>>>>>>>> adds a
>>>>>>>>>>>>>> new use of the WITH keyword that is unrelated to
>> the
>>> use of
>>>>>>>> WITH for
>>>>>>>>>>>>>> common-table expressions.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> A historical note. Microsoft SQL Server
>>> inherited its
>>>>>> hint
>>>>>>>> syntax
>>>>>>>>>>>>> from
>>>>>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
>>>>>>>> Programming”[1], page
>>>>>>>>>>>>> 632,
>>>>>>>>>>>>>> “Optimizer hints”. The book was written in 1999,
>> and
>>> covers
>>>>>>>> Microsoft
>>>>>>>>>>>> SQL
>>>>>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
>>> but the
>>>>>>>> syntax very
>>>>>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft
>> SQL
>>>>>> Server was
>>>>>>>>>>> forked
>>>>>>>>>>>> in
>>>>>>>>>>>>>> 1993.)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Microsoft later added the WITH keyword to make
>>> it less
>>>>>>>> ambiguous,
>>>>>>>>>>>> and
>>>>>>>>>>>>>> has now deprecated the syntax that does not use
>> WITH.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> They are forced to keep the syntax for
>> backwards
>>>>>>>> compatibility
>>>>>>>>>>> but
>>>>>>>>>>>>>> that doesn’t mean that we should shoulder their
>>> burden.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I think formatted comments are the right
>>> container for
>>>>>>>> hints
>>>>>>>>>>>> because
>>>>>>>>>>>>>> it allows us to change the hint syntax without
>>> changing
>>>>>> the SQL
>>>>>>>>>>> parser,
>>>>>>>>>>>>> and
>>>>>>>>>>>>>> makes clear that we are at liberty to ignore hints
>>>>>> entirely.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Julian
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
>>>>>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> Danny Chan
>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
>>>>>> twalthr@apache.org
>>>>>>>>> ,写道:
>>>>>>>>>>>>>>>>> Hi Danny,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> it is true that our DDL is not standard
>>> compliant by
>>>>>>>> using the
>>>>>>>>>>>> WITH
>>>>>>>>>>>>>>>>> clause. Nevertheless, we aim for not
>> diverging
>>> too
>>>>>> much
>>>>>>>> and the
>>>>>>>>>>>>> LIKE
>>>>>>>>>>>>>>>>> clause is an example of that. It will solve
>>> things
>>>>>> like
>>>>>>>>>>>> overwriting
>>>>>>>>>>>>>>>>> WATERMARKs, add additional/modifying
>>> properties and
>>>>>>>> inherit
>>>>>>>>>>>> schema.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3
>>> types
>>>>>>>> definition
>>>>>>>>>>>>>> together.
>>>>>>>>>>>>>>>>> We are not the first ones that try to solve
>>> this.
>>>>>> There
>>>>>>>> is also
>>>>>>>>>>>> the
>>>>>>>>>>>>>> SQL
>>>>>>>>>>>>>>>>> MED standard [1] that tried to tackle this
>>> problem. I
>>>>>>>> think it
>>>>>>>>>>>> was
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>> considered when designing the current DDL.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
>>>>>> offsets. I
>>>>>>>> will
>>>>>>>>>>>> give
>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>> examples and look forward to feedback here:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part
>>> of the
>>>>>>>> query
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>>>>>> - Parameters are part of the main query
>>>>>>>>>>>>>>>>> - No complicated hinting syntax
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>>>>> - Not SQL compliant
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> *Option 2* Use metadata in query
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
>>>>>>>>>>>>> SYSTEM_METADATA('offset'))`
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset >
>> TIMESTAMP
>>>>>>>> '2012-12-12
>>>>>>>>>>>>>> 12:34:22'`
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>>>>> - SQL compliant in the query
>>>>>>>>>>>>>>>>> - Access of metadata in the DDL which is
>>> required
>>>>>> anyway
>>>>>>>>>>>>>>>>> - Regular pushdown rules apply
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>>>>> - Users need to add an additional comlumn in
>>> the DDL
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> *Option 3*: Use hints for properties
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> `
>>>>>>>>>>>>>>>>> SELECT *
>>>>>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
>>>>>>>>>>>>>>>>> `
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>>>>>> - Parameters are not part of the main query
>>>>>>>>>>>>>>>>> - Cryptic syntax for new users
>>>>>>>>>>>>>>>>> - Not standard compliant.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> If we go with this option, I would suggest to
>>> make it
>>>>>>>> available
>>>>>>>>>>>> in
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> separate map and don't mix it with statically
>>> defined
>>>>>>>>>>> properties.
>>>>>>>>>>>>>> Such
>>>>>>>>>>>>>>>>> that the factory can decide which properties
>>> have the
>>>>>>>> right to
>>>>>>>>>>> be
>>>>>>>>>>>>>>>>> overwritten by the hints:
>>>>>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
>>>>>>>> ReadableConfig
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>> Timo
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Currently I see 3 options as a
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
>>>>>>>>>>>>>>>>>> Thanks Bowen ~
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I agree we should somehow categorize our
>>> connector
>>>>>>>>>>> parameters.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> For type1, I’m already preparing a solution
>>> like
>>>>>> the
>>>>>>>>>>> Confluent
>>>>>>>>>>>>>> schema registry + Avro schema inference thing, so
>>> this may
>>>>>> not
>>>>>>>> be a
>>>>>>>>>>>>> problem
>>>>>>>>>>>>>> in the near future.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> For type3, I have some questions:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset >
>> 12pm
>>>>>> yesterday”
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Where does the offset column come from, a
>>> virtual
>>>>>>>> column from
>>>>>>>>>>>> the
>>>>>>>>>>>>>> table schema, you said that
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> They change
>>>>>>>>>>>>>>>>>> almost every time a query starts and have
>>> nothing
>>>>>> to
>>>>>>>> do with
>>>>>>>>>>>>>> metadata, thus
>>>>>>>>>>>>>>>>>> should not be part of table definition/DDL
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> But why you can reference it in the query,
>>> I’m
>>>>>>>> confused for
>>>>>>>>>>>> that,
>>>>>>>>>>>>>> can you elaborate a little ?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> Danny Chan
>>>>>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
>>>>>>>> bowenli86@gmail.com
>>>>>>>>>>>> ,写道:
>>>>>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> The root cause of too much manual work is
>>> Flink
>>>>>> DDL
>>>>>>>> has
>>>>>>>>>>>> mixed 3
>>>>>>>>>>>>>> types of
>>>>>>>>>>>>>>>>>>> params together and doesn't handle each
>> of
>>> them
>>>>>> very
>>>>>>>> well.
>>>>>>>>>>>>> Below
>>>>>>>>>>>>>> are how I
>>>>>>>>>>>>>>>>>>> categorize them and corresponding
>>> solutions in my
>>>>>>>> mind:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
>>>>>> external
>>>>>>>>>>>>> endpoint/url,
>>>>>>>>>>>>>>>>>>> username/pwd, schemas, formats.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Such metadata are mostly already
>>> accessible in
>>>>>>>> external
>>>>>>>>>>>> system
>>>>>>>>>>>>>> as long as
>>>>>>>>>>>>>>>>>>> endpoints and credentials are provided.
>>> Flink can
>>>>>>>> get it
>>>>>>>>>>> thru
>>>>>>>>>>>>>> catalogs, but
>>>>>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus
>>> Flink
>>>>>> just
>>>>>>>> hasn't
>>>>>>>>>>>>> been
>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>> leverage that. So the solution should be
>>> building
>>>>>>>> more
>>>>>>>>>>>>> catalogs.
>>>>>>>>>>>>>> Such
>>>>>>>>>>>>>>>>>>> params should be part of a Flink table
>>>>>>>> DDL/definition, and
>>>>>>>>>>>> not
>>>>>>>>>>>>>> overridable
>>>>>>>>>>>>>>>>>>> in any means.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc
>>> connector's
>>>>>>>> fetch size,
>>>>>>>>>>>>>> elasticsearch
>>>>>>>>>>>>>>>>>>> connector's bulk flush size.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Such params don't affect query results,
>> but
>>>>>> affect
>>>>>>>> how
>>>>>>>>>>>> results
>>>>>>>>>>>>>> are produced
>>>>>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) -
>> they
>>> are
>>>>>>>> essentially
>>>>>>>>>>>>>> execution and
>>>>>>>>>>>>>>>>>>> implementation details. They change often
>>> in
>>>>>>>> exploration or
>>>>>>>>>>>>>> development
>>>>>>>>>>>>>>>>>>> stages, but not quite frequently in
>>> well-defined
>>>>>>>>>>> long-running
>>>>>>>>>>>>>> pipelines.
>>>>>>>>>>>>>>>>>>> They should always have default values
>> and
>>> can be
>>>>>>>> missing
>>>>>>>>>>> in
>>>>>>>>>>>>>> query. They
>>>>>>>>>>>>>>>>>>> can be part of a table DDL/definition,
>> but
>>> should
>>>>>>>> also be
>>>>>>>>>>>>>> replaceable in a
>>>>>>>>>>>>>>>>>>> query - *this is what table "hints" in
>>> FLIP-113
>>>>>>>> should
>>>>>>>>>>>> cover*.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka
>>> connector's
>>>>>>>> start
>>>>>>>>>>>> offset.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Such params affect query results - the
>>> semantics.
>>>>>>>> They'd
>>>>>>>>>>>> better
>>>>>>>>>>>>>> be as
>>>>>>>>>>>>>>>>>>> filter conditions in WHERE clause that
>> can
>>> be
>>>>>> pushed
>>>>>>>> down.
>>>>>>>>>>>> They
>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>> almost every time a query starts and have
>>>>>> nothing to
>>>>>>>> do
>>>>>>>>>>> with
>>>>>>>>>>>>>> metadata, thus
>>>>>>>>>>>>>>>>>>> should not be part of table
>>> definition/DDL, nor
>>>>>> be
>>>>>>>>>>> persisted
>>>>>>>>>>>> in
>>>>>>>>>>>>>> catalogs.
>>>>>>>>>>>>>>>>>>> If they will, users should create views
>> to
>>> keep
>>>>>> such
>>>>>>>> params
>>>>>>>>>>>>>> around (note
>>>>>>>>>>>>>>>>>>> this is different from variable
>>> substitution).
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we
>>> get these
>>>>>>>> params
>>>>>>>>>>>> right,
>>>>>>>>>>>>>> here're the
>>>>>>>>>>>>>>>>>>> steps users need to do to develop and run
>>> a Flink
>>>>>>>> job:
>>>>>>>>>>>>>>>>>>> - configure a Flink
>>> ConfluentSchemaRegistry with
>>>>>> url,
>>>>>>>>>>>> username,
>>>>>>>>>>>>>> and password
>>>>>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset
>>>> 12pm
>>>>>>>> yesterday"
>>>>>>>>>>>>>> (simplified
>>>>>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink
>> automatically
>>>>>> retrieves
>>>>>>>> all
>>>>>>>>>>>>>> metadata of
>>>>>>>>>>>>>>>>>>> schema, file format, etc and start the
>> job
>>>>>>>>>>>>>>>>>>> - users want to make the job read Kafka
>>> topic
>>>>>>>> faster, so it
>>>>>>>>>>>>> goes
>>>>>>>>>>>>>> as "SELECT
>>>>>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/
>>> WHERE
>>>>>>>> offset >
>>>>>>>>>>> 12pm
>>>>>>>>>>>>>> yesterday"
>>>>>>>>>>>>>>>>>>> - done and satisfied, users submit it to
>>>>>> production
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with
>> (k1=v1,
>>>>>> k2=v2),
>>>>>>>> I think
>>>>>>>>>>>>> it's
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> nice-to-have feature, but not a
>>> strategically
>>>>>>>> critical,
>>>>>>>>>>>>>> long-term solution,
>>>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>> 1) It may seem promising at the current
>>> stage to
>>>>>>>> solve the
>>>>>>>>>>>>>>>>>>> too-much-manual-work problem, but that's
>>> only
>>>>>>>> because Flink
>>>>>>>>>>>>>> hasn't
>>>>>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3
>>> types
>>>>>> of
>>>>>>>> params
>>>>>>>>>>>> above
>>>>>>>>>>>>>> properly.
>>>>>>>>>>>>>>>>>>> Once we get the params types right, the
>>> LIKE
>>>>>> syntax
>>>>>>>> won't
>>>>>>>>>>> be
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> important, and will be just an easier way
>>> to
>>>>>> create
>>>>>>>> tables
>>>>>>>>>>>>>> without retyping
>>>>>>>>>>>>>>>>>>> long fields like username and pwd.
>>>>>>>>>>>>>>>>>>> 2) Note that only some rare type of
>>> catalog can
>>>>>>>> store k-v
>>>>>>>>>>>>>> property pair, so
>>>>>>>>>>>>>>>>>>> table created this way often cannot be
>>>>>> persisted. In
>>>>>>>> the
>>>>>>>>>>>>>> foreseeable
>>>>>>>>>>>>>>>>>>> future, such catalog will only be
>>> HiveCatalog,
>>>>>> and
>>>>>>>> not
>>>>>>>>>>>> everyone
>>>>>>>>>>>>>> has a Hive
>>>>>>>>>>>>>>>>>>> metastore. To be honest, without
>>> persistence,
>>>>>>>> recreating
>>>>>>>>>>>> tables
>>>>>>>>>>>>>> every time
>>>>>>>>>>>>>>>>>>> this way is still a lot of keyboard
>> typing.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>> Bowen
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt
>> Young
>>> <
>>>>>>>>>>> ykt836@gmail.com
>>>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> If a specific connector want to have
>> such
>>>>>>>> parameter and
>>>>>>>>>>>> read
>>>>>>>>>>>>>> if out of
>>>>>>>>>>>>>>>>>>>> configuration, then that's fine.
>>>>>>>>>>>>>>>>>>>> If we are talking about a configuration
>>> for all
>>>>>>>> kinds of
>>>>>>>>>>>>>> sources, I would
>>>>>>>>>>>>>>>>>>>> be super careful about that.
>>>>>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases,
>>> but it
>>>>>>>> will also
>>>>>>>>>>>> make
>>>>>>>>>>>>>> the left 20%
>>>>>>>>>>>>>>>>>>>> feels weird.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>> Kurt
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark
>> Wu
>>> <
>>>>>>>>>>> imjark@gmail.com
>>>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi Kurt,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
>>>>>>>>>>>>>>>>>>>>> I'm not saying to use the global
>>>>>> configuration to
>>>>>>>>>>>> override
>>>>>>>>>>>>>> connector
>>>>>>>>>>>>>>>>>>>>> properties by the planner.
>>>>>>>>>>>>>>>>>>>>> But the connector should take this
>>>>>> configuration
>>>>>>>> and
>>>>>>>>>>>>>> translate into their
>>>>>>>>>>>>>>>>>>>>> client API.
>>>>>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues
>>> support
>>>>>>>> eariliest
>>>>>>>>>>>> and
>>>>>>>>>>>>>> latest and a
>>>>>>>>>>>>>>>>>>>>> timestamp value as start point.
>>>>>>>>>>>>>>>>>>>>> So we can support 3 options for this
>>>>>>>> configuration:
>>>>>>>>>>>>>> "eariliest", "latest"
>>>>>>>>>>>>>>>>>>>>> and a timestamp string value.
>>>>>>>>>>>>>>>>>>>>> Of course, this can't solve 100%
>>> cases, but I
>>>>>>>> guess can
>>>>>>>>>>>>>> sovle 80% or 90%
>>>>>>>>>>>>>>>>>>>>> cases.
>>>>>>>>>>>>>>>>>>>>> And the remaining cases can be
>>> resolved by
>>>>>> LIKE
>>>>>>>> syntax
>>>>>>>>>>>>> which
>>>>>>>>>>>>>> I guess is
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> very common cases.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>> Jark
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt
>>> Young <
>>>>>>>>>>>> ykt836@gmail.com
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Good to have such lovely
>>> discussions. I
>>>>>> also
>>>>>>>> want to
>>>>>>>>>>>>> share
>>>>>>>>>>>>>> some of my
>>>>>>>>>>>>>>>>>>>>>> opinions.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I
>>> also
>>>>>> think
>>>>>>>> ignore
>>>>>>>>>>>>>> invalid hints would
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> dangerous, maybe
>>>>>>>>>>>>>>>>>>>>>> the simplest solution is just throw
>>> an
>>>>>>>> exception.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> #2 Regarding to property
>>> replacement: I
>>>>>> don't
>>>>>>>> think
>>>>>>>>>>> we
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>> constraint
>>>>>>>>>>>>>>>>>>>>>> ourself to
>>>>>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
>>>>>> forbidden
>>>>>>>> it
>>>>>>>>>>>>> modifying
>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>> properties
>>>>>>>>>>>>>>>>>>>>>> which can effect
>>>>>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is
>>> one of
>>>>>> the
>>>>>>>> table
>>>>>>>>>>>>> hints,
>>>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>>>>>> powerful
>>>>>>>>>>>>>>>>>>>>>> one. It can
>>>>>>>>>>>>>>>>>>>>>> modify properties located in DDL's
>>> WITH
>>>>>> block.
>>>>>>>> But I
>>>>>>>>>>>> also
>>>>>>>>>>>>>> see the harm
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> if we make it
>>>>>>>>>>>>>>>>>>>>>> too flexible like change the kafka
>>> topic
>>>>>> name
>>>>>>>> with a
>>>>>>>>>>>>> hint.
>>>>>>>>>>>>>> Such use
>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> not common and
>>>>>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I
>> would
>>>>>> propose
>>>>>>>> we have
>>>>>>>>>>> a
>>>>>>>>>>>>> map
>>>>>>>>>>>>>> of hintable
>>>>>>>>>>>>>>>>>>>>>> properties for each
>>>>>>>>>>>>>>>>>>>>>> connector, and should validate all
>>> passed
>>>>>> in
>>>>>>>>>>> properties
>>>>>>>>>>>>>> are actually
>>>>>>>>>>>>>>>>>>>>>> hintable. And combining with
>>>>>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
>>>>>> exception
>>>>>>>> once
>>>>>>>>>>>>> received
>>>>>>>>>>>>>> invalid
>>>>>>>>>>>>>>>>>>>>>> property.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm
>>> not sure
>>>>>>>> it's
>>>>>>>>>>>>> feasible.
>>>>>>>>>>>>>> Different
>>>>>>>>>>>>>>>>>>>>>> connectors will have totally
>>>>>>>>>>>>>>>>>>>>>> different properties to represent
>>> offset,
>>>>>> some
>>>>>>>> might
>>>>>>>>>>> be
>>>>>>>>>>>>>> timestamps,
>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>> might be string literals
>>>>>>>>>>>>>>>>>>>>>> like "earliest", and others might
>> be
>>> just
>>>>>>>> integers.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>> Kurt
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM
>>> Jark Wu <
>>>>>>>>>>>>> imjark@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>> twalthr@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 <
>>>>>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
>>>>>>>>>>>>>> twalthr@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
>>>>>>>>>>>>> <
>>>>>>>>>>>>>> imjark@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 <
>>>>>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>
>>>
>>
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <da...@apache.org>.
Thanks everyone who engaged in this discussion ~

Our goal is "Supports Dynamic Table Options for Flink SQL". After an
offline discussion with Kurt, Timo and Dawid, we have made the final
conclusion, here is the summary:


   - Use comment style syntax to specify the dynamic table options: "/*+
   *OPTIONS*(k1='v1', k2='v2') */"
   - Have constraint on the options keys: the options that may bring in
   security problems should not be allowed, i.e. Kafka connector zookeeper
   endpoint URL and topic name
   - Use white-list to control the allowed options for each connector,
   which is more safe for future extention
   - We allow to enable/disable this feature globally
   - Implement based on the current code base first, and when FLIP-95 is
   checked in, implement this feature based on new interface

Any suggestions are appreciated ~

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-113%3A+Supports+Dynamic+Table+Options+for+Flink+SQL

Best,
Danny Chan

Jark Wu <im...@gmail.com> 于2020年3月18日周三 下午10:38写道:

> Hi everyone,
>
> Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid it
> doesn't solve the problems but increases some development and learning
> burdens.
>
> # increase development and learning burden
>
> According to the discussion so far, we want to support overriding a subset
> of options in hints which doesn't affect semantics.
> With the `supportedHintOptions`, it's up to the connector developers to
> decide which options will not affect semantics, and to be hint options.
> However, the question is how to distinguish whether an option will *affect
> semantics*? What happens if an option will affect semantics but provided as
> hint options?
> From my point of view, it's not easy to distinguish. For example, the
> "format.ignore-parse-error" can be a very useful dynamic option but that
> will affect semantic, because the result is different (null vs exception).
> Another example, the "connector.lookup.cache.*" options are also very
> useful to tune jobs, however, it will also affect the job results. I can
> come up many more useful options but may affect semantics.
>
> I can see that the community will under endless discussion around "can this
> option to be a hint option?",  "wether this option will affect semantics?".
> You can also find that we already have different opinions on
> "ignore-parse-error". Those discussion is a waste of time! That's not what
> users want!
> The problem is user need this, this, this options and HOW to expose them?
> We should focus on that.
>
> Then there could be two endings in the future:
> 1) compromise on the usability, we drop the rule that hints don't affect
> semantics, allow all the useful options in the hints list.
> 2) stick on the rule, users will find this is a stumbling feature which
> doesn't solve their problems.
>     And they will be surprised why this option can't be set, but the other
> could. *semantic* is hard to be understood by users.
>
> # doesn't solve the problems
>
> I think the purpose of this FLIP is to allow users to quickly override some
> connectors' properties to tune their jobs.
> However, `supportedHintOptions` is off track. It only allows a subset
> options and for the users it's not *clear* which subset is allowed.
>
> Besides, I'm not sure `supportedHintOptions` can work well for all cases.
> How could you support kafka properties (`connector.properties.*`) as hint
> options? Some kafka properties may affect semantics (bootstrap.servers),
> some may not (max.poll.records). Besides, I think it's not possible to list
> all the possible kafka properties [1].
>
> In summary, IMO, `supportedHintOptions`
> (1) it increase the complexity to develop a connector
> (2) it confuses users which options can be used in hint, which are not,
> they have to check the docs again and again.
> (3) it doesn't solve the problems which we want to solve by this FLIP.
>
> I think we should avoid introducing some partial solutions. Otherwise, we
> will be stuck in a loop that introduce new API -> deprecate API ->
> introduce new API....
>
> I personally in favor of an explicit WITH syntax after the table as a part
> of the query which is mentioned by Kurt before, e.g. SELECT * from T
> WITH('key' = 'value') .
> It allows users to dynamically set options which can affect semantics. It
> will be very flexible to solve users' problems so far.
>
> Best,
> Jark
>
> [1]: https://kafka.apache.org/documentation/#consumerconfigs
>
> On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com> wrote:
>
> > My POC is here for the hints options merge [1].
> >
> > Personally, I have no strong objections for splitting hints with the
> > CatalogTable, the only cons is a more complex implementation but the
> > concept is more clear, and I have updated the WIKI.
> >
> > I think it would be nice if we can support the format “ignore-parse
> error”
> > option key, the CSV source already has a key [2] and we can use that in
> the
> > supportedHIntOptions, for the common CSV and JSON formats, we cal also
> give
> > a support. This is the only kind of key in formats that “do not change
> the
> > semantics” (somehow), what do you think about this ~
> >
> > [1]
> >
> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
> > [2]
> >
> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
> >
> > Best,
> > Danny Chan
> > 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
> > > Hi everyone,
> > >
> > > +1 to Kurt's suggestion. Let's just have it in source and sink
> factories
> > > for now. We can still move this method up in the future. Currently, I
> > > don't see a need for catalogs or formats. Because how would you target
> a
> > > format in the query?
> > >
> > > @Danny: Can you send a link to your PoC? I'm very skeptical about
> > > creating a new CatalogTable in planner. Actually CatalogTable should be
> > > immutable between Catalog and Factory. Because a catalog can return its
> > > own factory and fully control the instantiation. Depending on the
> > > implementation, that means it can be possible that the catalog has
> > > encoded more information in a concrete subclass implementing the
> > > interface. I vote for separating the concerns of catalog information
> and
> > > hints in the factory explicitly.
> > >
> > > Regards,
> > > Timo
> > >
> > >
> > > On 18.03.20 05:41, Jingsong Li wrote:
> > > > Hi,
> > > >
> > > > I am thinking we can provide hints to *table* related instances.
> > > > - TableFormatFactory: of cause we need hints support, there are many
> > format
> > > > options in DDL too.
> > > > - catalog and module: I don't know, maybe in future we can provide
> some
> > > > hints for them.
> > > >
> > > > Best,
> > > > Jingsong Lee
> > > >
> > > > On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com>
> > wrote:
> > > >
> > > > > Yes, I think we should move the `supportedHintOptions` from
> > TableFactory
> > > > > to TableSourceFactory, and we also need to add the interface to
> > > > > TableSinkFactory though because sink target table may also have
> hints
> > > > > attached.
> > > > >
> > > > > Best,
> > > > > Danny Chan
> > > > > 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> > > > > > Have one question for adding `supportedHintOptions` method to
> > > > > > `TableFactory`. It seems
> > > > > > `TableFactory` is a base factory interface for all *table module*
> > related
> > > > > > instances, such as
> > > > > > catalog, module, format and so on. It's not created only for
> > *table*. Is
> > > > > it
> > > > > > possible to move it
> > > > > > to `TableSourceFactory`?
> > > > > >
> > > > > > Best,
> > > > > > Kurt
> > > > > >
> > > > > >
> > > > > > On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <
> yuzhao.cyz@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > Thanks Timo ~
> > > > > > >
> > > > > > > For the naming itself, I also think the PROPERTIES is not that
> > > > > concise, so
> > > > > > > +1 for OPTIONS (I had thought about that, but there are many
> > codes in
> > > > > > > current Flink called it properties, i.e. the
> > DescriptorProperties,
> > > > > > > #getSupportedProperties), let’s use OPTIONS if this is our new
> > > > > preference.
> > > > > > >
> > > > > > > +1 to `Set<ConfigOption> supportedHintOptions()` because the
> > > > > ConfigOption
> > > > > > > can take more info. AFAIK, Spark also call their table options
> > instead
> > > > > of
> > > > > > > properties. [1]
> > > > > > >
> > > > > > > In my local POC, I did create a new CatalogTable, and it works
> > for
> > > > > current
> > > > > > > connectors well, all the DDL tables would finally yield a
> > CatalogTable
> > > > > > > instance and we can apply the options to that(in the
> > CatalogSourceTable
> > > > > > > when we generating the TableSource), the pros is that we do not
> > need to
> > > > > > > modify the codes of connectors itself. If we split the options
> > from
> > > > > > > CatalogTable, we may need to add some additional logic in each
> > > > > connector
> > > > > > > factories in order to merge these properties (and the logic are
> > almost
> > > > > the
> > > > > > > same), what do you think about this?
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > >
> >
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> > > > > > >
> > > > > > > Best,
> > > > > > > Danny Chan
> > > > > > > 在 2020年3月17日 +0800 PM10:10,Timo Walther <twalthr@apache.org
> >,写道:
> > > > > > > > Hi Danny,
> > > > > > > >
> > > > > > > > thanks for updating the FLIP. I think your current design is
> > > > > sufficient
> > > > > > > > to separate hints from result-related properties.
> > > > > > > >
> > > > > > > > One remark to the naming itself: I would vote for calling the
> > hints
> > > > > > > > around table scan `OPTIONS('k'='v')`. We used the term
> > "properties"
> > > > > in
> > > > > > > > the past but since we want to unify the Flink configuration
> > > > > experience,
> > > > > > > > we should use consistent naming and classes around
> > `ConfigOptions`.
> > > > > > > >
> > > > > > > > It would be nice to use `Set<ConfigOption>
> > supportedHintOptions();`
> > > > > to
> > > > > > > > start using config options instead of pure string properties.
> > This
> > > > > will
> > > > > > > > also allow us to generate documentation in the future around
> > > > > supported
> > > > > > > > data types, ranges, etc. for options. At some point we would
> > also
> > > > > like
> > > > > > > > to drop `DescriptorProperties` class. "Options" is also used
> > in the
> > > > > > > > documentation [1] and in the SQL/MED standard [2].
> > > > > > > >
> > > > > > > > Furthermore, I would still vote for separating CatalogTable
> > and hint
> > > > > > > > options. Otherwise the planner would need to create a new
> > > > > CatalogTable
> > > > > > > > instance which might not always be easy. We should offer them
> > via:
> > > > > > > >
> > > > > > > >
> > org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > > > > > > > ReadableConfig
> > > > > > > >
> > > > > > > > What do you think?
> > > > > > > >
> > > > > > > > Regards,
> > > > > > > > Timo
> > > > > > > >
> > > > > > > > [1]
> > > > > > > >
> > > > > > >
> > > > >
> >
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > > > > > > > [2] https://wiki.postgresql.org/wiki/SQL/MED
> > > > > > > >
> > > > > > > >
> > > > > > > > On 12.03.20 15:06, Stephan Ewen wrote:
> > > > > > > > > @Danny sounds good.
> > > > > > > > >
> > > > > > > > > Maybe it is worth listing all the classes of problems that
> > you
> > > > > want to
> > > > > > > > > address and then look at each class and see if hints are a
> > good
> > > > > default
> > > > > > > > > solution or a good optional way of simplifying things?
> > > > > > > > > The discussion has grown a lot and it is starting to be
> hard
> > to
> > > > > > > distinguish
> > > > > > > > > the parts where everyone agrees from the parts were there
> are
> > > > > concerns.
> > > > > > > > >
> > > > > > > > > On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
> > danny0405@apache.org>
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks Stephan ~
> > > > > > > > > >
> > > > > > > > > > We can remove the support for properties that may change
> > the
> > > > > > > semantics of
> > > > > > > > > > query if you think that is a trouble.
> > > > > > > > > >
> > > > > > > > > > How about we support the /*+ properties() */ hint only
> for
> > those
> > > > > > > optimize
> > > > > > > > > > parameters, such as the fetch size of source or something
> > like
> > > > > that,
> > > > > > > does
> > > > > > > > > > that make sense?
> > > > > > > > > >
> > > > > > > > > > Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > > > > > > > > >
> > > > > > > > > > > I think Bowen has actually put it very well.
> > > > > > > > > > >
> > > > > > > > > > > (1) Hints that change semantics looks like trouble
> > waiting to
> > > > > > > happen. For
> > > > > > > > > > > example Kafka offset handling should be in filters. The
> > Kafka
> > > > > > > source
> > > > > > > > > > should
> > > > > > > > > > > support predicate pushdown.
> > > > > > > > > > >
> > > > > > > > > > > (2) Hints should not be a workaround for current
> > shortcomings.
> > > > > A
> > > > > > > lot of
> > > > > > > > > > the
> > > > > > > > > > > suggested above sounds exactly like that. Working
> around
> > > > > > > catalog/DDL
> > > > > > > > > > > shortcomings, missing exposure of metadata (offsets),
> > missing
> > > > > > > predicate
> > > > > > > > > > > pushdown in Kafka. Abusing a feature like hints now as
> a
> > quick
> > > > > fix
> > > > > > > for
> > > > > > > > > > > these issues, rather than fixing the root causes, will
> > much
> > > > > likely
> > > > > > > bite
> > > > > > > > > > us
> > > > > > > > > > > back badly in the future.
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Stephan
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
> > ykt836@gmail.com>
> > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > It seems this FLIP's name is somewhat misleading.
> From
> > my
> > > > > > > > > > understanding,
> > > > > > > > > > > > this FLIP is trying to
> > > > > > > > > > > > address the dynamic parameter issue, and table hints
> > is the
> > > > > way
> > > > > > > we wan
> > > > > > > > > > to
> > > > > > > > > > > > choose. I think we should
> > > > > > > > > > > > be focus on "what's the right way to solve dynamic
> > property"
> > > > > > > instead of
> > > > > > > > > > > > discussing "whether table
> > > > > > > > > > > > hints can affect query semantics".
> > > > > > > > > > > >
> > > > > > > > > > > > For now, there are two proposed ways to achieve
> dynamic
> > > > > property:
> > > > > > > > > > > > 1. FLIP-110: create temporary table xx like xx with
> > (xxx)
> > > > > > > > > > > > 2. use custom "from t with (xxx)" syntax
> > > > > > > > > > > > 3. "Borrow" the table hints to have a special
> > PROPERTIES
> > > > > hint.
> > > > > > > > > > > >
> > > > > > > > > > > > The first one didn't break anything, but the only
> > problem i
> > > > > see
> > > > > > > is a
> > > > > > > > > > > little
> > > > > > > > > > > > more verbose than the table hint
> > > > > > > > > > > > approach. I can imagine when someone using SQL CLI to
> > have a
> > > > > sql
> > > > > > > > > > > > experience, it's quite often that
> > > > > > > > > > > > he will modify the table property, some use cases i
> can
> > > > > think of:
> > > > > > > > > > > > 1. the source contains some corrupted data, i want to
> > turn
> > > > > on the
> > > > > > > > > > > > "ignore-error" flag for certain formats.
> > > > > > > > > > > > 2. I have a kafka table and want to see some sample
> > data
> > > > > from the
> > > > > > > > > > > > beginning, so i change the offset
> > > > > > > > > > > > to "earliest", and then I want to observe the latest
> > data
> > > > > which
> > > > > > > keeps
> > > > > > > > > > > > coming in. I would write another query
> > > > > > > > > > > > to select from the latest table.
> > > > > > > > > > > > 3. I want to my jdbc sink flush data more eagerly
> then
> > i can
> > > > > > > observe
> > > > > > > > > > the
> > > > > > > > > > > > data from database side.
> > > > > > > > > > > >
> > > > > > > > > > > > Most of such use cases are quite ad-hoc. If every
> time
> > I
> > > > > want to
> > > > > > > have a
> > > > > > > > > > > > different experience, i need to create
> > > > > > > > > > > > a temporary table and then also modify my query, it
> > doesn't
> > > > > feel
> > > > > > > > > > smooth.
> > > > > > > > > > > > Embed such dynamic property into
> > > > > > > > > > > > query would have better user experience.
> > > > > > > > > > > >
> > > > > > > > > > > > Both 2 & 3 can make this happen. The cons of #2 is
> > breaking
> > > > > SQL
> > > > > > > > > > > compliant,
> > > > > > > > > > > > and for #3, it only breaks some
> > > > > > > > > > > > unwritten rules, but we can have an explanation on
> > that. And
> > > > > I
> > > > > > > really
> > > > > > > > > > > doubt
> > > > > > > > > > > > whether user would complain about
> > > > > > > > > > > > this when they actually have flexible and good
> > experience
> > > > > using
> > > > > > > this.
> > > > > > > > > > > >
> > > > > > > > > > > > My tendency would be #3 > #1 > #2, what do you think?
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Kurt
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> > > > > yuzhao.cyz@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Thanks Aljoscha ~
> > > > > > > > > > > > >
> > > > > > > > > > > > > I agree for most of the query hints, they are
> > optional as
> > > > > an
> > > > > > > > > > optimizer
> > > > > > > > > > > > > instruction, especially for the traditional RDBMS.
> > > > > > > > > > > > >
> > > > > > > > > > > > > But, just like BenChao said, Flink as a computation
> > engine
> > > > > has
> > > > > > > many
> > > > > > > > > > > > > different kind of data sources, thus, dynamic
> > parameters
> > > > > like
> > > > > > > > > > > > start_offest
> > > > > > > > > > > > > can only bind to each table scope, we can not set a
> > session
> > > > > > > config
> > > > > > > > > > like
> > > > > > > > > > > > > KSQL because they are all about Kafka:
> > > > > > > > > > > > > > SET ‘auto.offset.reset’=‘earliest’;
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thus the most flexible way to set up these dynamic
> > params
> > > > > is
> > > > > > > to bind
> > > > > > > > > > to
> > > > > > > > > > > > > the table scope in the query when we want to
> override
> > > > > > > something, so
> > > > > > > > > > we
> > > > > > > > > > > > have
> > > > > > > > > > > > > these solutions above (with pros and cons from my
> > side):
> > > > > > > > > > > > >
> > > > > > > > > > > > > • 1. Select * from t(offset=123) (from Timo)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > > - Parameters are part of the main query
> > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > - Not SQL compliant
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > • 2. Select * from t /*+ PROPERTIES(offset=123) */
> > (from
> > > > > me)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > > - SQL compliant because it is nested in the
> comments
> > > > > > > > > > > > >
> > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > > > > >
> > > > > > > > > > > > > The biggest problem for hints way may be the “if
> > hints
> > > > > must be
> > > > > > > > > > > optional”,
> > > > > > > > > > > > > actually we have though about 1 for a while but
> > aborted
> > > > > > > because it
> > > > > > > > > > > breaks
> > > > > > > > > > > > > the SQL standard too much. And we replace it with
> 2,
> > > > > because
> > > > > > > the
> > > > > > > > > > hints
> > > > > > > > > > > > > syntax do not break SQL standard(nested in
> comments).
> > > > > > > > > > > > >
> > > > > > > > > > > > > What if we have the special /*+ PROPERTIES */ hint
> > that
> > > > > allows
> > > > > > > > > > override
> > > > > > > > > > > > > some properties of table dynamically, it does not
> > break
> > > > > > > anything, at
> > > > > > > > > > > > lease
> > > > > > > > > > > > > for current Flink use cases.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Planner hints are optional just because they are
> > naturally
> > > > > > > enforcers
> > > > > > > > > > of
> > > > > > > > > > > > > the planner, most of them aim to instruct the
> > optimizer,
> > > > > but,
> > > > > > > the
> > > > > > > > > > table
> > > > > > > > > > > > > hints is a little different, table hints can
> specify
> > the
> > > > > table
> > > > > > > meta
> > > > > > > > > > > like
> > > > > > > > > > > > > index column, and it is very convenient to specify
> > table
> > > > > > > properties.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Or shall we not call /*+ PROPERTIES(offset=123) */
> > table
> > > > > hint,
> > > > > > > we
> > > > > > > > > > can
> > > > > > > > > > > > > call it table dynamic parameters.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> > > > > > > aljoscha@apache.org>,写道:
> > > > > > > > > > > > > > Hi,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I don't understand this discussion. Hints, as I
> > > > > understand
> > > > > > > them,
> > > > > > > > > > > should
> > > > > > > > > > > > > > work like this:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - hints are *optional* advice for the optimizer
> to
> > try
> > > > > and
> > > > > > > help it
> > > > > > > > > > to
> > > > > > > > > > > > > > find a good execution strategy
> > > > > > > > > > > > > > - hints should not change query semantics, i.e.
> > they
> > > > > should
> > > > > > > not
> > > > > > > > > > > change
> > > > > > > > > > > > > > connector properties executing a query with
> taking
> > into
> > > > > > > account the
> > > > > > > > > > > > > > hints *must* produce the same result as executing
> > the
> > > > > query
> > > > > > > without
> > > > > > > > > > > > > > taking into account the hints
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > From these simple requirements you can derive a
> > solution
> > > > > > > that makes
> > > > > > > > > > > > > > sense. I don't have a strong preference for the
> > syntax
> > > > > but we
> > > > > > > > > > should
> > > > > > > > > > > > > > strive to be in line with prior work.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Aljoscha
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On 11.03.20 11:53, Danny Chan wrote:
> > > > > > > > > > > > > > > Thanks Timo for summarize the 3 options ~
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I agree with Kurt that option2 is too
> > complicated to
> > > > > use
> > > > > > > because:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > • As a Kafka topic consumer, the user must
> > define both
> > > > > the
> > > > > > > > > > virtual
> > > > > > > > > > > > > column for start offset and he must apply a special
> > filter
> > > > > > > predicate
> > > > > > > > > > > > after
> > > > > > > > > > > > > each query
> > > > > > > > > > > > > > > • And for the internal implementation, the
> > metadata
> > > > > column
> > > > > > > push
> > > > > > > > > > > down
> > > > > > > > > > > > > is another hard topic, each kind of message queue
> > may have
> > > > > its
> > > > > > > offset
> > > > > > > > > > > > > attribute, we need to consider the expression type
> > for
> > > > > > > different
> > > > > > > > > > kind;
> > > > > > > > > > > > the
> > > > > > > > > > > > > source also need to recognize the constant column
> as
> > a
> > > > > config
> > > > > > > > > > > > option(which
> > > > > > > > > > > > > is weird because usually what we pushed down is a
> > table
> > > > > column)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > For option 1 and option3, I think there is no
> > > > > difference,
> > > > > > > option1
> > > > > > > > > > > is
> > > > > > > > > > > > > also a hint syntax which is introduced in Sybase
> and
> > > > > > > referenced then
> > > > > > > > > > > > > deprecated by MS-SQL in 199X years because of the
> > > > > > > ambitiousness.
> > > > > > > > > > > > Personally
> > > > > > > > > > > > > I prefer /*+ */ style table hint than WITH keyword
> > for
> > > > > these
> > > > > > > reasons:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > • We do not break the standard SQL, the hints
> are
> > > > > nested
> > > > > > > in SQL
> > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > • We do not need to introduce additional WITH
> > keyword
> > > > > > > which may
> > > > > > > > > > > > appear
> > > > > > > > > > > > > in a query if we use that because a table can be
> > > > > referenced in
> > > > > > > all
> > > > > > > > > > > kinds
> > > > > > > > > > > > of
> > > > > > > > > > > > > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
> > make
> > > > > our
> > > > > > > sql
> > > > > > > > > > query
> > > > > > > > > > > > > break too much of the SQL from standard
> > > > > > > > > > > > > > > • We would have uniform syntax for hints as
> query
> > > > > hint, one
> > > > > > > > > > syntax
> > > > > > > > > > > > > fits all and more easy to use
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > And here is the reason why we choose a uniform
> > Oracle
> > > > > > > style query
> > > > > > > > > > > > > hint syntax which is addressed by Julian Hyde when
> we
> > > > > design
> > > > > > > the
> > > > > > > > > > syntax
> > > > > > > > > > > > > from the Calcite community:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I don’t much like the MSSQL-style syntax for
> > table
> > > > > hints.
> > > > > > > It
> > > > > > > > > > adds a
> > > > > > > > > > > > > new use of the WITH keyword that is unrelated to
> the
> > use of
> > > > > > > WITH for
> > > > > > > > > > > > > common-table expressions.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > A historical note. Microsoft SQL Server
> > inherited its
> > > > > hint
> > > > > > > syntax
> > > > > > > > > > > > from
> > > > > > > > > > > > > Sybase a very long time ago. (See “Transact SQL
> > > > > > > Programming”[1], page
> > > > > > > > > > > > 632,
> > > > > > > > > > > > > “Optimizer hints”. The book was written in 1999,
> and
> > covers
> > > > > > > Microsoft
> > > > > > > > > > > SQL
> > > > > > > > > > > > > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
> > but the
> > > > > > > syntax very
> > > > > > > > > > > > > likely predates Sybase 4.3, from which Microsoft
> SQL
> > > > > Server was
> > > > > > > > > > forked
> > > > > > > > > > > in
> > > > > > > > > > > > > 1993.)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Microsoft later added the WITH keyword to make
> > it less
> > > > > > > ambiguous,
> > > > > > > > > > > and
> > > > > > > > > > > > > has now deprecated the syntax that does not use
> WITH.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > They are forced to keep the syntax for
> backwards
> > > > > > > compatibility
> > > > > > > > > > but
> > > > > > > > > > > > > that doesn’t mean that we should shoulder their
> > burden.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I think formatted comments are the right
> > container for
> > > > > > > hints
> > > > > > > > > > > because
> > > > > > > > > > > > > it allows us to change the hint syntax without
> > changing
> > > > > the SQL
> > > > > > > > > > parser,
> > > > > > > > > > > > and
> > > > > > > > > > > > > makes clear that we are at liberty to ignore hints
> > > > > entirely.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Julian
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > > > > > > > > > > > > https://www.amazon.com/s?k=9781565924017>
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> > > > > twalthr@apache.org
> > > > > > > > ,写道:
> > > > > > > > > > > > > > > > Hi Danny,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > it is true that our DDL is not standard
> > compliant by
> > > > > > > using the
> > > > > > > > > > > WITH
> > > > > > > > > > > > > > > > clause. Nevertheless, we aim for not
> diverging
> > too
> > > > > much
> > > > > > > and the
> > > > > > > > > > > > LIKE
> > > > > > > > > > > > > > > > clause is an example of that. It will solve
> > things
> > > > > like
> > > > > > > > > > > overwriting
> > > > > > > > > > > > > > > > WATERMARKs, add additional/modifying
> > properties and
> > > > > > > inherit
> > > > > > > > > > > schema.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Bowen is right that Flink's DDL is mixing 3
> > types
> > > > > > > definition
> > > > > > > > > > > > > together.
> > > > > > > > > > > > > > > > We are not the first ones that try to solve
> > this.
> > > > > There
> > > > > > > is also
> > > > > > > > > > > the
> > > > > > > > > > > > > SQL
> > > > > > > > > > > > > > > > MED standard [1] that tried to tackle this
> > problem. I
> > > > > > > think it
> > > > > > > > > > > was
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > considered when designing the current DDL.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Currently, I see 3 options for handling Kafka
> > > > > offsets. I
> > > > > > > will
> > > > > > > > > > > give
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > examples and look forward to feedback here:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > *Option 1* Runtime and semantic parms as part
> > of the
> > > > > > > query
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > > > > > - Parameters are part of the main query
> > > > > > > > > > > > > > > > - No complicated hinting syntax
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > > > > - Not SQL compliant
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > *Option 2* Use metadata in query
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > `CREATE TABLE MyTable (id INT, offset AS
> > > > > > > > > > > > SYSTEM_METADATA('offset'))`
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > `SELECT * FROM MyTable WHERE offset >
> TIMESTAMP
> > > > > > > '2012-12-12
> > > > > > > > > > > > > 12:34:22'`
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > > > > - SQL compliant in the query
> > > > > > > > > > > > > > > > - Access of metadata in the DDL which is
> > required
> > > > > anyway
> > > > > > > > > > > > > > > > - Regular pushdown rules apply
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > > > > - Users need to add an additional comlumn in
> > the DDL
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > *Option 3*: Use hints for properties
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > `
> > > > > > > > > > > > > > > > SELECT *
> > > > > > > > > > > > > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > > > > > > > > > > > > `
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > > > > > > > > - Not standard compliant.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > If we go with this option, I would suggest to
> > make it
> > > > > > > available
> > > > > > > > > > > in
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > separate map and don't mix it with statically
> > defined
> > > > > > > > > > properties.
> > > > > > > > > > > > > Such
> > > > > > > > > > > > > > > > that the factory can decide which properties
> > have the
> > > > > > > right to
> > > > > > > > > > be
> > > > > > > > > > > > > > > > overwritten by the hints:
> > > > > > > > > > > > > > > > TableSourceFactory.Context.getQueryHints():
> > > > > > > ReadableConfig
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > > > > Timo
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Currently I see 3 options as a
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > > > > > > > > > > > > Thanks Bowen ~
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I agree we should somehow categorize our
> > connector
> > > > > > > > > > parameters.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > For type1, I’m already preparing a solution
> > like
> > > > > the
> > > > > > > > > > Confluent
> > > > > > > > > > > > > schema registry + Avro schema inference thing, so
> > this may
> > > > > not
> > > > > > > be a
> > > > > > > > > > > > problem
> > > > > > > > > > > > > in the near future.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > For type3, I have some questions:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > "SELECT * FROM mykafka WHERE offset >
> 12pm
> > > > > yesterday”
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Where does the offset column come from, a
> > virtual
> > > > > > > column from
> > > > > > > > > > > the
> > > > > > > > > > > > > table schema, you said that
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > They change
> > > > > > > > > > > > > > > > > almost every time a query starts and have
> > nothing
> > > > > to
> > > > > > > do with
> > > > > > > > > > > > > metadata, thus
> > > > > > > > > > > > > > > > > should not be part of table definition/DDL
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > But why you can reference it in the query,
> > I’m
> > > > > > > confused for
> > > > > > > > > > > that,
> > > > > > > > > > > > > can you elaborate a little ?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> > > > > > > bowenli86@gmail.com
> > > > > > > > > > > ,写道:
> > > > > > > > > > > > > > > > > > Thanks Danny for kicking off the effort
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > The root cause of too much manual work is
> > Flink
> > > > > DDL
> > > > > > > has
> > > > > > > > > > > mixed 3
> > > > > > > > > > > > > types of
> > > > > > > > > > > > > > > > > > params together and doesn't handle each
> of
> > them
> > > > > very
> > > > > > > well.
> > > > > > > > > > > > Below
> > > > > > > > > > > > > are how I
> > > > > > > > > > > > > > > > > > categorize them and corresponding
> > solutions in my
> > > > > > > mind:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > - type 1: Metadata of external data, like
> > > > > external
> > > > > > > > > > > > endpoint/url,
> > > > > > > > > > > > > > > > > > username/pwd, schemas, formats.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Such metadata are mostly already
> > accessible in
> > > > > > > external
> > > > > > > > > > > system
> > > > > > > > > > > > > as long as
> > > > > > > > > > > > > > > > > > endpoints and credentials are provided.
> > Flink can
> > > > > > > get it
> > > > > > > > > > thru
> > > > > > > > > > > > > catalogs, but
> > > > > > > > > > > > > > > > > > we haven't had many catalogs yet and thus
> > Flink
> > > > > just
> > > > > > > hasn't
> > > > > > > > > > > > been
> > > > > > > > > > > > > able to
> > > > > > > > > > > > > > > > > > leverage that. So the solution should be
> > building
> > > > > > > more
> > > > > > > > > > > > catalogs.
> > > > > > > > > > > > > Such
> > > > > > > > > > > > > > > > > > params should be part of a Flink table
> > > > > > > DDL/definition, and
> > > > > > > > > > > not
> > > > > > > > > > > > > overridable
> > > > > > > > > > > > > > > > > > in any means.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > - type 2: Runtime params, like jdbc
> > connector's
> > > > > > > fetch size,
> > > > > > > > > > > > > elasticsearch
> > > > > > > > > > > > > > > > > > connector's bulk flush size.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Such params don't affect query results,
> but
> > > > > affect
> > > > > > > how
> > > > > > > > > > > results
> > > > > > > > > > > > > are produced
> > > > > > > > > > > > > > > > > > (eg. fast or slow, aka performance) -
> they
> > are
> > > > > > > essentially
> > > > > > > > > > > > > execution and
> > > > > > > > > > > > > > > > > > implementation details. They change often
> > in
> > > > > > > exploration or
> > > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > > stages, but not quite frequently in
> > well-defined
> > > > > > > > > > long-running
> > > > > > > > > > > > > pipelines.
> > > > > > > > > > > > > > > > > > They should always have default values
> and
> > can be
> > > > > > > missing
> > > > > > > > > > in
> > > > > > > > > > > > > query. They
> > > > > > > > > > > > > > > > > > can be part of a table DDL/definition,
> but
> > should
> > > > > > > also be
> > > > > > > > > > > > > replaceable in a
> > > > > > > > > > > > > > > > > > query - *this is what table "hints" in
> > FLIP-113
> > > > > > > should
> > > > > > > > > > > cover*.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > - type 3: Semantic params, like kafka
> > connector's
> > > > > > > start
> > > > > > > > > > > offset.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Such params affect query results - the
> > semantics.
> > > > > > > They'd
> > > > > > > > > > > better
> > > > > > > > > > > > > be as
> > > > > > > > > > > > > > > > > > filter conditions in WHERE clause that
> can
> > be
> > > > > pushed
> > > > > > > down.
> > > > > > > > > > > They
> > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > almost every time a query starts and have
> > > > > nothing to
> > > > > > > do
> > > > > > > > > > with
> > > > > > > > > > > > > metadata, thus
> > > > > > > > > > > > > > > > > > should not be part of table
> > definition/DDL, nor
> > > > > be
> > > > > > > > > > persisted
> > > > > > > > > > > in
> > > > > > > > > > > > > catalogs.
> > > > > > > > > > > > > > > > > > If they will, users should create views
> to
> > keep
> > > > > such
> > > > > > > params
> > > > > > > > > > > > > around (note
> > > > > > > > > > > > > > > > > > this is different from variable
> > substitution).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Take Flink-Kafka as an example. Once we
> > get these
> > > > > > > params
> > > > > > > > > > > right,
> > > > > > > > > > > > > here're the
> > > > > > > > > > > > > > > > > > steps users need to do to develop and run
> > a Flink
> > > > > > > job:
> > > > > > > > > > > > > > > > > > - configure a Flink
> > ConfluentSchemaRegistry with
> > > > > url,
> > > > > > > > > > > username,
> > > > > > > > > > > > > and password
> > > > > > > > > > > > > > > > > > - run "SELECT * FROM mykafka WHERE offset
> > > 12pm
> > > > > > > yesterday"
> > > > > > > > > > > > > (simplified
> > > > > > > > > > > > > > > > > > timestamp) in SQL CLI, Flink
> automatically
> > > > > retrieves
> > > > > > > all
> > > > > > > > > > > > > metadata of
> > > > > > > > > > > > > > > > > > schema, file format, etc and start the
> job
> > > > > > > > > > > > > > > > > > - users want to make the job read Kafka
> > topic
> > > > > > > faster, so it
> > > > > > > > > > > > goes
> > > > > > > > > > > > > as "SELECT
> > > > > > > > > > > > > > > > > > * FROM mykafka /* faster_read_key=value*/
> > WHERE
> > > > > > > offset >
> > > > > > > > > > 12pm
> > > > > > > > > > > > > yesterday"
> > > > > > > > > > > > > > > > > > - done and satisfied, users submit it to
> > > > > production
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Regarding "CREATE TABLE t LIKE with
> (k1=v1,
> > > > > k2=v2),
> > > > > > > I think
> > > > > > > > > > > > it's
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > nice-to-have feature, but not a
> > strategically
> > > > > > > critical,
> > > > > > > > > > > > > long-term solution,
> > > > > > > > > > > > > > > > > > because
> > > > > > > > > > > > > > > > > > 1) It may seem promising at the current
> > stage to
> > > > > > > solve the
> > > > > > > > > > > > > > > > > > too-much-manual-work problem, but that's
> > only
> > > > > > > because Flink
> > > > > > > > > > > > > hasn't
> > > > > > > > > > > > > > > > > > leveraged catalogs well and handled the 3
> > types
> > > > > of
> > > > > > > params
> > > > > > > > > > > above
> > > > > > > > > > > > > properly.
> > > > > > > > > > > > > > > > > > Once we get the params types right, the
> > LIKE
> > > > > syntax
> > > > > > > won't
> > > > > > > > > > be
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > important, and will be just an easier way
> > to
> > > > > create
> > > > > > > tables
> > > > > > > > > > > > > without retyping
> > > > > > > > > > > > > > > > > > long fields like username and pwd.
> > > > > > > > > > > > > > > > > > 2) Note that only some rare type of
> > catalog can
> > > > > > > store k-v
> > > > > > > > > > > > > property pair, so
> > > > > > > > > > > > > > > > > > table created this way often cannot be
> > > > > persisted. In
> > > > > > > the
> > > > > > > > > > > > > foreseeable
> > > > > > > > > > > > > > > > > > future, such catalog will only be
> > HiveCatalog,
> > > > > and
> > > > > > > not
> > > > > > > > > > > everyone
> > > > > > > > > > > > > has a Hive
> > > > > > > > > > > > > > > > > > metastore. To be honest, without
> > persistence,
> > > > > > > recreating
> > > > > > > > > > > tables
> > > > > > > > > > > > > every time
> > > > > > > > > > > > > > > > > > this way is still a lot of keyboard
> typing.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Cheers,
> > > > > > > > > > > > > > > > > > Bowen
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt
> Young
> > <
> > > > > > > > > > ykt836@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > If a specific connector want to have
> such
> > > > > > > parameter and
> > > > > > > > > > > read
> > > > > > > > > > > > > if out of
> > > > > > > > > > > > > > > > > > > configuration, then that's fine.
> > > > > > > > > > > > > > > > > > > If we are talking about a configuration
> > for all
> > > > > > > kinds of
> > > > > > > > > > > > > sources, I would
> > > > > > > > > > > > > > > > > > > be super careful about that.
> > > > > > > > > > > > > > > > > > > It's true it can solve maybe 80% cases,
> > but it
> > > > > > > will also
> > > > > > > > > > > make
> > > > > > > > > > > > > the left 20%
> > > > > > > > > > > > > > > > > > > feels weird.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark
> Wu
> > <
> > > > > > > > > > imjark@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi Kurt,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > > > > > > > > > > > > I'm not saying to use the global
> > > > > configuration to
> > > > > > > > > > > override
> > > > > > > > > > > > > connector
> > > > > > > > > > > > > > > > > > > > properties by the planner.
> > > > > > > > > > > > > > > > > > > > But the connector should take this
> > > > > configuration
> > > > > > > and
> > > > > > > > > > > > > translate into their
> > > > > > > > > > > > > > > > > > > > client API.
> > > > > > > > > > > > > > > > > > > > AFAIK, almost all the message queues
> > support
> > > > > > > eariliest
> > > > > > > > > > > and
> > > > > > > > > > > > > latest and a
> > > > > > > > > > > > > > > > > > > > timestamp value as start point.
> > > > > > > > > > > > > > > > > > > > So we can support 3 options for this
> > > > > > > configuration:
> > > > > > > > > > > > > "eariliest", "latest"
> > > > > > > > > > > > > > > > > > > > and a timestamp string value.
> > > > > > > > > > > > > > > > > > > > Of course, this can't solve 100%
> > cases, but I
> > > > > > > guess can
> > > > > > > > > > > > > sovle 80% or 90%
> > > > > > > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > > > And the remaining cases can be
> > resolved by
> > > > > LIKE
> > > > > > > syntax
> > > > > > > > > > > > which
> > > > > > > > > > > > > I guess is
> > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > very common cases.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > > Jark
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt
> > Young <
> > > > > > > > > > > ykt836@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Good to have such lovely
> > discussions. I
> > > > > also
> > > > > > > want to
> > > > > > > > > > > > share
> > > > > > > > > > > > > some of my
> > > > > > > > > > > > > > > > > > > > > opinions.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > #1 Regarding to error handling: I
> > also
> > > > > think
> > > > > > > ignore
> > > > > > > > > > > > > invalid hints would
> > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > dangerous, maybe
> > > > > > > > > > > > > > > > > > > > > the simplest solution is just throw
> > an
> > > > > > > exception.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > #2 Regarding to property
> > replacement: I
> > > > > don't
> > > > > > > think
> > > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > constraint
> > > > > > > > > > > > > > > > > > > > > ourself to
> > > > > > > > > > > > > > > > > > > > > the meaning of the word "hint", and
> > > > > forbidden
> > > > > > > it
> > > > > > > > > > > > modifying
> > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > > properties
> > > > > > > > > > > > > > > > > > > > > which can effect
> > > > > > > > > > > > > > > > > > > > > query results. IMO `PROPERTIES` is
> > one of
> > > > > the
> > > > > > > table
> > > > > > > > > > > > hints,
> > > > > > > > > > > > > and a
> > > > > > > > > > > > > > > > > > > powerful
> > > > > > > > > > > > > > > > > > > > > one. It can
> > > > > > > > > > > > > > > > > > > > > modify properties located in DDL's
> > WITH
> > > > > block.
> > > > > > > But I
> > > > > > > > > > > also
> > > > > > > > > > > > > see the harm
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > if we make it
> > > > > > > > > > > > > > > > > > > > > too flexible like change the kafka
> > topic
> > > > > name
> > > > > > > with a
> > > > > > > > > > > > hint.
> > > > > > > > > > > > > Such use
> > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > not common and
> > > > > > > > > > > > > > > > > > > > > sounds very dangerous to me. I
> would
> > > > > propose
> > > > > > > we have
> > > > > > > > > > a
> > > > > > > > > > > > map
> > > > > > > > > > > > > of hintable
> > > > > > > > > > > > > > > > > > > > > properties for each
> > > > > > > > > > > > > > > > > > > > > connector, and should validate all
> > passed
> > > > > in
> > > > > > > > > > properties
> > > > > > > > > > > > > are actually
> > > > > > > > > > > > > > > > > > > > > hintable. And combining with
> > > > > > > > > > > > > > > > > > > > > #1 error handling, we can throw an
> > > > > exception
> > > > > > > once
> > > > > > > > > > > > received
> > > > > > > > > > > > > invalid
> > > > > > > > > > > > > > > > > > > > > property.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > #3 Regarding to global offset: I'm
> > not sure
> > > > > > > it's
> > > > > > > > > > > > feasible.
> > > > > > > > > > > > > Different
> > > > > > > > > > > > > > > > > > > > > connectors will have totally
> > > > > > > > > > > > > > > > > > > > > different properties to represent
> > offset,
> > > > > some
> > > > > > > might
> > > > > > > > > > be
> > > > > > > > > > > > > timestamps,
> > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > might be string literals
> > > > > > > > > > > > > > > > > > > > > like "earliest", and others might
> be
> > just
> > > > > > > integers.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM
> > Jark Wu <
> > > > > > > > > > > > imjark@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 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 <
> > > > > > > > > > > > > twalthr@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 <
> > > > > > > > > > > > > twalthr@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.lwl@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.cyz@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 <
> > > > > > > > > > > > > twalthr@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
> > > > > > > > > > > > <
> > > > > > > > > > > > > imjark@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 <
> > > > > > > > > > > > > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jark Wu <im...@gmail.com>.
Hi everyone,

Sorry, but I'm not sure about the `supportedHintOptions`. I'm afraid it
doesn't solve the problems but increases some development and learning
burdens.

# increase development and learning burden

According to the discussion so far, we want to support overriding a subset
of options in hints which doesn't affect semantics.
With the `supportedHintOptions`, it's up to the connector developers to
decide which options will not affect semantics, and to be hint options.
However, the question is how to distinguish whether an option will *affect
semantics*? What happens if an option will affect semantics but provided as
hint options?
From my point of view, it's not easy to distinguish. For example, the
"format.ignore-parse-error" can be a very useful dynamic option but that
will affect semantic, because the result is different (null vs exception).
Another example, the "connector.lookup.cache.*" options are also very
useful to tune jobs, however, it will also affect the job results. I can
come up many more useful options but may affect semantics.

I can see that the community will under endless discussion around "can this
option to be a hint option?",  "wether this option will affect semantics?".
You can also find that we already have different opinions on
"ignore-parse-error". Those discussion is a waste of time! That's not what
users want!
The problem is user need this, this, this options and HOW to expose them?
We should focus on that.

Then there could be two endings in the future:
1) compromise on the usability, we drop the rule that hints don't affect
semantics, allow all the useful options in the hints list.
2) stick on the rule, users will find this is a stumbling feature which
doesn't solve their problems.
    And they will be surprised why this option can't be set, but the other
could. *semantic* is hard to be understood by users.

# doesn't solve the problems

I think the purpose of this FLIP is to allow users to quickly override some
connectors' properties to tune their jobs.
However, `supportedHintOptions` is off track. It only allows a subset
options and for the users it's not *clear* which subset is allowed.

Besides, I'm not sure `supportedHintOptions` can work well for all cases.
How could you support kafka properties (`connector.properties.*`) as hint
options? Some kafka properties may affect semantics (bootstrap.servers),
some may not (max.poll.records). Besides, I think it's not possible to list
all the possible kafka properties [1].

In summary, IMO, `supportedHintOptions`
(1) it increase the complexity to develop a connector
(2) it confuses users which options can be used in hint, which are not,
they have to check the docs again and again.
(3) it doesn't solve the problems which we want to solve by this FLIP.

I think we should avoid introducing some partial solutions. Otherwise, we
will be stuck in a loop that introduce new API -> deprecate API ->
introduce new API....

I personally in favor of an explicit WITH syntax after the table as a part
of the query which is mentioned by Kurt before, e.g. SELECT * from T
WITH('key' = 'value') .
It allows users to dynamically set options which can affect semantics. It
will be very flexible to solve users' problems so far.

Best,
Jark

[1]: https://kafka.apache.org/documentation/#consumerconfigs

On Wed, 18 Mar 2020 at 21:44, Danny Chan <yu...@gmail.com> wrote:

> My POC is here for the hints options merge [1].
>
> Personally, I have no strong objections for splitting hints with the
> CatalogTable, the only cons is a more complex implementation but the
> concept is more clear, and I have updated the WIKI.
>
> I think it would be nice if we can support the format “ignore-parse error”
> option key, the CSV source already has a key [2] and we can use that in the
> supportedHIntOptions, for the common CSV and JSON formats, we cal also give
> a support. This is the only kind of key in formats that “do not change the
> semantics” (somehow), what do you think about this ~
>
> [1]
> https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
> [2]
> https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92
>
> Best,
> Danny Chan
> 在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
> > Hi everyone,
> >
> > +1 to Kurt's suggestion. Let's just have it in source and sink factories
> > for now. We can still move this method up in the future. Currently, I
> > don't see a need for catalogs or formats. Because how would you target a
> > format in the query?
> >
> > @Danny: Can you send a link to your PoC? I'm very skeptical about
> > creating a new CatalogTable in planner. Actually CatalogTable should be
> > immutable between Catalog and Factory. Because a catalog can return its
> > own factory and fully control the instantiation. Depending on the
> > implementation, that means it can be possible that the catalog has
> > encoded more information in a concrete subclass implementing the
> > interface. I vote for separating the concerns of catalog information and
> > hints in the factory explicitly.
> >
> > Regards,
> > Timo
> >
> >
> > On 18.03.20 05:41, Jingsong Li wrote:
> > > Hi,
> > >
> > > I am thinking we can provide hints to *table* related instances.
> > > - TableFormatFactory: of cause we need hints support, there are many
> format
> > > options in DDL too.
> > > - catalog and module: I don't know, maybe in future we can provide some
> > > hints for them.
> > >
> > > Best,
> > > Jingsong Lee
> > >
> > > On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com>
> wrote:
> > >
> > > > Yes, I think we should move the `supportedHintOptions` from
> TableFactory
> > > > to TableSourceFactory, and we also need to add the interface to
> > > > TableSinkFactory though because sink target table may also have hints
> > > > attached.
> > > >
> > > > Best,
> > > > Danny Chan
> > > > 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> > > > > Have one question for adding `supportedHintOptions` method to
> > > > > `TableFactory`. It seems
> > > > > `TableFactory` is a base factory interface for all *table module*
> related
> > > > > instances, such as
> > > > > catalog, module, format and so on. It's not created only for
> *table*. Is
> > > > it
> > > > > possible to move it
> > > > > to `TableSourceFactory`?
> > > > >
> > > > > Best,
> > > > > Kurt
> > > > >
> > > > >
> > > > > On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <yu...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Thanks Timo ~
> > > > > >
> > > > > > For the naming itself, I also think the PROPERTIES is not that
> > > > concise, so
> > > > > > +1 for OPTIONS (I had thought about that, but there are many
> codes in
> > > > > > current Flink called it properties, i.e. the
> DescriptorProperties,
> > > > > > #getSupportedProperties), let’s use OPTIONS if this is our new
> > > > preference.
> > > > > >
> > > > > > +1 to `Set<ConfigOption> supportedHintOptions()` because the
> > > > ConfigOption
> > > > > > can take more info. AFAIK, Spark also call their table options
> instead
> > > > of
> > > > > > properties. [1]
> > > > > >
> > > > > > In my local POC, I did create a new CatalogTable, and it works
> for
> > > > current
> > > > > > connectors well, all the DDL tables would finally yield a
> CatalogTable
> > > > > > instance and we can apply the options to that(in the
> CatalogSourceTable
> > > > > > when we generating the TableSource), the pros is that we do not
> need to
> > > > > > modify the codes of connectors itself. If we split the options
> from
> > > > > > CatalogTable, we may need to add some additional logic in each
> > > > connector
> > > > > > factories in order to merge these properties (and the logic are
> almost
> > > > the
> > > > > > same), what do you think about this?
> > > > > >
> > > > > > [1]
> > > > > >
> > > >
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> > > > > >
> > > > > > Best,
> > > > > > Danny Chan
> > > > > > 在 2020年3月17日 +0800 PM10:10,Timo Walther <tw...@apache.org>,写道:
> > > > > > > Hi Danny,
> > > > > > >
> > > > > > > thanks for updating the FLIP. I think your current design is
> > > > sufficient
> > > > > > > to separate hints from result-related properties.
> > > > > > >
> > > > > > > One remark to the naming itself: I would vote for calling the
> hints
> > > > > > > around table scan `OPTIONS('k'='v')`. We used the term
> "properties"
> > > > in
> > > > > > > the past but since we want to unify the Flink configuration
> > > > experience,
> > > > > > > we should use consistent naming and classes around
> `ConfigOptions`.
> > > > > > >
> > > > > > > It would be nice to use `Set<ConfigOption>
> supportedHintOptions();`
> > > > to
> > > > > > > start using config options instead of pure string properties.
> This
> > > > will
> > > > > > > also allow us to generate documentation in the future around
> > > > supported
> > > > > > > data types, ranges, etc. for options. At some point we would
> also
> > > > like
> > > > > > > to drop `DescriptorProperties` class. "Options" is also used
> in the
> > > > > > > documentation [1] and in the SQL/MED standard [2].
> > > > > > >
> > > > > > > Furthermore, I would still vote for separating CatalogTable
> and hint
> > > > > > > options. Otherwise the planner would need to create a new
> > > > CatalogTable
> > > > > > > instance which might not always be easy. We should offer them
> via:
> > > > > > >
> > > > > > >
> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > > > > > > ReadableConfig
> > > > > > >
> > > > > > > What do you think?
> > > > > > >
> > > > > > > Regards,
> > > > > > > Timo
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > > >
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > > > > > > [2] https://wiki.postgresql.org/wiki/SQL/MED
> > > > > > >
> > > > > > >
> > > > > > > On 12.03.20 15:06, Stephan Ewen wrote:
> > > > > > > > @Danny sounds good.
> > > > > > > >
> > > > > > > > Maybe it is worth listing all the classes of problems that
> you
> > > > want to
> > > > > > > > address and then look at each class and see if hints are a
> good
> > > > default
> > > > > > > > solution or a good optional way of simplifying things?
> > > > > > > > The discussion has grown a lot and it is starting to be hard
> to
> > > > > > distinguish
> > > > > > > > the parts where everyone agrees from the parts were there are
> > > > concerns.
> > > > > > > >
> > > > > > > > On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <
> danny0405@apache.org>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks Stephan ~
> > > > > > > > >
> > > > > > > > > We can remove the support for properties that may change
> the
> > > > > > semantics of
> > > > > > > > > query if you think that is a trouble.
> > > > > > > > >
> > > > > > > > > How about we support the /*+ properties() */ hint only for
> those
> > > > > > optimize
> > > > > > > > > parameters, such as the fetch size of source or something
> like
> > > > that,
> > > > > > does
> > > > > > > > > that make sense?
> > > > > > > > >
> > > > > > > > > Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > > > > > > > >
> > > > > > > > > > I think Bowen has actually put it very well.
> > > > > > > > > >
> > > > > > > > > > (1) Hints that change semantics looks like trouble
> waiting to
> > > > > > happen. For
> > > > > > > > > > example Kafka offset handling should be in filters. The
> Kafka
> > > > > > source
> > > > > > > > > should
> > > > > > > > > > support predicate pushdown.
> > > > > > > > > >
> > > > > > > > > > (2) Hints should not be a workaround for current
> shortcomings.
> > > > A
> > > > > > lot of
> > > > > > > > > the
> > > > > > > > > > suggested above sounds exactly like that. Working around
> > > > > > catalog/DDL
> > > > > > > > > > shortcomings, missing exposure of metadata (offsets),
> missing
> > > > > > predicate
> > > > > > > > > > pushdown in Kafka. Abusing a feature like hints now as a
> quick
> > > > fix
> > > > > > for
> > > > > > > > > > these issues, rather than fixing the root causes, will
> much
> > > > likely
> > > > > > bite
> > > > > > > > > us
> > > > > > > > > > back badly in the future.
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Stephan
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <
> ykt836@gmail.com>
> > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > It seems this FLIP's name is somewhat misleading. From
> my
> > > > > > > > > understanding,
> > > > > > > > > > > this FLIP is trying to
> > > > > > > > > > > address the dynamic parameter issue, and table hints
> is the
> > > > way
> > > > > > we wan
> > > > > > > > > to
> > > > > > > > > > > choose. I think we should
> > > > > > > > > > > be focus on "what's the right way to solve dynamic
> property"
> > > > > > instead of
> > > > > > > > > > > discussing "whether table
> > > > > > > > > > > hints can affect query semantics".
> > > > > > > > > > >
> > > > > > > > > > > For now, there are two proposed ways to achieve dynamic
> > > > property:
> > > > > > > > > > > 1. FLIP-110: create temporary table xx like xx with
> (xxx)
> > > > > > > > > > > 2. use custom "from t with (xxx)" syntax
> > > > > > > > > > > 3. "Borrow" the table hints to have a special
> PROPERTIES
> > > > hint.
> > > > > > > > > > >
> > > > > > > > > > > The first one didn't break anything, but the only
> problem i
> > > > see
> > > > > > is a
> > > > > > > > > > little
> > > > > > > > > > > more verbose than the table hint
> > > > > > > > > > > approach. I can imagine when someone using SQL CLI to
> have a
> > > > sql
> > > > > > > > > > > experience, it's quite often that
> > > > > > > > > > > he will modify the table property, some use cases i can
> > > > think of:
> > > > > > > > > > > 1. the source contains some corrupted data, i want to
> turn
> > > > on the
> > > > > > > > > > > "ignore-error" flag for certain formats.
> > > > > > > > > > > 2. I have a kafka table and want to see some sample
> data
> > > > from the
> > > > > > > > > > > beginning, so i change the offset
> > > > > > > > > > > to "earliest", and then I want to observe the latest
> data
> > > > which
> > > > > > keeps
> > > > > > > > > > > coming in. I would write another query
> > > > > > > > > > > to select from the latest table.
> > > > > > > > > > > 3. I want to my jdbc sink flush data more eagerly then
> i can
> > > > > > observe
> > > > > > > > > the
> > > > > > > > > > > data from database side.
> > > > > > > > > > >
> > > > > > > > > > > Most of such use cases are quite ad-hoc. If every time
> I
> > > > want to
> > > > > > have a
> > > > > > > > > > > different experience, i need to create
> > > > > > > > > > > a temporary table and then also modify my query, it
> doesn't
> > > > feel
> > > > > > > > > smooth.
> > > > > > > > > > > Embed such dynamic property into
> > > > > > > > > > > query would have better user experience.
> > > > > > > > > > >
> > > > > > > > > > > Both 2 & 3 can make this happen. The cons of #2 is
> breaking
> > > > SQL
> > > > > > > > > > compliant,
> > > > > > > > > > > and for #3, it only breaks some
> > > > > > > > > > > unwritten rules, but we can have an explanation on
> that. And
> > > > I
> > > > > > really
> > > > > > > > > > doubt
> > > > > > > > > > > whether user would complain about
> > > > > > > > > > > this when they actually have flexible and good
> experience
> > > > using
> > > > > > this.
> > > > > > > > > > >
> > > > > > > > > > > My tendency would be #3 > #1 > #2, what do you think?
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Kurt
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> > > > yuzhao.cyz@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Thanks Aljoscha ~
> > > > > > > > > > > >
> > > > > > > > > > > > I agree for most of the query hints, they are
> optional as
> > > > an
> > > > > > > > > optimizer
> > > > > > > > > > > > instruction, especially for the traditional RDBMS.
> > > > > > > > > > > >
> > > > > > > > > > > > But, just like BenChao said, Flink as a computation
> engine
> > > > has
> > > > > > many
> > > > > > > > > > > > different kind of data sources, thus, dynamic
> parameters
> > > > like
> > > > > > > > > > > start_offest
> > > > > > > > > > > > can only bind to each table scope, we can not set a
> session
> > > > > > config
> > > > > > > > > like
> > > > > > > > > > > > KSQL because they are all about Kafka:
> > > > > > > > > > > > > SET ‘auto.offset.reset’=‘earliest’;
> > > > > > > > > > > >
> > > > > > > > > > > > Thus the most flexible way to set up these dynamic
> params
> > > > is
> > > > > > to bind
> > > > > > > > > to
> > > > > > > > > > > > the table scope in the query when we want to override
> > > > > > something, so
> > > > > > > > > we
> > > > > > > > > > > have
> > > > > > > > > > > > these solutions above (with pros and cons from my
> side):
> > > > > > > > > > > >
> > > > > > > > > > > > • 1. Select * from t(offset=123) (from Timo)
> > > > > > > > > > > >
> > > > > > > > > > > > Pros:
> > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > - Parameters are part of the main query
> > > > > > > > > > > > Cons:
> > > > > > > > > > > > - Not SQL compliant
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > • 2. Select * from t /*+ PROPERTIES(offset=123) */
> (from
> > > > me)
> > > > > > > > > > > >
> > > > > > > > > > > > Pros:
> > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > - SQL compliant because it is nested in the comments
> > > > > > > > > > > >
> > > > > > > > > > > > Cons:
> > > > > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > > > >
> > > > > > > > > > > > The biggest problem for hints way may be the “if
> hints
> > > > must be
> > > > > > > > > > optional”,
> > > > > > > > > > > > actually we have though about 1 for a while but
> aborted
> > > > > > because it
> > > > > > > > > > breaks
> > > > > > > > > > > > the SQL standard too much. And we replace it with 2,
> > > > because
> > > > > > the
> > > > > > > > > hints
> > > > > > > > > > > > syntax do not break SQL standard(nested in comments).
> > > > > > > > > > > >
> > > > > > > > > > > > What if we have the special /*+ PROPERTIES */ hint
> that
> > > > allows
> > > > > > > > > override
> > > > > > > > > > > > some properties of table dynamically, it does not
> break
> > > > > > anything, at
> > > > > > > > > > > lease
> > > > > > > > > > > > for current Flink use cases.
> > > > > > > > > > > >
> > > > > > > > > > > > Planner hints are optional just because they are
> naturally
> > > > > > enforcers
> > > > > > > > > of
> > > > > > > > > > > > the planner, most of them aim to instruct the
> optimizer,
> > > > but,
> > > > > > the
> > > > > > > > > table
> > > > > > > > > > > > hints is a little different, table hints can specify
> the
> > > > table
> > > > > > meta
> > > > > > > > > > like
> > > > > > > > > > > > index column, and it is very convenient to specify
> table
> > > > > > properties.
> > > > > > > > > > > >
> > > > > > > > > > > > Or shall we not call /*+ PROPERTIES(offset=123) */
> table
> > > > hint,
> > > > > > we
> > > > > > > > > can
> > > > > > > > > > > > call it table dynamic parameters.
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> > > > > > aljoscha@apache.org>,写道:
> > > > > > > > > > > > > Hi,
> > > > > > > > > > > > >
> > > > > > > > > > > > > I don't understand this discussion. Hints, as I
> > > > understand
> > > > > > them,
> > > > > > > > > > should
> > > > > > > > > > > > > work like this:
> > > > > > > > > > > > >
> > > > > > > > > > > > > - hints are *optional* advice for the optimizer to
> try
> > > > and
> > > > > > help it
> > > > > > > > > to
> > > > > > > > > > > > > find a good execution strategy
> > > > > > > > > > > > > - hints should not change query semantics, i.e.
> they
> > > > should
> > > > > > not
> > > > > > > > > > change
> > > > > > > > > > > > > connector properties executing a query with taking
> into
> > > > > > account the
> > > > > > > > > > > > > hints *must* produce the same result as executing
> the
> > > > query
> > > > > > without
> > > > > > > > > > > > > taking into account the hints
> > > > > > > > > > > > >
> > > > > > > > > > > > > From these simple requirements you can derive a
> solution
> > > > > > that makes
> > > > > > > > > > > > > sense. I don't have a strong preference for the
> syntax
> > > > but we
> > > > > > > > > should
> > > > > > > > > > > > > strive to be in line with prior work.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Aljoscha
> > > > > > > > > > > > >
> > > > > > > > > > > > > On 11.03.20 11:53, Danny Chan wrote:
> > > > > > > > > > > > > > Thanks Timo for summarize the 3 options ~
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I agree with Kurt that option2 is too
> complicated to
> > > > use
> > > > > > because:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > • As a Kafka topic consumer, the user must
> define both
> > > > the
> > > > > > > > > virtual
> > > > > > > > > > > > column for start offset and he must apply a special
> filter
> > > > > > predicate
> > > > > > > > > > > after
> > > > > > > > > > > > each query
> > > > > > > > > > > > > > • And for the internal implementation, the
> metadata
> > > > column
> > > > > > push
> > > > > > > > > > down
> > > > > > > > > > > > is another hard topic, each kind of message queue
> may have
> > > > its
> > > > > > offset
> > > > > > > > > > > > attribute, we need to consider the expression type
> for
> > > > > > different
> > > > > > > > > kind;
> > > > > > > > > > > the
> > > > > > > > > > > > source also need to recognize the constant column as
> a
> > > > config
> > > > > > > > > > > option(which
> > > > > > > > > > > > is weird because usually what we pushed down is a
> table
> > > > column)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > For option 1 and option3, I think there is no
> > > > difference,
> > > > > > option1
> > > > > > > > > > is
> > > > > > > > > > > > also a hint syntax which is introduced in Sybase and
> > > > > > referenced then
> > > > > > > > > > > > deprecated by MS-SQL in 199X years because of the
> > > > > > ambitiousness.
> > > > > > > > > > > Personally
> > > > > > > > > > > > I prefer /*+ */ style table hint than WITH keyword
> for
> > > > these
> > > > > > reasons:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > • We do not break the standard SQL, the hints are
> > > > nested
> > > > > > in SQL
> > > > > > > > > > > > comments
> > > > > > > > > > > > > > • We do not need to introduce additional WITH
> keyword
> > > > > > which may
> > > > > > > > > > > appear
> > > > > > > > > > > > in a query if we use that because a table can be
> > > > referenced in
> > > > > > all
> > > > > > > > > > kinds
> > > > > > > > > > > of
> > > > > > > > > > > > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would
> make
> > > > our
> > > > > > sql
> > > > > > > > > query
> > > > > > > > > > > > break too much of the SQL from standard
> > > > > > > > > > > > > > • We would have uniform syntax for hints as query
> > > > hint, one
> > > > > > > > > syntax
> > > > > > > > > > > > fits all and more easy to use
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > And here is the reason why we choose a uniform
> Oracle
> > > > > > style query
> > > > > > > > > > > > hint syntax which is addressed by Julian Hyde when we
> > > > design
> > > > > > the
> > > > > > > > > syntax
> > > > > > > > > > > > from the Calcite community:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I don’t much like the MSSQL-style syntax for
> table
> > > > hints.
> > > > > > It
> > > > > > > > > adds a
> > > > > > > > > > > > new use of the WITH keyword that is unrelated to the
> use of
> > > > > > WITH for
> > > > > > > > > > > > common-table expressions.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > A historical note. Microsoft SQL Server
> inherited its
> > > > hint
> > > > > > syntax
> > > > > > > > > > > from
> > > > > > > > > > > > Sybase a very long time ago. (See “Transact SQL
> > > > > > Programming”[1], page
> > > > > > > > > > > 632,
> > > > > > > > > > > > “Optimizer hints”. The book was written in 1999, and
> covers
> > > > > > Microsoft
> > > > > > > > > > SQL
> > > > > > > > > > > > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5,
> but the
> > > > > > syntax very
> > > > > > > > > > > > likely predates Sybase 4.3, from which Microsoft SQL
> > > > Server was
> > > > > > > > > forked
> > > > > > > > > > in
> > > > > > > > > > > > 1993.)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Microsoft later added the WITH keyword to make
> it less
> > > > > > ambiguous,
> > > > > > > > > > and
> > > > > > > > > > > > has now deprecated the syntax that does not use WITH.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > They are forced to keep the syntax for backwards
> > > > > > compatibility
> > > > > > > > > but
> > > > > > > > > > > > that doesn’t mean that we should shoulder their
> burden.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I think formatted comments are the right
> container for
> > > > > > hints
> > > > > > > > > > because
> > > > > > > > > > > > it allows us to change the hint syntax without
> changing
> > > > the SQL
> > > > > > > > > parser,
> > > > > > > > > > > and
> > > > > > > > > > > > makes clear that we are at liberty to ignore hints
> > > > entirely.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Julian
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > > > > > > > > > > > https://www.amazon.com/s?k=9781565924017>
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> > > > twalthr@apache.org
> > > > > > > ,写道:
> > > > > > > > > > > > > > > Hi Danny,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > it is true that our DDL is not standard
> compliant by
> > > > > > using the
> > > > > > > > > > WITH
> > > > > > > > > > > > > > > clause. Nevertheless, we aim for not diverging
> too
> > > > much
> > > > > > and the
> > > > > > > > > > > LIKE
> > > > > > > > > > > > > > > clause is an example of that. It will solve
> things
> > > > like
> > > > > > > > > > overwriting
> > > > > > > > > > > > > > > WATERMARKs, add additional/modifying
> properties and
> > > > > > inherit
> > > > > > > > > > schema.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Bowen is right that Flink's DDL is mixing 3
> types
> > > > > > definition
> > > > > > > > > > > > together.
> > > > > > > > > > > > > > > We are not the first ones that try to solve
> this.
> > > > There
> > > > > > is also
> > > > > > > > > > the
> > > > > > > > > > > > SQL
> > > > > > > > > > > > > > > MED standard [1] that tried to tackle this
> problem. I
> > > > > > think it
> > > > > > > > > > was
> > > > > > > > > > > > not
> > > > > > > > > > > > > > > considered when designing the current DDL.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Currently, I see 3 options for handling Kafka
> > > > offsets. I
> > > > > > will
> > > > > > > > > > give
> > > > > > > > > > > > some
> > > > > > > > > > > > > > > examples and look forward to feedback here:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > *Option 1* Runtime and semantic parms as part
> of the
> > > > > > query
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > > > > - Parameters are part of the main query
> > > > > > > > > > > > > > > - No complicated hinting syntax
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > > > - Not SQL compliant
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > *Option 2* Use metadata in query
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > `CREATE TABLE MyTable (id INT, offset AS
> > > > > > > > > > > SYSTEM_METADATA('offset'))`
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP
> > > > > > '2012-12-12
> > > > > > > > > > > > 12:34:22'`
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > > > - SQL compliant in the query
> > > > > > > > > > > > > > > - Access of metadata in the DDL which is
> required
> > > > anyway
> > > > > > > > > > > > > > > - Regular pushdown rules apply
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > > > - Users need to add an additional comlumn in
> the DDL
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > *Option 3*: Use hints for properties
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > `
> > > > > > > > > > > > > > > SELECT *
> > > > > > > > > > > > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > > > > > > > > > > > `
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > > > > > > > - Not standard compliant.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > If we go with this option, I would suggest to
> make it
> > > > > > available
> > > > > > > > > > in
> > > > > > > > > > > a
> > > > > > > > > > > > > > > separate map and don't mix it with statically
> defined
> > > > > > > > > properties.
> > > > > > > > > > > > Such
> > > > > > > > > > > > > > > that the factory can decide which properties
> have the
> > > > > > right to
> > > > > > > > > be
> > > > > > > > > > > > > > > overwritten by the hints:
> > > > > > > > > > > > > > > TableSourceFactory.Context.getQueryHints():
> > > > > > ReadableConfig
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > > > Timo
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Currently I see 3 options as a
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > > > > > > > > > > > Thanks Bowen ~
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I agree we should somehow categorize our
> connector
> > > > > > > > > parameters.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > For type1, I’m already preparing a solution
> like
> > > > the
> > > > > > > > > Confluent
> > > > > > > > > > > > schema registry + Avro schema inference thing, so
> this may
> > > > not
> > > > > > be a
> > > > > > > > > > > problem
> > > > > > > > > > > > in the near future.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > For type3, I have some questions:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > "SELECT * FROM mykafka WHERE offset > 12pm
> > > > yesterday”
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Where does the offset column come from, a
> virtual
> > > > > > column from
> > > > > > > > > > the
> > > > > > > > > > > > table schema, you said that
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > They change
> > > > > > > > > > > > > > > > almost every time a query starts and have
> nothing
> > > > to
> > > > > > do with
> > > > > > > > > > > > metadata, thus
> > > > > > > > > > > > > > > > should not be part of table definition/DDL
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > But why you can reference it in the query,
> I’m
> > > > > > confused for
> > > > > > > > > > that,
> > > > > > > > > > > > can you elaborate a little ?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> > > > > > bowenli86@gmail.com
> > > > > > > > > > ,写道:
> > > > > > > > > > > > > > > > > Thanks Danny for kicking off the effort
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > The root cause of too much manual work is
> Flink
> > > > DDL
> > > > > > has
> > > > > > > > > > mixed 3
> > > > > > > > > > > > types of
> > > > > > > > > > > > > > > > > params together and doesn't handle each of
> them
> > > > very
> > > > > > well.
> > > > > > > > > > > Below
> > > > > > > > > > > > are how I
> > > > > > > > > > > > > > > > > categorize them and corresponding
> solutions in my
> > > > > > mind:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > - type 1: Metadata of external data, like
> > > > external
> > > > > > > > > > > endpoint/url,
> > > > > > > > > > > > > > > > > username/pwd, schemas, formats.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Such metadata are mostly already
> accessible in
> > > > > > external
> > > > > > > > > > system
> > > > > > > > > > > > as long as
> > > > > > > > > > > > > > > > > endpoints and credentials are provided.
> Flink can
> > > > > > get it
> > > > > > > > > thru
> > > > > > > > > > > > catalogs, but
> > > > > > > > > > > > > > > > > we haven't had many catalogs yet and thus
> Flink
> > > > just
> > > > > > hasn't
> > > > > > > > > > > been
> > > > > > > > > > > > able to
> > > > > > > > > > > > > > > > > leverage that. So the solution should be
> building
> > > > > > more
> > > > > > > > > > > catalogs.
> > > > > > > > > > > > Such
> > > > > > > > > > > > > > > > > params should be part of a Flink table
> > > > > > DDL/definition, and
> > > > > > > > > > not
> > > > > > > > > > > > overridable
> > > > > > > > > > > > > > > > > in any means.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > - type 2: Runtime params, like jdbc
> connector's
> > > > > > fetch size,
> > > > > > > > > > > > elasticsearch
> > > > > > > > > > > > > > > > > connector's bulk flush size.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Such params don't affect query results, but
> > > > affect
> > > > > > how
> > > > > > > > > > results
> > > > > > > > > > > > are produced
> > > > > > > > > > > > > > > > > (eg. fast or slow, aka performance) - they
> are
> > > > > > essentially
> > > > > > > > > > > > execution and
> > > > > > > > > > > > > > > > > implementation details. They change often
> in
> > > > > > exploration or
> > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > stages, but not quite frequently in
> well-defined
> > > > > > > > > long-running
> > > > > > > > > > > > pipelines.
> > > > > > > > > > > > > > > > > They should always have default values and
> can be
> > > > > > missing
> > > > > > > > > in
> > > > > > > > > > > > query. They
> > > > > > > > > > > > > > > > > can be part of a table DDL/definition, but
> should
> > > > > > also be
> > > > > > > > > > > > replaceable in a
> > > > > > > > > > > > > > > > > query - *this is what table "hints" in
> FLIP-113
> > > > > > should
> > > > > > > > > > cover*.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > - type 3: Semantic params, like kafka
> connector's
> > > > > > start
> > > > > > > > > > offset.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Such params affect query results - the
> semantics.
> > > > > > They'd
> > > > > > > > > > better
> > > > > > > > > > > > be as
> > > > > > > > > > > > > > > > > filter conditions in WHERE clause that can
> be
> > > > pushed
> > > > > > down.
> > > > > > > > > > They
> > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > almost every time a query starts and have
> > > > nothing to
> > > > > > do
> > > > > > > > > with
> > > > > > > > > > > > metadata, thus
> > > > > > > > > > > > > > > > > should not be part of table
> definition/DDL, nor
> > > > be
> > > > > > > > > persisted
> > > > > > > > > > in
> > > > > > > > > > > > catalogs.
> > > > > > > > > > > > > > > > > If they will, users should create views to
> keep
> > > > such
> > > > > > params
> > > > > > > > > > > > around (note
> > > > > > > > > > > > > > > > > this is different from variable
> substitution).
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Take Flink-Kafka as an example. Once we
> get these
> > > > > > params
> > > > > > > > > > right,
> > > > > > > > > > > > here're the
> > > > > > > > > > > > > > > > > steps users need to do to develop and run
> a Flink
> > > > > > job:
> > > > > > > > > > > > > > > > > - configure a Flink
> ConfluentSchemaRegistry with
> > > > url,
> > > > > > > > > > username,
> > > > > > > > > > > > and password
> > > > > > > > > > > > > > > > > - run "SELECT * FROM mykafka WHERE offset
> > 12pm
> > > > > > yesterday"
> > > > > > > > > > > > (simplified
> > > > > > > > > > > > > > > > > timestamp) in SQL CLI, Flink automatically
> > > > retrieves
> > > > > > all
> > > > > > > > > > > > metadata of
> > > > > > > > > > > > > > > > > schema, file format, etc and start the job
> > > > > > > > > > > > > > > > > - users want to make the job read Kafka
> topic
> > > > > > faster, so it
> > > > > > > > > > > goes
> > > > > > > > > > > > as "SELECT
> > > > > > > > > > > > > > > > > * FROM mykafka /* faster_read_key=value*/
> WHERE
> > > > > > offset >
> > > > > > > > > 12pm
> > > > > > > > > > > > yesterday"
> > > > > > > > > > > > > > > > > - done and satisfied, users submit it to
> > > > production
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1,
> > > > k2=v2),
> > > > > > I think
> > > > > > > > > > > it's
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > nice-to-have feature, but not a
> strategically
> > > > > > critical,
> > > > > > > > > > > > long-term solution,
> > > > > > > > > > > > > > > > > because
> > > > > > > > > > > > > > > > > 1) It may seem promising at the current
> stage to
> > > > > > solve the
> > > > > > > > > > > > > > > > > too-much-manual-work problem, but that's
> only
> > > > > > because Flink
> > > > > > > > > > > > hasn't
> > > > > > > > > > > > > > > > > leveraged catalogs well and handled the 3
> types
> > > > of
> > > > > > params
> > > > > > > > > > above
> > > > > > > > > > > > properly.
> > > > > > > > > > > > > > > > > Once we get the params types right, the
> LIKE
> > > > syntax
> > > > > > won't
> > > > > > > > > be
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > > important, and will be just an easier way
> to
> > > > create
> > > > > > tables
> > > > > > > > > > > > without retyping
> > > > > > > > > > > > > > > > > long fields like username and pwd.
> > > > > > > > > > > > > > > > > 2) Note that only some rare type of
> catalog can
> > > > > > store k-v
> > > > > > > > > > > > property pair, so
> > > > > > > > > > > > > > > > > table created this way often cannot be
> > > > persisted. In
> > > > > > the
> > > > > > > > > > > > foreseeable
> > > > > > > > > > > > > > > > > future, such catalog will only be
> HiveCatalog,
> > > > and
> > > > > > not
> > > > > > > > > > everyone
> > > > > > > > > > > > has a Hive
> > > > > > > > > > > > > > > > > metastore. To be honest, without
> persistence,
> > > > > > recreating
> > > > > > > > > > tables
> > > > > > > > > > > > every time
> > > > > > > > > > > > > > > > > this way is still a lot of keyboard typing.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Cheers,
> > > > > > > > > > > > > > > > > Bowen
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young
> <
> > > > > > > > > ykt836@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > If a specific connector want to have such
> > > > > > parameter and
> > > > > > > > > > read
> > > > > > > > > > > > if out of
> > > > > > > > > > > > > > > > > > configuration, then that's fine.
> > > > > > > > > > > > > > > > > > If we are talking about a configuration
> for all
> > > > > > kinds of
> > > > > > > > > > > > sources, I would
> > > > > > > > > > > > > > > > > > be super careful about that.
> > > > > > > > > > > > > > > > > > It's true it can solve maybe 80% cases,
> but it
> > > > > > will also
> > > > > > > > > > make
> > > > > > > > > > > > the left 20%
> > > > > > > > > > > > > > > > > > feels weird.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu
> <
> > > > > > > > > imjark@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi Kurt,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > > > > > > > > > > > I'm not saying to use the global
> > > > configuration to
> > > > > > > > > > override
> > > > > > > > > > > > connector
> > > > > > > > > > > > > > > > > > > properties by the planner.
> > > > > > > > > > > > > > > > > > > But the connector should take this
> > > > configuration
> > > > > > and
> > > > > > > > > > > > translate into their
> > > > > > > > > > > > > > > > > > > client API.
> > > > > > > > > > > > > > > > > > > AFAIK, almost all the message queues
> support
> > > > > > eariliest
> > > > > > > > > > and
> > > > > > > > > > > > latest and a
> > > > > > > > > > > > > > > > > > > timestamp value as start point.
> > > > > > > > > > > > > > > > > > > So we can support 3 options for this
> > > > > > configuration:
> > > > > > > > > > > > "eariliest", "latest"
> > > > > > > > > > > > > > > > > > > and a timestamp string value.
> > > > > > > > > > > > > > > > > > > Of course, this can't solve 100%
> cases, but I
> > > > > > guess can
> > > > > > > > > > > > sovle 80% or 90%
> > > > > > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > > And the remaining cases can be
> resolved by
> > > > LIKE
> > > > > > syntax
> > > > > > > > > > > which
> > > > > > > > > > > > I guess is
> > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > very common cases.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > Jark
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt
> Young <
> > > > > > > > > > ykt836@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Good to have such lovely
> discussions. I
> > > > also
> > > > > > want to
> > > > > > > > > > > share
> > > > > > > > > > > > some of my
> > > > > > > > > > > > > > > > > > > > opinions.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > #1 Regarding to error handling: I
> also
> > > > think
> > > > > > ignore
> > > > > > > > > > > > invalid hints would
> > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > dangerous, maybe
> > > > > > > > > > > > > > > > > > > > the simplest solution is just throw
> an
> > > > > > exception.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > #2 Regarding to property
> replacement: I
> > > > don't
> > > > > > think
> > > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > constraint
> > > > > > > > > > > > > > > > > > > > ourself to
> > > > > > > > > > > > > > > > > > > > the meaning of the word "hint", and
> > > > forbidden
> > > > > > it
> > > > > > > > > > > modifying
> > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > properties
> > > > > > > > > > > > > > > > > > > > which can effect
> > > > > > > > > > > > > > > > > > > > query results. IMO `PROPERTIES` is
> one of
> > > > the
> > > > > > table
> > > > > > > > > > > hints,
> > > > > > > > > > > > and a
> > > > > > > > > > > > > > > > > > powerful
> > > > > > > > > > > > > > > > > > > > one. It can
> > > > > > > > > > > > > > > > > > > > modify properties located in DDL's
> WITH
> > > > block.
> > > > > > But I
> > > > > > > > > > also
> > > > > > > > > > > > see the harm
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > if we make it
> > > > > > > > > > > > > > > > > > > > too flexible like change the kafka
> topic
> > > > name
> > > > > > with a
> > > > > > > > > > > hint.
> > > > > > > > > > > > Such use
> > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > not common and
> > > > > > > > > > > > > > > > > > > > sounds very dangerous to me. I would
> > > > propose
> > > > > > we have
> > > > > > > > > a
> > > > > > > > > > > map
> > > > > > > > > > > > of hintable
> > > > > > > > > > > > > > > > > > > > properties for each
> > > > > > > > > > > > > > > > > > > > connector, and should validate all
> passed
> > > > in
> > > > > > > > > properties
> > > > > > > > > > > > are actually
> > > > > > > > > > > > > > > > > > > > hintable. And combining with
> > > > > > > > > > > > > > > > > > > > #1 error handling, we can throw an
> > > > exception
> > > > > > once
> > > > > > > > > > > received
> > > > > > > > > > > > invalid
> > > > > > > > > > > > > > > > > > > > property.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > #3 Regarding to global offset: I'm
> not sure
> > > > > > it's
> > > > > > > > > > > feasible.
> > > > > > > > > > > > Different
> > > > > > > > > > > > > > > > > > > > connectors will have totally
> > > > > > > > > > > > > > > > > > > > different properties to represent
> offset,
> > > > some
> > > > > > might
> > > > > > > > > be
> > > > > > > > > > > > timestamps,
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > might be string literals
> > > > > > > > > > > > > > > > > > > > like "earliest", and others might be
> just
> > > > > > integers.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM
> Jark Wu <
> > > > > > > > > > > imjark@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 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 <
> > > > > > > > > > > > twalthr@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 <
> > > > > > > > > > > > twalthr@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.lwl@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.cyz@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 <
> > > > > > > > > > > > twalthr@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
> > > > > > > > > > > <
> > > > > > > > > > > > imjark@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 <
> > > > > > > > > > > > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
My POC is here for the hints options merge [1].

Personally, I have no strong objections for splitting hints with the CatalogTable, the only cons is a more complex implementation but the concept is more clear, and I have updated the WIKI.

I think it would be nice if we can support the format “ignore-parse error” option key, the CSV source already has a key [2] and we can use that in the supportedHIntOptions, for the common CSV and JSON formats, we cal also give a support. This is the only kind of key in formats that “do not change the semantics” (somehow), what do you think about this ~

[1] https://github.com/danny0405/flink/commit/5d925fa16c3c553423c4b7d93001521b8e6e6bee#diff-6e569a6dd124fd2091c18e2790fb49c5
[2] https://github.com/apache/flink/blob/b83060dff6d403b6994b6646b3f29a374f599530/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java#L92

Best,
Danny Chan
在 2020年3月18日 +0800 PM9:10,Timo Walther <tw...@apache.org>,写道:
> Hi everyone,
>
> +1 to Kurt's suggestion. Let's just have it in source and sink factories
> for now. We can still move this method up in the future. Currently, I
> don't see a need for catalogs or formats. Because how would you target a
> format in the query?
>
> @Danny: Can you send a link to your PoC? I'm very skeptical about
> creating a new CatalogTable in planner. Actually CatalogTable should be
> immutable between Catalog and Factory. Because a catalog can return its
> own factory and fully control the instantiation. Depending on the
> implementation, that means it can be possible that the catalog has
> encoded more information in a concrete subclass implementing the
> interface. I vote for separating the concerns of catalog information and
> hints in the factory explicitly.
>
> Regards,
> Timo
>
>
> On 18.03.20 05:41, Jingsong Li wrote:
> > Hi,
> >
> > I am thinking we can provide hints to *table* related instances.
> > - TableFormatFactory: of cause we need hints support, there are many format
> > options in DDL too.
> > - catalog and module: I don't know, maybe in future we can provide some
> > hints for them.
> >
> > Best,
> > Jingsong Lee
> >
> > On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com> wrote:
> >
> > > Yes, I think we should move the `supportedHintOptions` from TableFactory
> > > to TableSourceFactory, and we also need to add the interface to
> > > TableSinkFactory though because sink target table may also have hints
> > > attached.
> > >
> > > Best,
> > > Danny Chan
> > > 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> > > > Have one question for adding `supportedHintOptions` method to
> > > > `TableFactory`. It seems
> > > > `TableFactory` is a base factory interface for all *table module* related
> > > > instances, such as
> > > > catalog, module, format and so on. It's not created only for *table*. Is
> > > it
> > > > possible to move it
> > > > to `TableSourceFactory`?
> > > >
> > > > Best,
> > > > Kurt
> > > >
> > > >
> > > > On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <yu...@gmail.com>
> > > wrote:
> > > >
> > > > > Thanks Timo ~
> > > > >
> > > > > For the naming itself, I also think the PROPERTIES is not that
> > > concise, so
> > > > > +1 for OPTIONS (I had thought about that, but there are many codes in
> > > > > current Flink called it properties, i.e. the DescriptorProperties,
> > > > > #getSupportedProperties), let’s use OPTIONS if this is our new
> > > preference.
> > > > >
> > > > > +1 to `Set<ConfigOption> supportedHintOptions()` because the
> > > ConfigOption
> > > > > can take more info. AFAIK, Spark also call their table options instead
> > > of
> > > > > properties. [1]
> > > > >
> > > > > In my local POC, I did create a new CatalogTable, and it works for
> > > current
> > > > > connectors well, all the DDL tables would finally yield a CatalogTable
> > > > > instance and we can apply the options to that(in the CatalogSourceTable
> > > > > when we generating the TableSource), the pros is that we do not need to
> > > > > modify the codes of connectors itself. If we split the options from
> > > > > CatalogTable, we may need to add some additional logic in each
> > > connector
> > > > > factories in order to merge these properties (and the logic are almost
> > > the
> > > > > same), what do you think about this?
> > > > >
> > > > > [1]
> > > > >
> > > https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> > > > >
> > > > > Best,
> > > > > Danny Chan
> > > > > 在 2020年3月17日 +0800 PM10:10,Timo Walther <tw...@apache.org>,写道:
> > > > > > Hi Danny,
> > > > > >
> > > > > > thanks for updating the FLIP. I think your current design is
> > > sufficient
> > > > > > to separate hints from result-related properties.
> > > > > >
> > > > > > One remark to the naming itself: I would vote for calling the hints
> > > > > > around table scan `OPTIONS('k'='v')`. We used the term "properties"
> > > in
> > > > > > the past but since we want to unify the Flink configuration
> > > experience,
> > > > > > we should use consistent naming and classes around `ConfigOptions`.
> > > > > >
> > > > > > It would be nice to use `Set<ConfigOption> supportedHintOptions();`
> > > to
> > > > > > start using config options instead of pure string properties. This
> > > will
> > > > > > also allow us to generate documentation in the future around
> > > supported
> > > > > > data types, ranges, etc. for options. At some point we would also
> > > like
> > > > > > to drop `DescriptorProperties` class. "Options" is also used in the
> > > > > > documentation [1] and in the SQL/MED standard [2].
> > > > > >
> > > > > > Furthermore, I would still vote for separating CatalogTable and hint
> > > > > > options. Otherwise the planner would need to create a new
> > > CatalogTable
> > > > > > instance which might not always be easy. We should offer them via:
> > > > > >
> > > > > > org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > > > > > ReadableConfig
> > > > > >
> > > > > > What do you think?
> > > > > >
> > > > > > Regards,
> > > > > > Timo
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> > > https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > > > > > [2] https://wiki.postgresql.org/wiki/SQL/MED
> > > > > >
> > > > > >
> > > > > > On 12.03.20 15:06, Stephan Ewen wrote:
> > > > > > > @Danny sounds good.
> > > > > > >
> > > > > > > Maybe it is worth listing all the classes of problems that you
> > > want to
> > > > > > > address and then look at each class and see if hints are a good
> > > default
> > > > > > > solution or a good optional way of simplifying things?
> > > > > > > The discussion has grown a lot and it is starting to be hard to
> > > > > distinguish
> > > > > > > the parts where everyone agrees from the parts were there are
> > > concerns.
> > > > > > >
> > > > > > > On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <da...@apache.org>
> > > > > wrote:
> > > > > > >
> > > > > > > > Thanks Stephan ~
> > > > > > > >
> > > > > > > > We can remove the support for properties that may change the
> > > > > semantics of
> > > > > > > > query if you think that is a trouble.
> > > > > > > >
> > > > > > > > How about we support the /*+ properties() */ hint only for those
> > > > > optimize
> > > > > > > > parameters, such as the fetch size of source or something like
> > > that,
> > > > > does
> > > > > > > > that make sense?
> > > > > > > >
> > > > > > > > Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > > > > > > >
> > > > > > > > > I think Bowen has actually put it very well.
> > > > > > > > >
> > > > > > > > > (1) Hints that change semantics looks like trouble waiting to
> > > > > happen. For
> > > > > > > > > example Kafka offset handling should be in filters. The Kafka
> > > > > source
> > > > > > > > should
> > > > > > > > > support predicate pushdown.
> > > > > > > > >
> > > > > > > > > (2) Hints should not be a workaround for current shortcomings.
> > > A
> > > > > lot of
> > > > > > > > the
> > > > > > > > > suggested above sounds exactly like that. Working around
> > > > > catalog/DDL
> > > > > > > > > shortcomings, missing exposure of metadata (offsets), missing
> > > > > predicate
> > > > > > > > > pushdown in Kafka. Abusing a feature like hints now as a quick
> > > fix
> > > > > for
> > > > > > > > > these issues, rather than fixing the root causes, will much
> > > likely
> > > > > bite
> > > > > > > > us
> > > > > > > > > back badly in the future.
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Stephan
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com>
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > > It seems this FLIP's name is somewhat misleading. From my
> > > > > > > > understanding,
> > > > > > > > > > this FLIP is trying to
> > > > > > > > > > address the dynamic parameter issue, and table hints is the
> > > way
> > > > > we wan
> > > > > > > > to
> > > > > > > > > > choose. I think we should
> > > > > > > > > > be focus on "what's the right way to solve dynamic property"
> > > > > instead of
> > > > > > > > > > discussing "whether table
> > > > > > > > > > hints can affect query semantics".
> > > > > > > > > >
> > > > > > > > > > For now, there are two proposed ways to achieve dynamic
> > > property:
> > > > > > > > > > 1. FLIP-110: create temporary table xx like xx with (xxx)
> > > > > > > > > > 2. use custom "from t with (xxx)" syntax
> > > > > > > > > > 3. "Borrow" the table hints to have a special PROPERTIES
> > > hint.
> > > > > > > > > >
> > > > > > > > > > The first one didn't break anything, but the only problem i
> > > see
> > > > > is a
> > > > > > > > > little
> > > > > > > > > > more verbose than the table hint
> > > > > > > > > > approach. I can imagine when someone using SQL CLI to have a
> > > sql
> > > > > > > > > > experience, it's quite often that
> > > > > > > > > > he will modify the table property, some use cases i can
> > > think of:
> > > > > > > > > > 1. the source contains some corrupted data, i want to turn
> > > on the
> > > > > > > > > > "ignore-error" flag for certain formats.
> > > > > > > > > > 2. I have a kafka table and want to see some sample data
> > > from the
> > > > > > > > > > beginning, so i change the offset
> > > > > > > > > > to "earliest", and then I want to observe the latest data
> > > which
> > > > > keeps
> > > > > > > > > > coming in. I would write another query
> > > > > > > > > > to select from the latest table.
> > > > > > > > > > 3. I want to my jdbc sink flush data more eagerly then i can
> > > > > observe
> > > > > > > > the
> > > > > > > > > > data from database side.
> > > > > > > > > >
> > > > > > > > > > Most of such use cases are quite ad-hoc. If every time I
> > > want to
> > > > > have a
> > > > > > > > > > different experience, i need to create
> > > > > > > > > > a temporary table and then also modify my query, it doesn't
> > > feel
> > > > > > > > smooth.
> > > > > > > > > > Embed such dynamic property into
> > > > > > > > > > query would have better user experience.
> > > > > > > > > >
> > > > > > > > > > Both 2 & 3 can make this happen. The cons of #2 is breaking
> > > SQL
> > > > > > > > > compliant,
> > > > > > > > > > and for #3, it only breaks some
> > > > > > > > > > unwritten rules, but we can have an explanation on that. And
> > > I
> > > > > really
> > > > > > > > > doubt
> > > > > > > > > > whether user would complain about
> > > > > > > > > > this when they actually have flexible and good experience
> > > using
> > > > > this.
> > > > > > > > > >
> > > > > > > > > > My tendency would be #3 > #1 > #2, what do you think?
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Kurt
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> > > yuzhao.cyz@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Thanks Aljoscha ~
> > > > > > > > > > >
> > > > > > > > > > > I agree for most of the query hints, they are optional as
> > > an
> > > > > > > > optimizer
> > > > > > > > > > > instruction, especially for the traditional RDBMS.
> > > > > > > > > > >
> > > > > > > > > > > But, just like BenChao said, Flink as a computation engine
> > > has
> > > > > many
> > > > > > > > > > > different kind of data sources, thus, dynamic parameters
> > > like
> > > > > > > > > > start_offest
> > > > > > > > > > > can only bind to each table scope, we can not set a session
> > > > > config
> > > > > > > > like
> > > > > > > > > > > KSQL because they are all about Kafka:
> > > > > > > > > > > > SET ‘auto.offset.reset’=‘earliest’;
> > > > > > > > > > >
> > > > > > > > > > > Thus the most flexible way to set up these dynamic params
> > > is
> > > > > to bind
> > > > > > > > to
> > > > > > > > > > > the table scope in the query when we want to override
> > > > > something, so
> > > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > these solutions above (with pros and cons from my side):
> > > > > > > > > > >
> > > > > > > > > > > • 1. Select * from t(offset=123) (from Timo)
> > > > > > > > > > >
> > > > > > > > > > > Pros:
> > > > > > > > > > > - Easy to add
> > > > > > > > > > > - Parameters are part of the main query
> > > > > > > > > > > Cons:
> > > > > > > > > > > - Not SQL compliant
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from
> > > me)
> > > > > > > > > > >
> > > > > > > > > > > Pros:
> > > > > > > > > > > - Easy to add
> > > > > > > > > > > - SQL compliant because it is nested in the comments
> > > > > > > > > > >
> > > > > > > > > > > Cons:
> > > > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > > >
> > > > > > > > > > > The biggest problem for hints way may be the “if hints
> > > must be
> > > > > > > > > optional”,
> > > > > > > > > > > actually we have though about 1 for a while but aborted
> > > > > because it
> > > > > > > > > breaks
> > > > > > > > > > > the SQL standard too much. And we replace it with 2,
> > > because
> > > > > the
> > > > > > > > hints
> > > > > > > > > > > syntax do not break SQL standard(nested in comments).
> > > > > > > > > > >
> > > > > > > > > > > What if we have the special /*+ PROPERTIES */ hint that
> > > allows
> > > > > > > > override
> > > > > > > > > > > some properties of table dynamically, it does not break
> > > > > anything, at
> > > > > > > > > > lease
> > > > > > > > > > > for current Flink use cases.
> > > > > > > > > > >
> > > > > > > > > > > Planner hints are optional just because they are naturally
> > > > > enforcers
> > > > > > > > of
> > > > > > > > > > > the planner, most of them aim to instruct the optimizer,
> > > but,
> > > > > the
> > > > > > > > table
> > > > > > > > > > > hints is a little different, table hints can specify the
> > > table
> > > > > meta
> > > > > > > > > like
> > > > > > > > > > > index column, and it is very convenient to specify table
> > > > > properties.
> > > > > > > > > > >
> > > > > > > > > > > Or shall we not call /*+ PROPERTIES(offset=123) */ table
> > > hint,
> > > > > we
> > > > > > > > can
> > > > > > > > > > > call it table dynamic parameters.
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Danny Chan
> > > > > > > > > > > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> > > > > aljoscha@apache.org>,写道:
> > > > > > > > > > > > Hi,
> > > > > > > > > > > >
> > > > > > > > > > > > I don't understand this discussion. Hints, as I
> > > understand
> > > > > them,
> > > > > > > > > should
> > > > > > > > > > > > work like this:
> > > > > > > > > > > >
> > > > > > > > > > > > - hints are *optional* advice for the optimizer to try
> > > and
> > > > > help it
> > > > > > > > to
> > > > > > > > > > > > find a good execution strategy
> > > > > > > > > > > > - hints should not change query semantics, i.e. they
> > > should
> > > > > not
> > > > > > > > > change
> > > > > > > > > > > > connector properties executing a query with taking into
> > > > > account the
> > > > > > > > > > > > hints *must* produce the same result as executing the
> > > query
> > > > > without
> > > > > > > > > > > > taking into account the hints
> > > > > > > > > > > >
> > > > > > > > > > > > From these simple requirements you can derive a solution
> > > > > that makes
> > > > > > > > > > > > sense. I don't have a strong preference for the syntax
> > > but we
> > > > > > > > should
> > > > > > > > > > > > strive to be in line with prior work.
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Aljoscha
> > > > > > > > > > > >
> > > > > > > > > > > > On 11.03.20 11:53, Danny Chan wrote:
> > > > > > > > > > > > > Thanks Timo for summarize the 3 options ~
> > > > > > > > > > > > >
> > > > > > > > > > > > > I agree with Kurt that option2 is too complicated to
> > > use
> > > > > because:
> > > > > > > > > > > > >
> > > > > > > > > > > > > • As a Kafka topic consumer, the user must define both
> > > the
> > > > > > > > virtual
> > > > > > > > > > > column for start offset and he must apply a special filter
> > > > > predicate
> > > > > > > > > > after
> > > > > > > > > > > each query
> > > > > > > > > > > > > • And for the internal implementation, the metadata
> > > column
> > > > > push
> > > > > > > > > down
> > > > > > > > > > > is another hard topic, each kind of message queue may have
> > > its
> > > > > offset
> > > > > > > > > > > attribute, we need to consider the expression type for
> > > > > different
> > > > > > > > kind;
> > > > > > > > > > the
> > > > > > > > > > > source also need to recognize the constant column as a
> > > config
> > > > > > > > > > option(which
> > > > > > > > > > > is weird because usually what we pushed down is a table
> > > column)
> > > > > > > > > > > > >
> > > > > > > > > > > > > For option 1 and option3, I think there is no
> > > difference,
> > > > > option1
> > > > > > > > > is
> > > > > > > > > > > also a hint syntax which is introduced in Sybase and
> > > > > referenced then
> > > > > > > > > > > deprecated by MS-SQL in 199X years because of the
> > > > > ambitiousness.
> > > > > > > > > > Personally
> > > > > > > > > > > I prefer /*+ */ style table hint than WITH keyword for
> > > these
> > > > > reasons:
> > > > > > > > > > > > >
> > > > > > > > > > > > > • We do not break the standard SQL, the hints are
> > > nested
> > > > > in SQL
> > > > > > > > > > > comments
> > > > > > > > > > > > > • We do not need to introduce additional WITH keyword
> > > > > which may
> > > > > > > > > > appear
> > > > > > > > > > > in a query if we use that because a table can be
> > > referenced in
> > > > > all
> > > > > > > > > kinds
> > > > > > > > > > of
> > > > > > > > > > > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make
> > > our
> > > > > sql
> > > > > > > > query
> > > > > > > > > > > break too much of the SQL from standard
> > > > > > > > > > > > > • We would have uniform syntax for hints as query
> > > hint, one
> > > > > > > > syntax
> > > > > > > > > > > fits all and more easy to use
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > And here is the reason why we choose a uniform Oracle
> > > > > style query
> > > > > > > > > > > hint syntax which is addressed by Julian Hyde when we
> > > design
> > > > > the
> > > > > > > > syntax
> > > > > > > > > > > from the Calcite community:
> > > > > > > > > > > > >
> > > > > > > > > > > > > I don’t much like the MSSQL-style syntax for table
> > > hints.
> > > > > It
> > > > > > > > adds a
> > > > > > > > > > > new use of the WITH keyword that is unrelated to the use of
> > > > > WITH for
> > > > > > > > > > > common-table expressions.
> > > > > > > > > > > > >
> > > > > > > > > > > > > A historical note. Microsoft SQL Server inherited its
> > > hint
> > > > > syntax
> > > > > > > > > > from
> > > > > > > > > > > Sybase a very long time ago. (See “Transact SQL
> > > > > Programming”[1], page
> > > > > > > > > > 632,
> > > > > > > > > > > “Optimizer hints”. The book was written in 1999, and covers
> > > > > Microsoft
> > > > > > > > > SQL
> > > > > > > > > > > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the
> > > > > syntax very
> > > > > > > > > > > likely predates Sybase 4.3, from which Microsoft SQL
> > > Server was
> > > > > > > > forked
> > > > > > > > > in
> > > > > > > > > > > 1993.)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Microsoft later added the WITH keyword to make it less
> > > > > ambiguous,
> > > > > > > > > and
> > > > > > > > > > > has now deprecated the syntax that does not use WITH.
> > > > > > > > > > > > >
> > > > > > > > > > > > > They are forced to keep the syntax for backwards
> > > > > compatibility
> > > > > > > > but
> > > > > > > > > > > that doesn’t mean that we should shoulder their burden.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I think formatted comments are the right container for
> > > > > hints
> > > > > > > > > because
> > > > > > > > > > > it allows us to change the hint syntax without changing
> > > the SQL
> > > > > > > > parser,
> > > > > > > > > > and
> > > > > > > > > > > makes clear that we are at liberty to ignore hints
> > > entirely.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Julian
> > > > > > > > > > > > >
> > > > > > > > > > > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > > > > > > > > > > https://www.amazon.com/s?k=9781565924017>
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> > > twalthr@apache.org
> > > > > > ,写道:
> > > > > > > > > > > > > > Hi Danny,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > it is true that our DDL is not standard compliant by
> > > > > using the
> > > > > > > > > WITH
> > > > > > > > > > > > > > clause. Nevertheless, we aim for not diverging too
> > > much
> > > > > and the
> > > > > > > > > > LIKE
> > > > > > > > > > > > > > clause is an example of that. It will solve things
> > > like
> > > > > > > > > overwriting
> > > > > > > > > > > > > > WATERMARKs, add additional/modifying properties and
> > > > > inherit
> > > > > > > > > schema.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Bowen is right that Flink's DDL is mixing 3 types
> > > > > definition
> > > > > > > > > > > together.
> > > > > > > > > > > > > > We are not the first ones that try to solve this.
> > > There
> > > > > is also
> > > > > > > > > the
> > > > > > > > > > > SQL
> > > > > > > > > > > > > > MED standard [1] that tried to tackle this problem. I
> > > > > think it
> > > > > > > > > was
> > > > > > > > > > > not
> > > > > > > > > > > > > > considered when designing the current DDL.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Currently, I see 3 options for handling Kafka
> > > offsets. I
> > > > > will
> > > > > > > > > give
> > > > > > > > > > > some
> > > > > > > > > > > > > > examples and look forward to feedback here:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > *Option 1* Runtime and semantic parms as part of the
> > > > > query
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > > > - Parameters are part of the main query
> > > > > > > > > > > > > > - No complicated hinting syntax
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > > - Not SQL compliant
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > *Option 2* Use metadata in query
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > `CREATE TABLE MyTable (id INT, offset AS
> > > > > > > > > > SYSTEM_METADATA('offset'))`
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP
> > > > > '2012-12-12
> > > > > > > > > > > 12:34:22'`
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > > - SQL compliant in the query
> > > > > > > > > > > > > > - Access of metadata in the DDL which is required
> > > anyway
> > > > > > > > > > > > > > - Regular pushdown rules apply
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > > - Users need to add an additional comlumn in the DDL
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > *Option 3*: Use hints for properties
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > `
> > > > > > > > > > > > > > SELECT *
> > > > > > > > > > > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > > > > > > > > > > `
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Pros:
> > > > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Cons:
> > > > > > > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > > > > > > - Not standard compliant.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > If we go with this option, I would suggest to make it
> > > > > available
> > > > > > > > > in
> > > > > > > > > > a
> > > > > > > > > > > > > > separate map and don't mix it with statically defined
> > > > > > > > properties.
> > > > > > > > > > > Such
> > > > > > > > > > > > > > that the factory can decide which properties have the
> > > > > right to
> > > > > > > > be
> > > > > > > > > > > > > > overwritten by the hints:
> > > > > > > > > > > > > > TableSourceFactory.Context.getQueryHints():
> > > > > ReadableConfig
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > > Timo
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Currently I see 3 options as a
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > > > > > > > > > > Thanks Bowen ~
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I agree we should somehow categorize our connector
> > > > > > > > parameters.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > For type1, I’m already preparing a solution like
> > > the
> > > > > > > > Confluent
> > > > > > > > > > > schema registry + Avro schema inference thing, so this may
> > > not
> > > > > be a
> > > > > > > > > > problem
> > > > > > > > > > > in the near future.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > For type3, I have some questions:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > "SELECT * FROM mykafka WHERE offset > 12pm
> > > yesterday”
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Where does the offset column come from, a virtual
> > > > > column from
> > > > > > > > > the
> > > > > > > > > > > table schema, you said that
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > They change
> > > > > > > > > > > > > > > almost every time a query starts and have nothing
> > > to
> > > > > do with
> > > > > > > > > > > metadata, thus
> > > > > > > > > > > > > > > should not be part of table definition/DDL
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > But why you can reference it in the query, I’m
> > > > > confused for
> > > > > > > > > that,
> > > > > > > > > > > can you elaborate a little ?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> > > > > bowenli86@gmail.com
> > > > > > > > > ,写道:
> > > > > > > > > > > > > > > > Thanks Danny for kicking off the effort
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > The root cause of too much manual work is Flink
> > > DDL
> > > > > has
> > > > > > > > > mixed 3
> > > > > > > > > > > types of
> > > > > > > > > > > > > > > > params together and doesn't handle each of them
> > > very
> > > > > well.
> > > > > > > > > > Below
> > > > > > > > > > > are how I
> > > > > > > > > > > > > > > > categorize them and corresponding solutions in my
> > > > > mind:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - type 1: Metadata of external data, like
> > > external
> > > > > > > > > > endpoint/url,
> > > > > > > > > > > > > > > > username/pwd, schemas, formats.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Such metadata are mostly already accessible in
> > > > > external
> > > > > > > > > system
> > > > > > > > > > > as long as
> > > > > > > > > > > > > > > > endpoints and credentials are provided. Flink can
> > > > > get it
> > > > > > > > thru
> > > > > > > > > > > catalogs, but
> > > > > > > > > > > > > > > > we haven't had many catalogs yet and thus Flink
> > > just
> > > > > hasn't
> > > > > > > > > > been
> > > > > > > > > > > able to
> > > > > > > > > > > > > > > > leverage that. So the solution should be building
> > > > > more
> > > > > > > > > > catalogs.
> > > > > > > > > > > Such
> > > > > > > > > > > > > > > > params should be part of a Flink table
> > > > > DDL/definition, and
> > > > > > > > > not
> > > > > > > > > > > overridable
> > > > > > > > > > > > > > > > in any means.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - type 2: Runtime params, like jdbc connector's
> > > > > fetch size,
> > > > > > > > > > > elasticsearch
> > > > > > > > > > > > > > > > connector's bulk flush size.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Such params don't affect query results, but
> > > affect
> > > > > how
> > > > > > > > > results
> > > > > > > > > > > are produced
> > > > > > > > > > > > > > > > (eg. fast or slow, aka performance) - they are
> > > > > essentially
> > > > > > > > > > > execution and
> > > > > > > > > > > > > > > > implementation details. They change often in
> > > > > exploration or
> > > > > > > > > > > development
> > > > > > > > > > > > > > > > stages, but not quite frequently in well-defined
> > > > > > > > long-running
> > > > > > > > > > > pipelines.
> > > > > > > > > > > > > > > > They should always have default values and can be
> > > > > missing
> > > > > > > > in
> > > > > > > > > > > query. They
> > > > > > > > > > > > > > > > can be part of a table DDL/definition, but should
> > > > > also be
> > > > > > > > > > > replaceable in a
> > > > > > > > > > > > > > > > query - *this is what table "hints" in FLIP-113
> > > > > should
> > > > > > > > > cover*.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - type 3: Semantic params, like kafka connector's
> > > > > start
> > > > > > > > > offset.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Such params affect query results - the semantics.
> > > > > They'd
> > > > > > > > > better
> > > > > > > > > > > be as
> > > > > > > > > > > > > > > > filter conditions in WHERE clause that can be
> > > pushed
> > > > > down.
> > > > > > > > > They
> > > > > > > > > > > change
> > > > > > > > > > > > > > > > almost every time a query starts and have
> > > nothing to
> > > > > do
> > > > > > > > with
> > > > > > > > > > > metadata, thus
> > > > > > > > > > > > > > > > should not be part of table definition/DDL, nor
> > > be
> > > > > > > > persisted
> > > > > > > > > in
> > > > > > > > > > > catalogs.
> > > > > > > > > > > > > > > > If they will, users should create views to keep
> > > such
> > > > > params
> > > > > > > > > > > around (note
> > > > > > > > > > > > > > > > this is different from variable substitution).
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Take Flink-Kafka as an example. Once we get these
> > > > > params
> > > > > > > > > right,
> > > > > > > > > > > here're the
> > > > > > > > > > > > > > > > steps users need to do to develop and run a Flink
> > > > > job:
> > > > > > > > > > > > > > > > - configure a Flink ConfluentSchemaRegistry with
> > > url,
> > > > > > > > > username,
> > > > > > > > > > > and password
> > > > > > > > > > > > > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm
> > > > > yesterday"
> > > > > > > > > > > (simplified
> > > > > > > > > > > > > > > > timestamp) in SQL CLI, Flink automatically
> > > retrieves
> > > > > all
> > > > > > > > > > > metadata of
> > > > > > > > > > > > > > > > schema, file format, etc and start the job
> > > > > > > > > > > > > > > > - users want to make the job read Kafka topic
> > > > > faster, so it
> > > > > > > > > > goes
> > > > > > > > > > > as "SELECT
> > > > > > > > > > > > > > > > * FROM mykafka /* faster_read_key=value*/ WHERE
> > > > > offset >
> > > > > > > > 12pm
> > > > > > > > > > > yesterday"
> > > > > > > > > > > > > > > > - done and satisfied, users submit it to
> > > production
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1,
> > > k2=v2),
> > > > > I think
> > > > > > > > > > it's
> > > > > > > > > > > a
> > > > > > > > > > > > > > > > nice-to-have feature, but not a strategically
> > > > > critical,
> > > > > > > > > > > long-term solution,
> > > > > > > > > > > > > > > > because
> > > > > > > > > > > > > > > > 1) It may seem promising at the current stage to
> > > > > solve the
> > > > > > > > > > > > > > > > too-much-manual-work problem, but that's only
> > > > > because Flink
> > > > > > > > > > > hasn't
> > > > > > > > > > > > > > > > leveraged catalogs well and handled the 3 types
> > > of
> > > > > params
> > > > > > > > > above
> > > > > > > > > > > properly.
> > > > > > > > > > > > > > > > Once we get the params types right, the LIKE
> > > syntax
> > > > > won't
> > > > > > > > be
> > > > > > > > > > that
> > > > > > > > > > > > > > > > important, and will be just an easier way to
> > > create
> > > > > tables
> > > > > > > > > > > without retyping
> > > > > > > > > > > > > > > > long fields like username and pwd.
> > > > > > > > > > > > > > > > 2) Note that only some rare type of catalog can
> > > > > store k-v
> > > > > > > > > > > property pair, so
> > > > > > > > > > > > > > > > table created this way often cannot be
> > > persisted. In
> > > > > the
> > > > > > > > > > > foreseeable
> > > > > > > > > > > > > > > > future, such catalog will only be HiveCatalog,
> > > and
> > > > > not
> > > > > > > > > everyone
> > > > > > > > > > > has a Hive
> > > > > > > > > > > > > > > > metastore. To be honest, without persistence,
> > > > > recreating
> > > > > > > > > tables
> > > > > > > > > > > every time
> > > > > > > > > > > > > > > > this way is still a lot of keyboard typing.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Cheers,
> > > > > > > > > > > > > > > > Bowen
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <
> > > > > > > > ykt836@gmail.com
> > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > If a specific connector want to have such
> > > > > parameter and
> > > > > > > > > read
> > > > > > > > > > > if out of
> > > > > > > > > > > > > > > > > configuration, then that's fine.
> > > > > > > > > > > > > > > > > If we are talking about a configuration for all
> > > > > kinds of
> > > > > > > > > > > sources, I would
> > > > > > > > > > > > > > > > > be super careful about that.
> > > > > > > > > > > > > > > > > It's true it can solve maybe 80% cases, but it
> > > > > will also
> > > > > > > > > make
> > > > > > > > > > > the left 20%
> > > > > > > > > > > > > > > > > feels weird.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <
> > > > > > > > imjark@gmail.com
> > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi Kurt,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > > > > > > > > > > I'm not saying to use the global
> > > configuration to
> > > > > > > > > override
> > > > > > > > > > > connector
> > > > > > > > > > > > > > > > > > properties by the planner.
> > > > > > > > > > > > > > > > > > But the connector should take this
> > > configuration
> > > > > and
> > > > > > > > > > > translate into their
> > > > > > > > > > > > > > > > > > client API.
> > > > > > > > > > > > > > > > > > AFAIK, almost all the message queues support
> > > > > eariliest
> > > > > > > > > and
> > > > > > > > > > > latest and a
> > > > > > > > > > > > > > > > > > timestamp value as start point.
> > > > > > > > > > > > > > > > > > So we can support 3 options for this
> > > > > configuration:
> > > > > > > > > > > "eariliest", "latest"
> > > > > > > > > > > > > > > > > > and a timestamp string value.
> > > > > > > > > > > > > > > > > > Of course, this can't solve 100% cases, but I
> > > > > guess can
> > > > > > > > > > > sovle 80% or 90%
> > > > > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > And the remaining cases can be resolved by
> > > LIKE
> > > > > syntax
> > > > > > > > > > which
> > > > > > > > > > > I guess is
> > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > very common cases.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > Jark
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <
> > > > > > > > > ykt836@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Good to have such lovely discussions. I
> > > also
> > > > > want to
> > > > > > > > > > share
> > > > > > > > > > > some of my
> > > > > > > > > > > > > > > > > > > opinions.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > #1 Regarding to error handling: I also
> > > think
> > > > > ignore
> > > > > > > > > > > invalid hints would
> > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > dangerous, maybe
> > > > > > > > > > > > > > > > > > > the simplest solution is just throw an
> > > > > exception.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > #2 Regarding to property replacement: I
> > > don't
> > > > > think
> > > > > > > > we
> > > > > > > > > > > should
> > > > > > > > > > > > > > > > > constraint
> > > > > > > > > > > > > > > > > > > ourself to
> > > > > > > > > > > > > > > > > > > the meaning of the word "hint", and
> > > forbidden
> > > > > it
> > > > > > > > > > modifying
> > > > > > > > > > > any
> > > > > > > > > > > > > > > > > properties
> > > > > > > > > > > > > > > > > > > which can effect
> > > > > > > > > > > > > > > > > > > query results. IMO `PROPERTIES` is one of
> > > the
> > > > > table
> > > > > > > > > > hints,
> > > > > > > > > > > and a
> > > > > > > > > > > > > > > > > powerful
> > > > > > > > > > > > > > > > > > > one. It can
> > > > > > > > > > > > > > > > > > > modify properties located in DDL's WITH
> > > block.
> > > > > But I
> > > > > > > > > also
> > > > > > > > > > > see the harm
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > if we make it
> > > > > > > > > > > > > > > > > > > too flexible like change the kafka topic
> > > name
> > > > > with a
> > > > > > > > > > hint.
> > > > > > > > > > > Such use
> > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > not common and
> > > > > > > > > > > > > > > > > > > sounds very dangerous to me. I would
> > > propose
> > > > > we have
> > > > > > > > a
> > > > > > > > > > map
> > > > > > > > > > > of hintable
> > > > > > > > > > > > > > > > > > > properties for each
> > > > > > > > > > > > > > > > > > > connector, and should validate all passed
> > > in
> > > > > > > > properties
> > > > > > > > > > > are actually
> > > > > > > > > > > > > > > > > > > hintable. And combining with
> > > > > > > > > > > > > > > > > > > #1 error handling, we can throw an
> > > exception
> > > > > once
> > > > > > > > > > received
> > > > > > > > > > > invalid
> > > > > > > > > > > > > > > > > > > property.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > #3 Regarding to global offset: I'm not sure
> > > > > it's
> > > > > > > > > > feasible.
> > > > > > > > > > > Different
> > > > > > > > > > > > > > > > > > > connectors will have totally
> > > > > > > > > > > > > > > > > > > different properties to represent offset,
> > > some
> > > > > might
> > > > > > > > be
> > > > > > > > > > > timestamps,
> > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > might be string literals
> > > > > > > > > > > > > > > > > > > like "earliest", and others might be just
> > > > > integers.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
> > > > > > > > > > imjark@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 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 <
> > > > > > > > > > > twalthr@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 <
> > > > > > > > > > > twalthr@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.lwl@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.cyz@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 <
> > > > > > > > > > > twalthr@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
> > > > > > > > > > <
> > > > > > > > > > > imjark@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 <
> > > > > > > > > > > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@apache.org>.
Hi everyone,

+1 to Kurt's suggestion. Let's just have it in source and sink factories 
for now. We can still move this method up in the future. Currently, I 
don't see a need for catalogs or formats. Because how would you target a 
format in the query?

@Danny: Can you send a link to your PoC? I'm very skeptical about 
creating a new CatalogTable in planner. Actually CatalogTable should be 
immutable between Catalog and Factory. Because a catalog can return its 
own factory and fully control the instantiation. Depending on the 
implementation, that means it can be possible that the catalog has 
encoded more information in a concrete subclass implementing the 
interface. I vote for separating the concerns of catalog information and 
hints in the factory explicitly.

Regards,
Timo


On 18.03.20 05:41, Jingsong Li wrote:
> Hi,
> 
> I am thinking we can provide hints to *table* related instances.
> - TableFormatFactory: of cause we need hints support, there are many format
> options in DDL too.
> - catalog and module: I don't know, maybe in future we can provide some
> hints for them.
> 
> Best,
> Jingsong Lee
> 
> On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com> wrote:
> 
>> Yes, I think we should move the `supportedHintOptions` from TableFactory
>> to TableSourceFactory, and we also need to add the interface to
>> TableSinkFactory though because sink target table may also have hints
>> attached.
>>
>> Best,
>> Danny Chan
>> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
>>> Have one question for adding `supportedHintOptions` method to
>>> `TableFactory`. It seems
>>> `TableFactory` is a base factory interface for all *table module* related
>>> instances, such as
>>> catalog, module, format and so on. It's not created only for *table*. Is
>> it
>>> possible to move it
>>> to `TableSourceFactory`?
>>>
>>> Best,
>>> Kurt
>>>
>>>
>>> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <yu...@gmail.com>
>> wrote:
>>>
>>>> Thanks Timo ~
>>>>
>>>> For the naming itself, I also think the PROPERTIES is not that
>> concise, so
>>>> +1 for OPTIONS (I had thought about that, but there are many codes in
>>>> current Flink called it properties, i.e. the DescriptorProperties,
>>>> #getSupportedProperties), let’s use OPTIONS if this is our new
>> preference.
>>>>
>>>> +1 to `Set<ConfigOption> supportedHintOptions()` because the
>> ConfigOption
>>>> can take more info. AFAIK, Spark also call their table options instead
>> of
>>>> properties. [1]
>>>>
>>>> In my local POC, I did create a new CatalogTable, and it works for
>> current
>>>> connectors well, all the DDL tables would finally yield a CatalogTable
>>>> instance and we can apply the options to that(in the CatalogSourceTable
>>>> when we generating the TableSource), the pros is that we do not need to
>>>> modify the codes of connectors itself. If we split the options from
>>>> CatalogTable, we may need to add some additional logic in each
>> connector
>>>> factories in order to merge these properties (and the logic are almost
>> the
>>>> same), what do you think about this?
>>>>
>>>> [1]
>>>>
>> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
>>>>
>>>> Best,
>>>> Danny Chan
>>>> 在 2020年3月17日 +0800 PM10:10,Timo Walther <tw...@apache.org>,写道:
>>>>> Hi Danny,
>>>>>
>>>>> thanks for updating the FLIP. I think your current design is
>> sufficient
>>>>> to separate hints from result-related properties.
>>>>>
>>>>> One remark to the naming itself: I would vote for calling the hints
>>>>> around table scan `OPTIONS('k'='v')`. We used the term "properties"
>> in
>>>>> the past but since we want to unify the Flink configuration
>> experience,
>>>>> we should use consistent naming and classes around `ConfigOptions`.
>>>>>
>>>>> It would be nice to use `Set<ConfigOption> supportedHintOptions();`
>> to
>>>>> start using config options instead of pure string properties. This
>> will
>>>>> also allow us to generate documentation in the future around
>> supported
>>>>> data types, ranges, etc. for options. At some point we would also
>> like
>>>>> to drop `DescriptorProperties` class. "Options" is also used in the
>>>>> documentation [1] and in the SQL/MED standard [2].
>>>>>
>>>>> Furthermore, I would still vote for separating CatalogTable and hint
>>>>> options. Otherwise the planner would need to create a new
>> CatalogTable
>>>>> instance which might not always be easy. We should offer them via:
>>>>>
>>>>> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
>>>>> ReadableConfig
>>>>>
>>>>> What do you think?
>>>>>
>>>>> Regards,
>>>>> Timo
>>>>>
>>>>> [1]
>>>>>
>>>>
>> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
>>>>> [2] https://wiki.postgresql.org/wiki/SQL/MED
>>>>>
>>>>>
>>>>> On 12.03.20 15:06, Stephan Ewen wrote:
>>>>>> @Danny sounds good.
>>>>>>
>>>>>> Maybe it is worth listing all the classes of problems that you
>> want to
>>>>>> address and then look at each class and see if hints are a good
>> default
>>>>>> solution or a good optional way of simplifying things?
>>>>>> The discussion has grown a lot and it is starting to be hard to
>>>> distinguish
>>>>>> the parts where everyone agrees from the parts were there are
>> concerns.
>>>>>>
>>>>>> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <da...@apache.org>
>>>> wrote:
>>>>>>
>>>>>>> Thanks Stephan ~
>>>>>>>
>>>>>>> We can remove the support for properties that may change the
>>>> semantics of
>>>>>>> query if you think that is a trouble.
>>>>>>>
>>>>>>> How about we support the /*+ properties() */ hint only for those
>>>> optimize
>>>>>>> parameters, such as the fetch size of source or something like
>> that,
>>>> does
>>>>>>> that make sense?
>>>>>>>
>>>>>>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
>>>>>>>
>>>>>>>> I think Bowen has actually put it very well.
>>>>>>>>
>>>>>>>> (1) Hints that change semantics looks like trouble waiting to
>>>> happen. For
>>>>>>>> example Kafka offset handling should be in filters. The Kafka
>>>> source
>>>>>>> should
>>>>>>>> support predicate pushdown.
>>>>>>>>
>>>>>>>> (2) Hints should not be a workaround for current shortcomings.
>> A
>>>> lot of
>>>>>>> the
>>>>>>>> suggested above sounds exactly like that. Working around
>>>> catalog/DDL
>>>>>>>> shortcomings, missing exposure of metadata (offsets), missing
>>>> predicate
>>>>>>>> pushdown in Kafka. Abusing a feature like hints now as a quick
>> fix
>>>> for
>>>>>>>> these issues, rather than fixing the root causes, will much
>> likely
>>>> bite
>>>>>>> us
>>>>>>>> back badly in the future.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Stephan
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com>
>>>> wrote:
>>>>>>>>
>>>>>>>>> It seems this FLIP's name is somewhat misleading. From my
>>>>>>> understanding,
>>>>>>>>> this FLIP is trying to
>>>>>>>>> address the dynamic parameter issue, and table hints is the
>> way
>>>> we wan
>>>>>>> to
>>>>>>>>> choose. I think we should
>>>>>>>>> be focus on "what's the right way to solve dynamic property"
>>>> instead of
>>>>>>>>> discussing "whether table
>>>>>>>>> hints can affect query semantics".
>>>>>>>>>
>>>>>>>>> For now, there are two proposed ways to achieve dynamic
>> property:
>>>>>>>>> 1. FLIP-110: create temporary table xx like xx with (xxx)
>>>>>>>>> 2. use custom "from t with (xxx)" syntax
>>>>>>>>> 3. "Borrow" the table hints to have a special PROPERTIES
>> hint.
>>>>>>>>>
>>>>>>>>> The first one didn't break anything, but the only problem i
>> see
>>>> is a
>>>>>>>> little
>>>>>>>>> more verbose than the table hint
>>>>>>>>> approach. I can imagine when someone using SQL CLI to have a
>> sql
>>>>>>>>> experience, it's quite often that
>>>>>>>>> he will modify the table property, some use cases i can
>> think of:
>>>>>>>>> 1. the source contains some corrupted data, i want to turn
>> on the
>>>>>>>>> "ignore-error" flag for certain formats.
>>>>>>>>> 2. I have a kafka table and want to see some sample data
>> from the
>>>>>>>>> beginning, so i change the offset
>>>>>>>>> to "earliest", and then I want to observe the latest data
>> which
>>>> keeps
>>>>>>>>> coming in. I would write another query
>>>>>>>>> to select from the latest table.
>>>>>>>>> 3. I want to my jdbc sink flush data more eagerly then i can
>>>> observe
>>>>>>> the
>>>>>>>>> data from database side.
>>>>>>>>>
>>>>>>>>> Most of such use cases are quite ad-hoc. If every time I
>> want to
>>>> have a
>>>>>>>>> different experience, i need to create
>>>>>>>>> a temporary table and then also modify my query, it doesn't
>> feel
>>>>>>> smooth.
>>>>>>>>> Embed such dynamic property into
>>>>>>>>> query would have better user experience.
>>>>>>>>>
>>>>>>>>> Both 2 & 3 can make this happen. The cons of #2 is breaking
>> SQL
>>>>>>>> compliant,
>>>>>>>>> and for #3, it only breaks some
>>>>>>>>> unwritten rules, but we can have an explanation on that. And
>> I
>>>> really
>>>>>>>> doubt
>>>>>>>>> whether user would complain about
>>>>>>>>> this when they actually have flexible and good experience
>> using
>>>> this.
>>>>>>>>>
>>>>>>>>> My tendency would be #3 > #1 > #2, what do you think?
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Kurt
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
>> yuzhao.cyz@gmail.com
>>>>>
>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Thanks Aljoscha ~
>>>>>>>>>>
>>>>>>>>>> I agree for most of the query hints, they are optional as
>> an
>>>>>>> optimizer
>>>>>>>>>> instruction, especially for the traditional RDBMS.
>>>>>>>>>>
>>>>>>>>>> But, just like BenChao said, Flink as a computation engine
>> has
>>>> many
>>>>>>>>>> different kind of data sources, thus, dynamic parameters
>> like
>>>>>>>>> start_offest
>>>>>>>>>> can only bind to each table scope, we can not set a session
>>>> config
>>>>>>> like
>>>>>>>>>> KSQL because they are all about Kafka:
>>>>>>>>>>> SET ‘auto.offset.reset’=‘earliest’;
>>>>>>>>>>
>>>>>>>>>> Thus the most flexible way to set up these dynamic params
>> is
>>>> to bind
>>>>>>> to
>>>>>>>>>> the table scope in the query when we want to override
>>>> something, so
>>>>>>> we
>>>>>>>>> have
>>>>>>>>>> these solutions above (with pros and cons from my side):
>>>>>>>>>>
>>>>>>>>>> • 1. Select * from t(offset=123) (from Timo)
>>>>>>>>>>
>>>>>>>>>> Pros:
>>>>>>>>>> - Easy to add
>>>>>>>>>> - Parameters are part of the main query
>>>>>>>>>> Cons:
>>>>>>>>>> - Not SQL compliant
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from
>> me)
>>>>>>>>>>
>>>>>>>>>> Pros:
>>>>>>>>>> - Easy to add
>>>>>>>>>> - SQL compliant because it is nested in the comments
>>>>>>>>>>
>>>>>>>>>> Cons:
>>>>>>>>>> - Parameters are not part of the main query
>>>>>>>>>> - Cryptic syntax for new users
>>>>>>>>>>
>>>>>>>>>> The biggest problem for hints way may be the “if hints
>> must be
>>>>>>>> optional”,
>>>>>>>>>> actually we have though about 1 for a while but aborted
>>>> because it
>>>>>>>> breaks
>>>>>>>>>> the SQL standard too much. And we replace it with 2,
>> because
>>>> the
>>>>>>> hints
>>>>>>>>>> syntax do not break SQL standard(nested in comments).
>>>>>>>>>>
>>>>>>>>>> What if we have the special /*+ PROPERTIES */ hint that
>> allows
>>>>>>> override
>>>>>>>>>> some properties of table dynamically, it does not break
>>>> anything, at
>>>>>>>>> lease
>>>>>>>>>> for current Flink use cases.
>>>>>>>>>>
>>>>>>>>>> Planner hints are optional just because they are naturally
>>>> enforcers
>>>>>>> of
>>>>>>>>>> the planner, most of them aim to instruct the optimizer,
>> but,
>>>> the
>>>>>>> table
>>>>>>>>>> hints is a little different, table hints can specify the
>> table
>>>> meta
>>>>>>>> like
>>>>>>>>>> index column, and it is very convenient to specify table
>>>> properties.
>>>>>>>>>>
>>>>>>>>>> Or shall we not call /*+ PROPERTIES(offset=123) */ table
>> hint,
>>>> we
>>>>>>> can
>>>>>>>>>> call it table dynamic parameters.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Danny Chan
>>>>>>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
>>>> aljoscha@apache.org>,写道:
>>>>>>>>>>> Hi,
>>>>>>>>>>>
>>>>>>>>>>> I don't understand this discussion. Hints, as I
>> understand
>>>> them,
>>>>>>>> should
>>>>>>>>>>> work like this:
>>>>>>>>>>>
>>>>>>>>>>> - hints are *optional* advice for the optimizer to try
>> and
>>>> help it
>>>>>>> to
>>>>>>>>>>> find a good execution strategy
>>>>>>>>>>> - hints should not change query semantics, i.e. they
>> should
>>>> not
>>>>>>>> change
>>>>>>>>>>> connector properties executing a query with taking into
>>>> account the
>>>>>>>>>>> hints *must* produce the same result as executing the
>> query
>>>> without
>>>>>>>>>>> taking into account the hints
>>>>>>>>>>>
>>>>>>>>>>>  From these simple requirements you can derive a solution
>>>> that makes
>>>>>>>>>>> sense. I don't have a strong preference for the syntax
>> but we
>>>>>>> should
>>>>>>>>>>> strive to be in line with prior work.
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> Aljoscha
>>>>>>>>>>>
>>>>>>>>>>> On 11.03.20 11:53, Danny Chan wrote:
>>>>>>>>>>>> Thanks Timo for summarize the 3 options ~
>>>>>>>>>>>>
>>>>>>>>>>>> I agree with Kurt that option2 is too complicated to
>> use
>>>> because:
>>>>>>>>>>>>
>>>>>>>>>>>> • As a Kafka topic consumer, the user must define both
>> the
>>>>>>> virtual
>>>>>>>>>> column for start offset and he must apply a special filter
>>>> predicate
>>>>>>>>> after
>>>>>>>>>> each query
>>>>>>>>>>>> • And for the internal implementation, the metadata
>> column
>>>> push
>>>>>>>> down
>>>>>>>>>> is another hard topic, each kind of message queue may have
>> its
>>>> offset
>>>>>>>>>> attribute, we need to consider the expression type for
>>>> different
>>>>>>> kind;
>>>>>>>>> the
>>>>>>>>>> source also need to recognize the constant column as a
>> config
>>>>>>>>> option(which
>>>>>>>>>> is weird because usually what we pushed down is a table
>> column)
>>>>>>>>>>>>
>>>>>>>>>>>> For option 1 and option3, I think there is no
>> difference,
>>>> option1
>>>>>>>> is
>>>>>>>>>> also a hint syntax which is introduced in Sybase and
>>>> referenced then
>>>>>>>>>> deprecated by MS-SQL in 199X years because of the
>>>> ambitiousness.
>>>>>>>>> Personally
>>>>>>>>>> I prefer /*+ */ style table hint than WITH keyword for
>> these
>>>> reasons:
>>>>>>>>>>>>
>>>>>>>>>>>> • We do not break the standard SQL, the hints are
>> nested
>>>> in SQL
>>>>>>>>>> comments
>>>>>>>>>>>> • We do not need to introduce additional WITH keyword
>>>> which may
>>>>>>>>> appear
>>>>>>>>>> in a query if we use that because a table can be
>> referenced in
>>>> all
>>>>>>>> kinds
>>>>>>>>> of
>>>>>>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make
>> our
>>>> sql
>>>>>>> query
>>>>>>>>>> break too much of the SQL from standard
>>>>>>>>>>>> • We would have uniform syntax for hints as query
>> hint, one
>>>>>>> syntax
>>>>>>>>>> fits all and more easy to use
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> And here is the reason why we choose a uniform Oracle
>>>> style query
>>>>>>>>>> hint syntax which is addressed by Julian Hyde when we
>> design
>>>> the
>>>>>>> syntax
>>>>>>>>>> from the Calcite community:
>>>>>>>>>>>>
>>>>>>>>>>>> I don’t much like the MSSQL-style syntax for table
>> hints.
>>>> It
>>>>>>> adds a
>>>>>>>>>> new use of the WITH keyword that is unrelated to the use of
>>>> WITH for
>>>>>>>>>> common-table expressions.
>>>>>>>>>>>>
>>>>>>>>>>>> A historical note. Microsoft SQL Server inherited its
>> hint
>>>> syntax
>>>>>>>>> from
>>>>>>>>>> Sybase a very long time ago. (See “Transact SQL
>>>> Programming”[1], page
>>>>>>>>> 632,
>>>>>>>>>> “Optimizer hints”. The book was written in 1999, and covers
>>>> Microsoft
>>>>>>>> SQL
>>>>>>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the
>>>> syntax very
>>>>>>>>>> likely predates Sybase 4.3, from which Microsoft SQL
>> Server was
>>>>>>> forked
>>>>>>>> in
>>>>>>>>>> 1993.)
>>>>>>>>>>>>
>>>>>>>>>>>> Microsoft later added the WITH keyword to make it less
>>>> ambiguous,
>>>>>>>> and
>>>>>>>>>> has now deprecated the syntax that does not use WITH.
>>>>>>>>>>>>
>>>>>>>>>>>> They are forced to keep the syntax for backwards
>>>> compatibility
>>>>>>> but
>>>>>>>>>> that doesn’t mean that we should shoulder their burden.
>>>>>>>>>>>>
>>>>>>>>>>>> I think formatted comments are the right container for
>>>> hints
>>>>>>>> because
>>>>>>>>>> it allows us to change the hint syntax without changing
>> the SQL
>>>>>>> parser,
>>>>>>>>> and
>>>>>>>>>> makes clear that we are at liberty to ignore hints
>> entirely.
>>>>>>>>>>>>
>>>>>>>>>>>> Julian
>>>>>>>>>>>>
>>>>>>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
>>>>>>>>>> https://www.amazon.com/s?k=9781565924017>
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> Danny Chan
>>>>>>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <
>> twalthr@apache.org
>>>>> ,写道:
>>>>>>>>>>>>> Hi Danny,
>>>>>>>>>>>>>
>>>>>>>>>>>>> it is true that our DDL is not standard compliant by
>>>> using the
>>>>>>>> WITH
>>>>>>>>>>>>> clause. Nevertheless, we aim for not diverging too
>> much
>>>> and the
>>>>>>>>> LIKE
>>>>>>>>>>>>> clause is an example of that. It will solve things
>> like
>>>>>>>> overwriting
>>>>>>>>>>>>> WATERMARKs, add additional/modifying properties and
>>>> inherit
>>>>>>>> schema.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Bowen is right that Flink's DDL is mixing 3 types
>>>> definition
>>>>>>>>>> together.
>>>>>>>>>>>>> We are not the first ones that try to solve this.
>> There
>>>> is also
>>>>>>>> the
>>>>>>>>>> SQL
>>>>>>>>>>>>> MED standard [1] that tried to tackle this problem. I
>>>> think it
>>>>>>>> was
>>>>>>>>>> not
>>>>>>>>>>>>> considered when designing the current DDL.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Currently, I see 3 options for handling Kafka
>> offsets. I
>>>> will
>>>>>>>> give
>>>>>>>>>> some
>>>>>>>>>>>>> examples and look forward to feedback here:
>>>>>>>>>>>>>
>>>>>>>>>>>>> *Option 1* Runtime and semantic parms as part of the
>>>> query
>>>>>>>>>>>>>
>>>>>>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
>>>>>>>>>>>>>
>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>> - Parameters are part of the main query
>>>>>>>>>>>>> - No complicated hinting syntax
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>> - Not SQL compliant
>>>>>>>>>>>>>
>>>>>>>>>>>>> *Option 2* Use metadata in query
>>>>>>>>>>>>>
>>>>>>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
>>>>>>>>> SYSTEM_METADATA('offset'))`
>>>>>>>>>>>>>
>>>>>>>>>>>>> `SELECT * FROM MyTable WHERE offset > TIMESTAMP
>>>> '2012-12-12
>>>>>>>>>> 12:34:22'`
>>>>>>>>>>>>>
>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>> - SQL compliant in the query
>>>>>>>>>>>>> - Access of metadata in the DDL which is required
>> anyway
>>>>>>>>>>>>> - Regular pushdown rules apply
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>> - Users need to add an additional comlumn in the DDL
>>>>>>>>>>>>>
>>>>>>>>>>>>> *Option 3*: Use hints for properties
>>>>>>>>>>>>>
>>>>>>>>>>>>> `
>>>>>>>>>>>>> SELECT *
>>>>>>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
>>>>>>>>>>>>> `
>>>>>>>>>>>>>
>>>>>>>>>>>>> Pros:
>>>>>>>>>>>>> - Easy to add
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cons:
>>>>>>>>>>>>> - Parameters are not part of the main query
>>>>>>>>>>>>> - Cryptic syntax for new users
>>>>>>>>>>>>> - Not standard compliant.
>>>>>>>>>>>>>
>>>>>>>>>>>>> If we go with this option, I would suggest to make it
>>>> available
>>>>>>>> in
>>>>>>>>> a
>>>>>>>>>>>>> separate map and don't mix it with statically defined
>>>>>>> properties.
>>>>>>>>>> Such
>>>>>>>>>>>>> that the factory can decide which properties have the
>>>> right to
>>>>>>> be
>>>>>>>>>>>>> overwritten by the hints:
>>>>>>>>>>>>> TableSourceFactory.Context.getQueryHints():
>>>> ReadableConfig
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>> Timo
>>>>>>>>>>>>>
>>>>>>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
>>>>>>>>>>>>>
>>>>>>>>>>>>> Currently I see 3 options as a
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
>>>>>>>>>>>>>> Thanks Bowen ~
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I agree we should somehow categorize our connector
>>>>>>> parameters.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> For type1, I’m already preparing a solution like
>> the
>>>>>>> Confluent
>>>>>>>>>> schema registry + Avro schema inference thing, so this may
>> not
>>>> be a
>>>>>>>>> problem
>>>>>>>>>> in the near future.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> For type3, I have some questions:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> "SELECT * FROM mykafka WHERE offset > 12pm
>> yesterday”
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Where does the offset column come from, a virtual
>>>> column from
>>>>>>>> the
>>>>>>>>>> table schema, you said that
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> They change
>>>>>>>>>>>>>> almost every time a query starts and have nothing
>> to
>>>> do with
>>>>>>>>>> metadata, thus
>>>>>>>>>>>>>> should not be part of table definition/DDL
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> But why you can reference it in the query, I’m
>>>> confused for
>>>>>>>> that,
>>>>>>>>>> can you elaborate a little ?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>> Danny Chan
>>>>>>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <
>>>> bowenli86@gmail.com
>>>>>>>> ,写道:
>>>>>>>>>>>>>>> Thanks Danny for kicking off the effort
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The root cause of too much manual work is Flink
>> DDL
>>>> has
>>>>>>>> mixed 3
>>>>>>>>>> types of
>>>>>>>>>>>>>>> params together and doesn't handle each of them
>> very
>>>> well.
>>>>>>>>> Below
>>>>>>>>>> are how I
>>>>>>>>>>>>>>> categorize them and corresponding solutions in my
>>>> mind:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> - type 1: Metadata of external data, like
>> external
>>>>>>>>> endpoint/url,
>>>>>>>>>>>>>>> username/pwd, schemas, formats.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Such metadata are mostly already accessible in
>>>> external
>>>>>>>> system
>>>>>>>>>> as long as
>>>>>>>>>>>>>>> endpoints and credentials are provided. Flink can
>>>> get it
>>>>>>> thru
>>>>>>>>>> catalogs, but
>>>>>>>>>>>>>>> we haven't had many catalogs yet and thus Flink
>> just
>>>> hasn't
>>>>>>>>> been
>>>>>>>>>> able to
>>>>>>>>>>>>>>> leverage that. So the solution should be building
>>>> more
>>>>>>>>> catalogs.
>>>>>>>>>> Such
>>>>>>>>>>>>>>> params should be part of a Flink table
>>>> DDL/definition, and
>>>>>>>> not
>>>>>>>>>> overridable
>>>>>>>>>>>>>>> in any means.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> - type 2: Runtime params, like jdbc connector's
>>>> fetch size,
>>>>>>>>>> elasticsearch
>>>>>>>>>>>>>>> connector's bulk flush size.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Such params don't affect query results, but
>> affect
>>>> how
>>>>>>>> results
>>>>>>>>>> are produced
>>>>>>>>>>>>>>> (eg. fast or slow, aka performance) - they are
>>>> essentially
>>>>>>>>>> execution and
>>>>>>>>>>>>>>> implementation details. They change often in
>>>> exploration or
>>>>>>>>>> development
>>>>>>>>>>>>>>> stages, but not quite frequently in well-defined
>>>>>>> long-running
>>>>>>>>>> pipelines.
>>>>>>>>>>>>>>> They should always have default values and can be
>>>> missing
>>>>>>> in
>>>>>>>>>> query. They
>>>>>>>>>>>>>>> can be part of a table DDL/definition, but should
>>>> also be
>>>>>>>>>> replaceable in a
>>>>>>>>>>>>>>> query - *this is what table "hints" in FLIP-113
>>>> should
>>>>>>>> cover*.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> - type 3: Semantic params, like kafka connector's
>>>> start
>>>>>>>> offset.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Such params affect query results - the semantics.
>>>> They'd
>>>>>>>> better
>>>>>>>>>> be as
>>>>>>>>>>>>>>> filter conditions in WHERE clause that can be
>> pushed
>>>> down.
>>>>>>>> They
>>>>>>>>>> change
>>>>>>>>>>>>>>> almost every time a query starts and have
>> nothing to
>>>> do
>>>>>>> with
>>>>>>>>>> metadata, thus
>>>>>>>>>>>>>>> should not be part of table definition/DDL, nor
>> be
>>>>>>> persisted
>>>>>>>> in
>>>>>>>>>> catalogs.
>>>>>>>>>>>>>>> If they will, users should create views to keep
>> such
>>>> params
>>>>>>>>>> around (note
>>>>>>>>>>>>>>> this is different from variable substitution).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Take Flink-Kafka as an example. Once we get these
>>>> params
>>>>>>>> right,
>>>>>>>>>> here're the
>>>>>>>>>>>>>>> steps users need to do to develop and run a Flink
>>>> job:
>>>>>>>>>>>>>>> - configure a Flink ConfluentSchemaRegistry with
>> url,
>>>>>>>> username,
>>>>>>>>>> and password
>>>>>>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset > 12pm
>>>> yesterday"
>>>>>>>>>> (simplified
>>>>>>>>>>>>>>> timestamp) in SQL CLI, Flink automatically
>> retrieves
>>>> all
>>>>>>>>>> metadata of
>>>>>>>>>>>>>>> schema, file format, etc and start the job
>>>>>>>>>>>>>>> - users want to make the job read Kafka topic
>>>> faster, so it
>>>>>>>>> goes
>>>>>>>>>> as "SELECT
>>>>>>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/ WHERE
>>>> offset >
>>>>>>> 12pm
>>>>>>>>>> yesterday"
>>>>>>>>>>>>>>> - done and satisfied, users submit it to
>> production
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regarding "CREATE TABLE t LIKE with (k1=v1,
>> k2=v2),
>>>> I think
>>>>>>>>> it's
>>>>>>>>>> a
>>>>>>>>>>>>>>> nice-to-have feature, but not a strategically
>>>> critical,
>>>>>>>>>> long-term solution,
>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>> 1) It may seem promising at the current stage to
>>>> solve the
>>>>>>>>>>>>>>> too-much-manual-work problem, but that's only
>>>> because Flink
>>>>>>>>>> hasn't
>>>>>>>>>>>>>>> leveraged catalogs well and handled the 3 types
>> of
>>>> params
>>>>>>>> above
>>>>>>>>>> properly.
>>>>>>>>>>>>>>> Once we get the params types right, the LIKE
>> syntax
>>>> won't
>>>>>>> be
>>>>>>>>> that
>>>>>>>>>>>>>>> important, and will be just an easier way to
>> create
>>>> tables
>>>>>>>>>> without retyping
>>>>>>>>>>>>>>> long fields like username and pwd.
>>>>>>>>>>>>>>> 2) Note that only some rare type of catalog can
>>>> store k-v
>>>>>>>>>> property pair, so
>>>>>>>>>>>>>>> table created this way often cannot be
>> persisted. In
>>>> the
>>>>>>>>>> foreseeable
>>>>>>>>>>>>>>> future, such catalog will only be HiveCatalog,
>> and
>>>> not
>>>>>>>> everyone
>>>>>>>>>> has a Hive
>>>>>>>>>>>>>>> metastore. To be honest, without persistence,
>>>> recreating
>>>>>>>> tables
>>>>>>>>>> every time
>>>>>>>>>>>>>>> this way is still a lot of keyboard typing.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> Bowen
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <
>>>>>>> ykt836@gmail.com
>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If a specific connector want to have such
>>>> parameter and
>>>>>>>> read
>>>>>>>>>> if out of
>>>>>>>>>>>>>>>> configuration, then that's fine.
>>>>>>>>>>>>>>>> If we are talking about a configuration for all
>>>> kinds of
>>>>>>>>>> sources, I would
>>>>>>>>>>>>>>>> be super careful about that.
>>>>>>>>>>>>>>>> It's true it can solve maybe 80% cases, but it
>>>> will also
>>>>>>>> make
>>>>>>>>>> the left 20%
>>>>>>>>>>>>>>>> feels weird.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> Kurt
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <
>>>>>>> imjark@gmail.com
>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Kurt,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> #3 Regarding to global offset:
>>>>>>>>>>>>>>>>> I'm not saying to use the global
>> configuration to
>>>>>>>> override
>>>>>>>>>> connector
>>>>>>>>>>>>>>>>> properties by the planner.
>>>>>>>>>>>>>>>>> But the connector should take this
>> configuration
>>>> and
>>>>>>>>>> translate into their
>>>>>>>>>>>>>>>>> client API.
>>>>>>>>>>>>>>>>> AFAIK, almost all the message queues support
>>>> eariliest
>>>>>>>> and
>>>>>>>>>> latest and a
>>>>>>>>>>>>>>>>> timestamp value as start point.
>>>>>>>>>>>>>>>>> So we can support 3 options for this
>>>> configuration:
>>>>>>>>>> "eariliest", "latest"
>>>>>>>>>>>>>>>>> and a timestamp string value.
>>>>>>>>>>>>>>>>> Of course, this can't solve 100% cases, but I
>>>> guess can
>>>>>>>>>> sovle 80% or 90%
>>>>>>>>>>>>>>>>> cases.
>>>>>>>>>>>>>>>>> And the remaining cases can be resolved by
>> LIKE
>>>> syntax
>>>>>>>>> which
>>>>>>>>>> I guess is
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>> very common cases.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>> Jark
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt Young <
>>>>>>>> ykt836@gmail.com
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Good to have such lovely discussions. I
>> also
>>>> want to
>>>>>>>>> share
>>>>>>>>>> some of my
>>>>>>>>>>>>>>>>>> opinions.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> #1 Regarding to error handling: I also
>> think
>>>> ignore
>>>>>>>>>> invalid hints would
>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> dangerous, maybe
>>>>>>>>>>>>>>>>>> the simplest solution is just throw an
>>>> exception.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> #2 Regarding to property replacement: I
>> don't
>>>> think
>>>>>>> we
>>>>>>>>>> should
>>>>>>>>>>>>>>>> constraint
>>>>>>>>>>>>>>>>>> ourself to
>>>>>>>>>>>>>>>>>> the meaning of the word "hint", and
>> forbidden
>>>> it
>>>>>>>>> modifying
>>>>>>>>>> any
>>>>>>>>>>>>>>>> properties
>>>>>>>>>>>>>>>>>> which can effect
>>>>>>>>>>>>>>>>>> query results. IMO `PROPERTIES` is one of
>> the
>>>> table
>>>>>>>>> hints,
>>>>>>>>>> and a
>>>>>>>>>>>>>>>> powerful
>>>>>>>>>>>>>>>>>> one. It can
>>>>>>>>>>>>>>>>>> modify properties located in DDL's WITH
>> block.
>>>> But I
>>>>>>>> also
>>>>>>>>>> see the harm
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> if we make it
>>>>>>>>>>>>>>>>>> too flexible like change the kafka topic
>> name
>>>> with a
>>>>>>>>> hint.
>>>>>>>>>> Such use
>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> not common and
>>>>>>>>>>>>>>>>>> sounds very dangerous to me. I would
>> propose
>>>> we have
>>>>>>> a
>>>>>>>>> map
>>>>>>>>>> of hintable
>>>>>>>>>>>>>>>>>> properties for each
>>>>>>>>>>>>>>>>>> connector, and should validate all passed
>> in
>>>>>>> properties
>>>>>>>>>> are actually
>>>>>>>>>>>>>>>>>> hintable. And combining with
>>>>>>>>>>>>>>>>>> #1 error handling, we can throw an
>> exception
>>>> once
>>>>>>>>> received
>>>>>>>>>> invalid
>>>>>>>>>>>>>>>>>> property.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> #3 Regarding to global offset: I'm not sure
>>>> it's
>>>>>>>>> feasible.
>>>>>>>>>> Different
>>>>>>>>>>>>>>>>>> connectors will have totally
>>>>>>>>>>>>>>>>>> different properties to represent offset,
>> some
>>>> might
>>>>>>> be
>>>>>>>>>> timestamps,
>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>> might be string literals
>>>>>>>>>>>>>>>>>> like "earliest", and others might be just
>>>> integers.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> Kurt
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
>>>>>>>>> imjark@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>> twalthr@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 <
>>>>>>>>>> twalthr@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.lwl@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.cyz@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 <
>>>>>>>>>> twalthr@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
>>>>>>>>> <
>>>>>>>>>> imjark@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 <
>>>>>>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>
> 
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jingsong Li <ji...@gmail.com>.
Hi,

I am thinking we can provide hints to *table* related instances.
- TableFormatFactory: of cause we need hints support, there are many format
options in DDL too.
- catalog and module: I don't know, maybe in future we can provide some
hints for them.

Best,
Jingsong Lee

On Wed, Mar 18, 2020 at 12:28 PM Danny Chan <yu...@gmail.com> wrote:

> Yes, I think we should move the `supportedHintOptions` from TableFactory
> to TableSourceFactory, and we also need to add the interface to
> TableSinkFactory though because sink target table may also have hints
> attached.
>
> Best,
> Danny Chan
> 在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> > Have one question for adding `supportedHintOptions` method to
> > `TableFactory`. It seems
> > `TableFactory` is a base factory interface for all *table module* related
> > instances, such as
> > catalog, module, format and so on. It's not created only for *table*. Is
> it
> > possible to move it
> > to `TableSourceFactory`?
> >
> > Best,
> > Kurt
> >
> >
> > On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <yu...@gmail.com>
> wrote:
> >
> > > Thanks Timo ~
> > >
> > > For the naming itself, I also think the PROPERTIES is not that
> concise, so
> > > +1 for OPTIONS (I had thought about that, but there are many codes in
> > > current Flink called it properties, i.e. the DescriptorProperties,
> > > #getSupportedProperties), let’s use OPTIONS if this is our new
> preference.
> > >
> > > +1 to `Set<ConfigOption> supportedHintOptions()` because the
> ConfigOption
> > > can take more info. AFAIK, Spark also call their table options instead
> of
> > > properties. [1]
> > >
> > > In my local POC, I did create a new CatalogTable, and it works for
> current
> > > connectors well, all the DDL tables would finally yield a CatalogTable
> > > instance and we can apply the options to that(in the CatalogSourceTable
> > > when we generating the TableSource), the pros is that we do not need to
> > > modify the codes of connectors itself. If we split the options from
> > > CatalogTable, we may need to add some additional logic in each
> connector
> > > factories in order to merge these properties (and the logic are almost
> the
> > > same), what do you think about this?
> > >
> > > [1]
> > >
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> > >
> > > Best,
> > > Danny Chan
> > > 在 2020年3月17日 +0800 PM10:10,Timo Walther <tw...@apache.org>,写道:
> > > > Hi Danny,
> > > >
> > > > thanks for updating the FLIP. I think your current design is
> sufficient
> > > > to separate hints from result-related properties.
> > > >
> > > > One remark to the naming itself: I would vote for calling the hints
> > > > around table scan `OPTIONS('k'='v')`. We used the term "properties"
> in
> > > > the past but since we want to unify the Flink configuration
> experience,
> > > > we should use consistent naming and classes around `ConfigOptions`.
> > > >
> > > > It would be nice to use `Set<ConfigOption> supportedHintOptions();`
> to
> > > > start using config options instead of pure string properties. This
> will
> > > > also allow us to generate documentation in the future around
> supported
> > > > data types, ranges, etc. for options. At some point we would also
> like
> > > > to drop `DescriptorProperties` class. "Options" is also used in the
> > > > documentation [1] and in the SQL/MED standard [2].
> > > >
> > > > Furthermore, I would still vote for separating CatalogTable and hint
> > > > options. Otherwise the planner would need to create a new
> CatalogTable
> > > > instance which might not always be easy. We should offer them via:
> > > >
> > > > org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > > > ReadableConfig
> > > >
> > > > What do you think?
> > > >
> > > > Regards,
> > > > Timo
> > > >
> > > > [1]
> > > >
> > >
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > > > [2] https://wiki.postgresql.org/wiki/SQL/MED
> > > >
> > > >
> > > > On 12.03.20 15:06, Stephan Ewen wrote:
> > > > > @Danny sounds good.
> > > > >
> > > > > Maybe it is worth listing all the classes of problems that you
> want to
> > > > > address and then look at each class and see if hints are a good
> default
> > > > > solution or a good optional way of simplifying things?
> > > > > The discussion has grown a lot and it is starting to be hard to
> > > distinguish
> > > > > the parts where everyone agrees from the parts were there are
> concerns.
> > > > >
> > > > > On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <da...@apache.org>
> > > wrote:
> > > > >
> > > > > > Thanks Stephan ~
> > > > > >
> > > > > > We can remove the support for properties that may change the
> > > semantics of
> > > > > > query if you think that is a trouble.
> > > > > >
> > > > > > How about we support the /*+ properties() */ hint only for those
> > > optimize
> > > > > > parameters, such as the fetch size of source or something like
> that,
> > > does
> > > > > > that make sense?
> > > > > >
> > > > > > Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > > > > >
> > > > > > > I think Bowen has actually put it very well.
> > > > > > >
> > > > > > > (1) Hints that change semantics looks like trouble waiting to
> > > happen. For
> > > > > > > example Kafka offset handling should be in filters. The Kafka
> > > source
> > > > > > should
> > > > > > > support predicate pushdown.
> > > > > > >
> > > > > > > (2) Hints should not be a workaround for current shortcomings.
> A
> > > lot of
> > > > > > the
> > > > > > > suggested above sounds exactly like that. Working around
> > > catalog/DDL
> > > > > > > shortcomings, missing exposure of metadata (offsets), missing
> > > predicate
> > > > > > > pushdown in Kafka. Abusing a feature like hints now as a quick
> fix
> > > for
> > > > > > > these issues, rather than fixing the root causes, will much
> likely
> > > bite
> > > > > > us
> > > > > > > back badly in the future.
> > > > > > >
> > > > > > > Best,
> > > > > > > Stephan
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com>
> > > wrote:
> > > > > > >
> > > > > > > > It seems this FLIP's name is somewhat misleading. From my
> > > > > > understanding,
> > > > > > > > this FLIP is trying to
> > > > > > > > address the dynamic parameter issue, and table hints is the
> way
> > > we wan
> > > > > > to
> > > > > > > > choose. I think we should
> > > > > > > > be focus on "what's the right way to solve dynamic property"
> > > instead of
> > > > > > > > discussing "whether table
> > > > > > > > hints can affect query semantics".
> > > > > > > >
> > > > > > > > For now, there are two proposed ways to achieve dynamic
> property:
> > > > > > > > 1. FLIP-110: create temporary table xx like xx with (xxx)
> > > > > > > > 2. use custom "from t with (xxx)" syntax
> > > > > > > > 3. "Borrow" the table hints to have a special PROPERTIES
> hint.
> > > > > > > >
> > > > > > > > The first one didn't break anything, but the only problem i
> see
> > > is a
> > > > > > > little
> > > > > > > > more verbose than the table hint
> > > > > > > > approach. I can imagine when someone using SQL CLI to have a
> sql
> > > > > > > > experience, it's quite often that
> > > > > > > > he will modify the table property, some use cases i can
> think of:
> > > > > > > > 1. the source contains some corrupted data, i want to turn
> on the
> > > > > > > > "ignore-error" flag for certain formats.
> > > > > > > > 2. I have a kafka table and want to see some sample data
> from the
> > > > > > > > beginning, so i change the offset
> > > > > > > > to "earliest", and then I want to observe the latest data
> which
> > > keeps
> > > > > > > > coming in. I would write another query
> > > > > > > > to select from the latest table.
> > > > > > > > 3. I want to my jdbc sink flush data more eagerly then i can
> > > observe
> > > > > > the
> > > > > > > > data from database side.
> > > > > > > >
> > > > > > > > Most of such use cases are quite ad-hoc. If every time I
> want to
> > > have a
> > > > > > > > different experience, i need to create
> > > > > > > > a temporary table and then also modify my query, it doesn't
> feel
> > > > > > smooth.
> > > > > > > > Embed such dynamic property into
> > > > > > > > query would have better user experience.
> > > > > > > >
> > > > > > > > Both 2 & 3 can make this happen. The cons of #2 is breaking
> SQL
> > > > > > > compliant,
> > > > > > > > and for #3, it only breaks some
> > > > > > > > unwritten rules, but we can have an explanation on that. And
> I
> > > really
> > > > > > > doubt
> > > > > > > > whether user would complain about
> > > > > > > > this when they actually have flexible and good experience
> using
> > > this.
> > > > > > > >
> > > > > > > > My tendency would be #3 > #1 > #2, what do you think?
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Kurt
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <
> yuzhao.cyz@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks Aljoscha ~
> > > > > > > > >
> > > > > > > > > I agree for most of the query hints, they are optional as
> an
> > > > > > optimizer
> > > > > > > > > instruction, especially for the traditional RDBMS.
> > > > > > > > >
> > > > > > > > > But, just like BenChao said, Flink as a computation engine
> has
> > > many
> > > > > > > > > different kind of data sources, thus, dynamic parameters
> like
> > > > > > > > start_offest
> > > > > > > > > can only bind to each table scope, we can not set a session
> > > config
> > > > > > like
> > > > > > > > > KSQL because they are all about Kafka:
> > > > > > > > > > SET ‘auto.offset.reset’=‘earliest’;
> > > > > > > > >
> > > > > > > > > Thus the most flexible way to set up these dynamic params
> is
> > > to bind
> > > > > > to
> > > > > > > > > the table scope in the query when we want to override
> > > something, so
> > > > > > we
> > > > > > > > have
> > > > > > > > > these solutions above (with pros and cons from my side):
> > > > > > > > >
> > > > > > > > > • 1. Select * from t(offset=123) (from Timo)
> > > > > > > > >
> > > > > > > > > Pros:
> > > > > > > > > - Easy to add
> > > > > > > > > - Parameters are part of the main query
> > > > > > > > > Cons:
> > > > > > > > > - Not SQL compliant
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from
> me)
> > > > > > > > >
> > > > > > > > > Pros:
> > > > > > > > > - Easy to add
> > > > > > > > > - SQL compliant because it is nested in the comments
> > > > > > > > >
> > > > > > > > > Cons:
> > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > - Cryptic syntax for new users
> > > > > > > > >
> > > > > > > > > The biggest problem for hints way may be the “if hints
> must be
> > > > > > > optional”,
> > > > > > > > > actually we have though about 1 for a while but aborted
> > > because it
> > > > > > > breaks
> > > > > > > > > the SQL standard too much. And we replace it with 2,
> because
> > > the
> > > > > > hints
> > > > > > > > > syntax do not break SQL standard(nested in comments).
> > > > > > > > >
> > > > > > > > > What if we have the special /*+ PROPERTIES */ hint that
> allows
> > > > > > override
> > > > > > > > > some properties of table dynamically, it does not break
> > > anything, at
> > > > > > > > lease
> > > > > > > > > for current Flink use cases.
> > > > > > > > >
> > > > > > > > > Planner hints are optional just because they are naturally
> > > enforcers
> > > > > > of
> > > > > > > > > the planner, most of them aim to instruct the optimizer,
> but,
> > > the
> > > > > > table
> > > > > > > > > hints is a little different, table hints can specify the
> table
> > > meta
> > > > > > > like
> > > > > > > > > index column, and it is very convenient to specify table
> > > properties.
> > > > > > > > >
> > > > > > > > > Or shall we not call /*+ PROPERTIES(offset=123) */ table
> hint,
> > > we
> > > > > > can
> > > > > > > > > call it table dynamic parameters.
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Danny Chan
> > > > > > > > > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> > > aljoscha@apache.org>,写道:
> > > > > > > > > > Hi,
> > > > > > > > > >
> > > > > > > > > > I don't understand this discussion. Hints, as I
> understand
> > > them,
> > > > > > > should
> > > > > > > > > > work like this:
> > > > > > > > > >
> > > > > > > > > > - hints are *optional* advice for the optimizer to try
> and
> > > help it
> > > > > > to
> > > > > > > > > > find a good execution strategy
> > > > > > > > > > - hints should not change query semantics, i.e. they
> should
> > > not
> > > > > > > change
> > > > > > > > > > connector properties executing a query with taking into
> > > account the
> > > > > > > > > > hints *must* produce the same result as executing the
> query
> > > without
> > > > > > > > > > taking into account the hints
> > > > > > > > > >
> > > > > > > > > > From these simple requirements you can derive a solution
> > > that makes
> > > > > > > > > > sense. I don't have a strong preference for the syntax
> but we
> > > > > > should
> > > > > > > > > > strive to be in line with prior work.
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Aljoscha
> > > > > > > > > >
> > > > > > > > > > On 11.03.20 11:53, Danny Chan wrote:
> > > > > > > > > > > Thanks Timo for summarize the 3 options ~
> > > > > > > > > > >
> > > > > > > > > > > I agree with Kurt that option2 is too complicated to
> use
> > > because:
> > > > > > > > > > >
> > > > > > > > > > > • As a Kafka topic consumer, the user must define both
> the
> > > > > > virtual
> > > > > > > > > column for start offset and he must apply a special filter
> > > predicate
> > > > > > > > after
> > > > > > > > > each query
> > > > > > > > > > > • And for the internal implementation, the metadata
> column
> > > push
> > > > > > > down
> > > > > > > > > is another hard topic, each kind of message queue may have
> its
> > > offset
> > > > > > > > > attribute, we need to consider the expression type for
> > > different
> > > > > > kind;
> > > > > > > > the
> > > > > > > > > source also need to recognize the constant column as a
> config
> > > > > > > > option(which
> > > > > > > > > is weird because usually what we pushed down is a table
> column)
> > > > > > > > > > >
> > > > > > > > > > > For option 1 and option3, I think there is no
> difference,
> > > option1
> > > > > > > is
> > > > > > > > > also a hint syntax which is introduced in Sybase and
> > > referenced then
> > > > > > > > > deprecated by MS-SQL in 199X years because of the
> > > ambitiousness.
> > > > > > > > Personally
> > > > > > > > > I prefer /*+ */ style table hint than WITH keyword for
> these
> > > reasons:
> > > > > > > > > > >
> > > > > > > > > > > • We do not break the standard SQL, the hints are
> nested
> > > in SQL
> > > > > > > > > comments
> > > > > > > > > > > • We do not need to introduce additional WITH keyword
> > > which may
> > > > > > > > appear
> > > > > > > > > in a query if we use that because a table can be
> referenced in
> > > all
> > > > > > > kinds
> > > > > > > > of
> > > > > > > > > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make
> our
> > > sql
> > > > > > query
> > > > > > > > > break too much of the SQL from standard
> > > > > > > > > > > • We would have uniform syntax for hints as query
> hint, one
> > > > > > syntax
> > > > > > > > > fits all and more easy to use
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > And here is the reason why we choose a uniform Oracle
> > > style query
> > > > > > > > > hint syntax which is addressed by Julian Hyde when we
> design
> > > the
> > > > > > syntax
> > > > > > > > > from the Calcite community:
> > > > > > > > > > >
> > > > > > > > > > > I don’t much like the MSSQL-style syntax for table
> hints.
> > > It
> > > > > > adds a
> > > > > > > > > new use of the WITH keyword that is unrelated to the use of
> > > WITH for
> > > > > > > > > common-table expressions.
> > > > > > > > > > >
> > > > > > > > > > > A historical note. Microsoft SQL Server inherited its
> hint
> > > syntax
> > > > > > > > from
> > > > > > > > > Sybase a very long time ago. (See “Transact SQL
> > > Programming”[1], page
> > > > > > > > 632,
> > > > > > > > > “Optimizer hints”. The book was written in 1999, and covers
> > > Microsoft
> > > > > > > SQL
> > > > > > > > > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the
> > > syntax very
> > > > > > > > > likely predates Sybase 4.3, from which Microsoft SQL
> Server was
> > > > > > forked
> > > > > > > in
> > > > > > > > > 1993.)
> > > > > > > > > > >
> > > > > > > > > > > Microsoft later added the WITH keyword to make it less
> > > ambiguous,
> > > > > > > and
> > > > > > > > > has now deprecated the syntax that does not use WITH.
> > > > > > > > > > >
> > > > > > > > > > > They are forced to keep the syntax for backwards
> > > compatibility
> > > > > > but
> > > > > > > > > that doesn’t mean that we should shoulder their burden.
> > > > > > > > > > >
> > > > > > > > > > > I think formatted comments are the right container for
> > > hints
> > > > > > > because
> > > > > > > > > it allows us to change the hint syntax without changing
> the SQL
> > > > > > parser,
> > > > > > > > and
> > > > > > > > > makes clear that we are at liberty to ignore hints
> entirely.
> > > > > > > > > > >
> > > > > > > > > > > Julian
> > > > > > > > > > >
> > > > > > > > > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > > > > > > > > https://www.amazon.com/s?k=9781565924017>
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Danny Chan
> > > > > > > > > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <
> twalthr@apache.org
> > > > ,写道:
> > > > > > > > > > > > Hi Danny,
> > > > > > > > > > > >
> > > > > > > > > > > > it is true that our DDL is not standard compliant by
> > > using the
> > > > > > > WITH
> > > > > > > > > > > > clause. Nevertheless, we aim for not diverging too
> much
> > > and the
> > > > > > > > LIKE
> > > > > > > > > > > > clause is an example of that. It will solve things
> like
> > > > > > > overwriting
> > > > > > > > > > > > WATERMARKs, add additional/modifying properties and
> > > inherit
> > > > > > > schema.
> > > > > > > > > > > >
> > > > > > > > > > > > Bowen is right that Flink's DDL is mixing 3 types
> > > definition
> > > > > > > > > together.
> > > > > > > > > > > > We are not the first ones that try to solve this.
> There
> > > is also
> > > > > > > the
> > > > > > > > > SQL
> > > > > > > > > > > > MED standard [1] that tried to tackle this problem. I
> > > think it
> > > > > > > was
> > > > > > > > > not
> > > > > > > > > > > > considered when designing the current DDL.
> > > > > > > > > > > >
> > > > > > > > > > > > Currently, I see 3 options for handling Kafka
> offsets. I
> > > will
> > > > > > > give
> > > > > > > > > some
> > > > > > > > > > > > examples and look forward to feedback here:
> > > > > > > > > > > >
> > > > > > > > > > > > *Option 1* Runtime and semantic parms as part of the
> > > query
> > > > > > > > > > > >
> > > > > > > > > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > > > > > > > > >
> > > > > > > > > > > > Pros:
> > > > > > > > > > > > - Easy to add
> > > > > > > > > > > > - Parameters are part of the main query
> > > > > > > > > > > > - No complicated hinting syntax
> > > > > > > > > > > >
> > > > > > > > > > > > Cons:
> > > > > > > > > > > > - Not SQL compliant
> > > > > > > > > > > >
> > > > > > > > > > > > *Option 2* Use metadata in query
> > > > > > > > > > > >
> > > > > > > > > > > > `CREATE TABLE MyTable (id INT, offset AS
> > > > > > > > SYSTEM_METADATA('offset'))`
> > > > > > > > > > > >
> > > > > > > > > > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP
> > > '2012-12-12
> > > > > > > > > 12:34:22'`
> > > > > > > > > > > >
> > > > > > > > > > > > Pros:
> > > > > > > > > > > > - SQL compliant in the query
> > > > > > > > > > > > - Access of metadata in the DDL which is required
> anyway
> > > > > > > > > > > > - Regular pushdown rules apply
> > > > > > > > > > > >
> > > > > > > > > > > > Cons:
> > > > > > > > > > > > - Users need to add an additional comlumn in the DDL
> > > > > > > > > > > >
> > > > > > > > > > > > *Option 3*: Use hints for properties
> > > > > > > > > > > >
> > > > > > > > > > > > `
> > > > > > > > > > > > SELECT *
> > > > > > > > > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > > > > > > > > `
> > > > > > > > > > > >
> > > > > > > > > > > > Pros:
> > > > > > > > > > > > - Easy to add
> > > > > > > > > > > >
> > > > > > > > > > > > Cons:
> > > > > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > > > > - Not standard compliant.
> > > > > > > > > > > >
> > > > > > > > > > > > If we go with this option, I would suggest to make it
> > > available
> > > > > > > in
> > > > > > > > a
> > > > > > > > > > > > separate map and don't mix it with statically defined
> > > > > > properties.
> > > > > > > > > Such
> > > > > > > > > > > > that the factory can decide which properties have the
> > > right to
> > > > > > be
> > > > > > > > > > > > overwritten by the hints:
> > > > > > > > > > > > TableSourceFactory.Context.getQueryHints():
> > > ReadableConfig
> > > > > > > > > > > >
> > > > > > > > > > > > Regards,
> > > > > > > > > > > > Timo
> > > > > > > > > > > >
> > > > > > > > > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > > > > > > > > >
> > > > > > > > > > > > Currently I see 3 options as a
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > > > > > > > > Thanks Bowen ~
> > > > > > > > > > > > >
> > > > > > > > > > > > > I agree we should somehow categorize our connector
> > > > > > parameters.
> > > > > > > > > > > > >
> > > > > > > > > > > > > For type1, I’m already preparing a solution like
> the
> > > > > > Confluent
> > > > > > > > > schema registry + Avro schema inference thing, so this may
> not
> > > be a
> > > > > > > > problem
> > > > > > > > > in the near future.
> > > > > > > > > > > > >
> > > > > > > > > > > > > For type3, I have some questions:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > "SELECT * FROM mykafka WHERE offset > 12pm
> yesterday”
> > > > > > > > > > > > >
> > > > > > > > > > > > > Where does the offset column come from, a virtual
> > > column from
> > > > > > > the
> > > > > > > > > table schema, you said that
> > > > > > > > > > > > >
> > > > > > > > > > > > > > They change
> > > > > > > > > > > > > almost every time a query starts and have nothing
> to
> > > do with
> > > > > > > > > metadata, thus
> > > > > > > > > > > > > should not be part of table definition/DDL
> > > > > > > > > > > > >
> > > > > > > > > > > > > But why you can reference it in the query, I’m
> > > confused for
> > > > > > > that,
> > > > > > > > > can you elaborate a little ?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> > > bowenli86@gmail.com
> > > > > > > ,写道:
> > > > > > > > > > > > > > Thanks Danny for kicking off the effort
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > The root cause of too much manual work is Flink
> DDL
> > > has
> > > > > > > mixed 3
> > > > > > > > > types of
> > > > > > > > > > > > > > params together and doesn't handle each of them
> very
> > > well.
> > > > > > > > Below
> > > > > > > > > are how I
> > > > > > > > > > > > > > categorize them and corresponding solutions in my
> > > mind:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - type 1: Metadata of external data, like
> external
> > > > > > > > endpoint/url,
> > > > > > > > > > > > > > username/pwd, schemas, formats.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Such metadata are mostly already accessible in
> > > external
> > > > > > > system
> > > > > > > > > as long as
> > > > > > > > > > > > > > endpoints and credentials are provided. Flink can
> > > get it
> > > > > > thru
> > > > > > > > > catalogs, but
> > > > > > > > > > > > > > we haven't had many catalogs yet and thus Flink
> just
> > > hasn't
> > > > > > > > been
> > > > > > > > > able to
> > > > > > > > > > > > > > leverage that. So the solution should be building
> > > more
> > > > > > > > catalogs.
> > > > > > > > > Such
> > > > > > > > > > > > > > params should be part of a Flink table
> > > DDL/definition, and
> > > > > > > not
> > > > > > > > > overridable
> > > > > > > > > > > > > > in any means.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - type 2: Runtime params, like jdbc connector's
> > > fetch size,
> > > > > > > > > elasticsearch
> > > > > > > > > > > > > > connector's bulk flush size.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Such params don't affect query results, but
> affect
> > > how
> > > > > > > results
> > > > > > > > > are produced
> > > > > > > > > > > > > > (eg. fast or slow, aka performance) - they are
> > > essentially
> > > > > > > > > execution and
> > > > > > > > > > > > > > implementation details. They change often in
> > > exploration or
> > > > > > > > > development
> > > > > > > > > > > > > > stages, but not quite frequently in well-defined
> > > > > > long-running
> > > > > > > > > pipelines.
> > > > > > > > > > > > > > They should always have default values and can be
> > > missing
> > > > > > in
> > > > > > > > > query. They
> > > > > > > > > > > > > > can be part of a table DDL/definition, but should
> > > also be
> > > > > > > > > replaceable in a
> > > > > > > > > > > > > > query - *this is what table "hints" in FLIP-113
> > > should
> > > > > > > cover*.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - type 3: Semantic params, like kafka connector's
> > > start
> > > > > > > offset.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Such params affect query results - the semantics.
> > > They'd
> > > > > > > better
> > > > > > > > > be as
> > > > > > > > > > > > > > filter conditions in WHERE clause that can be
> pushed
> > > down.
> > > > > > > They
> > > > > > > > > change
> > > > > > > > > > > > > > almost every time a query starts and have
> nothing to
> > > do
> > > > > > with
> > > > > > > > > metadata, thus
> > > > > > > > > > > > > > should not be part of table definition/DDL, nor
> be
> > > > > > persisted
> > > > > > > in
> > > > > > > > > catalogs.
> > > > > > > > > > > > > > If they will, users should create views to keep
> such
> > > params
> > > > > > > > > around (note
> > > > > > > > > > > > > > this is different from variable substitution).
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Take Flink-Kafka as an example. Once we get these
> > > params
> > > > > > > right,
> > > > > > > > > here're the
> > > > > > > > > > > > > > steps users need to do to develop and run a Flink
> > > job:
> > > > > > > > > > > > > > - configure a Flink ConfluentSchemaRegistry with
> url,
> > > > > > > username,
> > > > > > > > > and password
> > > > > > > > > > > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm
> > > yesterday"
> > > > > > > > > (simplified
> > > > > > > > > > > > > > timestamp) in SQL CLI, Flink automatically
> retrieves
> > > all
> > > > > > > > > metadata of
> > > > > > > > > > > > > > schema, file format, etc and start the job
> > > > > > > > > > > > > > - users want to make the job read Kafka topic
> > > faster, so it
> > > > > > > > goes
> > > > > > > > > as "SELECT
> > > > > > > > > > > > > > * FROM mykafka /* faster_read_key=value*/ WHERE
> > > offset >
> > > > > > 12pm
> > > > > > > > > yesterday"
> > > > > > > > > > > > > > - done and satisfied, users submit it to
> production
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1,
> k2=v2),
> > > I think
> > > > > > > > it's
> > > > > > > > > a
> > > > > > > > > > > > > > nice-to-have feature, but not a strategically
> > > critical,
> > > > > > > > > long-term solution,
> > > > > > > > > > > > > > because
> > > > > > > > > > > > > > 1) It may seem promising at the current stage to
> > > solve the
> > > > > > > > > > > > > > too-much-manual-work problem, but that's only
> > > because Flink
> > > > > > > > > hasn't
> > > > > > > > > > > > > > leveraged catalogs well and handled the 3 types
> of
> > > params
> > > > > > > above
> > > > > > > > > properly.
> > > > > > > > > > > > > > Once we get the params types right, the LIKE
> syntax
> > > won't
> > > > > > be
> > > > > > > > that
> > > > > > > > > > > > > > important, and will be just an easier way to
> create
> > > tables
> > > > > > > > > without retyping
> > > > > > > > > > > > > > long fields like username and pwd.
> > > > > > > > > > > > > > 2) Note that only some rare type of catalog can
> > > store k-v
> > > > > > > > > property pair, so
> > > > > > > > > > > > > > table created this way often cannot be
> persisted. In
> > > the
> > > > > > > > > foreseeable
> > > > > > > > > > > > > > future, such catalog will only be HiveCatalog,
> and
> > > not
> > > > > > > everyone
> > > > > > > > > has a Hive
> > > > > > > > > > > > > > metastore. To be honest, without persistence,
> > > recreating
> > > > > > > tables
> > > > > > > > > every time
> > > > > > > > > > > > > > this way is still a lot of keyboard typing.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Cheers,
> > > > > > > > > > > > > > Bowen
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <
> > > > > > ykt836@gmail.com
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > If a specific connector want to have such
> > > parameter and
> > > > > > > read
> > > > > > > > > if out of
> > > > > > > > > > > > > > > configuration, then that's fine.
> > > > > > > > > > > > > > > If we are talking about a configuration for all
> > > kinds of
> > > > > > > > > sources, I would
> > > > > > > > > > > > > > > be super careful about that.
> > > > > > > > > > > > > > > It's true it can solve maybe 80% cases, but it
> > > will also
> > > > > > > make
> > > > > > > > > the left 20%
> > > > > > > > > > > > > > > feels weird.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <
> > > > > > imjark@gmail.com
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Kurt,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > > > > > > > > I'm not saying to use the global
> configuration to
> > > > > > > override
> > > > > > > > > connector
> > > > > > > > > > > > > > > > properties by the planner.
> > > > > > > > > > > > > > > > But the connector should take this
> configuration
> > > and
> > > > > > > > > translate into their
> > > > > > > > > > > > > > > > client API.
> > > > > > > > > > > > > > > > AFAIK, almost all the message queues support
> > > eariliest
> > > > > > > and
> > > > > > > > > latest and a
> > > > > > > > > > > > > > > > timestamp value as start point.
> > > > > > > > > > > > > > > > So we can support 3 options for this
> > > configuration:
> > > > > > > > > "eariliest", "latest"
> > > > > > > > > > > > > > > > and a timestamp string value.
> > > > > > > > > > > > > > > > Of course, this can't solve 100% cases, but I
> > > guess can
> > > > > > > > > sovle 80% or 90%
> > > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > And the remaining cases can be resolved by
> LIKE
> > > syntax
> > > > > > > > which
> > > > > > > > > I guess is
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > very common cases.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Jark
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <
> > > > > > > ykt836@gmail.com
> > > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Good to have such lovely discussions. I
> also
> > > want to
> > > > > > > > share
> > > > > > > > > some of my
> > > > > > > > > > > > > > > > > opinions.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > #1 Regarding to error handling: I also
> think
> > > ignore
> > > > > > > > > invalid hints would
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > dangerous, maybe
> > > > > > > > > > > > > > > > > the simplest solution is just throw an
> > > exception.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > #2 Regarding to property replacement: I
> don't
> > > think
> > > > > > we
> > > > > > > > > should
> > > > > > > > > > > > > > > constraint
> > > > > > > > > > > > > > > > > ourself to
> > > > > > > > > > > > > > > > > the meaning of the word "hint", and
> forbidden
> > > it
> > > > > > > > modifying
> > > > > > > > > any
> > > > > > > > > > > > > > > properties
> > > > > > > > > > > > > > > > > which can effect
> > > > > > > > > > > > > > > > > query results. IMO `PROPERTIES` is one of
> the
> > > table
> > > > > > > > hints,
> > > > > > > > > and a
> > > > > > > > > > > > > > > powerful
> > > > > > > > > > > > > > > > > one. It can
> > > > > > > > > > > > > > > > > modify properties located in DDL's WITH
> block.
> > > But I
> > > > > > > also
> > > > > > > > > see the harm
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > if we make it
> > > > > > > > > > > > > > > > > too flexible like change the kafka topic
> name
> > > with a
> > > > > > > > hint.
> > > > > > > > > Such use
> > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > not common and
> > > > > > > > > > > > > > > > > sounds very dangerous to me. I would
> propose
> > > we have
> > > > > > a
> > > > > > > > map
> > > > > > > > > of hintable
> > > > > > > > > > > > > > > > > properties for each
> > > > > > > > > > > > > > > > > connector, and should validate all passed
> in
> > > > > > properties
> > > > > > > > > are actually
> > > > > > > > > > > > > > > > > hintable. And combining with
> > > > > > > > > > > > > > > > > #1 error handling, we can throw an
> exception
> > > once
> > > > > > > > received
> > > > > > > > > invalid
> > > > > > > > > > > > > > > > > property.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > #3 Regarding to global offset: I'm not sure
> > > it's
> > > > > > > > feasible.
> > > > > > > > > Different
> > > > > > > > > > > > > > > > > connectors will have totally
> > > > > > > > > > > > > > > > > different properties to represent offset,
> some
> > > might
> > > > > > be
> > > > > > > > > timestamps,
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > might be string literals
> > > > > > > > > > > > > > > > > like "earliest", and others might be just
> > > integers.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
> > > > > > > > imjark@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 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 <
> > > > > > > > > twalthr@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 <
> > > > > > > > > twalthr@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.lwl@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.cyz@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 <
> > > > > > > > > twalthr@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
> > > > > > > > <
> > > > > > > > > imjark@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 <
> > > > > > > > > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
>


-- 
Best, Jingsong Lee

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
Yes, I think we should move the `supportedHintOptions` from TableFactory to TableSourceFactory, and we also need to add the interface to TableSinkFactory though because sink target table may also have hints attached.

Best,
Danny Chan
在 2020年3月18日 +0800 AM11:08,Kurt Young <yk...@gmail.com>,写道:
> Have one question for adding `supportedHintOptions` method to
> `TableFactory`. It seems
> `TableFactory` is a base factory interface for all *table module* related
> instances, such as
> catalog, module, format and so on. It's not created only for *table*. Is it
> possible to move it
> to `TableSourceFactory`?
>
> Best,
> Kurt
>
>
> On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <yu...@gmail.com> wrote:
>
> > Thanks Timo ~
> >
> > For the naming itself, I also think the PROPERTIES is not that concise, so
> > +1 for OPTIONS (I had thought about that, but there are many codes in
> > current Flink called it properties, i.e. the DescriptorProperties,
> > #getSupportedProperties), let’s use OPTIONS if this is our new preference.
> >
> > +1 to `Set<ConfigOption> supportedHintOptions()` because the ConfigOption
> > can take more info. AFAIK, Spark also call their table options instead of
> > properties. [1]
> >
> > In my local POC, I did create a new CatalogTable, and it works for current
> > connectors well, all the DDL tables would finally yield a CatalogTable
> > instance and we can apply the options to that(in the CatalogSourceTable
> > when we generating the TableSource), the pros is that we do not need to
> > modify the codes of connectors itself. If we split the options from
> > CatalogTable, we may need to add some additional logic in each connector
> > factories in order to merge these properties (and the logic are almost the
> > same), what do you think about this?
> >
> > [1]
> > https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
> >
> > Best,
> > Danny Chan
> > 在 2020年3月17日 +0800 PM10:10,Timo Walther <tw...@apache.org>,写道:
> > > Hi Danny,
> > >
> > > thanks for updating the FLIP. I think your current design is sufficient
> > > to separate hints from result-related properties.
> > >
> > > One remark to the naming itself: I would vote for calling the hints
> > > around table scan `OPTIONS('k'='v')`. We used the term "properties" in
> > > the past but since we want to unify the Flink configuration experience,
> > > we should use consistent naming and classes around `ConfigOptions`.
> > >
> > > It would be nice to use `Set<ConfigOption> supportedHintOptions();` to
> > > start using config options instead of pure string properties. This will
> > > also allow us to generate documentation in the future around supported
> > > data types, ranges, etc. for options. At some point we would also like
> > > to drop `DescriptorProperties` class. "Options" is also used in the
> > > documentation [1] and in the SQL/MED standard [2].
> > >
> > > Furthermore, I would still vote for separating CatalogTable and hint
> > > options. Otherwise the planner would need to create a new CatalogTable
> > > instance which might not always be easy. We should offer them via:
> > >
> > > org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > > ReadableConfig
> > >
> > > What do you think?
> > >
> > > Regards,
> > > Timo
> > >
> > > [1]
> > >
> > https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > > [2] https://wiki.postgresql.org/wiki/SQL/MED
> > >
> > >
> > > On 12.03.20 15:06, Stephan Ewen wrote:
> > > > @Danny sounds good.
> > > >
> > > > Maybe it is worth listing all the classes of problems that you want to
> > > > address and then look at each class and see if hints are a good default
> > > > solution or a good optional way of simplifying things?
> > > > The discussion has grown a lot and it is starting to be hard to
> > distinguish
> > > > the parts where everyone agrees from the parts were there are concerns.
> > > >
> > > > On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <da...@apache.org>
> > wrote:
> > > >
> > > > > Thanks Stephan ~
> > > > >
> > > > > We can remove the support for properties that may change the
> > semantics of
> > > > > query if you think that is a trouble.
> > > > >
> > > > > How about we support the /*+ properties() */ hint only for those
> > optimize
> > > > > parameters, such as the fetch size of source or something like that,
> > does
> > > > > that make sense?
> > > > >
> > > > > Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > > > >
> > > > > > I think Bowen has actually put it very well.
> > > > > >
> > > > > > (1) Hints that change semantics looks like trouble waiting to
> > happen. For
> > > > > > example Kafka offset handling should be in filters. The Kafka
> > source
> > > > > should
> > > > > > support predicate pushdown.
> > > > > >
> > > > > > (2) Hints should not be a workaround for current shortcomings. A
> > lot of
> > > > > the
> > > > > > suggested above sounds exactly like that. Working around
> > catalog/DDL
> > > > > > shortcomings, missing exposure of metadata (offsets), missing
> > predicate
> > > > > > pushdown in Kafka. Abusing a feature like hints now as a quick fix
> > for
> > > > > > these issues, rather than fixing the root causes, will much likely
> > bite
> > > > > us
> > > > > > back badly in the future.
> > > > > >
> > > > > > Best,
> > > > > > Stephan
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com>
> > wrote:
> > > > > >
> > > > > > > It seems this FLIP's name is somewhat misleading. From my
> > > > > understanding,
> > > > > > > this FLIP is trying to
> > > > > > > address the dynamic parameter issue, and table hints is the way
> > we wan
> > > > > to
> > > > > > > choose. I think we should
> > > > > > > be focus on "what's the right way to solve dynamic property"
> > instead of
> > > > > > > discussing "whether table
> > > > > > > hints can affect query semantics".
> > > > > > >
> > > > > > > For now, there are two proposed ways to achieve dynamic property:
> > > > > > > 1. FLIP-110: create temporary table xx like xx with (xxx)
> > > > > > > 2. use custom "from t with (xxx)" syntax
> > > > > > > 3. "Borrow" the table hints to have a special PROPERTIES hint.
> > > > > > >
> > > > > > > The first one didn't break anything, but the only problem i see
> > is a
> > > > > > little
> > > > > > > more verbose than the table hint
> > > > > > > approach. I can imagine when someone using SQL CLI to have a sql
> > > > > > > experience, it's quite often that
> > > > > > > he will modify the table property, some use cases i can think of:
> > > > > > > 1. the source contains some corrupted data, i want to turn on the
> > > > > > > "ignore-error" flag for certain formats.
> > > > > > > 2. I have a kafka table and want to see some sample data from the
> > > > > > > beginning, so i change the offset
> > > > > > > to "earliest", and then I want to observe the latest data which
> > keeps
> > > > > > > coming in. I would write another query
> > > > > > > to select from the latest table.
> > > > > > > 3. I want to my jdbc sink flush data more eagerly then i can
> > observe
> > > > > the
> > > > > > > data from database side.
> > > > > > >
> > > > > > > Most of such use cases are quite ad-hoc. If every time I want to
> > have a
> > > > > > > different experience, i need to create
> > > > > > > a temporary table and then also modify my query, it doesn't feel
> > > > > smooth.
> > > > > > > Embed such dynamic property into
> > > > > > > query would have better user experience.
> > > > > > >
> > > > > > > Both 2 & 3 can make this happen. The cons of #2 is breaking SQL
> > > > > > compliant,
> > > > > > > and for #3, it only breaks some
> > > > > > > unwritten rules, but we can have an explanation on that. And I
> > really
> > > > > > doubt
> > > > > > > whether user would complain about
> > > > > > > this when they actually have flexible and good experience using
> > this.
> > > > > > >
> > > > > > > My tendency would be #3 > #1 > #2, what do you think?
> > > > > > >
> > > > > > > Best,
> > > > > > > Kurt
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <yuzhao.cyz@gmail.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > Thanks Aljoscha ~
> > > > > > > >
> > > > > > > > I agree for most of the query hints, they are optional as an
> > > > > optimizer
> > > > > > > > instruction, especially for the traditional RDBMS.
> > > > > > > >
> > > > > > > > But, just like BenChao said, Flink as a computation engine has
> > many
> > > > > > > > different kind of data sources, thus, dynamic parameters like
> > > > > > > start_offest
> > > > > > > > can only bind to each table scope, we can not set a session
> > config
> > > > > like
> > > > > > > > KSQL because they are all about Kafka:
> > > > > > > > > SET ‘auto.offset.reset’=‘earliest’;
> > > > > > > >
> > > > > > > > Thus the most flexible way to set up these dynamic params is
> > to bind
> > > > > to
> > > > > > > > the table scope in the query when we want to override
> > something, so
> > > > > we
> > > > > > > have
> > > > > > > > these solutions above (with pros and cons from my side):
> > > > > > > >
> > > > > > > > • 1. Select * from t(offset=123) (from Timo)
> > > > > > > >
> > > > > > > > Pros:
> > > > > > > > - Easy to add
> > > > > > > > - Parameters are part of the main query
> > > > > > > > Cons:
> > > > > > > > - Not SQL compliant
> > > > > > > >
> > > > > > > >
> > > > > > > > • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from me)
> > > > > > > >
> > > > > > > > Pros:
> > > > > > > > - Easy to add
> > > > > > > > - SQL compliant because it is nested in the comments
> > > > > > > >
> > > > > > > > Cons:
> > > > > > > > - Parameters are not part of the main query
> > > > > > > > - Cryptic syntax for new users
> > > > > > > >
> > > > > > > > The biggest problem for hints way may be the “if hints must be
> > > > > > optional”,
> > > > > > > > actually we have though about 1 for a while but aborted
> > because it
> > > > > > breaks
> > > > > > > > the SQL standard too much. And we replace it with 2, because
> > the
> > > > > hints
> > > > > > > > syntax do not break SQL standard(nested in comments).
> > > > > > > >
> > > > > > > > What if we have the special /*+ PROPERTIES */ hint that allows
> > > > > override
> > > > > > > > some properties of table dynamically, it does not break
> > anything, at
> > > > > > > lease
> > > > > > > > for current Flink use cases.
> > > > > > > >
> > > > > > > > Planner hints are optional just because they are naturally
> > enforcers
> > > > > of
> > > > > > > > the planner, most of them aim to instruct the optimizer, but,
> > the
> > > > > table
> > > > > > > > hints is a little different, table hints can specify the table
> > meta
> > > > > > like
> > > > > > > > index column, and it is very convenient to specify table
> > properties.
> > > > > > > >
> > > > > > > > Or shall we not call /*+ PROPERTIES(offset=123) */ table hint,
> > we
> > > > > can
> > > > > > > > call it table dynamic parameters.
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Danny Chan
> > > > > > > > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> > aljoscha@apache.org>,写道:
> > > > > > > > > Hi,
> > > > > > > > >
> > > > > > > > > I don't understand this discussion. Hints, as I understand
> > them,
> > > > > > should
> > > > > > > > > work like this:
> > > > > > > > >
> > > > > > > > > - hints are *optional* advice for the optimizer to try and
> > help it
> > > > > to
> > > > > > > > > find a good execution strategy
> > > > > > > > > - hints should not change query semantics, i.e. they should
> > not
> > > > > > change
> > > > > > > > > connector properties executing a query with taking into
> > account the
> > > > > > > > > hints *must* produce the same result as executing the query
> > without
> > > > > > > > > taking into account the hints
> > > > > > > > >
> > > > > > > > > From these simple requirements you can derive a solution
> > that makes
> > > > > > > > > sense. I don't have a strong preference for the syntax but we
> > > > > should
> > > > > > > > > strive to be in line with prior work.
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Aljoscha
> > > > > > > > >
> > > > > > > > > On 11.03.20 11:53, Danny Chan wrote:
> > > > > > > > > > Thanks Timo for summarize the 3 options ~
> > > > > > > > > >
> > > > > > > > > > I agree with Kurt that option2 is too complicated to use
> > because:
> > > > > > > > > >
> > > > > > > > > > • As a Kafka topic consumer, the user must define both the
> > > > > virtual
> > > > > > > > column for start offset and he must apply a special filter
> > predicate
> > > > > > > after
> > > > > > > > each query
> > > > > > > > > > • And for the internal implementation, the metadata column
> > push
> > > > > > down
> > > > > > > > is another hard topic, each kind of message queue may have its
> > offset
> > > > > > > > attribute, we need to consider the expression type for
> > different
> > > > > kind;
> > > > > > > the
> > > > > > > > source also need to recognize the constant column as a config
> > > > > > > option(which
> > > > > > > > is weird because usually what we pushed down is a table column)
> > > > > > > > > >
> > > > > > > > > > For option 1 and option3, I think there is no difference,
> > option1
> > > > > > is
> > > > > > > > also a hint syntax which is introduced in Sybase and
> > referenced then
> > > > > > > > deprecated by MS-SQL in 199X years because of the
> > ambitiousness.
> > > > > > > Personally
> > > > > > > > I prefer /*+ */ style table hint than WITH keyword for these
> > reasons:
> > > > > > > > > >
> > > > > > > > > > • We do not break the standard SQL, the hints are nested
> > in SQL
> > > > > > > > comments
> > > > > > > > > > • We do not need to introduce additional WITH keyword
> > which may
> > > > > > > appear
> > > > > > > > in a query if we use that because a table can be referenced in
> > all
> > > > > > kinds
> > > > > > > of
> > > > > > > > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our
> > sql
> > > > > query
> > > > > > > > break too much of the SQL from standard
> > > > > > > > > > • We would have uniform syntax for hints as query hint, one
> > > > > syntax
> > > > > > > > fits all and more easy to use
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > And here is the reason why we choose a uniform Oracle
> > style query
> > > > > > > > hint syntax which is addressed by Julian Hyde when we design
> > the
> > > > > syntax
> > > > > > > > from the Calcite community:
> > > > > > > > > >
> > > > > > > > > > I don’t much like the MSSQL-style syntax for table hints.
> > It
> > > > > adds a
> > > > > > > > new use of the WITH keyword that is unrelated to the use of
> > WITH for
> > > > > > > > common-table expressions.
> > > > > > > > > >
> > > > > > > > > > A historical note. Microsoft SQL Server inherited its hint
> > syntax
> > > > > > > from
> > > > > > > > Sybase a very long time ago. (See “Transact SQL
> > Programming”[1], page
> > > > > > > 632,
> > > > > > > > “Optimizer hints”. The book was written in 1999, and covers
> > Microsoft
> > > > > > SQL
> > > > > > > > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the
> > syntax very
> > > > > > > > likely predates Sybase 4.3, from which Microsoft SQL Server was
> > > > > forked
> > > > > > in
> > > > > > > > 1993.)
> > > > > > > > > >
> > > > > > > > > > Microsoft later added the WITH keyword to make it less
> > ambiguous,
> > > > > > and
> > > > > > > > has now deprecated the syntax that does not use WITH.
> > > > > > > > > >
> > > > > > > > > > They are forced to keep the syntax for backwards
> > compatibility
> > > > > but
> > > > > > > > that doesn’t mean that we should shoulder their burden.
> > > > > > > > > >
> > > > > > > > > > I think formatted comments are the right container for
> > hints
> > > > > > because
> > > > > > > > it allows us to change the hint syntax without changing the SQL
> > > > > parser,
> > > > > > > and
> > > > > > > > makes clear that we are at liberty to ignore hints entirely.
> > > > > > > > > >
> > > > > > > > > > Julian
> > > > > > > > > >
> > > > > > > > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > > > > > > > https://www.amazon.com/s?k=9781565924017>
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Danny Chan
> > > > > > > > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <twalthr@apache.org
> > > ,写道:
> > > > > > > > > > > Hi Danny,
> > > > > > > > > > >
> > > > > > > > > > > it is true that our DDL is not standard compliant by
> > using the
> > > > > > WITH
> > > > > > > > > > > clause. Nevertheless, we aim for not diverging too much
> > and the
> > > > > > > LIKE
> > > > > > > > > > > clause is an example of that. It will solve things like
> > > > > > overwriting
> > > > > > > > > > > WATERMARKs, add additional/modifying properties and
> > inherit
> > > > > > schema.
> > > > > > > > > > >
> > > > > > > > > > > Bowen is right that Flink's DDL is mixing 3 types
> > definition
> > > > > > > > together.
> > > > > > > > > > > We are not the first ones that try to solve this. There
> > is also
> > > > > > the
> > > > > > > > SQL
> > > > > > > > > > > MED standard [1] that tried to tackle this problem. I
> > think it
> > > > > > was
> > > > > > > > not
> > > > > > > > > > > considered when designing the current DDL.
> > > > > > > > > > >
> > > > > > > > > > > Currently, I see 3 options for handling Kafka offsets. I
> > will
> > > > > > give
> > > > > > > > some
> > > > > > > > > > > examples and look forward to feedback here:
> > > > > > > > > > >
> > > > > > > > > > > *Option 1* Runtime and semantic parms as part of the
> > query
> > > > > > > > > > >
> > > > > > > > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > > > > > > > >
> > > > > > > > > > > Pros:
> > > > > > > > > > > - Easy to add
> > > > > > > > > > > - Parameters are part of the main query
> > > > > > > > > > > - No complicated hinting syntax
> > > > > > > > > > >
> > > > > > > > > > > Cons:
> > > > > > > > > > > - Not SQL compliant
> > > > > > > > > > >
> > > > > > > > > > > *Option 2* Use metadata in query
> > > > > > > > > > >
> > > > > > > > > > > `CREATE TABLE MyTable (id INT, offset AS
> > > > > > > SYSTEM_METADATA('offset'))`
> > > > > > > > > > >
> > > > > > > > > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP
> > '2012-12-12
> > > > > > > > 12:34:22'`
> > > > > > > > > > >
> > > > > > > > > > > Pros:
> > > > > > > > > > > - SQL compliant in the query
> > > > > > > > > > > - Access of metadata in the DDL which is required anyway
> > > > > > > > > > > - Regular pushdown rules apply
> > > > > > > > > > >
> > > > > > > > > > > Cons:
> > > > > > > > > > > - Users need to add an additional comlumn in the DDL
> > > > > > > > > > >
> > > > > > > > > > > *Option 3*: Use hints for properties
> > > > > > > > > > >
> > > > > > > > > > > `
> > > > > > > > > > > SELECT *
> > > > > > > > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > > > > > > > `
> > > > > > > > > > >
> > > > > > > > > > > Pros:
> > > > > > > > > > > - Easy to add
> > > > > > > > > > >
> > > > > > > > > > > Cons:
> > > > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > > > - Not standard compliant.
> > > > > > > > > > >
> > > > > > > > > > > If we go with this option, I would suggest to make it
> > available
> > > > > > in
> > > > > > > a
> > > > > > > > > > > separate map and don't mix it with statically defined
> > > > > properties.
> > > > > > > > Such
> > > > > > > > > > > that the factory can decide which properties have the
> > right to
> > > > > be
> > > > > > > > > > > overwritten by the hints:
> > > > > > > > > > > TableSourceFactory.Context.getQueryHints():
> > ReadableConfig
> > > > > > > > > > >
> > > > > > > > > > > Regards,
> > > > > > > > > > > Timo
> > > > > > > > > > >
> > > > > > > > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > > > > > > > >
> > > > > > > > > > > Currently I see 3 options as a
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > > > > > > > Thanks Bowen ~
> > > > > > > > > > > >
> > > > > > > > > > > > I agree we should somehow categorize our connector
> > > > > parameters.
> > > > > > > > > > > >
> > > > > > > > > > > > For type1, I’m already preparing a solution like the
> > > > > Confluent
> > > > > > > > schema registry + Avro schema inference thing, so this may not
> > be a
> > > > > > > problem
> > > > > > > > in the near future.
> > > > > > > > > > > >
> > > > > > > > > > > > For type3, I have some questions:
> > > > > > > > > > > >
> > > > > > > > > > > > > "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> > > > > > > > > > > >
> > > > > > > > > > > > Where does the offset column come from, a virtual
> > column from
> > > > > > the
> > > > > > > > table schema, you said that
> > > > > > > > > > > >
> > > > > > > > > > > > > They change
> > > > > > > > > > > > almost every time a query starts and have nothing to
> > do with
> > > > > > > > metadata, thus
> > > > > > > > > > > > should not be part of table definition/DDL
> > > > > > > > > > > >
> > > > > > > > > > > > But why you can reference it in the query, I’m
> > confused for
> > > > > > that,
> > > > > > > > can you elaborate a little ?
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Danny Chan
> > > > > > > > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> > bowenli86@gmail.com
> > > > > > ,写道:
> > > > > > > > > > > > > Thanks Danny for kicking off the effort
> > > > > > > > > > > > >
> > > > > > > > > > > > > The root cause of too much manual work is Flink DDL
> > has
> > > > > > mixed 3
> > > > > > > > types of
> > > > > > > > > > > > > params together and doesn't handle each of them very
> > well.
> > > > > > > Below
> > > > > > > > are how I
> > > > > > > > > > > > > categorize them and corresponding solutions in my
> > mind:
> > > > > > > > > > > > >
> > > > > > > > > > > > > - type 1: Metadata of external data, like external
> > > > > > > endpoint/url,
> > > > > > > > > > > > > username/pwd, schemas, formats.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Such metadata are mostly already accessible in
> > external
> > > > > > system
> > > > > > > > as long as
> > > > > > > > > > > > > endpoints and credentials are provided. Flink can
> > get it
> > > > > thru
> > > > > > > > catalogs, but
> > > > > > > > > > > > > we haven't had many catalogs yet and thus Flink just
> > hasn't
> > > > > > > been
> > > > > > > > able to
> > > > > > > > > > > > > leverage that. So the solution should be building
> > more
> > > > > > > catalogs.
> > > > > > > > Such
> > > > > > > > > > > > > params should be part of a Flink table
> > DDL/definition, and
> > > > > > not
> > > > > > > > overridable
> > > > > > > > > > > > > in any means.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > - type 2: Runtime params, like jdbc connector's
> > fetch size,
> > > > > > > > elasticsearch
> > > > > > > > > > > > > connector's bulk flush size.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Such params don't affect query results, but affect
> > how
> > > > > > results
> > > > > > > > are produced
> > > > > > > > > > > > > (eg. fast or slow, aka performance) - they are
> > essentially
> > > > > > > > execution and
> > > > > > > > > > > > > implementation details. They change often in
> > exploration or
> > > > > > > > development
> > > > > > > > > > > > > stages, but not quite frequently in well-defined
> > > > > long-running
> > > > > > > > pipelines.
> > > > > > > > > > > > > They should always have default values and can be
> > missing
> > > > > in
> > > > > > > > query. They
> > > > > > > > > > > > > can be part of a table DDL/definition, but should
> > also be
> > > > > > > > replaceable in a
> > > > > > > > > > > > > query - *this is what table "hints" in FLIP-113
> > should
> > > > > > cover*.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > - type 3: Semantic params, like kafka connector's
> > start
> > > > > > offset.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Such params affect query results - the semantics.
> > They'd
> > > > > > better
> > > > > > > > be as
> > > > > > > > > > > > > filter conditions in WHERE clause that can be pushed
> > down.
> > > > > > They
> > > > > > > > change
> > > > > > > > > > > > > almost every time a query starts and have nothing to
> > do
> > > > > with
> > > > > > > > metadata, thus
> > > > > > > > > > > > > should not be part of table definition/DDL, nor be
> > > > > persisted
> > > > > > in
> > > > > > > > catalogs.
> > > > > > > > > > > > > If they will, users should create views to keep such
> > params
> > > > > > > > around (note
> > > > > > > > > > > > > this is different from variable substitution).
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Take Flink-Kafka as an example. Once we get these
> > params
> > > > > > right,
> > > > > > > > here're the
> > > > > > > > > > > > > steps users need to do to develop and run a Flink
> > job:
> > > > > > > > > > > > > - configure a Flink ConfluentSchemaRegistry with url,
> > > > > > username,
> > > > > > > > and password
> > > > > > > > > > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm
> > yesterday"
> > > > > > > > (simplified
> > > > > > > > > > > > > timestamp) in SQL CLI, Flink automatically retrieves
> > all
> > > > > > > > metadata of
> > > > > > > > > > > > > schema, file format, etc and start the job
> > > > > > > > > > > > > - users want to make the job read Kafka topic
> > faster, so it
> > > > > > > goes
> > > > > > > > as "SELECT
> > > > > > > > > > > > > * FROM mykafka /* faster_read_key=value*/ WHERE
> > offset >
> > > > > 12pm
> > > > > > > > yesterday"
> > > > > > > > > > > > > - done and satisfied, users submit it to production
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2),
> > I think
> > > > > > > it's
> > > > > > > > a
> > > > > > > > > > > > > nice-to-have feature, but not a strategically
> > critical,
> > > > > > > > long-term solution,
> > > > > > > > > > > > > because
> > > > > > > > > > > > > 1) It may seem promising at the current stage to
> > solve the
> > > > > > > > > > > > > too-much-manual-work problem, but that's only
> > because Flink
> > > > > > > > hasn't
> > > > > > > > > > > > > leveraged catalogs well and handled the 3 types of
> > params
> > > > > > above
> > > > > > > > properly.
> > > > > > > > > > > > > Once we get the params types right, the LIKE syntax
> > won't
> > > > > be
> > > > > > > that
> > > > > > > > > > > > > important, and will be just an easier way to create
> > tables
> > > > > > > > without retyping
> > > > > > > > > > > > > long fields like username and pwd.
> > > > > > > > > > > > > 2) Note that only some rare type of catalog can
> > store k-v
> > > > > > > > property pair, so
> > > > > > > > > > > > > table created this way often cannot be persisted. In
> > the
> > > > > > > > foreseeable
> > > > > > > > > > > > > future, such catalog will only be HiveCatalog, and
> > not
> > > > > > everyone
> > > > > > > > has a Hive
> > > > > > > > > > > > > metastore. To be honest, without persistence,
> > recreating
> > > > > > tables
> > > > > > > > every time
> > > > > > > > > > > > > this way is still a lot of keyboard typing.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Cheers,
> > > > > > > > > > > > > Bowen
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <
> > > > > ykt836@gmail.com
> > > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > If a specific connector want to have such
> > parameter and
> > > > > > read
> > > > > > > > if out of
> > > > > > > > > > > > > > configuration, then that's fine.
> > > > > > > > > > > > > > If we are talking about a configuration for all
> > kinds of
> > > > > > > > sources, I would
> > > > > > > > > > > > > > be super careful about that.
> > > > > > > > > > > > > > It's true it can solve maybe 80% cases, but it
> > will also
> > > > > > make
> > > > > > > > the left 20%
> > > > > > > > > > > > > > feels weird.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <
> > > > > imjark@gmail.com
> > > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Kurt,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > > > > > > > I'm not saying to use the global configuration to
> > > > > > override
> > > > > > > > connector
> > > > > > > > > > > > > > > properties by the planner.
> > > > > > > > > > > > > > > But the connector should take this configuration
> > and
> > > > > > > > translate into their
> > > > > > > > > > > > > > > client API.
> > > > > > > > > > > > > > > AFAIK, almost all the message queues support
> > eariliest
> > > > > > and
> > > > > > > > latest and a
> > > > > > > > > > > > > > > timestamp value as start point.
> > > > > > > > > > > > > > > So we can support 3 options for this
> > configuration:
> > > > > > > > "eariliest", "latest"
> > > > > > > > > > > > > > > and a timestamp string value.
> > > > > > > > > > > > > > > Of course, this can't solve 100% cases, but I
> > guess can
> > > > > > > > sovle 80% or 90%
> > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > And the remaining cases can be resolved by LIKE
> > syntax
> > > > > > > which
> > > > > > > > I guess is
> > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > very common cases.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Jark
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <
> > > > > > ykt836@gmail.com
> > > > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Good to have such lovely discussions. I also
> > want to
> > > > > > > share
> > > > > > > > some of my
> > > > > > > > > > > > > > > > opinions.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > #1 Regarding to error handling: I also think
> > ignore
> > > > > > > > invalid hints would
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > dangerous, maybe
> > > > > > > > > > > > > > > > the simplest solution is just throw an
> > exception.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > #2 Regarding to property replacement: I don't
> > think
> > > > > we
> > > > > > > > should
> > > > > > > > > > > > > > constraint
> > > > > > > > > > > > > > > > ourself to
> > > > > > > > > > > > > > > > the meaning of the word "hint", and forbidden
> > it
> > > > > > > modifying
> > > > > > > > any
> > > > > > > > > > > > > > properties
> > > > > > > > > > > > > > > > which can effect
> > > > > > > > > > > > > > > > query results. IMO `PROPERTIES` is one of the
> > table
> > > > > > > hints,
> > > > > > > > and a
> > > > > > > > > > > > > > powerful
> > > > > > > > > > > > > > > > one. It can
> > > > > > > > > > > > > > > > modify properties located in DDL's WITH block.
> > But I
> > > > > > also
> > > > > > > > see the harm
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > if we make it
> > > > > > > > > > > > > > > > too flexible like change the kafka topic name
> > with a
> > > > > > > hint.
> > > > > > > > Such use
> > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > not common and
> > > > > > > > > > > > > > > > sounds very dangerous to me. I would propose
> > we have
> > > > > a
> > > > > > > map
> > > > > > > > of hintable
> > > > > > > > > > > > > > > > properties for each
> > > > > > > > > > > > > > > > connector, and should validate all passed in
> > > > > properties
> > > > > > > > are actually
> > > > > > > > > > > > > > > > hintable. And combining with
> > > > > > > > > > > > > > > > #1 error handling, we can throw an exception
> > once
> > > > > > > received
> > > > > > > > invalid
> > > > > > > > > > > > > > > > property.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > #3 Regarding to global offset: I'm not sure
> > it's
> > > > > > > feasible.
> > > > > > > > Different
> > > > > > > > > > > > > > > > connectors will have totally
> > > > > > > > > > > > > > > > different properties to represent offset, some
> > might
> > > > > be
> > > > > > > > timestamps,
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > might be string literals
> > > > > > > > > > > > > > > > like "earliest", and others might be just
> > integers.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
> > > > > > > imjark@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 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 <
> > > > > > > > twalthr@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 <
> > > > > > > > twalthr@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.lwl@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.cyz@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 <
> > > > > > > > twalthr@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
> > > > > > > <
> > > > > > > > imjark@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 <
> > > > > > > > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Kurt Young <yk...@gmail.com>.
Have one question for adding `supportedHintOptions` method to
`TableFactory`. It seems
`TableFactory` is a base factory interface for all *table module* related
instances, such as
catalog, module, format and so on. It's not created only for *table*. Is it
possible to move it
to `TableSourceFactory`?

Best,
Kurt


On Wed, Mar 18, 2020 at 10:59 AM Danny Chan <yu...@gmail.com> wrote:

> Thanks Timo ~
>
> For the naming itself, I also think the PROPERTIES is not that concise, so
> +1 for OPTIONS (I had thought about that, but there are many codes in
> current Flink called it properties, i.e. the DescriptorProperties,
> #getSupportedProperties), let’s use OPTIONS if this is our new preference.
>
> +1 to `Set<ConfigOption> supportedHintOptions()` because the ConfigOption
> can take more info. AFAIK, Spark also call their table options instead of
> properties. [1]
>
> In my local POC, I did create a new CatalogTable, and it works for current
> connectors well, all the DDL tables would finally yield a CatalogTable
> instance and we can apply the options to that(in the CatalogSourceTable
> when we generating the TableSource), the pros is that we do not need to
> modify the codes of connectors itself. If we split the options from
> CatalogTable, we may need to add some additional logic in each connector
> factories in order to merge these properties (and the logic are almost the
> same), what do you think about this?
>
> [1]
> https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html
>
> Best,
> Danny Chan
> 在 2020年3月17日 +0800 PM10:10,Timo Walther <tw...@apache.org>,写道:
> > Hi Danny,
> >
> > thanks for updating the FLIP. I think your current design is sufficient
> > to separate hints from result-related properties.
> >
> > One remark to the naming itself: I would vote for calling the hints
> > around table scan `OPTIONS('k'='v')`. We used the term "properties" in
> > the past but since we want to unify the Flink configuration experience,
> > we should use consistent naming and classes around `ConfigOptions`.
> >
> > It would be nice to use `Set<ConfigOption> supportedHintOptions();` to
> > start using config options instead of pure string properties. This will
> > also allow us to generate documentation in the future around supported
> > data types, ranges, etc. for options. At some point we would also like
> > to drop `DescriptorProperties` class. "Options" is also used in the
> > documentation [1] and in the SQL/MED standard [2].
> >
> > Furthermore, I would still vote for separating CatalogTable and hint
> > options. Otherwise the planner would need to create a new CatalogTable
> > instance which might not always be easy. We should offer them via:
> >
> > org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> > ReadableConfig
> >
> > What do you think?
> >
> > Regards,
> > Timo
> >
> > [1]
> >
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> > [2] https://wiki.postgresql.org/wiki/SQL/MED
> >
> >
> > On 12.03.20 15:06, Stephan Ewen wrote:
> > > @Danny sounds good.
> > >
> > > Maybe it is worth listing all the classes of problems that you want to
> > > address and then look at each class and see if hints are a good default
> > > solution or a good optional way of simplifying things?
> > > The discussion has grown a lot and it is starting to be hard to
> distinguish
> > > the parts where everyone agrees from the parts were there are concerns.
> > >
> > > On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <da...@apache.org>
> wrote:
> > >
> > > > Thanks Stephan ~
> > > >
> > > > We can remove the support for properties that may change the
> semantics of
> > > > query if you think that is a trouble.
> > > >
> > > > How about we support the /*+ properties() */ hint only for those
> optimize
> > > > parameters, such as the fetch size of source or something like that,
> does
> > > > that make sense?
> > > >
> > > > Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > > >
> > > > > I think Bowen has actually put it very well.
> > > > >
> > > > > (1) Hints that change semantics looks like trouble waiting to
> happen. For
> > > > > example Kafka offset handling should be in filters. The Kafka
> source
> > > > should
> > > > > support predicate pushdown.
> > > > >
> > > > > (2) Hints should not be a workaround for current shortcomings. A
> lot of
> > > > the
> > > > > suggested above sounds exactly like that. Working around
> catalog/DDL
> > > > > shortcomings, missing exposure of metadata (offsets), missing
> predicate
> > > > > pushdown in Kafka. Abusing a feature like hints now as a quick fix
> for
> > > > > these issues, rather than fixing the root causes, will much likely
> bite
> > > > us
> > > > > back badly in the future.
> > > > >
> > > > > Best,
> > > > > Stephan
> > > > >
> > > > >
> > > > > On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com>
> wrote:
> > > > >
> > > > > > It seems this FLIP's name is somewhat misleading. From my
> > > > understanding,
> > > > > > this FLIP is trying to
> > > > > > address the dynamic parameter issue, and table hints is the way
> we wan
> > > > to
> > > > > > choose. I think we should
> > > > > > be focus on "what's the right way to solve dynamic property"
> instead of
> > > > > > discussing "whether table
> > > > > > hints can affect query semantics".
> > > > > >
> > > > > > For now, there are two proposed ways to achieve dynamic property:
> > > > > > 1. FLIP-110: create temporary table xx like xx with (xxx)
> > > > > > 2. use custom "from t with (xxx)" syntax
> > > > > > 3. "Borrow" the table hints to have a special PROPERTIES hint.
> > > > > >
> > > > > > The first one didn't break anything, but the only problem i see
> is a
> > > > > little
> > > > > > more verbose than the table hint
> > > > > > approach. I can imagine when someone using SQL CLI to have a sql
> > > > > > experience, it's quite often that
> > > > > > he will modify the table property, some use cases i can think of:
> > > > > > 1. the source contains some corrupted data, i want to turn on the
> > > > > > "ignore-error" flag for certain formats.
> > > > > > 2. I have a kafka table and want to see some sample data from the
> > > > > > beginning, so i change the offset
> > > > > > to "earliest", and then I want to observe the latest data which
> keeps
> > > > > > coming in. I would write another query
> > > > > > to select from the latest table.
> > > > > > 3. I want to my jdbc sink flush data more eagerly then i can
> observe
> > > > the
> > > > > > data from database side.
> > > > > >
> > > > > > Most of such use cases are quite ad-hoc. If every time I want to
> have a
> > > > > > different experience, i need to create
> > > > > > a temporary table and then also modify my query, it doesn't feel
> > > > smooth.
> > > > > > Embed such dynamic property into
> > > > > > query would have better user experience.
> > > > > >
> > > > > > Both 2 & 3 can make this happen. The cons of #2 is breaking SQL
> > > > > compliant,
> > > > > > and for #3, it only breaks some
> > > > > > unwritten rules, but we can have an explanation on that. And I
> really
> > > > > doubt
> > > > > > whether user would complain about
> > > > > > this when they actually have flexible and good experience using
> this.
> > > > > >
> > > > > > My tendency would be #3 > #1 > #2, what do you think?
> > > > > >
> > > > > > Best,
> > > > > > Kurt
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <yuzhao.cyz@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > Thanks Aljoscha ~
> > > > > > >
> > > > > > > I agree for most of the query hints, they are optional as an
> > > > optimizer
> > > > > > > instruction, especially for the traditional RDBMS.
> > > > > > >
> > > > > > > But, just like BenChao said, Flink as a computation engine has
> many
> > > > > > > different kind of data sources, thus, dynamic parameters like
> > > > > > start_offest
> > > > > > > can only bind to each table scope, we can not set a session
> config
> > > > like
> > > > > > > KSQL because they are all about Kafka:
> > > > > > > > SET ‘auto.offset.reset’=‘earliest’;
> > > > > > >
> > > > > > > Thus the most flexible way to set up these dynamic params is
> to bind
> > > > to
> > > > > > > the table scope in the query when we want to override
> something, so
> > > > we
> > > > > > have
> > > > > > > these solutions above (with pros and cons from my side):
> > > > > > >
> > > > > > > • 1. Select * from t(offset=123) (from Timo)
> > > > > > >
> > > > > > > Pros:
> > > > > > > - Easy to add
> > > > > > > - Parameters are part of the main query
> > > > > > > Cons:
> > > > > > > - Not SQL compliant
> > > > > > >
> > > > > > >
> > > > > > > • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from me)
> > > > > > >
> > > > > > > Pros:
> > > > > > > - Easy to add
> > > > > > > - SQL compliant because it is nested in the comments
> > > > > > >
> > > > > > > Cons:
> > > > > > > - Parameters are not part of the main query
> > > > > > > - Cryptic syntax for new users
> > > > > > >
> > > > > > > The biggest problem for hints way may be the “if hints must be
> > > > > optional”,
> > > > > > > actually we have though about 1 for a while but aborted
> because it
> > > > > breaks
> > > > > > > the SQL standard too much. And we replace it with 2, because
> the
> > > > hints
> > > > > > > syntax do not break SQL standard(nested in comments).
> > > > > > >
> > > > > > > What if we have the special /*+ PROPERTIES */ hint that allows
> > > > override
> > > > > > > some properties of table dynamically, it does not break
> anything, at
> > > > > > lease
> > > > > > > for current Flink use cases.
> > > > > > >
> > > > > > > Planner hints are optional just because they are naturally
> enforcers
> > > > of
> > > > > > > the planner, most of them aim to instruct the optimizer, but,
> the
> > > > table
> > > > > > > hints is a little different, table hints can specify the table
> meta
> > > > > like
> > > > > > > index column, and it is very convenient to specify table
> properties.
> > > > > > >
> > > > > > > Or shall we not call /*+ PROPERTIES(offset=123) */ table hint,
> we
> > > > can
> > > > > > > call it table dynamic parameters.
> > > > > > >
> > > > > > > Best,
> > > > > > > Danny Chan
> > > > > > > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <
> aljoscha@apache.org>,写道:
> > > > > > > > Hi,
> > > > > > > >
> > > > > > > > I don't understand this discussion. Hints, as I understand
> them,
> > > > > should
> > > > > > > > work like this:
> > > > > > > >
> > > > > > > > - hints are *optional* advice for the optimizer to try and
> help it
> > > > to
> > > > > > > > find a good execution strategy
> > > > > > > > - hints should not change query semantics, i.e. they should
> not
> > > > > change
> > > > > > > > connector properties executing a query with taking into
> account the
> > > > > > > > hints *must* produce the same result as executing the query
> without
> > > > > > > > taking into account the hints
> > > > > > > >
> > > > > > > > From these simple requirements you can derive a solution
> that makes
> > > > > > > > sense. I don't have a strong preference for the syntax but we
> > > > should
> > > > > > > > strive to be in line with prior work.
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Aljoscha
> > > > > > > >
> > > > > > > > On 11.03.20 11:53, Danny Chan wrote:
> > > > > > > > > Thanks Timo for summarize the 3 options ~
> > > > > > > > >
> > > > > > > > > I agree with Kurt that option2 is too complicated to use
> because:
> > > > > > > > >
> > > > > > > > > • As a Kafka topic consumer, the user must define both the
> > > > virtual
> > > > > > > column for start offset and he must apply a special filter
> predicate
> > > > > > after
> > > > > > > each query
> > > > > > > > > • And for the internal implementation, the metadata column
> push
> > > > > down
> > > > > > > is another hard topic, each kind of message queue may have its
> offset
> > > > > > > attribute, we need to consider the expression type for
> different
> > > > kind;
> > > > > > the
> > > > > > > source also need to recognize the constant column as a config
> > > > > > option(which
> > > > > > > is weird because usually what we pushed down is a table column)
> > > > > > > > >
> > > > > > > > > For option 1 and option3, I think there is no difference,
> option1
> > > > > is
> > > > > > > also a hint syntax which is introduced in Sybase and
> referenced then
> > > > > > > deprecated by MS-SQL in 199X years because of the
> ambitiousness.
> > > > > > Personally
> > > > > > > I prefer /*+ */ style table hint than WITH keyword for these
> reasons:
> > > > > > > > >
> > > > > > > > > • We do not break the standard SQL, the hints are nested
> in SQL
> > > > > > > comments
> > > > > > > > > • We do not need to introduce additional WITH keyword
> which may
> > > > > > appear
> > > > > > > in a query if we use that because a table can be referenced in
> all
> > > > > kinds
> > > > > > of
> > > > > > > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our
> sql
> > > > query
> > > > > > > break too much of the SQL from standard
> > > > > > > > > • We would have uniform syntax for hints as query hint, one
> > > > syntax
> > > > > > > fits all and more easy to use
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > And here is the reason why we choose a uniform Oracle
> style query
> > > > > > > hint syntax which is addressed by Julian Hyde when we design
> the
> > > > syntax
> > > > > > > from the Calcite community:
> > > > > > > > >
> > > > > > > > > I don’t much like the MSSQL-style syntax for table hints.
> It
> > > > adds a
> > > > > > > new use of the WITH keyword that is unrelated to the use of
> WITH for
> > > > > > > common-table expressions.
> > > > > > > > >
> > > > > > > > > A historical note. Microsoft SQL Server inherited its hint
> syntax
> > > > > > from
> > > > > > > Sybase a very long time ago. (See “Transact SQL
> Programming”[1], page
> > > > > > 632,
> > > > > > > “Optimizer hints”. The book was written in 1999, and covers
> Microsoft
> > > > > SQL
> > > > > > > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the
> syntax very
> > > > > > > likely predates Sybase 4.3, from which Microsoft SQL Server was
> > > > forked
> > > > > in
> > > > > > > 1993.)
> > > > > > > > >
> > > > > > > > > Microsoft later added the WITH keyword to make it less
> ambiguous,
> > > > > and
> > > > > > > has now deprecated the syntax that does not use WITH.
> > > > > > > > >
> > > > > > > > > They are forced to keep the syntax for backwards
> compatibility
> > > > but
> > > > > > > that doesn’t mean that we should shoulder their burden.
> > > > > > > > >
> > > > > > > > > I think formatted comments are the right container for
> hints
> > > > > because
> > > > > > > it allows us to change the hint syntax without changing the SQL
> > > > parser,
> > > > > > and
> > > > > > > makes clear that we are at liberty to ignore hints entirely.
> > > > > > > > >
> > > > > > > > > Julian
> > > > > > > > >
> > > > > > > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > > > > > > https://www.amazon.com/s?k=9781565924017>
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Danny Chan
> > > > > > > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <twalthr@apache.org
> >,写道:
> > > > > > > > > > Hi Danny,
> > > > > > > > > >
> > > > > > > > > > it is true that our DDL is not standard compliant by
> using the
> > > > > WITH
> > > > > > > > > > clause. Nevertheless, we aim for not diverging too much
> and the
> > > > > > LIKE
> > > > > > > > > > clause is an example of that. It will solve things like
> > > > > overwriting
> > > > > > > > > > WATERMARKs, add additional/modifying properties and
> inherit
> > > > > schema.
> > > > > > > > > >
> > > > > > > > > > Bowen is right that Flink's DDL is mixing 3 types
> definition
> > > > > > > together.
> > > > > > > > > > We are not the first ones that try to solve this. There
> is also
> > > > > the
> > > > > > > SQL
> > > > > > > > > > MED standard [1] that tried to tackle this problem. I
> think it
> > > > > was
> > > > > > > not
> > > > > > > > > > considered when designing the current DDL.
> > > > > > > > > >
> > > > > > > > > > Currently, I see 3 options for handling Kafka offsets. I
> will
> > > > > give
> > > > > > > some
> > > > > > > > > > examples and look forward to feedback here:
> > > > > > > > > >
> > > > > > > > > > *Option 1* Runtime and semantic parms as part of the
> query
> > > > > > > > > >
> > > > > > > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > > > > > > >
> > > > > > > > > > Pros:
> > > > > > > > > > - Easy to add
> > > > > > > > > > - Parameters are part of the main query
> > > > > > > > > > - No complicated hinting syntax
> > > > > > > > > >
> > > > > > > > > > Cons:
> > > > > > > > > > - Not SQL compliant
> > > > > > > > > >
> > > > > > > > > > *Option 2* Use metadata in query
> > > > > > > > > >
> > > > > > > > > > `CREATE TABLE MyTable (id INT, offset AS
> > > > > > SYSTEM_METADATA('offset'))`
> > > > > > > > > >
> > > > > > > > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP
> '2012-12-12
> > > > > > > 12:34:22'`
> > > > > > > > > >
> > > > > > > > > > Pros:
> > > > > > > > > > - SQL compliant in the query
> > > > > > > > > > - Access of metadata in the DDL which is required anyway
> > > > > > > > > > - Regular pushdown rules apply
> > > > > > > > > >
> > > > > > > > > > Cons:
> > > > > > > > > > - Users need to add an additional comlumn in the DDL
> > > > > > > > > >
> > > > > > > > > > *Option 3*: Use hints for properties
> > > > > > > > > >
> > > > > > > > > > `
> > > > > > > > > > SELECT *
> > > > > > > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > > > > > > `
> > > > > > > > > >
> > > > > > > > > > Pros:
> > > > > > > > > > - Easy to add
> > > > > > > > > >
> > > > > > > > > > Cons:
> > > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > > - Not standard compliant.
> > > > > > > > > >
> > > > > > > > > > If we go with this option, I would suggest to make it
> available
> > > > > in
> > > > > > a
> > > > > > > > > > separate map and don't mix it with statically defined
> > > > properties.
> > > > > > > Such
> > > > > > > > > > that the factory can decide which properties have the
> right to
> > > > be
> > > > > > > > > > overwritten by the hints:
> > > > > > > > > > TableSourceFactory.Context.getQueryHints():
> ReadableConfig
> > > > > > > > > >
> > > > > > > > > > Regards,
> > > > > > > > > > Timo
> > > > > > > > > >
> > > > > > > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > > > > > > >
> > > > > > > > > > Currently I see 3 options as a
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > > > > > > Thanks Bowen ~
> > > > > > > > > > >
> > > > > > > > > > > I agree we should somehow categorize our connector
> > > > parameters.
> > > > > > > > > > >
> > > > > > > > > > > For type1, I’m already preparing a solution like the
> > > > Confluent
> > > > > > > schema registry + Avro schema inference thing, so this may not
> be a
> > > > > > problem
> > > > > > > in the near future.
> > > > > > > > > > >
> > > > > > > > > > > For type3, I have some questions:
> > > > > > > > > > >
> > > > > > > > > > > > "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> > > > > > > > > > >
> > > > > > > > > > > Where does the offset column come from, a virtual
> column from
> > > > > the
> > > > > > > table schema, you said that
> > > > > > > > > > >
> > > > > > > > > > > > They change
> > > > > > > > > > > almost every time a query starts and have nothing to
> do with
> > > > > > > metadata, thus
> > > > > > > > > > > should not be part of table definition/DDL
> > > > > > > > > > >
> > > > > > > > > > > But why you can reference it in the query, I’m
> confused for
> > > > > that,
> > > > > > > can you elaborate a little ?
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Danny Chan
> > > > > > > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <
> bowenli86@gmail.com
> > > > > ,写道:
> > > > > > > > > > > > Thanks Danny for kicking off the effort
> > > > > > > > > > > >
> > > > > > > > > > > > The root cause of too much manual work is Flink DDL
> has
> > > > > mixed 3
> > > > > > > types of
> > > > > > > > > > > > params together and doesn't handle each of them very
> well.
> > > > > > Below
> > > > > > > are how I
> > > > > > > > > > > > categorize them and corresponding solutions in my
> mind:
> > > > > > > > > > > >
> > > > > > > > > > > > - type 1: Metadata of external data, like external
> > > > > > endpoint/url,
> > > > > > > > > > > > username/pwd, schemas, formats.
> > > > > > > > > > > >
> > > > > > > > > > > > Such metadata are mostly already accessible in
> external
> > > > > system
> > > > > > > as long as
> > > > > > > > > > > > endpoints and credentials are provided. Flink can
> get it
> > > > thru
> > > > > > > catalogs, but
> > > > > > > > > > > > we haven't had many catalogs yet and thus Flink just
> hasn't
> > > > > > been
> > > > > > > able to
> > > > > > > > > > > > leverage that. So the solution should be building
> more
> > > > > > catalogs.
> > > > > > > Such
> > > > > > > > > > > > params should be part of a Flink table
> DDL/definition, and
> > > > > not
> > > > > > > overridable
> > > > > > > > > > > > in any means.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > - type 2: Runtime params, like jdbc connector's
> fetch size,
> > > > > > > elasticsearch
> > > > > > > > > > > > connector's bulk flush size.
> > > > > > > > > > > >
> > > > > > > > > > > > Such params don't affect query results, but affect
> how
> > > > > results
> > > > > > > are produced
> > > > > > > > > > > > (eg. fast or slow, aka performance) - they are
> essentially
> > > > > > > execution and
> > > > > > > > > > > > implementation details. They change often in
> exploration or
> > > > > > > development
> > > > > > > > > > > > stages, but not quite frequently in well-defined
> > > > long-running
> > > > > > > pipelines.
> > > > > > > > > > > > They should always have default values and can be
> missing
> > > > in
> > > > > > > query. They
> > > > > > > > > > > > can be part of a table DDL/definition, but should
> also be
> > > > > > > replaceable in a
> > > > > > > > > > > > query - *this is what table "hints" in FLIP-113
> should
> > > > > cover*.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > - type 3: Semantic params, like kafka connector's
> start
> > > > > offset.
> > > > > > > > > > > >
> > > > > > > > > > > > Such params affect query results - the semantics.
> They'd
> > > > > better
> > > > > > > be as
> > > > > > > > > > > > filter conditions in WHERE clause that can be pushed
> down.
> > > > > They
> > > > > > > change
> > > > > > > > > > > > almost every time a query starts and have nothing to
> do
> > > > with
> > > > > > > metadata, thus
> > > > > > > > > > > > should not be part of table definition/DDL, nor be
> > > > persisted
> > > > > in
> > > > > > > catalogs.
> > > > > > > > > > > > If they will, users should create views to keep such
> params
> > > > > > > around (note
> > > > > > > > > > > > this is different from variable substitution).
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Take Flink-Kafka as an example. Once we get these
> params
> > > > > right,
> > > > > > > here're the
> > > > > > > > > > > > steps users need to do to develop and run a Flink
> job:
> > > > > > > > > > > > - configure a Flink ConfluentSchemaRegistry with url,
> > > > > username,
> > > > > > > and password
> > > > > > > > > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm
> yesterday"
> > > > > > > (simplified
> > > > > > > > > > > > timestamp) in SQL CLI, Flink automatically retrieves
> all
> > > > > > > metadata of
> > > > > > > > > > > > schema, file format, etc and start the job
> > > > > > > > > > > > - users want to make the job read Kafka topic
> faster, so it
> > > > > > goes
> > > > > > > as "SELECT
> > > > > > > > > > > > * FROM mykafka /* faster_read_key=value*/ WHERE
> offset >
> > > > 12pm
> > > > > > > yesterday"
> > > > > > > > > > > > - done and satisfied, users submit it to production
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2),
> I think
> > > > > > it's
> > > > > > > a
> > > > > > > > > > > > nice-to-have feature, but not a strategically
> critical,
> > > > > > > long-term solution,
> > > > > > > > > > > > because
> > > > > > > > > > > > 1) It may seem promising at the current stage to
> solve the
> > > > > > > > > > > > too-much-manual-work problem, but that's only
> because Flink
> > > > > > > hasn't
> > > > > > > > > > > > leveraged catalogs well and handled the 3 types of
> params
> > > > > above
> > > > > > > properly.
> > > > > > > > > > > > Once we get the params types right, the LIKE syntax
> won't
> > > > be
> > > > > > that
> > > > > > > > > > > > important, and will be just an easier way to create
> tables
> > > > > > > without retyping
> > > > > > > > > > > > long fields like username and pwd.
> > > > > > > > > > > > 2) Note that only some rare type of catalog can
> store k-v
> > > > > > > property pair, so
> > > > > > > > > > > > table created this way often cannot be persisted. In
> the
> > > > > > > foreseeable
> > > > > > > > > > > > future, such catalog will only be HiveCatalog, and
> not
> > > > > everyone
> > > > > > > has a Hive
> > > > > > > > > > > > metastore. To be honest, without persistence,
> recreating
> > > > > tables
> > > > > > > every time
> > > > > > > > > > > > this way is still a lot of keyboard typing.
> > > > > > > > > > > >
> > > > > > > > > > > > Cheers,
> > > > > > > > > > > > Bowen
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <
> > > > ykt836@gmail.com
> > > > > >
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > If a specific connector want to have such
> parameter and
> > > > > read
> > > > > > > if out of
> > > > > > > > > > > > > configuration, then that's fine.
> > > > > > > > > > > > > If we are talking about a configuration for all
> kinds of
> > > > > > > sources, I would
> > > > > > > > > > > > > be super careful about that.
> > > > > > > > > > > > > It's true it can solve maybe 80% cases, but it
> will also
> > > > > make
> > > > > > > the left 20%
> > > > > > > > > > > > > feels weird.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Kurt
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <
> > > > imjark@gmail.com
> > > > > >
> > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi Kurt,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > > > > > > I'm not saying to use the global configuration to
> > > > > override
> > > > > > > connector
> > > > > > > > > > > > > > properties by the planner.
> > > > > > > > > > > > > > But the connector should take this configuration
> and
> > > > > > > translate into their
> > > > > > > > > > > > > > client API.
> > > > > > > > > > > > > > AFAIK, almost all the message queues support
> eariliest
> > > > > and
> > > > > > > latest and a
> > > > > > > > > > > > > > timestamp value as start point.
> > > > > > > > > > > > > > So we can support 3 options for this
> configuration:
> > > > > > > "eariliest", "latest"
> > > > > > > > > > > > > > and a timestamp string value.
> > > > > > > > > > > > > > Of course, this can't solve 100% cases, but I
> guess can
> > > > > > > sovle 80% or 90%
> > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > And the remaining cases can be resolved by LIKE
> syntax
> > > > > > which
> > > > > > > I guess is
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > very common cases.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Jark
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <
> > > > > ykt836@gmail.com
> > > > > > >
> > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Good to have such lovely discussions. I also
> want to
> > > > > > share
> > > > > > > some of my
> > > > > > > > > > > > > > > opinions.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > #1 Regarding to error handling: I also think
> ignore
> > > > > > > invalid hints would
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > dangerous, maybe
> > > > > > > > > > > > > > > the simplest solution is just throw an
> exception.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > #2 Regarding to property replacement: I don't
> think
> > > > we
> > > > > > > should
> > > > > > > > > > > > > constraint
> > > > > > > > > > > > > > > ourself to
> > > > > > > > > > > > > > > the meaning of the word "hint", and forbidden
> it
> > > > > > modifying
> > > > > > > any
> > > > > > > > > > > > > properties
> > > > > > > > > > > > > > > which can effect
> > > > > > > > > > > > > > > query results. IMO `PROPERTIES` is one of the
> table
> > > > > > hints,
> > > > > > > and a
> > > > > > > > > > > > > powerful
> > > > > > > > > > > > > > > one. It can
> > > > > > > > > > > > > > > modify properties located in DDL's WITH block.
> But I
> > > > > also
> > > > > > > see the harm
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > if we make it
> > > > > > > > > > > > > > > too flexible like change the kafka topic name
> with a
> > > > > > hint.
> > > > > > > Such use
> > > > > > > > > > > > > case
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > not common and
> > > > > > > > > > > > > > > sounds very dangerous to me. I would propose
> we have
> > > > a
> > > > > > map
> > > > > > > of hintable
> > > > > > > > > > > > > > > properties for each
> > > > > > > > > > > > > > > connector, and should validate all passed in
> > > > properties
> > > > > > > are actually
> > > > > > > > > > > > > > > hintable. And combining with
> > > > > > > > > > > > > > > #1 error handling, we can throw an exception
> once
> > > > > > received
> > > > > > > invalid
> > > > > > > > > > > > > > > property.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > #3 Regarding to global offset: I'm not sure
> it's
> > > > > > feasible.
> > > > > > > Different
> > > > > > > > > > > > > > > connectors will have totally
> > > > > > > > > > > > > > > different properties to represent offset, some
> might
> > > > be
> > > > > > > timestamps,
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > might be string literals
> > > > > > > > > > > > > > > like "earliest", and others might be just
> integers.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
> > > > > > imjark@gmail.com>
> > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 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 <
> > > > > > > twalthr@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 <
> > > > > > > twalthr@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.lwl@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.cyz@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 <
> > > > > > > twalthr@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
> > > > > > <
> > > > > > > imjark@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 <
> > > > > > > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
Thanks Timo ~

For the naming itself, I also think the PROPERTIES is not that concise, so +1 for OPTIONS (I had thought about that, but there are many codes in current Flink called it properties, i.e. the DescriptorProperties, #getSupportedProperties), let’s use OPTIONS if this is our new preference.

+1 to `Set<ConfigOption> supportedHintOptions()` because the ConfigOption can take more info. AFAIK, Spark also call their table options instead of properties. [1]

In my local POC, I did create a new CatalogTable, and it works for current connectors well, all the DDL tables would finally yield a CatalogTable instance and we can apply the options to that(in the CatalogSourceTable when we generating the TableSource), the pros is that we do not need to modify the codes of connectors itself. If we split the options from CatalogTable, we may need to add some additional logic in each connector factories in order to merge these properties (and the logic are almost the same), what do you think about this?

[1] https://docs.databricks.com/spark/latest/spark-sql/language-manual/create-table.html

Best,
Danny Chan
在 2020年3月17日 +0800 PM10:10,Timo Walther <tw...@apache.org>,写道:
> Hi Danny,
>
> thanks for updating the FLIP. I think your current design is sufficient
> to separate hints from result-related properties.
>
> One remark to the naming itself: I would vote for calling the hints
> around table scan `OPTIONS('k'='v')`. We used the term "properties" in
> the past but since we want to unify the Flink configuration experience,
> we should use consistent naming and classes around `ConfigOptions`.
>
> It would be nice to use `Set<ConfigOption> supportedHintOptions();` to
> start using config options instead of pure string properties. This will
> also allow us to generate documentation in the future around supported
> data types, ranges, etc. for options. At some point we would also like
> to drop `DescriptorProperties` class. "Options" is also used in the
> documentation [1] and in the SQL/MED standard [2].
>
> Furthermore, I would still vote for separating CatalogTable and hint
> options. Otherwise the planner would need to create a new CatalogTable
> instance which might not always be easy. We should offer them via:
>
> org.apache.flink.table.factories.TableSourceFactory.Context#getHints:
> ReadableConfig
>
> What do you think?
>
> Regards,
> Timo
>
> [1]
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
> [2] https://wiki.postgresql.org/wiki/SQL/MED
>
>
> On 12.03.20 15:06, Stephan Ewen wrote:
> > @Danny sounds good.
> >
> > Maybe it is worth listing all the classes of problems that you want to
> > address and then look at each class and see if hints are a good default
> > solution or a good optional way of simplifying things?
> > The discussion has grown a lot and it is starting to be hard to distinguish
> > the parts where everyone agrees from the parts were there are concerns.
> >
> > On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <da...@apache.org> wrote:
> >
> > > Thanks Stephan ~
> > >
> > > We can remove the support for properties that may change the semantics of
> > > query if you think that is a trouble.
> > >
> > > How about we support the /*+ properties() */ hint only for those optimize
> > > parameters, such as the fetch size of source or something like that, does
> > > that make sense?
> > >
> > > Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
> > >
> > > > I think Bowen has actually put it very well.
> > > >
> > > > (1) Hints that change semantics looks like trouble waiting to happen. For
> > > > example Kafka offset handling should be in filters. The Kafka source
> > > should
> > > > support predicate pushdown.
> > > >
> > > > (2) Hints should not be a workaround for current shortcomings. A lot of
> > > the
> > > > suggested above sounds exactly like that. Working around catalog/DDL
> > > > shortcomings, missing exposure of metadata (offsets), missing predicate
> > > > pushdown in Kafka. Abusing a feature like hints now as a quick fix for
> > > > these issues, rather than fixing the root causes, will much likely bite
> > > us
> > > > back badly in the future.
> > > >
> > > > Best,
> > > > Stephan
> > > >
> > > >
> > > > On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com> wrote:
> > > >
> > > > > It seems this FLIP's name is somewhat misleading. From my
> > > understanding,
> > > > > this FLIP is trying to
> > > > > address the dynamic parameter issue, and table hints is the way we wan
> > > to
> > > > > choose. I think we should
> > > > > be focus on "what's the right way to solve dynamic property" instead of
> > > > > discussing "whether table
> > > > > hints can affect query semantics".
> > > > >
> > > > > For now, there are two proposed ways to achieve dynamic property:
> > > > > 1. FLIP-110: create temporary table xx like xx with (xxx)
> > > > > 2. use custom "from t with (xxx)" syntax
> > > > > 3. "Borrow" the table hints to have a special PROPERTIES hint.
> > > > >
> > > > > The first one didn't break anything, but the only problem i see is a
> > > > little
> > > > > more verbose than the table hint
> > > > > approach. I can imagine when someone using SQL CLI to have a sql
> > > > > experience, it's quite often that
> > > > > he will modify the table property, some use cases i can think of:
> > > > > 1. the source contains some corrupted data, i want to turn on the
> > > > > "ignore-error" flag for certain formats.
> > > > > 2. I have a kafka table and want to see some sample data from the
> > > > > beginning, so i change the offset
> > > > > to "earliest", and then I want to observe the latest data which keeps
> > > > > coming in. I would write another query
> > > > > to select from the latest table.
> > > > > 3. I want to my jdbc sink flush data more eagerly then i can observe
> > > the
> > > > > data from database side.
> > > > >
> > > > > Most of such use cases are quite ad-hoc. If every time I want to have a
> > > > > different experience, i need to create
> > > > > a temporary table and then also modify my query, it doesn't feel
> > > smooth.
> > > > > Embed such dynamic property into
> > > > > query would have better user experience.
> > > > >
> > > > > Both 2 & 3 can make this happen. The cons of #2 is breaking SQL
> > > > compliant,
> > > > > and for #3, it only breaks some
> > > > > unwritten rules, but we can have an explanation on that. And I really
> > > > doubt
> > > > > whether user would complain about
> > > > > this when they actually have flexible and good experience using this.
> > > > >
> > > > > My tendency would be #3 > #1 > #2, what do you think?
> > > > >
> > > > > Best,
> > > > > Kurt
> > > > >
> > > > >
> > > > > On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <yu...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Thanks Aljoscha ~
> > > > > >
> > > > > > I agree for most of the query hints, they are optional as an
> > > optimizer
> > > > > > instruction, especially for the traditional RDBMS.
> > > > > >
> > > > > > But, just like BenChao said, Flink as a computation engine has many
> > > > > > different kind of data sources, thus, dynamic parameters like
> > > > > start_offest
> > > > > > can only bind to each table scope, we can not set a session config
> > > like
> > > > > > KSQL because they are all about Kafka:
> > > > > > > SET ‘auto.offset.reset’=‘earliest’;
> > > > > >
> > > > > > Thus the most flexible way to set up these dynamic params is to bind
> > > to
> > > > > > the table scope in the query when we want to override something, so
> > > we
> > > > > have
> > > > > > these solutions above (with pros and cons from my side):
> > > > > >
> > > > > > • 1. Select * from t(offset=123) (from Timo)
> > > > > >
> > > > > > Pros:
> > > > > > - Easy to add
> > > > > > - Parameters are part of the main query
> > > > > > Cons:
> > > > > > - Not SQL compliant
> > > > > >
> > > > > >
> > > > > > • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from me)
> > > > > >
> > > > > > Pros:
> > > > > > - Easy to add
> > > > > > - SQL compliant because it is nested in the comments
> > > > > >
> > > > > > Cons:
> > > > > > - Parameters are not part of the main query
> > > > > > - Cryptic syntax for new users
> > > > > >
> > > > > > The biggest problem for hints way may be the “if hints must be
> > > > optional”,
> > > > > > actually we have though about 1 for a while but aborted because it
> > > > breaks
> > > > > > the SQL standard too much. And we replace it with 2, because the
> > > hints
> > > > > > syntax do not break SQL standard(nested in comments).
> > > > > >
> > > > > > What if we have the special /*+ PROPERTIES */ hint that allows
> > > override
> > > > > > some properties of table dynamically, it does not break anything, at
> > > > > lease
> > > > > > for current Flink use cases.
> > > > > >
> > > > > > Planner hints are optional just because they are naturally enforcers
> > > of
> > > > > > the planner, most of them aim to instruct the optimizer, but, the
> > > table
> > > > > > hints is a little different, table hints can specify the table meta
> > > > like
> > > > > > index column, and it is very convenient to specify table properties.
> > > > > >
> > > > > > Or shall we not call /*+ PROPERTIES(offset=123) */ table hint, we
> > > can
> > > > > > call it table dynamic parameters.
> > > > > >
> > > > > > Best,
> > > > > > Danny Chan
> > > > > > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <al...@apache.org>,写道:
> > > > > > > Hi,
> > > > > > >
> > > > > > > I don't understand this discussion. Hints, as I understand them,
> > > > should
> > > > > > > work like this:
> > > > > > >
> > > > > > > - hints are *optional* advice for the optimizer to try and help it
> > > to
> > > > > > > find a good execution strategy
> > > > > > > - hints should not change query semantics, i.e. they should not
> > > > change
> > > > > > > connector properties executing a query with taking into account the
> > > > > > > hints *must* produce the same result as executing the query without
> > > > > > > taking into account the hints
> > > > > > >
> > > > > > > From these simple requirements you can derive a solution that makes
> > > > > > > sense. I don't have a strong preference for the syntax but we
> > > should
> > > > > > > strive to be in line with prior work.
> > > > > > >
> > > > > > > Best,
> > > > > > > Aljoscha
> > > > > > >
> > > > > > > On 11.03.20 11:53, Danny Chan wrote:
> > > > > > > > Thanks Timo for summarize the 3 options ~
> > > > > > > >
> > > > > > > > I agree with Kurt that option2 is too complicated to use because:
> > > > > > > >
> > > > > > > > • As a Kafka topic consumer, the user must define both the
> > > virtual
> > > > > > column for start offset and he must apply a special filter predicate
> > > > > after
> > > > > > each query
> > > > > > > > • And for the internal implementation, the metadata column push
> > > > down
> > > > > > is another hard topic, each kind of message queue may have its offset
> > > > > > attribute, we need to consider the expression type for different
> > > kind;
> > > > > the
> > > > > > source also need to recognize the constant column as a config
> > > > > option(which
> > > > > > is weird because usually what we pushed down is a table column)
> > > > > > > >
> > > > > > > > For option 1 and option3, I think there is no difference, option1
> > > > is
> > > > > > also a hint syntax which is introduced in Sybase and referenced then
> > > > > > deprecated by MS-SQL in 199X years because of the ambitiousness.
> > > > > Personally
> > > > > > I prefer /*+ */ style table hint than WITH keyword for these reasons:
> > > > > > > >
> > > > > > > > • We do not break the standard SQL, the hints are nested in SQL
> > > > > > comments
> > > > > > > > • We do not need to introduce additional WITH keyword which may
> > > > > appear
> > > > > > in a query if we use that because a table can be referenced in all
> > > > kinds
> > > > > of
> > > > > > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql
> > > query
> > > > > > break too much of the SQL from standard
> > > > > > > > • We would have uniform syntax for hints as query hint, one
> > > syntax
> > > > > > fits all and more easy to use
> > > > > > > >
> > > > > > > >
> > > > > > > > And here is the reason why we choose a uniform Oracle style query
> > > > > > hint syntax which is addressed by Julian Hyde when we design the
> > > syntax
> > > > > > from the Calcite community:
> > > > > > > >
> > > > > > > > I don’t much like the MSSQL-style syntax for table hints. It
> > > adds a
> > > > > > new use of the WITH keyword that is unrelated to the use of WITH for
> > > > > > common-table expressions.
> > > > > > > >
> > > > > > > > A historical note. Microsoft SQL Server inherited its hint syntax
> > > > > from
> > > > > > Sybase a very long time ago. (See “Transact SQL Programming”[1], page
> > > > > 632,
> > > > > > “Optimizer hints”. The book was written in 1999, and covers Microsoft
> > > > SQL
> > > > > > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very
> > > > > > likely predates Sybase 4.3, from which Microsoft SQL Server was
> > > forked
> > > > in
> > > > > > 1993.)
> > > > > > > >
> > > > > > > > Microsoft later added the WITH keyword to make it less ambiguous,
> > > > and
> > > > > > has now deprecated the syntax that does not use WITH.
> > > > > > > >
> > > > > > > > They are forced to keep the syntax for backwards compatibility
> > > but
> > > > > > that doesn’t mean that we should shoulder their burden.
> > > > > > > >
> > > > > > > > I think formatted comments are the right container for hints
> > > > because
> > > > > > it allows us to change the hint syntax without changing the SQL
> > > parser,
> > > > > and
> > > > > > makes clear that we are at liberty to ignore hints entirely.
> > > > > > > >
> > > > > > > > Julian
> > > > > > > >
> > > > > > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > > > > > https://www.amazon.com/s?k=9781565924017>
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Danny Chan
> > > > > > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
> > > > > > > > > Hi Danny,
> > > > > > > > >
> > > > > > > > > it is true that our DDL is not standard compliant by using the
> > > > WITH
> > > > > > > > > clause. Nevertheless, we aim for not diverging too much and the
> > > > > LIKE
> > > > > > > > > clause is an example of that. It will solve things like
> > > > overwriting
> > > > > > > > > WATERMARKs, add additional/modifying properties and inherit
> > > > schema.
> > > > > > > > >
> > > > > > > > > Bowen is right that Flink's DDL is mixing 3 types definition
> > > > > > together.
> > > > > > > > > We are not the first ones that try to solve this. There is also
> > > > the
> > > > > > SQL
> > > > > > > > > MED standard [1] that tried to tackle this problem. I think it
> > > > was
> > > > > > not
> > > > > > > > > considered when designing the current DDL.
> > > > > > > > >
> > > > > > > > > Currently, I see 3 options for handling Kafka offsets. I will
> > > > give
> > > > > > some
> > > > > > > > > examples and look forward to feedback here:
> > > > > > > > >
> > > > > > > > > *Option 1* Runtime and semantic parms as part of the query
> > > > > > > > >
> > > > > > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > > > > > >
> > > > > > > > > Pros:
> > > > > > > > > - Easy to add
> > > > > > > > > - Parameters are part of the main query
> > > > > > > > > - No complicated hinting syntax
> > > > > > > > >
> > > > > > > > > Cons:
> > > > > > > > > - Not SQL compliant
> > > > > > > > >
> > > > > > > > > *Option 2* Use metadata in query
> > > > > > > > >
> > > > > > > > > `CREATE TABLE MyTable (id INT, offset AS
> > > > > SYSTEM_METADATA('offset'))`
> > > > > > > > >
> > > > > > > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12
> > > > > > 12:34:22'`
> > > > > > > > >
> > > > > > > > > Pros:
> > > > > > > > > - SQL compliant in the query
> > > > > > > > > - Access of metadata in the DDL which is required anyway
> > > > > > > > > - Regular pushdown rules apply
> > > > > > > > >
> > > > > > > > > Cons:
> > > > > > > > > - Users need to add an additional comlumn in the DDL
> > > > > > > > >
> > > > > > > > > *Option 3*: Use hints for properties
> > > > > > > > >
> > > > > > > > > `
> > > > > > > > > SELECT *
> > > > > > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > > > > > `
> > > > > > > > >
> > > > > > > > > Pros:
> > > > > > > > > - Easy to add
> > > > > > > > >
> > > > > > > > > Cons:
> > > > > > > > > - Parameters are not part of the main query
> > > > > > > > > - Cryptic syntax for new users
> > > > > > > > > - Not standard compliant.
> > > > > > > > >
> > > > > > > > > If we go with this option, I would suggest to make it available
> > > > in
> > > > > a
> > > > > > > > > separate map and don't mix it with statically defined
> > > properties.
> > > > > > Such
> > > > > > > > > that the factory can decide which properties have the right to
> > > be
> > > > > > > > > overwritten by the hints:
> > > > > > > > > TableSourceFactory.Context.getQueryHints(): ReadableConfig
> > > > > > > > >
> > > > > > > > > Regards,
> > > > > > > > > Timo
> > > > > > > > >
> > > > > > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > > > > > >
> > > > > > > > > Currently I see 3 options as a
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > > > > > Thanks Bowen ~
> > > > > > > > > >
> > > > > > > > > > I agree we should somehow categorize our connector
> > > parameters.
> > > > > > > > > >
> > > > > > > > > > For type1, I’m already preparing a solution like the
> > > Confluent
> > > > > > schema registry + Avro schema inference thing, so this may not be a
> > > > > problem
> > > > > > in the near future.
> > > > > > > > > >
> > > > > > > > > > For type3, I have some questions:
> > > > > > > > > >
> > > > > > > > > > > "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> > > > > > > > > >
> > > > > > > > > > Where does the offset column come from, a virtual column from
> > > > the
> > > > > > table schema, you said that
> > > > > > > > > >
> > > > > > > > > > > They change
> > > > > > > > > > almost every time a query starts and have nothing to do with
> > > > > > metadata, thus
> > > > > > > > > > should not be part of table definition/DDL
> > > > > > > > > >
> > > > > > > > > > But why you can reference it in the query, I’m confused for
> > > > that,
> > > > > > can you elaborate a little ?
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Danny Chan
> > > > > > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <bowenli86@gmail.com
> > > > ,写道:
> > > > > > > > > > > Thanks Danny for kicking off the effort
> > > > > > > > > > >
> > > > > > > > > > > The root cause of too much manual work is Flink DDL has
> > > > mixed 3
> > > > > > types of
> > > > > > > > > > > params together and doesn't handle each of them very well.
> > > > > Below
> > > > > > are how I
> > > > > > > > > > > categorize them and corresponding solutions in my mind:
> > > > > > > > > > >
> > > > > > > > > > > - type 1: Metadata of external data, like external
> > > > > endpoint/url,
> > > > > > > > > > > username/pwd, schemas, formats.
> > > > > > > > > > >
> > > > > > > > > > > Such metadata are mostly already accessible in external
> > > > system
> > > > > > as long as
> > > > > > > > > > > endpoints and credentials are provided. Flink can get it
> > > thru
> > > > > > catalogs, but
> > > > > > > > > > > we haven't had many catalogs yet and thus Flink just hasn't
> > > > > been
> > > > > > able to
> > > > > > > > > > > leverage that. So the solution should be building more
> > > > > catalogs.
> > > > > > Such
> > > > > > > > > > > params should be part of a Flink table DDL/definition, and
> > > > not
> > > > > > overridable
> > > > > > > > > > > in any means.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > - type 2: Runtime params, like jdbc connector's fetch size,
> > > > > > elasticsearch
> > > > > > > > > > > connector's bulk flush size.
> > > > > > > > > > >
> > > > > > > > > > > Such params don't affect query results, but affect how
> > > > results
> > > > > > are produced
> > > > > > > > > > > (eg. fast or slow, aka performance) - they are essentially
> > > > > > execution and
> > > > > > > > > > > implementation details. They change often in exploration or
> > > > > > development
> > > > > > > > > > > stages, but not quite frequently in well-defined
> > > long-running
> > > > > > pipelines.
> > > > > > > > > > > They should always have default values and can be missing
> > > in
> > > > > > query. They
> > > > > > > > > > > can be part of a table DDL/definition, but should also be
> > > > > > replaceable in a
> > > > > > > > > > > query - *this is what table "hints" in FLIP-113 should
> > > > cover*.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > - type 3: Semantic params, like kafka connector's start
> > > > offset.
> > > > > > > > > > >
> > > > > > > > > > > Such params affect query results - the semantics. They'd
> > > > better
> > > > > > be as
> > > > > > > > > > > filter conditions in WHERE clause that can be pushed down.
> > > > They
> > > > > > change
> > > > > > > > > > > almost every time a query starts and have nothing to do
> > > with
> > > > > > metadata, thus
> > > > > > > > > > > should not be part of table definition/DDL, nor be
> > > persisted
> > > > in
> > > > > > catalogs.
> > > > > > > > > > > If they will, users should create views to keep such params
> > > > > > around (note
> > > > > > > > > > > this is different from variable substitution).
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Take Flink-Kafka as an example. Once we get these params
> > > > right,
> > > > > > here're the
> > > > > > > > > > > steps users need to do to develop and run a Flink job:
> > > > > > > > > > > - configure a Flink ConfluentSchemaRegistry with url,
> > > > username,
> > > > > > and password
> > > > > > > > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday"
> > > > > > (simplified
> > > > > > > > > > > timestamp) in SQL CLI, Flink automatically retrieves all
> > > > > > metadata of
> > > > > > > > > > > schema, file format, etc and start the job
> > > > > > > > > > > - users want to make the job read Kafka topic faster, so it
> > > > > goes
> > > > > > as "SELECT
> > > > > > > > > > > * FROM mykafka /* faster_read_key=value*/ WHERE offset >
> > > 12pm
> > > > > > yesterday"
> > > > > > > > > > > - done and satisfied, users submit it to production
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think
> > > > > it's
> > > > > > a
> > > > > > > > > > > nice-to-have feature, but not a strategically critical,
> > > > > > long-term solution,
> > > > > > > > > > > because
> > > > > > > > > > > 1) It may seem promising at the current stage to solve the
> > > > > > > > > > > too-much-manual-work problem, but that's only because Flink
> > > > > > hasn't
> > > > > > > > > > > leveraged catalogs well and handled the 3 types of params
> > > > above
> > > > > > properly.
> > > > > > > > > > > Once we get the params types right, the LIKE syntax won't
> > > be
> > > > > that
> > > > > > > > > > > important, and will be just an easier way to create tables
> > > > > > without retyping
> > > > > > > > > > > long fields like username and pwd.
> > > > > > > > > > > 2) Note that only some rare type of catalog can store k-v
> > > > > > property pair, so
> > > > > > > > > > > table created this way often cannot be persisted. In the
> > > > > > foreseeable
> > > > > > > > > > > future, such catalog will only be HiveCatalog, and not
> > > > everyone
> > > > > > has a Hive
> > > > > > > > > > > metastore. To be honest, without persistence, recreating
> > > > tables
> > > > > > every time
> > > > > > > > > > > this way is still a lot of keyboard typing.
> > > > > > > > > > >
> > > > > > > > > > > Cheers,
> > > > > > > > > > > Bowen
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <
> > > ykt836@gmail.com
> > > > >
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > If a specific connector want to have such parameter and
> > > > read
> > > > > > if out of
> > > > > > > > > > > > configuration, then that's fine.
> > > > > > > > > > > > If we are talking about a configuration for all kinds of
> > > > > > sources, I would
> > > > > > > > > > > > be super careful about that.
> > > > > > > > > > > > It's true it can solve maybe 80% cases, but it will also
> > > > make
> > > > > > the left 20%
> > > > > > > > > > > > feels weird.
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Kurt
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <
> > > imjark@gmail.com
> > > > >
> > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi Kurt,
> > > > > > > > > > > > >
> > > > > > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > > > > > I'm not saying to use the global configuration to
> > > > override
> > > > > > connector
> > > > > > > > > > > > > properties by the planner.
> > > > > > > > > > > > > But the connector should take this configuration and
> > > > > > translate into their
> > > > > > > > > > > > > client API.
> > > > > > > > > > > > > AFAIK, almost all the message queues support eariliest
> > > > and
> > > > > > latest and a
> > > > > > > > > > > > > timestamp value as start point.
> > > > > > > > > > > > > So we can support 3 options for this configuration:
> > > > > > "eariliest", "latest"
> > > > > > > > > > > > > and a timestamp string value.
> > > > > > > > > > > > > Of course, this can't solve 100% cases, but I guess can
> > > > > > sovle 80% or 90%
> > > > > > > > > > > > > cases.
> > > > > > > > > > > > > And the remaining cases can be resolved by LIKE syntax
> > > > > which
> > > > > > I guess is
> > > > > > > > > > > > not
> > > > > > > > > > > > > very common cases.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Jark
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <
> > > > ykt836@gmail.com
> > > > > >
> > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Good to have such lovely discussions. I also want to
> > > > > share
> > > > > > some of my
> > > > > > > > > > > > > > opinions.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > #1 Regarding to error handling: I also think ignore
> > > > > > invalid hints would
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > dangerous, maybe
> > > > > > > > > > > > > > the simplest solution is just throw an exception.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > #2 Regarding to property replacement: I don't think
> > > we
> > > > > > should
> > > > > > > > > > > > constraint
> > > > > > > > > > > > > > ourself to
> > > > > > > > > > > > > > the meaning of the word "hint", and forbidden it
> > > > > modifying
> > > > > > any
> > > > > > > > > > > > properties
> > > > > > > > > > > > > > which can effect
> > > > > > > > > > > > > > query results. IMO `PROPERTIES` is one of the table
> > > > > hints,
> > > > > > and a
> > > > > > > > > > > > powerful
> > > > > > > > > > > > > > one. It can
> > > > > > > > > > > > > > modify properties located in DDL's WITH block. But I
> > > > also
> > > > > > see the harm
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > if we make it
> > > > > > > > > > > > > > too flexible like change the kafka topic name with a
> > > > > hint.
> > > > > > Such use
> > > > > > > > > > > > case
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > not common and
> > > > > > > > > > > > > > sounds very dangerous to me. I would propose we have
> > > a
> > > > > map
> > > > > > of hintable
> > > > > > > > > > > > > > properties for each
> > > > > > > > > > > > > > connector, and should validate all passed in
> > > properties
> > > > > > are actually
> > > > > > > > > > > > > > hintable. And combining with
> > > > > > > > > > > > > > #1 error handling, we can throw an exception once
> > > > > received
> > > > > > invalid
> > > > > > > > > > > > > > property.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > #3 Regarding to global offset: I'm not sure it's
> > > > > feasible.
> > > > > > Different
> > > > > > > > > > > > > > connectors will have totally
> > > > > > > > > > > > > > different properties to represent offset, some might
> > > be
> > > > > > timestamps,
> > > > > > > > > > > > some
> > > > > > > > > > > > > > might be string literals
> > > > > > > > > > > > > > like "earliest", and others might be just integers.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Kurt
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
> > > > > imjark@gmail.com>
> > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 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 <
> > > > > > twalthr@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 <
> > > > > > twalthr@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.lwl@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.cyz@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 <
> > > > > > twalthr@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
> > > > > <
> > > > > > imjark@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 <
> > > > > > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@apache.org>.
Hi Danny,

thanks for updating the FLIP. I think your current design is sufficient 
to separate hints from result-related properties.

One remark to the naming itself: I would vote for calling the hints 
around table scan `OPTIONS('k'='v')`. We used the term "properties" in 
the past but since we want to unify the Flink configuration experience, 
we should use consistent naming and classes around `ConfigOptions`.

It would be nice to use `Set<ConfigOption> supportedHintOptions();` to 
start using config options instead of pure string properties. This will 
also allow us to generate documentation in the future around supported 
data types, ranges, etc. for options. At some point we would also like 
to drop `DescriptorProperties` class. "Options" is also used in the 
documentation [1] and in the SQL/MED standard [2].

Furthermore, I would still vote for separating CatalogTable and hint 
options. Otherwise the planner would need to create a new CatalogTable 
instance which might not always be easy. We should offer them via:

org.apache.flink.table.factories.TableSourceFactory.Context#getHints: 
ReadableConfig

What do you think?

Regards,
Timo

[1] 
https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table
[2] https://wiki.postgresql.org/wiki/SQL/MED


On 12.03.20 15:06, Stephan Ewen wrote:
> @Danny sounds good.
> 
> Maybe it is worth listing all the classes of problems that you want to
> address and then look at each class and see if hints are a good default
> solution or a good optional way of simplifying things?
> The discussion has grown a lot and it is starting to be hard to distinguish
> the parts where everyone agrees from the parts were there are concerns.
> 
> On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <da...@apache.org> wrote:
> 
>> Thanks Stephan ~
>>
>> We can remove the support for properties that may change the semantics of
>> query if you think that is a trouble.
>>
>> How about we support the /*+ properties() */ hint only for those optimize
>> parameters, such as the fetch size of source or something like that, does
>> that make sense?
>>
>> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
>>
>>> I think Bowen has actually put it very well.
>>>
>>> (1) Hints that change semantics looks like trouble waiting to happen. For
>>> example Kafka offset handling should be in filters. The Kafka source
>> should
>>> support predicate pushdown.
>>>
>>> (2) Hints should not be a workaround for current shortcomings. A lot of
>> the
>>> suggested above sounds exactly like that. Working around catalog/DDL
>>> shortcomings, missing exposure of metadata (offsets), missing predicate
>>> pushdown in Kafka. Abusing a feature like hints now as a quick fix for
>>> these issues, rather than fixing the root causes, will much likely bite
>> us
>>> back badly in the future.
>>>
>>> Best,
>>> Stephan
>>>
>>>
>>> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com> wrote:
>>>
>>>> It seems this FLIP's name is somewhat misleading. From my
>> understanding,
>>>> this FLIP is trying to
>>>> address the dynamic parameter issue, and table hints is the way we wan
>> to
>>>> choose. I think we should
>>>> be focus on "what's the right way to solve dynamic property" instead of
>>>> discussing "whether table
>>>> hints can affect query semantics".
>>>>
>>>> For now, there are two proposed ways to achieve dynamic property:
>>>> 1. FLIP-110: create temporary table xx like xx with (xxx)
>>>> 2. use custom "from t with (xxx)" syntax
>>>> 3. "Borrow" the table hints to have a special PROPERTIES hint.
>>>>
>>>> The first one didn't break anything, but the only problem i see is a
>>> little
>>>> more verbose than the table hint
>>>> approach. I can imagine when someone using SQL CLI to have a sql
>>>> experience, it's quite often that
>>>> he will modify the table property, some use cases i can think of:
>>>> 1. the source contains some corrupted data, i want to turn on the
>>>> "ignore-error" flag for certain formats.
>>>> 2. I have a kafka table and want to see some sample data from the
>>>> beginning, so i change the offset
>>>> to "earliest", and then I want to observe the latest data which keeps
>>>> coming in. I would write another query
>>>> to select from the latest table.
>>>> 3. I want to my jdbc sink flush data more eagerly then i can observe
>> the
>>>> data from database side.
>>>>
>>>> Most of such use cases are quite ad-hoc. If every time I want to have a
>>>> different experience, i need to create
>>>> a temporary table and then also modify my query, it doesn't feel
>> smooth.
>>>> Embed such dynamic property into
>>>> query would have better user experience.
>>>>
>>>> Both 2 & 3 can make this happen. The cons of #2 is breaking SQL
>>> compliant,
>>>> and for #3, it only breaks some
>>>> unwritten rules, but we can have an explanation on that. And I really
>>> doubt
>>>> whether user would complain about
>>>> this when they actually have flexible and good experience using this.
>>>>
>>>> My tendency would be #3 > #1 > #2, what do you think?
>>>>
>>>> Best,
>>>> Kurt
>>>>
>>>>
>>>> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <yu...@gmail.com>
>> wrote:
>>>>
>>>>> Thanks Aljoscha ~
>>>>>
>>>>> I agree for most of the query hints, they are optional as an
>> optimizer
>>>>> instruction, especially for the traditional RDBMS.
>>>>>
>>>>> But, just like BenChao said, Flink as a computation engine has many
>>>>> different kind of data sources, thus, dynamic parameters like
>>>> start_offest
>>>>> can only bind to each table scope, we can not set a session config
>> like
>>>>> KSQL because they are all about Kafka:
>>>>>> SET ‘auto.offset.reset’=‘earliest’;
>>>>>
>>>>> Thus the most flexible way to set up these dynamic params is to bind
>> to
>>>>> the table scope in the query when we want to override something, so
>> we
>>>> have
>>>>> these solutions above (with pros and cons from my side):
>>>>>
>>>>> • 1. Select * from t(offset=123) (from Timo)
>>>>>
>>>>>             Pros:
>>>>>               - Easy to add
>>>>>               - Parameters are part of the main query
>>>>>             Cons:
>>>>>               - Not SQL compliant
>>>>>
>>>>>
>>>>> • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from me)
>>>>>
>>>>>             Pros:
>>>>>             - Easy to add
>>>>>             - SQL compliant because it is nested in the comments
>>>>>
>>>>>             Cons:
>>>>>             - Parameters are not part of the main query
>>>>>             - Cryptic syntax for new users
>>>>>
>>>>> The biggest problem for hints way may be the “if hints must be
>>> optional”,
>>>>> actually we have though about 1 for a while but aborted because it
>>> breaks
>>>>> the SQL standard too much. And we replace it with 2, because the
>> hints
>>>>> syntax do not break SQL standard(nested in comments).
>>>>>
>>>>> What if we have the special /*+ PROPERTIES */ hint that allows
>> override
>>>>> some properties of table dynamically, it does not break anything, at
>>>> lease
>>>>> for current Flink use cases.
>>>>>
>>>>> Planner hints are optional just because they are naturally enforcers
>> of
>>>>> the planner, most of them aim to instruct the optimizer, but, the
>> table
>>>>> hints is a little different, table hints can specify the table meta
>>> like
>>>>> index column, and it is very convenient to specify table properties.
>>>>>
>>>>> Or shall we not call  /*+ PROPERTIES(offset=123) */ table hint, we
>> can
>>>>> call it table dynamic parameters.
>>>>>
>>>>> Best,
>>>>> Danny Chan
>>>>> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <al...@apache.org>,写道:
>>>>>> Hi,
>>>>>>
>>>>>> I don't understand this discussion. Hints, as I understand them,
>>> should
>>>>>> work like this:
>>>>>>
>>>>>> - hints are *optional* advice for the optimizer to try and help it
>> to
>>>>>> find a good execution strategy
>>>>>> - hints should not change query semantics, i.e. they should not
>>> change
>>>>>> connector properties executing a query with taking into account the
>>>>>> hints *must* produce the same result as executing the query without
>>>>>> taking into account the hints
>>>>>>
>>>>>>  From these simple requirements you can derive a solution that makes
>>>>>> sense. I don't have a strong preference for the syntax but we
>> should
>>>>>> strive to be in line with prior work.
>>>>>>
>>>>>> Best,
>>>>>> Aljoscha
>>>>>>
>>>>>> On 11.03.20 11:53, Danny Chan wrote:
>>>>>>> Thanks Timo for summarize the 3 options ~
>>>>>>>
>>>>>>> I agree with Kurt that option2 is too complicated to use because:
>>>>>>>
>>>>>>> • As a Kafka topic consumer, the user must define both the
>> virtual
>>>>> column for start offset and he must apply a special filter predicate
>>>> after
>>>>> each query
>>>>>>> • And for the internal implementation, the metadata column push
>>> down
>>>>> is another hard topic, each kind of message queue may have its offset
>>>>> attribute, we need to consider the expression type for different
>> kind;
>>>> the
>>>>> source also need to recognize the constant column as a config
>>>> option(which
>>>>> is weird because usually what we pushed down is a table column)
>>>>>>>
>>>>>>> For option 1 and option3, I think there is no difference, option1
>>> is
>>>>> also a hint syntax which is introduced in Sybase and referenced then
>>>>> deprecated by MS-SQL in 199X years because of the ambitiousness.
>>>> Personally
>>>>> I prefer /*+ */ style table hint than WITH keyword for these reasons:
>>>>>>>
>>>>>>> • We do not break the standard SQL, the hints are nested in SQL
>>>>> comments
>>>>>>> • We do not need to introduce additional WITH keyword which may
>>>> appear
>>>>> in a query if we use that because a table can be referenced in all
>>> kinds
>>>> of
>>>>> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql
>> query
>>>>> break too much of the SQL from standard
>>>>>>> • We would have uniform syntax for hints as query hint, one
>> syntax
>>>>> fits all and more easy to use
>>>>>>>
>>>>>>>
>>>>>>> And here is the reason why we choose a uniform Oracle style query
>>>>> hint syntax which is addressed by Julian Hyde when we design the
>> syntax
>>>>> from the Calcite community:
>>>>>>>
>>>>>>> I don’t much like the MSSQL-style syntax for table hints. It
>> adds a
>>>>> new use of the WITH keyword that is unrelated to the use of WITH for
>>>>> common-table expressions.
>>>>>>>
>>>>>>> A historical note. Microsoft SQL Server inherited its hint syntax
>>>> from
>>>>> Sybase a very long time ago. (See “Transact SQL Programming”[1], page
>>>> 632,
>>>>> “Optimizer hints”. The book was written in 1999, and covers Microsoft
>>> SQL
>>>>> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very
>>>>> likely predates Sybase 4.3, from which Microsoft SQL Server was
>> forked
>>> in
>>>>> 1993.)
>>>>>>>
>>>>>>> Microsoft later added the WITH keyword to make it less ambiguous,
>>> and
>>>>> has now deprecated the syntax that does not use WITH.
>>>>>>>
>>>>>>> They are forced to keep the syntax for backwards compatibility
>> but
>>>>> that doesn’t mean that we should shoulder their burden.
>>>>>>>
>>>>>>> I think formatted comments are the right container for hints
>>> because
>>>>> it allows us to change the hint syntax without changing the SQL
>> parser,
>>>> and
>>>>> makes clear that we are at liberty to ignore hints entirely.
>>>>>>>
>>>>>>> Julian
>>>>>>>
>>>>>>> [1] https://www.amazon.com/s?k=9781565924017 <
>>>>> https://www.amazon.com/s?k=9781565924017>
>>>>>>>
>>>>>>> Best,
>>>>>>> Danny Chan
>>>>>>> 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
>>>>>>>> Hi Danny,
>>>>>>>>
>>>>>>>> it is true that our DDL is not standard compliant by using the
>>> WITH
>>>>>>>> clause. Nevertheless, we aim for not diverging too much and the
>>>> LIKE
>>>>>>>> clause is an example of that. It will solve things like
>>> overwriting
>>>>>>>> WATERMARKs, add additional/modifying properties and inherit
>>> schema.
>>>>>>>>
>>>>>>>> Bowen is right that Flink's DDL is mixing 3 types definition
>>>>> together.
>>>>>>>> We are not the first ones that try to solve this. There is also
>>> the
>>>>> SQL
>>>>>>>> MED standard [1] that tried to tackle this problem. I think it
>>> was
>>>>> not
>>>>>>>> considered when designing the current DDL.
>>>>>>>>
>>>>>>>> Currently, I see 3 options for handling Kafka offsets. I will
>>> give
>>>>> some
>>>>>>>> examples and look forward to feedback here:
>>>>>>>>
>>>>>>>> *Option 1* Runtime and semantic parms as part of the query
>>>>>>>>
>>>>>>>> `SELECT * FROM MyTable('offset'=123)`
>>>>>>>>
>>>>>>>> Pros:
>>>>>>>> - Easy to add
>>>>>>>> - Parameters are part of the main query
>>>>>>>> - No complicated hinting syntax
>>>>>>>>
>>>>>>>> Cons:
>>>>>>>> - Not SQL compliant
>>>>>>>>
>>>>>>>> *Option 2* Use metadata in query
>>>>>>>>
>>>>>>>> `CREATE TABLE MyTable (id INT, offset AS
>>>> SYSTEM_METADATA('offset'))`
>>>>>>>>
>>>>>>>> `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12
>>>>> 12:34:22'`
>>>>>>>>
>>>>>>>> Pros:
>>>>>>>> - SQL compliant in the query
>>>>>>>> - Access of metadata in the DDL which is required anyway
>>>>>>>> - Regular pushdown rules apply
>>>>>>>>
>>>>>>>> Cons:
>>>>>>>> - Users need to add an additional comlumn in the DDL
>>>>>>>>
>>>>>>>> *Option 3*: Use hints for properties
>>>>>>>>
>>>>>>>> `
>>>>>>>> SELECT *
>>>>>>>> FROM MyTable /*+ PROPERTIES('offset'=123) */
>>>>>>>> `
>>>>>>>>
>>>>>>>> Pros:
>>>>>>>> - Easy to add
>>>>>>>>
>>>>>>>> Cons:
>>>>>>>> - Parameters are not part of the main query
>>>>>>>> - Cryptic syntax for new users
>>>>>>>> - Not standard compliant.
>>>>>>>>
>>>>>>>> If we go with this option, I would suggest to make it available
>>> in
>>>> a
>>>>>>>> separate map and don't mix it with statically defined
>> properties.
>>>>> Such
>>>>>>>> that the factory can decide which properties have the right to
>> be
>>>>>>>> overwritten by the hints:
>>>>>>>> TableSourceFactory.Context.getQueryHints(): ReadableConfig
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>> Timo
>>>>>>>>
>>>>>>>> [1] https://en.wikipedia.org/wiki/SQL/MED
>>>>>>>>
>>>>>>>> Currently I see 3 options as a
>>>>>>>>
>>>>>>>>
>>>>>>>> On 11.03.20 07:21, Danny Chan wrote:
>>>>>>>>> Thanks Bowen ~
>>>>>>>>>
>>>>>>>>> I agree we should somehow categorize our connector
>> parameters.
>>>>>>>>>
>>>>>>>>> For type1, I’m already preparing a solution like the
>> Confluent
>>>>> schema registry + Avro schema inference thing, so this may not be a
>>>> problem
>>>>> in the near future.
>>>>>>>>>
>>>>>>>>> For type3, I have some questions:
>>>>>>>>>
>>>>>>>>>> "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
>>>>>>>>>
>>>>>>>>> Where does the offset column come from, a virtual column from
>>> the
>>>>> table schema, you said that
>>>>>>>>>
>>>>>>>>>> They change
>>>>>>>>> almost every time a query starts and have nothing to do with
>>>>> metadata, thus
>>>>>>>>> should not be part of table definition/DDL
>>>>>>>>>
>>>>>>>>> But why you can reference it in the query, I’m confused for
>>> that,
>>>>> can you elaborate a little ?
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Danny Chan
>>>>>>>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <bowenli86@gmail.com
>>> ,写道:
>>>>>>>>>> Thanks Danny for kicking off the effort
>>>>>>>>>>
>>>>>>>>>> The root cause of too much manual work is Flink DDL has
>>> mixed 3
>>>>> types of
>>>>>>>>>> params together and doesn't handle each of them very well.
>>>> Below
>>>>> are how I
>>>>>>>>>> categorize them and corresponding solutions in my mind:
>>>>>>>>>>
>>>>>>>>>> - type 1: Metadata of external data, like external
>>>> endpoint/url,
>>>>>>>>>> username/pwd, schemas, formats.
>>>>>>>>>>
>>>>>>>>>> Such metadata are mostly already accessible in external
>>> system
>>>>> as long as
>>>>>>>>>> endpoints and credentials are provided. Flink can get it
>> thru
>>>>> catalogs, but
>>>>>>>>>> we haven't had many catalogs yet and thus Flink just hasn't
>>>> been
>>>>> able to
>>>>>>>>>> leverage that. So the solution should be building more
>>>> catalogs.
>>>>> Such
>>>>>>>>>> params should be part of a Flink table DDL/definition, and
>>> not
>>>>> overridable
>>>>>>>>>> in any means.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> - type 2: Runtime params, like jdbc connector's fetch size,
>>>>> elasticsearch
>>>>>>>>>> connector's bulk flush size.
>>>>>>>>>>
>>>>>>>>>> Such params don't affect query results, but affect how
>>> results
>>>>> are produced
>>>>>>>>>> (eg. fast or slow, aka performance) - they are essentially
>>>>> execution and
>>>>>>>>>> implementation details. They change often in exploration or
>>>>> development
>>>>>>>>>> stages, but not quite frequently in well-defined
>> long-running
>>>>> pipelines.
>>>>>>>>>> They should always have default values and can be missing
>> in
>>>>> query. They
>>>>>>>>>> can be part of a table DDL/definition, but should also be
>>>>> replaceable in a
>>>>>>>>>> query - *this is what table "hints" in FLIP-113 should
>>> cover*.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> - type 3: Semantic params, like kafka connector's start
>>> offset.
>>>>>>>>>>
>>>>>>>>>> Such params affect query results - the semantics. They'd
>>> better
>>>>> be as
>>>>>>>>>> filter conditions in WHERE clause that can be pushed down.
>>> They
>>>>> change
>>>>>>>>>> almost every time a query starts and have nothing to do
>> with
>>>>> metadata, thus
>>>>>>>>>> should not be part of table definition/DDL, nor be
>> persisted
>>> in
>>>>> catalogs.
>>>>>>>>>> If they will, users should create views to keep such params
>>>>> around (note
>>>>>>>>>> this is different from variable substitution).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Take Flink-Kafka as an example. Once we get these params
>>> right,
>>>>> here're the
>>>>>>>>>> steps users need to do to develop and run a Flink job:
>>>>>>>>>> - configure a Flink ConfluentSchemaRegistry with url,
>>> username,
>>>>> and password
>>>>>>>>>> - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday"
>>>>> (simplified
>>>>>>>>>> timestamp) in SQL CLI, Flink automatically retrieves all
>>>>> metadata of
>>>>>>>>>> schema, file format, etc and start the job
>>>>>>>>>> - users want to make the job read Kafka topic faster, so it
>>>> goes
>>>>> as "SELECT
>>>>>>>>>> * FROM mykafka /* faster_read_key=value*/ WHERE offset >
>> 12pm
>>>>> yesterday"
>>>>>>>>>> - done and satisfied, users submit it to production
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think
>>>> it's
>>>>> a
>>>>>>>>>> nice-to-have feature, but not a strategically critical,
>>>>> long-term solution,
>>>>>>>>>> because
>>>>>>>>>> 1) It may seem promising at the current stage to solve the
>>>>>>>>>> too-much-manual-work problem, but that's only because Flink
>>>>> hasn't
>>>>>>>>>> leveraged catalogs well and handled the 3 types of params
>>> above
>>>>> properly.
>>>>>>>>>> Once we get the params types right, the LIKE syntax won't
>> be
>>>> that
>>>>>>>>>> important, and will be just an easier way to create tables
>>>>> without retyping
>>>>>>>>>> long fields like username and pwd.
>>>>>>>>>> 2) Note that only some rare type of catalog can store k-v
>>>>> property pair, so
>>>>>>>>>> table created this way often cannot be persisted. In the
>>>>> foreseeable
>>>>>>>>>> future, such catalog will only be HiveCatalog, and not
>>> everyone
>>>>> has a Hive
>>>>>>>>>> metastore. To be honest, without persistence, recreating
>>> tables
>>>>> every time
>>>>>>>>>> this way is still a lot of keyboard typing.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Bowen
>>>>>>>>>>
>>>>>>>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <
>> ykt836@gmail.com
>>>>
>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> If a specific connector want to have such parameter and
>>> read
>>>>> if out of
>>>>>>>>>>> configuration, then that's fine.
>>>>>>>>>>> If we are talking about a configuration for all kinds of
>>>>> sources, I would
>>>>>>>>>>> be super careful about that.
>>>>>>>>>>> It's true it can solve maybe 80% cases, but it will also
>>> make
>>>>> the left 20%
>>>>>>>>>>> feels weird.
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> Kurt
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <
>> imjark@gmail.com
>>>>
>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Kurt,
>>>>>>>>>>>>
>>>>>>>>>>>> #3 Regarding to global offset:
>>>>>>>>>>>> I'm not saying to use the global configuration to
>>> override
>>>>> connector
>>>>>>>>>>>> properties by the planner.
>>>>>>>>>>>> But the connector should take this configuration and
>>>>> translate into their
>>>>>>>>>>>> client API.
>>>>>>>>>>>> AFAIK, almost all the message queues support eariliest
>>> and
>>>>> latest and a
>>>>>>>>>>>> timestamp value as start point.
>>>>>>>>>>>> So we can support 3 options for this configuration:
>>>>> "eariliest", "latest"
>>>>>>>>>>>> and a timestamp string value.
>>>>>>>>>>>> Of course, this can't solve 100% cases, but I guess can
>>>>> sovle 80% or 90%
>>>>>>>>>>>> cases.
>>>>>>>>>>>> And the remaining cases can be resolved by LIKE syntax
>>>> which
>>>>> I guess is
>>>>>>>>>>> not
>>>>>>>>>>>> very common cases.
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> Jark
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt Young <
>>> ykt836@gmail.com
>>>>>
>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Good to have such lovely discussions. I also want to
>>>> share
>>>>> some of my
>>>>>>>>>>>>> opinions.
>>>>>>>>>>>>>
>>>>>>>>>>>>> #1 Regarding to error handling: I also think ignore
>>>>> invalid hints would
>>>>>>>>>>>> be
>>>>>>>>>>>>> dangerous, maybe
>>>>>>>>>>>>> the simplest solution is just throw an exception.
>>>>>>>>>>>>>
>>>>>>>>>>>>> #2 Regarding to property replacement: I don't think
>> we
>>>>> should
>>>>>>>>>>> constraint
>>>>>>>>>>>>> ourself to
>>>>>>>>>>>>> the meaning of the word "hint", and forbidden it
>>>> modifying
>>>>> any
>>>>>>>>>>> properties
>>>>>>>>>>>>> which can effect
>>>>>>>>>>>>> query results. IMO `PROPERTIES` is one of the table
>>>> hints,
>>>>> and a
>>>>>>>>>>> powerful
>>>>>>>>>>>>> one. It can
>>>>>>>>>>>>> modify properties located in DDL's WITH block. But I
>>> also
>>>>> see the harm
>>>>>>>>>>>> that
>>>>>>>>>>>>> if we make it
>>>>>>>>>>>>> too flexible like change the kafka topic name with a
>>>> hint.
>>>>> Such use
>>>>>>>>>>> case
>>>>>>>>>>>> is
>>>>>>>>>>>>> not common and
>>>>>>>>>>>>> sounds very dangerous to me. I would propose we have
>> a
>>>> map
>>>>> of hintable
>>>>>>>>>>>>> properties for each
>>>>>>>>>>>>> connector, and should validate all passed in
>> properties
>>>>> are actually
>>>>>>>>>>>>> hintable. And combining with
>>>>>>>>>>>>> #1 error handling, we can throw an exception once
>>>> received
>>>>> invalid
>>>>>>>>>>>>> property.
>>>>>>>>>>>>>
>>>>>>>>>>>>> #3 Regarding to global offset: I'm not sure it's
>>>> feasible.
>>>>> Different
>>>>>>>>>>>>> connectors will have totally
>>>>>>>>>>>>> different properties to represent offset, some might
>> be
>>>>> timestamps,
>>>>>>>>>>> some
>>>>>>>>>>>>> might be string literals
>>>>>>>>>>>>> like "earliest", and others might be just integers.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> Kurt
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
>>>> imjark@gmail.com>
>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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 <
>>>>> twalthr@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 <
>>>>> twalthr@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.lwl@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.cyz@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 <
>>>>> twalthr@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
>>>> <
>>>>> imjark@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 <
>>>>>>>>>>>>>> kyledong@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.cyz@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.cyz@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
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Kurt Young <yk...@gmail.com>.
Hi Timo,

option 1 & option 3 are almost the same in my opinion. Even though we call
it table hints, the biggest
motivation for now is to modify table properties. I also see other vendor
using syntax like option 1 to
implement table hints, e.g. sql server [1]. It uses syntax like SELECT *
from T WITH('key' = 'value') which
i personally in favor of. It's also align with our DDL syntax.

Some thoughts about option 2: form implementation perspective, this
approach is not very easy to
design and implement. It's more complicate than normal fields pushdown like
projection and filter.
Such `SYSTEM_METADATA('offset')` metadata has special meaning, kafka table
source should
recognize `offset` is a special metadata, and when such metadata pushed
down, it will modify the
fetcher's offset. And we also have to invent some special value for such
metadata, like how to
represent "earliest"?

Specifying offset is a necessary step for all users who trying to use kafka
and option 2 seems to be
too complicated to me.

[1]
https://docs.microsoft.com/en-us/sql/t-sql/queries/hints-transact-sql-table?view=sql-server-ver15

Best,
Kurt


On Wed, Mar 11, 2020 at 4:03 PM Timo Walther <tw...@apache.org> wrote:

> Hi Danny,
>
> it is true that our DDL is not standard compliant by using the WITH
> clause. Nevertheless, we aim for not diverging too much and the LIKE
> clause is an example of that. It will solve things like overwriting
> WATERMARKs, add additional/modifying properties and inherit schema.
>
> Bowen is right that Flink's DDL is mixing 3 types definition together.
> We are not the first ones that try to solve this. There is also the SQL
> MED standard [1] that tried to tackle this problem. I think it was not
> considered when designing the current DDL.
>
> Currently, I see 3 options for handling Kafka offsets. I will give some
> examples and look forward to feedback here:
>
> *Option 1* Runtime and semantic parms as part of the query
>
> `SELECT * FROM MyTable('offset'=123)`
>
> Pros:
> - Easy to add
> - Parameters are part of the main query
> - No complicated hinting syntax
>
> Cons:
> - Not SQL compliant
>
> *Option 2* Use metadata in query
>
> `CREATE TABLE MyTable (id INT, offset AS SYSTEM_METADATA('offset'))`
>
> `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12 12:34:22'`
>
> Pros:
> - SQL compliant in the query
> - Access of metadata in the DDL which is required anyway
> - Regular pushdown rules apply
>
> Cons:
> - Users need to add an additional comlumn in the DDL
>
> *Option 3*: Use hints for properties
>
> `
> SELECT *
> FROM MyTable /*+ PROPERTIES('offset'=123) */
> `
>
> Pros:
> - Easy to add
>
> Cons:
> - Parameters are not part of the main query
> - Cryptic syntax for new users
> - Not standard compliant.
>
> If we go with this option, I would suggest to make it available in a
> separate map and don't mix it with statically defined properties. Such
> that the factory can decide which properties have the right to be
> overwritten by the hints:
> TableSourceFactory.Context.getQueryHints(): ReadableConfig
>
> Regards,
> Timo
>
> [1] https://en.wikipedia.org/wiki/SQL/MED
>
> Currently I see 3 options as a
>
>
> On 11.03.20 07:21, Danny Chan wrote:
> > Thanks Bowen ~
> >
> > I agree we should somehow categorize our connector parameters.
> >
> > For type1, I’m already preparing a solution like the Confluent schema
> registry + Avro schema inference thing, so this may not be a problem in the
> near future.
> >
> > For type3, I have some questions:
> >
> >> "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> >
> > Where does the offset column come from, a virtual column from the table
> schema, you said that
> >
> >> They change
> > almost every time a query starts and have nothing to do with metadata,
> thus
> > should not be part of table definition/DDL
> >
> > But why you can reference it in the query, I’m confused for that, can
> you elaborate a little ?
> >
> > Best,
> > Danny Chan
> > 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
> >> Thanks Danny for kicking off the effort
> >>
> >> The root cause of too much manual work is Flink DDL has mixed 3 types of
> >> params together and doesn't handle each of them very well. Below are
> how I
> >> categorize them and corresponding solutions in my mind:
> >>
> >> - type 1: Metadata of external data, like external endpoint/url,
> >> username/pwd, schemas, formats.
> >>
> >> Such metadata are mostly already accessible in external system as long
> as
> >> endpoints and credentials are provided. Flink can get it thru catalogs,
> but
> >> we haven't had many catalogs yet and thus Flink just hasn't been able to
> >> leverage that. So the solution should be building more catalogs. Such
> >> params should be part of a Flink table DDL/definition, and not
> overridable
> >> in any means.
> >>
> >>
> >> - type 2: Runtime params, like jdbc connector's fetch size,
> elasticsearch
> >> connector's bulk flush size.
> >>
> >> Such params don't affect query results, but affect how results are
> produced
> >> (eg. fast or slow, aka performance) - they are essentially execution and
> >> implementation details. They change often in exploration or development
> >> stages, but not quite frequently in well-defined long-running pipelines.
> >> They should always have default values and can be missing in query. They
> >> can be part of a table DDL/definition, but should also be replaceable
> in a
> >> query - *this is what table "hints" in FLIP-113 should cover*.
> >>
> >>
> >> - type 3: Semantic params, like kafka connector's start offset.
> >>
> >> Such params affect query results - the semantics. They'd better be as
> >> filter conditions in WHERE clause that can be pushed down. They change
> >> almost every time a query starts and have nothing to do with metadata,
> thus
> >> should not be part of table definition/DDL, nor be persisted in
> catalogs.
> >> If they will, users should create views to keep such params around (note
> >> this is different from variable substitution).
> >>
> >>
> >> Take Flink-Kafka as an example. Once we get these params right, here're
> the
> >> steps users need to do to develop and run a Flink job:
> >> - configure a Flink ConfluentSchemaRegistry with url, username, and
> password
> >> - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday" (simplified
> >> timestamp) in SQL CLI, Flink automatically retrieves all metadata of
> >> schema, file format, etc and start the job
> >> - users want to make the job read Kafka topic faster, so it goes as
> "SELECT
> >> * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm yesterday"
> >> - done and satisfied, users submit it to production
> >>
> >>
> >> Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's a
> >> nice-to-have feature, but not a strategically critical, long-term
> solution,
> >> because
> >> 1) It may seem promising at the current stage to solve the
> >> too-much-manual-work problem, but that's only because Flink hasn't
> >> leveraged catalogs well and handled the 3 types of params above
> properly.
> >> Once we get the params types right, the LIKE syntax won't be that
> >> important, and will be just an easier way to create tables without
> retyping
> >> long fields like username and pwd.
> >> 2) Note that only some rare type of catalog can store k-v property
> pair, so
> >> table created this way often cannot be persisted. In the foreseeable
> >> future, such catalog will only be HiveCatalog, and not everyone has a
> Hive
> >> metastore. To be honest, without persistence, recreating tables every
> time
> >> this way is still a lot of keyboard typing.
> >>
> >> Cheers,
> >> Bowen
> >>
> >> On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com> wrote:
> >>
> >>> If a specific connector want to have such parameter and read if out of
> >>> configuration, then that's fine.
> >>> If we are talking about a configuration for all kinds of sources, I
> would
> >>> be super careful about that.
> >>> It's true it can solve maybe 80% cases, but it will also make the left
> 20%
> >>> feels weird.
> >>>
> >>> Best,
> >>> Kurt
> >>>
> >>>
> >>> On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:
> >>>
> >>>> Hi Kurt,
> >>>>
> >>>> #3 Regarding to global offset:
> >>>> I'm not saying to use the global configuration to override connector
> >>>> properties by the planner.
> >>>> But the connector should take this configuration and translate into
> their
> >>>> client API.
> >>>> AFAIK, almost all the message queues support eariliest and latest and
> a
> >>>> timestamp value as start point.
> >>>> So we can support 3 options for this configuration: "eariliest",
> "latest"
> >>>> and a timestamp string value.
> >>>> Of course, this can't solve 100% cases, but I guess can sovle 80% or
> 90%
> >>>> cases.
> >>>> And the remaining cases can be resolved by LIKE syntax which I guess
> is
> >>> not
> >>>> very common cases.
> >>>>
> >>>> Best,
> >>>> Jark
> >>>>
> >>>>
> >>>> On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:
> >>>>
> >>>>> Good to have such lovely discussions. I also want to share some of my
> >>>>> opinions.
> >>>>>
> >>>>> #1 Regarding to error handling: I also think ignore invalid hints
> would
> >>>> be
> >>>>> dangerous, maybe
> >>>>> the simplest solution is just throw an exception.
> >>>>>
> >>>>> #2 Regarding to property replacement: I don't think we should
> >>> constraint
> >>>>> ourself to
> >>>>> the meaning of the word "hint", and forbidden it modifying any
> >>> properties
> >>>>> which can effect
> >>>>> query results. IMO `PROPERTIES` is one of the table hints, and a
> >>> powerful
> >>>>> one. It can
> >>>>> modify properties located in DDL's WITH block. But I also see the
> harm
> >>>> that
> >>>>> if we make it
> >>>>> too flexible like change the kafka topic name with a hint. Such use
> >>> case
> >>>> is
> >>>>> not common and
> >>>>> sounds very dangerous to me. I would propose we have a map of
> hintable
> >>>>> properties for each
> >>>>> connector, and should validate all passed in properties are actually
> >>>>> hintable. And combining with
> >>>>> #1 error handling, we can throw an exception once received invalid
> >>>>> property.
> >>>>>
> >>>>> #3 Regarding to global offset: I'm not sure it's feasible. Different
> >>>>> connectors will have totally
> >>>>> different properties to represent offset, some might be timestamps,
> >>> some
> >>>>> might be string literals
> >>>>> like "earliest", and others might be just integers.
> >>>>>
> >>>>> Best,
> >>>>> Kurt
> >>>>>
> >>>>>
> >>>>> On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
> >>>>>
> >>>>>> 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 <tw...@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 <tw...@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.lwl@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.cyz@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 <twalthr@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 <im...@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 <
> >>>>>> kyledong@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.cyz@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.cyz@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
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >
>
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Benchao Li <li...@gmail.com>.
Hi all,

Thanks Danny for bring up this great discussion, generally hints is a great
feature for SQL.
And the discussions are very insightful, I'd like to share some ideas here.

About error handling,
+1 to throw exception by default. IMHO, hints are parts of the query, which
should be validated before
execution. If we just log errors, maybe it's hard for users to debug their
job when the behavior of the
job differs from what they expects. An early validation and error throwing
can enforce users to make sure
their query is valid for both SQL and hints.

About whether hints can affect results,
Before the disscution, I thought that hints maybe useful to do many things
we cannot do with SQL standard,
e.g.
- Fast Emit for WindowAggregate
- State retention time for general group by and join
- allow lateness for WindowAggregate
- keyby before temporal join
- mini batch config
- local-global aggregate
Most of these things are configured globally, which is very inconvenient to
use in complex scenario.
Some of these can affect results, and IMHO it's ok to do so, because
Steaming SQL is a relative new
concept which differs in many aspects from Batch SQL.

About whether allow to add connector properties in hints,
IMHO, it's good to keep the grammar concise. If we can do that already with
connector properties, and
LIKE feature recently proposed, maybe we don't need to add another way to
do that. A clean and concise
grammar can benefit users a lot especially for users like inf team in each
compony. They maybe face
a lot internal users, and the variety of ways to do the same thing can
bring much burden to them.

About global `set xxx=xxx`,
it's a good idea to have this ability. IMHO, it's useful for some global
job configurations,
e.g.
- global parallelism
- checkpoint config
- auto watermark interval
- time zone
- serializers
However, I don't think it's a good idea to have some connector configs like
'auto.offset.reset'.
KSQL does that because it only deals with Kafka, and 'auto.offset.reset' is
indeed a global config
for that scenario. Flink is a general streaming (and batch) engine, which
IMHO we cannot assume all
connectors have some common configs like that. It's good to keep connectors
config in each table source
definition, even we need to set it multiple times.

Bowen Li <bo...@gmail.com> 于2020年3月12日周四 上午1:12写道:

> A quick summary that focus of the discussion now shifts to be whether
> semantic params like kafka 'starting offset' should be table
> hints/properties, and if so, in what form.
>
> I strongly believe the action of setting offset should *not* be part of a
> table, neither hints nor properties, for all the good reasons mentioned
> above. It doesn't describe the physical table/storage. It's indeed a filter
> describing what kind of data users want to query on top of all present,
> physical data.
>
> One way to think of that is 'offset' is a system column of Kafka, and Flink
> queries can filter on this system column. E.g. Postgres has a column 'xmin'
> to track commit timestamp automatically and users can filter on it [1].
> Flink connectors should be able to define it's own system columns,
> recognize them from filters pushed down by Flink SQL framework, and apply
> accordingly.
>
> [1]
>
> https://wiki.postgresql.org/wiki/What's_new_in_PostgreSQL_9.5#Commit_timestamp_tracking
>
>
>
> On Wed, Mar 11, 2020 at 6:20 AM Aljoscha Krettek <al...@apache.org>
> wrote:
>
> > Hi,
> >
> > I don't understand this discussion. Hints, as I understand them, should
> > work like this:
> >
> > - hints are *optional* advice for the optimizer to try and help it to
> > find a good execution strategy
> > - hints should not change query semantics, i.e. they should not change
> > connector properties executing a query with taking into account the
> > hints *must* produce the same result as executing the query without
> > taking into account the hints
> >
> >  From these simple requirements you can derive a solution that makes
> > sense. I don't have a strong preference for the syntax but we should
> > strive to be in line with prior work.
> >
> > Best,
> > Aljoscha
> >
> > On 11.03.20 11:53, Danny Chan wrote:
> > > Thanks Timo for summarize the 3 options ~
> > >
> > > I agree with Kurt that option2 is too complicated to use because:
> > >
> > > • As a Kafka topic consumer, the user must define both the virtual
> > column for start offset and he must apply a special filter predicate
> after
> > each query
> > > • And for the internal implementation, the metadata column push down is
> > another hard topic, each kind of message queue may have its offset
> > attribute, we need to consider the expression type for different kind;
> the
> > source also need to recognize the constant column as a config
> option(which
> > is weird because usually what we pushed down is a table column)
> > >
> > > For option 1 and option3, I think there is no difference, option1 is
> > also a hint syntax which is introduced in Sybase and referenced then
> > deprecated by MS-SQL in 199X years because of the ambitiousness.
> Personally
> > I prefer /*+ */ style table hint than WITH keyword for these reasons:
> > >
> > > • We do not break the standard SQL, the hints are nested in SQL
> comments
> > > • We do not need to introduce additional WITH keyword which may appear
> > in a query if we use that because a table can be referenced in all kinds
> of
> > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql query
> > break too much of the SQL from standard
> > > • We would have uniform syntax for hints as query hint, one syntax fits
> > all and more easy to use
> > >
> > >
> > > And here is the reason why we choose a uniform Oracle style query
> > hint syntax which is addressed by Julian Hyde when we design the syntax
> > from the Calcite community:
> > >
> > > I don’t much like the MSSQL-style syntax for table hints. It adds a new
> > use of the WITH keyword that is unrelated to the use of WITH for
> > common-table expressions.
> > >
> > > A historical note. Microsoft SQL Server inherited its hint syntax from
> > Sybase a very long time ago. (See “Transact SQL Programming”[1], page
> 632,
> > “Optimizer hints”. The book was written in 1999, and covers Microsoft SQL
> > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very
> > likely predates Sybase 4.3, from which Microsoft SQL Server was forked in
> > 1993.)
> > >
> > > Microsoft later added the WITH keyword to make it less ambiguous, and
> > has now deprecated the syntax that does not use WITH.
> > >
> > > They are forced to keep the syntax for backwards compatibility but that
> > doesn’t mean that we should shoulder their burden.
> > >
> > > I think formatted comments are the right container for hints because it
> > allows us to change the hint syntax without changing the SQL parser, and
> > makes clear that we are at liberty to ignore hints entirely.
> > >
> > > Julian
> > >
> > > [1] https://www.amazon.com/s?k=9781565924017 <
> > https://www.amazon.com/s?k=9781565924017>
> > >
> > > Best,
> > > Danny Chan
> > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
> > >> Hi Danny,
> > >>
> > >> it is true that our DDL is not standard compliant by using the WITH
> > >> clause. Nevertheless, we aim for not diverging too much and the LIKE
> > >> clause is an example of that. It will solve things like overwriting
> > >> WATERMARKs, add additional/modifying properties and inherit schema.
> > >>
> > >> Bowen is right that Flink's DDL is mixing 3 types definition together.
> > >> We are not the first ones that try to solve this. There is also the
> SQL
> > >> MED standard [1] that tried to tackle this problem. I think it was not
> > >> considered when designing the current DDL.
> > >>
> > >> Currently, I see 3 options for handling Kafka offsets. I will give
> some
> > >> examples and look forward to feedback here:
> > >>
> > >> *Option 1* Runtime and semantic parms as part of the query
> > >>
> > >> `SELECT * FROM MyTable('offset'=123)`
> > >>
> > >> Pros:
> > >> - Easy to add
> > >> - Parameters are part of the main query
> > >> - No complicated hinting syntax
> > >>
> > >> Cons:
> > >> - Not SQL compliant
> > >>
> > >> *Option 2* Use metadata in query
> > >>
> > >> `CREATE TABLE MyTable (id INT, offset AS SYSTEM_METADATA('offset'))`
> > >>
> > >> `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12 12:34:22'`
> > >>
> > >> Pros:
> > >> - SQL compliant in the query
> > >> - Access of metadata in the DDL which is required anyway
> > >> - Regular pushdown rules apply
> > >>
> > >> Cons:
> > >> - Users need to add an additional comlumn in the DDL
> > >>
> > >> *Option 3*: Use hints for properties
> > >>
> > >> `
> > >> SELECT *
> > >> FROM MyTable /*+ PROPERTIES('offset'=123) */
> > >> `
> > >>
> > >> Pros:
> > >> - Easy to add
> > >>
> > >> Cons:
> > >> - Parameters are not part of the main query
> > >> - Cryptic syntax for new users
> > >> - Not standard compliant.
> > >>
> > >> If we go with this option, I would suggest to make it available in a
> > >> separate map and don't mix it with statically defined properties. Such
> > >> that the factory can decide which properties have the right to be
> > >> overwritten by the hints:
> > >> TableSourceFactory.Context.getQueryHints(): ReadableConfig
> > >>
> > >> Regards,
> > >> Timo
> > >>
> > >> [1] https://en.wikipedia.org/wiki/SQL/MED
> > >>
> > >> Currently I see 3 options as a
> > >>
> > >>
> > >> On 11.03.20 07:21, Danny Chan wrote:
> > >>> Thanks Bowen ~
> > >>>
> > >>> I agree we should somehow categorize our connector parameters.
> > >>>
> > >>> For type1, I’m already preparing a solution like the Confluent schema
> > registry + Avro schema inference thing, so this may not be a problem in
> the
> > near future.
> > >>>
> > >>> For type3, I have some questions:
> > >>>
> > >>>> "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> > >>>
> > >>> Where does the offset column come from, a virtual column from the
> > table schema, you said that
> > >>>
> > >>>> They change
> > >>> almost every time a query starts and have nothing to do with
> metadata,
> > thus
> > >>> should not be part of table definition/DDL
> > >>>
> > >>> But why you can reference it in the query, I’m confused for that, can
> > you elaborate a little ?
> > >>>
> > >>> Best,
> > >>> Danny Chan
> > >>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
> > >>>> Thanks Danny for kicking off the effort
> > >>>>
> > >>>> The root cause of too much manual work is Flink DDL has mixed 3
> types
> > of
> > >>>> params together and doesn't handle each of them very well. Below are
> > how I
> > >>>> categorize them and corresponding solutions in my mind:
> > >>>>
> > >>>> - type 1: Metadata of external data, like external endpoint/url,
> > >>>> username/pwd, schemas, formats.
> > >>>>
> > >>>> Such metadata are mostly already accessible in external system as
> > long as
> > >>>> endpoints and credentials are provided. Flink can get it thru
> > catalogs, but
> > >>>> we haven't had many catalogs yet and thus Flink just hasn't been
> able
> > to
> > >>>> leverage that. So the solution should be building more catalogs.
> Such
> > >>>> params should be part of a Flink table DDL/definition, and not
> > overridable
> > >>>> in any means.
> > >>>>
> > >>>>
> > >>>> - type 2: Runtime params, like jdbc connector's fetch size,
> > elasticsearch
> > >>>> connector's bulk flush size.
> > >>>>
> > >>>> Such params don't affect query results, but affect how results are
> > produced
> > >>>> (eg. fast or slow, aka performance) - they are essentially execution
> > and
> > >>>> implementation details. They change often in exploration or
> > development
> > >>>> stages, but not quite frequently in well-defined long-running
> > pipelines.
> > >>>> They should always have default values and can be missing in query.
> > They
> > >>>> can be part of a table DDL/definition, but should also be
> replaceable
> > in a
> > >>>> query - *this is what table "hints" in FLIP-113 should cover*.
> > >>>>
> > >>>>
> > >>>> - type 3: Semantic params, like kafka connector's start offset.
> > >>>>
> > >>>> Such params affect query results - the semantics. They'd better be
> as
> > >>>> filter conditions in WHERE clause that can be pushed down. They
> change
> > >>>> almost every time a query starts and have nothing to do with
> > metadata, thus
> > >>>> should not be part of table definition/DDL, nor be persisted in
> > catalogs.
> > >>>> If they will, users should create views to keep such params around
> > (note
> > >>>> this is different from variable substitution).
> > >>>>
> > >>>>
> > >>>> Take Flink-Kafka as an example. Once we get these params right,
> > here're the
> > >>>> steps users need to do to develop and run a Flink job:
> > >>>> - configure a Flink ConfluentSchemaRegistry with url, username, and
> > password
> > >>>> - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday"
> > (simplified
> > >>>> timestamp) in SQL CLI, Flink automatically retrieves all metadata of
> > >>>> schema, file format, etc and start the job
> > >>>> - users want to make the job read Kafka topic faster, so it goes as
> > "SELECT
> > >>>> * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm
> > yesterday"
> > >>>> - done and satisfied, users submit it to production
> > >>>>
> > >>>>
> > >>>> Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's a
> > >>>> nice-to-have feature, but not a strategically critical, long-term
> > solution,
> > >>>> because
> > >>>> 1) It may seem promising at the current stage to solve the
> > >>>> too-much-manual-work problem, but that's only because Flink hasn't
> > >>>> leveraged catalogs well and handled the 3 types of params above
> > properly.
> > >>>> Once we get the params types right, the LIKE syntax won't be that
> > >>>> important, and will be just an easier way to create tables without
> > retyping
> > >>>> long fields like username and pwd.
> > >>>> 2) Note that only some rare type of catalog can store k-v property
> > pair, so
> > >>>> table created this way often cannot be persisted. In the foreseeable
> > >>>> future, such catalog will only be HiveCatalog, and not everyone has
> a
> > Hive
> > >>>> metastore. To be honest, without persistence, recreating tables
> every
> > time
> > >>>> this way is still a lot of keyboard typing.
> > >>>>
> > >>>> Cheers,
> > >>>> Bowen
> > >>>>
> > >>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com>
> wrote:
> > >>>>
> > >>>>> If a specific connector want to have such parameter and read if out
> > of
> > >>>>> configuration, then that's fine.
> > >>>>> If we are talking about a configuration for all kinds of sources, I
> > would
> > >>>>> be super careful about that.
> > >>>>> It's true it can solve maybe 80% cases, but it will also make the
> > left 20%
> > >>>>> feels weird.
> > >>>>>
> > >>>>> Best,
> > >>>>> Kurt
> > >>>>>
> > >>>>>
> > >>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:
> > >>>>>
> > >>>>>> Hi Kurt,
> > >>>>>>
> > >>>>>> #3 Regarding to global offset:
> > >>>>>> I'm not saying to use the global configuration to override
> connector
> > >>>>>> properties by the planner.
> > >>>>>> But the connector should take this configuration and translate
> into
> > their
> > >>>>>> client API.
> > >>>>>> AFAIK, almost all the message queues support eariliest and latest
> > and a
> > >>>>>> timestamp value as start point.
> > >>>>>> So we can support 3 options for this configuration: "eariliest",
> > "latest"
> > >>>>>> and a timestamp string value.
> > >>>>>> Of course, this can't solve 100% cases, but I guess can sovle 80%
> > or 90%
> > >>>>>> cases.
> > >>>>>> And the remaining cases can be resolved by LIKE syntax which I
> > guess is
> > >>>>> not
> > >>>>>> very common cases.
> > >>>>>>
> > >>>>>> Best,
> > >>>>>> Jark
> > >>>>>>
> > >>>>>>
> > >>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com>
> wrote:
> > >>>>>>
> > >>>>>>> Good to have such lovely discussions. I also want to share some
> of
> > my
> > >>>>>>> opinions.
> > >>>>>>>
> > >>>>>>> #1 Regarding to error handling: I also think ignore invalid hints
> > would
> > >>>>>> be
> > >>>>>>> dangerous, maybe
> > >>>>>>> the simplest solution is just throw an exception.
> > >>>>>>>
> > >>>>>>> #2 Regarding to property replacement: I don't think we should
> > >>>>> constraint
> > >>>>>>> ourself to
> > >>>>>>> the meaning of the word "hint", and forbidden it modifying any
> > >>>>> properties
> > >>>>>>> which can effect
> > >>>>>>> query results. IMO `PROPERTIES` is one of the table hints, and a
> > >>>>> powerful
> > >>>>>>> one. It can
> > >>>>>>> modify properties located in DDL's WITH block. But I also see the
> > harm
> > >>>>>> that
> > >>>>>>> if we make it
> > >>>>>>> too flexible like change the kafka topic name with a hint. Such
> use
> > >>>>> case
> > >>>>>> is
> > >>>>>>> not common and
> > >>>>>>> sounds very dangerous to me. I would propose we have a map of
> > hintable
> > >>>>>>> properties for each
> > >>>>>>> connector, and should validate all passed in properties are
> > actually
> > >>>>>>> hintable. And combining with
> > >>>>>>> #1 error handling, we can throw an exception once received
> invalid
> > >>>>>>> property.
> > >>>>>>>
> > >>>>>>> #3 Regarding to global offset: I'm not sure it's feasible.
> > Different
> > >>>>>>> connectors will have totally
> > >>>>>>> different properties to represent offset, some might be
> timestamps,
> > >>>>> some
> > >>>>>>> might be string literals
> > >>>>>>> like "earliest", and others might be just integers.
> > >>>>>>>
> > >>>>>>> Best,
> > >>>>>>> Kurt
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com>
> wrote:
> > >>>>>>>
> > >>>>>>>> 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 <tw...@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 <twalthr@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.lwl@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.cyz@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 <twalthr@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 <
> imjark@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 <
> > >>>>>>>> kyledong@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.cyz@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.cyz@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
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>
> > >>
> > >
> >
> >
>


-- 

Benchao Li
School of Electronics Engineering and Computer Science, Peking University
Tel:+86-15650713730
Email: libenchao@gmail.com; libenchao@pku.edu.cn

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Bowen Li <bo...@gmail.com>.
A quick summary that focus of the discussion now shifts to be whether
semantic params like kafka 'starting offset' should be table
hints/properties, and if so, in what form.

I strongly believe the action of setting offset should *not* be part of a
table, neither hints nor properties, for all the good reasons mentioned
above. It doesn't describe the physical table/storage. It's indeed a filter
describing what kind of data users want to query on top of all present,
physical data.

One way to think of that is 'offset' is a system column of Kafka, and Flink
queries can filter on this system column. E.g. Postgres has a column 'xmin'
to track commit timestamp automatically and users can filter on it [1].
Flink connectors should be able to define it's own system columns,
recognize them from filters pushed down by Flink SQL framework, and apply
accordingly.

[1]
https://wiki.postgresql.org/wiki/What's_new_in_PostgreSQL_9.5#Commit_timestamp_tracking



On Wed, Mar 11, 2020 at 6:20 AM Aljoscha Krettek <al...@apache.org>
wrote:

> Hi,
>
> I don't understand this discussion. Hints, as I understand them, should
> work like this:
>
> - hints are *optional* advice for the optimizer to try and help it to
> find a good execution strategy
> - hints should not change query semantics, i.e. they should not change
> connector properties executing a query with taking into account the
> hints *must* produce the same result as executing the query without
> taking into account the hints
>
>  From these simple requirements you can derive a solution that makes
> sense. I don't have a strong preference for the syntax but we should
> strive to be in line with prior work.
>
> Best,
> Aljoscha
>
> On 11.03.20 11:53, Danny Chan wrote:
> > Thanks Timo for summarize the 3 options ~
> >
> > I agree with Kurt that option2 is too complicated to use because:
> >
> > • As a Kafka topic consumer, the user must define both the virtual
> column for start offset and he must apply a special filter predicate after
> each query
> > • And for the internal implementation, the metadata column push down is
> another hard topic, each kind of message queue may have its offset
> attribute, we need to consider the expression type for different kind; the
> source also need to recognize the constant column as a config option(which
> is weird because usually what we pushed down is a table column)
> >
> > For option 1 and option3, I think there is no difference, option1 is
> also a hint syntax which is introduced in Sybase and referenced then
> deprecated by MS-SQL in 199X years because of the ambitiousness. Personally
> I prefer /*+ */ style table hint than WITH keyword for these reasons:
> >
> > • We do not break the standard SQL, the hints are nested in SQL comments
> > • We do not need to introduce additional WITH keyword which may appear
> in a query if we use that because a table can be referenced in all kinds of
> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql query
> break too much of the SQL from standard
> > • We would have uniform syntax for hints as query hint, one syntax fits
> all and more easy to use
> >
> >
> > And here is the reason why we choose a uniform Oracle style query
> hint syntax which is addressed by Julian Hyde when we design the syntax
> from the Calcite community:
> >
> > I don’t much like the MSSQL-style syntax for table hints. It adds a new
> use of the WITH keyword that is unrelated to the use of WITH for
> common-table expressions.
> >
> > A historical note. Microsoft SQL Server inherited its hint syntax from
> Sybase a very long time ago. (See “Transact SQL Programming”[1], page 632,
> “Optimizer hints”. The book was written in 1999, and covers Microsoft SQL
> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very
> likely predates Sybase 4.3, from which Microsoft SQL Server was forked in
> 1993.)
> >
> > Microsoft later added the WITH keyword to make it less ambiguous, and
> has now deprecated the syntax that does not use WITH.
> >
> > They are forced to keep the syntax for backwards compatibility but that
> doesn’t mean that we should shoulder their burden.
> >
> > I think formatted comments are the right container for hints because it
> allows us to change the hint syntax without changing the SQL parser, and
> makes clear that we are at liberty to ignore hints entirely.
> >
> > Julian
> >
> > [1] https://www.amazon.com/s?k=9781565924017 <
> https://www.amazon.com/s?k=9781565924017>
> >
> > Best,
> > Danny Chan
> > 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
> >> Hi Danny,
> >>
> >> it is true that our DDL is not standard compliant by using the WITH
> >> clause. Nevertheless, we aim for not diverging too much and the LIKE
> >> clause is an example of that. It will solve things like overwriting
> >> WATERMARKs, add additional/modifying properties and inherit schema.
> >>
> >> Bowen is right that Flink's DDL is mixing 3 types definition together.
> >> We are not the first ones that try to solve this. There is also the SQL
> >> MED standard [1] that tried to tackle this problem. I think it was not
> >> considered when designing the current DDL.
> >>
> >> Currently, I see 3 options for handling Kafka offsets. I will give some
> >> examples and look forward to feedback here:
> >>
> >> *Option 1* Runtime and semantic parms as part of the query
> >>
> >> `SELECT * FROM MyTable('offset'=123)`
> >>
> >> Pros:
> >> - Easy to add
> >> - Parameters are part of the main query
> >> - No complicated hinting syntax
> >>
> >> Cons:
> >> - Not SQL compliant
> >>
> >> *Option 2* Use metadata in query
> >>
> >> `CREATE TABLE MyTable (id INT, offset AS SYSTEM_METADATA('offset'))`
> >>
> >> `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12 12:34:22'`
> >>
> >> Pros:
> >> - SQL compliant in the query
> >> - Access of metadata in the DDL which is required anyway
> >> - Regular pushdown rules apply
> >>
> >> Cons:
> >> - Users need to add an additional comlumn in the DDL
> >>
> >> *Option 3*: Use hints for properties
> >>
> >> `
> >> SELECT *
> >> FROM MyTable /*+ PROPERTIES('offset'=123) */
> >> `
> >>
> >> Pros:
> >> - Easy to add
> >>
> >> Cons:
> >> - Parameters are not part of the main query
> >> - Cryptic syntax for new users
> >> - Not standard compliant.
> >>
> >> If we go with this option, I would suggest to make it available in a
> >> separate map and don't mix it with statically defined properties. Such
> >> that the factory can decide which properties have the right to be
> >> overwritten by the hints:
> >> TableSourceFactory.Context.getQueryHints(): ReadableConfig
> >>
> >> Regards,
> >> Timo
> >>
> >> [1] https://en.wikipedia.org/wiki/SQL/MED
> >>
> >> Currently I see 3 options as a
> >>
> >>
> >> On 11.03.20 07:21, Danny Chan wrote:
> >>> Thanks Bowen ~
> >>>
> >>> I agree we should somehow categorize our connector parameters.
> >>>
> >>> For type1, I’m already preparing a solution like the Confluent schema
> registry + Avro schema inference thing, so this may not be a problem in the
> near future.
> >>>
> >>> For type3, I have some questions:
> >>>
> >>>> "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> >>>
> >>> Where does the offset column come from, a virtual column from the
> table schema, you said that
> >>>
> >>>> They change
> >>> almost every time a query starts and have nothing to do with metadata,
> thus
> >>> should not be part of table definition/DDL
> >>>
> >>> But why you can reference it in the query, I’m confused for that, can
> you elaborate a little ?
> >>>
> >>> Best,
> >>> Danny Chan
> >>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
> >>>> Thanks Danny for kicking off the effort
> >>>>
> >>>> The root cause of too much manual work is Flink DDL has mixed 3 types
> of
> >>>> params together and doesn't handle each of them very well. Below are
> how I
> >>>> categorize them and corresponding solutions in my mind:
> >>>>
> >>>> - type 1: Metadata of external data, like external endpoint/url,
> >>>> username/pwd, schemas, formats.
> >>>>
> >>>> Such metadata are mostly already accessible in external system as
> long as
> >>>> endpoints and credentials are provided. Flink can get it thru
> catalogs, but
> >>>> we haven't had many catalogs yet and thus Flink just hasn't been able
> to
> >>>> leverage that. So the solution should be building more catalogs. Such
> >>>> params should be part of a Flink table DDL/definition, and not
> overridable
> >>>> in any means.
> >>>>
> >>>>
> >>>> - type 2: Runtime params, like jdbc connector's fetch size,
> elasticsearch
> >>>> connector's bulk flush size.
> >>>>
> >>>> Such params don't affect query results, but affect how results are
> produced
> >>>> (eg. fast or slow, aka performance) - they are essentially execution
> and
> >>>> implementation details. They change often in exploration or
> development
> >>>> stages, but not quite frequently in well-defined long-running
> pipelines.
> >>>> They should always have default values and can be missing in query.
> They
> >>>> can be part of a table DDL/definition, but should also be replaceable
> in a
> >>>> query - *this is what table "hints" in FLIP-113 should cover*.
> >>>>
> >>>>
> >>>> - type 3: Semantic params, like kafka connector's start offset.
> >>>>
> >>>> Such params affect query results - the semantics. They'd better be as
> >>>> filter conditions in WHERE clause that can be pushed down. They change
> >>>> almost every time a query starts and have nothing to do with
> metadata, thus
> >>>> should not be part of table definition/DDL, nor be persisted in
> catalogs.
> >>>> If they will, users should create views to keep such params around
> (note
> >>>> this is different from variable substitution).
> >>>>
> >>>>
> >>>> Take Flink-Kafka as an example. Once we get these params right,
> here're the
> >>>> steps users need to do to develop and run a Flink job:
> >>>> - configure a Flink ConfluentSchemaRegistry with url, username, and
> password
> >>>> - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday"
> (simplified
> >>>> timestamp) in SQL CLI, Flink automatically retrieves all metadata of
> >>>> schema, file format, etc and start the job
> >>>> - users want to make the job read Kafka topic faster, so it goes as
> "SELECT
> >>>> * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm
> yesterday"
> >>>> - done and satisfied, users submit it to production
> >>>>
> >>>>
> >>>> Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's a
> >>>> nice-to-have feature, but not a strategically critical, long-term
> solution,
> >>>> because
> >>>> 1) It may seem promising at the current stage to solve the
> >>>> too-much-manual-work problem, but that's only because Flink hasn't
> >>>> leveraged catalogs well and handled the 3 types of params above
> properly.
> >>>> Once we get the params types right, the LIKE syntax won't be that
> >>>> important, and will be just an easier way to create tables without
> retyping
> >>>> long fields like username and pwd.
> >>>> 2) Note that only some rare type of catalog can store k-v property
> pair, so
> >>>> table created this way often cannot be persisted. In the foreseeable
> >>>> future, such catalog will only be HiveCatalog, and not everyone has a
> Hive
> >>>> metastore. To be honest, without persistence, recreating tables every
> time
> >>>> this way is still a lot of keyboard typing.
> >>>>
> >>>> Cheers,
> >>>> Bowen
> >>>>
> >>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com> wrote:
> >>>>
> >>>>> If a specific connector want to have such parameter and read if out
> of
> >>>>> configuration, then that's fine.
> >>>>> If we are talking about a configuration for all kinds of sources, I
> would
> >>>>> be super careful about that.
> >>>>> It's true it can solve maybe 80% cases, but it will also make the
> left 20%
> >>>>> feels weird.
> >>>>>
> >>>>> Best,
> >>>>> Kurt
> >>>>>
> >>>>>
> >>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:
> >>>>>
> >>>>>> Hi Kurt,
> >>>>>>
> >>>>>> #3 Regarding to global offset:
> >>>>>> I'm not saying to use the global configuration to override connector
> >>>>>> properties by the planner.
> >>>>>> But the connector should take this configuration and translate into
> their
> >>>>>> client API.
> >>>>>> AFAIK, almost all the message queues support eariliest and latest
> and a
> >>>>>> timestamp value as start point.
> >>>>>> So we can support 3 options for this configuration: "eariliest",
> "latest"
> >>>>>> and a timestamp string value.
> >>>>>> Of course, this can't solve 100% cases, but I guess can sovle 80%
> or 90%
> >>>>>> cases.
> >>>>>> And the remaining cases can be resolved by LIKE syntax which I
> guess is
> >>>>> not
> >>>>>> very common cases.
> >>>>>>
> >>>>>> Best,
> >>>>>> Jark
> >>>>>>
> >>>>>>
> >>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:
> >>>>>>
> >>>>>>> Good to have such lovely discussions. I also want to share some of
> my
> >>>>>>> opinions.
> >>>>>>>
> >>>>>>> #1 Regarding to error handling: I also think ignore invalid hints
> would
> >>>>>> be
> >>>>>>> dangerous, maybe
> >>>>>>> the simplest solution is just throw an exception.
> >>>>>>>
> >>>>>>> #2 Regarding to property replacement: I don't think we should
> >>>>> constraint
> >>>>>>> ourself to
> >>>>>>> the meaning of the word "hint", and forbidden it modifying any
> >>>>> properties
> >>>>>>> which can effect
> >>>>>>> query results. IMO `PROPERTIES` is one of the table hints, and a
> >>>>> powerful
> >>>>>>> one. It can
> >>>>>>> modify properties located in DDL's WITH block. But I also see the
> harm
> >>>>>> that
> >>>>>>> if we make it
> >>>>>>> too flexible like change the kafka topic name with a hint. Such use
> >>>>> case
> >>>>>> is
> >>>>>>> not common and
> >>>>>>> sounds very dangerous to me. I would propose we have a map of
> hintable
> >>>>>>> properties for each
> >>>>>>> connector, and should validate all passed in properties are
> actually
> >>>>>>> hintable. And combining with
> >>>>>>> #1 error handling, we can throw an exception once received invalid
> >>>>>>> property.
> >>>>>>>
> >>>>>>> #3 Regarding to global offset: I'm not sure it's feasible.
> Different
> >>>>>>> connectors will have totally
> >>>>>>> different properties to represent offset, some might be timestamps,
> >>>>> some
> >>>>>>> might be string literals
> >>>>>>> like "earliest", and others might be just integers.
> >>>>>>>
> >>>>>>> Best,
> >>>>>>> Kurt
> >>>>>>>
> >>>>>>>
> >>>>>>> On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
> >>>>>>>
> >>>>>>>> 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 <tw...@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 <tw...@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.lwl@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.cyz@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 <twalthr@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 <im...@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 <
> >>>>>>>> kyledong@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.cyz@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.cyz@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
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
> >
>
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Stephan Ewen <se...@apache.org>.
@Danny sounds good.

Maybe it is worth listing all the classes of problems that you want to
address and then look at each class and see if hints are a good default
solution or a good optional way of simplifying things?
The discussion has grown a lot and it is starting to be hard to distinguish
the parts where everyone agrees from the parts were there are concerns.

On Thu, Mar 12, 2020 at 2:31 PM Danny Chan <da...@apache.org> wrote:

> Thanks Stephan ~
>
> We can remove the support for properties that may change the semantics of
> query if you think that is a trouble.
>
> How about we support the /*+ properties() */ hint only for those optimize
> parameters, such as the fetch size of source or something like that, does
> that make sense?
>
> Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:
>
> > I think Bowen has actually put it very well.
> >
> > (1) Hints that change semantics looks like trouble waiting to happen. For
> > example Kafka offset handling should be in filters. The Kafka source
> should
> > support predicate pushdown.
> >
> > (2) Hints should not be a workaround for current shortcomings. A lot of
> the
> > suggested above sounds exactly like that. Working around catalog/DDL
> > shortcomings, missing exposure of metadata (offsets), missing predicate
> > pushdown in Kafka. Abusing a feature like hints now as a quick fix for
> > these issues, rather than fixing the root causes, will much likely bite
> us
> > back badly in the future.
> >
> > Best,
> > Stephan
> >
> >
> > On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com> wrote:
> >
> > > It seems this FLIP's name is somewhat misleading. From my
> understanding,
> > > this FLIP is trying to
> > > address the dynamic parameter issue, and table hints is the way we wan
> to
> > > choose. I think we should
> > > be focus on "what's the right way to solve dynamic property" instead of
> > > discussing "whether table
> > > hints can affect query semantics".
> > >
> > > For now, there are two proposed ways to achieve dynamic property:
> > > 1. FLIP-110: create temporary table xx like xx with (xxx)
> > > 2. use custom "from t with (xxx)" syntax
> > > 3. "Borrow" the table hints to have a special PROPERTIES hint.
> > >
> > > The first one didn't break anything, but the only problem i see is a
> > little
> > > more verbose than the table hint
> > > approach. I can imagine when someone using SQL CLI to have a sql
> > > experience, it's quite often that
> > > he will modify the table property, some use cases i can think of:
> > > 1. the source contains some corrupted data, i want to turn on the
> > > "ignore-error" flag for certain formats.
> > > 2. I have a kafka table and want to see some sample data from the
> > > beginning, so i change the offset
> > > to "earliest", and then I want to observe the latest data which keeps
> > > coming in. I would write another query
> > > to select from the latest table.
> > > 3. I want to my jdbc sink flush data more eagerly then i can observe
> the
> > > data from database side.
> > >
> > > Most of such use cases are quite ad-hoc. If every time I want to have a
> > > different experience, i need to create
> > > a temporary table and then also modify my query, it doesn't feel
> smooth.
> > > Embed such dynamic property into
> > > query would have better user experience.
> > >
> > > Both 2 & 3 can make this happen. The cons of #2 is breaking SQL
> > compliant,
> > > and for #3, it only breaks some
> > > unwritten rules, but we can have an explanation on that. And I really
> > doubt
> > > whether user would complain about
> > > this when they actually have flexible and good experience using this.
> > >
> > > My tendency would be #3 > #1 > #2, what do you think?
> > >
> > > Best,
> > > Kurt
> > >
> > >
> > > On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <yu...@gmail.com>
> wrote:
> > >
> > > > Thanks Aljoscha ~
> > > >
> > > > I agree for most of the query hints, they are optional as an
> optimizer
> > > > instruction, especially for the traditional RDBMS.
> > > >
> > > > But, just like BenChao said, Flink as a computation engine has many
> > > > different kind of data sources, thus, dynamic parameters like
> > > start_offest
> > > > can only bind to each table scope, we can not set a session config
> like
> > > > KSQL because they are all about Kafka:
> > > > > SET ‘auto.offset.reset’=‘earliest’;
> > > >
> > > > Thus the most flexible way to set up these dynamic params is to bind
> to
> > > > the table scope in the query when we want to override something, so
> we
> > > have
> > > > these solutions above (with pros and cons from my side):
> > > >
> > > > • 1. Select * from t(offset=123) (from Timo)
> > > >
> > > >            Pros:
> > > >              - Easy to add
> > > >              - Parameters are part of the main query
> > > >            Cons:
> > > >              - Not SQL compliant
> > > >
> > > >
> > > > • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from me)
> > > >
> > > >            Pros:
> > > >            - Easy to add
> > > >            - SQL compliant because it is nested in the comments
> > > >
> > > >            Cons:
> > > >            - Parameters are not part of the main query
> > > >            - Cryptic syntax for new users
> > > >
> > > > The biggest problem for hints way may be the “if hints must be
> > optional”,
> > > > actually we have though about 1 for a while but aborted because it
> > breaks
> > > > the SQL standard too much. And we replace it with 2, because the
> hints
> > > > syntax do not break SQL standard(nested in comments).
> > > >
> > > > What if we have the special /*+ PROPERTIES */ hint that allows
> override
> > > > some properties of table dynamically, it does not break anything, at
> > > lease
> > > > for current Flink use cases.
> > > >
> > > > Planner hints are optional just because they are naturally enforcers
> of
> > > > the planner, most of them aim to instruct the optimizer, but, the
> table
> > > > hints is a little different, table hints can specify the table meta
> > like
> > > > index column, and it is very convenient to specify table properties.
> > > >
> > > > Or shall we not call  /*+ PROPERTIES(offset=123) */ table hint, we
> can
> > > > call it table dynamic parameters.
> > > >
> > > > Best,
> > > > Danny Chan
> > > > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <al...@apache.org>,写道:
> > > > > Hi,
> > > > >
> > > > > I don't understand this discussion. Hints, as I understand them,
> > should
> > > > > work like this:
> > > > >
> > > > > - hints are *optional* advice for the optimizer to try and help it
> to
> > > > > find a good execution strategy
> > > > > - hints should not change query semantics, i.e. they should not
> > change
> > > > > connector properties executing a query with taking into account the
> > > > > hints *must* produce the same result as executing the query without
> > > > > taking into account the hints
> > > > >
> > > > > From these simple requirements you can derive a solution that makes
> > > > > sense. I don't have a strong preference for the syntax but we
> should
> > > > > strive to be in line with prior work.
> > > > >
> > > > > Best,
> > > > > Aljoscha
> > > > >
> > > > > On 11.03.20 11:53, Danny Chan wrote:
> > > > > > Thanks Timo for summarize the 3 options ~
> > > > > >
> > > > > > I agree with Kurt that option2 is too complicated to use because:
> > > > > >
> > > > > > • As a Kafka topic consumer, the user must define both the
> virtual
> > > > column for start offset and he must apply a special filter predicate
> > > after
> > > > each query
> > > > > > • And for the internal implementation, the metadata column push
> > down
> > > > is another hard topic, each kind of message queue may have its offset
> > > > attribute, we need to consider the expression type for different
> kind;
> > > the
> > > > source also need to recognize the constant column as a config
> > > option(which
> > > > is weird because usually what we pushed down is a table column)
> > > > > >
> > > > > > For option 1 and option3, I think there is no difference, option1
> > is
> > > > also a hint syntax which is introduced in Sybase and referenced then
> > > > deprecated by MS-SQL in 199X years because of the ambitiousness.
> > > Personally
> > > > I prefer /*+ */ style table hint than WITH keyword for these reasons:
> > > > > >
> > > > > > • We do not break the standard SQL, the hints are nested in SQL
> > > > comments
> > > > > > • We do not need to introduce additional WITH keyword which may
> > > appear
> > > > in a query if we use that because a table can be referenced in all
> > kinds
> > > of
> > > > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql
> query
> > > > break too much of the SQL from standard
> > > > > > • We would have uniform syntax for hints as query hint, one
> syntax
> > > > fits all and more easy to use
> > > > > >
> > > > > >
> > > > > > And here is the reason why we choose a uniform Oracle style query
> > > > hint syntax which is addressed by Julian Hyde when we design the
> syntax
> > > > from the Calcite community:
> > > > > >
> > > > > > I don’t much like the MSSQL-style syntax for table hints. It
> adds a
> > > > new use of the WITH keyword that is unrelated to the use of WITH for
> > > > common-table expressions.
> > > > > >
> > > > > > A historical note. Microsoft SQL Server inherited its hint syntax
> > > from
> > > > Sybase a very long time ago. (See “Transact SQL Programming”[1], page
> > > 632,
> > > > “Optimizer hints”. The book was written in 1999, and covers Microsoft
> > SQL
> > > > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very
> > > > likely predates Sybase 4.3, from which Microsoft SQL Server was
> forked
> > in
> > > > 1993.)
> > > > > >
> > > > > > Microsoft later added the WITH keyword to make it less ambiguous,
> > and
> > > > has now deprecated the syntax that does not use WITH.
> > > > > >
> > > > > > They are forced to keep the syntax for backwards compatibility
> but
> > > > that doesn’t mean that we should shoulder their burden.
> > > > > >
> > > > > > I think formatted comments are the right container for hints
> > because
> > > > it allows us to change the hint syntax without changing the SQL
> parser,
> > > and
> > > > makes clear that we are at liberty to ignore hints entirely.
> > > > > >
> > > > > > Julian
> > > > > >
> > > > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > > > https://www.amazon.com/s?k=9781565924017>
> > > > > >
> > > > > > Best,
> > > > > > Danny Chan
> > > > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
> > > > > > > Hi Danny,
> > > > > > >
> > > > > > > it is true that our DDL is not standard compliant by using the
> > WITH
> > > > > > > clause. Nevertheless, we aim for not diverging too much and the
> > > LIKE
> > > > > > > clause is an example of that. It will solve things like
> > overwriting
> > > > > > > WATERMARKs, add additional/modifying properties and inherit
> > schema.
> > > > > > >
> > > > > > > Bowen is right that Flink's DDL is mixing 3 types definition
> > > > together.
> > > > > > > We are not the first ones that try to solve this. There is also
> > the
> > > > SQL
> > > > > > > MED standard [1] that tried to tackle this problem. I think it
> > was
> > > > not
> > > > > > > considered when designing the current DDL.
> > > > > > >
> > > > > > > Currently, I see 3 options for handling Kafka offsets. I will
> > give
> > > > some
> > > > > > > examples and look forward to feedback here:
> > > > > > >
> > > > > > > *Option 1* Runtime and semantic parms as part of the query
> > > > > > >
> > > > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > > > >
> > > > > > > Pros:
> > > > > > > - Easy to add
> > > > > > > - Parameters are part of the main query
> > > > > > > - No complicated hinting syntax
> > > > > > >
> > > > > > > Cons:
> > > > > > > - Not SQL compliant
> > > > > > >
> > > > > > > *Option 2* Use metadata in query
> > > > > > >
> > > > > > > `CREATE TABLE MyTable (id INT, offset AS
> > > SYSTEM_METADATA('offset'))`
> > > > > > >
> > > > > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12
> > > > 12:34:22'`
> > > > > > >
> > > > > > > Pros:
> > > > > > > - SQL compliant in the query
> > > > > > > - Access of metadata in the DDL which is required anyway
> > > > > > > - Regular pushdown rules apply
> > > > > > >
> > > > > > > Cons:
> > > > > > > - Users need to add an additional comlumn in the DDL
> > > > > > >
> > > > > > > *Option 3*: Use hints for properties
> > > > > > >
> > > > > > > `
> > > > > > > SELECT *
> > > > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > > > `
> > > > > > >
> > > > > > > Pros:
> > > > > > > - Easy to add
> > > > > > >
> > > > > > > Cons:
> > > > > > > - Parameters are not part of the main query
> > > > > > > - Cryptic syntax for new users
> > > > > > > - Not standard compliant.
> > > > > > >
> > > > > > > If we go with this option, I would suggest to make it available
> > in
> > > a
> > > > > > > separate map and don't mix it with statically defined
> properties.
> > > > Such
> > > > > > > that the factory can decide which properties have the right to
> be
> > > > > > > overwritten by the hints:
> > > > > > > TableSourceFactory.Context.getQueryHints(): ReadableConfig
> > > > > > >
> > > > > > > Regards,
> > > > > > > Timo
> > > > > > >
> > > > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > > > >
> > > > > > > Currently I see 3 options as a
> > > > > > >
> > > > > > >
> > > > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > > > Thanks Bowen ~
> > > > > > > >
> > > > > > > > I agree we should somehow categorize our connector
> parameters.
> > > > > > > >
> > > > > > > > For type1, I’m already preparing a solution like the
> Confluent
> > > > schema registry + Avro schema inference thing, so this may not be a
> > > problem
> > > > in the near future.
> > > > > > > >
> > > > > > > > For type3, I have some questions:
> > > > > > > >
> > > > > > > > > "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> > > > > > > >
> > > > > > > > Where does the offset column come from, a virtual column from
> > the
> > > > table schema, you said that
> > > > > > > >
> > > > > > > > > They change
> > > > > > > > almost every time a query starts and have nothing to do with
> > > > metadata, thus
> > > > > > > > should not be part of table definition/DDL
> > > > > > > >
> > > > > > > > But why you can reference it in the query, I’m confused for
> > that,
> > > > can you elaborate a little ?
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Danny Chan
> > > > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <bowenli86@gmail.com
> >,写道:
> > > > > > > > > Thanks Danny for kicking off the effort
> > > > > > > > >
> > > > > > > > > The root cause of too much manual work is Flink DDL has
> > mixed 3
> > > > types of
> > > > > > > > > params together and doesn't handle each of them very well.
> > > Below
> > > > are how I
> > > > > > > > > categorize them and corresponding solutions in my mind:
> > > > > > > > >
> > > > > > > > > - type 1: Metadata of external data, like external
> > > endpoint/url,
> > > > > > > > > username/pwd, schemas, formats.
> > > > > > > > >
> > > > > > > > > Such metadata are mostly already accessible in external
> > system
> > > > as long as
> > > > > > > > > endpoints and credentials are provided. Flink can get it
> thru
> > > > catalogs, but
> > > > > > > > > we haven't had many catalogs yet and thus Flink just hasn't
> > > been
> > > > able to
> > > > > > > > > leverage that. So the solution should be building more
> > > catalogs.
> > > > Such
> > > > > > > > > params should be part of a Flink table DDL/definition, and
> > not
> > > > overridable
> > > > > > > > > in any means.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > - type 2: Runtime params, like jdbc connector's fetch size,
> > > > elasticsearch
> > > > > > > > > connector's bulk flush size.
> > > > > > > > >
> > > > > > > > > Such params don't affect query results, but affect how
> > results
> > > > are produced
> > > > > > > > > (eg. fast or slow, aka performance) - they are essentially
> > > > execution and
> > > > > > > > > implementation details. They change often in exploration or
> > > > development
> > > > > > > > > stages, but not quite frequently in well-defined
> long-running
> > > > pipelines.
> > > > > > > > > They should always have default values and can be missing
> in
> > > > query. They
> > > > > > > > > can be part of a table DDL/definition, but should also be
> > > > replaceable in a
> > > > > > > > > query - *this is what table "hints" in FLIP-113 should
> > cover*.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > - type 3: Semantic params, like kafka connector's start
> > offset.
> > > > > > > > >
> > > > > > > > > Such params affect query results - the semantics. They'd
> > better
> > > > be as
> > > > > > > > > filter conditions in WHERE clause that can be pushed down.
> > They
> > > > change
> > > > > > > > > almost every time a query starts and have nothing to do
> with
> > > > metadata, thus
> > > > > > > > > should not be part of table definition/DDL, nor be
> persisted
> > in
> > > > catalogs.
> > > > > > > > > If they will, users should create views to keep such params
> > > > around (note
> > > > > > > > > this is different from variable substitution).
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Take Flink-Kafka as an example. Once we get these params
> > right,
> > > > here're the
> > > > > > > > > steps users need to do to develop and run a Flink job:
> > > > > > > > > - configure a Flink ConfluentSchemaRegistry with url,
> > username,
> > > > and password
> > > > > > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday"
> > > > (simplified
> > > > > > > > > timestamp) in SQL CLI, Flink automatically retrieves all
> > > > metadata of
> > > > > > > > > schema, file format, etc and start the job
> > > > > > > > > - users want to make the job read Kafka topic faster, so it
> > > goes
> > > > as "SELECT
> > > > > > > > > * FROM mykafka /* faster_read_key=value*/ WHERE offset >
> 12pm
> > > > yesterday"
> > > > > > > > > - done and satisfied, users submit it to production
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think
> > > it's
> > > > a
> > > > > > > > > nice-to-have feature, but not a strategically critical,
> > > > long-term solution,
> > > > > > > > > because
> > > > > > > > > 1) It may seem promising at the current stage to solve the
> > > > > > > > > too-much-manual-work problem, but that's only because Flink
> > > > hasn't
> > > > > > > > > leveraged catalogs well and handled the 3 types of params
> > above
> > > > properly.
> > > > > > > > > Once we get the params types right, the LIKE syntax won't
> be
> > > that
> > > > > > > > > important, and will be just an easier way to create tables
> > > > without retyping
> > > > > > > > > long fields like username and pwd.
> > > > > > > > > 2) Note that only some rare type of catalog can store k-v
> > > > property pair, so
> > > > > > > > > table created this way often cannot be persisted. In the
> > > > foreseeable
> > > > > > > > > future, such catalog will only be HiveCatalog, and not
> > everyone
> > > > has a Hive
> > > > > > > > > metastore. To be honest, without persistence, recreating
> > tables
> > > > every time
> > > > > > > > > this way is still a lot of keyboard typing.
> > > > > > > > >
> > > > > > > > > Cheers,
> > > > > > > > > Bowen
> > > > > > > > >
> > > > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <
> ykt836@gmail.com
> > >
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > If a specific connector want to have such parameter and
> > read
> > > > if out of
> > > > > > > > > > configuration, then that's fine.
> > > > > > > > > > If we are talking about a configuration for all kinds of
> > > > sources, I would
> > > > > > > > > > be super careful about that.
> > > > > > > > > > It's true it can solve maybe 80% cases, but it will also
> > make
> > > > the left 20%
> > > > > > > > > > feels weird.
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Kurt
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <
> imjark@gmail.com
> > >
> > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Kurt,
> > > > > > > > > > >
> > > > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > > > I'm not saying to use the global configuration to
> > override
> > > > connector
> > > > > > > > > > > properties by the planner.
> > > > > > > > > > > But the connector should take this configuration and
> > > > translate into their
> > > > > > > > > > > client API.
> > > > > > > > > > > AFAIK, almost all the message queues support eariliest
> > and
> > > > latest and a
> > > > > > > > > > > timestamp value as start point.
> > > > > > > > > > > So we can support 3 options for this configuration:
> > > > "eariliest", "latest"
> > > > > > > > > > > and a timestamp string value.
> > > > > > > > > > > Of course, this can't solve 100% cases, but I guess can
> > > > sovle 80% or 90%
> > > > > > > > > > > cases.
> > > > > > > > > > > And the remaining cases can be resolved by LIKE syntax
> > > which
> > > > I guess is
> > > > > > > > > > not
> > > > > > > > > > > very common cases.
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Jark
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <
> > ykt836@gmail.com
> > > >
> > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Good to have such lovely discussions. I also want to
> > > share
> > > > some of my
> > > > > > > > > > > > opinions.
> > > > > > > > > > > >
> > > > > > > > > > > > #1 Regarding to error handling: I also think ignore
> > > > invalid hints would
> > > > > > > > > > > be
> > > > > > > > > > > > dangerous, maybe
> > > > > > > > > > > > the simplest solution is just throw an exception.
> > > > > > > > > > > >
> > > > > > > > > > > > #2 Regarding to property replacement: I don't think
> we
> > > > should
> > > > > > > > > > constraint
> > > > > > > > > > > > ourself to
> > > > > > > > > > > > the meaning of the word "hint", and forbidden it
> > > modifying
> > > > any
> > > > > > > > > > properties
> > > > > > > > > > > > which can effect
> > > > > > > > > > > > query results. IMO `PROPERTIES` is one of the table
> > > hints,
> > > > and a
> > > > > > > > > > powerful
> > > > > > > > > > > > one. It can
> > > > > > > > > > > > modify properties located in DDL's WITH block. But I
> > also
> > > > see the harm
> > > > > > > > > > > that
> > > > > > > > > > > > if we make it
> > > > > > > > > > > > too flexible like change the kafka topic name with a
> > > hint.
> > > > Such use
> > > > > > > > > > case
> > > > > > > > > > > is
> > > > > > > > > > > > not common and
> > > > > > > > > > > > sounds very dangerous to me. I would propose we have
> a
> > > map
> > > > of hintable
> > > > > > > > > > > > properties for each
> > > > > > > > > > > > connector, and should validate all passed in
> properties
> > > > are actually
> > > > > > > > > > > > hintable. And combining with
> > > > > > > > > > > > #1 error handling, we can throw an exception once
> > > received
> > > > invalid
> > > > > > > > > > > > property.
> > > > > > > > > > > >
> > > > > > > > > > > > #3 Regarding to global offset: I'm not sure it's
> > > feasible.
> > > > Different
> > > > > > > > > > > > connectors will have totally
> > > > > > > > > > > > different properties to represent offset, some might
> be
> > > > timestamps,
> > > > > > > > > > some
> > > > > > > > > > > > might be string literals
> > > > > > > > > > > > like "earliest", and others might be just integers.
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Kurt
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
> > > imjark@gmail.com>
> > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > 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 <
> > > > twalthr@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 <
> > > > twalthr@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.lwl@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.cyz@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 <
> > > > twalthr@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
> > > <
> > > > imjark@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 <
> > > > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <da...@apache.org>.
Thanks Stephan ~

We can remove the support for properties that may change the semantics of
query if you think that is a trouble.

How about we support the /*+ properties() */ hint only for those optimize
parameters, such as the fetch size of source or something like that, does
that make sense?

Stephan Ewen <se...@apache.org>于2020年3月12日 周四下午7:45写道:

> I think Bowen has actually put it very well.
>
> (1) Hints that change semantics looks like trouble waiting to happen. For
> example Kafka offset handling should be in filters. The Kafka source should
> support predicate pushdown.
>
> (2) Hints should not be a workaround for current shortcomings. A lot of the
> suggested above sounds exactly like that. Working around catalog/DDL
> shortcomings, missing exposure of metadata (offsets), missing predicate
> pushdown in Kafka. Abusing a feature like hints now as a quick fix for
> these issues, rather than fixing the root causes, will much likely bite us
> back badly in the future.
>
> Best,
> Stephan
>
>
> On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com> wrote:
>
> > It seems this FLIP's name is somewhat misleading. From my understanding,
> > this FLIP is trying to
> > address the dynamic parameter issue, and table hints is the way we wan to
> > choose. I think we should
> > be focus on "what's the right way to solve dynamic property" instead of
> > discussing "whether table
> > hints can affect query semantics".
> >
> > For now, there are two proposed ways to achieve dynamic property:
> > 1. FLIP-110: create temporary table xx like xx with (xxx)
> > 2. use custom "from t with (xxx)" syntax
> > 3. "Borrow" the table hints to have a special PROPERTIES hint.
> >
> > The first one didn't break anything, but the only problem i see is a
> little
> > more verbose than the table hint
> > approach. I can imagine when someone using SQL CLI to have a sql
> > experience, it's quite often that
> > he will modify the table property, some use cases i can think of:
> > 1. the source contains some corrupted data, i want to turn on the
> > "ignore-error" flag for certain formats.
> > 2. I have a kafka table and want to see some sample data from the
> > beginning, so i change the offset
> > to "earliest", and then I want to observe the latest data which keeps
> > coming in. I would write another query
> > to select from the latest table.
> > 3. I want to my jdbc sink flush data more eagerly then i can observe the
> > data from database side.
> >
> > Most of such use cases are quite ad-hoc. If every time I want to have a
> > different experience, i need to create
> > a temporary table and then also modify my query, it doesn't feel smooth.
> > Embed such dynamic property into
> > query would have better user experience.
> >
> > Both 2 & 3 can make this happen. The cons of #2 is breaking SQL
> compliant,
> > and for #3, it only breaks some
> > unwritten rules, but we can have an explanation on that. And I really
> doubt
> > whether user would complain about
> > this when they actually have flexible and good experience using this.
> >
> > My tendency would be #3 > #1 > #2, what do you think?
> >
> > Best,
> > Kurt
> >
> >
> > On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <yu...@gmail.com> wrote:
> >
> > > Thanks Aljoscha ~
> > >
> > > I agree for most of the query hints, they are optional as an optimizer
> > > instruction, especially for the traditional RDBMS.
> > >
> > > But, just like BenChao said, Flink as a computation engine has many
> > > different kind of data sources, thus, dynamic parameters like
> > start_offest
> > > can only bind to each table scope, we can not set a session config like
> > > KSQL because they are all about Kafka:
> > > > SET ‘auto.offset.reset’=‘earliest’;
> > >
> > > Thus the most flexible way to set up these dynamic params is to bind to
> > > the table scope in the query when we want to override something, so we
> > have
> > > these solutions above (with pros and cons from my side):
> > >
> > > • 1. Select * from t(offset=123) (from Timo)
> > >
> > >            Pros:
> > >              - Easy to add
> > >              - Parameters are part of the main query
> > >            Cons:
> > >              - Not SQL compliant
> > >
> > >
> > > • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from me)
> > >
> > >            Pros:
> > >            - Easy to add
> > >            - SQL compliant because it is nested in the comments
> > >
> > >            Cons:
> > >            - Parameters are not part of the main query
> > >            - Cryptic syntax for new users
> > >
> > > The biggest problem for hints way may be the “if hints must be
> optional”,
> > > actually we have though about 1 for a while but aborted because it
> breaks
> > > the SQL standard too much. And we replace it with 2, because the hints
> > > syntax do not break SQL standard(nested in comments).
> > >
> > > What if we have the special /*+ PROPERTIES */ hint that allows override
> > > some properties of table dynamically, it does not break anything, at
> > lease
> > > for current Flink use cases.
> > >
> > > Planner hints are optional just because they are naturally enforcers of
> > > the planner, most of them aim to instruct the optimizer, but, the table
> > > hints is a little different, table hints can specify the table meta
> like
> > > index column, and it is very convenient to specify table properties.
> > >
> > > Or shall we not call  /*+ PROPERTIES(offset=123) */ table hint, we can
> > > call it table dynamic parameters.
> > >
> > > Best,
> > > Danny Chan
> > > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <al...@apache.org>,写道:
> > > > Hi,
> > > >
> > > > I don't understand this discussion. Hints, as I understand them,
> should
> > > > work like this:
> > > >
> > > > - hints are *optional* advice for the optimizer to try and help it to
> > > > find a good execution strategy
> > > > - hints should not change query semantics, i.e. they should not
> change
> > > > connector properties executing a query with taking into account the
> > > > hints *must* produce the same result as executing the query without
> > > > taking into account the hints
> > > >
> > > > From these simple requirements you can derive a solution that makes
> > > > sense. I don't have a strong preference for the syntax but we should
> > > > strive to be in line with prior work.
> > > >
> > > > Best,
> > > > Aljoscha
> > > >
> > > > On 11.03.20 11:53, Danny Chan wrote:
> > > > > Thanks Timo for summarize the 3 options ~
> > > > >
> > > > > I agree with Kurt that option2 is too complicated to use because:
> > > > >
> > > > > • As a Kafka topic consumer, the user must define both the virtual
> > > column for start offset and he must apply a special filter predicate
> > after
> > > each query
> > > > > • And for the internal implementation, the metadata column push
> down
> > > is another hard topic, each kind of message queue may have its offset
> > > attribute, we need to consider the expression type for different kind;
> > the
> > > source also need to recognize the constant column as a config
> > option(which
> > > is weird because usually what we pushed down is a table column)
> > > > >
> > > > > For option 1 and option3, I think there is no difference, option1
> is
> > > also a hint syntax which is introduced in Sybase and referenced then
> > > deprecated by MS-SQL in 199X years because of the ambitiousness.
> > Personally
> > > I prefer /*+ */ style table hint than WITH keyword for these reasons:
> > > > >
> > > > > • We do not break the standard SQL, the hints are nested in SQL
> > > comments
> > > > > • We do not need to introduce additional WITH keyword which may
> > appear
> > > in a query if we use that because a table can be referenced in all
> kinds
> > of
> > > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql query
> > > break too much of the SQL from standard
> > > > > • We would have uniform syntax for hints as query hint, one syntax
> > > fits all and more easy to use
> > > > >
> > > > >
> > > > > And here is the reason why we choose a uniform Oracle style query
> > > hint syntax which is addressed by Julian Hyde when we design the syntax
> > > from the Calcite community:
> > > > >
> > > > > I don’t much like the MSSQL-style syntax for table hints. It adds a
> > > new use of the WITH keyword that is unrelated to the use of WITH for
> > > common-table expressions.
> > > > >
> > > > > A historical note. Microsoft SQL Server inherited its hint syntax
> > from
> > > Sybase a very long time ago. (See “Transact SQL Programming”[1], page
> > 632,
> > > “Optimizer hints”. The book was written in 1999, and covers Microsoft
> SQL
> > > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very
> > > likely predates Sybase 4.3, from which Microsoft SQL Server was forked
> in
> > > 1993.)
> > > > >
> > > > > Microsoft later added the WITH keyword to make it less ambiguous,
> and
> > > has now deprecated the syntax that does not use WITH.
> > > > >
> > > > > They are forced to keep the syntax for backwards compatibility but
> > > that doesn’t mean that we should shoulder their burden.
> > > > >
> > > > > I think formatted comments are the right container for hints
> because
> > > it allows us to change the hint syntax without changing the SQL parser,
> > and
> > > makes clear that we are at liberty to ignore hints entirely.
> > > > >
> > > > > Julian
> > > > >
> > > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > > https://www.amazon.com/s?k=9781565924017>
> > > > >
> > > > > Best,
> > > > > Danny Chan
> > > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
> > > > > > Hi Danny,
> > > > > >
> > > > > > it is true that our DDL is not standard compliant by using the
> WITH
> > > > > > clause. Nevertheless, we aim for not diverging too much and the
> > LIKE
> > > > > > clause is an example of that. It will solve things like
> overwriting
> > > > > > WATERMARKs, add additional/modifying properties and inherit
> schema.
> > > > > >
> > > > > > Bowen is right that Flink's DDL is mixing 3 types definition
> > > together.
> > > > > > We are not the first ones that try to solve this. There is also
> the
> > > SQL
> > > > > > MED standard [1] that tried to tackle this problem. I think it
> was
> > > not
> > > > > > considered when designing the current DDL.
> > > > > >
> > > > > > Currently, I see 3 options for handling Kafka offsets. I will
> give
> > > some
> > > > > > examples and look forward to feedback here:
> > > > > >
> > > > > > *Option 1* Runtime and semantic parms as part of the query
> > > > > >
> > > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > > >
> > > > > > Pros:
> > > > > > - Easy to add
> > > > > > - Parameters are part of the main query
> > > > > > - No complicated hinting syntax
> > > > > >
> > > > > > Cons:
> > > > > > - Not SQL compliant
> > > > > >
> > > > > > *Option 2* Use metadata in query
> > > > > >
> > > > > > `CREATE TABLE MyTable (id INT, offset AS
> > SYSTEM_METADATA('offset'))`
> > > > > >
> > > > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12
> > > 12:34:22'`
> > > > > >
> > > > > > Pros:
> > > > > > - SQL compliant in the query
> > > > > > - Access of metadata in the DDL which is required anyway
> > > > > > - Regular pushdown rules apply
> > > > > >
> > > > > > Cons:
> > > > > > - Users need to add an additional comlumn in the DDL
> > > > > >
> > > > > > *Option 3*: Use hints for properties
> > > > > >
> > > > > > `
> > > > > > SELECT *
> > > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > > `
> > > > > >
> > > > > > Pros:
> > > > > > - Easy to add
> > > > > >
> > > > > > Cons:
> > > > > > - Parameters are not part of the main query
> > > > > > - Cryptic syntax for new users
> > > > > > - Not standard compliant.
> > > > > >
> > > > > > If we go with this option, I would suggest to make it available
> in
> > a
> > > > > > separate map and don't mix it with statically defined properties.
> > > Such
> > > > > > that the factory can decide which properties have the right to be
> > > > > > overwritten by the hints:
> > > > > > TableSourceFactory.Context.getQueryHints(): ReadableConfig
> > > > > >
> > > > > > Regards,
> > > > > > Timo
> > > > > >
> > > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > > >
> > > > > > Currently I see 3 options as a
> > > > > >
> > > > > >
> > > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > > Thanks Bowen ~
> > > > > > >
> > > > > > > I agree we should somehow categorize our connector parameters.
> > > > > > >
> > > > > > > For type1, I’m already preparing a solution like the Confluent
> > > schema registry + Avro schema inference thing, so this may not be a
> > problem
> > > in the near future.
> > > > > > >
> > > > > > > For type3, I have some questions:
> > > > > > >
> > > > > > > > "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> > > > > > >
> > > > > > > Where does the offset column come from, a virtual column from
> the
> > > table schema, you said that
> > > > > > >
> > > > > > > > They change
> > > > > > > almost every time a query starts and have nothing to do with
> > > metadata, thus
> > > > > > > should not be part of table definition/DDL
> > > > > > >
> > > > > > > But why you can reference it in the query, I’m confused for
> that,
> > > can you elaborate a little ?
> > > > > > >
> > > > > > > Best,
> > > > > > > Danny Chan
> > > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
> > > > > > > > Thanks Danny for kicking off the effort
> > > > > > > >
> > > > > > > > The root cause of too much manual work is Flink DDL has
> mixed 3
> > > types of
> > > > > > > > params together and doesn't handle each of them very well.
> > Below
> > > are how I
> > > > > > > > categorize them and corresponding solutions in my mind:
> > > > > > > >
> > > > > > > > - type 1: Metadata of external data, like external
> > endpoint/url,
> > > > > > > > username/pwd, schemas, formats.
> > > > > > > >
> > > > > > > > Such metadata are mostly already accessible in external
> system
> > > as long as
> > > > > > > > endpoints and credentials are provided. Flink can get it thru
> > > catalogs, but
> > > > > > > > we haven't had many catalogs yet and thus Flink just hasn't
> > been
> > > able to
> > > > > > > > leverage that. So the solution should be building more
> > catalogs.
> > > Such
> > > > > > > > params should be part of a Flink table DDL/definition, and
> not
> > > overridable
> > > > > > > > in any means.
> > > > > > > >
> > > > > > > >
> > > > > > > > - type 2: Runtime params, like jdbc connector's fetch size,
> > > elasticsearch
> > > > > > > > connector's bulk flush size.
> > > > > > > >
> > > > > > > > Such params don't affect query results, but affect how
> results
> > > are produced
> > > > > > > > (eg. fast or slow, aka performance) - they are essentially
> > > execution and
> > > > > > > > implementation details. They change often in exploration or
> > > development
> > > > > > > > stages, but not quite frequently in well-defined long-running
> > > pipelines.
> > > > > > > > They should always have default values and can be missing in
> > > query. They
> > > > > > > > can be part of a table DDL/definition, but should also be
> > > replaceable in a
> > > > > > > > query - *this is what table "hints" in FLIP-113 should
> cover*.
> > > > > > > >
> > > > > > > >
> > > > > > > > - type 3: Semantic params, like kafka connector's start
> offset.
> > > > > > > >
> > > > > > > > Such params affect query results - the semantics. They'd
> better
> > > be as
> > > > > > > > filter conditions in WHERE clause that can be pushed down.
> They
> > > change
> > > > > > > > almost every time a query starts and have nothing to do with
> > > metadata, thus
> > > > > > > > should not be part of table definition/DDL, nor be persisted
> in
> > > catalogs.
> > > > > > > > If they will, users should create views to keep such params
> > > around (note
> > > > > > > > this is different from variable substitution).
> > > > > > > >
> > > > > > > >
> > > > > > > > Take Flink-Kafka as an example. Once we get these params
> right,
> > > here're the
> > > > > > > > steps users need to do to develop and run a Flink job:
> > > > > > > > - configure a Flink ConfluentSchemaRegistry with url,
> username,
> > > and password
> > > > > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday"
> > > (simplified
> > > > > > > > timestamp) in SQL CLI, Flink automatically retrieves all
> > > metadata of
> > > > > > > > schema, file format, etc and start the job
> > > > > > > > - users want to make the job read Kafka topic faster, so it
> > goes
> > > as "SELECT
> > > > > > > > * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm
> > > yesterday"
> > > > > > > > - done and satisfied, users submit it to production
> > > > > > > >
> > > > > > > >
> > > > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think
> > it's
> > > a
> > > > > > > > nice-to-have feature, but not a strategically critical,
> > > long-term solution,
> > > > > > > > because
> > > > > > > > 1) It may seem promising at the current stage to solve the
> > > > > > > > too-much-manual-work problem, but that's only because Flink
> > > hasn't
> > > > > > > > leveraged catalogs well and handled the 3 types of params
> above
> > > properly.
> > > > > > > > Once we get the params types right, the LIKE syntax won't be
> > that
> > > > > > > > important, and will be just an easier way to create tables
> > > without retyping
> > > > > > > > long fields like username and pwd.
> > > > > > > > 2) Note that only some rare type of catalog can store k-v
> > > property pair, so
> > > > > > > > table created this way often cannot be persisted. In the
> > > foreseeable
> > > > > > > > future, such catalog will only be HiveCatalog, and not
> everyone
> > > has a Hive
> > > > > > > > metastore. To be honest, without persistence, recreating
> tables
> > > every time
> > > > > > > > this way is still a lot of keyboard typing.
> > > > > > > >
> > > > > > > > Cheers,
> > > > > > > > Bowen
> > > > > > > >
> > > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <ykt836@gmail.com
> >
> > > wrote:
> > > > > > > >
> > > > > > > > > If a specific connector want to have such parameter and
> read
> > > if out of
> > > > > > > > > configuration, then that's fine.
> > > > > > > > > If we are talking about a configuration for all kinds of
> > > sources, I would
> > > > > > > > > be super careful about that.
> > > > > > > > > It's true it can solve maybe 80% cases, but it will also
> make
> > > the left 20%
> > > > > > > > > feels weird.
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Kurt
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <imjark@gmail.com
> >
> > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Kurt,
> > > > > > > > > >
> > > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > > I'm not saying to use the global configuration to
> override
> > > connector
> > > > > > > > > > properties by the planner.
> > > > > > > > > > But the connector should take this configuration and
> > > translate into their
> > > > > > > > > > client API.
> > > > > > > > > > AFAIK, almost all the message queues support eariliest
> and
> > > latest and a
> > > > > > > > > > timestamp value as start point.
> > > > > > > > > > So we can support 3 options for this configuration:
> > > "eariliest", "latest"
> > > > > > > > > > and a timestamp string value.
> > > > > > > > > > Of course, this can't solve 100% cases, but I guess can
> > > sovle 80% or 90%
> > > > > > > > > > cases.
> > > > > > > > > > And the remaining cases can be resolved by LIKE syntax
> > which
> > > I guess is
> > > > > > > > > not
> > > > > > > > > > very common cases.
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Jark
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <
> ykt836@gmail.com
> > >
> > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Good to have such lovely discussions. I also want to
> > share
> > > some of my
> > > > > > > > > > > opinions.
> > > > > > > > > > >
> > > > > > > > > > > #1 Regarding to error handling: I also think ignore
> > > invalid hints would
> > > > > > > > > > be
> > > > > > > > > > > dangerous, maybe
> > > > > > > > > > > the simplest solution is just throw an exception.
> > > > > > > > > > >
> > > > > > > > > > > #2 Regarding to property replacement: I don't think we
> > > should
> > > > > > > > > constraint
> > > > > > > > > > > ourself to
> > > > > > > > > > > the meaning of the word "hint", and forbidden it
> > modifying
> > > any
> > > > > > > > > properties
> > > > > > > > > > > which can effect
> > > > > > > > > > > query results. IMO `PROPERTIES` is one of the table
> > hints,
> > > and a
> > > > > > > > > powerful
> > > > > > > > > > > one. It can
> > > > > > > > > > > modify properties located in DDL's WITH block. But I
> also
> > > see the harm
> > > > > > > > > > that
> > > > > > > > > > > if we make it
> > > > > > > > > > > too flexible like change the kafka topic name with a
> > hint.
> > > Such use
> > > > > > > > > case
> > > > > > > > > > is
> > > > > > > > > > > not common and
> > > > > > > > > > > sounds very dangerous to me. I would propose we have a
> > map
> > > of hintable
> > > > > > > > > > > properties for each
> > > > > > > > > > > connector, and should validate all passed in properties
> > > are actually
> > > > > > > > > > > hintable. And combining with
> > > > > > > > > > > #1 error handling, we can throw an exception once
> > received
> > > invalid
> > > > > > > > > > > property.
> > > > > > > > > > >
> > > > > > > > > > > #3 Regarding to global offset: I'm not sure it's
> > feasible.
> > > Different
> > > > > > > > > > > connectors will have totally
> > > > > > > > > > > different properties to represent offset, some might be
> > > timestamps,
> > > > > > > > > some
> > > > > > > > > > > might be string literals
> > > > > > > > > > > like "earliest", and others might be just integers.
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Kurt
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
> > imjark@gmail.com>
> > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > 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 <
> > > twalthr@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 <
> > > twalthr@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.lwl@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.cyz@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 <
> > > twalthr@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
> > <
> > > imjark@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 <
> > > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Stephan Ewen <se...@apache.org>.
I think Bowen has actually put it very well.

(1) Hints that change semantics looks like trouble waiting to happen. For
example Kafka offset handling should be in filters. The Kafka source should
support predicate pushdown.

(2) Hints should not be a workaround for current shortcomings. A lot of the
suggested above sounds exactly like that. Working around catalog/DDL
shortcomings, missing exposure of metadata (offsets), missing predicate
pushdown in Kafka. Abusing a feature like hints now as a quick fix for
these issues, rather than fixing the root causes, will much likely bite us
back badly in the future.

Best,
Stephan


On Thu, Mar 12, 2020 at 10:43 AM Kurt Young <yk...@gmail.com> wrote:

> It seems this FLIP's name is somewhat misleading. From my understanding,
> this FLIP is trying to
> address the dynamic parameter issue, and table hints is the way we wan to
> choose. I think we should
> be focus on "what's the right way to solve dynamic property" instead of
> discussing "whether table
> hints can affect query semantics".
>
> For now, there are two proposed ways to achieve dynamic property:
> 1. FLIP-110: create temporary table xx like xx with (xxx)
> 2. use custom "from t with (xxx)" syntax
> 3. "Borrow" the table hints to have a special PROPERTIES hint.
>
> The first one didn't break anything, but the only problem i see is a little
> more verbose than the table hint
> approach. I can imagine when someone using SQL CLI to have a sql
> experience, it's quite often that
> he will modify the table property, some use cases i can think of:
> 1. the source contains some corrupted data, i want to turn on the
> "ignore-error" flag for certain formats.
> 2. I have a kafka table and want to see some sample data from the
> beginning, so i change the offset
> to "earliest", and then I want to observe the latest data which keeps
> coming in. I would write another query
> to select from the latest table.
> 3. I want to my jdbc sink flush data more eagerly then i can observe the
> data from database side.
>
> Most of such use cases are quite ad-hoc. If every time I want to have a
> different experience, i need to create
> a temporary table and then also modify my query, it doesn't feel smooth.
> Embed such dynamic property into
> query would have better user experience.
>
> Both 2 & 3 can make this happen. The cons of #2 is breaking SQL compliant,
> and for #3, it only breaks some
> unwritten rules, but we can have an explanation on that. And I really doubt
> whether user would complain about
> this when they actually have flexible and good experience using this.
>
> My tendency would be #3 > #1 > #2, what do you think?
>
> Best,
> Kurt
>
>
> On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <yu...@gmail.com> wrote:
>
> > Thanks Aljoscha ~
> >
> > I agree for most of the query hints, they are optional as an optimizer
> > instruction, especially for the traditional RDBMS.
> >
> > But, just like BenChao said, Flink as a computation engine has many
> > different kind of data sources, thus, dynamic parameters like
> start_offest
> > can only bind to each table scope, we can not set a session config like
> > KSQL because they are all about Kafka:
> > > SET ‘auto.offset.reset’=‘earliest’;
> >
> > Thus the most flexible way to set up these dynamic params is to bind to
> > the table scope in the query when we want to override something, so we
> have
> > these solutions above (with pros and cons from my side):
> >
> > • 1. Select * from t(offset=123) (from Timo)
> >
> >            Pros:
> >              - Easy to add
> >              - Parameters are part of the main query
> >            Cons:
> >              - Not SQL compliant
> >
> >
> > • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from me)
> >
> >            Pros:
> >            - Easy to add
> >            - SQL compliant because it is nested in the comments
> >
> >            Cons:
> >            - Parameters are not part of the main query
> >            - Cryptic syntax for new users
> >
> > The biggest problem for hints way may be the “if hints must be optional”,
> > actually we have though about 1 for a while but aborted because it breaks
> > the SQL standard too much. And we replace it with 2, because the hints
> > syntax do not break SQL standard(nested in comments).
> >
> > What if we have the special /*+ PROPERTIES */ hint that allows override
> > some properties of table dynamically, it does not break anything, at
> lease
> > for current Flink use cases.
> >
> > Planner hints are optional just because they are naturally enforcers of
> > the planner, most of them aim to instruct the optimizer, but, the table
> > hints is a little different, table hints can specify the table meta like
> > index column, and it is very convenient to specify table properties.
> >
> > Or shall we not call  /*+ PROPERTIES(offset=123) */ table hint, we can
> > call it table dynamic parameters.
> >
> > Best,
> > Danny Chan
> > 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <al...@apache.org>,写道:
> > > Hi,
> > >
> > > I don't understand this discussion. Hints, as I understand them, should
> > > work like this:
> > >
> > > - hints are *optional* advice for the optimizer to try and help it to
> > > find a good execution strategy
> > > - hints should not change query semantics, i.e. they should not change
> > > connector properties executing a query with taking into account the
> > > hints *must* produce the same result as executing the query without
> > > taking into account the hints
> > >
> > > From these simple requirements you can derive a solution that makes
> > > sense. I don't have a strong preference for the syntax but we should
> > > strive to be in line with prior work.
> > >
> > > Best,
> > > Aljoscha
> > >
> > > On 11.03.20 11:53, Danny Chan wrote:
> > > > Thanks Timo for summarize the 3 options ~
> > > >
> > > > I agree with Kurt that option2 is too complicated to use because:
> > > >
> > > > • As a Kafka topic consumer, the user must define both the virtual
> > column for start offset and he must apply a special filter predicate
> after
> > each query
> > > > • And for the internal implementation, the metadata column push down
> > is another hard topic, each kind of message queue may have its offset
> > attribute, we need to consider the expression type for different kind;
> the
> > source also need to recognize the constant column as a config
> option(which
> > is weird because usually what we pushed down is a table column)
> > > >
> > > > For option 1 and option3, I think there is no difference, option1 is
> > also a hint syntax which is introduced in Sybase and referenced then
> > deprecated by MS-SQL in 199X years because of the ambitiousness.
> Personally
> > I prefer /*+ */ style table hint than WITH keyword for these reasons:
> > > >
> > > > • We do not break the standard SQL, the hints are nested in SQL
> > comments
> > > > • We do not need to introduce additional WITH keyword which may
> appear
> > in a query if we use that because a table can be referenced in all kinds
> of
> > SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql query
> > break too much of the SQL from standard
> > > > • We would have uniform syntax for hints as query hint, one syntax
> > fits all and more easy to use
> > > >
> > > >
> > > > And here is the reason why we choose a uniform Oracle style query
> > hint syntax which is addressed by Julian Hyde when we design the syntax
> > from the Calcite community:
> > > >
> > > > I don’t much like the MSSQL-style syntax for table hints. It adds a
> > new use of the WITH keyword that is unrelated to the use of WITH for
> > common-table expressions.
> > > >
> > > > A historical note. Microsoft SQL Server inherited its hint syntax
> from
> > Sybase a very long time ago. (See “Transact SQL Programming”[1], page
> 632,
> > “Optimizer hints”. The book was written in 1999, and covers Microsoft SQL
> > Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very
> > likely predates Sybase 4.3, from which Microsoft SQL Server was forked in
> > 1993.)
> > > >
> > > > Microsoft later added the WITH keyword to make it less ambiguous, and
> > has now deprecated the syntax that does not use WITH.
> > > >
> > > > They are forced to keep the syntax for backwards compatibility but
> > that doesn’t mean that we should shoulder their burden.
> > > >
> > > > I think formatted comments are the right container for hints because
> > it allows us to change the hint syntax without changing the SQL parser,
> and
> > makes clear that we are at liberty to ignore hints entirely.
> > > >
> > > > Julian
> > > >
> > > > [1] https://www.amazon.com/s?k=9781565924017 <
> > https://www.amazon.com/s?k=9781565924017>
> > > >
> > > > Best,
> > > > Danny Chan
> > > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
> > > > > Hi Danny,
> > > > >
> > > > > it is true that our DDL is not standard compliant by using the WITH
> > > > > clause. Nevertheless, we aim for not diverging too much and the
> LIKE
> > > > > clause is an example of that. It will solve things like overwriting
> > > > > WATERMARKs, add additional/modifying properties and inherit schema.
> > > > >
> > > > > Bowen is right that Flink's DDL is mixing 3 types definition
> > together.
> > > > > We are not the first ones that try to solve this. There is also the
> > SQL
> > > > > MED standard [1] that tried to tackle this problem. I think it was
> > not
> > > > > considered when designing the current DDL.
> > > > >
> > > > > Currently, I see 3 options for handling Kafka offsets. I will give
> > some
> > > > > examples and look forward to feedback here:
> > > > >
> > > > > *Option 1* Runtime and semantic parms as part of the query
> > > > >
> > > > > `SELECT * FROM MyTable('offset'=123)`
> > > > >
> > > > > Pros:
> > > > > - Easy to add
> > > > > - Parameters are part of the main query
> > > > > - No complicated hinting syntax
> > > > >
> > > > > Cons:
> > > > > - Not SQL compliant
> > > > >
> > > > > *Option 2* Use metadata in query
> > > > >
> > > > > `CREATE TABLE MyTable (id INT, offset AS
> SYSTEM_METADATA('offset'))`
> > > > >
> > > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12
> > 12:34:22'`
> > > > >
> > > > > Pros:
> > > > > - SQL compliant in the query
> > > > > - Access of metadata in the DDL which is required anyway
> > > > > - Regular pushdown rules apply
> > > > >
> > > > > Cons:
> > > > > - Users need to add an additional comlumn in the DDL
> > > > >
> > > > > *Option 3*: Use hints for properties
> > > > >
> > > > > `
> > > > > SELECT *
> > > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > > `
> > > > >
> > > > > Pros:
> > > > > - Easy to add
> > > > >
> > > > > Cons:
> > > > > - Parameters are not part of the main query
> > > > > - Cryptic syntax for new users
> > > > > - Not standard compliant.
> > > > >
> > > > > If we go with this option, I would suggest to make it available in
> a
> > > > > separate map and don't mix it with statically defined properties.
> > Such
> > > > > that the factory can decide which properties have the right to be
> > > > > overwritten by the hints:
> > > > > TableSourceFactory.Context.getQueryHints(): ReadableConfig
> > > > >
> > > > > Regards,
> > > > > Timo
> > > > >
> > > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > > >
> > > > > Currently I see 3 options as a
> > > > >
> > > > >
> > > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > > Thanks Bowen ~
> > > > > >
> > > > > > I agree we should somehow categorize our connector parameters.
> > > > > >
> > > > > > For type1, I’m already preparing a solution like the Confluent
> > schema registry + Avro schema inference thing, so this may not be a
> problem
> > in the near future.
> > > > > >
> > > > > > For type3, I have some questions:
> > > > > >
> > > > > > > "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> > > > > >
> > > > > > Where does the offset column come from, a virtual column from the
> > table schema, you said that
> > > > > >
> > > > > > > They change
> > > > > > almost every time a query starts and have nothing to do with
> > metadata, thus
> > > > > > should not be part of table definition/DDL
> > > > > >
> > > > > > But why you can reference it in the query, I’m confused for that,
> > can you elaborate a little ?
> > > > > >
> > > > > > Best,
> > > > > > Danny Chan
> > > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
> > > > > > > Thanks Danny for kicking off the effort
> > > > > > >
> > > > > > > The root cause of too much manual work is Flink DDL has mixed 3
> > types of
> > > > > > > params together and doesn't handle each of them very well.
> Below
> > are how I
> > > > > > > categorize them and corresponding solutions in my mind:
> > > > > > >
> > > > > > > - type 1: Metadata of external data, like external
> endpoint/url,
> > > > > > > username/pwd, schemas, formats.
> > > > > > >
> > > > > > > Such metadata are mostly already accessible in external system
> > as long as
> > > > > > > endpoints and credentials are provided. Flink can get it thru
> > catalogs, but
> > > > > > > we haven't had many catalogs yet and thus Flink just hasn't
> been
> > able to
> > > > > > > leverage that. So the solution should be building more
> catalogs.
> > Such
> > > > > > > params should be part of a Flink table DDL/definition, and not
> > overridable
> > > > > > > in any means.
> > > > > > >
> > > > > > >
> > > > > > > - type 2: Runtime params, like jdbc connector's fetch size,
> > elasticsearch
> > > > > > > connector's bulk flush size.
> > > > > > >
> > > > > > > Such params don't affect query results, but affect how results
> > are produced
> > > > > > > (eg. fast or slow, aka performance) - they are essentially
> > execution and
> > > > > > > implementation details. They change often in exploration or
> > development
> > > > > > > stages, but not quite frequently in well-defined long-running
> > pipelines.
> > > > > > > They should always have default values and can be missing in
> > query. They
> > > > > > > can be part of a table DDL/definition, but should also be
> > replaceable in a
> > > > > > > query - *this is what table "hints" in FLIP-113 should cover*.
> > > > > > >
> > > > > > >
> > > > > > > - type 3: Semantic params, like kafka connector's start offset.
> > > > > > >
> > > > > > > Such params affect query results - the semantics. They'd better
> > be as
> > > > > > > filter conditions in WHERE clause that can be pushed down. They
> > change
> > > > > > > almost every time a query starts and have nothing to do with
> > metadata, thus
> > > > > > > should not be part of table definition/DDL, nor be persisted in
> > catalogs.
> > > > > > > If they will, users should create views to keep such params
> > around (note
> > > > > > > this is different from variable substitution).
> > > > > > >
> > > > > > >
> > > > > > > Take Flink-Kafka as an example. Once we get these params right,
> > here're the
> > > > > > > steps users need to do to develop and run a Flink job:
> > > > > > > - configure a Flink ConfluentSchemaRegistry with url, username,
> > and password
> > > > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday"
> > (simplified
> > > > > > > timestamp) in SQL CLI, Flink automatically retrieves all
> > metadata of
> > > > > > > schema, file format, etc and start the job
> > > > > > > - users want to make the job read Kafka topic faster, so it
> goes
> > as "SELECT
> > > > > > > * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm
> > yesterday"
> > > > > > > - done and satisfied, users submit it to production
> > > > > > >
> > > > > > >
> > > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think
> it's
> > a
> > > > > > > nice-to-have feature, but not a strategically critical,
> > long-term solution,
> > > > > > > because
> > > > > > > 1) It may seem promising at the current stage to solve the
> > > > > > > too-much-manual-work problem, but that's only because Flink
> > hasn't
> > > > > > > leveraged catalogs well and handled the 3 types of params above
> > properly.
> > > > > > > Once we get the params types right, the LIKE syntax won't be
> that
> > > > > > > important, and will be just an easier way to create tables
> > without retyping
> > > > > > > long fields like username and pwd.
> > > > > > > 2) Note that only some rare type of catalog can store k-v
> > property pair, so
> > > > > > > table created this way often cannot be persisted. In the
> > foreseeable
> > > > > > > future, such catalog will only be HiveCatalog, and not everyone
> > has a Hive
> > > > > > > metastore. To be honest, without persistence, recreating tables
> > every time
> > > > > > > this way is still a lot of keyboard typing.
> > > > > > >
> > > > > > > Cheers,
> > > > > > > Bowen
> > > > > > >
> > > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com>
> > wrote:
> > > > > > >
> > > > > > > > If a specific connector want to have such parameter and read
> > if out of
> > > > > > > > configuration, then that's fine.
> > > > > > > > If we are talking about a configuration for all kinds of
> > sources, I would
> > > > > > > > be super careful about that.
> > > > > > > > It's true it can solve maybe 80% cases, but it will also make
> > the left 20%
> > > > > > > > feels weird.
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Kurt
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com>
> > wrote:
> > > > > > > >
> > > > > > > > > Hi Kurt,
> > > > > > > > >
> > > > > > > > > #3 Regarding to global offset:
> > > > > > > > > I'm not saying to use the global configuration to override
> > connector
> > > > > > > > > properties by the planner.
> > > > > > > > > But the connector should take this configuration and
> > translate into their
> > > > > > > > > client API.
> > > > > > > > > AFAIK, almost all the message queues support eariliest and
> > latest and a
> > > > > > > > > timestamp value as start point.
> > > > > > > > > So we can support 3 options for this configuration:
> > "eariliest", "latest"
> > > > > > > > > and a timestamp string value.
> > > > > > > > > Of course, this can't solve 100% cases, but I guess can
> > sovle 80% or 90%
> > > > > > > > > cases.
> > > > > > > > > And the remaining cases can be resolved by LIKE syntax
> which
> > I guess is
> > > > > > > > not
> > > > > > > > > very common cases.
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Jark
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <ykt836@gmail.com
> >
> > wrote:
> > > > > > > > >
> > > > > > > > > > Good to have such lovely discussions. I also want to
> share
> > some of my
> > > > > > > > > > opinions.
> > > > > > > > > >
> > > > > > > > > > #1 Regarding to error handling: I also think ignore
> > invalid hints would
> > > > > > > > > be
> > > > > > > > > > dangerous, maybe
> > > > > > > > > > the simplest solution is just throw an exception.
> > > > > > > > > >
> > > > > > > > > > #2 Regarding to property replacement: I don't think we
> > should
> > > > > > > > constraint
> > > > > > > > > > ourself to
> > > > > > > > > > the meaning of the word "hint", and forbidden it
> modifying
> > any
> > > > > > > > properties
> > > > > > > > > > which can effect
> > > > > > > > > > query results. IMO `PROPERTIES` is one of the table
> hints,
> > and a
> > > > > > > > powerful
> > > > > > > > > > one. It can
> > > > > > > > > > modify properties located in DDL's WITH block. But I also
> > see the harm
> > > > > > > > > that
> > > > > > > > > > if we make it
> > > > > > > > > > too flexible like change the kafka topic name with a
> hint.
> > Such use
> > > > > > > > case
> > > > > > > > > is
> > > > > > > > > > not common and
> > > > > > > > > > sounds very dangerous to me. I would propose we have a
> map
> > of hintable
> > > > > > > > > > properties for each
> > > > > > > > > > connector, and should validate all passed in properties
> > are actually
> > > > > > > > > > hintable. And combining with
> > > > > > > > > > #1 error handling, we can throw an exception once
> received
> > invalid
> > > > > > > > > > property.
> > > > > > > > > >
> > > > > > > > > > #3 Regarding to global offset: I'm not sure it's
> feasible.
> > Different
> > > > > > > > > > connectors will have totally
> > > > > > > > > > different properties to represent offset, some might be
> > timestamps,
> > > > > > > > some
> > > > > > > > > > might be string literals
> > > > > > > > > > like "earliest", and others might be just integers.
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Kurt
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <
> imjark@gmail.com>
> > wrote:
> > > > > > > > > >
> > > > > > > > > > > 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 <
> > twalthr@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 <
> > twalthr@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.lwl@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.cyz@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 <
> > twalthr@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
> <
> > imjark@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 <
> > > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Kurt Young <yk...@gmail.com>.
It seems this FLIP's name is somewhat misleading. From my understanding,
this FLIP is trying to
address the dynamic parameter issue, and table hints is the way we wan to
choose. I think we should
be focus on "what's the right way to solve dynamic property" instead of
discussing "whether table
hints can affect query semantics".

For now, there are two proposed ways to achieve dynamic property:
1. FLIP-110: create temporary table xx like xx with (xxx)
2. use custom "from t with (xxx)" syntax
3. "Borrow" the table hints to have a special PROPERTIES hint.

The first one didn't break anything, but the only problem i see is a little
more verbose than the table hint
approach. I can imagine when someone using SQL CLI to have a sql
experience, it's quite often that
he will modify the table property, some use cases i can think of:
1. the source contains some corrupted data, i want to turn on the
"ignore-error" flag for certain formats.
2. I have a kafka table and want to see some sample data from the
beginning, so i change the offset
to "earliest", and then I want to observe the latest data which keeps
coming in. I would write another query
to select from the latest table.
3. I want to my jdbc sink flush data more eagerly then i can observe the
data from database side.

Most of such use cases are quite ad-hoc. If every time I want to have a
different experience, i need to create
a temporary table and then also modify my query, it doesn't feel smooth.
Embed such dynamic property into
query would have better user experience.

Both 2 & 3 can make this happen. The cons of #2 is breaking SQL compliant,
and for #3, it only breaks some
unwritten rules, but we can have an explanation on that. And I really doubt
whether user would complain about
this when they actually have flexible and good experience using this.

My tendency would be #3 > #1 > #2, what do you think?

Best,
Kurt


On Thu, Mar 12, 2020 at 1:11 PM Danny Chan <yu...@gmail.com> wrote:

> Thanks Aljoscha ~
>
> I agree for most of the query hints, they are optional as an optimizer
> instruction, especially for the traditional RDBMS.
>
> But, just like BenChao said, Flink as a computation engine has many
> different kind of data sources, thus, dynamic parameters like start_offest
> can only bind to each table scope, we can not set a session config like
> KSQL because they are all about Kafka:
> > SET ‘auto.offset.reset’=‘earliest’;
>
> Thus the most flexible way to set up these dynamic params is to bind to
> the table scope in the query when we want to override something, so we have
> these solutions above (with pros and cons from my side):
>
> • 1. Select * from t(offset=123) (from Timo)
>
>            Pros:
>              - Easy to add
>              - Parameters are part of the main query
>            Cons:
>              - Not SQL compliant
>
>
> • 2. Select * from t /*+ PROPERTIES(offset=123) */ (from me)
>
>            Pros:
>            - Easy to add
>            - SQL compliant because it is nested in the comments
>
>            Cons:
>            - Parameters are not part of the main query
>            - Cryptic syntax for new users
>
> The biggest problem for hints way may be the “if hints must be optional”,
> actually we have though about 1 for a while but aborted because it breaks
> the SQL standard too much. And we replace it with 2, because the hints
> syntax do not break SQL standard(nested in comments).
>
> What if we have the special /*+ PROPERTIES */ hint that allows override
> some properties of table dynamically, it does not break anything, at lease
> for current Flink use cases.
>
> Planner hints are optional just because they are naturally enforcers of
> the planner, most of them aim to instruct the optimizer, but, the table
> hints is a little different, table hints can specify the table meta like
> index column, and it is very convenient to specify table properties.
>
> Or shall we not call  /*+ PROPERTIES(offset=123) */ table hint, we can
> call it table dynamic parameters.
>
> Best,
> Danny Chan
> 在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <al...@apache.org>,写道:
> > Hi,
> >
> > I don't understand this discussion. Hints, as I understand them, should
> > work like this:
> >
> > - hints are *optional* advice for the optimizer to try and help it to
> > find a good execution strategy
> > - hints should not change query semantics, i.e. they should not change
> > connector properties executing a query with taking into account the
> > hints *must* produce the same result as executing the query without
> > taking into account the hints
> >
> > From these simple requirements you can derive a solution that makes
> > sense. I don't have a strong preference for the syntax but we should
> > strive to be in line with prior work.
> >
> > Best,
> > Aljoscha
> >
> > On 11.03.20 11:53, Danny Chan wrote:
> > > Thanks Timo for summarize the 3 options ~
> > >
> > > I agree with Kurt that option2 is too complicated to use because:
> > >
> > > • As a Kafka topic consumer, the user must define both the virtual
> column for start offset and he must apply a special filter predicate after
> each query
> > > • And for the internal implementation, the metadata column push down
> is another hard topic, each kind of message queue may have its offset
> attribute, we need to consider the expression type for different kind; the
> source also need to recognize the constant column as a config option(which
> is weird because usually what we pushed down is a table column)
> > >
> > > For option 1 and option3, I think there is no difference, option1 is
> also a hint syntax which is introduced in Sybase and referenced then
> deprecated by MS-SQL in 199X years because of the ambitiousness. Personally
> I prefer /*+ */ style table hint than WITH keyword for these reasons:
> > >
> > > • We do not break the standard SQL, the hints are nested in SQL
> comments
> > > • We do not need to introduce additional WITH keyword which may appear
> in a query if we use that because a table can be referenced in all kinds of
> SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql query
> break too much of the SQL from standard
> > > • We would have uniform syntax for hints as query hint, one syntax
> fits all and more easy to use
> > >
> > >
> > > And here is the reason why we choose a uniform Oracle style query
> hint syntax which is addressed by Julian Hyde when we design the syntax
> from the Calcite community:
> > >
> > > I don’t much like the MSSQL-style syntax for table hints. It adds a
> new use of the WITH keyword that is unrelated to the use of WITH for
> common-table expressions.
> > >
> > > A historical note. Microsoft SQL Server inherited its hint syntax from
> Sybase a very long time ago. (See “Transact SQL Programming”[1], page 632,
> “Optimizer hints”. The book was written in 1999, and covers Microsoft SQL
> Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very
> likely predates Sybase 4.3, from which Microsoft SQL Server was forked in
> 1993.)
> > >
> > > Microsoft later added the WITH keyword to make it less ambiguous, and
> has now deprecated the syntax that does not use WITH.
> > >
> > > They are forced to keep the syntax for backwards compatibility but
> that doesn’t mean that we should shoulder their burden.
> > >
> > > I think formatted comments are the right container for hints because
> it allows us to change the hint syntax without changing the SQL parser, and
> makes clear that we are at liberty to ignore hints entirely.
> > >
> > > Julian
> > >
> > > [1] https://www.amazon.com/s?k=9781565924017 <
> https://www.amazon.com/s?k=9781565924017>
> > >
> > > Best,
> > > Danny Chan
> > > 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
> > > > Hi Danny,
> > > >
> > > > it is true that our DDL is not standard compliant by using the WITH
> > > > clause. Nevertheless, we aim for not diverging too much and the LIKE
> > > > clause is an example of that. It will solve things like overwriting
> > > > WATERMARKs, add additional/modifying properties and inherit schema.
> > > >
> > > > Bowen is right that Flink's DDL is mixing 3 types definition
> together.
> > > > We are not the first ones that try to solve this. There is also the
> SQL
> > > > MED standard [1] that tried to tackle this problem. I think it was
> not
> > > > considered when designing the current DDL.
> > > >
> > > > Currently, I see 3 options for handling Kafka offsets. I will give
> some
> > > > examples and look forward to feedback here:
> > > >
> > > > *Option 1* Runtime and semantic parms as part of the query
> > > >
> > > > `SELECT * FROM MyTable('offset'=123)`
> > > >
> > > > Pros:
> > > > - Easy to add
> > > > - Parameters are part of the main query
> > > > - No complicated hinting syntax
> > > >
> > > > Cons:
> > > > - Not SQL compliant
> > > >
> > > > *Option 2* Use metadata in query
> > > >
> > > > `CREATE TABLE MyTable (id INT, offset AS SYSTEM_METADATA('offset'))`
> > > >
> > > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12
> 12:34:22'`
> > > >
> > > > Pros:
> > > > - SQL compliant in the query
> > > > - Access of metadata in the DDL which is required anyway
> > > > - Regular pushdown rules apply
> > > >
> > > > Cons:
> > > > - Users need to add an additional comlumn in the DDL
> > > >
> > > > *Option 3*: Use hints for properties
> > > >
> > > > `
> > > > SELECT *
> > > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > > `
> > > >
> > > > Pros:
> > > > - Easy to add
> > > >
> > > > Cons:
> > > > - Parameters are not part of the main query
> > > > - Cryptic syntax for new users
> > > > - Not standard compliant.
> > > >
> > > > If we go with this option, I would suggest to make it available in a
> > > > separate map and don't mix it with statically defined properties.
> Such
> > > > that the factory can decide which properties have the right to be
> > > > overwritten by the hints:
> > > > TableSourceFactory.Context.getQueryHints(): ReadableConfig
> > > >
> > > > Regards,
> > > > Timo
> > > >
> > > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > > >
> > > > Currently I see 3 options as a
> > > >
> > > >
> > > > On 11.03.20 07:21, Danny Chan wrote:
> > > > > Thanks Bowen ~
> > > > >
> > > > > I agree we should somehow categorize our connector parameters.
> > > > >
> > > > > For type1, I’m already preparing a solution like the Confluent
> schema registry + Avro schema inference thing, so this may not be a problem
> in the near future.
> > > > >
> > > > > For type3, I have some questions:
> > > > >
> > > > > > "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> > > > >
> > > > > Where does the offset column come from, a virtual column from the
> table schema, you said that
> > > > >
> > > > > > They change
> > > > > almost every time a query starts and have nothing to do with
> metadata, thus
> > > > > should not be part of table definition/DDL
> > > > >
> > > > > But why you can reference it in the query, I’m confused for that,
> can you elaborate a little ?
> > > > >
> > > > > Best,
> > > > > Danny Chan
> > > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
> > > > > > Thanks Danny for kicking off the effort
> > > > > >
> > > > > > The root cause of too much manual work is Flink DDL has mixed 3
> types of
> > > > > > params together and doesn't handle each of them very well. Below
> are how I
> > > > > > categorize them and corresponding solutions in my mind:
> > > > > >
> > > > > > - type 1: Metadata of external data, like external endpoint/url,
> > > > > > username/pwd, schemas, formats.
> > > > > >
> > > > > > Such metadata are mostly already accessible in external system
> as long as
> > > > > > endpoints and credentials are provided. Flink can get it thru
> catalogs, but
> > > > > > we haven't had many catalogs yet and thus Flink just hasn't been
> able to
> > > > > > leverage that. So the solution should be building more catalogs.
> Such
> > > > > > params should be part of a Flink table DDL/definition, and not
> overridable
> > > > > > in any means.
> > > > > >
> > > > > >
> > > > > > - type 2: Runtime params, like jdbc connector's fetch size,
> elasticsearch
> > > > > > connector's bulk flush size.
> > > > > >
> > > > > > Such params don't affect query results, but affect how results
> are produced
> > > > > > (eg. fast or slow, aka performance) - they are essentially
> execution and
> > > > > > implementation details. They change often in exploration or
> development
> > > > > > stages, but not quite frequently in well-defined long-running
> pipelines.
> > > > > > They should always have default values and can be missing in
> query. They
> > > > > > can be part of a table DDL/definition, but should also be
> replaceable in a
> > > > > > query - *this is what table "hints" in FLIP-113 should cover*.
> > > > > >
> > > > > >
> > > > > > - type 3: Semantic params, like kafka connector's start offset.
> > > > > >
> > > > > > Such params affect query results - the semantics. They'd better
> be as
> > > > > > filter conditions in WHERE clause that can be pushed down. They
> change
> > > > > > almost every time a query starts and have nothing to do with
> metadata, thus
> > > > > > should not be part of table definition/DDL, nor be persisted in
> catalogs.
> > > > > > If they will, users should create views to keep such params
> around (note
> > > > > > this is different from variable substitution).
> > > > > >
> > > > > >
> > > > > > Take Flink-Kafka as an example. Once we get these params right,
> here're the
> > > > > > steps users need to do to develop and run a Flink job:
> > > > > > - configure a Flink ConfluentSchemaRegistry with url, username,
> and password
> > > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday"
> (simplified
> > > > > > timestamp) in SQL CLI, Flink automatically retrieves all
> metadata of
> > > > > > schema, file format, etc and start the job
> > > > > > - users want to make the job read Kafka topic faster, so it goes
> as "SELECT
> > > > > > * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm
> yesterday"
> > > > > > - done and satisfied, users submit it to production
> > > > > >
> > > > > >
> > > > > > Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's
> a
> > > > > > nice-to-have feature, but not a strategically critical,
> long-term solution,
> > > > > > because
> > > > > > 1) It may seem promising at the current stage to solve the
> > > > > > too-much-manual-work problem, but that's only because Flink
> hasn't
> > > > > > leveraged catalogs well and handled the 3 types of params above
> properly.
> > > > > > Once we get the params types right, the LIKE syntax won't be that
> > > > > > important, and will be just an easier way to create tables
> without retyping
> > > > > > long fields like username and pwd.
> > > > > > 2) Note that only some rare type of catalog can store k-v
> property pair, so
> > > > > > table created this way often cannot be persisted. In the
> foreseeable
> > > > > > future, such catalog will only be HiveCatalog, and not everyone
> has a Hive
> > > > > > metastore. To be honest, without persistence, recreating tables
> every time
> > > > > > this way is still a lot of keyboard typing.
> > > > > >
> > > > > > Cheers,
> > > > > > Bowen
> > > > > >
> > > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com>
> wrote:
> > > > > >
> > > > > > > If a specific connector want to have such parameter and read
> if out of
> > > > > > > configuration, then that's fine.
> > > > > > > If we are talking about a configuration for all kinds of
> sources, I would
> > > > > > > be super careful about that.
> > > > > > > It's true it can solve maybe 80% cases, but it will also make
> the left 20%
> > > > > > > feels weird.
> > > > > > >
> > > > > > > Best,
> > > > > > > Kurt
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com>
> wrote:
> > > > > > >
> > > > > > > > Hi Kurt,
> > > > > > > >
> > > > > > > > #3 Regarding to global offset:
> > > > > > > > I'm not saying to use the global configuration to override
> connector
> > > > > > > > properties by the planner.
> > > > > > > > But the connector should take this configuration and
> translate into their
> > > > > > > > client API.
> > > > > > > > AFAIK, almost all the message queues support eariliest and
> latest and a
> > > > > > > > timestamp value as start point.
> > > > > > > > So we can support 3 options for this configuration:
> "eariliest", "latest"
> > > > > > > > and a timestamp string value.
> > > > > > > > Of course, this can't solve 100% cases, but I guess can
> sovle 80% or 90%
> > > > > > > > cases.
> > > > > > > > And the remaining cases can be resolved by LIKE syntax which
> I guess is
> > > > > > > not
> > > > > > > > very common cases.
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Jark
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com>
> wrote:
> > > > > > > >
> > > > > > > > > Good to have such lovely discussions. I also want to share
> some of my
> > > > > > > > > opinions.
> > > > > > > > >
> > > > > > > > > #1 Regarding to error handling: I also think ignore
> invalid hints would
> > > > > > > > be
> > > > > > > > > dangerous, maybe
> > > > > > > > > the simplest solution is just throw an exception.
> > > > > > > > >
> > > > > > > > > #2 Regarding to property replacement: I don't think we
> should
> > > > > > > constraint
> > > > > > > > > ourself to
> > > > > > > > > the meaning of the word "hint", and forbidden it modifying
> any
> > > > > > > properties
> > > > > > > > > which can effect
> > > > > > > > > query results. IMO `PROPERTIES` is one of the table hints,
> and a
> > > > > > > powerful
> > > > > > > > > one. It can
> > > > > > > > > modify properties located in DDL's WITH block. But I also
> see the harm
> > > > > > > > that
> > > > > > > > > if we make it
> > > > > > > > > too flexible like change the kafka topic name with a hint.
> Such use
> > > > > > > case
> > > > > > > > is
> > > > > > > > > not common and
> > > > > > > > > sounds very dangerous to me. I would propose we have a map
> of hintable
> > > > > > > > > properties for each
> > > > > > > > > connector, and should validate all passed in properties
> are actually
> > > > > > > > > hintable. And combining with
> > > > > > > > > #1 error handling, we can throw an exception once received
> invalid
> > > > > > > > > property.
> > > > > > > > >
> > > > > > > > > #3 Regarding to global offset: I'm not sure it's feasible.
> Different
> > > > > > > > > connectors will have totally
> > > > > > > > > different properties to represent offset, some might be
> timestamps,
> > > > > > > some
> > > > > > > > > might be string literals
> > > > > > > > > like "earliest", and others might be just integers.
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Kurt
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com>
> wrote:
> > > > > > > > >
> > > > > > > > > > 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 <
> twalthr@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 <
> twalthr@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.lwl@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.cyz@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 <
> twalthr@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 <
> imjark@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 <
> > > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
Thanks Aljoscha ~

I agree for most of the query hints, they are optional as an optimizer instruction, especially for the traditional RDBMS.

But, just like BenChao said, Flink as a computation engine has many different kind of data sources, thus, dynamic parameters like start_offest can only bind to each table scope, we can not set a session config like KSQL because they are all about Kafka:
> SET ‘auto.offset.reset’=‘earliest’;

Thus the most flexible way to set up these dynamic params is to bind to the table scope in the query when we want to override something, so we have these solutions above (with pros and cons from my side):

• 1. Select * from t(offset=123) (from Timo)

           Pros:
             - Easy to add
             - Parameters are part of the main query
           Cons:
             - Not SQL compliant


• 2. Select * from t /*+ PROPERTIES(offset=123) */ (from me)

           Pros:
           - Easy to add
           - SQL compliant because it is nested in the comments

           Cons:
           - Parameters are not part of the main query
           - Cryptic syntax for new users

The biggest problem for hints way may be the “if hints must be optional”, actually we have though about 1 for a while but aborted because it breaks the SQL standard too much. And we replace it with 2, because the hints syntax do not break SQL standard(nested in comments).

What if we have the special /*+ PROPERTIES */ hint that allows override some properties of table dynamically, it does not break anything, at lease for current Flink use cases.

Planner hints are optional just because they are naturally enforcers of the planner, most of them aim to instruct the optimizer, but, the table hints is a little different, table hints can specify the table meta like index column, and it is very convenient to specify table properties.

Or shall we not call  /*+ PROPERTIES(offset=123) */ table hint, we can call it table dynamic parameters.

Best,
Danny Chan
在 2020年3月11日 +0800 PM9:20,Aljoscha Krettek <al...@apache.org>,写道:
> Hi,
>
> I don't understand this discussion. Hints, as I understand them, should
> work like this:
>
> - hints are *optional* advice for the optimizer to try and help it to
> find a good execution strategy
> - hints should not change query semantics, i.e. they should not change
> connector properties executing a query with taking into account the
> hints *must* produce the same result as executing the query without
> taking into account the hints
>
> From these simple requirements you can derive a solution that makes
> sense. I don't have a strong preference for the syntax but we should
> strive to be in line with prior work.
>
> Best,
> Aljoscha
>
> On 11.03.20 11:53, Danny Chan wrote:
> > Thanks Timo for summarize the 3 options ~
> >
> > I agree with Kurt that option2 is too complicated to use because:
> >
> > • As a Kafka topic consumer, the user must define both the virtual column for start offset and he must apply a special filter predicate after each query
> > • And for the internal implementation, the metadata column push down is another hard topic, each kind of message queue may have its offset attribute, we need to consider the expression type for different kind; the source also need to recognize the constant column as a config option(which is weird because usually what we pushed down is a table column)
> >
> > For option 1 and option3, I think there is no difference, option1 is also a hint syntax which is introduced in Sybase and referenced then deprecated by MS-SQL in 199X years because of the ambitiousness. Personally I prefer /*+ */ style table hint than WITH keyword for these reasons:
> >
> > • We do not break the standard SQL, the hints are nested in SQL comments
> > • We do not need to introduce additional WITH keyword which may appear in a query if we use that because a table can be referenced in all kinds of SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql query break too much of the SQL from standard
> > • We would have uniform syntax for hints as query hint, one syntax fits all and more easy to use
> >
> >
> > And here is the reason why we choose a uniform Oracle style query hint syntax which is addressed by Julian Hyde when we design the syntax from the Calcite community:
> >
> > I don’t much like the MSSQL-style syntax for table hints. It adds a new use of the WITH keyword that is unrelated to the use of WITH for common-table expressions.
> >
> > A historical note. Microsoft SQL Server inherited its hint syntax from Sybase a very long time ago. (See “Transact SQL Programming”[1], page 632, “Optimizer hints”. The book was written in 1999, and covers Microsoft SQL Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very likely predates Sybase 4.3, from which Microsoft SQL Server was forked in 1993.)
> >
> > Microsoft later added the WITH keyword to make it less ambiguous, and has now deprecated the syntax that does not use WITH.
> >
> > They are forced to keep the syntax for backwards compatibility but that doesn’t mean that we should shoulder their burden.
> >
> > I think formatted comments are the right container for hints because it allows us to change the hint syntax without changing the SQL parser, and makes clear that we are at liberty to ignore hints entirely.
> >
> > Julian
> >
> > [1] https://www.amazon.com/s?k=9781565924017 <https://www.amazon.com/s?k=9781565924017>
> >
> > Best,
> > Danny Chan
> > 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
> > > Hi Danny,
> > >
> > > it is true that our DDL is not standard compliant by using the WITH
> > > clause. Nevertheless, we aim for not diverging too much and the LIKE
> > > clause is an example of that. It will solve things like overwriting
> > > WATERMARKs, add additional/modifying properties and inherit schema.
> > >
> > > Bowen is right that Flink's DDL is mixing 3 types definition together.
> > > We are not the first ones that try to solve this. There is also the SQL
> > > MED standard [1] that tried to tackle this problem. I think it was not
> > > considered when designing the current DDL.
> > >
> > > Currently, I see 3 options for handling Kafka offsets. I will give some
> > > examples and look forward to feedback here:
> > >
> > > *Option 1* Runtime and semantic parms as part of the query
> > >
> > > `SELECT * FROM MyTable('offset'=123)`
> > >
> > > Pros:
> > > - Easy to add
> > > - Parameters are part of the main query
> > > - No complicated hinting syntax
> > >
> > > Cons:
> > > - Not SQL compliant
> > >
> > > *Option 2* Use metadata in query
> > >
> > > `CREATE TABLE MyTable (id INT, offset AS SYSTEM_METADATA('offset'))`
> > >
> > > `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12 12:34:22'`
> > >
> > > Pros:
> > > - SQL compliant in the query
> > > - Access of metadata in the DDL which is required anyway
> > > - Regular pushdown rules apply
> > >
> > > Cons:
> > > - Users need to add an additional comlumn in the DDL
> > >
> > > *Option 3*: Use hints for properties
> > >
> > > `
> > > SELECT *
> > > FROM MyTable /*+ PROPERTIES('offset'=123) */
> > > `
> > >
> > > Pros:
> > > - Easy to add
> > >
> > > Cons:
> > > - Parameters are not part of the main query
> > > - Cryptic syntax for new users
> > > - Not standard compliant.
> > >
> > > If we go with this option, I would suggest to make it available in a
> > > separate map and don't mix it with statically defined properties. Such
> > > that the factory can decide which properties have the right to be
> > > overwritten by the hints:
> > > TableSourceFactory.Context.getQueryHints(): ReadableConfig
> > >
> > > Regards,
> > > Timo
> > >
> > > [1] https://en.wikipedia.org/wiki/SQL/MED
> > >
> > > Currently I see 3 options as a
> > >
> > >
> > > On 11.03.20 07:21, Danny Chan wrote:
> > > > Thanks Bowen ~
> > > >
> > > > I agree we should somehow categorize our connector parameters.
> > > >
> > > > For type1, I’m already preparing a solution like the Confluent schema registry + Avro schema inference thing, so this may not be a problem in the near future.
> > > >
> > > > For type3, I have some questions:
> > > >
> > > > > "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> > > >
> > > > Where does the offset column come from, a virtual column from the table schema, you said that
> > > >
> > > > > They change
> > > > almost every time a query starts and have nothing to do with metadata, thus
> > > > should not be part of table definition/DDL
> > > >
> > > > But why you can reference it in the query, I’m confused for that, can you elaborate a little ?
> > > >
> > > > Best,
> > > > Danny Chan
> > > > 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
> > > > > Thanks Danny for kicking off the effort
> > > > >
> > > > > The root cause of too much manual work is Flink DDL has mixed 3 types of
> > > > > params together and doesn't handle each of them very well. Below are how I
> > > > > categorize them and corresponding solutions in my mind:
> > > > >
> > > > > - type 1: Metadata of external data, like external endpoint/url,
> > > > > username/pwd, schemas, formats.
> > > > >
> > > > > Such metadata are mostly already accessible in external system as long as
> > > > > endpoints and credentials are provided. Flink can get it thru catalogs, but
> > > > > we haven't had many catalogs yet and thus Flink just hasn't been able to
> > > > > leverage that. So the solution should be building more catalogs. Such
> > > > > params should be part of a Flink table DDL/definition, and not overridable
> > > > > in any means.
> > > > >
> > > > >
> > > > > - type 2: Runtime params, like jdbc connector's fetch size, elasticsearch
> > > > > connector's bulk flush size.
> > > > >
> > > > > Such params don't affect query results, but affect how results are produced
> > > > > (eg. fast or slow, aka performance) - they are essentially execution and
> > > > > implementation details. They change often in exploration or development
> > > > > stages, but not quite frequently in well-defined long-running pipelines.
> > > > > They should always have default values and can be missing in query. They
> > > > > can be part of a table DDL/definition, but should also be replaceable in a
> > > > > query - *this is what table "hints" in FLIP-113 should cover*.
> > > > >
> > > > >
> > > > > - type 3: Semantic params, like kafka connector's start offset.
> > > > >
> > > > > Such params affect query results - the semantics. They'd better be as
> > > > > filter conditions in WHERE clause that can be pushed down. They change
> > > > > almost every time a query starts and have nothing to do with metadata, thus
> > > > > should not be part of table definition/DDL, nor be persisted in catalogs.
> > > > > If they will, users should create views to keep such params around (note
> > > > > this is different from variable substitution).
> > > > >
> > > > >
> > > > > Take Flink-Kafka as an example. Once we get these params right, here're the
> > > > > steps users need to do to develop and run a Flink job:
> > > > > - configure a Flink ConfluentSchemaRegistry with url, username, and password
> > > > > - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday" (simplified
> > > > > timestamp) in SQL CLI, Flink automatically retrieves all metadata of
> > > > > schema, file format, etc and start the job
> > > > > - users want to make the job read Kafka topic faster, so it goes as "SELECT
> > > > > * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm yesterday"
> > > > > - done and satisfied, users submit it to production
> > > > >
> > > > >
> > > > > Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's a
> > > > > nice-to-have feature, but not a strategically critical, long-term solution,
> > > > > because
> > > > > 1) It may seem promising at the current stage to solve the
> > > > > too-much-manual-work problem, but that's only because Flink hasn't
> > > > > leveraged catalogs well and handled the 3 types of params above properly.
> > > > > Once we get the params types right, the LIKE syntax won't be that
> > > > > important, and will be just an easier way to create tables without retyping
> > > > > long fields like username and pwd.
> > > > > 2) Note that only some rare type of catalog can store k-v property pair, so
> > > > > table created this way often cannot be persisted. In the foreseeable
> > > > > future, such catalog will only be HiveCatalog, and not everyone has a Hive
> > > > > metastore. To be honest, without persistence, recreating tables every time
> > > > > this way is still a lot of keyboard typing.
> > > > >
> > > > > Cheers,
> > > > > Bowen
> > > > >
> > > > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com> wrote:
> > > > >
> > > > > > If a specific connector want to have such parameter and read if out of
> > > > > > configuration, then that's fine.
> > > > > > If we are talking about a configuration for all kinds of sources, I would
> > > > > > be super careful about that.
> > > > > > It's true it can solve maybe 80% cases, but it will also make the left 20%
> > > > > > feels weird.
> > > > > >
> > > > > > Best,
> > > > > > Kurt
> > > > > >
> > > > > >
> > > > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:
> > > > > >
> > > > > > > Hi Kurt,
> > > > > > >
> > > > > > > #3 Regarding to global offset:
> > > > > > > I'm not saying to use the global configuration to override connector
> > > > > > > properties by the planner.
> > > > > > > But the connector should take this configuration and translate into their
> > > > > > > client API.
> > > > > > > AFAIK, almost all the message queues support eariliest and latest and a
> > > > > > > timestamp value as start point.
> > > > > > > So we can support 3 options for this configuration: "eariliest", "latest"
> > > > > > > and a timestamp string value.
> > > > > > > Of course, this can't solve 100% cases, but I guess can sovle 80% or 90%
> > > > > > > cases.
> > > > > > > And the remaining cases can be resolved by LIKE syntax which I guess is
> > > > > > not
> > > > > > > very common cases.
> > > > > > >
> > > > > > > Best,
> > > > > > > Jark
> > > > > > >
> > > > > > >
> > > > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:
> > > > > > >
> > > > > > > > Good to have such lovely discussions. I also want to share some of my
> > > > > > > > opinions.
> > > > > > > >
> > > > > > > > #1 Regarding to error handling: I also think ignore invalid hints would
> > > > > > > be
> > > > > > > > dangerous, maybe
> > > > > > > > the simplest solution is just throw an exception.
> > > > > > > >
> > > > > > > > #2 Regarding to property replacement: I don't think we should
> > > > > > constraint
> > > > > > > > ourself to
> > > > > > > > the meaning of the word "hint", and forbidden it modifying any
> > > > > > properties
> > > > > > > > which can effect
> > > > > > > > query results. IMO `PROPERTIES` is one of the table hints, and a
> > > > > > powerful
> > > > > > > > one. It can
> > > > > > > > modify properties located in DDL's WITH block. But I also see the harm
> > > > > > > that
> > > > > > > > if we make it
> > > > > > > > too flexible like change the kafka topic name with a hint. Such use
> > > > > > case
> > > > > > > is
> > > > > > > > not common and
> > > > > > > > sounds very dangerous to me. I would propose we have a map of hintable
> > > > > > > > properties for each
> > > > > > > > connector, and should validate all passed in properties are actually
> > > > > > > > hintable. And combining with
> > > > > > > > #1 error handling, we can throw an exception once received invalid
> > > > > > > > property.
> > > > > > > >
> > > > > > > > #3 Regarding to global offset: I'm not sure it's feasible. Different
> > > > > > > > connectors will have totally
> > > > > > > > different properties to represent offset, some might be timestamps,
> > > > > > some
> > > > > > > > might be string literals
> > > > > > > > like "earliest", and others might be just integers.
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Kurt
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
> > > > > > > >
> > > > > > > > > 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 <tw...@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 <tw...@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.lwl@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.cyz@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 <twalthr@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 <im...@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 <
> > > > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Aljoscha Krettek <al...@apache.org>.
Hi,

I don't understand this discussion. Hints, as I understand them, should 
work like this:

- hints are *optional* advice for the optimizer to try and help it to 
find a good execution strategy
- hints should not change query semantics, i.e. they should not change 
connector properties executing a query with taking into account the 
hints *must* produce the same result as executing the query without 
taking into account the hints

 From these simple requirements you can derive a solution that makes 
sense. I don't have a strong preference for the syntax but we should 
strive to be in line with prior work.

Best,
Aljoscha

On 11.03.20 11:53, Danny Chan wrote:
> Thanks Timo for summarize the 3 options ~
> 
> I agree with Kurt that option2 is too complicated to use because:
> 
> • As a Kafka topic consumer, the user must define both the virtual column for start offset and he must apply a special filter predicate after each query
> • And for the internal implementation, the metadata column push down is another hard topic, each kind of message queue may have its offset attribute, we need to consider the expression type for different kind; the source also need to recognize the constant column as a config option(which is weird because usually what we pushed down is a table column)
> 
> For option 1 and option3, I think there is no difference, option1 is also a hint syntax which is introduced in Sybase and referenced then deprecated by MS-SQL in 199X years because of the ambitiousness. Personally I prefer /*+ */ style table hint than WITH keyword for these reasons:
> 
> • We do not break the standard SQL, the hints are nested in SQL comments
> • We do not need to introduce additional WITH keyword which may appear in a query if we use that because a table can be referenced in all kinds of SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql query break too much of the SQL from standard
> • We would have uniform syntax for hints as query hint, one syntax fits all and more easy to use
> 
> 
> And here is the reason why we choose a uniform Oracle style query hint syntax which is addressed by Julian Hyde when we design the syntax from the Calcite community:
> 
> I don’t much like the MSSQL-style syntax for table hints. It adds a new use of the WITH keyword that is unrelated to the use of WITH for common-table expressions.
> 
> A historical note. Microsoft SQL Server inherited its hint syntax from Sybase a very long time ago. (See “Transact SQL Programming”[1], page 632, “Optimizer hints”. The book was written in 1999, and covers Microsoft SQL Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very likely predates Sybase 4.3, from which Microsoft SQL Server was forked in 1993.)
> 
> Microsoft later added the WITH keyword to make it less ambiguous, and has now deprecated the syntax that does not use WITH.
> 
> They are forced to keep the syntax for backwards compatibility but that doesn’t mean that we should shoulder their burden.
> 
> I think formatted comments are the right container for hints because it allows us to change the hint syntax without changing the SQL parser, and makes clear that we are at liberty to ignore hints entirely.
> 
> Julian
> 
> [1] https://www.amazon.com/s?k=9781565924017 <https://www.amazon.com/s?k=9781565924017>
> 
> Best,
> Danny Chan
> 在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
>> Hi Danny,
>>
>> it is true that our DDL is not standard compliant by using the WITH
>> clause. Nevertheless, we aim for not diverging too much and the LIKE
>> clause is an example of that. It will solve things like overwriting
>> WATERMARKs, add additional/modifying properties and inherit schema.
>>
>> Bowen is right that Flink's DDL is mixing 3 types definition together.
>> We are not the first ones that try to solve this. There is also the SQL
>> MED standard [1] that tried to tackle this problem. I think it was not
>> considered when designing the current DDL.
>>
>> Currently, I see 3 options for handling Kafka offsets. I will give some
>> examples and look forward to feedback here:
>>
>> *Option 1* Runtime and semantic parms as part of the query
>>
>> `SELECT * FROM MyTable('offset'=123)`
>>
>> Pros:
>> - Easy to add
>> - Parameters are part of the main query
>> - No complicated hinting syntax
>>
>> Cons:
>> - Not SQL compliant
>>
>> *Option 2* Use metadata in query
>>
>> `CREATE TABLE MyTable (id INT, offset AS SYSTEM_METADATA('offset'))`
>>
>> `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12 12:34:22'`
>>
>> Pros:
>> - SQL compliant in the query
>> - Access of metadata in the DDL which is required anyway
>> - Regular pushdown rules apply
>>
>> Cons:
>> - Users need to add an additional comlumn in the DDL
>>
>> *Option 3*: Use hints for properties
>>
>> `
>> SELECT *
>> FROM MyTable /*+ PROPERTIES('offset'=123) */
>> `
>>
>> Pros:
>> - Easy to add
>>
>> Cons:
>> - Parameters are not part of the main query
>> - Cryptic syntax for new users
>> - Not standard compliant.
>>
>> If we go with this option, I would suggest to make it available in a
>> separate map and don't mix it with statically defined properties. Such
>> that the factory can decide which properties have the right to be
>> overwritten by the hints:
>> TableSourceFactory.Context.getQueryHints(): ReadableConfig
>>
>> Regards,
>> Timo
>>
>> [1] https://en.wikipedia.org/wiki/SQL/MED
>>
>> Currently I see 3 options as a
>>
>>
>> On 11.03.20 07:21, Danny Chan wrote:
>>> Thanks Bowen ~
>>>
>>> I agree we should somehow categorize our connector parameters.
>>>
>>> For type1, I’m already preparing a solution like the Confluent schema registry + Avro schema inference thing, so this may not be a problem in the near future.
>>>
>>> For type3, I have some questions:
>>>
>>>> "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
>>>
>>> Where does the offset column come from, a virtual column from the table schema, you said that
>>>
>>>> They change
>>> almost every time a query starts and have nothing to do with metadata, thus
>>> should not be part of table definition/DDL
>>>
>>> But why you can reference it in the query, I’m confused for that, can you elaborate a little ?
>>>
>>> Best,
>>> Danny Chan
>>> 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
>>>> Thanks Danny for kicking off the effort
>>>>
>>>> The root cause of too much manual work is Flink DDL has mixed 3 types of
>>>> params together and doesn't handle each of them very well. Below are how I
>>>> categorize them and corresponding solutions in my mind:
>>>>
>>>> - type 1: Metadata of external data, like external endpoint/url,
>>>> username/pwd, schemas, formats.
>>>>
>>>> Such metadata are mostly already accessible in external system as long as
>>>> endpoints and credentials are provided. Flink can get it thru catalogs, but
>>>> we haven't had many catalogs yet and thus Flink just hasn't been able to
>>>> leverage that. So the solution should be building more catalogs. Such
>>>> params should be part of a Flink table DDL/definition, and not overridable
>>>> in any means.
>>>>
>>>>
>>>> - type 2: Runtime params, like jdbc connector's fetch size, elasticsearch
>>>> connector's bulk flush size.
>>>>
>>>> Such params don't affect query results, but affect how results are produced
>>>> (eg. fast or slow, aka performance) - they are essentially execution and
>>>> implementation details. They change often in exploration or development
>>>> stages, but not quite frequently in well-defined long-running pipelines.
>>>> They should always have default values and can be missing in query. They
>>>> can be part of a table DDL/definition, but should also be replaceable in a
>>>> query - *this is what table "hints" in FLIP-113 should cover*.
>>>>
>>>>
>>>> - type 3: Semantic params, like kafka connector's start offset.
>>>>
>>>> Such params affect query results - the semantics. They'd better be as
>>>> filter conditions in WHERE clause that can be pushed down. They change
>>>> almost every time a query starts and have nothing to do with metadata, thus
>>>> should not be part of table definition/DDL, nor be persisted in catalogs.
>>>> If they will, users should create views to keep such params around (note
>>>> this is different from variable substitution).
>>>>
>>>>
>>>> Take Flink-Kafka as an example. Once we get these params right, here're the
>>>> steps users need to do to develop and run a Flink job:
>>>> - configure a Flink ConfluentSchemaRegistry with url, username, and password
>>>> - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday" (simplified
>>>> timestamp) in SQL CLI, Flink automatically retrieves all metadata of
>>>> schema, file format, etc and start the job
>>>> - users want to make the job read Kafka topic faster, so it goes as "SELECT
>>>> * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm yesterday"
>>>> - done and satisfied, users submit it to production
>>>>
>>>>
>>>> Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's a
>>>> nice-to-have feature, but not a strategically critical, long-term solution,
>>>> because
>>>> 1) It may seem promising at the current stage to solve the
>>>> too-much-manual-work problem, but that's only because Flink hasn't
>>>> leveraged catalogs well and handled the 3 types of params above properly.
>>>> Once we get the params types right, the LIKE syntax won't be that
>>>> important, and will be just an easier way to create tables without retyping
>>>> long fields like username and pwd.
>>>> 2) Note that only some rare type of catalog can store k-v property pair, so
>>>> table created this way often cannot be persisted. In the foreseeable
>>>> future, such catalog will only be HiveCatalog, and not everyone has a Hive
>>>> metastore. To be honest, without persistence, recreating tables every time
>>>> this way is still a lot of keyboard typing.
>>>>
>>>> Cheers,
>>>> Bowen
>>>>
>>>> On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com> wrote:
>>>>
>>>>> If a specific connector want to have such parameter and read if out of
>>>>> configuration, then that's fine.
>>>>> If we are talking about a configuration for all kinds of sources, I would
>>>>> be super careful about that.
>>>>> It's true it can solve maybe 80% cases, but it will also make the left 20%
>>>>> feels weird.
>>>>>
>>>>> Best,
>>>>> Kurt
>>>>>
>>>>>
>>>>> On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:
>>>>>
>>>>>> Hi Kurt,
>>>>>>
>>>>>> #3 Regarding to global offset:
>>>>>> I'm not saying to use the global configuration to override connector
>>>>>> properties by the planner.
>>>>>> But the connector should take this configuration and translate into their
>>>>>> client API.
>>>>>> AFAIK, almost all the message queues support eariliest and latest and a
>>>>>> timestamp value as start point.
>>>>>> So we can support 3 options for this configuration: "eariliest", "latest"
>>>>>> and a timestamp string value.
>>>>>> Of course, this can't solve 100% cases, but I guess can sovle 80% or 90%
>>>>>> cases.
>>>>>> And the remaining cases can be resolved by LIKE syntax which I guess is
>>>>> not
>>>>>> very common cases.
>>>>>>
>>>>>> Best,
>>>>>> Jark
>>>>>>
>>>>>>
>>>>>> On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:
>>>>>>
>>>>>>> Good to have such lovely discussions. I also want to share some of my
>>>>>>> opinions.
>>>>>>>
>>>>>>> #1 Regarding to error handling: I also think ignore invalid hints would
>>>>>> be
>>>>>>> dangerous, maybe
>>>>>>> the simplest solution is just throw an exception.
>>>>>>>
>>>>>>> #2 Regarding to property replacement: I don't think we should
>>>>> constraint
>>>>>>> ourself to
>>>>>>> the meaning of the word "hint", and forbidden it modifying any
>>>>> properties
>>>>>>> which can effect
>>>>>>> query results. IMO `PROPERTIES` is one of the table hints, and a
>>>>> powerful
>>>>>>> one. It can
>>>>>>> modify properties located in DDL's WITH block. But I also see the harm
>>>>>> that
>>>>>>> if we make it
>>>>>>> too flexible like change the kafka topic name with a hint. Such use
>>>>> case
>>>>>> is
>>>>>>> not common and
>>>>>>> sounds very dangerous to me. I would propose we have a map of hintable
>>>>>>> properties for each
>>>>>>> connector, and should validate all passed in properties are actually
>>>>>>> hintable. And combining with
>>>>>>> #1 error handling, we can throw an exception once received invalid
>>>>>>> property.
>>>>>>>
>>>>>>> #3 Regarding to global offset: I'm not sure it's feasible. Different
>>>>>>> connectors will have totally
>>>>>>> different properties to represent offset, some might be timestamps,
>>>>> some
>>>>>>> might be string literals
>>>>>>> like "earliest", and others might be just integers.
>>>>>>>
>>>>>>> Best,
>>>>>>> Kurt
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
>>>>>>>
>>>>>>>> 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 <tw...@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 <tw...@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.lwl@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.cyz@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 <twalthr@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 <im...@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 <
>>>>>>>> kyledong@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.cyz@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.cyz@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
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>
>>
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
Thanks Timo for summarize the 3 options ~

I agree with Kurt that option2 is too complicated to use because:

• As a Kafka topic consumer, the user must define both the virtual column for start offset and he must apply a special filter predicate after each query
• And for the internal implementation, the metadata column push down is another hard topic, each kind of message queue may have its offset attribute, we need to consider the expression type for different kind; the source also need to recognize the constant column as a config option(which is weird because usually what we pushed down is a table column)

For option 1 and option3, I think there is no difference, option1 is also a hint syntax which is introduced in Sybase and referenced then deprecated by MS-SQL in 199X years because of the ambitiousness. Personally I prefer /*+ */ style table hint than WITH keyword for these reasons:

• We do not break the standard SQL, the hints are nested in SQL comments
• We do not need to introduce additional WITH keyword which may appear in a query if we use that because a table can be referenced in all kinds of SQL contexts: INSERT/DELETE/FROM/JOIN …. That would make our sql query break too much of the SQL from standard
• We would have uniform syntax for hints as query hint, one syntax fits all and more easy to use


And here is the reason why we choose a uniform Oracle style query hint syntax which is addressed by Julian Hyde when we design the syntax from the Calcite community:

I don’t much like the MSSQL-style syntax for table hints. It adds a new use of the WITH keyword that is unrelated to the use of WITH for common-table expressions.

A historical note. Microsoft SQL Server inherited its hint syntax from Sybase a very long time ago. (See “Transact SQL Programming”[1], page 632, “Optimizer hints”. The book was written in 1999, and covers Microsoft SQL Server 6.5 / 7.0 and Sybase Adaptive Server 11.5, but the syntax very likely predates Sybase 4.3, from which Microsoft SQL Server was forked in 1993.)

Microsoft later added the WITH keyword to make it less ambiguous, and has now deprecated the syntax that does not use WITH.

They are forced to keep the syntax for backwards compatibility but that doesn’t mean that we should shoulder their burden.

I think formatted comments are the right container for hints because it allows us to change the hint syntax without changing the SQL parser, and makes clear that we are at liberty to ignore hints entirely.

Julian

[1] https://www.amazon.com/s?k=9781565924017 <https://www.amazon.com/s?k=9781565924017>

Best,
Danny Chan
在 2020年3月11日 +0800 PM4:03,Timo Walther <tw...@apache.org>,写道:
> Hi Danny,
>
> it is true that our DDL is not standard compliant by using the WITH
> clause. Nevertheless, we aim for not diverging too much and the LIKE
> clause is an example of that. It will solve things like overwriting
> WATERMARKs, add additional/modifying properties and inherit schema.
>
> Bowen is right that Flink's DDL is mixing 3 types definition together.
> We are not the first ones that try to solve this. There is also the SQL
> MED standard [1] that tried to tackle this problem. I think it was not
> considered when designing the current DDL.
>
> Currently, I see 3 options for handling Kafka offsets. I will give some
> examples and look forward to feedback here:
>
> *Option 1* Runtime and semantic parms as part of the query
>
> `SELECT * FROM MyTable('offset'=123)`
>
> Pros:
> - Easy to add
> - Parameters are part of the main query
> - No complicated hinting syntax
>
> Cons:
> - Not SQL compliant
>
> *Option 2* Use metadata in query
>
> `CREATE TABLE MyTable (id INT, offset AS SYSTEM_METADATA('offset'))`
>
> `SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12 12:34:22'`
>
> Pros:
> - SQL compliant in the query
> - Access of metadata in the DDL which is required anyway
> - Regular pushdown rules apply
>
> Cons:
> - Users need to add an additional comlumn in the DDL
>
> *Option 3*: Use hints for properties
>
> `
> SELECT *
> FROM MyTable /*+ PROPERTIES('offset'=123) */
> `
>
> Pros:
> - Easy to add
>
> Cons:
> - Parameters are not part of the main query
> - Cryptic syntax for new users
> - Not standard compliant.
>
> If we go with this option, I would suggest to make it available in a
> separate map and don't mix it with statically defined properties. Such
> that the factory can decide which properties have the right to be
> overwritten by the hints:
> TableSourceFactory.Context.getQueryHints(): ReadableConfig
>
> Regards,
> Timo
>
> [1] https://en.wikipedia.org/wiki/SQL/MED
>
> Currently I see 3 options as a
>
>
> On 11.03.20 07:21, Danny Chan wrote:
> > Thanks Bowen ~
> >
> > I agree we should somehow categorize our connector parameters.
> >
> > For type1, I’m already preparing a solution like the Confluent schema registry + Avro schema inference thing, so this may not be a problem in the near future.
> >
> > For type3, I have some questions:
> >
> > > "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> >
> > Where does the offset column come from, a virtual column from the table schema, you said that
> >
> > > They change
> > almost every time a query starts and have nothing to do with metadata, thus
> > should not be part of table definition/DDL
> >
> > But why you can reference it in the query, I’m confused for that, can you elaborate a little ?
> >
> > Best,
> > Danny Chan
> > 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
> > > Thanks Danny for kicking off the effort
> > >
> > > The root cause of too much manual work is Flink DDL has mixed 3 types of
> > > params together and doesn't handle each of them very well. Below are how I
> > > categorize them and corresponding solutions in my mind:
> > >
> > > - type 1: Metadata of external data, like external endpoint/url,
> > > username/pwd, schemas, formats.
> > >
> > > Such metadata are mostly already accessible in external system as long as
> > > endpoints and credentials are provided. Flink can get it thru catalogs, but
> > > we haven't had many catalogs yet and thus Flink just hasn't been able to
> > > leverage that. So the solution should be building more catalogs. Such
> > > params should be part of a Flink table DDL/definition, and not overridable
> > > in any means.
> > >
> > >
> > > - type 2: Runtime params, like jdbc connector's fetch size, elasticsearch
> > > connector's bulk flush size.
> > >
> > > Such params don't affect query results, but affect how results are produced
> > > (eg. fast or slow, aka performance) - they are essentially execution and
> > > implementation details. They change often in exploration or development
> > > stages, but not quite frequently in well-defined long-running pipelines.
> > > They should always have default values and can be missing in query. They
> > > can be part of a table DDL/definition, but should also be replaceable in a
> > > query - *this is what table "hints" in FLIP-113 should cover*.
> > >
> > >
> > > - type 3: Semantic params, like kafka connector's start offset.
> > >
> > > Such params affect query results - the semantics. They'd better be as
> > > filter conditions in WHERE clause that can be pushed down. They change
> > > almost every time a query starts and have nothing to do with metadata, thus
> > > should not be part of table definition/DDL, nor be persisted in catalogs.
> > > If they will, users should create views to keep such params around (note
> > > this is different from variable substitution).
> > >
> > >
> > > Take Flink-Kafka as an example. Once we get these params right, here're the
> > > steps users need to do to develop and run a Flink job:
> > > - configure a Flink ConfluentSchemaRegistry with url, username, and password
> > > - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday" (simplified
> > > timestamp) in SQL CLI, Flink automatically retrieves all metadata of
> > > schema, file format, etc and start the job
> > > - users want to make the job read Kafka topic faster, so it goes as "SELECT
> > > * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm yesterday"
> > > - done and satisfied, users submit it to production
> > >
> > >
> > > Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's a
> > > nice-to-have feature, but not a strategically critical, long-term solution,
> > > because
> > > 1) It may seem promising at the current stage to solve the
> > > too-much-manual-work problem, but that's only because Flink hasn't
> > > leveraged catalogs well and handled the 3 types of params above properly.
> > > Once we get the params types right, the LIKE syntax won't be that
> > > important, and will be just an easier way to create tables without retyping
> > > long fields like username and pwd.
> > > 2) Note that only some rare type of catalog can store k-v property pair, so
> > > table created this way often cannot be persisted. In the foreseeable
> > > future, such catalog will only be HiveCatalog, and not everyone has a Hive
> > > metastore. To be honest, without persistence, recreating tables every time
> > > this way is still a lot of keyboard typing.
> > >
> > > Cheers,
> > > Bowen
> > >
> > > On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com> wrote:
> > >
> > > > If a specific connector want to have such parameter and read if out of
> > > > configuration, then that's fine.
> > > > If we are talking about a configuration for all kinds of sources, I would
> > > > be super careful about that.
> > > > It's true it can solve maybe 80% cases, but it will also make the left 20%
> > > > feels weird.
> > > >
> > > > Best,
> > > > Kurt
> > > >
> > > >
> > > > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:
> > > >
> > > > > Hi Kurt,
> > > > >
> > > > > #3 Regarding to global offset:
> > > > > I'm not saying to use the global configuration to override connector
> > > > > properties by the planner.
> > > > > But the connector should take this configuration and translate into their
> > > > > client API.
> > > > > AFAIK, almost all the message queues support eariliest and latest and a
> > > > > timestamp value as start point.
> > > > > So we can support 3 options for this configuration: "eariliest", "latest"
> > > > > and a timestamp string value.
> > > > > Of course, this can't solve 100% cases, but I guess can sovle 80% or 90%
> > > > > cases.
> > > > > And the remaining cases can be resolved by LIKE syntax which I guess is
> > > > not
> > > > > very common cases.
> > > > >
> > > > > Best,
> > > > > Jark
> > > > >
> > > > >
> > > > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:
> > > > >
> > > > > > Good to have such lovely discussions. I also want to share some of my
> > > > > > opinions.
> > > > > >
> > > > > > #1 Regarding to error handling: I also think ignore invalid hints would
> > > > > be
> > > > > > dangerous, maybe
> > > > > > the simplest solution is just throw an exception.
> > > > > >
> > > > > > #2 Regarding to property replacement: I don't think we should
> > > > constraint
> > > > > > ourself to
> > > > > > the meaning of the word "hint", and forbidden it modifying any
> > > > properties
> > > > > > which can effect
> > > > > > query results. IMO `PROPERTIES` is one of the table hints, and a
> > > > powerful
> > > > > > one. It can
> > > > > > modify properties located in DDL's WITH block. But I also see the harm
> > > > > that
> > > > > > if we make it
> > > > > > too flexible like change the kafka topic name with a hint. Such use
> > > > case
> > > > > is
> > > > > > not common and
> > > > > > sounds very dangerous to me. I would propose we have a map of hintable
> > > > > > properties for each
> > > > > > connector, and should validate all passed in properties are actually
> > > > > > hintable. And combining with
> > > > > > #1 error handling, we can throw an exception once received invalid
> > > > > > property.
> > > > > >
> > > > > > #3 Regarding to global offset: I'm not sure it's feasible. Different
> > > > > > connectors will have totally
> > > > > > different properties to represent offset, some might be timestamps,
> > > > some
> > > > > > might be string literals
> > > > > > like "earliest", and others might be just integers.
> > > > > >
> > > > > > Best,
> > > > > > Kurt
> > > > > >
> > > > > >
> > > > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
> > > > > >
> > > > > > > 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 <tw...@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 <tw...@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.lwl@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.cyz@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 <twalthr@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 <im...@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 <
> > > > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@apache.org>.
Hi Danny,

it is true that our DDL is not standard compliant by using the WITH 
clause. Nevertheless, we aim for not diverging too much and the LIKE 
clause is an example of that. It will solve things like overwriting 
WATERMARKs, add additional/modifying properties and inherit schema.

Bowen is right that Flink's DDL is mixing 3 types definition together. 
We are not the first ones that try to solve this. There is also the SQL 
MED standard [1] that tried to tackle this problem. I think it was not 
considered when designing the current DDL.

Currently, I see 3 options for handling Kafka offsets. I will give some 
examples and look forward to feedback here:

*Option 1* Runtime and semantic parms as part of the query

`SELECT * FROM MyTable('offset'=123)`

Pros:
- Easy to add
- Parameters are part of the main query
- No complicated hinting syntax

Cons:
- Not SQL compliant

*Option 2* Use metadata in query

`CREATE TABLE MyTable (id INT, offset AS SYSTEM_METADATA('offset'))`

`SELECT * FROM MyTable WHERE offset > TIMESTAMP '2012-12-12 12:34:22'`

Pros:
- SQL compliant in the query
- Access of metadata in the DDL which is required anyway
- Regular pushdown rules apply

Cons:
- Users need to add an additional comlumn in the DDL

*Option 3*: Use hints for properties

`
SELECT *
FROM MyTable /*+ PROPERTIES('offset'=123) */
`

Pros:
- Easy to add

Cons:
- Parameters are not part of the main query
- Cryptic syntax for new users
- Not standard compliant.

If we go with this option, I would suggest to make it available in a 
separate map and don't mix it with statically defined properties. Such 
that the factory can decide which properties have the right to be 
overwritten by the hints:
TableSourceFactory.Context.getQueryHints(): ReadableConfig

Regards,
Timo

[1] https://en.wikipedia.org/wiki/SQL/MED

Currently I see 3 options as a


On 11.03.20 07:21, Danny Chan wrote:
> Thanks Bowen ~
> 
> I agree we should somehow categorize our connector parameters.
> 
> For type1, I’m already preparing a solution like the Confluent schema registry + Avro schema inference thing, so this may not be a problem in the near future.
> 
> For type3, I have some questions:
> 
>> "SELECT * FROM mykafka WHERE offset > 12pm yesterday”
> 
> Where does the offset column come from, a virtual column from the table schema, you said that
> 
>> They change
> almost every time a query starts and have nothing to do with metadata, thus
> should not be part of table definition/DDL
> 
> But why you can reference it in the query, I’m confused for that, can you elaborate a little ?
> 
> Best,
> Danny Chan
> 在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
>> Thanks Danny for kicking off the effort
>>
>> The root cause of too much manual work is Flink DDL has mixed 3 types of
>> params together and doesn't handle each of them very well. Below are how I
>> categorize them and corresponding solutions in my mind:
>>
>> - type 1: Metadata of external data, like external endpoint/url,
>> username/pwd, schemas, formats.
>>
>> Such metadata are mostly already accessible in external system as long as
>> endpoints and credentials are provided. Flink can get it thru catalogs, but
>> we haven't had many catalogs yet and thus Flink just hasn't been able to
>> leverage that. So the solution should be building more catalogs. Such
>> params should be part of a Flink table DDL/definition, and not overridable
>> in any means.
>>
>>
>> - type 2: Runtime params, like jdbc connector's fetch size, elasticsearch
>> connector's bulk flush size.
>>
>> Such params don't affect query results, but affect how results are produced
>> (eg. fast or slow, aka performance) - they are essentially execution and
>> implementation details. They change often in exploration or development
>> stages, but not quite frequently in well-defined long-running pipelines.
>> They should always have default values and can be missing in query. They
>> can be part of a table DDL/definition, but should also be replaceable in a
>> query - *this is what table "hints" in FLIP-113 should cover*.
>>
>>
>> - type 3: Semantic params, like kafka connector's start offset.
>>
>> Such params affect query results - the semantics. They'd better be as
>> filter conditions in WHERE clause that can be pushed down. They change
>> almost every time a query starts and have nothing to do with metadata, thus
>> should not be part of table definition/DDL, nor be persisted in catalogs.
>> If they will, users should create views to keep such params around (note
>> this is different from variable substitution).
>>
>>
>> Take Flink-Kafka as an example. Once we get these params right, here're the
>> steps users need to do to develop and run a Flink job:
>> - configure a Flink ConfluentSchemaRegistry with url, username, and password
>> - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday" (simplified
>> timestamp) in SQL CLI, Flink automatically retrieves all metadata of
>> schema, file format, etc and start the job
>> - users want to make the job read Kafka topic faster, so it goes as "SELECT
>> * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm yesterday"
>> - done and satisfied, users submit it to production
>>
>>
>> Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's a
>> nice-to-have feature, but not a strategically critical, long-term solution,
>> because
>> 1) It may seem promising at the current stage to solve the
>> too-much-manual-work problem, but that's only because Flink hasn't
>> leveraged catalogs well and handled the 3 types of params above properly.
>> Once we get the params types right, the LIKE syntax won't be that
>> important, and will be just an easier way to create tables without retyping
>> long fields like username and pwd.
>> 2) Note that only some rare type of catalog can store k-v property pair, so
>> table created this way often cannot be persisted. In the foreseeable
>> future, such catalog will only be HiveCatalog, and not everyone has a Hive
>> metastore. To be honest, without persistence, recreating tables every time
>> this way is still a lot of keyboard typing.
>>
>> Cheers,
>> Bowen
>>
>> On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com> wrote:
>>
>>> If a specific connector want to have such parameter and read if out of
>>> configuration, then that's fine.
>>> If we are talking about a configuration for all kinds of sources, I would
>>> be super careful about that.
>>> It's true it can solve maybe 80% cases, but it will also make the left 20%
>>> feels weird.
>>>
>>> Best,
>>> Kurt
>>>
>>>
>>> On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:
>>>
>>>> Hi Kurt,
>>>>
>>>> #3 Regarding to global offset:
>>>> I'm not saying to use the global configuration to override connector
>>>> properties by the planner.
>>>> But the connector should take this configuration and translate into their
>>>> client API.
>>>> AFAIK, almost all the message queues support eariliest and latest and a
>>>> timestamp value as start point.
>>>> So we can support 3 options for this configuration: "eariliest", "latest"
>>>> and a timestamp string value.
>>>> Of course, this can't solve 100% cases, but I guess can sovle 80% or 90%
>>>> cases.
>>>> And the remaining cases can be resolved by LIKE syntax which I guess is
>>> not
>>>> very common cases.
>>>>
>>>> Best,
>>>> Jark
>>>>
>>>>
>>>> On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:
>>>>
>>>>> Good to have such lovely discussions. I also want to share some of my
>>>>> opinions.
>>>>>
>>>>> #1 Regarding to error handling: I also think ignore invalid hints would
>>>> be
>>>>> dangerous, maybe
>>>>> the simplest solution is just throw an exception.
>>>>>
>>>>> #2 Regarding to property replacement: I don't think we should
>>> constraint
>>>>> ourself to
>>>>> the meaning of the word "hint", and forbidden it modifying any
>>> properties
>>>>> which can effect
>>>>> query results. IMO `PROPERTIES` is one of the table hints, and a
>>> powerful
>>>>> one. It can
>>>>> modify properties located in DDL's WITH block. But I also see the harm
>>>> that
>>>>> if we make it
>>>>> too flexible like change the kafka topic name with a hint. Such use
>>> case
>>>> is
>>>>> not common and
>>>>> sounds very dangerous to me. I would propose we have a map of hintable
>>>>> properties for each
>>>>> connector, and should validate all passed in properties are actually
>>>>> hintable. And combining with
>>>>> #1 error handling, we can throw an exception once received invalid
>>>>> property.
>>>>>
>>>>> #3 Regarding to global offset: I'm not sure it's feasible. Different
>>>>> connectors will have totally
>>>>> different properties to represent offset, some might be timestamps,
>>> some
>>>>> might be string literals
>>>>> like "earliest", and others might be just integers.
>>>>>
>>>>> Best,
>>>>> Kurt
>>>>>
>>>>>
>>>>> On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
>>>>>
>>>>>> 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 <tw...@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 <tw...@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.lwl@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.cyz@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 <twalthr@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 <im...@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 <
>>>>>> kyledong@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.cyz@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.cyz@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
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
Thanks Bowen ~

I agree we should somehow categorize our connector parameters.

For type1, I’m already preparing a solution like the Confluent schema registry + Avro schema inference thing, so this may not be a problem in the near future.

For type3, I have some questions:

> "SELECT * FROM mykafka WHERE offset > 12pm yesterday”

Where does the offset column come from, a virtual column from the table schema, you said that

>They change
almost every time a query starts and have nothing to do with metadata, thus
should not be part of table definition/DDL

But why you can reference it in the query, I’m confused for that, can you elaborate a little ?

Best,
Danny Chan
在 2020年3月11日 +0800 PM12:52,Bowen Li <bo...@gmail.com>,写道:
> Thanks Danny for kicking off the effort
>
> The root cause of too much manual work is Flink DDL has mixed 3 types of
> params together and doesn't handle each of them very well. Below are how I
> categorize them and corresponding solutions in my mind:
>
> - type 1: Metadata of external data, like external endpoint/url,
> username/pwd, schemas, formats.
>
> Such metadata are mostly already accessible in external system as long as
> endpoints and credentials are provided. Flink can get it thru catalogs, but
> we haven't had many catalogs yet and thus Flink just hasn't been able to
> leverage that. So the solution should be building more catalogs. Such
> params should be part of a Flink table DDL/definition, and not overridable
> in any means.
>
>
> - type 2: Runtime params, like jdbc connector's fetch size, elasticsearch
> connector's bulk flush size.
>
> Such params don't affect query results, but affect how results are produced
> (eg. fast or slow, aka performance) - they are essentially execution and
> implementation details. They change often in exploration or development
> stages, but not quite frequently in well-defined long-running pipelines.
> They should always have default values and can be missing in query. They
> can be part of a table DDL/definition, but should also be replaceable in a
> query - *this is what table "hints" in FLIP-113 should cover*.
>
>
> - type 3: Semantic params, like kafka connector's start offset.
>
> Such params affect query results - the semantics. They'd better be as
> filter conditions in WHERE clause that can be pushed down. They change
> almost every time a query starts and have nothing to do with metadata, thus
> should not be part of table definition/DDL, nor be persisted in catalogs.
> If they will, users should create views to keep such params around (note
> this is different from variable substitution).
>
>
> Take Flink-Kafka as an example. Once we get these params right, here're the
> steps users need to do to develop and run a Flink job:
> - configure a Flink ConfluentSchemaRegistry with url, username, and password
> - run "SELECT * FROM mykafka WHERE offset > 12pm yesterday" (simplified
> timestamp) in SQL CLI, Flink automatically retrieves all metadata of
> schema, file format, etc and start the job
> - users want to make the job read Kafka topic faster, so it goes as "SELECT
> * FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm yesterday"
> - done and satisfied, users submit it to production
>
>
> Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's a
> nice-to-have feature, but not a strategically critical, long-term solution,
> because
> 1) It may seem promising at the current stage to solve the
> too-much-manual-work problem, but that's only because Flink hasn't
> leveraged catalogs well and handled the 3 types of params above properly.
> Once we get the params types right, the LIKE syntax won't be that
> important, and will be just an easier way to create tables without retyping
> long fields like username and pwd.
> 2) Note that only some rare type of catalog can store k-v property pair, so
> table created this way often cannot be persisted. In the foreseeable
> future, such catalog will only be HiveCatalog, and not everyone has a Hive
> metastore. To be honest, without persistence, recreating tables every time
> this way is still a lot of keyboard typing.
>
> Cheers,
> Bowen
>
> On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com> wrote:
>
> > If a specific connector want to have such parameter and read if out of
> > configuration, then that's fine.
> > If we are talking about a configuration for all kinds of sources, I would
> > be super careful about that.
> > It's true it can solve maybe 80% cases, but it will also make the left 20%
> > feels weird.
> >
> > Best,
> > Kurt
> >
> >
> > On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:
> >
> > > Hi Kurt,
> > >
> > > #3 Regarding to global offset:
> > > I'm not saying to use the global configuration to override connector
> > > properties by the planner.
> > > But the connector should take this configuration and translate into their
> > > client API.
> > > AFAIK, almost all the message queues support eariliest and latest and a
> > > timestamp value as start point.
> > > So we can support 3 options for this configuration: "eariliest", "latest"
> > > and a timestamp string value.
> > > Of course, this can't solve 100% cases, but I guess can sovle 80% or 90%
> > > cases.
> > > And the remaining cases can be resolved by LIKE syntax which I guess is
> > not
> > > very common cases.
> > >
> > > Best,
> > > Jark
> > >
> > >
> > > On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:
> > >
> > > > Good to have such lovely discussions. I also want to share some of my
> > > > opinions.
> > > >
> > > > #1 Regarding to error handling: I also think ignore invalid hints would
> > > be
> > > > dangerous, maybe
> > > > the simplest solution is just throw an exception.
> > > >
> > > > #2 Regarding to property replacement: I don't think we should
> > constraint
> > > > ourself to
> > > > the meaning of the word "hint", and forbidden it modifying any
> > properties
> > > > which can effect
> > > > query results. IMO `PROPERTIES` is one of the table hints, and a
> > powerful
> > > > one. It can
> > > > modify properties located in DDL's WITH block. But I also see the harm
> > > that
> > > > if we make it
> > > > too flexible like change the kafka topic name with a hint. Such use
> > case
> > > is
> > > > not common and
> > > > sounds very dangerous to me. I would propose we have a map of hintable
> > > > properties for each
> > > > connector, and should validate all passed in properties are actually
> > > > hintable. And combining with
> > > > #1 error handling, we can throw an exception once received invalid
> > > > property.
> > > >
> > > > #3 Regarding to global offset: I'm not sure it's feasible. Different
> > > > connectors will have totally
> > > > different properties to represent offset, some might be timestamps,
> > some
> > > > might be string literals
> > > > like "earliest", and others might be just integers.
> > > >
> > > > Best,
> > > > Kurt
> > > >
> > > >
> > > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
> > > >
> > > > > 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 <tw...@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 <tw...@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.lwl@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.cyz@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 <twalthr@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 <im...@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 <
> > > > > kyledong@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.cyz@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.cyz@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
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Bowen Li <bo...@gmail.com>.
Thanks Danny for kicking off the effort

The root cause of too much manual work is Flink DDL has mixed 3 types of
params together and doesn't handle each of them very well. Below are how I
categorize them and corresponding solutions in my mind:

- type 1: Metadata of external data, like external endpoint/url,
username/pwd, schemas, formats.

Such metadata are mostly already accessible in external system as long as
endpoints and credentials are provided. Flink can get it thru catalogs, but
we haven't had many catalogs yet and thus Flink just hasn't been able to
leverage that. So the solution should be building more catalogs. Such
params should be part of a Flink table DDL/definition, and not overridable
in any means.


- type 2: Runtime params, like jdbc connector's fetch size, elasticsearch
connector's bulk flush size.

Such params don't affect query results, but affect how results are produced
(eg. fast or slow, aka performance) - they are essentially execution and
implementation details. They change often in exploration or development
stages, but not quite frequently in well-defined long-running pipelines.
They should always have default values and can be missing in query. They
can be part of a table DDL/definition, but should also be replaceable in a
query - *this is what table "hints" in FLIP-113 should cover*.


- type 3: Semantic params, like kafka connector's start offset.

Such params affect query results - the semantics. They'd better be as
filter conditions in WHERE clause that can be pushed down. They change
almost every time a query starts and have nothing to do with metadata, thus
should not be part of table definition/DDL, nor be persisted in catalogs.
If they will, users should create views to keep such params around (note
this is different from variable substitution).


Take Flink-Kafka as an example. Once we get these params right, here're the
steps users need to do to develop and run a Flink job:
- configure a Flink ConfluentSchemaRegistry with url, username, and password
- run "SELECT * FROM mykafka WHERE offset > 12pm yesterday" (simplified
timestamp) in SQL CLI, Flink automatically retrieves all metadata of
schema, file format, etc and start the job
- users want to make the job read Kafka topic faster, so it goes as "SELECT
* FROM mykafka /* faster_read_key=value*/ WHERE offset > 12pm yesterday"
- done and satisfied, users submit it to production


Regarding "CREATE TABLE t LIKE with (k1=v1, k2=v2), I think it's a
nice-to-have feature, but not a strategically critical, long-term solution,
because
1) It may seem promising at the current stage to solve the
too-much-manual-work problem, but that's only because Flink hasn't
leveraged catalogs well and handled the 3 types of params above properly.
Once we get the params types right, the LIKE syntax won't be that
important, and will be just an easier way to create tables without retyping
long fields like username and pwd.
2) Note that only some rare type of catalog can store k-v property pair, so
table created this way often cannot be persisted. In the foreseeable
future, such catalog will only be HiveCatalog, and not everyone has a Hive
metastore. To be honest, without persistence, recreating tables every time
this way is still a lot of keyboard typing.

Cheers,
Bowen

On Tue, Mar 10, 2020 at 8:07 PM Kurt Young <yk...@gmail.com> wrote:

> If a specific connector want to have such parameter and read if out of
> configuration, then that's fine.
> If we are talking about a configuration for all kinds of sources, I would
> be super careful about that.
> It's true it can solve maybe 80% cases, but it will also make the left 20%
> feels weird.
>
> Best,
> Kurt
>
>
> On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:
>
> > Hi Kurt,
> >
> > #3 Regarding to global offset:
> > I'm not saying to use the global configuration to override connector
> > properties by the planner.
> > But the connector should take this configuration and translate into their
> > client API.
> > AFAIK, almost all the message queues support eariliest and latest and a
> > timestamp value as start point.
> > So we can support 3 options for this configuration: "eariliest", "latest"
> > and a timestamp string value.
> > Of course, this can't solve 100% cases, but I guess can sovle 80% or 90%
> > cases.
> > And the remaining cases can be resolved by LIKE syntax which I guess is
> not
> > very common cases.
> >
> > Best,
> > Jark
> >
> >
> > On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:
> >
> > > Good to have such lovely discussions. I also want to share some of my
> > > opinions.
> > >
> > > #1 Regarding to error handling: I also think ignore invalid hints would
> > be
> > > dangerous, maybe
> > > the simplest solution is just throw an exception.
> > >
> > > #2 Regarding to property replacement: I don't think we should
> constraint
> > > ourself to
> > > the meaning of the word "hint", and forbidden it modifying any
> properties
> > > which can effect
> > > query results. IMO `PROPERTIES` is one of the table hints, and a
> powerful
> > > one. It can
> > > modify properties located in DDL's WITH block. But I also see the harm
> > that
> > > if we make it
> > > too flexible like change the kafka topic name with a hint. Such use
> case
> > is
> > > not common and
> > > sounds very dangerous to me. I would propose we have a map of hintable
> > > properties for each
> > > connector, and should validate all passed in properties are actually
> > > hintable. And combining with
> > > #1 error handling, we can throw an exception once received invalid
> > > property.
> > >
> > > #3 Regarding to global offset: I'm not sure it's feasible. Different
> > > connectors will have totally
> > > different properties to represent offset, some might be timestamps,
> some
> > > might be string literals
> > > like "earliest", and others might be just integers.
> > >
> > > Best,
> > > Kurt
> > >
> > >
> > > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
> > >
> > > > 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 <tw...@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 <tw...@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.lwl@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.cyz@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 <twalthr@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 <im...@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 <
> > > > kyledong@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.cyz@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.cyz@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
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>>
> > > > > >>
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Kurt Young <yk...@gmail.com>.
If a specific connector want to have such parameter and read if out of
configuration, then that's fine.
If we are talking about a configuration for all kinds of sources, I would
be super careful about that.
It's true it can solve maybe 80% cases, but it will also make the left 20%
feels weird.

Best,
Kurt


On Wed, Mar 11, 2020 at 11:00 AM Jark Wu <im...@gmail.com> wrote:

> Hi Kurt,
>
> #3 Regarding to global offset:
> I'm not saying to use the global configuration to override connector
> properties by the planner.
> But the connector should take this configuration and translate into their
> client API.
> AFAIK, almost all the message queues support eariliest and latest and a
> timestamp value as start point.
> So we can support 3 options for this configuration: "eariliest", "latest"
> and a timestamp string value.
> Of course, this can't solve 100% cases, but I guess can sovle 80% or 90%
> cases.
> And the remaining cases can be resolved by LIKE syntax which I guess is not
> very common cases.
>
> Best,
> Jark
>
>
> On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:
>
> > Good to have such lovely discussions. I also want to share some of my
> > opinions.
> >
> > #1 Regarding to error handling: I also think ignore invalid hints would
> be
> > dangerous, maybe
> > the simplest solution is just throw an exception.
> >
> > #2 Regarding to property replacement: I don't think we should constraint
> > ourself to
> > the meaning of the word "hint", and forbidden it modifying any properties
> > which can effect
> > query results. IMO `PROPERTIES` is one of the table hints, and a powerful
> > one. It can
> > modify properties located in DDL's WITH block. But I also see the harm
> that
> > if we make it
> > too flexible like change the kafka topic name with a hint. Such use case
> is
> > not common and
> > sounds very dangerous to me. I would propose we have a map of hintable
> > properties for each
> > connector, and should validate all passed in properties are actually
> > hintable. And combining with
> > #1 error handling, we can throw an exception once received invalid
> > property.
> >
> > #3 Regarding to global offset: I'm not sure it's feasible. Different
> > connectors will have totally
> > different properties to represent offset, some might be timestamps, some
> > might be string literals
> > like "earliest", and others might be just integers.
> >
> > Best,
> > Kurt
> >
> >
> > On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
> >
> > > 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 <tw...@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 <tw...@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.lwl@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 <yu...@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 <tw...@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 <im...@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 <
> > > kyledong@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.cyz@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.cyz@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
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>
> > > > >>
> > > > >
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jark Wu <im...@gmail.com>.
Hi Kurt,

#3 Regarding to global offset:
I'm not saying to use the global configuration to override connector
properties by the planner.
But the connector should take this configuration and translate into their
client API.
AFAIK, almost all the message queues support eariliest and latest and a
timestamp value as start point.
So we can support 3 options for this configuration: "eariliest", "latest"
and a timestamp string value.
Of course, this can't solve 100% cases, but I guess can sovle 80% or 90%
cases.
And the remaining cases can be resolved by LIKE syntax which I guess is not
very common cases.

Best,
Jark


On Wed, 11 Mar 2020 at 10:33, Kurt Young <yk...@gmail.com> wrote:

> Good to have such lovely discussions. I also want to share some of my
> opinions.
>
> #1 Regarding to error handling: I also think ignore invalid hints would be
> dangerous, maybe
> the simplest solution is just throw an exception.
>
> #2 Regarding to property replacement: I don't think we should constraint
> ourself to
> the meaning of the word "hint", and forbidden it modifying any properties
> which can effect
> query results. IMO `PROPERTIES` is one of the table hints, and a powerful
> one. It can
> modify properties located in DDL's WITH block. But I also see the harm that
> if we make it
> too flexible like change the kafka topic name with a hint. Such use case is
> not common and
> sounds very dangerous to me. I would propose we have a map of hintable
> properties for each
> connector, and should validate all passed in properties are actually
> hintable. And combining with
> #1 error handling, we can throw an exception once received invalid
> property.
>
> #3 Regarding to global offset: I'm not sure it's feasible. Different
> connectors will have totally
> different properties to represent offset, some might be timestamps, some
> might be string literals
> like "earliest", and others might be just integers.
>
> Best,
> Kurt
>
>
> On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:
>
> > 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 <tw...@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 <tw...@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.lwl@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 <yu...@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 <tw...@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 <im...@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 <
> > kyledong@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.cyz@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.cyz@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
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>
> > > >>
> > > >
> > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Kurt Young <yk...@gmail.com>.
Good to have such lovely discussions. I also want to share some of my
opinions.

#1 Regarding to error handling: I also think ignore invalid hints would be
dangerous, maybe
the simplest solution is just throw an exception.

#2 Regarding to property replacement: I don't think we should constraint
ourself to
the meaning of the word "hint", and forbidden it modifying any properties
which can effect
query results. IMO `PROPERTIES` is one of the table hints, and a powerful
one. It can
modify properties located in DDL's WITH block. But I also see the harm that
if we make it
too flexible like change the kafka topic name with a hint. Such use case is
not common and
sounds very dangerous to me. I would propose we have a map of hintable
properties for each
connector, and should validate all passed in properties are actually
hintable. And combining with
#1 error handling, we can throw an exception once received invalid property.

#3 Regarding to global offset: I'm not sure it's feasible. Different
connectors will have totally
different properties to represent offset, some might be timestamps, some
might be string literals
like "earliest", and others might be just integers.

Best,
Kurt


On Tue, Mar 10, 2020 at 11:46 PM Jark Wu <im...@gmail.com> wrote:

> 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 <tw...@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 <tw...@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 <we...@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 <yu...@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 <tw...@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 <im...@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 <
> kyledong@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.cyz@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.cyz@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
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>
> > >>
> > >
> >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jark Wu <im...@gmail.com>.
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 <tw...@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 <tw...@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 <we...@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 <yu...@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 <tw...@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 <im...@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 <kyledong@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 <yu...@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 <yu...@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
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
> >
>
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@apache.org>.
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 <tw...@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 <we...@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 <yu...@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 <tw...@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 <im...@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 <ky...@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 <yu...@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 <yu...@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
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>
>>
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
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 <tw...@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 <we...@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 <yu...@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 <tw...@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 <im...@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 <ky...@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 <yu...@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 <yu...@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
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@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 <we...@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 <yu...@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 <tw...@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 <im...@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 <ky...@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 <yu...@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 <yu...@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
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>
>>>
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
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 <we...@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 <yu...@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 <tw...@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 <im...@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 <ky...@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 <yu...@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 <yu...@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
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > >
> >

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by "wenlong.lwl" <we...@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 <yu...@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 <tw...@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 <im...@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 <ky...@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 <yu...@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 <yu...@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
> > > > > >
> > > > >
> > > >
> > >
> > >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
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 <tw...@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 <im...@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 <ky...@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 <yu...@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 <yu...@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
> > > > >
> > > >
> > >
> >
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Timo Walther <tw...@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 <im...@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 <ky...@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 <yu...@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 <yu...@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
>>>>
>>>
>>
> 
> 


Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jingsong Li <ji...@gmail.com>.
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 <im...@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 <ky...@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 <yu...@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 <yu...@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
> > >
> >
>


-- 
Best, Jingsong Lee

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Jark Wu <im...@gmail.com>.
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 <ky...@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 <yu...@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 <yu...@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
> >
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by "DONG, Weike" <ky...@connect.hku.hk>.
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 <yu...@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 <yu...@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
>

Re: [DISCUSS]FLIP-113: Support SQL and planner hints

Posted by Danny Chan <yu...@gmail.com>.
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 <yu...@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