hi kaibo,
As we discuss offline, I think it's a clean way that flink-table provides
an interface (or a tool) to do the sql validation for platform users.
`tEnv.sqlUpdate` or `tEnv.explain(false)` is a temporary solution which
contains too many unrelated logic (just consider the functionality whether
a sql is valid).

Best,
godfrey



Arvid Heise <ar...@ververica.com> 于2020年1月8日周三 下午9:40写道:

> A common approach is to add the connector jar as test dependencies and have
> a smoke test that just starts the job with a temporary external system
> spawned with docker. I usually use test containers [1]. Then you simply
> need to execute the integration tests in your IDE and usually can even
> debug non-obvious errors.
>
> [1] https://www.testcontainers.org/
>
> On Mon, Dec 30, 2019 at 1:39 PM Kaibo Zhou <zkb...@gmail.com> wrote:
>
> > Hi, Jingsong,
> >
> > Thank you very much for your suggestion.
> >
> > I verified that use `tEnv.sqlUpdate("xxx")` and `tEnv.explain(false)` to
> do
> > validation, it works.
> > But this method needs the connector jar, which is very inconvenient to
> use.
> >
> >
> > Hi, Danny,
> >
> > Many thanks for providing very useful explanations.
> >
> > The user case is users will register some source/sink tables, udf to
> > catalog service first, and then they will write and modify SQL like
> "insert
> > into sinkTable select * from sourceTable where a>1" on Web SQLEditor. The
> > platform wants to tell the user whether the SQL is valid includes the
> > detailed position if an error occurs.
> >
> > For the `insert target table`, the platform wants to validate the table
> > exists, field name and field type.
> >
> > Best,
> > Kaibo
> >
> > Danny Chan <yuzhao....@gmail.com> 于2019年12月30日周一 下午5:37写道:
> >
> > > Hi, Kaibo Zhou ~
> > >
> > > There are several phrases that a SQL text get to execution graph what
> can
> > > be run with Flink runtime:
> > >
> > >
> > > 1. Sql Parse: parse the sql text to AST(sql node tree)
> > > 2. Sql node(row type) validation, this includes the tables/schema
> > inference
> > > 3. Sql-to-rel conversion, convert the sql node to RelNode(relational
> > > algebra)
> > > 4. Promote the relational expression with planner(Volcano or Hep) then
> > > converts to execution convention nodes
> > > 5. Genegate the code and the execution graph
> > >
> > > For the first 3 steps, Apache Flink uses the Apache Calcite as the
> > > implementation, that means a SQL test passed to table environment would
> > > always have a SQL parse/validation/sql-to-rel conversion.
> > >
> > > For example, a code snippet like tableEnv.sqlQuery("INSERT INTO
> sinkTable
> > > SELECT f1,f2 FROM sourceTable”), the query part “SELECT f1,f2 FROM
> > > sourceTable” was validated.
> > >
> > > But you are right, for Flink SQL, an insert statement target table is
> not
> > > validated during the validation phrase, actually we validate the
> “select”
> > > clause first, extract the target table identifier and we validate the
> > > schema of “select” clause and target table are the same when we invoke
> > > write to sink(after step 4).
> > >
> > >
> > > For most of the cases this is okey, can you share your cases ? What
> kind
> > > of validation do you want for the insert target table ?
> > >
> > > We are planning to include the insert target table validation in the
> > step2
> > > for 2 reasons:
> > >
> > > • The computed column validation(stored or virtual)
> > > • The insert implicit type coercion
> > >
> > > But this would comes for Flink version 1.11 ~
> > >
> > >
> > > Best,
> > > Danny Chan
> > > 在 2019年12月27日 +0800 PM5:44,dev@flink.apache.org,写道:
> > > >
> > > > "INSERT INTO
> > > > sinkTable SELECT f1,f2 FROM sourceTable"
> > >
> >
>

Reply via email to