Skip to content

Commit

Permalink
New constructor for NonFateSharingCache
Browse files Browse the repository at this point in the history
  • Loading branch information
ericm-db committed Nov 28, 2024
1 parent 6697725 commit d39fee9
Show file tree
Hide file tree
Showing 2 changed files with 23 additions and 10 deletions.
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

0 comments on commit d39fee9

Please sign in to comment.