fix(compiler): Streams in closures generates wrong AIR (#1055)

This commit is contained in:
Dima 2024-01-30 11:34:50 +03:00 committed by GitHub
parent 11f8d3b11b
commit 1a0b3a22c1
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 224 additions and 66 deletions

View File

@ -1,33 +1,92 @@
aqua M aqua A
export returnSrvAsAbility export get_logs
ability MyAb: service Op("op"):
call() -> string id(s1: string)
identity(s: string) -> string
service MySrv("default-id"): service MyOp("op"):
call() -> string id(s1: string)
identity(s: string) -> string
func mySrvDefault() -> MyAb: func get_logs(a: string):
<- MySrv if a == "sdf":
streamA <- Op.identity("some serv")
Op.id(streamA)
streamA: *string
streamA <- Op.identity("stream")
func mySrvResolved() -> MyAb: -- ability WorkerJob:
MySrv "resolved-id" -- runOnSingleWorker(w: string) -> []string
<- MySrv --
-- func runJob(j: -> []string) -> []string:
func mySrvThird() -> MyAb: -- <- j()
MySrv "third-id" --
<- MySrv -- func disjoint_run{WorkerJob}() -> -> []string:
-- run = func () -> []string:
func useMyAb{MyAb}() -> string: -- r <- WorkerJob.runOnSingleWorker("a")
<- MyAb.call() -- <- r
-- <- run
func returnSrvAsAbility() -> []string: --
result: *string -- func empty() -> string:
MySrvDefault <- mySrvDefault() -- a = "empty"
MySrvResolved <- mySrvResolved() -- <- a
MySrvThird <- mySrvThird() --
result <- useMyAb{MySrvDefault}() -- func lng317Bug() -> []string:
result <- useMyAb{MySrvResolved}() --
result <- useMyAb{MySrvThird}() -- res: *string
<- result --
-- outer = () -> string:
-- <- empty()
--
-- clos = () -> -> []string:
-- job2 = () -> []string:
-- res <- outer()
-- res <- MyOp.identity("identity")
-- <- res
-- <- job2
-- worker_job = WorkerJob(runOnSingleWorker = clos())
-- subnet_job <- disjoint_run{worker_job}()
-- finalRes <- runJob(subnet_job)
-- <- finalRes
--
-- ability Job:
-- run(s: string) -> string
--
-- func disrun(getJob: -> Job) -> Job:
-- j <- getJob()
-- <- j
--
-- func lng325Bug() -> string:
-- brokenStream: *string
--
-- job = () -> Job:
-- run = (str: string) -> string:
-- brokenStream <- MyOp.identity(str)
-- <- "run"
--
-- <- Job(run = run)
--
-- subnetJob <- disrun(job)
-- subnetJob.run("firstStream string")
-- <- brokenStream!
--
-- func secondStream() -> string:
-- brokenStream: *string
--
-- secondJob = () -> Job:
-- secondRun = (str: string) -> string:
-- brokenStream <- MyOp.identity(str)
-- <- "run"
--
-- <- Job(run = secondRun)
--
-- subnetJob <- disrun(secondJob)
-- subnetJob.run("secondStream string")
-- <- brokenStream!
--
-- func lng325BugTwoFuncs() -> string, string:
-- res1 <- lng325Bug()
-- res2 <- secondStream()
-- <- res1, res2

View File

@ -144,7 +144,7 @@ object Air {
showNext(inst) showNext(inst)
sb.append(space) sb.append(space)
case Air.Fold(iter, label, inst, lastInst) case Air.Fold(iter, label, inst, lastInst)
sb.append(" ").append(s" ${iter.show} $label\n") sb.append(s" ${iter.show} $label\n")
showNext(inst) showNext(inst)
showNext(lastInst) showNext(lastInst)
sb.append(space) sb.append(space)

View File

@ -1,6 +1,7 @@
aqua Closure declares * aqua Closure declares *
export LocalSrv, closureIn, closureOut, closureBig, closureOut2, lng58Bug, multipleClosuresBugLNG262, lng317Bug export LocalSrv, closureIn, closureOut, closureBig, closureOut2, lng58Bug, multipleClosuresBugLNG262, lng317Bug
export lng325Bug, lng325BugTwoFuncs
import "@fluencelabs/aqua-lib/builtin.aqua" import "@fluencelabs/aqua-lib/builtin.aqua"
@ -114,4 +115,44 @@ func lng317Bug() -> []string:
worker_job = WorkerJob(runOnSingleWorker = clos()) worker_job = WorkerJob(runOnSingleWorker = clos())
subnet_job <- disjoint_run{worker_job}() subnet_job <- disjoint_run{worker_job}()
finalRes <- runJob(subnet_job) finalRes <- runJob(subnet_job)
<- finalRes <- finalRes
ability Job:
run(s: string) -> string
func disrun(getJob: -> Job) -> Job:
j <- getJob()
<- j
func lng325Bug() -> string:
brokenStream: *string
job = () -> Job:
run = (str: string) -> string:
brokenStream <- MyOp.identity(str)
<- "run"
<- Job(run = run)
subnetJob <- disrun(job)
subnetJob.run("firstStream string")
<- brokenStream!
func secondStream() -> string:
brokenStream: *string
secondJob = () -> Job:
secondRun = (str: string) -> string:
brokenStream <- MyOp.identity(str)
<- "run"
<- Job(run = secondRun)
subnetJob <- disrun(secondJob)
subnetJob.run("secondStream string")
<- brokenStream!
func lng325BugTwoFuncs() -> string, string:
res1 <- lng325Bug()
res2 <- secondStream()
<- res1, res2

View File

@ -122,6 +122,8 @@ import {
closuresCall, closuresCall,
multipleClosuresLNG262BugCall, multipleClosuresLNG262BugCall,
lng317BugCall, lng317BugCall,
lng325BugCall,
lng325BugTwoFuncsCall
} from "../examples/closures.js"; } from "../examples/closures.js";
import { closureArrowCaptureCall } from "../examples/closureArrowCapture.js"; import { closureArrowCaptureCall } from "../examples/closureArrowCapture.js";
import { import {
@ -1106,6 +1108,16 @@ describe("Testing examples", () => {
expect(result).toEqual(["empty", "identity"]); expect(result).toEqual(["empty", "identity"]);
}); });
it("closures.aqua bug LNG-325", async () => {
let result = await lng325BugCall();
expect(result).toEqual("firstStream string");
});
it("closures.aqua bug LNG-325 two functions", async () => {
let result = await lng325BugTwoFuncsCall();
expect(result).toEqual(["firstStream string", "secondStream string"]);
});
it("closureArrowCapture.aqua", async () => { it("closureArrowCapture.aqua", async () => {
let result = await closureArrowCaptureCall("input"); let result = await closureArrowCaptureCall("input");
expect(result).toEqual("call: ".repeat(4) + "input"); expect(result).toEqual("call: ".repeat(4) + "input");

View File

@ -6,7 +6,9 @@ import {
closureOut2, closureOut2,
lng58Bug, lng58Bug,
lng317Bug, lng317Bug,
multipleClosuresBugLNG262 multipleClosuresBugLNG262,
lng325Bug,
lng325BugTwoFuncs
} from "../compiled/examples/closures.js"; } from "../compiled/examples/closures.js";
import { config } from "../config.js"; import { config } from "../config.js";
@ -42,3 +44,11 @@ export async function multipleClosuresLNG262BugCall(): Promise<[number, number]>
export async function lng317BugCall(): Promise<string[]> { export async function lng317BugCall(): Promise<string[]> {
return lng317Bug(); return lng317Bug();
} }
export async function lng325BugCall(): Promise<string> {
return lng325Bug();
}
export async function lng325BugTwoFuncsCall(): Promise<[string, string]> {
return lng325BugTwoFuncs();
}

View File

@ -113,20 +113,34 @@ object RawValueInliner extends Logging {
call: Call, call: Call,
flatStreamArguments: Boolean flatStreamArguments: Boolean
): State[S, (CallModel, Option[OpModel.Tree])] = { ): State[S, (CallModel, Option[OpModel.Tree])] = {
valueListToModel(call.args).flatMap { args => for {
if (flatStreamArguments) args <- valueListToModel(call.args)
args.map { arg => args <- {
TagInliner.flat(arg._1, arg._2) if (flatStreamArguments)
}.sequence args.traverse(TagInliner.flat.tupled)
else else
State.pure(args) State.pure(args)
}.map { list => }
exportTo <- call.exportTo.traverse {
case c@Call.Export(_, _, isExistingStream) if isExistingStream =>
// process streams, because they can be stored in Exports outside function/closure with different name
valueToModel(c.toRaw)
case ce =>
State.pure((VarModel(ce.name, ce.`type`), None))
}
} yield {
val (argsVars, argsOps) = args.unzip.map(_.flatten)
val (exportVars, exportOps) = exportTo.unzip.map(_.flatten)
val exportModel = exportVars.collect {
// exportTo can be only a variable
case VarModel(name, baseType, _) => CallModel.Export(name, baseType)
}
( (
CallModel( CallModel(
list.map(_._1), argsVars,
call.exportTo.map(CallModel.callExport) exportModel
), ),
parDesugarPrefix(list.flatMap(_._2)) parDesugarPrefix(exportOps ++ argsOps)
) )
} }
} }

View File

@ -403,12 +403,8 @@ object TagInliner extends Logging {
case DeclareStreamTag(value) => case DeclareStreamTag(value) =>
value match value match
case VarRaw(name, _) => case VarRaw(name, t) =>
for { Exports[S].resolved(name, VarModel(name, t)).as(TagInlined.Empty())
cd <- valueToModel(value)
(vm, prefix) = cd
_ <- Exports[S].resolved(name, vm)
} yield TagInlined.Empty(prefix = prefix)
case _ => none case _ => none
case ServiceIdTag(id, serviceType, name) => case ServiceIdTag(id, serviceType, name) =>

View File

@ -8,17 +8,17 @@ import aqua.model.inline.state.{Arrows, Exports, Mangler}
import aqua.model.inline.RawValueInliner.valueToModel import aqua.model.inline.RawValueInliner.valueToModel
import aqua.model.inline.TagInliner.canonicalizeIfStream import aqua.model.inline.TagInliner.canonicalizeIfStream
import aqua.model.inline.Inline.parDesugarPrefixOpt import aqua.model.inline.Inline.parDesugarPrefixOpt
import cats.data.{Chain, State}
import cats.data.Chain
import cats.syntax.flatMap.* import cats.syntax.flatMap.*
import cats.syntax.apply.* import cats.syntax.apply.*
import cats.Eval
final case class IfTagInliner( final case class IfTagInliner(
valueRaw: ValueRaw valueRaw: ValueRaw
) { ) {
import IfTagInliner.* import IfTagInliner.*
def inlined[S: Mangler: Exports: Arrows] = def inlined[S: Mangler: Exports: Arrows]: State[S, IfTagInlined] =
(valueRaw match { (valueRaw match {
// Optimize in case last operation is equality check // Optimize in case last operation is equality check
case ApplyBinaryOpRaw(op @ (BinOp.Eq | BinOp.Neq), left, right, _) => case ApplyBinaryOpRaw(op @ (BinOp.Eq | BinOp.Neq), left, right, _) =>

View File

@ -1,5 +1,6 @@
package aqua.raw.ops package aqua.raw.ops
import aqua.errors.Errors.internalError
import aqua.raw.value.{ValueRaw, VarRaw} import aqua.raw.value.{ValueRaw, VarRaw}
import aqua.types.{ArrowType, ProductType, Type} import aqua.types.{ArrowType, ProductType, Type}
@ -27,9 +28,26 @@ case class Call(args: List[ValueRaw], exportTo: List[Call.Export]) {
object Call { object Call {
// TODO docs // TODO docs
case class Export(name: String, `type`: Type) { case class Export(name: String, `type`: Type, isExistingStream: Boolean = false) {
def mapName(f: String => String): Export = copy(f(name)) def mapName(f: String => String): Export = copy(f(name))
def mapStream(f: ValueRaw => ValueRaw): Call.Export =
this match {
// map streams from "exportTo", because they are not exports, but variables
case ce @ Call.Export(_, _, true) =>
f(ce.toRaw) match {
case VarRaw(name, baseType) => Call.Export(name, baseType, true)
case _ => internalError(s"Stream '$ce' can be only VarRaw")
}
case ce => ce
}
def renameNonStream(map: Map[String, String]): Call.Export =
this match {
case ce @ Call.Export(_, _, true) => ce
case ce => ce.mapName(n => map.getOrElse(n, n))
}
def toRaw: VarRaw = VarRaw(name, `type`) def toRaw: VarRaw = VarRaw(name, `type`)
override def toString: String = s"$name:${`type`} <-" override def toString: String = s"$name:${`type`} <-"

View File

@ -1,11 +1,9 @@
package aqua.raw.ops package aqua.raw.ops
import aqua.raw.arrow.FuncRaw import aqua.raw.arrow.FuncRaw
import aqua.raw.ops.RawTag.Tree import aqua.raw.value.{CallArrowRaw, CallServiceRaw, ValueRaw, VarRaw}
import aqua.raw.value.{CallArrowRaw, CallServiceRaw, ValueRaw}
import aqua.tree.{TreeNode, TreeNodeCompanion} import aqua.tree.{TreeNode, TreeNodeCompanion}
import aqua.types.* import aqua.types.*
import cats.Show import cats.Show
import cats.data.{Chain, NonEmptyList} import cats.data.{Chain, NonEmptyList}
import cats.free.Cofree import cats.free.Cofree
@ -208,15 +206,20 @@ case class CallArrowRawTag(
value: ValueRaw value: ValueRaw
) extends RawTag { ) extends RawTag {
override def exportsVarNames: Set[String] = exportTo.map(_.name).toSet private lazy val usesExportStreams = exportTo.collect { case Call.Export(name, _, true) =>
name
}.toSet
override def usesVarNames: Set[String] = value.varNames // don't use existing streams in exports
override def exportsVarNames: Set[String] = exportTo.map(_.name).toSet -- usesExportStreams
override def usesVarNames: Set[String] = value.varNames ++ usesExportStreams
override def mapValues(f: ValueRaw => ValueRaw): RawTag = override def mapValues(f: ValueRaw => ValueRaw): RawTag =
CallArrowRawTag(exportTo, value.map(f)) CallArrowRawTag(exportTo.map(_.mapStream(f)), value.map(f))
override def renameExports(map: Map[String, String]): RawTag = override def renameExports(map: Map[String, String]): RawTag =
copy(exportTo = exportTo.map(_.mapName(n => map.getOrElse(n, n)))) copy(exportTo = exportTo.map(_.renameNonStream(map)))
} }
object CallArrowRawTag { object CallArrowRawTag {

View File

@ -3,8 +3,7 @@ package aqua.model
import aqua.raw.arrow.FuncRaw import aqua.raw.arrow.FuncRaw
import aqua.raw.ops.{Call, CallArrowRawTag, RawTag} import aqua.raw.ops.{Call, CallArrowRawTag, RawTag}
import aqua.raw.value.{ValueRaw, VarRaw} import aqua.raw.value.{ValueRaw, VarRaw}
import aqua.types.{ArrowType, Type} import aqua.types.{ArrowType, MutableStreamType, Type}
import cats.syntax.option.* import cats.syntax.option.*
case class FuncArrow( case class FuncArrow(
@ -73,7 +72,9 @@ object FuncArrow {
val call = Call( val call = Call(
methodType.domain.toLabelledList().map(VarRaw.apply), methodType.domain.toLabelledList().map(VarRaw.apply),
retVar.map(r => Call.Export(r.name, r.`type`)).toList retVar.map { r =>
Call.Export(r.name, r.`type`, Type.isStreamType(r.`type`))
}.toList
) )
val body = CallArrowRawTag.service( val body = CallArrowRawTag.service(

View File

@ -31,7 +31,7 @@ class CallArrowSem[S[_]](val expr: CallArrowExpr[S]) extends AnyVal {
(variables zip codomain.toList).traverse { case (v, t) => (variables zip codomain.toList).traverse { case (v, t) =>
N.read(v, mustBeDefined = false).flatMap { N.read(v, mustBeDefined = false).flatMap {
case Some(stream @ StreamType(st)) => case Some(stream @ StreamType(st)) =>
T.ensureTypeMatches(v, st, t).as(Call.Export(v.value, stream)) T.ensureTypeMatches(v, st, t).as(Call.Export(v.value, stream, isExistingStream = true))
case _ => case _ =>
N.define(v, t).as(Call.Export(v.value, t)) N.define(v, t).as(Call.Export(v.value, t))
} }

View File

@ -636,7 +636,7 @@ class SemanticsSpec extends AnyFlatSpec with Matchers with Inside {
stream.`type` shouldBe StreamType(ScalarType.i32) stream.`type` shouldBe StreamType(ScalarType.i32)
matchChildren(forTag) { case (ParTag, parTag) => matchChildren(forTag) { case (ParTag, parTag) =>
matchChildren(parTag)( matchChildren(parTag)(
{ case (PushToStreamTag(VarRaw(varName, _), Call.Export(streamName, _)), _) => { case (PushToStreamTag(VarRaw(varName, _), Call.Export(streamName, _, _)), _) =>
varName shouldBe "i" varName shouldBe "i"
streamName shouldBe "stream" streamName shouldBe "stream"
}, },

View File

@ -519,6 +519,12 @@ object Type {
* `StreamType` is collectible with canonicalization * `StreamType` is collectible with canonicalization
*/ */
type CollectibleType = DataType | StreamType type CollectibleType = DataType | StreamType
def isStreamType(t: Type): Boolean =
t match {
case _: MutableStreamType => true
case _ => false
}
given PartialOrder[Type] = given PartialOrder[Type] =
CompareTypes.partialOrder CompareTypes.partialOrder

View File

@ -2,8 +2,6 @@ package aqua.mangler
import cats.Monoid import cats.Monoid
import scala.annotation.tailrec
case class ManglerState(namesNumbers: Map[String, Int] = Map.empty) { case class ManglerState(namesNumbers: Map[String, Int] = Map.empty) {
private def genName(name: String, n: Int) = private def genName(name: String, n: Int) =