rahil-c commented on code in PR #18432:
URL: https://github.com/apache/hudi/pull/18432#discussion_r3034783953


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieVectorSearchPlanBuilder.scala:
##########
@@ -0,0 +1,318 @@
+/*
+ * 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.Window
+import org.apache.spark.sql.functions.{broadcast, col, 
monotonically_increasing_id, row_number}
+import org.apache.spark.sql.hudi.command.exception.HoodieAnalysisException
+import org.apache.spark.sql.types.{ArrayType, ByteType, DataType, DoubleType, 
FloatType}
+
+import scala.util.{Failure, Success, Try}
+
+/**
+ * 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) + 
`_hudi_distance: Double`
+ *  - Batch-query: all corpus columns (minus the embedding column) + clashing 
query columns
+ *    (prefixed with `_hudi_query_`) + `_hudi_distance: Double` + `_hudi_qid: 
Long`
+ *  - Results are ordered by `_hudi_distance` ascending (lower = more similar)
+ *  - `_hudi_qid` is an opaque grouping identifier (not a sequential index). 
Values may be
+ *    large non-contiguous numbers because they are generated by 
`monotonically_increasing_id()`.
+ */
+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
+   * @note Batch mode broadcasts the query table to all executors via a 
cross-join.
+   *       This is designed for small-to-medium query sets (tens to low 
hundreds of rows).
+   *       For large query tables, memory pressure on executors may occur.
+   */
+  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 = "_hudi_distance"
+  private[analysis] val QUERY_ID_COL = "_hudi_qid"
+  private[analysis] val QUERY_EMB_ALIAS = "_hudi_query_emb"
+  private[analysis] val RANK_COL = "_hudi_rank"
+  private[analysis] val QUERY_COL_PREFIX = "_hudi_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")
+  }
+
+  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] = {
+    df.schema.fields.find(_.name == colName).flatMap { field =>
+      val meta = field.metadata
+      if (meta.contains(HoodieSchema.TYPE_METADATA_FIELD)) {
+        val typeDesc = meta.getString(HoodieSchema.TYPE_METADATA_FIELD)
+        Try(HoodieSchema.parseTypeDescriptor(typeDesc)) match {
+          case Success(v: HoodieSchema.Vector) => Some(v.getDimension)
+          case Success(_) => None
+          case Failure(e) => throw new HoodieAnalysisException(
+            s"Column '$colName' has malformed type metadata '$typeDesc': 
${e.getMessage}")
+        }
+      } 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 filteredDf = corpusDf.filter(col(embeddingCol).isNotNull)
+
+    // Validate byte corpus query vector values before creating the UDF.
+    if (elemType == ByteType) {
+      queryVector.foreach { v =>
+        if (v < Byte.MinValue || v > Byte.MaxValue)
+          throw new HoodieAnalysisException(
+            s"Query vector value $v is out of range for byte corpus 
[${Byte.MinValue}, ${Byte.MaxValue}]")
+        if (v != v.toByte.toDouble)
+          throw new HoodieAnalysisException(
+            s"Query vector value $v is not a whole number. " +
+              "Byte corpus embeddings require integer-valued query vectors.")
+      }
+    }
+
+    // The single-query UDF closes over the pre-computed query DenseVector,
+    // so only the corpus column is passed per row.
+    val distanceUdf = VectorDistanceUtils.createSingleQueryDistanceUdf(metric, 
elemType, queryVector)
+
+    val result = filteredDf
+      .withColumn(DISTANCE_COL, distanceUdf(col(embeddingCol)))
+      .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 = VectorDistanceUtils.createDistanceUdf(metric, 
corpusElemType)
+    val filteredCorpus = corpusDf.filter(col(corpusEmbeddingCol).isNotNull)
+
+    // Prefix clashing query columns with "_hudi_query_" to avoid cross-join 
ambiguity when
+    // corpus and query share column names (e.g. both have "id" or 
"embedding").
+    // QUERY_ID_COL uses a distinct name (_hudi_qid) so it never clashes with 
prefixed query cols.
+    val corpusCols = filteredCorpus.columns.toSet
+    val queryWithId = queryDf.filter(col(queryEmbeddingCol).isNotNull)
+      .withColumnRenamed(queryEmbeddingCol, QUERY_EMB_ALIAS)
+      .withColumn(QUERY_ID_COL, monotonically_increasing_id())
+
+    val renamedQuery = queryWithId.select(queryWithId.columns.map { qCol =>
+      if (qCol != QUERY_ID_COL && qCol != QUERY_EMB_ALIAS && 
corpusCols.contains(qCol))
+        col(qCol).as(s"$QUERY_COL_PREFIX$qCol")
+      else
+        col(qCol)
+    }: _*)
+
+    // Cross join corpus with broadcast queries, compute distance, then rank
+    val scored = filteredCorpus.crossJoin(broadcast(renamedQuery))
+      .withColumn(DISTANCE_COL,
+        distanceUdf(col(corpusEmbeddingCol), col(QUERY_EMB_ALIAS)))
+      .drop(corpusEmbeddingCol)
+      .drop(QUERY_EMB_ALIAS)
+
+    val window = 
Window.partitionBy(QUERY_ID_COL).orderBy(col(DISTANCE_COL).asc)
+    val result = scored
+      .withColumn(RANK_COL, row_number().over(window))
+      .filter(col(RANK_COL) <= k)
+      .drop(RANK_COL)
+      .orderBy(col(QUERY_ID_COL), col(DISTANCE_COL))
+
+    result.queryExecution.analyzed

Review Comment:
   I think we can check the logical plan right with `describe command` to see 
this info. I can check on spark ui as well



-- 
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]

Reply via email to