Skip to content

Commit

Permalink
Fix #408 - reified output value codec
Browse files Browse the repository at this point in the history
- add support for serialization and deserialization of Output with full OutputData metadata
  • Loading branch information
pawelprazak committed Mar 24, 2024
1 parent 2c0effe commit 4668012
Show file tree
Hide file tree
Showing 18 changed files with 1,374 additions and 1,156 deletions.
19 changes: 19 additions & 0 deletions CONTRIBUTING.md
Original file line number Diff line number Diff line change
Expand Up @@ -530,6 +530,25 @@ jcmd 25776 VM.flags
jcmd 25776 GC.heap_info
```

## Implementation details

### Serde - gRPC/Protobuf serialization and deserialization

The most important information about our serde:
- unknown values are neither empty nor non-empty - we simply don't know
- maps (structs) don't preserve `Null` protobuf value (but we do preserve unknown values)
- upstream uses special structures signatures [besom.internal.Constants.SpecialSig](core/src/main/scala/besom/internal/codecs.scala)
to encode internal metadata that Pulumi uses

Serialization entry points:
- [besom.internal.RegistersOutputs](core/src/main/scala/besom/internal/RegistersOutputs.scala)
- [besom.internal.PropertiesSerializer](core/src/main/scala/besom/internal/PropertiesSerializer.scala)
- [besom.internal.Encoder](core/src/main/scala/besom/internal/codecs.scala)

Deserialization entry points:
- [besom.internal.ResourceDecoder](core/src/main/scala/besom/internal/ResourceDecoder.scala)
- [besom.internal.Decoder](core/src/main/scala/besom/internal/codecs.scala)

## Getting Help

We are sure there are rough edges, and we appreciate you helping out.
Expand Down
16 changes: 0 additions & 16 deletions core/src/main/scala/besom/internal/CodecMacros.scala
Original file line number Diff line number Diff line change
Expand Up @@ -50,22 +50,6 @@ object CodecMacros:
exprOfEncoder :: recSummonEncodersImpl(Type.of[tail])
case _ => report.errorAndAbort("This can be ONLY called on tuples!")

inline def summonJsonEncoders[A]: List[JsonEncoder[?]] = ${ summonJsonEncodersImpl[A] }

private def summonJsonEncodersImpl[A: Type](using Quotes): Expr[List[JsonEncoder[?]]] =
Expr.ofList(recSummonJsonEncodersImpl(Type.of[A]))

private def recSummonJsonEncodersImpl(t: Type[?])(using Quotes): List[Expr[JsonEncoder[?]]] =
import quotes.reflect.*
t match
case '[EmptyTuple] => Nil
case '[head *: tail] =>
val exprOfJsonEncoder = Expr.summon[JsonEncoder[head]].getOrElse {
report.errorAndAbort(s"JsonEncoder for ${Type.show[head]} was not found!")
}
exprOfJsonEncoder :: recSummonJsonEncodersImpl(Type.of[tail])
case _ => report.errorAndAbort("This can be ONLY called on tuples!")

// inline def summonTypeclasses[A, TC[_]]: List[TC[Any]] = ${ summonTypeclassesImpl[A, TC] }

// private def summonTypeclassesImpl[A: Type, TC[_]: Type](using Quotes): Expr[List[TC[Any]]] =
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/besom/internal/OutputData.scala
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ object OutputData:
def empty[A](resources: Set[Resource] = Set.empty, isSecret: Boolean = false): OutputData[A] =
Known(resources, isSecret, None)

def traverseResult[A](using ctx: Context)(value: => Result[A]): Result[OutputData[A]] =
def traverseResult[A](using Context)(value: => Result[A]): Result[OutputData[A]] =
value.map(OutputData.apply(_))

def sequence[A, CC[X] <: IterableOnce[X], To](
Expand Down
21 changes: 7 additions & 14 deletions core/src/main/scala/besom/internal/PropertiesSerializer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -28,19 +28,12 @@ object PropertiesSerializer:
args: A,
filter: String => Boolean
)(using Context): Result[SerializationResult] =
summon[ArgsEncoder[A]].encode(args, filter).map { case (fieldsToResources, value) =>
SerializationResult(value, detectUnknowns(Value(Kind.StructValue(value))), fieldsToResources)
summon[ArgsEncoder[A]].encode(args, filter).map { case (fieldsToMetadata, value) =>
SerializationResult(value, detectUnknowns(fieldsToMetadata), fieldsToResources(fieldsToMetadata))
}

private[internal] def detectUnknowns(value: Value): Boolean =
value.kind match
case StringValue(str) => str == Constants.UnknownStringValue
case StructValue(struct) =>
struct.fields.foldLeft(false) { case (prev, (_, value)) =>
prev || detectUnknowns(value)
}
case ListValue(list) =>
list.values.foldLeft(false) { case (prev, value) =>
prev || detectUnknowns(value)
}
case _ => false // all other leaf types
private[internal] def detectUnknowns(metadata: Map[String, Metadata]): Boolean = metadata.values.exists(_.unknown)
private[internal] def fieldsToResources(metadata: Map[String, Metadata])(using Context): Map[String, Set[Resource]] =
metadata.map { case (k, m) =>
(k, m.dependencies.map(urn => DependencyResource(Output(urn)).asInstanceOf[Resource]).toSet)
}
159 changes: 100 additions & 59 deletions core/src/main/scala/besom/internal/ProtobufUtil.scala
Original file line number Diff line number Diff line change
@@ -1,11 +1,12 @@
package besom.internal

import besom.internal.Constants.*
import besom.types.URN
import besom.types.{URN, ResourceId}
import com.google.protobuf.struct.*
import com.google.protobuf.struct.Value.Kind
import com.google.protobuf.util.JsonFormat

import scala.annotation.unused
import scala.util.*

object ProtobufUtil:
Expand All @@ -14,7 +15,7 @@ object ProtobufUtil:
val Null: Value = Value(Kind.NullValue(NullValue.NULL_VALUE))

trait ToValue[A]:
extension (a: A) def asValue: Value
extension (@unused a: A) def asValue: Value

given ToValue[Int] with
extension (i: Int) def asValue: Value = Value(Kind.NumberValue(i))
Expand Down Expand Up @@ -54,51 +55,25 @@ object ProtobufUtil:
given ToValue[SpecialSig] with
extension (s: SpecialSig) def asValue: Value = s.asString.asValue

given ToValue[SecretValue] with
extension (s: SecretValue)
def asValue: Value = Map(
SpecialSig.Key -> SpecialSig.SecretSig.asValue,
ValueName -> s.value
).asValue

given ToValue[OutputValue] with
extension (o: OutputValue)
def asValue: Value = {
Map(
SpecialSig.Key -> SpecialSig.OutputSig.asValue
) ++ (if o.isKnown then Map(ValueName -> o.value) else Map.empty)
++ (if o.isSecret then Map(SecretName -> true.asValue) else Map.empty)
++ (if o.dependencies.nonEmpty
then Map(DependenciesName -> o.dependencies.map(_.asString.asValue).asValue)
else Map.empty)
}.asValue

extension (v: Value)
def asJsonString: Either[Throwable, String] = Try(printer.print(Value.toJavaProto(v))).toEither
def asJsonStringOrThrow: String = asJsonString.fold(t => throw Exception("Expected a JSON", t), identity)

def struct: Option[Struct] = v.kind.structValue
def outputValue: Option[OutputValue] = v.struct.flatMap(_.outputValue)
def secretValue: Option[SecretValue] = v.struct.flatMap(_.secretValue)
def isKnown: Boolean = !v.kind.isNullValue

def asSecretValue: Value = SecretValue(v).asValue
def asOutputValue(
isSecret: Boolean,
dependencies: List[URN]
): Value = OutputValue(
value = v,
isSecret = isSecret,
dependencies = dependencies
).asValue
): Value = OutputValue.known(value = v, isSecret = isSecret, dependencies = dependencies).asValue

def withSpecialSignature[A](f: PartialFunction[(Struct, SpecialSig), A]): Option[A] =
v.struct.flatMap(_.withSpecialSignature(f))
v.kind.structValue.flatMap(_.withSpecialSignature(f))

def valueType: String = v.kind.getClass.getSimpleName

end extension

extension (s: Struct)
def specialSignatureString: Option[String] =
private def specialSignatureString: Option[String] =
s.fields.get(SpecialSig.Key).flatMap(_.kind.stringValue)
def specialSignature: Option[SpecialSig] =
s.specialSignatureString.flatMap(SpecialSig.fromString)
Expand All @@ -108,38 +83,104 @@ object ProtobufUtil:
yield (s, sig)
maybeSpecial.collect(f)
end withSpecialSignature
def outputValue: Option[OutputValue] =
withSpecialSignature { case (struct, SpecialSig.OutputSig) =>
val value = struct.fields.getOrElse(ValueName, Null)
val isSecret = struct.fields.get(SecretName).flatMap(_.kind.boolValue).getOrElse(false)
val dependencyValues = struct.fields.get(DependenciesName).flatMap(_.kind.listValue).map(_.values.toList).getOrElse(Nil)
val dependencies = dependencyValues.flatMap(_.kind.stringValue).flatMap(s => URN.from(s).toOption)
OutputValue(value, isSecret, dependencies)
}
end outputValue
def secretValue: Option[SecretValue] =
withSpecialSignature { case (struct, SpecialSig.SecretSig) =>
val value = struct.fields.getOrElse(ValueName, Null)
SecretValue(value)
}

end ProtobufUtil

import ProtobufUtil.*
import ProtobufUtil.{*, given}

case class ResourceValue private (urn: Value, id: Option[Value])
object ResourceValue:
def apply(urn: Value, id: Value): Either[Exception, ResourceValue] =
val fixedIdValue = id match
case OutputValue(o) => Right(o.value.getOrElse(ResourceId.empty.asValue))
case Value(Kind.StringValue(id), _) if id == UnknownStringValue => Right(ResourceId.empty.asValue)
case v@Value(Kind.StringValue(_), _) => Right(v)
case v => Left(Exception(s"Unexpected id value type: ${v.valueType}"))
for
base <- apply(urn)
id <- fixedIdValue
yield base.copy(id = Some(id))
def apply(urn: Value): Either[Exception, ResourceValue] =
urn match
case OutputValue(o) => Right(ResourceValue(o.value.getOrElse(URN.empty.asString.asValue), None))
case Value(Kind.StringValue(urn), _) => Right(ResourceValue(urn.asValue, None))
case v => Left(Exception(s"Unexpected urn value type: ${v.valueType}"))

extension (r: ResourceValue)
def asValue: Value = {
Map(
SpecialSig.Key -> SpecialSig.ResourceSig.asValue,
ResourceUrnName -> r.urn
) ++ r.id.map(i => Map(ResourceIdName -> i)).getOrElse(Map.empty)
}.asValue

case class SecretValue(value: Value):
def isKnown: Boolean = !notKnown
def notKnown: Boolean = value.kind.isNullValue
def isKnown: Boolean = value.kind.stringValue.map(_ == UnknownStringValue).getOrElse(false)

object SecretValue:
def unapply(value: Value): Option[SecretValue] = value.secretValue
def unapply(struct: Struct): Option[SecretValue] = struct.secretValue
def unknown: SecretValue = SecretValue(UnknownStringValue.asValue)
def unapply(value: Value): Option[SecretValue] =
value.withSpecialSignature { case (struct, SpecialSig.SecretSig) =>
val value = struct.fields.getOrElse(ValueName, Null)
SecretValue(value)
}

extension (s: SecretValue)
def asValue: Value = Map(
SpecialSig.Key -> SpecialSig.SecretSig.asValue,
ValueName -> s.value
).asValue

case class OutputValue(value: Value, isSecret: Boolean, dependencies: List[URN]):
def isKnown: Boolean = value.isKnown
def notKnown: Boolean = !isKnown
case class OutputValue(value: Option[Value], isSecret: Boolean, dependencies: List[URN]):
def isKnown: Boolean = value.isDefined

object OutputValue:
def unapply(value: Value): Option[OutputValue] = value.outputValue
def unapply(struct: Struct): Option[OutputValue] = struct.outputValue
def unknown(isSecret: Boolean, dependencies: List[URN]): OutputValue =
OutputValue(Null, isSecret, dependencies)
def known(value: Value, isSecret: Boolean, dependencies: List[URN]): OutputValue = OutputValue(Some(value), isSecret, dependencies)
def unknown(isSecret: Boolean, dependencies: List[URN]): OutputValue = OutputValue(None, isSecret, dependencies)
def unapply(value: Value): Option[OutputValue] =
value.withSpecialSignature { case (struct, SpecialSig.OutputSig) =>
val value = struct.fields.get(ValueName)
val isSecret = struct.fields.get(SecretName).flatMap(_.kind.boolValue).getOrElse(false)
val dependencyValues = struct.fields.get(DependenciesName).flatMap(_.kind.listValue).map(_.values.toList).getOrElse(Nil)
val dependencies = dependencyValues.flatMap(_.kind.stringValue).flatMap(s => URN.from(s).toOption)
OutputValue(value, isSecret, dependencies)
}

extension (o: OutputValue)
// noinspection ScalaUnusedSymbol
private[internal] def asRawValueOrUnknown: Value = o.value.getOrElse(UnknownStringValue.asValue)
def asValue: Value = {
Map(
SpecialSig.Key -> SpecialSig.OutputSig.asValue
) ++ (if o.isKnown then Map(ValueName -> o.value.get) else Map.empty)
++ (if o.isSecret then Map(SecretName -> true.asValue) else Map.empty)
++ (if o.dependencies.nonEmpty
then Map(DependenciesName -> o.dependencies.map(_.asString.asValue).asValue)
else Map.empty)
}.asValue

case class Metadata(
known: Boolean,
secret: Boolean,
empty: Boolean,
dependencies: List[URN]
):
def unknown: Boolean = !known
def combine(that: Metadata): Metadata =
Metadata(
known && that.known,
secret || that.secret,
empty && that.empty,
dependencies ++ that.dependencies
)

def render(value: Value)(using ctx: Context): Value =
if ctx.featureSupport.keepOutputValues then OutputValue(if known then Some(value) else None, secret, dependencies).asValue
else
val v = if known then value else UnknownStringValue.asValue
if secret then SecretValue(v).asValue else v

end Metadata
object Metadata:
/* Default values that can be used safely with combine, e.g. as zero in fold operation */
def empty: Metadata = Metadata(known = true, secret = false, empty = false, Nil)
4 changes: 2 additions & 2 deletions core/src/main/scala/besom/internal/RegistersOutputs.scala
Original file line number Diff line number Diff line change
Expand Up @@ -52,9 +52,9 @@ object RegistersOutputs:
.unique(encoder.asTerm, "encode")
.appliedTo(fieldExpr.asTerm)
.appliedTo(ctx.asTerm)
.asExprOf[Result[(Set[Resource], Value)]]
.asExprOf[Result[(Metadata, Value)]]

'{ $encoded.map(depsAndValue => (${ Expr(fieldName) }, depsAndValue._2)) } // discard dependencies
'{ $encoded.map(metaAndValue => (${ Expr(fieldName) }, metaAndValue._2)) } // discard dependencies
}

val listOfResults = Expr.ofSeq(extractedFields)
Expand Down
15 changes: 8 additions & 7 deletions core/src/main/scala/besom/internal/ResourceDecoder.scala
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,11 @@ trait ResourceDecoder[A <: Resource]: // TODO rename to something more sensible
def makeResolver(using Context, BesomMDC[Label]): Result[(A, ResourceResolver[A])]

object ResourceDecoder:
inline def derived[A <: Resource]: ResourceDecoder[A] = ${ derivedImpl[A] }

class CustomPropertyExtractor[A](propertyName: String, decoder: Decoder[A]):

// noinspection ScalaUnusedSymbol
def extract(
fields: Map[String, Value],
dependencies: Map[String, Set[Resource]],
Expand All @@ -23,7 +26,7 @@ object ResourceDecoder:
): ValidatedResult[DecodingError, OutputData[A]] =
val resourceLabel = mdc.get(Key.LabelKey)

val fieldDependencies = dependencies.get(propertyName).getOrElse(Set.empty)
val fieldDependencies = dependencies.getOrElse(propertyName, Set.empty)
val propertyLabel = resourceLabel.withKey(propertyName)

val outputData =
Expand Down Expand Up @@ -67,8 +70,8 @@ object ResourceDecoder:
)

Promise[OutputData[URN]]().zip(Promise[OutputData[ResourceId]]()).zip(customPropertiesResults).map {
case (urnPromise, idPromise, customPopertiesPromises) =>
val allPromises = Vector(urnPromise, idPromise) ++ customPopertiesPromises.toList
case (urnPromise, idPromise, customPropertiesPromises) =>
val allPromises = Vector(urnPromise, idPromise) ++ customPropertiesPromises.toList

val propertiesOutputs = allPromises.map(promise => Output.ofData(promise.get)).toArray
val resource = fromProduct(Tuple.fromArray(propertiesOutputs))
Expand Down Expand Up @@ -102,7 +105,7 @@ object ResourceDecoder:
val dependencies = rawResourceResult.dependencies

val propertiesFulfilmentResults =
customPopertiesPromises.zip(customPropertyExtractors).map { (promise, extractor) =>
customPropertiesPromises.zip(customPropertyExtractors).map { (promise, extractor) =>
extractor.extract(fields, dependencies, resource).asResult.flatMap {
case Validated.Valid(outputData) => promise.fulfillAny(outputData)
case Validated.Invalid(errs) => promise.fail(AggregatedDecodingError(errs))
Expand All @@ -125,9 +128,7 @@ object ResourceDecoder:
}
end makeResolver

inline def derived[A <: Resource]: ResourceDecoder[A] = ${ derivedImpl[A] }

def derivedImpl[A <: Resource: Type](using q: Quotes): Expr[ResourceDecoder[A]] =
private def derivedImpl[A <: Resource: Type](using q: Quotes): Expr[ResourceDecoder[A]] =
Expr.summon[Mirror.Of[A]].get match
case '{
$m: Mirror.ProductOf[A] { type MirroredElemLabels = elementLabels; type MirroredElemTypes = elementTypes }
Expand Down
6 changes: 3 additions & 3 deletions core/src/main/scala/besom/internal/ResourceOps.scala
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ class ResourceOps(using ctx: Context, mdc: BesomMDC[Label]):
val resourceLabel = mdc.get(Key.LabelKey)
summon[Decoder[R]].decode(resultAsValue, resourceLabel).asResult.flatMap {
case Validated.Invalid(errs) =>
Result.fail(new AggregatedDecodingError(errs))
Result.fail(AggregatedDecodingError(errs))
case Validated.Valid(value) =>
Result.pure(value.withDependencies(props.values.flatten.toSet))
}
Expand Down Expand Up @@ -199,14 +199,14 @@ class ResourceOps(using ctx: Context, mdc: BesomMDC[Label]):
resolver: ResourceResolver[R],
state: ResourceState
): Result[Unit] =
val debugMessageSuffix = eitherErrorOrResult.fold(t => s"an error: ${t.getMessage()}", _ => "a result")
val debugMessageSuffix = eitherErrorOrResult.fold(t => s"an error: ${t.getMessage}", _ => "a result")

for
_ <- log.debug(s"Resolving resource ${state.asLabel} with $debugMessageSuffix")
_ <- log.trace(s"Resolving resource ${state.asLabel} with: ${pprint(eitherErrorOrResult)}")
errOrUnit <- resolver.resolve(eitherErrorOrResult).either
_ <- errOrUnit.fold(ctx.fail, _ => Result.unit) // fail context if resource resolution fails
errOrUnitMsg = errOrUnit.fold(t => s"with an error: ${t.getMessage()}", _ => "successfully")
errOrUnitMsg = errOrUnit.fold(t => s"with an error: ${t.getMessage}", _ => "successfully")
failResult = errOrUnit.fold(t => Result.fail(t), _ => Result.unit)
_ <- log.debug(s"Resolved resource ${state.asLabel} $errOrUnitMsg") *> failResult
yield ()
Expand Down
Loading

0 comments on commit 4668012

Please sign in to comment.