From 1a5d1321a10dd6961aa2b672ee0f7d1d4184ad4c Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Thu, 2 May 2024 14:45:55 -0700 Subject: [PATCH 01/46] Implement initial background indexing of a project Implements an initial background index when the project is opened. The following will be implemented in follow-up PRs: - Resolving package dependencies - Preparing dependent modules - Watching for file updates --- .gitignore | 1 + Package.swift | 2 + Sources/LSPLogging/Logging.swift | 2 +- Sources/LSPLogging/NonDarwinLogging.swift | 2 +- Sources/SKCore/BuildSystemManager.swift | 22 +- .../SKTestSupport/SwiftPMTestProject.swift | 7 +- Sources/SKTestSupport/WrappedSemaphore.swift | 36 +++ Sources/SemanticIndex/CMakeLists.txt | 2 + .../SemanticIndex/SemanticIndexManager.swift | 167 ++++++++++ .../UpdateIndexStoreTaskDescription.swift | 302 ++++++++++++++++++ Sources/SourceKitLSP/SourceKitLSPServer.swift | 19 +- Sources/SourceKitLSP/Workspace.swift | 51 ++- Tests/SKCoreTests/TaskSchedulerTests.swift | 23 +- .../BackgroundIndexingTests.swift | 155 +++++++++ .../SourceKitLSPTests/BuildSystemTests.swift | 3 +- 15 files changed, 759 insertions(+), 35 deletions(-) create mode 100644 Sources/SKTestSupport/WrappedSemaphore.swift create mode 100644 Sources/SemanticIndex/SemanticIndexManager.swift create mode 100644 Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift create mode 100644 Tests/SourceKitLSPTests/BackgroundIndexingTests.swift diff --git a/.gitignore b/.gitignore index b13934290..32c80ecff 100644 --- a/.gitignore +++ b/.gitignore @@ -2,6 +2,7 @@ default.profraw Package.resolved /.build +/.index-build /Packages /*.xcodeproj /*.sublime-project diff --git a/Package.swift b/Package.swift index b93f1079f..0f8e29557 100644 --- a/Package.swift +++ b/Package.swift @@ -173,6 +173,8 @@ let package = Package( .target( name: "SemanticIndex", dependencies: [ + "CAtomics", + "LanguageServerProtocol", "LSPLogging", "SKCore", .product(name: "IndexStoreDB", package: "indexstore-db"), diff --git a/Sources/LSPLogging/Logging.swift b/Sources/LSPLogging/Logging.swift index 079924750..35437e71e 100644 --- a/Sources/LSPLogging/Logging.swift +++ b/Sources/LSPLogging/Logging.swift @@ -13,7 +13,7 @@ /// Which log level to use (from https://developer.apple.com/wwdc20/10168?time=604) /// - Debug: Useful only during debugging (only logged during debugging) /// - Info: Helpful but not essential for troubleshooting (not persisted, logged to memory) -/// - Notice/log (Default): Essential for troubleshooting +/// - Notice/log/default: Essential for troubleshooting /// - Error: Error seen during execution /// - Used eg. if the user sends an erroneous request or if a request fails /// - Fault: Bug in program diff --git a/Sources/LSPLogging/NonDarwinLogging.swift b/Sources/LSPLogging/NonDarwinLogging.swift index f89d568b9..332e3e7c9 100644 --- a/Sources/LSPLogging/NonDarwinLogging.swift +++ b/Sources/LSPLogging/NonDarwinLogging.swift @@ -330,7 +330,7 @@ public struct NonDarwinLogger: Sendable { log(level: .info, message) } - /// Log a message at the `log` level. + /// Log a message at the `default` level. public func log(_ message: NonDarwinLogMessage) { log(level: .default, message) } diff --git a/Sources/SKCore/BuildSystemManager.swift b/Sources/SKCore/BuildSystemManager.swift index ea726f942..a7100f9ac 100644 --- a/Sources/SKCore/BuildSystemManager.swift +++ b/Sources/SKCore/BuildSystemManager.swift @@ -159,7 +159,8 @@ extension BuildSystemManager { /// references to that C file in the build settings by the header file. public func buildSettingsInferredFromMainFile( for document: DocumentURI, - language: Language + language: Language, + logBuildSettings: Bool = true ) async -> FileBuildSettings? { let mainFile = await mainFile(for: document, language: language) guard var settings = await buildSettings(for: mainFile, language: language) else { @@ -170,7 +171,9 @@ extension BuildSystemManager { // to reference `document` instead of `mainFile`. settings = settings.patching(newFile: document.pseudoPath, originalFile: mainFile.pseudoPath) } - await BuildSettingsLogger.shared.log(settings: settings, for: document) + if logBuildSettings { + await BuildSettingsLogger.shared.log(settings: settings, for: document) + } return settings } @@ -349,16 +352,24 @@ extension BuildSystemManager { // MARK: - Build settings logger /// Shared logger that only logs build settings for a file once unless they change -fileprivate actor BuildSettingsLogger { - static let shared = BuildSettingsLogger() +public actor BuildSettingsLogger { + public static let shared = BuildSettingsLogger() private var loggedSettings: [DocumentURI: FileBuildSettings] = [:] - func log(settings: FileBuildSettings, for uri: DocumentURI) { + public func log(level: LogLevel = .default, settings: FileBuildSettings, for uri: DocumentURI) { guard loggedSettings[uri] != settings else { return } loggedSettings[uri] = settings + Self.log(level: level, settings: settings, for: uri) + } + + /// Log the given build settings. + /// + /// In contrast to the instance method `log`, this will always log the build settings. The instance method only logs + /// the build settings if they have changed. + public static func log(level: LogLevel = .default, settings: FileBuildSettings, for uri: DocumentURI) { let log = """ Compiler Arguments: \(settings.compilerArguments.joined(separator: "\n")) @@ -370,6 +381,7 @@ fileprivate actor BuildSettingsLogger { let chunks = splitLongMultilineMessage(message: log) for (index, chunk) in chunks.enumerated() { logger.log( + level: level, """ Build settings for \(uri.forLogging) (\(index + 1)/\(chunks.count)) \(chunk) diff --git a/Sources/SKTestSupport/SwiftPMTestProject.swift b/Sources/SKTestSupport/SwiftPMTestProject.swift index aa2737cea..72334c0d6 100644 --- a/Sources/SKTestSupport/SwiftPMTestProject.swift +++ b/Sources/SKTestSupport/SwiftPMTestProject.swift @@ -43,6 +43,7 @@ public class SwiftPMTestProject: MultiFileTestProject { build: Bool = false, allowBuildFailure: Bool = false, serverOptions: SourceKitLSPServer.Options = .testDefault, + pollIndex: Bool = true, usePullDiagnostics: Bool = true, testName: String = #function ) async throws { @@ -77,8 +78,10 @@ public class SwiftPMTestProject: MultiFileTestProject { try await Self.build(at: self.scratchDirectory) } } - // Wait for the indexstore-db to finish indexing - _ = try await testClient.send(PollIndexRequest()) + if pollIndex { + // Wait for the indexstore-db to finish indexing + _ = try await testClient.send(PollIndexRequest()) + } } /// Build a SwiftPM package package manifest is located in the directory at `path`. diff --git a/Sources/SKTestSupport/WrappedSemaphore.swift b/Sources/SKTestSupport/WrappedSemaphore.swift new file mode 100644 index 000000000..ee2036557 --- /dev/null +++ b/Sources/SKTestSupport/WrappedSemaphore.swift @@ -0,0 +1,36 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift.org open source project +// +// Copyright (c) 2014 - 2024 Apple Inc. and the Swift project authors +// Licensed under Apache License v2.0 with Runtime Library Exception +// +// See https://swift.org/LICENSE.txt for license information +// See https://swift.org/CONTRIBUTORS.txt for the list of Swift project authors +// +//===----------------------------------------------------------------------===// + +import Dispatch + +/// Wrapper around `DispatchSemaphore` so that Swift Concurrency doesn't complain about the usage of semaphores in the +/// tests. +/// +/// This should only be used for tests that test priority escalation and thus cannot await a `Task` (which would cause +/// priority elevations). +public struct WrappedSemaphore { + let semaphore = DispatchSemaphore(value: 0) + + public init() {} + + public func signal(value: Int = 1) { + for _ in 0..) +} + +/// Schedules index tasks and keeps track of the index status of files. +public final actor SemanticIndexManager { + /// The underlying index. This is used to check if the index of a file is already up-to-date, in which case it doesn't + /// need to be indexed again. + private let index: CheckedIndex + + /// The build system manager that is used to get compiler arguments for a file. + private let buildSystemManager: BuildSystemManager + + /// The index status of the source files that the `SemanticIndexManager` knows about. + /// + /// Files that have never been indexed are not in this dictionary. + private var indexStatus: [DocumentURI: FileIndexStatus] = [:] + + /// The `TaskScheduler` that manages the scheduling of index tasks. This is shared among all `SemanticIndexManager`s + /// in the process, to ensure that we don't schedule more index operations than processor cores from multiple + /// workspaces. + private let indexTaskScheduler: TaskScheduler + + /// Callback that is called when an index task has finished. + /// + /// Currently only used for testing. + private let indexTaskDidFinish: (@Sendable (UpdateIndexStoreTaskDescription) -> Void)? + + // MARK: - Public API + + public init( + index: UncheckedIndex, + buildSystemManager: BuildSystemManager, + indexTaskScheduler: TaskScheduler, + indexTaskDidFinish: (@Sendable (UpdateIndexStoreTaskDescription) -> Void)? + ) { + self.index = index.checked(for: .modifiedFiles) + self.buildSystemManager = buildSystemManager + self.indexTaskScheduler = indexTaskScheduler + self.indexTaskDidFinish = indexTaskDidFinish + } + + /// Schedules a task to index all files in `files` that don't already have an up-to-date index. + /// Returns immediately after scheduling that task. + /// + /// Indexing is being performed with a low priority. + public func scheduleBackgroundIndex(files: some Collection) { + self.index(files: files, priority: .low) + } + + /// Wait for all in-progress index tasks to finish. + public func waitForUpToDateIndex() async { + logger.info("Waiting for up-to-date index") + await withTaskGroup(of: Void.self) { taskGroup in + for (_, status) in indexStatus { + switch status { + case .inProgress(let task): + taskGroup.addTask { + await task.value + } + case .upToDate: + break + } + } + await taskGroup.waitForAll() + } + index.pollForUnitChangesAndWait() + logger.debug("Done waiting for up-to-date index") + } + + /// Ensure that the index for the given files is up-to-date. + /// + /// This tries to produce an up-to-date index for the given files as quickly as possible. To achieve this, it might + /// suspend previous target-wide index tasks in favor of index tasks that index a fewer files. + public func waitForUpToDateIndex(for uris: some Collection) async { + logger.info( + "Waiting for up-to-date index for \(uris.map { $0.fileURL?.lastPathComponent ?? $0.stringValue }.joined(separator: ", "))" + ) + let filesWithOutOfDateIndex = uris.filter { uri in + switch indexStatus[uri] { + case .inProgress, nil: return true + case .upToDate: return false + } + } + // Create a new index task for the files that aren't up-to-date. The newly scheduled index tasks will + // - Wait for the existing index operations to finish if they have the same number of files. + // - Reschedule the background index task in favor of an index task with fewer source files. + await self.index(files: filesWithOutOfDateIndex, priority: nil).value + index.pollForUnitChangesAndWait() + logger.debug("Done waiting for up-to-date index") + } + + // MARK: - Helper functions + + /// Index the given set of files at the given priority. + /// + /// The returned task finishes when all files are indexed. + @discardableResult + private func index(files: some Collection, priority: TaskPriority?) -> Task { + let outOfDateFiles = files.filter { + if case .upToDate = indexStatus[$0] { + return false + } + return true + } + + var indexTasks: [Task] = [] + + // TODO (indexing): Group index operations by target when we support background preparation. + for files in outOfDateFiles.partition(intoNumberOfBatches: ProcessInfo.processInfo.processorCount * 5) { + let indexTask = Task(priority: priority) { + await self.indexTaskScheduler.schedule( + priority: priority, + UpdateIndexStoreTaskDescription( + filesToIndex: Set(files), + buildSystemManager: self.buildSystemManager, + index: self.index, + didFinishCallback: { [weak self] taskDescription in + self?.indexTaskDidFinish?(taskDescription) + } + ) + ).value + for file in files { + indexStatus[file] = .upToDate + } + } + indexTasks.append(indexTask) + + for file in files { + indexStatus[file] = .inProgress(indexTask) + } + } + let indexTasksImmutable = indexTasks + + return Task(priority: priority) { + await withTaskGroup(of: Void.self) { taskGroup in + for indexTask in indexTasksImmutable { + taskGroup.addTask(priority: priority) { + await indexTask.value + } + } + await taskGroup.waitForAll() + } + } + } +} diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift new file mode 100644 index 000000000..0aa542857 --- /dev/null +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -0,0 +1,302 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift.org open source project +// +// Copyright (c) 2014 - 2024 Apple Inc. and the Swift project authors +// Licensed under Apache License v2.0 with Runtime Library Exception +// +// See https://swift.org/LICENSE.txt for license information +// See https://swift.org/CONTRIBUTORS.txt for the list of Swift project authors +// +//===----------------------------------------------------------------------===// + +import CAtomics +import Foundation +import LSPLogging +import LanguageServerProtocol +import SKCore + +import struct TSCBasic.AbsolutePath +import class TSCBasic.Process + +private var updateIndexStoreIDForLogging = AtomicUInt32(initialValue: 1) + +/// Describes a task to index a set of source files. +/// +/// This task description can be scheduled in a `TaskScheduler`. +public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { + public let id = updateIndexStoreIDForLogging.fetchAndIncrement() + + /// The files that should be indexed. + private let filesToIndex: Set + + /// The build system manager that is used to get the toolchain and build settings for the files to index. + private let buildSystemManager: BuildSystemManager + + /// A reference to the underlying index store. Used to check if the index is already up-to-date for a file, in which + /// case we don't need to index it again. + private let index: CheckedIndex + + /// A callback that is called when the index task finishes + private let didFinishCallback: @Sendable (UpdateIndexStoreTaskDescription) -> Void + + /// The task is idempotent because indexing the same file twice produces the same result as indexing it once. + public var isIdempotent: Bool { true } + + public var estimatedCPUCoreCount: Int { 1 } + + public var description: String { + return self.redactedDescription + } + + public var redactedDescription: String { + return "indexing-\(id)" + } + + init( + filesToIndex: Set, + buildSystemManager: BuildSystemManager, + index: CheckedIndex, + didFinishCallback: @escaping @Sendable (UpdateIndexStoreTaskDescription) -> Void + ) { + self.filesToIndex = filesToIndex + self.buildSystemManager = buildSystemManager + self.index = index + self.didFinishCallback = didFinishCallback + } + + public func execute() async { + defer { + didFinishCallback(self) + } + // Only use the last two digits of the indexing ID for the logging scope to avoid creating too many scopes. + // See comment in `withLoggingScope`. + // The last 2 digits should be sufficient to differentiate between multiple concurrently running indexing operation. + await withLoggingSubsystemAndScope( + subsystem: "org.swift.sourcekit-lsp.indexing", + scope: "update-indexstore-\(id % 100)" + ) { + let startDate = Date() + + let filesToIndexDescription = filesToIndex.map { $0.fileURL?.lastPathComponent ?? $0.stringValue } + .joined(separator: ", ") + logger.log( + "Starting updating index store with priority \(Task.currentPriority.rawValue, privacy: .public): \(filesToIndexDescription)" + ) + let filesToIndex = filesToIndex.sorted(by: { $0.stringValue < $1.stringValue }) + // TODO (indexing): Once swiftc supports it, we should group files by target and index files within the same + // target together in one swiftc invocation. + for file in filesToIndex { + await updateIndexStoreForSingleFile(file) + } + logger.log( + "Finished updating index store in \(Date().timeIntervalSince(startDate) * 1000, privacy: .public)ms: \(filesToIndexDescription)" + ) + } + } + + public func dependencies( + to currentlyExecutingTasks: [UpdateIndexStoreTaskDescription] + ) -> [TaskDependencyAction] { + return currentlyExecutingTasks.compactMap { (other) -> TaskDependencyAction? in + guard !other.filesToIndex.intersection(filesToIndex).isEmpty else { + // Disjoint sets of files can be indexed concurrently. + return nil + } + if self.filesToIndex.count < other.filesToIndex.count { + // If there is an index operation with more files already running, suspend it. + // The most common use case for this is if we schedule an entire target to be indexed in the background and then + // need a single file indexed for use interaction. We should suspend the target-wide indexing and just index + // the current file to get index data for it ASAP. + return .cancelAndRescheduleDependency(other) + } else { + return .waitAndElevatePriorityOfDependency(other) + } + } + } + + private func updateIndexStoreForSingleFile(_ uri: DocumentURI) async { + guard let url = uri.fileURL else { + // The URI is not a file, so there's nothing we can index. + return + } + guard !index.hasUpToDateUnit(for: url) else { + // We consider a file's index up-to-date if we have any up-to-date unit. Changing build settings does not + // invalidate the up-to-date status of the index. + return + } + guard let language = await buildSystemManager.defaultLanguage(for: uri) else { + logger.error("Not indexing \(uri.forLogging) because its language could not be determined") + return + } + let buildSettings = await buildSystemManager.buildSettingsInferredFromMainFile( + for: uri, + language: language, + logBuildSettings: false + ) + guard let buildSettings else { + logger.error("Not indexing \(uri.forLogging) because it has no compiler arguments") + return + } + guard !buildSettings.isFallback else { + // Only index with real build settings. Indexing with fallback arguments could result in worse results than not + // indexing at all: If a file has been indexed with real build settings before, had a tiny modification made but + // we don't have any real build settings when it should get re-indexed. Then it's better to have the stale index + // from correct compiler arguments than no index at all. + logger.error("Not updating index store for \(uri.forLogging) because it has fallback compiler arguments") + return + } + guard let toolchain = await buildSystemManager.toolchain(for: uri, language) else { + logger.error( + "Not updating index store for \(uri.forLogging) because no toolchain could be determined for the document" + ) + return + } + switch language { + case .swift: + do { + try await updateIndexStore(forSwiftFile: uri, buildSettings: buildSettings, toolchain: toolchain) + } catch { + logger.error("Updating index store for \(uri) failed: \(error.forLogging)") + BuildSettingsLogger.log(settings: buildSettings, for: uri) + } + case .c, .cpp, .objective_c, .objective_cpp: + // TODO (indexing): Support indexing of clang files, including headers. + break + default: + logger.error( + "Not updating index store for \(uri) because it is a language that is not supported by background indexing" + ) + } + } + + private func updateIndexStore( + forSwiftFile uri: DocumentURI, + buildSettings: FileBuildSettings, + toolchain: Toolchain + ) async throws { + let indexingArguments = adjustSwiftCompilerArgumentsForIndexStoreUpdate( + buildSettings.compilerArguments, + fileToIndex: uri + ) + + guard let swiftc = toolchain.swiftc else { + logger.error( + "Not updating index store for \(uri.forLogging) because toolchain \(toolchain.identifier) does not contain a Swift compiler" + ) + return + } + + let process = + if let workingDirectory = buildSettings.workingDirectory { + Process( + arguments: [swiftc.pathString] + indexingArguments, + workingDirectory: try AbsolutePath(validating: workingDirectory) + ) + } else { + Process(arguments: [swiftc.pathString] + indexingArguments) + } + try process.launch() + let result = try await process.waitUntilExitSendingSigIntOnTaskCancellation() + switch result.exitStatus { + case .terminated(code: 0): + break + case .terminated(code: let code): + // This most likely happens if there are compilation errors in the source file. This is nothing to worry about. + let stdout = (try? String(bytes: result.output.get(), encoding: .utf8)) ?? "" + let stderr = (try? String(bytes: result.stderrOutput.get(), encoding: .utf8)) ?? "" + // Indexing will frequently fail if the source code is in an invalid state. Thus, log the failure at a low level. + logger.debug( + """ + Updating index store for Swift file \(uri.forLogging) terminated with non-zero exit code \(code) + Stderr: + \(stderr) + Stdout: + \(stdout) + """ + ) + BuildSettingsLogger.log(level: .debug, settings: buildSettings, for: uri) + case .signalled(signal: let signal): + if !Task.isCancelled { + // The indexing job finished with a signal. Could be because the compiler crashed. + // Ignore signal exit codes if this task has been cancelled because the compiler exits with SIGINT if it gets + // interrupted. + logger.error("Updating index store for Swift file \(uri.forLogging) signaled \(signal)") + BuildSettingsLogger.log(level: .error, settings: buildSettings, for: uri) + } + } + } +} + +/// Adjust compiler arguments that were created for building to compiler arguments that should be used for indexing. +/// +/// This removes compiler arguments that produce output files and adds arguments to index the file. +private func adjustSwiftCompilerArgumentsForIndexStoreUpdate( + _ compilerArguments: [String], + fileToIndex: DocumentURI +) -> [String] { + let removeFlags: Set = [ + "-c", + "-disable-cmo", + "-emit-dependencies", + "-emit-module-interface", + "-emit-module", + "-emit-module", + "-emit-objc-header", + "-incremental", + "-no-color-diagnostics", + "-parseable-output", + "-save-temps", + "-serialize-diagnostics", + "-use-frontend-parseable-output", + "-validate-clang-modules-once", + "-whole-module-optimization", + ] + + let removeArguments: Set = [ + "-clang-build-session-file", + "-emit-module-interface-path", + "-emit-module-path", + "-emit-objc-header-path", + "-emit-package-module-interface-path", + "-emit-private-module-interface-path", + "-num-threads", + "-o", + "-output-file-map", + ] + + let removeFrontendFlags: Set = [ + "-experimental-skip-non-inlinable-function-bodies", + "-experimental-skip-all-function-bodies", + ] + + var result: [String] = [] + result.reserveCapacity(compilerArguments.count) + var iterator = compilerArguments.makeIterator() + while let argument = iterator.next() { + if removeFlags.contains(argument) { + continue + } + if removeArguments.contains(argument) { + _ = iterator.next() + continue + } + if argument == "-Xfrontend" { + if let nextArgument = iterator.next() { + if removeFrontendFlags.contains(nextArgument) { + continue + } + result += [argument, nextArgument] + continue + } + result.append(argument) + } + result.append(argument) + } + result += [ + "-index-file", + "-index-file-path", fileToIndex.pseudoPath, + "-disable-batch-mode", // batch mode is not compatible with -index-file + ] + return result +} diff --git a/Sources/SourceKitLSP/SourceKitLSPServer.swift b/Sources/SourceKitLSP/SourceKitLSPServer.swift index fce17f037..cf3bc2811 100644 --- a/Sources/SourceKitLSP/SourceKitLSPServer.swift +++ b/Sources/SourceKitLSP/SourceKitLSPServer.swift @@ -449,6 +449,12 @@ public actor SourceKitLSPServer { let documentManager = DocumentManager() + /// The `TaskScheduler` that schedules all background indexing tasks. + /// + /// Shared process-wide to ensure the scheduled index operations across multiple workspaces don't exceed the maximum + /// number of processor cores that the user allocated to background indexing. + private let indexTaskScheduler: TaskScheduler + private var packageLoadingWorkDoneProgress = WorkDoneProgressState( "SourceKitLSP.SourceKitLSPServer.reloadPackage", title: "SourceKit-LSP: Reloading Package" @@ -519,6 +525,12 @@ public actor SourceKitLSPServer { self.onExit = onExit self.client = client + let processorCount = ProcessInfo.processInfo.processorCount + let lowPriorityCores = options.indexOptions.maxCoresPercentageToUseForBackgroundIndexing * Double(processorCount) + self.indexTaskScheduler = TaskScheduler(maxConcurrentTasksByPriority: [ + (TaskPriority.medium, processorCount), + (TaskPriority.low, max(Int(lowPriorityCores), 1)), + ]) } /// Search through all the parent directories of `uri` and check if any of these directories contain a workspace @@ -1152,6 +1164,7 @@ extension SourceKitLSPServer { options: options, compilationDatabaseSearchPaths: self.options.compilationDatabaseSearchPaths, indexOptions: self.options.indexOptions, + indexTaskScheduler: indexTaskScheduler, reloadPackageStatusCallback: { [weak self] status in guard let self else { return } guard capabilityRegistry.clientCapabilities.window?.workDoneProgress ?? false else { @@ -1220,7 +1233,8 @@ extension SourceKitLSPServer { options: self.options, underlyingBuildSystem: nil, index: nil, - indexDelegate: nil + indexDelegate: nil, + indexTaskScheduler: self.indexTaskScheduler ) self.workspacesAndIsImplicit.append((workspace: workspace, isImplicit: false)) @@ -2414,7 +2428,8 @@ extension SourceKitLSPServer { func pollIndex(_ req: PollIndexRequest) async throws -> VoidResponse { for workspace in workspaces { - workspace.uncheckedIndex?.underlyingIndexStoreDB.pollForUnitChangesAndWait() + await workspace.semanticIndexManager?.waitForUpToDateIndex() + workspace.index(checkedFor: .deletedFiles)?.pollForUnitChangesAndWait() } return VoidResponse() } diff --git a/Sources/SourceKitLSP/Workspace.swift b/Sources/SourceKitLSP/Workspace.swift index cb8d8ef52..5306b66f4 100644 --- a/Sources/SourceKitLSP/Workspace.swift +++ b/Sources/SourceKitLSP/Workspace.swift @@ -74,6 +74,12 @@ public final class Workspace { /// Language service for an open document, if available. var documentService: [DocumentURI: LanguageService] = [:] + /// The `SemanticIndexManager` that keeps track of whose file's index is up-to-date in the workspace and schedules + /// indexing and preparation tasks for files with out-of-date index. + /// + /// `nil` if background indexing is not enabled. + let semanticIndexManager: SemanticIndexManager? + public init( documentManager: DocumentManager, rootUri: DocumentURI?, @@ -82,7 +88,8 @@ public final class Workspace { options: SourceKitLSPServer.Options, underlyingBuildSystem: BuildSystem?, index uncheckedIndex: UncheckedIndex?, - indexDelegate: SourceKitIndexDelegate? + indexDelegate: SourceKitIndexDelegate?, + indexTaskScheduler: TaskScheduler ) async { self.documentManager = documentManager self.buildSetup = options.buildSetup @@ -95,6 +102,16 @@ public final class Workspace { mainFilesProvider: uncheckedIndex, toolchainRegistry: toolchainRegistry ) + if let uncheckedIndex, options.indexOptions.enableBackgroundIndexing { + self.semanticIndexManager = SemanticIndexManager( + index: uncheckedIndex, + buildSystemManager: buildSystemManager, + indexTaskScheduler: indexTaskScheduler, + indexTaskDidFinish: options.indexOptions.indexTaskDidFinish + ) + } else { + self.semanticIndexManager = nil + } await indexDelegate?.addMainFileChangedCallback { [weak self] in await self?.buildSystemManager.mainFilesChanged() } @@ -106,6 +123,9 @@ public final class Workspace { } // Trigger an initial population of `syntacticTestIndex`. await syntacticTestIndex.listOfTestFilesDidChange(buildSystemManager.testFiles()) + if let semanticIndexManager, let underlyingBuildSystem { + await semanticIndexManager.scheduleBackgroundIndex(files: await underlyingBuildSystem.sourceFiles().map(\.uri)) + } } /// Creates a workspace for a given root `URL`, inferring the `ExternalWorkspace` if possible. @@ -122,11 +142,16 @@ public final class Workspace { options: SourceKitLSPServer.Options, compilationDatabaseSearchPaths: [RelativePath], indexOptions: IndexOptions = IndexOptions(), + indexTaskScheduler: TaskScheduler, reloadPackageStatusCallback: @escaping (ReloadPackageStatus) async -> Void ) async throws { var buildSystem: BuildSystem? = nil if let rootUrl = rootUri.fileURL, let rootPath = try? AbsolutePath(validating: rootUrl.path) { + var options = options + if options.indexOptions.enableBackgroundIndexing, options.buildSetup.path == nil { + options.buildSetup.path = rootPath.appending(component: ".index-build") + } func createSwiftPMBuildSystem(rootUrl: URL) async -> SwiftPMBuildSystem? { return await SwiftPMBuildSystem( url: rootUrl, @@ -218,7 +243,8 @@ public final class Workspace { options: options, underlyingBuildSystem: buildSystem, index: UncheckedIndex(index), - indexDelegate: indexDelegate + indexDelegate: indexDelegate, + indexTaskScheduler: indexTaskScheduler ) } @@ -258,15 +284,34 @@ public struct IndexOptions { /// explicit calls to pollForUnitChangesAndWait(). public var listenToUnitEvents: Bool + /// Whether background indexing should be enabled. + public var enableBackgroundIndexing: Bool + + /// The percentage of the machine's cores that should at most be used for background indexing. + /// + /// Setting this to a value < 1 ensures that background indexing doesn't use all CPU resources. + public var maxCoresPercentageToUseForBackgroundIndexing: Double + + /// A callback that is called when an index task finishes. + /// + /// Intended for testing purposes. + public var indexTaskDidFinish: (@Sendable (UpdateIndexStoreTaskDescription) -> Void)? + public init( indexStorePath: AbsolutePath? = nil, indexDatabasePath: AbsolutePath? = nil, indexPrefixMappings: [PathPrefixMapping]? = nil, - listenToUnitEvents: Bool = true + listenToUnitEvents: Bool = true, + enableBackgroundIndexing: Bool = false, + maxCoresPercentageToUseForBackgroundIndexing: Double = 1, + indexTaskDidFinish: (@Sendable (UpdateIndexStoreTaskDescription) -> Void)? = nil ) { self.indexStorePath = indexStorePath self.indexDatabasePath = indexDatabasePath self.indexPrefixMappings = indexPrefixMappings self.listenToUnitEvents = listenToUnitEvents + self.enableBackgroundIndexing = enableBackgroundIndexing + self.maxCoresPercentageToUseForBackgroundIndexing = maxCoresPercentageToUseForBackgroundIndexing + self.indexTaskDidFinish = indexTaskDidFinish } } diff --git a/Tests/SKCoreTests/TaskSchedulerTests.swift b/Tests/SKCoreTests/TaskSchedulerTests.swift index db81c320b..9ca7a1bd0 100644 --- a/Tests/SKCoreTests/TaskSchedulerTests.swift +++ b/Tests/SKCoreTests/TaskSchedulerTests.swift @@ -11,6 +11,7 @@ //===----------------------------------------------------------------------===// import SKCore +import SKTestSupport import XCTest final class TaskSchedulerTests: XCTestCase { @@ -285,24 +286,6 @@ fileprivate actor TaskExecutionRecorder { } } -/// Wrapper around `DispatchSemaphore` so that Swift Concurrency doesn't complain about the usage of semaphores in the -/// tests. -fileprivate struct UnsafeSemaphore { - let semaphore = DispatchSemaphore(value: 0) - - func signal(value: Int = 1) { - for _ in 0.. Date: Thu, 2 May 2024 16:17:27 -0700 Subject: [PATCH 02/46] Conjure up an output file path so that indexing of two files produces different unit files --- Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index 0aa542857..7058abae6 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -296,7 +296,10 @@ private func adjustSwiftCompilerArgumentsForIndexStoreUpdate( result += [ "-index-file", "-index-file-path", fileToIndex.pseudoPath, - "-disable-batch-mode", // batch mode is not compatible with -index-file + // batch mode is not compatible with -index-file + "-disable-batch-mode", + // Fake an output path so that we get a different unit file for every Swift file we background index + "-o", fileToIndex.pseudoPath + ".o", ] return result } From 0ae3d80c5ccbe4478993a8c674b7788fd05f0b45 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Sun, 5 May 2024 14:53:42 -0700 Subject: [PATCH 03/46] Fix Windows build failure Windows has `ProcessResult.ExitStatus.abnormal` instead of ` `ProcessResult.ExitStatus.signalled` --- Sources/SKSupport/CMakeLists.txt | 1 + .../SwitchableProcessResultExitStatus.swift | 45 +++++++++++++++++++ .../UpdateIndexStoreTaskDescription.swift | 8 +++- 3 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 Sources/SKSupport/SwitchableProcessResultExitStatus.swift diff --git a/Sources/SKSupport/CMakeLists.txt b/Sources/SKSupport/CMakeLists.txt index 5453bb59d..7cb96d058 100644 --- a/Sources/SKSupport/CMakeLists.txt +++ b/Sources/SKSupport/CMakeLists.txt @@ -14,6 +14,7 @@ add_library(SKSupport STATIC Process+WaitUntilExitWithCancellation.swift Random.swift Result.swift + SwitchableProcessResultExitStatus.swift ThreadSafeBox.swift WorkspaceType.swift ) diff --git a/Sources/SKSupport/SwitchableProcessResultExitStatus.swift b/Sources/SKSupport/SwitchableProcessResultExitStatus.swift new file mode 100644 index 000000000..8e6f85733 --- /dev/null +++ b/Sources/SKSupport/SwitchableProcessResultExitStatus.swift @@ -0,0 +1,45 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift.org open source project +// +// Copyright (c) 2014 - 2024 Apple Inc. and the Swift project authors +// Licensed under Apache License v2.0 with Runtime Library Exception +// +// See https://swift.org/LICENSE.txt for license information +// See https://swift.org/CONTRIBUTORS.txt for the list of Swift project authors +// +//===----------------------------------------------------------------------===// + +// We need to import all of TSCBasic because otherwise we can't refer to Process.ExitStatus (rdar://127577691) +import struct TSCBasic.ProcessResult + +/// Same as `ProcessResult.ExitStatus` in tools-support-core but has the same cases on all platforms and is thus easier +/// to switch over +public enum SwitchableProcessResultExitStatus { + /// The process was terminated normally with a exit code. + case terminated(code: Int32) + /// The process was terminated abnormally. + case abnormal(exception: UInt32) + /// The process was terminated due to a signal. + case signalled(signal: Int32) +} + +extension ProcessResult.ExitStatus { + public var exhaustivelySwitchable: SwitchableProcessResultExitStatus { + #if os(Windows) + switch self { + case .terminated(let code): + return .terminated(code: code) + case .abnormal(let exception): + return .abnormal(exception: exception) + } + #else + switch self { + case .terminated(let code): + return .terminated(code: code) + case .signalled(let signal): + return .signalled(signal: signal) + } + #endif + } +} diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index 7058abae6..9ef7924a6 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -15,6 +15,7 @@ import Foundation import LSPLogging import LanguageServerProtocol import SKCore +import SKSupport import struct TSCBasic.AbsolutePath import class TSCBasic.Process @@ -198,7 +199,7 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { } try process.launch() let result = try await process.waitUntilExitSendingSigIntOnTaskCancellation() - switch result.exitStatus { + switch result.exitStatus.exhaustivelySwitchable { case .terminated(code: 0): break case .terminated(code: let code): @@ -224,6 +225,11 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { logger.error("Updating index store for Swift file \(uri.forLogging) signaled \(signal)") BuildSettingsLogger.log(level: .error, settings: buildSettings, for: uri) } + case .abnormal(exception: let exception): + if !Task.isCancelled { + logger.error("Updating index store for Swift file \(uri.forLogging) exited abnormally \(exception)") + BuildSettingsLogger.log(level: .error, settings: buildSettings, for: uri) + } } } } From f3f8860d7a6d31c6ba128485e50358d628d9bfd5 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Mon, 6 May 2024 18:30:22 -0700 Subject: [PATCH 04/46] Return `fileHandlingCapability.handled` for all files that `SwiftPMBuildSystem` has build settings for MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, `SwiftPMBuildSystem` would report `.unhandled` as the file handling capability for `Package.swift`. In single workspace-folder setups, this was not an issue because we would always default to using the build system of the first/only workspace folder, which was the only `SwiftPMBuildSystem` and then returned compiler arguments despite saying that it couldn’t handle `Package.swift`. This, however, breaks if you have a setup with multi workspace folders, where we can’t rely on the first workspace folder being able to provide build settings for the package manifest. To fix this, report `.handled` file handling capability for all files that `SwiftPMBuildSystem` can compute build settings for. Fixes #1210 rdar://127351614 --- .../SwiftPMBuildSystem.swift | 17 +++++----- Tests/SourceKitLSPTests/WorkspaceTests.swift | 33 +++++++++++++++++++ 2 files changed, 42 insertions(+), 8 deletions(-) diff --git a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift index 881ebc0f0..bd33e1596 100644 --- a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift +++ b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift @@ -315,6 +315,11 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { public var indexPrefixMappings: [PathPrefixMapping] { return [] } public func buildSettings(for uri: DocumentURI, language: Language) throws -> FileBuildSettings? { + // SwiftPMBuildSystem doesn't respect the langue specified by the editor. + return try buildSettings(for: uri) + } + + private func buildSettings(for uri: DocumentURI) throws -> FileBuildSettings? { guard let url = uri.fileURL else { // We can't determine build settings for non-file URIs. return nil @@ -335,7 +340,7 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { } if path.extension == "h" { - return try settings(forHeader: path, language) + return try settings(forHeader: path) } return nil @@ -432,14 +437,10 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { } public func fileHandlingCapability(for uri: DocumentURI) -> FileHandlingCapability { - guard let fileUrl = uri.fileURL else { - return .unhandled - } - if (try? buildTarget(for: AbsolutePath(validating: fileUrl.path))) != nil { + if (try? buildSettings(for: uri)) != nil { return .handled - } else { - return .unhandled } + return .unhandled } public func sourceFiles() -> [SourceFileInfo] { @@ -491,7 +492,7 @@ extension SwiftPMBuildSystem { /// file. /// This is safe because all files within one target have the same build settings except for reference to the file /// itself, which we are replacing. - private func settings(forHeader path: AbsolutePath, _ language: Language) throws -> FileBuildSettings? { + private func settings(forHeader path: AbsolutePath) throws -> FileBuildSettings? { func impl(_ path: AbsolutePath) throws -> FileBuildSettings? { var dir = path.parentDirectory while !dir.isRoot { diff --git a/Tests/SourceKitLSPTests/WorkspaceTests.swift b/Tests/SourceKitLSPTests/WorkspaceTests.swift index 27e547590..10af11fce 100644 --- a/Tests/SourceKitLSPTests/WorkspaceTests.swift +++ b/Tests/SourceKitLSPTests/WorkspaceTests.swift @@ -161,6 +161,39 @@ final class WorkspaceTests: XCTestCase { ) } + func testOpenPackageManifestInMultiSwiftPMWorkspaceSetup() async throws { + let project = try await MultiFileTestProject( + files: [ + // PackageA + "PackageA/Sources/MyLibrary/libA.swift": "", + "PackageA/Package.swift": SwiftPMTestProject.defaultPackageManifest, + + // PackageB + "PackageB/Sources/MyLibrary/libB.swift": "", + "PackageB/Package.swift": SwiftPMTestProject.defaultPackageManifest, + ], + workspaces: { scratchDir in + return [ + WorkspaceFolder(uri: DocumentURI(scratchDir)), + WorkspaceFolder(uri: DocumentURI(scratchDir.appendingPathComponent("PackageA"))), + WorkspaceFolder(uri: DocumentURI(scratchDir.appendingPathComponent("PackageB"))), + ] + } + ) + + let bPackageManifestUri = DocumentURI( + project.scratchDirectory.appendingPathComponent("PackageB").appendingPathComponent("Package.swift") + ) + + project.testClient.openDocument(SwiftPMTestProject.defaultPackageManifest, uri: bPackageManifestUri) + + // Ensure that we get proper build settings for Package.swift and no error about `No such module: PackageDescription` + let diags = try await project.testClient.send( + DocumentDiagnosticsRequest(textDocument: TextDocumentIdentifier(bPackageManifestUri)) + ) + XCTAssertEqual(diags, .full(RelatedFullDocumentDiagnosticReport(items: []))) + } + func testSwiftPMPackageInSubfolder() async throws { try await SkipUnless.swiftpmStoresModulesInSubdirectory() From 2270631a32eeb83b9a63bca8d643c9240be91151 Mon Sep 17 00:00:00 2001 From: Pavel Yaskevich Date: Tue, 7 May 2024 00:07:19 -0700 Subject: [PATCH 05/46] Adopt SwiftPM ModulesGraph initializer API adjustment `ModulesGraph.init` was changed by https://github.com/apple/swift-package-manager/pull/7530 to accept `packages` as a way to avoid having to recompute the full list of packages by walking roots. --- Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift index 881ebc0f0..b5692f3e0 100644 --- a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift +++ b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift @@ -26,6 +26,7 @@ import SourceKitLSPAPI import Workspace import struct Basics.AbsolutePath +import struct Basics.IdentifiableSet import struct Basics.TSCAbsolutePath import struct Foundation.URL import protocol TSCBasic.FileSystem @@ -176,7 +177,12 @@ public actor SwiftPMBuildSystem { flags: buildSetup.flags ) - self.modulesGraph = try ModulesGraph(rootPackages: [], dependencies: [], binaryArtifacts: [:]) + self.modulesGraph = try ModulesGraph( + rootPackages: [], + packages: IdentifiableSet(), + dependencies: [], + binaryArtifacts: [:] + ) self.reloadPackageStatusCallback = reloadPackageStatusCallback // The debounce duration of 500ms was chosen arbitrarily without scientific research. From 7b315680c9ff32be9953336aa6481675bf1419e1 Mon Sep 17 00:00:00 2001 From: Paul LeMarquand Date: Mon, 6 May 2024 12:13:07 -0400 Subject: [PATCH 06/46] Merge tests defined in extensions Merge the XCTests and swift-testing tests defined in extensions into their parent TestItems. This is done as another pass after the TestScanner visitors have walked the tree. Fixes #1218 --- .../Swift/SwiftTestingScanner.swift | 2 +- Sources/SourceKitLSP/TestDiscovery.swift | 41 +++++++- .../DocumentTestDiscoveryTests.swift | 99 ++++++++++++++++++- 3 files changed, 135 insertions(+), 7 deletions(-) diff --git a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift index ecda43039..0af4701f0 100644 --- a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift +++ b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift @@ -201,7 +201,7 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { let syntaxTree = await syntaxTreeManager.syntaxTree(for: snapshot) let visitor = SyntacticSwiftTestingTestScanner(snapshot: snapshot, allTestsDisabled: false, parentTypeNames: []) visitor.walk(syntaxTree) - return visitor.result + return visitor.result.mergeTestsInExtensions() } /// Visit a class/struct/... or extension declaration. diff --git a/Sources/SourceKitLSP/TestDiscovery.swift b/Sources/SourceKitLSP/TestDiscovery.swift index 0a185df1a..2207df4c0 100644 --- a/Sources/SourceKitLSP/TestDiscovery.swift +++ b/Sources/SourceKitLSP/TestDiscovery.swift @@ -357,7 +357,7 @@ final class SyntacticSwiftXCTestScanner: SyntaxVisitor { let syntaxTree = await syntaxTreeManager.syntaxTree(for: snapshot) let visitor = SyntacticSwiftXCTestScanner(snapshot: snapshot) visitor.walk(syntaxTree) - return visitor.result + return visitor.result.mergeTestsInExtensions() } private func findTestMethods(in members: MemberBlockItemListSyntax, containerName: String) -> [TestItem] { @@ -462,6 +462,45 @@ extension TestItem { } } +extension Collection where Element == TestItem { + /// Walks the TestItem tree of each item in the collection and merges orphaned leaf nodes + /// into their parent, if a parent exists. + /// + /// A node's parent is identified by the node's ID with the last component dropped. + func mergeTestsInExtensions() -> [TestItem] { + var itemDict: [String: TestItem] = [:] + for item in self { + if var existingItem = itemDict[item.id] { + existingItem.children = (existingItem.children + item.children) + itemDict[item.id] = existingItem + } else { + itemDict[item.id] = item + } + } + + for item in self { + let parentID = item.id.components(separatedBy: "/").dropLast().joined(separator: "/") + // If the parent exists, add the current item to its children and remove it from the root + if var parent = itemDict[parentID] { + parent.children.append(item) + itemDict[parent.id] = parent + itemDict[item.id] = nil + } + } + + // Filter out the items that have been merged into their parents, sorting the tests by location + var reorganizedItems = itemDict.values.compactMap { $0 }.sorted { $0.location < $1.location } + + reorganizedItems = reorganizedItems.map({ + var newItem = $0 + newItem.children = $0.children.mergeTestsInExtensions() + return newItem + }) + + return reorganizedItems + } +} + extension SwiftLanguageService { public func syntacticDocumentTests(for uri: DocumentURI, in workspace: Workspace) async throws -> [TestItem]? { let snapshot = try documentManager.latestSnapshot(uri) diff --git a/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift b/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift index c1133305b..659101b11 100644 --- a/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift +++ b/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift @@ -827,29 +827,118 @@ final class DocumentTestDiscoveryTests: XCTestCase { location: Location(uri: uri, range: positions["2️⃣"].. Date: Tue, 7 May 2024 09:15:06 -0400 Subject: [PATCH 07/46] Document mergeTestsInExtensions() --- Sources/SourceKitLSP/TestDiscovery.swift | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/Sources/SourceKitLSP/TestDiscovery.swift b/Sources/SourceKitLSP/TestDiscovery.swift index 2207df4c0..dd15e17e7 100644 --- a/Sources/SourceKitLSP/TestDiscovery.swift +++ b/Sources/SourceKitLSP/TestDiscovery.swift @@ -463,8 +463,19 @@ extension TestItem { } extension Collection where Element == TestItem { - /// Walks the TestItem tree of each item in the collection and merges orphaned leaf nodes - /// into their parent, if a parent exists. + /// When the test scanners discover tests in extensions they are captured in their own parent `TestItem`, not the + /// `TestItem` generated from the class/struct's definition. This is largely because of the syntatic nature of the + /// test scanners as they are today, which only know about tests within the context of the current file. Extensions + /// defined in separate files must be organized in their own `TestItem` since at the time of their creation there + /// isn't enough information to connect them back to the tests defined in the main type definition. + /// + /// This is a more syntatic than semantic view of the `TestItem` hierarchy than the end user likely wants. + /// If we think of the enclosing class or struct as the test suite, then extensions on that class or struct should be + /// additions to that suite, just like extensions on types are, from the user's perspective, transparently added to + /// their type. + /// + /// This method walks the `TestItem` tree produced by the test scanners and merges in the tests defined in extensions + /// into the `TestItem` that represents the type definition. /// /// A node's parent is identified by the node's ID with the last component dropped. func mergeTestsInExtensions() -> [TestItem] { From adfae1a77f0147fb685ea3546b3147a31c50c303 Mon Sep 17 00:00:00 2001 From: Paul LeMarquand Date: Tue, 7 May 2024 13:20:18 -0400 Subject: [PATCH 08/46] Prioritize tests defined in type definition over those in extensions Sort the list of test items prioritizing those defined in the originating type definition over those in extensions. --- .../SupportTypes/TestItem.swift | 5 + .../Swift/SwiftTestingScanner.swift | 23 +++- Sources/SourceKitLSP/TestDiscovery.swift | 88 ++++++++++-- .../DocumentTestDiscoveryTests.swift | 126 +++++++++++++++++- .../WorkspaceTestDiscoveryTests.swift | 99 ++++++++++++++ 5 files changed, 321 insertions(+), 20 deletions(-) diff --git a/Sources/LanguageServerProtocol/SupportTypes/TestItem.swift b/Sources/LanguageServerProtocol/SupportTypes/TestItem.swift index a764dabc6..477da701e 100644 --- a/Sources/LanguageServerProtocol/SupportTypes/TestItem.swift +++ b/Sources/LanguageServerProtocol/SupportTypes/TestItem.swift @@ -56,12 +56,16 @@ public struct TestItem: ResponseType, Equatable { /// Tags associated with this test item. public var tags: [TestTag] + /// Whether the `TestItem` is declared in an extension. + public var isExtension: Bool + public init( id: String, label: String, description: String? = nil, sortText: String? = nil, disabled: Bool, + isExtension: Bool, style: String, location: Location, children: [TestItem], @@ -72,6 +76,7 @@ public struct TestItem: ResponseType, Equatable { self.description = description self.sortText = sortText self.disabled = disabled + self.isExtension = isExtension self.style = style self.location = location self.children = children diff --git a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift index 0af4701f0..e19c29eed 100644 --- a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift +++ b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift @@ -163,6 +163,9 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { /// This is the case when the scanner is looking for tests inside a disabled suite. private let allTestsDisabled: Bool + /// Whether the tests discovered by the scanner should be marked as being delcared in an extension. + private let isScanningExtension: Bool + /// The names of the types that this scanner is scanning members for. /// /// For example, when scanning for tests inside `Bar` in the following, this is `["Foo", "Bar"]` @@ -179,10 +182,16 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { /// The discovered test items. private var result: [TestItem] = [] - private init(snapshot: DocumentSnapshot, allTestsDisabled: Bool, parentTypeNames: [String]) { + private init( + snapshot: DocumentSnapshot, + allTestsDisabled: Bool, + isScanningExtension: Bool, + parentTypeNames: [String] + ) { self.snapshot = snapshot self.allTestsDisabled = allTestsDisabled self.parentTypeNames = parentTypeNames + self.isScanningExtension = isScanningExtension super.init(viewMode: .fixedUp) } @@ -199,9 +208,14 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { return [] } let syntaxTree = await syntaxTreeManager.syntaxTree(for: snapshot) - let visitor = SyntacticSwiftTestingTestScanner(snapshot: snapshot, allTestsDisabled: false, parentTypeNames: []) + let visitor = SyntacticSwiftTestingTestScanner( + snapshot: snapshot, + allTestsDisabled: false, + isScanningExtension: false, + parentTypeNames: [] + ) visitor.walk(syntaxTree) - return visitor.result.mergeTestsInExtensions() + return visitor.result } /// Visit a class/struct/... or extension declaration. @@ -231,6 +245,7 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { let memberScanner = SyntacticSwiftTestingTestScanner( snapshot: snapshot, allTestsDisabled: attributeData?.isDisabled ?? false, + isScanningExtension: node is ExtensionDeclSyntax, parentTypeNames: parentTypeNames + typeNames ) memberScanner.walk(node.memberBlock) @@ -245,6 +260,7 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { id: (parentTypeNames + typeNames).joined(separator: "/"), label: attributeData?.displayName ?? typeNames.last!, disabled: (attributeData?.isDisabled ?? false) || allTestsDisabled, + isExtension: node is ExtensionDeclSyntax, style: TestStyle.swiftTesting, location: Location(uri: snapshot.uri, range: range), children: memberScanner.result, @@ -299,6 +315,7 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { id: (parentTypeNames + [name]).joined(separator: "/"), label: attributeData.displayName ?? name, disabled: attributeData.isDisabled || allTestsDisabled, + isExtension: isScanningExtension, style: TestStyle.swiftTesting, location: Location(uri: snapshot.uri, range: range), children: [], diff --git a/Sources/SourceKitLSP/TestDiscovery.swift b/Sources/SourceKitLSP/TestDiscovery.swift index dd15e17e7..1ca6d384d 100644 --- a/Sources/SourceKitLSP/TestDiscovery.swift +++ b/Sources/SourceKitLSP/TestDiscovery.swift @@ -154,6 +154,7 @@ extension SourceKitLSPServer { id: id, label: testSymbolOccurrence.symbol.name, disabled: false, + isExtension: false, style: TestStyle.xcTest, location: location, children: children, @@ -197,7 +198,7 @@ extension SourceKitLSPServer { return [] } return await orLog("Getting document tests for \(uri)") { - try await self.documentTests( + try await self.getDocumentTests( DocumentTestsRequest(textDocument: TextDocumentIdentifier(uri)), workspace: workspace, languageService: languageService @@ -258,6 +259,7 @@ extension SourceKitLSPServer { .concurrentMap { await self.tests(in: $0) } .flatMap { $0 } .sorted { $0.location < $1.location } + .mergingTestsInExtensions() } /// Extracts a flat dictionary mapping test IDs to their locations from the given `testItems`. @@ -274,6 +276,15 @@ extension SourceKitLSPServer { _ req: DocumentTestsRequest, workspace: Workspace, languageService: LanguageService + ) async throws -> [TestItem] { + return try await getDocumentTests(req, workspace: workspace, languageService: languageService) + .mergingTestsInExtensions() + } + + private func getDocumentTests( + _ req: DocumentTestsRequest, + workspace: Workspace, + languageService: LanguageService ) async throws -> [TestItem] { let snapshot = try self.documentManager.latestSnapshot(req.textDocument.uri) let mainFileUri = await workspace.buildSystemManager.mainFile( @@ -357,7 +368,7 @@ final class SyntacticSwiftXCTestScanner: SyntaxVisitor { let syntaxTree = await syntaxTreeManager.syntaxTree(for: snapshot) let visitor = SyntacticSwiftXCTestScanner(snapshot: snapshot) visitor.walk(syntaxTree) - return visitor.result.mergeTestsInExtensions() + return visitor.result } private func findTestMethods(in members: MemberBlockItemListSyntax, containerName: String) -> [TestItem] { @@ -387,6 +398,7 @@ final class SyntacticSwiftXCTestScanner: SyntaxVisitor { id: "\(containerName)/\(function.name.text)()", label: "\(function.name.text)()", disabled: false, + isExtension: false, style: TestStyle.xcTest, location: Location(uri: snapshot.uri, range: range), children: [], @@ -418,6 +430,7 @@ final class SyntacticSwiftXCTestScanner: SyntaxVisitor { id: node.name.text, label: node.name.text, disabled: false, + isExtension: false, style: TestStyle.xcTest, location: Location(uri: snapshot.uri, range: range), children: testMethods, @@ -462,7 +475,7 @@ extension TestItem { } } -extension Collection where Element == TestItem { +extension Array { /// When the test scanners discover tests in extensions they are captured in their own parent `TestItem`, not the /// `TestItem` generated from the class/struct's definition. This is largely because of the syntatic nature of the /// test scanners as they are today, which only know about tests within the context of the current file. Extensions @@ -477,18 +490,61 @@ extension Collection where Element == TestItem { /// This method walks the `TestItem` tree produced by the test scanners and merges in the tests defined in extensions /// into the `TestItem` that represents the type definition. /// + /// This causes extensions to be merged into their type's definition if the type's definition exists in the list of + /// test items. If the type's definition is not a test item in this collection, the first extension of that type will + /// be used as the primary test location. + /// + /// For example if there are two files + /// + /// FileA.swift + /// ```swift + /// @Suite struct MyTests { + /// @Test func oneIsTwo {} + /// } + /// ``` + /// + /// FileB.swift + /// ```swift + /// extension MyTests { + /// @Test func twoIsThree() {} + /// } + /// ``` + /// + /// Then `workspace/tests` will return + /// - `MyTests` (FileA.swift:1) + /// - `oneIsTwo` + /// - `twoIsThree` + /// + /// And `textDocument/tests` for FileB.swift will return + /// - `MyTests` (FileB.swift:1) + /// - `twoIsThree` + /// /// A node's parent is identified by the node's ID with the last component dropped. - func mergeTestsInExtensions() -> [TestItem] { + func mergingTestsInExtensions() -> [TestItem] { var itemDict: [String: TestItem] = [:] for item in self { - if var existingItem = itemDict[item.id] { - existingItem.children = (existingItem.children + item.children) - itemDict[item.id] = existingItem + if var rootItem = itemDict[item.id] { + // If we've encountered an extension first, and this is the + // type declaration, then use the type declaration TestItem + // as the root item. + if rootItem.isExtension && !item.isExtension { + var newItem = item + newItem.children += rootItem.children + rootItem = newItem + } else { + rootItem.children += item.children + } + + itemDict[item.id] = rootItem } else { itemDict[item.id] = item } } + if itemDict.isEmpty { + return [] + } + for item in self { let parentID = item.id.components(separatedBy: "/").dropLast().joined(separator: "/") // If the parent exists, add the current item to its children and remove it from the root @@ -499,16 +555,22 @@ extension Collection where Element == TestItem { } } - // Filter out the items that have been merged into their parents, sorting the tests by location - var reorganizedItems = itemDict.values.compactMap { $0 }.sorted { $0.location < $1.location } + // Filter out the items that have been merged into their parents, sorting the tests by location. + // TestItems not in extensions should be priotitized first. + var sortedItems = itemDict.values.compactMap { $0 }.sorted { + ($0.location.uri != $1.location.uri && $0.isExtension != $1.isExtension) ? !$0.isExtension : ($0.location < $1.location) + } - reorganizedItems = reorganizedItems.map({ + sortedItems = sortedItems.map { + guard !$0.children.isEmpty else { + return $0 + } var newItem = $0 - newItem.children = $0.children.mergeTestsInExtensions() + newItem.children = $0.children.mergingTestsInExtensions() return newItem - }) + } - return reorganizedItems + return sortedItems } } diff --git a/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift b/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift index 659101b11..019e9b414 100644 --- a/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift +++ b/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift @@ -60,6 +60,7 @@ final class DocumentTestDiscoveryTests: XCTestCase { id: "MyTests", label: "MyTests", disabled: false, + isExtension: false, style: TestStyle.xcTest, location: Location(uri: uri, range: positions["1️⃣"].. Date: Tue, 7 May 2024 13:31:55 -0400 Subject: [PATCH 09/46] Test case for multiple extensions with no type declaration --- .../DocumentTestDiscoveryTests.swift | 58 +++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift b/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift index 019e9b414..c7fcbe555 100644 --- a/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift +++ b/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift @@ -1149,6 +1149,64 @@ final class DocumentTestDiscoveryTests: XCTestCase { ) } + func testSwiftTestingTwoExtensionsNoDeclaration() async throws { + let testClient = try await TestSourceKitLSPClient() + let uri = DocumentURI.for(.swift) + + let positions = testClient.openDocument( + """ + import Testing + + 1️⃣extension MyTests { + 3️⃣@Test func oneIsTwo() {}4️⃣ + }2️⃣ + + extension MyTests { + 5️⃣@Test func twoIsThree() {}6️⃣ + } + """, + uri: uri + ) + + let tests = try await testClient.send(DocumentTestsRequest(textDocument: TextDocumentIdentifier(uri))) + XCTAssertEqual( + tests, + [ + TestItem( + id: "MyTests", + label: "MyTests", + disabled: false, + isExtension: true, + style: TestStyle.swiftTesting, + location: Location(uri: uri, range: positions["1️⃣"].. Date: Mon, 22 Apr 2024 22:08:20 -0700 Subject: [PATCH 10/46] Add separate "add test" manifest actions for XCTest and Swift Testing --- .../CodeActions/PackageManifestEdits.swift | 41 +++++++++++++------ Tests/SourceKitLSPTests/CodeActionTests.swift | 4 +- 2 files changed, 30 insertions(+), 15 deletions(-) diff --git a/Sources/SourceKitLSP/Swift/CodeActions/PackageManifestEdits.swift b/Sources/SourceKitLSP/Swift/CodeActions/PackageManifestEdits.swift index 27603f144..306d4698f 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/PackageManifestEdits.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/PackageManifestEdits.swift @@ -43,21 +43,36 @@ struct PackageManifestEdits: SyntaxCodeActionProvider { } do { - // Describe the target we are going to create. - let target = try TargetDescription( - name: "\(targetName)Tests", - dependencies: [.byName(name: targetName, condition: nil)], - type: .test - ) + var actions: [CodeAction] = [] - let edits = try AddTarget.addTarget(target, to: scope.file) - return [ - CodeAction( - title: "Add test target", - kind: .refactor, - edit: edits.asWorkspaceEdit(snapshot: scope.snapshot) - ) + let variants: [(AddTarget.TestHarness, String)] = [ + (.swiftTesting, "Swift Testing"), + (.xctest, "XCTest"), ] + for (testingLibrary, libraryName) in variants { + // Describe the target we are going to create. + let target = try TargetDescription( + name: "\(targetName)Tests", + dependencies: [.byName(name: targetName, condition: nil)], + type: .test + ) + + let edits = try AddTarget.addTarget( + target, + to: scope.file, + configuration: .init(testHarness: testingLibrary) + ) + + actions.append( + CodeAction( + title: "Add test target (\(libraryName))", + kind: .refactor, + edit: edits.asWorkspaceEdit(snapshot: scope.snapshot) + ) + ) + } + + return actions } catch { return [] } diff --git a/Tests/SourceKitLSPTests/CodeActionTests.swift b/Tests/SourceKitLSPTests/CodeActionTests.swift index 27752e902..7939c3d75 100644 --- a/Tests/SourceKitLSPTests/CodeActionTests.swift +++ b/Tests/SourceKitLSPTests/CodeActionTests.swift @@ -609,12 +609,12 @@ final class CodeActionTests: XCTestCase { // Make sure we get the expected package manifest editing actions. let addTestAction = codeActions.first { action in - return action.title == "Add test target" + return action.title == "Add test target (Swift Testing)" } XCTAssertNotNil(addTestAction) guard let addTestChanges = addTestAction?.edit?.documentChanges else { - XCTFail("Didn't have changes in the 'Add test target' action") + XCTFail("Didn't have changes in the 'Add test target (Swift Testing)' action") return } From b844dd3235e1d3d8ac917304e2cbd62ec302883f Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Wed, 8 May 2024 10:38:39 -0700 Subject: [PATCH 11/46] Fix PR comments to bug report issue template I forgot to push these changes before merging the original PR. --- .github/ISSUE_TEMPLATE/BUG_REPORT.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/BUG_REPORT.yml b/.github/ISSUE_TEMPLATE/BUG_REPORT.yml index f7d5ff735..c5321ed65 100644 --- a/.github/ISSUE_TEMPLATE/BUG_REPORT.yml +++ b/.github/ISSUE_TEMPLATE/BUG_REPORT.yml @@ -18,8 +18,8 @@ body: id: editor attributes: label: Editor - description: Which text editor are you using? - placeholder: Eg. Visual Studio Code with Swift plugin 1.9.0, neovim + description: Which text editor are you using (and LSP extension/plugin if applicable)? + placeholder: Eg. Visual Studio Code with Swift extension 1.9.0, Neovim - type: dropdown id: reproduces-with-swift-6 attributes: @@ -50,4 +50,4 @@ body: label: Logging description: | If you are using SourceKit-LSP from Swift 6, running `sourcekit-lsp diagnose` in terminal and attaching the generated bundle helps us diagnose the issue. - The generated bundle might contain portions of your source code, so please only attach it if you feel comfortable sharing it. + The generated bundle may contain paths to files on disk as well as portions of your source code. This greatly helps in reproducing issues, but you should only attach it if you feel comfortable doing so. From 2cc272e3f223b98c689fd59f4231b017f6478dd5 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Thu, 2 May 2024 20:57:56 -0700 Subject: [PATCH 12/46] Make waitUntilExitSendingSigIntOnTaskCancellation a public function instead of a public extension --- .../SKSupport/Process+WaitUntilExitWithCancellation.swift | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/Sources/SKSupport/Process+WaitUntilExitWithCancellation.swift b/Sources/SKSupport/Process+WaitUntilExitWithCancellation.swift index aa6f1c13f..b265161e6 100644 --- a/Sources/SKSupport/Process+WaitUntilExitWithCancellation.swift +++ b/Sources/SKSupport/Process+WaitUntilExitWithCancellation.swift @@ -15,9 +15,10 @@ import Foundation import class TSCBasic.Process import struct TSCBasic.ProcessResult -public extension Process { +extension Process { /// Wait for the process to exit. If the task gets cancelled, during this time, send a `SIGINT` to the process. - func waitUntilExitSendingSigIntOnTaskCancellation() async throws -> ProcessResult { + @discardableResult + public func waitUntilExitSendingSigIntOnTaskCancellation() async throws -> ProcessResult { return try await withTaskCancellationHandler { try await waitUntilExit() } onCancel: { From a643749def006599401bf364758dbbe837da1c54 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Thu, 2 May 2024 20:58:16 -0700 Subject: [PATCH 13/46] Log an error if we couldn't find the definition of a USR in the index --- Sources/SourceKitLSP/SourceKitLSPServer.swift | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/Sources/SourceKitLSP/SourceKitLSPServer.swift b/Sources/SourceKitLSP/SourceKitLSPServer.swift index 8ad25dac0..f925eb9bb 100644 --- a/Sources/SourceKitLSP/SourceKitLSPServer.swift +++ b/Sources/SourceKitLSP/SourceKitLSPServer.swift @@ -2471,7 +2471,11 @@ fileprivate extension CheckedIndex { /// If the USR has an ambiguous definition, the most important role of this function is to deterministically return /// the same result every time. func primaryDefinitionOrDeclarationOccurrence(ofUSR usr: String) -> SymbolOccurrence? { - return definitionOrDeclarationOccurrences(ofUSR: usr).sorted().first + let result = definitionOrDeclarationOccurrences(ofUSR: usr).sorted().first + if result == nil { + logger.error("Failed to find definition of \(usr) in index") + } + return result } } From e740bb33943e5067d9a2caa7f6ea2037ca889a5f Mon Sep 17 00:00:00 2001 From: Paul LeMarquand Date: Wed, 8 May 2024 14:29:16 -0400 Subject: [PATCH 14/46] Dont expose isExtension on TestItem --- .../SupportTypes/TestItem.swift | 5 - Sources/SourceKitLSP/LanguageService.swift | 2 +- .../Swift/SwiftTestingScanner.swift | 52 +++---- .../Swift/SyntacticTestIndex.swift | 8 +- Sources/SourceKitLSP/TestDiscovery.swift | 146 ++++++++++-------- .../DocumentTestDiscoveryTests.swift | 54 ------- .../WorkspaceTestDiscoveryTests.swift | 39 ----- 7 files changed, 113 insertions(+), 193 deletions(-) diff --git a/Sources/LanguageServerProtocol/SupportTypes/TestItem.swift b/Sources/LanguageServerProtocol/SupportTypes/TestItem.swift index 477da701e..a764dabc6 100644 --- a/Sources/LanguageServerProtocol/SupportTypes/TestItem.swift +++ b/Sources/LanguageServerProtocol/SupportTypes/TestItem.swift @@ -56,16 +56,12 @@ public struct TestItem: ResponseType, Equatable { /// Tags associated with this test item. public var tags: [TestTag] - /// Whether the `TestItem` is declared in an extension. - public var isExtension: Bool - public init( id: String, label: String, description: String? = nil, sortText: String? = nil, disabled: Bool, - isExtension: Bool, style: String, location: Location, children: [TestItem], @@ -76,7 +72,6 @@ public struct TestItem: ResponseType, Equatable { self.description = description self.sortText = sortText self.disabled = disabled - self.isExtension = isExtension self.style = style self.location = location self.children = children diff --git a/Sources/SourceKitLSP/LanguageService.swift b/Sources/SourceKitLSP/LanguageService.swift index 29ed79fa8..2a1ad8497 100644 --- a/Sources/SourceKitLSP/LanguageService.swift +++ b/Sources/SourceKitLSP/LanguageService.swift @@ -200,7 +200,7 @@ public protocol LanguageService: AnyObject { /// This is used as a fallback to show the test cases in a file if the index for a given file is not up-to-date. /// /// A return value of `nil` indicates that this language service does not support syntactic test discovery. - func syntacticDocumentTests(for uri: DocumentURI, in workspace: Workspace) async throws -> [TestItem]? + func syntacticDocumentTests(for uri: DocumentURI, in workspace: Workspace) async throws -> [AnnotatedTestItem]? /// Crash the language server. Should be used for crash recovery testing only. func _crash() async diff --git a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift index e19c29eed..dbcfc7e9a 100644 --- a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift +++ b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift @@ -163,9 +163,6 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { /// This is the case when the scanner is looking for tests inside a disabled suite. private let allTestsDisabled: Bool - /// Whether the tests discovered by the scanner should be marked as being delcared in an extension. - private let isScanningExtension: Bool - /// The names of the types that this scanner is scanning members for. /// /// For example, when scanning for tests inside `Bar` in the following, this is `["Foo", "Bar"]` @@ -180,18 +177,16 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { private let parentTypeNames: [String] /// The discovered test items. - private var result: [TestItem] = [] + private var result: [AnnotatedTestItem] = [] private init( snapshot: DocumentSnapshot, allTestsDisabled: Bool, - isScanningExtension: Bool, parentTypeNames: [String] ) { self.snapshot = snapshot self.allTestsDisabled = allTestsDisabled self.parentTypeNames = parentTypeNames - self.isScanningExtension = isScanningExtension super.init(viewMode: .fixedUp) } @@ -199,7 +194,7 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { public static func findTestSymbols( in snapshot: DocumentSnapshot, syntaxTreeManager: SyntaxTreeManager - ) async -> [TestItem] { + ) async -> [AnnotatedTestItem] { guard snapshot.text.contains("Suite") || snapshot.text.contains("Test") else { // If the file contains swift-testing tests, it must contain a `@Suite` or `@Test` attribute. // Only check for the attribute name because the attribute may be module qualified and contain an arbitrary amount @@ -211,7 +206,6 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { let visitor = SyntacticSwiftTestingTestScanner( snapshot: snapshot, allTestsDisabled: false, - isScanningExtension: false, parentTypeNames: [] ) visitor.walk(syntaxTree) @@ -245,7 +239,6 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { let memberScanner = SyntacticSwiftTestingTestScanner( snapshot: snapshot, allTestsDisabled: attributeData?.isDisabled ?? false, - isScanningExtension: node is ExtensionDeclSyntax, parentTypeNames: parentTypeNames + typeNames ) memberScanner.walk(node.memberBlock) @@ -256,15 +249,18 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { } let range = snapshot.range(of: node.positionAfterSkippingLeadingTrivia.. [TestItem] { +fileprivate func testItems(in url: URL) async -> [AnnotatedTestItem] { guard url.pathExtension == "swift" else { return [] } @@ -79,6 +79,8 @@ fileprivate func testItems(in url: URL) async -> [TestItem] { syntaxTreeManager: syntaxTreeManager ) async let xcTests = SyntacticSwiftXCTestScanner.findTestSymbols(in: snapshot, syntaxTreeManager: syntaxTreeManager) + .map { AnnotatedTestItem(testItem: $0, isExtension: false) } + return await swiftTestingTests + xcTests } @@ -206,7 +208,7 @@ actor SyntacticTestIndex { /// Gets all the tests in the syntactic index. /// /// This waits for any pending document updates to be indexed before returning a result. - nonisolated func tests() async -> [TestItem] { + nonisolated func tests() async -> [AnnotatedTestItem] { let readTask = indexingQueue.async(metadata: .read) { return await self.indexedTests.values.flatMap { $0.tests } } diff --git a/Sources/SourceKitLSP/TestDiscovery.swift b/Sources/SourceKitLSP/TestDiscovery.swift index 1ca6d384d..6949c1188 100644 --- a/Sources/SourceKitLSP/TestDiscovery.swift +++ b/Sources/SourceKitLSP/TestDiscovery.swift @@ -21,6 +21,22 @@ public enum TestStyle { public static let swiftTesting = "swift-testing" } +public struct AnnotatedTestItem { + /// The test item to be annotated + public var testItem: TestItem + + /// Whether the `TestItem` is declared in an extension. + public var isExtension: Bool + + public init( + testItem: TestItem, + isExtension: Bool + ) { + self.testItem = testItem + self.isExtension = isExtension + } +} + fileprivate extension SymbolOccurrence { /// Assuming that this is a symbol occurrence returned by the index, return whether it can constitute the definition /// of a test case. @@ -95,7 +111,7 @@ extension SourceKitLSPServer { private func testItems( for testSymbolOccurrences: [SymbolOccurrence], resolveLocation: (DocumentURI, Position) -> Location - ) -> [TestItem] { + ) -> [AnnotatedTestItem] { // Arrange tests by the USR they are contained in. This allows us to emit test methods as children of test classes. // `occurrencesByParent[nil]` are the root test symbols that aren't a child of another test symbol. var occurrencesByParent: [String?: [SymbolOccurrence]] = [:] @@ -126,7 +142,7 @@ extension SourceKitLSPServer { for testSymbolOccurrence: SymbolOccurrence, documentManager: DocumentManager, context: [String] - ) -> TestItem { + ) -> AnnotatedTestItem { let symbolPosition: Position if let snapshot = try? documentManager.latestSnapshot( testSymbolOccurrence.location.documentUri @@ -150,15 +166,17 @@ extension SourceKitLSPServer { .map { testItem(for: $0, documentManager: documentManager, context: context + [testSymbolOccurrence.symbol.name]) } - return TestItem( - id: id, - label: testSymbolOccurrence.symbol.name, - disabled: false, - isExtension: false, - style: TestStyle.xcTest, - location: location, - children: children, - tags: [] + return AnnotatedTestItem( + testItem: TestItem( + id: id, + label: testSymbolOccurrence.symbol.name, + disabled: false, + style: TestStyle.xcTest, + location: location, + children: children.map(\.testItem), + tags: [] + ), + isExtension: false ) } @@ -172,7 +190,7 @@ extension SourceKitLSPServer { /// This merges tests from the semantic index, the syntactic index and in-memory file states. /// /// The returned list of tests is not sorted. It should be sorted before being returned to the editor. - private func tests(in workspace: Workspace) async -> [TestItem] { + private func tests(in workspace: Workspace) async -> [AnnotatedTestItem] { // Gather all tests classes and test methods. We include test from different sources: // - For all files that have been not been modified since they were last indexed in the semantic index, include // XCTests from the semantic index. @@ -193,12 +211,12 @@ extension SourceKitLSPServer { return index?.fileHasInMemoryModifications(url) ?? documentManager.fileHasInMemoryModifications(url) } - let testsFromFilesWithInMemoryState = await filesWithInMemoryState.concurrentMap { (uri) -> [TestItem] in + let testsFromFilesWithInMemoryState = await filesWithInMemoryState.concurrentMap { (uri) -> [AnnotatedTestItem] in guard let languageService = workspace.documentService[uri] else { return [] } return await orLog("Getting document tests for \(uri)") { - try await self.getDocumentTests( + try await self.documentTestsWithoutMergingExtensions( DocumentTestsRequest(textDocument: TextDocumentIdentifier(uri)), workspace: workspace, languageService: languageService @@ -213,16 +231,17 @@ extension SourceKitLSPServer { for: semanticTestSymbolOccurrences, resolveLocation: { uri, position in Location(uri: uri, range: Range(position)) } ) - let filesWithTestsFromSemanticIndex = Set(testsFromSemanticIndex.map(\.location.uri)) + let filesWithTestsFromSemanticIndex = Set(testsFromSemanticIndex.map(\.testItem.location.uri)) let indexOnlyDiscardingDeletedFiles = workspace.index(checkedFor: .deletedFiles) let syntacticTestsToInclude = testsFromSyntacticIndex - .compactMap { (testItem) -> TestItem? in + .compactMap { (item) -> AnnotatedTestItem? in + let testItem = item.testItem if testItem.style == TestStyle.swiftTesting { // Swift-testing tests aren't part of the semantic index. Always include them. - return testItem + return item } if filesWithTestsFromSemanticIndex.contains(testItem.location.uri) { // If we have an semantic tests from this file, then the semantic index is up-to-date for this file. We thus @@ -245,9 +264,12 @@ extension SourceKitLSPServer { // XCTestCase subclasses, swift-testing handled above) for the same file. In practice test files usually contain // a single XCTestCase subclass, so caching doesn't make sense here. // Also, this is only called for files containing test cases but for which the semantic index is out-of-date. - return testItem.filterUsing( + if let filtered = testItem.filterUsing( semanticSymbols: indexOnlyDiscardingDeletedFiles?.symbols(inFilePath: testItem.location.uri.pseudoPath) - ) + ) { + return AnnotatedTestItem(testItem: filtered, isExtension: item.isExtension) + } + return nil } // We don't need to sort the tests here because they will get @@ -258,34 +280,24 @@ extension SourceKitLSPServer { return await self.workspaces .concurrentMap { await self.tests(in: $0) } .flatMap { $0 } - .sorted { $0.location < $1.location } + .sorted { $0.testItem.location < $1.testItem.location } .mergingTestsInExtensions() } - /// Extracts a flat dictionary mapping test IDs to their locations from the given `testItems`. - private func testLocations(from testItems: [TestItem]) -> [String: Location] { - var result: [String: Location] = [:] - for testItem in testItems { - result[testItem.id] = testItem.location - result.merge(testLocations(from: testItem.children)) { old, new in new } - } - return result - } - func documentTests( _ req: DocumentTestsRequest, workspace: Workspace, languageService: LanguageService ) async throws -> [TestItem] { - return try await getDocumentTests(req, workspace: workspace, languageService: languageService) + return try await documentTestsWithoutMergingExtensions(req, workspace: workspace, languageService: languageService) .mergingTestsInExtensions() } - private func getDocumentTests( + private func documentTestsWithoutMergingExtensions( _ req: DocumentTestsRequest, workspace: Workspace, languageService: LanguageService - ) async throws -> [TestItem] { + ) async throws -> [AnnotatedTestItem] { let snapshot = try self.documentManager.latestSnapshot(req.textDocument.uri) let mainFileUri = await workspace.buildSystemManager.mainFile( for: req.textDocument.uri, @@ -301,8 +313,8 @@ extension SourceKitLSPServer { syntacticTests == nil ? .deletedFiles : .inMemoryModifiedFiles(documentManager) if let index = workspace.index(checkedFor: indexCheckLevel) { - var syntacticSwiftTestingTests: [TestItem] { - syntacticTests?.filter { $0.style == TestStyle.swiftTesting } ?? [] + var syntacticSwiftTestingTests: [AnnotatedTestItem] { + syntacticTests?.filter { $0.testItem.style == TestStyle.swiftTesting } ?? [] } let testSymbols = @@ -398,7 +410,6 @@ final class SyntacticSwiftXCTestScanner: SyntaxVisitor { id: "\(containerName)/\(function.name.text)()", label: "\(function.name.text)()", disabled: false, - isExtension: false, style: TestStyle.xcTest, location: Location(uri: snapshot.uri, range: range), children: [], @@ -430,7 +441,6 @@ final class SyntacticSwiftXCTestScanner: SyntaxVisitor { id: node.name.text, label: node.name.text, disabled: false, - isExtension: false, style: TestStyle.xcTest, location: Location(uri: snapshot.uri, range: range), children: testMethods, @@ -475,7 +485,7 @@ extension TestItem { } } -extension Array { +extension Array { /// When the test scanners discover tests in extensions they are captured in their own parent `TestItem`, not the /// `TestItem` generated from the class/struct's definition. This is largely because of the syntatic nature of the /// test scanners as they are today, which only know about tests within the context of the current file. Extensions @@ -487,8 +497,8 @@ extension Array { /// additions to that suite, just like extensions on types are, from the user's perspective, transparently added to /// their type. /// - /// This method walks the `TestItem` tree produced by the test scanners and merges in the tests defined in extensions - /// into the `TestItem` that represents the type definition. + /// This method walks the `AnnotatedTestItem` tree produced by the test scanners and merges in the tests defined in + /// extensions into the final `TestItem`s that represent the type definition. /// /// This causes extensions to be merged into their type's definition if the type's definition exists in the list of /// test items. If the type's definition is not a test item in this collection, the first extension of that type will @@ -521,23 +531,24 @@ extension Array { /// /// A node's parent is identified by the node's ID with the last component dropped. func mergingTestsInExtensions() -> [TestItem] { - var itemDict: [String: TestItem] = [:] + var itemDict: [String: AnnotatedTestItem] = [:] for item in self { - if var rootItem = itemDict[item.id] { + let id = item.testItem.id + if var rootItem = itemDict[id] { // If we've encountered an extension first, and this is the // type declaration, then use the type declaration TestItem // as the root item. if rootItem.isExtension && !item.isExtension { var newItem = item - newItem.children += rootItem.children + newItem.testItem.children += rootItem.testItem.children rootItem = newItem } else { - rootItem.children += item.children + rootItem.testItem.children += item.testItem.children } - itemDict[item.id] = rootItem + itemDict[id] = rootItem } else { - itemDict[item.id] = item + itemDict[id] = item } } @@ -545,53 +556,60 @@ extension Array { return [] } + var mergedIds = Set() for item in self { - let parentID = item.id.components(separatedBy: "/").dropLast().joined(separator: "/") + let id = item.testItem.id + let parentID = id.components(separatedBy: "/").dropLast().joined(separator: "/") // If the parent exists, add the current item to its children and remove it from the root if var parent = itemDict[parentID] { - parent.children.append(item) - itemDict[parent.id] = parent - itemDict[item.id] = nil + parent.testItem.children.append(item.testItem) + mergedIds.insert(parent.testItem.id) + itemDict[parent.testItem.id] = parent + itemDict[id] = nil } } // Filter out the items that have been merged into their parents, sorting the tests by location. // TestItems not in extensions should be priotitized first. - var sortedItems = itemDict.values.compactMap { $0 }.sorted { - ($0.location.uri != $1.location.uri && $0.isExtension != $1.isExtension) ? !$0.isExtension : ($0.location < $1.location) - } + let sortedItems = itemDict.values + .compactMap { $0 } + .sorted { ($0.isExtension != $1.isExtension) ? !$0.isExtension : ($0.testItem.location < $1.testItem.location) } - sortedItems = sortedItems.map { - guard !$0.children.isEmpty else { - return $0 + return sortedItems.map { + guard !$0.testItem.children.isEmpty, mergedIds.contains($0.testItem.id) else { + return $0.testItem } - var newItem = $0 - newItem.children = $0.children.mergingTestsInExtensions() + var newItem = $0.testItem + newItem.children = newItem.children + .map { AnnotatedTestItem(testItem: $0, isExtension: false) } + .mergingTestsInExtensions() return newItem } - - return sortedItems } } extension SwiftLanguageService { - public func syntacticDocumentTests(for uri: DocumentURI, in workspace: Workspace) async throws -> [TestItem]? { + public func syntacticDocumentTests(for uri: DocumentURI, in workspace: Workspace) async throws -> [AnnotatedTestItem]? + { let snapshot = try documentManager.latestSnapshot(uri) let semanticSymbols = workspace.index(checkedFor: .deletedFiles)?.symbols(inFilePath: snapshot.uri.pseudoPath) let xctestSymbols = await SyntacticSwiftXCTestScanner.findTestSymbols( in: snapshot, syntaxTreeManager: syntaxTreeManager - ).compactMap { $0.filterUsing(semanticSymbols: semanticSymbols) } + ) + .compactMap { $0.filterUsing(semanticSymbols: semanticSymbols) } + .map { AnnotatedTestItem(testItem: $0, isExtension: false) } + let swiftTestingSymbols = await SyntacticSwiftTestingTestScanner.findTestSymbols( in: snapshot, syntaxTreeManager: syntaxTreeManager ) - return (xctestSymbols + swiftTestingSymbols).sorted { $0.location < $1.location } + return (xctestSymbols + swiftTestingSymbols).sorted { $0.testItem.location < $1.testItem.location } } } extension ClangLanguageService { - public func syntacticDocumentTests(for uri: DocumentURI, in workspace: Workspace) async -> [TestItem]? { + public func syntacticDocumentTests(for uri: DocumentURI, in workspace: Workspace) async -> [AnnotatedTestItem]? { return nil } } diff --git a/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift b/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift index c7fcbe555..cd4a835fb 100644 --- a/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift +++ b/Tests/SourceKitLSPTests/DocumentTestDiscoveryTests.swift @@ -60,7 +60,6 @@ final class DocumentTestDiscoveryTests: XCTestCase { id: "MyTests", label: "MyTests", disabled: false, - isExtension: false, style: TestStyle.xcTest, location: Location(uri: uri, range: positions["1️⃣"].. Date: Tue, 7 May 2024 16:21:07 -0700 Subject: [PATCH 15/46] Address my own review comments to #1179 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Addresses a few minor comments and the following major ones: - Add test cases for the syntax refactorings - Don’t report code actions for refactorings that don’t actually modify the source - Instead of just looking at the parent of the token of the selected range, walk up the syntax tree to find the syntax node to refactor. This makes the refactorings available in a lot more locations. --- Sources/SourceKitLSP/CMakeLists.txt | 1 + Sources/SourceKitLSP/Rename.swift | 10 - .../Swift/CodeActions/AddDocumentation.swift | 7 + .../CodeActions/ConvertIntegerLiteral.swift | 5 +- .../ConvertJSONToCodableStruct.swift | 11 + .../CodeActions/PackageManifestEdits.swift | 4 +- .../SyntaxCodeActionProvider.swift | 52 ++- .../SyntaxRefactoringCodeActionProvider.swift | 79 ++++- .../Swift/SwiftLanguageService.swift | 15 +- Sources/SourceKitLSP/TextEdit+IsNoop.swift | 23 ++ Tests/SourceKitLSPTests/CodeActionTests.swift | 328 ++++++++++++++++-- .../SyntaxRefactorTests.swift | 20 ++ 12 files changed, 475 insertions(+), 80 deletions(-) create mode 100644 Sources/SourceKitLSP/TextEdit+IsNoop.swift diff --git a/Sources/SourceKitLSP/CMakeLists.txt b/Sources/SourceKitLSP/CMakeLists.txt index e56f172fa..a86c621f9 100644 --- a/Sources/SourceKitLSP/CMakeLists.txt +++ b/Sources/SourceKitLSP/CMakeLists.txt @@ -14,6 +14,7 @@ add_library(SourceKitLSP STATIC SourceKitLSPServer+Options.swift SymbolLocation+DocumentURI.swift TestDiscovery.swift + TextEdit+IsNoop.swift WorkDoneProgressManager.swift Workspace.swift ) diff --git a/Sources/SourceKitLSP/Rename.swift b/Sources/SourceKitLSP/Rename.swift index a8bb0b15d..3ac6984b6 100644 --- a/Sources/SourceKitLSP/Rename.swift +++ b/Sources/SourceKitLSP/Rename.swift @@ -1490,13 +1490,3 @@ fileprivate extension RelatedIdentifiersResponse { } } } - -fileprivate extension TextEdit { - /// Returns `true` the replaced text is the same as the new text - func isNoOp(in snapshot: DocumentSnapshot) -> Bool { - if snapshot.text[snapshot.indexRange(of: range)] == newText { - return true - } - return false - } -} diff --git a/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift b/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift index f37a6b661..ee1e9babd 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift @@ -94,6 +94,13 @@ public struct AddDocumentation: EditRefactoringProvider { } extension AddDocumentation: SyntaxRefactoringCodeActionProvider { + static func nodeToRefactor(in scope: SyntaxCodeActionScope) -> Input? { + return scope.innermostNodeContainingRange?.findParentOfSelf( + ofType: DeclSyntax.self, + stoppingIf: { $0.is(CodeBlockItemSyntax.self) || $0.is(MemberBlockItemSyntax.self) || $0.is(ExprSyntax.self) } + ) + } + static var title: String { "Add documentation" } } diff --git a/Sources/SourceKitLSP/Swift/CodeActions/ConvertIntegerLiteral.swift b/Sources/SourceKitLSP/Swift/CodeActions/ConvertIntegerLiteral.swift index b541f1c91..f00d0a80c 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/ConvertIntegerLiteral.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/ConvertIntegerLiteral.swift @@ -14,9 +14,6 @@ import LanguageServerProtocol import SwiftRefactor import SwiftSyntax -// TODO: Make the type IntegerLiteralExprSyntax.Radix conform to CaseEnumerable -// in swift-syntax. - extension IntegerLiteralExprSyntax.Radix { static var allCases: [Self] = [.binary, .octal, .decimal, .hex] } @@ -26,7 +23,7 @@ extension IntegerLiteralExprSyntax.Radix { struct ConvertIntegerLiteral: SyntaxCodeActionProvider { static func codeActions(in scope: SyntaxCodeActionScope) -> [CodeAction] { guard - let token = scope.firstToken, + let token = scope.innermostNodeContainingRange, let integerExpr = token.parent?.as(IntegerLiteralExprSyntax.self), let integerValue = Int( integerExpr.split().value.filter { $0 != "_" }, diff --git a/Sources/SourceKitLSP/Swift/CodeActions/ConvertJSONToCodableStruct.swift b/Sources/SourceKitLSP/Swift/CodeActions/ConvertJSONToCodableStruct.swift index 73f583627..dc1fab7a0 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/ConvertJSONToCodableStruct.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/ConvertJSONToCodableStruct.swift @@ -192,6 +192,17 @@ public struct ConvertJSONToCodableStruct: EditRefactoringProvider { } extension ConvertJSONToCodableStruct: SyntaxRefactoringCodeActionProvider { + static func nodeToRefactor(in scope: SyntaxCodeActionScope) -> Syntax? { + var node: Syntax? = scope.innermostNodeContainingRange + while let unwrappedNode = node, ![.codeBlockItem, .memberBlockItem].contains(unwrappedNode.kind) { + if preflightRefactoring(unwrappedNode) != nil { + return unwrappedNode + } + node = unwrappedNode.parent + } + return nil + } + static var title = "Create Codable structs from JSON" } diff --git a/Sources/SourceKitLSP/Swift/CodeActions/PackageManifestEdits.swift b/Sources/SourceKitLSP/Swift/CodeActions/PackageManifestEdits.swift index 306d4698f..439513e2b 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/PackageManifestEdits.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/PackageManifestEdits.swift @@ -20,9 +20,7 @@ import SwiftSyntax /// edit a package manifest. struct PackageManifestEdits: SyntaxCodeActionProvider { static func codeActions(in scope: SyntaxCodeActionScope) -> [CodeAction] { - guard let token = scope.firstToken, - let call = token.findEnclosingCall() - else { + guard let call = scope.innermostNodeContainingRange?.findEnclosingCall() else { return [] } diff --git a/Sources/SourceKitLSP/Swift/CodeActions/SyntaxCodeActionProvider.swift b/Sources/SourceKitLSP/Swift/CodeActions/SyntaxCodeActionProvider.swift index 45bebc378..0674c5859 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/SyntaxCodeActionProvider.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/SyntaxCodeActionProvider.swift @@ -40,26 +40,50 @@ struct SyntaxCodeActionScope { /// considered, i.e., where the cursor or selection is. var range: Range - init( + /// The innermost node that contains the entire selected source range + var innermostNodeContainingRange: Syntax? + + init?( snapshot: DocumentSnapshot, - syntaxTree tree: SourceFileSyntax, + syntaxTree file: SourceFileSyntax, request: CodeActionRequest - ) throws { + ) { self.snapshot = snapshot self.request = request - self.file = tree + self.file = file - let start = snapshot.absolutePosition(of: request.range.lowerBound) - let end = snapshot.absolutePosition(of: request.range.upperBound) - let left = file.token(at: start) - let right = file.token(at: end) - let leftOff = left?.position ?? AbsolutePosition(utf8Offset: 0) - let rightOff = right?.endPosition ?? leftOff - self.range = leftOff.. TokenSyntax? { + let absolutePosition = snapshot.absolutePosition(of: position) + if absolutePosition == syntaxTree.endPosition { + // token(at:) will not find the end of file token if the end of file token has length 0. Special case this and + // return the last proper token in this case. + return syntaxTree.endOfFileToken.previousToken(viewMode: .sourceAccurate) + } + guard let token = syntaxTree.token(at: absolutePosition) else { + return nil + } + // See `adjustPositionToStartOfIdentifier`. We need to be a little more aggressive for the refactorings and also + // adjust to the start of punctuation eg. if the end of the selected range is after a `}`, we want the end token for + // the refactoring to be the `}`, not the token after `}`. + if absolutePosition == token.position, + let previousToken = token.previousToken(viewMode: .sourceAccurate), + previousToken.endPositionBeforeTrailingTrivia == absolutePosition + { + return previousToken } + return token } diff --git a/Sources/SourceKitLSP/Swift/CodeActions/SyntaxRefactoringCodeActionProvider.swift b/Sources/SourceKitLSP/Swift/CodeActions/SyntaxRefactoringCodeActionProvider.swift index 1fb72e0ed..4a42f52f4 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/SyntaxRefactoringCodeActionProvider.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/SyntaxRefactoringCodeActionProvider.swift @@ -18,29 +18,35 @@ import SwiftSyntax /// swift-syntax) into a SyntaxCodeActionProvider. protocol SyntaxRefactoringCodeActionProvider: SyntaxCodeActionProvider, EditRefactoringProvider { static var title: String { get } + + /// Returns the node that the syntax refactoring should be performed on, if code actions are requested for the given + /// scope. + static func nodeToRefactor(in scope: SyntaxCodeActionScope) -> Input? } /// SyntaxCodeActionProviders with a \c Void context can automatically be /// adapted provide a code action based on their refactoring operation. extension SyntaxRefactoringCodeActionProvider where Self.Context == Void { static func codeActions(in scope: SyntaxCodeActionScope) -> [CodeAction] { - guard - let token = scope.firstToken, - let node = token.parent?.as(Input.self) - else { + guard let node = nodeToRefactor(in: scope) else { return [] } let sourceEdits = Self.textRefactor(syntax: node) - if sourceEdits.isEmpty { - return [] - } - let textEdits = sourceEdits.map { edit in - TextEdit( + let textEdits = sourceEdits.compactMap { (edit) -> TextEdit? in + let edit = TextEdit( range: scope.snapshot.range(of: edit.range), newText: edit.replacement ) + if edit.isNoOp(in: scope.snapshot) { + return nil + } + return edit + } + + if textEdits.isEmpty { + return [] } return [ @@ -57,22 +63,77 @@ extension SyntaxRefactoringCodeActionProvider where Self.Context == Void { extension AddSeparatorsToIntegerLiteral: SyntaxRefactoringCodeActionProvider { public static var title: String { "Add digit separators" } + + static func nodeToRefactor(in scope: SyntaxCodeActionScope) -> Input? { + return scope.innermostNodeContainingRange?.findParentOfSelf( + ofType: IntegerLiteralExprSyntax.self, + stoppingIf: { $0.is(CodeBlockSyntax.self) || $0.is(MemberBlockSyntax.self) } + ) + } } extension FormatRawStringLiteral: SyntaxRefactoringCodeActionProvider { public static var title: String { "Convert string literal to minimal number of '#'s" } + + static func nodeToRefactor(in scope: SyntaxCodeActionScope) -> Input? { + return scope.innermostNodeContainingRange?.findParentOfSelf( + ofType: StringLiteralExprSyntax.self, + stoppingIf: { + $0.is(CodeBlockSyntax.self) || $0.is(MemberBlockSyntax.self) + || $0.keyPathInParent == \ExpressionSegmentSyntax.expressions + } + ) + } } extension MigrateToNewIfLetSyntax: SyntaxRefactoringCodeActionProvider { public static var title: String { "Migrate to shorthand 'if let' syntax" } + + static func nodeToRefactor(in scope: SyntaxCodeActionScope) -> Input? { + return scope.innermostNodeContainingRange?.findParentOfSelf( + ofType: IfExprSyntax.self, + stoppingIf: { $0.is(CodeBlockSyntax.self) || $0.is(MemberBlockSyntax.self) } + ) + } } extension OpaqueParameterToGeneric: SyntaxRefactoringCodeActionProvider { public static var title: String { "Expand 'some' parameters to generic parameters" } + + static func nodeToRefactor(in scope: SyntaxCodeActionScope) -> Input? { + return scope.innermostNodeContainingRange?.findParentOfSelf( + ofType: DeclSyntax.self, + stoppingIf: { $0.is(CodeBlockSyntax.self) || $0.is(MemberBlockSyntax.self) } + ) + } } extension RemoveSeparatorsFromIntegerLiteral: SyntaxRefactoringCodeActionProvider { public static var title: String { "Remove digit separators" } + + static func nodeToRefactor(in scope: SyntaxCodeActionScope) -> Input? { + return scope.innermostNodeContainingRange?.findParentOfSelf( + ofType: IntegerLiteralExprSyntax.self, + stoppingIf: { $0.is(CodeBlockSyntax.self) || $0.is(MemberBlockSyntax.self) } + ) + } +} + +extension Syntax { + /// Finds the innermost parent of the given type while not walking outside of nodes that satisfy `stoppingIf`. + func findParentOfSelf( + ofType: ParentType.Type, + stoppingIf: (Syntax) -> Bool + ) -> ParentType? { + var node: Syntax? = self + while let unwrappedNode = node, !stoppingIf(unwrappedNode) { + if let expectedType = unwrappedNode.as(ParentType.self) { + return expectedType + } + node = unwrappedNode.parent + } + return nil + } } diff --git a/Sources/SourceKitLSP/Swift/SwiftLanguageService.swift b/Sources/SourceKitLSP/Swift/SwiftLanguageService.swift index 6f21b250f..49095bea3 100644 --- a/Sources/SourceKitLSP/Swift/SwiftLanguageService.swift +++ b/Sources/SourceKitLSP/Swift/SwiftLanguageService.swift @@ -755,9 +755,10 @@ extension SwiftLanguageService { return response } - func retrieveCodeActions(_ req: CodeActionRequest, providers: [CodeActionProvider]) async throws - -> [CodeAction] - { + func retrieveCodeActions( + _ req: CodeActionRequest, + providers: [CodeActionProvider] + ) async throws -> [CodeAction] { guard providers.isEmpty == false else { return [] } @@ -776,7 +777,9 @@ extension SwiftLanguageService { let snapshot = try documentManager.latestSnapshot(uri) let syntaxTree = await syntaxTreeManager.syntaxTree(for: snapshot) - let scope = try SyntaxCodeActionScope(snapshot: snapshot, syntaxTree: syntaxTree, request: request) + guard let scope = SyntaxCodeActionScope(snapshot: snapshot, syntaxTree: syntaxTree, request: request) else { + return [] + } return await allSyntaxCodeActions.concurrentMap { provider in return provider.codeActions(in: scope) }.flatMap { $0 } @@ -1152,8 +1155,8 @@ extension DocumentSnapshot { callerFile: StaticString = #fileID, callerLine: UInt = #line ) -> Range { - let lowerBound = self.position(of: node.position) - let upperBound = self.position(of: node.endPosition) + let lowerBound = self.position(of: node.position, callerFile: callerFile, callerLine: callerLine) + let upperBound = self.position(of: node.endPosition, callerFile: callerFile, callerLine: callerLine) return lowerBound.. Bool { + if snapshot.text[snapshot.indexRange(of: range)] == newText { + return true + } + return false + } +} diff --git a/Tests/SourceKitLSPTests/CodeActionTests.swift b/Tests/SourceKitLSPTests/CodeActionTests.swift index 7939c3d75..481ebe175 100644 --- a/Tests/SourceKitLSPTests/CodeActionTests.swift +++ b/Tests/SourceKitLSPTests/CodeActionTests.swift @@ -16,23 +16,22 @@ import SKTestSupport import SourceKitLSP import XCTest -final class CodeActionTests: XCTestCase { - - typealias CodeActionCapabilities = TextDocumentClientCapabilities.CodeAction - typealias CodeActionLiteralSupport = CodeActionCapabilities.CodeActionLiteralSupport - typealias CodeActionKindCapabilities = CodeActionLiteralSupport.CodeActionKind - - private func clientCapabilitiesWithCodeActionSupport() -> ClientCapabilities { - var documentCapabilities = TextDocumentClientCapabilities() - var codeActionCapabilities = CodeActionCapabilities() - let codeActionKinds = CodeActionKindCapabilities(valueSet: [.refactor, .quickFix]) - let codeActionLiteralSupport = CodeActionLiteralSupport(codeActionKind: codeActionKinds) - codeActionCapabilities.codeActionLiteralSupport = codeActionLiteralSupport - documentCapabilities.codeAction = codeActionCapabilities - documentCapabilities.completion = .init(completionItem: .init(snippetSupport: true)) - return ClientCapabilities(workspace: nil, textDocument: documentCapabilities) - } +private typealias CodeActionCapabilities = TextDocumentClientCapabilities.CodeAction +private typealias CodeActionLiteralSupport = CodeActionCapabilities.CodeActionLiteralSupport +private typealias CodeActionKindCapabilities = CodeActionLiteralSupport.CodeActionKind + +private var clientCapabilitiesWithCodeActionSupport: ClientCapabilities = { + var documentCapabilities = TextDocumentClientCapabilities() + var codeActionCapabilities = CodeActionCapabilities() + let codeActionKinds = CodeActionKindCapabilities(valueSet: [.refactor, .quickFix]) + let codeActionLiteralSupport = CodeActionLiteralSupport(codeActionKind: codeActionKinds) + codeActionCapabilities.codeActionLiteralSupport = codeActionLiteralSupport + documentCapabilities.codeAction = codeActionCapabilities + documentCapabilities.completion = .init(completionItem: .init(snippetSupport: true)) + return ClientCapabilities(workspace: nil, textDocument: documentCapabilities) +}() +final class CodeActionTests: XCTestCase { func testCodeActionResponseLegacySupport() throws { let command = Command(title: "Title", command: "Command", arguments: [1, "text", 2.2, nil]) let codeAction = CodeAction(title: "1") @@ -191,7 +190,7 @@ final class CodeActionTests: XCTestCase { } func testEmptyCodeActionResult() async throws { - let testClient = try await TestSourceKitLSPClient(capabilities: clientCapabilitiesWithCodeActionSupport()) + let testClient = try await TestSourceKitLSPClient(capabilities: clientCapabilitiesWithCodeActionSupport) let uri = DocumentURI.for(.swift) let positions = testClient.openDocument( """ @@ -214,7 +213,7 @@ final class CodeActionTests: XCTestCase { } func testSemanticRefactorLocalRenameResult() async throws { - let testClient = try await TestSourceKitLSPClient(capabilities: clientCapabilitiesWithCodeActionSupport()) + let testClient = try await TestSourceKitLSPClient(capabilities: clientCapabilitiesWithCodeActionSupport) let uri = DocumentURI.for(.swift) let positions = testClient.openDocument( """ @@ -227,16 +226,20 @@ final class CodeActionTests: XCTestCase { ) let request = CodeActionRequest( - range: positions["1️⃣"]..(_ input: T1) {}" + ) + ] + ] + ), + command: nil + ) + ] + } + } + + func testOpaqueParameterToGenericIsNotShownFromTheBody() async throws { + try await assertCodeActions( + ##""" + func someFunction(_ input: some Value) 1️⃣{ + 2️⃣print("x") + }3️⃣ + """##, + exhaustive: false + ) { uri, positions in + [] + } + } + + func testConvertJSONToCodable() async throws { + try await assertCodeActions( + ##""" + 1️⃣{ + 2️⃣"id": 3️⃣1, + "values": 4️⃣["foo", "bar"] + }5️⃣ + + """##, + exhaustive: false + ) { uri, positions in + [ + CodeAction( + title: "Create Codable structs from JSON", + kind: .refactorInline, + diagnostics: nil, + edit: WorkspaceEdit( + changes: [ + uri: [ + TextEdit( + range: positions["1️⃣"].. [CodeAction], + file: StaticString = #filePath, + line: UInt = #line + ) async throws { + let testClient = try await TestSourceKitLSPClient(capabilities: clientCapabilitiesWithCodeActionSupport) + let uri = DocumentURI.for(.swift) + let positions = testClient.openDocument(markedText, uri: uri) + + for marker in markers ?? extractMarkers(markedText).markers.map(\.key) { + let result = try await testClient.send( + CodeActionRequest( + range: Range(positions[marker]), + context: .init(), + textDocument: TextDocumentIdentifier(uri) + ) + ) + guard case .codeActions(let codeActions) = result else { + XCTFail("Expected code actions at marker \(marker)", file: file, line: line) + return + } + if exhaustive { + XCTAssertEqual( + codeActions, + expected(uri, positions), + "Found unexpected code actions at \(marker)", + file: file, + line: line + ) + } else { + XCTAssert( + codeActions.contains(expected(uri, positions)), + """ + Code actions did not contain expected at \(marker): + \(codeActions) + """, + file: file, + line: line + ) + } + } + } } diff --git a/Tests/SourceKitLSPTests/SyntaxRefactorTests.swift b/Tests/SourceKitLSPTests/SyntaxRefactorTests.swift index f908385c4..45039bec2 100644 --- a/Tests/SourceKitLSPTests/SyntaxRefactorTests.swift +++ b/Tests/SourceKitLSPTests/SyntaxRefactorTests.swift @@ -42,6 +42,26 @@ final class SyntaxRefactorTests: XCTestCase { ) } + func testAddDocumentationRefactorSingleParameter() throws { + try assertRefactor( + """ + 1️⃣func 2️⃣refactor(syntax: DeclSyntax) { } + """, + context: (), + provider: AddDocumentation.self, + expected: [ + SourceEdit( + range: AbsolutePosition(utf8Offset: 0).. Date: Wed, 8 May 2024 10:06:03 -0700 Subject: [PATCH 16/46] =?UTF-8?q?Fix=20a=20bug=20that=20caused=20documenta?= =?UTF-8?q?tion=20to=20be=20added=20at=20the=20start=20of=20the=20declarat?= =?UTF-8?q?ion=E2=80=99s=20trivia?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This meant that if there were two newlines before the declaration, the documentation would be separated to the declaration by one newline and if the declaration was at the start of a line, the declaration would be on the same line as the doc comment, effectively making the documentation part of a comment. --- .../Swift/CodeActions/AddDocumentation.swift | 4 ++-- Tests/SourceKitLSPTests/SyntaxRefactorTests.swift | 12 ++++++------ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift b/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift index ee1e9babd..408c47745 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift @@ -52,7 +52,6 @@ public struct AddDocumentation: EditRefactoringProvider { let newlineAndIndentation = [.newlines(1)] + (syntax.firstToken(viewMode: .sourceAccurate)?.indentationOfLine ?? []) var content: [TriviaPiece] = [] - content += newlineAndIndentation content.append(.docLineComment("/// A description")) if let parameters = syntax.parameters?.parameters { @@ -82,8 +81,9 @@ public struct AddDocumentation: EditRefactoringProvider { content += newlineAndIndentation content.append(.docLineComment("/// - Returns:")) } + content += newlineAndIndentation - let insertPos = syntax.position + let insertPos = syntax.positionAfterSkippingLeadingTrivia return [ SourceEdit( range: insertPos.. Date: Wed, 8 May 2024 10:06:20 -0700 Subject: [PATCH 17/46] Add test cases that check code actions invoked from a range --- Tests/SourceKitLSPTests/CodeActionTests.swift | 80 ++++++++++++++++--- 1 file changed, 69 insertions(+), 11 deletions(-) diff --git a/Tests/SourceKitLSPTests/CodeActionTests.swift b/Tests/SourceKitLSPTests/CodeActionTests.swift index 481ebe175..28260123b 100644 --- a/Tests/SourceKitLSPTests/CodeActionTests.swift +++ b/Tests/SourceKitLSPTests/CodeActionTests.swift @@ -689,7 +689,8 @@ final class CodeActionTests: XCTestCase { try await assertCodeActions( """ let x = 1️⃣12️⃣63️⃣ - """ + """, + ranges: [("1️⃣", "2️⃣"), ("1️⃣", "3️⃣")] ) { uri, positions in [ CodeAction( @@ -728,6 +729,7 @@ final class CodeActionTests: XCTestCase { """ let x = 1️⃣#"Hello 2️⃣world"#3️⃣ """, + ranges: [("1️⃣", "3️⃣")], exhaustive: false ) { uri, positions in [ @@ -749,6 +751,7 @@ final class CodeActionTests: XCTestCase { ##""" let x = 1️⃣#"Hello 2️⃣\#(name)"#3️⃣ """##, + ranges: [("1️⃣", "3️⃣")], exhaustive: false ) { uri, positions in [ @@ -787,9 +790,10 @@ final class CodeActionTests: XCTestCase { func testMigrateIfLetSyntax() async throws { try await assertCodeActions( ##""" - 1️⃣if 2️⃣let 3️⃣foo = 4️⃣foo {}9️⃣ + 1️⃣if 2️⃣let 3️⃣foo = 4️⃣foo {}5️⃣ """##, - markers: ["1️⃣", "2️⃣", "3️⃣", "4️⃣"] + markers: ["1️⃣", "2️⃣", "3️⃣", "4️⃣"], + ranges: [("1️⃣", "4️⃣"), ("1️⃣", "5️⃣")] ) { uri, positions in [ CodeAction( @@ -800,7 +804,7 @@ final class CodeActionTests: XCTestCase { changes: [ uri: [ TextEdit( - range: positions["1️⃣"]..(_ input: T1) {}" ) ] @@ -875,6 +880,7 @@ final class CodeActionTests: XCTestCase { }5️⃣ """##, + ranges: [("1️⃣", "5️⃣")], exhaustive: false ) { uri, positions in [ @@ -903,9 +909,54 @@ final class CodeActionTests: XCTestCase { } } + func testAddDocumentationRefactorSingleParameter() async throws { + try await assertCodeActions( + """ + 1️⃣func 2️⃣refactor(3️⃣syntax: 4️⃣Decl5️⃣Syntax)6️⃣ { }7️⃣ + """, + ranges: [("1️⃣", "2️⃣"), ("1️⃣", "6️⃣"), ("1️⃣", "7️⃣")], + exhaustive: false + ) { uri, positions in + [ + CodeAction( + title: "Add documentation", + kind: .refactorInline, + diagnostics: nil, + edit: WorkspaceEdit( + changes: [ + uri: [ + TextEdit( + range: Range(positions["1️⃣"]), + newText: """ + /// A description + /// - Parameter syntax: + \("") + """ + ) + ] + ] + ), + command: nil + ) + ] + } + } + + /// Retrieves the code action at a set of markers and asserts that it matches a list of expected code actions. + /// + /// - Parameters: + /// - markedText: The source file input to get the code actions for. + /// - markers: The list of markers to retrieve code actions at. If `nil` code actions will be retrieved for all + /// markers in `markedText` + /// - ranges: If specified, code actions are also requested for selection ranges between these markers. + /// - exhaustive: Whether `expected` is expected to be a subset of the returned code actions or whether it is + /// expected to exhaustively match all code actions. + /// - expected: A closure that returns the list of expected code actions, given the URI of the test document and the + /// marker positions within. private func assertCodeActions( _ markedText: String, markers: [String]? = nil, + ranges: [(String, String)] = [], exhaustive: Bool = true, expected: (_ uri: DocumentURI, _ positions: DocumentPositions) -> [CodeAction], file: StaticString = #filePath, @@ -915,23 +966,30 @@ final class CodeActionTests: XCTestCase { let uri = DocumentURI.for(.swift) let positions = testClient.openDocument(markedText, uri: uri) - for marker in markers ?? extractMarkers(markedText).markers.map(\.key) { + var ranges = ranges + if let markers { + ranges += markers.map { ($0, $0) } + } else { + ranges += extractMarkers(markedText).markers.map(\.key).map { ($0, $0) } + } + + for (startMarker, endMarker) in ranges { let result = try await testClient.send( CodeActionRequest( - range: Range(positions[marker]), + range: positions[startMarker].. Date: Wed, 8 May 2024 10:11:59 -0700 Subject: [PATCH 18/46] =?UTF-8?q?Don=E2=80=99t=20show=20`Add=20documentati?= =?UTF-8?q?on`=20refactoring=20for=20declarations=20that=20are=20not=20on?= =?UTF-8?q?=20a=20new=20line?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../Swift/CodeActions/AddDocumentation.swift | 12 +++-- Tests/SourceKitLSPTests/CodeActionTests.swift | 47 ++++++++++++++++++- 2 files changed, 54 insertions(+), 5 deletions(-) diff --git a/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift b/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift index 408c47745..1d3543cd7 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/AddDocumentation.swift @@ -37,16 +37,20 @@ import SwiftSyntax public struct AddDocumentation: EditRefactoringProvider { @_spi(Testing) public static func textRefactor(syntax: DeclSyntax, in context: Void) -> [SourceEdit] { - let hasDocumentation = syntax.leadingTrivia.contains(where: { trivia in + let hasDocumentation = syntax.leadingTrivia.contains { trivia in switch trivia { - case .blockComment(_), .docBlockComment(_), .lineComment(_), .docLineComment(_): + case .blockComment, .docBlockComment, .lineComment, .docLineComment: return true default: return false } - }) + } + + // We consider nodes at the start of the source file at being on a new line + let isOnNewLine = + syntax.leadingTrivia.contains(where: \.isNewline) || syntax.previousToken(viewMode: .sourceAccurate) == nil - guard !hasDocumentation else { + guard !hasDocumentation && isOnNewLine else { return [] } diff --git a/Tests/SourceKitLSPTests/CodeActionTests.swift b/Tests/SourceKitLSPTests/CodeActionTests.swift index 28260123b..8a340a21e 100644 --- a/Tests/SourceKitLSPTests/CodeActionTests.swift +++ b/Tests/SourceKitLSPTests/CodeActionTests.swift @@ -909,10 +909,12 @@ final class CodeActionTests: XCTestCase { } } - func testAddDocumentationRefactorSingleParameter() async throws { + func testAddDocumentationRefactorNotAtStartOfFile() async throws { try await assertCodeActions( """ + struct Foo { 1️⃣func 2️⃣refactor(3️⃣syntax: 4️⃣Decl5️⃣Syntax)6️⃣ { }7️⃣ + } """, ranges: [("1️⃣", "2️⃣"), ("1️⃣", "6️⃣"), ("1️⃣", "7️⃣")], exhaustive: false @@ -942,6 +944,49 @@ final class CodeActionTests: XCTestCase { } } + func testAddDocumentationRefactorAtStartOfFile() async throws { + try await assertCodeActions( + """ + 1️⃣func 2️⃣refactor(3️⃣syntax: 4️⃣Decl5️⃣Syntax)6️⃣ { }7️⃣ + """, + ranges: [("1️⃣", "2️⃣"), ("1️⃣", "6️⃣"), ("1️⃣", "7️⃣")], + exhaustive: false + ) { uri, positions in + [ + CodeAction( + title: "Add documentation", + kind: .refactorInline, + diagnostics: nil, + edit: WorkspaceEdit( + changes: [ + uri: [ + TextEdit( + range: Range(positions["1️⃣"]), + newText: """ + /// A description + /// - Parameter syntax: + \("") + """ + ) + ] + ] + ), + command: nil + ) + ] + } + } + + func testAddDocumentationDoesNotShowUpIfItIsNotOnItsOwnLine() async throws { + try await assertCodeActions( + """ + var x = 1; var 1️⃣y = 2 + """ + ) { uri, positions in + [] + } + } + /// Retrieves the code action at a set of markers and asserts that it matches a list of expected code actions. /// /// - Parameters: From d4dd57861c051e9e77666fa5b96ff5d705b6370d Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Wed, 8 May 2024 16:02:11 -0700 Subject: [PATCH 19/46] Introduce a notion of `ConfiguredTargets` into the build system Instead of asking for build settings of a file, the build system manager asks for the targets of a file and then asks for the build settings of that file in a specific target. This has two advantages: - We know about targets and can prepare the targets for background indexing - Once we support build systems in which a single file can be part of multiple targets, we can have a centralized place that picks preferred targets for a file, eg. based on user configuration --- Sources/SKCore/BuildServerBuildSystem.swift | 10 +- Sources/SKCore/BuildSystem.swift | 31 +++++- Sources/SKCore/BuildSystemManager.swift | 45 ++++++-- .../CompilationDatabaseBuildSystem.swift | 11 +- .../SwiftPMBuildSystem.swift | 104 +++++++++++------- .../SKCoreTests/BuildSystemManagerTests.swift | 6 +- .../CompilationDatabaseTests.swift | 1 + .../SwiftPMBuildSystemTests.swift | 10 +- .../SourceKitLSPTests/BuildSystemTests.swift | 10 +- 9 files changed, 174 insertions(+), 54 deletions(-) diff --git a/Sources/SKCore/BuildServerBuildSystem.swift b/Sources/SKCore/BuildServerBuildSystem.swift index f45dd93fa..361e1a981 100644 --- a/Sources/SKCore/BuildServerBuildSystem.swift +++ b/Sources/SKCore/BuildServerBuildSystem.swift @@ -263,7 +263,11 @@ extension BuildServerBuildSystem: BuildSystem { /// /// Returns `nil` if no build settings have been received from the build /// server yet or if no build settings are available for this file. - public func buildSettings(for document: DocumentURI, language: Language) async -> FileBuildSettings? { + public func buildSettings( + for document: DocumentURI, + in target: ConfiguredTarget, + language: Language + ) async -> FileBuildSettings? { return buildSettings[document] } @@ -271,6 +275,10 @@ extension BuildServerBuildSystem: BuildSystem { return nil } + public func configuredTargets(for document: DocumentURI) async -> [ConfiguredTarget] { + return [ConfiguredTarget(targetID: "dummy", runDestinationID: "dummy")] + } + public func registerForChangeNotifications(for uri: DocumentURI) { let request = RegisterForChanges(uri: uri, action: .register) _ = self.buildServer?.send(request) { result in diff --git a/Sources/SKCore/BuildSystem.swift b/Sources/SKCore/BuildSystem.swift index ee48db020..03b2bdad9 100644 --- a/Sources/SKCore/BuildSystem.swift +++ b/Sources/SKCore/BuildSystem.swift @@ -43,6 +43,27 @@ public struct SourceFileInfo: Sendable { } } +/// A target / run destination combination. For example, a configured target can represent building the target +/// `MyLibrary` for iOS. +public struct ConfiguredTarget: Hashable, Sendable { + /// An opaque string that represents the target. + /// + /// The target's ID should be generated by the build system that handles the target and only interpreted by that + /// build system. + public let targetID: String + + /// An opaque string that represents the run destination. + /// + /// The run destination's ID should be generated by the build system that handles the target and only interpreted by + /// that build system. + public let runDestinationID: String + + public init(targetID: String, runDestinationID: String) { + self.targetID = targetID + self.runDestinationID = runDestinationID + } +} + /// Provider of FileBuildSettings and other build-related information. /// /// The primary role of the build system is to answer queries for @@ -53,7 +74,6 @@ public struct SourceFileInfo: Sendable { /// For example, a SwiftPMWorkspace provides compiler arguments for the files /// contained in a SwiftPM package root directory. public protocol BuildSystem: AnyObject, Sendable { - /// The root of the project that this build system manages. For example, for SwiftPM packages, this is the folder /// containing Package.swift. For compilation databases it is the root folder based on which the compilation database /// was found. @@ -85,7 +105,14 @@ public protocol BuildSystem: AnyObject, Sendable { /// /// Returns `nil` if the build system can't provide build settings for this /// file or if it hasn't computed build settings for the file yet. - func buildSettings(for document: DocumentURI, language: Language) async throws -> FileBuildSettings? + func buildSettings( + for document: DocumentURI, + in target: ConfiguredTarget, + language: Language + ) async throws -> FileBuildSettings? + + /// Return the list of targets and run destinations that the given document can be built for. + func configuredTargets(for document: DocumentURI) async -> [ConfiguredTarget] /// If the build system has knowledge about the language that this document should be compiled in, return it. /// diff --git a/Sources/SKCore/BuildSystemManager.swift b/Sources/SKCore/BuildSystemManager.swift index a7100f9ac..bc83e7830 100644 --- a/Sources/SKCore/BuildSystemManager.swift +++ b/Sources/SKCore/BuildSystemManager.swift @@ -122,22 +122,53 @@ extension BuildSystemManager { } } + /// Returns the `ConfiguredTarget` that should be used for semantic functionality of the given document. + public func canonicalConfiguredTarget(for document: DocumentURI) async -> ConfiguredTarget? { + // Sort the configured targets to deterministically pick the same `ConfiguredTarget` every time. + // We could allow the user to specify a preference of one target over another. For now this is not necessary because + // no build system currently returns multiple targets for a source file. + return await buildSystem?.configuredTargets(for: document) + .sorted { ($0.targetID, $0.runDestinationID) < ($1.targetID, $1.runDestinationID) } + .first + } + + /// Returns the build settings for `document` from `buildSystem`. + /// + /// Implementation detail of `buildSettings(for:language:)`. + private func buildSettingsFromPrimaryBuildSystem( + for document: DocumentURI, + language: Language + ) async throws -> FileBuildSettings? { + guard let buildSystem else { + return nil + } + guard let target = await canonicalConfiguredTarget(for: document) else { + logger.error("Failed to get target for \(document.forLogging)") + return nil + } + // FIXME: (async) We should only wait `fallbackSettingsTimeout` for build + // settings and return fallback afterwards. I am not sure yet, how best to + // implement that with Swift concurrency. + // For now, this should be fine because all build systems return + // very quickly from `settings(for:language:)`. + guard let settings = try await buildSystem.buildSettings(for: document, in: target, language: language) else { + return nil + } + return settings + } + private func buildSettings( for document: DocumentURI, language: Language ) async -> FileBuildSettings? { do { - // FIXME: (async) We should only wait `fallbackSettingsTimeout` for build - // settings and return fallback afterwards. I am not sure yet, how best to - // implement that with Swift concurrency. - // For now, this should be fine because all build systems return - // very quickly from `settings(for:language:)`. - if let settings = try await buildSystem?.buildSettings(for: document, language: language) { - return settings + if let buildSettings = try await buildSettingsFromPrimaryBuildSystem(for: document, language: language) { + return buildSettings } } catch { logger.error("Getting build settings failed: \(error.forLogging)") } + guard var settings = fallbackBuildSystem?.buildSettings(for: document, language: language) else { return nil } diff --git a/Sources/SKCore/CompilationDatabaseBuildSystem.swift b/Sources/SKCore/CompilationDatabaseBuildSystem.swift index 031b1d10c..b8eaf03b9 100644 --- a/Sources/SKCore/CompilationDatabaseBuildSystem.swift +++ b/Sources/SKCore/CompilationDatabaseBuildSystem.swift @@ -93,14 +93,17 @@ public actor CompilationDatabaseBuildSystem { } extension CompilationDatabaseBuildSystem: BuildSystem { - public var indexDatabasePath: AbsolutePath? { indexStorePath?.parentDirectory.appending(component: "IndexDatabase") } public var indexPrefixMappings: [PathPrefixMapping] { return [] } - public func buildSettings(for document: DocumentURI, language: Language) async -> FileBuildSettings? { + public func buildSettings( + for document: DocumentURI, + in buildTarget: ConfiguredTarget, + language: Language + ) async -> FileBuildSettings? { guard let url = document.fileURL else { // We can't determine build settings for non-file URIs. return nil @@ -118,6 +121,10 @@ extension CompilationDatabaseBuildSystem: BuildSystem { return nil } + public func configuredTargets(for document: DocumentURI) async -> [ConfiguredTarget] { + return [ConfiguredTarget(targetID: "dummy", runDestinationID: "dummy")] + } + public func registerForChangeNotifications(for uri: DocumentURI) async { self.watchedFiles.insert(uri) } diff --git a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift index bd33e1596..6d6f6a334 100644 --- a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift +++ b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift @@ -28,10 +28,14 @@ import Workspace import struct Basics.AbsolutePath import struct Basics.TSCAbsolutePath import struct Foundation.URL +import struct TSCBasic.AbsolutePath import protocol TSCBasic.FileSystem +import class TSCBasic.Process import var TSCBasic.localFileSystem import func TSCBasic.resolveSymlinks +typealias AbsolutePath = Basics.AbsolutePath + #if canImport(SPMBuildCore) import SPMBuildCore #endif @@ -91,9 +95,11 @@ public actor SwiftPMBuildSystem { let workspace: Workspace public let buildParameters: BuildParameters let fileSystem: FileSystem + private let toolchainRegistry: ToolchainRegistry var fileToTarget: [AbsolutePath: SwiftBuildTarget] = [:] var sourceDirToTarget: [AbsolutePath: SwiftBuildTarget] = [:] + var targets: [SwiftBuildTarget] = [] /// The URIs for which the delegate has registered for change notifications, /// mapped to the language the delegate specified when registering for change notifications. @@ -129,6 +135,7 @@ public actor SwiftPMBuildSystem { ) async throws { self.workspacePath = workspacePath self.fileSystem = fileSystem + self.toolchainRegistry = toolchainRegistry guard let packageRoot = findPackageDirectory(containing: workspacePath, fileSystem) else { throw Error.noManifest(workspacePath: workspacePath) @@ -259,6 +266,8 @@ extension SwiftPMBuildSystem { /// with only some properties modified. self.modulesGraph = modulesGraph + self.targets = try buildDescription.allTargetsInTopologicalOrder(in: modulesGraph) + self.fileToTarget = [AbsolutePath: SwiftBuildTarget]( modulesGraph.allTargets.flatMap { target in return target.sources.paths.compactMap { @@ -314,36 +323,42 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { public var indexPrefixMappings: [PathPrefixMapping] { return [] } - public func buildSettings(for uri: DocumentURI, language: Language) throws -> FileBuildSettings? { - // SwiftPMBuildSystem doesn't respect the langue specified by the editor. - return try buildSettings(for: uri) - } - - private func buildSettings(for uri: DocumentURI) throws -> FileBuildSettings? { - guard let url = uri.fileURL else { + public func buildSettings( + for uri: DocumentURI, + in configuredTarget: ConfiguredTarget, + language: Language + ) throws -> FileBuildSettings? { + guard let url = uri.fileURL, let path = try? AbsolutePath(validating: url.path) else { // We can't determine build settings for non-file URIs. return nil } - guard let path = try? AbsolutePath(validating: url.path) else { - return nil - } - if let buildTarget = try buildTarget(for: path) { - return FileBuildSettings( - compilerArguments: try buildTarget.compileArguments(for: path.asURL), - workingDirectory: workspacePath.pathString - ) + if configuredTarget.targetID == "" { + return try settings(forPackageManifest: path) } - if path.basename == "Package.swift" { - return try settings(forPackageManifest: path) + let buildTargets = self.targets.filter({ $0.name == configuredTarget.targetID }) + if buildTargets.count > 1 { + logger.error("Found multiple targets with name \(configuredTarget.targetID). Picking the first one") + } + guard let buildTarget = buildTargets.first else { + if buildTargets.isEmpty { + logger.error("Did not find target with name \(configuredTarget.targetID)") + } + return nil } - if path.extension == "h" { - return try settings(forHeader: path) + if url.pathExtension == "h", let substituteFile = buildTarget.sources.first { + return FileBuildSettings( + compilerArguments: try buildTarget.compileArguments(for: substituteFile), + workingDirectory: workspacePath.pathString + ).patching(newFile: try resolveSymlinks(path).pathString, originalFile: substituteFile.absoluteString) } - return nil + return FileBuildSettings( + compilerArguments: try buildTarget.compileArguments(for: url), + workingDirectory: workspacePath.pathString + ) } public func defaultLanguage(for document: DocumentURI) async -> Language? { @@ -351,6 +366,29 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { return nil } + public func configuredTargets(for uri: DocumentURI) -> [ConfiguredTarget] { + guard let url = uri.fileURL, let path = try? AbsolutePath(validating: url.path) else { + // We can't determine targets for non-file URIs. + return [] + } + + if let target = try? buildTarget(for: path) { + return [ConfiguredTarget(targetID: target.name, runDestinationID: "dummy")] + } + + if path.basename == "Package.swift" { + // We use an empty target name to represent the package manifest since an empty target name is not valid for any + // user-defined target. + return [ConfiguredTarget(targetID: "", runDestinationID: "dummy")] + } + + if url.pathExtension == "h", let target = try? target(forHeader: path) { + return [target] + } + + return [] + } + public func registerForChangeNotifications(for uri: DocumentURI) async { self.watchedFiles.insert(uri) } @@ -437,10 +475,10 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { } public func fileHandlingCapability(for uri: DocumentURI) -> FileHandlingCapability { - if (try? buildSettings(for: uri)) != nil { - return .handled + if configuredTargets(for: uri).isEmpty { + return .unhandled } - return .unhandled + return .handled } public func sourceFiles() -> [SourceFileInfo] { @@ -485,25 +523,13 @@ extension SwiftPMBuildSystem { return canonicalPath == path ? nil : impl(canonicalPath) } - /// Retrieve settings for a given header file. - /// - /// This finds the target the header belongs to based on its location in the file system, retrieves the build settings - /// for any file within that target and generates compiler arguments by replacing that picked file with the header - /// file. - /// This is safe because all files within one target have the same build settings except for reference to the file - /// itself, which we are replacing. - private func settings(forHeader path: AbsolutePath) throws -> FileBuildSettings? { - func impl(_ path: AbsolutePath) throws -> FileBuildSettings? { + /// This finds the target the header belongs to based on its location in the file system. + private func target(forHeader path: AbsolutePath) throws -> ConfiguredTarget? { + func impl(_ path: AbsolutePath) throws -> ConfiguredTarget? { var dir = path.parentDirectory while !dir.isRoot { if let buildTarget = sourceDirToTarget[dir] { - if let sourceFile = buildTarget.sources.first { - return FileBuildSettings( - compilerArguments: try buildTarget.compileArguments(for: sourceFile), - workingDirectory: workspacePath.pathString - ).patching(newFile: path.pathString, originalFile: sourceFile.absoluteString) - } - return nil + return ConfiguredTarget(targetID: buildTarget.name, runDestinationID: "dummy") } dir = dir.parentDirectory } diff --git a/Tests/SKCoreTests/BuildSystemManagerTests.swift b/Tests/SKCoreTests/BuildSystemManagerTests.swift index 8dd89ccf7..09a201cd6 100644 --- a/Tests/SKCoreTests/BuildSystemManagerTests.swift +++ b/Tests/SKCoreTests/BuildSystemManagerTests.swift @@ -445,7 +445,7 @@ class ManualBuildSystem: BuildSystem { self.delegate = delegate } - func buildSettings(for uri: DocumentURI, language: Language) -> FileBuildSettings? { + func buildSettings(for uri: DocumentURI, in buildTarget: ConfiguredTarget, language: Language) -> FileBuildSettings? { return map[uri] } @@ -453,6 +453,10 @@ class ManualBuildSystem: BuildSystem { return nil } + public func configuredTargets(for document: DocumentURI) async -> [ConfiguredTarget] { + return [ConfiguredTarget(targetID: "dummy", runDestinationID: "dummy")] + } + func registerForChangeNotifications(for uri: DocumentURI) async { } diff --git a/Tests/SKCoreTests/CompilationDatabaseTests.swift b/Tests/SKCoreTests/CompilationDatabaseTests.swift index 3498ba425..934c1aaa7 100644 --- a/Tests/SKCoreTests/CompilationDatabaseTests.swift +++ b/Tests/SKCoreTests/CompilationDatabaseTests.swift @@ -290,6 +290,7 @@ final class CompilationDatabaseTests: XCTestCase { ) { buildSystem in let settings = await buildSystem.buildSettings( for: DocumentURI(URL(fileURLWithPath: "/a/a.swift")), + in: ConfiguredTarget(targetID: "dummy", runDestinationID: "dummy"), language: .swift ) XCTAssertNotNil(settings) diff --git a/Tests/SKSwiftPMWorkspaceTests/SwiftPMBuildSystemTests.swift b/Tests/SKSwiftPMWorkspaceTests/SwiftPMBuildSystemTests.swift index 7aed19472..900915f7e 100644 --- a/Tests/SKSwiftPMWorkspaceTests/SwiftPMBuildSystemTests.swift +++ b/Tests/SKSwiftPMWorkspaceTests/SwiftPMBuildSystemTests.swift @@ -27,8 +27,16 @@ import struct PackageModel.BuildFlags import SPMBuildCore #endif -final class SwiftPMWorkspaceTests: XCTestCase { +fileprivate extension SwiftPMBuildSystem { + func buildSettings(for uri: DocumentURI, language: Language) throws -> FileBuildSettings? { + guard let target = self.configuredTargets(for: uri).only else { + return nil + } + return try buildSettings(for: uri, in: target, language: language) + } +} +final class SwiftPMBuildSystemTests: XCTestCase { func testNoPackage() async throws { let fs = InMemoryFileSystem() try await withTestScratchDir { tempDir in diff --git a/Tests/SourceKitLSPTests/BuildSystemTests.swift b/Tests/SourceKitLSPTests/BuildSystemTests.swift index 7e43505ff..252abea30 100644 --- a/Tests/SourceKitLSPTests/BuildSystemTests.swift +++ b/Tests/SourceKitLSPTests/BuildSystemTests.swift @@ -39,7 +39,11 @@ final class TestBuildSystem: BuildSystem { /// Files currently being watched by our delegate. var watchedFiles: Set = [] - func buildSettings(for document: DocumentURI, language: Language) async throws -> FileBuildSettings? { + func buildSettings( + for document: DocumentURI, + in buildTarget: ConfiguredTarget, + language: Language + ) async throws -> FileBuildSettings? { return buildSettingsByFile[document] } @@ -47,6 +51,10 @@ final class TestBuildSystem: BuildSystem { return nil } + public func configuredTargets(for document: DocumentURI) async -> [ConfiguredTarget] { + return [ConfiguredTarget(targetID: "dummy", runDestinationID: "dummy")] + } + func registerForChangeNotifications(for uri: DocumentURI) async { watchedFiles.insert(uri) } From baa3f616c94ea65bb0e01cb59c89cf9df771c22c Mon Sep 17 00:00:00 2001 From: Paul LeMarquand Date: Wed, 8 May 2024 23:01:41 -0400 Subject: [PATCH 20/46] Handle XCTest extensions --- .../Swift/SwiftTestingScanner.swift | 2 +- .../Swift/SyntacticTestIndex.swift | 1 - Sources/SourceKitLSP/TestDiscovery.swift | 54 ++++++++++++------- 3 files changed, 37 insertions(+), 20 deletions(-) diff --git a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift index dbcfc7e9a..9a5c3e50b 100644 --- a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift +++ b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift @@ -249,7 +249,7 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { } let range = snapshot.range(of: node.positionAfterSkippingLeadingTrivia.. [AnnotatedTestItem] { syntaxTreeManager: syntaxTreeManager ) async let xcTests = SyntacticSwiftXCTestScanner.findTestSymbols(in: snapshot, syntaxTreeManager: syntaxTreeManager) - .map { AnnotatedTestItem(testItem: $0, isExtension: false) } return await swiftTestingTests + xcTests } diff --git a/Sources/SourceKitLSP/TestDiscovery.swift b/Sources/SourceKitLSP/TestDiscovery.swift index 6949c1188..becdba115 100644 --- a/Sources/SourceKitLSP/TestDiscovery.swift +++ b/Sources/SourceKitLSP/TestDiscovery.swift @@ -25,7 +25,7 @@ public struct AnnotatedTestItem { /// The test item to be annotated public var testItem: TestItem - /// Whether the `TestItem` is declared in an extension. + /// Whether the `TestItem` is an extension. public var isExtension: Bool public init( @@ -360,7 +360,7 @@ final class SyntacticSwiftXCTestScanner: SyntaxVisitor { private var snapshot: DocumentSnapshot /// The workspace symbols representing the found `XCTestCase` subclasses and test methods. - private var result: [TestItem] = [] + private var result: [AnnotatedTestItem] = [] private init(snapshot: DocumentSnapshot) { self.snapshot = snapshot @@ -370,7 +370,7 @@ final class SyntacticSwiftXCTestScanner: SyntaxVisitor { public static func findTestSymbols( in snapshot: DocumentSnapshot, syntaxTreeManager: SyntaxTreeManager - ) async -> [TestItem] { + ) async -> [AnnotatedTestItem] { guard snapshot.text.contains("XCTestCase") || snapshot.text.contains("test") else { // If the file contains tests that can be discovered syntactically, it needs to have a class inheriting from // `XCTestCase` or a function starting with `test`. @@ -437,14 +437,17 @@ final class SyntacticSwiftXCTestScanner: SyntaxVisitor { return .visitChildren } let range = snapshot.range(of: node.positionAfterSkippingLeadingTrivia.. SyntaxVisitorContinueKind { result += findTestMethods(in: node.memberBlock.members, containerName: node.extendedType.trimmedDescription) + .map { AnnotatedTestItem(testItem: $0, isExtension: true) } return .visitChildren } } @@ -485,6 +489,20 @@ extension TestItem { } } +extension AnnotatedTestItem { + /// Use out-of-date semantic information to filter syntactic symbols. + /// + /// Delegates to the `TestItem`'s `filterUsing(semanticSymbols:)` method to perform the filtering. + fileprivate func filterUsing(semanticSymbols: [Symbol]?) -> AnnotatedTestItem? { + guard let testItem = self.testItem.filterUsing(semanticSymbols: semanticSymbols) else { + return nil + } + var test = self + test.testItem = testItem + return test + } +} + extension Array { /// When the test scanners discover tests in extensions they are captured in their own parent `TestItem`, not the /// `TestItem` generated from the class/struct's definition. This is largely because of the syntatic nature of the @@ -569,13 +587,11 @@ extension Array { } } - // Filter out the items that have been merged into their parents, sorting the tests by location. - // TestItems not in extensions should be priotitized first. + // Sort the tests by location, prioritizing TestItems not in extensions. let sortedItems = itemDict.values - .compactMap { $0 } .sorted { ($0.isExtension != $1.isExtension) ? !$0.isExtension : ($0.testItem.location < $1.testItem.location) } - return sortedItems.map { + let result = sortedItems.map { guard !$0.testItem.children.isEmpty, mergedIds.contains($0.testItem.id) else { return $0.testItem } @@ -585,12 +601,15 @@ extension Array { .mergingTestsInExtensions() return newItem } + return result } } extension SwiftLanguageService { - public func syntacticDocumentTests(for uri: DocumentURI, in workspace: Workspace) async throws -> [AnnotatedTestItem]? - { + public func syntacticDocumentTests( + for uri: DocumentURI, + in workspace: Workspace + ) async throws -> [AnnotatedTestItem]? { let snapshot = try documentManager.latestSnapshot(uri) let semanticSymbols = workspace.index(checkedFor: .deletedFiles)?.symbols(inFilePath: snapshot.uri.pseudoPath) let xctestSymbols = await SyntacticSwiftXCTestScanner.findTestSymbols( @@ -598,7 +617,6 @@ extension SwiftLanguageService { syntaxTreeManager: syntaxTreeManager ) .compactMap { $0.filterUsing(semanticSymbols: semanticSymbols) } - .map { AnnotatedTestItem(testItem: $0, isExtension: false) } let swiftTestingSymbols = await SyntacticSwiftTestingTestScanner.findTestSymbols( in: snapshot, From a1d10d8bedd8d3eeea4142445ddc7f0592d86ec7 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Thu, 9 May 2024 10:26:23 -0700 Subject: [PATCH 21/46] Fall back to launching subprocesses without a working directory MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Details from https://github.com/apple/sourcekit-lsp/issues/1271 > Amazon Linux 2 and CentOS 7 have a glibc that doesn’t support `posix_spawn_file_actions_addchdir_np` and thus `TSCBasic.Process` can’t launch a process on these platforms with the working directory set. We currently fall back to launching the index tasks without a working directory on these platforms, which I think is fine because SwiftPM gives us compiler arguments with absolute paths. But we should figure something out. > > Using `Foundation.Process` is not an option because it runs `chdir` on the current process for Posix platforms, which is racy if there are multiple subprocesses being spawned simultaneously. On Windows `TSCBasic.Processs` uses `Foundation.Process` and `Foundation.Process` properly set the working directory of the subprocesses on Windows, so Windows is not a problem. rdar://127797048 --- Sources/SKSupport/CMakeLists.txt | 1 + ...LaunchWithWorkingDirectoryIfPossible.swift | 70 +++++++++++++++++++ .../UpdateIndexStoreTaskDescription.swift | 14 ++-- 3 files changed, 75 insertions(+), 10 deletions(-) create mode 100644 Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift diff --git a/Sources/SKSupport/CMakeLists.txt b/Sources/SKSupport/CMakeLists.txt index 7cb96d058..8c5e1685a 100644 --- a/Sources/SKSupport/CMakeLists.txt +++ b/Sources/SKSupport/CMakeLists.txt @@ -11,6 +11,7 @@ add_library(SKSupport STATIC DocumentURI+CustomLogStringConvertible.swift FileSystem.swift LineTable.swift + Process+LaunchWithWorkingDirectoryIfPossible.swift Process+WaitUntilExitWithCancellation.swift Random.swift Result.swift diff --git a/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift b/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift new file mode 100644 index 000000000..732331969 --- /dev/null +++ b/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift @@ -0,0 +1,70 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift.org open source project +// +// Copyright (c) 2014 - 2024 Apple Inc. and the Swift project authors +// Licensed under Apache License v2.0 with Runtime Library Exception +// +// See https://swift.org/LICENSE.txt for license information +// See https://swift.org/CONTRIBUTORS.txt for the list of Swift project authors +// +//===----------------------------------------------------------------------===// + +import LSPLogging + +import struct TSCBasic.AbsolutePath +import class TSCBasic.Process +import enum TSCBasic.ProcessEnv +import struct TSCBasic.ProcessEnvironmentBlock + +extension Process { + /// Launches a new process with the given parameters. + /// + /// - Important: If `workingDirectory` is not supported on this platform, this logs an error and falls back to launching the + /// process without the working directory set. + public static func launch( + arguments: [String], + environmentBlock: ProcessEnvironmentBlock = ProcessEnv.block, + workingDirectory: AbsolutePath?, + outputRedirection: OutputRedirection = .collect, + startNewProcessGroup: Bool = true, + loggingHandler: LoggingHandler? = .none + ) throws -> Process { + let process = + if let workingDirectory { + Process( + arguments: arguments, + environmentBlock: environmentBlock, + workingDirectory: workingDirectory, + outputRedirection: outputRedirection, + startNewProcessGroup: startNewProcessGroup, + loggingHandler: loggingHandler + ) + } else { + self.init( + arguments: arguments, + environmentBlock: environmentBlock, + outputRedirection: outputRedirection, + startNewProcessGroup: startNewProcessGroup, + loggingHandler: loggingHandler + ) + } + do { + try process.launch() + } catch Process.Error.workingDirectoryNotSupported where workingDirectory != nil { + // TODO (indexing): We need to figure out how to set the working directory on all platforms. + logger.error( + "Working directory not supported on the platform. Launching process without working directory \(workingDirectory!.pathString)" + ) + return try Process.launch( + arguments: arguments, + environmentBlock: environmentBlock, + workingDirectory: nil, + outputRedirection: outputRedirection, + startNewProcessGroup: startNewProcessGroup, + loggingHandler: loggingHandler + ) + } + return process + } +} diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index 9ef7924a6..56331f874 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -188,16 +188,10 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { return } - let process = - if let workingDirectory = buildSettings.workingDirectory { - Process( - arguments: [swiftc.pathString] + indexingArguments, - workingDirectory: try AbsolutePath(validating: workingDirectory) - ) - } else { - Process(arguments: [swiftc.pathString] + indexingArguments) - } - try process.launch() + let process = try Process.launch( + arguments: [swiftc.pathString] + indexingArguments, + workingDirectory: buildSettings.workingDirectory.map(AbsolutePath.init(validating:)) + ) let result = try await process.waitUntilExitSendingSigIntOnTaskCancellation() switch result.exitStatus.exhaustivelySwitchable { case .terminated(code: 0): From 7b4bbeb39d73c745b891208056b15fa0c746af09 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Thu, 9 May 2024 15:19:31 -0700 Subject: [PATCH 22/46] Address review comments to #1216 --- Sources/SemanticIndex/SemanticIndexManager.swift | 8 +------- .../UpdateIndexStoreTaskDescription.swift | 13 ++++++------- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/Sources/SemanticIndex/SemanticIndexManager.swift b/Sources/SemanticIndex/SemanticIndexManager.swift index 037509f50..057a13b8f 100644 --- a/Sources/SemanticIndex/SemanticIndexManager.swift +++ b/Sources/SemanticIndex/SemanticIndexManager.swift @@ -97,16 +97,10 @@ public final actor SemanticIndexManager { logger.info( "Waiting for up-to-date index for \(uris.map { $0.fileURL?.lastPathComponent ?? $0.stringValue }.joined(separator: ", "))" ) - let filesWithOutOfDateIndex = uris.filter { uri in - switch indexStatus[uri] { - case .inProgress, nil: return true - case .upToDate: return false - } - } // Create a new index task for the files that aren't up-to-date. The newly scheduled index tasks will // - Wait for the existing index operations to finish if they have the same number of files. // - Reschedule the background index task in favor of an index task with fewer source files. - await self.index(files: filesWithOutOfDateIndex, priority: nil).value + await self.index(files: uris, priority: nil).value index.pollForUnitChangesAndWait() logger.debug("Done waiting for up-to-date index") } diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index 56331f874..9af7a5081 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -176,11 +176,6 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { buildSettings: FileBuildSettings, toolchain: Toolchain ) async throws { - let indexingArguments = adjustSwiftCompilerArgumentsForIndexStoreUpdate( - buildSettings.compilerArguments, - fileToIndex: uri - ) - guard let swiftc = toolchain.swiftc else { logger.error( "Not updating index store for \(uri.forLogging) because toolchain \(toolchain.identifier) does not contain a Swift compiler" @@ -188,6 +183,11 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { return } + let indexingArguments = adjustSwiftCompilerArgumentsForIndexStoreUpdate( + buildSettings.compilerArguments, + fileToIndex: uri + ) + let process = try Process.launch( arguments: [swiftc.pathString] + indexingArguments, workingDirectory: buildSettings.workingDirectory.map(AbsolutePath.init(validating:)) @@ -289,7 +289,6 @@ private func adjustSwiftCompilerArgumentsForIndexStoreUpdate( result += [argument, nextArgument] continue } - result.append(argument) } result.append(argument) } @@ -299,7 +298,7 @@ private func adjustSwiftCompilerArgumentsForIndexStoreUpdate( // batch mode is not compatible with -index-file "-disable-batch-mode", // Fake an output path so that we get a different unit file for every Swift file we background index - "-o", fileToIndex.pseudoPath + ".o", + "-index-unit-output-path", fileToIndex.pseudoPath + ".o", ] return result } From a71b428568d86b3ba77a15a287e1a145720deb2f Mon Sep 17 00:00:00 2001 From: Kim de Vos Date: Thu, 9 May 2024 15:53:25 +0200 Subject: [PATCH 23/46] Remove `?? position` as it never was used --- .../SourceKitLSP/Swift/AdjustPositionToStartOfIdentifier.swift | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Sources/SourceKitLSP/Swift/AdjustPositionToStartOfIdentifier.swift b/Sources/SourceKitLSP/Swift/AdjustPositionToStartOfIdentifier.swift index f77e6ec4c..11397a781 100644 --- a/Sources/SourceKitLSP/Swift/AdjustPositionToStartOfIdentifier.swift +++ b/Sources/SourceKitLSP/Swift/AdjustPositionToStartOfIdentifier.swift @@ -56,7 +56,7 @@ extension SwiftLanguageService { let visitor = StartOfIdentifierFinder(position: snapshot.absolutePosition(of: position)) visitor.walk(tree) if let resolvedPosition = visitor.resolvedPosition { - return snapshot.position(of: resolvedPosition) ?? position + return snapshot.position(of: resolvedPosition) } return position } From c5699fb4dde964283c18c97255ac426bcf7f6439 Mon Sep 17 00:00:00 2001 From: Kim de Vos Date: Thu, 9 May 2024 16:09:08 +0200 Subject: [PATCH 24/46] Fix deprecated `ByteSourceRange` --- Sources/SourceKitLSP/Rename.swift | 2 +- .../Swift/CodeActions/PackageManifestEdits.swift | 2 +- .../SyntaxRefactoringCodeActionProvider.swift | 2 +- Sources/SourceKitLSP/Swift/Diagnostic.swift | 6 +++--- Sources/SourceKitLSP/Swift/DocumentSymbols.swift | 8 +++++--- Sources/SourceKitLSP/Swift/SemanticTokens.swift | 4 ++-- Sources/SourceKitLSP/Swift/SwiftLanguageService.swift | 11 +++++++---- Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift | 8 ++++++-- Sources/SourceKitLSP/TestDiscovery.swift | 6 ++++-- 9 files changed, 30 insertions(+), 19 deletions(-) diff --git a/Sources/SourceKitLSP/Rename.swift b/Sources/SourceKitLSP/Rename.swift index 3ac6984b6..0ce10e968 100644 --- a/Sources/SourceKitLSP/Rename.swift +++ b/Sources/SourceKitLSP/Rename.swift @@ -989,7 +989,7 @@ extension SwiftLanguageService { } if let startToken, let endToken { - return snapshot.range( + return snapshot.absolutePositionRange( of: startToken.positionAfterSkippingLeadingTrivia.. TextEdit? in let edit = TextEdit( - range: scope.snapshot.range(of: edit.range), + range: scope.snapshot.absolutePositionRange(of: edit.range), newText: edit.replacement ) if edit.isNoOp(in: scope.snapshot) { diff --git a/Sources/SourceKitLSP/Swift/Diagnostic.swift b/Sources/SourceKitLSP/Swift/Diagnostic.swift index b47e28d5b..136cc7540 100644 --- a/Sources/SourceKitLSP/Swift/Diagnostic.swift +++ b/Sources/SourceKitLSP/Swift/Diagnostic.swift @@ -61,7 +61,7 @@ extension CodeAction { init?(_ fixIt: FixIt, in snapshot: DocumentSnapshot) { var textEdits = [TextEdit]() for edit in fixIt.edits { - textEdits.append(TextEdit(range: snapshot.range(of: edit.range), newText: edit.replacement)) + textEdits.append(TextEdit(range: snapshot.absolutePositionRange(of: edit.range), newText: edit.replacement)) } self.init( @@ -281,7 +281,7 @@ extension Diagnostic { var range = Range(snapshot.position(of: diag.position)) for highlight in diag.highlights { let swiftSyntaxRange = highlight.positionAfterSkippingLeadingTrivia.., callerFile: StaticString = #fileID, callerLine: UInt = #line @@ -1169,9 +1169,12 @@ extension DocumentSnapshot { of range: Range, callerFile: StaticString = #fileID, callerLine: UInt = #line - ) -> ByteSourceRange { + ) -> Range { let utf8OffsetRange = utf8OffsetRange(of: range, callerFile: callerFile, callerLine: callerLine) - return ByteSourceRange(offset: utf8OffsetRange.startIndex, length: utf8OffsetRange.count) + return Range( + position: AbsolutePosition(utf8Offset: utf8OffsetRange.startIndex), + length: SourceLength(utf8Length: utf8OffsetRange.count) + ) } // MARK: Position <-> RenameLocation diff --git a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift index 9a5c3e50b..31f640fef 100644 --- a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift +++ b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift @@ -248,7 +248,9 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { return .skipChildren } - let range = snapshot.range(of: node.positionAfterSkippingLeadingTrivia.. Date: Thu, 9 May 2024 15:45:59 +0200 Subject: [PATCH 25/46] Replace `IncrementalEdit` with `SourceEdit` --- .../Swift/DocumentFormatting.swift | 18 ++++-------------- .../Swift/SwiftLanguageService.swift | 10 ++-------- 2 files changed, 6 insertions(+), 22 deletions(-) diff --git a/Sources/SourceKitLSP/Swift/DocumentFormatting.swift b/Sources/SourceKitLSP/Swift/DocumentFormatting.swift index ed58a21a5..42e75dfcb 100644 --- a/Sources/SourceKitLSP/Swift/DocumentFormatting.swift +++ b/Sources/SourceKitLSP/Swift/DocumentFormatting.swift @@ -106,10 +106,10 @@ private func edits(from original: DocumentSnapshot, to edited: String) -> [TextE switch change { case .insert(offset: let offset, element: let element, associatedWith: _): let absolutePosition = AbsolutePosition(utf8Offset: offset) - return IncrementalEdit(range: absolutePosition.. [TextE // Map the offset-based edits to line-column based edits to be consumed by LSP - return concurrentEdits.edits.compactMap { (edit) -> TextEdit? in - let (startLine, startColumn) = original.lineTable.lineAndUTF16ColumnOf(utf8Offset: edit.offset) - let (endLine, endColumn) = original.lineTable.lineAndUTF16ColumnOf(utf8Offset: edit.endOffset) - guard let newText = String(bytes: edit.replacement, encoding: .utf8) else { - logger.fault("Failed to get String from UTF-8 bytes \(edit.replacement)") - return nil - } - - return TextEdit( - range: Position(line: startLine, utf16index: startColumn).. Date: Thu, 9 May 2024 12:58:49 -0700 Subject: [PATCH 26/46] Support background preparation of targets --- Sources/SKCore/BuildServerBuildSystem.swift | 10 ++ Sources/SKCore/BuildSystem.swift | 32 ++++- Sources/SKCore/BuildSystemManager.swift | 14 +- .../CompilationDatabaseBuildSystem.swift | 12 +- .../Collection+PartitionIntoBatches.swift | 16 ++- .../SwiftPMBuildSystem.swift | 89 +++++++++++- .../SKTestSupport/FileManager+findFiles.swift | 12 ++ .../SwiftPMDependencyProject.swift | 4 +- Sources/SemanticIndex/CMakeLists.txt | 2 + .../SemanticIndex/IndexTaskDescription.swift | 105 ++++++++++++++ .../PreparationTaskDescription.swift | 107 ++++++++++++++ .../SemanticIndex/SemanticIndexManager.swift | 135 +++++++++++++++--- .../UpdateIndexStoreTaskDescription.swift | 2 + Sources/SourceKitLSP/SourceKitLSPServer.swift | 2 +- Sources/SourceKitLSP/Workspace.swift | 12 +- .../SKCoreTests/BuildSystemManagerTests.swift | 10 ++ .../BackgroundIndexingTests.swift | 130 +++++++++++++++++ .../SourceKitLSPTests/BuildSystemTests.swift | 10 ++ 18 files changed, 661 insertions(+), 43 deletions(-) create mode 100644 Sources/SemanticIndex/IndexTaskDescription.swift create mode 100644 Sources/SemanticIndex/PreparationTaskDescription.swift diff --git a/Sources/SKCore/BuildServerBuildSystem.swift b/Sources/SKCore/BuildServerBuildSystem.swift index 361e1a981..01561d721 100644 --- a/Sources/SKCore/BuildServerBuildSystem.swift +++ b/Sources/SKCore/BuildServerBuildSystem.swift @@ -279,6 +279,16 @@ extension BuildServerBuildSystem: BuildSystem { return [ConfiguredTarget(targetID: "dummy", runDestinationID: "dummy")] } + public func generateBuildGraph() {} + + public func topologicalSort(of targets: [ConfiguredTarget]) async -> [ConfiguredTarget]? { + return nil + } + + public func prepare(targets: [ConfiguredTarget]) async throws { + throw PrepareNotSupportedError() + } + public func registerForChangeNotifications(for uri: DocumentURI) { let request = RegisterForChanges(uri: uri, action: .register) _ = self.buildServer?.send(request) { result in diff --git a/Sources/SKCore/BuildSystem.swift b/Sources/SKCore/BuildSystem.swift index 03b2bdad9..f6880a00b 100644 --- a/Sources/SKCore/BuildSystem.swift +++ b/Sources/SKCore/BuildSystem.swift @@ -31,14 +31,19 @@ public struct SourceFileInfo: Sendable { /// The URI of the source file. public let uri: DocumentURI + /// `true` if this file belongs to the root project that the user is working on. It is false, if the file belongs + /// to a dependency of the project. + public let isPartOfRootProject: Bool + /// Whether the file might contain test cases. This property is an over-approximation. It might be true for files /// from non-test targets or files that don't actually contain any tests. Keeping this list of files with /// `mayContainTets` minimal as possible helps reduce the amount of work that the syntactic test indexer needs to /// perform. public let mayContainTests: Bool - public init(uri: DocumentURI, mayContainTests: Bool) { + public init(uri: DocumentURI, isPartOfRootProject: Bool, mayContainTests: Bool) { self.uri = uri + self.isPartOfRootProject = isPartOfRootProject self.mayContainTests = mayContainTests } } @@ -64,6 +69,13 @@ public struct ConfiguredTarget: Hashable, Sendable { } } +/// An error build systems can throw from `prepare` if they don't support preparation of targets. +public struct PrepareNotSupportedError: Error, CustomStringConvertible { + public init() {} + + public var description: String { "Preparation not supported" } +} + /// Provider of FileBuildSettings and other build-related information. /// /// The primary role of the build system is to answer queries for @@ -114,6 +126,22 @@ public protocol BuildSystem: AnyObject, Sendable { /// Return the list of targets and run destinations that the given document can be built for. func configuredTargets(for document: DocumentURI) async -> [ConfiguredTarget] + /// Re-generate the build graph including all the tasks that are necessary for building the entire build graph, like + /// resolving package versions. + func generateBuildGraph() async throws + + /// Sort the targets so that low-level targets occur before high-level targets. + /// + /// This sorting is best effort but allows the indexer to prepare and index low-level targets first, which allows + /// index data to be available earlier. + /// + /// `nil` if the build system doesn't support topological sorting of targets. + func topologicalSort(of targets: [ConfiguredTarget]) async -> [ConfiguredTarget]? + + /// Prepare the given targets for indexing and semantic functionality. This should build all swift modules of target + /// dependencies. + func prepare(targets: [ConfiguredTarget]) async throws + /// If the build system has knowledge about the language that this document should be compiled in, return it. /// /// This is used to determine the language in which a source file should be background indexed. @@ -146,5 +174,3 @@ public protocol BuildSystem: AnyObject, Sendable { /// The callback might also be called without an actual change to `sourceFiles`. func addSourceFilesDidChangeCallback(_ callback: @Sendable @escaping () async -> Void) async } - -public let buildTargetsNotSupported = ResponseError.methodNotFound(BuildTargets.method) diff --git a/Sources/SKCore/BuildSystemManager.swift b/Sources/SKCore/BuildSystemManager.swift index bc83e7830..49a23aa42 100644 --- a/Sources/SKCore/BuildSystemManager.swift +++ b/Sources/SKCore/BuildSystemManager.swift @@ -208,6 +208,18 @@ extension BuildSystemManager { return settings } + public func generateBuildGraph() async throws { + try await self.buildSystem?.generateBuildGraph() + } + + public func topologicalSort(of targets: [ConfiguredTarget]) async throws -> [ConfiguredTarget] { + return await buildSystem?.topologicalSort(of: targets) ?? targets + } + + public func prepare(targets: [ConfiguredTarget]) async throws { + try await buildSystem?.prepare(targets: targets) + } + public func registerForChangeNotifications(for uri: DocumentURI, language: Language) async { logger.debug("registerForChangeNotifications(\(uri.forLogging))") let mainFile = await mainFile(for: uri, language: language) @@ -247,7 +259,7 @@ extension BuildSystemManager { public func testFiles() async -> [DocumentURI] { return await sourceFiles().compactMap { (info: SourceFileInfo) -> DocumentURI? in - guard info.mayContainTests else { + guard info.isPartOfRootProject, info.mayContainTests else { return nil } return info.uri diff --git a/Sources/SKCore/CompilationDatabaseBuildSystem.swift b/Sources/SKCore/CompilationDatabaseBuildSystem.swift index b8eaf03b9..eab796d52 100644 --- a/Sources/SKCore/CompilationDatabaseBuildSystem.swift +++ b/Sources/SKCore/CompilationDatabaseBuildSystem.swift @@ -125,6 +125,16 @@ extension CompilationDatabaseBuildSystem: BuildSystem { return [ConfiguredTarget(targetID: "dummy", runDestinationID: "dummy")] } + public func prepare(targets: [ConfiguredTarget]) async throws { + throw PrepareNotSupportedError() + } + + public func generateBuildGraph() {} + + public func topologicalSort(of targets: [ConfiguredTarget]) -> [ConfiguredTarget]? { + return nil + } + public func registerForChangeNotifications(for uri: DocumentURI) async { self.watchedFiles.insert(uri) } @@ -208,7 +218,7 @@ extension CompilationDatabaseBuildSystem: BuildSystem { return [] } return compdb.allCommands.map { - SourceFileInfo(uri: DocumentURI($0.url), mayContainTests: true) + SourceFileInfo(uri: DocumentURI($0.url), isPartOfRootProject: true, mayContainTests: true) } } diff --git a/Sources/SKSupport/Collection+PartitionIntoBatches.swift b/Sources/SKSupport/Collection+PartitionIntoBatches.swift index ed1effdc9..674c11cff 100644 --- a/Sources/SKSupport/Collection+PartitionIntoBatches.swift +++ b/Sources/SKSupport/Collection+PartitionIntoBatches.swift @@ -10,7 +10,7 @@ // //===----------------------------------------------------------------------===// -public extension Collection { +public extension Collection where Index == Int { /// Partition the elements of the collection into `numberOfBatches` roughly equally sized batches. /// /// Elements are assigned to the batches round-robin. This ensures that elements that are close to each other in the @@ -32,4 +32,18 @@ public extension Collection { } return batches.filter { !$0.isEmpty } } + + /// Partition the collection into batches that have a maximum size of `batchSize`. + /// + /// The last batch will contain the remainder elements. + func partition(intoBatchesOfSize batchSize: Int) -> [[Element]] { + var batches: [[Element]] = [] + batches.reserveCapacity(self.count / batchSize) + var lastIndex = self.startIndex + for index in stride(from: self.startIndex, to: self.endIndex, by: batchSize).dropFirst() + [self.endIndex] { + batches.append(Array(self[lastIndex.. Language? { - // TODO (indexing): Query The SwiftPM build system for the document's language + // TODO (indexing): Query The SwiftPM build system for the document's language. + // https://github.com/apple/sourcekit-lsp/issues/1267 return nil } @@ -395,6 +406,77 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { return [] } + public func topologicalSort(of targets: [ConfiguredTarget]) -> [ConfiguredTarget]? { + return targets.sorted { (lhs: ConfiguredTarget, rhs: ConfiguredTarget) -> Bool in + let lhsIndex = self.targets.firstIndex(where: { $0.name == lhs.targetID }) ?? self.targets.count + let rhsIndex = self.targets.firstIndex(where: { $0.name == rhs.targetID }) ?? self.targets.count + return lhsIndex < rhsIndex + } + } + + public func prepare(targets: [ConfiguredTarget]) async throws { + // TODO (indexing): Support preparation of multiple targets at once. + // https://github.com/apple/sourcekit-lsp/issues/1262 + for target in targets { + try await prepare(singleTarget: target) + } + } + + private func prepare(singleTarget target: ConfiguredTarget) async throws { + // TODO (indexing): Add a proper 'prepare' job in SwiftPM instead of building the target. + // https://github.com/apple/sourcekit-lsp/issues/1254 + guard let toolchain = await toolchainRegistry.default else { + logger.error("Not preparing because not toolchain exists") + return + } + guard let swift = toolchain.swift else { + logger.error( + "Not preparing because toolchain at \(toolchain.identifier) does not contain a Swift compiler" + ) + return + } + let arguments = [ + swift.pathString, "build", + "--scratch-path", self.workspace.location.scratchDirectory.pathString, + "--disable-index-store", + "--target", target.targetID, + ] + let process = Process( + arguments: arguments, + workingDirectory: try TSCBasic.AbsolutePath(validating: workspacePath.pathString) + ) + try process.launch() + let result = try await process.waitUntilExitSendingSigIntOnTaskCancellation() + switch result.exitStatus.exhaustivelySwitchable { + case .terminated(code: 0): + break + case .terminated(code: let code): + // This most likely happens if there are compilation errors in the source file. This is nothing to worry about. + let stdout = (try? String(bytes: result.output.get(), encoding: .utf8)) ?? "" + let stderr = (try? String(bytes: result.stderrOutput.get(), encoding: .utf8)) ?? "" + logger.debug( + """ + Preparation of targets \(target.targetID) terminated with non-zero exit code \(code) + Stderr: + \(stderr) + Stdout: + \(stdout) + """ + ) + case .signalled(signal: let signal): + if !Task.isCancelled { + // The indexing job finished with a signal. Could be because the compiler crashed. + // Ignore signal exit codes if this task has been cancelled because the compiler exits with SIGINT if it gets + // interrupted. + logger.error("Preparation of targets \(target.targetID) signaled \(signal)") + } + case .abnormal(exception: let exception): + if !Task.isCancelled { + logger.error("Preparation of targets \(target.targetID) exited abnormally \(exception)") + } + } + } + public func registerForChangeNotifications(for uri: DocumentURI) async { self.watchedFiles.insert(uri) } @@ -489,14 +571,11 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { public func sourceFiles() -> [SourceFileInfo] { return fileToTarget.compactMap { (path, target) -> SourceFileInfo? in - guard target.isPartOfRootPackage else { - // Don't consider files from package dependencies as possible test files. - return nil - } // We should only set mayContainTests to `true` for files from test targets // (https://github.com/apple/sourcekit-lsp/issues/1174). return SourceFileInfo( uri: DocumentURI(path.asURL), + isPartOfRootProject: target.isPartOfRootPackage, mayContainTests: true ) } diff --git a/Sources/SKTestSupport/FileManager+findFiles.swift b/Sources/SKTestSupport/FileManager+findFiles.swift index 232d158a2..97bd1befc 100644 --- a/Sources/SKTestSupport/FileManager+findFiles.swift +++ b/Sources/SKTestSupport/FileManager+findFiles.swift @@ -24,4 +24,16 @@ extension FileManager { } return result } + + /// Returns the URLs of all files with the given file extension in the given directory (recursively). + public func findFiles(named name: String, in directory: URL) -> [URL] { + var result: [URL] = [] + let enumerator = self.enumerator(at: directory, includingPropertiesForKeys: nil) + while let url = enumerator?.nextObject() as? URL { + if url.lastPathComponent == name { + result.append(url) + } + } + return result + } } diff --git a/Sources/SKTestSupport/SwiftPMDependencyProject.swift b/Sources/SKTestSupport/SwiftPMDependencyProject.swift index c35c7ddd9..496834c5c 100644 --- a/Sources/SKTestSupport/SwiftPMDependencyProject.swift +++ b/Sources/SKTestSupport/SwiftPMDependencyProject.swift @@ -71,13 +71,13 @@ public class SwiftPMDependencyProject { var files = files files["Package.swift"] = manifest - for (fileLocation, contents) in files { + for (fileLocation, markedContents) in files { let fileURL = fileLocation.url(relativeTo: packageDirectory) try FileManager.default.createDirectory( at: fileURL.deletingLastPathComponent(), withIntermediateDirectories: true ) - try contents.write(to: fileURL, atomically: true, encoding: .utf8) + try extractMarkers(markedContents).textWithoutMarkers.write(to: fileURL, atomically: true, encoding: .utf8) } try await runGitCommand(["init"], workingDirectory: packageDirectory) diff --git a/Sources/SemanticIndex/CMakeLists.txt b/Sources/SemanticIndex/CMakeLists.txt index 2d805df8c..197bfde6e 100644 --- a/Sources/SemanticIndex/CMakeLists.txt +++ b/Sources/SemanticIndex/CMakeLists.txt @@ -1,6 +1,8 @@ add_library(SemanticIndex STATIC CheckedIndex.swift + IndexTaskDescription.swift + PreparationTaskDescription.swift SemanticIndexManager.swift UpdateIndexStoreTaskDescription.swift ) diff --git a/Sources/SemanticIndex/IndexTaskDescription.swift b/Sources/SemanticIndex/IndexTaskDescription.swift new file mode 100644 index 000000000..e2e2c21f2 --- /dev/null +++ b/Sources/SemanticIndex/IndexTaskDescription.swift @@ -0,0 +1,105 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift.org open source project +// +// Copyright (c) 2014 - 2024 Apple Inc. and the Swift project authors +// Licensed under Apache License v2.0 with Runtime Library Exception +// +// See https://swift.org/LICENSE.txt for license information +// See https://swift.org/CONTRIBUTORS.txt for the list of Swift project authors +// +//===----------------------------------------------------------------------===// + +import SKCore + +/// A task that either prepares targets or updates the index store for a set of files. +public enum IndexTaskDescription: TaskDescriptionProtocol { + case updateIndexStore(UpdateIndexStoreTaskDescription) + case preparation(PreparationTaskDescription) + + public var isIdempotent: Bool { + switch self { + case .updateIndexStore(let taskDescription): return taskDescription.isIdempotent + case .preparation(let taskDescription): return taskDescription.isIdempotent + } + } + + public var estimatedCPUCoreCount: Int { + switch self { + case .updateIndexStore(let taskDescription): return taskDescription.estimatedCPUCoreCount + case .preparation(let taskDescription): return taskDescription.estimatedCPUCoreCount + } + } + + public var id: String { + switch self { + case .updateIndexStore(let taskDescription): return "indexing-\(taskDescription.id)" + case .preparation(let taskDescription): return "preparation-\(taskDescription.id)" + } + } + + public var description: String { + switch self { + case .updateIndexStore(let taskDescription): return taskDescription.description + case .preparation(let taskDescription): return taskDescription.description + } + } + + public var redactedDescription: String { + switch self { + case .updateIndexStore(let taskDescription): return taskDescription.redactedDescription + case .preparation(let taskDescription): return taskDescription.redactedDescription + } + } + + public func execute() async { + switch self { + case .updateIndexStore(let taskDescription): return await taskDescription.execute() + case .preparation(let taskDescription): return await taskDescription.execute() + } + } + + /// Forward to the underlying task to compute the dependencies. Preparation and index tasks don't have any + /// dependencies that are managed by `TaskScheduler`. `SemanticIndexManager` awaits the preparation of a target before + /// indexing files within it. + public func dependencies( + to currentlyExecutingTasks: [IndexTaskDescription] + ) -> [TaskDependencyAction] { + switch self { + case .updateIndexStore(let taskDescription): + let currentlyExecutingTasks = + currentlyExecutingTasks + .compactMap { (currentlyExecutingTask) -> UpdateIndexStoreTaskDescription? in + if case .updateIndexStore(let currentlyExecutingTask) = currentlyExecutingTask { + return currentlyExecutingTask + } + return nil + } + return taskDescription.dependencies(to: currentlyExecutingTasks).map { + switch $0 { + case .waitAndElevatePriorityOfDependency(let td): + return .waitAndElevatePriorityOfDependency(.updateIndexStore(td)) + case .cancelAndRescheduleDependency(let td): + return .cancelAndRescheduleDependency(.updateIndexStore(td)) + } + } + case .preparation(let taskDescription): + let currentlyExecutingTasks = + currentlyExecutingTasks + .compactMap { (currentlyExecutingTask) -> PreparationTaskDescription? in + if case .preparation(let currentlyExecutingTask) = currentlyExecutingTask { + return currentlyExecutingTask + } + return nil + } + return taskDescription.dependencies(to: currentlyExecutingTasks).map { + switch $0 { + case .waitAndElevatePriorityOfDependency(let td): + return .waitAndElevatePriorityOfDependency(.preparation(td)) + case .cancelAndRescheduleDependency(let td): + return .cancelAndRescheduleDependency(.preparation(td)) + } + } + } + } +} diff --git a/Sources/SemanticIndex/PreparationTaskDescription.swift b/Sources/SemanticIndex/PreparationTaskDescription.swift new file mode 100644 index 000000000..d674dd5b6 --- /dev/null +++ b/Sources/SemanticIndex/PreparationTaskDescription.swift @@ -0,0 +1,107 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift.org open source project +// +// Copyright (c) 2014 - 2024 Apple Inc. and the Swift project authors +// Licensed under Apache License v2.0 with Runtime Library Exception +// +// See https://swift.org/LICENSE.txt for license information +// See https://swift.org/CONTRIBUTORS.txt for the list of Swift project authors +// +//===----------------------------------------------------------------------===// + +import CAtomics +import Foundation +import LSPLogging +import LanguageServerProtocol +import SKCore + +import struct TSCBasic.AbsolutePath +import class TSCBasic.Process + +private var preparationIDForLogging = AtomicUInt32(initialValue: 1) + +/// Describes a task to index a set of source files. +/// +/// This task description can be scheduled in a `TaskScheduler`. +public struct PreparationTaskDescription: TaskDescriptionProtocol { + public let id = preparationIDForLogging.fetchAndIncrement() + + /// The files that should be indexed. + private let targetsToPrepare: [ConfiguredTarget] + + /// The build system manager that is used to get the toolchain and build settings for the files to index. + private let buildSystemManager: BuildSystemManager + + /// A callback that is called when the task finishes. + /// + /// Intended for testing purposes. + private let didFinishCallback: @Sendable (PreparationTaskDescription) -> Void + + /// The task is idempotent because indexing the same file twice produces the same result as indexing it once. + public var isIdempotent: Bool { true } + + public var estimatedCPUCoreCount: Int { 1 } + + public var description: String { + return self.redactedDescription + } + + public var redactedDescription: String { + return "preparation-\(id)" + } + + init( + targetsToPrepare: [ConfiguredTarget], + buildSystemManager: BuildSystemManager, + didFinishCallback: @escaping @Sendable (PreparationTaskDescription) -> Void + ) { + self.targetsToPrepare = targetsToPrepare + self.buildSystemManager = buildSystemManager + self.didFinishCallback = didFinishCallback + } + + public func execute() async { + defer { + didFinishCallback(self) + } + // Only use the last two digits of the preparation ID for the logging scope to avoid creating too many scopes. + // See comment in `withLoggingScope`. + // The last 2 digits should be sufficient to differentiate between multiple concurrently running preparation operations + await withLoggingScope("preparation-\(id % 100)") { + let startDate = Date() + let targetsToPrepare = targetsToPrepare.sorted(by: { + ($0.targetID, $0.runDestinationID) < ($1.targetID, $1.runDestinationID) + }) + let targetsToPrepareDescription = targetsToPrepare.map { $0.targetID }.joined(separator: ", ") + logger.log( + "Starting preparation with priority \(Task.currentPriority.rawValue, privacy: .public): \(targetsToPrepareDescription)" + ) + do { + try await buildSystemManager.prepare(targets: targetsToPrepare) + } catch { + logger.error( + "Preparation failed: \(error.forLogging)" + ) + } + logger.log( + "Finished preparation in \(Date().timeIntervalSince(startDate) * 1000, privacy: .public)ms: \(targetsToPrepareDescription)" + ) + } + } + + public func dependencies( + to currentlyExecutingTasks: [PreparationTaskDescription] + ) -> [TaskDependencyAction] { + return currentlyExecutingTasks.compactMap { (other) -> TaskDependencyAction? in + if other.targetsToPrepare.count > self.targetsToPrepare.count { + // If there is an prepare operation with more targets already running, suspend it. + // The most common use case for this is if we prepare all targets simultaneously during the initial preparation + // when a project is opened and need a single target indexed for user interaction. We should suspend the + // workspace-wide preparation and just prepare the currently needed target. + return .cancelAndRescheduleDependency(other) + } + return .waitAndElevatePriorityOfDependency(other) + } + } +} diff --git a/Sources/SemanticIndex/SemanticIndexManager.swift b/Sources/SemanticIndex/SemanticIndexManager.swift index 057a13b8f..b326c4d75 100644 --- a/Sources/SemanticIndex/SemanticIndexManager.swift +++ b/Sources/SemanticIndex/SemanticIndexManager.swift @@ -37,23 +37,27 @@ public final actor SemanticIndexManager { /// Files that have never been indexed are not in this dictionary. private var indexStatus: [DocumentURI: FileIndexStatus] = [:] + /// The task to generate the build graph (resolving package dependencies, generating the build description, + /// ...). `nil` if no build graph is currently being generated. + private var generateBuildGraphTask: Task? + /// The `TaskScheduler` that manages the scheduling of index tasks. This is shared among all `SemanticIndexManager`s /// in the process, to ensure that we don't schedule more index operations than processor cores from multiple /// workspaces. - private let indexTaskScheduler: TaskScheduler + private let indexTaskScheduler: TaskScheduler /// Callback that is called when an index task has finished. /// /// Currently only used for testing. - private let indexTaskDidFinish: (@Sendable (UpdateIndexStoreTaskDescription) -> Void)? + private let indexTaskDidFinish: (@Sendable (IndexTaskDescription) -> Void)? // MARK: - Public API public init( index: UncheckedIndex, buildSystemManager: BuildSystemManager, - indexTaskScheduler: TaskScheduler, - indexTaskDidFinish: (@Sendable (UpdateIndexStoreTaskDescription) -> Void)? + indexTaskScheduler: TaskScheduler, + indexTaskDidFinish: (@Sendable (IndexTaskDescription) -> Void)? ) { self.index = index.checked(for: .modifiedFiles) self.buildSystemManager = buildSystemManager @@ -65,13 +69,27 @@ public final actor SemanticIndexManager { /// Returns immediately after scheduling that task. /// /// Indexing is being performed with a low priority. - public func scheduleBackgroundIndex(files: some Collection) { - self.index(files: files, priority: .low) + public func scheduleBackgroundIndex(files: some Collection) async { + await self.index(files: files, priority: .low) + } + + /// Regenerate the build graph (also resolving package dependencies) and then index all the source files known to the + /// build system. + public func scheduleBuildGraphGenerationAndBackgroundIndexAllFiles() async { + generateBuildGraphTask = Task(priority: .low) { + await orLog("Generating build graph") { try await self.buildSystemManager.generateBuildGraph() } + await scheduleBackgroundIndex(files: await self.buildSystemManager.sourceFiles().map(\.uri)) + generateBuildGraphTask = nil + } } /// Wait for all in-progress index tasks to finish. public func waitForUpToDateIndex() async { logger.info("Waiting for up-to-date index") + // Wait for a build graph update first, if one is in progress. This will add all index tasks to `indexStatus`, so we + // can await the index tasks below. + await generateBuildGraphTask?.value + await withTaskGroup(of: Void.self) { taskGroup in for (_, status) in indexStatus { switch status { @@ -97,6 +115,10 @@ public final actor SemanticIndexManager { logger.info( "Waiting for up-to-date index for \(uris.map { $0.fileURL?.lastPathComponent ?? $0.stringValue }.joined(separator: ", "))" ) + // If there's a build graph update in progress wait for that to finish so we can discover new files in the build + // system. + await generateBuildGraphTask?.value + // Create a new index task for the files that aren't up-to-date. The newly scheduled index tasks will // - Wait for the existing index operations to finish if they have the same number of files. // - Reschedule the background index task in favor of an index task with fewer source files. @@ -107,41 +129,108 @@ public final actor SemanticIndexManager { // MARK: - Helper functions + /// Prepare the given targets for indexing + private func prepare(targets: [ConfiguredTarget], priority: TaskPriority?) async { + await self.indexTaskScheduler.schedule( + priority: priority, + .preparation( + PreparationTaskDescription( + targetsToPrepare: targets, + buildSystemManager: self.buildSystemManager, + didFinishCallback: { [weak self] taskDescription in + self?.indexTaskDidFinish?(.preparation(taskDescription)) + } + ) + ) + ).value + } + + /// Update the index store for the given files, assuming that their targets have already been prepared. + private func updateIndexStore(for files: [DocumentURI], priority: TaskPriority?) async { + await self.indexTaskScheduler.schedule( + priority: priority, + .updateIndexStore( + UpdateIndexStoreTaskDescription( + filesToIndex: Set(files), + buildSystemManager: self.buildSystemManager, + index: self.index, + didFinishCallback: { [weak self] taskDescription in + self?.indexTaskDidFinish?(.updateIndexStore(taskDescription)) + } + ) + ) + ).value + for file in files { + self.indexStatus[file] = .upToDate + } + } + /// Index the given set of files at the given priority. /// /// The returned task finishes when all files are indexed. @discardableResult - private func index(files: some Collection, priority: TaskPriority?) -> Task { + private func index(files: some Collection, priority: TaskPriority?) async -> Task { let outOfDateFiles = files.filter { if case .upToDate = indexStatus[$0] { return false } return true } + .sorted(by: { $0.stringValue < $1.stringValue }) // sort files to get deterministic indexing order + + // Sort the targets in topological order so that low-level targets get built before high-level targets, allowing us + // to index the low-level targets ASAP. + var filesByTarget: [ConfiguredTarget: [DocumentURI]] = [:] + for file in outOfDateFiles { + guard let target = await buildSystemManager.canonicalConfiguredTarget(for: file) else { + logger.error("Not indexing \(file.forLogging) because the target could not be determined") + continue + } + filesByTarget[target, default: []].append(file) + } + + var sortedTargets: [ConfiguredTarget] = + await orLog("Sorting targets") { try await buildSystemManager.topologicalSort(of: Array(filesByTarget.keys)) } + ?? Array(filesByTarget.keys).sorted(by: { $0.targetID < $1.targetID }) + + if Set(sortedTargets) != Set(filesByTarget.keys) { + logger.fault( + """ + Sorting targets topologically changed set of targets: + \(sortedTargets.map(\.targetID).joined(separator: ", ")) != \(filesByTarget.keys.map(\.targetID).joined(separator: ", ")) + """ + ) + sortedTargets = Array(filesByTarget.keys).sorted(by: { $0.targetID < $1.targetID }) + } var indexTasks: [Task] = [] - // TODO (indexing): Group index operations by target when we support background preparation. - for files in outOfDateFiles.partition(intoNumberOfBatches: ProcessInfo.processInfo.processorCount * 5) { + // TODO (indexing): When we can index multiple targets concurrently in SwiftPM, increase the batch size to half the + // processor count, so we can get parallelism during preparation. + // https://github.com/apple/sourcekit-lsp/issues/1262 + for targetsBatch in sortedTargets.partition(intoBatchesOfSize: 1) { let indexTask = Task(priority: priority) { - await self.indexTaskScheduler.schedule( - priority: priority, - UpdateIndexStoreTaskDescription( - filesToIndex: Set(files), - buildSystemManager: self.buildSystemManager, - index: self.index, - didFinishCallback: { [weak self] taskDescription in - self?.indexTaskDidFinish?(taskDescription) + // First prepare the targets. + await prepare(targets: targetsBatch, priority: priority) + + // And after preparation is done, index the files in the targets. + await withTaskGroup(of: Void.self) { taskGroup in + for target in targetsBatch { + // TODO (indexing): Once swiftc supports indexing of multiple files in a single invocation, increase the + // batch size to allow it to share AST builds between multiple files within a target. + // https://github.com/apple/sourcekit-lsp/issues/1268 + for fileBatch in filesByTarget[target]!.partition(intoBatchesOfSize: 1) { + taskGroup.addTask { + await self.updateIndexStore(for: fileBatch, priority: priority) + } } - ) - ).value - for file in files { - indexStatus[file] = .upToDate + } + await taskGroup.waitForAll() } } indexTasks.append(indexTask) - for file in files { + for file in targetsBatch.flatMap({ filesByTarget[$0]! }) { indexStatus[file] = .inProgress(indexTask) } } @@ -150,7 +239,7 @@ public final actor SemanticIndexManager { return Task(priority: priority) { await withTaskGroup(of: Void.self) { taskGroup in for indexTask in indexTasksImmutable { - taskGroup.addTask(priority: priority) { + taskGroup.addTask { await indexTask.value } } diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index 9af7a5081..ac3f491af 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -87,6 +87,7 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { let filesToIndex = filesToIndex.sorted(by: { $0.stringValue < $1.stringValue }) // TODO (indexing): Once swiftc supports it, we should group files by target and index files within the same // target together in one swiftc invocation. + // https://github.com/apple/sourcekit-lsp/issues/1268 for file in filesToIndex { await updateIndexStoreForSingleFile(file) } @@ -163,6 +164,7 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { } case .c, .cpp, .objective_c, .objective_cpp: // TODO (indexing): Support indexing of clang files, including headers. + // https://github.com/apple/sourcekit-lsp/issues/1253 break default: logger.error( diff --git a/Sources/SourceKitLSP/SourceKitLSPServer.swift b/Sources/SourceKitLSP/SourceKitLSPServer.swift index f925eb9bb..3c4f67518 100644 --- a/Sources/SourceKitLSP/SourceKitLSPServer.swift +++ b/Sources/SourceKitLSP/SourceKitLSPServer.swift @@ -453,7 +453,7 @@ public actor SourceKitLSPServer { /// /// Shared process-wide to ensure the scheduled index operations across multiple workspaces don't exceed the maximum /// number of processor cores that the user allocated to background indexing. - private let indexTaskScheduler: TaskScheduler + private let indexTaskScheduler: TaskScheduler private var packageLoadingWorkDoneProgress = WorkDoneProgressState( "SourceKitLSP.SourceKitLSPServer.reloadPackage", diff --git a/Sources/SourceKitLSP/Workspace.swift b/Sources/SourceKitLSP/Workspace.swift index 5306b66f4..831b06775 100644 --- a/Sources/SourceKitLSP/Workspace.swift +++ b/Sources/SourceKitLSP/Workspace.swift @@ -89,7 +89,7 @@ public final class Workspace { underlyingBuildSystem: BuildSystem?, index uncheckedIndex: UncheckedIndex?, indexDelegate: SourceKitIndexDelegate?, - indexTaskScheduler: TaskScheduler + indexTaskScheduler: TaskScheduler ) async { self.documentManager = documentManager self.buildSetup = options.buildSetup @@ -123,8 +123,8 @@ public final class Workspace { } // Trigger an initial population of `syntacticTestIndex`. await syntacticTestIndex.listOfTestFilesDidChange(buildSystemManager.testFiles()) - if let semanticIndexManager, let underlyingBuildSystem { - await semanticIndexManager.scheduleBackgroundIndex(files: await underlyingBuildSystem.sourceFiles().map(\.uri)) + if let semanticIndexManager { + await semanticIndexManager.scheduleBuildGraphGenerationAndBackgroundIndexAllFiles() } } @@ -142,7 +142,7 @@ public final class Workspace { options: SourceKitLSPServer.Options, compilationDatabaseSearchPaths: [RelativePath], indexOptions: IndexOptions = IndexOptions(), - indexTaskScheduler: TaskScheduler, + indexTaskScheduler: TaskScheduler, reloadPackageStatusCallback: @escaping (ReloadPackageStatus) async -> Void ) async throws { var buildSystem: BuildSystem? = nil @@ -295,7 +295,7 @@ public struct IndexOptions { /// A callback that is called when an index task finishes. /// /// Intended for testing purposes. - public var indexTaskDidFinish: (@Sendable (UpdateIndexStoreTaskDescription) -> Void)? + public var indexTaskDidFinish: (@Sendable (IndexTaskDescription) -> Void)? public init( indexStorePath: AbsolutePath? = nil, @@ -304,7 +304,7 @@ public struct IndexOptions { listenToUnitEvents: Bool = true, enableBackgroundIndexing: Bool = false, maxCoresPercentageToUseForBackgroundIndexing: Double = 1, - indexTaskDidFinish: (@Sendable (UpdateIndexStoreTaskDescription) -> Void)? = nil + indexTaskDidFinish: (@Sendable (IndexTaskDescription) -> Void)? = nil ) { self.indexStorePath = indexStorePath self.indexDatabasePath = indexDatabasePath diff --git a/Tests/SKCoreTests/BuildSystemManagerTests.swift b/Tests/SKCoreTests/BuildSystemManagerTests.swift index 09a201cd6..8baa896ca 100644 --- a/Tests/SKCoreTests/BuildSystemManagerTests.swift +++ b/Tests/SKCoreTests/BuildSystemManagerTests.swift @@ -457,6 +457,16 @@ class ManualBuildSystem: BuildSystem { return [ConfiguredTarget(targetID: "dummy", runDestinationID: "dummy")] } + public func prepare(targets: [ConfiguredTarget]) async throws { + throw PrepareNotSupportedError() + } + + public func generateBuildGraph() {} + + public func topologicalSort(of targets: [ConfiguredTarget]) -> [ConfiguredTarget]? { + return nil + } + func registerForChangeNotifications(for uri: DocumentURI) async { } diff --git a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift index 22b2af822..4c10bec27 100644 --- a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift +++ b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift @@ -105,6 +105,63 @@ final class BackgroundIndexingTests: XCTestCase { ) } + func testBackgroundIndexingOfMultiModuleProject() async throws { + let project = try await SwiftPMTestProject( + files: [ + "LibA/MyFile.swift": """ + public func 1️⃣foo() {} + """, + "LibB/MyOtherFile.swift": """ + import LibA + func 2️⃣bar() { + 3️⃣foo() + } + """, + ], + manifest: """ + // swift-tools-version: 5.7 + + import PackageDescription + + let package = Package( + name: "MyLibrary", + targets: [ + .target(name: "LibA"), + .target(name: "LibB", dependencies: ["LibA"]), + ] + ) + """, + serverOptions: backgroundIndexingOptions + ) + + let (uri, positions) = try project.openDocument("MyFile.swift") + let prepare = try await project.testClient.send( + CallHierarchyPrepareRequest(textDocument: TextDocumentIdentifier(uri), position: positions["1️⃣"]) + ) + let initialItem = try XCTUnwrap(prepare?.only) + let calls = try await project.testClient.send(CallHierarchyIncomingCallsRequest(item: initialItem)) + XCTAssertEqual( + calls, + [ + CallHierarchyIncomingCall( + from: CallHierarchyItem( + name: "bar()", + kind: .function, + tags: nil, + uri: try project.uri(for: "MyOtherFile.swift"), + range: Range(try project.position(of: "2️⃣", in: "MyOtherFile.swift")), + selectionRange: Range(try project.position(of: "2️⃣", in: "MyOtherFile.swift")), + data: .dictionary([ + "usr": .string("s:4LibB3baryyF"), + "uri": .string(try project.uri(for: "MyOtherFile.swift").stringValue), + ]) + ), + fromRanges: [Range(try project.position(of: "3️⃣", in: "MyOtherFile.swift"))] + ) + ] + ) + } + func testBackgroundIndexingHappensWithLowPriority() async throws { var serverOptions = backgroundIndexingOptions serverOptions.indexOptions.indexTaskDidFinish = { taskDescription in @@ -152,4 +209,77 @@ final class BackgroundIndexingTests: XCTestCase { } semaphore.wait() } + + func testBackgroundIndexingOfPackageDependency() async throws { + let dependencyContents = """ + public func 1️⃣doSomething() {} + """ + + let dependencyProject = try await SwiftPMDependencyProject(files: [ + "Sources/MyDependency/MyDependency.swift": dependencyContents + ]) + defer { dependencyProject.keepAlive() } + + let project = try await SwiftPMTestProject( + files: [ + "Test.swift": """ + import MyDependency + + func 2️⃣test() { + 3️⃣doSomething() + } + """ + ], + manifest: """ + // swift-tools-version: 5.7 + import PackageDescription + let package = Package( + name: "MyLibrary", + dependencies: [.package(url: "\(dependencyProject.packageDirectory)", from: "1.0.0")], + targets: [ + .target( + name: "MyLibrary", + dependencies: [.product(name: "MyDependency", package: "MyDependency")] + ) + ] + ) + """, + serverOptions: backgroundIndexingOptions + ) + + let dependencyUrl = try XCTUnwrap( + FileManager.default.findFiles(named: "MyDependency.swift", in: project.scratchDirectory).only + ) + let dependencyUri = DocumentURI(dependencyUrl) + let testFileUri = try project.uri(for: "Test.swift") + let positions = project.testClient.openDocument(dependencyContents, uri: dependencyUri) + let prepare = try await project.testClient.send( + CallHierarchyPrepareRequest(textDocument: TextDocumentIdentifier(dependencyUri), position: positions["1️⃣"]) + ) + + let calls = try await project.testClient.send( + CallHierarchyIncomingCallsRequest(item: try XCTUnwrap(prepare?.only)) + ) + + XCTAssertEqual( + calls, + [ + CallHierarchyIncomingCall( + from: CallHierarchyItem( + name: "test()", + kind: .function, + tags: nil, + uri: testFileUri, + range: try project.range(from: "2️⃣", to: "2️⃣", in: "Test.swift"), + selectionRange: try project.range(from: "2️⃣", to: "2️⃣", in: "Test.swift"), + data: .dictionary([ + "usr": .string("s:9MyLibrary4testyyF"), + "uri": .string(testFileUri.stringValue), + ]) + ), + fromRanges: [try project.range(from: "3️⃣", to: "3️⃣", in: "Test.swift")] + ) + ] + ) + } } diff --git a/Tests/SourceKitLSPTests/BuildSystemTests.swift b/Tests/SourceKitLSPTests/BuildSystemTests.swift index 252abea30..0116b962f 100644 --- a/Tests/SourceKitLSPTests/BuildSystemTests.swift +++ b/Tests/SourceKitLSPTests/BuildSystemTests.swift @@ -55,6 +55,16 @@ final class TestBuildSystem: BuildSystem { return [ConfiguredTarget(targetID: "dummy", runDestinationID: "dummy")] } + public func prepare(targets: [ConfiguredTarget]) async throws { + throw PrepareNotSupportedError() + } + + public func generateBuildGraph() {} + + public func topologicalSort(of targets: [ConfiguredTarget]) -> [ConfiguredTarget]? { + return nil + } + func registerForChangeNotifications(for uri: DocumentURI) async { watchedFiles.insert(uri) } From 13176b012d2c8905527cf66ebc6c93925c2b45a4 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Fri, 10 May 2024 12:56:37 -0700 Subject: [PATCH 27/46] Address review comments --- Sources/SKCore/BuildSystemManager.swift | 4 +- .../SwiftPMBuildSystem.swift | 50 ++++++++++--------- .../SKTestSupport/FileManager+findFiles.swift | 2 +- .../PreparationTaskDescription.swift | 11 ++-- .../SemanticIndex/SemanticIndexManager.swift | 8 ++- .../BackgroundIndexingTests.swift | 2 + 6 files changed, 45 insertions(+), 32 deletions(-) diff --git a/Sources/SKCore/BuildSystemManager.swift b/Sources/SKCore/BuildSystemManager.swift index 49a23aa42..8c6c49d30 100644 --- a/Sources/SKCore/BuildSystemManager.swift +++ b/Sources/SKCore/BuildSystemManager.swift @@ -212,8 +212,8 @@ extension BuildSystemManager { try await self.buildSystem?.generateBuildGraph() } - public func topologicalSort(of targets: [ConfiguredTarget]) async throws -> [ConfiguredTarget] { - return await buildSystem?.topologicalSort(of: targets) ?? targets + public func topologicalSort(of targets: [ConfiguredTarget]) async throws -> [ConfiguredTarget]? { + return await buildSystem?.topologicalSort(of: targets) } public func prepare(targets: [ConfiguredTarget]) async throws { diff --git a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift index 06f7c0138..5c9479e56 100644 --- a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift +++ b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift @@ -100,7 +100,11 @@ public actor SwiftPMBuildSystem { var fileToTarget: [AbsolutePath: SwiftBuildTarget] = [:] var sourceDirToTarget: [AbsolutePath: SwiftBuildTarget] = [:] - var targets: [SwiftBuildTarget] = [] + + /// Maps target ids (aka. `ConfiguredTarget.targetID`) to their SwiftPM build target as well as an index in their + /// topological sorting. Targets with lower index are more low level, ie. targets with higher indices depend on + /// targets with lower indices. + var targets: [String: (index: Int, buildTarget: SwiftBuildTarget)] = [:] /// The URIs for which the delegate has registered for change notifications, /// mapped to the language the delegate specified when registering for change notifications. @@ -119,6 +123,11 @@ public actor SwiftPMBuildSystem { /// Force-unwrapped optional because initializing it requires access to `self`. var fileDependenciesUpdatedDebouncer: Debouncer>! = nil + /// A `ObservabilitySystem` from `SwiftPM` that logs. + private let observabilitySystem = ObservabilitySystem({ scope, diagnostic in + logger.log(level: diagnostic.severity.asLogLevel, "SwiftPM log: \(diagnostic.description)") + }) + /// Creates a build system using the Swift Package Manager, if this workspace is a package. /// /// - Parameters: @@ -238,9 +247,6 @@ public actor SwiftPMBuildSystem { extension SwiftPMBuildSystem { public func generateBuildGraph() async throws { - let observabilitySystem = ObservabilitySystem({ scope, diagnostic in - logger.log(level: diagnostic.severity.asLogLevel, "SwiftPM log: \(diagnostic.description)") - }) try self.workspace.resolve( root: PackageGraphRootInput(packages: [AbsolutePath(projectRoot)]), observabilityScope: observabilitySystem.topScope @@ -258,10 +264,6 @@ extension SwiftPMBuildSystem { } } - let observabilitySystem = ObservabilitySystem({ scope, diagnostic in - logger.log(level: diagnostic.severity.asLogLevel, "SwiftPM log: \(diagnostic.description)") - }) - let modulesGraph = try self.workspace.loadPackageGraph( rootInput: PackageGraphRootInput(packages: [AbsolutePath(projectRoot)]), forceResolvedVersions: true, @@ -282,7 +284,15 @@ extension SwiftPMBuildSystem { /// with only some properties modified. self.modulesGraph = modulesGraph - self.targets = try buildDescription.allTargetsInTopologicalOrder(in: modulesGraph) + self.targets = Dictionary( + try buildDescription.allTargetsInTopologicalOrder(in: modulesGraph).enumerated().map { (index, target) in + return (key: target.name, (index, target)) + }, + uniquingKeysWith: { first, second in + logger.fault("Found two targets with the same name \(first.buildTarget.name)") + return second + } + ) self.fileToTarget = [AbsolutePath: SwiftBuildTarget]( modulesGraph.allTargets.flatMap { target in @@ -353,14 +363,8 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { return try settings(forPackageManifest: path) } - let buildTargets = self.targets.filter({ $0.name == configuredTarget.targetID }) - if buildTargets.count > 1 { - logger.error("Found multiple targets with name \(configuredTarget.targetID). Picking the first one") - } - guard let buildTarget = buildTargets.first else { - if buildTargets.isEmpty { - logger.error("Did not find target with name \(configuredTarget.targetID)") - } + guard let buildTarget = self.targets[configuredTarget.targetID]?.buildTarget else { + logger.error("Did not find target with name \(configuredTarget.targetID)") return nil } @@ -408,8 +412,8 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { public func topologicalSort(of targets: [ConfiguredTarget]) -> [ConfiguredTarget]? { return targets.sorted { (lhs: ConfiguredTarget, rhs: ConfiguredTarget) -> Bool in - let lhsIndex = self.targets.firstIndex(where: { $0.name == lhs.targetID }) ?? self.targets.count - let rhsIndex = self.targets.firstIndex(where: { $0.name == rhs.targetID }) ?? self.targets.count + let lhsIndex = self.targets[lhs.targetID]?.index ?? self.targets.count + let rhsIndex = self.targets[lhs.targetID]?.index ?? self.targets.count return lhsIndex < rhsIndex } } @@ -443,7 +447,7 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { ] let process = Process( arguments: arguments, - workingDirectory: try TSCBasic.AbsolutePath(validating: workspacePath.pathString) + workingDirectory: workspacePath ) try process.launch() let result = try await process.waitUntilExitSendingSigIntOnTaskCancellation() @@ -456,7 +460,7 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { let stderr = (try? String(bytes: result.stderrOutput.get(), encoding: .utf8)) ?? "" logger.debug( """ - Preparation of targets \(target.targetID) terminated with non-zero exit code \(code) + Preparation of target \(target.targetID) terminated with non-zero exit code \(code) Stderr: \(stderr) Stdout: @@ -468,11 +472,11 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { // The indexing job finished with a signal. Could be because the compiler crashed. // Ignore signal exit codes if this task has been cancelled because the compiler exits with SIGINT if it gets // interrupted. - logger.error("Preparation of targets \(target.targetID) signaled \(signal)") + logger.error("Preparation of target \(target.targetID) signaled \(signal)") } case .abnormal(exception: let exception): if !Task.isCancelled { - logger.error("Preparation of targets \(target.targetID) exited abnormally \(exception)") + logger.error("Preparation of target \(target.targetID) exited abnormally \(exception)") } } } diff --git a/Sources/SKTestSupport/FileManager+findFiles.swift b/Sources/SKTestSupport/FileManager+findFiles.swift index 97bd1befc..b4bf60d12 100644 --- a/Sources/SKTestSupport/FileManager+findFiles.swift +++ b/Sources/SKTestSupport/FileManager+findFiles.swift @@ -25,7 +25,7 @@ extension FileManager { return result } - /// Returns the URLs of all files with the given file extension in the given directory (recursively). + /// Returns the URLs of all files with the given file name in the given directory (recursively). public func findFiles(named name: String, in directory: URL) -> [URL] { var result: [URL] = [] let enumerator = self.enumerator(at: directory, includingPropertiesForKeys: nil) diff --git a/Sources/SemanticIndex/PreparationTaskDescription.swift b/Sources/SemanticIndex/PreparationTaskDescription.swift index d674dd5b6..d2348b272 100644 --- a/Sources/SemanticIndex/PreparationTaskDescription.swift +++ b/Sources/SemanticIndex/PreparationTaskDescription.swift @@ -21,13 +21,13 @@ import class TSCBasic.Process private var preparationIDForLogging = AtomicUInt32(initialValue: 1) -/// Describes a task to index a set of source files. +/// Describes a task to prepare a set of targets. /// /// This task description can be scheduled in a `TaskScheduler`. public struct PreparationTaskDescription: TaskDescriptionProtocol { public let id = preparationIDForLogging.fetchAndIncrement() - /// The files that should be indexed. + /// The targets that should be prepared. private let targetsToPrepare: [ConfiguredTarget] /// The build system manager that is used to get the toolchain and build settings for the files to index. @@ -38,7 +38,7 @@ public struct PreparationTaskDescription: TaskDescriptionProtocol { /// Intended for testing purposes. private let didFinishCallback: @Sendable (PreparationTaskDescription) -> Void - /// The task is idempotent because indexing the same file twice produces the same result as indexing it once. + /// The task is idempotent because preparing the same target twice produces the same result as preparing it once. public var isIdempotent: Bool { true } public var estimatedCPUCoreCount: Int { 1 } @@ -73,7 +73,10 @@ public struct PreparationTaskDescription: TaskDescriptionProtocol { let targetsToPrepare = targetsToPrepare.sorted(by: { ($0.targetID, $0.runDestinationID) < ($1.targetID, $1.runDestinationID) }) - let targetsToPrepareDescription = targetsToPrepare.map { $0.targetID }.joined(separator: ", ") + let targetsToPrepareDescription = + targetsToPrepare + .map { "\($0.targetID)-\($0.runDestinationID)" } + .joined(separator: ", ") logger.log( "Starting preparation with priority \(Task.currentPriority.rawValue, privacy: .public): \(targetsToPrepareDescription)" ) diff --git a/Sources/SemanticIndex/SemanticIndexManager.swift b/Sources/SemanticIndex/SemanticIndexManager.swift index b326c4d75..4b6ed22a8 100644 --- a/Sources/SemanticIndex/SemanticIndexManager.swift +++ b/Sources/SemanticIndex/SemanticIndexManager.swift @@ -191,7 +191,9 @@ public final actor SemanticIndexManager { var sortedTargets: [ConfiguredTarget] = await orLog("Sorting targets") { try await buildSystemManager.topologicalSort(of: Array(filesByTarget.keys)) } - ?? Array(filesByTarget.keys).sorted(by: { $0.targetID < $1.targetID }) + ?? Array(filesByTarget.keys).sorted(by: { + ($0.targetID, $0.runDestinationID) < ($1.targetID, $1.runDestinationID) + }) if Set(sortedTargets) != Set(filesByTarget.keys) { logger.fault( @@ -200,7 +202,9 @@ public final actor SemanticIndexManager { \(sortedTargets.map(\.targetID).joined(separator: ", ")) != \(filesByTarget.keys.map(\.targetID).joined(separator: ", ")) """ ) - sortedTargets = Array(filesByTarget.keys).sorted(by: { $0.targetID < $1.targetID }) + sortedTargets = Array(filesByTarget.keys).sorted(by: { + ($0.targetID, $0.runDestinationID) < ($1.targetID, $1.runDestinationID) + }) } var indexTasks: [Task] = [] diff --git a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift index 4c10bec27..a0ee73ff1 100644 --- a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift +++ b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift @@ -106,6 +106,7 @@ final class BackgroundIndexingTests: XCTestCase { } func testBackgroundIndexingOfMultiModuleProject() async throws { + try await SkipUnless.swiftpmStoresModulesInSubdirectory() let project = try await SwiftPMTestProject( files: [ "LibA/MyFile.swift": """ @@ -211,6 +212,7 @@ final class BackgroundIndexingTests: XCTestCase { } func testBackgroundIndexingOfPackageDependency() async throws { + try await SkipUnless.swiftpmStoresModulesInSubdirectory() let dependencyContents = """ public func 1️⃣doSomething() {} """ From 740262cb24ca7663ec9adecc28f1706170ea69dd Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Fri, 10 May 2024 14:42:22 -0700 Subject: [PATCH 28/46] =?UTF-8?q?Don=E2=80=99t=20set=20`forceResolvedVersi?= =?UTF-8?q?ons`=20to=20`true`=20when=20we=20have=20an=20index=20build=20di?= =?UTF-8?q?rectory?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit When `SwiftPMBuildSystem` operates on a ` .index-build` directory, it owns the checkouts and is thus also allowed to resolve the package versions. This is necessary --- .../SwiftPMBuildSystem.swift | 18 ++++--- Sources/SourceKitLSP/Workspace.swift | 3 ++ .../SwiftPMBuildSystemTests.swift | 48 ++++++++++++------- 3 files changed, 47 insertions(+), 22 deletions(-) diff --git a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift index 5c9479e56..13f7457d6 100644 --- a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift +++ b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift @@ -128,6 +128,13 @@ public actor SwiftPMBuildSystem { logger.log(level: diagnostic.severity.asLogLevel, "SwiftPM log: \(diagnostic.description)") }) + /// Whether the SwiftPMBuildSystem may modify `Package.resolved` or not. + /// + /// This is `false` if the `SwiftPMBuildSystem` is pointed at a `.index-build` directory that's independent of the + /// user's build. In this case `SwiftPMBuildSystem` is allowed to clone repositories even if no `Package.resolved` + /// exists. + private let forceResolvedVersions: Bool + /// Creates a build system using the Swift Package Manager, if this workspace is a package. /// /// - Parameters: @@ -141,11 +148,13 @@ public actor SwiftPMBuildSystem { toolchainRegistry: ToolchainRegistry, fileSystem: FileSystem = localFileSystem, buildSetup: BuildSetup, + forceResolvedVersions: Bool, reloadPackageStatusCallback: @escaping (ReloadPackageStatus) async -> Void = { _ in } ) async throws { self.workspacePath = workspacePath self.fileSystem = fileSystem self.toolchainRegistry = toolchainRegistry + self.forceResolvedVersions = forceResolvedVersions guard let packageRoot = findPackageDirectory(containing: workspacePath, fileSystem) else { throw Error.noManifest(workspacePath: workspacePath) @@ -213,7 +222,6 @@ public actor SwiftPMBuildSystem { } await delegate.filesDependenciesUpdated(filesWithUpdatedDependencies) } - try await reloadPackage() } @@ -226,6 +234,7 @@ public actor SwiftPMBuildSystem { url: URL, toolchainRegistry: ToolchainRegistry, buildSetup: BuildSetup, + forceResolvedVersions: Bool, reloadPackageStatusCallback: @escaping (ReloadPackageStatus) async -> Void ) async { do { @@ -234,6 +243,7 @@ public actor SwiftPMBuildSystem { toolchainRegistry: toolchainRegistry, fileSystem: localFileSystem, buildSetup: buildSetup, + forceResolvedVersions: forceResolvedVersions, reloadPackageStatusCallback: reloadPackageStatusCallback ) } catch Error.noManifest { @@ -247,10 +257,6 @@ public actor SwiftPMBuildSystem { extension SwiftPMBuildSystem { public func generateBuildGraph() async throws { - try self.workspace.resolve( - root: PackageGraphRootInput(packages: [AbsolutePath(projectRoot)]), - observabilityScope: observabilitySystem.topScope - ) try await self.reloadPackage() } @@ -266,7 +272,7 @@ extension SwiftPMBuildSystem { let modulesGraph = try self.workspace.loadPackageGraph( rootInput: PackageGraphRootInput(packages: [AbsolutePath(projectRoot)]), - forceResolvedVersions: true, + forceResolvedVersions: forceResolvedVersions, observabilityScope: observabilitySystem.topScope ) diff --git a/Sources/SourceKitLSP/Workspace.swift b/Sources/SourceKitLSP/Workspace.swift index 831b06775..565d53fea 100644 --- a/Sources/SourceKitLSP/Workspace.swift +++ b/Sources/SourceKitLSP/Workspace.swift @@ -149,14 +149,17 @@ public final class Workspace { if let rootUrl = rootUri.fileURL, let rootPath = try? AbsolutePath(validating: rootUrl.path) { var options = options + var forceResolvedVersions = true if options.indexOptions.enableBackgroundIndexing, options.buildSetup.path == nil { options.buildSetup.path = rootPath.appending(component: ".index-build") + forceResolvedVersions = false } func createSwiftPMBuildSystem(rootUrl: URL) async -> SwiftPMBuildSystem? { return await SwiftPMBuildSystem( url: rootUrl, toolchainRegistry: toolchainRegistry, buildSetup: options.buildSetup, + forceResolvedVersions: forceResolvedVersions, reloadPackageStatusCallback: reloadPackageStatusCallback ) } diff --git a/Tests/SKSwiftPMWorkspaceTests/SwiftPMBuildSystemTests.swift b/Tests/SKSwiftPMWorkspaceTests/SwiftPMBuildSystemTests.swift index 900915f7e..2ad593636 100644 --- a/Tests/SKSwiftPMWorkspaceTests/SwiftPMBuildSystemTests.swift +++ b/Tests/SKSwiftPMWorkspaceTests/SwiftPMBuildSystemTests.swift @@ -53,7 +53,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) ) } @@ -80,7 +81,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) ) } @@ -107,7 +109,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: ToolchainRegistry(toolchains: []), fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) ) } @@ -134,7 +137,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let aswift = packageRoot.appending(components: "Sources", "lib", "a.swift") @@ -202,7 +206,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: config + buildSetup: config, + forceResolvedVersions: true ) let aswift = packageRoot.appending(components: "Sources", "lib", "a.swift") @@ -239,7 +244,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let source = try resolveSymlinks(packageRoot.appending(component: "Package.swift")) @@ -272,7 +278,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let aswift = packageRoot.appending(components: "Sources", "lib", "a.swift") @@ -317,7 +324,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let aswift = packageRoot.appending(components: "Sources", "libA", "a.swift") @@ -379,7 +387,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let aswift = packageRoot.appending(components: "Sources", "libA", "a.swift") @@ -419,7 +428,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let acxx = packageRoot.appending(components: "Sources", "lib", "a.cpp") @@ -498,7 +508,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: ToolchainRegistry.forTesting, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let aswift = packageRoot.appending(components: "Sources", "lib", "a.swift") @@ -545,7 +556,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let aswift1 = packageRoot.appending(components: "Sources", "lib", "a.swift") @@ -609,7 +621,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: symlinkRoot, toolchainRegistry: ToolchainRegistry.forTesting, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) for file in [acpp, ah] { @@ -649,7 +662,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let aswift = packageRoot.appending(components: "Sources", "lib", "a.swift") @@ -685,7 +699,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) assertEqual(await swiftpmBuildSystem.projectRoot, try resolveSymlinks(tempDir.appending(component: "pkg"))) @@ -721,7 +736,8 @@ final class SwiftPMBuildSystemTests: XCTestCase { workspacePath: packageRoot, toolchainRegistry: tr, fileSystem: fs, - buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup + buildSetup: SourceKitLSPServer.Options.testDefault.buildSetup, + forceResolvedVersions: true ) let aswift = packageRoot.appending(components: "Plugins", "MyPlugin", "a.swift") From 70e373c1fa3ba3ca313b5aa881daf418ae601c48 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Sun, 12 May 2024 22:44:20 -0700 Subject: [PATCH 29/46] Make all non-test modules except for SourceKitLSP build in Swift 6 mode This would have caught a race condition in background indexing that was caused by accessing `CheckedIndex` from multiple threads despite it not being thread-safe. --- Sources/SKCore/FallbackBuildSystem.swift | 2 +- Sources/SKCore/Toolchain.swift | 2 +- ...LaunchWithWorkingDirectoryIfPossible.swift | 4 --- Sources/SemanticIndex/CheckedIndex.swift | 14 +++++++-- .../SemanticIndex/SemanticIndexManager.swift | 2 +- .../UpdateIndexStoreTaskDescription.swift | 8 ++--- Sources/SourceKitD/SKDResponse.swift | 2 +- .../SKCoreTests/ToolchainRegistryTests.swift | 30 ------------------- 8 files changed, 19 insertions(+), 45 deletions(-) diff --git a/Sources/SKCore/FallbackBuildSystem.swift b/Sources/SKCore/FallbackBuildSystem.swift index 0c6fc167a..92c154582 100644 --- a/Sources/SKCore/FallbackBuildSystem.swift +++ b/Sources/SKCore/FallbackBuildSystem.swift @@ -16,7 +16,7 @@ import Foundation import LanguageServerProtocol import SKSupport -@preconcurrency import enum PackageLoading.Platform +import enum PackageLoading.Platform import struct TSCBasic.AbsolutePath import class TSCBasic.Process diff --git a/Sources/SKCore/Toolchain.swift b/Sources/SKCore/Toolchain.swift index 746086e39..fcb369620 100644 --- a/Sources/SKCore/Toolchain.swift +++ b/Sources/SKCore/Toolchain.swift @@ -14,7 +14,7 @@ import LSPLogging import LanguageServerProtocol import SKSupport -@preconcurrency import enum PackageLoading.Platform +import enum PackageLoading.Platform import struct TSCBasic.AbsolutePath import protocol TSCBasic.FileSystem import var TSCBasic.localFileSystem diff --git a/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift b/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift index 732331969..871fa1b57 100644 --- a/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift +++ b/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift @@ -26,7 +26,6 @@ extension Process { arguments: [String], environmentBlock: ProcessEnvironmentBlock = ProcessEnv.block, workingDirectory: AbsolutePath?, - outputRedirection: OutputRedirection = .collect, startNewProcessGroup: Bool = true, loggingHandler: LoggingHandler? = .none ) throws -> Process { @@ -36,7 +35,6 @@ extension Process { arguments: arguments, environmentBlock: environmentBlock, workingDirectory: workingDirectory, - outputRedirection: outputRedirection, startNewProcessGroup: startNewProcessGroup, loggingHandler: loggingHandler ) @@ -44,7 +42,6 @@ extension Process { self.init( arguments: arguments, environmentBlock: environmentBlock, - outputRedirection: outputRedirection, startNewProcessGroup: startNewProcessGroup, loggingHandler: loggingHandler ) @@ -60,7 +57,6 @@ extension Process { arguments: arguments, environmentBlock: environmentBlock, workingDirectory: nil, - outputRedirection: outputRedirection, startNewProcessGroup: startNewProcessGroup, loggingHandler: loggingHandler ) diff --git a/Sources/SemanticIndex/CheckedIndex.swift b/Sources/SemanticIndex/CheckedIndex.swift index 9df02bea5..d6a6da1ae 100644 --- a/Sources/SemanticIndex/CheckedIndex.swift +++ b/Sources/SemanticIndex/CheckedIndex.swift @@ -11,7 +11,7 @@ //===----------------------------------------------------------------------===// import Foundation -import IndexStoreDB +@preconcurrency import IndexStoreDB import LSPLogging import LanguageServerProtocol @@ -47,7 +47,7 @@ public enum IndexCheckLevel { /// `IndexCheckLevel`. /// /// - SeeAlso: Comment on `IndexOutOfDateChecker` -public final class CheckedIndex: Sendable { +public final class CheckedIndex { private var checker: IndexOutOfDateChecker private let index: IndexStoreDB @@ -56,6 +56,10 @@ public final class CheckedIndex: Sendable { self.checker = IndexOutOfDateChecker(checkLevel: checkLevel) } + public var unchecked: UncheckedIndex { + return UncheckedIndex(index) + } + @discardableResult public func forEachSymbolOccurrence( byUSR usr: String, @@ -146,7 +150,7 @@ public final class CheckedIndex: Sendable { /// access of the underlying `IndexStoreDB`. This makes sure that accesses to the raw `IndexStoreDB` are explicit (by /// calling `underlyingIndexStoreDB`) and we don't accidentally call into the `IndexStoreDB` when we wanted a /// `CheckedIndex`. -public struct UncheckedIndex { +public struct UncheckedIndex: Sendable { public let underlyingIndexStoreDB: IndexStoreDB public init?(_ index: IndexStoreDB?) { @@ -156,6 +160,10 @@ public struct UncheckedIndex { self.underlyingIndexStoreDB = index } + public init(_ index: IndexStoreDB) { + self.underlyingIndexStoreDB = index + } + public func checked(for checkLevel: IndexCheckLevel) -> CheckedIndex { return CheckedIndex(index: underlyingIndexStoreDB, checkLevel: checkLevel) } diff --git a/Sources/SemanticIndex/SemanticIndexManager.swift b/Sources/SemanticIndex/SemanticIndexManager.swift index 4b6ed22a8..8069818ae 100644 --- a/Sources/SemanticIndex/SemanticIndexManager.swift +++ b/Sources/SemanticIndex/SemanticIndexManager.swift @@ -153,7 +153,7 @@ public final actor SemanticIndexManager { UpdateIndexStoreTaskDescription( filesToIndex: Set(files), buildSystemManager: self.buildSystemManager, - index: self.index, + index: self.index.unchecked, didFinishCallback: { [weak self] taskDescription in self?.indexTaskDidFinish?(.updateIndexStore(taskDescription)) } diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index ac3f491af..b02676fb6 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -20,7 +20,7 @@ import SKSupport import struct TSCBasic.AbsolutePath import class TSCBasic.Process -private var updateIndexStoreIDForLogging = AtomicUInt32(initialValue: 1) +private nonisolated(unsafe) var updateIndexStoreIDForLogging = AtomicUInt32(initialValue: 1) /// Describes a task to index a set of source files. /// @@ -36,7 +36,7 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { /// A reference to the underlying index store. Used to check if the index is already up-to-date for a file, in which /// case we don't need to index it again. - private let index: CheckedIndex + private let index: UncheckedIndex /// A callback that is called when the index task finishes private let didFinishCallback: @Sendable (UpdateIndexStoreTaskDescription) -> Void @@ -57,7 +57,7 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { init( filesToIndex: Set, buildSystemManager: BuildSystemManager, - index: CheckedIndex, + index: UncheckedIndex, didFinishCallback: @escaping @Sendable (UpdateIndexStoreTaskDescription) -> Void ) { self.filesToIndex = filesToIndex @@ -122,7 +122,7 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { // The URI is not a file, so there's nothing we can index. return } - guard !index.hasUpToDateUnit(for: url) else { + guard !index.checked(for: .modifiedFiles).hasUpToDateUnit(for: url) else { // We consider a file's index up-to-date if we have any up-to-date unit. Changing build settings does not // invalidate the up-to-date status of the index. return diff --git a/Sources/SourceKitD/SKDResponse.swift b/Sources/SourceKitD/SKDResponse.swift index c3d4c08b6..6f7ec1ba2 100644 --- a/Sources/SourceKitD/SKDResponse.swift +++ b/Sources/SourceKitD/SKDResponse.swift @@ -22,7 +22,7 @@ import CRT #endif public final class SKDResponse: Sendable { - private nonisolated let response: sourcekitd_api_response_t + private nonisolated(unsafe) let response: sourcekitd_api_response_t let sourcekitd: SourceKitD /// Creates a new `SKDResponse` that exclusively manages the raw `sourcekitd_api_response_t`. diff --git a/Tests/SKCoreTests/ToolchainRegistryTests.swift b/Tests/SKCoreTests/ToolchainRegistryTests.swift index d3bd55eb7..fea574732 100644 --- a/Tests/SKCoreTests/ToolchainRegistryTests.swift +++ b/Tests/SKCoreTests/ToolchainRegistryTests.swift @@ -36,36 +36,6 @@ final class ToolchainRegistryTests: XCTestCase { await assertTrue(tr.default === tr.toolchain(identifier: "a")) } - func testDefaultDarwin() async throws { - let prevPlatform = Platform.current - defer { Platform.current = prevPlatform } - Platform.current = .darwin - - let tr = ToolchainRegistry( - toolchains: [ - Toolchain(identifier: "a", displayName: "a", path: nil), - Toolchain(identifier: ToolchainRegistry.darwinDefaultToolchainIdentifier, displayName: "a", path: nil), - ] - ) - await assertEqual(tr.default?.identifier, ToolchainRegistry.darwinDefaultToolchainIdentifier) - } - - func testUnknownPlatform() throws { - let prevPlatform = Platform.current - defer { Platform.current = prevPlatform } - Platform.current = nil - - let fs = InMemoryFileSystem() - let binPath = try AbsolutePath(validating: "/foo/bar/my_toolchain/bin") - try makeToolchain(binPath: binPath, fs, sourcekitdInProc: true) - - guard let t = Toolchain(binPath, fs) else { - XCTFail("could not find any tools") - return - } - XCTAssertNotNil(t.sourcekitd) - } - func testFindXcodeDefaultToolchain() async throws { try SkipUnless.platformIsDarwin("Finding toolchains in Xcode is only supported on macOS") let fs = InMemoryFileSystem() From 61a21347da5167d3fbb20bab23db410473f15b39 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Mon, 13 May 2024 17:51:15 -0700 Subject: [PATCH 30/46] Index clang files Fixes #1253 rdar://127474135 --- .../UpdateIndexStoreTaskDescription.swift | 117 ++++++++++++++++-- .../BackgroundIndexingTests.swift | 44 +++++++ 2 files changed, 150 insertions(+), 11 deletions(-) diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index ac3f491af..b6675ccb2 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -163,9 +163,12 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { BuildSettingsLogger.log(settings: buildSettings, for: uri) } case .c, .cpp, .objective_c, .objective_cpp: - // TODO (indexing): Support indexing of clang files, including headers. - // https://github.com/apple/sourcekit-lsp/issues/1253 - break + do { + try await updateIndexStore(forClangFile: uri, buildSettings: buildSettings, toolchain: toolchain) + } catch { + logger.error("Updating index store for \(uri) failed: \(error.forLogging)") + BuildSettingsLogger.log(settings: buildSettings, for: uri) + } default: logger.error( "Not updating index store for \(uri) because it is a language that is not supported by background indexing" @@ -190,10 +193,49 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { fileToIndex: uri ) - let process = try Process.launch( - arguments: [swiftc.pathString] + indexingArguments, + try await runIndexingProcess( + indexFile: uri, + buildSettings: buildSettings, + processArguments: [swiftc.pathString] + indexingArguments, workingDirectory: buildSettings.workingDirectory.map(AbsolutePath.init(validating:)) ) + } + + private func updateIndexStore( + forClangFile uri: DocumentURI, + buildSettings: FileBuildSettings, + toolchain: Toolchain + ) async throws { + guard let clang = toolchain.clang else { + logger.error( + "Not updating index store for \(uri.forLogging) because toolchain \(toolchain.identifier) does not contain clang" + ) + return + } + + let indexingArguments = adjustClangCompilerArgumentsForIndexStoreUpdate( + buildSettings.compilerArguments, + fileToIndex: uri + ) + + try await runIndexingProcess( + indexFile: uri, + buildSettings: buildSettings, + processArguments: [clang.pathString] + indexingArguments, + workingDirectory: buildSettings.workingDirectory.map(AbsolutePath.init(validating:)) + ) + } + + private func runIndexingProcess( + indexFile: DocumentURI, + buildSettings: FileBuildSettings, + processArguments: [String], + workingDirectory: AbsolutePath? + ) async throws { + let process = try Process.launch( + arguments: processArguments, + workingDirectory: workingDirectory + ) let result = try await process.waitUntilExitSendingSigIntOnTaskCancellation() switch result.exitStatus.exhaustivelySwitchable { case .terminated(code: 0): @@ -205,26 +247,26 @@ public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { // Indexing will frequently fail if the source code is in an invalid state. Thus, log the failure at a low level. logger.debug( """ - Updating index store for Swift file \(uri.forLogging) terminated with non-zero exit code \(code) + Updating index store for \(indexFile.forLogging) terminated with non-zero exit code \(code) Stderr: \(stderr) Stdout: \(stdout) """ ) - BuildSettingsLogger.log(level: .debug, settings: buildSettings, for: uri) + BuildSettingsLogger.log(level: .debug, settings: buildSettings, for: indexFile) case .signalled(signal: let signal): if !Task.isCancelled { // The indexing job finished with a signal. Could be because the compiler crashed. // Ignore signal exit codes if this task has been cancelled because the compiler exits with SIGINT if it gets // interrupted. - logger.error("Updating index store for Swift file \(uri.forLogging) signaled \(signal)") - BuildSettingsLogger.log(level: .error, settings: buildSettings, for: uri) + logger.error("Updating index store for \(indexFile.forLogging) signaled \(signal)") + BuildSettingsLogger.log(level: .error, settings: buildSettings, for: indexFile) } case .abnormal(exception: let exception): if !Task.isCancelled { - logger.error("Updating index store for Swift file \(uri.forLogging) exited abnormally \(exception)") - BuildSettingsLogger.log(level: .error, settings: buildSettings, for: uri) + logger.error("Updating index store for \(indexFile.forLogging) exited abnormally \(exception)") + BuildSettingsLogger.log(level: .error, settings: buildSettings, for: indexFile) } } } @@ -304,3 +346,56 @@ private func adjustSwiftCompilerArgumentsForIndexStoreUpdate( ] return result } + +/// Adjust compiler arguments that were created for building to compiler arguments that should be used for indexing. +/// +/// This removes compiler arguments that produce output files and adds arguments to index the file. +private func adjustClangCompilerArgumentsForIndexStoreUpdate( + _ compilerArguments: [String], + fileToIndex: DocumentURI +) -> [String] { + let removeFlags: Set = [ + // Disable writing of a depfile + "-M", + "-MD", + "-MMD", + "-MG", + "-MM", + "-MV", + // Don't create phony targets + "-MP", + // Don't writ out compilation databases + "-MJ", + // Continue in the presence of errors during indexing + "-fmodules-validate-once-per-build-session", + // Don't compile + "-c", + ] + + let removeArguments: Set = [ + // Disable writing of a depfile + "-MT", + "-MF", + "-MQ", + // Don't write serialized diagnostic files + "--serialize-diagnostics", + ] + + var result: [String] = [] + result.reserveCapacity(compilerArguments.count) + var iterator = compilerArguments.makeIterator() + while let argument = iterator.next() { + if removeFlags.contains(argument) || argument.starts(with: "-fbuild-session-file=") { + continue + } + if removeArguments.contains(argument) { + _ = iterator.next() + continue + } + result.append(argument) + } + result.append( + "-fsyntax-only" + ) + return result +} diff --git a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift index a0ee73ff1..a2c4c3539 100644 --- a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift +++ b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift @@ -284,4 +284,48 @@ final class BackgroundIndexingTests: XCTestCase { ] ) } + + func testIndexCFile() async throws { + let project = try await SwiftPMTestProject( + files: [ + "MyLibrary/include/dummy.h": "", + "MyFile.c": """ + void 1️⃣someFunc() {} + + void 2️⃣test() { + 3️⃣someFunc(); + } + """, + ], + serverOptions: backgroundIndexingOptions + ) + + let (uri, positions) = try project.openDocument("MyFile.c") + let prepare = try await project.testClient.send( + CallHierarchyPrepareRequest(textDocument: TextDocumentIdentifier(uri), position: positions["1️⃣"]) + ) + let calls = try await project.testClient.send( + CallHierarchyIncomingCallsRequest(item: try XCTUnwrap(prepare?.only)) + ) + XCTAssertEqual( + calls, + [ + CallHierarchyIncomingCall( + from: CallHierarchyItem( + name: "test", + kind: .function, + tags: nil, + uri: uri, + range: Range(positions["2️⃣"]), + selectionRange: Range(positions["2️⃣"]), + data: .dictionary([ + "usr": .string("c:@F@test"), + "uri": .string(uri.stringValue), + ]) + ), + fromRanges: [Range(positions["3️⃣"])] + ) + ] + ) + } } From 31f2674b80bb6425570ae84dcdeae2c3f0e116e9 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Mon, 13 May 2024 21:07:18 -0700 Subject: [PATCH 31/46] =?UTF-8?q?Don=E2=80=99t=20rely=20on=20working=20dir?= =?UTF-8?q?ectory=20during=20index=20preparation?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Amazon Linux and CentOS don’t support working directory. We can work around this by passing `--package-path` to `swift build`. rdar://128037023 --- .../Process+LaunchWithWorkingDirectoryIfPossible.swift | 2 +- Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift | 7 ++----- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift b/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift index 871fa1b57..f57f905a3 100644 --- a/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift +++ b/Sources/SKSupport/Process+LaunchWithWorkingDirectoryIfPossible.swift @@ -39,7 +39,7 @@ extension Process { loggingHandler: loggingHandler ) } else { - self.init( + Process( arguments: arguments, environmentBlock: environmentBlock, startNewProcessGroup: startNewProcessGroup, diff --git a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift index 13f7457d6..db08b2e9f 100644 --- a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift +++ b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift @@ -447,15 +447,12 @@ extension SwiftPMBuildSystem: SKCore.BuildSystem { } let arguments = [ swift.pathString, "build", + "--package-path", workspacePath.pathString, "--scratch-path", self.workspace.location.scratchDirectory.pathString, "--disable-index-store", "--target", target.targetID, ] - let process = Process( - arguments: arguments, - workingDirectory: workspacePath - ) - try process.launch() + let process = try Process.launch(arguments: arguments, workingDirectory: nil) let result = try await process.waitUntilExitSendingSigIntOnTaskCancellation() switch result.exitStatus.exhaustivelySwitchable { case .terminated(code: 0): From 7e7df04b481c816ad85f34c6011a0e9ee890946e Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Mon, 13 May 2024 21:22:09 -0700 Subject: [PATCH 32/46] Make the `SourceKitLSP` module build in Swift 6 mode MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Swift 6 mode didn’t find any notable data races. But it’s good to know Swift 6 will prevent future ones. --- Sources/LSPLogging/Logging.swift | 10 ++ Sources/LSPLogging/LoggingScope.swift | 2 +- Sources/LSPLogging/NonDarwinLogging.swift | 6 +- .../LanguageServerProtocol/Connection.swift | 4 +- Sources/LanguageServerProtocol/Message.swift | 4 +- .../Requests/CodeActionRequest.swift | 2 +- .../SupportTypes/ServerCapabilities.swift | 2 +- Sources/SKCore/BuildSystemManager.swift | 6 +- Sources/SKCore/FallbackBuildSystem.swift | 6 +- Sources/SKSupport/LineTable.swift | 5 +- Sources/SKSupport/ThreadSafeBox.swift | 6 + .../Clang/ClangLanguageService.swift | 18 ++- Sources/SourceKitLSP/DocumentManager.swift | 12 +- Sources/SourceKitLSP/LanguageService.swift | 6 +- Sources/SourceKitLSP/Rename.swift | 35 +++-- Sources/SourceKitLSP/Sequence+AsyncMap.swift | 4 +- .../SourceKitLSP/SourceKitIndexDelegate.swift | 4 +- .../SourceKitLSPServer+Options.swift | 2 +- Sources/SourceKitLSP/SourceKitLSPServer.swift | 67 +++++----- .../CodeActions/ConvertIntegerLiteral.swift | 2 +- .../ConvertJSONToCodableStruct.swift | 2 +- .../Swift/CodeCompletionSession.swift | 12 +- .../Swift/SwiftLanguageService.swift | 8 +- .../Swift/SyntaxHighlightingToken.swift | 2 +- .../Swift/SyntaxHighlightingTokens.swift | 2 +- .../SourceKitLSP/Swift/VariableTypeInfo.swift | 4 +- Sources/SourceKitLSP/TestDiscovery.swift | 5 +- Sources/SourceKitLSP/Workspace.swift | 22 +++- .../SKCoreTests/BuildSystemManagerTests.swift | 2 +- .../FallbackBuildSystemTests.swift | 121 +++++++++--------- .../SourceKitLSPTests/BuildSystemTests.swift | 6 +- 31 files changed, 232 insertions(+), 157 deletions(-) diff --git a/Sources/LSPLogging/Logging.swift b/Sources/LSPLogging/Logging.swift index 35437e71e..c8d84a287 100644 --- a/Sources/LSPLogging/Logging.swift +++ b/Sources/LSPLogging/Logging.swift @@ -28,6 +28,16 @@ public typealias LogLevel = os.OSLogType public typealias Logger = os.Logger public typealias Signposter = OSSignposter +#if compiler(<5.11) +extension OSSignposter: @unchecked Sendable {} +extension OSSignpostID: @unchecked Sendable {} +extension OSSignpostIntervalState: @unchecked Sendable {} +#else +extension OSSignposter: @retroactive @unchecked Sendable {} +extension OSSignpostID: @retroactive @unchecked Sendable {} +extension OSSignpostIntervalState: @retroactive @unchecked Sendable {} +#endif + extension os.Logger { public func makeSignposter() -> Signposter { return OSSignposter(logger: self) diff --git a/Sources/LSPLogging/LoggingScope.swift b/Sources/LSPLogging/LoggingScope.swift index 5ef40a303..1d18513b3 100644 --- a/Sources/LSPLogging/LoggingScope.swift +++ b/Sources/LSPLogging/LoggingScope.swift @@ -80,7 +80,7 @@ public func withLoggingScope( /// - SeeAlso: ``withLoggingScope(_:_:)-6qtga`` public func withLoggingScope( _ scope: String, - _ operation: () async throws -> Result + @_inheritActorContext _ operation: @Sendable () async throws -> Result ) async rethrows -> Result { return try await LoggingScope.$_scope.withValue( scope, diff --git a/Sources/LSPLogging/NonDarwinLogging.swift b/Sources/LSPLogging/NonDarwinLogging.swift index 332e3e7c9..b838a29db 100644 --- a/Sources/LSPLogging/NonDarwinLogging.swift +++ b/Sources/LSPLogging/NonDarwinLogging.swift @@ -361,14 +361,14 @@ public struct NonDarwinLogger: Sendable { // MARK: - Signposter -public struct NonDarwinSignpostID {} +public struct NonDarwinSignpostID: Sendable {} -public struct NonDarwinSignpostIntervalState {} +public struct NonDarwinSignpostIntervalState: Sendable {} /// A type that is API-compatible to `OSLogMessage` for all uses within sourcekit-lsp. /// /// Since non-Darwin platforms don't have signposts, the type just has no-op operations. -public struct NonDarwinSignposter { +public struct NonDarwinSignposter: Sendable { public func makeSignpostID() -> NonDarwinSignpostID { return NonDarwinSignpostID() } diff --git a/Sources/LanguageServerProtocol/Connection.swift b/Sources/LanguageServerProtocol/Connection.swift index 3bb27d5ad..e33a40f9a 100644 --- a/Sources/LanguageServerProtocol/Connection.swift +++ b/Sources/LanguageServerProtocol/Connection.swift @@ -44,7 +44,7 @@ public protocol MessageHandler: AnyObject, Sendable { func handle( _ request: Request, id: RequestID, - reply: @escaping (LSPResult) -> Void + reply: @Sendable @escaping (LSPResult) -> Void ) } @@ -110,7 +110,7 @@ public final class LocalConnection: Connection, @unchecked Sendable { public func send( _ request: Request, - reply: @escaping (LSPResult) -> Void + reply: @Sendable @escaping (LSPResult) -> Void ) -> RequestID { let id = nextRequestID() diff --git a/Sources/LanguageServerProtocol/Message.swift b/Sources/LanguageServerProtocol/Message.swift index 508cec8b9..97c1d5972 100644 --- a/Sources/LanguageServerProtocol/Message.swift +++ b/Sources/LanguageServerProtocol/Message.swift @@ -24,7 +24,7 @@ public protocol _RequestType: MessageType { func _handle( _ handler: MessageHandler, id: RequestID, - reply: @escaping (LSPResult, RequestID) -> Void + reply: @Sendable @escaping (LSPResult, RequestID) -> Void ) } @@ -49,7 +49,7 @@ extension RequestType { public func _handle( _ handler: MessageHandler, id: RequestID, - reply: @escaping (LSPResult, RequestID) -> Void + reply: @Sendable @escaping (LSPResult, RequestID) -> Void ) { handler.handle(self, id: id) { response in reply(response.map({ $0 as ResponseType }), id) diff --git a/Sources/LanguageServerProtocol/Requests/CodeActionRequest.swift b/Sources/LanguageServerProtocol/Requests/CodeActionRequest.swift index fe2357478..09db58055 100644 --- a/Sources/LanguageServerProtocol/Requests/CodeActionRequest.swift +++ b/Sources/LanguageServerProtocol/Requests/CodeActionRequest.swift @@ -10,7 +10,7 @@ // //===----------------------------------------------------------------------===// -public typealias CodeActionProvider = (CodeActionRequest) async throws -> [CodeAction] +public typealias CodeActionProvider = @Sendable (CodeActionRequest) async throws -> [CodeAction] /// Request for returning all possible code actions for a given text document and range. /// diff --git a/Sources/LanguageServerProtocol/SupportTypes/ServerCapabilities.swift b/Sources/LanguageServerProtocol/SupportTypes/ServerCapabilities.swift index 0ce3bbd6a..36e4e03d8 100644 --- a/Sources/LanguageServerProtocol/SupportTypes/ServerCapabilities.swift +++ b/Sources/LanguageServerProtocol/SupportTypes/ServerCapabilities.swift @@ -587,7 +587,7 @@ public struct DocumentRangeFormattingOptions: WorkDoneProgressOptions, Codable, } } -public struct FoldingRangeOptions: Codable, Hashable { +public struct FoldingRangeOptions: Codable, Hashable, Sendable { /// Currently empty in the spec. public init() {} } diff --git a/Sources/SKCore/BuildSystemManager.swift b/Sources/SKCore/BuildSystemManager.swift index 8c6c49d30..33d755c57 100644 --- a/Sources/SKCore/BuildSystemManager.swift +++ b/Sources/SKCore/BuildSystemManager.swift @@ -17,6 +17,10 @@ import LanguageServerProtocol import struct TSCBasic.AbsolutePath +#if canImport(os) +import os +#endif + /// `BuildSystem` that integrates client-side information such as main-file lookup as well as providing /// common functionality such as caching. /// @@ -169,7 +173,7 @@ extension BuildSystemManager { logger.error("Getting build settings failed: \(error.forLogging)") } - guard var settings = fallbackBuildSystem?.buildSettings(for: document, language: language) else { + guard var settings = await fallbackBuildSystem?.buildSettings(for: document, language: language) else { return nil } if buildSystem == nil { diff --git a/Sources/SKCore/FallbackBuildSystem.swift b/Sources/SKCore/FallbackBuildSystem.swift index 92c154582..30c1e73b1 100644 --- a/Sources/SKCore/FallbackBuildSystem.swift +++ b/Sources/SKCore/FallbackBuildSystem.swift @@ -21,7 +21,7 @@ import struct TSCBasic.AbsolutePath import class TSCBasic.Process /// A simple BuildSystem suitable as a fallback when accurate settings are unknown. -public final class FallbackBuildSystem { +public actor FallbackBuildSystem { let buildSetup: BuildSetup @@ -38,6 +38,10 @@ public final class FallbackBuildSystem { ) }() + @_spi(Testing) public func setSdkPath(_ newValue: AbsolutePath?) { + self.sdkpath = newValue + } + /// Delegate to handle any build system events. public weak var delegate: BuildSystemDelegate? = nil diff --git a/Sources/SKSupport/LineTable.swift b/Sources/SKSupport/LineTable.swift index 343e81f86..e13488a43 100644 --- a/Sources/SKSupport/LineTable.swift +++ b/Sources/SKSupport/LineTable.swift @@ -11,8 +11,11 @@ //===----------------------------------------------------------------------===// import LSPLogging +#if canImport(os) +import os +#endif -public struct LineTable: Hashable { +public struct LineTable: Hashable, Sendable { @usableFromInline var impl: [String.Index] diff --git a/Sources/SKSupport/ThreadSafeBox.swift b/Sources/SKSupport/ThreadSafeBox.swift index 761331079..5f56b8c66 100644 --- a/Sources/SKSupport/ThreadSafeBox.swift +++ b/Sources/SKSupport/ThreadSafeBox.swift @@ -47,6 +47,12 @@ public class ThreadSafeBox: @unchecked Sendable { _value = initialValue } + public func withLock(_ body: (inout T) -> Result) -> Result { + return lock.withLock { + return body(&_value) + } + } + /// If the value in the box is an optional, return it and reset it to `nil` /// in an atomic operation. public func takeValue() -> T where U? == T { diff --git a/Sources/SourceKitLSP/Clang/ClangLanguageService.swift b/Sources/SourceKitLSP/Clang/ClangLanguageService.swift index 7eaea0b34..6e9a63d0e 100644 --- a/Sources/SourceKitLSP/Clang/ClangLanguageService.swift +++ b/Sources/SourceKitLSP/Clang/ClangLanguageService.swift @@ -33,10 +33,13 @@ extension NSLock { } /// Gathers data from clangd's stderr pipe. When it has accumulated a full line, writes the the line to the logger. -fileprivate class ClangdStderrLogForwarder { +fileprivate actor ClangdStderrLogForwarder { + /// Queue on which all data from `clangd`’s stderr will be forwarded to `stderr`. This allows us to have a + /// nonisolated `handle` function but ensure that data gets processed in order. + private let queue = AsyncQueue() private var buffer = Data() - func handle(_ newData: Data) { + private func handleImpl(_ newData: Data) { self.buffer += newData while let newlineIndex = self.buffer.firstIndex(of: UInt8(ascii: "\n")) { // Output a separate log message for every line in clangd's stderr. @@ -50,6 +53,12 @@ fileprivate class ClangdStderrLogForwarder { buffer = buffer[buffer.index(after: newlineIndex)...] } } + + nonisolated func handle(_ newData: Data) { + queue.async { + await self.handleImpl(newData) + } + } } /// A thin wrapper over a connection to a clangd server providing build setting handling. @@ -328,7 +337,7 @@ actor ClangLanguageService: LanguageService, MessageHandler { nonisolated func handle( _ params: R, id: RequestID, - reply: @escaping (LSPResult) -> Void + reply: @Sendable @escaping (LSPResult) -> Void ) { logger.info( """ @@ -442,10 +451,11 @@ extension ClangLanguageService { } public func shutdown() async { + let clangd = clangd! await withCheckedContinuation { continuation in _ = clangd.send(ShutdownRequest()) { _ in Task { - await self.clangd.send(ExitNotification()) + clangd.send(ExitNotification()) continuation.resume() } } diff --git a/Sources/SourceKitLSP/DocumentManager.swift b/Sources/SourceKitLSP/DocumentManager.swift index 2798dd72e..243b4eb4a 100644 --- a/Sources/SourceKitLSP/DocumentManager.swift +++ b/Sources/SourceKitLSP/DocumentManager.swift @@ -23,10 +23,10 @@ import SwiftSyntax /// data structure that is stored internally by the ``DocumentManager`` is a /// ``Document``. The purpose of a ``DocumentSnapshot`` is to be able to work /// with one version of a document without having to think about it changing. -public struct DocumentSnapshot: Identifiable { +public struct DocumentSnapshot: Identifiable, Sendable { /// An ID that uniquely identifies the version of the document stored in this /// snapshot. - public struct ID: Hashable, Comparable { + public struct ID: Hashable, Comparable, Sendable { public let uri: DocumentURI public let version: Int @@ -84,16 +84,18 @@ public final class Document { } } -public final class DocumentManager: InMemoryDocumentManager { +public final class DocumentManager: InMemoryDocumentManager, Sendable { public enum Error: Swift.Error { case alreadyOpen(DocumentURI) case missingDocument(DocumentURI) } - let queue: DispatchQueue = DispatchQueue(label: "document-manager-queue") + // FIXME: (async) Migrate this to be an AsyncQueue + private let queue: DispatchQueue = DispatchQueue(label: "document-manager-queue") - var documents: [DocumentURI: Document] = [:] + // `nonisolated(unsafe)` is fine because `documents` is guarded by queue. + nonisolated(unsafe) var documents: [DocumentURI: Document] = [:] public init() {} diff --git a/Sources/SourceKitLSP/LanguageService.swift b/Sources/SourceKitLSP/LanguageService.swift index 2a1ad8497..517ce98d9 100644 --- a/Sources/SourceKitLSP/LanguageService.swift +++ b/Sources/SourceKitLSP/LanguageService.swift @@ -24,7 +24,7 @@ public enum LanguageServerState { case semanticFunctionalityDisabled } -public struct RenameLocation { +public struct RenameLocation: Sendable { /// How the identifier at a given location is being used. /// /// This is primarily used to influence how argument labels should be renamed in Swift and if a location should be @@ -64,7 +64,7 @@ public struct RenameLocation { /// /// For example, we may have a language service that provides semantic functionality for c-family using a clangd server, /// launched from a specific toolchain or from sourcekitd. -public protocol LanguageService: AnyObject { +public protocol LanguageService: AnyObject, Sendable { // MARK: - Creation @@ -90,7 +90,7 @@ public protocol LanguageService: AnyObject { /// Add a handler that is called whenever the state of the language server changes. func addStateChangeHandler( - handler: @escaping (_ oldState: LanguageServerState, _ newState: LanguageServerState) -> Void + handler: @Sendable @escaping (_ oldState: LanguageServerState, _ newState: LanguageServerState) -> Void ) async // MARK: - Text synchronization diff --git a/Sources/SourceKitLSP/Rename.swift b/Sources/SourceKitLSP/Rename.swift index 0ce10e968..ee01c3231 100644 --- a/Sources/SourceKitLSP/Rename.swift +++ b/Sources/SourceKitLSP/Rename.swift @@ -10,7 +10,7 @@ // //===----------------------------------------------------------------------===// -import IndexStoreDB +@preconcurrency import IndexStoreDB import LSPLogging import LanguageServerProtocol import SKSupport @@ -445,7 +445,7 @@ extension SwiftLanguageService { /// These names might differ. For example, an Objective-C method gets translated by the clang importer to form the Swift /// name or it could have a `SWIFT_NAME` attribute that defines the method's name in Swift. Similarly, a Swift symbol /// might specify the name by which it gets exposed to Objective-C using the `@objc` attribute. -public struct CrossLanguageName { +public struct CrossLanguageName: Sendable { /// The name of the symbol in clang languages or `nil` if the symbol is defined in Swift, doesn't have any references /// from clang languages and thus hasn't been translated. fileprivate let clangName: String? @@ -564,6 +564,22 @@ extension SourceKitLSPServer { return nil } + // FIXME: (async-workaround): Needed to work around rdar://127977642 + private func translateClangNameToSwift( + _ swiftLanguageService: SwiftLanguageService, + at symbolLocation: SymbolLocation, + in snapshot: DocumentSnapshot, + isObjectiveCSelector: Bool, + name: String + ) async throws -> String { + return try await swiftLanguageService.translateClangNameToSwift( + at: symbolLocation, + in: snapshot, + isObjectiveCSelector: isObjectiveCSelector, + name: name + ) + } + private func getCrossLanguageName( forDefinitionOccurrence definitionOccurrence: SymbolOccurrence, overrideName: String? = nil, @@ -598,7 +614,8 @@ extension SourceKitLSPServer { let swiftName: String? if let swiftReference = await getReferenceFromSwift(usr: usr, index: index, workspace: workspace) { let isObjectiveCSelector = definitionLanguage == .objective_c && definitionSymbol.kind.isMethod - swiftName = try await swiftReference.swiftLanguageService.translateClangNameToSwift( + swiftName = try await self.translateClangNameToSwift( + swiftReference.swiftLanguageService, at: swiftReference.location, in: swiftReference.snapshot, isObjectiveCSelector: isObjectiveCSelector, @@ -670,7 +687,7 @@ extension SourceKitLSPServer { guard let workspace = await workspaceForDocument(uri: uri) else { throw ResponseError.workspaceNotOpen(uri) } - guard let primaryFileLanguageService = workspace.documentService[uri] else { + guard let primaryFileLanguageService = workspace.documentService.value[uri] else { return nil } @@ -716,10 +733,10 @@ extension SourceKitLSPServer { var locationsByFile: [URL: [RenameLocation]] = [:] actor LanguageServerTypesCache { - let index: CheckedIndex + let index: UncheckedIndex var languageServerTypesCache: [URL: LanguageServerType?] = [:] - init(index: CheckedIndex) { + init(index: UncheckedIndex) { self.index = index } @@ -727,13 +744,15 @@ extension SourceKitLSPServer { if let cachedValue = languageServerTypesCache[url] { return cachedValue } - let serverType = LanguageServerType(symbolProvider: index.symbolProvider(for: url.path)) + let serverType = LanguageServerType( + symbolProvider: index.checked(for: .deletedFiles).symbolProvider(for: url.path) + ) languageServerTypesCache[url] = serverType return serverType } } - let languageServerTypesCache = LanguageServerTypesCache(index: index) + let languageServerTypesCache = LanguageServerTypesCache(index: index.unchecked) let usrsToRename = overridingAndOverriddenUsrs(of: usr, index: index) let occurrencesToRename = usrsToRename.flatMap { index.occurrences(ofUSR: $0, roles: renameRoles) } diff --git a/Sources/SourceKitLSP/Sequence+AsyncMap.swift b/Sources/SourceKitLSP/Sequence+AsyncMap.swift index 5a39dda39..7ec4e100e 100644 --- a/Sources/SourceKitLSP/Sequence+AsyncMap.swift +++ b/Sources/SourceKitLSP/Sequence+AsyncMap.swift @@ -13,7 +13,7 @@ extension Sequence { /// Just like `Sequence.map` but allows an `async` transform function. func asyncMap( - _ transform: (Element) async throws -> T + @_inheritActorContext _ transform: @Sendable (Element) async throws -> T ) async rethrows -> [T] { var result: [T] = [] result.reserveCapacity(self.underestimatedCount) @@ -27,7 +27,7 @@ extension Sequence { /// Just like `Sequence.compactMap` but allows an `async` transform function. func asyncCompactMap( - _ transform: (Element) async throws -> T? + @_inheritActorContext _ transform: @Sendable (Element) async throws -> T? ) async rethrows -> [T] { var result: [T] = [] diff --git a/Sources/SourceKitLSP/SourceKitIndexDelegate.swift b/Sources/SourceKitLSP/SourceKitIndexDelegate.swift index 2b5aeaa3d..8168534ee 100644 --- a/Sources/SourceKitLSP/SourceKitIndexDelegate.swift +++ b/Sources/SourceKitLSP/SourceKitIndexDelegate.swift @@ -24,7 +24,7 @@ public actor SourceKitIndexDelegate: IndexDelegate { let queue = AsyncQueue() /// Registered `MainFilesDelegate`s to notify when main files change. - var mainFilesChangedCallbacks: [() async -> Void] = [] + var mainFilesChangedCallbacks: [@Sendable () async -> Void] = [] /// The count of pending unit events. Whenever this transitions to 0, it represents a time where /// the index finished processing known events. Of course, that may have already changed by the @@ -72,7 +72,7 @@ public actor SourceKitIndexDelegate: IndexDelegate { } /// Register a delegate to receive notifications when main files change. - public func addMainFileChangedCallback(_ callback: @escaping () async -> Void) { + public func addMainFileChangedCallback(_ callback: @escaping @Sendable () async -> Void) { mainFilesChangedCallbacks.append(callback) } diff --git a/Sources/SourceKitLSP/SourceKitLSPServer+Options.swift b/Sources/SourceKitLSP/SourceKitLSPServer+Options.swift index 558404baa..747eaee62 100644 --- a/Sources/SourceKitLSP/SourceKitLSPServer+Options.swift +++ b/Sources/SourceKitLSP/SourceKitLSPServer+Options.swift @@ -21,7 +21,7 @@ import struct TSCBasic.RelativePath extension SourceKitLSPServer { /// Configuration options for the SourceKitServer. - public struct Options { + public struct Options: Sendable { /// Additional compiler flags (e.g. `-Xswiftc` for SwiftPM projects) and other build-related /// configuration. diff --git a/Sources/SourceKitLSP/SourceKitLSPServer.swift b/Sources/SourceKitLSP/SourceKitLSPServer.swift index 3c4f67518..410a44c71 100644 --- a/Sources/SourceKitLSP/SourceKitLSPServer.swift +++ b/Sources/SourceKitLSP/SourceKitLSPServer.swift @@ -70,26 +70,27 @@ enum LanguageServerType: Hashable { } /// A request and a callback that returns the request's reply -fileprivate final class RequestAndReply { +fileprivate final class RequestAndReply: Sendable { let params: Params - private let replyBlock: (LSPResult) -> Void + private let replyBlock: @Sendable (LSPResult) -> Void /// Whether a reply has been made. Every request must reply exactly once. - private var replied: Bool = false + /// `nonisolated(unsafe)` is fine because `replied` is atomic. + private nonisolated(unsafe) var replied: AtomicBool = AtomicBool(initialValue: false) - public init(_ request: Params, reply: @escaping (LSPResult) -> Void) { + public init(_ request: Params, reply: @escaping @Sendable (LSPResult) -> Void) { self.params = request self.replyBlock = reply } deinit { - precondition(replied, "request never received a reply") + precondition(replied.value, "request never received a reply") } /// Call the `replyBlock` with the result produced by the given closure. - func reply(_ body: () async throws -> Params.Response) async { - precondition(!replied, "replied to request more than once") - replied = true + func reply(_ body: @Sendable () async throws -> Params.Response) async { + precondition(!replied.value, "replied to request more than once") + replied.value = true do { replyBlock(.success(try await body())) } catch { @@ -631,7 +632,7 @@ public actor SourceKitLSPServer { // This should be created as soon as we receive an open call, even if the document // isn't yet ready. - guard let languageService = workspace.documentService[doc] else { + guard let languageService = workspace.documentService.value[doc] else { return } @@ -640,7 +641,9 @@ public actor SourceKitLSPServer { private func handleRequest( for request: RequestAndReply, - requestHandler: @escaping (RequestType, Workspace, LanguageService) async throws -> + requestHandler: @Sendable @escaping ( + RequestType, Workspace, LanguageService + ) async throws -> RequestType.Response ) async { await request.reply { @@ -649,7 +652,7 @@ public actor SourceKitLSPServer { guard let workspace = await self.workspaceForDocument(uri: request.textDocument.uri) else { throw ResponseError.workspaceNotOpen(request.textDocument.uri) } - guard let languageService = workspace.documentService[doc] else { + guard let languageService = workspace.documentService.value[doc] else { throw ResponseError.unknown("No language service for '\(request.textDocument.uri)' found") } return try await requestHandler(request, workspace, languageService) @@ -673,7 +676,7 @@ public actor SourceKitLSPServer { guard let workspace = await self.workspaceForDocument(uri: documentUri) else { continue } - guard workspace.documentService[documentUri] === languageService else { + guard workspace.documentService.value[documentUri] === languageService else { continue } guard let snapshot = try? self.documentManager.latestSnapshot(documentUri) else { @@ -799,7 +802,7 @@ public actor SourceKitLSPServer { _ language: Language, in workspace: Workspace ) async -> LanguageService? { - if let service = workspace.documentService[uri] { + if let service = workspace.documentService.value[uri] { return service } @@ -811,21 +814,24 @@ public actor SourceKitLSPServer { logger.log("Using toolchain \(toolchain.displayName) (\(toolchain.identifier)) for \(uri.forLogging)") - if let concurrentlySetService = workspace.documentService[uri] { - // Since we await the construction of `service`, another call to this - // function might have happened and raced us, setting - // `workspace.documentServices[uri]`. If this is the case, return the - // existing value and discard the service that we just retrieved. - return concurrentlySetService + return workspace.documentService.withLock { documentService in + if let concurrentlySetService = documentService[uri] { + // Since we await the construction of `service`, another call to this + // function might have happened and raced us, setting + // `workspace.documentServices[uri]`. If this is the case, return the + // existing value and discard the service that we just retrieved. + return concurrentlySetService + } + documentService[uri] = service + return service } - workspace.documentService[uri] = service - return service } } // MARK: - MessageHandler -private var notificationIDForLogging = AtomicUInt32(initialValue: 1) +// nonisolated(unsafe) is fine because `notificationIDForLogging` is atomic. +private nonisolated(unsafe) var notificationIDForLogging = AtomicUInt32(initialValue: 1) extension SourceKitLSPServer: MessageHandler { public nonisolated func handle(_ params: some NotificationType) { @@ -886,7 +892,7 @@ extension SourceKitLSPServer: MessageHandler { public nonisolated func handle( _ params: R, id: RequestID, - reply: @escaping (LSPResult) -> Void + reply: @Sendable @escaping (LSPResult) -> Void ) { let signposter = Logger(subsystem: LoggingScope.subsystem, category: "request-\(id)").makeSignposter() let signpostID = signposter.makeSignpostID() @@ -918,7 +924,7 @@ extension SourceKitLSPServer: MessageHandler { private func handleImpl( _ params: R, id: RequestID, - reply: @escaping (LSPResult) -> Void + reply: @Sendable @escaping (LSPResult) -> Void ) async { let startDate = Date() @@ -1059,7 +1065,7 @@ extension SourceKitLSPServer: BuildSystemDelegate { continue } - guard let service = await self.workspaceForDocument(uri: uri)?.documentService[uri] else { + guard let service = await self.workspaceForDocument(uri: uri)?.documentService.value[uri] else { continue } @@ -1083,7 +1089,7 @@ extension SourceKitLSPServer: BuildSystemDelegate { } for uri in self.affectedOpenDocumentsForChangeSet(changedFilesForWorkspace, self.documentManager) { logger.log("Dependencies updated for opened file \(uri.forLogging)") - if let service = workspace.documentService[uri] { + if let service = workspace.documentService.value[uri] { await service.documentDependenciesUpdated(uri) } } @@ -1430,8 +1436,7 @@ extension SourceKitLSPServer { // the client to open new workspaces. for workspace in self.workspaces { await workspace.buildSystemManager.setMainFilesProvider(nil) - // Close the index, which will flush to disk. - workspace.uncheckedIndex = nil + workspace.closeIndex() // Break retain cycle with the BSM. await workspace.buildSystemManager.setDelegate(nil) @@ -1523,7 +1528,7 @@ extension SourceKitLSPServer { await workspace.buildSystemManager.unregisterForChangeNotifications(for: uri) - await workspace.documentService[uri]?.closeDocument(note) + await workspace.documentService.value[uri]?.closeDocument(note) } func changeDocument(_ notification: DidChangeTextDocumentNotification) async { @@ -1538,7 +1543,7 @@ extension SourceKitLSPServer { // If the document is ready, we can handle the change right now. documentManager.edit(notification) - await workspace.documentService[uri]?.changeDocument(notification) + await workspace.documentService.value[uri]?.changeDocument(notification) } func willSaveDocument( @@ -1788,7 +1793,7 @@ extension SourceKitLSPServer { guard let workspace = await workspaceForDocument(uri: uri) else { throw ResponseError.workspaceNotOpen(uri) } - guard let languageService = workspace.documentService[uri] else { + guard let languageService = workspace.documentService.value[uri] else { return nil } diff --git a/Sources/SourceKitLSP/Swift/CodeActions/ConvertIntegerLiteral.swift b/Sources/SourceKitLSP/Swift/CodeActions/ConvertIntegerLiteral.swift index f00d0a80c..f1253ea2c 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/ConvertIntegerLiteral.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/ConvertIntegerLiteral.swift @@ -15,7 +15,7 @@ import SwiftRefactor import SwiftSyntax extension IntegerLiteralExprSyntax.Radix { - static var allCases: [Self] = [.binary, .octal, .decimal, .hex] + static let allCases: [Self] = [.binary, .octal, .decimal, .hex] } /// Syntactic code action provider to convert integer literals between diff --git a/Sources/SourceKitLSP/Swift/CodeActions/ConvertJSONToCodableStruct.swift b/Sources/SourceKitLSP/Swift/CodeActions/ConvertJSONToCodableStruct.swift index dc1fab7a0..0db53cc0e 100644 --- a/Sources/SourceKitLSP/Swift/CodeActions/ConvertJSONToCodableStruct.swift +++ b/Sources/SourceKitLSP/Swift/CodeActions/ConvertJSONToCodableStruct.swift @@ -203,7 +203,7 @@ extension ConvertJSONToCodableStruct: SyntaxRefactoringCodeActionProvider { return nil } - static var title = "Create Codable structs from JSON" + static let title = "Create Codable structs from JSON" } /// A JSON object, which is has a set of fields, each of which has the given diff --git a/Sources/SourceKitLSP/Swift/CodeCompletionSession.swift b/Sources/SourceKitLSP/Swift/CodeCompletionSession.swift index 91c28152a..baae728bb 100644 --- a/Sources/SourceKitLSP/Swift/CodeCompletionSession.swift +++ b/Sources/SourceKitLSP/Swift/CodeCompletionSession.swift @@ -52,9 +52,9 @@ class CodeCompletionSession { /// have a global mapping from `sourcekitd` to its currently active code /// completion session. /// - /// Modification of code completion sessions should only happen on - /// `completionQueue`. - private static var completionSessions: [ObjectIdentifier: CodeCompletionSession] = [:] + /// - Important: Must only be accessed on `completionQueue`. + /// `nonisolated(unsafe)` fine because this is guarded by `completionQueue`. + private static nonisolated(unsafe) var completionSessions: [ObjectIdentifier: CodeCompletionSession] = [:] /// Gets the code completion results for the given parameters. /// @@ -180,7 +180,7 @@ class CodeCompletionSession { in snapshot: DocumentSnapshot, options: SKCompletionOptions ) async throws -> CompletionList { - logger.info("Opening code completion session: \(self, privacy: .private) filter=\(filterText)") + logger.info("Opening code completion session: \(self.description) filter=\(filterText)") guard snapshot.version == self.snapshot.version else { throw ResponseError(code: .invalidRequest, message: "open must use the original snapshot") } @@ -221,7 +221,7 @@ class CodeCompletionSession { ) async throws -> CompletionList { // FIXME: Assertion for prefix of snapshot matching what we started with. - logger.info("Updating code completion session: \(self, privacy: .private) filter=\(filterText)") + logger.info("Updating code completion session: \(self.description) filter=\(filterText)") let req = sourcekitd.dictionary([ keys.request: sourcekitd.requests.codeCompleteUpdate, keys.offset: utf8StartOffset, @@ -272,7 +272,7 @@ class CodeCompletionSession { keys.offset: utf8StartOffset, keys.name: snapshot.uri.pseudoPath, ]) - logger.info("Closing code completion session: \(self, privacy: .private)") + logger.info("Closing code completion session: \(self.description)") _ = try? await sourcekitd.send(req, fileContents: nil) self.state = .closed } diff --git a/Sources/SourceKitLSP/Swift/SwiftLanguageService.swift b/Sources/SourceKitLSP/Swift/SwiftLanguageService.swift index d522588e1..acadc4609 100644 --- a/Sources/SourceKitLSP/Swift/SwiftLanguageService.swift +++ b/Sources/SourceKitLSP/Swift/SwiftLanguageService.swift @@ -68,7 +68,7 @@ fileprivate func diagnosticsEnabled(for document: DocumentURI) -> Bool { } /// A swift compiler command derived from a `FileBuildSettingsChange`. -public struct SwiftCompileCommand: Equatable { +public struct SwiftCompileCommand: Sendable, Equatable { /// The compiler arguments, including working directory. This is required since sourcekitd only /// accepts the working directory via the compiler arguments. @@ -90,7 +90,7 @@ public struct SwiftCompileCommand: Equatable { } } -public actor SwiftLanguageService: LanguageService { +public actor SwiftLanguageService: LanguageService, Sendable { /// The ``SourceKitLSPServer`` instance that created this `ClangLanguageService`. weak var sourceKitLSPServer: SourceKitLSPServer? @@ -218,7 +218,7 @@ public actor SwiftLanguageService: LanguageService { } /// - Important: For testing only - public func setReusedNodeCallback(_ callback: ReusedNodeCallback?) async { + public func setReusedNodeCallback(_ callback: (@Sendable (_ node: Syntax) -> ())?) async { await self.syntaxTreeManager.setReusedNodeCallback(callback) } @@ -246,7 +246,7 @@ public actor SwiftLanguageService: LanguageService { } public func addStateChangeHandler( - handler: @escaping (_ oldState: LanguageServerState, _ newState: LanguageServerState) -> Void + handler: @Sendable @escaping (_ oldState: LanguageServerState, _ newState: LanguageServerState) -> Void ) { self.stateChangeHandlers.append(handler) } diff --git a/Sources/SourceKitLSP/Swift/SyntaxHighlightingToken.swift b/Sources/SourceKitLSP/Swift/SyntaxHighlightingToken.swift index f2fa24798..db2402917 100644 --- a/Sources/SourceKitLSP/Swift/SyntaxHighlightingToken.swift +++ b/Sources/SourceKitLSP/Swift/SyntaxHighlightingToken.swift @@ -15,7 +15,7 @@ import LanguageServerProtocol import SourceKitD /// A ranged token in the document used for syntax highlighting. -public struct SyntaxHighlightingToken: Hashable { +public struct SyntaxHighlightingToken: Hashable, Sendable { /// The range of the token in the document. Must be on a single line. public var range: Range { didSet { diff --git a/Sources/SourceKitLSP/Swift/SyntaxHighlightingTokens.swift b/Sources/SourceKitLSP/Swift/SyntaxHighlightingTokens.swift index d04dcbe14..4515861cf 100644 --- a/Sources/SourceKitLSP/Swift/SyntaxHighlightingTokens.swift +++ b/Sources/SourceKitLSP/Swift/SyntaxHighlightingTokens.swift @@ -15,7 +15,7 @@ import LanguageServerProtocol import SourceKitD /// A wrapper around an array of syntax highlighting tokens. -public struct SyntaxHighlightingTokens { +public struct SyntaxHighlightingTokens: Sendable { public var tokens: [SyntaxHighlightingToken] public init(tokens: [SyntaxHighlightingToken]) { diff --git a/Sources/SourceKitLSP/Swift/VariableTypeInfo.swift b/Sources/SourceKitLSP/Swift/VariableTypeInfo.swift index fd45dd0be..0ed2bcdc3 100644 --- a/Sources/SourceKitLSP/Swift/VariableTypeInfo.swift +++ b/Sources/SourceKitLSP/Swift/VariableTypeInfo.swift @@ -22,11 +22,11 @@ fileprivate extension TokenSyntax { var node = Syntax(self) LOOP: while let parent = node.parent { switch parent.kind { - case .caseItem, .closureParam: + case .switchCaseItem, .closureShorthandParameter: // case items (inside a switch) and closure parameters can’t have type // annotations. return false - case .codeBlockItem, .memberDeclListItem: + case .codeBlockItem, .memberBlockItem: // Performance optimization. If we walked the parents up to code block item, // we can’t enter a case item or closure param anymore. No need walking // the tree any further. diff --git a/Sources/SourceKitLSP/TestDiscovery.swift b/Sources/SourceKitLSP/TestDiscovery.swift index e35d0eea5..3a5509e8d 100644 --- a/Sources/SourceKitLSP/TestDiscovery.swift +++ b/Sources/SourceKitLSP/TestDiscovery.swift @@ -21,7 +21,7 @@ public enum TestStyle { public static let swiftTesting = "swift-testing" } -public struct AnnotatedTestItem { +public struct AnnotatedTestItem: Sendable { /// The test item to be annotated public var testItem: TestItem @@ -180,6 +180,7 @@ extension SourceKitLSPServer { ) } + let documentManager = self.documentManager return occurrencesByParent[nil, default: []] .sorted() .map { testItem(for: $0, documentManager: documentManager, context: []) } @@ -212,7 +213,7 @@ extension SourceKitLSPServer { } let testsFromFilesWithInMemoryState = await filesWithInMemoryState.concurrentMap { (uri) -> [AnnotatedTestItem] in - guard let languageService = workspace.documentService[uri] else { + guard let languageService = workspace.documentService.value[uri] else { return [] } return await orLog("Getting document tests for \(uri)") { diff --git a/Sources/SourceKitLSP/Workspace.swift b/Sources/SourceKitLSP/Workspace.swift index 565d53fea..04c63b9b2 100644 --- a/Sources/SourceKitLSP/Workspace.swift +++ b/Sources/SourceKitLSP/Workspace.swift @@ -46,7 +46,7 @@ fileprivate func firstNonNil( /// "initialize" request has been made. /// /// Typically a workspace is contained in a root directory. -public final class Workspace { +public final class Workspace: Sendable { /// The root directory of the workspace. public let rootUri: DocumentURI? @@ -63,7 +63,7 @@ public final class Workspace { /// The source code index, if available. /// /// Usually a checked index (retrieved using `index(checkedFor:)`) should be used instead of the unchecked index. - var uncheckedIndex: UncheckedIndex? = nil + private let uncheckedIndex: ThreadSafeBox /// The index that syntactically scans the workspace for tests. let syntacticTestIndex = SyntacticTestIndex() @@ -72,7 +72,7 @@ public final class Workspace { private let documentManager: DocumentManager /// Language service for an open document, if available. - var documentService: [DocumentURI: LanguageService] = [:] + let documentService: ThreadSafeBox<[DocumentURI: LanguageService]> = ThreadSafeBox(initialValue: [:]) /// The `SemanticIndexManager` that keeps track of whose file's index is up-to-date in the workspace and schedules /// indexing and preparation tasks for files with out-of-date index. @@ -95,7 +95,7 @@ public final class Workspace { self.buildSetup = options.buildSetup self.rootUri = rootUri self.capabilityRegistry = capabilityRegistry - self.uncheckedIndex = uncheckedIndex + self.uncheckedIndex = ThreadSafeBox(initialValue: uncheckedIndex) self.buildSystemManager = await BuildSystemManager( buildSystem: underlyingBuildSystem, fallbackBuildSystem: FallbackBuildSystem(buildSetup: buildSetup), @@ -143,7 +143,7 @@ public final class Workspace { compilationDatabaseSearchPaths: [RelativePath], indexOptions: IndexOptions = IndexOptions(), indexTaskScheduler: TaskScheduler, - reloadPackageStatusCallback: @escaping (ReloadPackageStatus) async -> Void + reloadPackageStatusCallback: @Sendable @escaping (ReloadPackageStatus) async -> Void ) async throws { var buildSystem: BuildSystem? = nil @@ -254,7 +254,15 @@ public final class Workspace { /// Returns a `CheckedIndex` that verifies that all the returned entries are up-to-date with the given /// `IndexCheckLevel`. func index(checkedFor checkLevel: IndexCheckLevel) -> CheckedIndex? { - return uncheckedIndex?.checked(for: checkLevel) + return uncheckedIndex.value?.checked(for: checkLevel) + } + + /// Write the index to disk. + /// + /// After this method is called, the workspace will no longer have an index associated with it. It should only be + /// called when SourceKit-LSP shuts down. + func closeIndex() { + uncheckedIndex.value = nil } public func filesDidChange(_ events: [FileEvent]) async { @@ -272,7 +280,7 @@ struct WeakWorkspace { } } -public struct IndexOptions { +public struct IndexOptions: Sendable { /// Override the index-store-path provided by the build system. public var indexStorePath: AbsolutePath? diff --git a/Tests/SKCoreTests/BuildSystemManagerTests.swift b/Tests/SKCoreTests/BuildSystemManagerTests.swift index 8baa896ca..bb15ad142 100644 --- a/Tests/SKCoreTests/BuildSystemManagerTests.swift +++ b/Tests/SKCoreTests/BuildSystemManagerTests.swift @@ -147,7 +147,7 @@ final class BuildSystemManagerTests: XCTestCase { ) defer { withExtendedLifetime(bsm) {} } // Keep BSM alive for callbacks. let del = await BSMDelegate(bsm) - let fallbackSettings = fallback.buildSettings(for: a, language: .swift) + let fallbackSettings = await fallback.buildSettings(for: a, language: .swift) await bsm.registerForChangeNotifications(for: a, language: .swift) assertEqual(await bsm.buildSettingsInferredFromMainFile(for: a, language: .swift), fallbackSettings) diff --git a/Tests/SKCoreTests/FallbackBuildSystemTests.swift b/Tests/SKCoreTests/FallbackBuildSystemTests.swift index a6be3920a..748171e02 100644 --- a/Tests/SKCoreTests/FallbackBuildSystemTests.swift +++ b/Tests/SKCoreTests/FallbackBuildSystemTests.swift @@ -10,8 +10,9 @@ // //===----------------------------------------------------------------------===// +import LSPTestSupport import LanguageServerProtocol -import SKCore +@_spi(Testing) import SKCore import TSCBasic import XCTest @@ -19,17 +20,17 @@ import struct PackageModel.BuildFlags final class FallbackBuildSystemTests: XCTestCase { - func testSwift() throws { + func testSwift() async throws { let sdk = try AbsolutePath(validating: "/my/sdk") let source = try AbsolutePath(validating: "/my/source.swift") let bs = FallbackBuildSystem(buildSetup: .default) - bs.sdkpath = sdk + await bs.setSdkPath(sdk) - XCTAssertNil(bs.indexStorePath) - XCTAssertNil(bs.indexDatabasePath) + assertNil(await bs.indexStorePath) + assertNil(await bs.indexDatabasePath) - let settings = bs.buildSettings(for: source.asURI, language: .swift)! + let settings = await bs.buildSettings(for: source.asURI, language: .swift)! XCTAssertNil(settings.workingDirectory) let args = settings.compilerArguments @@ -42,17 +43,17 @@ final class FallbackBuildSystemTests: XCTestCase { ] ) - bs.sdkpath = nil + await bs.setSdkPath(nil) - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .swift)?.compilerArguments, + assertEqual( + await bs.buildSettings(for: source.asURI, language: .swift)?.compilerArguments, [ source.pathString ] ) } - func testSwiftWithCustomFlags() throws { + func testSwiftWithCustomFlags() async throws { let sdk = try AbsolutePath(validating: "/my/sdk") let source = try AbsolutePath(validating: "/my/source.swift") @@ -66,9 +67,9 @@ final class FallbackBuildSystemTests: XCTestCase { ]) ) let bs = FallbackBuildSystem(buildSetup: buildSetup) - bs.sdkpath = sdk + await bs.setSdkPath(sdk) - let args = bs.buildSettings(for: source.asURI, language: .swift)?.compilerArguments + let args = await bs.buildSettings(for: source.asURI, language: .swift)?.compilerArguments XCTAssertEqual( args, [ @@ -80,10 +81,10 @@ final class FallbackBuildSystemTests: XCTestCase { ] ) - bs.sdkpath = nil + await bs.setSdkPath(nil) - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .swift)?.compilerArguments, + assertEqual( + await bs.buildSettings(for: source.asURI, language: .swift)?.compilerArguments, [ "-Xfrontend", "-debug-constraints", @@ -92,7 +93,7 @@ final class FallbackBuildSystemTests: XCTestCase { ) } - func testSwiftWithCustomSDKFlag() throws { + func testSwiftWithCustomSDKFlag() async throws { let sdk = try AbsolutePath(validating: "/my/sdk") let source = try AbsolutePath(validating: "/my/source.swift") @@ -108,10 +109,10 @@ final class FallbackBuildSystemTests: XCTestCase { ]) ) let bs = FallbackBuildSystem(buildSetup: buildSetup) - bs.sdkpath = sdk + await bs.setSdkPath(sdk) - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .swift)!.compilerArguments, + assertEqual( + await bs.buildSettings(for: source.asURI, language: .swift)!.compilerArguments, [ "-sdk", "/some/custom/sdk", @@ -121,10 +122,10 @@ final class FallbackBuildSystemTests: XCTestCase { ] ) - bs.sdkpath = nil + await bs.setSdkPath(nil) - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .swift)!.compilerArguments, + assertEqual( + await bs.buildSettings(for: source.asURI, language: .swift)!.compilerArguments, [ "-sdk", "/some/custom/sdk", @@ -135,14 +136,14 @@ final class FallbackBuildSystemTests: XCTestCase { ) } - func testCXX() throws { + func testCXX() async throws { let sdk = try AbsolutePath(validating: "/my/sdk") let source = try AbsolutePath(validating: "/my/source.cpp") let bs = FallbackBuildSystem(buildSetup: .default) - bs.sdkpath = sdk + await bs.setSdkPath(sdk) - let settings = bs.buildSettings(for: source.asURI, language: .cpp)! + let settings = await bs.buildSettings(for: source.asURI, language: .cpp)! XCTAssertNil(settings.workingDirectory) let args = settings.compilerArguments @@ -155,17 +156,17 @@ final class FallbackBuildSystemTests: XCTestCase { ] ) - bs.sdkpath = nil + await bs.setSdkPath(nil) - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, + assertEqual( + await bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, [ source.pathString ] ) } - func testCXXWithCustomFlags() throws { + func testCXXWithCustomFlags() async throws { let sdk = try AbsolutePath(validating: "/my/sdk") let source = try AbsolutePath(validating: "/my/source.cpp") @@ -178,10 +179,10 @@ final class FallbackBuildSystemTests: XCTestCase { ]) ) let bs = FallbackBuildSystem(buildSetup: buildSetup) - bs.sdkpath = sdk + await bs.setSdkPath(sdk) - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, + assertEqual( + await bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, [ "-v", "-isysroot", @@ -190,10 +191,10 @@ final class FallbackBuildSystemTests: XCTestCase { ] ) - bs.sdkpath = nil + await bs.setSdkPath(nil) - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, + assertEqual( + await bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, [ "-v", source.pathString, @@ -201,7 +202,7 @@ final class FallbackBuildSystemTests: XCTestCase { ) } - func testCXXWithCustomIsysroot() throws { + func testCXXWithCustomIsysroot() async throws { let sdk = try AbsolutePath(validating: "/my/sdk") let source = try AbsolutePath(validating: "/my/source.cpp") @@ -216,10 +217,10 @@ final class FallbackBuildSystemTests: XCTestCase { ]) ) let bs = FallbackBuildSystem(buildSetup: buildSetup) - bs.sdkpath = sdk + await bs.setSdkPath(sdk) - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, + assertEqual( + await bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, [ "-isysroot", "/my/custom/sdk", @@ -228,10 +229,10 @@ final class FallbackBuildSystemTests: XCTestCase { ] ) - bs.sdkpath = nil + await bs.setSdkPath(nil) - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, + assertEqual( + await bs.buildSettings(for: source.asURI, language: .cpp)?.compilerArguments, [ "-isysroot", "/my/custom/sdk", @@ -241,19 +242,19 @@ final class FallbackBuildSystemTests: XCTestCase { ) } - func testC() throws { + func testC() async throws { let source = try AbsolutePath(validating: "/my/source.c") let bs = FallbackBuildSystem(buildSetup: .default) - bs.sdkpath = nil - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .c)?.compilerArguments, + await bs.setSdkPath(nil) + assertEqual( + await bs.buildSettings(for: source.asURI, language: .c)?.compilerArguments, [ source.pathString ] ) } - func testCWithCustomFlags() throws { + func testCWithCustomFlags() async throws { let source = try AbsolutePath(validating: "/my/source.c") let buildSetup = BuildSetup( @@ -265,9 +266,9 @@ final class FallbackBuildSystemTests: XCTestCase { ]) ) let bs = FallbackBuildSystem(buildSetup: buildSetup) - bs.sdkpath = nil - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .c)?.compilerArguments, + await bs.setSdkPath(nil) + assertEqual( + await bs.buildSettings(for: source.asURI, language: .c)?.compilerArguments, [ "-v", source.pathString, @@ -275,33 +276,33 @@ final class FallbackBuildSystemTests: XCTestCase { ) } - func testObjC() throws { + func testObjC() async throws { let source = try AbsolutePath(validating: "/my/source.m") let bs = FallbackBuildSystem(buildSetup: .default) - bs.sdkpath = nil - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .objective_c)?.compilerArguments, + await bs.setSdkPath(nil) + assertEqual( + await bs.buildSettings(for: source.asURI, language: .objective_c)?.compilerArguments, [ source.pathString ] ) } - func testObjCXX() throws { + func testObjCXX() async throws { let source = try AbsolutePath(validating: "/my/source.mm") let bs = FallbackBuildSystem(buildSetup: .default) - bs.sdkpath = nil - XCTAssertEqual( - bs.buildSettings(for: source.asURI, language: .objective_cpp)?.compilerArguments, + await bs.setSdkPath(nil) + assertEqual( + await bs.buildSettings(for: source.asURI, language: .objective_cpp)?.compilerArguments, [ source.pathString ] ) } - func testUnknown() throws { + func testUnknown() async throws { let source = try AbsolutePath(validating: "/my/source.mm") let bs = FallbackBuildSystem(buildSetup: .default) - XCTAssertNil(bs.buildSettings(for: source.asURI, language: Language(rawValue: "unknown"))) + assertNil(await bs.buildSettings(for: source.asURI, language: Language(rawValue: "unknown"))) } } diff --git a/Tests/SourceKitLSPTests/BuildSystemTests.swift b/Tests/SourceKitLSPTests/BuildSystemTests.swift index 0116b962f..501abd0b7 100644 --- a/Tests/SourceKitLSPTests/BuildSystemTests.swift +++ b/Tests/SourceKitLSPTests/BuildSystemTests.swift @@ -188,7 +188,9 @@ final class BuildSystemTests: XCTestCase { func testSwiftDocumentUpdatedBuildSettings() async throws { let doc = DocumentURI.for(.swift) - let args = FallbackBuildSystem(buildSetup: .default).buildSettings(for: doc, language: .swift)!.compilerArguments + let args = await FallbackBuildSystem(buildSetup: .default) + .buildSettings(for: doc, language: .swift)! + .compilerArguments buildSystem.buildSettingsByFile[doc] = FileBuildSettings(compilerArguments: args) @@ -257,7 +259,7 @@ final class BuildSystemTests: XCTestCase { let doc = DocumentURI.for(.swift) // Primary settings must be different than the fallback settings. - var primarySettings = FallbackBuildSystem(buildSetup: .default).buildSettings(for: doc, language: .swift)! + var primarySettings = await FallbackBuildSystem(buildSetup: .default).buildSettings(for: doc, language: .swift)! primarySettings.isFallback = false primarySettings.compilerArguments.append("-DPRIMARY") From 59ef3f657b0706b392df3cb05da4cf7b94c99d6d Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Mon, 13 May 2024 21:43:20 -0700 Subject: [PATCH 33/46] Make the `Diagnose` module build in Swift 6 mode MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This was mostly about slapping `@MainActor` on a bunch of declarations since `diagnose` doesn’t leverage concurrency at the moment. --- Sources/Diagnose/CMakeLists.txt | 4 +++- .../CommandConfiguration+Sendable.swift | 17 +++++++++++++ .../CommandLineArgumentsReducer.swift | 3 +++ Sources/Diagnose/DiagnoseCommand.swift | 21 ++++++++++++---- Sources/Diagnose/MergeSwiftFiles.swift | 1 + Sources/Diagnose/ReduceCommand.swift | 4 +++- Sources/Diagnose/ReduceFrontendCommand.swift | 9 +++++-- .../Diagnose/ReduceSourceKitDRequest.swift | 1 + Sources/Diagnose/ReduceSwiftFrontend.swift | 1 + Sources/Diagnose/RequestInfo.swift | 4 ++-- .../Diagnose/SourceKitDRequestExecutor.swift | 7 +++--- Sources/Diagnose/SourceReducer.swift | 6 +++++ .../Diagnose/SourcekitdRequestCommand.swift | 2 +- .../StderrStreamConcurrencySafe.swift | 24 +++++++++++++++++++ Tests/DiagnoseTests/DiagnoseTests.swift | 4 +++- 15 files changed, 93 insertions(+), 15 deletions(-) create mode 100644 Sources/Diagnose/CommandConfiguration+Sendable.swift create mode 100644 Sources/Diagnose/StderrStreamConcurrencySafe.swift diff --git a/Sources/Diagnose/CMakeLists.txt b/Sources/Diagnose/CMakeLists.txt index 719f8c7ec..e6c73f3ec 100644 --- a/Sources/Diagnose/CMakeLists.txt +++ b/Sources/Diagnose/CMakeLists.txt @@ -1,4 +1,5 @@ add_library(Diagnose STATIC + CommandConfiguration+Sendable.swift CommandLineArgumentsReducer.swift DiagnoseCommand.swift MergeSwiftFiles.swift @@ -11,9 +12,10 @@ add_library(Diagnose STATIC ReproducerBundle.swift RequestInfo.swift SourceKitD+RunWithYaml.swift + SourcekitdRequestCommand.swift SourceKitDRequestExecutor.swift SourceReducer.swift - SourcekitdRequestCommand.swift + StderrStreamConcurrencySafe.swift SwiftFrontendCrashScraper.swift Toolchain+SwiftFrontend.swift) diff --git a/Sources/Diagnose/CommandConfiguration+Sendable.swift b/Sources/Diagnose/CommandConfiguration+Sendable.swift new file mode 100644 index 000000000..8277251e1 --- /dev/null +++ b/Sources/Diagnose/CommandConfiguration+Sendable.swift @@ -0,0 +1,17 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift.org open source project +// +// Copyright (c) 2014 - 2024 Apple Inc. and the Swift project authors +// Licensed under Apache License v2.0 with Runtime Library Exception +// +// See https://swift.org/LICENSE.txt for license information +// See https://swift.org/CONTRIBUTORS.txt for the list of Swift project authors +// +//===----------------------------------------------------------------------===// + +import ArgumentParser + +// If `CommandConfiguration` is not sendable, commands can't have static `configuration` properties. +// Needed until we update Swift CI to swift-argument-parser 1.3.1, which has this conformance (rdar://128042447). +extension CommandConfiguration: @unchecked @retroactive Sendable {} diff --git a/Sources/Diagnose/CommandLineArgumentsReducer.swift b/Sources/Diagnose/CommandLineArgumentsReducer.swift index 45ad000a0..d13cf22f1 100644 --- a/Sources/Diagnose/CommandLineArgumentsReducer.swift +++ b/Sources/Diagnose/CommandLineArgumentsReducer.swift @@ -16,6 +16,7 @@ import LSPLogging // MARK: - Entry point extension RequestInfo { + @MainActor func reduceCommandLineArguments( using executor: SourceKitRequestExecutor, progressUpdate: (_ progress: Double, _ message: String) -> Void @@ -49,6 +50,7 @@ fileprivate class CommandLineArgumentReducer { self.progressUpdate = progressUpdate } + @MainActor func run(initialRequestInfo: RequestInfo) async throws -> RequestInfo { var requestInfo = initialRequestInfo requestInfo = try await reduce(initialRequestInfo: requestInfo, simultaneousRemove: 10) @@ -113,6 +115,7 @@ fileprivate class CommandLineArgumentReducer { return requestInfo } + @MainActor private func tryRemoving( _ argumentsToRemove: ClosedRange, from requestInfo: RequestInfo diff --git a/Sources/Diagnose/DiagnoseCommand.swift b/Sources/Diagnose/DiagnoseCommand.swift index ca01dfdca..acc528227 100644 --- a/Sources/Diagnose/DiagnoseCommand.swift +++ b/Sources/Diagnose/DiagnoseCommand.swift @@ -16,13 +16,13 @@ import SKCore import struct TSCBasic.AbsolutePath import class TSCBasic.Process -import var TSCBasic.stderrStream import class TSCUtility.PercentProgressAnimation /// When diagnosis is started, a progress bar displayed on the terminal that shows how far the diagnose command has /// progressed. /// Can't be a member of `DiagnoseCommand` because then `DiagnoseCommand` is no longer codable, which it needs to be /// to be a `AsyncParsableCommand`. +@MainActor private var progressBar: PercentProgressAnimation? = nil /// A component of the diagnostic bundle that's collected in independent stages. @@ -35,7 +35,7 @@ fileprivate enum BundleComponent: String, CaseIterable, ExpressibleByArgument { } public struct DiagnoseCommand: AsyncParsableCommand { - public static var configuration: CommandConfiguration = CommandConfiguration( + public static let configuration: CommandConfiguration = CommandConfiguration( commandName: "diagnose", abstract: "Creates a bundle containing information that help diagnose issues with sourcekit-lsp" ) @@ -72,6 +72,7 @@ public struct DiagnoseCommand: AsyncParsableCommand { } } + @MainActor var toolchain: Toolchain? { get async throws { if let toolchainOverride { @@ -96,6 +97,7 @@ public struct DiagnoseCommand: AsyncParsableCommand { public init() {} + @MainActor private func addSourcekitdCrashReproducer(toBundle bundlePath: URL) async throws { reportProgress(.reproducingSourcekitdCrash(progress: 0), message: "Trying to reduce recent sourcekitd crashes") for (name, requestInfo) in try requestInfos() { @@ -121,6 +123,7 @@ public struct DiagnoseCommand: AsyncParsableCommand { } } + @MainActor private func addSwiftFrontendCrashReproducer(toBundle bundlePath: URL) async throws { reportProgress( .reproducingSwiftFrontendCrash(progress: 0), @@ -182,7 +185,8 @@ public struct DiagnoseCommand: AsyncParsableCommand { } /// Execute body and if it throws, log the error. - private func orPrintError(_ body: () async throws -> Void) async { + @MainActor + private func orPrintError(_ body: @MainActor () async throws -> Void) async { do { try await body() } catch { @@ -190,6 +194,7 @@ public struct DiagnoseCommand: AsyncParsableCommand { } } + @MainActor private func addOsLog(toBundle bundlePath: URL) async throws { #if os(macOS) reportProgress(.collectingLogMessages(progress: 0), message: "Collecting log messages") @@ -227,6 +232,7 @@ public struct DiagnoseCommand: AsyncParsableCommand { #endif } + @MainActor private func addCrashLogs(toBundle bundlePath: URL) throws { #if os(macOS) reportProgress(.collectingCrashReports, message: "Collecting crash reports") @@ -252,6 +258,7 @@ public struct DiagnoseCommand: AsyncParsableCommand { #endif } + @MainActor private func addSwiftVersion(toBundle bundlePath: URL) async throws { let outputFileUrl = bundlePath.appendingPathComponent("swift-versions.txt") FileManager.default.createFile(atPath: outputFileUrl.path, contents: nil) @@ -283,10 +290,12 @@ public struct DiagnoseCommand: AsyncParsableCommand { } } + @MainActor private func reportProgress(_ state: DiagnoseProgressState, message: String) { progressBar?.update(step: Int(state.progress * 100), total: 100, text: message) } + @MainActor public func run() async throws { print( """ @@ -303,7 +312,10 @@ public struct DiagnoseCommand: AsyncParsableCommand { """ ) - progressBar = PercentProgressAnimation(stream: stderrStream, header: "Diagnosing sourcekit-lsp issues") + progressBar = PercentProgressAnimation( + stream: stderrStreamConcurrencySafe, + header: "Diagnosing sourcekit-lsp issues" + ) let dateFormatter = ISO8601DateFormatter() dateFormatter.timeZone = NSTimeZone.local @@ -342,6 +354,7 @@ public struct DiagnoseCommand: AsyncParsableCommand { } + @MainActor private func reduce( requestInfo: RequestInfo, toolchain: Toolchain?, diff --git a/Sources/Diagnose/MergeSwiftFiles.swift b/Sources/Diagnose/MergeSwiftFiles.swift index 2c5a492e6..ee0c30000 100644 --- a/Sources/Diagnose/MergeSwiftFiles.swift +++ b/Sources/Diagnose/MergeSwiftFiles.swift @@ -17,6 +17,7 @@ extension RequestInfo { /// Check if the issue reproduces when merging all `.swift` input files into a single file. /// /// Returns `nil` if the issue didn't reproduce with all `.swift` files merged. + @MainActor func mergeSwiftFiles( using executor: SourceKitRequestExecutor, progressUpdate: (_ progress: Double, _ message: String) -> Void diff --git a/Sources/Diagnose/ReduceCommand.swift b/Sources/Diagnose/ReduceCommand.swift index 416e11719..51bbfe4bc 100644 --- a/Sources/Diagnose/ReduceCommand.swift +++ b/Sources/Diagnose/ReduceCommand.swift @@ -20,7 +20,7 @@ import var TSCBasic.stderrStream import class TSCUtility.PercentProgressAnimation public struct ReduceCommand: AsyncParsableCommand { - public static var configuration: CommandConfiguration = CommandConfiguration( + public static let configuration: CommandConfiguration = CommandConfiguration( commandName: "reduce", abstract: "Reduce a single sourcekitd crash", shouldDisplay: false @@ -56,6 +56,7 @@ public struct ReduceCommand: AsyncParsableCommand { private var nsPredicate: NSPredicate? { nil } #endif + @MainActor var toolchain: Toolchain? { get async throws { if let toolchainOverride { @@ -68,6 +69,7 @@ public struct ReduceCommand: AsyncParsableCommand { public init() {} + @MainActor public func run() async throws { guard let sourcekitd = try await toolchain?.sourcekitd else { throw ReductionError("Unable to find sourcekitd.framework") diff --git a/Sources/Diagnose/ReduceFrontendCommand.swift b/Sources/Diagnose/ReduceFrontendCommand.swift index f8184aa90..fe57124ae 100644 --- a/Sources/Diagnose/ReduceFrontendCommand.swift +++ b/Sources/Diagnose/ReduceFrontendCommand.swift @@ -20,7 +20,7 @@ import var TSCBasic.stderrStream import class TSCUtility.PercentProgressAnimation public struct ReduceFrontendCommand: AsyncParsableCommand { - public static var configuration: CommandConfiguration = CommandConfiguration( + public static let configuration: CommandConfiguration = CommandConfiguration( commandName: "reduce-frontend", abstract: "Reduce a single swift-frontend crash", shouldDisplay: false @@ -64,6 +64,7 @@ public struct ReduceFrontendCommand: AsyncParsableCommand { ) var frontendArgs: [String] + @MainActor var toolchain: Toolchain? { get async throws { if let toolchainOverride { @@ -76,6 +77,7 @@ public struct ReduceFrontendCommand: AsyncParsableCommand { public init() {} + @MainActor public func run() async throws { guard let sourcekitd = try await toolchain?.sourcekitd else { throw ReductionError("Unable to find sourcekitd.framework") @@ -84,7 +86,10 @@ public struct ReduceFrontendCommand: AsyncParsableCommand { throw ReductionError("Unable to find swift-frontend") } - let progressBar = PercentProgressAnimation(stream: stderrStream, header: "Reducing swift-frontend crash") + let progressBar = PercentProgressAnimation( + stream: stderrStream, + header: "Reducing swift-frontend crash" + ) let executor = OutOfProcessSourceKitRequestExecutor( sourcekitd: sourcekitd.asURL, diff --git a/Sources/Diagnose/ReduceSourceKitDRequest.swift b/Sources/Diagnose/ReduceSourceKitDRequest.swift index ada0c8e87..b532254da 100644 --- a/Sources/Diagnose/ReduceSourceKitDRequest.swift +++ b/Sources/Diagnose/ReduceSourceKitDRequest.swift @@ -12,6 +12,7 @@ extension RequestInfo { /// Reduce the input file of this request and the command line arguments. + @MainActor func reduce( using executor: SourceKitRequestExecutor, progressUpdate: (_ progress: Double, _ message: String) -> Void diff --git a/Sources/Diagnose/ReduceSwiftFrontend.swift b/Sources/Diagnose/ReduceSwiftFrontend.swift index 0ac7a1e7b..87628ce87 100644 --- a/Sources/Diagnose/ReduceSwiftFrontend.swift +++ b/Sources/Diagnose/ReduceSwiftFrontend.swift @@ -10,6 +10,7 @@ // //===----------------------------------------------------------------------===// +@MainActor @_spi(Testing) public func reduceFrontendIssue( frontendArgs: [String], diff --git a/Sources/Diagnose/RequestInfo.swift b/Sources/Diagnose/RequestInfo.swift index 4b951c8d3..b10fc6562 100644 --- a/Sources/Diagnose/RequestInfo.swift +++ b/Sources/Diagnose/RequestInfo.swift @@ -15,7 +15,7 @@ import RegexBuilder /// All the information necessary to replay a sourcektid request. @_spi(Testing) -public struct RequestInfo { +public struct RequestInfo: Sendable { /// The JSON request object. Contains the following dynamic placeholders: /// - `$OFFSET`: To be replaced by `offset` before running the request /// - `$FILE`: Will be replaced with a path to the file that contains the reduced source code. @@ -51,7 +51,7 @@ public struct RequestInfo { } /// A fake value that is used to indicate that we are reducing a `swift-frontend` issue instead of a sourcekitd issue. - static var fakeRequestTemplateForFrontendIssues = """ + static let fakeRequestTemplateForFrontendIssues = """ { key.request: sourcekit-lsp-fake-request-for-frontend-crash key.compilerargs: [ diff --git a/Sources/Diagnose/SourceKitDRequestExecutor.swift b/Sources/Diagnose/SourceKitDRequestExecutor.swift index f5cb5ec99..389549a0d 100644 --- a/Sources/Diagnose/SourceKitDRequestExecutor.swift +++ b/Sources/Diagnose/SourceKitDRequestExecutor.swift @@ -19,7 +19,7 @@ import struct TSCBasic.ProcessResult /// The different states in which a sourcekitd request can finish. @_spi(Testing) -public enum SourceKitDRequestResult { +public enum SourceKitDRequestResult: Sendable { /// The request succeeded. case success(response: String) @@ -46,11 +46,12 @@ fileprivate extension String { /// An executor that can run a sourcekitd request and indicate whether the request reprodes a specified issue. @_spi(Testing) public protocol SourceKitRequestExecutor { - func runSourceKitD(request: RequestInfo) async throws -> SourceKitDRequestResult - func runSwiftFrontend(request: RequestInfo) async throws -> SourceKitDRequestResult + @MainActor func runSourceKitD(request: RequestInfo) async throws -> SourceKitDRequestResult + @MainActor func runSwiftFrontend(request: RequestInfo) async throws -> SourceKitDRequestResult } extension SourceKitRequestExecutor { + @MainActor func run(request: RequestInfo) async throws -> SourceKitDRequestResult { if request.requestTemplate == RequestInfo.fakeRequestTemplateForFrontendIssues { return try await runSwiftFrontend(request: request) diff --git a/Sources/Diagnose/SourceReducer.swift b/Sources/Diagnose/SourceReducer.swift index 2acddca8d..8dfe8b3e1 100644 --- a/Sources/Diagnose/SourceReducer.swift +++ b/Sources/Diagnose/SourceReducer.swift @@ -21,6 +21,7 @@ import SwiftSyntax extension RequestInfo { @_spi(Testing) + @MainActor public func reduceInputFile( using executor: SourceKitRequestExecutor, progressUpdate: (_ progress: Double, _ message: String) -> Void @@ -61,6 +62,7 @@ fileprivate enum ReductionStepResult { } /// Reduces an input source file while continuing to reproduce the crash +@MainActor fileprivate class SourceReducer { /// The executor that is used to run a sourcekitd request and check whether it /// still crashes. @@ -84,6 +86,7 @@ fileprivate class SourceReducer { } /// Reduce the file contents in `initialRequest` to a smaller file that still reproduces a crash. + @MainActor func run(initialRequestInfo: RequestInfo) async throws -> RequestInfo { var requestInfo = initialRequestInfo self.initialImportCount = Parser.parse(source: requestInfo.fileContents).numberOfImports @@ -242,6 +245,7 @@ fileprivate class SourceReducer { /// /// If the request still crashes after applying the edits computed by `reduce`, return the reduced request info. /// Otherwise, return `nil` + @MainActor private func runReductionStep( requestInfo: RequestInfo, reportProgress: Bool = true, @@ -608,6 +612,7 @@ fileprivate class FirstImportFinder: SyntaxAnyVisitor { /// the file that imports the module. If `areFallbackArgs` is set, we have synthesized fallback arguments that only /// contain a target and SDK. This is useful when reducing a swift-frontend crash because sourcekitd requires driver /// arguments but the swift-frontend crash has frontend args. +@MainActor fileprivate func getSwiftInterface( _ moduleName: String, executor: SourceKitRequestExecutor, @@ -680,6 +685,7 @@ fileprivate func getSwiftInterface( return try JSONDecoder().decode(String.self, from: sanitizedData) } +@MainActor fileprivate func inlineFirstImport( in tree: SourceFileSyntax, executor: SourceKitRequestExecutor, diff --git a/Sources/Diagnose/SourcekitdRequestCommand.swift b/Sources/Diagnose/SourcekitdRequestCommand.swift index 14cabbad4..ba4e0686f 100644 --- a/Sources/Diagnose/SourcekitdRequestCommand.swift +++ b/Sources/Diagnose/SourcekitdRequestCommand.swift @@ -18,7 +18,7 @@ import SourceKitD import struct TSCBasic.AbsolutePath public struct SourceKitdRequestCommand: AsyncParsableCommand { - public static var configuration = CommandConfiguration( + public static let configuration = CommandConfiguration( commandName: "run-sourcekitd-request", abstract: "Run a sourcekitd request and print its result", shouldDisplay: false diff --git a/Sources/Diagnose/StderrStreamConcurrencySafe.swift b/Sources/Diagnose/StderrStreamConcurrencySafe.swift new file mode 100644 index 000000000..93bc48f14 --- /dev/null +++ b/Sources/Diagnose/StderrStreamConcurrencySafe.swift @@ -0,0 +1,24 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift.org open source project +// +// Copyright (c) 2014 - 2024 Apple Inc. and the Swift project authors +// Licensed under Apache License v2.0 with Runtime Library Exception +// +// See https://swift.org/LICENSE.txt for license information +// See https://swift.org/CONTRIBUTORS.txt for the list of Swift project authors +// +//===----------------------------------------------------------------------===// + +import TSCLibc + +import class TSCBasic.LocalFileOutputByteStream +import class TSCBasic.ThreadSafeOutputByteStream + +// A version of `stderrStream` from `TSCBasic` that is a `let` and can thus be used from Swift 6. +let stderrStreamConcurrencySafe: ThreadSafeOutputByteStream = try! ThreadSafeOutputByteStream( + LocalFileOutputByteStream( + filePointer: TSCLibc.stderr, + closeOnDeinit: false + ) +) diff --git a/Tests/DiagnoseTests/DiagnoseTests.swift b/Tests/DiagnoseTests/DiagnoseTests.swift index b7dab78a4..f4502a40f 100644 --- a/Tests/DiagnoseTests/DiagnoseTests.swift +++ b/Tests/DiagnoseTests/DiagnoseTests.swift @@ -114,7 +114,7 @@ final class DiagnoseTests: XCTestCase { let foo = 1️⃣Foo() } - /* + /* Block comment With another line */ @@ -146,6 +146,7 @@ final class DiagnoseTests: XCTestCase { ) } + @MainActor func testReduceFrontend() async throws { try await withTestScratchDir { scratchDir in let fileAContents = """ @@ -227,6 +228,7 @@ final class DiagnoseTests: XCTestCase { /// - `$OFFSET`: The UTF-8 offset of the 1️⃣ location marker in `markedFileContents` /// - reproducerPredicate: A predicate that indicates whether a run request reproduces the issue. /// - expectedReducedFileContents: The contents of the file that the reducer is expected to produce. +@MainActor private func assertReduceSourceKitD( _ markedFileContents: String, request: String, From 5cce99b920fcf55ebd9eab02efa830076e7fd4f8 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Thu, 9 May 2024 16:52:58 -0700 Subject: [PATCH 34/46] Make `IndexTaskDescription` protocol-based instead of enum-based This simplifies the implementation. --- .../SemanticIndex/IndexTaskDescription.swift | 110 +++++++----------- .../PreparationTaskDescription.swift | 4 +- .../SemanticIndex/SemanticIndexManager.swift | 16 +-- .../UpdateIndexStoreTaskDescription.swift | 3 +- Sources/SourceKitLSP/SourceKitLSPServer.swift | 2 +- Sources/SourceKitLSP/Workspace.swift | 8 +- 6 files changed, 62 insertions(+), 81 deletions(-) diff --git a/Sources/SemanticIndex/IndexTaskDescription.swift b/Sources/SemanticIndex/IndexTaskDescription.swift index e2e2c21f2..33f032dcd 100644 --- a/Sources/SemanticIndex/IndexTaskDescription.swift +++ b/Sources/SemanticIndex/IndexTaskDescription.swift @@ -12,94 +12,72 @@ import SKCore -/// A task that either prepares targets or updates the index store for a set of files. -public enum IndexTaskDescription: TaskDescriptionProtocol { - case updateIndexStore(UpdateIndexStoreTaskDescription) - case preparation(PreparationTaskDescription) +/// Protocol of tasks that are executed on the index task scheduler. +/// +/// It is assumed that `IndexTaskDescription` of different types are allowed to execute in parallel. +protocol IndexTaskDescription: TaskDescriptionProtocol { + /// A string that is unique to this type of `IndexTaskDescription`. It is used to produce unique IDs for tasks of + /// different types in `AnyIndexTaskDescription` + static var idPrefix: String { get } + + var id: UInt32 { get } +} + +extension IndexTaskDescription { + func dependencies( + to currentlyExecutingTasks: [AnyIndexTaskDescription] + ) -> [TaskDependencyAction] { + return self.dependencies(to: currentlyExecutingTasks.compactMap { $0.wrapped as? Self }) + .map { + switch $0 { + case .cancelAndRescheduleDependency(let td): + return .cancelAndRescheduleDependency(AnyIndexTaskDescription(td)) + case .waitAndElevatePriorityOfDependency(let td): + return .waitAndElevatePriorityOfDependency(AnyIndexTaskDescription(td)) + } + } + + } +} + +/// Type-erased wrapper of an `IndexTaskDescription`. +public struct AnyIndexTaskDescription: TaskDescriptionProtocol { + let wrapped: any IndexTaskDescription + + init(_ wrapped: any IndexTaskDescription) { + self.wrapped = wrapped + } public var isIdempotent: Bool { - switch self { - case .updateIndexStore(let taskDescription): return taskDescription.isIdempotent - case .preparation(let taskDescription): return taskDescription.isIdempotent - } + return wrapped.isIdempotent } public var estimatedCPUCoreCount: Int { - switch self { - case .updateIndexStore(let taskDescription): return taskDescription.estimatedCPUCoreCount - case .preparation(let taskDescription): return taskDescription.estimatedCPUCoreCount - } + return wrapped.estimatedCPUCoreCount } public var id: String { - switch self { - case .updateIndexStore(let taskDescription): return "indexing-\(taskDescription.id)" - case .preparation(let taskDescription): return "preparation-\(taskDescription.id)" - } + return "\(type(of: wrapped).idPrefix)-\(wrapped.id)" } public var description: String { - switch self { - case .updateIndexStore(let taskDescription): return taskDescription.description - case .preparation(let taskDescription): return taskDescription.description - } + return wrapped.description } public var redactedDescription: String { - switch self { - case .updateIndexStore(let taskDescription): return taskDescription.redactedDescription - case .preparation(let taskDescription): return taskDescription.redactedDescription - } + return wrapped.redactedDescription } public func execute() async { - switch self { - case .updateIndexStore(let taskDescription): return await taskDescription.execute() - case .preparation(let taskDescription): return await taskDescription.execute() - } + return await wrapped.execute() } /// Forward to the underlying task to compute the dependencies. Preparation and index tasks don't have any /// dependencies that are managed by `TaskScheduler`. `SemanticIndexManager` awaits the preparation of a target before /// indexing files within it. public func dependencies( - to currentlyExecutingTasks: [IndexTaskDescription] - ) -> [TaskDependencyAction] { - switch self { - case .updateIndexStore(let taskDescription): - let currentlyExecutingTasks = - currentlyExecutingTasks - .compactMap { (currentlyExecutingTask) -> UpdateIndexStoreTaskDescription? in - if case .updateIndexStore(let currentlyExecutingTask) = currentlyExecutingTask { - return currentlyExecutingTask - } - return nil - } - return taskDescription.dependencies(to: currentlyExecutingTasks).map { - switch $0 { - case .waitAndElevatePriorityOfDependency(let td): - return .waitAndElevatePriorityOfDependency(.updateIndexStore(td)) - case .cancelAndRescheduleDependency(let td): - return .cancelAndRescheduleDependency(.updateIndexStore(td)) - } - } - case .preparation(let taskDescription): - let currentlyExecutingTasks = - currentlyExecutingTasks - .compactMap { (currentlyExecutingTask) -> PreparationTaskDescription? in - if case .preparation(let currentlyExecutingTask) = currentlyExecutingTask { - return currentlyExecutingTask - } - return nil - } - return taskDescription.dependencies(to: currentlyExecutingTasks).map { - switch $0 { - case .waitAndElevatePriorityOfDependency(let td): - return .waitAndElevatePriorityOfDependency(.preparation(td)) - case .cancelAndRescheduleDependency(let td): - return .cancelAndRescheduleDependency(.preparation(td)) - } - } - } + to currentlyExecutingTasks: [AnyIndexTaskDescription] + ) -> [TaskDependencyAction] { + return wrapped.dependencies(to: currentlyExecutingTasks) } } diff --git a/Sources/SemanticIndex/PreparationTaskDescription.swift b/Sources/SemanticIndex/PreparationTaskDescription.swift index d2348b272..b48e6b93e 100644 --- a/Sources/SemanticIndex/PreparationTaskDescription.swift +++ b/Sources/SemanticIndex/PreparationTaskDescription.swift @@ -24,7 +24,9 @@ private var preparationIDForLogging = AtomicUInt32(initialValue: 1) /// Describes a task to prepare a set of targets. /// /// This task description can be scheduled in a `TaskScheduler`. -public struct PreparationTaskDescription: TaskDescriptionProtocol { +public struct PreparationTaskDescription: IndexTaskDescription { + public static let idPrefix = "prepare" + public let id = preparationIDForLogging.fetchAndIncrement() /// The targets that should be prepared. diff --git a/Sources/SemanticIndex/SemanticIndexManager.swift b/Sources/SemanticIndex/SemanticIndexManager.swift index 8069818ae..e1aac5862 100644 --- a/Sources/SemanticIndex/SemanticIndexManager.swift +++ b/Sources/SemanticIndex/SemanticIndexManager.swift @@ -44,20 +44,20 @@ public final actor SemanticIndexManager { /// The `TaskScheduler` that manages the scheduling of index tasks. This is shared among all `SemanticIndexManager`s /// in the process, to ensure that we don't schedule more index operations than processor cores from multiple /// workspaces. - private let indexTaskScheduler: TaskScheduler + private let indexTaskScheduler: TaskScheduler /// Callback that is called when an index task has finished. /// /// Currently only used for testing. - private let indexTaskDidFinish: (@Sendable (IndexTaskDescription) -> Void)? + private let indexTaskDidFinish: (@Sendable (AnyIndexTaskDescription) -> Void)? // MARK: - Public API public init( index: UncheckedIndex, buildSystemManager: BuildSystemManager, - indexTaskScheduler: TaskScheduler, - indexTaskDidFinish: (@Sendable (IndexTaskDescription) -> Void)? + indexTaskScheduler: TaskScheduler, + indexTaskDidFinish: (@Sendable (AnyIndexTaskDescription) -> Void)? ) { self.index = index.checked(for: .modifiedFiles) self.buildSystemManager = buildSystemManager @@ -133,12 +133,12 @@ public final actor SemanticIndexManager { private func prepare(targets: [ConfiguredTarget], priority: TaskPriority?) async { await self.indexTaskScheduler.schedule( priority: priority, - .preparation( + AnyIndexTaskDescription( PreparationTaskDescription( targetsToPrepare: targets, buildSystemManager: self.buildSystemManager, didFinishCallback: { [weak self] taskDescription in - self?.indexTaskDidFinish?(.preparation(taskDescription)) + self?.indexTaskDidFinish?(AnyIndexTaskDescription(taskDescription)) } ) ) @@ -149,13 +149,13 @@ public final actor SemanticIndexManager { private func updateIndexStore(for files: [DocumentURI], priority: TaskPriority?) async { await self.indexTaskScheduler.schedule( priority: priority, - .updateIndexStore( + AnyIndexTaskDescription( UpdateIndexStoreTaskDescription( filesToIndex: Set(files), buildSystemManager: self.buildSystemManager, index: self.index.unchecked, didFinishCallback: { [weak self] taskDescription in - self?.indexTaskDidFinish?(.updateIndexStore(taskDescription)) + self?.indexTaskDidFinish?(AnyIndexTaskDescription(taskDescription)) } ) ) diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index 46a8b6732..6b9968a14 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -25,7 +25,8 @@ private nonisolated(unsafe) var updateIndexStoreIDForLogging = AtomicUInt32(init /// Describes a task to index a set of source files. /// /// This task description can be scheduled in a `TaskScheduler`. -public struct UpdateIndexStoreTaskDescription: TaskDescriptionProtocol { +public struct UpdateIndexStoreTaskDescription: IndexTaskDescription { + public static let idPrefix = "update-indexstore" public let id = updateIndexStoreIDForLogging.fetchAndIncrement() /// The files that should be indexed. diff --git a/Sources/SourceKitLSP/SourceKitLSPServer.swift b/Sources/SourceKitLSP/SourceKitLSPServer.swift index 410a44c71..484310375 100644 --- a/Sources/SourceKitLSP/SourceKitLSPServer.swift +++ b/Sources/SourceKitLSP/SourceKitLSPServer.swift @@ -454,7 +454,7 @@ public actor SourceKitLSPServer { /// /// Shared process-wide to ensure the scheduled index operations across multiple workspaces don't exceed the maximum /// number of processor cores that the user allocated to background indexing. - private let indexTaskScheduler: TaskScheduler + private let indexTaskScheduler: TaskScheduler private var packageLoadingWorkDoneProgress = WorkDoneProgressState( "SourceKitLSP.SourceKitLSPServer.reloadPackage", diff --git a/Sources/SourceKitLSP/Workspace.swift b/Sources/SourceKitLSP/Workspace.swift index 04c63b9b2..050671edc 100644 --- a/Sources/SourceKitLSP/Workspace.swift +++ b/Sources/SourceKitLSP/Workspace.swift @@ -89,7 +89,7 @@ public final class Workspace: Sendable { underlyingBuildSystem: BuildSystem?, index uncheckedIndex: UncheckedIndex?, indexDelegate: SourceKitIndexDelegate?, - indexTaskScheduler: TaskScheduler + indexTaskScheduler: TaskScheduler ) async { self.documentManager = documentManager self.buildSetup = options.buildSetup @@ -142,7 +142,7 @@ public final class Workspace: Sendable { options: SourceKitLSPServer.Options, compilationDatabaseSearchPaths: [RelativePath], indexOptions: IndexOptions = IndexOptions(), - indexTaskScheduler: TaskScheduler, + indexTaskScheduler: TaskScheduler, reloadPackageStatusCallback: @Sendable @escaping (ReloadPackageStatus) async -> Void ) async throws { var buildSystem: BuildSystem? = nil @@ -306,7 +306,7 @@ public struct IndexOptions: Sendable { /// A callback that is called when an index task finishes. /// /// Intended for testing purposes. - public var indexTaskDidFinish: (@Sendable (IndexTaskDescription) -> Void)? + public var indexTaskDidFinish: (@Sendable (AnyIndexTaskDescription) -> Void)? public init( indexStorePath: AbsolutePath? = nil, @@ -315,7 +315,7 @@ public struct IndexOptions: Sendable { listenToUnitEvents: Bool = true, enableBackgroundIndexing: Bool = false, maxCoresPercentageToUseForBackgroundIndexing: Double = 1, - indexTaskDidFinish: (@Sendable (IndexTaskDescription) -> Void)? = nil + indexTaskDidFinish: (@Sendable (AnyIndexTaskDescription) -> Void)? = nil ) { self.indexStorePath = indexStorePath self.indexDatabasePath = indexDatabasePath From 0dc5cdd7c79e44e71fc0373bc188bec1630c7c85 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Fri, 10 May 2024 06:47:51 -0700 Subject: [PATCH 35/46] Call `indexTaskDidFinish` from `SemanticIndexManager` This fixes a bug where `indexTaskDidFinish` would also get called when a task is cancelled to be rescheduled. --- .../PreparationTaskDescription.swift | 12 +----- .../SemanticIndex/SemanticIndexManager.swift | 38 ++++++++----------- .../UpdateIndexStoreTaskDescription.swift | 10 +---- 3 files changed, 17 insertions(+), 43 deletions(-) diff --git a/Sources/SemanticIndex/PreparationTaskDescription.swift b/Sources/SemanticIndex/PreparationTaskDescription.swift index b48e6b93e..7765c63e3 100644 --- a/Sources/SemanticIndex/PreparationTaskDescription.swift +++ b/Sources/SemanticIndex/PreparationTaskDescription.swift @@ -35,11 +35,6 @@ public struct PreparationTaskDescription: IndexTaskDescription { /// The build system manager that is used to get the toolchain and build settings for the files to index. private let buildSystemManager: BuildSystemManager - /// A callback that is called when the task finishes. - /// - /// Intended for testing purposes. - private let didFinishCallback: @Sendable (PreparationTaskDescription) -> Void - /// The task is idempotent because preparing the same target twice produces the same result as preparing it once. public var isIdempotent: Bool { true } @@ -55,18 +50,13 @@ public struct PreparationTaskDescription: IndexTaskDescription { init( targetsToPrepare: [ConfiguredTarget], - buildSystemManager: BuildSystemManager, - didFinishCallback: @escaping @Sendable (PreparationTaskDescription) -> Void + buildSystemManager: BuildSystemManager ) { self.targetsToPrepare = targetsToPrepare self.buildSystemManager = buildSystemManager - self.didFinishCallback = didFinishCallback } public func execute() async { - defer { - didFinishCallback(self) - } // Only use the last two digits of the preparation ID for the logging scope to avoid creating too many scopes. // See comment in `withLoggingScope`. // The last 2 digits should be sufficient to differentiate between multiple concurrently running preparation operations diff --git a/Sources/SemanticIndex/SemanticIndexManager.swift b/Sources/SemanticIndex/SemanticIndexManager.swift index e1aac5862..9f615cbf3 100644 --- a/Sources/SemanticIndex/SemanticIndexManager.swift +++ b/Sources/SemanticIndex/SemanticIndexManager.swift @@ -131,38 +131,30 @@ public final actor SemanticIndexManager { /// Prepare the given targets for indexing private func prepare(targets: [ConfiguredTarget], priority: TaskPriority?) async { - await self.indexTaskScheduler.schedule( - priority: priority, - AnyIndexTaskDescription( - PreparationTaskDescription( - targetsToPrepare: targets, - buildSystemManager: self.buildSystemManager, - didFinishCallback: { [weak self] taskDescription in - self?.indexTaskDidFinish?(AnyIndexTaskDescription(taskDescription)) - } - ) + let taskDescription = AnyIndexTaskDescription( + PreparationTaskDescription( + targetsToPrepare: targets, + buildSystemManager: self.buildSystemManager ) - ).value + ) + await self.indexTaskScheduler.schedule(priority: priority, taskDescription).value + self.indexTaskDidFinish?(taskDescription) } /// Update the index store for the given files, assuming that their targets have already been prepared. private func updateIndexStore(for files: [DocumentURI], priority: TaskPriority?) async { - await self.indexTaskScheduler.schedule( - priority: priority, - AnyIndexTaskDescription( - UpdateIndexStoreTaskDescription( - filesToIndex: Set(files), - buildSystemManager: self.buildSystemManager, - index: self.index.unchecked, - didFinishCallback: { [weak self] taskDescription in - self?.indexTaskDidFinish?(AnyIndexTaskDescription(taskDescription)) - } - ) + let taskDescription = AnyIndexTaskDescription( + UpdateIndexStoreTaskDescription( + filesToIndex: Set(files), + buildSystemManager: self.buildSystemManager, + index: self.index.unchecked ) - ).value + ) + await self.indexTaskScheduler.schedule(priority: priority, taskDescription).value for file in files { self.indexStatus[file] = .upToDate } + self.indexTaskDidFinish?(taskDescription) } /// Index the given set of files at the given priority. diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index 6b9968a14..2de7ea4c0 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -39,9 +39,6 @@ public struct UpdateIndexStoreTaskDescription: IndexTaskDescription { /// case we don't need to index it again. private let index: UncheckedIndex - /// A callback that is called when the index task finishes - private let didFinishCallback: @Sendable (UpdateIndexStoreTaskDescription) -> Void - /// The task is idempotent because indexing the same file twice produces the same result as indexing it once. public var isIdempotent: Bool { true } @@ -58,19 +55,14 @@ public struct UpdateIndexStoreTaskDescription: IndexTaskDescription { init( filesToIndex: Set, buildSystemManager: BuildSystemManager, - index: UncheckedIndex, - didFinishCallback: @escaping @Sendable (UpdateIndexStoreTaskDescription) -> Void + index: UncheckedIndex ) { self.filesToIndex = filesToIndex self.buildSystemManager = buildSystemManager self.index = index - self.didFinishCallback = didFinishCallback } public func execute() async { - defer { - didFinishCallback(self) - } // Only use the last two digits of the indexing ID for the logging scope to avoid creating too many scopes. // See comment in `withLoggingScope`. // The last 2 digits should be sufficient to differentiate between multiple concurrently running indexing operation. From c399b438584f520455dda4d90e946eebc240998c Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Fri, 10 May 2024 06:50:40 -0700 Subject: [PATCH 36/46] =?UTF-8?q?Don=E2=80=99t=20pass=20the=20finished=20`?= =?UTF-8?q?taskDescription`=20to=20`indexTaskDidFinish`=20callback?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This follows the general paradigm that callbacks shouldn’t carry much state and instead only notify an observer that state has changed, which the observer can then poll. --- Sources/SemanticIndex/SemanticIndexManager.swift | 8 ++++---- Sources/SourceKitLSP/Workspace.swift | 4 ++-- Tests/SourceKitLSPTests/BackgroundIndexingTests.swift | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/Sources/SemanticIndex/SemanticIndexManager.swift b/Sources/SemanticIndex/SemanticIndexManager.swift index 9f615cbf3..250a0ec0f 100644 --- a/Sources/SemanticIndex/SemanticIndexManager.swift +++ b/Sources/SemanticIndex/SemanticIndexManager.swift @@ -49,7 +49,7 @@ public final actor SemanticIndexManager { /// Callback that is called when an index task has finished. /// /// Currently only used for testing. - private let indexTaskDidFinish: (@Sendable (AnyIndexTaskDescription) -> Void)? + private let indexTaskDidFinish: (@Sendable () -> Void)? // MARK: - Public API @@ -57,7 +57,7 @@ public final actor SemanticIndexManager { index: UncheckedIndex, buildSystemManager: BuildSystemManager, indexTaskScheduler: TaskScheduler, - indexTaskDidFinish: (@Sendable (AnyIndexTaskDescription) -> Void)? + indexTaskDidFinish: (@Sendable () -> Void)? ) { self.index = index.checked(for: .modifiedFiles) self.buildSystemManager = buildSystemManager @@ -138,7 +138,7 @@ public final actor SemanticIndexManager { ) ) await self.indexTaskScheduler.schedule(priority: priority, taskDescription).value - self.indexTaskDidFinish?(taskDescription) + self.indexTaskDidFinish?() } /// Update the index store for the given files, assuming that their targets have already been prepared. @@ -154,7 +154,7 @@ public final actor SemanticIndexManager { for file in files { self.indexStatus[file] = .upToDate } - self.indexTaskDidFinish?(taskDescription) + self.indexTaskDidFinish?() } /// Index the given set of files at the given priority. diff --git a/Sources/SourceKitLSP/Workspace.swift b/Sources/SourceKitLSP/Workspace.swift index 050671edc..c89cdd53b 100644 --- a/Sources/SourceKitLSP/Workspace.swift +++ b/Sources/SourceKitLSP/Workspace.swift @@ -306,7 +306,7 @@ public struct IndexOptions: Sendable { /// A callback that is called when an index task finishes. /// /// Intended for testing purposes. - public var indexTaskDidFinish: (@Sendable (AnyIndexTaskDescription) -> Void)? + public var indexTaskDidFinish: (@Sendable () -> Void)? public init( indexStorePath: AbsolutePath? = nil, @@ -315,7 +315,7 @@ public struct IndexOptions: Sendable { listenToUnitEvents: Bool = true, enableBackgroundIndexing: Bool = false, maxCoresPercentageToUseForBackgroundIndexing: Double = 1, - indexTaskDidFinish: (@Sendable (AnyIndexTaskDescription) -> Void)? = nil + indexTaskDidFinish: (@Sendable () -> Void)? = nil ) { self.indexStorePath = indexStorePath self.indexDatabasePath = indexDatabasePath diff --git a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift index a2c4c3539..7ec0dd82a 100644 --- a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift +++ b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift @@ -165,10 +165,10 @@ final class BackgroundIndexingTests: XCTestCase { func testBackgroundIndexingHappensWithLowPriority() async throws { var serverOptions = backgroundIndexingOptions - serverOptions.indexOptions.indexTaskDidFinish = { taskDescription in + serverOptions.indexOptions.indexTaskDidFinish = { XCTAssert( Task.currentPriority == .low, - "\(taskDescription.description) ran with priority \(Task.currentPriority)" + "An index task ran with priority \(Task.currentPriority)" ) } let project = try await SwiftPMTestProject( From d70a68f37cc08c9d4f74c5d78e4e68fa4463eff8 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Fri, 10 May 2024 11:44:17 -0700 Subject: [PATCH 37/46] Wait until initialization has finished before starting a work done progress --- Sources/SourceKitLSP/SourceKitLSPServer.swift | 54 +++++++++++++++---- .../WorkDoneProgressManager.swift | 21 +++++++- 2 files changed, 62 insertions(+), 13 deletions(-) diff --git a/Sources/SourceKitLSP/SourceKitLSPServer.swift b/Sources/SourceKitLSP/SourceKitLSPServer.swift index 484310375..14553c26a 100644 --- a/Sources/SourceKitLSP/SourceKitLSPServer.swift +++ b/Sources/SourceKitLSP/SourceKitLSPServer.swift @@ -115,6 +115,10 @@ final actor WorkDoneProgressState { case progressCreationFailed } + /// A queue so we can have synchronous `startProgress` and `endProgress` functions that don't need to wait for the + /// work done progress to be started or ended. + private let queue = AsyncQueue() + /// How many active tasks are running. /// /// A work done progress should be displayed if activeTasks > 0 @@ -135,13 +139,16 @@ final actor WorkDoneProgressState { /// Start a new task, creating a new `WorkDoneProgress` if none is running right now. /// /// - Parameter server: The server that is used to create the `WorkDoneProgress` on the client - func startProgress(server: SourceKitLSPServer) async { + nonisolated func startProgress(server: SourceKitLSPServer) { + queue.async { + await self.startProgressImpl(server: server) + } + } + + func startProgressImpl(server: SourceKitLSPServer) async { + await server.waitUntilInitialized() activeTasks += 1 guard await server.capabilityRegistry?.clientCapabilities.window?.workDoneProgress ?? false else { - // If the client doesn't support workDoneProgress, keep track of the active task count but don't update the state. - // That way, if we call `startProgress` before initialization finishes, we won't send the - // `CreateWorkDoneProgressRequest` but if we call `startProgress` again after initialization finished (and thus - // the capability is set), we will create the work done progress. return } if state == .noProgress { @@ -180,7 +187,13 @@ final actor WorkDoneProgressState { /// If this drops the active task count to 0, the work done progress is ended on the client. /// /// - Parameter server: The server that is used to send and update of the `WorkDoneProgress` to the client - func endProgress(server: SourceKitLSPServer) async { + nonisolated func endProgress(server: SourceKitLSPServer) { + queue.async { + await self.endProgressImpl(server: server) + } + } + + func endProgressImpl(server: SourceKitLSPServer) async { assert(activeTasks > 0, "Unbalanced startProgress/endProgress calls") activeTasks -= 1 guard await server.capabilityRegistry?.clientCapabilities.window?.workDoneProgress ?? false else { @@ -440,11 +453,16 @@ public actor SourceKitLSPServer { /// The connection to the editor. public let client: Connection + /// Set to `true` after the `SourceKitLSPServer` has send the reply to the `InitializeRequest`. + /// + /// Initialization can be awaited using `waitUntilInitialized`. + private var initialized: Bool = false + var options: Options let toolchainRegistry: ToolchainRegistry - var capabilityRegistry: CapabilityRegistry? + public var capabilityRegistry: CapabilityRegistry? var languageServices: [LanguageServerType: [LanguageService]] = [:] @@ -508,19 +526,15 @@ public actor SourceKitLSPServer { self.inProgressRequests[id] = task } - let fs: FileSystem - var onExit: () -> Void /// Creates a language server for the given client. public init( client: Connection, - fileSystem: FileSystem = localFileSystem, toolchainRegistry: ToolchainRegistry, options: Options, onExit: @escaping () -> Void = {} ) { - self.fs = fileSystem self.toolchainRegistry = toolchainRegistry self.options = options self.onExit = onExit @@ -534,6 +548,22 @@ public actor SourceKitLSPServer { ]) } + /// Await until the server has send the reply to the initialize request. + func waitUntilInitialized() async { + // The polling of `initialized` is not perfect but it should be OK, because + // - In almost all cases the server should already be initialized. + // - If it's not initialized, we expect initialization to finish fairly quickly. Even if initialization takes 5s + // this only results in 50 polls, which is acceptable. + // Alternative solutions that signal via an async sequence seem overkill here. + while !initialized { + do { + try await Task.sleep(for: .seconds(0.1)) + } catch { + break + } + } + } + /// Search through all the parent directories of `uri` and check if any of these directories contain a workspace /// capable of handling `uri`. /// @@ -958,6 +988,8 @@ extension SourceKitLSPServer: MessageHandler { switch request { case let request as RequestAndReply: await request.reply { try await initialize(request.params) } + // Only set `initialized` to `true` after we have sent the response to the initialize request to the client. + initialized = true case let request as RequestAndReply: await request.reply { try await shutdown(request.params) } case let request as RequestAndReply: diff --git a/Sources/SourceKitLSP/WorkDoneProgressManager.swift b/Sources/SourceKitLSP/WorkDoneProgressManager.swift index 089e76f90..f4f59d496 100644 --- a/Sources/SourceKitLSP/WorkDoneProgressManager.swift +++ b/Sources/SourceKitLSP/WorkDoneProgressManager.swift @@ -23,20 +23,37 @@ final class WorkDoneProgressManager { private let queue = AsyncQueue() private let server: SourceKitLSPServer - init?(server: SourceKitLSPServer, capabilityRegistry: CapabilityRegistry, title: String, message: String? = nil) { + convenience init?(server: SourceKitLSPServer, title: String, message: String? = nil, percentage: Int? = nil) async { + guard let capabilityRegistry = await server.capabilityRegistry else { + return nil + } + self.init(server: server, capabilityRegistry: capabilityRegistry, title: title, message: message) + } + + init?( + server: SourceKitLSPServer, + capabilityRegistry: CapabilityRegistry, + title: String, + message: String? = nil, + percentage: Int? = nil + ) { guard capabilityRegistry.clientCapabilities.window?.workDoneProgress ?? false else { return nil } self.token = .string("WorkDoneProgress-\(UUID())") self.server = server queue.async { [server, token] in + await server.waitUntilInitialized() await withCheckedContinuation { (continuation: CheckedContinuation) in _ = server.client.send(CreateWorkDoneProgressRequest(token: token)) { result in continuation.resume() } } await server.sendNotificationToClient( - WorkDoneProgress(token: token, value: .begin(WorkDoneProgressBegin(title: title, message: message))) + WorkDoneProgress( + token: token, + value: .begin(WorkDoneProgressBegin(title: title, message: message, percentage: percentage)) + ) ) } } From eaa378f3903d3d88447b57b82a5adce7396164b9 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Fri, 10 May 2024 11:45:16 -0700 Subject: [PATCH 38/46] Plumb `capabilities` to `SwiftPMTestProject` and add a closure that can be executed before SourceKit-LSP is initialized --- Sources/SKTestSupport/MultiFileTestProject.swift | 4 ++++ Sources/SKTestSupport/SwiftPMTestProject.swift | 4 ++++ Sources/SKTestSupport/TestSourceKitLSPClient.swift | 6 +++++- 3 files changed, 13 insertions(+), 1 deletion(-) diff --git a/Sources/SKTestSupport/MultiFileTestProject.swift b/Sources/SKTestSupport/MultiFileTestProject.swift index 68baba274..ea300f497 100644 --- a/Sources/SKTestSupport/MultiFileTestProject.swift +++ b/Sources/SKTestSupport/MultiFileTestProject.swift @@ -80,8 +80,10 @@ public class MultiFileTestProject { public init( files: [RelativeFileLocation: String], workspaces: (URL) async throws -> [WorkspaceFolder] = { [WorkspaceFolder(uri: DocumentURI($0))] }, + capabilities: ClientCapabilities = ClientCapabilities(), serverOptions: SourceKitLSPServer.Options = .testDefault, usePullDiagnostics: Bool = true, + preInitialization: ((TestSourceKitLSPClient) -> Void)? = nil, testName: String = #function ) async throws { scratchDirectory = try testScratchDir(testName: testName) @@ -112,8 +114,10 @@ public class MultiFileTestProject { self.testClient = try await TestSourceKitLSPClient( serverOptions: serverOptions, + capabilities: capabilities, usePullDiagnostics: usePullDiagnostics, workspaceFolders: workspaces(scratchDirectory), + preInitialization: preInitialization, cleanUp: { [scratchDirectory] in if cleanScratchDirectories { try? FileManager.default.removeItem(at: scratchDirectory) diff --git a/Sources/SKTestSupport/SwiftPMTestProject.swift b/Sources/SKTestSupport/SwiftPMTestProject.swift index 72334c0d6..abd4a20d2 100644 --- a/Sources/SKTestSupport/SwiftPMTestProject.swift +++ b/Sources/SKTestSupport/SwiftPMTestProject.swift @@ -42,8 +42,10 @@ public class SwiftPMTestProject: MultiFileTestProject { workspaces: (URL) async throws -> [WorkspaceFolder] = { [WorkspaceFolder(uri: DocumentURI($0))] }, build: Bool = false, allowBuildFailure: Bool = false, + capabilities: ClientCapabilities = ClientCapabilities(), serverOptions: SourceKitLSPServer.Options = .testDefault, pollIndex: Bool = true, + preInitialization: ((TestSourceKitLSPClient) -> Void)? = nil, usePullDiagnostics: Bool = true, testName: String = #function ) async throws { @@ -66,8 +68,10 @@ public class SwiftPMTestProject: MultiFileTestProject { try await super.init( files: filesByPath, workspaces: workspaces, + capabilities: capabilities, serverOptions: serverOptions, usePullDiagnostics: usePullDiagnostics, + preInitialization: preInitialization, testName: testName ) diff --git a/Sources/SKTestSupport/TestSourceKitLSPClient.swift b/Sources/SKTestSupport/TestSourceKitLSPClient.swift index 1ab9a7541..833b0f02a 100644 --- a/Sources/SKTestSupport/TestSourceKitLSPClient.swift +++ b/Sources/SKTestSupport/TestSourceKitLSPClient.swift @@ -83,6 +83,8 @@ public final class TestSourceKitLSPClient: MessageHandler { /// - capabilities: The test client's capabilities. /// - usePullDiagnostics: Whether to use push diagnostics or use push-based diagnostics /// - workspaceFolders: Workspace folders to open. + /// - preInitialization: A closure that is called after the test client is created but before SourceKit-LSP is + /// initialized. This can be used to eg. register request handlers. /// - cleanUp: A closure that is called when the `TestSourceKitLSPClient` is destructed. /// This allows e.g. a `IndexedSingleSwiftFileTestProject` to delete its temporary files when they are no longer /// needed. @@ -94,6 +96,7 @@ public final class TestSourceKitLSPClient: MessageHandler { capabilities: ClientCapabilities = ClientCapabilities(), usePullDiagnostics: Bool = true, workspaceFolders: [WorkspaceFolder]? = nil, + preInitialization: ((TestSourceKitLSPClient) -> Void)? = nil, cleanUp: @escaping () -> Void = {} ) async throws { if !useGlobalModuleCache { @@ -135,7 +138,7 @@ public final class TestSourceKitLSPClient: MessageHandler { guard capabilities.textDocument!.diagnostic == nil else { struct ConflictingDiagnosticsError: Error, CustomStringConvertible { var description: String { - "usePushDiagnostics = false is not supported if capabilities already contain diagnostic options" + "usePullDiagnostics = false is not supported if capabilities already contain diagnostic options" } } throw ConflictingDiagnosticsError() @@ -145,6 +148,7 @@ public final class TestSourceKitLSPClient: MessageHandler { XCTAssertEqual(request.registrations.only?.method, DocumentDiagnosticsRequest.method) return VoidResponse() } + preInitialization?(self) } if initialize { _ = try await self.send( From c9b51f9b345fe8d7fcaa844df7f214628c10e9cc Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Fri, 10 May 2024 11:45:44 -0700 Subject: [PATCH 39/46] Allow client request handlers specified on `TestSourceKitLSPClient` to be executed in any order --- .../SKTestSupport/TestSourceKitLSPClient.swift | 17 +++++++++-------- .../UpdateIndexStoreTaskDescription.swift | 2 +- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/Sources/SKTestSupport/TestSourceKitLSPClient.swift b/Sources/SKTestSupport/TestSourceKitLSPClient.swift index 833b0f02a..f3854c039 100644 --- a/Sources/SKTestSupport/TestSourceKitLSPClient.swift +++ b/Sources/SKTestSupport/TestSourceKitLSPClient.swift @@ -290,18 +290,19 @@ public final class TestSourceKitLSPClient: MessageHandler { id: LanguageServerProtocol.RequestID, reply: @escaping (LSPResult) -> Void ) { - guard let requestHandler = requestHandlers.first else { - reply(.failure(.methodNotFound(Request.method))) - return - } - guard let requestHandler = requestHandler as? RequestHandler else { - print("\(RequestHandler.self)") - XCTFail("Received request of unexpected type \(Request.method)") + let requestHandlerAndIndex = requestHandlers.enumerated().compactMap { + (index, handler) -> (RequestHandler, Int)? in + guard let handler = handler as? RequestHandler else { + return nil + } + return (handler, index) + }.first + guard let (requestHandler, index) = requestHandlerAndIndex else { reply(.failure(.methodNotFound(Request.method))) return } reply(.success(requestHandler(params))) - requestHandlers.removeFirst() + requestHandlers.remove(at: index) } // MARK: - Convenience functions diff --git a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift index 2de7ea4c0..cd8b48ea6 100644 --- a/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift +++ b/Sources/SemanticIndex/UpdateIndexStoreTaskDescription.swift @@ -49,7 +49,7 @@ public struct UpdateIndexStoreTaskDescription: IndexTaskDescription { } public var redactedDescription: String { - return "indexing-\(id)" + return "update-indexstore-\(id)" } init( From a6389e58e2df161c2ef064d2e15f2ee3005d533c Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Mon, 13 May 2024 21:59:49 -0700 Subject: [PATCH 40/46] Show a work done progress that shows the index progress This makes it a lot easier to work on background indexing because you can easily see how background indexing is making progress. Resolves #1257 rdar://127474057 --- Sources/SKCore/TaskScheduler.swift | 38 +++++- .../SemanticIndex/SemanticIndexManager.swift | 108 +++++++++++++++--- Sources/SourceKitLSP/CMakeLists.txt | 1 + .../SourceKitLSP/IndexProgressManager.swift | 103 +++++++++++++++++ .../SourceKitLSPServer+Options.swift | 5 + Sources/SourceKitLSP/SourceKitLSPServer.swift | 30 ++++- Sources/SourceKitLSP/Workspace.swift | 24 ++-- .../BackgroundIndexingTests.swift | 47 +++++++- .../SourceKitLSPTests/BuildSystemTests.swift | 4 +- 9 files changed, 324 insertions(+), 36 deletions(-) create mode 100644 Sources/SourceKitLSP/IndexProgressManager.swift diff --git a/Sources/SKCore/TaskScheduler.swift b/Sources/SKCore/TaskScheduler.swift index deb3a3399..58cf052fb 100644 --- a/Sources/SKCore/TaskScheduler.swift +++ b/Sources/SKCore/TaskScheduler.swift @@ -76,6 +76,19 @@ public protocol TaskDescriptionProtocol: Identifiable, Sendable, CustomLogString var estimatedCPUCoreCount: Int { get } } +/// Parameter that's passed to `executionStateChangedCallback` to indicate the new state of a scheduled task. +public enum TaskExecutionState { + /// The task started executing. + case executing + + /// The task was cancelled and will be re-scheduled for execution later. Will be followed by another call with + /// `executing`. + case cancelledToBeRescheduled + + /// The task has finished executing. Now more state updates will come after this one. + case finished +} + fileprivate actor QueuedTask { /// Result of `executionTask` / the tasks in `executionTaskCreatedContinuation`. /// See doc comment on `executionTask`. @@ -136,9 +149,18 @@ fileprivate actor QueuedTask { /// Gets reset every time `executionTask` finishes. nonisolated(unsafe) private var cancelledToBeRescheduled: AtomicBool = .init(initialValue: false) - init(priority: TaskPriority? = nil, description: TaskDescription) async { + /// A callback that will be called when the task starts executing, is cancelled to be rescheduled, or when it finishes + /// execution. + private let executionStateChangedCallback: (@Sendable (TaskExecutionState) async -> Void)? + + init( + priority: TaskPriority? = nil, + description: TaskDescription, + executionStateChangedCallback: (@Sendable (TaskExecutionState) async -> Void)? + ) async { self._priority = .init(initialValue: priority?.rawValue ?? Task.currentPriority.rawValue) self.description = description + self.executionStateChangedCallback = executionStateChangedCallback var updatePriorityContinuation: AsyncStream.Continuation! let updatePriorityStream = AsyncStream { @@ -194,16 +216,19 @@ fileprivate actor QueuedTask { } executionTask = task executionTaskCreatedContinuation.yield(task) + await executionStateChangedCallback?(.executing) return await task.value } /// Implementation detail of `execute` that is called after `self.description.execute()` finishes. - private func finalizeExecution() -> ExecutionTaskFinishStatus { + private func finalizeExecution() async -> ExecutionTaskFinishStatus { self.executionTask = nil if Task.isCancelled && self.cancelledToBeRescheduled.value { + await executionStateChangedCallback?(.cancelledToBeRescheduled) self.cancelledToBeRescheduled.value = false return ExecutionTaskFinishStatus.cancelledToBeRescheduled } else { + await executionStateChangedCallback?(.finished) return ExecutionTaskFinishStatus.terminated } } @@ -308,12 +333,17 @@ public actor TaskScheduler { @discardableResult public func schedule( priority: TaskPriority? = nil, - _ taskDescription: TaskDescription + _ taskDescription: TaskDescription, + @_inheritActorContext executionStateChangedCallback: (@Sendable (TaskExecutionState) async -> Void)? = nil ) async -> Task { withLoggingSubsystemAndScope(subsystem: taskSchedulerSubsystem, scope: nil) { logger.debug("Scheduling \(taskDescription.forLogging)") } - let queuedTask = await QueuedTask(priority: priority, description: taskDescription) + let queuedTask = await QueuedTask( + priority: priority, + description: taskDescription, + executionStateChangedCallback: executionStateChangedCallback + ) pendingTasks.append(queuedTask) Task.detached(priority: priority ?? Task.currentPriority) { // Poke the `TaskScheduler` to execute a new task. If the `TaskScheduler` is already working at its capacity diff --git a/Sources/SemanticIndex/SemanticIndexManager.swift b/Sources/SemanticIndex/SemanticIndexManager.swift index 250a0ec0f..c60a12a3e 100644 --- a/Sources/SemanticIndex/SemanticIndexManager.swift +++ b/Sources/SemanticIndex/SemanticIndexManager.swift @@ -19,8 +19,22 @@ import SKCore private enum FileIndexStatus { /// The index is up-to-date. case upToDate - /// The file is being indexed by the given task. - case inProgress(Task) + /// The file is not up to date and we have scheduled a task to index it but that index operation hasn't been started + /// yet. + case scheduled(Task) + /// We are currently actively indexing this file, ie. we are running a subprocess that indexes the file. + case executing(Task) + + var description: String { + switch self { + case .upToDate: + return "upToDate" + case .scheduled: + return "scheduled" + case .executing: + return "executing" + } + } } /// Schedules index tasks and keeps track of the index status of files. @@ -46,22 +60,51 @@ public final actor SemanticIndexManager { /// workspaces. private let indexTaskScheduler: TaskScheduler + /// Called when files are scheduled to be indexed. + /// + /// The parameter is the number of files that were scheduled to be indexed. + private let indexTasksWereScheduled: @Sendable (_ numberOfFileScheduled: Int) -> Void + /// Callback that is called when an index task has finished. /// - /// Currently only used for testing. - private let indexTaskDidFinish: (@Sendable () -> Void)? + /// An object observing this property probably wants to check `inProgressIndexTasks` when the callback is called to + /// get the current list of in-progress index tasks. + /// + /// The number of `indexTaskDidFinish` calls does not have to relate to the number of `indexTasksWereScheduled` calls. + private let indexTaskDidFinish: @Sendable () -> Void // MARK: - Public API + /// The files that still need to be indexed. + /// + /// See `FileIndexStatus` for the distinction between `scheduled` and `executing`. + public var inProgressIndexTasks: (scheduled: [DocumentURI], executing: [DocumentURI]) { + let scheduled = indexStatus.compactMap { (uri: DocumentURI, status: FileIndexStatus) in + if case .scheduled = status { + return uri + } + return nil + } + let inProgress = indexStatus.compactMap { (uri: DocumentURI, status: FileIndexStatus) in + if case .executing = status { + return uri + } + return nil + } + return (scheduled, inProgress) + } + public init( index: UncheckedIndex, buildSystemManager: BuildSystemManager, indexTaskScheduler: TaskScheduler, - indexTaskDidFinish: (@Sendable () -> Void)? + indexTasksWereScheduled: @escaping @Sendable (Int) -> Void, + indexTaskDidFinish: @escaping @Sendable () -> Void ) { self.index = index.checked(for: .modifiedFiles) self.buildSystemManager = buildSystemManager self.indexTaskScheduler = indexTaskScheduler + self.indexTasksWereScheduled = indexTasksWereScheduled self.indexTaskDidFinish = indexTaskDidFinish } @@ -93,7 +136,7 @@ public final actor SemanticIndexManager { await withTaskGroup(of: Void.self) { taskGroup in for (_, status) in indexStatus { switch status { - case .inProgress(let task): + case .scheduled(let task), .executing(let task): taskGroup.addTask { await task.value } @@ -138,7 +181,7 @@ public final actor SemanticIndexManager { ) ) await self.indexTaskScheduler.schedule(priority: priority, taskDescription).value - self.indexTaskDidFinish?() + self.indexTaskDidFinish() } /// Update the index store for the given files, assuming that their targets have already been prepared. @@ -150,11 +193,44 @@ public final actor SemanticIndexManager { index: self.index.unchecked ) ) - await self.indexTaskScheduler.schedule(priority: priority, taskDescription).value - for file in files { - self.indexStatus[file] = .upToDate + let updateIndexStoreTask = await self.indexTaskScheduler.schedule(priority: priority, taskDescription) { newState in + switch newState { + case .executing: + for file in files { + if case .scheduled(let task) = self.indexStatus[file] { + self.indexStatus[file] = .executing(task) + } else { + logger.fault( + """ + Index status of \(file) is in an unexpected state \ + '\(self.indexStatus[file]?.description ?? "", privacy: .public)' when update index store task \ + started executing + """ + ) + } + } + case .cancelledToBeRescheduled: + for file in files { + if case .executing(let task) = self.indexStatus[file] { + self.indexStatus[file] = .scheduled(task) + } else { + logger.fault( + """ + Index status of \(file) is in an unexpected state \ + '\(self.indexStatus[file]?.description ?? "", privacy: .public)' when update index store task \ + is cancelled to be rescheduled. + """ + ) + } + } + case .finished: + for file in files { + self.indexStatus[file] = .upToDate + } + self.indexTaskDidFinish() + } } - self.indexTaskDidFinish?() + await updateIndexStoreTask.value } /// Index the given set of files at the given priority. @@ -226,9 +302,15 @@ public final actor SemanticIndexManager { } indexTasks.append(indexTask) - for file in targetsBatch.flatMap({ filesByTarget[$0]! }) { - indexStatus[file] = .inProgress(indexTask) + let filesToIndex = targetsBatch.flatMap({ filesByTarget[$0]! }) + for file in filesToIndex { + // indexStatus will get set to `.upToDate` by `updateIndexStore`. Setting it to `.upToDate` cannot race with + // setting it to `.scheduled` because we don't have an `await` call between the creation of `indexTask` and + // this loop, so we still have exclusive access to the `SemanticIndexManager` actor and hence `updateIndexStore` + // can't execute until we have set all index statuses to `.scheduled`. + indexStatus[file] = .scheduled(indexTask) } + indexTasksWereScheduled(filesToIndex.count) } let indexTasksImmutable = indexTasks diff --git a/Sources/SourceKitLSP/CMakeLists.txt b/Sources/SourceKitLSP/CMakeLists.txt index a86c621f9..45684cec9 100644 --- a/Sources/SourceKitLSP/CMakeLists.txt +++ b/Sources/SourceKitLSP/CMakeLists.txt @@ -3,6 +3,7 @@ add_library(SourceKitLSP STATIC CapabilityRegistry.swift DocumentManager.swift DocumentSnapshot+FromFileContents.swift + IndexProgressManager.swift IndexStoreDB+MainFilesProvider.swift LanguageService.swift Rename.swift diff --git a/Sources/SourceKitLSP/IndexProgressManager.swift b/Sources/SourceKitLSP/IndexProgressManager.swift new file mode 100644 index 000000000..fdc745fa3 --- /dev/null +++ b/Sources/SourceKitLSP/IndexProgressManager.swift @@ -0,0 +1,103 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift.org open source project +// +// Copyright (c) 2014 - 2024 Apple Inc. and the Swift project authors +// Licensed under Apache License v2.0 with Runtime Library Exception +// +// See https://swift.org/LICENSE.txt for license information +// See https://swift.org/CONTRIBUTORS.txt for the list of Swift project authors +// +//===----------------------------------------------------------------------===// + +import LanguageServerProtocol +import SKSupport +import SemanticIndex + +/// Listens for index status updates from `SemanticIndexManagers`. From that information, it manages a +/// `WorkDoneProgress` that communicates the index progress to the editor. +actor IndexProgressManager { + /// A queue on which `indexTaskWasQueued` and `indexStatusDidChange` are handled. + /// + /// This allows the two functions two be `nonisolated` (and eg. the caller of `indexStatusDidChange` doesn't have to + /// wait for the work done progress to be updated) while still guaranteeing that we handle them in the order they + /// were called. + private let queue = AsyncQueue() + + /// The `SourceKitLSPServer` for which this manages the index progress. It gathers all `SemanticIndexManagers` from + /// the workspaces in the `SourceKitLSPServer`. + private weak var sourceKitLSPServer: SourceKitLSPServer? + + /// This is the target number of index tasks (eg. the `3` in `1/3 done`). + /// + /// Every time a new index task is scheduled, this number gets incremented, so that it only ever increases. + /// When indexing of one session is done (ie. when there are no more `scheduled` or `executing` tasks in any + /// `SemanticIndexManager`), `queuedIndexTasks` gets reset to 0 and the work done progress gets ended. + /// This way, when the next work done progress is started, it starts at zero again. + /// + /// The number of outstanding tasks is determined from the `scheduled` and `executing` tasks in all the + /// `SemanticIndexManager`s. + private var queuedIndexTasks = 0 + + /// While there are ongoing index tasks, a `WorkDoneProgressManager` that displays the work done progress. + private var workDoneProgress: WorkDoneProgressManager? + + init(sourceKitLSPServer: SourceKitLSPServer) { + self.sourceKitLSPServer = sourceKitLSPServer + } + + /// Called when a new file is scheduled to be indexed. Increments the target index count, eg. the 3 in `1/3`. + nonisolated func indexTaskWasQueued(count: Int) { + queue.async { + await self.indexTaskWasQueuedImpl(count: count) + } + } + + private func indexTaskWasQueuedImpl(count: Int) async { + queuedIndexTasks += count + await indexStatusDidChangeImpl() + } + + /// Called when a `SemanticIndexManager` finishes indexing a file. Adjusts the done index count, eg. the 1 in `1/3`. + nonisolated func indexStatusDidChange() { + queue.async { + await self.indexStatusDidChangeImpl() + } + } + + private func indexStatusDidChangeImpl() async { + guard let sourceKitLSPServer else { + workDoneProgress = nil + return + } + var scheduled: [DocumentURI] = [] + var executing: [DocumentURI] = [] + for indexManager in await sourceKitLSPServer.workspaces.compactMap({ $0.semanticIndexManager }) { + let inProgress = await indexManager.inProgressIndexTasks + scheduled += inProgress.scheduled + executing += inProgress.executing + } + + if scheduled.isEmpty && executing.isEmpty { + // Nothing left to index. Reset the target count and dismiss the work done progress. + queuedIndexTasks = 0 + workDoneProgress = nil + return + } + + let finishedTasks = queuedIndexTasks - scheduled.count - executing.count + let message = "\(finishedTasks) / \(queuedIndexTasks)" + + let percentage = Int(Double(finishedTasks) / Double(queuedIndexTasks) * 100) + if let workDoneProgress { + workDoneProgress.update(message: message, percentage: percentage) + } else { + workDoneProgress = await WorkDoneProgressManager( + server: sourceKitLSPServer, + title: "Indexing", + message: message, + percentage: percentage + ) + } + } +} diff --git a/Sources/SourceKitLSP/SourceKitLSPServer+Options.swift b/Sources/SourceKitLSP/SourceKitLSPServer+Options.swift index 747eaee62..b7005c989 100644 --- a/Sources/SourceKitLSP/SourceKitLSPServer+Options.swift +++ b/Sources/SourceKitLSP/SourceKitLSPServer+Options.swift @@ -49,6 +49,11 @@ extension SourceKitLSPServer { /// notification when running unit tests. public var swiftPublishDiagnosticsDebounceDuration: TimeInterval + /// A callback that is called when an index task finishes. + /// + /// Intended for testing purposes. + public var indexTaskDidFinish: (@Sendable () -> Void)? + public init( buildSetup: BuildSetup = .default, clangdOptions: [String] = [], diff --git a/Sources/SourceKitLSP/SourceKitLSPServer.swift b/Sources/SourceKitLSP/SourceKitLSPServer.swift index 14553c26a..6c82f148d 100644 --- a/Sources/SourceKitLSP/SourceKitLSPServer.swift +++ b/Sources/SourceKitLSP/SourceKitLSPServer.swift @@ -474,6 +474,12 @@ public actor SourceKitLSPServer { /// number of processor cores that the user allocated to background indexing. private let indexTaskScheduler: TaskScheduler + /// Implicitly unwrapped optional so we can create an `IndexProgressManager` that has a weak reference to + /// `SourceKitLSPServer`. + /// `nonisolated(unsafe)` because `indexProgressManager` will not be modified after it is assigned from the + /// initializer. + private nonisolated(unsafe) var indexProgressManager: IndexProgressManager! + private var packageLoadingWorkDoneProgress = WorkDoneProgressState( "SourceKitLSP.SourceKitLSPServer.reloadPackage", title: "SourceKit-LSP: Reloading Package" @@ -546,6 +552,8 @@ public actor SourceKitLSPServer { (TaskPriority.medium, processorCount), (TaskPriority.low, max(Int(lowPriorityCores), 1)), ]) + self.indexProgressManager = nil + self.indexProgressManager = IndexProgressManager(sourceKitLSPServer: self) } /// Await until the server has send the reply to the initialize request. @@ -1192,6 +1200,7 @@ extension SourceKitLSPServer { logger.log("Cannot open workspace before server is initialized") return nil } + let indexTaskDidFinishCallback = options.indexTaskDidFinish var options = self.options options.buildSetup = self.options.buildSetup.merging(buildSetup(for: workspaceFolder)) return try? await Workspace( @@ -1205,16 +1214,19 @@ extension SourceKitLSPServer { indexTaskScheduler: indexTaskScheduler, reloadPackageStatusCallback: { [weak self] status in guard let self else { return } - guard capabilityRegistry.clientCapabilities.window?.workDoneProgress ?? false else { - // Client doesn’t support work done progress - return - } switch status { case .start: await self.packageLoadingWorkDoneProgress.startProgress(server: self) case .end: await self.packageLoadingWorkDoneProgress.endProgress(server: self) } + }, + indexTasksWereScheduled: { [weak self] count in + self?.indexProgressManager.indexTaskWasQueued(count: count) + }, + indexTaskDidFinish: { [weak self] in + self?.indexProgressManager.indexStatusDidChange() + indexTaskDidFinishCallback?() } ) } @@ -1263,6 +1275,7 @@ extension SourceKitLSPServer { if self.workspaces.isEmpty { logger.error("no workspace found") + let indexTaskDidFinishCallback = self.options.indexTaskDidFinish let workspace = await Workspace( documentManager: self.documentManager, rootUri: req.rootURI, @@ -1272,7 +1285,14 @@ extension SourceKitLSPServer { underlyingBuildSystem: nil, index: nil, indexDelegate: nil, - indexTaskScheduler: self.indexTaskScheduler + indexTaskScheduler: self.indexTaskScheduler, + indexTasksWereScheduled: { [weak self] count in + self?.indexProgressManager.indexTaskWasQueued(count: count) + }, + indexTaskDidFinish: { [weak self] in + self?.indexProgressManager.indexStatusDidChange() + indexTaskDidFinishCallback?() + } ) self.workspacesAndIsImplicit.append((workspace: workspace, isImplicit: false)) diff --git a/Sources/SourceKitLSP/Workspace.swift b/Sources/SourceKitLSP/Workspace.swift index c89cdd53b..cd7a083f4 100644 --- a/Sources/SourceKitLSP/Workspace.swift +++ b/Sources/SourceKitLSP/Workspace.swift @@ -89,7 +89,9 @@ public final class Workspace: Sendable { underlyingBuildSystem: BuildSystem?, index uncheckedIndex: UncheckedIndex?, indexDelegate: SourceKitIndexDelegate?, - indexTaskScheduler: TaskScheduler + indexTaskScheduler: TaskScheduler, + indexTasksWereScheduled: @escaping @Sendable (Int) -> Void, + indexTaskDidFinish: @escaping @Sendable () -> Void ) async { self.documentManager = documentManager self.buildSetup = options.buildSetup @@ -107,7 +109,8 @@ public final class Workspace: Sendable { index: uncheckedIndex, buildSystemManager: buildSystemManager, indexTaskScheduler: indexTaskScheduler, - indexTaskDidFinish: options.indexOptions.indexTaskDidFinish + indexTasksWereScheduled: indexTasksWereScheduled, + indexTaskDidFinish: indexTaskDidFinish ) } else { self.semanticIndexManager = nil @@ -143,7 +146,9 @@ public final class Workspace: Sendable { compilationDatabaseSearchPaths: [RelativePath], indexOptions: IndexOptions = IndexOptions(), indexTaskScheduler: TaskScheduler, - reloadPackageStatusCallback: @Sendable @escaping (ReloadPackageStatus) async -> Void + reloadPackageStatusCallback: @Sendable @escaping (ReloadPackageStatus) async -> Void, + indexTasksWereScheduled: @Sendable @escaping (Int) -> Void, + indexTaskDidFinish: @Sendable @escaping () -> Void ) async throws { var buildSystem: BuildSystem? = nil @@ -247,7 +252,9 @@ public final class Workspace: Sendable { underlyingBuildSystem: buildSystem, index: UncheckedIndex(index), indexDelegate: indexDelegate, - indexTaskScheduler: indexTaskScheduler + indexTaskScheduler: indexTaskScheduler, + indexTasksWereScheduled: indexTasksWereScheduled, + indexTaskDidFinish: indexTaskDidFinish ) } @@ -303,19 +310,13 @@ public struct IndexOptions: Sendable { /// Setting this to a value < 1 ensures that background indexing doesn't use all CPU resources. public var maxCoresPercentageToUseForBackgroundIndexing: Double - /// A callback that is called when an index task finishes. - /// - /// Intended for testing purposes. - public var indexTaskDidFinish: (@Sendable () -> Void)? - public init( indexStorePath: AbsolutePath? = nil, indexDatabasePath: AbsolutePath? = nil, indexPrefixMappings: [PathPrefixMapping]? = nil, listenToUnitEvents: Bool = true, enableBackgroundIndexing: Bool = false, - maxCoresPercentageToUseForBackgroundIndexing: Double = 1, - indexTaskDidFinish: (@Sendable () -> Void)? = nil + maxCoresPercentageToUseForBackgroundIndexing: Double = 1 ) { self.indexStorePath = indexStorePath self.indexDatabasePath = indexDatabasePath @@ -323,6 +324,5 @@ public struct IndexOptions: Sendable { self.listenToUnitEvents = listenToUnitEvents self.enableBackgroundIndexing = enableBackgroundIndexing self.maxCoresPercentageToUseForBackgroundIndexing = maxCoresPercentageToUseForBackgroundIndexing - self.indexTaskDidFinish = indexTaskDidFinish } } diff --git a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift index 7ec0dd82a..3e1bbf702 100644 --- a/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift +++ b/Tests/SourceKitLSPTests/BackgroundIndexingTests.swift @@ -165,7 +165,7 @@ final class BackgroundIndexingTests: XCTestCase { func testBackgroundIndexingHappensWithLowPriority() async throws { var serverOptions = backgroundIndexingOptions - serverOptions.indexOptions.indexTaskDidFinish = { + serverOptions.indexTaskDidFinish = { XCTAssert( Task.currentPriority == .low, "An index task ran with priority \(Task.currentPriority)" @@ -328,4 +328,49 @@ final class BackgroundIndexingTests: XCTestCase { ] ) } + + func testBackgroundIndexingStatusWorkDoneProgress() async throws { + let workDoneProgressCreated = self.expectation(description: "Work done progress created") + let project = try await SwiftPMTestProject( + files: [ + "MyFile.swift": """ + func foo() {} + func bar() { + foo() + } + """ + ], + capabilities: ClientCapabilities(window: WindowClientCapabilities(workDoneProgress: true)), + serverOptions: backgroundIndexingOptions, + preInitialization: { testClient in + testClient.handleNextRequest { (request: CreateWorkDoneProgressRequest) in + workDoneProgressCreated.fulfill() + return VoidResponse() + } + } + ) + try await fulfillmentOfOrThrow([workDoneProgressCreated]) + let workBeginProgress = try await project.testClient.nextNotification(ofType: WorkDoneProgress.self) + guard case .begin = workBeginProgress.value else { + XCTFail("Expected begin work done progress") + return + } + var didGetEndWorkDoneProgress = false + for _ in 0..<3 { + let workEndProgress = try await project.testClient.nextNotification(ofType: WorkDoneProgress.self) + switch workEndProgress.value { + case .begin: + XCTFail("Unexpected begin work done progress") + case .report: + // Allow up to 2 work done progress reports. + continue + case .end: + didGetEndWorkDoneProgress = true + } + break + } + XCTAssert(didGetEndWorkDoneProgress, "Expected end work done progress") + + withExtendedLifetime(project) {} + } } diff --git a/Tests/SourceKitLSPTests/BuildSystemTests.swift b/Tests/SourceKitLSPTests/BuildSystemTests.swift index 501abd0b7..c3eb343c8 100644 --- a/Tests/SourceKitLSPTests/BuildSystemTests.swift +++ b/Tests/SourceKitLSPTests/BuildSystemTests.swift @@ -127,7 +127,9 @@ final class BuildSystemTests: XCTestCase { underlyingBuildSystem: buildSystem, index: nil, indexDelegate: nil, - indexTaskScheduler: .forTesting + indexTaskScheduler: .forTesting, + indexTasksWereScheduled: { _ in }, + indexTaskDidFinish: {} ) await server.setWorkspaces([(workspace: workspace, isImplicit: false)]) From 365e3a5449fb114c97c4cc60b07e7eac3b261854 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Tue, 14 May 2024 08:47:03 -0700 Subject: [PATCH 41/46] Guard `@retroactive` for `CommandConfiguration` behind `#if compiler(>=5.11)` Fixes a build failure when building SourceKit-LSP using Xcode 15.4. --- Sources/Diagnose/CommandConfiguration+Sendable.swift | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/Sources/Diagnose/CommandConfiguration+Sendable.swift b/Sources/Diagnose/CommandConfiguration+Sendable.swift index 8277251e1..0504351e5 100644 --- a/Sources/Diagnose/CommandConfiguration+Sendable.swift +++ b/Sources/Diagnose/CommandConfiguration+Sendable.swift @@ -14,4 +14,8 @@ import ArgumentParser // If `CommandConfiguration` is not sendable, commands can't have static `configuration` properties. // Needed until we update Swift CI to swift-argument-parser 1.3.1, which has this conformance (rdar://128042447). +#if compiler(<5.11) +extension CommandConfiguration: @unchecked Sendable {} +#else extension CommandConfiguration: @unchecked @retroactive Sendable {} +#endif From eec5455e887fad6a0e74508811e95d87a4a1dcd7 Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Tue, 14 May 2024 09:49:33 -0700 Subject: [PATCH 42/46] Add a command-line option to enable background indexing This allows me to test background indexing more easily locally as it matures. It is still experimental and may contain bugs. Fixes #1256 rdar://127474761 --- Sources/sourcekit-lsp/SourceKitLSP.swift | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/Sources/sourcekit-lsp/SourceKitLSP.swift b/Sources/sourcekit-lsp/SourceKitLSP.swift index dc2f7cb1a..89f768110 100644 --- a/Sources/sourcekit-lsp/SourceKitLSP.swift +++ b/Sources/sourcekit-lsp/SourceKitLSP.swift @@ -200,6 +200,11 @@ struct SourceKitLSP: AsyncParsableCommand { ) var completionMaxResults = 200 + @Flag( + help: "Enable background indexing. This feature is still under active development and may be incomplete." + ) + var enableExperimentalBackgroundIndexing = false + func mapOptions() -> SourceKitLSPServer.Options { var serverOptions = SourceKitLSPServer.Options() @@ -215,6 +220,7 @@ struct SourceKitLSP: AsyncParsableCommand { serverOptions.indexOptions.indexStorePath = indexStorePath serverOptions.indexOptions.indexDatabasePath = indexDatabasePath serverOptions.indexOptions.indexPrefixMappings = indexPrefixMappings + serverOptions.indexOptions.enableBackgroundIndexing = enableExperimentalBackgroundIndexing serverOptions.completionOptions.maxResults = completionMaxResults serverOptions.generatedInterfacesPath = generatedInterfacesPath From 7baffde1db463e9c874098c1eb586b5fbe79ac1f Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Tue, 14 May 2024 10:02:35 -0700 Subject: [PATCH 43/46] =?UTF-8?q?Change=20compiler=E2=80=99s=20diagnostic?= =?UTF-8?q?=20provider=20name=20to=20`SourceKit`?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The fact that they are coming from a service named `sourcekitd` should be an implentation detail of SourceKit-LSP and shouldn’t be exposed to users. Use the generic `SourceKit` term, which is vague about which SourceKit the diagnostics are coming from. --- Sources/SourceKitLSP/Swift/Diagnostic.swift | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Sources/SourceKitLSP/Swift/Diagnostic.swift b/Sources/SourceKitLSP/Swift/Diagnostic.swift index 136cc7540..f1c41fa72 100644 --- a/Sources/SourceKitLSP/Swift/Diagnostic.swift +++ b/Sources/SourceKitLSP/Swift/Diagnostic.swift @@ -263,7 +263,7 @@ extension Diagnostic { severity: severity, code: code, codeDescription: codeDescription, - source: "sourcekitd", + source: "SourceKit", message: message, tags: tags, relatedInformation: notes, @@ -297,7 +297,7 @@ extension Diagnostic { severity: diag.diagMessage.severity.lspSeverity, code: nil, codeDescription: nil, - source: "SwiftSyntax", + source: "SourceKit", message: diag.message, tags: nil, relatedInformation: relatedInformation, From 280a50e29396cc818d1b68a470f4aa983cf2fcae Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Tue, 14 May 2024 15:36:18 -0700 Subject: [PATCH 44/46] Adjustment because `packages` is not a parameter of `ModulesGraph` in `release/6.0` --- Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift | 1 - 1 file changed, 1 deletion(-) diff --git a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift index 2ec951f09..69497b842 100644 --- a/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift +++ b/Sources/SKSwiftPMWorkspace/SwiftPMBuildSystem.swift @@ -204,7 +204,6 @@ public actor SwiftPMBuildSystem { self.modulesGraph = try ModulesGraph( rootPackages: [], - packages: IdentifiableSet(), dependencies: [], binaryArtifacts: [:] ) From 0783c9e6e6acb96584d698926330d8e6038e5bdf Mon Sep 17 00:00:00 2001 From: Alex Hoppen Date: Tue, 14 May 2024 15:37:15 -0700 Subject: [PATCH 45/46] Revert "Replace `IncrementalEdit` with `SourceEdit`" This reverts commit ed6163087586eb27382f693d1924aa6d8e882a05. --- .../SourceKitLSP/Swift/DocumentFormatting.swift | 14 ++++++++++++-- .../SourceKitLSP/Swift/SwiftLanguageService.swift | 10 ++++++++-- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/Sources/SourceKitLSP/Swift/DocumentFormatting.swift b/Sources/SourceKitLSP/Swift/DocumentFormatting.swift index cfe256ec8..1b0028948 100644 --- a/Sources/SourceKitLSP/Swift/DocumentFormatting.swift +++ b/Sources/SourceKitLSP/Swift/DocumentFormatting.swift @@ -115,8 +115,18 @@ private func edits(from original: DocumentSnapshot, to edited: String) -> [TextE // Map the offset-based edits to line-column based edits to be consumed by LSP - return concurrentEdits.edits.compactMap { - TextEdit(range: original.absolutePositionRange(of: $0.range), newText: $0.replacement) + return concurrentEdits.edits.compactMap { (edit) -> TextEdit? in + let (startLine, startColumn) = original.lineTable.lineAndUTF16ColumnOf(utf8Offset: edit.offset) + let (endLine, endColumn) = original.lineTable.lineAndUTF16ColumnOf(utf8Offset: edit.endOffset) + guard let newText = String(bytes: edit.replacement, encoding: .utf8) else { + logger.fault("Failed to get String from UTF-8 bytes \(edit.replacement)") + return nil + } + + return TextEdit( + range: Position(line: startLine, utf16index: startColumn).. Date: Tue, 14 May 2024 15:37:19 -0700 Subject: [PATCH 46/46] Revert "Fix deprecated `ByteSourceRange`" This reverts commit c5699fb4dde964283c18c97255ac426bcf7f6439. --- Sources/SourceKitLSP/Rename.swift | 2 +- .../Swift/CodeActions/PackageManifestEdits.swift | 2 +- .../SyntaxRefactoringCodeActionProvider.swift | 2 +- Sources/SourceKitLSP/Swift/Diagnostic.swift | 6 +++--- Sources/SourceKitLSP/Swift/DocumentSymbols.swift | 8 +++----- Sources/SourceKitLSP/Swift/SemanticTokens.swift | 4 ++-- Sources/SourceKitLSP/Swift/SwiftLanguageService.swift | 11 ++++------- Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift | 8 ++------ Sources/SourceKitLSP/TestDiscovery.swift | 6 ++---- 9 files changed, 19 insertions(+), 30 deletions(-) diff --git a/Sources/SourceKitLSP/Rename.swift b/Sources/SourceKitLSP/Rename.swift index ee01c3231..29a8f723c 100644 --- a/Sources/SourceKitLSP/Rename.swift +++ b/Sources/SourceKitLSP/Rename.swift @@ -1008,7 +1008,7 @@ extension SwiftLanguageService { } if let startToken, let endToken { - return snapshot.absolutePositionRange( + return snapshot.range( of: startToken.positionAfterSkippingLeadingTrivia.. TextEdit? in let edit = TextEdit( - range: scope.snapshot.absolutePositionRange(of: edit.range), + range: scope.snapshot.range(of: edit.range), newText: edit.replacement ) if edit.isNoOp(in: scope.snapshot) { diff --git a/Sources/SourceKitLSP/Swift/Diagnostic.swift b/Sources/SourceKitLSP/Swift/Diagnostic.swift index f1c41fa72..e290b30be 100644 --- a/Sources/SourceKitLSP/Swift/Diagnostic.swift +++ b/Sources/SourceKitLSP/Swift/Diagnostic.swift @@ -61,7 +61,7 @@ extension CodeAction { init?(_ fixIt: FixIt, in snapshot: DocumentSnapshot) { var textEdits = [TextEdit]() for edit in fixIt.edits { - textEdits.append(TextEdit(range: snapshot.absolutePositionRange(of: edit.range), newText: edit.replacement)) + textEdits.append(TextEdit(range: snapshot.range(of: edit.range), newText: edit.replacement)) } self.init( @@ -281,7 +281,7 @@ extension Diagnostic { var range = Range(snapshot.position(of: diag.position)) for highlight in diag.highlights { let swiftSyntaxRange = highlight.positionAfterSkippingLeadingTrivia.., callerFile: StaticString = #fileID, callerLine: UInt = #line @@ -1169,12 +1169,9 @@ extension DocumentSnapshot { of range: Range, callerFile: StaticString = #fileID, callerLine: UInt = #line - ) -> Range { + ) -> ByteSourceRange { let utf8OffsetRange = utf8OffsetRange(of: range, callerFile: callerFile, callerLine: callerLine) - return Range( - position: AbsolutePosition(utf8Offset: utf8OffsetRange.startIndex), - length: SourceLength(utf8Length: utf8OffsetRange.count) - ) + return ByteSourceRange(offset: utf8OffsetRange.startIndex, length: utf8OffsetRange.count) } // MARK: Position <-> RenameLocation diff --git a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift index 31f640fef..9a5c3e50b 100644 --- a/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift +++ b/Sources/SourceKitLSP/Swift/SwiftTestingScanner.swift @@ -248,9 +248,7 @@ final class SyntacticSwiftTestingTestScanner: SyntaxVisitor { return .skipChildren } - let range = snapshot.absolutePositionRange( - of: node.positionAfterSkippingLeadingTrivia..