Multiple fat DARs scala/codegen support, #109 (#366)

Add Dar Traverse and Equal, replace manually written test cases with
scalaz law checks;
Add test case for multiple DARs support;
Enforcing non empty list of input files
This commit is contained in:
Leonid Shlyapnikov 2019-04-15 17:12:07 -04:00 committed by GitHub
parent b16176338c
commit bdb5160403
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 205 additions and 184 deletions

View File

@ -125,6 +125,7 @@ da_scala_test_suite(
"//3rdparty/jvm/org/scalacheck",
"//3rdparty/jvm/org/scalatest",
"//3rdparty/jvm/org/scalaz:scalaz_core",
"//3rdparty/jvm/org/scalaz:scalaz_scalacheck_binding",
"//daml-lf/data",
],
)

View File

@ -3,15 +3,32 @@
package com.digitalasset.daml.lf
import scalaz.Functor
import scalaz.{Applicative, Equal, Traverse}
import scalaz.syntax.equal._
import scalaz.std.list._
import scala.language.higherKinds
case class Dar[A](main: A, dependencies: List[A]) {
final case class Dar[A](main: A, dependencies: List[A]) {
lazy val all: List[A] = main :: dependencies
}
object Dar {
implicit val darFunctor: Functor[Dar] = new Functor[Dar] {
implicit val darTraverse: Traverse[Dar] = new Traverse[Dar] {
override def map[A, B](fa: Dar[A])(f: A => B): Dar[B] =
Dar[B](main = f(fa.main), dependencies = fa.dependencies.map(f))
override def traverseImpl[G[_]: Applicative, A, B](fa: Dar[A])(f: A => G[B]): G[Dar[B]] = {
import scalaz.syntax.apply._
import scalaz.syntax.traverse._
import scalaz.std.list._
val gb: G[B] = f(fa.main)
val gbs: G[List[B]] = fa.dependencies.traverse(f)
^(gb, gbs)((b, bs) => Dar(b, bs))
}
}
implicit def darEqual[A: Equal]: Equal[Dar[A]] = new Equal[Dar[A]] {
override def equal(a1: Dar[A], a2: Dar[A]): Boolean =
a1.main === a2.main && a1.dependencies === a2.dependencies
}
}

View File

@ -2,27 +2,30 @@
// SPDX-License-Identifier: Apache-2.0
package com.digitalasset.daml.lf
import org.scalacheck.{Arbitrary, Gen}
import org.scalatest.prop.GeneratorDrivenPropertyChecks
import org.scalacheck.{Arbitrary, Gen, Properties}
import org.scalatest.prop.Checkers
import org.scalatest.{FlatSpec, Matchers}
import scalaz.std.anyVal._
import scalaz.scalacheck.ScalazProperties
class DarSpec extends FlatSpec with Matchers with GeneratorDrivenPropertyChecks {
behavior of Dar.getClass.getSimpleName
class DarSpec extends FlatSpec with Matchers with Checkers {
behavior of s"${Dar.getClass.getSimpleName} Equal"
checkLaws(ScalazProperties.equal.laws[Dar[Int]])
it should "implement Functor with proper map" in forAll(darGen[Int]) { dar =>
import scalaz.Functor
import scalaz.syntax.functor._
behavior of s"${Dar.getClass.getSimpleName} Traverse"
checkLaws(ScalazProperties.traverse.laws[Dar])
implicit val sut: Functor[Dar] = Dar.darFunctor
behavior of s"${Dar.getClass.getSimpleName} Functor"
checkLaws(ScalazProperties.functor.laws[Dar])
def f(a: Int): Int = a + 100
dar.map(f) shouldBe Dar(f(dar.main), dar.dependencies.map(f))
}
private def checkLaws(props: Properties): Unit =
props.properties foreach { case (s, p) => it should s in check(p) }
private def darGen[A: Arbitrary]: Gen[Dar[A]] =
for {
main <- Arbitrary.arbitrary[A]
dependencies <- Arbitrary.arbitrary[List[A]]
} yield Dar[A](main, dependencies)
private implicit def darArb[A: Arbitrary]: Arbitrary[Dar[A]] = Arbitrary(darGen)
}

View File

@ -28,7 +28,7 @@ object DamlLfV1ArchiveReader {
case _ => s"Unexpected exception: ${t.getMessage}"
}
private[this] def readPackage(lf: DamlLf.ArchivePayload): String \/ DamlLf1.Package = {
private[iface] def readPackage(lf: DamlLf.ArchivePayload): String \/ DamlLf1.Package = {
import DamlLf.ArchivePayload.{SumCase => SC}
lf.getSumCase match {
case SC.DAML_LF_1 => \/-(lf.getDamlLf1)

View File

@ -66,6 +66,10 @@ object Interface {
def read(lf: DamlLf.Archive): (Errors[ErrorLoc, InvalidDataTypeDefinition], Interface) =
readInterface(lf)
def read(lf: (PackageId, DamlLf.ArchivePayload))
: (Errors[ErrorLoc, InvalidDataTypeDefinition], Interface) =
readInterface(lf)
def read(f: () => String \/ (PackageId, DamlLf1.Package))
: (Errors[ErrorLoc, InvalidDataTypeDefinition], Interface) =
readInterface(f)
@ -123,13 +127,20 @@ object InterfaceReader {
def readInterface(lf: DamlLf.Archive): (Errors[ErrorLoc, InvalidDataTypeDefinition], Interface) =
readInterface(() => DamlLfV1ArchiveReader.readPackage(lf))
def readInterface(lf: (PackageId, DamlLf.ArchivePayload))
: (Errors[ErrorLoc, InvalidDataTypeDefinition], Interface) = {
readInterface(() => lf.traverseU(DamlLfV1ArchiveReader.readPackage))
}
private val dummyPkgId = SimpleString.assertFromString("-dummyPkg-")
private val dummyInterface = Interface(dummyPkgId, Map.empty)
def readInterface(f: () => String \/ (PackageId, DamlLf1.Package))
: (Errors[ErrorLoc, InvalidDataTypeDefinition], Interface) =
f() match {
case -\/(e) =>
(point(InvalidDataTypeDefinition(e)), Interface(dummyPkgId, Map.empty))
(point(InvalidDataTypeDefinition(e)), dummyInterface)
case \/-((templateGroupId, lfPackage)) =>
lfprintln(s"templateGroupId: $templateGroupId")
lfprintln(s"package: $lfPackage")

View File

@ -15,19 +15,27 @@ load("@os_info//:os_info.bzl", "is_linux")
daml_compile(
name = "MyMain",
damlc = "//daml-foundations/daml-tools/da-hs-damlc-app:da-hs-damlc-app-deprecated",
main_src = "src/main/daml/MyMain.daml",
target = "1.3",
)
daml_compile(
name = "MySecondMain",
main_src = "src/main/daml/MySecondMain.daml",
target = "1.3",
)
genrule(
name = "MyMain-codegen",
srcs = [":MyMain"],
srcs = [
":MyMain",
":MySecondMain",
],
outs = ["MyMain-codegen-out"],
cmd = "$(execpath //language-support/scala/codegen:codegen-main) --input-file $(location :MyMain.dalf) --dependencies $(location //daml-foundations/daml-ghc/package-database/deprecated:daml-prim-1.3.dalf) --package-name com.digitalasset.sample --output-dir $@",
cmd = "$(execpath //language-support/scala/codegen:codegen-main) --input-files $(location :MyMain.dar),$(location :MySecondMain.dar) --package-name com.digitalasset.sample --output-dir $@",
tools = [
":MyMain.dalf",
"//daml-foundations/daml-ghc/package-database/deprecated:daml-prim-1.3.dalf",
":MyMain.dar",
":MySecondMain.dar",
"//language-support/scala/codegen:codegen-main",
],
)
@ -130,8 +138,8 @@ da_scala_test(
],
data = [
"src/test/resources/logback-sandbox.xml",
":MyMain.dalf",
"//daml-foundations/daml-ghc/package-database/deprecated:daml-prim-1.3.dalf",
":MyMain.dar",
":MySecondMain.dar",
"//ledger/sandbox:sandbox-binary_deploy.jar",
],
jvm_flags = [

View File

@ -0,0 +1,12 @@
-- Copyright (c) 2019 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved.
-- SPDX-License-Identifier: Apache-2.0
daml 1.2
module MySecondMain where
template DummyTemplateFromAnotherDar
with
owner: Party
text: Text
where
signatory owner

View File

@ -54,6 +54,7 @@ import com.digitalasset.sample.MyMain.{
TemplateWithUnitParam,
VariantWithRecordWithVariant
}
import com.digitalasset.sample.MySecondMain
import com.digitalasset.util.Ctx
import com.google.protobuf.empty.Empty
import org.scalacheck.Arbitrary.arbitrary
@ -83,8 +84,8 @@ class ScalaCodeGenIT
private val ledgerId = this.getClass.getSimpleName
private val archives = List(
requiredResource("language-support/scala/codegen-sample-app/MyMain.dalf"),
requiredResource("daml-foundations/daml-ghc/package-database/deprecated/daml-prim-1.3.dalf")
requiredResource("language-support/scala/codegen-sample-app/MyMain.dar"),
requiredResource("language-support/scala/codegen-sample-app/MySecondMain.dar"),
)
private val asys = ActorSystem()
@ -333,6 +334,13 @@ class ScalaCodeGenIT
testCreateContractAndReceiveEvent(contract copy (party = alice), alice)
}
"alice creates DummyTemplateFromAnotherDar contract and receives corresponding event" in {
import com.digitalasset.ledger.client.binding.encoding.GenEncoding.Implicits._
val contract = arbitrary[MySecondMain.DummyTemplateFromAnotherDar].sample getOrElse sys.error(
"random DummyTemplateFromAnotherDar failed")
testCreateContractAndReceiveEvent(contract copy (owner = alice), alice)
}
private def testCreateContractAndReceiveEvent(
contract: Template[AnyRef],
party: P.Party): Assertion = {

View File

@ -4,19 +4,20 @@
package com.digitalasset.codegen
import com.digitalasset.codegen.types.Namespace
import com.digitalasset.daml.lf.iface, iface.{Type => _, _}
import com.digitalasset.daml.lf.iface.reader.InterfaceType
import com.digitalasset.daml.lf.{Dar, UniversalArchiveReader, iface}
import iface.{Type => _, _}
import com.digitalasset.daml.lf.iface.reader.{Errors, Interface, InterfaceReader}
import java.io._
import java.net.URL
import java.nio.file.Files
import scala.collection.breakOut
import com.digitalasset.codegen.dependencygraph._
import com.digitalasset.codegen.exception.PackageInterfaceException
import com.digitalasset.codegen.lf.EnvironmentInterface.environmentInterfaceSemigroup
import com.digitalasset.daml.lf.data.ImmArray.ImmArraySeq
import lf.{DefTemplateWithRecord, LFUtil, ScopedDataType}
import com.digitalasset.daml_lf.DamlLf
import lf.{DefTemplateWithRecord, EnvironmentInterface, LFUtil, ScopedDataType}
import com.digitalasset.daml.lf.data.Ref._
import com.digitalasset.daml.lf.iface.reader.Errors.ErrorLoc
import com.digitalasset.daml_lf.DamlLf
import scalaz._
import scalaz.std.tuple._
import scalaz.std.list._
@ -27,68 +28,14 @@ import scalaz.syntax.std.option._
import scalaz.syntax.bind._
import scalaz.syntax.traverse1._
import scala.util.{Failure, Success}
object CodeGen {
sealed abstract class Mode extends Serializable with Product { self =>
type Dialect <: Util { type Interface <: self.Interface }
type InterfaceElement
type Interface
private[CodeGen] val Dialect: (String, Interface, File) => Dialect
type Payload = (PackageId, DamlLf.ArchivePayload)
private[CodeGen] def decodeInterfaceFromStream(
format: PackageFormat,
bis: BufferedInputStream): String \/ InterfaceElement
private[CodeGen] def combineInterfaces(
leader: InterfaceElement,
dependencies: Seq[InterfaceElement]): Interface
private[CodeGen] def templateCount(interface: Interface): Int
}
case object Novel extends Mode {
import reader.InterfaceReader
type Dialect = LFUtil
type InterfaceElement = reader.Interface
type Interface = lf.EnvironmentInterface
private[CodeGen] val Dialect = LFUtil.apply _
private[CodeGen] override def decodeInterfaceFromStream(
format: PackageFormat,
bis: BufferedInputStream): String \/ InterfaceElement =
format match {
case PackageFormat.SDaml =>
\/.left("sdaml v1 not supported")
case PackageFormat.SDamlV2 =>
\/.left("sdaml v2 not supported")
case PackageFormat.DamlLF =>
\/.fromTryCatchNonFatal {
val (errors, out) = reader.Interface.read(
DamlLf.Archive.parser().parseFrom(bis)
)
println(
s"Codegen decoded archive with Package ID: ${out.packageId.underlyingString: String}")
if (!errors.empty)
\/.left(
("Errors reading LF archive:\n" +: InterfaceReader.InterfaceReaderError.treeReport(
errors)).toString)
else \/.right(out)
}.leftMap(_.getLocalizedMessage).join
}
private[CodeGen] override def combineInterfaces(
leader: InterfaceElement,
dependencies: Seq[InterfaceElement]): Interface =
lf.EnvironmentInterface fromReaderInterfaces (leader, dependencies: _*)
private[CodeGen] override def templateCount(interface: Interface): Int = {
interface.typeDecls.count {
case (_, InterfaceType.Template(_, _)) => true
case _ => false
}
}
}
sealed abstract class Mode extends Serializable with Product
case object Novel extends Mode
val universe: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe
import universe._
@ -96,7 +43,7 @@ object CodeGen {
import Util.{FilePlan, WriteParams, partitionEithers}
/*
* Given an DAML package (in sdaml format), a package name and an output
* Given a DAML package (in DAR or DALF format), a package name and an output
* directory, this function writes a bunch of generated .scala files
* to 'outputDir' that mirror the namespace of the DAML package.
*
@ -109,76 +56,83 @@ object CodeGen {
@throws[PackageInterfaceException](
cause = "either decoding a package from a file or extracting" +
" the package interface failed")
def generateCode(
sdamlFile: File,
otherDalfInputs: Seq[URL],
def generateCode(files: List[File], packageName: String, outputDir: File, mode: Mode): Unit =
files match {
case Nil =>
throw PackageInterfaceException("Expected at least one DAR or DALF input file.")
case f :: fs =>
generateCodeSafe(NonEmptyList(f, fs: _*), packageName, outputDir, mode)
.fold(es => throw PackageInterfaceException(formatErrors(es)), identity)
}
private def formatErrors(es: NonEmptyList[String]): String =
es.toList.mkString("\n")
def generateCodeSafe(
files: NonEmptyList[File],
packageName: String,
outputDir: File,
mode: Mode): Unit = {
val errorOrRun = for {
interface <- decodePackageFromFile(sdamlFile, mode)
dependencies <- decodePackagesFromURLs(otherDalfInputs, mode)
combined = mode.combineInterfaces(interface, dependencies)
} yield packageInterfaceToScalaCode(mode.Dialect(packageName, combined, outputDir))
errorOrRun fold (e => throw PackageInterfaceException(e), identity)
}
private def decodePackageFromFile(
sdamlFile: File,
mode: Mode): String \/ mode.InterfaceElement = {
val is = Files.newInputStream(sdamlFile.toPath)
println(s"Decoding ${sdamlFile.toPath}")
try decodePackageFrom(is, mode)
finally is.close() // close is in case of fatal throwables
}
private def decodePackagesFromURLs(
urls: Seq[URL],
mode: Mode): String \/ Seq[mode.InterfaceElement] =
urls
.map { url =>
val is = url.openStream()
try decodePackageFrom(is, mode)
finally is.close()
}
.toList
.sequenceU
@throws[FileNotFoundException](cause = "input file not found")
@throws[SecurityException](cause = "input file not readable")
private def decodePackageFrom(is: InputStream, mode: Mode): String \/ mode.InterfaceElement = {
val bis = new BufferedInputStream(is)
for {
format <- detectPackageFormat(bis)
result <- mode.decodeInterfaceFromStream(format, bis)
} yield result
}
sealed trait PackageFormat
object PackageFormat {
case object SDaml extends PackageFormat
case object SDamlV2 extends PackageFormat
case object DamlLF extends PackageFormat
}
private def detectPackageFormat(is: InputStream): String \/ PackageFormat = {
if (is.markSupported()) {
is.mark(1024)
val buf = Array.ofDim[Byte](11)
is.read(buf)
is.reset()
\/.right(
if (buf.startsWith("("))
if (buf.startsWith("(pkge\"v2.0\"")) PackageFormat.SDamlV2
else PackageFormat.SDaml
else
PackageFormat.DamlLF
)
} else {
\/.left("input stream doesn't support mark")
mode: Mode): ValidationNel[String, Unit] =
decodeInterfaces(files).map { ifaces: NonEmptyList[EnvironmentInterface] =>
val combinedIface: EnvironmentInterface = combineEnvInterfaces(ifaces)
packageInterfaceToScalaCode(util(mode, packageName, combinedIface, outputDir))
}
private def util(
mode: Mode,
packageName: String,
iface: EnvironmentInterface,
outputDir: File): Util = mode match {
case Novel => LFUtil(packageName, iface, outputDir)
}
private def decodeInterfaces(
files: NonEmptyList[File]): ValidationNel[String, NonEmptyList[EnvironmentInterface]] = {
val reader = UniversalArchiveReader()
val parse: File => String \/ Dar[Payload] = parseFile(reader)
files.traverseU(f => decodeInterface(parse)(f).validationNel)
}
private def parseFile(reader: UniversalArchiveReader[Payload])(f: File): String \/ Dar[Payload] =
reader.readFile(f) match {
case Success(p) => \/.right(p)
case Failure(e) =>
e.printStackTrace()
\/.left(e.getLocalizedMessage)
}
private def decodeInterface(parse: File => String \/ Dar[Payload])(
file: File): String \/ EnvironmentInterface =
parse(file).flatMap(decodeInterface)
private def decodeInterface(dar: Dar[Payload]): String \/ EnvironmentInterface = {
import scalaz.syntax.traverse._
dar.traverseU(decodeInterface).map(combineInterfaces)
}
private def decodeInterface(p: Payload): String \/ Interface =
\/.fromTryCatchNonFatal {
val packageId: PackageId = p._1
println(
s"Scala Codegen - decoding archive with Package ID: ${packageId.underlyingString: String}")
val (errors, out) = Interface.read(p)
if (!errors.empty) {
\/.left(formatDecodeErrors(packageId, errors))
} else \/.right(out)
}.leftMap(_.getLocalizedMessage).join
private def formatDecodeErrors(
packageId: PackageId,
errors: Errors[ErrorLoc, InterfaceReader.InvalidDataTypeDefinition]): String =
(Cord(s"Errors decoding LF archive (Package ID: ${packageId.underlyingString: String}):\n") ++
InterfaceReader.InterfaceReaderError.treeReport(errors)).toString
private def combineInterfaces(dar: Dar[Interface]): EnvironmentInterface =
EnvironmentInterface.fromReaderInterfaces(dar)
private def combineEnvInterfaces(as: NonEmptyList[EnvironmentInterface]): EnvironmentInterface = {
val z = EnvironmentInterface(Map.empty)
as.foldLeft(z)((a1, a2) => environmentInterfaceSemigroup.append(a1, a2))
}
private def packageInterfaceToScalaCode(util: Util): Unit = {
@ -212,7 +166,7 @@ object CodeGen {
println("Scala Codegen result:")
println(s"Number of generated templates: ${supportedTemplateIds.size}")
println(
s"Number of not generated templates: ${util.mode.templateCount(interface) - supportedTemplateIds.size}")
s"Number of not generated templates: ${util.templateCount(interface) - supportedTemplateIds.size}")
println(s"Details: ${orderedDependencies.errors.map(_.msg).mkString("\n")}")
}

View File

@ -4,31 +4,24 @@
package com.digitalasset.codegen
import java.io.File
import scopt.OptionParser
import java.net.URL
import scopt.OptionParser
object Main {
case class Config(
inputFile: File = new File("."),
otherDalfInputs: Seq[URL] = Seq(),
inputFiles: Seq[File] = Seq(),
packageName: String = "",
outputDir: File = new File("."),
codeGenMode: CodeGen.Mode = CodeGen.Novel)
val parser = new OptionParser[Config]("codegen") {
private val parser = new OptionParser[Config]("codegen") {
help("help").text("prints this usage text")
opt[File]("input-file").required
opt[Seq[File]]("input-files").required
.abbr("i")
.action((d, c) => c.copy(inputFile = d))
.text("input top-level DAML module")
opt[Seq[File]]("dependencies").optional
.abbr("d")
.action((d, c) => c.copy(otherDalfInputs = d.map(toURL)))
.text("list of DAML module dependencies")
.action((d, c) => c.copy(inputFiles = d))
.text("input DAR or DALF files")
opt[String]("package-name")
.required()
@ -43,14 +36,11 @@ object Main {
.text("output directory for Scala files")
}
private def toURL(f: File): URL = f.toURI.toURL
def main(args: Array[String]): Unit = {
parser.parse(args, Config()) match {
case Some(config) =>
CodeGen.generateCode(
config.inputFile,
config.otherDalfInputs,
config.inputFiles.toList,
config.packageName,
config.outputDir,
config.codeGenMode)

View File

@ -34,8 +34,6 @@ abstract class Util(val packageName: String, val outputDir: File) { self =>
val iface: Interface
val mode: CodeGen.Mode { type Interface >: self.Interface }
val packageNameElems: Array[String] = packageName.split('.')
private[codegen] def orderedDependencies(library: Interface)
@ -125,6 +123,8 @@ abstract class Util(val packageName: String, val outputDir: File) { self =>
def genArgumentValueToGenType(genType: IType): Tree
def paramRefAndGenTypeToArgumentValue(paramRef: Tree, genType: IType): Tree
def templateCount(interface: Interface): Int
}
object Util {

View File

@ -5,12 +5,13 @@ package com.digitalasset.codegen
package lf
import com.digitalasset.daml.lf.data.Ref.Identifier
import com.digitalasset.daml.lf.iface
import com.digitalasset.daml.lf.{Dar, iface}
import iface.reader
import scala.collection.breakOut
import scala.collection.immutable.Map
import scalaz.syntax.std.map._
import scalaz.Semigroup
final case class EnvironmentInterface(typeDecls: Map[Identifier, reader.InterfaceType])
@ -20,4 +21,15 @@ object EnvironmentInterface {
case reader.Interface(packageId, typeDecls) =>
typeDecls mapKeys (Identifier(packageId, _))
}(breakOut))
def fromReaderInterfaces(dar: Dar[reader.Interface]): EnvironmentInterface =
fromReaderInterfaces(dar.main, dar.dependencies: _*)
val environmentInterfaceSemigroup = new Semigroup[EnvironmentInterface] {
override def append(
f1: EnvironmentInterface,
f2: => EnvironmentInterface): EnvironmentInterface = {
EnvironmentInterface(f1.typeDecls ++ f2.typeDecls)
}
}
}

View File

@ -11,14 +11,14 @@ import parent.exception.UnsupportedDamlTypeException
import com.digitalasset.daml.lf.iface
import iface.{
PrimType => PT,
PrimTypeOptional => PTOptional,
PrimTypeMap => PTMap,
PrimTypeOptional => PTOptional,
Type => IType,
_
}
import com.digitalasset.daml.lf.iface.reader.InterfaceType
import java.io.File
import scalaz._
import scalaz.std.set._
import scalaz.syntax.id._
@ -50,8 +50,6 @@ final case class LFUtil(
type TemplateInterface = DefTemplateWithRecord.FWT
override val mode = parent.CodeGen.Novel
private[codegen] override def orderedDependencies(library: Interface) =
DependencyGraph(this).orderedDependencies(library)
@ -257,6 +255,13 @@ final case class LFUtil(
$choiceMethod(${TermName(actorParamName)}, $dctorName(..$dargs))"""
}.toList
}
override def templateCount(interface: Interface): Int = {
interface.typeDecls.count {
case (_, InterfaceType.Template(_, _)) => true
case _ => false
}
}
}
object LFUtil {