Caches belong to runtime, not IR compiler (#8178)

This commit is contained in:
Jaroslav Tulach 2023-10-30 16:53:44 +01:00 committed by GitHub
parent 487241317b
commit 646b47b246
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 191 additions and 174 deletions

View File

@ -1,9 +1,8 @@
package org.enso.interpreter.instrument.job; package org.enso.interpreter.instrument.job;
import java.util.logging.Level; import java.util.logging.Level;
import org.enso.compiler.SerializationManager;
import org.enso.interpreter.instrument.execution.RuntimeContext; import org.enso.interpreter.instrument.execution.RuntimeContext;
import org.enso.interpreter.runtime.EnsoContext; import org.enso.interpreter.runtime.SerializationManager;
import org.enso.pkg.QualifiedName; import org.enso.pkg.QualifiedName;
import org.enso.polyglot.CompilationStage; import org.enso.polyglot.CompilationStage;
@ -21,8 +20,9 @@ public final class SerializeModuleJob extends BackgroundJob<Void> {
@Override @Override
public Void run(RuntimeContext ctx) { public Void run(RuntimeContext ctx) {
EnsoContext ensoContext = ctx.executionService().getContext(); var ensoContext = ctx.executionService().getContext();
SerializationManager serializationManager = ensoContext.getCompiler().getSerializationManager(); var compiler = ensoContext.getCompiler();
SerializationManager serializationManager = SerializationManager.apply(compiler.context());
boolean useGlobalCacheLocations = ensoContext.isUseGlobalCache(); boolean useGlobalCacheLocations = ensoContext.isUseGlobalCache();
var writeLockTimestamp = ctx.locking().acquireWriteCompilationLock(); var writeLockTimestamp = ctx.locking().acquireWriteCompilationLock();
try { try {
@ -42,7 +42,7 @@ public final class SerializeModuleJob extends BackgroundJob<Void> {
} }
serializationManager.serializeModule( serializationManager.serializeModule(
module.asCompilerModule(), useGlobalCacheLocations, false); compiler, module.asCompilerModule(), useGlobalCacheLocations, false);
}); });
} finally { } finally {
ctx.locking().releaseWriteCompilationLock(); ctx.locking().releaseWriteCompilationLock();

View File

@ -1,6 +1,7 @@
package org.enso.interpreter.instrument.job package org.enso.interpreter.instrument.job
import org.enso.editions.LibraryName import org.enso.editions.LibraryName
import org.enso.interpreter.runtime.SerializationManager
import org.enso.interpreter.instrument.execution.RuntimeContext import org.enso.interpreter.instrument.execution.RuntimeContext
import org.enso.polyglot.runtime.Runtime.Api import org.enso.polyglot.runtime.Runtime.Api
@ -31,8 +32,9 @@ final class DeserializeLibrarySuggestionsJob(
Level.FINE, Level.FINE,
s"Deserializing suggestions for library [$libraryName]." s"Deserializing suggestions for library [$libraryName]."
) )
val serializationManager = val serializationManager = SerializationManager(
ctx.executionService.getContext.getCompiler.getSerializationManager ctx.executionService.getContext.getCompiler.context
)
serializationManager serializationManager
.deserializeSuggestions(libraryName) .deserializeSuggestions(libraryName)
.foreach { cachedSuggestions => .foreach { cachedSuggestions =>

View File

@ -5,17 +5,15 @@ import com.oracle.truffle.api.source.Source;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream; import java.io.PrintStream;
import java.util.List; import java.util.List;
import java.util.Optional; import java.util.concurrent.Future;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.logging.Level; import java.util.logging.Level;
import org.enso.compiler.Cache;
import org.enso.compiler.Compiler; import org.enso.compiler.Compiler;
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.data.BindingsMap; import org.enso.compiler.data.BindingsMap;
import org.enso.compiler.data.CompilerConfig; import org.enso.compiler.data.CompilerConfig;
import org.enso.editions.LibraryName;
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;
@ -66,10 +64,7 @@ public interface CompilerContext {
boolean typeContainsValues(String name); boolean typeContainsValues(String name);
void initializeBuiltinsIr( void initializeBuiltinsIr(
boolean irCachingEnabled, Compiler compiler, boolean irCachingEnabled, FreshNameSupply freshNameSupply, Passes passes);
SerializationManager serializationManager,
FreshNameSupply freshNameSupply,
Passes passes);
QualifiedName getModuleName(Module module); QualifiedName getModuleName(Module module);
@ -89,12 +84,18 @@ public interface CompilerContext {
CompilationStage getCompilationStage(Module module); CompilationStage getCompilationStage(Module module);
<T> Optional<T> loadCache(Cache<T, ?> cache);
<T> Optional<TruffleFile> saveCache(Cache<T, ?> cache, T entry, boolean useGlobalCacheLocations);
TypeGraph getTypeHierarchy(); TypeGraph getTypeHierarchy();
Future<Boolean> serializeLibrary(
Compiler compiler, LibraryName libraryName, boolean useGlobalCacheLocations);
Future<Boolean> serializeModule(
Compiler compiler, Module module, boolean useGlobalCacheLocations);
boolean deserializeModule(Compiler compiler, Module module);
void shutdown(boolean waitForPendingJobCompletion);
public static interface Updater { public static interface Updater {
void bindingsMap(BindingsMap map); void bindingsMap(BindingsMap map);
@ -128,8 +129,6 @@ public interface CompilerContext {
public abstract List<QualifiedName> getDirectModulesRefs(); public abstract List<QualifiedName> getDirectModulesRefs();
public abstract ModuleCache getCache();
public abstract CompilationStage getCompilationStage(); public abstract CompilationStage getCompilationStage();
public abstract boolean isSynthetic(); public abstract boolean isSynthetic();

View File

@ -1,4 +1,4 @@
package org.enso.compiler; package org.enso.interpreter.caches;
import com.oracle.truffle.api.TruffleFile; import com.oracle.truffle.api.TruffleFile;
import com.oracle.truffle.api.TruffleLogger; import com.oracle.truffle.api.TruffleLogger;

View File

@ -1,4 +1,4 @@
package org.enso.compiler; package org.enso.interpreter.caches;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
@ -114,40 +114,19 @@ public final class ImportExportCache extends Cache<ImportExportCache.CachedBindi
return byteStream.toByteArray(); return byteStream.toByteArray();
} }
static class MapToBindings implements Serializable { public static final class MapToBindings implements Serializable {
private final Map<QualifiedName, BindingsMap> _entries; private final Map<QualifiedName, BindingsMap> _entries;
public MapToBindings(Map<QualifiedName, BindingsMap> entries) { public MapToBindings(Map<QualifiedName, BindingsMap> entries) {
this._entries = entries; this._entries = entries;
} }
Map<QualifiedName, BindingsMap> entries() { public Map<QualifiedName, BindingsMap> entries() {
return _entries; return _entries;
} }
} }
// CachedBindings is not a record **on purpose**. There appears to be a Frgaal bug leading to invalid compilation error. public static record CachedBindings(LibraryName libraryName, MapToBindings bindings, Optional<List<SourceFile<TruffleFile>>> sources) {
public static final class CachedBindings {
private final LibraryName _libraryName;
private final MapToBindings _bindings;
private final Optional<List<SourceFile<TruffleFile>>> _sources;
CachedBindings(LibraryName libraryName, MapToBindings bindings, Optional<List<SourceFile<TruffleFile>>> sources) {
this._libraryName = libraryName;
this._bindings = bindings;
this._sources = sources;
}
LibraryName libraryName() {
return _libraryName;
}
MapToBindings bindings() {
return _bindings;
}
Optional<List<SourceFile<TruffleFile>>> sources() {
return _sources;
}
} }
public record Metadata( public record Metadata(

View File

@ -1,4 +1,4 @@
package org.enso.compiler; package org.enso.interpreter.caches;
import buildinfo.Info; import buildinfo.Info;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
@ -175,27 +175,7 @@ public final class ModuleCache extends Cache<ModuleCache.CachedModule, ModuleCac
return byteStream.toByteArray(); return byteStream.toByteArray();
} }
// CachedModule is not a record **on purpose**. There appears to be a Frgaal bug leading to invalid compilation error. public record CachedModule(Module moduleIR, CompilationStage compilationStage, Source source) {
static class CachedModule {
private final Module _moduleIR;
private final CompilationStage _compilationStage;
private final Source _source;
public CachedModule(Module moduleIR, CompilationStage compilationStage, Source source) {
this._moduleIR = moduleIR;
this._compilationStage = compilationStage;
this._source = source;
}
Module moduleIR() {
return _moduleIR;
}
CompilationStage compilationStage() {
return _compilationStage;
}
Source source() {
return _source;
}
} }
public record Metadata( public record Metadata(

View File

@ -1,4 +1,4 @@
package org.enso.compiler; package org.enso.interpreter.caches;
import buildinfo.Info; import buildinfo.Info;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
@ -115,7 +115,7 @@ public final class SuggestionsCache
} }
// Suggestions class is not a record because of a Frgaal bug leading to invalid compilation error. // Suggestions class is not a record because of a Frgaal bug leading to invalid compilation error.
final static class Suggestions implements Serializable { public final static class Suggestions implements Serializable {
private final List<Suggestion> suggestions; private final List<Suggestion> suggestions;

View File

@ -22,11 +22,11 @@ import java.util.Map;
import java.util.UUID; import java.util.UUID;
import java.util.WeakHashMap; import java.util.WeakHashMap;
import java.util.logging.Level; import java.util.logging.Level;
import org.enso.compiler.ModuleCache;
import org.enso.compiler.context.CompilerContext; import org.enso.compiler.context.CompilerContext;
import org.enso.compiler.context.SimpleUpdate; import org.enso.compiler.context.SimpleUpdate;
import org.enso.compiler.core.IR; import org.enso.compiler.core.IR;
import org.enso.compiler.core.ir.Expression; import org.enso.compiler.core.ir.Expression;
import org.enso.interpreter.caches.ModuleCache;
import org.enso.interpreter.node.callable.dispatch.CallOptimiserNode; import org.enso.interpreter.node.callable.dispatch.CallOptimiserNode;
import org.enso.interpreter.node.callable.dispatch.LoopingCallOptimiserNode; import org.enso.interpreter.node.callable.dispatch.LoopingCallOptimiserNode;
import org.enso.interpreter.runtime.builtin.BuiltinFunction; import org.enso.interpreter.runtime.builtin.BuiltinFunction;

View File

@ -1,5 +1,6 @@
package org.enso.interpreter.runtime; package org.enso.interpreter.runtime;
import org.enso.compiler.Passes;
import org.enso.compiler.pass.analyse.BindingAnalysis$; import org.enso.compiler.pass.analyse.BindingAnalysis$;
import org.enso.compiler.context.CompilerContext; import org.enso.compiler.context.CompilerContext;
import org.enso.compiler.context.FreshNameSupply; import org.enso.compiler.context.FreshNameSupply;
@ -13,21 +14,17 @@ import java.io.PrintStream;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Optional; import java.util.Optional;
import java.util.concurrent.Future;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.logging.Level; import java.util.logging.Level;
import org.enso.compiler.Cache;
import org.enso.compiler.Compiler; import org.enso.compiler.Compiler;
import org.enso.compiler.ModuleCache;
import org.enso.compiler.PackageRepository; import org.enso.compiler.PackageRepository;
import org.enso.compiler.Passes;
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.editions.LibraryName;
import org.enso.interpreter.runtime.data.Type; import org.enso.interpreter.caches.Cache;
import org.enso.interpreter.runtime.scope.LocalScope; import org.enso.interpreter.caches.ModuleCache;
import org.enso.interpreter.runtime.type.Types; 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;
@ -39,14 +36,16 @@ import scala.Option;
final class TruffleCompilerContext implements CompilerContext { final class TruffleCompilerContext implements CompilerContext {
private final EnsoContext context; private final EnsoContext context;
private final TruffleLogger compiler; private final TruffleLogger loggerCompiler;
private final TruffleLogger serializationManager; private final TruffleLogger loggerManager;
private final RuntimeStubsGenerator stubsGenerator; private final RuntimeStubsGenerator stubsGenerator;
private final SerializationManager serializationManager;
TruffleCompilerContext(EnsoContext context) { TruffleCompilerContext(EnsoContext context) {
this.context = context; this.context = context;
this.compiler = context.getLogger(Compiler.class); this.loggerCompiler = context.getLogger(Compiler.class);
this.serializationManager = context.getLogger(SerializationManager.class); this.loggerManager = context.getLogger(SerializationManager.class);
this.serializationManager = new SerializationManager(this);
this.stubsGenerator = new RuntimeStubsGenerator(context.getBuiltins()); this.stubsGenerator = new RuntimeStubsGenerator(context.getBuiltins());
} }
@ -70,6 +69,10 @@ final class TruffleCompilerContext implements CompilerContext {
return context.getPackageRepository(); return context.getPackageRepository();
} }
final SerializationManager getSerializationManager() {
return serializationManager;
}
@Override @Override
public PrintStream getErr() { public PrintStream getErr() {
return context.getErr(); return context.getErr();
@ -82,12 +85,12 @@ final class TruffleCompilerContext implements CompilerContext {
@Override @Override
public void log(Level level, String msg, Object... args) { public void log(Level level, String msg, Object... args) {
compiler.log(level, msg, args); loggerCompiler.log(level, msg, args);
} }
@Override @Override
public void logSerializationManager(Level level, String msg, Object... args) { public void logSerializationManager(Level level, String msg, Object... args) {
serializationManager.log(level, msg, args); loggerManager.log(level, msg, args);
} }
@Override @Override
@ -169,12 +172,10 @@ final class TruffleCompilerContext implements CompilerContext {
} }
} }
@Override
public <T> Optional<T> loadCache(Cache<T, ?> cache) { public <T> Optional<T> loadCache(Cache<T, ?> cache) {
return cache.load(context); return cache.load(context);
} }
@Override
public <T> Optional<TruffleFile> saveCache( public <T> Optional<TruffleFile> saveCache(
Cache<T, ?> cache, T entry, boolean useGlobalCacheLocations) { Cache<T, ?> cache, T entry, boolean useGlobalCacheLocations) {
return cache.save(entry, context, useGlobalCacheLocations); return cache.save(entry, context, useGlobalCacheLocations);
@ -191,7 +192,8 @@ final class TruffleCompilerContext implements CompilerContext {
*/ */
@Override @Override
public void initializeBuiltinsIr( public void initializeBuiltinsIr(
boolean irCachingEnabled, SerializationManager serializationManager, Compiler compiler,
boolean irCachingEnabled,
FreshNameSupply freshNameSupply, Passes passes FreshNameSupply freshNameSupply, Passes passes
) { ) {
var builtins = context.getBuiltins(); var builtins = context.getBuiltins();
@ -207,7 +209,7 @@ final class TruffleCompilerContext implements CompilerContext {
if (irCachingEnabled) { if (irCachingEnabled) {
if ( if (
serializationManager.deserialize(builtinsModule) instanceof Option<?> op && serializationManager.deserialize(compiler, builtinsModule) instanceof Option<?> op &&
op.isDefined() && op.isDefined() &&
op.get() instanceof Boolean b && b op.get() instanceof Boolean b && b
) { ) {
@ -233,7 +235,7 @@ final class TruffleCompilerContext implements CompilerContext {
if (irCachingEnabled && !wasLoadedFromCache(builtinsModule)) { if (irCachingEnabled && !wasLoadedFromCache(builtinsModule)) {
serializationManager.serializeModule( serializationManager.serializeModule(
builtinsModule, true, true compiler, builtinsModule, true, true
); );
} }
} }
@ -250,6 +252,30 @@ final class TruffleCompilerContext implements CompilerContext {
return option.isEmpty() ? null : option.get().asCompilerModule(); return option.isEmpty() ? null : option.get().asCompilerModule();
} }
@SuppressWarnings("unchecked")
@Override
public Future<Boolean> serializeLibrary(Compiler compiler, LibraryName libraryName, boolean useGlobalCacheLocations) {
Object res = serializationManager.serializeLibrary(compiler, libraryName, useGlobalCacheLocations);
return (Future<Boolean>) res;
}
@SuppressWarnings("unchecked")
@Override
public Future<Boolean> serializeModule(Compiler compiler, CompilerContext.Module module, boolean useGlobalCacheLocations) {
Object res = serializationManager.serializeModule(compiler, module, useGlobalCacheLocations, true);
return (Future<Boolean>) res;
}
@Override
public boolean deserializeModule(Compiler compiler, CompilerContext.Module module) {
var result = serializationManager.deserialize(compiler, module);
return result.nonEmpty();
}
@Override
public void shutdown(boolean waitForPendingJobCompletion) {
serializationManager.shutdown(waitForPendingJobCompletion);
}
private final class ModuleUpdater implements Updater, AutoCloseable { private final class ModuleUpdater implements Updater, AutoCloseable {
private final Module module; private final Module module;
@ -389,7 +415,6 @@ final class TruffleCompilerContext implements CompilerContext {
return module.getDirectModulesRefs(); return module.getDirectModulesRefs();
} }
@Override
public ModuleCache getCache() { public ModuleCache getCache() {
return module.getCache(); return module.getCache();
} }

View File

@ -69,8 +69,6 @@ class Compiler(
private val irCachingEnabled = !context.isIrCachingDisabled private val irCachingEnabled = !context.isIrCachingDisabled
private val useGlobalCacheLocations = context.isUseGlobalCacheLocations private val useGlobalCacheLocations = context.isUseGlobalCacheLocations
private val isInteractiveMode = context.isInteractiveMode() private val isInteractiveMode = context.isInteractiveMode()
private val serializationManager: SerializationManager =
new SerializationManager(this)
private val output: PrintStream = private val output: PrintStream =
if (config.outputRedirect.isDefined) if (config.outputRedirect.isDefined)
new PrintStream(config.outputRedirect.get) new PrintStream(config.outputRedirect.get)
@ -105,18 +103,14 @@ class Compiler(
/** Run the initialization sequence. */ /** Run the initialization sequence. */
def initialize(): Unit = { def initialize(): Unit = {
context.initializeBuiltinsIr( context.initializeBuiltinsIr(
this,
irCachingEnabled, irCachingEnabled,
serializationManager,
freshNameSupply, freshNameSupply,
passes passes
) )
packageRepository.initialize().left.foreach(reportPackageError) packageRepository.initialize().left.foreach(reportPackageError)
} }
/** @return the serialization manager instance. */
def getSerializationManager: SerializationManager =
serializationManager
/** @return the package repository instance. */ /** @return the package repository instance. */
def getPackageRepository(): PackageRepository = def getPackageRepository(): PackageRepository =
context.getPackageRepository context.getPackageRepository
@ -147,7 +141,7 @@ class Compiler(
def compile( def compile(
shouldCompileDependencies: Boolean, shouldCompileDependencies: Boolean,
useGlobalCacheLocations: Boolean useGlobalCacheLocations: Boolean
): Future[Boolean] = { ): Future[java.lang.Boolean] = {
getPackageRepository().getMainProjectPackage match { getPackageRepository().getMainProjectPackage match {
case None => case None =>
context.log( context.log(
@ -188,9 +182,10 @@ class Compiler(
shouldCompileDependencies shouldCompileDependencies
) )
serializationManager.serializeLibrary( context.serializeLibrary(
this,
pkg.libraryName, pkg.libraryName,
useGlobalCacheLocations = useGlobalCacheLocations useGlobalCacheLocations
) )
} }
} }
@ -440,7 +435,8 @@ class Compiler(
if (isInteractiveMode) { if (isInteractiveMode) {
context.notifySerializeModule(context.getModuleName(module)) context.notifySerializeModule(context.getModuleName(module))
} else { } else {
serializationManager.serializeModule( context.serializeModule(
this,
module, module,
useGlobalCacheLocations useGlobalCacheLocations
) )
@ -580,9 +576,8 @@ class Compiler(
context.updateModule(module, _.resetScope) context.updateModule(module, _.resetScope)
if (irCachingEnabled && !context.isInteractive(module)) { if (irCachingEnabled && !context.isInteractive(module)) {
serializationManager.deserialize(module) match { if (context.deserializeModule(this, module)) {
case Some(_) => return return
case _ =>
} }
} }
@ -597,9 +592,9 @@ class Compiler(
def importExportBindings(module: Module): Option[BindingsMap] = { def importExportBindings(module: Module): Option[BindingsMap] = {
if (irCachingEnabled && !context.isInteractive(module)) { if (irCachingEnabled && !context.isInteractive(module)) {
val libraryName = Option(module.getPackage).map(_.libraryName) val libraryName = Option(module.getPackage).map(_.libraryName)
libraryName libraryName.flatMap(
.flatMap(packageRepository.getLibraryBindings(_, serializationManager)) packageRepository.getLibraryBindings(_, module.getName, context)
.flatMap(_.bindings.entries.get(module.getName)) )
} else None } else None
} }
@ -1234,7 +1229,7 @@ class Compiler(
* jobs to complete * jobs to complete
*/ */
def shutdown(waitForPendingJobCompletion: Boolean): Unit = { def shutdown(waitForPendingJobCompletion: Boolean): Unit = {
serializationManager.shutdown(waitForPendingJobCompletion) context.shutdown(waitForPendingJobCompletion)
shutdownParsingPool(waitForPendingJobCompletion) shutdownParsingPool(waitForPendingJobCompletion)
} }

View File

@ -3,7 +3,8 @@ package org.enso.compiler
import com.oracle.truffle.api.TruffleFile import com.oracle.truffle.api.TruffleFile
import org.enso.editions.LibraryName import org.enso.editions.LibraryName
import org.enso.compiler.context.CompilerContext import org.enso.compiler.context.CompilerContext
import org.enso.pkg.{ComponentGroups, Package} import org.enso.compiler.data.BindingsMap
import org.enso.pkg.{ComponentGroups, Package, QualifiedName}
import scala.collection.immutable.ListSet import scala.collection.immutable.ListSet
import scala.jdk.OptionConverters.RichOption import scala.jdk.OptionConverters.RichOption
@ -110,8 +111,9 @@ trait PackageRepository {
/** Returns a deserialized bindings map for the whole library, if available */ /** Returns a deserialized bindings map for the whole library, if available */
def getLibraryBindings( def getLibraryBindings(
libraryName: LibraryName, libraryName: LibraryName,
serializationManager: SerializationManager moduleName: QualifiedName,
): Option[ImportExportCache.CachedBindings] context: CompilerContext
): Option[BindingsMap]
} }

View File

@ -1,13 +1,13 @@
package org.enso.interpreter.runtime package org.enso.interpreter.runtime
import org.enso.compiler.PackageRepository import org.enso.compiler.PackageRepository
import org.enso.compiler.ImportExportCache
import org.enso.compiler.SerializationManager
import org.enso.compiler.context.CompilerContext import org.enso.compiler.context.CompilerContext
import org.enso.compiler.data.BindingsMap
import com.oracle.truffle.api.TruffleFile import com.oracle.truffle.api.TruffleFile
import com.typesafe.scalalogging.Logger import com.typesafe.scalalogging.Logger
import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.StringUtils
import org.enso.editions.LibraryVersion import org.enso.editions.LibraryVersion
import org.enso.interpreter.caches.ImportExportCache
import org.enso.interpreter.runtime.util.TruffleFileSystem import org.enso.interpreter.runtime.util.TruffleFileSystem
import org.enso.librarymanager.published.repository.LibraryManifest import org.enso.librarymanager.published.repository.LibraryManifest
import org.enso.librarymanager.resolved.LibraryRoot import org.enso.librarymanager.resolved.LibraryRoot
@ -572,17 +572,21 @@ private class DefaultPackageRepository(
override def getLibraryBindings( override def getLibraryBindings(
libraryName: LibraryName, libraryName: LibraryName,
serializationManager: SerializationManager moduleName: QualifiedName,
): Option[ImportExportCache.CachedBindings] = { context: CompilerContext
ensurePackageIsLoaded(libraryName).toOption.flatMap { _ => ): Option[BindingsMap] = {
val cache = ensurePackageIsLoaded(libraryName).toOption.flatMap { _ =>
if (!loadedLibraryBindings.contains(libraryName)) { if (!loadedLibraryBindings.contains(libraryName)) {
loadedPackages.get(libraryName).flatten.foreach(loadDependencies(_)) loadedPackages.get(libraryName).flatten.foreach(loadDependencies(_))
serializationManager context
.asInstanceOf[TruffleCompilerContext]
.getSerializationManager()
.deserializeLibraryBindings(libraryName) .deserializeLibraryBindings(libraryName)
.foreach(cache => loadedLibraryBindings.addOne((libraryName, cache))) .foreach(cache => loadedLibraryBindings.addOne((libraryName, cache)))
} }
loadedLibraryBindings.get(libraryName) loadedLibraryBindings.get(libraryName)
} }
cache.flatMap(_.bindings.entries().get(moduleName))
} }
private def loadDependencies(pkg: Package[TruffleFile]): Unit = { private def loadDependencies(pkg: Package[TruffleFile]): Unit = {

View File

@ -1,15 +1,20 @@
package org.enso.compiler package org.enso.interpreter.runtime
import com.oracle.truffle.api.source.Source import com.oracle.truffle.api.source.Source
import org.enso.compiler.Compiler
import org.enso.compiler.core.Implicits.AsMetadata import org.enso.compiler.core.Implicits.AsMetadata
import org.enso.compiler.core.ir.{Module => IRModule} import org.enso.compiler.core.ir.{Module => IRModule}
import org.enso.compiler.context.{ExportsBuilder, ExportsMap, SuggestionBuilder} import org.enso.compiler.context.{ExportsBuilder, ExportsMap, SuggestionBuilder}
import org.enso.compiler.context.CompilerContext
import org.enso.compiler.context.CompilerContext.Module import org.enso.compiler.context.CompilerContext.Module
import org.enso.compiler.pass.analyse.BindingAnalysis import org.enso.compiler.pass.analyse.BindingAnalysis
import org.enso.editions.LibraryName import org.enso.editions.LibraryName
import org.enso.pkg.QualifiedName import org.enso.pkg.QualifiedName
import org.enso.polyglot.Suggestion import org.enso.polyglot.Suggestion
import org.enso.polyglot.CompilationStage import org.enso.polyglot.CompilationStage
import org.enso.interpreter.caches.ImportExportCache
import org.enso.interpreter.caches.ModuleCache
import org.enso.interpreter.caches.SuggestionsCache
import java.io.NotSerializableException import java.io.NotSerializableException
import java.util import java.util
@ -27,7 +32,11 @@ import java.util.logging.Level
import scala.collection.mutable import scala.collection.mutable
import scala.jdk.OptionConverters.RichOptional import scala.jdk.OptionConverters.RichOptional
final class SerializationManager(compiler: Compiler) { final class SerializationManager(private val context: TruffleCompilerContext) {
def this(compiler: Compiler) = {
this(compiler.context.asInstanceOf[TruffleCompilerContext])
}
import SerializationManager._ import SerializationManager._
@ -57,13 +66,13 @@ final class SerializationManager(compiler: Compiler) {
TimeUnit.SECONDS, TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable](), new LinkedBlockingDeque[Runnable](),
(runnable: Runnable) => { (runnable: Runnable) => {
compiler.context.createSystemThread(runnable) context.createSystemThread(runnable)
} }
) )
// Make sure it is started to avoid races with language shutdown with low job // Make sure it is started to avoid races with language shutdown with low job
// count. // count.
if (compiler.context.isCreateThreadAllowed) { if (context.isCreateThreadAllowed) {
pool.prestartAllCoreThreads() pool.prestartAllCoreThreads()
} }
@ -91,6 +100,7 @@ final class SerializationManager(compiler: Compiler) {
* `true` if `module` has been successfully serialized, `false` otherwise * `true` if `module` has been successfully serialized, `false` otherwise
*/ */
def serializeModule( def serializeModule(
compiler: Compiler,
module: Module, module: Module,
useGlobalCacheLocations: Boolean, useGlobalCacheLocations: Boolean,
useThreadPool: Boolean = true useThreadPool: Boolean = true
@ -100,7 +110,7 @@ final class SerializationManager(compiler: Compiler) {
"Cannot serialize synthetic module [" + module.getName + "]" "Cannot serialize synthetic module [" + module.getName + "]"
); );
} }
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
"Requesting serialization for module [{0}].", "Requesting serialization for module [{0}].",
module.getName module.getName
@ -112,7 +122,7 @@ final class SerializationManager(compiler: Compiler) {
duplicatedIr.preorder.foreach(_.passData.prepareForSerialization(compiler)) duplicatedIr.preorder.foreach(_.passData.prepareForSerialization(compiler))
val task = doSerializeModule( val task = doSerializeModule(
module.getCache, getCache(module),
duplicatedIr, duplicatedIr,
module.getCompilationStage, module.getCompilationStage,
module.getName, module.getName,
@ -130,7 +140,7 @@ final class SerializationManager(compiler: Compiler) {
CompletableFuture.completedFuture(task.call()) CompletableFuture.completedFuture(task.call())
} catch { } catch {
case e: Throwable => case e: Throwable =>
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
s"Serialization task failed in module [${module.getName}].", s"Serialization task failed in module [${module.getName}].",
e e
@ -141,18 +151,19 @@ final class SerializationManager(compiler: Compiler) {
} }
def serializeLibrary( def serializeLibrary(
compiler: Compiler,
libraryName: LibraryName, libraryName: LibraryName,
useGlobalCacheLocations: Boolean useGlobalCacheLocations: Boolean
): Future[Boolean] = { ): Future[Boolean] = {
compiler.context.logSerializationManager( context.logSerializationManager(
Level.INFO, Level.INFO,
"Requesting serialization for library [{0}].", "Requesting serialization for library [{0}].",
libraryName libraryName
) )
val task: Callable[Boolean] = val task: Callable[Boolean] =
doSerializeLibrary(libraryName, useGlobalCacheLocations) doSerializeLibrary(compiler, libraryName, useGlobalCacheLocations)
if (compiler.context.isCreateThreadAllowed) { if (context.isCreateThreadAllowed) {
isWaitingForSerialization.synchronized { isWaitingForSerialization.synchronized {
val future = pool.submit(task) val future = pool.submit(task)
isWaitingForSerialization.put(libraryName.toQualifiedName, future) isWaitingForSerialization.put(libraryName.toQualifiedName, future)
@ -163,7 +174,7 @@ final class SerializationManager(compiler: Compiler) {
CompletableFuture.completedFuture(task.call()) CompletableFuture.completedFuture(task.call())
} catch { } catch {
case e: Throwable => case e: Throwable =>
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
s"Serialization task failed for library [$libraryName].", s"Serialization task failed for library [$libraryName].",
e e
@ -174,6 +185,7 @@ final class SerializationManager(compiler: Compiler) {
} }
private def doSerializeLibrary( private def doSerializeLibrary(
compiler: Compiler,
libraryName: LibraryName, libraryName: LibraryName,
useGlobalCacheLocations: Boolean useGlobalCacheLocations: Boolean
): Callable[Boolean] = () => { ): Callable[Boolean] = () => {
@ -181,7 +193,7 @@ final class SerializationManager(compiler: Compiler) {
Thread.sleep(100) Thread.sleep(100)
} }
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
"Running serialization for bindings [{0}].", "Running serialization for bindings [{0}].",
libraryName libraryName
@ -190,7 +202,8 @@ final class SerializationManager(compiler: Compiler) {
val bindingsCache = new ImportExportCache.CachedBindings( val bindingsCache = new ImportExportCache.CachedBindings(
libraryName, libraryName,
new ImportExportCache.MapToBindings( new ImportExportCache.MapToBindings(
compiler.packageRepository context
.getPackageRepository()
.getModulesForLibrary(libraryName) .getModulesForLibrary(libraryName)
.map { module => .map { module =>
val ir = module.getIr val ir = module.getIr
@ -203,7 +216,8 @@ final class SerializationManager(compiler: Compiler) {
} }
.toMap .toMap
), ),
compiler.packageRepository context
.getPackageRepository()
.getPackageForLibraryJava(libraryName) .getPackageForLibraryJava(libraryName)
.map(_.listSourcesJava()) .map(_.listSourcesJava())
) )
@ -211,7 +225,7 @@ final class SerializationManager(compiler: Compiler) {
val result = val result =
try { try {
val cache = new ImportExportCache(libraryName) val cache = new ImportExportCache(libraryName)
val file = compiler.context.saveCache( val file = context.saveCache(
cache, cache,
bindingsCache, bindingsCache,
useGlobalCacheLocations useGlobalCacheLocations
@ -219,14 +233,14 @@ final class SerializationManager(compiler: Compiler) {
file.isPresent file.isPresent
} catch { } catch {
case e: NotSerializableException => case e: NotSerializableException =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.SEVERE, Level.SEVERE,
s"Could not serialize bindings [$libraryName].", s"Could not serialize bindings [$libraryName].",
e e
) )
throw e throw e
case e: Throwable => case e: Throwable =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.SEVERE, Level.SEVERE,
s"Serialization of bindings `$libraryName` failed: ${e.getMessage}`", s"Serialization of bindings `$libraryName` failed: ${e.getMessage}`",
e e
@ -234,7 +248,11 @@ final class SerializationManager(compiler: Compiler) {
throw e throw e
} }
doSerializeLibrarySuggestions(libraryName, useGlobalCacheLocations) doSerializeLibrarySuggestions(
compiler,
libraryName,
useGlobalCacheLocations
)
result result
} finally { } finally {
@ -243,6 +261,7 @@ final class SerializationManager(compiler: Compiler) {
} }
private def doSerializeLibrarySuggestions( private def doSerializeLibrarySuggestions(
compiler: Compiler,
libraryName: LibraryName, libraryName: LibraryName,
useGlobalCacheLocations: Boolean useGlobalCacheLocations: Boolean
): Boolean = { ): Boolean = {
@ -252,7 +271,7 @@ final class SerializationManager(compiler: Compiler) {
try { try {
val libraryModules = val libraryModules =
compiler.packageRepository.getModulesForLibrary(libraryName) context.getPackageRepository().getModulesForLibrary(libraryName)
libraryModules libraryModules
.flatMap { module => .flatMap { module =>
val suggestions = SuggestionBuilder(module, compiler) val suggestions = SuggestionBuilder(module, compiler)
@ -272,12 +291,13 @@ final class SerializationManager(compiler: Compiler) {
new SuggestionsCache.CachedSuggestions( new SuggestionsCache.CachedSuggestions(
libraryName, libraryName,
new SuggestionsCache.Suggestions(suggestions), new SuggestionsCache.Suggestions(suggestions),
compiler.packageRepository context
.getPackageRepository()
.getPackageForLibraryJava(libraryName) .getPackageForLibraryJava(libraryName)
.map(_.listSourcesJava()) .map(_.listSourcesJava())
) )
val cache = new SuggestionsCache(libraryName) val cache = new SuggestionsCache(libraryName)
val file = compiler.context.saveCache( val file = context.saveCache(
cache, cache,
cachedSuggestions, cachedSuggestions,
useGlobalCacheLocations useGlobalCacheLocations
@ -285,14 +305,14 @@ final class SerializationManager(compiler: Compiler) {
file.isPresent file.isPresent
} catch { } catch {
case e: NotSerializableException => case e: NotSerializableException =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.SEVERE, Level.SEVERE,
s"Could not serialize suggestions [$libraryName].", s"Could not serialize suggestions [$libraryName].",
e e
) )
throw e throw e
case e: Throwable => case e: Throwable =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.SEVERE, Level.SEVERE,
s"Serialization of suggestions `$libraryName` failed: ${e.getMessage}`", s"Serialization of suggestions `$libraryName` failed: ${e.getMessage}`",
e e
@ -312,16 +332,16 @@ final class SerializationManager(compiler: Compiler) {
Thread.sleep(100) Thread.sleep(100)
} }
val cache = new SuggestionsCache(libraryName) val cache = new SuggestionsCache(libraryName)
compiler.context.loadCache(cache).toScala match { context.loadCache(cache).toScala match {
case result @ Some(_: SuggestionsCache.CachedSuggestions) => case result @ Some(_: SuggestionsCache.CachedSuggestions) =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.FINE, Level.FINE,
"Restored suggestions for library [{0}].", "Restored suggestions for library [{0}].",
libraryName libraryName
) )
result result
case None => case None =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.FINE, Level.FINE,
"Unable to load suggestions for library [{0}].", "Unable to load suggestions for library [{0}].",
libraryName libraryName
@ -342,16 +362,16 @@ final class SerializationManager(compiler: Compiler) {
Thread.sleep(100) Thread.sleep(100)
} }
val cache = new ImportExportCache(libraryName) val cache = new ImportExportCache(libraryName)
compiler.context.loadCache(cache).toScala match { context.loadCache(cache).toScala match {
case result @ Some(_: ImportExportCache.CachedBindings) => case result @ Some(_: ImportExportCache.CachedBindings) =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.FINE, Level.FINE,
"Restored bindings for library [{0}].", "Restored bindings for library [{0}].",
libraryName libraryName
) )
result result
case _ => case _ =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.FINEST, Level.FINEST,
"Unable to load bindings for library [{0}].", "Unable to load bindings for library [{0}].",
libraryName libraryName
@ -375,6 +395,7 @@ final class SerializationManager(compiler: Compiler) {
* cache could not be deserialized. * cache could not be deserialized.
*/ */
def deserialize( def deserialize(
compiler: Compiler,
module: Module module: Module
): Option[Boolean] = { ): Option[Boolean] = {
if (isWaitingForSerialization(module)) { if (isWaitingForSerialization(module)) {
@ -385,47 +406,45 @@ final class SerializationManager(compiler: Compiler) {
Thread.sleep(100) Thread.sleep(100)
} }
compiler.context.loadCache(module.getCache).toScala match { context.loadCache(getCache(module)).toScala match {
case Some(loadedCache) => case Some(loadedCache) =>
val relinkedIrChecks = val relinkedIrChecks =
loadedCache loadedCache.moduleIR.preorder
.moduleIR() .map(_.passData.restoreFromSerialization(compiler))
.preorder context.updateModule(
.map(_.passData.restoreFromSerialization(this.compiler))
compiler.context.updateModule(
module, module,
{ u => { u =>
u.ir(loadedCache.moduleIR()) u.ir(loadedCache.moduleIR)
u.compilationStage(loadedCache.compilationStage()) u.compilationStage(loadedCache.compilationStage)
u.loadedFromCache(true) u.loadedFromCache(true)
} }
) )
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
"Restored IR from cache for module [{0}] at stage [{1}].", "Restored IR from cache for module [{0}] at stage [{1}].",
module.getName, module.getName,
loadedCache.compilationStage() loadedCache.compilationStage
) )
if (!relinkedIrChecks.contains(false)) { if (!relinkedIrChecks.contains(false)) {
compiler.context.updateModule(module, _.hasCrossModuleLinks(true)) context.updateModule(module, _.hasCrossModuleLinks(true))
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
"Restored links (early phase) in module [{0}].", "Restored links (early phase) in module [{0}].",
module.getName module.getName
) )
Some(true) Some(true)
} else { } else {
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
"Could not restore links (early phase) in module [{0}].", "Could not restore links (early phase) in module [{0}].",
module.getName module.getName
) )
compiler.context.updateModule(module, _.hasCrossModuleLinks(false)) context.updateModule(module, _.hasCrossModuleLinks(false))
Some(false) Some(false)
} }
case None => case None =>
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
"Unable to load a cache for module [{0}].", "Unable to load a cache for module [{0}].",
module.getName module.getName
@ -524,7 +543,7 @@ final class SerializationManager(compiler: Compiler) {
isWaitingForSerialization.size isWaitingForSerialization.size
} }
val jobCount = waitingCount + isSerializing.size val jobCount = waitingCount + isSerializing.size
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
"Waiting for #{0} serialization jobs to complete.", "Waiting for #{0} serialization jobs to complete.",
jobCount jobCount
@ -551,7 +570,7 @@ final class SerializationManager(compiler: Compiler) {
pool.shutdownNow() pool.shutdownNow()
Thread.sleep(100) Thread.sleep(100)
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
"Serialization manager has been shut down." "Serialization manager has been shut down."
) )
@ -591,7 +610,7 @@ final class SerializationManager(compiler: Compiler) {
Thread.sleep(100) Thread.sleep(100)
} }
compiler.context.logSerializationManager( context.logSerializationManager(
debugLogLevel, debugLogLevel,
"Running serialization for module [{0}].", "Running serialization for module [{0}].",
name name
@ -602,7 +621,7 @@ final class SerializationManager(compiler: Compiler) {
if (stage.isAtLeast(CompilationStage.AFTER_STATIC_PASSES)) { if (stage.isAtLeast(CompilationStage.AFTER_STATIC_PASSES)) {
CompilationStage.AFTER_STATIC_PASSES CompilationStage.AFTER_STATIC_PASSES
} else stage } else stage
compiler.context context
.saveCache( .saveCache(
cache, cache,
new ModuleCache.CachedModule(ir, fixedStage, source), new ModuleCache.CachedModule(ir, fixedStage, source),
@ -612,14 +631,14 @@ final class SerializationManager(compiler: Compiler) {
.orElse(false) .orElse(false)
} catch { } catch {
case e: NotSerializableException => case e: NotSerializableException =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.SEVERE, Level.SEVERE,
s"Could not serialize module [$name].", s"Could not serialize module [$name].",
e e
) )
throw e throw e
case e: Throwable => case e: Throwable =>
compiler.context.logSerializationManager( context.logSerializationManager(
Level.SEVERE, Level.SEVERE,
s"Serialization of module `$name` failed: ${e.getMessage}`", s"Serialization of module `$name` failed: ${e.getMessage}`",
e e
@ -663,6 +682,10 @@ final class SerializationManager(compiler: Compiler) {
) )
.asScala .asScala
} }
private def getCache(module: Module): ModuleCache = {
module.asInstanceOf[TruffleCompilerContext.Module].getCache
}
} }
object SerializationManager { object SerializationManager {
@ -682,4 +705,7 @@ object SerializationManager {
QualifiedName(List(libraryName.namespace), libraryName.name) QualifiedName(List(libraryName.namespace), libraryName.name)
} }
def apply(context: CompilerContext): SerializationManager = {
context.asInstanceOf[TruffleCompilerContext].getSerializationManager()
}
} }

View File

@ -18,6 +18,7 @@ import java.util.logging.LogRecord;
import java.util.logging.SimpleFormatter; import java.util.logging.SimpleFormatter;
import org.enso.compiler.core.ir.Module; import org.enso.compiler.core.ir.Module;
import org.enso.interpreter.runtime.EnsoContext; import org.enso.interpreter.runtime.EnsoContext;
import org.enso.interpreter.runtime.SerializationManager$;
import org.enso.pkg.PackageManager; import org.enso.pkg.PackageManager;
import org.enso.polyglot.LanguageInfo; import org.enso.polyglot.LanguageInfo;
import org.enso.polyglot.MethodNames; import org.enso.polyglot.MethodNames;
@ -54,13 +55,14 @@ public class SerdeCompilerTest {
var result = compiler.run(module); var result = compiler.run(module);
assertEquals("Two library modules are compiled", result.compiledModules().size(), 2); assertEquals("Two library modules are compiled", result.compiledModules().size(), 2);
assertEquals(result.compiledModules().exists(m -> m == module), true); assertEquals(result.compiledModules().exists(m -> m == module), true);
var serializationManager = ensoContext.getCompiler().getSerializationManager(); var serializationManager =
SerializationManager$.MODULE$.apply(ensoContext.getCompiler().context());
var futures = new ArrayList<Future<?>>(); var futures = new ArrayList<Future<?>>();
result result
.compiledModules() .compiledModules()
.foreach( .foreach(
(m) -> { (m) -> {
var future = serializationManager.serializeModule(m, true, true); var future = serializationManager.serializeModule(compiler, m, true, true);
futures.add(future); futures.add(future);
return null; return null;
}); });

View File

@ -10,7 +10,9 @@ import java.util.function.Supplier;
import java.util.stream.Stream; import java.util.stream.Stream;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.enso.editions.LibraryName; import org.enso.editions.LibraryName;
import org.enso.interpreter.caches.SuggestionsCache;
import org.enso.interpreter.runtime.EnsoContext; import org.enso.interpreter.runtime.EnsoContext;
import org.enso.interpreter.runtime.SerializationManager;
import org.enso.interpreter.runtime.util.TruffleFileSystem; import org.enso.interpreter.runtime.util.TruffleFileSystem;
import org.enso.interpreter.test.InterpreterContext; import org.enso.interpreter.test.InterpreterContext;
import org.enso.pkg.Package; import org.enso.pkg.Package;

View File

@ -9,6 +9,7 @@ import java.io.OutputStream;
import java.nio.file.Paths; import java.nio.file.Paths;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.enso.interpreter.runtime.EnsoContext; import org.enso.interpreter.runtime.EnsoContext;
import org.enso.interpreter.runtime.SerializationManager;
import org.enso.pkg.PackageManager; import org.enso.pkg.PackageManager;
import org.enso.polyglot.LanguageInfo; import org.enso.polyglot.LanguageInfo;
import org.enso.polyglot.MethodNames; import org.enso.polyglot.MethodNames;
@ -57,10 +58,10 @@ public class SerializerTest {
assertEquals(result.compiledModules().exists(m -> m == module), true); assertEquals(result.compiledModules().exists(m -> m == module), true);
var serializationManager = new SerializationManager(ensoContext.getCompiler()); var serializationManager = new SerializationManager(ensoContext.getCompiler());
var useThreadPool = compiler.context().isCreateThreadAllowed(); var useThreadPool = compiler.context().isCreateThreadAllowed();
var future = serializationManager.serializeModule(module, true, useThreadPool); var future = serializationManager.serializeModule(compiler, module, true, useThreadPool);
var serialized = future.get(5, TimeUnit.SECONDS); var serialized = future.get(5, TimeUnit.SECONDS);
assertEquals(serialized, true); assertEquals(serialized, true);
var deserialized = serializationManager.deserialize(module); var deserialized = serializationManager.deserialize(compiler, module);
assertEquals(deserialized.isDefined() && (Boolean) deserialized.get(), true); assertEquals(deserialized.isDefined() && (Boolean) deserialized.get(), true);
serializationManager.shutdown(true); serializationManager.shutdown(true);
ctx.leave(); ctx.leave();