Skip to content
Closed
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 @@ -17,7 +17,7 @@

package org.apache.spark.util

import java.util.concurrent.Callable
import java.util.concurrent.{Callable, TimeUnit}

import com.google.common.cache.{Cache, CacheBuilder, CacheLoader, LoadingCache}

Expand Down Expand Up @@ -68,6 +68,20 @@ private[spark] object NonFateSharingCache {
override def load(k: K): V = loadingFunc.apply(k)
}))
}

def apply[K, V](
maximumSize: Long,
expireAfterAccessTime: Long,
expireAfterAccessTimeUnit: TimeUnit): NonFateSharingCache[K, V] = {
val builder = CacheBuilder.newBuilder().asInstanceOf[CacheBuilder[K, V]]
if (maximumSize > 0L) {
builder.maximumSize(maximumSize)
}
if(expireAfterAccessTime > 0) {
builder.expireAfterAccess(expireAfterAccessTime, expireAfterAccessTimeUnit)
}
new NonFateSharingCache(builder.build[K, V]())
}
}

private[spark] class NonFateSharingCache[K, V](protected val cache: Cache[K, V]) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit}

import scala.util.control.NonFatal

import com.google.common.cache.CacheBuilder
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path

Expand Down Expand Up @@ -613,15 +612,15 @@ object RocksDBStateStoreProvider {
val VIRTUAL_COL_FAMILY_PREFIX_BYTES = 2

private val MAX_AVRO_ENCODERS_IN_CACHE = 1000
// Add the cache at companion object level so it persists across provider instances
private val avroEncoderMap: NonFateSharingCache[String, AvroEncoder] = {
val guavaCache = CacheBuilder.newBuilder()
.maximumSize(MAX_AVRO_ENCODERS_IN_CACHE) // Adjust size based on your needs
.expireAfterAccess(1, TimeUnit.HOURS) // Optional: Add expiration if needed
.build[String, AvroEncoder]()
private val AVRO_ENCODER_LIFETIME_HOURS = 1L

new NonFateSharingCache(guavaCache)
}
// Add the cache at companion object level so it persists across provider instances
private val avroEncoderMap: NonFateSharingCache[String, AvroEncoder] =
NonFateSharingCache(
maximumSize = MAX_AVRO_ENCODERS_IN_CACHE,
expireAfterAccessTime = AVRO_ENCODER_LIFETIME_HOURS,
expireAfterAccessTimeUnit = TimeUnit.HOURS
)

def getAvroEnc(
stateStoreEncoding: String,
Expand Down