You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/04/01 09:55:46 UTC

[GitHub] [iceberg] chenjunjiedada opened a new issue #2405: NPE when reading metadata for manifests.

chenjunjiedada opened a new issue #2405:
URL: https://github.com/apache/iceberg/issues/2405


   Caused by: java.lang.NullPointerException
   	at org.apache.iceberg.spark.source.StructInternalRow.getBoolean(StructInternalRow.java:109)
   	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	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:197)
   	at org.apache.iceberg.io.CloseableIterable$4$1.next(CloseableIterable.java:114)
   	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:458)
   	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:729)
   	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
   	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
   	at org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:221)
   	at org.apache.spark.storage.memory.MemoryStore.putIteratorAsValues(MemoryStore.scala:299)
   	at org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1371)
   	at org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1298)
   	at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1362)
   	at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1186)
   	at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:360)
   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:311)
   	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:349)
   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:313)
   	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
   	at org.apache.spark.scheduler.Task.run(Task.scala:127)
   	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:446)
   	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1391)
   	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:449)
   	... 3 more


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826281826






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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada closed issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada closed issue #2405:
URL: https://github.com/apache/iceberg/issues/2405


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] yyanyy commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
yyanyy commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826398213


   I think it's expected that `summary.containsNaN()` could return null? looking from the code it seems that `StaticDataTask.Row` should also be able to handle null? when does the issue occur? 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826903986


   I think if we make the changes @chenjunjiedada suggests (which I think are a good idea too), we should probably just change the signature to be a primitive and not the Boxed bool.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-827171835


   @yyanyy np :) It's a team effort. I think i'm pro just changing required -> Optional and in the future i'm going to be very strict about anyone using boxed primitives :P


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-825356399


   I thought it is caused by the `ManifestFileUtil` bug. But we had fixed that issue internally.  After digging more, we found this is caused by our internal flink iceberg sink which appends manifest without partition summary. 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada edited a comment on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada edited a comment on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826281826






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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826927825


   I think for the StaticDataRow error we should be changing the Schema of Manifests table to mark containsNan as Optional 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada closed issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada closed issue #2405:
