Eliminate references to Truffle nodes & co. in the compiler (#8172)

This commit is contained in:
Jaroslav Tulach 2023-10-30 10:57:21 +01:00 committed by GitHub
parent 7019de70b7
commit a862ea7948
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 215 additions and 208 deletions

View File

@ -14,16 +14,12 @@ import org.enso.compiler.ModuleCache;
import org.enso.compiler.PackageRepository; import org.enso.compiler.PackageRepository;
import org.enso.compiler.Passes; import org.enso.compiler.Passes;
import org.enso.compiler.SerializationManager; import org.enso.compiler.SerializationManager;
import org.enso.compiler.core.ir.Expression;
import org.enso.compiler.data.BindingsMap; import org.enso.compiler.data.BindingsMap;
import org.enso.compiler.data.CompilerConfig; import org.enso.compiler.data.CompilerConfig;
import org.enso.interpreter.node.ExpressionNode;
import org.enso.interpreter.runtime.data.Type;
import org.enso.interpreter.runtime.scope.LocalScope;
import org.enso.interpreter.runtime.scope.ModuleScope;
import org.enso.pkg.Package; import org.enso.pkg.Package;
import org.enso.pkg.QualifiedName; import org.enso.pkg.QualifiedName;
import org.enso.polyglot.CompilationStage; import org.enso.polyglot.CompilationStage;
import org.enso.polyglot.data.TypeGraph;
/** /**
* Interface that encapsulate all services {@link Compiler} needs from Truffle or other environment. * Interface that encapsulate all services {@link Compiler} needs from Truffle or other environment.
@ -63,12 +59,6 @@ public interface CompilerContext {
void truffleRunCodegen(Module module, CompilerConfig config) throws IOException; void truffleRunCodegen(Module module, CompilerConfig config) throws IOException;
void truffleRunCodegen(
Source source, ModuleScope scope, CompilerConfig config, org.enso.compiler.core.ir.Module ir);
ExpressionNode truffleRunInline(
Source source, LocalScope localScope, Module module, CompilerConfig config, Expression ir);
// module related // module related
void runStubsGenerator(Module module); void runStubsGenerator(Module module);
@ -103,6 +93,8 @@ public interface CompilerContext {
<T> Optional<TruffleFile> saveCache(Cache<T, ?> cache, T entry, boolean useGlobalCacheLocations); <T> Optional<TruffleFile> saveCache(Cache<T, ?> cache, T entry, boolean useGlobalCacheLocations);
TypeGraph getTypeHierarchy();
public static interface Updater { public static interface Updater {
void bindingsMap(BindingsMap map); void bindingsMap(BindingsMap map);
@ -128,12 +120,8 @@ public interface CompilerContext {
public abstract boolean isSameAs(org.enso.interpreter.runtime.Module m); public abstract boolean isSameAs(org.enso.interpreter.runtime.Module m);
public abstract org.enso.interpreter.runtime.scope.ModuleScope getScope();
public abstract QualifiedName getName(); public abstract QualifiedName getName();
public abstract Type findType(String name);
public abstract BindingsMap getBindingsMap(); public abstract BindingsMap getBindingsMap();
public abstract TruffleFile getSourceFile(); public abstract TruffleFile getSourceFile();

View File

@ -3,19 +3,18 @@ package org.enso.interpreter;
import org.enso.compiler.core.ConstantsNames; import org.enso.compiler.core.ConstantsNames;
/** Language-level constants for use throughout the program. */ /** Language-level constants for use throughout the program. */
public class Constants { public final class Constants {
private Constants() {}
public static final String SCOPE_SEPARATOR = "."; public static final String SCOPE_SEPARATOR = ".";
/** Names for different language elements. */ /** Names for different language elements. */
public static class Names { public static interface Names extends ConstantsNames {}
public static final String SELF_ARGUMENT = ConstantsNames.SELF_ARGUMENT;
public static final String SELF_TYPE_ARGUMENT = ConstantsNames.SELF_TYPE_ARGUMENT;
public static final String THAT_ARGUMENT = ConstantsNames.THAT_ARGUMENT;
public static final String FROM_MEMBER = ConstantsNames.FROM_MEMBER;
}
/** Cache sizes for different AST nodes. */ /** Cache sizes for different AST nodes. */
public static class CacheSizes { public static class CacheSizes {
private CacheSizes() {}
public static final String ARGUMENT_SORTER_NODE = "10"; public static final String ARGUMENT_SORTER_NODE = "10";
public static final String FUNCTION_INTEROP_LIBRARY = "10"; public static final String FUNCTION_INTEROP_LIBRARY = "10";
public static final String THUNK_EXECUTOR_NODE = "10"; public static final String THUNK_EXECUTOR_NODE = "10";

View File

@ -23,6 +23,8 @@ import org.enso.interpreter.node.EnsoRootNode;
import org.enso.interpreter.node.ExpressionNode; import org.enso.interpreter.node.ExpressionNode;
import org.enso.interpreter.node.ProgramRootNode; import org.enso.interpreter.node.ProgramRootNode;
import org.enso.interpreter.runtime.EnsoContext; import org.enso.interpreter.runtime.EnsoContext;
import org.enso.interpreter.runtime.IrToTruffle;
import org.enso.interpreter.runtime.scope.LocalScope;
import org.enso.interpreter.runtime.state.ExecutionEnvironment; import org.enso.interpreter.runtime.state.ExecutionEnvironment;
import org.enso.interpreter.runtime.tag.AvoidIdInstrumentationTag; import org.enso.interpreter.runtime.tag.AvoidIdInstrumentationTag;
import org.enso.interpreter.runtime.tag.IdentifiedTag; import org.enso.interpreter.runtime.tag.IdentifiedTag;
@ -260,13 +262,23 @@ public final class EnsoLanguage extends TruffleLanguage<EnsoContext> {
scala.Option.empty() scala.Option.empty()
); );
Compiler silentCompiler = context.getCompiler().duplicateWithConfig(redirectConfigWithStrictErrors); Compiler silentCompiler = context.getCompiler().duplicateWithConfig(redirectConfigWithStrictErrors);
scala.Option<ExpressionNode> exprNode; ExpressionNode exprNode;
try { try {
exprNode = silentCompiler var optionTupple = silentCompiler.runInline(
.runInline( request.getSource().getCharacters().toString(),
request.getSource().getCharacters().toString(), inlineContext
inlineContext );
); if (optionTupple.nonEmpty()) {
var newInlineContext = optionTupple.get()._1();
var ir = optionTupple.get()._2();
var sco = newInlineContext.localScope().getOrElse(LocalScope::root);
var mod = newInlineContext.module$access$0().module$access$0();
var m = org.enso.interpreter.runtime.Module.fromCompilerModule(mod);
var toTruffle = new IrToTruffle(context, request.getSource(), m.getScope(), redirectConfigWithStrictErrors);
exprNode = toTruffle.runInline(ir, sco, "<inline_source>");
} else {
exprNode = null;
}
} catch (UnhandledEntity e) { } catch (UnhandledEntity e) {
throw new InlineParsingException("Unhandled entity: " + e.entity(), e); throw new InlineParsingException("Unhandled entity: " + e.entity(), e);
} catch (CompilationAbortedException e) { } catch (CompilationAbortedException e) {
@ -277,8 +289,8 @@ public final class EnsoLanguage extends TruffleLanguage<EnsoContext> {
silentCompiler.shutdown(false); silentCompiler.shutdown(false);
} }
if (exprNode.isDefined()) { if (exprNode != null) {
var language = EnsoLanguage.get(exprNode.get()); var language = EnsoLanguage.get(exprNode);
return new ExecutableNode(language) { return new ExecutableNode(language) {
@Child @Child
private ExpressionNode expr; private ExpressionNode expr;
@ -286,7 +298,7 @@ public final class EnsoLanguage extends TruffleLanguage<EnsoContext> {
@Override @Override
public Object execute(VirtualFrame frame) { public Object execute(VirtualFrame frame) {
if (expr == null) { if (expr == null) {
expr = insert(exprNode.get()); expr = insert(exprNode);
} }
return expr.executeGeneric(frame); return expr.executeGeneric(frame);
} }

View File

@ -11,10 +11,10 @@ import org.enso.compiler.context.InlineContext;
import org.enso.interpreter.Constants; import org.enso.interpreter.Constants;
import org.enso.interpreter.node.BaseNode; import org.enso.interpreter.node.BaseNode;
import org.enso.interpreter.node.ClosureRootNode; import org.enso.interpreter.node.ClosureRootNode;
import org.enso.interpreter.node.ExpressionNode;
import org.enso.interpreter.node.callable.thunk.ThunkExecutorNode; import org.enso.interpreter.node.callable.thunk.ThunkExecutorNode;
import org.enso.interpreter.node.expression.builtin.text.util.ToJavaStringNode; import org.enso.interpreter.node.expression.builtin.text.util.ToJavaStringNode;
import org.enso.interpreter.runtime.EnsoContext; import org.enso.interpreter.runtime.EnsoContext;
import org.enso.interpreter.runtime.IrToTruffle;
import org.enso.interpreter.runtime.callable.CallerInfo; import org.enso.interpreter.runtime.callable.CallerInfo;
import org.enso.interpreter.runtime.callable.function.Function; import org.enso.interpreter.runtime.callable.function.Function;
import org.enso.interpreter.runtime.data.text.Text; import org.enso.interpreter.runtime.data.text.Text;
@ -66,12 +66,26 @@ public abstract class EvalNode extends BaseNode {
LocalScope localScope = scope.createChild(); LocalScope localScope = scope.createChild();
InlineContext inlineContext = InlineContext inlineContext =
InlineContext.fromJava( InlineContext.fromJava(
localScope, moduleScope, getTailStatus(), context.getCompilerConfig()); localScope,
ExpressionNode expr = moduleScope.getModule().asCompilerModule(),
context.getCompiler().runInline(expression, inlineContext).getOrElse(() -> null); scala.Option.apply(getTailStatus() != TailStatus.NOT_TAIL),
if (expr == null) { context.getCompilerConfig());
var compiler = context.getCompiler();
var tuppleOption = compiler.runInline(expression, inlineContext);
if (tuppleOption.isEmpty()) {
throw new RuntimeException("Invalid code passed to `eval`: " + expression); throw new RuntimeException("Invalid code passed to `eval`: " + expression);
} }
var newInlineContext = tuppleOption.get()._1();
var ir = tuppleOption.get()._2();
var src = tuppleOption.get()._3();
var sco = newInlineContext.localScope().getOrElse(LocalScope::root);
var mod = newInlineContext.module$access$0().module$access$0();
var m = org.enso.interpreter.runtime.Module.fromCompilerModule(mod);
var toTruffle =
new IrToTruffle(context, src, m.getScope(), compiler.org$enso$compiler$Compiler$$config);
var expr = toTruffle.runInline(ir, sco, "<inline_source>");
if (shouldCaptureResultScope) { if (shouldCaptureResultScope) {
expr = CaptureResultScopeNode.build(expr); expr = CaptureResultScopeNode.build(expr);

View File

@ -1,8 +1,6 @@
package org.enso.interpreter.runtime; package org.enso.interpreter.runtime;
import org.enso.compiler.pass.analyse.BindingAnalysis$; import org.enso.compiler.pass.analyse.BindingAnalysis$;
import org.enso.compiler.codegen.IrToTruffle;
import org.enso.compiler.codegen.RuntimeStubsGenerator;
import org.enso.compiler.context.CompilerContext; import org.enso.compiler.context.CompilerContext;
import org.enso.compiler.context.FreshNameSupply; import org.enso.compiler.context.FreshNameSupply;
@ -30,10 +28,11 @@ import org.enso.compiler.data.CompilerConfig;
import org.enso.interpreter.node.ExpressionNode; import org.enso.interpreter.node.ExpressionNode;
import org.enso.interpreter.runtime.data.Type; import org.enso.interpreter.runtime.data.Type;
import org.enso.interpreter.runtime.scope.LocalScope; import org.enso.interpreter.runtime.scope.LocalScope;
import org.enso.interpreter.runtime.scope.ModuleScope; import org.enso.interpreter.runtime.type.Types;
import org.enso.pkg.Package; import org.enso.pkg.Package;
import org.enso.pkg.QualifiedName; import org.enso.pkg.QualifiedName;
import org.enso.polyglot.CompilationStage; import org.enso.polyglot.CompilationStage;
import org.enso.polyglot.data.TypeGraph;
import scala.Option; import scala.Option;
@ -113,18 +112,8 @@ final class TruffleCompilerContext implements CompilerContext {
@Override @Override
public void truffleRunCodegen(CompilerContext.Module module, CompilerConfig config) throws IOException { public void truffleRunCodegen(CompilerContext.Module module, CompilerConfig config) throws IOException {
truffleRunCodegen(module.getSource(), module.getScope(), config, module.getIr()); var m = org.enso.interpreter.runtime.Module.fromCompilerModule(module);
} new IrToTruffle(context, module.getSource(), m.getScope(), config).run(module.getIr());
@Override
public void truffleRunCodegen(Source source, ModuleScope scope, CompilerConfig config, org.enso.compiler.core.ir.Module ir) {
new IrToTruffle(context, source, scope, config).run(ir);
}
@Override
public ExpressionNode truffleRunInline(Source source, LocalScope localScope, CompilerContext.Module module, CompilerConfig config, Expression ir) {
return new IrToTruffle(context, source, module.getScope(), config)
.runInline(ir, localScope, "<inline_source>");
} }
// module related // module related
@ -168,6 +157,11 @@ final class TruffleCompilerContext implements CompilerContext {
return module.getCompilationStage(); return module.getCompilationStage();
} }
@Override
public TypeGraph getTypeHierarchy() {
return Types.getTypeHierarchy();
}
@Override @Override
public void updateModule(CompilerContext.Module module, Consumer<Updater> callback) { public void updateModule(CompilerContext.Module module, Consumer<Updater> callback) {
try (var u = new ModuleUpdater((Module)module)) { try (var u = new ModuleUpdater((Module)module)) {
@ -369,21 +363,11 @@ final class TruffleCompilerContext implements CompilerContext {
return module == m; return module == m;
} }
// XXX
public org.enso.interpreter.runtime.scope.ModuleScope getScope() {
return module.getScope();
}
@Override @Override
public QualifiedName getName() { public QualifiedName getName() {
return module.getName(); return module.getName();
} }
@Override
public Type findType(String name) {
return module.getScope().getTypes().get(name);
}
@Override @Override
public BindingsMap getBindingsMap() { public BindingsMap getBindingsMap() {
if (module.getIr() != null) { if (module.getIr() != null) {

View File

@ -34,8 +34,6 @@ import org.enso.compiler.phase.{
ImportResolver ImportResolver
} }
import org.enso.editions.LibraryName import org.enso.editions.LibraryName
import org.enso.interpreter.node.{ExpressionNode => RuntimeExpression}
import org.enso.interpreter.runtime.scope.ModuleScope
import org.enso.pkg.QualifiedName import org.enso.pkg.QualifiedName
import org.enso.polyglot.LanguageInfo import org.enso.polyglot.LanguageInfo
import org.enso.polyglot.CompilationStage import org.enso.polyglot.CompilationStage
@ -691,7 +689,7 @@ class Compiler(
def runInline( def runInline(
srcString: String, srcString: String,
inlineContext: InlineContext inlineContext: InlineContext
): Option[RuntimeExpression] = { ): Option[(InlineContext, Expression, Source)] = {
val newContext = inlineContext.copy(freshNameSupply = Some(freshNameSupply)) val newContext = inlineContext.copy(freshNameSupply = Some(freshNameSupply))
val source = Source val source = Source
.newBuilder( .newBuilder(
@ -705,7 +703,7 @@ class Compiler(
ensoCompiler.generateIRInline(tree).flatMap { ir => ensoCompiler.generateIRInline(tree).flatMap { ir =>
val compilerOutput = runCompilerPhasesInline(ir, newContext) val compilerOutput = runCompilerPhasesInline(ir, newContext)
runErrorHandlingInline(compilerOutput, source, newContext) runErrorHandlingInline(compilerOutput, source, newContext)
Some(newContext.truffleRunInline(context, source, config, compilerOutput)) Some((newContext, compilerOutput, source))
} }
} }
@ -722,7 +720,7 @@ class Compiler(
qualifiedName: String, qualifiedName: String,
loc: Option[IdentifiedLocation], loc: Option[IdentifiedLocation],
source: Source source: Source
): ModuleScope = { ): Unit = {
val module = Option(context.findTopScopeModule(qualifiedName)) val module = Option(context.findTopScopeModule(qualifiedName))
.getOrElse { .getOrElse {
val locStr = fileLocationFromSection(loc, source) val locStr = fileLocationFromSection(loc, source)
@ -740,7 +738,6 @@ class Compiler(
"Trying to use a module in codegen without generating runtime stubs" "Trying to use a module in codegen without generating runtime stubs"
) )
} }
module.getScope
} }
/** Parses the given source with the new Rust parser. /** Parses the given source with the new Rust parser.
@ -1231,20 +1228,6 @@ class Compiler(
source.getPath + ":" + srcLocation source.getPath + ":" + srcLocation
} }
/** Generates code for the truffle interpreter.
*
* @param ir the program to translate
* @param source the source code of the program represented by `ir`
* @param scope the module scope in which the code is to be generated
*/
def truffleCodegen(
ir: IRModule,
source: Source,
scope: ModuleScope
): Unit = {
context.truffleRunCodegen(source, scope, config, ir)
}
/** Performs shutdown actions for the compiler. /** Performs shutdown actions for the compiler.
* *
* @param waitForPendingJobCompletion whether or not shutdown should wait for * @param waitForPendingJobCompletion whether or not shutdown should wait for

View File

@ -1,13 +1,9 @@
package org.enso.compiler.context package org.enso.compiler.context
import org.enso.compiler.core.ir.Expression
import org.enso.compiler.PackageRepository import org.enso.compiler.PackageRepository
import org.enso.compiler.data.CompilerConfig import org.enso.compiler.data.CompilerConfig
import org.enso.compiler.pass.PassConfiguration import org.enso.compiler.pass.PassConfiguration
import org.enso.interpreter.node.BaseNode.TailStatus import org.enso.interpreter.runtime.scope.LocalScope
import org.enso.interpreter.runtime.scope.{LocalScope, ModuleScope}
import org.enso.interpreter.node.ExpressionNode
import com.oracle.truffle.api.source.Source
/** A type containing the information about the execution context for an inline /** A type containing the information about the execution context for an inline
* expression. * expression.
@ -31,16 +27,6 @@ case class InlineContext(
pkgRepo: Option[PackageRepository] = None pkgRepo: Option[PackageRepository] = None
) { ) {
def bindingsAnalysis() = module.bindingsAnalysis() def bindingsAnalysis() = module.bindingsAnalysis()
def truffleRunInline(
context: CompilerContext,
source: Source,
config: CompilerConfig,
ir: Expression
): ExpressionNode = {
val s = localScope.getOrElse(LocalScope.root)
return module.truffleRunInline(context, source, s, config, ir)
}
} }
object InlineContext { object InlineContext {
@ -48,24 +34,21 @@ object InlineContext {
* internally. * internally.
* *
* @param localScope the local scope instance * @param localScope the local scope instance
* @param moduleScope the module scope instance * @param module the module defining the context
* @param isInTailPosition whether or not the inline expression occurs in a * @param isInTailPosition whether or not the inline expression occurs in a
* tail position * tail position
* @return the [[InlineContext]] instance corresponding to the arguments * @return the [[InlineContext]] instance corresponding to the arguments
*/ */
def fromJava( def fromJava(
localScope: LocalScope, localScope: LocalScope,
moduleScope: ModuleScope, module: CompilerContext.Module,
isInTailPosition: TailStatus, isInTailPosition: Option[Boolean],
compilerConfig: CompilerConfig compilerConfig: CompilerConfig
): InlineContext = { ): InlineContext = {
InlineContext( InlineContext(
localScope = Option(localScope), localScope = Option(localScope),
module = ModuleContext( module = ModuleContext(module, compilerConfig),
moduleScope.getModule().asCompilerModule(), isInTailPosition = isInTailPosition,
compilerConfig
),
isInTailPosition = Option(isInTailPosition != TailStatus.NOT_TAIL),
compilerConfig = compilerConfig compilerConfig = compilerConfig
) )
} }

View File

@ -1,13 +1,10 @@
package org.enso.compiler.context package org.enso.compiler.context
import org.enso.compiler.core.ir.Expression
import org.enso.compiler.PackageRepository import org.enso.compiler.PackageRepository
import org.enso.compiler.data.CompilerConfig import org.enso.compiler.data.CompilerConfig
import org.enso.compiler.pass.PassConfiguration import org.enso.compiler.pass.PassConfiguration
import org.enso.pkg.Package; import org.enso.pkg.Package;
import org.enso.pkg.QualifiedName; import org.enso.pkg.QualifiedName;
import org.enso.interpreter.runtime.scope.LocalScope
import org.enso.interpreter.node.ExpressionNode
import com.oracle.truffle.api.source.Source import com.oracle.truffle.api.source.Source
import org.enso.compiler.data.BindingsMap.ModuleReference import org.enso.compiler.data.BindingsMap.ModuleReference
@ -28,17 +25,8 @@ case class ModuleContext(
isGeneratingDocs: Boolean = false, isGeneratingDocs: Boolean = false,
pkgRepo: Option[PackageRepository] = None pkgRepo: Option[PackageRepository] = None
) { ) {
def isSynthetic() = module.isSynthetic() def isSynthetic() = module.isSynthetic()
def bindingsAnalysis() = module.getBindingsMap() def bindingsAnalysis() = module.getBindingsMap()
def truffleRunInline(
context: CompilerContext,
source: Source,
s: LocalScope,
config: CompilerConfig,
ir: Expression
): ExpressionNode = {
return context.truffleRunInline(source, s, module, config, ir)
}
def getName(): QualifiedName = module.getName() def getName(): QualifiedName = module.getName()
def getPackage(): Package[_] = module.getPackage() def getPackage(): Package[_] = module.getPackage()
def getSource(): Source = module.getSource() def getSource(): Source = module.getSource()

View File

@ -25,7 +25,6 @@ import org.enso.compiler.pass.resolve.{
TypeNames, TypeNames,
TypeSignatures TypeSignatures
} }
import org.enso.interpreter.runtime.`type`.Types
import org.enso.pkg.QualifiedName import org.enso.pkg.QualifiedName
import org.enso.polyglot.Suggestion import org.enso.polyglot.Suggestion
import org.enso.polyglot.data.{Tree, TypeGraph} import org.enso.polyglot.data.{Tree, TypeGraph}
@ -793,7 +792,11 @@ object SuggestionBuilder {
source: A, source: A,
compiler: Compiler compiler: Compiler
): SuggestionBuilder[A] = ): SuggestionBuilder[A] =
new SuggestionBuilder[A](source, Types.getTypeHierarchy, compiler) new SuggestionBuilder[A](
source,
compiler.context.getTypeHierarchy(),
compiler
)
/** A single level of an `IR`. /** A single level of an `IR`.
* *

View File

@ -787,9 +787,6 @@ object BindingsMap {
override lazy val exportedSymbols: Map[String, List[ResolvedName]] = override lazy val exportedSymbols: Map[String, List[ResolvedName]] =
tp.members.map(m => (m.name, List(ResolvedConstructor(this, m)))).toMap tp.members.map(m => (m.name, List(ResolvedConstructor(this, m)))).toMap
def unsafeToRuntimeType(): org.enso.interpreter.runtime.data.Type =
module.unsafeAsModule().findType(tp.name)
} }
/** A result of successful name resolution. /** A result of successful name resolution.

View File

@ -68,7 +68,7 @@ import scala.reflect.ClassTag
* *
* - A [[org.enso.compiler.pass.PassConfiguration]] containing an instance of * - A [[org.enso.compiler.pass.PassConfiguration]] containing an instance of
* [[AliasAnalysis.Configuration]]. * [[AliasAnalysis.Configuration]].
* - A [[org.enso.interpreter.runtime.scope.LocalScope]], where relevant. * - A [[LocalScope]], where relevant.
*/ */
case object AliasAnalysis extends IRPass { case object AliasAnalysis extends IRPass {

View File

@ -42,7 +42,7 @@ import scala.collection.mutable
* *
* This pass requires the context to provide: * This pass requires the context to provide:
* *
* - A [[org.enso.interpreter.runtime.scope.LocalScope]], where relevant. * - A [[LocalScope]], where relevant.
* *
* It requires that all members of [[org.enso.compiler.core.ir.IRKind.Primitive]] have been removed * It requires that all members of [[org.enso.compiler.core.ir.IRKind.Primitive]] have been removed
* from the IR by the time it runs. * from the IR by the time it runs.

View File

@ -1,6 +1,7 @@
package org.enso.compiler.pass.desugar package org.enso.compiler.pass.desugar
import org.enso.compiler.context.{InlineContext, ModuleContext} import org.enso.compiler.context.{InlineContext, ModuleContext}
import org.enso.compiler.core.ConstantsNames
import org.enso.compiler.core.Implicits.AsMetadata import org.enso.compiler.core.Implicits.AsMetadata
import org.enso.compiler.core.ir.module.scope.Definition import org.enso.compiler.core.ir.module.scope.Definition
import org.enso.compiler.core.ir.module.scope.definition import org.enso.compiler.core.ir.module.scope.definition
@ -24,7 +25,6 @@ import org.enso.compiler.pass.analyse.{
} }
import org.enso.compiler.pass.optimise.LambdaConsolidate import org.enso.compiler.pass.optimise.LambdaConsolidate
import org.enso.compiler.pass.resolve.IgnoredBindings import org.enso.compiler.pass.resolve.IgnoredBindings
import org.enso.interpreter.Constants
/** This pass handles the desugaring of long-form function and method /** This pass handles the desugaring of long-form function and method
* definitions into standard bindings using lambdas. * definitions into standard bindings using lambdas.
@ -185,7 +185,7 @@ case object FunctionBinding extends IRPass {
else else
Name Name
.Literal( .Literal(
Constants.Names.THAT_ARGUMENT, ConstantsNames.THAT_ARGUMENT,
firstArgumentName.isMethod, firstArgumentName.isMethod,
firstArgumentName.location firstArgumentName.location
) )
@ -203,7 +203,7 @@ case object FunctionBinding extends IRPass {
if (sndArgName.isInstanceOf[Name.Blank]) { if (sndArgName.isInstanceOf[Name.Blank]) {
val newName = Name val newName = Name
.Literal( .Literal(
Constants.Names.THAT_ARGUMENT, ConstantsNames.THAT_ARGUMENT,
sndArgName.isMethod, sndArgName.isMethod,
sndArgName.location sndArgName.location
) )
@ -217,7 +217,7 @@ case object FunctionBinding extends IRPass {
), ),
rest rest
) )
} else if (snd.name.name != Constants.Names.THAT_ARGUMENT) { } else if (snd.name.name != ConstantsNames.THAT_ARGUMENT) {
(None, restArgs) (None, restArgs)
} else { } else {
(Some(snd), rest) (Some(snd), rest)
@ -230,7 +230,7 @@ case object FunctionBinding extends IRPass {
remainingArgs: List[DefinitionArgument] remainingArgs: List[DefinitionArgument]
): Either[Error, definition.Method] = { ): Either[Error, definition.Method] = {
remaining remaining
.filter(_.name.name != Constants.Names.SELF_ARGUMENT) .filter(_.name.name != ConstantsNames.SELF_ARGUMENT)
.find(_.defaultValue.isEmpty) match { .find(_.defaultValue.isEmpty) match {
case Some(nonDefaultedArg) => case Some(nonDefaultedArg) =>
Left( Left(
@ -262,9 +262,9 @@ case object FunctionBinding extends IRPass {
val failures = sndArgument match { val failures = sndArgument match {
case Some(newSndArgument) => case Some(newSndArgument) =>
if ( if (
newFirstArgument.name.name == Constants.Names.SELF_ARGUMENT newFirstArgument.name.name == ConstantsNames.SELF_ARGUMENT
) { ) {
if (newSndArgument.name.name != Constants.Names.THAT_ARGUMENT) if (newSndArgument.name.name != ConstantsNames.THAT_ARGUMENT)
Left( Left(
errors.Conversion( errors.Conversion(
newSndArgument, newSndArgument,
@ -275,7 +275,7 @@ case object FunctionBinding extends IRPass {
) )
else Right(()) else Right(())
} else if ( } else if (
newFirstArgument.name.name != Constants.Names.THAT_ARGUMENT newFirstArgument.name.name != ConstantsNames.THAT_ARGUMENT
) { ) {
Left( Left(
errors.Conversion( errors.Conversion(
@ -288,7 +288,7 @@ case object FunctionBinding extends IRPass {
} else Right(()) } else Right(())
case None => case None =>
if ( if (
newFirstArgument.name.name != Constants.Names.THAT_ARGUMENT newFirstArgument.name.name != ConstantsNames.THAT_ARGUMENT
) { ) {
Left( Left(
errors.Conversion( errors.Conversion(

View File

@ -23,10 +23,10 @@ import org.enso.compiler.data.BindingsMap.{
ResolvedModule ResolvedModule
} }
import org.enso.compiler.core.CompilerError import org.enso.compiler.core.CompilerError
import org.enso.compiler.core.ConstantsNames
import org.enso.compiler.core.ir.expression.Application import org.enso.compiler.core.ir.expression.Application
import org.enso.compiler.pass.IRPass import org.enso.compiler.pass.IRPass
import org.enso.compiler.pass.analyse.{AliasAnalysis, BindingAnalysis} import org.enso.compiler.pass.analyse.{AliasAnalysis, BindingAnalysis}
import org.enso.interpreter.Constants
/** Resolves name occurences in non-pattern contexts. /** Resolves name occurences in non-pattern contexts.
* *
@ -401,7 +401,7 @@ case object GlobalNames extends IRPass {
private def findThisPosition(args: List[CallArgument]): Option[Int] = { private def findThisPosition(args: List[CallArgument]): Option[Int] = {
val ix = args.indexWhere(arg => val ix = args.indexWhere(arg =>
arg.name.exists( arg.name.exists(
_.name == Constants.Names.SELF_ARGUMENT _.name == ConstantsNames.SELF_ARGUMENT
) || arg.name.isEmpty ) || arg.name.isEmpty
) )
if (ix == -1) None else Some(ix) if (ix == -1) None else Some(ix)

View File

@ -1,8 +1,10 @@
package org.enso.compiler.codegen package org.enso.interpreter.runtime
import com.oracle.truffle.api.source.{Source, SourceSection} import com.oracle.truffle.api.source.{Source, SourceSection}
import com.oracle.truffle.api.interop.InteropLibrary import com.oracle.truffle.api.interop.InteropLibrary
import org.enso.compiler.context.CompilerContext
import org.enso.compiler.core.CompilerError import org.enso.compiler.core.CompilerError
import org.enso.compiler.core.ConstantsNames
import org.enso.compiler.core.Implicits.AsMetadata import org.enso.compiler.core.Implicits.AsMetadata
import org.enso.compiler.core.IR import org.enso.compiler.core.IR
import org.enso.compiler.core.ir.{ import org.enso.compiler.core.ir.{
@ -197,7 +199,12 @@ class IrToTruffle(
module module
} }
.foreach { exp => .foreach { exp =>
moduleScope.addExport(exp.unsafeAsModule().getScope) moduleScope.addExport(
asScope(
exp
.unsafeAsModule()
)
)
} }
val importDefs = module.imports val importDefs = module.imports
val methodDefs = module.bindings.collect { val methodDefs = module.bindings.collect {
@ -208,17 +215,18 @@ class IrToTruffle(
imp.target match { imp.target match {
case BindingsMap.ResolvedType(_, _) => case BindingsMap.ResolvedType(_, _) =>
case ResolvedModule(module) => case ResolvedModule(module) =>
val mod = module.unsafeAsModule() val mod = module
.unsafeAsModule()
val scope: ModuleScope = imp.importDef.onlyNames val scope: ModuleScope = imp.importDef.onlyNames
.map(only => { .map(only => {
val requestedTypes = only.map(_.name).asJava val requestedTypes = only.map(_.name).asJava
if (requestedTypes.isEmpty()) { if (requestedTypes.isEmpty()) {
mod.getScope() asScope(mod)
} else { } else {
mod.getScope().withTypes(requestedTypes) asScope(mod).withTypes(requestedTypes)
} }
}) })
.getOrElse(mod.getScope()) .getOrElse(asScope(mod))
moduleScope.addImport(scope) moduleScope.addImport(scope)
} }
} }
@ -244,10 +252,10 @@ class IrToTruffle(
val typeDefs = module.bindings.collect { case tp: Definition.Type => tp } val typeDefs = module.bindings.collect { case tp: Definition.Type => tp }
typeDefs.foreach { tpDef => typeDefs.foreach { tpDef =>
// Register the atoms and their constructors in scope // Register the atoms and their constructors in scope
val atomDefs = tpDef.members val atomDefs = tpDef.members
val runtimeType = moduleScope.getTypes.get(tpDef.name.name) val asType = moduleScope.getTypes.get(tpDef.name.name)
val atomConstructors = val atomConstructors =
atomDefs.map(cons => runtimeType.getConstructors.get(cons.name.name)) atomDefs.map(cons => asType.getConstructors.get(cons.name.name))
atomConstructors atomConstructors
.zip(atomDefs) .zip(atomDefs)
.foreach { case (atomCons, atomDefn) => .foreach { case (atomCons, atomDefn) =>
@ -280,7 +288,7 @@ class IrToTruffle(
for (idx <- atomDefn.arguments.indices) { for (idx <- atomDefn.arguments.indices) {
val unprocessedArg = atomDefn.arguments(idx) val unprocessedArg = atomDefn.arguments(idx)
val checkNode = checkRuntimeTypes(unprocessedArg) val checkNode = checkAsTypes(unprocessedArg)
val arg = argFactory.run(unprocessedArg, idx, checkNode) val arg = argFactory.run(unprocessedArg, idx, checkNode)
val occInfo = unprocessedArg val occInfo = unprocessedArg
.unsafeGetMetadata( .unsafeGetMetadata(
@ -345,7 +353,7 @@ class IrToTruffle(
) )
} }
} }
runtimeType.generateGetters(language) asType.generateGetters(language)
} }
// Register the method definitions in scope // Register the method definitions in scope
@ -387,9 +395,16 @@ class IrToTruffle(
.map { res => .map { res =>
res.target match { res.target match {
case BindingsMap.ResolvedType(module, tp) => case BindingsMap.ResolvedType(module, tp) =>
module.unsafeAsModule().getScope.getTypes.get(tp.name) asScope(
module
.unsafeAsModule()
).getTypes
.get(tp.name)
case BindingsMap.ResolvedModule(module) => case BindingsMap.ResolvedModule(module) =>
module.unsafeAsModule().getScope.getAssociatedType asScope(
module
.unsafeAsModule()
).getAssociatedType
case BindingsMap.ResolvedConstructor(_, _) => case BindingsMap.ResolvedConstructor(_, _) =>
throw new CompilerError( throw new CompilerError(
"Impossible, should be caught by MethodDefinitions pass" "Impossible, should be caught by MethodDefinitions pass"
@ -717,7 +732,12 @@ class IrToTruffle(
) => ) =>
ReadArgumentCheckNode.build( ReadArgumentCheckNode.build(
name, name,
mod.unsafeAsModule().getScope.getTypes.get(tpe.name) asScope(
mod
.unsafeAsModule()
.asInstanceOf[TruffleCompilerContext.Module]
).getTypes
.get(tpe.name)
) )
case Some( case Some(
BindingsMap BindingsMap
@ -725,14 +745,18 @@ class IrToTruffle(
) => ) =>
ReadArgumentCheckNode.meta( ReadArgumentCheckNode.meta(
name, name,
mod.unsafeAsModule().getScope.getPolyglotSymbol(symbol.name) asScope(
mod
.unsafeAsModule()
.asInstanceOf[TruffleCompilerContext.Module]
).getPolyglotSymbol(symbol.name)
) )
case _ => null case _ => null
} }
} }
} }
private def checkRuntimeTypes( private def checkAsTypes(
arg: DefinitionArgument arg: DefinitionArgument
): ReadArgumentCheckNode = { ): ReadArgumentCheckNode = {
arg.ascribedType.map(extractAscribedType(arg.name, _)).getOrElse(null) arg.ascribedType.map(extractAscribedType(arg.name, _)).getOrElse(null)
@ -777,13 +801,16 @@ class IrToTruffle(
expr.getMetadata(MethodDefinitions).map { res => expr.getMetadata(MethodDefinitions).map { res =>
res.target match { res.target match {
case BindingsMap.ResolvedType(definitionModule, tp) => case BindingsMap.ResolvedType(definitionModule, tp) =>
definitionModule asScope(
.unsafeAsModule() definitionModule
.getScope .unsafeAsModule()
.getTypes ).getTypes
.get(tp.name) .get(tp.name)
case BindingsMap.ResolvedModule(module) => case BindingsMap.ResolvedModule(module) =>
module.unsafeAsModule().getScope.getAssociatedType asScope(
module
.unsafeAsModule()
).getAssociatedType
case BindingsMap.ResolvedConstructor(_, _) => case BindingsMap.ResolvedConstructor(_, _) =>
throw new CompilerError( throw new CompilerError(
"Impossible here, should be caught by MethodDefinitions pass." "Impossible here, should be caught by MethodDefinitions pass."
@ -847,7 +874,7 @@ class IrToTruffle(
new FunctionSchema( new FunctionSchema(
new ArgumentDefinition( new ArgumentDefinition(
0, 0,
Constants.Names.SELF_ARGUMENT, ConstantsNames.SELF_ARGUMENT,
null, null,
null, null,
ArgumentDefinition.ExecutionMode.EXECUTE ArgumentDefinition.ExecutionMode.EXECUTE
@ -863,7 +890,7 @@ class IrToTruffle(
new FunctionSchema( new FunctionSchema(
new ArgumentDefinition( new ArgumentDefinition(
0, 0,
Constants.Names.SELF_ARGUMENT, ConstantsNames.SELF_ARGUMENT,
null, null,
null, null,
ArgumentDefinition.ExecutionMode.EXECUTE ArgumentDefinition.ExecutionMode.EXECUTE
@ -886,7 +913,11 @@ class IrToTruffle(
resolution match { resolution match {
case BindingsMap.ResolvedType(module, tp) => case BindingsMap.ResolvedType(module, tp) =>
val runtimeTp = val runtimeTp =
module.unsafeAsModule().getScope.getTypes.get(tp.name) asScope(
module
.unsafeAsModule()
).getTypes
.get(tp.name)
val fun = mkTypeGetter(runtimeTp) val fun = mkTypeGetter(runtimeTp)
moduleScope.registerMethod( moduleScope.registerMethod(
moduleScope.getAssociatedType, moduleScope.getAssociatedType,
@ -894,9 +925,7 @@ class IrToTruffle(
fun fun
) )
case BindingsMap.ResolvedConstructor(definitionType, cons) => case BindingsMap.ResolvedConstructor(definitionType, cons) =>
val runtimeCons = definitionType val runtimeCons = asType(definitionType).getConstructors
.unsafeToRuntimeType()
.getConstructors
.get(cons.name) .get(cons.name)
val fun = mkConsGetter(runtimeCons) val fun = mkConsGetter(runtimeCons)
moduleScope.registerMethod( moduleScope.registerMethod(
@ -906,7 +935,10 @@ class IrToTruffle(
) )
case BindingsMap.ResolvedModule(module) => case BindingsMap.ResolvedModule(module) =>
val runtimeCons = val runtimeCons =
module.unsafeAsModule().getScope.getAssociatedType asScope(
module
.unsafeAsModule()
).getAssociatedType
val fun = mkTypeGetter(runtimeCons) val fun = mkTypeGetter(runtimeCons)
moduleScope.registerMethod( moduleScope.registerMethod(
moduleScope.getAssociatedType, moduleScope.getAssociatedType,
@ -915,11 +947,10 @@ class IrToTruffle(
) )
case BindingsMap.ResolvedMethod(module, method) => case BindingsMap.ResolvedMethod(module, method) =>
val actualModule = module.unsafeAsModule() val actualModule = module.unsafeAsModule()
val fun = actualModule.getScope val fun = asScope(actualModule).getMethodForType(
.getMethodForType( asScope(actualModule).getAssociatedType,
actualModule.getScope.getAssociatedType, method.name
method.name )
)
assert( assert(
fun != null, fun != null,
s"exported symbol `${method.name}` needs to be registered first in the module " s"exported symbol `${method.name}` needs to be registered first in the module "
@ -1229,7 +1260,10 @@ class IrToTruffle(
Right( Right(
ObjectEqualityBranchNode.build( ObjectEqualityBranchNode.build(
branchCodeNode.getCallTarget, branchCodeNode.getCallTarget,
mod.unsafeAsModule().getScope.getAssociatedType, asScope(
mod
.unsafeAsModule()
).getAssociatedType,
branch.terminalBranch branch.terminalBranch
) )
) )
@ -1239,7 +1273,7 @@ class IrToTruffle(
) )
) => ) =>
val atomCons = val atomCons =
tp.unsafeToRuntimeType().getConstructors.get(cons.name) asType(tp).getConstructors.get(cons.name)
val r = if (atomCons == context.getBuiltins.bool().getTrue) { val r = if (atomCons == context.getBuiltins.bool().getTrue) {
BooleanBranchNode.build( BooleanBranchNode.build(
true, true,
@ -1264,7 +1298,11 @@ class IrToTruffle(
BindingsMap.Resolution(BindingsMap.ResolvedType(mod, tp)) BindingsMap.Resolution(BindingsMap.ResolvedType(mod, tp))
) => ) =>
val tpe = val tpe =
mod.unsafeAsModule().getScope.getTypes.get(tp.name) asScope(
mod
.unsafeAsModule()
).getTypes
.get(tp.name)
val polyglot = context.getBuiltins.polyglot val polyglot = context.getBuiltins.polyglot
val branchNode = if (tpe == polyglot) { val branchNode = if (tpe == polyglot) {
PolyglotBranchNode.build( PolyglotBranchNode.build(
@ -1285,10 +1323,10 @@ class IrToTruffle(
BindingsMap.ResolvedPolyglotSymbol(mod, symbol) BindingsMap.ResolvedPolyglotSymbol(mod, symbol)
) )
) => ) =>
val polyglotSymbol = mod val polyglotSymbol = asScope(
.unsafeAsModule() mod
.getScope .unsafeAsModule()
.getPolyglotSymbol(symbol.name) ).getPolyglotSymbol(symbol.name)
Either.cond( Either.cond(
polyglotSymbol != null, polyglotSymbol != null,
ObjectEqualityBranchNode ObjectEqualityBranchNode
@ -1305,10 +1343,10 @@ class IrToTruffle(
) )
) => ) =>
val mod = typ.module val mod = typ.module
val polyClass = mod val polyClass = asScope(
.unsafeAsModule() mod
.getScope .unsafeAsModule()
.getPolyglotSymbol(typ.symbol.name) ).getPolyglotSymbol(typ.symbol.name)
val polyValueOrError = val polyValueOrError =
if (polyClass == null) if (polyClass == null)
@ -1410,7 +1448,11 @@ class IrToTruffle(
) => ) =>
// Using .getTypes because .getType may return an associated type // Using .getTypes because .getType may return an associated type
Option( Option(
mod.unsafeAsModule().getScope.getTypes.get(tpe.name) asScope(
mod
.unsafeAsModule()
).getTypes
.get(tpe.name)
) match { ) match {
case Some(tpe) => case Some(tpe) =>
val argOfType = List( val argOfType = List(
@ -1445,10 +1487,10 @@ class IrToTruffle(
) )
) => ) =>
val polySymbol = val polySymbol =
mod asScope(
.unsafeAsModule() mod
.getScope .unsafeAsModule()
.getPolyglotSymbol(symbol.name) ).getPolyglotSymbol(symbol.name)
if (polySymbol != null) { if (polySymbol != null) {
val argOfType = List( val argOfType = List(
DefinitionArgument.Specified( DefinitionArgument.Specified(
@ -1609,7 +1651,7 @@ class IrToTruffle(
} else if (global.isDefined) { } else if (global.isDefined) {
val resolution = global.get.target val resolution = global.get.target
nodeForResolution(resolution) nodeForResolution(resolution)
} else if (nameStr == Constants.Names.FROM_MEMBER) { } else if (nameStr == ConstantsNames.FROM_MEMBER) {
ConstantObjectNode.build(UnresolvedConversion.build(moduleScope)) ConstantObjectNode.build(UnresolvedConversion.build(moduleScope))
} else { } else {
DynamicSymbolNode.build( DynamicSymbolNode.build(
@ -1619,7 +1661,7 @@ class IrToTruffle(
case Name.Self(location, _, passData, _) => case Name.Self(location, _, passData, _) =>
processName( processName(
Name.Literal( Name.Literal(
Constants.Names.SELF_ARGUMENT, ConstantsNames.SELF_ARGUMENT,
isMethod = false, isMethod = false,
location, location,
None, None,
@ -1672,12 +1714,14 @@ class IrToTruffle(
resolution match { resolution match {
case tp: BindingsMap.ResolvedType => case tp: BindingsMap.ResolvedType =>
ConstantObjectNode.build( ConstantObjectNode.build(
tp.module.unsafeAsModule().getScope.getTypes.get(tp.tp.name) asScope(
tp.module
.unsafeAsModule()
).getTypes
.get(tp.tp.name)
) )
case BindingsMap.ResolvedConstructor(definitionType, cons) => case BindingsMap.ResolvedConstructor(definitionType, cons) =>
val c = definitionType val c = asType(definitionType).getConstructors
.unsafeToRuntimeType()
.getConstructors
.get(cons.name) .get(cons.name)
if (c == null) { if (c == null) {
throw new CompilerError(s"Constructor for $cons is null") throw new CompilerError(s"Constructor for $cons is null")
@ -1685,13 +1729,16 @@ class IrToTruffle(
ConstructorNode.build(c) ConstructorNode.build(c)
case BindingsMap.ResolvedModule(module) => case BindingsMap.ResolvedModule(module) =>
ConstantObjectNode.build( ConstantObjectNode.build(
module.unsafeAsModule().getScope.getAssociatedType asScope(
module
.unsafeAsModule()
).getAssociatedType
) )
case BindingsMap.ResolvedPolyglotSymbol(module, symbol) => case BindingsMap.ResolvedPolyglotSymbol(module, symbol) =>
val s = module val s = asScope(
.unsafeAsModule() module
.getScope .unsafeAsModule()
.getPolyglotSymbol(symbol.name) ).getPolyglotSymbol(symbol.name)
if (s == null) { if (s == null) {
throw new CompilerError( throw new CompilerError(
s"No polyglot symbol for ${symbol.name}" s"No polyglot symbol for ${symbol.name}"
@ -1699,10 +1746,10 @@ class IrToTruffle(
} }
ConstantObjectNode.build(s) ConstantObjectNode.build(s)
case BindingsMap.ResolvedPolyglotField(symbol, name) => case BindingsMap.ResolvedPolyglotField(symbol, name) =>
val s = symbol.module val s = asScope(
.unsafeAsModule() symbol.module
.getScope .unsafeAsModule()
.getPolyglotSymbol(name) ).getPolyglotSymbol(name)
if (s == null) { if (s == null) {
throw new CompilerError( throw new CompilerError(
s"No polyglot field for ${name}" s"No polyglot field for ${name}"
@ -1846,7 +1893,7 @@ class IrToTruffle(
// Note [Rewriting Arguments] // Note [Rewriting Arguments]
val argSlots = arguments.zipWithIndex.map { val argSlots = arguments.zipWithIndex.map {
case (unprocessedArg, idx) => case (unprocessedArg, idx) =>
val checkNode = checkRuntimeTypes(unprocessedArg) val checkNode = checkAsTypes(unprocessedArg)
val arg = argFactory.run(unprocessedArg, idx, checkNode) val arg = argFactory.run(unprocessedArg, idx, checkNode)
argDefinitions(idx) = arg argDefinitions(idx) = arg
val occInfo = unprocessedArg val occInfo = unprocessedArg
@ -1870,7 +1917,7 @@ class IrToTruffle(
val argName = arg.getName val argName = arg.getName
if ( if (
argName != Constants.Names.SELF_ARGUMENT && seenArgNames.contains( argName != ConstantsNames.SELF_ARGUMENT && seenArgNames.contains(
argName argName
) )
) { ) {
@ -2228,4 +2275,13 @@ class IrToTruffle(
) )
} }
} }
private def asScope(module: CompilerContext.Module): ModuleScope = {
val m = org.enso.interpreter.runtime.Module.fromCompilerModule(module)
m.getScope()
}
private def asType(typ: BindingsMap.ResolvedType): Type = {
asScope(typ.module.unsafeAsModule()).getTypes().get(typ.tp.name)
}
} }

View File

@ -1,4 +1,4 @@
package org.enso.compiler.codegen package org.enso.interpreter.runtime
import org.enso.compiler.data.BindingsMap import org.enso.compiler.data.BindingsMap
import org.enso.compiler.core.CompilerError import org.enso.compiler.core.CompilerError

View File

@ -2,6 +2,7 @@ package org.enso.compiler.test.pass.desugar
import org.enso.compiler.Passes import org.enso.compiler.Passes
import org.enso.compiler.context.ModuleContext import org.enso.compiler.context.ModuleContext
import org.enso.compiler.core.ConstantsNames
import org.enso.compiler.core.ir.{ import org.enso.compiler.core.ir.{
DefinitionArgument, DefinitionArgument,
Function, Function,
@ -15,7 +16,6 @@ import org.enso.compiler.core.ir.module.scope.definition
import org.enso.compiler.pass.desugar.{FunctionBinding, GenerateMethodBodies} import org.enso.compiler.pass.desugar.{FunctionBinding, GenerateMethodBodies}
import org.enso.compiler.pass.{PassConfiguration, PassGroup, PassManager} import org.enso.compiler.pass.{PassConfiguration, PassGroup, PassManager}
import org.enso.compiler.test.CompilerTest import org.enso.compiler.test.CompilerTest
import org.enso.interpreter.Constants
class GenerateMethodBodiesTest extends CompilerTest { class GenerateMethodBodiesTest extends CompilerTest {
@ -338,7 +338,7 @@ class GenerateMethodBodiesTest extends CompilerTest {
val nestedBody = body.body.asInstanceOf[Function.Lambda] val nestedBody = body.body.asInstanceOf[Function.Lambda]
nestedBody.arguments.length shouldEqual 1 nestedBody.arguments.length shouldEqual 1
nestedBody.arguments.head.name shouldBe an[Name.Literal] nestedBody.arguments.head.name shouldBe an[Name.Literal]
nestedBody.arguments.head.name.name shouldEqual Constants.Names.THAT_ARGUMENT nestedBody.arguments.head.name.name shouldEqual ConstantsNames.THAT_ARGUMENT
} }
"have report a warning when defining `self` at a wrong position" in { "have report a warning when defining `self` at a wrong position" in {
@ -352,7 +352,7 @@ class GenerateMethodBodiesTest extends CompilerTest {
val body = conversion.body.asInstanceOf[Function.Lambda] val body = conversion.body.asInstanceOf[Function.Lambda]
body.arguments.length shouldEqual 1 body.arguments.length shouldEqual 1
body.arguments.head.name shouldBe an[Name.Literal] body.arguments.head.name shouldBe an[Name.Literal]
body.arguments.head.name.name shouldBe Constants.Names.THAT_ARGUMENT body.arguments.head.name.name shouldBe ConstantsNames.THAT_ARGUMENT
conversion.body.diagnostics.collect { case w: Warning => conversion.body.diagnostics.collect { case w: Warning =>
w w
@ -370,7 +370,7 @@ class GenerateMethodBodiesTest extends CompilerTest {
val body = conversion.body.asInstanceOf[Function.Lambda] val body = conversion.body.asInstanceOf[Function.Lambda]
body.arguments.length shouldEqual 1 body.arguments.length shouldEqual 1
body.arguments.head.name shouldBe an[Name.Literal] body.arguments.head.name shouldBe an[Name.Literal]
body.arguments.head.name.name shouldBe Constants.Names.THAT_ARGUMENT body.arguments.head.name.name shouldBe ConstantsNames.THAT_ARGUMENT
conversion.body.diagnostics.collect { case w: Warning => conversion.body.diagnostics.collect { case w: Warning =>
w w