From a4564f0f079395ca2c07a6a00741161bf85e5228 Mon Sep 17 00:00:00 2001 From: Oscar Boykin Date: Tue, 5 Mar 2024 12:46:18 -1000 Subject: [PATCH 1/4] Error on unused lets --- .../src/main/resources/bosatsu/predef.bosatsu | 2 + .../scala/org/bykn/bosatsu/Evaluation.scala | 7 + .../scala/org/bykn/bosatsu/Indented.scala | 2 - .../scala/org/bykn/bosatsu/MainModule.scala | 10 +- .../main/scala/org/bykn/bosatsu/Package.scala | 5 + .../org/bykn/bosatsu/PackageCustoms.scala | 131 ++++++++++++++---- .../scala/org/bykn/bosatsu/PackageError.scala | 32 +++++ .../scala/org/bykn/bosatsu/PackageMap.scala | 2 + .../org/bykn/bosatsu/SourceConverter.scala | 2 +- .../scala/org/bykn/bosatsu/graph/Dag.scala | 20 ++- .../org/bykn/bosatsu/EvaluationTest.scala | 41 ++++-- .../scala/org/bykn/bosatsu/PackageTest.scala | 3 + .../org/bykn/bosatsu/graph/TreeTest.scala | 6 +- test_workspace/ApplicativeTraverse.bosatsu | 3 +- test_workspace/AvlTree.bosatsu | 42 +----- test_workspace/BuildLibrary.bosatsu | 3 +- test_workspace/Nat.bosatsu | 4 - test_workspace/Parser.bosatsu | 2 + test_workspace/Properties.bosatsu | 4 - test_workspace/TreeList.bosatsu | 4 +- test_workspace/TypeConstraint.bosatsu | 25 +++- test_workspace/euler4.bosatsu | 2 - test_workspace/euler5.bosatsu | 2 - 23 files changed, 237 insertions(+), 117 deletions(-) diff --git a/core/src/main/resources/bosatsu/predef.bosatsu b/core/src/main/resources/bosatsu/predef.bosatsu index 4cd58f237..db0e67cd9 100644 --- a/core/src/main/resources/bosatsu/predef.bosatsu +++ b/core/src/main/resources/bosatsu/predef.bosatsu @@ -46,6 +46,7 @@ export ( Dict, add, add_key, + build_List, char_to_String, cmp_Int, concat, @@ -56,6 +57,7 @@ export ( eq_Int, flat_map_List, foldLeft, + foldr_List, gcd_Int, get_key, int_loop, diff --git a/core/src/main/scala/org/bykn/bosatsu/Evaluation.scala b/core/src/main/scala/org/bykn/bosatsu/Evaluation.scala index 08cf1dfac..50717d73c 100644 --- a/core/src/main/scala/org/bykn/bosatsu/Evaluation.scala +++ b/core/src/main/scala/org/bykn/bosatsu/Evaluation.scala @@ -104,6 +104,13 @@ case class Evaluation[T](pm: PackageMap.Typed[T], externals: Externals) { value <- evaluate(p).get(name) } yield value + def evaluateMain(p: PackageName): Option[(Eval[Value], Type)] = + for { + pack <- pm.toMap.get(p) + (name, _, te) <- Package.mainValue(pack) + value <- evaluate(p).get(name) + } yield (value, te.getType) + /* TODO: this is useful for debugging, but we should probably test it and write a parser for the * list syntax diff --git a/core/src/main/scala/org/bykn/bosatsu/Indented.scala b/core/src/main/scala/org/bykn/bosatsu/Indented.scala index 107fbf363..e6584488c 100644 --- a/core/src/main/scala/org/bykn/bosatsu/Indented.scala +++ b/core/src/main/scala/org/bykn/bosatsu/Indented.scala @@ -4,8 +4,6 @@ import org.typelevel.paiges.{Doc, Document} import cats.parse.{Parser => P} -import cats.implicits._ - case class Indented[T](spaces: Int, value: T) { require(spaces > 0, s"need non-empty indentation: $spaces") } diff --git a/core/src/main/scala/org/bykn/bosatsu/MainModule.scala b/core/src/main/scala/org/bykn/bosatsu/MainModule.scala index 5fb098bf2..b9a6cac4d 100644 --- a/core/src/main/scala/org/bykn/bosatsu/MainModule.scala +++ b/core/src/main/scala/org/bykn/bosatsu/MainModule.scala @@ -521,13 +521,13 @@ abstract class MainModule[IO[_]](implicit ps: List[(Path, PackageName)] ): IO[(PackageName, Option[Bindable])] = ps.collectFirst { case (path, pn) if path == mainFile => pn } match { + case Some(p) => moduleIOMonad.pure((p, None)) case None => moduleIOMonad.raiseError( new Exception( s"could not find file $mainFile in parsed sources" ) ) - case Some(p) => moduleIOMonad.pure((p, None)) } } @@ -1018,16 +1018,14 @@ abstract class MainModule[IO[_]](implicit def runEval: IO[(Evaluation[Any], Output.EvaluationResult)] = withEC { implicit ec => for { - pn <- inputs.packMap(this, List(mainPackage), errColor) - (packs, names) = pn - mainPackageNameValue <- mainPackage.getMain(names) - (mainPackageName, value) = mainPackageNameValue + (packs, names) <- inputs.packMap(this, List(mainPackage), errColor) + (mainPackageName, value) <- mainPackage.getMain(names) out <- if (packs.toMap.contains(mainPackageName)) { val ev = Evaluation(packs, Predef.jvmExternals) val res = value match { - case None => ev.evaluateLast(mainPackageName) + case None => ev.evaluateMain(mainPackageName) case Some(ident) => ev.evaluateName(mainPackageName, ident) } diff --git a/core/src/main/scala/org/bykn/bosatsu/Package.scala b/core/src/main/scala/org/bykn/bosatsu/Package.scala index e7b400702..3d1285c36 100644 --- a/core/src/main/scala/org/bykn/bosatsu/Package.scala +++ b/core/src/main/scala/org/bykn/bosatsu/Package.scala @@ -97,6 +97,11 @@ object Package { te.getType == Type.TestType }.lastOption + def mainValue[A]( + tp: Typed[A] + ): Option[(Identifier.Bindable, RecursionKind, TypedExpr[A])] = + tp.program.lets.lastOption + /** Discard any top level values that are not referenced, exported, the final * test value, or the final expression * diff --git a/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala b/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala index 1cca0e604..6ceb81b96 100644 --- a/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala +++ b/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala @@ -17,15 +17,18 @@ import cats.syntax.all._ import org.bykn.bosatsu.Referant.Constructor import org.bykn.bosatsu.Referant.DefinedT import org.bykn.bosatsu.TypedExpr.Match +import org.bykn.bosatsu.Identifier.Bindable +import org.bykn.bosatsu.graph.Dag /** This checks the imports and exports of compiled packages and makes sure they * are valid */ object PackageCustoms { - def apply[A]( + def apply[A: HasRegion]( pack: Package.Typed[A] ): ValidatedNec[PackageError, Package.Typed[A]] = checkValuesHaveExportedTypes(pack.name, pack.exports) *> + noUselessBinds(pack) *> allImportsAreUsed(pack) private def removeUnused[A]( @@ -59,6 +62,39 @@ object PackageCustoms { pack.copy(imports = i) } + private type VSet = Set[(PackageName, Identifier)] + private type VState[X] = State[VSet, X] + + private def usedGlobals[A](te: TypedExpr[A]): VState[TypedExpr[A]] = + te.traverseUp[VState] { + case g @ TypedExpr.Global(p, n, _, _) => + State(s => (s + ((p, n)), g)) + case m @ Match(_, branches, _) => + branches + .traverse_ { case (pat, _) => + pat + .traverseStruct[ + VState, + (PackageName, Identifier.Constructor) + ] { (n, parts) => + State.modify[VSet](_ + n) *> + parts.map { inner => + Pattern.PositionalStruct(n, inner) + } + } + .void + } + .as(m) + case te => Monad[VState].pure(te) + } + + private def usedGlobals[A](pack: Package.Typed[A]): Set[(PackageName, Identifier)] = { + val usedValuesSt: VState[Unit] = + pack.program.lets.traverse_ { case (_, _, te) => usedGlobals(te) } + + usedValuesSt.runS(Set.empty).value + } + private def allImportsAreUsed[A]( pack: Package.Typed[A] ): ValidatedNec[PackageError, Package.Typed[A]] = { @@ -94,34 +130,7 @@ object PackageCustoms { if (impValues.isEmpty && impTypes.isEmpty) Validated.valid(pack) else { - type VSet = Set[(PackageName, Identifier)] - type VState[X] = State[VSet, X] - val usedValuesSt: VState[Unit] = - pack.program.lets.traverse_ { case (_, _, te) => - te.traverseUp { - case g @ TypedExpr.Global(p, n, _, _) => - State(s => (s + ((p, n)), g)) - case m @ Match(_, branches, _) => - branches - .traverse_ { case (pat, _) => - pat - .traverseStruct[ - VState, - (PackageName, Identifier.Constructor) - ] { (n, parts) => - State.modify[VSet](_ + n) *> - parts.map { inner => - Pattern.PositionalStruct(n, inner) - } - } - .void - } - .as(m) - case te => Monad[VState].pure(te) - } - } - - val usedValues = usedValuesSt.runS(Set.empty).value + val usedValues = usedGlobals(pack) val usedTypes: Set[Type.Const] = pack.program.lets.iterator @@ -224,4 +233,66 @@ object PackageCustoms { case Some(nel) => Validated.invalid(nel) } } -} + + private def noUselessBinds[A: HasRegion](pack: Package.Typed[A]): ValidatedNec[PackageError, Unit] = { + type Node = Either[pack.exports.type, Bindable] + implicit val ordNode: Ordering[Node] = + new Ordering[Node] { + def compare(x: Node, y: Node): Int = + x match { + case Right(bx) => + y match { + case Left(_) => 1 + case Right(by) => + Ordering[Identifier].compare(bx, by) + } + case Left(_) => + y match { + case Left(_) => 0 + case Right(_) => -1 + } + } + } + + val exports: Node = Left(pack.exports) + val roots: List[Node] = + (exports :: + pack.program.lets.collect { case (b, _, _) if b.asString.startsWith("_") => Right(b) } ::: + Package.testValue(pack).map { case (b, _, _) => Right(b) }.toList ::: + Package.mainValue(pack).map { case (b, _, _) => Right(b) }.toList).distinct + + val bindMap: Map[Bindable, TypedExpr[A]] = + pack.program.lets.iterator.map { case (b, _, te) => (b, te) }.toMap + + def internalDeps(te: TypedExpr[A]): Set[Bindable] = + usedGlobals(te).runS(Set.empty).value.collect { + case (pn, i: Identifier.Bindable) if pn == pack.name => i + } + + def depsOf(n: Node): Iterable[Node] = + n match { + case Left(_) => pack.exports.flatMap { + case ExportedName.Binding(n, _) => Right(n) :: Nil + case _ => Nil + } + case Right(value) => + bindMap.get(value) match { + case None => Nil + case Some(te) => internalDeps(te).map(Right(_)) + } + } + val canReach: SortedSet[Node] = Dag.transitiveSet(roots)(depsOf _) + + val unused = pack.program.lets.filter { + case (bn, _, _) => !(bn.asString.startsWith("_") || canReach.contains(Right(bn))) + } + + NonEmptyList.fromList(unused) match { + case None => Validated.unit + case Some(value) => + Validated.invalidNec(PackageError.UnusedLets(pack.name, value.map { case (b, r, te) => + (b, r, te, HasRegion.region(te)) + })) + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/bykn/bosatsu/PackageError.scala b/core/src/main/scala/org/bykn/bosatsu/PackageError.scala index 2a9855b80..e7745c4a3 100644 --- a/core/src/main/scala/org/bykn/bosatsu/PackageError.scala +++ b/core/src/main/scala/org/bykn/bosatsu/PackageError.scala @@ -886,4 +886,36 @@ object PackageError { ) + di + Doc.hardLine).render(80) } } + + case class UnusedLets( + inPack: PackageName, + unusedLets: NonEmptyList[(Identifier.Bindable, RecursionKind, TypedExpr[Any], Region)] + ) extends PackageError { + def message( + sourceMap: Map[PackageName, (LocationMap, String)], + errColor: Colorize + ) = { + val prefix = sourceMap.headLine(inPack, None) + val (lm, _) = sourceMap.getMapSrc(inPack) + val di = (Doc.hardLine + Doc.intercalate( + Doc.line, + unusedLets.toList.map { case (b, _, _, region) => + lm.showRegion(region, 2, errColor) match { + case Some(regionDoc) => + Doc.text(b.sourceCodeRepr) + Doc.hardLine + + regionDoc + case None => + Doc.text(b.sourceCodeRepr) + } + } + )) + .nested(2) + + val lets = + if (unusedLets.tail.lengthCompare(0) == 0) "let" else "lets" + (prefix + Doc.hardLine + Doc.text( + s"unused $lets of:" + ) + di + Doc.hardLine).render(80) + } + } } diff --git a/core/src/main/scala/org/bykn/bosatsu/PackageMap.scala b/core/src/main/scala/org/bykn/bosatsu/PackageMap.scala index eedd3db62..4ae918888 100644 --- a/core/src/main/scala/org/bykn/bosatsu/PackageMap.scala +++ b/core/src/main/scala/org/bykn/bosatsu/PackageMap.scala @@ -426,6 +426,8 @@ object PackageMap { // We have a result, which we can continue to check val pack = Package(nm, imps, exports, program) val res = (fte, pack) + // We have to check the "customs" before any normalization + // or optimization PackageCustoms(pack) match { case Validated.Valid(p1) => Ior.right((fte, p1)) case Validated.Invalid(errs) => diff --git a/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala b/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala index f069a8c55..be7384cb6 100644 --- a/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala +++ b/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala @@ -1218,7 +1218,7 @@ final class SourceConverter( private def anonNameStrings(): Iterator[String] = rankn.Type.allBinders.iterator - .map(_.name) + .map { bn => "_" + bn.name } private def unusedNames(allNames: Bindable => Boolean): Iterator[Bindable] = anonNameStrings() diff --git a/core/src/main/scala/org/bykn/bosatsu/graph/Dag.scala b/core/src/main/scala/org/bykn/bosatsu/graph/Dag.scala index 0ca1b4b55..44585ee5c 100644 --- a/core/src/main/scala/org/bykn/bosatsu/graph/Dag.scala +++ b/core/src/main/scala/org/bykn/bosatsu/graph/Dag.scala @@ -78,7 +78,10 @@ object Dag { m.iterator .map { case (a, current) => val next = nfn(a).iterator.foldLeft(SortedSet.empty[A]) { (s, a) => - s | m(a) + s | (m.get(a) match { + case Some(value) => value + case None => SortedSet.empty + }) } a -> (current | next) } @@ -139,6 +142,19 @@ object Dag { val depCache = MMap.empty[A, SortedSet[A]] def deps(a: A): SortedSet[A] = depCache.getOrElseUpdate(a, s.nfn(a).to(SortedSet)) - } + + def transitiveSet[A: Ordering](nodes: List[A])(nfn: A => Iterable[A]): SortedSet[A] = { + def loop(stack: List[A], inStack: SortedSet[A], reached: SortedSet[A]): SortedSet[A] = + stack match { + case head :: tail => + val next = nfn(head).iterator.filterNot { n => inStack(n) || reached(n) } + .toList + .sorted + loop(next ::: tail, inStack ++ next, reached + head) + case Nil => reached + } + + loop(nodes, SortedSet.empty, SortedSet.empty) + } } diff --git a/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala b/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala index fe57084c3..cce2b1c28 100644 --- a/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala +++ b/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala @@ -452,6 +452,8 @@ main = 6.gcd_Int(3) List(""" package Foo +export zip + three = [0, 1] # exercise the built-in range function threer = range(3) @@ -486,6 +488,7 @@ same = eq_list(three, threer, eq_Int) evalTest( List(""" package Foo +export zip def zip(as: List[a], bs: List[b]) -> List[(a, b)]: recur as: @@ -700,7 +703,7 @@ package Foo def concat(a): a -main = [1, 2] +main = concat([1, 2]) """), "Foo", VList.Cons(VInt(1), VList.Cons(VInt(2), VList.VNil)) @@ -817,6 +820,8 @@ main = one List(""" package A +export just_foo + struct Leib(subst: forall f: * -> *. f[a] -> f[b]) struct Id(a) @@ -842,6 +847,7 @@ def getValue(v: StringOrInt[a]) -> a: case IsStr(s, leib): coerce(s, leib) case IsInt(i, leib): coerce(i, leib) +just_foo = getValue(str) main = getValue(int) """), "A", @@ -2584,6 +2590,7 @@ tests = TestSuite("test", // test an example using a predef function, like add runBosatsuTest( List("""package A +export add # this should be add from predef two = add(1, 1) @@ -3084,6 +3091,8 @@ test = Assertion(True, "") List(""" package VarSet/Recursion +test = Assertion(True, "") + enum Thing: Thing1, Thing2(a: Int, t: Thing) @@ -3091,8 +3100,6 @@ def bar(y, _: String, x): recur x: Thing1: y Thing2(i, t): bar(i, "boom", t) - -test = Assertion(True, "") """), "VarSet/Recursion", 1 @@ -3319,6 +3326,8 @@ def get[shape](sh: shape[RecordValue], RecordGetter(getter): RecordGetter[shape, List(""" package Foo +export comp, ignore + def f(fn: forall a. List[a] -> List[a]) -> Int: fn([1]).foldLeft(0, (x, _) -> x.add(1)) @@ -3349,7 +3358,9 @@ struct Pair1(fst: a, snd: a) pair = Pair1(single_id1, single_id2) comp = x -> f(g(x)) - + +ignore: exists a. a = (pair, h, count, foo1) + test = Assertion(True, "") """), "Foo", @@ -3405,12 +3416,12 @@ def lengths2(l1: List[Int], l2: List[String], maybeFn: forall tt. Option[List[tt Some(fn): fn(l1).add(fn(l2)) None: 0 +test = Assertion(lengths([], [], None).add(lengths2([], [], None)) matches 0, "test") + # this is a test that doesn't forget that we have the empty list: x = match []: case []: 0 case [h, *_]: (h: forall a. a) - -test = Assertion(lengths([], [], None) matches 0, "test") """ :: Nil, "SubsumeTest", 1 @@ -3594,6 +3605,8 @@ test = Assertion(last(One(True)), "") List(""" package PolyRec +test = Assertion(True, "") + enum Nat: NZero, NSucc(n: Nat) def poly_rec(count: Nat, a: a) -> a: @@ -3603,8 +3616,6 @@ def poly_rec(count: Nat, a: a) -> a: # make a call with a different type (_, b) = poly_rec(prev, ("foo", a)) b - -test = Assertion(True, "") """), "PolyRec", 1 @@ -3614,6 +3625,8 @@ test = Assertion(True, "") List(""" package PolyRec +test = Assertion(True, "") + enum Nat: NZero, NSucc(n: Nat) def call(a): @@ -3628,8 +3641,6 @@ def call(a): b # call a polymorphic recursion internally to exercise different code paths poly_rec(NZero, a) - -test = Assertion(True, "") """), "PolyRec", 1 @@ -3666,6 +3677,9 @@ main = v evalTest( List(""" package A + +export ignore + enum Cont[a: *]: Item(a: a) Next(use: (Cont[a] -> a) -> a) @@ -3679,6 +3693,9 @@ def loop[a](box: Cont[a]) -> a: case Next(cont_fn): cont_fn(loop) loopgen: forall a. Cont[a] -> a = loop + +ignore: exists a. a = loopgen + b: Cont[Int] = Item(1).map(x -> x.add(1)) main: Int = loop(b) """), @@ -3694,9 +3711,6 @@ enum Box[a: +*]: Item(a: a) Next(fn: forall res. (forall b. (Box[b], b -> a) -> res) -> res) -def map[a, b](box: Box[a], fn: a -> b) -> Box[b]: - Next(cont -> cont(box, fn)) - b = Item(1) def loop[a](box: Box[a]) -> a: @@ -3806,6 +3820,7 @@ test = Assertion(res matches 0, "one") runBosatsuTest( List(""" package Foo +export maybeMapped, FreeF enum FreeF[a]: Pure(a: a) diff --git a/core/src/test/scala/org/bykn/bosatsu/PackageTest.scala b/core/src/test/scala/org/bykn/bosatsu/PackageTest.scala index 3c4846b53..23eb5278b 100644 --- a/core/src/test/scala/org/bykn/bosatsu/PackageTest.scala +++ b/core/src/test/scala/org/bykn/bosatsu/PackageTest.scala @@ -115,6 +115,8 @@ package P7 from P6 import data as p6_data from P5 import Option, List, NonEmpty as Cons, Empty as Nil, head +export data + data = Cons(1, Cons(2, Nil)) data1 = Cons(0, p6_data) @@ -160,6 +162,7 @@ def takeFoo(foo): val p2 = parse(""" package R2 from R1 import Foo as Bar, mkFoo, takeFoo +export main, main2 # note Bar is the same as foo struct Baz(b: Bar) diff --git a/core/src/test/scala/org/bykn/bosatsu/graph/TreeTest.scala b/core/src/test/scala/org/bykn/bosatsu/graph/TreeTest.scala index 2b80c71de..e54fccddd 100644 --- a/core/src/test/scala/org/bykn/bosatsu/graph/TreeTest.scala +++ b/core/src/test/scala/org/bykn/bosatsu/graph/TreeTest.scala @@ -76,7 +76,11 @@ class TreeTest extends AnyFunSuite { else s } - assert(reachable(Set(start)).toList.sorted == cycles.head.toList) + val reached = Dag.transitiveSet(start :: Nil)(nfn) + + val expectReachable = reachable(Set(start)).toList.sorted + assert(reached.toList == expectReachable) + assert(expectReachable == cycles.head.toList) assert(Tree.dagToTree(start)(nfn).isInvalid) } } diff --git a/test_workspace/ApplicativeTraverse.bosatsu b/test_workspace/ApplicativeTraverse.bosatsu index 411e304fd..7bc3dff48 100644 --- a/test_workspace/ApplicativeTraverse.bosatsu +++ b/test_workspace/ApplicativeTraverse.bosatsu @@ -2,7 +2,8 @@ package Bosatsu/Example/ApplicativeTraverse from Bosatsu/List import eq_List from Bosatsu/Option import eq_Option -export Applicative(), Traverse(), traverse_List, applicative_Option +export (Applicative(), Traverse(), traverse_List, applicative_Option, + applicative_from_pure_ap, applicative_from_pure_map_product) # Represents the Applicative typeclass struct Applicative( diff --git a/test_workspace/AvlTree.bosatsu b/test_workspace/AvlTree.bosatsu index de1095dd4..fe80f13e6 100644 --- a/test_workspace/AvlTree.bosatsu +++ b/test_workspace/AvlTree.bosatsu @@ -1,6 +1,6 @@ package AvlTree -export Tree, Module, module +export Tree, Module, module, min enum Tree[a]: Empty, Branch(size: Int, height: Int, key: a, left: Tree[a], right: Tree[a]) @@ -162,6 +162,7 @@ def fold_right_Tree(t: Tree[a], right_v: b, fn: (a, b) -> b) -> b: # Module pattern to associate some methods with a typeclass (Order) struct Module[a]( order: Order[a], + empty: Tree[a], add: (Tree[a], a) -> Tree[a], single: a -> Tree[a], contains: (Tree[a], a) -> Option[a], @@ -173,6 +174,7 @@ struct Module[a]( def module(ord: Order[a]) -> Module[a]: Module { order: ord, + empty: Empty, add: (t, a) -> add_item(ord, t, a), single: a -> add_item(ord, Empty, a), contains: (t, a) -> contains(ord, t, a), @@ -181,44 +183,6 @@ def module(ord: Order[a]) -> Module[a]: fold_right: fold_right_Tree, } -struct Dict[k, v](order: Order[(k, v)], tree: Tree[(k, v)]) - -def empty_Dict(comp: Order[k]) -> forall v. Dict[k, v]: - Order(fn) = comp - pair_ord = Order(\(k1, _), (k2, _) -> fn(k1, k2)) - Dict(pair_ord, Empty) - -def add_key(dict: Dict[k, v], key: k, value: v) -> Dict[k, v]: - Dict(ord, tree) = dict - new_tree = add_item(ord, tree, (key, value)) - Dict(ord, new_tree) - -def get_key(dict: Dict[k, v], key: k) -> Option[v]: - Dict(ord, tree) = dict - match min(tree): - None: None - Some((_, v)): - # fill in a fake v - match contains(ord, tree, (key, v)): - Some((_, v)): Some(v) - None: None - -def remove_key(dict: Dict[k, v], key: k) -> Dict[k, v]: - Dict(ord, tree) = dict - match min(tree): - None: dict - Some((_, v)): - # fill in a fake v - tree1 = remove_item(ord, tree, (key, v)) - Dict(ord, tree1) - -def items(dict: Dict[k, v]) -> List[(k, v)]: - Dict(_, tree) = dict - tree.fold_right_Tree([], \kv, tail -> [kv, *tail]) - -def clear_Dict(dict: Dict[k, v]) -> Dict[k, v]: - Dict(ord, _) = dict - Dict(ord, Empty) ##################### # test code ##################### diff --git a/test_workspace/BuildLibrary.bosatsu b/test_workspace/BuildLibrary.bosatsu index 51932fa3e..5dba01a8d 100644 --- a/test_workspace/BuildLibrary.bosatsu +++ b/test_workspace/BuildLibrary.bosatsu @@ -1,6 +1,7 @@ package BuildLibrary -export File(), Library(), LibArgs(), build, Build, file, files, build_all, library, empty +export (File(), Library(), LibArgs(), build, Build, file, files, build_all, library, empty, + map_Build, map2_Build) struct Leibniz[a, b](cast: forall f: * -> *. f[a] -> f[b]) (refl: forall a. Leibniz[a, a]) = Leibniz(x -> x) diff --git a/test_workspace/Nat.bosatsu b/test_workspace/Nat.bosatsu index 2cf8687e4..5b12ed46a 100644 --- a/test_workspace/Nat.bosatsu +++ b/test_workspace/Nat.bosatsu @@ -49,10 +49,6 @@ def to_Nat(i: Int) -> Nat: n1 = Succ(Zero) n2 = Succ(n1) -n3 = Succ(n2) -n4 = Succ(n3) -n5 = Succ(n4) -n6 = Succ(n5) def operator ==(i0: Int, i1: Int): cmp_Int(i0, i1) matches EQ diff --git a/test_workspace/Parser.bosatsu b/test_workspace/Parser.bosatsu index 24d8b9501..435390b96 100644 --- a/test_workspace/Parser.bosatsu +++ b/test_workspace/Parser.bosatsu @@ -1,5 +1,7 @@ package Parser +export Parser, empty, parse, flat_map, expect, one_of, map + struct Parser[a](parse: String -> Option[(String, a)]) empty = Parser(_ -> None) diff --git a/test_workspace/Properties.bosatsu b/test_workspace/Properties.bosatsu index 7f3d222b4..182363db5 100644 --- a/test_workspace/Properties.bosatsu +++ b/test_workspace/Properties.bosatsu @@ -8,13 +8,9 @@ export (Prop, forall_Prop, suite_Prop, run_Prop) struct Prop(fn: Int -> Rand[Test]) def operator -(a, b): a.sub(b) -def operator +(a, b): a.add(b) -def operator *(a, b): a.times(b) def operator &(a, b): a.and_Int(b) def operator <<(a, b): a.shift_left_Int(b) def operator >>(a, b): a.shift_right_Int(b) -def operator |(a, b): a.or_Int(b) -def operator ^(a, b): a.xor_Int(b) def forall_Prop[a](rand: Rand[a], name: String, fn: a -> Test) -> Prop: diff --git a/test_workspace/TreeList.bosatsu b/test_workspace/TreeList.bosatsu index 56040f692..ad70dff96 100644 --- a/test_workspace/TreeList.bosatsu +++ b/test_workspace/TreeList.bosatsu @@ -1,7 +1,6 @@ package TreeList -from Bosatsu/List import eq_List -export TreeList, empty, cons, decons, head, get, fold, eq_TreeList +export TreeList, empty, cons, decons, head, get, fold, eq_TreeList, to_List # implementation of O(1) cons/uncons O(log N) list-like data structure @@ -113,7 +112,6 @@ def eq_Opt(fn)(a, b): eq_oi = eq_Opt(eq_Int) eq_ti = eq_TreeList(eq_Int) -eq_li = eq_List(eq_Int) operator +/ = cons diff --git a/test_workspace/TypeConstraint.bosatsu b/test_workspace/TypeConstraint.bosatsu index a2621edbb..547330375 100644 --- a/test_workspace/TypeConstraint.bosatsu +++ b/test_workspace/TypeConstraint.bosatsu @@ -6,6 +6,7 @@ export ( cast, upcast, downcast, compose_eq, compose_sub, compose_sup, eq_to_sub, eq_to_sup, sub_to_sup, sup_to_sub, + flip_eq, ) struct Sub[a: -*, b: +*](sub: forall f: +* -> *. f[a] -> f[b]) @@ -72,9 +73,21 @@ def flip_eq[a, b](eq: Eq[a, b]) -> Eq[b, a]: def compose_eq[a, b, c](first: Eq[a, b], second: Eq[b, c]) -> Eq[a, c]: second.substitute(first) -refl_bottom: forall b. Sub[forall a. a, b] = refl_sub -refl_bottom1: Sub[forall a. a, forall a. a] = refl_sub -refl_Int: Sub[forall a. a, Int] = refl_sub -refl_any: Sub[forall a. a, exists a. a] = refl_sub -refl_any1: Sub[exists a. a, exists a. a] = refl_sub -refl_Int_any: Sub[Int, exists a. a] = refl_sub +# TODO: these typecheck, but using them fails with errors +# like: type ?211 does not subsume type forall a: *. a +# 84| +# 85|ignore: List[exists a. a] = [ +# 86| hide(refl_bottom), hide(refl_bottom1) +# +#refl_bottom: forall b. Sub[forall a. a, b] = refl_sub +#refl_bottom1: Sub[forall a. a, forall a. a] = refl_sub +#refl_Int: Sub[forall a. a, Int] = refl_sub +#refl_any: Sub[forall a. a, exists a. a] = refl_sub +#refl_any1: Sub[exists a. a, exists a. a] = refl_sub +#refl_Int_any: Sub[Int, exists a. a] = refl_sub +# +#def hide[a](a: a) -> exists b. b: a +# +#ignore: List[exists a. a] = [ +# hide(refl_bottom), hide(refl_bottom1) +#] \ No newline at end of file diff --git a/test_workspace/euler4.bosatsu b/test_workspace/euler4.bosatsu index de4dba30d..954ca4d3e 100644 --- a/test_workspace/euler4.bosatsu +++ b/test_workspace/euler4.bosatsu @@ -12,8 +12,6 @@ from Bosatsu/Nat import Nat, Succ, Zero, to_Nat def operator >(a, b): a.cmp_Int(b) matches GT -operator - = sub - def max_opt(o1, o2): match o1: None: o2 diff --git a/test_workspace/euler5.bosatsu b/test_workspace/euler5.bosatsu index 0c5fbb41a..648d0ba35 100644 --- a/test_workspace/euler5.bosatsu +++ b/test_workspace/euler5.bosatsu @@ -24,8 +24,6 @@ def int_loop_up(top, res, fn): (next_rev, next_res) = fn(top - i, res) (top - next_rev, next_res)) -sum_10 = int_loop_up(6, 0, \i, s -> (i + 1, s + i)) - bound = max_candidate.add(1) def divides(denom, num): From fe2b1ef8fb40ea6626a5215c0a5b6ca4c737bdce Mon Sep 17 00:00:00 2001 From: Oscar Boykin Date: Mon, 11 Mar 2024 09:14:31 -0700 Subject: [PATCH 2/4] don't root synthetics --- core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala | 3 +-- core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala | 6 ++---- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala b/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala index 1223e20c3..71c01277a 100644 --- a/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala +++ b/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala @@ -257,7 +257,6 @@ object PackageCustoms { val exports: Node = Left(pack.exports) val roots: List[Node] = (exports :: - pack.program.lets.collect { case (b, _, _) if b.isSynthetic => Right(b) } ::: Package.testValue(pack).map { case (b, _, _) => Right(b) }.toList ::: Package.mainValue(pack).map { case (b, _, _) => Right(b) }.toList).distinct @@ -284,7 +283,7 @@ object PackageCustoms { val canReach: SortedSet[Node] = Dag.transitiveSet(roots)(depsOf _) val unused = pack.program.lets.filter { - case (bn, _, _) => !(bn.asString.startsWith("_") || canReach.contains(Right(bn))) + case (bn, _, _) => !(bn.isSynthetic || canReach.contains(Right(bn))) } NonEmptyList.fromList(unused) match { diff --git a/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala b/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala index cce2b1c28..9147ebb0b 100644 --- a/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala +++ b/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala @@ -3061,8 +3061,7 @@ test = Assertion(True, "") package A # this is basically a typecheck only -x = (1, "1") -(_, _) = x +(_, _) = (1, "1") test = Assertion(True, "") """), @@ -3076,8 +3075,7 @@ package A struct Foo(x, y) # this is basically a typecheck only -x = Foo(1, "1") -Foo(_, _) = x +Foo(_, _) = Foo(1, "1") test = Assertion(True, "") """), From ca7746c427eb793736dc104bd218219135f36087 Mon Sep 17 00:00:00 2001 From: Oscar Boykin Date: Fri, 15 Mar 2024 10:05:44 -0700 Subject: [PATCH 3/4] ban non-binding patterns, add tests --- .../org/bykn/bosatsu/PackageCustoms.scala | 31 +------- .../scala/org/bykn/bosatsu/PackageError.scala | 28 ++----- .../org/bykn/bosatsu/SourceConverter.scala | 22 +++++- .../scala/org/bykn/bosatsu/TypedExpr.scala | 27 ++++++- .../org/bykn/bosatsu/EvaluationTest.scala | 73 +++++++++++++------ test_workspace/Foo.bosatsu | 3 + 6 files changed, 107 insertions(+), 77 deletions(-) diff --git a/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala b/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala index 71c01277a..94c367189 100644 --- a/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala +++ b/core/src/main/scala/org/bykn/bosatsu/PackageCustoms.scala @@ -1,6 +1,5 @@ package org.bykn.bosatsu -import cats.Monad import cats.data.{ Chain, NonEmptyList, @@ -16,7 +15,6 @@ import scala.collection.immutable.SortedSet import cats.syntax.all._ import org.bykn.bosatsu.Referant.Constructor import org.bykn.bosatsu.Referant.DefinedT -import org.bykn.bosatsu.TypedExpr.Match import org.bykn.bosatsu.Identifier.Bindable import org.bykn.bosatsu.graph.Dag @@ -65,32 +63,9 @@ object PackageCustoms { private type VSet = Set[(PackageName, Identifier)] private type VState[X] = State[VSet, X] - private def usedGlobals[A](te: TypedExpr[A]): VState[TypedExpr[A]] = - te.traverseUp[VState] { - case g @ TypedExpr.Global(p, n, _, _) => - State(s => (s + ((p, n)), g)) - case m @ Match(_, branches, _) => - branches - .traverse_ { case (pat, _) => - pat - .traverseStruct[ - VState, - (PackageName, Identifier.Constructor) - ] { (n, parts) => - State.modify[VSet](_ + n) *> - parts.map { inner => - Pattern.PositionalStruct(n, inner) - } - } - .void - } - .as(m) - case te => Monad[VState].pure(te) - } - private def usedGlobals[A](pack: Package.Typed[A]): Set[(PackageName, Identifier)] = { val usedValuesSt: VState[Unit] = - pack.program.lets.traverse_ { case (_, _, te) => usedGlobals(te) } + pack.program.lets.traverse_ { case (_, _, te) => TypedExpr.usedGlobals(te) } usedValuesSt.runS(Set.empty).value } @@ -264,7 +239,7 @@ object PackageCustoms { pack.program.lets.iterator.map { case (b, _, te) => (b, te) }.toMap def internalDeps(te: TypedExpr[A]): Set[Bindable] = - usedGlobals(te).runS(Set.empty).value.collect { + TypedExpr.usedGlobals(te).runS(Set.empty).value.collect { case (pn, i: Identifier.Bindable) if pn == pack.name => i } @@ -283,7 +258,7 @@ object PackageCustoms { val canReach: SortedSet[Node] = Dag.transitiveSet(roots)(depsOf _) val unused = pack.program.lets.filter { - case (bn, _, _) => !(bn.isSynthetic || canReach.contains(Right(bn))) + case (bn, _, _) => !canReach.contains(Right(bn)) } NonEmptyList.fromList(unused) match { diff --git a/core/src/main/scala/org/bykn/bosatsu/PackageError.scala b/core/src/main/scala/org/bykn/bosatsu/PackageError.scala index e7745c4a3..a029dd4f6 100644 --- a/core/src/main/scala/org/bykn/bosatsu/PackageError.scala +++ b/core/src/main/scala/org/bykn/bosatsu/PackageError.scala @@ -894,28 +894,10 @@ object PackageError { def message( sourceMap: Map[PackageName, (LocationMap, String)], errColor: Colorize - ) = { - val prefix = sourceMap.headLine(inPack, None) - val (lm, _) = sourceMap.getMapSrc(inPack) - val di = (Doc.hardLine + Doc.intercalate( - Doc.line, - unusedLets.toList.map { case (b, _, _, region) => - lm.showRegion(region, 2, errColor) match { - case Some(regionDoc) => - Doc.text(b.sourceCodeRepr) + Doc.hardLine + - regionDoc - case None => - Doc.text(b.sourceCodeRepr) - } - } - )) - .nested(2) - - val lets = - if (unusedLets.tail.lengthCompare(0) == 0) "let" else "lets" - (prefix + Doc.hardLine + Doc.text( - s"unused $lets of:" - ) + di + Doc.hardLine).render(80) - } + ) = + UnusedLetError( + inPack, + unusedLets.map { case (b, _, _, r) => (b, r)} + ).message(sourceMap, errColor) } } diff --git a/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala b/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala index 061cb88a1..ae07515ac 100644 --- a/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala +++ b/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala @@ -1401,6 +1401,12 @@ final class SourceConverter( } } + val noBinds: Result[Unit] = stmts.parTraverse_ { + case Bind(BindingStatement(b, d, _)) if b.names.isEmpty => + Ior.Both(NonEmptyChain.one(SourceConverter.NonBindingPattern(b, d)), ()) + case _ => SourceConverter.successUnit + } + val flatIn: List[(Bindable, RecursionKind, Flattened)] = SourceConverter.makeLetsUnique(flatList) { (bind, _) => // rename all but the last item @@ -1457,6 +1463,7 @@ final class SourceConverter( case (b, _, Right((_, d))) => Right(Right((b, d))) } + noBinds.parProductR( parFold(Set.empty[Bindable], withEx) { case (topBound, stmt) => stmt match { case Right(Right((nm, decl))) => @@ -1509,8 +1516,7 @@ final class SourceConverter( case Left(ExternalDef(n, _, _)) => (topBound + n, success(Nil)) } - }(SourceConverter.parallelIor) - .map(_.flatten) + }(SourceConverter.parallelIor)).map(_.flatten) } def toProgram( @@ -1589,6 +1595,12 @@ object SourceConverter { def addError[A](r: Result[A], err: Error): Result[A] = parallelIor.<*(r)(failure(err)) + def maybeError[A](r: Result[A], opt: Option[Error]): Result[A] = + opt match { + case None => r + case Some(e) => addError(r, e) + } + // use this when we want to accumulate errors in parallel private val parallelIor: Applicative[Result] = Ior.catsDataParallelForIor[NonEmptyChain[Error]].applicative @@ -1932,4 +1944,10 @@ object SourceConverter { .render(80) } } + + final case class NonBindingPattern(pattern: Pattern.Parsed, bound: Declaration) extends Error { + def message = + (Document[Pattern.Parsed].document(pattern) + Doc.text(" does not bind any names.")).render(80) + def region = bound.region + } } diff --git a/core/src/main/scala/org/bykn/bosatsu/TypedExpr.scala b/core/src/main/scala/org/bykn/bosatsu/TypedExpr.scala index eedc57b24..fbf39044b 100644 --- a/core/src/main/scala/org/bykn/bosatsu/TypedExpr.scala +++ b/core/src/main/scala/org/bykn/bosatsu/TypedExpr.scala @@ -2,7 +2,7 @@ package org.bykn.bosatsu import cats.{Applicative, Eval, Monad, Traverse} import cats.arrow.FunctionK -import cats.data.{NonEmptyList, Writer} +import cats.data.{NonEmptyList, Writer, State} import cats.implicits._ import org.bykn.bosatsu.rankn.Type import org.typelevel.paiges.{Doc, Document} @@ -1531,4 +1531,29 @@ object TypedExpr { } } + def usedGlobals[A](te: TypedExpr[A]): State[Set[(PackageName, Identifier)], TypedExpr[A]] = { + type VSet = Set[(PackageName, Identifier)] + type VState[X] = State[VSet, X] + te.traverseUp[VState] { + case g @ TypedExpr.Global(p, n, _, _) => + State(s => (s + ((p, n)), g)) + case m @ Match(_, branches, _) => + branches + .traverse_ { case (pat, _) => + pat + .traverseStruct[ + VState, + (PackageName, Identifier.Constructor) + ] { (n, parts) => + State.modify[VSet](_ + n) *> + parts.map { inner => + Pattern.PositionalStruct(n, inner) + } + } + .void + } + .as(m) + case te => Monad[VState].pure(te) + } + } } diff --git a/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala b/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala index 9147ebb0b..3eea45f7e 100644 --- a/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala +++ b/core/src/test/scala/org/bykn/bosatsu/EvaluationTest.scala @@ -909,6 +909,23 @@ main = plus(1, 2) assert(msg.contains("unused let binding: z\n Region(68,73)")) () } + + evalFail(List(""" +package A + +# this shouldn't compile, z is unused +z = 1 + +def plus(x, y): + x.add(y) + +main = plus(1, 2) +""")) { case le @ PackageError.UnusedLets(_, _) => + val msg = le.message(Map.empty, Colorize.None) + assert(!msg.contains("Name(")) + assert(msg.contains("unused let binding: z\n Region(54,55)")) + () + } } test("structual recursion is allowed") { @@ -3042,34 +3059,41 @@ main = Foo(1) } } - test("non binding top levels work") { - runBosatsuTest( + test("non binding top levels don't work") { + evalFail( List(""" package A # this is basically a typecheck only _ = add(1, 2) -test = Assertion(True, "") -"""), - "A", - 1 - ) +""")) { case sce @ PackageError.SourceConverterErrorsIn(_, _, _) => + assert( + sce.message( + Map.empty, + Colorize.None + ) == "in file: , package A\n_ does not bind any names.\nRegion(53,62)" + ) + () + } - runBosatsuTest( + evalFail( List(""" package A # this is basically a typecheck only (_, _) = (1, "1") +""")) { case sce @ PackageError.SourceConverterErrorsIn(_, _, _) => + assert( + sce.message( + Map.empty, + Colorize.None + ) == "in file: , package A\n(_, _) does not bind any names.\nRegion(58,66)" + ) + () + } -test = Assertion(True, "") -"""), - "A", - 1 - ) - - runBosatsuTest( + evalFail( List(""" package A @@ -3077,11 +3101,15 @@ struct Foo(x, y) # this is basically a typecheck only Foo(_, _) = Foo(1, "1") -test = Assertion(True, "") -"""), - "A", - 1 - ) +""")) { case sce @ PackageError.SourceConverterErrorsIn(_, _, _) => + assert( + sce.message( + Map.empty, + Colorize.None + ) == "in file: , package A\nFoo(_, _) does not bind any names.\nRegion(78,89)" + ) + () + } } test("recursion check with _ pattern: issue 573") { @@ -3345,7 +3373,7 @@ def foo1(fn) -> Int: def foo2(fn: List[forall a. a -> a]) -> Int: fn.foldLeft(0, (x, _) -> x.add(1)) -count = foo1(single(id)) +count0 = foo1(single(id)) count = foo2(single(id)) single_id1: forall a. List[a -> a] = single(id) @@ -3357,7 +3385,7 @@ pair = Pair1(single_id1, single_id2) comp = x -> f(g(x)) -ignore: exists a. a = (pair, h, count, foo1) +ignore: exists a. a = (pair, h, count, foo1, count0) test = Assertion(True, "") """), @@ -3961,6 +3989,5 @@ test = TestSuite("bases", "Foo", 12 ) - } } diff --git a/test_workspace/Foo.bosatsu b/test_workspace/Foo.bosatsu index 376118635..f3fcdeb72 100644 --- a/test_workspace/Foo.bosatsu +++ b/test_workspace/Foo.bosatsu @@ -3,5 +3,8 @@ export x, # testing an implicit package name x = 1 +y = x # now we shadow x with a different type x = "this is Foo" + +ignore = y From fc46ced7522bf72433b03ec32250aa96235e07e6 Mon Sep 17 00:00:00 2001 From: Oscar Boykin Date: Fri, 15 Mar 2024 10:28:01 -0700 Subject: [PATCH 4/4] remove unused code --- core/src/main/scala/org/bykn/bosatsu/Identifier.scala | 5 +---- .../src/main/scala/org/bykn/bosatsu/SourceConverter.scala | 8 +------- core/src/main/scala/org/bykn/bosatsu/graph/Dag.scala | 5 +---- 3 files changed, 3 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/bykn/bosatsu/Identifier.scala b/core/src/main/scala/org/bykn/bosatsu/Identifier.scala index 0f40a2a86..6eb584e55 100644 --- a/core/src/main/scala/org/bykn/bosatsu/Identifier.scala +++ b/core/src/main/scala/org/bykn/bosatsu/Identifier.scala @@ -41,10 +41,7 @@ object Identifier { /** These are names that can appear in bindings. Importantly, we can't bind * constructor names except to define types */ - sealed abstract class Bindable extends Identifier { - def isSynthetic: Boolean = - asString.startsWith("_") - } + sealed abstract class Bindable extends Identifier final case class Constructor(asString: String) extends Identifier final case class Name(asString: String) extends Bindable diff --git a/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala b/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala index ae07515ac..a31c37ff1 100644 --- a/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala +++ b/core/src/main/scala/org/bykn/bosatsu/SourceConverter.scala @@ -1403,7 +1403,7 @@ final class SourceConverter( val noBinds: Result[Unit] = stmts.parTraverse_ { case Bind(BindingStatement(b, d, _)) if b.names.isEmpty => - Ior.Both(NonEmptyChain.one(SourceConverter.NonBindingPattern(b, d)), ()) + SourceConverter.partial(SourceConverter.NonBindingPattern(b, d), ()) case _ => SourceConverter.successUnit } @@ -1595,12 +1595,6 @@ object SourceConverter { def addError[A](r: Result[A], err: Error): Result[A] = parallelIor.<*(r)(failure(err)) - def maybeError[A](r: Result[A], opt: Option[Error]): Result[A] = - opt match { - case None => r - case Some(e) => addError(r, e) - } - // use this when we want to accumulate errors in parallel private val parallelIor: Applicative[Result] = Ior.catsDataParallelForIor[NonEmptyChain[Error]].applicative diff --git a/core/src/main/scala/org/bykn/bosatsu/graph/Dag.scala b/core/src/main/scala/org/bykn/bosatsu/graph/Dag.scala index 44585ee5c..92b58c6f6 100644 --- a/core/src/main/scala/org/bykn/bosatsu/graph/Dag.scala +++ b/core/src/main/scala/org/bykn/bosatsu/graph/Dag.scala @@ -78,10 +78,7 @@ object Dag { m.iterator .map { case (a, current) => val next = nfn(a).iterator.foldLeft(SortedSet.empty[A]) { (s, a) => - s | (m.get(a) match { - case Some(value) => value - case None => SortedSet.empty - }) + s | m(a) } a -> (current | next) }