mirror of
https://github.com/fluencelabs/aqua.git
synced 2025-04-24 14:32:13 +00:00
fix(compiler): Use strict
to check types availability [LNG-334] (#1071)
This commit is contained in:
parent
f7194f0a54
commit
17fb77c40f
@ -1,92 +1,29 @@
|
|||||||
aqua A
|
aqua A
|
||||||
|
|
||||||
export get_logs
|
export main
|
||||||
|
|
||||||
service Op("op"):
|
alias SomeAlias: string
|
||||||
id(s1: string)
|
|
||||||
identity(s: string) -> string
|
|
||||||
|
|
||||||
service MyOp("op"):
|
data NestedStruct:
|
||||||
id(s1: string)
|
a: SomeAlias
|
||||||
identity(s: string) -> string
|
|
||||||
|
|
||||||
func get_logs(a: string):
|
data SomeStruct:
|
||||||
if a == "sdf":
|
al: SomeAlias
|
||||||
streamA <- Op.identity("some serv")
|
nested: NestedStruct
|
||||||
Op.id(streamA)
|
|
||||||
streamA: *string
|
|
||||||
streamA <- Op.identity("stream")
|
|
||||||
|
|
||||||
-- ability WorkerJob:
|
ability SomeAbility:
|
||||||
-- runOnSingleWorker(w: string) -> []string
|
someStr: SomeStruct
|
||||||
--
|
nested: NestedStruct
|
||||||
-- func runJob(j: -> []string) -> []string:
|
al: SomeAlias
|
||||||
-- <- j()
|
someFunc(ss: SomeStruct, nest: NestedStruct, al: SomeAlias) -> NestedStruct, SomeStruct, SomeAlias
|
||||||
--
|
|
||||||
-- func disjoint_run{WorkerJob}() -> -> []string:
|
service Srv("a"):
|
||||||
-- run = func () -> []string:
|
check(ss: SomeStruct, nest: NestedStruct, al: SomeAlias) -> NestedStruct
|
||||||
-- r <- WorkerJob.runOnSingleWorker("a")
|
check2() -> SomeStruct
|
||||||
-- <- r
|
check3() -> SomeAlias
|
||||||
-- <- run
|
|
||||||
--
|
func withAb{SomeAbility}() -> SomeStruct:
|
||||||
-- func empty() -> string:
|
<- SomeAbility.someStr
|
||||||
-- a = "empty"
|
|
||||||
-- <- a
|
func main(ss: SomeStruct, nest: NestedStruct, al: SomeAlias) -> string:
|
||||||
--
|
<- ""
|
||||||
-- func lng317Bug() -> []string:
|
|
||||||
--
|
|
||||||
-- res: *string
|
|
||||||
--
|
|
||||||
-- 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
|
|
@ -1,7 +1,15 @@
|
|||||||
module FooBars declares *
|
aqua FooBars declares *
|
||||||
|
|
||||||
const DECLARE_CONST = "declare_const"
|
const DECLARE_CONST = "declare_const"
|
||||||
const DECLARE_CONST2 = "declare_const2"
|
const DECLARE_CONST2 = "declare_const2"
|
||||||
|
|
||||||
|
-- alias SomeStruct: string
|
||||||
|
|
||||||
|
data SomeStruct:
|
||||||
|
a: string
|
||||||
|
|
||||||
|
-- service SomeStruct("ss"):
|
||||||
|
-- a() -> string
|
||||||
|
|
||||||
func foo() -> string:
|
func foo() -> string:
|
||||||
<- "I am MyFooBar foo"
|
<- "I am MyFooBar foo"
|
@ -85,7 +85,7 @@ class AquaLSPSpec extends AnyFlatSpec with Matchers with Inside {
|
|||||||
println(
|
println(
|
||||||
c.variables
|
c.variables
|
||||||
.map(_.definition)
|
.map(_.definition)
|
||||||
.filter(v => v.name == fullName.getOrElse(checkName) && v.`type` == `type`)
|
.filter(v => v.name == fullName.getOrElse(checkName))
|
||||||
.map { case DefinitionInfo(name, token, t) =>
|
.map { case DefinitionInfo(name, token, t) =>
|
||||||
val span = token.unit._1
|
val span = token.unit._1
|
||||||
s"$name(${span.startIndex}:${span.endIndex}) $t"
|
s"$name(${span.startIndex}:${span.endIndex}) $t"
|
||||||
@ -348,4 +348,110 @@ class AquaLSPSpec extends AnyFlatSpec with Matchers with Inside {
|
|||||||
ArrowType(NilType, ProductType(ScalarType.string :: Nil))
|
ArrowType(NilType, ProductType(ScalarType.string :: Nil))
|
||||||
) shouldBe true
|
) shouldBe true
|
||||||
}
|
}
|
||||||
|
|
||||||
|
it should "resolve type tokens in one file correctly" in {
|
||||||
|
val main =
|
||||||
|
"""
|
||||||
|
|aqua A declares withAb
|
||||||
|
|
|
||||||
|
|export main
|
||||||
|
|
|
||||||
|
|alias SomeAlias: string
|
||||||
|
|
|
||||||
|
|data NestedStruct:
|
||||||
|
| a: SomeAlias
|
||||||
|
|
|
||||||
|
|data SomeStruct:
|
||||||
|
| al: SomeAlias
|
||||||
|
| nested: NestedStruct
|
||||||
|
|
|
||||||
|
|ability SomeAbility:
|
||||||
|
| someStr: SomeStruct
|
||||||
|
| nested: NestedStruct
|
||||||
|
| al: SomeAlias
|
||||||
|
| someFunc(ss: SomeStruct, nest: NestedStruct, al: SomeAlias) -> NestedStruct, SomeStruct, SomeAlias
|
||||||
|
|
|
||||||
|
|service Srv("a"):
|
||||||
|
| check(ss: SomeStruct, nest: NestedStruct, al: SomeAlias) -> NestedStruct
|
||||||
|
| check2() -> SomeStruct
|
||||||
|
| check3() -> SomeAlias
|
||||||
|
|
|
||||||
|
|func withAb{SomeAbility}() -> SomeStruct:
|
||||||
|
| Srv.check()
|
||||||
|
| Srv.check2()
|
||||||
|
| <- SomeAbility.someStr
|
||||||
|
|
|
||||||
|
|func main(ss: SomeStruct, nest: NestedStruct, al: SomeAlias) -> string:
|
||||||
|
| Srv.check3()
|
||||||
|
| <- ""
|
||||||
|
|""".stripMargin
|
||||||
|
|
||||||
|
val src = Map(
|
||||||
|
"index.aqua" -> main
|
||||||
|
)
|
||||||
|
|
||||||
|
val res = compile(src, Map.empty).toOption.get.values.head
|
||||||
|
|
||||||
|
val nestedType = StructType("NestedStruct", NonEmptyMap.of(("a", ScalarType.string)))
|
||||||
|
val someStr =
|
||||||
|
StructType("SomeStruct", NonEmptyMap.of(("nested", nestedType), ("al", ScalarType.string)))
|
||||||
|
|
||||||
|
val abFuncType = ArrowType(
|
||||||
|
ProductType.labelled(
|
||||||
|
("ss", someStr) :: ("nest", nestedType) :: ("al", ScalarType.string) :: Nil
|
||||||
|
),
|
||||||
|
ProductType(nestedType :: someStr :: ScalarType.string :: Nil)
|
||||||
|
)
|
||||||
|
val someAb = AbilityType(
|
||||||
|
"SomeAbility",
|
||||||
|
NonEmptyMap.of(
|
||||||
|
("someStr", someStr),
|
||||||
|
("nested", nestedType),
|
||||||
|
("al", ScalarType.string),
|
||||||
|
("someFunc", abFuncType)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
val srvType = ServiceType(
|
||||||
|
"Srv",
|
||||||
|
NonEmptyMap.of(
|
||||||
|
(
|
||||||
|
"check",
|
||||||
|
ArrowType(
|
||||||
|
ProductType.labelled(
|
||||||
|
("ss", someStr) :: ("nest", nestedType) :: ("al", ScalarType.string) :: Nil
|
||||||
|
),
|
||||||
|
ProductType(nestedType :: Nil)
|
||||||
|
)
|
||||||
|
), ("check2", ArrowType(NilType, ProductType(someStr :: Nil))),
|
||||||
|
("check3", ArrowType(NilType, ProductType(ScalarType.string :: Nil)))
|
||||||
|
)
|
||||||
|
)
|
||||||
|
|
||||||
|
res.checkTokenLoc(main, "SomeAlias", 0, ScalarType.string) shouldBe true
|
||||||
|
Range.inclusive(1, 8).foreach { n =>
|
||||||
|
res.checkLocations("SomeAlias", 0, n, main) shouldBe true
|
||||||
|
}
|
||||||
|
|
||||||
|
res.checkTokenLoc(main, "NestedStruct", 0, nestedType) shouldBe true
|
||||||
|
Range.inclusive(1, 7).foreach { n =>
|
||||||
|
res.checkLocations("NestedStruct", 0, n, main) shouldBe true
|
||||||
|
}
|
||||||
|
|
||||||
|
res.checkTokenLoc(main, "SomeStruct", 0, someStr) shouldBe true
|
||||||
|
Range.inclusive(1, 7).foreach { n =>
|
||||||
|
res.checkLocations("SomeStruct", 0, n, main) shouldBe true
|
||||||
|
}
|
||||||
|
|
||||||
|
res.checkTokenLoc(main, "SomeAbility", 0, someAb) shouldBe true
|
||||||
|
// from {SomeAbility} to 'ability SomeAbility'
|
||||||
|
res.checkLocations("SomeAbility", 0, 1, main) shouldBe true
|
||||||
|
// from 'SomeAbility.someStr' to {SomeAbility}
|
||||||
|
res.checkLocations("SomeAbility", 1, 2, main) shouldBe true
|
||||||
|
|
||||||
|
res.checkTokenLoc(main, "Srv", 0, srvType) shouldBe true
|
||||||
|
Range.inclusive(1, 3).foreach { n =>
|
||||||
|
res.checkLocations("Srv", 0, n, main) shouldBe true
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -1,11 +1,10 @@
|
|||||||
package aqua.parser.expr.func
|
package aqua.parser.expr.func
|
||||||
|
|
||||||
import aqua.parser.lexer.{ArrowTypeToken, BasicTypeToken, TypeToken, ValueToken}
|
import aqua.parser.lexer.{ArrowTypeToken, BasicTypeToken, NamedTypeToken, TypeToken, ValueToken}
|
||||||
import aqua.parser.lift.LiftParser
|
import aqua.parser.lift.LiftParser
|
||||||
import aqua.parser.lift.Span
|
import aqua.parser.lift.Span
|
||||||
import aqua.parser.lift.Span.{P0ToSpan, PToSpan}
|
import aqua.parser.lift.Span.{P0ToSpan, PToSpan}
|
||||||
import aqua.parser.{ArrowReturnError, Ast, Expr, ParserError}
|
import aqua.parser.{ArrowReturnError, Ast, Expr, ParserError}
|
||||||
|
|
||||||
import cats.Comonad
|
import cats.Comonad
|
||||||
import cats.parse.Parser
|
import cats.parse.Parser
|
||||||
import cats.~>
|
import cats.~>
|
||||||
@ -15,7 +14,6 @@ case class ArrowExpr[F[_]](arrowTypeExpr: ArrowTypeToken[F])
|
|||||||
|
|
||||||
override def mapK[K[_]: Comonad](fk: F ~> K): ArrowExpr[K] =
|
override def mapK[K[_]: Comonad](fk: F ~> K): ArrowExpr[K] =
|
||||||
copy(arrowTypeExpr.mapK(fk))
|
copy(arrowTypeExpr.mapK(fk))
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
object ArrowExpr extends Expr.AndIndented {
|
object ArrowExpr extends Expr.AndIndented {
|
||||||
|
@ -110,7 +110,8 @@ object ScalarTypeToken {
|
|||||||
case class ArrowTypeToken[S[_]: Comonad](
|
case class ArrowTypeToken[S[_]: Comonad](
|
||||||
override val unit: S[Unit],
|
override val unit: S[Unit],
|
||||||
args: List[(Option[Name[S]], TypeToken[S])],
|
args: List[(Option[Name[S]], TypeToken[S])],
|
||||||
res: List[TypeToken[S]]
|
res: List[TypeToken[S]],
|
||||||
|
abilities: List[NamedTypeToken[S]] = Nil
|
||||||
) extends TypeToken[S] {
|
) extends TypeToken[S] {
|
||||||
override def as[T](v: T): S[T] = unit.as(v)
|
override def as[T](v: T): S[T] = unit.as(v)
|
||||||
|
|
||||||
@ -118,9 +119,11 @@ case class ArrowTypeToken[S[_]: Comonad](
|
|||||||
copy(
|
copy(
|
||||||
fk(unit),
|
fk(unit),
|
||||||
args.map { case (n, t) => (n.map(_.mapK(fk)), t.mapK(fk)) },
|
args.map { case (n, t) => (n.map(_.mapK(fk)), t.mapK(fk)) },
|
||||||
res.map(_.mapK(fk))
|
res.map(_.mapK(fk)),
|
||||||
)
|
abilities.map(_.mapK(fk))
|
||||||
def argTypes: List[TypeToken[S]] = args.map(_._2)
|
)
|
||||||
|
def argTypes: List[TypeToken[S]] = abilities ++ args.map(_._2)
|
||||||
|
lazy val absWithArgs: List[(Option[Name[S]], TypeToken[S])] = abilities.map(n => Some(n.asName) -> n) ++ args
|
||||||
}
|
}
|
||||||
|
|
||||||
object ArrowTypeToken {
|
object ArrowTypeToken {
|
||||||
@ -133,8 +136,8 @@ object ArrowTypeToken {
|
|||||||
).map(_.toList)
|
).map(_.toList)
|
||||||
|
|
||||||
// {SomeAb, SecondAb} for NamedTypeToken
|
// {SomeAb, SecondAb} for NamedTypeToken
|
||||||
def abilities(): P0[List[(Option[Name[S]], NamedTypeToken[S])]] =
|
def abilities(): P0[List[NamedTypeToken[S]]] =
|
||||||
(`{` *> comma(`Class`.surroundedBy(`/s*`).lift.map(s => Option(Name(s)) -> NamedTypeToken(s)))
|
(`{` *> comma(`Class`.surroundedBy(`/s*`).lift.map(NamedTypeToken(_)))
|
||||||
.map(_.toList) <* `}`).?.map(_.getOrElse(List.empty))
|
.map(_.toList) <* `}`).?.map(_.getOrElse(List.empty))
|
||||||
|
|
||||||
def `arrowdef`(argTypeP: P[TypeToken[Span.S]]): P[ArrowTypeToken[Span.S]] =
|
def `arrowdef`(argTypeP: P[TypeToken[Span.S]]): P[ArrowTypeToken[Span.S]] =
|
||||||
@ -144,8 +147,9 @@ object ArrowTypeToken {
|
|||||||
val args = argsList.map(Option.empty[Name[Span.S]] -> _)
|
val args = argsList.map(Option.empty[Name[Span.S]] -> _)
|
||||||
ArrowTypeToken(
|
ArrowTypeToken(
|
||||||
point,
|
point,
|
||||||
abs ++ args,
|
args,
|
||||||
res
|
res,
|
||||||
|
abs
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -155,7 +159,7 @@ object ArrowTypeToken {
|
|||||||
.surroundedBy(`/s*`)
|
.surroundedBy(`/s*`)
|
||||||
) <* (`/s*` *> `)` <* ` `.?)) ~
|
) <* (`/s*` *> `)` <* ` `.?)) ~
|
||||||
(` -> ` *> returnDef()).?).map { case (((abilities, point), args), res) =>
|
(` -> ` *> returnDef()).?).map { case (((abilities, point), args), res) =>
|
||||||
ArrowTypeToken(point, abilities ++ args, res.toList.flatMap(_.toList))
|
ArrowTypeToken(point, args, res.toList.flatMap(_.toList), abilities)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -75,15 +75,17 @@ object AquaSpec {
|
|||||||
|
|
||||||
def toArrowType(
|
def toArrowType(
|
||||||
args: List[BasicTypeToken[Id]],
|
args: List[BasicTypeToken[Id]],
|
||||||
res: Option[BasicTypeToken[Id]]
|
res: Option[BasicTypeToken[Id]],
|
||||||
|
abilities: List[NamedTypeToken[Id]] = Nil
|
||||||
): ArrowTypeToken[Id] =
|
): ArrowTypeToken[Id] =
|
||||||
ArrowTypeToken[Id]((), args.map(None -> _), res.toList)
|
ArrowTypeToken[Id]((), args.map(None -> _), res.toList, abilities)
|
||||||
|
|
||||||
def toNamedArrow(
|
def toNamedArrow(
|
||||||
args: List[(String, TypeToken[Id])],
|
args: List[(String, TypeToken[Id])],
|
||||||
res: List[BasicTypeToken[Id]]
|
res: List[BasicTypeToken[Id]],
|
||||||
|
abilities: List[NamedTypeToken[Id]] = Nil
|
||||||
): ArrowTypeToken[Id] =
|
): ArrowTypeToken[Id] =
|
||||||
ArrowTypeToken[Id]((), args.map(ab => Some(Name[Id](ab._1)) -> ab._2), res)
|
ArrowTypeToken[Id]((), args.map(ab => Some(Name[Id](ab._1)) -> ab._2), res, abilities)
|
||||||
|
|
||||||
def toNamedArg(str: String, customType: String): Arg[Id] =
|
def toNamedArg(str: String, customType: String): Arg[Id] =
|
||||||
Arg[Id](str, toNamedType(customType))
|
Arg[Id](str, toNamedType(customType))
|
||||||
|
@ -33,11 +33,11 @@ class ArrowTypeExprSpec extends AnyFlatSpec with Matchers with AquaSpec {
|
|||||||
"onIn",
|
"onIn",
|
||||||
toNamedArrow(
|
toNamedArrow(
|
||||||
List(
|
List(
|
||||||
"SomeAb" -> toNamedType("SomeAb"),
|
|
||||||
"a" -> toNamedType("Custom"),
|
"a" -> toNamedType("Custom"),
|
||||||
"b" -> toNamedType("Custom2")
|
"b" -> toNamedType("Custom2")
|
||||||
),
|
),
|
||||||
Nil
|
Nil,
|
||||||
|
toNamedType("SomeAb") :: Nil
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
@ -165,14 +165,11 @@ class TypeTokenSpec extends AnyFlatSpec with Matchers with EitherValues {
|
|||||||
arrowWithNames("{SomeAb, SecondAb}(a: A) -> B") should be(
|
arrowWithNames("{SomeAb, SecondAb}(a: A) -> B") should be(
|
||||||
ArrowTypeToken[Id](
|
ArrowTypeToken[Id](
|
||||||
(),
|
(),
|
||||||
(Some(Name[Id]("SomeAb")) -> NamedTypeToken[Id]("SomeAb")) :: (Some(
|
(
|
||||||
Name[Id](
|
|
||||||
"SecondAb"
|
|
||||||
)
|
|
||||||
) -> NamedTypeToken[Id]("SecondAb")) :: (
|
|
||||||
Some(Name[Id]("a")) -> NamedTypeToken[Id]("A")
|
Some(Name[Id]("a")) -> NamedTypeToken[Id]("A")
|
||||||
) :: Nil,
|
) :: Nil,
|
||||||
List(NamedTypeToken[Id]("B"))
|
List(NamedTypeToken[Id]("B")),
|
||||||
|
NamedTypeToken[Id]("SomeAb") :: NamedTypeToken[Id]("SecondAb") :: Nil
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -29,11 +29,15 @@ class ArrowSem[S[_]](val expr: ArrowExpr[S]) extends AnyVal {
|
|||||||
|
|
||||||
def before[Alg[_]: Monad](implicit
|
def before[Alg[_]: Monad](implicit
|
||||||
T: TypesAlgebra[S, Alg],
|
T: TypesAlgebra[S, Alg],
|
||||||
N: NamesAlgebra[S, Alg]
|
N: NamesAlgebra[S, Alg],
|
||||||
|
L: LocationsAlgebra[S, Alg]
|
||||||
): Alg[ArrowType] = for {
|
): Alg[ArrowType] = for {
|
||||||
arrowType <- T.beginArrowScope(arrowTypeExpr)
|
arrowType <- T.beginArrowScope(arrowTypeExpr)
|
||||||
|
// add locations before ability will be defined as new variable definition
|
||||||
|
_ <- L.pointLocations(arrowTypeExpr.abilities.map(n => n.value -> n))
|
||||||
|
absAsArgs = arrowTypeExpr.abilities.map(_.asName)
|
||||||
// Create local variables
|
// Create local variables
|
||||||
_ <- expr.arrowTypeExpr.args.flatMap { case (name, _) => name }
|
_ <- arrowTypeExpr.absWithArgs.flatMap { case (name, _) => name }
|
||||||
.zip(arrowType.domain.toList)
|
.zip(arrowType.domain.toList)
|
||||||
.traverse {
|
.traverse {
|
||||||
case (argName, t: ArrowType) =>
|
case (argName, t: ArrowType) =>
|
||||||
@ -137,7 +141,8 @@ class ArrowSem[S[_]](val expr: ArrowExpr[S]) extends AnyVal {
|
|||||||
T: TypesAlgebra[S, Alg],
|
T: TypesAlgebra[S, Alg],
|
||||||
N: NamesAlgebra[S, Alg],
|
N: NamesAlgebra[S, Alg],
|
||||||
A: AbilitiesAlgebra[S, Alg],
|
A: AbilitiesAlgebra[S, Alg],
|
||||||
M: ManglerAlgebra[Alg]
|
M: ManglerAlgebra[Alg],
|
||||||
|
L: LocationsAlgebra[S, Alg]
|
||||||
): Prog[Alg, Raw] =
|
): Prog[Alg, Raw] =
|
||||||
Prog
|
Prog
|
||||||
.around(
|
.around(
|
||||||
|
@ -1,10 +1,8 @@
|
|||||||
package aqua.semantics.expr.func
|
package aqua.semantics.expr.func
|
||||||
|
|
||||||
import aqua.parser.expr.func.CallArrowExpr
|
import aqua.parser.expr.func.CallArrowExpr
|
||||||
import aqua.parser.lexer.{CallArrowToken, IntoArrow, IntoField, PropertyToken, VarToken}
|
|
||||||
import aqua.raw.Raw
|
import aqua.raw.Raw
|
||||||
import aqua.raw.ops.{Call, CallArrowRawTag, FuncOp}
|
import aqua.raw.ops.{Call, CallArrowRawTag, FuncOp}
|
||||||
import aqua.raw.value.CallArrowRaw
|
|
||||||
import aqua.semantics.Prog
|
import aqua.semantics.Prog
|
||||||
import aqua.semantics.rules.ValuesAlgebra
|
import aqua.semantics.rules.ValuesAlgebra
|
||||||
import aqua.semantics.rules.names.NamesAlgebra
|
import aqua.semantics.rules.names.NamesAlgebra
|
||||||
@ -12,12 +10,9 @@ import aqua.semantics.rules.types.TypesAlgebra
|
|||||||
import aqua.types.{ProductType, StreamType, Type}
|
import aqua.types.{ProductType, StreamType, Type}
|
||||||
|
|
||||||
import cats.Monad
|
import cats.Monad
|
||||||
import cats.syntax.applicative.*
|
|
||||||
import cats.syntax.apply.*
|
import cats.syntax.apply.*
|
||||||
import cats.syntax.comonad.*
|
|
||||||
import cats.syntax.flatMap.*
|
import cats.syntax.flatMap.*
|
||||||
import cats.syntax.functor.*
|
import cats.syntax.functor.*
|
||||||
import cats.syntax.option.*
|
|
||||||
import cats.syntax.traverse.*
|
import cats.syntax.traverse.*
|
||||||
|
|
||||||
class CallArrowSem[S[_]](val expr: CallArrowExpr[S]) extends AnyVal {
|
class CallArrowSem[S[_]](val expr: CallArrowExpr[S]) extends AnyVal {
|
||||||
|
@ -99,7 +99,7 @@ class ValuesAlgebra[S[_], Alg[_]: Monad](using
|
|||||||
case None =>
|
case None =>
|
||||||
(for {
|
(for {
|
||||||
t <- OptionT(
|
t <- OptionT(
|
||||||
T.getType(name.value)
|
T.resolveType(name.asTypeToken, mustBeDefined = false)
|
||||||
).collect { case st: ServiceType => st }
|
).collect { case st: ServiceType => st }
|
||||||
// A hack to report name error, better to refactor
|
// A hack to report name error, better to refactor
|
||||||
.flatTapNone(N.read(name))
|
.flatTapNone(N.read(name))
|
||||||
@ -429,7 +429,7 @@ class ValuesAlgebra[S[_], Alg[_]: Monad](using
|
|||||||
|
|
||||||
lazy val nameTypeFromService = for {
|
lazy val nameTypeFromService = for {
|
||||||
st <- OptionT(
|
st <- OptionT(
|
||||||
T.getType(ab.value)
|
T.resolveType(ab, mustBeDefined = false)
|
||||||
).collect { case st: ServiceType => st }
|
).collect { case st: ServiceType => st }
|
||||||
rename <- OptionT(
|
rename <- OptionT(
|
||||||
A.getServiceRename(ab)
|
A.getServiceRename(ab)
|
||||||
|
@ -4,15 +4,17 @@ import aqua.parser.lexer.{Name, NamedTypeToken, Token}
|
|||||||
import aqua.raw.RawContext
|
import aqua.raw.RawContext
|
||||||
import aqua.raw.value.ValueRaw
|
import aqua.raw.value.ValueRaw
|
||||||
import aqua.semantics.Levenshtein
|
import aqua.semantics.Levenshtein
|
||||||
import aqua.semantics.rules.locations.{DefinitionInfo, LocationsAlgebra}
|
import aqua.semantics.rules.locations.LocationsAlgebra
|
||||||
import aqua.semantics.rules.mangler.ManglerAlgebra
|
import aqua.semantics.rules.mangler.ManglerAlgebra
|
||||||
import aqua.semantics.rules.report.ReportAlgebra
|
import aqua.semantics.rules.report.ReportAlgebra
|
||||||
import aqua.semantics.rules.{abilities, StackInterpreter}
|
import aqua.semantics.rules.{StackInterpreter, abilities}
|
||||||
import aqua.types.ArrowType
|
import aqua.types.ArrowType
|
||||||
import cats.data.{NonEmptyMap, State}
|
|
||||||
|
import cats.data.*
|
||||||
import cats.syntax.applicative.*
|
import cats.syntax.applicative.*
|
||||||
import cats.syntax.apply.*
|
import cats.syntax.apply.*
|
||||||
import cats.syntax.functor.*
|
import cats.syntax.functor.*
|
||||||
|
import cats.syntax.flatMap.*
|
||||||
import cats.syntax.option.*
|
import cats.syntax.option.*
|
||||||
import monocle.Lens
|
import monocle.Lens
|
||||||
import monocle.macros.GenLens
|
import monocle.macros.GenLens
|
||||||
@ -62,7 +64,9 @@ class AbilitiesInterpreter[S[_], X](using
|
|||||||
}
|
}
|
||||||
|
|
||||||
override def isDefinedAbility(name: NamedTypeToken[S]): State[X, Boolean] =
|
override def isDefinedAbility(name: NamedTypeToken[S]): State[X, Boolean] =
|
||||||
getState.map(_.abilities.contains(name.value))
|
OptionT(getState.map(_.abilities.get(name.value))).semiflatTap { _ =>
|
||||||
|
locations.pointLocation(name.value, name)
|
||||||
|
}.isDefined
|
||||||
|
|
||||||
override def getArrow(name: NamedTypeToken[S], arrow: Name[S]): SX[Option[ArrowType]] =
|
override def getArrow(name: NamedTypeToken[S], arrow: Name[S]): SX[Option[ArrowType]] =
|
||||||
getAbility(name.value).flatMap {
|
getAbility(name.value).flatMap {
|
||||||
|
@ -14,7 +14,7 @@ import cats.syntax.validated.*
|
|||||||
|
|
||||||
final case class TypeResolution[S[_], +T](
|
final case class TypeResolution[S[_], +T](
|
||||||
`type`: T,
|
`type`: T,
|
||||||
definitions: List[(Token[S], NamedTypeToken[S])]
|
occurrences: List[(Token[S], String)]
|
||||||
)
|
)
|
||||||
|
|
||||||
object TypeResolution {
|
object TypeResolution {
|
||||||
@ -59,10 +59,7 @@ object TypeResolution {
|
|||||||
case OptionTypeToken(_, dtt) =>
|
case OptionTypeToken(_, dtt) =>
|
||||||
resolveCollection(dtt, "Option", OptionType.apply)(state)
|
resolveCollection(dtt, "Option", OptionType.apply)(state)
|
||||||
case ntt: NamedTypeToken[S] =>
|
case ntt: NamedTypeToken[S] =>
|
||||||
val defs = state
|
val defs = (ntt -> ntt.value) :: Nil
|
||||||
.getTypeDefinition(ntt.value)
|
|
||||||
.toList
|
|
||||||
.map(ntt -> _)
|
|
||||||
|
|
||||||
state
|
state
|
||||||
.getType(ntt.value)
|
.getType(ntt.value)
|
||||||
@ -84,7 +81,7 @@ object TypeResolution {
|
|||||||
)(state: TypesState[S]): Res[S, ArrowType] = {
|
)(state: TypesState[S]): Res[S, ArrowType] = {
|
||||||
val res = arrowTypeToken.res
|
val res = arrowTypeToken.res
|
||||||
.traverse(typeToken => resolveTypeToken(typeToken)(state).toEither)
|
.traverse(typeToken => resolveTypeToken(typeToken)(state).toEither)
|
||||||
val args = arrowTypeToken.args.traverse { case (argName, typeToken) =>
|
val args = arrowTypeToken.absWithArgs.traverse { case (argName, typeToken) =>
|
||||||
resolveTypeToken(typeToken)(state)
|
resolveTypeToken(typeToken)(state)
|
||||||
.map(argName.map(_.value) -> _)
|
.map(argName.map(_.value) -> _)
|
||||||
.toEither
|
.toEither
|
||||||
@ -104,7 +101,7 @@ object TypeResolution {
|
|||||||
ProductType.maybeLabelled(argsLabeledTypes),
|
ProductType.maybeLabelled(argsLabeledTypes),
|
||||||
ProductType(resTypes)
|
ProductType(resTypes)
|
||||||
)
|
)
|
||||||
val defs = (argsTokens ++ resTokens)
|
val defs = argsTokens ++ resTokens
|
||||||
|
|
||||||
TypeResolution(typ, defs)
|
TypeResolution(typ, defs)
|
||||||
}.toValidated
|
}.toValidated
|
||||||
|
@ -11,14 +11,12 @@ import cats.data.OptionT
|
|||||||
|
|
||||||
trait TypesAlgebra[S[_], Alg[_]] {
|
trait TypesAlgebra[S[_], Alg[_]] {
|
||||||
|
|
||||||
def resolveType(token: TypeToken[S]): Alg[Option[Type]]
|
def resolveType(token: TypeToken[S], mustBeDefined: Boolean = true): Alg[Option[Type]]
|
||||||
|
|
||||||
def resolveStreamType(token: TypeToken[S]): Alg[Option[StreamType]]
|
def resolveStreamType(token: TypeToken[S]): Alg[Option[StreamType]]
|
||||||
|
|
||||||
def resolveNamedType(token: TypeToken[S]): Alg[Option[AbilityType | StructType]]
|
def resolveNamedType(token: TypeToken[S]): Alg[Option[AbilityType | StructType]]
|
||||||
|
|
||||||
def getType(name: String): Alg[Option[Type]]
|
|
||||||
|
|
||||||
def resolveArrowDef(arrowDef: ArrowTypeToken[S]): Alg[Option[ArrowType]]
|
def resolveArrowDef(arrowDef: ArrowTypeToken[S]): Alg[Option[ArrowType]]
|
||||||
|
|
||||||
def resolveServiceType(name: NamedTypeToken[S]): Alg[Option[ServiceType]]
|
def resolveServiceType(name: NamedTypeToken[S]): Alg[Option[ServiceType]]
|
||||||
|
@ -11,6 +11,7 @@ import aqua.semantics.rules.types.TypeResolution.TypeResolutionError
|
|||||||
import aqua.types.*
|
import aqua.types.*
|
||||||
import aqua.types.Type.*
|
import aqua.types.Type.*
|
||||||
|
|
||||||
|
import cats.Applicative
|
||||||
import cats.data.*
|
import cats.data.*
|
||||||
import cats.data.Validated.{Invalid, Valid}
|
import cats.data.Validated.{Invalid, Valid}
|
||||||
import cats.syntax.applicative.*
|
import cats.syntax.applicative.*
|
||||||
@ -18,12 +19,11 @@ import cats.syntax.apply.*
|
|||||||
import cats.syntax.flatMap.*
|
import cats.syntax.flatMap.*
|
||||||
import cats.syntax.foldable.*
|
import cats.syntax.foldable.*
|
||||||
import cats.syntax.functor.*
|
import cats.syntax.functor.*
|
||||||
import cats.syntax.monad.*
|
|
||||||
import cats.syntax.option.*
|
import cats.syntax.option.*
|
||||||
import cats.syntax.traverse.*
|
import cats.syntax.traverse.*
|
||||||
import cats.{Applicative, ~>}
|
|
||||||
import monocle.Lens
|
import monocle.Lens
|
||||||
import monocle.macros.GenLens
|
import monocle.macros.GenLens
|
||||||
|
import scala.annotation.tailrec
|
||||||
import scala.collection.immutable.SortedMap
|
import scala.collection.immutable.SortedMap
|
||||||
import scala.reflect.TypeTest
|
import scala.reflect.TypeTest
|
||||||
|
|
||||||
@ -41,18 +41,16 @@ class TypesInterpreter[S[_], X](using
|
|||||||
|
|
||||||
type ST[A] = State[X, A]
|
type ST[A] = State[X, A]
|
||||||
|
|
||||||
override def getType(name: String): State[X, Option[Type]] =
|
override def resolveType(token: TypeToken[S], mustBeDefined: Boolean = true): State[X, Option[Type]] =
|
||||||
getState.map(st => st.strict.get(name))
|
|
||||||
|
|
||||||
override def resolveType(token: TypeToken[S]): State[X, Option[Type]] =
|
|
||||||
getState.map(TypeResolution.resolveTypeToken(token)).flatMap {
|
getState.map(TypeResolution.resolveTypeToken(token)).flatMap {
|
||||||
case Valid(TypeResolution(typ, tokens)) =>
|
case Valid(TypeResolution(typ, tokens)) =>
|
||||||
val tokensLocs = tokens.map { case (t, n) => n.value -> t }
|
val tokensLocs = tokens.map { case (t, n) => n -> t }
|
||||||
locations.pointLocations(tokensLocs).as(typ.some)
|
locations.pointLocations(tokensLocs).as(typ.some)
|
||||||
case Invalid(errors) =>
|
case Invalid(errors) if mustBeDefined =>
|
||||||
errors.traverse_ { case TypeResolutionError(token, hint) =>
|
errors.traverse_ { case TypeResolutionError(token, hint) =>
|
||||||
report.error(token, hint)
|
report.error(token, hint)
|
||||||
}.as(none)
|
}.as(none)
|
||||||
|
case _ => none.pure
|
||||||
}
|
}
|
||||||
|
|
||||||
override def resolveStreamType(token: TypeToken[S]): State[X, Option[StreamType]] =
|
override def resolveStreamType(token: TypeToken[S]): State[X, Option[StreamType]] =
|
||||||
@ -70,7 +68,7 @@ class TypesInterpreter[S[_], X](using
|
|||||||
override def resolveArrowDef(arrowDef: ArrowTypeToken[S]): State[X, Option[ArrowType]] =
|
override def resolveArrowDef(arrowDef: ArrowTypeToken[S]): State[X, Option[ArrowType]] =
|
||||||
getState.map(TypeResolution.resolveArrowDef(arrowDef)).flatMap {
|
getState.map(TypeResolution.resolveArrowDef(arrowDef)).flatMap {
|
||||||
case Valid(TypeResolution(tt, tokens)) =>
|
case Valid(TypeResolution(tt, tokens)) =>
|
||||||
val tokensLocs = tokens.map { case (t, n) => n.value -> t }
|
val tokensLocs = tokens.map { case (t, n) => n -> t }
|
||||||
locations.pointLocations(tokensLocs).as(tt.some)
|
locations.pointLocations(tokensLocs).as(tt.some)
|
||||||
case Invalid(errs) =>
|
case Invalid(errs) =>
|
||||||
errs.traverse_ { case TypeResolutionError(token, hint) =>
|
errs.traverse_ { case TypeResolutionError(token, hint) =>
|
||||||
@ -155,7 +153,7 @@ class TypesInterpreter[S[_], X](using
|
|||||||
): State[X, Option[StructType]] =
|
): State[X, Option[StructType]] =
|
||||||
ensureNameNotDefined(name.value, name, ifDefined = none)(
|
ensureNameNotDefined(name.value, name, ifDefined = none)(
|
||||||
fields.toList.traverse {
|
fields.toList.traverse {
|
||||||
case (field, (fieldName, t: DataType)) =>
|
case (field, (_, t: DataType)) =>
|
||||||
(field -> t).some.pure[ST]
|
(field -> t).some.pure[ST]
|
||||||
case (field, (fieldName, t)) =>
|
case (field, (fieldName, t)) =>
|
||||||
report
|
report
|
||||||
@ -180,8 +178,7 @@ class TypesInterpreter[S[_], X](using
|
|||||||
)
|
)
|
||||||
|
|
||||||
override def defineAlias(name: NamedTypeToken[S], target: Type): State[X, Boolean] =
|
override def defineAlias(name: NamedTypeToken[S], target: Type): State[X, Boolean] =
|
||||||
getState.map(_.definitions.get(name.value)).flatMap {
|
getState.map(_.getType(name.value)).flatMap {
|
||||||
case Some(n) if n == name => State.pure(false)
|
|
||||||
case Some(_) => report.error(name, s"Type `${name.value}` was already defined").as(false)
|
case Some(_) => report.error(name, s"Type `${name.value}` was already defined").as(false)
|
||||||
case None =>
|
case None =>
|
||||||
modify(_.defineType(name, target))
|
modify(_.defineType(name, target))
|
||||||
@ -269,7 +266,7 @@ class TypesInterpreter[S[_], X](using
|
|||||||
ensureTypeMatches(arg, expectedType, argType)
|
ensureTypeMatches(arg, expectedType, argType)
|
||||||
}
|
}
|
||||||
|
|
||||||
locations.pointFieldLocation(abName, opName, op) *>
|
locations.pointFieldLocation(ab.name, opName, op) *>
|
||||||
reportNotEnoughArguments *>
|
reportNotEnoughArguments *>
|
||||||
reportTooManyArguments *>
|
reportTooManyArguments *>
|
||||||
checkArgumentTypes.map(typesMatch =>
|
checkArgumentTypes.map(typesMatch =>
|
||||||
@ -367,6 +364,7 @@ class TypesInterpreter[S[_], X](using
|
|||||||
right: Type
|
right: Type
|
||||||
): State[X, Boolean] = {
|
): State[X, Boolean] = {
|
||||||
// TODO: This needs more comprehensive logic
|
// TODO: This needs more comprehensive logic
|
||||||
|
@tailrec
|
||||||
def isComparable(lt: Type, rt: Type): Boolean =
|
def isComparable(lt: Type, rt: Type): Boolean =
|
||||||
(lt, rt) match {
|
(lt, rt) match {
|
||||||
// All numbers are comparable
|
// All numbers are comparable
|
||||||
@ -485,7 +483,7 @@ class TypesInterpreter[S[_], X](using
|
|||||||
}
|
}
|
||||||
expectedKeys = expected.fields.keys.toNonEmptyList
|
expectedKeys = expected.fields.keys.toNonEmptyList
|
||||||
/* Report unexpected arguments */
|
/* Report unexpected arguments */
|
||||||
_ <- arguments.toNel.traverse_ { case (name, arg -> typ) =>
|
_ <- arguments.toNel.traverse_ { case (name, arg -> _) =>
|
||||||
expected.fields.lookup(name) match {
|
expected.fields.lookup(name) match {
|
||||||
case Some(_) => State.pure(())
|
case Some(_) => State.pure(())
|
||||||
case None =>
|
case None =>
|
||||||
@ -605,7 +603,7 @@ class TypesInterpreter[S[_], X](using
|
|||||||
report
|
report
|
||||||
.error(
|
.error(
|
||||||
token,
|
token,
|
||||||
s"Number of arguments doesn't match the function type, expected: ${expected}, given: $givenNum"
|
s"Number of arguments doesn't match the function type, expected: $expected, given: $givenNum"
|
||||||
)
|
)
|
||||||
.as(false)
|
.as(false)
|
||||||
|
|
||||||
@ -613,7 +611,7 @@ class TypesInterpreter[S[_], X](using
|
|||||||
Applicative[ST]
|
Applicative[ST]
|
||||||
.product(
|
.product(
|
||||||
// Collect argument types
|
// Collect argument types
|
||||||
token.args
|
token.absWithArgs
|
||||||
.foldLeft(Chain.empty[(String, Type)].pure[ST]) {
|
.foldLeft(Chain.empty[(String, Type)].pure[ST]) {
|
||||||
case (f, (Some(argName), argType)) =>
|
case (f, (Some(argName), argType)) =>
|
||||||
f.flatMap(acc =>
|
f.flatMap(acc =>
|
||||||
@ -720,7 +718,7 @@ class TypesInterpreter[S[_], X](using
|
|||||||
)(
|
)(
|
||||||
ifNotDefined: => State[X, A]
|
ifNotDefined: => State[X, A]
|
||||||
): State[X, A] = getState
|
): State[X, A] = getState
|
||||||
.map(_.definitions.get(name))
|
.map(_.getType(name))
|
||||||
.flatMap {
|
.flatMap {
|
||||||
case Some(_) =>
|
case Some(_) =>
|
||||||
// TODO: Point to both locations here
|
// TODO: Point to both locations here
|
||||||
|
@ -2,38 +2,25 @@ package aqua.semantics.rules.types
|
|||||||
|
|
||||||
import aqua.parser.lexer.*
|
import aqua.parser.lexer.*
|
||||||
import aqua.raw.RawContext
|
import aqua.raw.RawContext
|
||||||
import aqua.raw.value.{FunctorRaw, IntoIndexRaw, LiteralRaw, PropertyRaw, ValueRaw}
|
import aqua.raw.value.ValueRaw
|
||||||
import aqua.types.*
|
import aqua.types.*
|
||||||
|
|
||||||
import cats.data.Validated.{Invalid, Valid}
|
|
||||||
import cats.data.{Chain, NonEmptyChain, ValidatedNec}
|
|
||||||
import cats.kernel.Monoid
|
import cats.kernel.Monoid
|
||||||
import cats.syntax.apply.*
|
|
||||||
import cats.syntax.bifunctor.*
|
|
||||||
import cats.syntax.functor.*
|
|
||||||
import cats.syntax.option.*
|
|
||||||
import cats.syntax.traverse.*
|
|
||||||
import cats.syntax.validated.*
|
|
||||||
|
|
||||||
case class TypesState[S[_]](
|
case class TypesState[S[_]](
|
||||||
fields: Map[String, (Name[S], Type)] = Map(),
|
fields: Map[String, (Name[S], Type)] = Map(),
|
||||||
strict: Map[String, Type] = Map.empty,
|
strict: Map[String, Type] = Map.empty,
|
||||||
definitions: Map[String, NamedTypeToken[S]] = Map(),
|
|
||||||
stack: List[TypesState.Frame[S]] = Nil
|
stack: List[TypesState.Frame[S]] = Nil
|
||||||
) {
|
) {
|
||||||
def isDefined(t: String): Boolean = strict.contains(t)
|
def isDefined(t: String): Boolean = strict.contains(t)
|
||||||
|
|
||||||
def defineType(name: NamedTypeToken[S], `type`: Type): TypesState[S] =
|
def defineType(name: NamedTypeToken[S], `type`: Type): TypesState[S] =
|
||||||
copy(
|
copy(
|
||||||
strict = strict.updated(name.value, `type`),
|
strict = strict.updated(name.value, `type`)
|
||||||
definitions = definitions.updated(name.value, name)
|
|
||||||
)
|
)
|
||||||
|
|
||||||
def getType(name: String): Option[Type] =
|
def getType(name: String): Option[Type] =
|
||||||
strict.get(name)
|
strict.get(name)
|
||||||
|
|
||||||
def getTypeDefinition(name: String): Option[NamedTypeToken[S]] =
|
|
||||||
definitions.get(name)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
object TypesState {
|
object TypesState {
|
||||||
@ -55,7 +42,6 @@ object TypesState {
|
|||||||
override def combine(x: TypesState[S], y: TypesState[S]): TypesState[S] =
|
override def combine(x: TypesState[S], y: TypesState[S]): TypesState[S] =
|
||||||
TypesState(
|
TypesState(
|
||||||
strict = x.strict ++ y.strict,
|
strict = x.strict ++ y.strict,
|
||||||
definitions = x.definitions ++ y.definitions
|
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -78,7 +78,7 @@ class TypeResolutionSpec extends AnyFlatSpec with Matchers with Inside {
|
|||||||
base <- baseTypes
|
base <- baseTypes
|
||||||
(token, expected) = base
|
(token, expected) = base
|
||||||
} inside(resolve(token, Map("Struct" -> structType))) {
|
} inside(resolve(token, Map("Struct" -> structType))) {
|
||||||
case Valid(TypeResolution(result, Nil)) =>
|
case Valid(TypeResolution(result, _)) =>
|
||||||
result shouldEqual expected
|
result shouldEqual expected
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -100,7 +100,7 @@ class TypeResolutionSpec extends AnyFlatSpec with Matchers with Inside {
|
|||||||
(btoken, btype) = base
|
(btoken, btype) = base
|
||||||
(mod, typ) = modifier
|
(mod, typ) = modifier
|
||||||
} inside(resolve(mod(btoken), Map("Struct" -> structType))) {
|
} inside(resolve(mod(btoken), Map("Struct" -> structType))) {
|
||||||
case Valid(TypeResolution(result, Nil)) =>
|
case Valid(TypeResolution(result, _)) =>
|
||||||
result shouldEqual typ(btype)
|
result shouldEqual typ(btype)
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
|
Loading…
x
Reference in New Issue
Block a user