diff --git a/build.sbt b/build.sbt index 9bb10b39e..39a839f74 100644 --- a/build.sbt +++ b/build.sbt @@ -168,6 +168,15 @@ mimaBinaryIssueFilters ++= Seq( ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.com$fasterxml$jackson$module$scala$introspect$ScalaAnnotationIntrospectorModule$_setter_$overrideMap_="), ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.com$fasterxml$jackson$module$scala$introspect$ScalaAnnotationIntrospectorModule$$_shouldSupportScala3Classes_="), ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.com$fasterxml$jackson$module$scala$introspect$ScalaAnnotationIntrospectorModule$$_shouldSupportScala3Classes"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.com$fasterxml$jackson$module$scala$introspect$ScalaAnnotationIntrospectorModule$$_lookupCacheFactory"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.com$fasterxml$jackson$module$scala$introspect$ScalaAnnotationIntrospectorModule$$_lookupCacheFactory_="), + ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.com$fasterxml$jackson$module$scala$introspect$ScalaAnnotationIntrospectorModule$$_descriptorCacheSize"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.com$fasterxml$jackson$module$scala$introspect$ScalaAnnotationIntrospectorModule$$_descriptorCacheSize_="), + ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.com$fasterxml$jackson$module$scala$introspect$ScalaAnnotationIntrospectorModule$$_scalaTypeCacheSize"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.com$fasterxml$jackson$module$scala$introspect$ScalaAnnotationIntrospectorModule$$_scalaTypeCacheSize_="), + ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.setLookupCacheFactory"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.setDescriptorCacheSize"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.setScalaTypeCacheSize"), ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.clearRegisteredReferencedTypes"), ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.registerReferencedValueType"), ProblemFilters.exclude[ReversedMissingMethodProblem]("com.fasterxml.jackson.module.scala.introspect.ScalaAnnotationIntrospectorModule.getRegisteredReferencedValueType"), diff --git a/src/main/scala-2.+/com/fasterxml/jackson/module/scala/ScalaObjectMapper.scala b/src/main/scala-2.+/com/fasterxml/jackson/module/scala/ScalaObjectMapper.scala index 5122e0498..fea05822f 100644 --- a/src/main/scala-2.+/com/fasterxml/jackson/module/scala/ScalaObjectMapper.scala +++ b/src/main/scala-2.+/com/fasterxml/jackson/module/scala/ScalaObjectMapper.scala @@ -343,9 +343,9 @@ trait ScalaObjectMapper { * JSON. Same converters (serializers, deserializers) will be used as for * data binding, meaning same object mapper configuration works. * - * @throws IllegalArgumentException If conversion fails due to incompatible type; - * if so, root cause will contain underlying checked exception data binding - * functionality threw + * @throws java.lang.IllegalArgumentException If conversion fails due to incompatible type; + * if so, root cause will contain underlying checked exception data + * binding functionality threw */ def convertValue[T: Manifest](fromValue: Any): T = { convertValue(fromValue, constructType[T]) diff --git a/src/main/scala/com/fasterxml/jackson/module/scala/ClassTagExtensions.scala b/src/main/scala/com/fasterxml/jackson/module/scala/ClassTagExtensions.scala index 492103039..cd55dea13 100644 --- a/src/main/scala/com/fasterxml/jackson/module/scala/ClassTagExtensions.scala +++ b/src/main/scala/com/fasterxml/jackson/module/scala/ClassTagExtensions.scala @@ -255,9 +255,9 @@ trait ClassTagExtensions { * JSON. Same converters (serializers, deserializers) will be used as for * data binding, meaning same object mapper configuration works. * - * @throws IllegalArgumentException If conversion fails due to incompatible type; - * if so, root cause will contain underlying checked exception data binding - * functionality threw + * @throws java.lang.IllegalArgumentException If conversion fails due to incompatible type; + * if so, root cause will contain underlying checked exception data + * binding functionality threw */ def convertValue[T: JavaTypeable](fromValue: Any): T = { convertValue(fromValue, constructType[T]) diff --git a/src/main/scala/com/fasterxml/jackson/module/scala/LookupCacheFactory.scala b/src/main/scala/com/fasterxml/jackson/module/scala/LookupCacheFactory.scala new file mode 100644 index 000000000..e1730d18a --- /dev/null +++ b/src/main/scala/com/fasterxml/jackson/module/scala/LookupCacheFactory.scala @@ -0,0 +1,16 @@ +package com.fasterxml.jackson.module.scala + +import com.fasterxml.jackson.databind.util.{LRUMap, LookupCache} + +/** + * Factory for creating [[com.fasterxml.jackson.databind.util.LookupCache]] instances + */ +trait LookupCacheFactory { + def createLookupCache[K, V](initialEntries: Int, maxEntries: Int): LookupCache[K, V] +} + +object DefaultLookupCacheFactory extends LookupCacheFactory { + override def createLookupCache[K, V](initialEntries: Int, maxEntries: Int): LookupCache[K, V] = { + new LRUMap[K, V](initialEntries, maxEntries) + } +} diff --git a/src/main/scala/com/fasterxml/jackson/module/scala/introspect/ScalaAnnotationIntrospectorModule.scala b/src/main/scala/com/fasterxml/jackson/module/scala/introspect/ScalaAnnotationIntrospectorModule.scala index f15b2cdbe..4b1dc9266 100644 --- a/src/main/scala/com/fasterxml/jackson/module/scala/introspect/ScalaAnnotationIntrospectorModule.scala +++ b/src/main/scala/com/fasterxml/jackson/module/scala/introspect/ScalaAnnotationIntrospectorModule.scala @@ -2,14 +2,14 @@ package com.fasterxml.jackson.module.scala.introspect import com.fasterxml.jackson.annotation.JsonCreator import com.fasterxml.jackson.core.Version -import com.fasterxml.jackson.databind.`type`.{ClassKey, CollectionLikeType, MapLikeType, ReferenceType, SimpleType} +import com.fasterxml.jackson.databind.`type`.{CollectionLikeType, MapLikeType, ReferenceType, SimpleType} import com.fasterxml.jackson.databind.cfg.MapperConfig import com.fasterxml.jackson.databind.deser.std.StdValueInstantiator import com.fasterxml.jackson.databind.deser._ import com.fasterxml.jackson.databind.introspect._ -import com.fasterxml.jackson.databind.util.{AccessPattern, LRUMap, LookupCache} +import com.fasterxml.jackson.databind.util.{AccessPattern, LookupCache} import com.fasterxml.jackson.databind.{BeanDescription, DeserializationConfig, DeserializationContext, JavaType, MapperFeature} -import com.fasterxml.jackson.module.scala.JacksonModule +import com.fasterxml.jackson.module.scala.{DefaultLookupCacheFactory, JacksonModule, LookupCacheFactory} import com.fasterxml.jackson.module.scala.util.Implicits._ import java.lang.annotation.Annotation @@ -178,7 +178,7 @@ object ScalaAnnotationIntrospector extends NopAnnotationIntrospector with ValueI } private def _descriptorFor(clz: Class[_]): Option[BeanDescriptor] = { - val key = new ClassKey(clz) + val key = clz.getName val isScala = { Option(ScalaAnnotationIntrospectorModule._scalaTypeCache.get(key)) match { case Some(result) => result @@ -247,15 +247,80 @@ trait ScalaAnnotationIntrospectorModule extends JacksonModule { this += { _.appendAnnotationIntrospector(ScalaAnnotationIntrospector) } this += { _.addValueInstantiators(ScalaAnnotationIntrospector) } - private[introspect] var _descriptorCache: LookupCache[ClassKey, BeanDescriptor] = - new LRUMap[ClassKey, BeanDescriptor](16, 100) + private var _lookupCacheFactory: LookupCacheFactory = DefaultLookupCacheFactory + private var _shouldSupportScala3Classes: Boolean = true + private var _descriptorCacheSize: Int = 100 + private var _scalaTypeCacheSize: Int = 1000 - private[introspect] var _scalaTypeCache: LookupCache[ClassKey, Boolean] = - new LRUMap[ClassKey, Boolean](16, 1000) + private[introspect] var _descriptorCache: LookupCache[String, BeanDescriptor] = + _lookupCacheFactory.createLookupCache(16, _descriptorCacheSize) + + private[introspect] var _scalaTypeCache: LookupCache[String, Boolean] = + _lookupCacheFactory.createLookupCache(16, _scalaTypeCacheSize) private[introspect] val overrideMap = MutableMap[String, ClassOverrides]() - private var _shouldSupportScala3Classes = true + /** + * Replaces the [[LookupCacheFactory]]. The default factory uses [[com.fasterxml.jackson.databind.util.LRUMap]]. + *
+ * Note that this clears the existing cache entries. It is best to set this up before you start using + * the Jackson Scala Module for serializing/deserializing. + *
+ * + * @param lookupCacheFactory new factory + * @see [[setDescriptorCacheSize]] + * @see [[setScalaTypeCacheSize]] + * @since 2.14.3 + */ + def setLookupCacheFactory(lookupCacheFactory: LookupCacheFactory): Unit = { + _lookupCacheFactory = lookupCacheFactory + recreateDescriptorCache() + recreateScalaTypeCache() + } + + /** + * Resize thedescriptorCache
. The default size is 100.
+ * + * Note that this clears the existing cache entries. It is best to set this up before you start using + * the Jackson Scala Module for serializing/deserializing. + *
+ * + * @param size new size for the cache + * @see [[setScalaTypeCacheSize]] + * @see [[setLookupCacheFactory]] + * @since 2.14.3 + */ + def setDescriptorCacheSize(size: Int): Unit = { + _descriptorCacheSize = size + recreateDescriptorCache() + } + + /** + * Resize thescalaTypeCache
. The default size is 1000.
+ * + * Note that this clears the existing cache entries. It is best to set this up before you start using + * the Jackson Scala Module for serializing/deserializing. + *
+ * + * @param size new size for the cache + * @see [[setDescriptorCacheSize]] + * @see [[setLookupCacheFactory]] + * @since 2.14.3 + */ + def setScalaTypeCacheSize(size: Int): Unit = { + _scalaTypeCacheSize = size + recreateScalaTypeCache() + } + + private def recreateDescriptorCache(): Unit = { + _descriptorCache.clear() + _descriptorCache = _lookupCacheFactory.createLookupCache(16, _descriptorCacheSize) + } + + private def recreateScalaTypeCache(): Unit = { + _scalaTypeCache.clear() + _scalaTypeCache = _lookupCacheFactory.createLookupCache(16, _scalaTypeCacheSize) + } /** * jackson-module-scala does not always properly handle deserialization of Options or Collections wrapping @@ -316,7 +381,7 @@ trait ScalaAnnotationIntrospectorModule extends JacksonModule { } /** - * clears all the state associated with reference types + * Clears all the state associated with reference types * * @see [[registerReferencedValueType]] * @see [[clearRegisteredReferencedTypes(Class[_])]] @@ -326,7 +391,17 @@ trait ScalaAnnotationIntrospectorModule extends JacksonModule { overrideMap.clear() } - def setDescriptorCache(cache: LookupCache[ClassKey, BeanDescriptor]): LookupCache[ClassKey, BeanDescriptor] = { + /** + * Replace thedescriptorCachescalaTypeCache
.
*
* @param cache new cache instance
- * @return old cache instance
+ * @return existing cache instance
* @since 2.14.0
+ * @see [[setScalaTypeCacheSize]]
+ * @see [[setLookupCacheFactory]]
+ * @deprecated key type will change to String in v2.15.0 and this function will be removed in a later release
*/
- def setScalaTypeCache(cache: LookupCache[ClassKey, Boolean]): LookupCache[ClassKey, Boolean] = {
+ @deprecated("key type will change to String in v2.15.0 and this function will be removed in a later release", "2.14.3")
+ def setScalaTypeCache(cache: LookupCache[String, Boolean]): LookupCache[String, Boolean] = {
val existingCache = _scalaTypeCache
_scalaTypeCache = cache
existingCache
diff --git a/src/test/scala/com/fasterxml/jackson/module/scala/introspect/ScalaAnnotationIntrospectorTest.scala b/src/test/scala/com/fasterxml/jackson/module/scala/introspect/ScalaAnnotationIntrospectorTest.scala
index a568704ac..f7bb3ac22 100644
--- a/src/test/scala/com/fasterxml/jackson/module/scala/introspect/ScalaAnnotationIntrospectorTest.scala
+++ b/src/test/scala/com/fasterxml/jackson/module/scala/introspect/ScalaAnnotationIntrospectorTest.scala
@@ -4,7 +4,6 @@ package introspect
import com.fasterxml.jackson.annotation.JsonProperty
import com.fasterxml.jackson.core.JsonGenerator
-import com.fasterxml.jackson.databind.`type`.ClassKey
import com.fasterxml.jackson.databind.json.JsonMapper
import com.fasterxml.jackson.databind.module.SimpleModule
import com.fasterxml.jackson.databind.ser.ContextualSerializer
@@ -38,18 +37,17 @@ object ScalaAnnotationIntrospectorTest {
case class CaseClassWithDefault(a: String = "defaultParam", b: Option[String] = Some("optionDefault"), c: Option[String])
- class ConcurrentLookupCache[T]() extends LookupCache[ClassKey, T] {
- final private val cache = TrieMap.empty[ClassKey, T]
+ class ConcurrentLookupCache[K, V]() extends LookupCache[K, V] {
+ final private val cache = TrieMap.empty[K, V]
- override def put(key: ClassKey, value: T): T =
- cache.put(key, value).getOrElse(None.orNull).asInstanceOf[T]
+ override def put(key: K, value: V): V =
+ cache.put(key, value).getOrElse(None.orNull).asInstanceOf[V]
- override def putIfAbsent(key: ClassKey, value: T): T =
- cache.putIfAbsent(key, value).getOrElse(None.orNull).asInstanceOf[T]
+ override def putIfAbsent(key: K, value: V): V =
+ cache.putIfAbsent(key, value).getOrElse(None.orNull).asInstanceOf[V]
- override def get(key: Any): T = key match {
- case classKey: ClassKey => cache.get(classKey).getOrElse(None.orNull).asInstanceOf[T]
- case _ => None.orNull.asInstanceOf[T]
+ override def get(key: Any): V = {
+ cache.get(key.asInstanceOf[K]).getOrElse(None.orNull).asInstanceOf[V]
}
override def clear(): Unit = {
@@ -58,6 +56,11 @@ object ScalaAnnotationIntrospectorTest {
override def size: Int = cache.size
}
+
+ object ConcurrentLookupCacheFactory extends LookupCacheFactory {
+ override def createLookupCache[K, V](initialEntries: Int, maxEntries: Int): LookupCache[K, V] =
+ new ConcurrentLookupCache[K, V]
+ }
}
class ScalaAnnotationIntrospectorTest extends FixtureAnyFlatSpec with Matchers {
@@ -231,10 +234,10 @@ class ScalaAnnotationIntrospectorTest extends FixtureAnyFlatSpec with Matchers {
}
}
- it should "allow descriptor cache to be replaced" in { _ =>
+ it should "allow descriptor cache to be replaced (old style)" in { _ =>
val defaultCache = ScalaAnnotationIntrospectorModule._descriptorCache
try {
- val cache = new ConcurrentLookupCache[BeanDescriptor]()
+ val cache = new ConcurrentLookupCache[String, BeanDescriptor]()
ScalaAnnotationIntrospectorModule.setDescriptorCache(cache)
val builder = JsonMapper.builder().addModule(DefaultScalaModule)
val mapper = builder.build()
@@ -244,16 +247,38 @@ class ScalaAnnotationIntrospectorTest extends FixtureAnyFlatSpec with Matchers {
withoutDefault.a shouldEqual "notDefault"
cache.size shouldBe >=(1)
- cache.get(new ClassKey(classOf[CaseClassWithDefault])) should not be (null)
+ cache.get(classOf[CaseClassWithDefault].getName) should not be (null)
} finally {
ScalaAnnotationIntrospectorModule.setDescriptorCache(defaultCache)
}
}
- it should "allow scala type cache to be replaced" in { _ =>
+ it should "allow descriptor cache to be replaced (new style)" in { _ =>
+ val defaultCache = ScalaAnnotationIntrospectorModule._descriptorCache
+ try {
+ ScalaAnnotationIntrospectorModule.setLookupCacheFactory(ConcurrentLookupCacheFactory)
+ val builder = JsonMapper.builder().addModule(DefaultScalaModule)
+ val mapper = builder.build()
+ val jsonWithKey = """{"a": "notDefault"}"""
+
+ val withoutDefault = mapper.readValue(jsonWithKey, classOf[CaseClassWithDefault])
+ withoutDefault.a shouldEqual "notDefault"
+
+ ScalaAnnotationIntrospectorModule._descriptorCache shouldBe a[ConcurrentLookupCache[String, BeanDescriptor]]
+ val cache = ScalaAnnotationIntrospectorModule._descriptorCache
+ .asInstanceOf[ConcurrentLookupCache[String, BeanDescriptor]]
+ cache.size shouldBe >=(1)
+ cache.get(classOf[CaseClassWithDefault].getName) should not be (null)
+ defaultCache.size() shouldEqual 0
+ } finally {
+ ScalaAnnotationIntrospectorModule.setLookupCacheFactory(DefaultLookupCacheFactory)
+ }
+ }
+
+ it should "allow scala type cache to be replaced (old style)" in { _ =>
val defaultCache = ScalaAnnotationIntrospectorModule._scalaTypeCache
try {
- val cache = new ConcurrentLookupCache[Boolean]()
+ val cache = new ConcurrentLookupCache[String, Boolean]()
ScalaAnnotationIntrospectorModule.setScalaTypeCache(cache)
val builder = JsonMapper.builder().addModule(DefaultScalaModule)
val mapper = builder.build()
@@ -263,16 +288,43 @@ class ScalaAnnotationIntrospectorTest extends FixtureAnyFlatSpec with Matchers {
withoutDefault.a shouldEqual "notDefault"
cache.size shouldBe >=(1)
- cache.get(new ClassKey(classOf[CaseClassWithDefault])) shouldBe true
+ cache.get(classOf[CaseClassWithDefault].getName) shouldBe true
val javaValueHolder = mapper.readValue("\"2\"", classOf[ValueHolder])
javaValueHolder should not be(null)
- cache.get(new ClassKey(classOf[ValueHolder])) shouldBe false
+ cache.get(classOf[ValueHolder].getName) shouldBe false
} finally {
ScalaAnnotationIntrospectorModule.setScalaTypeCache(defaultCache)
}
}
+ it should "allow scala type cache to be replaced (new style)" in { _ =>
+ val defaultCache = ScalaAnnotationIntrospectorModule._scalaTypeCache
+ try {
+ ScalaAnnotationIntrospectorModule.setLookupCacheFactory(ConcurrentLookupCacheFactory)
+ val builder = JsonMapper.builder().addModule(DefaultScalaModule)
+ val mapper = builder.build()
+ val jsonWithKey = """{"a": "notDefault"}"""
+
+ val withoutDefault = mapper.readValue(jsonWithKey, classOf[CaseClassWithDefault])
+ withoutDefault.a shouldEqual "notDefault"
+
+ ScalaAnnotationIntrospectorModule._scalaTypeCache shouldBe a[ConcurrentLookupCache[String, Boolean]]
+ val cache = ScalaAnnotationIntrospectorModule._scalaTypeCache
+ .asInstanceOf[ConcurrentLookupCache[String, Boolean]]
+ cache.size shouldBe >=(1)
+ cache.get(classOf[CaseClassWithDefault].getName) shouldBe true
+
+ val javaValueHolder = mapper.readValue("\"2\"", classOf[ValueHolder])
+ javaValueHolder should not be (null)
+ cache.get(classOf[ValueHolder].getName) shouldBe false
+
+ defaultCache.size() shouldEqual 0
+ } finally {
+ ScalaAnnotationIntrospectorModule.setLookupCacheFactory(DefaultLookupCacheFactory)
+ }
+ }
+
it should "allow scala3 check to be disabled" in { _ =>
ScalaAnnotationIntrospectorModule.shouldSupportScala3Classes() shouldBe true
try {