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

Reply via email to