URL: https://github.com/apache/iceberg/issues/2405


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada edited a comment on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada edited a comment on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826281826


   Still hit this problem.  Now I know how to reproduce this.
   
   1. Write the iceberg table with the 0.10.0 version.
   2. Read the iceberg table manifests with the 0.11.1 version.
   
   `ManifestTable.partitionSummarirsToRows` puts `summary.containsNaN` to the spark row which will be parsed to `Boolean` class and show out.
   
   The following code could fix the issue but I think we should consider with #2495 as well:
   ```
   --- a/api/src/main/java/org/apache/iceberg/ManifestFile.java
   +++ b/api/src/main/java/org/apache/iceberg/ManifestFile.java
   @@ -207,7 +207,7 @@ public interface ManifestFile {
         * Default to return null to ensure backward compatibility.
         */
        default Boolean containsNaN() {
   -      return null;
   +      return true;
        }
    
        /**
   --- a/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   +++ b/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   @@ -121,7 +121,7 @@ public class GenericPartitionFieldSummary
    
      @Override
      public Boolean containsNaN() {
   -    return containsNaN;
   +    return containsNaN == null || containsNull;
      }
   ```
   
   @RussellSpitzer @yyanyy 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826860482


   @chenjunjiedada Good catch, we should definitely fix that, I also was a little worried about keeping the boxed boolean as the return value in containsNaN since we usually treat the parameter as a primitive, but I'm ok with just fixing the other occurrence for now, wdyt?


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] yyanyy commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
yyanyy commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826398213


   I think it's expected that `summary.containsNaN()` could return null? looking from the code it seems that `StaticDataTask.Row` should also be able to handle null? when does the issue occur? 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826359240


   I think I have this fixed in #2495
   
   Sent from my iPhone
   
   > On Apr 25, 2021, at 3:24 AM, Chen, Junjie ***@***.***> wrote:
   > 
   > 
   > Still hit this problem. Now I know how to reproduce this.
   > 
   > Write the iceberg table with the 0.10.0 version.
   > Read the iceberg table manifests with the 0.11.1 version.
   > The following code could fix the issue:
   > 
   > --- a/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   > +++ b/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   > @@ -121,7 +121,7 @@ public class GenericPartitionFieldSummary
   >  
   >    @Override
   >    public Boolean containsNaN() {
   > -    return containsNaN;
   > +    return containsNaN == null || containsNull;
   >    }
   > @RussellSpitzer @yyanyy, We may need to re-consider this with #2492 again.
   > 
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub, or unsubscribe.
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826402145


   @RussellSpitzer, `summary.containsNaN()` is called in two places, #2495 fixes one place. Another place is in #2509.
   
   @yyanyy , I thought return boxed value in `summary.containsNaN()` may fix the problem for all places, while I don't want to break the original assumption. There is also a unit test for the null value. This issue happens when reading manifests as a table.
   
   > ManifestTable.partitionSummarirsToRows puts summary.containsNaN(), which is null, to the spark row that will be parsed to Boolean class and show out.
   
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826359240


   I think I have this fixed in #2495
   
   Sent from my iPhone
   
   > On Apr 25, 2021, at 3:24 AM, Chen, Junjie ***@***.***> wrote:
   > 
   > 
   > Still hit this problem. Now I know how to reproduce this.
   > 
   > Write the iceberg table with the 0.10.0 version.
   > Read the iceberg table manifests with the 0.11.1 version.
   > The following code could fix the issue:
   > 
   > --- a/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   > +++ b/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   > @@ -121,7 +121,7 @@ public class GenericPartitionFieldSummary
   >  
   >    @Override
   >    public Boolean containsNaN() {
   > -    return containsNaN;
   > +    return containsNaN == null || containsNull;
   >    }
   > @RussellSpitzer @yyanyy, We may need to re-consider this with #2492 again.
   > 
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub, or unsubscribe.
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826402826


   @yyanyy Looks like `StaticDataTask.Row` doesn't handle the null case. Do you think we should fix in that?


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] yyanyy closed issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
yyanyy closed issue #2405:
URL: https://github.com/apache/iceberg/issues/2405


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] yyanyy commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
yyanyy commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-827171072


   I think the `required` keyword in manifests table @RussellSpitzer pointed out is the culprit, I think `StaticDataTask.Row` should be able to handle null but since we marked `containsNan` as required in manifests table, it then throws the exception. And I agree with Russell that at least for showing manifest table, we probably don't want to mark `containsNan` as true if originally it's null/missing as it could be misleading.
   
   If we indeed want to make `containsNan` a primitive everywhere, there would be other places to change including a [public facing interface](https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/ManifestFile.java#L209-L211) that are introduced in #1872 and [a case](https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/expressions/ManifestEvaluator.java#L164) that will produce less accurate result. However we haven't specify if we want `contains_nan` to be nullable or not [in spec](https://iceberg.apache.org/spec/#manifest-lists) so this decision might still be revertible.
   
   I'll submit a PR to fix `required` field in manifests table and a separate PR to update spec so that we can have a wider audience and make a decision there. Meanwhile @RussellSpitzer @chenjunjiedada please let me know if you have further comments/concerns, and sorry for causing the issue!


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer edited a comment on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer edited a comment on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826927825


   I think for the StaticDataRow error we should be changing the Schema of Manifests table to mark containsNan as Optional since in this context we are actually reading the value out, and it isn't necessarily true, it may be missing.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826281826


   Still hit this problem.  Now I know how to reproduce this.
   
   1. Write the iceberg table with the 0.10.0 version.
   2. Read the iceberg table manifests with the 0.11.1 version.
   
   The following code could fix the issue:
   ```
   --- a/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   +++ b/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   @@ -121,7 +121,7 @@ public class GenericPartitionFieldSummary
    
      @Override
      public Boolean containsNaN() {
   -    return containsNaN;
   +    return containsNaN == null || containsNull;
      }
   ```
   
   @RussellSpitzer @yyanyy, We may need to re-consider this with #2492 again.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada edited a comment on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada edited a comment on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826281826


   Still hit this problem.  Now I know how to reproduce this.
   
   1. Write the iceberg table with the 0.10.0 version.
   2. Read the iceberg table manifests with the 0.11.1 version.
   
   `ManifestTable.partitionSummarirsToRows` puts `summary.containsNaN()`,  which is null,  to the spark row that will be parsed to `Boolean` class and show out.
   
   The following code could fix the issue but I think we should consider with #2495 as well:
   ```
   --- a/api/src/main/java/org/apache/iceberg/ManifestFile.java
   +++ b/api/src/main/java/org/apache/iceberg/ManifestFile.java
   @@ -207,7 +207,7 @@ public interface ManifestFile {
         * Default to return null to ensure backward compatibility.
         */
        default Boolean containsNaN() {
   -      return null;
   +      return true;
        }
    
        /**
   --- a/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   +++ b/core/src/main/java/org/apache/iceberg/GenericPartitionFieldSummary.java
   @@ -121,7 +121,7 @@ public class GenericPartitionFieldSummary
    
      @Override
      public Boolean containsNaN() {
   -    return containsNaN;
   +    return containsNaN == null || containsNull;
      }
   ```
   
   @RussellSpitzer @yyanyy 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-813070744


   Thanks @RussellSpitzer to take care! It happens at the customer's table. I opened the issue to remind myself as I was busy. Let me inspect what's happened!


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-812879873


   Do you have any more info on the repo for this?


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada edited a comment on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
chenjunjiedada edited a comment on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-826402145


   @RussellSpitzer, `summary.containsNaN()` is called in two places, #2495 fixes one place. Another place is in #2509.
   
   @yyanyy , I thought not return null in `summary.containsNaN()` may fix the problem for all places, while I don't want to break the original assumption. There is also a unit test for the null value. This issue happens when reading manifests as a table.
   
   > ManifestTable.partitionSummarirsToRows puts summary.containsNaN(), which is null, to the spark row that will be parsed to Boolean class and show out.
   
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] yyanyy commented on issue #2405: NPE when reading metadata for manifests.

Posted by GitBox <gi...@apache.org>.
yyanyy commented on issue #2405:
URL: https://github.com/apache/iceberg/issues/2405#issuecomment-827222241


   Apologies for the delay, #2521 contains the two changes I planned to include as mentioned earlier, please take a look when you have time @RussellSpitzer @chenjunjiedada Thank you! 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org