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

jiayu pushed a commit to branch fix-shapefile-databricks-2659
in repository https://gitbox.apache.org/repos/asf/sedona.git

commit aff0507bb11248971fce4df11af053df2e97a0b2
Author: Jia Yu <[email protected]>
AuthorDate: Wed Feb 18 11:58:51 2026 -0800

    [GH-2659] Fix Shapefile metadata reader on Databricks by using reflection 
for DataSource.checkAndGlobPathIfNecessary
    
    On Databricks Runtime (both Spark 3.5 and 4.0), 
DataSource.checkAndGlobPathIfNecessary
    has all required parameters with no defaults, unlike OSS Apache Spark where 
some
    parameters have default values.
    
    The previous code used named parameters, causing the Scala compiler to 
generate
    $default$N() synthetic method calls in bytecode. On Databricks these 
accessors
    don't exist, resulting in NoSuchMethodError at runtime.
    
    Using reflection resolves the method at runtime, avoiding any dependency on
    compiler-generated default parameter accessors.
---
 .../datasources/SedonaFileIndexHelper.scala        | 59 +++++++++++++++++++++-
 1 file changed, 58 insertions(+), 1 deletion(-)

diff --git 
a/spark/common/src/main/scala/org/apache/spark/sql/execution/datasources/SedonaFileIndexHelper.scala
 
b/spark/common/src/main/scala/org/apache/spark/sql/execution/datasources/SedonaFileIndexHelper.scala
index 7971a1b4ba..6d001bedd2 100644
--- 
a/spark/common/src/main/scala/org/apache/spark/sql/execution/datasources/SedonaFileIndexHelper.scala
+++ 
b/spark/common/src/main/scala/org/apache/spark/sql/execution/datasources/SedonaFileIndexHelper.scala
@@ -18,6 +18,8 @@
  */
 package org.apache.spark.sql.execution.datasources
 
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.util.CaseInsensitiveStringMap
@@ -36,6 +38,61 @@ import scala.collection.JavaConverters._
  */
 object SedonaFileIndexHelper {
 
+  /**
+   * Cached reflective reference to [[DataSource.checkAndGlobPathIfNecessary]].
+   *
+   * <p>We call this method via reflection to avoid binary incompatibility 
between OSS Apache Spark
+   * and Databricks Runtime. On OSS Spark (3.5, 4.0, 4.1, etc.) this method 
has default parameter
+   * values, but on Databricks Runtime (both Spark 3.5 and 4.0) the same 
method has all required
+   * parameters with no defaults, and may also differ in parameter count.
+   *
+   * <p>Direct calls with named/default parameters cause the Scala compiler to 
generate synthetic
+   * {@code $default$N()} accessor methods in the bytecode. When these 
accessors do not exist at
+   * runtime (as is the case on Databricks), a {@link NoSuchMethodError} is 
thrown. Reflection
+   * avoids this by resolving the method at runtime.
+   */
+  private lazy val checkAndGlobMethod: java.lang.reflect.Method = {
+    DataSource.getClass.getMethods
+      .filter(_.getName == "checkAndGlobPathIfNecessary")
+      .headOption
+      .getOrElse(throw new NoSuchMethodException(
+        "DataSource.checkAndGlobPathIfNecessary not found"))
+  }
+
+  private def checkAndGlobPathIfNecessary(
+      paths: Seq[String],
+      hadoopConf: Configuration,
+      checkEmptyGlobPath: Boolean,
+      checkFilesExist: Boolean,
+      enableGlobbing: Boolean): Seq[Path] = {
+    val method = checkAndGlobMethod
+    val args: Array[AnyRef] = method.getParameterCount match {
+      case 6 =>
+        // OSS Spark 4.x has an extra numThreads parameter at position 5
+        Array(
+          paths,
+          hadoopConf,
+          java.lang.Boolean.valueOf(checkEmptyGlobPath),
+          java.lang.Boolean.valueOf(checkFilesExist),
+          Integer.valueOf(40),
+          java.lang.Boolean.valueOf(enableGlobbing))
+      case _ =>
+        // OSS Spark 3.x / Databricks Runtime: 5 parameters
+        Array(
+          paths,
+          hadoopConf,
+          java.lang.Boolean.valueOf(checkEmptyGlobPath),
+          java.lang.Boolean.valueOf(checkFilesExist),
+          java.lang.Boolean.valueOf(enableGlobbing))
+    }
+    try {
+      method.invoke(DataSource, args: _*).asInstanceOf[Seq[Path]]
+    } catch {
+      case e: java.lang.reflect.InvocationTargetException =>
+        throw e.getCause
+    }
+  }
+
   /**
    * Build an [[InMemoryFileIndex]] for the given paths, resolving globs if 
necessary, without the
    * streaming metadata directory check.
@@ -49,7 +106,7 @@ object SedonaFileIndexHelper {
     val hadoopConf = 
sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap)
     val globPathsEnabled =
       Option(options.get("globPaths")).map(v => 
java.lang.Boolean.parseBoolean(v)).getOrElse(true)
-    val rootPathsSpecified = DataSource.checkAndGlobPathIfNecessary(
+    val rootPathsSpecified = checkAndGlobPathIfNecessary(
       paths,
       hadoopConf,
       checkEmptyGlobPath = true,

Reply via email to