Skip to content

Commit 7298b00

Browse files
author
yangyx
committed
HiveFileStatusCache unit test
1 parent ff11997 commit 7298b00

File tree

2 files changed

+40
-2
lines changed

2 files changed

+40
-2
lines changed

extensions/spark/kyuubi-spark-connector-hive/src/test/scala/org/apache/kyuubi/spark/connector/hive/HiveFileStatusCacheSuite.scala

Lines changed: 39 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -39,9 +39,9 @@ class HiveFileStatusCacheSuite extends KyuubiHiveTest {
3939
val files = (1 to 3).map(_ => new FileStatus())
4040

4141
HiveFileStatusCache.resetForTesting()
42-
val fileStatusCacheTabel1 = HiveFileStatusCache.getOrCreate(spark, "catalog.db.table1")
42+
val fileStatusCacheTabel1 = HiveFileStatusCache.getOrCreate(spark, "catalog.db.cat1Table")
4343
fileStatusCacheTabel1.putLeafFiles(path, files.toArray)
44-
val fileStatusCacheTabel2 = HiveFileStatusCache.getOrCreate(spark, "catalog.db.table1")
44+
val fileStatusCacheTabel2 = HiveFileStatusCache.getOrCreate(spark, "catalog.db.cat1Table")
4545
val fileStatusCacheTabel3 = HiveFileStatusCache.getOrCreate(spark, "catalog.db.table2")
4646

4747
// Exactly 3 files are cached.
@@ -216,4 +216,41 @@ class HiveFileStatusCacheSuite extends KyuubiHiveTest {
216216
.isEmpty)
217217
}
218218
}
219+
220+
test("FileStatusCache isolated between different catalogs with same database.table") {
221+
val catalog1 = catalogName
222+
val catalog2 = "hive2"
223+
val dbName = "default"
224+
val tbName = "tbl_partition"
225+
val dbTableShortName = s"${dbName}.${tbName}"
226+
val cat1Table = s"${catalog1}.${dbTableShortName}"
227+
val cat2Table = s"${catalog2}.${dbTableShortName}"
228+
229+
withTable(cat1Table, cat2Table) {
230+
spark.sql(s"CREATE TABLE IF NOT EXISTS $cat1Table (age int)partitioned by(city string)" +
231+
s" stored as orc").collect()
232+
val location = newCatalog()
233+
.loadTable(Identifier.of(Array(dbName), tbName))
234+
.asInstanceOf[HiveTable]
235+
.catalogTable.location.toString
236+
237+
spark.sql(s"use $catalog1").collect()
238+
spark.sql(s"insert into $dbTableShortName partition(city='ct1') " +
239+
s"values(11),(12),(13),(14),(15)").collect()
240+
spark.sql(s"select * from $cat1Table where city='ct1'").collect()
241+
assert(HiveFileStatusCache.getOrCreate(spark, cat1Table)
242+
.getLeafFiles(new Path(s"$location/city=ct1"))
243+
.get.length === 1)
244+
245+
spark.sql(s"use $catalog2").collect()
246+
spark.sql(s"insert into $dbTableShortName partition(city='ct2') " +
247+
s"values(21),(22),(23),(24),(25)").collect()
248+
spark.sql(s"select * from $cat2Table where city='ct2'").collect()
249+
assert(HiveFileStatusCache.getOrCreate(spark, cat2Table)
250+
.getLeafFiles(new Path(s"$location/city=ct1")).isEmpty)
251+
assert(HiveFileStatusCache.getOrCreate(spark, cat2Table)
252+
.getLeafFiles(new Path(s"$location/city=ct2"))
253+
.get.length === 1)
254+
}
255+
}
219256
}

extensions/spark/kyuubi-spark-connector-hive/src/test/scala/org/apache/kyuubi/spark/connector/hive/KyuubiHiveTest.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,7 @@ abstract class KyuubiHiveTest extends QueryTest with Logging {
6666
.set("spark.ui.enabled", "false")
6767
.set("spark.sql.catalogImplementation", "hive")
6868
.set("spark.sql.catalog.hive", classOf[HiveTableCatalog].getName)
69+
.set("spark.sql.catalog.hive2", classOf[HiveTableCatalog].getName)
6970
.set("javax.jdo.option.ConnectionURL", "jdbc:derby:memory:memorydb;create=true")
7071
.set("javax.jdo.option.ConnectionDriverName", "org.apache.derby.jdbc.EmbeddedDriver")
7172

0 commit comments

Comments
 (0)