aboutsummaryrefslogtreecommitdiff
path: root/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
blob: 4fb78ac02cb55c5b4366b0894a2cc93bb4c1badd (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.sql.hive.orc

import java.io.File
import java.nio.charset.StandardCharsets

import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hadoop.hive.ql.io.orc.CompressionKind
import org.scalatest.BeforeAndAfterAll

import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.hive.HiveUtils
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.apache.spark.sql.internal.SQLConf

case class AllDataTypesWithNonPrimitiveType(
    stringField: String,
    intField: Int,
    longField: Long,
    floatField: Float,
    doubleField: Double,
    shortField: Short,
    byteField: Byte,
    booleanField: Boolean,
    array: Seq[Int],
    arrayContainsNull: Seq[Option[Int]],
    map: Map[Int, Long],
    mapValueContainsNull: Map[Int, Option[Long]],
    data: (Seq[Int], (Int, String)))

case class BinaryData(binaryData: Array[Byte])

case class Contact(name: String, phone: String)

case class Person(name: String, age: Int, contacts: Seq[Contact])

class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {

  def getTempFilePath(prefix: String, suffix: String = ""): File = {
    val tempFile = File.createTempFile(prefix, suffix)
    tempFile.delete()
    tempFile
  }

  test("Read/write All Types") {
    val data = (0 to 255).map { i =>
      (s"$i", i, i.toLong, i.toFloat, i.toDouble, i.toShort, i.toByte, i % 2 == 0)
    }

    withOrcFile(data) { file =>
      checkAnswer(
        sqlContext.read.orc(file),
        data.toDF().collect())
    }
  }

  test("Read/write binary data") {
    withOrcFile(BinaryData("test".getBytes(StandardCharsets.UTF_8)) :: Nil) { file =>
      val bytes = read.orc(file).head().getAs[Array[Byte]](0)
      assert(new String(bytes, StandardCharsets.UTF_8) === "test")
    }
  }

  test("Read/write all types with non-primitive type") {
    val data: Seq[AllDataTypesWithNonPrimitiveType] = (0 to 255).map { i =>
      AllDataTypesWithNonPrimitiveType(
        s"$i", i, i.toLong, i.toFloat, i.toDouble, i.toShort, i.toByte, i % 2 == 0,
        0 until i,
        (0 until i).map(Option(_).filter(_ % 3 == 0)),
        (0 until i).map(i => i -> i.toLong).toMap,
        (0 until i).map(i => i -> Option(i.toLong)).toMap + (i -> None),
        (0 until i, (i, s"$i")))
    }

    withOrcFile(data) { file =>
      checkAnswer(
        read.orc(file),
        data.toDF().collect())
    }
  }

  test("Creating case class RDD table") {
    val data = (1 to 100).map(i => (i, s"val_$i"))
    sparkContext.parallelize(data).toDF().registerTempTable("t")
    withTempTable("t") {
      checkAnswer(sql("SELECT * FROM t"), data.toDF().collect())
    }
  }

  test("Simple selection form ORC table") {
    val data = (1 to 10).map { i =>
      Person(s"name_$i", i, (0 to 1).map { m => Contact(s"contact_$m", s"phone_$m") })
    }

    withOrcTable(data, "t") {
      // ppd:
      // leaf-0 = (LESS_THAN_EQUALS age 5)
      // expr = leaf-0
      assert(sql("SELECT name FROM t WHERE age <= 5").count() === 5)

      // ppd:
      // leaf-0 = (LESS_THAN_EQUALS age 5)
      // expr = (not leaf-0)
      assertResult(10) {
        sql("SELECT name, contacts FROM t where age > 5")
          .rdd
          .flatMap(_.getAs[Seq[_]]("contacts"))
          .count()
      }

      // ppd:
      // leaf-0 = (LESS_THAN_EQUALS age 5)
      // leaf-1 = (LESS_THAN age 8)
      // expr = (and (not leaf-0) leaf-1)
      {
        val df = sql("SELECT name, contacts FROM t WHERE age > 5 AND age < 8")
        assert(df.count() === 2)
        assertResult(4) {
          df.rdd.flatMap(_.getAs[Seq[_]]("contacts")).count()
        }
      }

      // ppd:
      // leaf-0 = (LESS_THAN age 2)
      // leaf-1 = (LESS_THAN_EQUALS age 8)
      // expr = (or leaf-0 (not leaf-1))
      {
        val df = sql("SELECT name, contacts FROM t WHERE age < 2 OR age > 8")
        assert(df.count() === 3)
        assertResult(6) {
          df.rdd.flatMap(_.getAs[Seq[_]]("contacts")).count()
        }
      }
    }
  }

  test("save and load case class RDD with `None`s as orc") {
    val data = (
      None: Option[Int],
      None: Option[Long],
      None: Option[Float],
      None: Option[Double],
      None: Option[Boolean]
    ) :: Nil

    withOrcFile(data) { file =>
      checkAnswer(
        read.orc(file),
        Row(Seq.fill(5)(null): _*))
    }
  }

  // We only support zlib in Hive 0.12.0 now
  test("Default compression options for writing to an ORC file") {
    withOrcFile((1 to 100).map(i => (i, s"val_$i"))) { file =>
      assertResult(CompressionKind.ZLIB) {
        OrcFileOperator.getFileReader(file).get.getCompression
      }
    }
  }

  // Following codec is supported in hive-0.13.1, ignore it now
  ignore("Other compression options for writing to an ORC file - 0.13.1 and above") {
    val data = (1 to 100).map(i => (i, s"val_$i"))
    val conf = sparkContext.hadoopConfiguration

    conf.set(ConfVars.HIVE_ORC_DEFAULT_COMPRESS.varname, "SNAPPY")
    withOrcFile(data) { file =>
      assertResult(CompressionKind.SNAPPY) {
        OrcFileOperator.getFileReader(file).get.getCompression
      }
    }

    conf.set(ConfVars.HIVE_ORC_DEFAULT_COMPRESS.varname, "NONE")
    withOrcFile(data) { file =>
      assertResult(CompressionKind.NONE) {
        OrcFileOperator.getFileReader(file).get.getCompression
      }
    }

    conf.set(ConfVars.HIVE_ORC_DEFAULT_COMPRESS.varname, "LZO")
    withOrcFile(data) { file =>
      assertResult(CompressionKind.LZO) {
        OrcFileOperator.getFileReader(file).get.getCompression
      }
    }
  }

  test("simple select queries") {
    withOrcTable((0 until 10).map(i => (i, i.toString)), "t") {
      checkAnswer(
        sql("SELECT `_1` FROM t where t.`_1` > 5"),
        (6 until 10).map(Row.apply(_)))

      checkAnswer(
        sql("SELECT `_1` FROM t as tmp where tmp.`_1` < 5"),
        (0 until 5).map(Row.apply(_)))
    }
  }

  test("appending") {
    val data = (0 until 10).map(i => (i, i.toString))
    createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp")
    withOrcTable(data, "t") {
      sql("INSERT INTO TABLE t SELECT * FROM tmp")
      checkAnswer(table("t"), (data ++ data).map(Row.fromTuple))
    }
    sessionState.catalog.dropTable(TableIdentifier("tmp"), ignoreIfNotExists = true)
  }

  test("overwriting") {
    val data = (0 until 10).map(i => (i, i.toString))
    createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp")
    withOrcTable(data, "t") {
      sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp")
      checkAnswer(table("t"), data.map(Row.fromTuple))
    }
    sessionState.catalog.dropTable(TableIdentifier("tmp"), ignoreIfNotExists = true)
  }

  test("self-join") {
    // 4 rows, cells of column 1 of row 2 and row 4 are null
    val data = (1 to 4).map { i =>
      val maybeInt = if (i % 2 == 0) None else Some(i)
      (maybeInt, i.toString)
    }

    withOrcTable(data, "t") {
      val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x.`_1` = y.`_1`")
      val queryOutput = selfJoin.queryExecution.analyzed.output

      assertResult(4, "Field count mismatches")(queryOutput.size)
      assertResult(2, "Duplicated expression ID in query plan:\n $selfJoin") {
        queryOutput.filter(_.name == "_1").map(_.exprId).size
      }

      checkAnswer(selfJoin, List(Row(1, "1", 1, "1"), Row(3, "3", 3, "3")))
    }
  }

  test("nested data - struct with array field") {
    val data = (1 to 10).map(i => Tuple1((i, Seq("val_$i"))))
    withOrcTable(data, "t") {
      checkAnswer(sql("SELECT `_1`.`_2`[0] FROM t"), data.map {
        case Tuple1((_, Seq(string))) => Row(string)
      })
    }
  }

  test("nested data - array of struct") {
    val data = (1 to 10).map(i => Tuple1(Seq(i -> "val_$i")))
    withOrcTable(data, "t") {
      checkAnswer(sql("SELECT `_1`[0].`_2` FROM t"), data.map {
        case Tuple1(Seq((_, string))) => Row(string)
      })
    }
  }

  test("columns only referenced by pushed down filters should remain") {
    withOrcTable((1 to 10).map(Tuple1.apply), "t") {
      checkAnswer(sql("SELECT `_1` FROM t WHERE `_1` < 10"), (1 to 9).map(Row.apply(_)))
    }
  }

  test("SPARK-5309 strings stored using dictionary compression in orc") {
    withOrcTable((0 until 1000).map(i => ("same", "run_" + i / 100, 1)), "t") {
      checkAnswer(
        sql("SELECT `_1`, `_2`, SUM(`_3`) FROM t GROUP BY `_1`, `_2`"),
        (0 until 10).map(i => Row("same", "run_" + i, 100)))

      checkAnswer(
        sql("SELECT `_1`, `_2`, SUM(`_3`) FROM t WHERE `_2` = 'run_5' GROUP BY `_1`, `_2`"),
        List(Row("same", "run_5", 100)))
    }
  }

  test("SPARK-9170: Don't implicitly lowercase of user-provided columns") {
    withTempPath { dir =>
      val path = dir.getCanonicalPath

      sqlContext.range(0, 10).select('id as "Acol").write.format("orc").save(path)
      sqlContext.read.format("orc").load(path).schema("Acol")
      intercept[IllegalArgumentException] {
        sqlContext.read.format("orc").load(path).schema("acol")
      }
      checkAnswer(sqlContext.read.format("orc").load(path).select("acol").sort("acol"),
        (0 until 10).map(Row(_)))
    }
  }

  test("SPARK-8501: Avoids discovery schema from empty ORC files") {
    withTempPath { dir =>
      val path = dir.getCanonicalPath

      withTable("empty_orc") {
        withTempTable("empty", "single") {
          sqlContext.sql(
            s"""CREATE TABLE empty_orc(key INT, value STRING)
               |STORED AS ORC
               |LOCATION '$path'
             """.stripMargin)

          val emptyDF = Seq.empty[(Int, String)].toDF("key", "value").coalesce(1)
          emptyDF.registerTempTable("empty")

          // This creates 1 empty ORC file with Hive ORC SerDe.  We are using this trick because
          // Spark SQL ORC data source always avoids write empty ORC files.
          sqlContext.sql(
            s"""INSERT INTO TABLE empty_orc
               |SELECT key, value FROM empty
             """.stripMargin)

          val errorMessage = intercept[AnalysisException] {
            sqlContext.read.orc(path)
          }.getMessage

          assert(errorMessage.contains("Unable to infer schema for ORC"))

          val singleRowDF = Seq((0, "foo")).toDF("key", "value").coalesce(1)
          singleRowDF.registerTempTable("single")

          sqlContext.sql(
            s"""INSERT INTO TABLE empty_orc
               |SELECT key, value FROM single
             """.stripMargin)

          val df = sqlContext.read.orc(path)
          assert(df.schema === singleRowDF.schema.asNullable)
          checkAnswer(df, singleRowDF)
        }
      }
    }
  }

  test("SPARK-10623 Enable ORC PPD") {
    withTempPath { dir =>
      withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") {
        import testImplicits._
        val path = dir.getCanonicalPath

        // For field "a", the first column has odds integers. This is to check the filtered count
        // when `isNull` is performed. For Field "b", `isNotNull` of ORC file filters rows
        // only when all the values are null (maybe this works differently when the data
        // or query is complicated). So, simply here a column only having `null` is added.
        val data = (0 until 10).map { i =>
          val maybeInt = if (i % 2 == 0) None else Some(i)
          val nullValue: Option[String] = None
          (maybeInt, nullValue)
        }
        // It needs to repartition data so that we can have several ORC files
        // in order to skip stripes in ORC.
        createDataFrame(data).toDF("a", "b").repartition(10).write.orc(path)
        val df = sqlContext.read.orc(path)

        def checkPredicate(pred: Column, answer: Seq[Row]): Unit = {
          val sourceDf = stripSparkFilter(df.where(pred))
          val data = sourceDf.collect().toSet
          val expectedData = answer.toSet

          // When a filter is pushed to ORC, ORC can apply it to rows. So, we can check
          // the number of rows returned from the ORC to make sure our filter pushdown work.
          // A tricky part is, ORC does not process filter rows fully but return some possible
          // results. So, this checks if the number of result is less than the original count
          // of data, and then checks if it contains the expected data.
          assert(
            sourceDf.count < 10 && expectedData.subsetOf(data),
            s"No data was filtered for predicate: $pred")
        }

        checkPredicate('a === 5, List(5).map(Row(_, null)))
        checkPredicate('a <=> 5, List(5).map(Row(_, null)))
        checkPredicate('a < 5, List(1, 3).map(Row(_, null)))
        checkPredicate('a <= 5, List(1, 3, 5).map(Row(_, null)))
        checkPredicate('a > 5, List(7, 9).map(Row(_, null)))
        checkPredicate('a >= 5, List(5, 7, 9).map(Row(_, null)))
        checkPredicate('a.isNull, List(null).map(Row(_, null)))
        checkPredicate('b.isNotNull, List())
        checkPredicate('a.isin(3, 5, 7), List(3, 5, 7).map(Row(_, null)))
        checkPredicate('a > 0 && 'a < 3, List(1).map(Row(_, null)))
        checkPredicate('a < 1 || 'a > 8, List(9).map(Row(_, null)))
        checkPredicate(!('a > 3), List(1, 3).map(Row(_, null)))
        checkPredicate(!('a > 0 && 'a < 3), List(3, 5, 7, 9).map(Row(_, null)))
      }
    }
  }

  test("SPARK-14070 Use ORC data source for SQL queries on ORC tables") {
    withTempPath { dir =>
      withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true",
        HiveUtils.CONVERT_METASTORE_ORC.key -> "true") {
        val path = dir.getCanonicalPath

        withTable("dummy_orc") {
          withTempTable("single") {
            sqlContext.sql(
              s"""CREATE TABLE dummy_orc(key INT, value STRING)
                  |STORED AS ORC
                  |LOCATION '$path'
               """.stripMargin)

            val singleRowDF = Seq((0, "foo")).toDF("key", "value").coalesce(1)
            singleRowDF.registerTempTable("single")

            sqlContext.sql(
              s"""INSERT INTO TABLE dummy_orc
                  |SELECT key, value FROM single
               """.stripMargin)

            val df = sqlContext.sql("SELECT * FROM dummy_orc WHERE key=0")
            checkAnswer(df, singleRowDF)

            val queryExecution = df.queryExecution
            queryExecution.analyzed.collectFirst {
              case _: LogicalRelation => ()
            }.getOrElse {
              fail(s"Expecting the query plan to have LogicalRelation, but got:\n$queryExecution")
            }
          }
        }
      }
    }
  }
}