szehon-ho opened a new issue #2783:
URL: https://github.com/apache/iceberg/issues/2783


   As of Spark 3, it seems more predicates are pushed down.  For example, take 
this join:
   
   ```
   Dataset<Row> stringDs = spark.createDataset(Arrays.asList("my_path"), 
Encoders.STRING())
           .toDF("file_path");
   
       SparkCatalog catalog = (SparkCatalog) 
spark.sessionState().catalogManager().catalog(catalogName);
       String[] tableIdentifiers = tableName.split("\\.");
       Identifier metaId = Identifier.of(
           new String[]{tableIdentifiers[1], tableIdentifiers[2]}, "entries");
       SparkTable metaTable = catalog.loadTable(metaId);
       Dataset<Row> entriesDs = Dataset.ofRows(spark, 
DataSourceV2Relation.create(metaTable, Some.apply(catalog), Some.apply(
           metaId)));
   
       Column joinCond = 
entriesDs.col("data_file.file_path").equalTo(stringDs.col("file_path"));
       Dataset<Row> res = entriesDs.join(stringDs, joinCond);
       boolean empty = res.isEmpty();
       Assert.assertEquals(true, empty);
   ```
   
   It will result in the following NPE: 
   ```Driver stacktrace:
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in 
stage 5.0 failed 1 times, most recent failure: Lost task 0.0 in stage 5.0 (TID 
5) (10.0.0.45 executor driver): java.lang.IllegalStateException: Unknown type 
for int field. Type name: java.lang.String
        at 
org.apache.iceberg.spark.source.StructInternalRow.getInt(StructInternalRow.java:131)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.writeFields_0_0$(Unknown
 Source)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.base/java.lang.reflect.Method.invoke(Method.java:566)
        at 
org.apache.iceberg.common.DynMethods$UnboundMethod.invokeChecked(DynMethods.java:65)
        at 
org.apache.iceberg.common.DynMethods$UnboundMethod.invoke(DynMethods.java:77)
        at 
org.apache.iceberg.common.DynMethods$BoundMethod.invoke(DynMethods.java:180)
        at 
org.apache.iceberg.spark.source.RowDataReader.lambda$newDataIterable$3(RowDataReader.java:193)
        at 
org.apache.iceberg.io.CloseableIterable$4$1.next(CloseableIterable.java:113)
        at org.apache.iceberg.io.FilterIterator.advance(FilterIterator.java:66)
        at org.apache.iceberg.io.FilterIterator.hasNext(FilterIterator.java:50)
        at 
org.apache.iceberg.spark.source.BaseDataReader.next(BaseDataReader.java:87)
        at 
org.apache.spark.sql.execution.datasources.v2.PartitionIterator.hasNext(DataSourceRDD.scala:79)
        at 
org.apache.spark.sql.execution.datasources.v2.MetricsIterator.hasNext(DataSourceRDD.scala:112)
        at 
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:755)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:345)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:131)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   ```


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



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

Reply via email to