Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-12417

Orc bloom filter options are not propagated during file write in spark

Details

    • Improvement
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • None
    • 2.0.0
    • SQL
    • None

    Description

      ORC bloom filter is supported by the version of hive used in Spark 1.5.2. However, when trying to create orc file with bloom filter option, it does not make use of it.

      E.g, following orc output does not create the bloom filter even though the options are specified.

          Map<String, String> orcOption = new HashMap<String, String>();
          orcOption.put("orc.bloom.filter.columns", "*");
          hiveContext.sql("select * from accounts where effective_date='2015-12-30'").write().
              format("orc").options(orcOption).save("/tmp/accounts");
      

      Attachments

        1. SPARK-12417.1.patch
          2 kB
          Rajesh Balamohan

        Issue Links

          Activity

            apachespark Apache Spark added a comment -

            User 'rajeshbalamohan' has created a pull request for this issue:
            https://github.com/apache/spark/pull/10375

            apachespark Apache Spark added a comment - User 'rajeshbalamohan' has created a pull request for this issue: https://github.com/apache/spark/pull/10375
            apachespark Apache Spark added a comment -

            User 'rajeshbalamohan' has created a pull request for this issue:
            https://github.com/apache/spark/pull/10842

            apachespark Apache Spark added a comment - User 'rajeshbalamohan' has created a pull request for this issue: https://github.com/apache/spark/pull/10842
            dongjoon Dongjoon Hyun added a comment - - edited

            This is fixed since 2.0.0.

            scala> spark.version
            res0: String = 2.0.0
            
            scala> Seq((1,2)).toDF("a", "b").write.option("orc.bloom.filter.columns", "*").orc("/tmp/orc200")
            
            $ hive --orcfiledump /tmp/orc200/part-r-00007-d36ca145-1e23-4d3a-ba99-09506e4ed8cc.snappy.orc
            ...
            Stripes:
              Stripe: offset: 3 data: 12 rows: 1 tail: 92 index: 1390
                Stream: column 0 section ROW_INDEX start: 3 length 11
                Stream: column 0 section BLOOM_FILTER start: 14 length 426
                Stream: column 1 section ROW_INDEX start: 440 length 24
                Stream: column 1 section BLOOM_FILTER start: 464 length 456
                Stream: column 2 section ROW_INDEX start: 920 length 24
                Stream: column 2 section BLOOM_FILTER start: 944 length 449
                Stream: column 1 section DATA start: 1393 length 6
                Stream: column 2 section DATA start: 1399 length 6
            ...
            
            dongjoon Dongjoon Hyun added a comment - - edited This is fixed since 2.0.0. scala> spark.version res0: String = 2.0.0 scala> Seq((1,2)).toDF( "a" , "b" ).write.option( "orc.bloom.filter.columns" , "*" ).orc( "/tmp/orc200" ) $ hive --orcfiledump /tmp/orc200/part-r-00007-d36ca145-1e23-4d3a-ba99-09506e4ed8cc.snappy.orc ... Stripes: Stripe: offset: 3 data: 12 rows: 1 tail: 92 index: 1390 Stream: column 0 section ROW_INDEX start: 3 length 11 Stream: column 0 section BLOOM_FILTER start: 14 length 426 Stream: column 1 section ROW_INDEX start: 440 length 24 Stream: column 1 section BLOOM_FILTER start: 464 length 456 Stream: column 2 section ROW_INDEX start: 920 length 24 Stream: column 2 section BLOOM_FILTER start: 944 length 449 Stream: column 1 section DATA start: 1393 length 6 Stream: column 2 section DATA start: 1399 length 6 ...

            People

              apachespark Apache Spark
              rajesh.balamohan Rajesh Balamohan
              Votes:
              1 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: