twalthr commented on code in PR #24769:
URL: https://github.com/apache/flink/pull/24769#discussion_r1609502734


##########
flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl:
##########
@@ -3037,3 +3067,185 @@ SqlTruncateTable SqlTruncateTable() :
         return new SqlTruncateTable(getPos(), sqlIdentifier);
     }
 }
+
+/**
+* SHOW MODELS [FROM [catalog.] database] sql call.
+*/
+SqlShowModels SqlShowModels() :
+{
+    SqlIdentifier databaseName = null;
+    SqlCharStringLiteral likeLiteral = null;
+    String prep = null;
+    boolean notLike = false;
+    SqlParserPos pos;
+}
+{
+    <SHOW> <MODELS>
+    { pos = getPos(); }
+    [
+        ( <FROM> { prep = "FROM"; } | <IN> { prep = "IN"; } )
+        { pos = getPos(); }
+        databaseName = CompoundIdentifier()
+    ]
+    [
+        [
+            <NOT>
+            {
+                notLike = true;
+            }
+        ]
+        <LIKE>  <QUOTED_STRING>
+        {
+            String likeCondition = SqlParserUtil.parseString(token.image);
+            likeLiteral = SqlLiteral.createCharString(likeCondition, getPos());
+        }
+    ]
+    {
+        return new SqlShowModels(pos, prep, databaseName, notLike, 
likeLiteral);
+    }
+}
+
+/**
+* ALTER MODEL [IF EXISTS] modelName SET (property_key = property_val, ...)
+* ALTER MODEL [IF EXISTS] modelName RENAME TO newModelName
+*/
+SqlAlterModel SqlAlterModel() :
+{
+    SqlParserPos startPos;
+    boolean ifExists = false;
+    SqlIdentifier modelIdentifier;
+    SqlIdentifier newModelIdentifier = null;
+    SqlNodeList modelOptionList = SqlNodeList.EMPTY;
+}
+{
+    <ALTER> <MODEL> { startPos = getPos(); }
+    ifExists = IfExistsOpt()
+    modelIdentifier = CompoundIdentifier()
+    (
+        LOOKAHEAD(2)
+        <RENAME> <TO>
+        newModelIdentifier = CompoundIdentifier()
+        {
+            return new SqlAlterModel(
+                        startPos.plus(getPos()),
+                        modelIdentifier,
+                        newModelIdentifier,
+                        ifExists);
+        }
+    |
+        <SET>
+        modelOptionList = TableProperties()

Review Comment:
   rename `TableProperties()` to `ObjectOptions()` or just `Options()` for 
consistency. I guess we are also using `TableProperties()` for the catalog WITH 
clause? we should leave the code base in a better shape with every PR.



##########
flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java:
##########
@@ -2996,6 +3011,144 @@ public void describeTo(Description description) {
         };
     }
 
+    @Test
+    void testShowModels() {
+        sql("show models").ok("SHOW MODELS");
+        sql("show models from db1").ok("SHOW MODELS FROM `DB1`");
+        sql("show models from catalog1.db1").ok("SHOW MODELS FROM 
`CATALOG1`.`DB1`");
+        sql("show models in db1").ok("SHOW MODELS IN `DB1`");
+        sql("show models in catalog1.db1").ok("SHOW MODELS IN 
`CATALOG1`.`DB1`");
+    }
+
+    @Test
+    void testDropModel() {
+        sql("drop model m1").ok("DROP MODEL `M1`");
+        sql("drop model db1.m1").ok("DROP MODEL `DB1`.`M1`");
+        sql("drop model catalog1.db1.m1").ok("DROP MODEL 
`CATALOG1`.`DB1`.`M1`");
+    }
+
+    @Test
+    void testDropModelIfExists() {
+        sql("drop model if exists catalog1.db1.m1")
+                .ok("DROP MODEL IF EXISTS `CATALOG1`.`DB1`.`M1`");
+    }
+
+    @Test
+    void testAlterModel() {
+        final String sql = "alter model m1 set ('key1' = 'value1','key2' = 
'value2')";
+        final String expected =
+                "ALTER MODEL `M1` SET (\n"
+                        + "  'key1' = 'value1',\n"
+                        + "  'key2' = 'value2'\n"
+                        + ")";
+        sql(sql).ok(expected);
+    }
+
+    @Test
+    void testAlterModelIfExists() {
+        final String sql = "alter model if exists m1 set ('key1' = 
'value1','key2' = 'value2')";
+        final String expected =
+                "ALTER MODEL IF EXISTS `M1` SET (\n"
+                        + "  'key1' = 'value1',\n"
+                        + "  'key2' = 'value2'\n"
+                        + ")";
+        sql(sql).ok(expected);
+    }
+
+    @Test
+    void testAlterModelRename() {
+        final String sql = "alter model m1 rename to m2";
+        final String expected = "ALTER MODEL `M1` RENAME TO `M2`";
+        sql(sql).ok(expected);
+    }
+
+    @Test
+    void testAlterModelRenameIfExists() {
+        final String sql = "alter model if exists m1 rename to m2";
+        final String expected = "ALTER MODEL IF EXISTS `M1` RENAME TO `M2`";
+        sql(sql).ok(expected);
+    }
+
+    @Test
+    void testCreateModel() {
+        sql("create model m1\n"
+                        + " INPUT(col1 INT, col2 STRING)\n"
+                        + " OUTPUT(label DOUBLE)\n"
+                        + " COMMENT 'model_comment'\n"
+                        + " WITH (\n"
+                        + "  'key1'='value1',\n"
+                        + "  'key2'='value2'\n"
+                        + " )\n")
+                .ok(
+                        "CREATE MODEL `M1` INPUT (\n"
+                                + "  `COL1` INTEGER,\n"
+                                + "  `COL2` STRING\n"
+                                + ") OUTPUT (\n"
+                                + "  `LABEL` DOUBLE\n"
+                                + ")\n"
+                                + "COMMENT 'model_comment' WITH (\n"
+                                + "  'key1' = 'value1',\n"
+                                + "  'key2' = 'value2'\n"
+                                + ")");
+    }
+
+    @Test
+    void testCreateModelIfNotExists() {
+        sql("create model if not exists m1\n"
+                        + " INPUT(col1 INT, col2 STRING)\n"
+                        + " OUTPUT(label DOUBLE)\n"
+                        + " COMMENT 'model_comment'\n"
+                        + " WITH (\n"
+                        + "  'key1'='value1',\n"
+                        + "  'key2'='value2'\n"
+                        + " )\n")
+                .ok(
+                        "CREATE MODEL IF NOT EXISTS `M1` INPUT (\n"
+                                + "  `COL1` INTEGER,\n"
+                                + "  `COL2` STRING\n"
+                                + ") OUTPUT (\n"
+                                + "  `LABEL` DOUBLE\n"
+                                + ")\n"
+                                + "COMMENT 'model_comment' WITH (\n"
+                                + "  'key1' = 'value1',\n"
+                                + "  'key2' = 'value2'\n"
+                                + ")");
+    }
+
+    @Test
+    void testCreateModelAs() {
+        sql("create model m1\n"
+                        + " WITH (\n"
+                        + "  'key1'='value1',\n"
+                        + "  'key2'='value2'\n"
+                        + " ) as select f1, f2 from t1\n")
+                .ok(
+                        "CREATE MODEL `M1` WITH (\n"
+                                + "  'key1' = 'value1',\n"
+                                + "  'key2' = 'value2'\n"
+                                + ")\n"
+                                + "AS\n"
+                                + "SELECT `F1`, `F2`\n"
+                                + "FROM `T1`");
+    }
+
+    @Test
+    void testCreateModelAsIfNotExists() {
+        sql("create model if not exists m1\n"
+                        + " WITH (\n"
+                        + "  'key1'='value1',\n"
+                        + "  'key2'='value2'\n"
+                        + " ) as select f1, f2 from t1\n")
+                .ok(
+                        "CREATE MODEL IF NOT EXISTS `M1` WITH (\n"

Review Comment:
   Also test the error behavior when a user adds explicit columns to input and 
output using `CREATE MODEL AS`



##########
flink-table/flink-sql-parser/pom.xml:
##########
@@ -322,6 +322,14 @@ under the License.
                                        </execution>
                                </executions>
                        </plugin>
+                       <plugin>
+                               <groupId>org.apache.maven.plugins</groupId>
+                               <artifactId>maven-checkstyle-plugin</artifactId>
+
+                               <configuration>

Review Comment:
   Should not be required. See comment below.



##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateModelAs.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.sql.parser.ddl;
+
+import org.apache.flink.sql.parser.error.SqlValidateException;
+
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+
+import javax.annotation.Nonnull;
+
+import java.util.List;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * {@link SqlNode} to describe the CREATE MODEL AS syntax. The CTAS would 
create a pipeline to
+ * compute the result of the given query and use the data to train the model.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * CREATE MODEL my_model WITH (
+ *     ‘task’ = ‘regression’,

Review Comment:
   Wrong single quotes, should be `'`



##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlRichDescribeModel.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.sql.parser.dql;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * DESCRIBE MODEL [EXTENDED] [[catalogName.] dataBasesName].sqlIdentifier sql 
call. Here we add Rich
+ * in className to follow the convention of SqlDescribeTable, which only had 
it to distinguish from

Review Comment:
   use `{@link` when referring to a class like `SqlDescribeTable` this help the 
reader to quickly navigate the code base



##########
tools/maven/suppressions-parser.xml:
##########
@@ -0,0 +1,26 @@
+<?xml version="1.0"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<!DOCTYPE suppressions PUBLIC
+       "-//Puppy Crawl//DTD Suppressions 1.1//EN"
+       "http://www.puppycrawl.com/dtds/suppressions_1_1.dtd";>
+
+<suppressions>

Review Comment:
   Why do we need a dedicated file? Shouldn't adding this to the regular Flink 
suppressions be enough?



##########
flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl:
##########
@@ -2412,6 +2440,8 @@ SqlDrop SqlDropExtended(Span s, boolean replace) :
         drop = SqlDropDatabase(s, replace)
         |
         drop = SqlDropFunction(s, replace, isTemporary)
+        |
+        drop = SqlDropModel(s) /** temporary model is not supported **/

Review Comment:
   Why don't we add this as part of this PR? Should be straight forward to add.



##########
flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl:
##########
@@ -3037,3 +3067,185 @@ SqlTruncateTable SqlTruncateTable() :
         return new SqlTruncateTable(getPos(), sqlIdentifier);
     }
 }
+
+/**
+* SHOW MODELS [FROM [catalog.] database] sql call.
+*/
+SqlShowModels SqlShowModels() :
+{
+    SqlIdentifier databaseName = null;
+    SqlCharStringLiteral likeLiteral = null;
+    String prep = null;
+    boolean notLike = false;
+    SqlParserPos pos;
+}
+{
+    <SHOW> <MODELS>
+    { pos = getPos(); }
+    [
+        ( <FROM> { prep = "FROM"; } | <IN> { prep = "IN"; } )
+        { pos = getPos(); }
+        databaseName = CompoundIdentifier()
+    ]
+    [
+        [
+            <NOT>
+            {
+                notLike = true;
+            }
+        ]
+        <LIKE>  <QUOTED_STRING>
+        {
+            String likeCondition = SqlParserUtil.parseString(token.image);
+            likeLiteral = SqlLiteral.createCharString(likeCondition, getPos());
+        }
+    ]
+    {
+        return new SqlShowModels(pos, prep, databaseName, notLike, 
likeLiteral);
+    }
+}
+
+/**
+* ALTER MODEL [IF EXISTS] modelName SET (property_key = property_val, ...)
+* ALTER MODEL [IF EXISTS] modelName RENAME TO newModelName
+*/
+SqlAlterModel SqlAlterModel() :
+{
+    SqlParserPos startPos;
+    boolean ifExists = false;
+    SqlIdentifier modelIdentifier;
+    SqlIdentifier newModelIdentifier = null;
+    SqlNodeList modelOptionList = SqlNodeList.EMPTY;
+}
+{
+    <ALTER> <MODEL> { startPos = getPos(); }
+    ifExists = IfExistsOpt()
+    modelIdentifier = CompoundIdentifier()
+    (
+        LOOKAHEAD(2)
+        <RENAME> <TO>
+        newModelIdentifier = CompoundIdentifier()
+        {
+            return new SqlAlterModel(
+                        startPos.plus(getPos()),
+                        modelIdentifier,
+                        newModelIdentifier,
+                        ifExists);
+        }
+    |
+        <SET>
+        modelOptionList = TableProperties()
+        {
+            return new SqlAlterModel(
+                        startPos.plus(getPos()),
+                        modelIdentifier,
+                        modelOptionList,
+                        ifExists);
+        }
+    )
+}
+
+/**
+* DROP MODEL [IF EXIST] modelName
+*/
+SqlDrop SqlDropModel(Span s) :
+{
+    SqlIdentifier modelIdentifier = null;
+    boolean ifExists = false;
+}
+{
+    <MODEL>
+
+    ifExists = IfExistsOpt()
+
+    modelIdentifier = CompoundIdentifier()
+
+    {
+         return new SqlDropModel(s.pos(), modelIdentifier, ifExists);
+    }
+}
+
+/**
+* CREATE MODEL [IF NOT EXIST] modelName

Review Comment:
   add temporary as part of this PR, should be easily copyable from `CREATE 
TABLE`?



##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateModelAs.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.sql.parser.ddl;
+
+import org.apache.flink.sql.parser.error.SqlValidateException;
+
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+
+import javax.annotation.Nonnull;
+
+import java.util.List;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * {@link SqlNode} to describe the CREATE MODEL AS syntax. The CTAS would 
create a pipeline to
+ * compute the result of the given query and use the data to train the model.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * CREATE MODEL my_model WITH (
+ *     ‘task’ = ‘regression’,
+ *     ‘algorithm’: ‘boosted_tree’,

Review Comment:
   `colon` is also not supported. A similar to `SqlAlterModel` would be more 
appropriate here.



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala:
##########
@@ -147,6 +148,7 @@ class FlinkPlannerImpl(
         || sqlNode.isInstanceOf[SqlShowProcedures]
         || sqlNode.isInstanceOf[SqlShowJobs]
         || sqlNode.isInstanceOf[SqlDescribeJob]
+        || sqlNode.isInstanceOf[SqlRichDescribeModel]

Review Comment:
   What is the current error behavior when people start using the syntax 
without an implementation for it? Does Flink generate a helpful message? Like 
"Unsupported operatin: CREATE MODEL AS"? It's unlikely that we will provide 
implementation for all keywords until feature freeze in 3 weeks.



##########
flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateModel.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.sql.parser.ddl;
+
+import org.apache.flink.sql.parser.ExtendedSqlNode;
+import org.apache.flink.sql.parser.SqlUnparseUtils;
+import org.apache.flink.sql.parser.error.SqlValidateException;
+
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlCreate;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.List;
+import java.util.Optional;
+
+import static java.util.Objects.requireNonNull;
+
+/** {@link SqlNode} to describe the CREATE MODEL syntax. */

Review Comment:
   also give a full syntax example in the JavaDoc here similar to 
`SqlAlterModel`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to