lihaosky commented on code in PR #26644: URL: https://github.com/apache/flink/pull/26644#discussion_r2140558849
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlMLEvaluateTableFunction.java: ########## @@ -0,0 +1,208 @@ +/* + * 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.flink.table.planner.functions.utils.SqlValidatorUtils; + +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.SqlCharStringLiteral; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +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 org.apache.calcite.util.NlsString; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * SqlMLEvaluateTableFunction implements an operator for model evaluation. + * + * <p>It allows six parameters: + * + * <ol> + * <li>a table name + * <li>a model name + * <li>a descriptor to provide label column names from the input table + * <li>a descriptor to provide feature column names from the input table + * <li>an optional task override string + * <li>an optional config map + * </ol> + * + * <p>The function returns a MAP type containing evaluation metrics and results. + */ +public class SqlMLEvaluateTableFunction extends SqlMLTableFunction { + + public static final String PARAM_LABEL = "LABEL"; + public static final String PARAM_FEATURE = "FEATURE"; + public static final String PARAM_TASK = "TASK"; + + public static final List<String> SUPPORTED_TASKS = Review Comment: Sure ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlMLEvaluateTableFunction.java: ########## @@ -0,0 +1,208 @@ +/* + * 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.flink.table.planner.functions.utils.SqlValidatorUtils; + +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.SqlCharStringLiteral; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +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 org.apache.calcite.util.NlsString; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * SqlMLEvaluateTableFunction implements an operator for model evaluation. + * + * <p>It allows six parameters: + * + * <ol> + * <li>a table name + * <li>a model name + * <li>a descriptor to provide label column names from the input table + * <li>a descriptor to provide feature column names from the input table + * <li>an optional task override string + * <li>an optional config map + * </ol> + * + * <p>The function returns a MAP type containing evaluation metrics and results. + */ +public class SqlMLEvaluateTableFunction extends SqlMLTableFunction { + + public static final String PARAM_LABEL = "LABEL"; + public static final String PARAM_FEATURE = "FEATURE"; Review Comment: Sure ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlMLEvaluateTableFunction.java: ########## @@ -0,0 +1,208 @@ +/* + * 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.flink.table.planner.functions.utils.SqlValidatorUtils; + +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.SqlCharStringLiteral; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +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 org.apache.calcite.util.NlsString; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * SqlMLEvaluateTableFunction implements an operator for model evaluation. + * + * <p>It allows six parameters: + * + * <ol> + * <li>a table name + * <li>a model name + * <li>a descriptor to provide label column names from the input table + * <li>a descriptor to provide feature column names from the input table + * <li>an optional task override string + * <li>an optional config map + * </ol> + * + * <p>The function returns a MAP type containing evaluation metrics and results. + */ +public class SqlMLEvaluateTableFunction extends SqlMLTableFunction { + + public static final String PARAM_LABEL = "LABEL"; + public static final String PARAM_FEATURE = "FEATURE"; + public static final String PARAM_TASK = "TASK"; + + public static final List<String> SUPPORTED_TASKS = + List.of("regression", "clustering", "classification", "embedding", "text_generation"); + + public SqlMLEvaluateTableFunction() { + super("ML_EVALUATE", new EvaluateOperandMetadata()); + } + + /** + * {@inheritDoc} + * + * <p>Overrides because the first parameter of ML table-value function is an explicit TABLE + * parameter, which is not scalar. + */ + @Override + public boolean argumentMustBeScalar(int ordinal) { + return ordinal != 0; + } + + @Override + protected RelDataType inferRowType(SqlOperatorBinding opBinding) { + final RelDataTypeFactory typeFactory = opBinding.getTypeFactory(); + final RelDataType inputRowType = opBinding.getOperandType(0); + + // Create a MAP type for evaluation results + RelDataType keyType = typeFactory.createSqlType(SqlTypeName.VARCHAR); + RelDataType valueType = typeFactory.createSqlType(SqlTypeName.DOUBLE); + RelDataType mapType = typeFactory.createMapType(keyType, valueType); + + return typeFactory + .builder() + .kind(inputRowType.getStructKind()) + .add("result", mapType) + .build(); + } + + private static class EvaluateOperandMetadata implements SqlOperandMetadata { + private static final List<String> PARAM_NAMES = + List.of( + PARAM_INPUT, + PARAM_MODEL, + PARAM_LABEL, + PARAM_FEATURE, + PARAM_TASK, + PARAM_CONFIG); + private static final List<String> MANDATORY_PARAM_NAMES = + List.of(PARAM_INPUT, PARAM_MODEL, PARAM_LABEL, PARAM_FEATURE); + + EvaluateOperandMetadata() {} + + @Override + public List<RelDataType> paramTypes(RelDataTypeFactory typeFactory) { + return Collections.nCopies( + PARAM_NAMES.size(), typeFactory.createSqlType(SqlTypeName.ANY)); + } + + @Override + public List<String> paramNames() { + return PARAM_NAMES; + } + + @Override + public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) { + if (!SqlValidatorUtils.checkTableAndDescriptorOperands(callBinding, 2, 3)) { + return SqlValidatorUtils.throwValidationSignatureErrorOrReturnFalse( + callBinding, throwOnFailure); + } + + if (!SqlValidatorUtils.throwExceptionOrReturnFalse( + checkModelSignature(callBinding, 3, 2), throwOnFailure)) { + return false; + } + + if (callBinding.getOperandCount() == PARAM_NAMES.size()) { + // Check task and config parameters + if (!SqlValidatorUtils.throwExceptionOrReturnFalse( + checkTask(callBinding.operand(4)), throwOnFailure)) { + return false; + } + + if (!SqlValidatorUtils.throwExceptionOrReturnFalse( + checkConfig(callBinding.operand(5)), throwOnFailure)) { + return false; + } + } + + if (callBinding.getOperandCount() == MANDATORY_PARAM_NAMES.size() + 1) { + // Last param can be config or task Review Comment: 5th param is optional, it can be task or config map. 6th param is also optional and it must be config if exist ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlMLPredictTableFunction.java: ########## @@ -134,100 +123,13 @@ public SqlOperandCountRange getOperandCountRange() { @Override public boolean isOptional(int i) { - return i > getOperandCountRange().getMin() && i <= getOperandCountRange().getMax(); + return i >= getOperandCountRange().getMin() && i <= getOperandCountRange().getMax(); Review Comment: I'm confused. If `i` is operand index, it should be `>=` min count? But it's `>` in `SqlWindowTableFunction`. ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlMLEvaluateTableFunction.java: ########## @@ -0,0 +1,208 @@ +/* + * 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.flink.table.planner.functions.utils.SqlValidatorUtils; + +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.SqlCharStringLiteral; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +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 org.apache.calcite.util.NlsString; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * SqlMLEvaluateTableFunction implements an operator for model evaluation. + * + * <p>It allows six parameters: + * + * <ol> + * <li>a table name + * <li>a model name + * <li>a descriptor to provide label column names from the input table + * <li>a descriptor to provide feature column names from the input table + * <li>an optional task override string + * <li>an optional config map + * </ol> + * + * <p>The function returns a MAP type containing evaluation metrics and results. + */ +public class SqlMLEvaluateTableFunction extends SqlMLTableFunction { + + public static final String PARAM_LABEL = "LABEL"; + public static final String PARAM_FEATURE = "FEATURE"; + public static final String PARAM_TASK = "TASK"; + + public static final List<String> SUPPORTED_TASKS = + List.of("regression", "clustering", "classification", "embedding", "text_generation"); + + public SqlMLEvaluateTableFunction() { + super("ML_EVALUATE", new EvaluateOperandMetadata()); + } + + /** + * {@inheritDoc} + * + * <p>Overrides because the first parameter of ML table-value function is an explicit TABLE + * parameter, which is not scalar. + */ + @Override + public boolean argumentMustBeScalar(int ordinal) { + return ordinal != 0; + } + + @Override + protected RelDataType inferRowType(SqlOperatorBinding opBinding) { + final RelDataTypeFactory typeFactory = opBinding.getTypeFactory(); + final RelDataType inputRowType = opBinding.getOperandType(0); + + // Create a MAP type for evaluation results + RelDataType keyType = typeFactory.createSqlType(SqlTypeName.VARCHAR); + RelDataType valueType = typeFactory.createSqlType(SqlTypeName.DOUBLE); + RelDataType mapType = typeFactory.createMapType(keyType, valueType); + + return typeFactory + .builder() + .kind(inputRowType.getStructKind()) + .add("result", mapType) + .build(); + } + + private static class EvaluateOperandMetadata implements SqlOperandMetadata { + private static final List<String> PARAM_NAMES = + List.of( + PARAM_INPUT, + PARAM_MODEL, + PARAM_LABEL, + PARAM_FEATURE, + PARAM_TASK, + PARAM_CONFIG); + private static final List<String> MANDATORY_PARAM_NAMES = Review Comment: `task` is optional here. It can be used to override the task in model or if task doesn't appear in model. ``` SELECT * FROM ML_EVALUATE(TABLE `eval_data`, MODEL `classifier_model`, DESCRIPTOR(label), DESCRIPTOR(f1, f2)) ``` ``` SELECT * FROM ML_EVALUATE(input => TABLE `eval_data`, model_input => MODEL `classifier_model`, label => DESCRIPTOR(label), features => DESCRIPTOR(f1, f2), task => 'classification') ``` are both supported in the FLIP. First one will use task in model option. If it doesn't exist, it's an error ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMLPredictTableFunction.java: ########## @@ -81,8 +81,8 @@ name = "stream-exec-ml-predict-table-function", version = 1, producedTransformations = StreamExecMLPredictTableFunction.ML_PREDICT_TRANSFORMATION, - minPlanVersion = FlinkVersion.V2_1, Review Comment: It's fixed in https://github.com/apache/flink/pull/26661. I may need to rebase -- 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