aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
blob: e66fe97afad4594d8913537bfdea62c69cd5dac2 (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
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
/*
 * 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

import scala.collection.mutable.HashSet
import scala.concurrent.duration._
import scala.language.postfixOps

import org.scalatest.concurrent.Eventually._

import org.apache.spark.CleanerListener
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
import org.apache.spark.sql.execution.{RDDScanExec, SparkPlan}
import org.apache.spark.sql.execution.columnar._
import org.apache.spark.sql.execution.exchange.ShuffleExchange
import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
import org.apache.spark.storage.{RDDBlockId, StorageLevel}
import org.apache.spark.util.{AccumulatorContext, Utils}

private case class BigData(s: String)

class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext {
  import testImplicits._

  setupTestData()

  override def afterEach(): Unit = {
    try {
      spark.catalog.clearCache()
    } finally {
      super.afterEach()
    }
  }

  def rddIdOf(tableName: String): Int = {
    val plan = spark.table(tableName).queryExecution.sparkPlan
    plan.collect {
      case InMemoryTableScanExec(_, _, relation) =>
        relation.cachedColumnBuffers.id
      case _ =>
        fail(s"Table $tableName is not cached\n" + plan)
    }.head
  }

  def isMaterialized(rddId: Int): Boolean = {
    val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0))
    maybeBlock.foreach(_ => sparkContext.env.blockManager.releaseLock(RDDBlockId(rddId, 0)))
    maybeBlock.nonEmpty
  }

  private def getNumInMemoryRelations(ds: Dataset[_]): Int = {
    val plan = ds.queryExecution.withCachedData
    var sum = plan.collect { case _: InMemoryRelation => 1 }.sum
    plan.transformAllExpressions {
      case e: SubqueryExpression =>
        sum += getNumInMemoryRelations(e.plan)
        e
    }
    sum
  }

  private def getNumInMemoryTablesRecursively(plan: SparkPlan): Int = {
    plan.collect {
      case InMemoryTableScanExec(_, _, relation) =>
        getNumInMemoryTablesRecursively(relation.child) + 1
    }.sum
  }

  test("withColumn doesn't invalidate cached dataframe") {
    var evalCount = 0
    val myUDF = udf((x: String) => { evalCount += 1; "result" })
    val df = Seq(("test", 1)).toDF("s", "i").select(myUDF($"s"))
    df.cache()

    df.collect()
    assert(evalCount === 1)

    df.collect()
    assert(evalCount === 1)

    val df2 = df.withColumn("newColumn", lit(1))
    df2.collect()

    // We should not reevaluate the cached dataframe
    assert(evalCount === 1)
  }

  test("cache temp table") {
    withTempView("tempTable") {
      testData.select('key).createOrReplaceTempView("tempTable")
      assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0)
      spark.catalog.cacheTable("tempTable")
      assertCached(sql("SELECT COUNT(*) FROM tempTable"))
      spark.catalog.uncacheTable("tempTable")
    }
  }

  test("unpersist an uncached table will not raise exception") {
    assert(None == spark.sharedState.cacheManager.lookupCachedData(testData))
    testData.unpersist(blocking = true)
    assert(None == spark.sharedState.cacheManager.lookupCachedData(testData))
    testData.unpersist(blocking = false)
    assert(None == spark.sharedState.cacheManager.lookupCachedData(testData))
    testData.persist()
    assert(None != spark.sharedState.cacheManager.lookupCachedData(testData))
    testData.unpersist(blocking = true)
    assert(None == spark.sharedState.cacheManager.lookupCachedData(testData))
    testData.unpersist(blocking = false)
    assert(None == spark.sharedState.cacheManager.lookupCachedData(testData))
  }

  test("cache table as select") {
    withTempView("tempTable") {
      sql("CACHE TABLE tempTable AS SELECT key FROM testData")
      assertCached(sql("SELECT COUNT(*) FROM tempTable"))
      spark.catalog.uncacheTable("tempTable")
    }
  }

  test("uncaching temp table") {
    testData.select('key).createOrReplaceTempView("tempTable1")
    testData.select('key).createOrReplaceTempView("tempTable2")
    spark.catalog.cacheTable("tempTable1")

    assertCached(sql("SELECT COUNT(*) FROM tempTable1"))
    assertCached(sql("SELECT COUNT(*) FROM tempTable2"))

    // Is this valid?
    spark.catalog.uncacheTable("tempTable2")

    // Should this be cached?
    assertCached(sql("SELECT COUNT(*) FROM tempTable1"), 0)
  }

  test("too big for memory") {
    val data = "*" * 1000
    sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF()
      .createOrReplaceTempView("bigData")
    spark.table("bigData").persist(StorageLevel.MEMORY_AND_DISK)
    assert(spark.table("bigData").count() === 200000L)
    spark.table("bigData").unpersist(blocking = true)
  }

  test("calling .cache() should use in-memory columnar caching") {
    spark.table("testData").cache()
    assertCached(spark.table("testData"))
    spark.table("testData").unpersist(blocking = true)
  }

  test("calling .unpersist() should drop in-memory columnar cache") {
    spark.table("testData").cache()
    spark.table("testData").count()
    spark.table("testData").unpersist(blocking = true)
    assertCached(spark.table("testData"), 0)
  }

  test("isCached") {
    spark.catalog.cacheTable("testData")

    assertCached(spark.table("testData"))
    assert(spark.table("testData").queryExecution.withCachedData match {
      case _: InMemoryRelation => true
      case _ => false
    })

    spark.catalog.uncacheTable("testData")
    assert(!spark.catalog.isCached("testData"))
    assert(spark.table("testData").queryExecution.withCachedData match {
      case _: InMemoryRelation => false
      case _ => true
    })
  }

  test("SPARK-1669: cacheTable should be idempotent") {
    assume(!spark.table("testData").logicalPlan.isInstanceOf[InMemoryRelation])

    spark.catalog.cacheTable("testData")
    assertCached(spark.table("testData"))

    assertResult(1, "InMemoryRelation not found, testData should have been cached") {
      getNumInMemoryRelations(spark.table("testData"))
    }

    spark.catalog.cacheTable("testData")
    assertResult(0, "Double InMemoryRelations found, cacheTable() is not idempotent") {
      spark.table("testData").queryExecution.withCachedData.collect {
        case r @ InMemoryRelation(_, _, _, _, _: InMemoryTableScanExec, _) => r
      }.size
    }

    spark.catalog.uncacheTable("testData")
  }

  test("read from cached table and uncache") {
    spark.catalog.cacheTable("testData")
    checkAnswer(spark.table("testData"), testData.collect().toSeq)
    assertCached(spark.table("testData"))

    spark.catalog.uncacheTable("testData")
    checkAnswer(spark.table("testData"), testData.collect().toSeq)
    assertCached(spark.table("testData"), 0)
  }

  test("SELECT star from cached table") {
    sql("SELECT * FROM testData").createOrReplaceTempView("selectStar")
    spark.catalog.cacheTable("selectStar")
    checkAnswer(
      sql("SELECT * FROM selectStar WHERE key = 1"),
      Seq(Row(1, "1")))
    spark.catalog.uncacheTable("selectStar")
  }

  test("Self-join cached") {
    val unCachedAnswer =
      sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key").collect()
    spark.catalog.cacheTable("testData")
    checkAnswer(
      sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key"),
      unCachedAnswer.toSeq)
    spark.catalog.uncacheTable("testData")
  }

  test("'CACHE TABLE' and 'UNCACHE TABLE' SQL statement") {
    sql("CACHE TABLE testData")
    assertCached(spark.table("testData"))

    val rddId = rddIdOf("testData")
    assert(
      isMaterialized(rddId),
      "Eagerly cached in-memory table should have already been materialized")

    sql("UNCACHE TABLE testData")
    assert(!spark.catalog.isCached("testData"), "Table 'testData' should not be cached")

    eventually(timeout(10 seconds)) {
      assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted")
    }
  }

  test("CACHE TABLE tableName AS SELECT * FROM anotherTable") {
    withTempView("testCacheTable") {
      sql("CACHE TABLE testCacheTable AS SELECT * FROM testData")
      assertCached(spark.table("testCacheTable"))

      val rddId = rddIdOf("testCacheTable")
      assert(
        isMaterialized(rddId),
        "Eagerly cached in-memory table should have already been materialized")

      spark.catalog.uncacheTable("testCacheTable")
      eventually(timeout(10 seconds)) {
        assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted")
      }
    }
  }

  test("CACHE TABLE tableName AS SELECT ...") {
    withTempView("testCacheTable") {
      sql("CACHE TABLE testCacheTable AS SELECT key FROM testData LIMIT 10")
      assertCached(spark.table("testCacheTable"))

      val rddId = rddIdOf("testCacheTable")
      assert(
        isMaterialized(rddId),
        "Eagerly cached in-memory table should have already been materialized")

      spark.catalog.uncacheTable("testCacheTable")
      eventually(timeout(10 seconds)) {
        assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted")
      }
    }
  }

  test("CACHE LAZY TABLE tableName") {
    sql("CACHE LAZY TABLE testData")
    assertCached(spark.table("testData"))

    val rddId = rddIdOf("testData")
    assert(
      !isMaterialized(rddId),
      "Lazily cached in-memory table shouldn't be materialized eagerly")

    sql("SELECT COUNT(*) FROM testData").collect()
    assert(
      isMaterialized(rddId),
      "Lazily cached in-memory table should have been materialized")

    spark.catalog.uncacheTable("testData")
    eventually(timeout(10 seconds)) {
      assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted")
    }
  }

  test("InMemoryRelation statistics") {
    sql("CACHE TABLE testData")
    spark.table("testData").queryExecution.withCachedData.collect {
      case cached: InMemoryRelation =>
        val actualSizeInBytes = (1 to 100).map(i => 4 + i.toString.length + 4).sum
        assert(cached.stats(sqlConf).sizeInBytes === actualSizeInBytes)
    }
  }

  test("Drops temporary table") {
    testData.select('key).createOrReplaceTempView("t1")
    spark.table("t1")
    spark.catalog.dropTempView("t1")
    intercept[AnalysisException](spark.table("t1"))
  }

  test("Drops cached temporary table") {
    testData.select('key).createOrReplaceTempView("t1")
    testData.select('key).createOrReplaceTempView("t2")
    spark.catalog.cacheTable("t1")

    assert(spark.catalog.isCached("t1"))
    assert(spark.catalog.isCached("t2"))

    spark.catalog.dropTempView("t1")
    intercept[AnalysisException](spark.table("t1"))
    assert(!spark.catalog.isCached("t2"))
  }

  test("Clear all cache") {
    sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1")
    sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2")
    spark.catalog.cacheTable("t1")
    spark.catalog.cacheTable("t2")
    spark.catalog.clearCache()
    assert(spark.sharedState.cacheManager.isEmpty)

    sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1")
    sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2")
    spark.catalog.cacheTable("t1")
    spark.catalog.cacheTable("t2")
    sql("Clear CACHE")
    assert(spark.sharedState.cacheManager.isEmpty)
  }

  test("Ensure accumulators to be cleared after GC when uncacheTable") {
    sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1")
    sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2")

    spark.catalog.cacheTable("t1")
    spark.catalog.cacheTable("t2")

    sql("SELECT * FROM t1").count()
    sql("SELECT * FROM t2").count()
    sql("SELECT * FROM t1").count()
    sql("SELECT * FROM t2").count()

    val toBeCleanedAccIds = new HashSet[Long]

    val accId1 = spark.table("t1").queryExecution.withCachedData.collect {
      case i: InMemoryRelation => i.batchStats.id
    }.head
    toBeCleanedAccIds += accId1

    val accId2 = spark.table("t1").queryExecution.withCachedData.collect {
      case i: InMemoryRelation => i.batchStats.id
    }.head
    toBeCleanedAccIds += accId2

    val cleanerListener = new CleanerListener {
      def rddCleaned(rddId: Int): Unit = {}
      def shuffleCleaned(shuffleId: Int): Unit = {}
      def broadcastCleaned(broadcastId: Long): Unit = {}
      def accumCleaned(accId: Long): Unit = {
        toBeCleanedAccIds.synchronized { toBeCleanedAccIds -= accId }
      }
      def checkpointCleaned(rddId: Long): Unit = {}
    }
    spark.sparkContext.cleaner.get.attachListener(cleanerListener)

    spark.catalog.uncacheTable("t1")
    spark.catalog.uncacheTable("t2")

    System.gc()

    eventually(timeout(10 seconds)) {
      assert(toBeCleanedAccIds.synchronized { toBeCleanedAccIds.isEmpty },
        "batchStats accumulators should be cleared after GC when uncacheTable")
    }

    assert(AccumulatorContext.get(accId1).isEmpty)
    assert(AccumulatorContext.get(accId2).isEmpty)
  }

  test("SPARK-10327 Cache Table is not working while subquery has alias in its project list") {
    sparkContext.parallelize((1, 1) :: (2, 2) :: Nil)
      .toDF("key", "value").selectExpr("key", "value", "key+1").createOrReplaceTempView("abc")
    spark.catalog.cacheTable("abc")

    val sparkPlan = sql(
      """select a.key, b.key, c.key from
        |abc a join abc b on a.key=b.key
        |join abc c on a.key=c.key""".stripMargin).queryExecution.sparkPlan

    assert(sparkPlan.collect { case e: InMemoryTableScanExec => e }.size === 3)
    assert(sparkPlan.collect { case e: RDDScanExec => e }.size === 0)
  }

  /**
   * Verifies that the plan for `df` contains `expected` number of Exchange operators.
   */
  private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = {
    assert(df.queryExecution.executedPlan.collect { case e: ShuffleExchange => e }.size == expected)
  }

  test("A cached table preserves the partitioning and ordering of its cached SparkPlan") {
    val table3x = testData.union(testData).union(testData)
    table3x.createOrReplaceTempView("testData3x")

    sql("SELECT key, value FROM testData3x ORDER BY key").createOrReplaceTempView("orderedTable")
    spark.catalog.cacheTable("orderedTable")
    assertCached(spark.table("orderedTable"))
    // Should not have an exchange as the query is already sorted on the group by key.
    verifyNumExchanges(sql("SELECT key, count(*) FROM orderedTable GROUP BY key"), 0)
    checkAnswer(
      sql("SELECT key, count(*) FROM orderedTable GROUP BY key ORDER BY key"),
      sql("SELECT key, count(*) FROM testData3x GROUP BY key ORDER BY key").collect())
    spark.catalog.uncacheTable("orderedTable")
    spark.catalog.dropTempView("orderedTable")

    // Set up two tables distributed in the same way. Try this with the data distributed into
    // different number of partitions.
    for (numPartitions <- 1 until 10 by 4) {
      withTempView("t1", "t2") {
        testData.repartition(numPartitions, $"key").createOrReplaceTempView("t1")
        testData2.repartition(numPartitions, $"a").createOrReplaceTempView("t2")
        spark.catalog.cacheTable("t1")
        spark.catalog.cacheTable("t2")

        // Joining them should result in no exchanges.
        verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a"), 0)
        checkAnswer(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a"),
          sql("SELECT * FROM testData t1 JOIN testData2 t2 ON t1.key = t2.a"))

        // Grouping on the partition key should result in no exchanges
        verifyNumExchanges(sql("SELECT count(*) FROM t1 GROUP BY key"), 0)
        checkAnswer(sql("SELECT count(*) FROM t1 GROUP BY key"),
          sql("SELECT count(*) FROM testData GROUP BY key"))

        spark.catalog.uncacheTable("t1")
        spark.catalog.uncacheTable("t2")
      }
    }

    // Distribute the tables into non-matching number of partitions. Need to shuffle one side.
    withTempView("t1", "t2") {
      testData.repartition(6, $"key").createOrReplaceTempView("t1")
      testData2.repartition(3, $"a").createOrReplaceTempView("t2")
      spark.catalog.cacheTable("t1")
      spark.catalog.cacheTable("t2")

      val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a")
      verifyNumExchanges(query, 1)
      assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 6)
      checkAnswer(
        query,
        testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b"))
      spark.catalog.uncacheTable("t1")
      spark.catalog.uncacheTable("t2")
    }

    // One side of join is not partitioned in the desired way. Need to shuffle one side.
    withTempView("t1", "t2") {
      testData.repartition(6, $"value").createOrReplaceTempView("t1")
      testData2.repartition(6, $"a").createOrReplaceTempView("t2")
      spark.catalog.cacheTable("t1")
      spark.catalog.cacheTable("t2")

      val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a")
      verifyNumExchanges(query, 1)
      assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 6)
      checkAnswer(
        query,
        testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b"))
      spark.catalog.uncacheTable("t1")
      spark.catalog.uncacheTable("t2")
    }

    withTempView("t1", "t2") {
      testData.repartition(6, $"value").createOrReplaceTempView("t1")
      testData2.repartition(12, $"a").createOrReplaceTempView("t2")
      spark.catalog.cacheTable("t1")
      spark.catalog.cacheTable("t2")

      val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a")
      verifyNumExchanges(query, 1)
      assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 12)
      checkAnswer(
        query,
        testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b"))
      spark.catalog.uncacheTable("t1")
      spark.catalog.uncacheTable("t2")
    }

    // One side of join is not partitioned in the desired way. Since the number of partitions of
    // the side that has already partitioned is smaller than the side that is not partitioned,
    // we shuffle both side.
    withTempView("t1", "t2") {
      testData.repartition(6, $"value").createOrReplaceTempView("t1")
      testData2.repartition(3, $"a").createOrReplaceTempView("t2")
      spark.catalog.cacheTable("t1")
      spark.catalog.cacheTable("t2")

      val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a")
      verifyNumExchanges(query, 2)
      checkAnswer(
        query,
        testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b"))
      spark.catalog.uncacheTable("t1")
      spark.catalog.uncacheTable("t2")
    }

    // repartition's column ordering is different from group by column ordering.
    // But they use the same set of columns.
    withTempView("t1") {
      testData.repartition(6, $"value", $"key").createOrReplaceTempView("t1")
      spark.catalog.cacheTable("t1")

      val query = sql("SELECT value, key from t1 group by key, value")
      verifyNumExchanges(query, 0)
      checkAnswer(
        query,
        testData.distinct().select($"value", $"key"))
      spark.catalog.uncacheTable("t1")
    }

    // repartition's column ordering is different from join condition's column ordering.
    // We will still shuffle because hashcodes of a row depend on the column ordering.
    // If we do not shuffle, we may actually partition two tables in totally two different way.
    // See PartitioningSuite for more details.
    withTempView("t1", "t2") {
      val df1 = testData
      df1.repartition(6, $"value", $"key").createOrReplaceTempView("t1")
      val df2 = testData2.select($"a", $"b".cast("string"))
      df2.repartition(6, $"a", $"b").createOrReplaceTempView("t2")
      spark.catalog.cacheTable("t1")
      spark.catalog.cacheTable("t2")

      val query =
        sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a and t1.value = t2.b")
      verifyNumExchanges(query, 1)
      assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 6)
      checkAnswer(
        query,
        df1.join(df2, $"key" === $"a" && $"value" === $"b").select($"key", $"value", $"a", $"b"))
      spark.catalog.uncacheTable("t1")
      spark.catalog.uncacheTable("t2")
    }
  }

  test("SPARK-15870 DataFrame can't execute after uncacheTable") {
    val selectStar = sql("SELECT * FROM testData WHERE key = 1")
    selectStar.createOrReplaceTempView("selectStar")

    spark.catalog.cacheTable("selectStar")
    checkAnswer(
      selectStar,
      Seq(Row(1, "1")))

    spark.catalog.uncacheTable("selectStar")
    checkAnswer(
      selectStar,
      Seq(Row(1, "1")))
  }

  test("SPARK-15915 Logical plans should use canonicalized plan when override sameResult") {
    val localRelation = Seq(1, 2, 3).toDF()
    localRelation.createOrReplaceTempView("localRelation")

    spark.catalog.cacheTable("localRelation")
    assert(getNumInMemoryRelations(localRelation) == 1)
  }

  test("SPARK-19093 Caching in side subquery") {
    withTempView("t1") {
      Seq(1).toDF("c1").createOrReplaceTempView("t1")
      spark.catalog.cacheTable("t1")
      val ds =
        sql(
          """
            |SELECT * FROM t1
            |WHERE
            |NOT EXISTS (SELECT * FROM t1)
          """.stripMargin)
      assert(getNumInMemoryRelations(ds) == 2)
    }
  }

  test("SPARK-19093 scalar and nested predicate query") {
    withTempView("t1", "t2", "t3", "t4") {
      Seq(1).toDF("c1").createOrReplaceTempView("t1")
      Seq(2).toDF("c1").createOrReplaceTempView("t2")
      Seq(1).toDF("c1").createOrReplaceTempView("t3")
      Seq(1).toDF("c1").createOrReplaceTempView("t4")
      spark.catalog.cacheTable("t1")
      spark.catalog.cacheTable("t2")
      spark.catalog.cacheTable("t3")
      spark.catalog.cacheTable("t4")

      // Nested predicate subquery
      val ds =
        sql(
        """
          |SELECT * FROM t1
          |WHERE
          |c1 IN (SELECT c1 FROM t2 WHERE c1 IN (SELECT c1 FROM t3 WHERE c1 = 1))
        """.stripMargin)
      assert(getNumInMemoryRelations(ds) == 3)

      // Scalar subquery and predicate subquery
      val ds2 =
        sql(
          """
            |SELECT * FROM (SELECT max(c1) FROM t1 GROUP BY c1)
            |WHERE
            |c1 = (SELECT max(c1) FROM t2 GROUP BY c1)
            |OR
            |EXISTS (SELECT c1 FROM t3)
            |OR
            |c1 IN (SELECT c1 FROM t4)
          """.stripMargin)
      assert(getNumInMemoryRelations(ds2) == 4)
    }
  }

  test("SPARK-19765: UNCACHE TABLE should un-cache all cached plans that refer to this table") {
    withTable("t") {
      withTempPath { path =>
        Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath)
        sql(s"CREATE TABLE t USING parquet LOCATION '$path'")
        spark.catalog.cacheTable("t")
        spark.table("t").select($"i").cache()
        checkAnswer(spark.table("t").select($"i"), Row(1))
        assertCached(spark.table("t").select($"i"))

        Utils.deleteRecursively(path)
        spark.sessionState.catalog.refreshTable(TableIdentifier("t"))
        spark.catalog.uncacheTable("t")
        assert(spark.table("t").select($"i").count() == 0)
        assert(getNumInMemoryRelations(spark.table("t").select($"i")) == 0)
      }
    }
  }

  test("refreshByPath should refresh all cached plans with the specified path") {
    withTempDir { dir =>
      val path = dir.getCanonicalPath()

      spark.range(10).write.mode("overwrite").parquet(path)
      spark.read.parquet(path).cache()
      spark.read.parquet(path).filter($"id" > 4).cache()
      assert(spark.read.parquet(path).filter($"id" > 4).count() == 5)

      spark.range(20).write.mode("overwrite").parquet(path)
      spark.catalog.refreshByPath(path)
      assert(spark.read.parquet(path).count() == 20)
      assert(spark.read.parquet(path).filter($"id" > 4).count() == 15)
    }
  }

  test("SPARK-19993 simple subquery caching") {
    withTempView("t1", "t2") {
      Seq(1).toDF("c1").createOrReplaceTempView("t1")
      Seq(2).toDF("c1").createOrReplaceTempView("t2")

      sql(
        """
          |SELECT * FROM t1
          |WHERE
          |NOT EXISTS (SELECT * FROM t2)
        """.stripMargin).cache()

      val cachedDs =
        sql(
          """
            |SELECT * FROM t1
            |WHERE
            |NOT EXISTS (SELECT * FROM t2)
          """.stripMargin)
      assert(getNumInMemoryRelations(cachedDs) == 1)

      // Additional predicate in the subquery plan should cause a cache miss
      val cachedMissDs =
      sql(
        """
          |SELECT * FROM t1
          |WHERE
          |NOT EXISTS (SELECT * FROM t2 where c1 = 0)
        """.stripMargin)
      assert(getNumInMemoryRelations(cachedMissDs) == 0)
    }
  }

  test("SPARK-19993 subquery caching with correlated predicates") {
    withTempView("t1", "t2") {
      Seq(1).toDF("c1").createOrReplaceTempView("t1")
      Seq(1).toDF("c1").createOrReplaceTempView("t2")

      // Simple correlated predicate in subquery
      sql(
        """
          |SELECT * FROM t1
          |WHERE
          |t1.c1 in (SELECT t2.c1 FROM t2 where t1.c1 = t2.c1)
        """.stripMargin).cache()

      val cachedDs =
        sql(
          """
            |SELECT * FROM t1
            |WHERE
            |t1.c1 in (SELECT t2.c1 FROM t2 where t1.c1 = t2.c1)
          """.stripMargin)
      assert(getNumInMemoryRelations(cachedDs) == 1)
    }
  }

  test("SPARK-19993 subquery with cached underlying relation") {
    withTempView("t1") {
      Seq(1).toDF("c1").createOrReplaceTempView("t1")
      spark.catalog.cacheTable("t1")

      // underlying table t1 is cached as well as the query that refers to it.
      val ds =
      sql(
        """
          |SELECT * FROM t1
          |WHERE
          |NOT EXISTS (SELECT * FROM t1)
        """.stripMargin)
      assert(getNumInMemoryRelations(ds) == 2)

      val cachedDs =
        sql(
          """
            |SELECT * FROM t1
            |WHERE
            |NOT EXISTS (SELECT * FROM t1)
          """.stripMargin).cache()
      assert(getNumInMemoryTablesRecursively(cachedDs.queryExecution.sparkPlan) == 3)
    }
  }

  test("SPARK-19993 nested subquery caching and scalar + predicate subqueris") {
    withTempView("t1", "t2", "t3", "t4") {
      Seq(1).toDF("c1").createOrReplaceTempView("t1")
      Seq(2).toDF("c1").createOrReplaceTempView("t2")
      Seq(1).toDF("c1").createOrReplaceTempView("t3")
      Seq(1).toDF("c1").createOrReplaceTempView("t4")

      // Nested predicate subquery
      sql(
        """
          |SELECT * FROM t1
          |WHERE
          |c1 IN (SELECT c1 FROM t2 WHERE c1 IN (SELECT c1 FROM t3 WHERE c1 = 1))
        """.stripMargin).cache()

      val cachedDs =
        sql(
          """
            |SELECT * FROM t1
            |WHERE
            |c1 IN (SELECT c1 FROM t2 WHERE c1 IN (SELECT c1 FROM t3 WHERE c1 = 1))
          """.stripMargin)
      assert(getNumInMemoryRelations(cachedDs) == 1)

      // Scalar subquery and predicate subquery
      sql(
        """
          |SELECT * FROM (SELECT max(c1) FROM t1 GROUP BY c1)
          |WHERE
          |c1 = (SELECT max(c1) FROM t2 GROUP BY c1)
          |OR
          |EXISTS (SELECT c1 FROM t3)
          |OR
          |c1 IN (SELECT c1 FROM t4)
        """.stripMargin).cache()

      val cachedDs2 =
        sql(
          """
            |SELECT * FROM (SELECT max(c1) FROM t1 GROUP BY c1)
            |WHERE
            |c1 = (SELECT max(c1) FROM t2 GROUP BY c1)
            |OR
            |EXISTS (SELECT c1 FROM t3)
            |OR
            |c1 IN (SELECT c1 FROM t4)
          """.stripMargin)
      assert(getNumInMemoryRelations(cachedDs2) == 1)
    }
  }
}