Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.util.quoteIfNeeded
import org.apache.spark.sql.connector.catalog.{Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableChange}
import org.apache.spark.sql.connector.catalog._
import org.apache.spark.sql.connector.catalog.NamespaceChange.RemoveProperty
import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.execution.command.DDLUtils
Expand All @@ -49,6 +49,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.kyuubi.spark.connector.hive.HiveConnectorUtils.withSparkSQLConf
import org.apache.kyuubi.spark.connector.hive.HiveTableCatalog.{getStorageFormatAndProvider, toCatalogDatabase, CatalogDatabaseHelper, IdentifierHelper, NamespaceHelper}
import org.apache.kyuubi.spark.connector.hive.KyuubiHiveConnectorDelegationTokenProvider.metastoreTokenSignature
import org.apache.kyuubi.spark.connector.hive.read.HiveFileStatusCache
import org.apache.kyuubi.util.reflect.{DynClasses, DynConstructors}

/**
Expand Down Expand Up @@ -377,29 +378,45 @@ class HiveTableCatalog(sparkSession: SparkSession)
}

try {
catalog.alterTable(
catalogTable.copy(
properties = properties,
schema = schema,
owner = owner,
comment = comment,
storage = storage))
catalog.alterTable(newCatalogTable(
identifier = catalogTable.identifier,
tableType = catalogTable.tableType,
storage = storage,
schema = schema,
provider = catalogTable.provider,
partitionColumnNames = catalogTable.partitionColumnNames,
bucketSpec = catalogTable.bucketSpec,
owner = owner,
createTime = catalogTable.createTime,
lastAccessTime = catalogTable.lastAccessTime,
createVersion = catalogTable.createVersion,
properties = properties,
stats = catalogTable.stats,
viewText = catalogTable.viewText,
comment = comment,
unsupportedFeatures = catalogTable.unsupportedFeatures,
tracksPartitionsInCatalog = catalogTable.tracksPartitionsInCatalog,
schemaPreservesCase = catalogTable.schemaPreservesCase,
ignoredProperties = catalogTable.ignoredProperties,
viewOriginalText = catalogTable.viewOriginalText))
} catch {
case _: NoSuchTableException =>
throw new NoSuchTableException(ident)
}

invalidateTable(ident)
loadTable(ident)
}

override def dropTable(ident: Identifier): Boolean =
withSparkSQLConf(LEGACY_NON_IDENTIFIER_OUTPUT_CATALOG_NAME -> "true") {
try {
if (loadTable(ident) != null) {
val table = loadTable(ident)
if (table != null) {
catalog.dropTable(
ident.asTableIdentifier,
ignoreIfNotExists = true,
purge = true /* skip HDFS trash */ )
invalidateTable(ident)
true
} else {
false
Expand All @@ -417,10 +434,17 @@ class HiveTableCatalog(sparkSession: SparkSession)
}

// Load table to make sure the table exists
loadTable(oldIdent)
val table = loadTable(oldIdent)
catalog.renameTable(oldIdent.asTableIdentifier, newIdent.asTableIdentifier)
invalidateTable(oldIdent)
}

override def invalidateTable(ident: Identifier): Unit = {
super.invalidateTable(ident)
val qualifiedName = s"$catalogName.$ident"
HiveFileStatusCache.getOrCreate(sparkSession, qualifiedName).invalidateAll()
}

