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

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


The following commit(s) were added to refs/heads/master by this push:
     new f03902c06bae [SPARK-53531][CORE] Better error message for 
HadoopRDD.getInputFormat
f03902c06bae is described below

commit f03902c06bae120f1a6becf7597a26734ea98f46
Author: Cheng Pan <[email protected]>
AuthorDate: Wed Sep 10 12:18:40 2025 +0800

    [SPARK-53531][CORE] Better error message for HadoopRDD.getInputFormat
    
    ### What changes were proposed in this pull request?
    
    When `HadoopRDD.getInputFormat` fails to initialize the InputFormat, the 
thrown error message is not clear. This PR enhances the exception message to 
include the class name of InputFormat.
    
    A typical case that triggers this issue is reading an Iceberg table without 
a properly configured Iceberg catalog.
    
    
https://stackoverflow.com/questions/72620351/getting-error-when-querying-iceberg-table-via-spark-thrift-server-using-beeline
    
    ### Why are the changes needed?
    
    Improve the error message.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes, the user would know more info from the error message.
    
    ### How was this patch tested?
    
    Before
    ```
    spark-sql (default)> select * from i;
    2025-09-09 05:52:13 ERROR SparkSQLDriver: Failed in [select * from i]
    java.lang.RuntimeException: java.lang.InstantiationException
            at 
org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:157)
            at 
org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:126)
            at 
org.apache.spark.rdd.HadoopRDD.getInputFormat(HadoopRDD.scala:219)
            at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:233)
            at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:301)
            ...
    Caused by: java.lang.InstantiationException
            at 
java.base/jdk.internal.reflect.InstantiationExceptionConstructorAccessorImpl.newInstance(InstantiationExceptionConstructorAccessorImpl.java:48)
            at 
java.base/java.lang.reflect.Constructor.newInstanceWithCaller(Constructor.java:500)
            at 
java.base/java.lang.reflect.Constructor.newInstance(Constructor.java:481)
            at 
org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:155)
            ... 67 more
    ```
    
    After
    ```
    spark-sql (default)> select * from i;
    2025-09-09 06:23:48 ERROR SparkSQLDriver: Failed in [select * from i]
    java.lang.RuntimeException: Failed to instantiate 
org.apache.hadoop.mapred.FileInputFormat
            at 
org.apache.spark.rdd.HadoopRDD.getInputFormat(HadoopRDD.scala:223)
            at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:231)
            at org.apache.spark.rdd.RDD.$anonfun$partitions$2(RDD.scala:301)
            ...
    Caused by: java.lang.InstantiationException
            at 
java.base/jdk.internal.reflect.InstantiationExceptionConstructorAccessorImpl.newInstance(InstantiationExceptionConstructorAccessorImpl.java:48)
            at 
java.base/java.lang.reflect.Constructor.newInstanceWithCaller(Constructor.java:500)
            at 
java.base/java.lang.reflect.Constructor.newInstance(Constructor.java:481)
            at 
org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:155)
            at 
org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:126)
            at 
org.apache.spark.rdd.HadoopRDD.getInputFormat(HadoopRDD.scala:218)
            ... 65 more
    ```
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No.
    
    Closes #52282 from pan3793/SPARK-53531.
    
    Authored-by: Cheng Pan <[email protected]>
    Signed-off-by: yangjie01 <[email protected]>
---
 core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 15 +++++++--------
 1 file changed, 7 insertions(+), 8 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala 
b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index a16bd98a7466..e594b086bfda 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -24,7 +24,7 @@ import java.util.{Date, Locale}
 
 import scala.reflect.ClassTag
 
-import org.apache.hadoop.conf.{Configurable, Configuration}
+import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.hdfs.BlockMissingException
 import org.apache.hadoop.io.compress.CompressionCodecFactory
 import org.apache.hadoop.mapred._
@@ -214,14 +214,13 @@ class HadoopRDD[K, V](
     }
   }
 
-  protected def getInputFormat(conf: JobConf): InputFormat[K, V] = {
-    val newInputFormat = 
ReflectionUtils.newInstance(inputFormatClass.asInstanceOf[Class[_]], conf)
+  protected def getInputFormat(conf: JobConf): InputFormat[K, V] = try {
+    ReflectionUtils.newInstance(inputFormatClass.asInstanceOf[Class[_]], conf)
       .asInstanceOf[InputFormat[K, V]]
-    newInputFormat match {
-      case c: Configurable => c.setConf(conf)
-      case _ =>
-    }
-    newInputFormat
+  } catch {
+    case r: RuntimeException
+      if r.getCause != null && r.getCause.isInstanceOf[InstantiationException] 
=>
+      throw new RuntimeException(s"Failed to instantiate 
${inputFormatClass.getName}", r.getCause)
   }
 
   override def getPartitions: Array[Partition] = {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to