This is an automated email from the ASF dual-hosted git repository.

sivabalan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new 6156b74414d5 feat(spark): Add HoodieSparkSqlUtils APIs for tooling 
(#18202)
6156b74414d5 is described below

commit 6156b74414d58250557b691fc7ae5fd6e25d5ae4
Author: Surya Prasanna <[email protected]>
AuthorDate: Tue Feb 24 11:08:26 2026 -0800

    feat(spark): Add HoodieSparkSqlUtils APIs for tooling (#18202)
---
 .../java/org/apache/hudi/HoodieSparkSQLUtils.java  | 118 +++++++++++++++++++++
 .../hudi/utils/TestHoodieSparkSQLUtils.scala       | 109 +++++++++++++++++++
 2 files changed, 227 insertions(+)

diff --git 
a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieSparkSQLUtils.java
 
b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieSparkSQLUtils.java
new file mode 100644
index 000000000000..399dc5ec648f
--- /dev/null
+++ 
b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieSparkSQLUtils.java
@@ -0,0 +1,118 @@
+/*
+ * 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.hudi;
+
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class HoodieSparkSQLUtils {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieSparkSQLUtils.class);
+
+  /**
+   * Returns basepath of Hudi table given the table name.
+   *
+   * @param jsc java spark context
+   * @param fullTableName this is full table name including database.
+   */
+  public static String getBasePathFromTableName(JavaSparkContext jsc, String 
fullTableName) {
+    SparkSession sparkSession = 
SparkSession.builder().enableHiveSupport().sparkContext(jsc.sc()).getOrCreate();
+    return getBasePathFromTableName(sparkSession, fullTableName);
+  }
+
+  /**
+   * Returns basepath of Hudi table given the table name.
+   *
+   * @param sparkSession spark session.
+   * @param fullTableName this is full table name including database.
+   */
+  public static String getBasePathFromTableName(SparkSession sparkSession, 
String fullTableName) {
+    try {
+      return sparkSession.sql(String.format("desc formatted %s", 
fullTableName))
+          .collectAsList()
+          .stream()
+          .filter(row -> "Location".equals(row.getString(0)))
+          .map(v -> v.getString(1))
+          .findFirst()
+          .get();
+    } catch (Exception e) {
+      throw new HoodieException(String.format("Failed to fetch basepath for: 
%s", fullTableName), e);
+    }
+  }
+
+  public static List<Pair<String, String>> 
loadHoodiePathsFromHive(SparkSession sparkSession, String database,
+                                                                    boolean 
filterHudiDatasets) {
+    return loadHoodiePathsFromHive(sparkSession, database, filterHudiDatasets, 
"");
+  }
+
+  /**
+   * Get valid hoodie base path by checking hive registered tables.
+   *
+   * @param sparkSession sparkSession
+   * @param database hive database to check
+   * @param filterHudiDatasets decides whether to consider only hudi datasets.
+   * @return List of a pair of (tablename, basepath)
+   */
+  public static List<Pair<String, String>> 
loadHoodiePathsFromHive(SparkSession sparkSession, String database,
+                                                                    boolean 
filterHudiDatasets, String tableNamePrefix) {
+    Dataset<Row> tables = sparkSession.sql("SHOW TABLES FROM " + database);
+    List<Row> tableRows = !StringUtils.isNullOrEmpty(tableNamePrefix)
+        ? 
tables.filter(tables.col("tableName").startsWith(tableNamePrefix)).collectAsList()
+        : tables.collectAsList();
+    List<Pair<String, String>> validHoodiePaths = new ArrayList<>();
+    tableRows.forEach(tableRow -> {
+      String fullTableName = null;
+      try {
+        fullTableName = tableRow.getString(0) + "." + tableRow.getString(1);
+        LOG.info("Table name {}", fullTableName);
+        List<Row> rows = sparkSession.sql(String.format("desc formatted %s", 
fullTableName))
+            .collectAsList()
+            .stream()
+            .filter(row -> "Location".equals(row.getString(0)) || 
"InputFormat".equals(row.getString(0)))
+            .collect(Collectors.toList());
+        boolean skipDataset = false;
+        if (filterHudiDatasets) {
+          skipDataset = rows.stream().noneMatch(row -> 
"InputFormat".equals(row.getString(0))
+              && row.getString(1).startsWith("Hoodie"));
+        }
+        if (!skipDataset) {
+          String basepath = rows.stream().filter(row -> 
"Location".equals(row.getString(0)))
+              .map(v -> v.getString(1))
+              .findFirst()
+              .get();
+          validHoodiePaths.add(Pair.of(fullTableName, basepath));
+        }
+      } catch (Exception e) {
+        LOG.error("Exception in fetching basepath for table {}", 
fullTableName, e);
+      }
+    });
+    return validHoodiePaths;
+  }
+}
diff --git 
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/utils/TestHoodieSparkSQLUtils.scala
 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/utils/TestHoodieSparkSQLUtils.scala
new file mode 100644
index 000000000000..0406a9523347
--- /dev/null
+++ 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/utils/TestHoodieSparkSQLUtils.scala
@@ -0,0 +1,109 @@
+/*
+ * 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.hudi.utils
+
+import org.apache.hudi.HoodieSparkSQLUtils
+
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+
+import scala.collection.JavaConverters._
+
+class TestHoodieSparkSQLUtils extends HoodieSparkSqlTestBase {
+
+  test("Test getBasePathFromTableName APIs") {
+    withTable(generateTableName) { tableName =>
+      val fullTableName = s"default.$tableName"
+      createHudiTable(tableName)
+
+      val expectedBasePath = getTableLocation(fullTableName)
+      assertEquals(expectedBasePath, 
HoodieSparkSQLUtils.getBasePathFromTableName(spark, fullTableName))
+
+      val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
+      assertEquals(expectedBasePath, 
HoodieSparkSQLUtils.getBasePathFromTableName(jsc, fullTableName))
+    }
+  }
+
+  test("Test loadHoodiePathsFromHive APIs with and without prefix") {
+    val tableName1 = generateTableName
+    val tableName2 = generateTableName
+    val otherPrefixTable = generateTableName
+    val tableNamePrefix = tableName1.take(6)
+
+    withTable(tableName1) { _ =>
+      withTable(tableName2) { _ =>
+        withTable(otherPrefixTable) { _ =>
+          createHudiTable(tableName1)
+          createHudiTable(tableName2)
+          createHudiTable(otherPrefixTable)
+
+          val fullTableName1 = s"default.$tableName1"
+          val fullTableName2 = s"default.$tableName2"
+          val fullOtherTableName = s"default.$otherPrefixTable"
+
+          val allPairs = HoodieSparkSQLUtils.loadHoodiePathsFromHive(spark, 
"default", false).asScala
+          val allTableNames = allPairs.map(_.getLeft).toSet
+          assertTrue(allTableNames.contains(fullTableName1))
+          assertTrue(allTableNames.contains(fullTableName2))
+          assertTrue(allTableNames.contains(fullOtherTableName))
+
+          val prefixPairs = HoodieSparkSQLUtils
+            .loadHoodiePathsFromHive(spark, "default", false, tableNamePrefix)
+            .asScala
+          
assertTrue(prefixPairs.forall(_.getLeft.split("\\.")(1).startsWith(tableNamePrefix)))
+        }
+      }
+    }
+  }
+
+  test("Test loadHoodiePathsFromHive filterHudiDatasets mode") {
+    withTable(generateTableName) { tableName =>
+      createHudiTable(tableName)
+      val filteredPairs = HoodieSparkSQLUtils.loadHoodiePathsFromHive(spark, 
"default", true).asScala
+      // The current filter checks InputFormat value with startsWith("Hoodie").
+      assertTrue(filteredPairs.isEmpty)
+    }
+  }
+
+  private def createHudiTable(tableName: String): Unit = {
+    spark.sql(
+      s"""
+         | create table $tableName (
+         |  id int,
+         |  name string,
+         |  ts long
+         | ) using hudi
+         | tblproperties (
+         |  primaryKey = 'id',
+         |  orderingFields = 'ts'
+         | )
+         |""".stripMargin)
+    spark.sql(s"insert into $tableName values (1, 'a1', 1000)")
+  }
+
+  private def getTableLocation(fullTableName: String): String = {
+    spark.sql(s"desc formatted $fullTableName")
+      .collect()
+      .find(row => "Location".equals(row.getString(0)))
+      .map(row => row.getString(1))
+      .get
+  }
+}

Reply via email to