private def toOptions(properties: Map[String, String]): Map[String, String] = {
properties.filterKeys(_.startsWith(TableCatalog.OPTION_PREFIX)).map {
case (key, value) => key.drop(TableCatalog.OPTION_PREFIX.length) -> value
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,4 +57,16 @@ object KyuubiHiveConnectorConf {
.version("1.11.0")
.booleanConf
.createWithDefault(true)

val HIVE_FILE_STATUS_CACHE_SCOPE =
buildConf("spark.sql.kyuubi.hive.file.status.cache.scope")
.doc("The scope of hive file status cache, globe and none.")
.version("1.11.0")
.stringConf
.transform(policy => policy.toUpperCase(Locale.ROOT))
.checkValue(
policy => Set("GLOBE", "NONE").contains(policy),
"Invalid value for 'spark.sql.kyuubi.hive.file.status.cache.scope'." +
"Valid values are 'GLOBE', 'NONE'.")
.createWithDefault("GLOBE")
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,9 @@ class HiveCatalogFileIndex(
private val partPathToBindHivePart: mutable.Map[PartitionPath, CatalogTablePartition] =
mutable.Map()

private val fileStatusCache = FileStatusCache.getOrCreate(sparkSession)
private val fileStatusCache = HiveFileStatusCache.getOrCreate(
sparkSession,
hiveCatalog.name() + "." + catalogTable.qualifiedName)

private val baseLocation: Option[URI] = table.storage.locationUri

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,157 @@
/*
* 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.kyuubi.spark.connector.hive.read

import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean

import scala.collection.JavaConverters._

import com.google.common.cache._
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.execution.datasources.{FileStatusCache, NoopCache}
import org.apache.spark.util.SizeEstimator

import org.apache.kyuubi.spark.connector.hive.KyuubiHiveConnectorConf.HIVE_FILE_STATUS_CACHE_SCOPE

/**
* Forked from Apache Spark's [[org.apache.spark.sql.execution.datasources.FileStatusCache]] 3.5.5.
*
* Because the original FileStatusCache cannot take effect (see https://github.com/apache/kyuubi
* /issues/7192).
*
* The main modification point is that at the globally level, the cache key is the qualified name
* of the table (in the form of `catalog.database.table`) + path. The previous key was an
* object + path generated during initialization, and the current scenario is that FileStatusCache
* is not preserved by the outside, resulting in different keys and ineffective caching.
*
* Use [[HiveFileStatusCache.getOrCreate()]] to construct a globe/none shared file status cache.
*/
object HiveFileStatusCache {
private var sharedCache: HiveSharedInMemoryCache = _

/**
* @return a new FileStatusCache based on session configuration. Cache memory quota is
* shared across all clients.
*/
def getOrCreate(session: SparkSession, qualifiedName: String): FileStatusCache =
synchronized {
val conf = session.sessionState.conf
if (conf.manageFilesourcePartitions && conf.filesourcePartitionFileCacheSize > 0) {
if (sharedCache == null) {
sharedCache = new HiveSharedInMemoryCache(
session.sessionState.conf.filesourcePartitionFileCacheSize,
session.sessionState.conf.metadataCacheTTL)
}
conf.getConf(HIVE_FILE_STATUS_CACHE_SCOPE) match {
case "GLOBE" => sharedCache.createForNewClient(qualifiedName)
case "NONE" => NoopCache
}
} else {
NoopCache
}
}

def resetForTesting(): Unit = synchronized {
sharedCache = null
}
}

/**
* An implementation that caches partition file statuses in memory.
*
* @param maxSizeInBytes max allowable cache size before entries start getting evicted
*/
private class HiveSharedInMemoryCache(maxSizeInBytes: Long, cacheTTL: Long) extends Logging {

// Opaque object that uniquely identifies a shared cache user
private type ClientId = Object

private val warnedAboutEviction = new AtomicBoolean(false)

// we use a composite cache key in order to distinguish entries inserted by different clients
private val cache: Cache[(ClientId, Path), Array[FileStatus]] = {
// [[Weigher]].weigh returns Int so we could only cache objects < 2GB
// instead, the weight is divided by this factor (which is smaller
// than the size of one [[FileStatus]]).
// so it will support objects up to 64GB in size.
val weightScale = 32
val weigher = new Weigher[(ClientId, Path), Array[FileStatus]] {
override def weigh(key: (ClientId, Path), value: Array[FileStatus]): Int = {
val estimate = (SizeEstimator.estimate(key) + SizeEstimator.estimate(value)) / weightScale
if (estimate > Int.MaxValue) {
logWarning(s"Cached table partition metadata size is too big. Approximating to " +
s"${Int.MaxValue.toLong * weightScale}.")
Int.MaxValue
} else {
estimate.toInt
}
}
}
val removalListener = new RemovalListener[(ClientId, Path), Array[FileStatus]]() {
override def onRemoval(
removed: RemovalNotification[(ClientId, Path), Array[FileStatus]]): Unit = {
if (removed.getCause == RemovalCause.SIZE &&
warnedAboutEviction.compareAndSet(false, true)) {
logWarning(
"Evicting cached table partition metadata from memory due to size constraints " +
"(spark.sql.hive.filesourcePartitionFileCacheSize = "
+ maxSizeInBytes + " bytes). This may impact query planning performance.")
}
}
}

var builder = CacheBuilder.newBuilder()
.weigher(weigher)
.removalListener(removalListener)
.maximumWeight(maxSizeInBytes / weightScale)

if (cacheTTL > 0) {
builder = builder.expireAfterWrite(cacheTTL, TimeUnit.SECONDS)
}

builder.build[(ClientId, Path), Array[FileStatus]]()
}

/**
* @return a FileStatusCache that does not share any entries with any other client, but does
* share memory resources for the purpose of cache eviction.
*/
def createForNewClient(clientId: Object): HiveFileStatusCache = new HiveFileStatusCache {

override def getLeafFiles(path: Path): Option[Array[FileStatus]] = {
Option(cache.getIfPresent((clientId, path)))
}

override def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit = {
cache.put((clientId, path), leafFiles)
}

override def invalidateAll(): Unit = {
cache.asMap.asScala.foreach { case (key, value) =>
if (key._1 == clientId) {
cache.invalidate(key)
}
}
}
}

abstract class HiveFileStatusCache extends FileStatusCache {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import org.apache.spark.internal.io.FileCommitProtocol
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.connector.catalog.Identifier
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage}
import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult}
import org.apache.spark.sql.execution.datasources.v2.FileBatchWrite
Expand Down Expand Up @@ -69,6 +70,8 @@ class HiveBatchWrite(

// un-cache this table.
hiveTableCatalog.catalog.invalidateCachedTable(table.identifier)
hiveTableCatalog.invalidateTable(
Identifier.of(Array(table.identifier.database.getOrElse("")), table.identifier.table))

val catalog = hiveTableCatalog.catalog
if (sparkSession.sessionState.conf.autoSizeUpdateEnabled) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap

import org.apache.kyuubi.spark.connector.hive.HiveTableCatalog.IdentifierHelper
import org.apache.kyuubi.spark.connector.hive.KyuubiHiveConnectorConf.{READ_CONVERT_METASTORE_ORC, READ_CONVERT_METASTORE_PARQUET}
import org.apache.kyuubi.spark.connector.hive.read.HiveScan
import org.apache.kyuubi.spark.connector.hive.read.{HiveFileStatusCache, HiveScan}

class HiveCatalogSuite extends KyuubiHiveTest {

Expand Down Expand Up @@ -284,16 +284,26 @@ class HiveCatalogSuite extends KyuubiHiveTest {
}

test("invalidateTable") {
val table = catalog.createTable(testIdent, schema, Array.empty[Transform], emptyProps)
// Hive v2 don't cache table
catalog.invalidateTable(testIdent)

val loaded = catalog.loadTable(testIdent)

assert(table.name == loaded.name)
assert(table.schema == loaded.schema)
assert(table.properties == loaded.properties)
catalog.dropTable(testIdent)
withSparkSession() { spark =>
val table = catalog.createTable(testIdent, schema, Array.empty[Transform], emptyProps)
val qualifiedName = s"$catalogName.$testIdent"
val location = table.asInstanceOf[HiveTable].catalogTable.location

spark.sql(s"select * from $qualifiedName").collect()
assert(HiveFileStatusCache.getOrCreate(spark, qualifiedName)
.getLeafFiles(new Path(location)).isDefined)

catalog.invalidateTable(testIdent)
// invalidate filestatus cache
assert(HiveFileStatusCache.getOrCreate(spark, qualifiedName)
.getLeafFiles(new Path(location)).isEmpty)

val loaded = catalog.loadTable(testIdent)
assert(table.name == loaded.name)
assert(table.schema == loaded.schema)
assert(table.properties == loaded.properties)
catalog.dropTable(testIdent)
}
}

test("listNamespaces: fail if missing namespace") {
Expand Down
Loading
Loading