@@ -26,8 +26,10 @@ import org.apache.spark.sql.{QueryTest, SQLConf, SaveMode}
26
26
import org .apache .spark .sql .catalyst .expressions .Row
27
27
import org .apache .spark .sql .execution .{ExecutedCommand , PhysicalRDD }
28
28
import org .apache .spark .sql .hive .execution .HiveTableScan
29
+ import org .apache .spark .sql .hive .test .TestHive
29
30
import org .apache .spark .sql .hive .test .TestHive ._
30
31
import org .apache .spark .sql .hive .test .TestHive .implicits ._
32
+ import org .apache .spark .sql .json .JSONRelation
31
33
import org .apache .spark .sql .sources .{InsertIntoDataSource , LogicalRelation }
32
34
import org .apache .spark .sql .parquet .{ParquetRelation2 , ParquetTableScan }
33
35
import org .apache .spark .sql .SaveMode
@@ -390,6 +392,116 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase {
390
392
391
393
sql(" DROP TABLE ms_convert" )
392
394
}
395
+
396
+ test(" Caching converted data source Parquet Relations" ) {
397
+ def checkCached (tableIdentifer : catalog.QualifiedTableName ): Unit = {
398
+ // Converted test_parquet should be cached.
399
+ catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) match {
400
+ case null => fail(" Converted test_parquet should be cached in the cache." )
401
+ case logical @ LogicalRelation (parquetRelation : ParquetRelation2 ) => // OK
402
+ case other =>
403
+ fail(
404
+ " The cached test_parquet should be a Parquet Relation. " +
405
+ s " However, $other is returned form the cache. " )
406
+ }
407
+ }
408
+
409
+ sql(" DROP TABLE IF EXISTS test_insert_parquet" )
410
+ sql(" DROP TABLE IF EXISTS test_parquet_partitioned_cache_test" )
411
+
412
+ sql(
413
+ """
414
+ |create table test_insert_parquet
415
+ |(
416
+ | intField INT,
417
+ | stringField STRING
418
+ |)
419
+ |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
420
+ |STORED AS
421
+ | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
422
+ | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
423
+ """ .stripMargin)
424
+
425
+ var tableIdentifer = catalog.QualifiedTableName (" default" , " test_insert_parquet" )
426
+
427
+ // First, make sure the converted test_parquet is not cached.
428
+ assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null )
429
+ // Table lookup will make the table cached.
430
+ table(" test_insert_parquet" )
431
+ checkCached(tableIdentifer)
432
+ // For insert into non-partitioned table, we will do the conversion,
433
+ // so the converted test_insert_parquet should be cached.
434
+ invalidateTable(" test_insert_parquet" )
435
+ assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null )
436
+ sql(
437
+ """
438
+ |INSERT INTO TABLE test_insert_parquet
439
+ |select a, b from jt
440
+ """ .stripMargin)
441
+ checkCached(tableIdentifer)
442
+ // Make sure we can read the data.
443
+ checkAnswer(
444
+ sql(" select * from test_insert_parquet" ),
445
+ sql(" select a, b from jt" ).collect())
446
+ // Invalidate the cache.
447
+ invalidateTable(" test_insert_parquet" )
448
+ assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null )
449
+
450
+ // Create a partitioned table.
451
+ sql(
452
+ """
453
+ |create table test_parquet_partitioned_cache_test
454
+ |(
455
+ | intField INT,
456
+ | stringField STRING
457
+ |)
458
+ |PARTITIONED BY (date string)
459
+ |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
460
+ |STORED AS
461
+ | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
462
+ | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
463
+ """ .stripMargin)
464
+
465
+ tableIdentifer = catalog.QualifiedTableName (" default" , " test_parquet_partitioned_cache_test" )
466
+ assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null )
467
+ sql(
468
+ """
469
+ |INSERT INTO TABLE test_parquet_partitioned_cache_test
470
+ |PARTITION (date='2015-04-01')
471
+ |select a, b from jt
472
+ """ .stripMargin)
473
+ // Right now, insert into a partitioned Parquet is not supported in data source Parquet.
474
+ // So, we expect it is not cached.
475
+ assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null )
476
+ conf.setConf(SQLConf .PARQUET_USE_DATA_SOURCE_API , " false" )
477
+ sql(
478
+ """
479
+ |INSERT INTO TABLE test_parquet_partitioned_cache_test
480
+ |PARTITION (date='2015-04-02')
481
+ |select a, b from jt
482
+ """ .stripMargin)
483
+ assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null )
484
+ conf.setConf(SQLConf .PARQUET_USE_DATA_SOURCE_API , " true" )
485
+
486
+ // Make sure we can cache the partitioned table.
487
+ table(" test_parquet_partitioned_cache_test" )
488
+ checkCached(tableIdentifer)
489
+ // Make sure we can read the data.
490
+ checkAnswer(
491
+ sql(" select STRINGField, date, intField from test_parquet_partitioned_cache_test" ),
492
+ sql(
493
+ """
494
+ |select b, '2015-04-01', a FROM jt
495
+ |UNION ALL
496
+ |select b, '2015-04-02', a FROM jt
497
+ """ .stripMargin).collect())
498
+
499
+ invalidateTable(" test_parquet_partitioned_cache_test" )
500
+ assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null )
501
+
502
+ sql(" DROP TABLE test_insert_parquet" )
503
+ sql(" DROP TABLE test_parquet_partitioned_cache_test" )
504
+ }
393
505
}
394
506
395
507
class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase {
0 commit comments