aboutsummaryrefslogtreecommitdiff
path: root/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
blob: 4bbf9259192ea519f88d83b22c7cf8dee0e00eb5 (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
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
/*
 * 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.execution

import java.io.{DataInput, DataOutput, File, PrintWriter}
import java.util.{ArrayList, Arrays, Properties}

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hive.ql.exec.UDF
import org.apache.hadoop.hive.ql.udf.{UDAFPercentile, UDFType}
import org.apache.hadoop.hive.ql.udf.generic._
import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject
import org.apache.hadoop.hive.serde2.{AbstractSerDe, SerDeStats}
import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory}
import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
import org.apache.hadoop.io.{LongWritable, Writable}

import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
import org.apache.spark.sql.catalyst.plans.logical.Project
import org.apache.spark.sql.functions.max
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.util.Utils

case class Fields(f1: Int, f2: Int, f3: Int, f4: Int, f5: Int)

// Case classes for the custom UDF's.
case class IntegerCaseClass(i: Int)
case class ListListIntCaseClass(lli: Seq[(Int, Int, Int)])
case class StringCaseClass(s: String)
case class ListStringCaseClass(l: Seq[String])

/**
 * A test suite for Hive custom UDFs.
 */
class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils {

  import spark.udf
  import spark.implicits._

  test("spark sql udf test that returns a struct") {
    udf.register("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5))
    assert(sql(
      """
        |SELECT getStruct(1).f1,
        |       getStruct(1).f2,
        |       getStruct(1).f3,
        |       getStruct(1).f4,
        |       getStruct(1).f5 FROM src LIMIT 1
      """.stripMargin).head() === Row(1, 2, 3, 4, 5))
  }

  test("SPARK-4785 When called with arguments referring column fields, PMOD throws NPE") {
    checkAnswer(
      sql("SELECT PMOD(CAST(key as INT), 10) FROM src LIMIT 1"),
      Row(8)
    )
  }

  test("hive struct udf") {
    sql(
      """
      |CREATE TABLE hiveUDFTestTable (
      |   pair STRUCT<id: INT, value: INT>
      |)
      |PARTITIONED BY (partition STRING)
      |ROW FORMAT SERDE '%s'
      |STORED AS SEQUENCEFILE
    """.
        stripMargin.format(classOf[PairSerDe].getName))

    val location = Utils.getSparkClassLoader.getResource("data/files/testUDF").getFile
    sql(s"""
      ALTER TABLE hiveUDFTestTable
      ADD IF NOT EXISTS PARTITION(partition='testUDF')
      LOCATION '$location'""")

    sql(s"CREATE TEMPORARY FUNCTION testUDF AS '${classOf[PairUDF].getName}'")
    sql("SELECT testUDF(pair) FROM hiveUDFTestTable")
    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDF")
  }

  test("Max/Min on named_struct") {
    checkAnswer(sql(
      """
        |SELECT max(named_struct(
        |           "key", key,
        |           "value", value)).value FROM src
      """.stripMargin), Seq(Row("val_498")))
    checkAnswer(sql(
      """
        |SELECT min(named_struct(
        |           "key", key,
        |           "value", value)).value FROM src
      """.stripMargin), Seq(Row("val_0")))

    // nested struct cases
    checkAnswer(sql(
      """
        |SELECT max(named_struct(
        |           "key", named_struct(
                            "key", key,
                            "value", value),
        |           "value", value)).value FROM src
      """.stripMargin), Seq(Row("val_498")))
    checkAnswer(sql(
      """
        |SELECT min(named_struct(
        |           "key", named_struct(
                           "key", key,
                           "value", value),
        |           "value", value)).value FROM src
      """.stripMargin), Seq(Row("val_0")))
  }

  test("SPARK-6409 UDAF Average test") {
    sql(s"CREATE TEMPORARY FUNCTION test_avg AS '${classOf[GenericUDAFAverage].getName}'")
    checkAnswer(
      sql("SELECT test_avg(1), test_avg(substr(value,5)) FROM src"),
      Seq(Row(1.0, 260.182)))
    sql("DROP TEMPORARY FUNCTION IF EXISTS test_avg")
    hiveContext.reset()
  }

  test("SPARK-2693 udaf aggregates test") {
    checkAnswer(sql("SELECT percentile(key, 1) FROM src LIMIT 1"),
      sql("SELECT max(key) FROM src").collect().toSeq)

    checkAnswer(sql("SELECT percentile(key, array(1, 1)) FROM src LIMIT 1"),
      sql("SELECT array(max(key), max(key)) FROM src").collect().toSeq)
  }

  test("SPARK-16228 Percentile needs explicit cast to double") {
    sql("select percentile(value, cast(0.5 as double)) from values 1,2,3 T(value)")
    sql("select percentile_approx(value, cast(0.5 as double)) from values 1.0,2.0,3.0 T(value)")
    sql("select percentile(value, 0.5) from values 1,2,3 T(value)")
    sql("select percentile_approx(value, 0.5) from values 1.0,2.0,3.0 T(value)")
  }

  test("Generic UDAF aggregates") {

    checkAnswer(sql(
     """
       |SELECT percentile_approx(2, 0.99999),
       |       sum(distinct 1),
       |       count(distinct 1,2,3,4) FROM src LIMIT 1
     """.stripMargin), sql("SELECT 2, 1, 1 FROM src LIMIT 1").collect().toSeq)

    checkAnswer(sql(
      """
        |SELECT ceiling(percentile_approx(distinct key, 0.99999)),
        |       count(distinct key),
        |       sum(distinct key),
        |       count(distinct 1),
        |       sum(distinct 1),
        |       sum(1) FROM src LIMIT 1
      """.stripMargin),
      sql(
        """
          |SELECT max(key),
          |       count(distinct key),
          |       sum(distinct key),
          |       1, 1, sum(1) FROM src LIMIT 1
        """.stripMargin).collect().toSeq)

    checkAnswer(sql(
      """
        |SELECT ceiling(percentile_approx(distinct key, 0.9 + 0.09999)),
        |       count(distinct key), sum(distinct key),
        |       count(distinct 1), sum(distinct 1),
        |       sum(1) FROM src LIMIT 1
      """.stripMargin),
      sql("SELECT max(key), count(distinct key), sum(distinct key), 1, 1, sum(1) FROM src LIMIT 1")
        .collect().toSeq)

    checkAnswer(sql("SELECT ceiling(percentile_approx(key, 0.99999D)) FROM src LIMIT 1"),
      sql("SELECT max(key) FROM src LIMIT 1").collect().toSeq)

    checkAnswer(sql("SELECT percentile_approx(100.0D, array(0.9D, 0.9D)) FROM src LIMIT 1"),
      sql("SELECT array(100, 100) FROM src LIMIT 1").collect().toSeq)
   }

  test("UDFIntegerToString") {
    val testData = spark.sparkContext.parallelize(
      IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF()
    testData.createOrReplaceTempView("integerTable")

    val udfName = classOf[UDFIntegerToString].getName
    sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '$udfName'")
    checkAnswer(
      sql("SELECT testUDFIntegerToString(i) FROM integerTable"),
      Seq(Row("1"), Row("2")))
    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFIntegerToString")

    hiveContext.reset()
  }

  test("UDFToListString") {
    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF()
    testData.createOrReplaceTempView("inputTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFToListString AS '${classOf[UDFToListString].getName}'")
    checkAnswer(
      sql("SELECT testUDFToListString(s) FROM inputTable"),
      Seq(Row(Seq("data1", "data2", "data3"))))

    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToListString")
    hiveContext.reset()
  }

  test("UDFToListInt") {
    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF()
    testData.createOrReplaceTempView("inputTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFToListInt AS '${classOf[UDFToListInt].getName}'")
    checkAnswer(
      sql("SELECT testUDFToListInt(s) FROM inputTable"),
      Seq(Row(Seq(1, 2, 3))))

    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToListInt")
    hiveContext.reset()
  }

  test("UDFToStringIntMap") {
    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF()
    testData.createOrReplaceTempView("inputTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFToStringIntMap " +
      s"AS '${classOf[UDFToStringIntMap].getName}'")
    checkAnswer(
      sql("SELECT testUDFToStringIntMap(s) FROM inputTable"),
      Seq(Row(Map("key1" -> 1, "key2" -> 2, "key3" -> 3))))

    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToStringIntMap")
    hiveContext.reset()
  }

  test("UDFToIntIntMap") {
    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF()
    testData.createOrReplaceTempView("inputTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFToIntIntMap " +
      s"AS '${classOf[UDFToIntIntMap].getName}'")
    checkAnswer(
      sql("SELECT testUDFToIntIntMap(s) FROM inputTable"),
      Seq(Row(Map(1 -> 1, 2 -> 1, 3 -> 1))))

    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToIntIntMap")
    hiveContext.reset()
  }

  test("UDFToListMapStringListInt") {
    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF()
    testData.createOrReplaceTempView("inputTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFToListMapStringListInt " +
      s"AS '${classOf[UDFToListMapStringListInt].getName}'")
    checkAnswer(
      sql("SELECT testUDFToListMapStringListInt(s) FROM inputTable"),
      Seq(Row(Seq(Map("a" -> Seq(1, 2), "b" -> Seq(3, 4))))))

    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToListMapStringListInt")
    hiveContext.reset()
  }

  test("UDFRawList") {
    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF()
    testData.createOrReplaceTempView("inputTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFRawList " +
      s"AS '${classOf[UDFRawList].getName}'")
    val err = intercept[AnalysisException](sql("SELECT testUDFRawList(s) FROM inputTable"))
    assert(err.getMessage.contains(
      "Raw list type in java is unsupported because Spark cannot infer the element type."))

    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFRawList")
    hiveContext.reset()
  }

  test("UDFRawMap") {
    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF()
    testData.createOrReplaceTempView("inputTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFRawMap " +
      s"AS '${classOf[UDFRawMap].getName}'")
    val err = intercept[AnalysisException](sql("SELECT testUDFRawMap(s) FROM inputTable"))
    assert(err.getMessage.contains(
      "Raw map type in java is unsupported because Spark cannot infer key and value types."))

    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFRawMap")
    hiveContext.reset()
  }

  test("UDFWildcardList") {
    val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF()
    testData.createOrReplaceTempView("inputTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFWildcardList " +
      s"AS '${classOf[UDFWildcardList].getName}'")
    val err = intercept[AnalysisException](sql("SELECT testUDFWildcardList(s) FROM inputTable"))
    assert(err.getMessage.contains(
      "Collection types with wildcards (e.g. List<?> or Map<?, ?>) are unsupported " +
        "because Spark cannot infer the data type for these type parameters."))

    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFWildcardList")
    hiveContext.reset()
  }

  test("UDFListListInt") {
    val testData = spark.sparkContext.parallelize(
      ListListIntCaseClass(Nil) ::
      ListListIntCaseClass(Seq((1, 2, 3))) ::
      ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF()
    testData.createOrReplaceTempView("listListIntTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'")
    checkAnswer(
      sql("SELECT testUDFListListInt(lli) FROM listListIntTable"),
      Seq(Row(0), Row(2), Row(13)))
    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListListInt")

    hiveContext.reset()
  }

  test("UDFListString") {
    val testData = spark.sparkContext.parallelize(
      ListStringCaseClass(Seq("a", "b", "c")) ::
      ListStringCaseClass(Seq("d", "e")) :: Nil).toDF()
    testData.createOrReplaceTempView("listStringTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'")
    checkAnswer(
      sql("SELECT testUDFListString(l) FROM listStringTable"),
      Seq(Row("a,b,c"), Row("d,e")))
    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListString")

    hiveContext.reset()
  }

  test("UDFStringString") {
    val testData = spark.sparkContext.parallelize(
      StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil).toDF()
    testData.createOrReplaceTempView("stringTable")

    sql(s"CREATE TEMPORARY FUNCTION testStringStringUDF AS '${classOf[UDFStringString].getName}'")
    checkAnswer(
      sql("SELECT testStringStringUDF(\"hello\", s) FROM stringTable"),
      Seq(Row("hello world"), Row("hello goodbye")))

    checkAnswer(
      sql("SELECT testStringStringUDF(\"\", testStringStringUDF(\"hello\", s)) FROM stringTable"),
      Seq(Row(" hello world"), Row(" hello goodbye")))

    sql("DROP TEMPORARY FUNCTION IF EXISTS testStringStringUDF")

    hiveContext.reset()
  }

  test("UDFTwoListList") {
    val testData = spark.sparkContext.parallelize(
      ListListIntCaseClass(Nil) ::
      ListListIntCaseClass(Seq((1, 2, 3))) ::
      ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) ::
      Nil).toDF()
    testData.createOrReplaceTempView("TwoListTable")

    sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'")
    checkAnswer(
      sql("SELECT testUDFTwoListList(lli, lli) FROM TwoListTable"),
      Seq(Row("0, 0"), Row("2, 2"), Row("13, 13")))
    sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFTwoListList")

    hiveContext.reset()
  }

  test("non-deterministic children of UDF") {
    withUserDefinedFunction("testStringStringUDF" -> true, "testGenericUDFHash" -> true) {
      // HiveSimpleUDF
      sql(s"CREATE TEMPORARY FUNCTION testStringStringUDF AS '${classOf[UDFStringString].getName}'")
      val df1 = sql("SELECT testStringStringUDF(rand(), \"hello\")")
      assert(!df1.logicalPlan.asInstanceOf[Project].projectList.forall(_.deterministic))

      // HiveGenericUDF
      sql(s"CREATE TEMPORARY FUNCTION testGenericUDFHash AS '${classOf[GenericUDFHash].getName}'")
      val df2 = sql("SELECT testGenericUDFHash(rand())")
      assert(!df2.logicalPlan.asInstanceOf[Project].projectList.forall(_.deterministic))
    }
  }

  test("Hive UDFs with insufficient number of input arguments should trigger an analysis error") {
    Seq((1, 2)).toDF("a", "b").createOrReplaceTempView("testUDF")

    {
      // HiveSimpleUDF
      sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'")
      val message = intercept[AnalysisException] {
        sql("SELECT testUDFTwoListList() FROM testUDF")
      }.getMessage
      assert(message.contains("No handler for Hive UDF"))
      sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFTwoListList")
    }

    {
      // HiveGenericUDF
      sql(s"CREATE TEMPORARY FUNCTION testUDFAnd AS '${classOf[GenericUDFOPAnd].getName}'")
      val message = intercept[AnalysisException] {
        sql("SELECT testUDFAnd() FROM testUDF")
      }.getMessage
      assert(message.contains("No handler for Hive UDF"))
      sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFAnd")
    }

    {
      // Hive UDAF
      sql(s"CREATE TEMPORARY FUNCTION testUDAFPercentile AS '${classOf[UDAFPercentile].getName}'")
      val message = intercept[AnalysisException] {
        sql("SELECT testUDAFPercentile(a) FROM testUDF GROUP BY b")
      }.getMessage
      assert(message.contains("No handler for Hive UDF"))
      sql("DROP TEMPORARY FUNCTION IF EXISTS testUDAFPercentile")
    }

    {
      // AbstractGenericUDAFResolver
      sql(s"CREATE TEMPORARY FUNCTION testUDAFAverage AS '${classOf[GenericUDAFAverage].getName}'")
      val message = intercept[AnalysisException] {
        sql("SELECT testUDAFAverage() FROM testUDF GROUP BY b")
      }.getMessage
      assert(message.contains("No handler for Hive UDF"))
      sql("DROP TEMPORARY FUNCTION IF EXISTS testUDAFAverage")
    }

    {
      // Hive UDTF
      sql(s"CREATE TEMPORARY FUNCTION testUDTFExplode AS '${classOf[GenericUDTFExplode].getName}'")
      val message = intercept[AnalysisException] {
        sql("SELECT testUDTFExplode() FROM testUDF")
      }.getMessage
      assert(message.contains("No handler for Hive UDF"))
      sql("DROP TEMPORARY FUNCTION IF EXISTS testUDTFExplode")
    }

    spark.catalog.dropTempView("testUDF")
  }

  test("Hive UDF in group by") {
    withTempView("tab1") {
      Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1")
      sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'")
      val count = sql("select testUDFToDate(cast(test_date as timestamp))" +
        " from tab1 group by testUDFToDate(cast(test_date as timestamp))").count()
      sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToDate")
      assert(count == 1)
    }
  }

  test("SPARK-11522 select input_file_name from non-parquet table") {

    withTempDir { tempDir =>

      // EXTERNAL OpenCSVSerde table pointing to LOCATION

      val file1 = new File(tempDir + "/data1")
      val writer1 = new PrintWriter(file1)
      writer1.write("1,2")
      writer1.close()

      val file2 = new File(tempDir + "/data2")
      val writer2 = new PrintWriter(file2)
      writer2.write("1,2")
      writer2.close()

      sql(
        s"""CREATE EXTERNAL TABLE csv_table(page_id INT, impressions INT)
        ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde'
        WITH SERDEPROPERTIES (
          \"separatorChar\" = \",\",
          \"quoteChar\"     = \"\\\"\",
          \"escapeChar\"    = \"\\\\\")
        LOCATION '${tempDir.toURI}'
      """)

      val answer1 =
        sql("SELECT input_file_name() FROM csv_table").head().getString(0)
      assert(answer1.contains("data1") || answer1.contains("data2"))

      val count1 = sql("SELECT input_file_name() FROM csv_table").distinct().count()
      assert(count1 == 2)
      sql("DROP TABLE csv_table")

      // EXTERNAL pointing to LOCATION

      sql(
        s"""CREATE EXTERNAL TABLE external_t5 (c1 int, c2 int)
        ROW FORMAT DELIMITED FIELDS TERMINATED BY ','
        LOCATION '${tempDir.toURI}'
      """)

      val answer2 =
        sql("SELECT input_file_name() as file FROM external_t5").head().getString(0)
      assert(answer1.contains("data1") || answer1.contains("data2"))

      val count2 = sql("SELECT input_file_name() as file FROM external_t5").distinct().count
      assert(count2 == 2)
      sql("DROP TABLE external_t5")
    }

    withTempDir { tempDir =>

      // External parquet pointing to LOCATION

      val parquetLocation = s"${tempDir.toURI}/external_parquet"
      sql("SELECT 1, 2").write.parquet(parquetLocation)

      sql(
        s"""CREATE EXTERNAL TABLE external_parquet(c1 int, c2 int)
        STORED AS PARQUET
        LOCATION '$parquetLocation'
      """)

      val answer3 =
        sql("SELECT input_file_name() as file FROM external_parquet").head().getString(0)
      assert(answer3.contains("external_parquet"))

      val count3 = sql("SELECT input_file_name() as file FROM external_parquet").distinct().count
      assert(count3 == 1)
      sql("DROP TABLE external_parquet")
    }

    // Non-External parquet pointing to /tmp/...
    sql("CREATE TABLE parquet_tmp STORED AS parquet AS SELECT 1, 2")

    val answer4 =
      sql("SELECT input_file_name() as file FROM parquet_tmp").head().getString(0)
    assert(answer4.contains("parquet_tmp"))

    val count4 = sql("SELECT input_file_name() as file FROM parquet_tmp").distinct().count
    assert(count4 == 1)
    sql("DROP TABLE parquet_tmp")
  }

  test("Hive Stateful UDF") {
    withUserDefinedFunction("statefulUDF" -> true, "statelessUDF" -> true) {
      sql(s"CREATE TEMPORARY FUNCTION statefulUDF AS '${classOf[StatefulUDF].getName}'")
      sql(s"CREATE TEMPORARY FUNCTION statelessUDF AS '${classOf[StatelessUDF].getName}'")
      val testData = spark.range(10).repartition(1)

      // Expected Max(s) is 10 as statefulUDF returns the sequence number starting from 1.
      checkAnswer(testData.selectExpr("statefulUDF() as s").agg(max($"s")), Row(10))

      // Expected Max(s) is 5 as statefulUDF returns the sequence number starting from 1,
      // and the data is evenly distributed into 2 partitions.
      checkAnswer(testData.repartition(2)
        .selectExpr("statefulUDF() as s").agg(max($"s")), Row(5))

      // Expected Max(s) is 1, as stateless UDF is deterministic and foldable and replaced
      // by constant 1 by ConstantFolding optimizer.
      checkAnswer(testData.selectExpr("statelessUDF() as s").agg(max($"s")), Row(1))
    }
  }
}

class TestPair(x: Int, y: Int) extends Writable with Serializable {
  def this() = this(0, 0)
  var entry: (Int, Int) = (x, y)

  override def write(output: DataOutput): Unit = {
    output.writeInt(entry._1)
    output.writeInt(entry._2)
  }

  override def readFields(input: DataInput): Unit = {
    val x = input.readInt()
    val y = input.readInt()
    entry = (x, y)
  }
}

class PairSerDe extends AbstractSerDe {
  override def initialize(p1: Configuration, p2: Properties): Unit = {}

  override def getObjectInspector: ObjectInspector = {
    ObjectInspectorFactory
      .getStandardStructObjectInspector(
        Arrays.asList("pair"),
        Arrays.asList(ObjectInspectorFactory.getStandardStructObjectInspector(
          Arrays.asList("id", "value"),
          Arrays.asList(PrimitiveObjectInspectorFactory.javaIntObjectInspector,
                        PrimitiveObjectInspectorFactory.javaIntObjectInspector))
    ))
  }

  override def getSerializedClass: Class[_ <: Writable] = classOf[TestPair]

  override def getSerDeStats: SerDeStats = null

  override def serialize(p1: scala.Any, p2: ObjectInspector): Writable = null

  override def deserialize(value: Writable): AnyRef = {
    val pair = value.asInstanceOf[TestPair]

    val row = new ArrayList[ArrayList[AnyRef]]
    row.add(new ArrayList[AnyRef](2))
    row.get(0).add(Integer.valueOf(pair.entry._1))
    row.get(0).add(Integer.valueOf(pair.entry._2))

    row
  }
}

class PairUDF extends GenericUDF {
  override def initialize(p1: Array[ObjectInspector]): ObjectInspector =
    ObjectInspectorFactory.getStandardStructObjectInspector(
      Arrays.asList("id", "value"),
      Arrays.asList(PrimitiveObjectInspectorFactory.javaIntObjectInspector,
                    PrimitiveObjectInspectorFactory.javaIntObjectInspector)
  )

  override def evaluate(args: Array[DeferredObject]): AnyRef = {
    Integer.valueOf(args(0).get.asInstanceOf[TestPair].entry._2)
  }

  override def getDisplayString(p1: Array[String]): String = ""
}

@UDFType(stateful = true)
class StatefulUDF extends UDF {
  private val result = new LongWritable(0)

  def evaluate(): LongWritable = {
    result.set(result.get() + 1)
    result
  }
}

class StatelessUDF extends UDF {
  private val result = new LongWritable(0)

  def evaluate(): LongWritable = {
    result.set(result.get() + 1)
    result
  }
}