Skip to content

Commit

Permalink
maxJsonDepth when validating JSON
Browse files Browse the repository at this point in the history
  • Loading branch information
istreeter authored and spenes committed Aug 20, 2024
1 parent 998bfae commit fd4909f
Show file tree
Hide file tree
Showing 6 changed files with 343 additions and 103 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,13 @@ import io.circe.{DecodingFailure, Json}
*/
final class IgluCirceClient[F[_]] private (
resolver: Resolver[F],
schemaEvaluationCache: SchemaEvaluationCache[F]
schemaEvaluationCache: SchemaEvaluationCache[F],
maxJsonDepth: Int
) {
@deprecated("Use `IgluCirceClient(resolver, cache, maxJsonDepth)`", "3.2.0")
def this(resolver: Resolver[F], cache: SchemaEvaluationCache[F]) =
this(resolver, cache, Int.MaxValue)

def check(
instance: SelfDescribingData[Json]
)(implicit
Expand All @@ -50,7 +55,8 @@ final class IgluCirceClient[F[_]] private (
resolver.lookupSchemaResult(instance.schema, resolveSupersedingSchema = true)
)
validation =
CirceValidator.WithCaching.validate(schemaEvaluationCache)(instance.data, resolverResult)
CirceValidator.WithCaching
.validate(schemaEvaluationCache)(instance.data, resolverResult, maxJsonDepth)
_ <- EitherT(validation).leftMap(e =>
e.toClientError(resolverResult.value.supersededBy.map(_.asString))
)
Expand All @@ -61,21 +67,36 @@ final class IgluCirceClient[F[_]] private (

object IgluCirceClient {

@deprecated("Use `parseDefault(json, maxJsonDepth)`", "3.2.0")
def parseDefault[F[_]: Monad: CreateResolverCache: InitValidatorCache](
json: Json
): EitherT[F, DecodingFailure, IgluCirceClient[F]] =
parseDefault(json, Int.MaxValue)

def parseDefault[F[_]: Monad: CreateResolverCache: InitValidatorCache](
json: Json,
maxJsonDepth: Int
): EitherT[F, DecodingFailure, IgluCirceClient[F]] =
for {
config <- EitherT.fromEither[F](Resolver.parseConfig(json))
resolver <- Resolver.fromConfig[F](config)
client <- EitherT.liftF(fromResolver(resolver, config.cacheSize))
client <- EitherT.liftF(fromResolver(resolver, config.cacheSize, maxJsonDepth))
} yield client

@deprecated("Use `fromResolver(resolver, cacheSize, maxJsonDepth)`", "3.2.0")
def fromResolver[F[_]: Monad: InitValidatorCache](
resolver: Resolver[F],
cacheSize: Int
): F[IgluCirceClient[F]] =
fromResolver(resolver, cacheSize, Int.MaxValue)

def fromResolver[F[_]: Monad: InitValidatorCache](
resolver: Resolver[F],
cacheSize: Int,
maxJsonDepth: Int
): F[IgluCirceClient[F]] = {
schemaEvaluationCache[F](cacheSize).map { cache =>
new IgluCirceClient(resolver, cache)
new IgluCirceClient(resolver, cache, maxJsonDepth)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import scala.jdk.CollectionConverters._

// Cats
import cats.Monad
import cats.data.{EitherNel, NonEmptyList}
import cats.data.NonEmptyList
import cats.syntax.all._

// Jackson
Expand Down Expand Up @@ -87,35 +87,37 @@ object CirceValidator extends Validator[Json] {
private lazy val V4Schema =
V4SchemaInstance.getSchema(new ObjectMapper().readTree(MetaSchemas.JsonSchemaV4Text))

def validate(data: Json, schema: Json): Either[ValidatorError, Unit] = {
val jacksonJson = circeToJackson(schema)
evaluateSchema(jacksonJson)
.flatMap { schema =>
validateOnReadySchema(schema, data).leftMap(ValidatorError.InvalidData.apply)
}
}

def checkSchema(schema: Json): List[ValidatorError.SchemaIssue] = {
val jacksonJson = circeToJackson(schema)
validateSchemaAgainstV4(jacksonJson)
def validate(data: Json, schema: Json): Either[ValidatorError, Unit] =
for {
jacksonJson <- circeToJackson(schema, Int.MaxValue).leftMap(_.toInvalidSchema)
schema <- evaluateSchema(jacksonJson)
_ <- validateOnReadySchema(schema, data, Int.MaxValue)
} yield ()

@deprecated("Use `checkSchema(schema, maxJsonDepth)`", "3.2.0")
def checkSchema(schema: Json): List[ValidatorError.SchemaIssue] =
checkSchema(schema, Int.MaxValue)

def checkSchema(schema: Json, maxJsonDepth: Int): List[ValidatorError.SchemaIssue] = {
circeToJackson(schema, maxJsonDepth) match {
case Left(e) => List(e.toSchemaIssue)
case Right(jacksonJson) => validateSchemaAgainstV4(jacksonJson)
}
}

/** Validate instance against schema and return same instance */
private def validateOnReadySchema(
schema: JsonSchema,
instance: Json
): EitherNel[ValidatorReport, Unit] = {
val messages = schema
.validate(circeToJackson(instance))
.asScala
.toList
.map(fromValidationMessage)

messages match {
case x :: xs => NonEmptyList(x, xs).asLeft
case Nil => ().asRight
}
}
instance: Json,
maxJsonDepth: Int
): Either[ValidatorError.InvalidData, Unit] =
for {
jacksonJson <- circeToJackson(instance, maxJsonDepth).leftMap(_.toInvalidData)
_ <- schema.validate(jacksonJson).asScala.toList.map(fromValidationMessage) match {
case x :: xs => ValidatorError.InvalidData(NonEmptyList(x, xs)).asLeft
case Nil => ().asRight
}
} yield ()

private def fromValidationMessage(m: ValidationMessage): ValidatorReport =
ValidatorReport(m.getMessage, m.getPath.some, m.getArguments.toList, m.getType.some)
Expand Down Expand Up @@ -154,41 +156,48 @@ object CirceValidator extends Validator[Json] {

def validate[F[_]: Monad](
schemaEvaluationCache: SchemaEvaluationCache[F]
)(data: Json, schema: SchemaLookupResult): F[Either[ValidatorError, Unit]] = {
getFromCacheOrEvaluate(schemaEvaluationCache)(schema)
)(
data: Json,
schema: SchemaLookupResult,
maxJsonDepth: Int
): F[Either[ValidatorError, Unit]] = {
getFromCacheOrEvaluate(schemaEvaluationCache)(schema, maxJsonDepth)
.map {
_.flatMap { jsonschema =>
validateOnReadySchema(jsonschema, data)
.leftMap(ValidatorError.InvalidData.apply)
validateOnReadySchema(jsonschema, data, maxJsonDepth)
}
}
}

private def getFromCacheOrEvaluate[F[_]: Monad](
evaluationCache: SchemaEvaluationCache[F]
)(result: SchemaLookupResult): F[Either[ValidatorError.InvalidSchema, JsonSchema]] = {
)(
result: SchemaLookupResult,
maxJsonDepth: Int
): F[Either[ValidatorError.InvalidSchema, JsonSchema]] = {
result match {
case ResolverResult.Cached(key, SchemaItem(schema, _), timestamp) =>
evaluationCache.get((key, timestamp)).flatMap {
case Some(alreadyEvaluatedSchema) =>
alreadyEvaluatedSchema.pure[F]
case None =>
provideNewJsonSchema(schema)
provideNewJsonSchema(schema, maxJsonDepth)
.pure[F]
.flatTap(result => evaluationCache.put((key, timestamp), result))
}
case ResolverResult.NotCached(SchemaItem(schema, _)) =>
provideNewJsonSchema(schema).pure[F]
provideNewJsonSchema(schema, maxJsonDepth).pure[F]
}
}

private def provideNewJsonSchema(
schema: Json
schema: Json,
maxJsonDepth: Int
): Either[ValidatorError.InvalidSchema, JsonSchema] = {
val schemaAsNode = circeToJackson(schema)
for {
_ <- validateSchema(schemaAsNode)
evaluated <- evaluateSchema(schemaAsNode)
schemaAsNode <- circeToJackson(schema, maxJsonDepth).leftMap(_.toInvalidSchema)
_ <- validateSchema(schemaAsNode)
evaluated <- evaluateSchema(schemaAsNode)
} yield evaluated
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* Copyright (c) 2014-2024 Snowplow Analytics Ltd. All rights reserved.
*
* This program is licensed to you under the Apache License Version 2.0,
* and you may not use this file except in compliance with the Apache License Version 2.0.
* You may obtain a copy of the Apache License Version 2.0 at http://www.apache.org/licenses/LICENSE-2.0.
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the Apache License Version 2.0 is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the Apache License Version 2.0 for the specific language governing permissions and limitations there under.
*/
package io.circe.jackson.snowplow

import cats.data.NonEmptyList
import com.snowplowanalytics.iglu.client.validator.{ValidatorError, ValidatorReport}

sealed trait CirceToJsonError extends Product with Serializable {
def message: String

def toInvalidData: ValidatorError.InvalidData =
ValidatorError.InvalidData(
NonEmptyList.one(
ValidatorReport(message, Some("/"), List.empty, None)
)
)

def toInvalidSchema: ValidatorError.InvalidSchema =
ValidatorError.InvalidSchema(NonEmptyList.one(toSchemaIssue))

def toSchemaIssue: ValidatorError.SchemaIssue =
ValidatorError.SchemaIssue(path = "/", message = message)
}

object CirceToJsonError {
case object MaxDepthExceeded extends CirceToJsonError {
override def message: String = "Maximum allowed JSON depth exceeded"
}
}
109 changes: 63 additions & 46 deletions modules/core/src/main/scala/io/circe/jackson/snowplow/package.scala
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,9 @@
package io.circe
package jackson

import cats.syntax.either._
import cats.syntax.traverse._

import scala.jdk.CollectionConverters._

import java.math.{BigDecimal => JBigDecimal}
Expand All @@ -30,60 +33,74 @@ package object snowplow {
/**
* Converts given circe's Json instance to Jackson's JsonNode
* Numbers with exponents exceeding Integer.MAX_VALUE are converted to strings
* '''Warning: This implementation is not stack safe and will fail on very deep structures'''
* @param json instance of circe's Json
* @return converted JsonNode
*/
final def circeToJackson(json: Json): JsonNode =
json.fold(
NullNode.instance,
BooleanNode.valueOf(_),
number =>
if (json == negativeZeroJson)
DoubleNode.valueOf(number.toDouble)
else
number match {
case _: JsonBiggerDecimal | _: JsonBigDecimal =>
number.toBigDecimal
.map(bigDecimal => {
if (bigDecimal.isValidInt)
IntNode.valueOf(bigDecimal.intValue)
else if (bigDecimal.isValidLong) {
LongNode.valueOf(bigDecimal.longValue)
} else if (bigDecimal.isWhole) {
BigIntegerNode.valueOf(bigDecimal.toBigInt.underlying)
} else
DecimalNode.valueOf(bigDecimal.underlying)
})
.getOrElse(TextNode.valueOf(number.toString))
case JsonLong(x) => LongNode.valueOf(x)
case JsonDouble(x) => DoubleNode.valueOf(x)
case JsonFloat(x) => FloatNode.valueOf(x)
case JsonDecimal(x) =>
try
if (x.contains('.') || x.toLowerCase.contains('e'))
DecimalNode.valueOf(new JBigDecimal(x))
else
getJsonNodeFromStringContent(x)
catch {
case _: NumberFormatException => TextNode.valueOf(x)
case _: JsonParseException => TextNode.valueOf(x)
final def circeToJackson(json: Json, maxJsonDepth: Int): Either[CirceToJsonError, JsonNode] =
if (maxJsonDepth <= 0) CirceToJsonError.MaxDepthExceeded.asLeft
else
json.fold(
NullNode.instance.asRight,
BooleanNode.valueOf(_).asRight,
number =>
{
if (json == negativeZeroJson)
DoubleNode.valueOf(number.toDouble)
else
number match {
case _: JsonBiggerDecimal | _: JsonBigDecimal =>
number.toBigDecimal
.map(bigDecimal => {
if (bigDecimal.isValidInt)
IntNode.valueOf(bigDecimal.intValue)
else if (bigDecimal.isValidLong) {
LongNode.valueOf(bigDecimal.longValue)
} else if (bigDecimal.isWhole) {
BigIntegerNode.valueOf(bigDecimal.toBigInt.underlying)
} else
DecimalNode.valueOf(bigDecimal.underlying)
})
.getOrElse(TextNode.valueOf(number.toString))
case JsonLong(x) => LongNode.valueOf(x)
case JsonDouble(x) => DoubleNode.valueOf(x)
case JsonFloat(x) => FloatNode.valueOf(x)
case JsonDecimal(x) =>
try
if (x.contains('.') || x.toLowerCase.contains('e'))
DecimalNode.valueOf(new JBigDecimal(x))
else
getJsonNodeFromStringContent(x)
catch {
case _: NumberFormatException => TextNode.valueOf(x)
case _: JsonParseException => TextNode.valueOf(x)
}
}
},
s => TextNode.valueOf(s),
array => JsonNodeFactory.instance.arrayNode.addAll(array.map(circeToJackson).asJava),
obj =>
objectNodeSetAll(
JsonNodeFactory.instance.objectNode,
obj.toMap.map { case (k, v) =>
(k, circeToJackson(v))
}.asJava
)
)
}.asRight,
s => TextNode.valueOf(s).asRight,
array =>
array
.traverse(circeToJackson(_, maxJsonDepth - 1))
.map(l => JsonNodeFactory.instance.arrayNode.addAll(l.asJava)),
obj =>
obj.toList
.traverse { case (k, v) =>
circeToJackson(v, maxJsonDepth - 1).map((k, _))
}
.map { l =>
objectNodeSetAll(
JsonNodeFactory.instance.objectNode,
l.toMap.asJava
)
}
)

def objectNodeSetAll(node: ObjectNode, fields: java.util.Map[String, JsonNode]): JsonNode =
node.setAll[JsonNode](fields)

private def getJsonNodeFromStringContent(content: String): JsonNode =
mapper.readTree(content)

@deprecated("Use `circeToJackson(json, maxJsonDepth)`", "3.2.0")
final def circeToJackson(json: Json): JsonNode =
circeToJackson(json, Int.MaxValue).toOption.get
}
Original file line number Diff line number Diff line change
Expand Up @@ -108,5 +108,7 @@ object SpecHelpers {
val TestResolver = Resolver.init[IO](10, None, EmbeddedTest)
val TestClient = for { resolver <- TestResolver } yield Client(resolver, CirceValidator)
val CachingTestClient =
TestResolver.flatMap(resolver => IgluCirceClient.fromResolver[IO](resolver, cacheSize = 10))
TestResolver.flatMap(resolver =>
IgluCirceClient.fromResolver[IO](resolver, cacheSize = 10, maxJsonDepth = 40)
)
}
Loading

0 comments on commit fd4909f

Please sign in to comment.