Github user dongjoon-hyun commented on a diff in the pull request:
https://github.com/apache/spark/pull/22418#discussion_r217970877
--- Diff:
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
---
@@ -50,6 +55,66 @@ abstract class OrcSuite extends OrcTest with
BeforeAndAfterAll {
.createOrReplaceTempView("orc_temp_table")
}
+ protected def testBloomFilterCreation(bloomFilterKind: Kind) {
+ val tableName = "bloomFilter"
+
+ withTempDir { dir =>
+ withTable(tableName) {
+ val sqlStatement = orcImp match {
+ case "native" =>
+ s"""
+ |CREATE TABLE $tableName (a INT, b STRING)
+ |USING ORC
+ |OPTIONS (
+ | path '${dir.toURI}',
+ | orc.bloom.filter.columns '*',
+ | orc.bloom.filter.fpp 0.1
+ |)
+ """.stripMargin
+ case "hive" =>
+ s"""
+ |CREATE TABLE $tableName (a INT, b STRING)
+ |STORED AS ORC
+ |LOCATION '${dir.toURI}'
+ |TBLPROPERTIES (
+ | orc.bloom.filter.columns='*',
+ | orc.bloom.filter.fpp=0.1
+ |)
+ """.stripMargin
+ case impl =>
+ throw new UnsupportedOperationException(s"Unknown ORC
implementation: $impl")
+ }
+
+ sql(sqlStatement)
+ sql(s"INSERT INTO $tableName VALUES (1, 'str')")
+
+ val partFiles = dir.listFiles()
+ .filter(f => f.isFile && !f.getName.startsWith(".") &&
!f.getName.startsWith("_"))
+ assert(partFiles.length === 1)
+
+ val orcFilePath = new Path(partFiles.head.getAbsolutePath)
+ val readerOptions = OrcFile.readerOptions(new Configuration())
+ val reader = OrcFile.createReader(orcFilePath, readerOptions)
+ var recordReader: RecordReaderImpl = null
+ try {
+ recordReader = reader.rows.asInstanceOf[RecordReaderImpl]
+
+ // BloomFilter array is created for all types; `struct`, int
(`a`), string (`b`)
+ val sargColumns = Array(true, true, true)
+ val orcIndex = recordReader.readRowIndex(0, null, sargColumns)
+
+ // Check the types and counts of bloom filters
+ assert(orcIndex.getBloomFilterKinds.forall(_ ===
bloomFilterKind))
--- End diff --
```scala
scala> spark.version
res1: String = 2.3.1
scala> sql("set spark.sql.orc.impl=native")
scala> sql("create table bf using orc options(orc.bloom.filter.columns '*')
as select 1 a")
scala> sql("create table bf2 using orc options(orc.bloom.filter.columns
'*', orc.bloom.filter.write.version 'original') as select 1 a")
```
```scala
$ orc-tools meta spark-warehouse/bf | grep BLOOM_FILTER
Stream: column 0 section BLOOM_FILTER_UTF8 start: 14 length 381
Stream: column 1 section BLOOM_FILTER_UTF8 start: 419 length 402
$ orc-tools meta spark-warehouse/bf2 | grep BLOOM_FILTER
Stream: column 0 section BLOOM_FILTER start: 14 length 426
Stream: column 0 section BLOOM_FILTER_UTF8 start: 440 length 381
Stream: column 1 section BLOOM_FILTER start: 845 length 456
Stream: column 1 section BLOOM_FILTER_UTF8 start: 1301 length 402
```
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]