From 6fc36ad66dba61282b0787e157aadb9b83ade7b4 Mon Sep 17 00:00:00 2001 From: rochala Date: Mon, 19 May 2025 19:53:17 +0200 Subject: [PATCH] Compiler cancellation with new compiler job queue --- .../src/dotty/tools/dotc/core/Contexts.scala | 2 +- .../dotc/interactive/InteractiveDriver.scala | 13 +- .../main/dotty/tools/pc/CachingDriver.scala | 11 +- .../main/dotty/tools/pc/DriverAccess.scala | 65 +++ .../dotty/tools/pc/PcDefinitionProvider.scala | 17 +- .../dotty/tools/pc/PcInlayHintsProvider.scala | 10 +- .../main/dotty/tools/pc/PcReportContext.scala | 15 + .../dotty/tools/pc/Scala3CompilerAccess.scala | 40 -- .../tools/pc/Scala3CompilerWrapper.scala | 29 -- .../tools/pc/ScalaPresentationCompiler.scala | 380 ++++++------------ .../src/main/dotty/tools/pc/TaskQueue.scala | 113 ++++++ .../src/main/dotty/tools/pc/Worker.scala | 107 +++++ .../pc/completions/CompletionProvider.scala | 235 +++++------ .../tools/pc/tests/CompilerCachingSuite.scala | 12 +- .../completion/CompletionCancelSuite.scala | 150 ++++--- .../AutoImplementAbstractMembersSuite.scala | 4 +- 16 files changed, 661 insertions(+), 542 deletions(-) create mode 100644 presentation-compiler/src/main/dotty/tools/pc/DriverAccess.scala create mode 100644 presentation-compiler/src/main/dotty/tools/pc/PcReportContext.scala delete mode 100644 presentation-compiler/src/main/dotty/tools/pc/Scala3CompilerAccess.scala delete mode 100644 presentation-compiler/src/main/dotty/tools/pc/Scala3CompilerWrapper.scala create mode 100644 presentation-compiler/src/main/dotty/tools/pc/TaskQueue.scala create mode 100644 presentation-compiler/src/main/dotty/tools/pc/Worker.scala diff --git a/compiler/src/dotty/tools/dotc/core/Contexts.scala b/compiler/src/dotty/tools/dotc/core/Contexts.scala index e12ab1cc2da2..9a310c98d111 100644 --- a/compiler/src/dotty/tools/dotc/core/Contexts.scala +++ b/compiler/src/dotty/tools/dotc/core/Contexts.scala @@ -180,7 +180,7 @@ object Contexts { val local = incCallback local != null && local.enabled || forceRun - /** The Zinc compile progress callback implementation if we are run from Zinc, null otherwise */ + /** The Zinc compile progress callback implementation if we are run from Zinc or used by presentation compiler, null otherwise */ def progressCallback: ProgressCallback | Null = store(progressCallbackLoc) /** Run `op` if there exists a Zinc progress callback */ diff --git a/compiler/src/dotty/tools/dotc/interactive/InteractiveDriver.scala b/compiler/src/dotty/tools/dotc/interactive/InteractiveDriver.scala index 673874ae2769..5bd6ddac3b27 100644 --- a/compiler/src/dotty/tools/dotc/interactive/InteractiveDriver.scala +++ b/compiler/src/dotty/tools/dotc/interactive/InteractiveDriver.scala @@ -23,6 +23,7 @@ import Denotations.staticRef import classpath.* import reporting.* import util.* +import sbt.interfaces.ProgressCallback /** A Driver subclass designed to be used from IDEs */ class InteractiveDriver(val settings: List[String]) extends Driver { @@ -30,6 +31,8 @@ class InteractiveDriver(val settings: List[String]) extends Driver { override def sourcesRequired: Boolean = false + private var myProgressCallback: ProgressCallback | Null = null + private val myInitCtx: Context = { val rootCtx = initCtx.fresh.addMode(Mode.ReadPositions).addMode(Mode.Interactive) rootCtx.setSetting(rootCtx.settings.YretainTrees, true) @@ -141,6 +144,11 @@ class InteractiveDriver(val settings: List[String]) extends Driver { (fromSource ++ fromClassPath).distinct } + def runWithProgressCallback(uri: URI, sourceCode: String, progressCallback: ProgressCallback | Null = null) = + myProgressCallback = progressCallback + run(uri, sourceCode) + myProgressCallback = null + def run(uri: URI, sourceCode: String): List[Diagnostic] = run(uri, SourceFile.virtual(uri, sourceCode)) def run(uri: URI, source: SourceFile): List[Diagnostic] = { @@ -151,7 +159,7 @@ class InteractiveDriver(val settings: List[String]) extends Driver { val reporter = new StoreReporter(null) with UniqueMessagePositions with HideNonSensicalMessages - val run = compiler.newRun(using myInitCtx.fresh.setReporter(reporter)) + val run = compiler.newRun(using myInitCtx.fresh.setReporter(reporter).setProgressCallback(myProgressCallback)) myCtx = run.runContext.withRootImports given Context = myCtx @@ -169,8 +177,7 @@ class InteractiveDriver(val settings: List[String]) extends Driver { myCtx = myCtx.fresh.setPhase(myInitCtx.base.typerPhase) reporter.removeBufferedMessages - } - catch { + } catch { case ex: FatalError => myCtx = previousCtx close(uri) diff --git a/presentation-compiler/src/main/dotty/tools/pc/CachingDriver.scala b/presentation-compiler/src/main/dotty/tools/pc/CachingDriver.scala index f5715c2780a9..0fa935e9543a 100644 --- a/presentation-compiler/src/main/dotty/tools/pc/CachingDriver.scala +++ b/presentation-compiler/src/main/dotty/tools/pc/CachingDriver.scala @@ -42,14 +42,9 @@ class CachingDriver(override val settings: List[String]) extends InteractiveDriv override def run(uri: URI, source: SourceFile): List[Diagnostic] = val diags = if alreadyCompiled(uri, source.content) then Nil - else super.run(uri, source) - lastCompiledURI = uri - diags - - override def run(uri: URI, sourceCode: String): List[Diagnostic] = - val diags = - if alreadyCompiled(uri, sourceCode.toCharArray().nn) then Nil - else super.run(uri, sourceCode) + else + val res = super.run(uri, source) + res lastCompiledURI = uri diags diff --git a/presentation-compiler/src/main/dotty/tools/pc/DriverAccess.scala b/presentation-compiler/src/main/dotty/tools/pc/DriverAccess.scala new file mode 100644 index 000000000000..8efffcf8c644 --- /dev/null +++ b/presentation-compiler/src/main/dotty/tools/pc/DriverAccess.scala @@ -0,0 +1,65 @@ +package dotty.tools.pc + + +import scala.meta.pc.PresentationCompilerConfig + +import dotty.tools.dotc.interactive.InteractiveDriver +import scala.meta.pc.CancelToken +import scala.meta.pc.VirtualFileParams +import java.util.concurrent.CompletableFuture +import scala.meta.internal.metals.EmptyCancelToken +import java.net.URI +import scala.meta.pc.OffsetParams + +case class CompilationInputs( + uri: URI, + code: String, + cancelToken: CancelToken = EmptyCancelToken, + cleanDriver: Boolean = false +): + def show: String = + s"""|uri: $uri + |code: $code + |cancelToken: $cancelToken + |cleanDriver: $cleanDriver + |""".stripMargin + +object CompilationInputs: + def empty: CompilationInputs = CompilationInputs(new URI(""), "", EmptyCancelToken, false) + def fromParams(params: VirtualFileParams | OffsetParams, cleanDriver: Boolean = false): CompilationInputs = + CompilationInputs(params.uri().nn, params.text().nn, params.token().nn, cleanDriver) + +class DriverAccess( + config: PresentationCompilerConfig, + driverSettings: List[String], + reportContext: PcReportContext +): + + private val taskQueue = TaskQueue(driverSettings, reportContext) + private val worker = Worker(taskQueue, config, reportContext) + + def lookup[T](f: InteractiveDriver => T): CompletableFuture[T] = + enqueueTask(taskQueue.LookupTask(f)) + + def enqueueCancellable[T](inputs: CompilationInputs) + (f: CancelToken ?=> InteractiveDriver => T): CompletableFuture[T] = + + given token: CancelToken = inputs.cancelToken + val task = taskQueue.CompilationTask(token, inputs)(f) + + enqueueTask(task) + + end enqueueCancellable + + private def enqueueTask[T](task: taskQueue.Task[T]): CompletableFuture[T] = + taskQueue.enqueue(task) + worker.scheduleProcessing() + task.future + + def shutdown(): Unit = + worker.shutdown() + taskQueue.shutdown() + + def restart() = + taskQueue.shutdown() + diff --git a/presentation-compiler/src/main/dotty/tools/pc/PcDefinitionProvider.scala b/presentation-compiler/src/main/dotty/tools/pc/PcDefinitionProvider.scala index ca5a36cefad0..abd6c7eee6fc 100644 --- a/presentation-compiler/src/main/dotty/tools/pc/PcDefinitionProvider.scala +++ b/presentation-compiler/src/main/dotty/tools/pc/PcDefinitionProvider.scala @@ -41,22 +41,21 @@ class PcDefinitionProvider( val uri = params.uri().nn val text = params.text().nn val filePath = Paths.get(uri) - driver.run( - uri, - SourceFile.virtual(filePath.toString, text) - ) + + val unit = driver.compilationUnits(uri) + val newCtx = driver.currentCtx.fresh.setCompilationUnit(unit) val pos = driver.sourcePosition(params) - val path = - Interactive.pathTo(driver.openedTrees(uri), pos)(using driver.currentCtx) + val path = Interactive.pathTo(unit.tpdTree, pos.span)(using newCtx) + + given localCtx: Context = Interactive.contextOfPath(path)(using newCtx) + val indexedContext = IndexedContext(pos)(using localCtx) - given ctx: Context = driver.localContext(params) - val indexedContext = IndexedContext(pos)(using ctx) val result = if findTypeDef then findTypeDefinitions(path, pos, indexedContext, uri) else findDefinitions(path, pos, indexedContext, uri) - if result.locations().nn.isEmpty() then fallbackToUntyped(pos, uri)(using ctx) + if result.locations().nn.isEmpty() then fallbackToUntyped(pos, uri) else result end definitions diff --git a/presentation-compiler/src/main/dotty/tools/pc/PcInlayHintsProvider.scala b/presentation-compiler/src/main/dotty/tools/pc/PcInlayHintsProvider.scala index 0734468754c0..a51ba5eb1fd0 100644 --- a/presentation-compiler/src/main/dotty/tools/pc/PcInlayHintsProvider.scala +++ b/presentation-compiler/src/main/dotty/tools/pc/PcInlayHintsProvider.scala @@ -40,20 +40,18 @@ class PcInlayHintsProvider( val filePath = Paths.get(uri).nn val sourceText = params.text().nn val text = sourceText.toCharArray().nn - val source = - SourceFile.virtual(filePath.toString, sourceText) - driver.run(uri, source) - given InlayHintsParams = params + given InlayHintsParams = params given InferredType.Text = InferredType.Text(text) given ctx: Context = driver.currentCtx - val unit = driver.currentCtx.run.nn.units.head + + val unit = driver.compilationUnits(uri) val pos = driver.sourcePosition(params) def provide(): List[InlayHint] = val deepFolder = DeepFolder[InlayHints](collectDecorations) Interactive - .pathTo(driver.openedTrees(uri), pos)(using driver.currentCtx) + .pathTo(unit.tpdTree, pos.span) .headOption .getOrElse(unit.tpdTree) .enclosedChildren(pos.span) diff --git a/presentation-compiler/src/main/dotty/tools/pc/PcReportContext.scala b/presentation-compiler/src/main/dotty/tools/pc/PcReportContext.scala new file mode 100644 index 000000000000..be76accebe54 --- /dev/null +++ b/presentation-compiler/src/main/dotty/tools/pc/PcReportContext.scala @@ -0,0 +1,15 @@ +package dotty.tools.pc + +import scala.meta.internal.metals.ReportContext +import scala.meta.internal.metals.Reporter +import scala.meta.internal.metals.EmptyReportContext + +class PcReportContext(underlying: ReportContext, pcDetails: Map[String, String]) extends ReportContext: + override def incognito: Reporter = underlying.incognito + override def unsanitized: Reporter = underlying.unsanitized + override def bloop: Reporter = underlying.bloop + def additionalData: String = pcDetails.mkString("\n") + +object PcReportContext: + def empty: PcReportContext = PcReportContext(EmptyReportContext, Map.empty) + diff --git a/presentation-compiler/src/main/dotty/tools/pc/Scala3CompilerAccess.scala b/presentation-compiler/src/main/dotty/tools/pc/Scala3CompilerAccess.scala deleted file mode 100644 index 1443fbcf37cc..000000000000 --- a/presentation-compiler/src/main/dotty/tools/pc/Scala3CompilerAccess.scala +++ /dev/null @@ -1,40 +0,0 @@ -package dotty.tools.pc - -import java.util.concurrent.ScheduledExecutorService - -import scala.concurrent.ExecutionContextExecutor -import scala.meta.internal.metals.ReportContext -import scala.meta.internal.pc.CompilerAccess -import scala.meta.pc.PresentationCompilerConfig - -import dotty.tools.dotc.reporting.StoreReporter -import dotty.tools.dotc.interactive.InteractiveDriver - -class Scala3CompilerAccess( - config: PresentationCompilerConfig, - sh: Option[ScheduledExecutorService], - newCompiler: () => Scala3CompilerWrapper -)(using ec: ExecutionContextExecutor, rc: ReportContext) - extends CompilerAccess[StoreReporter, InteractiveDriver]( - config, - sh, - newCompiler, - /* If running inside the executor, we need to reset the job queue - * Otherwise it will block indefinetely in case of infinite loops. - */ - shouldResetJobQueue = true - ): - - def newReporter = new StoreReporter(null) - - /** - * Handle the exception in order to make sure that - * we retry immediately. Otherwise, we will wait until - * the end of the timeout, which is 20s by default. - */ - protected def handleSharedCompilerException( - t: Throwable - ): Option[String] = None - - protected def ignoreException(t: Throwable): Boolean = false -end Scala3CompilerAccess diff --git a/presentation-compiler/src/main/dotty/tools/pc/Scala3CompilerWrapper.scala b/presentation-compiler/src/main/dotty/tools/pc/Scala3CompilerWrapper.scala deleted file mode 100644 index 968c144625a3..000000000000 --- a/presentation-compiler/src/main/dotty/tools/pc/Scala3CompilerWrapper.scala +++ /dev/null @@ -1,29 +0,0 @@ -package dotty.tools.pc - -import scala.meta.internal.pc.CompilerWrapper -import scala.meta.internal.pc.ReporterAccess - -import dotty.tools.dotc.reporting.StoreReporter -import dotty.tools.dotc.interactive.InteractiveDriver - -class Scala3CompilerWrapper(driver: InteractiveDriver) - extends CompilerWrapper[StoreReporter, InteractiveDriver]: - - override def compiler(): InteractiveDriver = driver - - override def resetReporter(): Unit = - val ctx = driver.currentCtx - ctx.reporter.removeBufferedMessages(using ctx) - - override def reporterAccess: ReporterAccess[StoreReporter] = - new ReporterAccess[StoreReporter]: - def reporter = driver.currentCtx.reporter.asInstanceOf[StoreReporter] - - override def askShutdown(): Unit = () - - override def isAlive(): Boolean = false - - override def stop(): Unit = {} - - override def presentationCompilerThread: Option[Thread] = None -end Scala3CompilerWrapper diff --git a/presentation-compiler/src/main/dotty/tools/pc/ScalaPresentationCompiler.scala b/presentation-compiler/src/main/dotty/tools/pc/ScalaPresentationCompiler.scala index 81bad6a2976d..f308a7495df6 100644 --- a/presentation-compiler/src/main/dotty/tools/pc/ScalaPresentationCompiler.scala +++ b/presentation-compiler/src/main/dotty/tools/pc/ScalaPresentationCompiler.scala @@ -7,41 +7,35 @@ import java.util.Optional import java.util.concurrent.CompletableFuture import java.util.concurrent.ExecutorService import java.util.concurrent.ScheduledExecutorService +import java.util.Collections import java.util as ju import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContextExecutor import scala.jdk.CollectionConverters._ import scala.language.unsafeNulls -import scala.meta.internal.metals.CompilerVirtualFileParams -import scala.meta.internal.metals.EmptyCancelToken -import scala.meta.internal.metals.EmptyReportContext -import scala.meta.internal.metals.PcQueryContext -import scala.meta.internal.metals.ReportContext import scala.meta.internal.metals.ReportLevel import scala.meta.internal.metals.StdReportContext import scala.meta.internal.mtags.CommonMtagsEnrichments.* -import scala.meta.internal.pc.CompilerAccess -import scala.meta.internal.pc.DefinitionResultImpl -import scala.meta.internal.pc.EmptyCompletionList import scala.meta.internal.pc.EmptySymbolSearch import scala.meta.internal.pc.PresentationCompilerConfigImpl import scala.meta.pc.* import scala.meta.pc.{PcSymbolInformation as IPcSymbolInformation} -import dotty.tools.dotc.reporting.StoreReporter import dotty.tools.pc.completions.CompletionProvider -import dotty.tools.pc.InferExpectedType import dotty.tools.pc.completions.OverrideCompletions import dotty.tools.pc.buildinfo.BuildInfo -import dotty.tools.pc.SymbolInformationProvider -import dotty.tools.dotc.interactive.InteractiveDriver import org.eclipse.lsp4j.DocumentHighlight import org.eclipse.lsp4j.TextEdit import org.eclipse.lsp4j as l +/** Implementation of Presentation Compiler + * + * NOTE: This class is not thread safe. Each consumer of this class should ensure that tasks are + * queued in sequence to guarantee correct caching. + */ case class ScalaPresentationCompiler( buildTargetIdentifier: String = "", buildTargetName: Option[String] = None, @@ -72,10 +66,10 @@ case class ScalaPresentationCompiler( private val forbiddenOptions = Set("-print-lines", "-print-tasty") private val forbiddenDoubleOptions = Set.empty[String] - given ReportContext = + given reportContext: PcReportContext = folderPath - .map(StdReportContext(_, _ => buildTargetName, reportsLevel)) - .getOrElse(EmptyReportContext) + .map(path => PcReportContext(StdReportContext(path, _ => buildTargetName, reportsLevel), additionalReportData)) + .getOrElse(PcReportContext.empty) override def codeAction[T]( params: OffsetParams, @@ -83,12 +77,9 @@ case class ScalaPresentationCompiler( codeActionPayload: Optional[T] ): CompletableFuture[ju.List[TextEdit]] = (codeActionId, codeActionPayload.asScala) match - case ( - CodeActionId.ConvertToNamedArguments, - Some(argIndices: ju.List[_]) - ) => - val payload = - argIndices.asScala.collect { case i: Integer => i.toInt }.toSet + case (CodeActionId.ConvertToNamedArguments, Some(argIndices: ju.List[?])) => + val payload: ju.List[Integer] = + argIndices.asScala.collect { case i: Integer => i.asInstanceOf[Integer] }.asJava convertToNamedArguments(params, payload) case (CodeActionId.ImplementAbstractMembers, _) => implementAbstractMembers(params) @@ -103,9 +94,8 @@ case class ScalaPresentationCompiler( case _ => failedFuture(new IllegalArgumentException(s"Expected range parameters")) } case (PcConvertToNamedLambdaParameters.codeActionId, _) => - compilerAccess.withNonInterruptableCompiler(List.empty[l.TextEdit].asJava, params.token) { - access => PcConvertToNamedLambdaParameters(access.compiler(), params).convertToNamedLambdaParameters - }(params.toQueryContext) + driverAccess.enqueueCancellable(CompilationInputs.fromParams(params)): driver => + PcConvertToNamedLambdaParameters(driver, params).convertToNamedLambdaParameters case (id, _) => failedFuture(new IllegalArgumentException(s"Unsupported action id $id")) private def failedFuture[T](e: Throwable): CompletableFuture[T] = @@ -124,13 +114,6 @@ case class ScalaPresentationCompiler( override def withReportsLoggerLevel(level: String): PresentationCompiler = copy(reportsLevel = ReportLevel.fromString(level)) - val compilerAccess: CompilerAccess[StoreReporter, InteractiveDriver] = - Scala3CompilerAccess( - config, - sh, - () => new Scala3CompilerWrapper(CachingDriver(driverSettings)) - )(using ec) - val driverSettings = val implicitSuggestionTimeout = List("-Ximport-suggestion-timeout", "0") val defaultFlags = List("-color:never") @@ -139,6 +122,8 @@ case class ScalaPresentationCompiler( filteredOptions ::: defaultFlags ::: implicitSuggestionTimeout ::: "-classpath" :: classpath .mkString(File.pathSeparator) :: Nil + lazy val driverAccess = DriverAccess(config, driverSettings, reportContext) + private def removeDoubleOptions(options: List[String]): List[String] = options match case head :: _ :: tail if forbiddenDoubleOptions(head) => @@ -149,168 +134,117 @@ case class ScalaPresentationCompiler( override def semanticTokens( params: VirtualFileParams ): CompletableFuture[ju.List[Node]] = - compilerAccess.withInterruptableCompiler( - new ju.ArrayList[Node](), - params.token() - ) { access => - val driver = access.compiler() - new PcSemanticTokensProvider(driver, params).provide().asJava - }(params.toQueryContext) + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => + new PcSemanticTokensProvider(driver, params) + .provide() + .asJava override def inlayHints( params: InlayHintsParams ): ju.concurrent.CompletableFuture[ju.List[l.InlayHint]] = - compilerAccess.withInterruptableCompiler( - new ju.ArrayList[l.InlayHint](), - params.token(), - ) { access => - val driver = access.compiler() - new PcInlayHintsProvider(driver, params, search) - .provide() - .asJava - }(params.toQueryContext) + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => + new PcInlayHintsProvider(driver, params, search) + .provide() + .asJava override def getTasty( targetUri: URI, isHttpEnabled: Boolean ): CompletableFuture[String] = - CompletableFuture.completedFuture { + CompletableFuture.completedFuture: TastyUtils.getTasty(targetUri, isHttpEnabled) - } def complete(params: OffsetParams): CompletableFuture[l.CompletionList] = - compilerAccess.withInterruptableCompiler( - EmptyCompletionList(), - params.token() - ) { access => - val driver = access.compiler() - new CompletionProvider( - search, - driver, - () => InteractiveDriver(driverSettings), - params, - config, - buildTargetIdentifier, - folderPath, - completionItemPriority - ).completions() - }(params.toQueryContext) + val (wasCursorApplied, completionText) = CompletionProvider.applyCompletionCursor(params) + val inputs = CompilationInputs(params.uri.nn, completionText, params.token().nn, wasCursorApplied) + + driverAccess + .enqueueCancellable(inputs): driver => + new CompletionProvider( + search, + driver, + driverSettings, + params, + config, + buildTargetIdentifier, + folderPath, + completionItemPriority, + wasCursorApplied + ).completions() def definition(params: OffsetParams): CompletableFuture[DefinitionResult] = - compilerAccess.withInterruptableCompiler( - DefinitionResultImpl.empty, - params.token() - ) { access => - val driver = access.compiler() - PcDefinitionProvider(driver, params, search).definitions() - }(params.toQueryContext) + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params.nn)): driver => + PcDefinitionProvider(driver, params, search).definitions() override def typeDefinition( params: OffsetParams ): CompletableFuture[DefinitionResult] = - compilerAccess.withInterruptableCompiler( - DefinitionResultImpl.empty, - params.token() - ) { access => - val driver = access.compiler() - PcDefinitionProvider(driver, params, search).typeDefinitions() - }(params.toQueryContext) + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => + PcDefinitionProvider(driver, params, search).typeDefinitions() def documentHighlight( params: OffsetParams ): CompletableFuture[ju.List[DocumentHighlight]] = - compilerAccess.withInterruptableCompiler( - List.empty[DocumentHighlight].asJava, - params.token() - ) { access => - val driver = access.compiler() - PcDocumentHighlightProvider(driver, params).highlights.asJava - }(params.toQueryContext) + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => + PcDocumentHighlightProvider(driver, params).highlights.asJava override def references( params: ReferencesRequest ): CompletableFuture[ju.List[ReferencesResult]] = - compilerAccess.withNonInterruptableCompiler( - List.empty[ReferencesResult].asJava, - params.file().token, - ) { access => - val driver = access.compiler() - PcReferencesProvider(driver, params) - .references() - .asJava - }(params.file().toQueryContext) + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params.file)): driver => + PcReferencesProvider(driver, params) + .references() + .asJava def inferExpectedType(params: OffsetParams): CompletableFuture[ju.Optional[String]] = - compilerAccess.withInterruptableCompiler( - Optional.empty(), - params.token, - ) { access => - val driver = access.compiler() - new InferExpectedType(search, driver, params).infer().asJava - }(params.toQueryContext) + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => + new InferExpectedType(search, driver, params).infer().asJava - def shutdown(): Unit = - compilerAccess.shutdown() + def shutdown(): Unit = driverAccess.shutdown() - def restart(): Unit = - compilerAccess.shutdownCurrentCompiler() + def restart(): Unit = driverAccess.restart() - def diagnosticsForDebuggingPurposes(): ju.List[String] = - List[String]().asJava + def diagnosticsForDebuggingPurposes(): ju.List[String] = Collections.emptyList() override def info( symbol: String ): CompletableFuture[Optional[IPcSymbolInformation]] = - compilerAccess.withNonInterruptableCompiler[Optional[IPcSymbolInformation]]( - Optional.empty(), - EmptyCancelToken, - ) { access => - SymbolInformationProvider(using access.compiler().currentCtx) - .info(symbol) - .map(_.asJava) - .asJava - }(emptyQueryContext) + driverAccess + .lookup: driver => + SymbolInformationProvider(using driver.currentCtx) + .info(symbol) + .map(_.asJava) + .asJava def semanticdbTextDocument( filename: URI, code: String ): CompletableFuture[Array[Byte]] = - val virtualFile = CompilerVirtualFileParams(filename, code) - compilerAccess.withNonInterruptableCompiler( - Array.empty[Byte], - EmptyCancelToken - ) { access => - val driver = access.compiler() - val provider = SemanticdbTextDocumentProvider(driver, folderPath) - provider.textDocument(filename, code) - }(virtualFile.toQueryContext) + driverAccess + .enqueueCancellable(CompilationInputs(filename, code)): driver => + SemanticdbTextDocumentProvider(driver, folderPath) + .textDocument(filename, code) def completionItemResolve( item: l.CompletionItem, symbol: String ): CompletableFuture[l.CompletionItem] = - compilerAccess.withNonInterruptableCompiler( - item, - EmptyCancelToken - ) { access => - val driver = access.compiler() - CompletionItemResolver.resolve(item, symbol, search, config)(using - driver.currentCtx - ) - }(emptyQueryContext) + driverAccess.lookup: driver => + CompletionItemResolver.resolve(item, symbol, search, config)(using driver.currentCtx) def autoImports( name: String, params: scala.meta.pc.OffsetParams, isExtension: java.lang.Boolean - ): CompletableFuture[ - ju.List[scala.meta.pc.AutoImportsResult] - ] = - compilerAccess.withNonInterruptableCompiler( - List.empty[scala.meta.pc.AutoImportsResult].asJava, - params.token() - ) { access => - val driver = access.compiler() + ): CompletableFuture[ju.List[scala.meta.pc.AutoImportsResult]] = + driverAccess.enqueueCancellable(CompilationInputs.fromParams(params)): driver => new AutoImportsProvider( search, driver, @@ -321,148 +255,98 @@ case class ScalaPresentationCompiler( ) .autoImports(isExtension) .asJava - }(params.toQueryContext) def implementAbstractMembers( params: OffsetParams ): CompletableFuture[ju.List[l.TextEdit]] = - val empty: ju.List[l.TextEdit] = new ju.ArrayList[l.TextEdit]() - compilerAccess.withNonInterruptableCompiler( - empty, - params.token() - ) { pc => - val driver = pc.compiler() - OverrideCompletions.implementAllAt( - params, - driver, - search, - config - ) - }(params.toQueryContext) - end implementAbstractMembers + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => + OverrideCompletions.implementAllAt( + params, + driver, + search, + config + ) override def insertInferredType( params: OffsetParams ): CompletableFuture[ju.List[l.TextEdit]] = - val empty: ju.List[l.TextEdit] = new ju.ArrayList[l.TextEdit]() - compilerAccess.withNonInterruptableCompiler( - empty, - params.token() - ) { pc => - new InferredTypeProvider(params, pc.compiler(), config, search) - .inferredTypeEdits() - .asJava - }(params.toQueryContext) + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => + new InferredTypeProvider(params, driver, config, search) + .inferredTypeEdits() + .asJava override def inlineValue( params: OffsetParams ): CompletableFuture[ju.List[l.TextEdit]] = - val empty: Either[String, List[l.TextEdit]] = Right(List()) - (compilerAccess - .withInterruptableCompiler(empty, params.token()) { pc => - new PcInlineValueProvider(pc.compiler(), params) + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => + new PcInlineValueProvider(driver, params) .getInlineTextEdits() - }(params.toQueryContext)) - .thenApply { + .thenApply: case Right(edits: List[TextEdit]) => edits.asJava case Left(error: String) => throw new DisplayableException(error) - } - end inlineValue override def extractMethod( range: RangeParams, extractionPos: OffsetParams ): CompletableFuture[ju.List[l.TextEdit]] = - val empty: ju.List[l.TextEdit] = new ju.ArrayList[l.TextEdit]() - compilerAccess.withInterruptableCompiler(empty, range.token()) { - pc => + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(range)): driver => new ExtractMethodProvider( range, extractionPos, - pc.compiler(), + driver, search, options.contains("-no-indent"), ) .extractMethod() .asJava - }(range.toQueryContext) - end extractMethod override def convertToNamedArguments( params: OffsetParams, argIndices: ju.List[Integer] ): CompletableFuture[ju.List[l.TextEdit]] = - convertToNamedArguments(params, argIndices.asScala.toSet.map(_.toInt)) - - def convertToNamedArguments( - params: OffsetParams, - argIndices: Set[Int] - ): CompletableFuture[ju.List[l.TextEdit]] = - val empty: Either[String, List[l.TextEdit]] = Right(List()) - (compilerAccess - .withNonInterruptableCompiler(empty, params.token()) { pc => + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => new ConvertToNamedArgumentsProvider( - pc.compiler(), + driver, params, - argIndices + argIndices.asScala.map(_.toInt).toSet ).convertToNamedArguments - }(params.toQueryContext)) - .thenApplyAsync { + .thenApply: case Left(error: String) => throw new DisplayableException(error) case Right(edits: List[l.TextEdit]) => edits.asJava - } - end convertToNamedArguments + override def selectionRange( params: ju.List[OffsetParams] ): CompletableFuture[ju.List[l.SelectionRange]] = - CompletableFuture.completedFuture { - compilerAccess.withSharedCompiler( - List.empty[l.SelectionRange].asJava - ) { pc => - new SelectionRangeProvider( - pc.compiler(), - params, - ).selectionRange().asJava - }(params.asScala.headOption.map(_.toQueryContext).getOrElse(emptyQueryContext)) - } - end selectionRange + if params.isEmpty then + CompletableFuture.completedFuture(Collections.emptyList()) + else + driverAccess.enqueueCancellable(CompilationInputs.fromParams(params.asScala.head)): driver => + new SelectionRangeProvider(driver, params).selectionRange().asJava def hover( params: OffsetParams ): CompletableFuture[ju.Optional[HoverSignature]] = - compilerAccess.withNonInterruptableCompiler( - ju.Optional.empty[HoverSignature](), - params.token() - ) { access => - val driver = access.compiler() - HoverProvider.hover(params, driver, search, config.hoverContentType()) - }(params.toQueryContext) - end hover + driverAccess + .enqueueCancellable(CompilationInputs.fromParams(params)): driver => + HoverProvider.hover(params, driver, search, config.hoverContentType()) def prepareRename( params: OffsetParams ): CompletableFuture[ju.Optional[l.Range]] = - compilerAccess.withNonInterruptableCompiler( - Optional.empty[l.Range](), - params.token() - ) { access => - val driver = access.compiler() - Optional.ofNullable( - PcRenameProvider(driver, params, None).prepareRename().orNull - ) - }(params.toQueryContext) + driverAccess.enqueueCancellable(CompilationInputs.fromParams(params)): driver => + PcRenameProvider(driver, params, None).prepareRename().asJava def rename( params: OffsetParams, name: String ): CompletableFuture[ju.List[l.TextEdit]] = - compilerAccess.withNonInterruptableCompiler( - List[l.TextEdit]().asJava, - params.token() - ) { access => - val driver = access.compiler() + driverAccess.enqueueCancellable(CompilationInputs.fromParams(params)): driver => PcRenameProvider(driver, params, Some(name)).rename().asJava - }(params.toQueryContext) def newInstance( buildTargetIdentifier: String, @@ -476,13 +360,8 @@ case class ScalaPresentationCompiler( ) def signatureHelp(params: OffsetParams): CompletableFuture[l.SignatureHelp] = - compilerAccess.withNonInterruptableCompiler( - new l.SignatureHelp(), - params.token() - ) { access => - val driver = access.compiler() + driverAccess.enqueueCancellable(CompilationInputs.fromParams(params)): driver => SignatureHelpProvider.signatureHelp(driver, params, search) - }(params.toQueryContext) override def didChange( params: VirtualFileParams @@ -490,10 +369,7 @@ case class ScalaPresentationCompiler( CompletableFuture.completedFuture(Nil.asJava) override def didClose(uri: URI): Unit = - compilerAccess.withNonInterruptableCompiler( - (), - EmptyCancelToken - ) { access => access.compiler().close(uri) }(emptyQueryContext) + driverAccess.lookup(_.close(uri)) override def withExecutorService( executorService: ExecutorService @@ -516,21 +392,15 @@ case class ScalaPresentationCompiler( def withWorkspace(workspace: Path): PresentationCompiler = copy(folderPath = Some(workspace)) - override def isLoaded() = compilerAccess.isLoaded() - - def additionalReportData() = - s"""|Scala version: $scalaVersion - |Classpath: - |${classpath - .map(path => s"$path [${if path.exists then "exists" else "missing"} ]") - .mkString(", ")} - |Options: - |${options.mkString(" ")} - |""".stripMargin + override def isLoaded() = true - extension (params: VirtualFileParams) - def toQueryContext = PcQueryContext(Some(params), additionalReportData) - - def emptyQueryContext = PcQueryContext(None, additionalReportData) + private def additionalReportData = + Map( + "scalaVersion" -> scalaVersion, + "classpath" -> classpath + .map(path => s"$path [ ${if path.exists then "exists" else "missing"} ]") + .mkString(", "), + "options" -> options.mkString(" ") + ) end ScalaPresentationCompiler diff --git a/presentation-compiler/src/main/dotty/tools/pc/TaskQueue.scala b/presentation-compiler/src/main/dotty/tools/pc/TaskQueue.scala new file mode 100644 index 000000000000..e9f6d0f3db47 --- /dev/null +++ b/presentation-compiler/src/main/dotty/tools/pc/TaskQueue.scala @@ -0,0 +1,113 @@ +package dotty.tools.pc + +import dotty.tools.dotc.interactive.InteractiveDriver +import scala.meta.pc.CancelToken +import java.util.concurrent.CompletableFuture +import java.util.concurrent.atomic.AtomicBoolean +import scala.meta.internal.pc.CompilerThrowable +import scala.meta.internal.metals.Report +import java.util.logging.Logger +import java.util.logging.Level +import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.CancellationException +import java.util.concurrent.ConcurrentLinkedQueue +import dotty.tools.dotc.sbt.interfaces.ProgressCallback + +class TaskQueue(driverSettings: List[String], reportContext: PcReportContext): + private val logger: Logger = Logger.getLogger(getClass.getName).nn + + private val queue: ConcurrentLinkedQueue[Task[?]] = ConcurrentLinkedQueue[Task[?]]() // Not started tasks only + private val _driver: AtomicReference[CachingDriver] = AtomicReference(newCompiler()) + + private def newCompiler(): CachingDriver = new CachingDriver(driverSettings) + + def enqueue(task: Task[?]): Boolean = queue.add(task) + def dequeue(): Option[Task[?]] = Option(queue.poll().nn) + + def shutdownCurrentCompiler(): Unit = + _driver.set(new CachingDriver(driverSettings)) + + private def getDriver(cleanDriver: Boolean): CachingDriver = + if cleanDriver then new CachingDriver(driverSettings) + else _driver.get().nn + + sealed trait Task[T]: + private val _interrupted = AtomicBoolean(false) + + val inputs: CompilationInputs + val future: CompletableFuture[T] = new CompletableFuture() + + def execute(): Unit + + final def interrupt(): Unit = _interrupted.set(true) + final def isInterrupted: Boolean = _interrupted.get() + + case class CompilationTask[T](token: CancelToken, val inputs: CompilationInputs)(f: InteractiveDriver => T) extends Task[T]: + + val callback = new ProgressCallback: + override def isCancelled(): Boolean = token.nn.isCanceled() || isInterrupted || Thread.interrupted() + + private def compile(): T = + import inputs.* + val driver = getDriver(cleanDriver) + driver.runWithProgressCallback(uri, code, callback) + f(driver) + + override def execute(): Unit = this.synchronized: + try + if callback.isCancelled then future.cancel(true) + else future.complete(compile()) + catch case ex: Exception => + // ProgressCallback.isCancelled = true throws random errors in the compiler, lets return CancellationException instead + if token.nn.isCanceled then future.completeExceptionally(new CancellationException()) + else if isInterrupted then future.completeExceptionally(new InterruptedException()) + else + handleError(ex, (inputs)) + future.completeExceptionally(ex) + + case class LookupTask[T](f: InteractiveDriver => T) extends Task[T]: + override val inputs: CompilationInputs = CompilationInputs.empty + + private def compile(): T = + val driver = _driver.get().nn + f(driver) + + override def execute(): Unit = + try + future.complete(compile()) + catch case ex: Exception => + future.completeExceptionally(ex) + + private def handleError(e: Throwable, compilationInputs: CompilationInputs): Unit = + val error = CompilerThrowable.trimStackTrace(e) + val report = + Report( + "compiler-error", + s"""|occurred in the Presentation Compiler. + | + |Presentation Compiler configuration: + | + |action parameters: + |${compilationInputs.show} + | + |Additional data: + | ${reportContext.additionalData} + |""".stripMargin, + error, + path = Some(compilationInputs.uri) + ) + + val pathToReport = reportContext.unsanitized.create(report) + pathToReport match { + case Some(path) => + logger.log( + Level.SEVERE, + s"A severe compiler error occurred, full details of the error can be found in the error report $path" + ) + case _ => + logger.log(Level.SEVERE, error.getMessage, error) + } + + shutdownCurrentCompiler() + + def shutdown() = shutdownCurrentCompiler() diff --git a/presentation-compiler/src/main/dotty/tools/pc/Worker.scala b/presentation-compiler/src/main/dotty/tools/pc/Worker.scala new file mode 100644 index 000000000000..343e9dd3cd5b --- /dev/null +++ b/presentation-compiler/src/main/dotty/tools/pc/Worker.scala @@ -0,0 +1,107 @@ +package dotty.tools.pc + +import java.util.concurrent.ScheduledExecutorService +import scala.meta.pc.PresentationCompilerConfig +import scala.meta.internal.metals.Report +import java.util.logging.Logger +import java.util.logging.Level +import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit +import java.util.concurrent.ScheduledFuture + +case class InfiniteCompilationException(msg: String) extends Exception // we should expose this in mtags-interfaces so we can catch it in IDE's + +class Worker(queue: TaskQueue, config: PresentationCompilerConfig, reportContext: PcReportContext): + private val logger: Logger = Logger.getLogger(getClass.getName).nn + + private val executor: ScheduledExecutorService = Executors.newSingleThreadScheduledExecutor().nn + private val timeoutExecutor: ScheduledExecutorService = Executors.newSingleThreadScheduledExecutor().nn + private var _currentTask: AtomicReference[Option[queue.Task[?]]] = AtomicReference(None) + + def scheduleProcessing(): Unit = + executor.execute(() => processQueue()) + + private def handleInfiniteCompilation(thread: Thread, task: queue.Task[?]): Unit = + val compilationInputs = task.inputs + val stacktrace = + thread.getStackTrace().nn + .map(_.nn.toString()) + .mkString("\n") + + val shortMessage = + """Fatal compiler error encountered. + |Please send this report to compiler team or create an issue at https://github.com/scala/scala3/issues""".stripMargin + + val report = + Report( + "fatal-compiler-error", + s"""|occurred in the presentation compiler. + | + |There is very high chance that you've just discovered infinite compilation. + |Please report this fatal error to the compiler team by uploading this report. + | + |You can do at https://github.com/scala/scala3/issues + | + |If your code is sensitive please make sure to remove code from this report. + | + |Additional data: + | ${reportContext.additionalData} + |Stacktrace: + | ${stacktrace} + | + |action parameters: + |${compilationInputs.show} + |""".stripMargin, + shortMessage, + path = Some(compilationInputs.uri) + ) + + reportContext.unsanitized.create(report) match + case Some(path) => + logger.log(Level.SEVERE, s"$shortMessage. Full details of the error can be found in the error report $path") + case _ => + logger.log(Level.SEVERE, shortMessage + "\n" + stacktrace.indent(2)) + task.future.completeExceptionally(new InfiniteCompilationException(shortMessage)) + + private def scheduleTaskTimeout(task: queue.Task[?]): ScheduledFuture[Unit] = + timeoutExecutor.schedule[Unit](() => { + if !task.future.isDone() then + logger.log(Level.WARNING, s"Task timeout detected. Attempting graceful cancellation.") + task.interrupt() // Attempt graceful cancellation + }, config.timeoutDelay(), config.timeoutUnit()).nn + + private def scheduleForcedShutdown(executorThread: Thread, task: queue.Task[?]): ScheduledFuture[Unit] = + timeoutExecutor.schedule[Unit](() => { + logger.log(Level.SEVERE, s"Task did not respond to cancellation after 10 seconds. Forcing executor shutdown.") + executor.shutdown() + try + if !executor.awaitTermination(5, TimeUnit.SECONDS) then + executor.shutdownNow() + catch + case _: InterruptedException => executor.shutdownNow() + handleInfiniteCompilation(executorThread, task) + () + }, 60, TimeUnit.SECONDS).nn // refactor and use value from config + + private def processQueue(): Unit = + try + val thisThread = Thread.currentThread().nn + queue.dequeue().foreach: task => + _currentTask.set(Some(task)) + + val initialTimeoutFuture = scheduleTaskTimeout(task) + val forceShutdownFuture = scheduleForcedShutdown(thisThread, task) + + task.execute() // actual compilation happens here + + initialTimeoutFuture.cancel(false) + forceShutdownFuture.cancel(false) + + processQueue() + finally + _currentTask.getAndSet(None).nn.foreach(_.interrupt()) + + def shutdown() = + executor.shutdown() + timeoutExecutor.shutdown() diff --git a/presentation-compiler/src/main/dotty/tools/pc/completions/CompletionProvider.scala b/presentation-compiler/src/main/dotty/tools/pc/completions/CompletionProvider.scala index ef9f77eb58fc..3fe01b227f69 100644 --- a/presentation-compiler/src/main/dotty/tools/pc/completions/CompletionProvider.scala +++ b/presentation-compiler/src/main/dotty/tools/pc/completions/CompletionProvider.scala @@ -44,144 +44,14 @@ object CompletionProvider: val softKeywords = Tokens.softModifierNames + nme.as + nme.derives + nme.extension + nme.throws + nme.using Tokens.keywords.toList.map(Tokens.tokenString) ++ softKeywords.map(_.toString) -class CompletionProvider( - search: SymbolSearch, - cachingDriver: InteractiveDriver, - freshDriver: () => InteractiveDriver, - params: OffsetParams, - config: PresentationCompilerConfig, - buildTargetIdentifier: String, - folderPath: Option[Path], - referenceCounter: CompletionItemPriority -)(using reports: ReportContext): - def completions(): CompletionList = - val uri = params.uri().nn - val text = params.text().nn - - val (wasCursorApplied, code) = applyCompletionCursor(params) - val sourceFile = SourceFile.virtual(uri, code) - - /** Creating a new fresh driver is way slower than reusing existing one, - * but runnig a compilation has side effects that modifies the state of the driver. - * We don't want to affect cachingDriver state with compilation including "CURSOR" suffix. - * - * We could in theory save this fresh driver for reuse, but it is a choice between extra memory usage and speed. - * The scenario in which "CURSOR" is applied (empty query or query equal to any keyword) has a slim chance of happening. - */ - - val driver = if wasCursorApplied then freshDriver() else cachingDriver - driver.run(uri, sourceFile) - - given ctx: Context = driver.currentCtx - val pos = driver.sourcePosition(params) - val (items, isIncomplete) = driver.compilationUnits.get(uri) match - case Some(unit) => - val newctx = ctx.fresh - .setCompilationUnit(unit) - .setProfiler(Profiler()(using ctx)) - .withPhase(Phases.typerPhase(using ctx)) - val tpdPath0 = Interactive.pathTo(unit.tpdTree, pos.span)(using newctx) - val adjustedPath = Interactive.resolveTypedOrUntypedPath(tpdPath0, pos)(using newctx) - - val tpdPath = tpdPath0 match - case Select(qual, name) :: tail - /** If for any reason we end up in param after lifting, we want to inline the synthetic val: - * List(1).iterator.sliding@@ will be transformed into: - * - * 1| val $1$: Iterator[Int] = List.apply[Int]([1 : Int]*).iterator - * 2| { - * 3| def $anonfun(size: Int, step: Int): $1$.GroupedIterator[Int] = - * 4| $1$.sliding[Int](size, step) - * 5| closure($anonfun) - * 6| }:((Int, Int) => Iterator[Int]#GroupedIterator[Int]) - * - * With completion being run at line 4 at @@: - * 4| $1$.sliding@@[Int](size, step) - * - */ - if qual.symbol.is(Flags.Synthetic) && qual.span.isZeroExtent && qual.symbol.name.isInstanceOf[DerivedName] => - qual.symbol.defTree match - case valdef: ValDef if !valdef.rhs.isEmpty => Select(valdef.rhs, name) :: tail - case _ => tpdPath0 - case _ => tpdPath0 - - - val locatedCtx = Interactive.contextOfPath(tpdPath)(using newctx) - val indexedCtx = IndexedContext(pos)(using locatedCtx) - - val completionPos = CompletionPos.infer(pos, params, adjustedPath, wasCursorApplied)(using locatedCtx) - - val autoImportsGen = AutoImports.generator( - completionPos.toSourcePosition, - text, - unit.tpdTree, - unit.comments, - indexedCtx, - config - ) - - val (completions, searchResult) = - new Completions( - text, - locatedCtx, - search, - buildTargetIdentifier, - completionPos, - indexedCtx, - tpdPath, - adjustedPath, - config, - folderPath, - autoImportsGen, - unit.comments, - driver.settings, - referenceCounter - ).completions() - - val items = completions.zipWithIndex.map { case (item, idx) => - completionItems( - item, - idx, - autoImportsGen, - completionPos, - tpdPath, - indexedCtx - )(using locatedCtx) - } - val isIncomplete = searchResult match - case SymbolSearch.Result.COMPLETE => false - case SymbolSearch.Result.INCOMPLETE => true - (items, isIncomplete) - case None => (Nil, false) - - new CompletionList( - isIncomplete, - items.asJava - ) - end completions - - /** - * In case if completion comes from empty line like: - * {{{ - * class Foo: - * val a = 1 - * @@ - * }}} - * it's required to modify actual code by additional Ident. - * - * Otherwise, completion poisition doesn't point at any tree - * because scala parser trim end position to the last statement pos. - */ - private def applyCompletionCursor(params: OffsetParams): (Boolean, String) = + def applyCompletionCursor(params: OffsetParams): (Boolean, String) = val text = params.text().nn val offset = params.offset().nn val query = Completion.naiveCompletionPrefix(text, offset) - if offset > 0 && text.charAt(offset - 1).isUnicodeIdentifierPart && !CompletionProvider.allKeywords.contains(query) then false -> text else val isStartMultilineComment = - val i = params.offset() i >= 3 && (text.charAt(i - 1) match case '*' => @@ -198,6 +68,109 @@ class CompletionProvider( ) end applyCompletionCursor +class CompletionProvider( + search: SymbolSearch, + driver: InteractiveDriver, + driverSettings: List[String], + params: OffsetParams, + config: PresentationCompilerConfig, + buildTargetIdentifier: String, + folderPath: Option[Path], + referenceCounter: CompletionItemPriority, + wasCursorApplied: Boolean +)(using reports: ReportContext): + def completions(): CompletionList = + val uri = params.uri().nn + val text = params.text().nn + + val unit = driver.compilationUnits(uri) + val ctx = driver.currentCtx + val pos = driver.sourcePosition(params) + + val newctx: Context = ctx.fresh + .setCompilationUnit(unit) + .setProfiler(Profiler()(using ctx)) + .withPhase(Phases.typerPhase(using ctx)) + + val tpdPath0 = Interactive.pathTo(unit.tpdTree, pos.span)(using newctx) + val adjustedPath = Interactive.resolveTypedOrUntypedPath(tpdPath0, pos)(using newctx) + + given context: Context = newctx + val tpdPath = tpdPath0 match + case Select(qual, name) :: tail + /** If for any reason we end up in param after lifting, we want to inline the synthetic val: + * List(1).iterator.sliding@@ will be transformed into: + * + * 1| val $1$: Iterator[Int] = List.apply[Int]([1 : Int]*).iterator + * 2| { + * 3| def $anonfun(size: Int, step: Int): $1$.GroupedIterator[Int] = + * 4| $1$.sliding[Int](size, step) + * 5| closure($anonfun) + * 6| }:((Int, Int) => Iterator[Int]#GroupedIterator[Int]) + * + * With completion being run at line 4 at @@: + * 4| $1$.sliding@@[Int](size, step) + * + */ + if qual.symbol.is(Flags.Synthetic) && qual.span.isZeroExtent && qual.symbol.name.isInstanceOf[DerivedName] => + qual.symbol.defTree match + case valdef: ValDef if !valdef.rhs.isEmpty => Select(valdef.rhs, name) :: tail + case _ => tpdPath0 + case _ => tpdPath0 + + + val locatedCtx: Context = Interactive.contextOfPath(tpdPath)(using newctx) + val indexedCtx = IndexedContext(pos)(using locatedCtx) + + val completionPos = CompletionPos.infer(pos, params, adjustedPath, wasCursorApplied)(using locatedCtx) + + val autoImportsGen = AutoImports.generator( + completionPos.toSourcePosition, + text, + unit.tpdTree, + unit.comments, + indexedCtx, + config + ) + + val (completions, searchResult) = + new Completions( + text, + locatedCtx, + search, + buildTargetIdentifier, + completionPos, + indexedCtx, + tpdPath, + adjustedPath, + config, + folderPath, + autoImportsGen, + unit.comments, + driverSettings, + referenceCounter + ).completions() + + val items = completions.zipWithIndex.map { case (item, idx) => + completionItems( + item, + idx, + autoImportsGen, + completionPos, + tpdPath, + indexedCtx + )(using locatedCtx) + } + val isIncomplete = searchResult match + case SymbolSearch.Result.COMPLETE => false + case SymbolSearch.Result.INCOMPLETE => true + + new CompletionList( + isIncomplete, + items.asJava + ) + end completions + private def completionItems( completion: CompletionValue, idx: Int, diff --git a/presentation-compiler/test/dotty/tools/pc/tests/CompilerCachingSuite.scala b/presentation-compiler/test/dotty/tools/pc/tests/CompilerCachingSuite.scala index b2d837e2ff50..72cc46ea48f1 100644 --- a/presentation-compiler/test/dotty/tools/pc/tests/CompilerCachingSuite.scala +++ b/presentation-compiler/test/dotty/tools/pc/tests/CompilerCachingSuite.scala @@ -28,18 +28,18 @@ class CompilerCachingSuite extends BasePCSuite: private def checkCompilationCount(expected: Int): Unit = presentationCompiler match case pc: ScalaPresentationCompiler => - val compilations = pc.compilerAccess.withNonInterruptableCompiler(-1, EmptyCancelToken) { driver => - driver.compiler().currentCtx.runId - }(emptyQueryContext).get(timeout.length, timeout.unit) + val compilations = pc.driverAccess.lookup { driver => + driver.currentCtx.runId + }.get(timeout.length, timeout.unit) assertEquals(expected, compilations, s"Expected $expected compilations but got $compilations") case _ => throw IllegalStateException("Presentation compiler should always be of type of ScalaPresentationCompiler") private def getContext(): Context = presentationCompiler match case pc: ScalaPresentationCompiler => - pc.compilerAccess.withNonInterruptableCompiler(null, EmptyCancelToken) { driver => - driver.compiler().currentCtx - }(emptyQueryContext).get(timeout.length, timeout.unit) + pc.driverAccess.lookup { driver => + driver.currentCtx + }.get(timeout.length, timeout.unit) case _ => throw IllegalStateException("Presentation compiler should always be of type of ScalaPresentationCompiler") private def emptyQueryContext = PcQueryContext(None, () => "")(using EmptyReportContext) diff --git a/presentation-compiler/test/dotty/tools/pc/tests/completion/CompletionCancelSuite.scala b/presentation-compiler/test/dotty/tools/pc/tests/completion/CompletionCancelSuite.scala index c1d0e017def7..5f7de5cb6842 100644 --- a/presentation-compiler/test/dotty/tools/pc/tests/completion/CompletionCancelSuite.scala +++ b/presentation-compiler/test/dotty/tools/pc/tests/completion/CompletionCancelSuite.scala @@ -21,6 +21,7 @@ import scala.language.unsafeNulls import dotty.tools.pc.base.BaseCompletionSuite import org.junit.Test +import org.junit.Assert class CompletionCancelSuite extends BaseCompletionSuite: @@ -35,20 +36,30 @@ class CompletionCancelSuite extends BaseCompletionSuite: */ class AlwaysCancelToken extends CancelToken: val cancel = new CompletableFuture[lang.Boolean]() - var isCancelled = new AtomicBoolean(false) override def onCancel(): CompletionStage[lang.Boolean] = cancel override def checkCanceled(): Unit = - if (isCancelled.compareAndSet(false, true)) { + cancel.complete(true) + throw new CancellationException("Always Cancel Token") + + /** + * A cancel token that cancels after 5 `checkCancelled` calls. + */ + class DelayedCancelToken extends CancelToken: + val cancel = new CompletableFuture[lang.Boolean]() + override def onCancel(): CompletionStage[lang.Boolean] = cancel + var i = 0 + override def checkCanceled(): Unit = + i += 1 + if i > 5 then cancel.complete(true) - } else - Thread.sleep(10) + throw new CancellationException("Delayed Cancel Token") def checkCancelled( + token: CancelToken, query: String, - expected: String + expected: String, ): Unit = val (code, offset) = params(query) - val token = new AlwaysCancelToken try presentationCompiler .complete( @@ -62,8 +73,8 @@ class CompletionCancelSuite extends BaseCompletionSuite: .get() fail("Expected completion request to be interrupted") catch - case InterruptException() => - assert(token.isCancelled.get()) + case _ => + assert(token.onCancel().toCompletableFuture.get()) // assert that regular completion works as expected. val completion = presentationCompiler @@ -83,8 +94,9 @@ class CompletionCancelSuite extends BaseCompletionSuite: assertNoDiff(expected, obtained) - @Test def `basic` = + @Test def `cancel-before-start` = checkCancelled( + AlwaysCancelToken(), """ |object A { | val x = asser@@ @@ -95,48 +107,80 @@ class CompletionCancelSuite extends BaseCompletionSuite: |""".stripMargin ) - /** - * A cancel token to simulate infinite compilation - */ - object FreezeCancelToken extends CancelToken: - val cancel = new CompletableFuture[lang.Boolean]() - var isCancelled = new AtomicBoolean(false) - override def onCancel(): CompletionStage[lang.Boolean] = cancel - override def checkCanceled(): Unit = - var hello = true - var i = 0 - while (hello) i += 1 - hello = false - - @Test def `break-compilation` = - val query = """ - |object A { - | val x = asser@@ - |} - """.stripMargin - val (code, offset) = params(query) - val uri = URI.create("file:///A.scala") - try - presentationCompiler - .complete( - CompilerOffsetParams( - uri, - code, - offset, - FreezeCancelToken - ) - ) - .get() - catch case _: CancellationException => () + @Test def `cancel-during-compilation` = + checkCancelled( + DelayedCancelToken(), + """ + |object A { + | val x = asse@@ + |} + """.stripMargin, + """|assert(inline assertion: Boolean): Unit + |assert(inline assertion: Boolean, inline message: => Any): Unit + |""".stripMargin + ) - val res = presentationCompiler - .complete( - CompilerOffsetParams( - uri, - code, - offset, - EmptyCancelToken - ) - ) - .get() - assert(res.getItems().asScala.nonEmpty) + // With new assumptions we can't run this test without forking the JVM. + // We now expect to handle infinite compilations at the call site. + // + // This is very dangerous and potentially will make users unresponsive, + // and there is also no good way to handle infinite compilations. + // + // We will try to catch all of them, but right now the approach that + // ensures no leaks is the way to go. + // + // In previous implementations this test was working, because we + // didn't log, catch the infinite computations, we've just created new + // presentation compiler driver and went straight back to work, while + // this zombie thread was still stuck. + // + // /** + // * A cancel token to simulate infinite compilation + // */ + // object FreezeCancelToken extends CancelToken: + // val cancel = new CompletableFuture[lang.Boolean]() + // var isCancelled = new AtomicBoolean(false) + // var count = 0 + // override def onCancel(): CompletionStage[lang.Boolean] = cancel + // override def checkCanceled(): Unit = + // var hello = false + // var i = 0 + // count += 1 + // if count > 2 then hello = true + // while (hello) i += 1 + // hello = false + + // @Test def `zombie-task-detection` = + // val query = """ + // |object A { + // | val x = asser@@ + // |} + // """.stripMargin + // val (code, offset) = params(query) + // val uri = URI.create("file:///A.scala") + + // Assert.assertThrows(classOf[InfiniteCompilationException], () => + // try + // presentationCompiler + // .complete( + // CompilerOffsetParams( + // uri, + // code, + // offset, + // FreezeCancelToken + // ) + // ).get() + // catch case _: CancellationException => () + // ) + + // val res = presentationCompiler + // .complete( + // CompilerOffsetParams( + // uri, + // code, + // offset, + // EmptyCancelToken + // ) + // ) + // .get() + // assert(res.getItems().asScala.nonEmpty) diff --git a/presentation-compiler/test/dotty/tools/pc/tests/edit/AutoImplementAbstractMembersSuite.scala b/presentation-compiler/test/dotty/tools/pc/tests/edit/AutoImplementAbstractMembersSuite.scala index 2df69cc85af2..16b87704957d 100644 --- a/presentation-compiler/test/dotty/tools/pc/tests/edit/AutoImplementAbstractMembersSuite.scala +++ b/presentation-compiler/test/dotty/tools/pc/tests/edit/AutoImplementAbstractMembersSuite.scala @@ -1,6 +1,7 @@ package dotty.tools.pc.tests.edit import java.net.URI +import java.nio.file.Paths import scala.meta.internal.jdk.CollectionConverters.* import scala.meta.internal.metals.CompilerOffsetParams @@ -1314,9 +1315,10 @@ class AutoImplementAbstractMembersSuite extends BaseCodeActionSuite: filename: String = "A.scala" ): List[l.TextEdit] = val (code, _, offset) = params(original) + val uri = Paths.get(filename).toUri() val result = presentationCompiler .implementAbstractMembers( - CompilerOffsetParams(URI.create(filename), code, offset, cancelToken) + CompilerOffsetParams(uri, code, offset, cancelToken) ) .get() result.asScala.toList