fsk119 commented on code in PR #26553:
URL: https://github.com/apache/flink/pull/26553#discussion_r2095247558


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlMlTableFunction.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.planner.functions.sql.ml;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+import java.util.List;
+
+/**
+ * Base class for a table-valued function that works with models. Examples 
include {@code
+ * ML_PREDICT}.
+ */
+public abstract class SqlMlTableFunction extends SqlFunction implements 
SqlTableFunction {
+
+    protected static final String PARAM_DATA = "data";

Review Comment:
   In the FLIP 437, we agree to use the following statement. Hope we can have 
the same arugment names as the proposal in FLIP.
   
   ```
   SELECT * FROM ML_EVALUATE(
     input => `eval_data`,
     model => `classifier_model`,
     args => DESCRIPTOR(f1, f2)
   )
   ```



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlMlTableFunction.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.planner.functions.sql.ml;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+import java.util.List;
+
+/**
+ * Base class for a table-valued function that works with models. Examples 
include {@code
+ * ML_PREDICT}.
+ */
+public abstract class SqlMlTableFunction extends SqlFunction implements 
SqlTableFunction {

Review Comment:
   MLTableFunction? I think ML is better than Ml. WDYT?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlMlPredictTableFunction.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.table.planner.functions.sql.ml;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * SqlMlPredictTableFunction implements an operator for prediction.
+ *
+ * <p>It allows four parameters:
+ *
+ * <ol>
+ *   <li>a table
+ *   <li>a model name
+ *   <li>a descriptor to provide a column name from the input table
+ *   <li>an optional config map
+ * </ol>
+ */
+public class SqlMlPredictTableFunction extends SqlMlTableFunction {
+
+    public SqlMlPredictTableFunction() {
+        super("ML_PREDICT", new PredictOperandMetadata());
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * <p>Overrides because the first parameter of table-value function 
windowing is an explicit
+     * TABLE parameter, which is not scalar.
+     */
+    @Override
+    public boolean argumentMustBeScalar(int ordinal) {
+        return ordinal != 0;
+    }
+
+    @Override
+    protected RelDataType inferRowType(SqlOperatorBinding opBinding) {
+        // TODO: FLINK-37780 output type based on table schema and model 
output schema
+        // model output schema to be available after integrated with 
SqlExplicitModelCall
+        return opBinding.getOperandType(1);

Review Comment:
   I think the output schema of the ML_PREDICT the combine of input table 
schema and the model output schema. For example, if the input table schema is 
<a INT, b INT> and the model output schema is <c INT>, then the output schema 
of the ML_PREDICT is <a INT, b INT, c INT>. But the current implementation only 
takes model output schema into the consideration.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlMlTableFunction.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.table.planner.functions.sql.ml;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+import java.util.List;
+
+/**
+ * Base class for a table-valued function that works with models. Examples 
include {@code
+ * ML_PREDICT}.
+ */
+public abstract class SqlMlTableFunction extends SqlFunction implements 
SqlTableFunction {
+
+    protected static final String PARAM_DATA = "data";
+    protected static final String PARAM_MODEL = "input_model";
+    protected static final String PARAM_COLUMN = "input_column";
+    protected static final String PARAM_CONFIG = "config";
+
+    public SqlMlTableFunction(String name, SqlOperandMetadata operandMetadata) 
{
+        super(
+                name,
+                SqlKind.OTHER_FUNCTION,
+                ReturnTypes.CURSOR,
+                null,
+                operandMetadata,
+                SqlFunctionCategory.SYSTEM);
+    }
+
+    @Override
+    public void validateCall(
+            SqlCall call,
+            SqlValidator validator,
+            SqlValidatorScope scope,
+            SqlValidatorScope operandScope) {
+        assert call.getOperator() == this;

Review Comment:
           super.validateCall(call, validator, scope, operandScope);
   



##########
flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlModelCall.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.calcite.sql;
+
+import org.apache.flink.table.planner.catalog.CatalogSchemaModel;
+import org.apache.flink.table.planner.plan.FlinkCalciteCatalogReader;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/** SqlModelCall to fetch and reference model based on identifier. */
+public class SqlModelCall extends SqlBasicCall {
+
+    private @Nullable CatalogSchemaModel model = null;
+
+    public SqlModelCall(SqlExplicitModelCall modelCall) {
+        super(
+                SqlModelOperator.create(

Review Comment:
   We can introduce a static field for this operator, e.g.
   ```
       private static final SqlOperator OPERATOR = new SqlModelCallOperator();
   
   ```



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to