rahil-c commented on code in PR #18432: URL: https://github.com/apache/hudi/pull/18432#discussion_r3033468954
########## hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieVectorSearchPlanBuilder.scala: ########## @@ -0,0 +1,397 @@ +/* + * 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.spark.sql.hudi.analysis + +import org.apache.hudi.common.schema.HoodieSchema + +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.catalyst.plans.logical.HoodieVectorSearchTableValuedFunction.{DistanceMetric, SearchAlgorithm} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.expressions.{UserDefinedFunction, Window} +import org.apache.spark.sql.functions.{array, broadcast, col, lit, monotonically_increasing_id, row_number, udf} +import org.apache.spark.sql.hudi.command.exception.HoodieAnalysisException +import org.apache.spark.sql.types.{ArrayType, ByteType, DataType, DoubleType, FloatType} + +/** + * Extension point for vector search algorithms. Each implementation provides + * the Spark logical plan for single-query and batch-query KNN search. + * + * To add a new algorithm (e.g. RowMatrix, HNSW): + * 1. Create an object extending this trait + * 2. Add a value to [[SearchAlgorithm]] + * 3. Register the mapping in [[HoodieVectorSearchPlanBuilder.resolveAlgorithm]] + * + * Implementations can use the shared validation helpers on + * [[HoodieVectorSearchPlanBuilder]] (validateEmbeddingColumn, validateBatchDimensions, etc.) + * and the raw distance functions on [[VectorDistanceUtils]]. + * + * The output schema contract: + * - Single-query: all corpus columns (minus the embedding column) + `_distance: Double` + * - Batch-query: all corpus columns (minus the embedding column) + renamed query columns + * (prefixed with `_query_`) + `_distance: Double` + `_query_id: Long` + * - Results are ordered by `_distance` ascending (lower = more similar) + */ +trait VectorSearchAlgorithm { + + /** Human-readable name for error messages and logging. */ + def name: String + + /** + * Build a plan that finds the k nearest corpus rows to a single query vector. + * + * @param spark active SparkSession + * @param corpusDf resolved corpus DataFrame (may be Hudi, Parquet, or temp view) + * @param embeddingCol name of the array-typed embedding column in corpusDf + * @param queryVector the query vector, normalized to Array[Double] + * @param k number of nearest neighbors to return + * @param metric distance metric (COSINE, L2, DOT_PRODUCT) + * @return an analyzed LogicalPlan whose output matches the single-query schema contract + */ + def buildSingleQueryPlan( + spark: SparkSession, + corpusDf: DataFrame, + embeddingCol: String, + queryVector: Array[Double], + k: Int, + metric: DistanceMetric.Value): LogicalPlan + + /** + * Build a plan that finds the k nearest corpus rows for each row in the query table. + * + * @param spark active SparkSession + * @param corpusDf resolved corpus DataFrame + * @param corpusEmbeddingCol name of the embedding column in corpusDf + * @param queryDf resolved query DataFrame + * @param queryEmbeddingCol name of the embedding column in queryDf + * @param k number of nearest neighbors per query + * @param metric distance metric (COSINE, L2, DOT_PRODUCT) + * @return an analyzed LogicalPlan whose output matches the batch-query schema contract + */ + def buildBatchQueryPlan( + spark: SparkSession, + corpusDf: DataFrame, + corpusEmbeddingCol: String, + queryDf: DataFrame, + queryEmbeddingCol: String, + k: Int, + metric: DistanceMetric.Value): LogicalPlan +} + +/** + * Resolves [[SearchAlgorithm]] values to [[VectorSearchAlgorithm]] implementations + * and provides shared validation helpers used across algorithms. + */ +object HoodieVectorSearchPlanBuilder { + + val DISTANCE_COL = "_distance" + private[analysis] val QUERY_ID_COL = "_query_id" + private[analysis] val QUERY_EMB_ALIAS = "_query_emb_internal" + private[analysis] val RANK_COL = "_rank" + private[analysis] val QUERY_COL_PREFIX = "_query_" + + /** Resolve a [[SearchAlgorithm]] enum value to its implementation. */ + def resolveAlgorithm(algorithm: SearchAlgorithm.Value): VectorSearchAlgorithm = algorithm match { + case SearchAlgorithm.BRUTE_FORCE => BruteForceSearchAlgorithm + case other => throw new HoodieAnalysisException( + s"Unsupported search algorithm: $other") + } + + // ======================== Shared Validation ======================== + + private[analysis] def validateEmbeddingColumn(df: DataFrame, colName: String): Unit = { + val fieldOpt = df.schema.fields.find(_.name == colName) + val field = fieldOpt.getOrElse( + throw new HoodieAnalysisException( + s"Embedding column '$colName' not found in table schema. " + + s"Available columns: ${df.schema.fieldNames.mkString(", ")}")) + field.dataType match { + case ArrayType(FloatType, _) | ArrayType(DoubleType, _) | ArrayType(ByteType, _) => // valid + case other => + throw new HoodieAnalysisException( + s"Embedding column '$colName' has type $other, " + + "expected array<float>, array<double>, or array<byte>") + } + } + + /** + * Validates that the query vector dimension matches the corpus embedding dimension + * when the corpus column has VECTOR(dim) metadata. + */ + private[analysis] def validateQueryVectorDimension( + df: DataFrame, embeddingCol: String, queryDim: Int): Unit = { + extractVectorDimension(df, embeddingCol).foreach { corpusDim => + if (corpusDim != queryDim) { + throw new HoodieAnalysisException( + s"Query vector dimension ($queryDim) does not match " + + s"corpus embedding dimension ($corpusDim) for column '$embeddingCol'") + } + } + } + + /** + * Validates that corpus and query embedding columns have the same element type. + */ + private[analysis] def validateElementTypeCompatibility( + corpusDf: DataFrame, corpusCol: String, + queryDf: DataFrame, queryCol: String): Unit = { + val corpusElemType = getElementType(corpusDf, corpusCol) + val queryElemType = getElementType(queryDf, queryCol) + if (corpusElemType != queryElemType) { + throw new HoodieAnalysisException( + s"Corpus embedding column '$corpusCol' has element type $corpusElemType " + + s"but query embedding column '$queryCol' has element type $queryElemType. " + + "Both must use the same element type (e.g. array<float>).") + } + } + + /** + * Validates that corpus and query embedding dimensions match when both have + * VECTOR(dim) metadata. + */ + private[analysis] def validateBatchDimensions( + corpusDf: DataFrame, corpusCol: String, + queryDf: DataFrame, queryCol: String): Unit = { + (extractVectorDimension(corpusDf, corpusCol), extractVectorDimension(queryDf, queryCol)) match { + case (Some(corpusDim), Some(queryDim)) if corpusDim != queryDim => + throw new HoodieAnalysisException( + s"Corpus embedding dimension ($corpusDim) does not match " + + s"query embedding dimension ($queryDim)") + case _ => // dimensions match or metadata not available + } + } + + private[analysis] def getElementType(df: DataFrame, colName: String): DataType = { + df.schema(colName).dataType match { + case ArrayType(elemType, _) => elemType + case other => + throw new HoodieAnalysisException( + s"Embedding column '$colName' has type $other, expected an array type") + } + } + + /** Extracts VECTOR(dim) dimension from column metadata, if present. */ + private def extractVectorDimension(df: DataFrame, colName: String): Option[Int] = { + val field = df.schema.fields.find(_.name == colName).get + val meta = field.metadata + if (meta.contains(HoodieSchema.TYPE_METADATA_FIELD)) { + val typeDesc = meta.getString(HoodieSchema.TYPE_METADATA_FIELD) + val dimPattern = """VECTOR\((\d+)""".r + dimPattern.findFirstMatchIn(typeDesc).map(_.group(1).toInt) + } else None + } +} + +/** + * Brute-force KNN vector search: computes distance between every corpus row and + * every query vector, then selects the top-K closest. + * + * <p>Complexity: O(|corpus| * |queries| * dimensions) — linear scan with no index. + * + * <p><b>Single-query mode:</b> applies a distance UDF per corpus row, then + * {@code orderBy + limit(k)} (Spark optimizes this to a partial sort via TakeOrderedAndProject). + * + * <p><b>Batch-query mode:</b> broadcast cross-joins the (small) query table with + * the corpus, computes pairwise distances, then uses a window function to rank + * and select top-K per query. The cross-join produces O(|corpus| * |queries|) + * intermediate rows, so this is suitable for small-to-medium query sets + * (tens to low hundreds of queries) against moderate corpora. + */ +object BruteForceSearchAlgorithm extends VectorSearchAlgorithm { + + import HoodieVectorSearchPlanBuilder._ + + override val name: String = "brute_force" + + override def buildSingleQueryPlan( + spark: SparkSession, + corpusDf: DataFrame, + embeddingCol: String, + queryVector: Array[Double], + k: Int, + metric: DistanceMetric.Value): LogicalPlan = { + validateEmbeddingColumn(corpusDf, embeddingCol) + validateQueryVectorDimension(corpusDf, embeddingCol, queryVector.length) + + val elemType = getElementType(corpusDf, embeddingCol) + val distanceUdf = createDistanceUdf(metric, elemType) + val filteredDf = corpusDf.filter(col(embeddingCol).isNotNull) + + // Convert query vector to match corpus element type. + // ByteType requires array() constructor because lit(Array[Byte]) creates BinaryType. + val queryLit = elemType match { + case FloatType => lit(queryVector.map(_.toFloat)) + case DoubleType => lit(queryVector) + case ByteType => array(queryVector.map(v => lit(v.toByte)): _*) + case _ => lit(queryVector) + } + + val result = filteredDf + .withColumn(DISTANCE_COL, distanceUdf(col(embeddingCol), queryLit)) + .drop(embeddingCol) + .orderBy(col(DISTANCE_COL).asc) + .limit(k) + + result.queryExecution.analyzed + } + + override def buildBatchQueryPlan( + spark: SparkSession, + corpusDf: DataFrame, + corpusEmbeddingCol: String, + queryDf: DataFrame, + queryEmbeddingCol: String, + k: Int, + metric: DistanceMetric.Value): LogicalPlan = { + validateEmbeddingColumn(corpusDf, corpusEmbeddingCol) + validateEmbeddingColumn(queryDf, queryEmbeddingCol) + validateElementTypeCompatibility(corpusDf, corpusEmbeddingCol, queryDf, queryEmbeddingCol) + validateBatchDimensions(corpusDf, corpusEmbeddingCol, queryDf, queryEmbeddingCol) + + val corpusElemType = getElementType(corpusDf, corpusEmbeddingCol) + val distanceUdf = createDistanceUdf(metric, corpusElemType) + val filteredCorpus = corpusDf.filter(col(corpusEmbeddingCol).isNotNull) + + // Prefix every query column with "_query_" to avoid cross-join column ambiguity: + // 1. when corpusEmbeddingCol == queryEmbeddingCol (both named "embedding") + // 2. when corpus and query share other non-embedding columns (e.g. both have "id") + val corpusCols = filteredCorpus.columns.toSet + val internalCols = Set(QUERY_ID_COL, QUERY_EMB_ALIAS, DISTANCE_COL, RANK_COL) + val queryWithId = queryDf.filter(col(queryEmbeddingCol).isNotNull) + .withColumnRenamed(queryEmbeddingCol, QUERY_EMB_ALIAS) + .withColumn(QUERY_ID_COL, monotonically_increasing_id()) + + // Rename any query column that clashes with a corpus column or internal columns. + // Uses a double prefix if the standard rename would itself clash (e.g. "id" -> "_query_id" + // would collide with the internal _query_id column). + val renamedQuery = queryWithId.columns.foldLeft(queryWithId) { (df, qCol) => Review Comment: @yihua i think you are right that seems to be the cleanest solution let me try this out. -- 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]
