lihaosky commented on code in PR #27129:
URL: https://github.com/apache/flink/pull/27129#discussion_r2455885402
##########
docs/layouts/shortcodes/generated/vector_search_runtime_config_configuration.html:
##########
@@ -0,0 +1,36 @@
+<table class="configuration table table-bordered">
+ <thead>
+ <tr>
+ <th class="text-left" style="width: 20%">Key</th>
+ <th class="text-left" style="width: 15%">Default</th>
+ <th class="text-left" style="width: 10%">Type</th>
+ <th class="text-left" style="width: 55%">Description</th>
+ </tr>
+ </thead>
+ <tbody>
+ <tr>
+ <td><h5>async</h5></td>
+ <td style="word-wrap: break-word;">(none)</td>
+ <td>Boolean</td>
+ <td>Value can be 'true' or 'false' to suggest the planner choose
the corresponding predict function. If the backend search function provider
does not support the suggested mode, it will throw exception to notify
users.</td>
Review Comment:
Mention the default value?
##########
docs/layouts/shortcodes/generated/vector_search_runtime_config_configuration.html:
##########
@@ -0,0 +1,36 @@
+<table class="configuration table table-bordered">
+ <thead>
+ <tr>
+ <th class="text-left" style="width: 20%">Key</th>
+ <th class="text-left" style="width: 15%">Default</th>
+ <th class="text-left" style="width: 10%">Type</th>
+ <th class="text-left" style="width: 55%">Description</th>
+ </tr>
+ </thead>
+ <tbody>
+ <tr>
+ <td><h5>async</h5></td>
+ <td style="word-wrap: break-word;">(none)</td>
+ <td>Boolean</td>
+ <td>Value can be 'true' or 'false' to suggest the planner choose
the corresponding predict function. If the backend search function provider
does not support the suggested mode, it will throw exception to notify
users.</td>
+ </tr>
+ <tr>
+ <td><h5>max-concurrent-operations</h5></td>
+ <td style="word-wrap: break-word;">(none)</td>
+ <td>Integer</td>
+ <td>The max number of async i/o operation that the async ml
predict can trigger.</td>
Review Comment:
Mention default value?
```suggestion
<td>The max number of async i/o operation that the async vector
search can trigger.</td>
```
##########
docs/layouts/shortcodes/generated/vector_search_runtime_config_configuration.html:
##########
@@ -0,0 +1,36 @@
+<table class="configuration table table-bordered">
+ <thead>
+ <tr>
+ <th class="text-left" style="width: 20%">Key</th>
+ <th class="text-left" style="width: 15%">Default</th>
+ <th class="text-left" style="width: 10%">Type</th>
+ <th class="text-left" style="width: 55%">Description</th>
+ </tr>
+ </thead>
+ <tbody>
+ <tr>
+ <td><h5>async</h5></td>
+ <td style="word-wrap: break-word;">(none)</td>
+ <td>Boolean</td>
+ <td>Value can be 'true' or 'false' to suggest the planner choose
the corresponding predict function. If the backend search function provider
does not support the suggested mode, it will throw exception to notify
users.</td>
+ </tr>
+ <tr>
+ <td><h5>max-concurrent-operations</h5></td>
+ <td style="word-wrap: break-word;">(none)</td>
+ <td>Integer</td>
+ <td>The max number of async i/o operation that the async ml
predict can trigger.</td>
+ </tr>
+ <tr>
+ <td><h5>output-mode</h5></td>
+ <td style="word-wrap: break-word;">(none)</td>
+ <td><p>Enum</p></td>
+ <td>Output mode for asynchronous operations which will convert to
{@see AsyncDataStream.OutputMode}, ORDERED by default. If set to
ALLOW_UNORDERED, will attempt to use {@see
AsyncDataStream.OutputMode.UNORDERED} when it does not affect the correctness
of the result, otherwise ORDERED will be still used.<br /><br />Possible
values:<ul><li>"ORDERED"</li><li>"ALLOW_UNORDERED"</li></ul></td>
Review Comment:
Mention default?
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/VectorSearchRuntimeConfigOptions.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.api.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+
+import org.apache.flink.shaded.guava33.com.google.common.collect.ImmutableSet;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * This class holds option name definitions for VECTOR_SEARCH runtime config
based on {@link
+ * ConfigOption}.
+ */
+@PublicEvolving
+public class VectorSearchRuntimeConfigOptions {
+
+ public static final ConfigOption<Boolean> ASYNC =
+ key("async")
+ .booleanType()
+ .noDefaultValue()
Review Comment:
Why there's no default value?
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/VectorSearchRuntimeConfigOptions.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.api.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+
+import org.apache.flink.shaded.guava33.com.google.common.collect.ImmutableSet;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * This class holds option name definitions for VECTOR_SEARCH runtime config
based on {@link
+ * ConfigOption}.
+ */
+@PublicEvolving
+public class VectorSearchRuntimeConfigOptions {
+
+ public static final ConfigOption<Boolean> ASYNC =
+ key("async")
+ .booleanType()
+ .noDefaultValue()
+ .withDescription(
+ "Value can be 'true' or 'false' to suggest the
planner choose the corresponding"
+ + " predict function. If the backend
search function provider does not support the"
+ + " suggested mode, it will throw
exception to notify users.");
+
+ public static final ConfigOption<ExecutionConfigOptions.AsyncOutputMode>
ASYNC_OUTPUT_MODE =
+ key("output-mode")
+ .enumType(ExecutionConfigOptions.AsyncOutputMode.class)
+ .noDefaultValue()
+ .withDescription(
+ "Output mode for asynchronous operations which
will convert to {@see AsyncDataStream.OutputMode}, ORDERED by default. "
+ + "If set to ALLOW_UNORDERED, will attempt
to use {@see AsyncDataStream.OutputMode.UNORDERED} when it does not "
+ + "affect the correctness of the result,
otherwise ORDERED will be still used.");
+
+ public static final ConfigOption<Integer> ASYNC_MAX_CONCURRENT_OPERATIONS =
+ key("max-concurrent-operations")
+ .intType()
+ .noDefaultValue()
Review Comment:
ditto. Mention default it's in
`TABLE_EXEC_ASYNC_VECTOR_SEARCH_MAX_CONCURRENT_OPERATIONS`?
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/VectorSearchRuntimeConfigOptions.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.api.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+
+import org.apache.flink.shaded.guava33.com.google.common.collect.ImmutableSet;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * This class holds option name definitions for VECTOR_SEARCH runtime config
based on {@link
+ * ConfigOption}.
+ */
+@PublicEvolving
+public class VectorSearchRuntimeConfigOptions {
+
+ public static final ConfigOption<Boolean> ASYNC =
+ key("async")
+ .booleanType()
+ .noDefaultValue()
+ .withDescription(
+ "Value can be 'true' or 'false' to suggest the
planner choose the corresponding"
+ + " predict function. If the backend
search function provider does not support the"
+ + " suggested mode, it will throw
exception to notify users.");
+
+ public static final ConfigOption<ExecutionConfigOptions.AsyncOutputMode>
ASYNC_OUTPUT_MODE =
+ key("output-mode")
+ .enumType(ExecutionConfigOptions.AsyncOutputMode.class)
+ .noDefaultValue()
+ .withDescription(
+ "Output mode for asynchronous operations which
will convert to {@see AsyncDataStream.OutputMode}, ORDERED by default. "
+ + "If set to ALLOW_UNORDERED, will attempt
to use {@see AsyncDataStream.OutputMode.UNORDERED} when it does not "
+ + "affect the correctness of the result,
otherwise ORDERED will be still used.");
+
+ public static final ConfigOption<Integer> ASYNC_MAX_CONCURRENT_OPERATIONS =
+ key("max-concurrent-operations")
+ .intType()
+ .noDefaultValue()
+ .withDescription(
+ "The max number of async i/o operation that the
async ml predict can trigger.");
+
+ public static final ConfigOption<Duration> ASYNC_TIMEOUT =
+ key("timeout")
+ .durationType()
+ .noDefaultValue()
Review Comment:
ditto
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/ml/SqlVectorSearchTableFunction.java:
##########
@@ -217,18 +236,23 @@ public boolean checkOperandTypes(SqlCallBinding
callBinding, boolean throwOnFail
topK))),
throwOnFailure);
}
- return true;
+
+ // check config type
+ return SqlValidatorUtils.throwExceptionOrReturnFalse(
+ checkOptionalConfigOperands(
+ callBinding, 4,
SqlValidatorUtils::checkConfigValue),
Review Comment:
nit: use `OPTIONAL_ARG_IDX`
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/VectorSearchRuntimeConfigOptions.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.api.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+
+import org.apache.flink.shaded.guava33.com.google.common.collect.ImmutableSet;
+
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * This class holds option name definitions for VECTOR_SEARCH runtime config
based on {@link
+ * ConfigOption}.
+ */
+@PublicEvolving
+public class VectorSearchRuntimeConfigOptions {
+
+ public static final ConfigOption<Boolean> ASYNC =
+ key("async")
+ .booleanType()
+ .noDefaultValue()
+ .withDescription(
+ "Value can be 'true' or 'false' to suggest the
planner choose the corresponding"
+ + " predict function. If the backend
search function provider does not support the"
+ + " suggested mode, it will throw
exception to notify users.");
+
+ public static final ConfigOption<ExecutionConfigOptions.AsyncOutputMode>
ASYNC_OUTPUT_MODE =
+ key("output-mode")
+ .enumType(ExecutionConfigOptions.AsyncOutputMode.class)
+ .noDefaultValue()
Review Comment:
ditto
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/utils/SqlValidatorUtils.java:
##########
@@ -191,6 +204,67 @@ public static List<RelDataTypeField> makeOutputUnique(
return result;
}
+ public static Optional<RuntimeException> checkConfigValue(Map<String,
String> runtimeConfig) {
Review Comment:
`checkVectorSearchConfigValue`? This looks to be mainly for vector search
config check
--
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]