diff --git a/engine/runtime-compiler/src/main/java/org/enso/compiler/pass/lint/ShadowedPatternFields.java b/engine/runtime-compiler/src/main/java/org/enso/compiler/pass/lint/ShadowedPatternFields.java new file mode 100644 index 000000000000..aff15a04191c --- /dev/null +++ b/engine/runtime-compiler/src/main/java/org/enso/compiler/pass/lint/ShadowedPatternFields.java @@ -0,0 +1,187 @@ +package org.enso.compiler.pass.lint; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.enso.compiler.context.InlineContext; +import org.enso.compiler.context.ModuleContext; +import org.enso.compiler.core.CompilerError; +import org.enso.compiler.core.IR; +import org.enso.compiler.core.ir.Expression; +import org.enso.compiler.core.ir.MetadataStorage; +import org.enso.compiler.core.ir.Name; +import org.enso.compiler.core.ir.Pattern; +import org.enso.compiler.core.ir.expression.Case; +import org.enso.compiler.core.ir.expression.Case.Branch; +import org.enso.compiler.core.ir.expression.warnings.Shadowed.PatternBinding; +import org.enso.compiler.pass.IRProcessingPass; +import org.enso.compiler.pass.MiniIRPass; +import org.enso.compiler.pass.MiniPassFactory; +import org.enso.compiler.pass.analyse.AliasAnalysis$; +import org.enso.compiler.pass.analyse.DataflowAnalysis$; +import org.enso.compiler.pass.analyse.DemandAnalysis$; +import org.enso.compiler.pass.analyse.TailCall; +import org.enso.compiler.pass.desugar.GenerateMethodBodies$; +import org.enso.compiler.pass.desugar.NestedPatternMatch$; +import org.enso.compiler.pass.resolve.IgnoredBindings$; +import scala.collection.immutable.List; +import scala.collection.immutable.Seq; +import scala.jdk.javaapi.CollectionConverters; + +/** + * This pass detects and renames shadowed pattern fields. + * + *

This is necessary both in order to create a warning, but also to ensure that alias analysis + * doesn't get confused. + * + *

This pass requires no configuration. + * + *

This pass requires the context to provide: + * + *

- Nothing + */ +public final class ShadowedPatternFields implements MiniPassFactory { + public static final ShadowedPatternFields INSTANCE = new ShadowedPatternFields(); + + private ShadowedPatternFields() {} + + @Override + public List precursorPasses() { + java.util.List list = java.util.List.of(GenerateMethodBodies$.MODULE$); + return CollectionConverters.asScala(list).toList(); + } + + @Override + public List invalidatedPasses() { + java.util.List list = + java.util.List.of( + AliasAnalysis$.MODULE$, + DataflowAnalysis$.MODULE$, + DemandAnalysis$.MODULE$, + IgnoredBindings$.MODULE$, + NestedPatternMatch$.MODULE$, + TailCall.INSTANCE); + return CollectionConverters.asScala(list).toList(); + } + + @Override + public MiniIRPass createForModuleCompilation(ModuleContext moduleContext) { + return new Mini(); + } + + @Override + public MiniIRPass createForInlineCompilation(InlineContext inlineContext) { + return new Mini(); + } + + private static final class Mini extends MiniIRPass { + @Override + @SuppressWarnings("unchecked") + public Expression transformExpression(Expression expr) { + return switch (expr) { + case Case.Branch branch -> lintCaseBranch(branch); + case Case.Expr caseExpr -> { + Seq newBranches = caseExpr.branches().map(this::lintCaseBranch).toSeq(); + yield caseExpr.copy( + caseExpr.scrutinee(), + newBranches, + caseExpr.isNested(), + caseExpr.location(), + caseExpr.passData(), + caseExpr.diagnostics(), + caseExpr.id()); + } + default -> expr; + }; + } + + /** + * Lints for shadowed pattern variables in a case branch. + * + * @param branch the case branch to lint + * @return `branch`, with warnings for any shadowed pattern variables + */ + private Case.Branch lintCaseBranch(Case.Branch branch) { + var newPattern = lintPattern(branch.pattern()); + return branch.copy( + newPattern, + branch.expression(), + branch.terminalBranch(), + branch.location(), + branch.passData(), + branch.diagnostics(), + branch.id()); + } + + /** + * Lints a pattern for shadowed pattern variables. + * + *

A later pattern variable shadows an earlier pattern variable with the same name. + * + * @param pattern the pattern to lint + * @return `pattern`, with a warning applied to any shadowed pattern variables + */ + private Pattern lintPattern(Pattern pattern) { + var seenNames = new HashSet(); + var lastSeen = new HashMap(); + + return go(pattern, seenNames, lastSeen); + } + + private Pattern go(Pattern pattern, Set seenNames, Map lastSeen) { + return switch (pattern) { + case Pattern.Name named -> { + var name = named.name().name(); + if (seenNames.contains(name)) { + var warning = new PatternBinding(name, lastSeen.get(name), named.identifiedLocation()); + lastSeen.put(name, named); + var blank = new Name.Blank(named.identifiedLocation(), new MetadataStorage()); + var patternCopy = named.copyWithName(blank); + patternCopy.getDiagnostics().add(warning); + yield patternCopy; + } else if (!(named.name() instanceof Name.Blank)) { + lastSeen.put(name, named); + seenNames.add(name); + yield named; + } else { + yield named; + } + } + case Pattern.Constructor cons -> { + var newFields = + cons.fields().reverse().map(field -> go(field, seenNames, lastSeen)).reverse(); + yield cons.copyWithFields(newFields); + } + case Pattern.Literal literal -> literal; + case Pattern.Type typed -> { + var name = typed.name().name(); + if (seenNames.contains(name)) { + var warning = new PatternBinding(name, lastSeen.get(name), typed.identifiedLocation()); + lastSeen.put(name, typed); + var blank = new Name.Blank(typed.identifiedLocation(), new MetadataStorage()); + var typedCopy = + typed.copy( + blank, + typed.tpe(), + typed.location(), + typed.passData(), + typed.diagnostics(), + typed.id()); + typedCopy.getDiagnostics().add(warning); + yield typedCopy; + } else if (!(typed.name() instanceof Name.Blank)) { + lastSeen.put(name, typed); + seenNames.add(name); + yield typed; + } else { + yield typed; + } + } + case Pattern.Documentation doc -> throw new CompilerError( + "Branch documentation should be desugared at an earlier stage."); + default -> pattern; + }; + } + } +} diff --git a/engine/runtime-compiler/src/main/java/org/enso/compiler/pass/optimise/UnreachableMatchBranches.java b/engine/runtime-compiler/src/main/java/org/enso/compiler/pass/optimise/UnreachableMatchBranches.java new file mode 100644 index 000000000000..713dcb8457e5 --- /dev/null +++ b/engine/runtime-compiler/src/main/java/org/enso/compiler/pass/optimise/UnreachableMatchBranches.java @@ -0,0 +1,175 @@ +package org.enso.compiler.pass.optimise; + +import java.util.ArrayList; +import java.util.stream.Stream; +import org.enso.compiler.context.InlineContext; +import org.enso.compiler.context.ModuleContext; +import org.enso.compiler.core.CompilerError; +import org.enso.compiler.core.ir.Expression; +import org.enso.compiler.core.ir.IdentifiedLocation; +import org.enso.compiler.core.ir.Pattern; +import org.enso.compiler.core.ir.expression.Case; +import org.enso.compiler.core.ir.expression.warnings.Unreachable; +import org.enso.compiler.pass.IRProcessingPass; +import org.enso.compiler.pass.MiniIRPass; +import org.enso.compiler.pass.MiniPassFactory; +import org.enso.compiler.pass.analyse.AliasAnalysis$; +import org.enso.compiler.pass.analyse.DataflowAnalysis$; +import org.enso.compiler.pass.analyse.DemandAnalysis$; +import org.enso.compiler.pass.analyse.TailCall; +import org.enso.compiler.pass.desugar.ComplexType$; +import org.enso.compiler.pass.desugar.FunctionBinding$; +import org.enso.compiler.pass.desugar.GenerateMethodBodies$; +import org.enso.compiler.pass.desugar.LambdaShorthandToLambda$; +import org.enso.compiler.pass.desugar.NestedPatternMatch$; +import org.enso.compiler.pass.resolve.DocumentationComments$; +import org.enso.compiler.pass.resolve.IgnoredBindings$; +import org.enso.scala.wrapper.ScalaConversions; +import scala.collection.immutable.List; +import scala.jdk.javaapi.CollectionConverters; + +/** + * This pass discovers and optimizes away unreachable case branches. + * + *

It removes these unreachable expressions from the IR, and attaches a {@link + * org.enso.compiler.core.ir.Warning} diagnostic to the case expression itself. + * + *

Currently, a branch is considered 'unreachable' by this pass if: + * + *

+ * + *

In the future, this pass should be expanded to consider patterns that are entirely subsumed by + * previous patterns in its definition of unreachable, but this requires doing sophisticated + * coverage analysis, and hence should happen as part of the broader refactor of nested patterns + * desugaring. + * + *

This pass requires no configuration. + * + *

This pass requires the context to provide: + * + *

+ */ +public final class UnreachableMatchBranches implements MiniPassFactory { + private UnreachableMatchBranches() {} + + public static final UnreachableMatchBranches INSTANCE = new UnreachableMatchBranches(); + + @Override + public List precursorPasses() { + java.util.List passes = new ArrayList<>(); + passes.add(ComplexType$.MODULE$); + passes.add(DocumentationComments$.MODULE$); + passes.add(FunctionBinding$.MODULE$); + passes.add(GenerateMethodBodies$.MODULE$); + passes.add(LambdaShorthandToLambda$.MODULE$); + return CollectionConverters.asScala(passes).toList(); + } + + @Override + public List invalidatedPasses() { + java.util.List passes = new ArrayList<>(); + passes.add(AliasAnalysis$.MODULE$); + passes.add(DataflowAnalysis$.MODULE$); + passes.add(DemandAnalysis$.MODULE$); + passes.add(IgnoredBindings$.MODULE$); + passes.add(NestedPatternMatch$.MODULE$); + passes.add(TailCall.INSTANCE); + return CollectionConverters.asScala(passes).toList(); + } + + @Override + public MiniIRPass createForInlineCompilation(InlineContext inlineContext) { + return new Mini(); + } + + @Override + public MiniIRPass createForModuleCompilation(ModuleContext moduleContext) { + return new Mini(); + } + + private static class Mini extends MiniIRPass { + @Override + public Expression transformExpression(Expression expr) { + return switch (expr) { + case Case cse -> optimizeCase(cse); + default -> expr; + }; + } + + /** + * Optimizes a case expression by removing unreachable branches. + * + *

Additionally, it will attach a warning about unreachable branches to the case expression. + * + * @param cse the case expression to optimize + * @return `cse` with unreachable branches removed + */ + private Case optimizeCase(Case cse) { + if (cse instanceof Case.Expr expr) { + var branches = CollectionConverters.asJava(expr.branches()); + var reachableNonCatchAllBranches = + branches.stream().takeWhile(branch -> !isCatchAll(branch)); + var firstCatchAll = branches.stream().filter(this::isCatchAll).findFirst(); + var unreachableBranches = + branches.stream().dropWhile(branch -> !isCatchAll(branch)).skip(1).toList(); + List reachableBranches; + if (firstCatchAll.isPresent()) { + reachableBranches = appended(reachableNonCatchAllBranches, firstCatchAll.get()); + } else { + reachableBranches = ScalaConversions.nil(); + } + + if (unreachableBranches.isEmpty()) { + return expr; + } else { + var firstUnreachableWithLoc = + unreachableBranches.stream() + .filter(branch -> branch.identifiedLocation() != null) + .findFirst(); + var lastUnreachableWithLoc = + unreachableBranches.stream() + .filter(branch -> branch.identifiedLocation() != null) + .reduce((first, second) -> second); + IdentifiedLocation unreachableLocation = null; + if (firstUnreachableWithLoc.isPresent() && lastUnreachableWithLoc.isPresent()) { + unreachableLocation = + new IdentifiedLocation( + firstUnreachableWithLoc.get().location().get().start(), + lastUnreachableWithLoc.get().location().get().end(), + firstUnreachableWithLoc.get().id()); + } + + var diagnostic = new Unreachable.Branches(unreachableLocation); + var copiedExpr = expr.copyWithBranches(reachableBranches); + copiedExpr.getDiagnostics().add(diagnostic); + return copiedExpr; + } + } else { + throw new CompilerError("Unexpected case branch."); + } + } + + /** + * Determines if a branch is a catch all branch. + * + * @param branch the branch to check + * @return `true` if `branch` is catch-all, otherwise `false` + */ + private boolean isCatchAll(Case.Branch branch) { + return switch (branch.pattern()) { + case Pattern.Name ignored -> true; + default -> false; + }; + } + + private static List appended(Stream branches, Case.Branch branch) { + var ret = new ArrayList<>(branches.toList()); + ret.add(branch); + return CollectionConverters.asScala(ret).toList(); + } + } +} diff --git a/engine/runtime-compiler/src/main/scala/org/enso/compiler/Passes.scala b/engine/runtime-compiler/src/main/scala/org/enso/compiler/Passes.scala index 9d0f42809031..0109879c024c 100644 --- a/engine/runtime-compiler/src/main/scala/org/enso/compiler/Passes.scala +++ b/engine/runtime-compiler/src/main/scala/org/enso/compiler/Passes.scala @@ -57,8 +57,8 @@ class Passes(config: CompilerConfig) { ) } else List()) ++ List( - ShadowedPatternFields, - UnreachableMatchBranches, + ShadowedPatternFields.INSTANCE, + UnreachableMatchBranches.INSTANCE, NestedPatternMatch, IgnoredBindings, TypeFunctions, diff --git a/engine/runtime-compiler/src/main/scala/org/enso/compiler/pass/lint/ShadowedPatternFields.scala b/engine/runtime-compiler/src/main/scala/org/enso/compiler/pass/lint/ShadowedPatternFields.scala deleted file mode 100644 index f1c277bd431a..000000000000 --- a/engine/runtime-compiler/src/main/scala/org/enso/compiler/pass/lint/ShadowedPatternFields.scala +++ /dev/null @@ -1,197 +0,0 @@ -package org.enso.compiler.pass.lint - -import org.enso.compiler.context.{InlineContext, ModuleContext} -import org.enso.compiler.core.Implicits.AsDiagnostics -import org.enso.compiler.core.IR -import org.enso.compiler.core.ir.{Expression, Module, Name, Pattern} -import org.enso.compiler.core.ir.expression.{errors, warnings, Case} -import org.enso.compiler.core.CompilerError -import org.enso.compiler.pass.IRPass -import org.enso.compiler.pass.IRProcessingPass -import org.enso.compiler.pass.analyse.{ - AliasAnalysis, - DataflowAnalysis, - DemandAnalysis, - TailCall -} -import org.enso.compiler.pass.desugar.{GenerateMethodBodies, NestedPatternMatch} -import org.enso.compiler.pass.resolve.IgnoredBindings - -import scala.annotation.unused -import scala.collection.mutable - -/** This pass detects and renames shadowed pattern fields. - * - * This is necessary both in order to create a warning, but also to ensure that - * alias analysis doesn't get confused. - * - * This pass requires no configuration. - * - * This pass requires the context to provide: - * - * - Nothing - */ -case object ShadowedPatternFields extends IRPass { - override type Metadata = IRPass.Metadata.Empty - override type Config = IRPass.Configuration.Default - - override lazy val precursorPasses: Seq[IRProcessingPass] = List( - GenerateMethodBodies - ) - override lazy val invalidatedPasses: Seq[IRProcessingPass] = List( - AliasAnalysis, - DataflowAnalysis, - DemandAnalysis, - IgnoredBindings, - NestedPatternMatch, - TailCall.INSTANCE - ) - - /** Lints for shadowed pattern fields. - * - * @param ir the Enso IR to process - * @param moduleContext a context object that contains the information needed - * to process a module - * @return `ir`, possibly having made transformations or annotations to that - * IR. - */ - override def runModule( - ir: Module, - @unused moduleContext: ModuleContext - ): Module = { - ir.mapExpressions(lintExpression) - } - - /** Lints for shadowed pattern fields. - * - * @param ir the Enso IR to process - * @param inlineContext a context object that contains the information needed - * for inline evaluation - * @return `ir`, possibly having made transformations or annotations to that - * IR. - */ - override def runExpression( - ir: Expression, - @unused inlineContext: InlineContext - ): Expression = { - ir.transformExpressions { case x => - lintExpression(x) - } - } - - // === Pass Internals ======================================================= - - /** Lints for shadowed pattern variables on an arbitrary expression. - * - * @param expression the expression to lint - * @return `expression`, with warnings for any shadowed pattern variables - */ - private def lintExpression( - expression: Expression - ): Expression = { - expression.transformExpressions { case cse: Case => - lintCase(cse) - } - } - - /** Lints for shadowed pattern variables in a case expression. - * - * @param cse the expression to lint - * @return `cse`, with warnings for any shadowed pattern variables - */ - def lintCase(cse: Case): Case = { - cse match { - case expr @ Case.Expr(scrutinee, branches, _, _, _) => - expr.copy( - scrutinee = lintExpression(scrutinee), - branches = branches.map(lintCaseBranch) - ) - case _: Case.Branch => - throw new CompilerError("Unexpected case branch.") - } - } - - /** Lints for shadowed pattern variables in a case branch. - * - * @param branch the case branch to lint - * @return `branch`, with warnings for any shadowed pattern variables - */ - def lintCaseBranch( - branch: Case.Branch - ): Case.Branch = { - branch.copy( - pattern = lintPattern(branch.pattern), - expression = lintExpression(branch.expression) - ) - } - - /** Lints a pattern for shadowed pattern variables. - * - * A later pattern variable shadows an earlier pattern variable with the same - * name. - * - * @param pattern the pattern to lint - * @return `pattern`, with a warning applied to any shadowed pattern - * variables - */ - def lintPattern(pattern: Pattern): Pattern = { - val seenNames: mutable.Set[String] = mutable.Set() - val lastSeen: mutable.Map[String, IR] = mutable.Map() - - def go(pattern: Pattern, seenNames: mutable.Set[String]): Pattern = { - pattern match { - case named @ Pattern.Name(name, location, _) => - if (seenNames.contains(name.name)) { - val warning = warnings.Shadowed - .PatternBinding(name.name, lastSeen(name.name), location) - - lastSeen(name.name) = named - named - .copy( - name = Name.Blank(name.identifiedLocation()) - ) - .addDiagnostic(warning) - } else if (!name.isInstanceOf[Name.Blank]) { - lastSeen(name.name) = named - seenNames += name.name - named - } else { - named - } - case cons @ Pattern.Constructor(_, fields, _, _) => - val newFields = fields.reverse.map(go(_, seenNames)).reverse - - cons.copy( - fields = newFields - ) - case literal: Pattern.Literal => - literal - case typed @ Pattern.Type(name, _, location, _) => - if (seenNames.contains(name.name)) { - val warning = warnings.Shadowed - .PatternBinding(name.name, lastSeen(name.name), location) - - lastSeen(name.name) = typed - typed - .copy( - name = Name.Blank(name.identifiedLocation()) - ) - .addDiagnostic(warning) - } else if (!name.isInstanceOf[Name.Blank]) { - lastSeen(name.name) = typed - seenNames += name.name - typed - } else { - typed - } - case _: Pattern.Documentation => - throw new CompilerError( - "Branch documentation should be desugared at an earlier stage." - ) - case err: errors.Pattern => err - } - } - - go(pattern, seenNames) - } -} diff --git a/engine/runtime-compiler/src/main/scala/org/enso/compiler/pass/optimise/UnreachableMatchBranches.scala b/engine/runtime-compiler/src/main/scala/org/enso/compiler/pass/optimise/UnreachableMatchBranches.scala deleted file mode 100644 index 00f9ab6734ea..000000000000 --- a/engine/runtime-compiler/src/main/scala/org/enso/compiler/pass/optimise/UnreachableMatchBranches.scala +++ /dev/null @@ -1,193 +0,0 @@ -package org.enso.compiler.pass.optimise - -import org.enso.compiler.context.{InlineContext, ModuleContext} -import org.enso.compiler.core.Implicits.AsDiagnostics -import org.enso.compiler.core.ir.{ - Expression, - IdentifiedLocation, - Module, - Pattern -} -import org.enso.compiler.core.ir.expression.{errors, warnings, Case} -import org.enso.compiler.core.CompilerError -import org.enso.compiler.pass.IRPass -import org.enso.compiler.pass.IRProcessingPass -import org.enso.compiler.pass.analyse.{ - AliasAnalysis, - DataflowAnalysis, - DemandAnalysis, - TailCall -} -import org.enso.compiler.pass.desugar._ -import org.enso.compiler.pass.resolve.{DocumentationComments, IgnoredBindings} - -import scala.annotation.unused - -/** This pass discovers and optimised away unreachable case branches. - * - * It removes these unreachable expressions from the IR, and attaches a - * [[org.enso.compiler.core.ir.Warning]] diagnostic to the case expression itself. - * - * Currently, a branch is considered 'unreachable' by this pass if: - * - * - It occurs after a catch-all branch. - * - * In the future, this pass should be expanded to consider patterns that are - * entirely subsumed by previous patterns in its definition of uncreachable, - * but this requires doing sophisticated coverage analysis, and hence should - * happen as part of the broader refactor of nested patterns desugaring. - * - * This pass requires no configuration. - * - * This pass requires the context to provide: - * - * - Nothing - */ -case object UnreachableMatchBranches extends IRPass { - override type Metadata = IRPass.Metadata.Empty - override type Config = IRPass.Configuration.Default - - override lazy val precursorPasses: Seq[IRProcessingPass] = List( - ComplexType, - DocumentationComments, - FunctionBinding, - GenerateMethodBodies, - LambdaShorthandToLambda - ) - override lazy val invalidatedPasses: Seq[IRProcessingPass] = List( - AliasAnalysis, - DataflowAnalysis, - DemandAnalysis, - IgnoredBindings, - NestedPatternMatch, - TailCall.INSTANCE - ) - - /** Runs unreachable branch optimisation on a module. - * - * @param ir the Enso IR to process - * @param moduleContext a context object that contains the information needed - * to process a module - * @return `ir`, possibly having made transformations or annotations to that - * IR. - */ - override def runModule( - ir: Module, - @unused moduleContext: ModuleContext - ): Module = { - ir.mapExpressions(optimizeExpression) - } - - /** Runs unreachable branch optimisation on an expression. - * - * @param ir the Enso IR to process - * @param inlineContext a context object that contains the information needed - * for inline evaluation - * @return `ir`, possibly having made transformations or annotations to that - * IR. - */ - override def runExpression( - ir: Expression, - @unused inlineContext: InlineContext - ): Expression = { - ir.transformExpressions { case x => - optimizeExpression(x) - } - } - - // === Pass Internals ======================================================= - - /** Optimizes an expression by removing unreachable branches in case - * expressions. - * - * @param expression the expression to optimize - * @return `expression` with unreachable case branches removed - */ - private def optimizeExpression(expression: Expression): Expression = { - expression.transformExpressions { case cse: Case => - optimizeCase(cse) - } - } - - /** Optimizes a case expression by removing unreachable branches. - * - * Additionally, it will attach a warning about unreachable branches to the - * case expression. - * - * @param cse the case expression to optimize - * @return `cse` with unreachable branches removed - */ - //noinspection DuplicatedCode - private def optimizeCase(cse: Case): Case = { - cse match { - case expr @ Case.Expr(scrutinee, branches, _, _, _) => - val reachableNonCatchAllBranches = branches.takeWhile(!isCatchAll(_)) - val firstCatchAll = branches.find(isCatchAll) - val unreachableBranches = - branches.dropWhile(!isCatchAll(_)).drop(1) - val reachableBranches = firstCatchAll - .flatMap(b => Some(reachableNonCatchAllBranches :+ b)) - .getOrElse(List()) - .toList - - if (unreachableBranches.isEmpty) { - expr.copy( - scrutinee = optimizeExpression(scrutinee), - branches = branches.map(b => - b.copy(expression = optimizeExpression(b.expression)) - ) - ) - } else { - val unreachableLocation = - unreachableBranches.foldLeft(None: Option[IdentifiedLocation])( - (loc, branch) => { - loc match { - case Some(loc) => - branch.location match { - case Some(branchLoc) => - Some( - new IdentifiedLocation( - loc.start, - branchLoc.end, - loc.uuid - ) - ) - case None => Some(loc) - } - case None => branch.location - } - } - ) - - val diagnostic = - warnings.Unreachable.Branches(unreachableLocation.orNull) - - expr - .copy( - scrutinee = optimizeExpression(scrutinee), - branches = reachableBranches - .map(b => b.copy(expression = optimizeExpression(b.expression))) - ) - .addDiagnostic(diagnostic) - } - case _: Case.Branch => - throw new CompilerError("Unexpected case branch.") - } - } - - /** Determines if a branch is a catch all branch. - * - * @param branch the branch to check - * @return `true` if `branch` is catch-all, otherwise `false` - */ - private def isCatchAll(branch: Case.Branch): Boolean = { - branch.pattern match { - case _: Pattern.Name => true - case _: Pattern.Constructor => false - case _: Pattern.Literal => false - case _: Pattern.Type => false - case _: Pattern.Documentation => false - case _: errors.Pattern => true - } - } -} diff --git a/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/PassesTest.scala b/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/PassesTest.scala index 4f117401b4d7..07f32699d223 100644 --- a/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/PassesTest.scala +++ b/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/PassesTest.scala @@ -68,8 +68,8 @@ class PassesTest extends CompilerTest { AmbiguousImportsAnalysis.INSTANCE, PrivateModuleAnalysis.INSTANCE, PrivateConstructorAnalysis.INSTANCE, - ShadowedPatternFields, - UnreachableMatchBranches, + ShadowedPatternFields.INSTANCE, + UnreachableMatchBranches.INSTANCE, NestedPatternMatch, IgnoredBindings, TypeFunctions, diff --git a/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/pass/lint/ShadowedPatternFieldsTest.scala b/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/pass/lint/ShadowedPatternFieldsTest.scala index 6a2886adbe4d..1fd41a09eed4 100644 --- a/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/pass/lint/ShadowedPatternFieldsTest.scala +++ b/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/pass/lint/ShadowedPatternFieldsTest.scala @@ -5,7 +5,12 @@ import org.enso.compiler.context.{FreshNameSupply, InlineContext} import org.enso.compiler.core.ir.{Expression, Name, Pattern} import org.enso.compiler.core.ir.expression.{warnings, Case} import org.enso.compiler.pass.lint.ShadowedPatternFields -import org.enso.compiler.pass.{PassConfiguration, PassGroup, PassManager} +import org.enso.compiler.pass.{ + MiniIRPass, + PassConfiguration, + PassGroup, + PassManager +} import org.enso.compiler.test.CompilerTest class ShadowedPatternFieldsTest extends CompilerTest { @@ -15,7 +20,7 @@ class ShadowedPatternFieldsTest extends CompilerTest { val passes = new Passes(defaultConfig) val precursorPasses: PassGroup = - passes.getPrecursors(ShadowedPatternFields).get + passes.getPrecursors(ShadowedPatternFields.INSTANCE).get val passConfig: PassConfiguration = PassConfiguration() implicit val passManager: PassManager = @@ -34,7 +39,9 @@ class ShadowedPatternFieldsTest extends CompilerTest { * @return [[ir]], with shadowed pattern variables linted */ def lint(implicit inlineContext: InlineContext): Expression = { - ShadowedPatternFields.runExpression(ir, inlineContext) + val miniPass = + ShadowedPatternFields.INSTANCE.createForInlineCompilation(inlineContext) + MiniIRPass.compile(classOf[Expression], ir, miniPass) } } diff --git a/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/pass/optimise/UnreachableMatchBranchesTest.scala b/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/pass/optimise/UnreachableMatchBranchesTest.scala index 617ef04a1b15..0d8258151c2c 100644 --- a/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/pass/optimise/UnreachableMatchBranchesTest.scala +++ b/engine/runtime-integration-tests/src/test/scala/org/enso/compiler/test/pass/optimise/UnreachableMatchBranchesTest.scala @@ -5,7 +5,12 @@ import org.enso.compiler.context.{FreshNameSupply, InlineContext} import org.enso.compiler.core.ir.Expression import org.enso.compiler.core.ir.expression.{warnings, Case} import org.enso.compiler.pass.optimise.UnreachableMatchBranches -import org.enso.compiler.pass.{PassConfiguration, PassGroup, PassManager} +import org.enso.compiler.pass.{ + MiniIRPass, + PassConfiguration, + PassGroup, + PassManager +} import org.enso.compiler.test.CompilerTest class UnreachableMatchBranchesTest extends CompilerTest { @@ -15,7 +20,7 @@ class UnreachableMatchBranchesTest extends CompilerTest { val passes = new Passes(defaultConfig) val precursorPasses: PassGroup = - passes.getPrecursors(UnreachableMatchBranches).get + passes.getPrecursors(UnreachableMatchBranches.INSTANCE).get val passConfig: PassConfiguration = PassConfiguration() implicit val passManager: PassManager = @@ -34,7 +39,11 @@ class UnreachableMatchBranchesTest extends CompilerTest { * @return [[ir]] with unreachable case branches removed */ def optimize(implicit inlineContext: InlineContext): Expression = { - UnreachableMatchBranches.runExpression(ir, inlineContext) + val miniPass = + UnreachableMatchBranches.INSTANCE.createForInlineCompilation( + inlineContext + ) + MiniIRPass.compile(classOf[Expression], ir, miniPass) } } diff --git a/engine/runtime-parser/src/main/scala/org/enso/compiler/core/ir/Pattern.scala b/engine/runtime-parser/src/main/scala/org/enso/compiler/core/ir/Pattern.scala index 7986ff635034..9d75eeb1b60e 100644 --- a/engine/runtime-parser/src/main/scala/org/enso/compiler/core/ir/Pattern.scala +++ b/engine/runtime-parser/src/main/scala/org/enso/compiler/core/ir/Pattern.scala @@ -77,6 +77,12 @@ object Pattern { } else this } + def copyWithName( + name: IRName + ) = { + copy(name = name) + } + /** @inheritdoc */ override def duplicate( keepLocations: Boolean = true, @@ -180,6 +186,12 @@ object Pattern { } else this } + def copyWithFields( + fields: List[Pattern] + ): Constructor = { + copy(fields = fields) + } + /** @inheritdoc */ override def duplicate( keepLocations: Boolean = true, diff --git a/engine/runtime-parser/src/main/scala/org/enso/compiler/core/ir/expression/Case.scala b/engine/runtime-parser/src/main/scala/org/enso/compiler/core/ir/expression/Case.scala index 62b74e3f6e34..9758ba6f0535 100644 --- a/engine/runtime-parser/src/main/scala/org/enso/compiler/core/ir/expression/Case.scala +++ b/engine/runtime-parser/src/main/scala/org/enso/compiler/core/ir/expression/Case.scala @@ -84,6 +84,12 @@ object Case { } else this } + def copyWithBranches( + branches: Seq[Branch] + ): Expr = { + copy(branches = branches) + } + /** @inheritdoc */ override def duplicate( keepLocations: Boolean = true,