lihaosky commented on code in PR #27126:
URL: https://github.com/apache/flink/pull/27126#discussion_r2450007664
##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesRuntimeFunctions.java:
##########
@@ -1171,4 +1172,46 @@ private double cosineDistance(double[] left, double[]
right) {
return sum;
}
}
+
+ public static class TestValueAsyncVectorSearchFunction extends
AsyncVectorSearchFunction {
+
+ private final TestValueVectorSearchFunction impl;
+ private transient ExecutorService executors;
+ private transient Random random;
+
+ public TestValueAsyncVectorSearchFunction(
+ List<Row> data, int[] searchIndices, DataType physicalRowType)
{
+ this.impl = new TestValueVectorSearchFunction(data, searchIndices,
physicalRowType);
+ }
+
+ @Override
+ public void open(FunctionContext context) throws Exception {
+ super.open(context);
+ impl.open(context);
+ executors = Executors.newCachedThreadPool();
+ random = new Random();
+ }
+
+ @Override
+ public CompletableFuture<Collection<RowData>> asyncVectorSearch(
+ int topK, RowData queryData) {
+ return CompletableFuture.supplyAsync(
+ () -> {
+ try {
+ Thread.sleep(random.nextInt(800) + 200);
Review Comment:
Do something like
https://github.com/apache/flink/blob/master/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesModelFactory.java#L260
to have more deterministic behavior?
##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/AsyncVectorSearchITCase.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.runtime.stream.table;
+
+import org.apache.flink.core.testutils.FlinkAssertions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase;
+import
org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CollectionUtil;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThatList;
+
+/** ITCase for async VECTOR_SEARCH. */
+@ExtendWith(ParameterizedTestExtension.class)
+public class AsyncVectorSearchITCase extends StreamingWithStateTestBase {
+
+ public AsyncVectorSearchITCase(StateBackendMode state) {
+ super(state);
+ }
+
+ private final List<Row> data =
+ Arrays.asList(
+ Row.of(1L, new Float[] {5f, 12f, 13f}),
+ Row.of(2L, new Float[] {11f, 60f, 61f}),
+ Row.of(3L, new Float[] {8f, 15f, 17f}));
+
+ private final List<Row> nullableData =
+ Arrays.asList(Row.of(1L, new Float[] {5f, 12f, 13f}), Row.of(4L,
null));
+
+ @BeforeEach
+ public void before() {
+ super.before();
+ createTable("src", data);
+ createTable("nullableSrc", nullableData);
+ createTable("vector", data);
+ }
+
+ @TestTemplate
+ void testSimple() {
+ List<Row> actual =
+ CollectionUtil.iteratorToList(
+ tEnv().executeSql(
+ "SELECT * FROM src, LATERAL
TABLE(VECTOR_SEARCH(TABLE vector, DESCRIPTOR(`vector`), src.vector, 2))")
+ .collect());
+ assertThatList(actual)
+ .containsExactlyInAnyOrder(
+ Row.of(
+ 1L,
+ new Float[] {5.0f, 12.0f, 13.0f},
+ 1L,
+ new Float[] {5.0f, 12.0f, 13.0f},
+ 1.0),
+ Row.of(
+ 1L,
+ new Float[] {5.0f, 12.0f, 13.0f},
+ 3L,
+ new Float[] {8f, 15f, 17f},
+ 0.9977375565610862),
+ Row.of(
+ 2L,
+ new Float[] {11f, 60f, 61f},
+ 2L,
+ new Float[] {11f, 60f, 61f},
+ 1.0),
+ Row.of(
+ 2L,
+ new Float[] {11f, 60f, 61f},
+ 1L,
+ new Float[] {5.0f, 12.0f, 13.0f},
+ 0.9886506935687265),
+ Row.of(
+ 3L,
+ new Float[] {8f, 15f, 17f},
+ 3L,
+ new Float[] {8f, 15f, 17f},
+ 1.0000000000000002),
+ Row.of(
+ 3L,
+ new Float[] {8f, 15f, 17f},
+ 1L,
+ new Float[] {5.0f, 12.0f, 13.0f},
+ 0.9977375565610862));
+ }
+
+ @TestTemplate
+ void testLeftLateralJoin() {
+ List<Row> actual =
+ CollectionUtil.iteratorToList(
+ tEnv().executeSql(
+ "SELECT * FROM nullableSrc LEFT JOIN
LATERAL TABLE(VECTOR_SEARCH(TABLE vector, DESCRIPTOR(`vector`),
nullableSrc.vector, 2)) ON TRUE")
Review Comment:
Looks we don't have config of `async = true`. How do we make sure it's
running async mode?
--
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]