Uploaded image for project: 'Parquet'
  1. Parquet
  2. PARQUET-136

NPE thrown in StatisticsFilter when all values in a string/binary column trunk are null

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.6.0
    • 1.6.0
    • parquet-mr
    • None

    Description

      For a string or a binary column, if all values in a single column trunk are null, so do the min & max values in the column trunk statistics. However, while checking the statistics for column trunk pruning, a null check is missing, and causes NPE. Corresponding code can be found here.

      This issue can be steadily reproduced with the following Spark shell snippet against Spark 1.2.0-SNAPSHOT (013089794d):

      import org.apache.spark.sql.SQLContext
      
      val sqlContext = new SQLContext(sc)
      import sqlContext._
      
      case class StringCol(value: String)
      
      sc.parallelize(StringCol(null) :: Nil, 1).saveAsParquetFile("/tmp/empty.parquet")
      parquetFile("/tmp/empty.parquet").registerTempTable("null_table")
      
      sql("SET spark.sql.parquet.filterPushdown=true")
      sql("SELECT * FROM null_table WHERE value = 'foo'").collect()
      

      Exception thrown:

      org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.NullPointerException
              at parquet.io.api.Binary$ByteArrayBackedBinary.compareTo(Binary.java:206)
              at parquet.io.api.Binary$ByteArrayBackedBinary.compareTo(Binary.java:162)
              at parquet.filter2.statisticslevel.StatisticsFilter.visit(StatisticsFilter.java:100)
              at parquet.filter2.statisticslevel.StatisticsFilter.visit(StatisticsFilter.java:47)
              at parquet.filter2.predicate.Operators$Eq.accept(Operators.java:162)
              at parquet.filter2.statisticslevel.StatisticsFilter.canDrop(StatisticsFilter.java:52)
              at parquet.filter2.compat.RowGroupFilter.visit(RowGroupFilter.java:46)
              at parquet.filter2.compat.RowGroupFilter.visit(RowGroupFilter.java:22)
              at parquet.filter2.compat.FilterCompat$FilterPredicateCompat.accept(FilterCompat.java:108)
              at parquet.filter2.compat.RowGroupFilter.filterRowGroups(RowGroupFilter.java:28)
              at parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:158)
              at parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:138)
              at org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(NewHadoopRDD.scala:135)
              at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:107)
              at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:69)
              at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
              at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
              at org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31)
              at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
              at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
              at org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31)
              at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
              at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
              at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
              at org.apache.spark.scheduler.Task.run(Task.scala:56)
              at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
              at java.lang.Thread.run(Thread.java:745)
      

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              lian cheng Cheng Lian
              Votes:
              3 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: