1//===--- TUScheduler.cpp -----------------------------------------*-C++-*-===//
2//
3// Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
4// See https://llvm.org/LICENSE.txt for license information.
5// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
6//
7//===----------------------------------------------------------------------===//
8// TUScheduler manages a worker per active file. This ASTWorker processes
9// updates (modifications to file contents) and reads (actions performed on
10// preamble/AST) to the file.
11//
12// Each ASTWorker owns a dedicated thread to process updates and reads to the
13// relevant file. Any request gets queued in FIFO order to be processed by that
14// thread.
15//
16// An update request replaces current praser inputs to ensure any subsequent
17// read sees the version of the file they were requested. It will also issue a
18// build for new inputs.
19//
20// ASTWorker processes the file in two parts, a preamble and a main-file
21// section. A preamble can be reused between multiple versions of the file until
22// invalidated by a modification to a header, compile commands or modification
23// to relevant part of the current file. Such a preamble is called compatible.
24// An update is considered dead if no read was issued for that version and
25// diagnostics weren't requested by client or could be generated for a later
26// version of the file. ASTWorker eliminates such requests as they are
27// redundant.
28//
29// In the presence of stale (non-compatible) preambles, ASTWorker won't publish
30// diagnostics for update requests. Read requests will be served with ASTs build
31// with stale preambles, unless the read is picky and requires a compatible
32// preamble. In such cases it will block until new preamble is built.
33//
34// ASTWorker owns a PreambleThread for building preambles. If the preamble gets
35// invalidated by an update request, a new build will be requested on
36// PreambleThread. Since PreambleThread only receives requests for newer
37// versions of the file, in case of multiple requests it will only build the
38// last one and skip requests in between. Unless client force requested
39// diagnostics(WantDiagnostics::Yes).
40//
41// When a new preamble is built, a "golden" AST is immediately built from that
42// version of the file. This ensures diagnostics get updated even if the queue
43// is full.
44//
45// Some read requests might just need preamble. Since preambles can be read
46// concurrently, ASTWorker runs these requests on their own thread. These
47// requests will receive latest build preamble, which might possibly be stale.
48
49#include "TUScheduler.h"
50#include "CompileCommands.h"
51#include "Compiler.h"
52#include "Config.h"
53#include "Diagnostics.h"
54#include "GlobalCompilationDatabase.h"
55#include "ParsedAST.h"
56#include "Preamble.h"
57#include "clang-include-cleaner/Record.h"
58#include "support/Cancellation.h"
59#include "support/Context.h"
60#include "support/Logger.h"
61#include "support/MemoryTree.h"
62#include "support/Path.h"
63#include "support/ThreadCrashReporter.h"
64#include "support/Threading.h"
65#include "support/Trace.h"
66#include "clang/Basic/Stack.h"
67#include "clang/Frontend/CompilerInvocation.h"
68#include "clang/Tooling/CompilationDatabase.h"
69#include "llvm/ADT/FunctionExtras.h"
70#include "llvm/ADT/STLExtras.h"
71#include "llvm/ADT/ScopeExit.h"
72#include "llvm/ADT/SmallVector.h"
73#include "llvm/ADT/StringExtras.h"
74#include "llvm/ADT/StringRef.h"
75#include "llvm/Support/Allocator.h"
76#include "llvm/Support/Errc.h"
77#include "llvm/Support/ErrorHandling.h"
78#include "llvm/Support/FormatVariadic.h"
79#include "llvm/Support/Path.h"
80#include "llvm/Support/Threading.h"
81#include "llvm/Support/raw_ostream.h"
82#include <algorithm>
83#include <atomic>
84#include <chrono>
85#include <condition_variable>
86#include <functional>
87#include <memory>
88#include <mutex>
89#include <optional>
90#include <queue>
91#include <string>
92#include <thread>
93#include <type_traits>
94#include <utility>
95#include <vector>
96
97namespace clang {
98namespace clangd {
99using std::chrono::steady_clock;
100
101namespace {
102// Tracks latency (in seconds) of FS operations done during a preamble build.
103// build_type allows to split by expected VFS cache state (cold on first
104// preamble, somewhat warm after that when building first preamble for new file,
105// likely ~everything cached on preamble rebuild.
106constexpr trace::Metric
107 PreambleBuildFilesystemLatency("preamble_fs_latency",
108 trace::Metric::Distribution, "build_type");
109// Tracks latency of FS operations done during a preamble build as a ratio of
110// preamble build time. build_type is same as above.
111constexpr trace::Metric PreambleBuildFilesystemLatencyRatio(
112 "preamble_fs_latency_ratio", trace::Metric::Distribution, "build_type");
113
114constexpr trace::Metric PreambleBuildSize("preamble_build_size",
115 trace::Metric::Distribution);
116constexpr trace::Metric PreambleSerializedSize("preamble_serialized_size",
117 trace::Metric::Distribution);
118
119void reportPreambleBuild(const PreambleBuildStats &Stats,
120 bool IsFirstPreamble) {
121 auto RecordWithLabel = [&Stats](llvm::StringRef Label) {
122 PreambleBuildFilesystemLatency.record(Value: Stats.FileSystemTime, Label);
123 if (Stats.TotalBuildTime > 0) // Avoid division by zero.
124 PreambleBuildFilesystemLatencyRatio.record(
125 Value: Stats.FileSystemTime / Stats.TotalBuildTime, Label);
126 };
127
128 static llvm::once_flag OnceFlag;
129 llvm::call_once(flag&: OnceFlag, F: [&] { RecordWithLabel("first_build"); });
130 RecordWithLabel(IsFirstPreamble ? "first_build_for_file" : "rebuild");
131
132 PreambleBuildSize.record(Value: Stats.BuildSize);
133 PreambleSerializedSize.record(Value: Stats.SerializedSize);
134}
135
136class ASTWorker;
137} // namespace
138
139static clang::clangd::Key<std::string> FileBeingProcessed;
140
141std::optional<llvm::StringRef> TUScheduler::getFileBeingProcessedInContext() {
142 if (auto *File = Context::current().get(Key: FileBeingProcessed))
143 return llvm::StringRef(*File);
144 return std::nullopt;
145}
146
147/// An LRU cache of idle ASTs.
148/// Because we want to limit the overall number of these we retain, the cache
149/// owns ASTs (and may evict them) while their workers are idle.
150/// Workers borrow ASTs when active, and return them when done.
151class TUScheduler::ASTCache {
152public:
153 using Key = const ASTWorker *;
154
155 ASTCache(unsigned MaxRetainedASTs) : MaxRetainedASTs(MaxRetainedASTs) {}
156
157 /// Returns result of getUsedBytes() for the AST cached by \p K.
158 /// If no AST is cached, 0 is returned.
159 std::size_t getUsedBytes(Key K) {
160 std::lock_guard<std::mutex> Lock(Mut);
161 auto It = findByKey(K);
162 if (It == LRU.end() || !It->second)
163 return 0;
164 return It->second->getUsedBytes();
165 }
166
167 /// Store the value in the pool, possibly removing the last used AST.
168 /// The value should not be in the pool when this function is called.
169 void put(Key K, std::unique_ptr<ParsedAST> V) {
170 std::unique_lock<std::mutex> Lock(Mut);
171 assert(findByKey(K) == LRU.end());
172
173 LRU.insert(position: LRU.begin(), x: {K, std::move(V)});
174 if (LRU.size() <= MaxRetainedASTs)
175 return;
176 // We're past the limit, remove the last element.
177 std::unique_ptr<ParsedAST> ForCleanup = std::move(LRU.back().second);
178 LRU.pop_back();
179 // Run the expensive destructor outside the lock.
180 Lock.unlock();
181 ForCleanup.reset();
182 }
183
184 /// Returns the cached value for \p K, or std::nullopt if the value is not in
185 /// the cache anymore. If nullptr was cached for \p K, this function will
186 /// return a null unique_ptr wrapped into an optional.
187 /// If \p AccessMetric is set records whether there was a hit or miss.
188 std::optional<std::unique_ptr<ParsedAST>>
189 take(Key K, const trace::Metric *AccessMetric = nullptr) {
190 // Record metric after unlocking the mutex.
191 std::unique_lock<std::mutex> Lock(Mut);
192 auto Existing = findByKey(K);
193 if (Existing == LRU.end()) {
194 if (AccessMetric)
195 AccessMetric->record(Value: 1, Label: "miss");
196 return std::nullopt;
197 }
198 if (AccessMetric)
199 AccessMetric->record(Value: 1, Label: "hit");
200 std::unique_ptr<ParsedAST> V = std::move(Existing->second);
201 LRU.erase(position: Existing);
202 // GCC 4.8 fails to compile `return V;`, as it tries to call the copy
203 // constructor of unique_ptr, so we call the move ctor explicitly to avoid
204 // this miscompile.
205 return std::optional<std::unique_ptr<ParsedAST>>(std::move(V));
206 }
207
208private:
209 using KVPair = std::pair<Key, std::unique_ptr<ParsedAST>>;
210
211 std::vector<KVPair>::iterator findByKey(Key K) {
212 return llvm::find_if(Range&: LRU, P: [K](const KVPair &P) { return P.first == K; });
213 }
214
215 std::mutex Mut;
216 unsigned MaxRetainedASTs;
217 /// Items sorted in LRU order, i.e. first item is the most recently accessed
218 /// one.
219 std::vector<KVPair> LRU; /* GUARDED_BY(Mut) */
220};
221
222/// A map from header files to an opened "proxy" file that includes them.
223/// If you open the header, the compile command from the proxy file is used.
224///
225/// This inclusion information could also naturally live in the index, but there
226/// are advantages to using open files instead:
227/// - it's easier to achieve a *stable* choice of proxy, which is important
228/// to avoid invalidating the preamble
229/// - context-sensitive flags for libraries with multiple configurations
230/// (e.g. C++ stdlib sensitivity to -std version)
231/// - predictable behavior, e.g. guarantees that go-to-def landing on a header
232/// will have a suitable command available
233/// - fewer scaling problems to solve (project include graphs are big!)
234///
235/// Implementation details:
236/// - We only record this for mainfiles where the command was trustworthy
237/// (i.e. not inferred). This avoids a bad inference "infecting" other files.
238/// - Once we've picked a proxy file for a header, we stick with it until the
239/// proxy file is invalidated *and* a new candidate proxy file is built.
240/// Switching proxies is expensive, as the compile flags will (probably)
241/// change and therefore we'll end up rebuilding the header's preamble.
242/// - We don't capture the actual compile command, but just the filename we
243/// should query to get it. This avoids getting out of sync with the CDB.
244///
245/// All methods are threadsafe. In practice, update() comes from preamble
246/// threads, remove()s mostly from the main thread, and get() from ASTWorker.
247/// Writes are rare and reads are cheap, so we don't expect much contention.
248class TUScheduler::HeaderIncluderCache {
249 // We should be a little careful how we store the include graph of open
250 // files, as each can have a large number of transitive headers.
251 // This representation is O(unique transitive source files).
252 llvm::BumpPtrAllocator Arena;
253 struct Association {
254 llvm::StringRef MainFile;
255 // Circular-linked-list of associations with the same mainFile.
256 // Null indicates that the mainfile was removed.
257 Association *Next;
258 };
259 llvm::StringMap<Association, llvm::BumpPtrAllocator &> HeaderToMain;
260 llvm::StringMap<Association *, llvm::BumpPtrAllocator &> MainToFirst;
261 std::atomic<size_t> UsedBytes; // Updated after writes.
262 mutable std::mutex Mu;
263
264 void invalidate(Association *First) {
265 Association *Current = First;
266 do {
267 Association *Next = Current->Next;
268 Current->Next = nullptr;
269 Current = Next;
270 } while (Current != First);
271 }
272
273 // Create the circular list and return the head of it.
274 Association *associate(llvm::StringRef MainFile,
275 llvm::ArrayRef<std::string> Headers) {
276 Association *First = nullptr, *Prev = nullptr;
277 for (const std::string &Header : Headers) {
278 auto &Assoc = HeaderToMain[Header];
279 if (Assoc.Next)
280 continue; // Already has a valid association.
281
282 Assoc.MainFile = MainFile;
283 Assoc.Next = Prev;
284 Prev = &Assoc;
285 if (!First)
286 First = &Assoc;
287 }
288 if (First)
289 First->Next = Prev;
290 return First;
291 }
292
293 void updateMemoryUsage() {
294 auto StringMapHeap = [](const auto &Map) {
295 // StringMap stores the hashtable on the heap.
296 // It contains pointers to the entries, and a hashcode for each.
297 return Map.getNumBuckets() * (sizeof(void *) + sizeof(unsigned));
298 };
299 size_t Usage = Arena.getTotalMemory() + StringMapHeap(MainToFirst) +
300 StringMapHeap(HeaderToMain) + sizeof(*this);
301 UsedBytes.store(i: Usage, m: std::memory_order_release);
302 }
303
304public:
305 HeaderIncluderCache() : HeaderToMain(Arena), MainToFirst(Arena) {
306 updateMemoryUsage();
307 }
308
309 // Associate each header with MainFile (unless already associated).
310 // Headers not in the list will have their associations removed.
311 void update(PathRef MainFile, llvm::ArrayRef<std::string> Headers) {
312 std::lock_guard<std::mutex> Lock(Mu);
313 auto It = MainToFirst.try_emplace(Key: MainFile, Args: nullptr);
314 Association *&First = It.first->second;
315 if (First)
316 invalidate(First);
317 First = associate(MainFile: It.first->first(), Headers);
318 updateMemoryUsage();
319 }
320
321 // Mark MainFile as gone.
322 // This will *not* disassociate headers with MainFile immediately, but they
323 // will be eligible for association with other files that get update()d.
324 void remove(PathRef MainFile) {
325 std::lock_guard<std::mutex> Lock(Mu);
326 Association *&First = MainToFirst[MainFile];
327 if (First) {
328 invalidate(First);
329 First = nullptr;
330 }
331 // MainToFirst entry should stay alive, as Associations might be pointing at
332 // its key.
333 }
334
335 /// Get the mainfile associated with Header, or the empty string if none.
336 std::string get(PathRef Header) const {
337 std::lock_guard<std::mutex> Lock(Mu);
338 return HeaderToMain.lookup(Key: Header).MainFile.str();
339 }
340
341 size_t getUsedBytes() const {
342 return UsedBytes.load(m: std::memory_order_acquire);
343 }
344};
345
346namespace {
347
348bool isReliable(const tooling::CompileCommand &Cmd) {
349 return Cmd.Heuristic.empty();
350}
351
352/// Threadsafe manager for updating a TUStatus and emitting it after each
353/// update.
354class SynchronizedTUStatus {
355public:
356 SynchronizedTUStatus(PathRef FileName, ParsingCallbacks &Callbacks)
357 : FileName(FileName), Callbacks(Callbacks) {}
358
359 void update(llvm::function_ref<void(TUStatus &)> Mutator) {
360 std::lock_guard<std::mutex> Lock(StatusMu);
361 Mutator(Status);
362 emitStatusLocked();
363 }
364
365 /// Prevents emitting of further updates.
366 void stop() {
367 std::lock_guard<std::mutex> Lock(StatusMu);
368 CanPublish = false;
369 }
370
371private:
372 void emitStatusLocked() {
373 if (CanPublish)
374 Callbacks.onFileUpdated(File: FileName, Status);
375 }
376
377 const Path FileName;
378
379 std::mutex StatusMu;
380 TUStatus Status;
381 bool CanPublish = true;
382 ParsingCallbacks &Callbacks;
383};
384
385// An attempt to acquire resources for a task using PreambleThrottler.
386// Initially it is unsatisfied, it (hopefully) becomes satisfied later but may
387// be destroyed before then. Destruction releases all resources.
388class PreambleThrottlerRequest {
389public:
390 // The condition variable is signalled when the request is satisfied.
391 PreambleThrottlerRequest(llvm::StringRef Filename,
392 PreambleThrottler *Throttler,
393 std::condition_variable &CV)
394 : Throttler(Throttler),
395 Satisfied(Throttler == nullptr) {
396 // If there is no throttler, this dummy request is always satisfied.
397 if (!Throttler)
398 return;
399 ID = Throttler->acquire(Filename, [&] {
400 Satisfied.store(i: true, m: std::memory_order_release);
401 CV.notify_all();
402 });
403 }
404
405 bool satisfied() const { return Satisfied.load(m: std::memory_order_acquire); }
406
407 // When the request is destroyed:
408 // - if resources are not yet obtained, stop trying to get them.
409 // - if resources were obtained, release them.
410 ~PreambleThrottlerRequest() {
411 if (Throttler)
412 Throttler->release(ID);
413 }
414 PreambleThrottlerRequest(const PreambleThrottlerRequest &) = delete;
415 PreambleThrottlerRequest &
416 operator=(const PreambleThrottlerRequest &) = delete;
417
418private:
419 PreambleThrottler::RequestID ID;
420 PreambleThrottler *Throttler;
421 std::atomic<bool> Satisfied = {false};
422};
423
424/// Responsible for building preambles. Whenever the thread is idle and the
425/// preamble is outdated, it starts to build a fresh preamble from the latest
426/// inputs. If RunSync is true, preambles are built synchronously in update()
427/// instead.
428class PreambleThread {
429public:
430 PreambleThread(llvm::StringRef FileName, ParsingCallbacks &Callbacks,
431 bool StorePreambleInMemory, bool RunSync,
432 PreambleThrottler *Throttler, SynchronizedTUStatus &Status,
433 TUScheduler::HeaderIncluderCache &HeaderIncluders,
434 ASTWorker &AW)
435 : FileName(FileName), Callbacks(Callbacks),
436 StoreInMemory(StorePreambleInMemory), RunSync(RunSync),
437 Throttler(Throttler), Status(Status), ASTPeer(AW),
438 HeaderIncluders(HeaderIncluders) {}
439
440 /// It isn't guaranteed that each requested version will be built. If there
441 /// are multiple update requests while building a preamble, only the last one
442 /// will be built.
443 void update(std::unique_ptr<CompilerInvocation> CI, ParseInputs PI,
444 std::vector<Diag> CIDiags, WantDiagnostics WantDiags) {
445 Request Req = {.CI: std::move(CI), .Inputs: std::move(PI), .CIDiags: std::move(CIDiags), .WantDiags: WantDiags,
446 .Ctx: Context::current().clone()};
447 if (RunSync) {
448 build(Req: std::move(Req));
449 Status.update(Mutator: [](TUStatus &Status) {
450 Status.PreambleActivity = PreambleAction::Idle;
451 });
452 return;
453 }
454 {
455 std::unique_lock<std::mutex> Lock(Mutex);
456 // If NextReq was requested with WantDiagnostics::Yes we cannot just drop
457 // that on the floor. Block until we start building it. This won't
458 // dead-lock as we are blocking the caller thread, while builds continue
459 // on preamble thread.
460 ReqCV.wait(lock&: Lock, p: [this] {
461 return !NextReq || NextReq->WantDiags != WantDiagnostics::Yes;
462 });
463 NextReq = std::move(Req);
464 }
465 // Let the worker thread know there's a request, notify_one is safe as there
466 // should be a single worker thread waiting on it.
467 ReqCV.notify_all();
468 }
469
470 void run() {
471 // We mark the current as the stack bottom so that clang running on this
472 // thread can notice the stack usage and prevent stack overflow with best
473 // efforts. Same applies to other calls thoughout clangd.
474 clang::noteBottomOfStack();
475 while (true) {
476 std::optional<PreambleThrottlerRequest> Throttle;
477 {
478 std::unique_lock<std::mutex> Lock(Mutex);
479 assert(!CurrentReq && "Already processing a request?");
480 // Wait until stop is called or there is a request.
481 ReqCV.wait(lock&: Lock, p: [&] { return NextReq || Done; });
482 if (Done)
483 break;
484
485 {
486 Throttle.emplace(args: FileName, args&: Throttler, args&: ReqCV);
487 std::optional<trace::Span> Tracer;
488 // If acquire succeeded synchronously, avoid status jitter.
489 if (!Throttle->satisfied()) {
490 Tracer.emplace(args: "PreambleThrottle");
491 Status.update(Mutator: [&](TUStatus &Status) {
492 Status.PreambleActivity = PreambleAction::Queued;
493 });
494 }
495 ReqCV.wait(lock&: Lock, p: [&] { return Throttle->satisfied() || Done; });
496 }
497 if (Done)
498 break;
499 // While waiting for the throttler, the request may have been updated!
500 // That's fine though, there's still guaranteed to be some request.
501
502 CurrentReq = std::move(*NextReq);
503 NextReq.reset();
504 }
505
506 {
507 WithContext Guard(std::move(CurrentReq->Ctx));
508 // Note that we don't make use of the ContextProvider here.
509 // Preamble tasks are always scheduled by ASTWorker tasks, and we
510 // reuse the context/config that was created at that level.
511
512 // Build the preamble and let the waiters know about it.
513 build(Req: std::move(*CurrentReq));
514 }
515 // Releasing the throttle before destroying the request assists testing.
516 Throttle.reset();
517 bool IsEmpty = false;
518 {
519 std::lock_guard<std::mutex> Lock(Mutex);
520 CurrentReq.reset();
521 IsEmpty = !NextReq;
522 }
523 if (IsEmpty) {
524 // We don't perform this above, before waiting for a request to make
525 // tests more deterministic. As there can be a race between this thread
526 // and client thread(clangdserver).
527 Status.update(Mutator: [](TUStatus &Status) {
528 Status.PreambleActivity = PreambleAction::Idle;
529 });
530 }
531 ReqCV.notify_all();
532 }
533 dlog("Preamble worker for {0} stopped", FileName);
534 }
535
536 /// Signals the run loop to exit.
537 void stop() {
538 dlog("Preamble worker for {0} received stop", FileName);
539 {
540 std::lock_guard<std::mutex> Lock(Mutex);
541 Done = true;
542 NextReq.reset();
543 }
544 // Let the worker thread know that it should stop.
545 ReqCV.notify_all();
546 }
547
548 bool blockUntilIdle(Deadline Timeout) const {
549 std::unique_lock<std::mutex> Lock(Mutex);
550 return wait(Lock, CV&: ReqCV, D: Timeout, F: [&] { return !NextReq && !CurrentReq; });
551 }
552
553private:
554 /// Holds inputs required for building a preamble. CI is guaranteed to be
555 /// non-null.
556 struct Request {
557 std::unique_ptr<CompilerInvocation> CI;
558 ParseInputs Inputs;
559 std::vector<Diag> CIDiags;
560 WantDiagnostics WantDiags;
561 Context Ctx;
562 };
563
564 bool isDone() {
565 std::lock_guard<std::mutex> Lock(Mutex);
566 return Done;
567 }
568
569 /// Builds a preamble for \p Req, might reuse LatestBuild if possible.
570 /// Notifies ASTWorker after build finishes.
571 void build(Request Req);
572
573 mutable std::mutex Mutex;
574 bool Done = false; /* GUARDED_BY(Mutex) */
575 std::optional<Request> NextReq; /* GUARDED_BY(Mutex) */
576 std::optional<Request> CurrentReq; /* GUARDED_BY(Mutex) */
577 // Signaled whenever a thread populates NextReq or worker thread builds a
578 // Preamble.
579 mutable std::condition_variable ReqCV; /* GUARDED_BY(Mutex) */
580 // Accessed only by preamble thread.
581 std::shared_ptr<const PreambleData> LatestBuild;
582
583 const Path FileName;
584 ParsingCallbacks &Callbacks;
585 const bool StoreInMemory;
586 const bool RunSync;
587 PreambleThrottler *Throttler;
588
589 SynchronizedTUStatus &Status;
590 ASTWorker &ASTPeer;
591 TUScheduler::HeaderIncluderCache &HeaderIncluders;
592};
593
594class ASTWorkerHandle;
595
596/// Owns one instance of the AST, schedules updates and reads of it.
597/// Also responsible for building and providing access to the preamble.
598/// Each ASTWorker processes the async requests sent to it on a separate
599/// dedicated thread.
600/// The ASTWorker that manages the AST is shared by both the processing thread
601/// and the TUScheduler. The TUScheduler should discard an ASTWorker when
602/// remove() is called, but its thread may be busy and we don't want to block.
603/// So the workers are accessed via an ASTWorkerHandle. Destroying the handle
604/// signals the worker to exit its run loop and gives up shared ownership of the
605/// worker.
606class ASTWorker {
607 friend class ASTWorkerHandle;
608 ASTWorker(PathRef FileName, const GlobalCompilationDatabase &CDB,
609 TUScheduler::ASTCache &LRUCache,
610 TUScheduler::HeaderIncluderCache &HeaderIncluders,
611 Semaphore &Barrier, bool RunSync, const TUScheduler::Options &Opts,
612 ParsingCallbacks &Callbacks);
613
614public:
615 /// Create a new ASTWorker and return a handle to it.
616 /// The processing thread is spawned using \p Tasks. However, when \p Tasks
617 /// is null, all requests will be processed on the calling thread
618 /// synchronously instead. \p Barrier is acquired when processing each
619 /// request, it is used to limit the number of actively running threads.
620 static ASTWorkerHandle
621 create(PathRef FileName, const GlobalCompilationDatabase &CDB,
622 TUScheduler::ASTCache &IdleASTs,
623 TUScheduler::HeaderIncluderCache &HeaderIncluders,
624 AsyncTaskRunner *Tasks, Semaphore &Barrier,
625 const TUScheduler::Options &Opts, ParsingCallbacks &Callbacks);
626 ~ASTWorker();
627 ASTWorker(const ASTWorker &other) = delete;
628 ASTWorker &operator=(const ASTWorker &other) = delete;
629 void update(ParseInputs Inputs, WantDiagnostics, bool ContentChanged);
630 void
631 runWithAST(llvm::StringRef Name,
632 llvm::unique_function<void(llvm::Expected<InputsAndAST>)> Action,
633 TUScheduler::ASTActionInvalidation);
634 bool blockUntilIdle(Deadline Timeout) const;
635
636 std::shared_ptr<const PreambleData> getPossiblyStalePreamble(
637 std::shared_ptr<const ASTSignals> *ASTSignals = nullptr) const;
638
639 /// Used to inform ASTWorker about a new preamble build by PreambleThread.
640 /// Diagnostics are only published through this callback. This ensures they
641 /// are always for newer versions of the file, as the callback gets called in
642 /// the same order as update requests.
643 void updatePreamble(std::unique_ptr<CompilerInvocation> CI, ParseInputs PI,
644 std::shared_ptr<const PreambleData> Preamble,
645 std::vector<Diag> CIDiags, WantDiagnostics WantDiags);
646
647 /// Returns compile command from the current file inputs.
648 tooling::CompileCommand getCurrentCompileCommand() const;
649
650 /// Wait for the first build of preamble to finish. Preamble itself can be
651 /// accessed via getPossiblyStalePreamble(). Note that this function will
652 /// return after an unsuccessful build of the preamble too, i.e. result of
653 /// getPossiblyStalePreamble() can be null even after this function returns.
654 void waitForFirstPreamble() const;
655
656 TUScheduler::FileStats stats() const;
657 bool isASTCached() const;
658
659private:
660 // Details of an update request that are relevant to scheduling.
661 struct UpdateType {
662 // Do we want diagnostics from this version?
663 // If Yes, we must always build this version.
664 // If No, we only need to build this version if it's read.
665 // If Auto, we build if it's read or if the debounce expires.
666 WantDiagnostics Diagnostics;
667 // Did the main-file content of the document change?
668 // If so, we're allowed to cancel certain invalidated preceding reads.
669 bool ContentChanged;
670 };
671
672 /// Publishes diagnostics for \p Inputs. It will build an AST or reuse the
673 /// cached one if applicable. Assumes LatestPreamble is compatible for \p
674 /// Inputs.
675 void generateDiagnostics(std::unique_ptr<CompilerInvocation> Invocation,
676 ParseInputs Inputs, std::vector<Diag> CIDiags);
677
678 void updateASTSignals(ParsedAST &AST);
679
680 // Must be called exactly once on processing thread. Will return after
681 // stop() is called on a separate thread and all pending requests are
682 // processed.
683 void run();
684 /// Signal that run() should finish processing pending requests and exit.
685 void stop();
686
687 /// Adds a new task to the end of the request queue.
688 void startTask(llvm::StringRef Name, llvm::unique_function<void()> Task,
689 std::optional<UpdateType> Update,
690 TUScheduler::ASTActionInvalidation);
691 /// Runs a task synchronously.
692 void runTask(llvm::StringRef Name, llvm::function_ref<void()> Task);
693
694 /// Determines the next action to perform.
695 /// All actions that should never run are discarded.
696 /// Returns a deadline for the next action. If it's expired, run now.
697 /// scheduleLocked() is called again at the deadline, or if requests arrive.
698 Deadline scheduleLocked();
699 /// Should the first task in the queue be skipped instead of run?
700 bool shouldSkipHeadLocked() const;
701
702 struct Request {
703 llvm::unique_function<void()> Action;
704 std::string Name;
705 steady_clock::time_point AddTime;
706 Context Ctx;
707 std::optional<Context> QueueCtx;
708 std::optional<UpdateType> Update;
709 TUScheduler::ASTActionInvalidation InvalidationPolicy;
710 Canceler Invalidate;
711 };
712
713 /// Handles retention of ASTs.
714 TUScheduler::ASTCache &IdleASTs;
715 TUScheduler::HeaderIncluderCache &HeaderIncluders;
716 const bool RunSync;
717 /// Time to wait after an update to see whether another update obsoletes it.
718 const DebouncePolicy UpdateDebounce;
719 /// File that ASTWorker is responsible for.
720 const Path FileName;
721 /// Callback to create processing contexts for tasks.
722 const std::function<Context(llvm::StringRef)> ContextProvider;
723 const GlobalCompilationDatabase &CDB;
724 /// Callback invoked when preamble or main file AST is built.
725 ParsingCallbacks &Callbacks;
726
727 Semaphore &Barrier;
728 /// Whether the 'onMainAST' callback ran for the current FileInputs.
729 bool RanASTCallback = false;
730 /// Guards members used by both TUScheduler and the worker thread.
731 mutable std::mutex Mutex;
732 /// File inputs, currently being used by the worker.
733 /// Writes and reads from unknown threads are locked. Reads from the worker
734 /// thread are not locked, as it's the only writer.
735 ParseInputs FileInputs; /* GUARDED_BY(Mutex) */
736 /// Times of recent AST rebuilds, used for UpdateDebounce computation.
737 llvm::SmallVector<DebouncePolicy::clock::duration>
738 RebuildTimes; /* GUARDED_BY(Mutex) */
739 /// Set to true to signal run() to finish processing.
740 bool Done; /* GUARDED_BY(Mutex) */
741 std::deque<Request> Requests; /* GUARDED_BY(Mutex) */
742 std::optional<Request> CurrentRequest; /* GUARDED_BY(Mutex) */
743 /// Signalled whenever a new request has been scheduled or processing of a
744 /// request has completed.
745 mutable std::condition_variable RequestsCV;
746 std::shared_ptr<const ASTSignals> LatestASTSignals; /* GUARDED_BY(Mutex) */
747 /// Latest build preamble for current TU.
748 /// std::nullopt means no builds yet, null means there was an error while
749 /// building. Only written by ASTWorker's thread.
750 std::optional<std::shared_ptr<const PreambleData>> LatestPreamble;
751 std::deque<Request> PreambleRequests; /* GUARDED_BY(Mutex) */
752 /// Signaled whenever LatestPreamble changes state or there's a new
753 /// PreambleRequest.
754 mutable std::condition_variable PreambleCV;
755 /// Guards the callback that publishes results of AST-related computations
756 /// (diagnostics) and file statuses.
757 std::mutex PublishMu;
758 // Used to prevent remove document + add document races that lead to
759 // out-of-order callbacks for publishing results of onMainAST callback.
760 //
761 // The lifetime of the old/new ASTWorkers will overlap, but their handles
762 // don't. When the old handle is destroyed, the old worker will stop reporting
763 // any results to the user.
764 bool CanPublishResults = true; /* GUARDED_BY(PublishMu) */
765 std::atomic<unsigned> ASTBuildCount = {0};
766 std::atomic<unsigned> PreambleBuildCount = {0};
767
768 SynchronizedTUStatus Status;
769 PreambleThread PreamblePeer;
770};
771
772/// A smart-pointer-like class that points to an active ASTWorker.
773/// In destructor, signals to the underlying ASTWorker that no new requests will
774/// be sent and the processing loop may exit (after running all pending
775/// requests).
776class ASTWorkerHandle {
777 friend class ASTWorker;
778 ASTWorkerHandle(std::shared_ptr<ASTWorker> Worker)
779 : Worker(std::move(Worker)) {
780 assert(this->Worker);
781 }
782
783public:
784 ASTWorkerHandle(const ASTWorkerHandle &) = delete;
785 ASTWorkerHandle &operator=(const ASTWorkerHandle &) = delete;
786 ASTWorkerHandle(ASTWorkerHandle &&) = default;
787 ASTWorkerHandle &operator=(ASTWorkerHandle &&) = default;
788
789 ~ASTWorkerHandle() {
790 if (Worker)
791 Worker->stop();
792 }
793
794 ASTWorker &operator*() {
795 assert(Worker && "Handle was moved from");
796 return *Worker;
797 }
798
799 ASTWorker *operator->() {
800 assert(Worker && "Handle was moved from");
801 return Worker.get();
802 }
803
804 /// Returns an owning reference to the underlying ASTWorker that can outlive
805 /// the ASTWorkerHandle. However, no new requests to an active ASTWorker can
806 /// be schedule via the returned reference, i.e. only reads of the preamble
807 /// are possible.
808 std::shared_ptr<const ASTWorker> lock() { return Worker; }
809
810private:
811 std::shared_ptr<ASTWorker> Worker;
812};
813
814ASTWorkerHandle
815ASTWorker::create(PathRef FileName, const GlobalCompilationDatabase &CDB,
816 TUScheduler::ASTCache &IdleASTs,
817 TUScheduler::HeaderIncluderCache &HeaderIncluders,
818 AsyncTaskRunner *Tasks, Semaphore &Barrier,
819 const TUScheduler::Options &Opts,
820 ParsingCallbacks &Callbacks) {
821 std::shared_ptr<ASTWorker> Worker(
822 new ASTWorker(FileName, CDB, IdleASTs, HeaderIncluders, Barrier,
823 /*RunSync=*/!Tasks, Opts, Callbacks));
824 if (Tasks) {
825 Tasks->runAsync(Name: "ASTWorker:" + llvm::sys::path::filename(path: FileName),
826 Action: [Worker]() { Worker->run(); });
827 Tasks->runAsync(Name: "PreambleWorker:" + llvm::sys::path::filename(path: FileName),
828 Action: [Worker]() { Worker->PreamblePeer.run(); });
829 }
830
831 return ASTWorkerHandle(std::move(Worker));
832}
833
834ASTWorker::ASTWorker(PathRef FileName, const GlobalCompilationDatabase &CDB,
835 TUScheduler::ASTCache &LRUCache,
836 TUScheduler::HeaderIncluderCache &HeaderIncluders,
837 Semaphore &Barrier, bool RunSync,
838 const TUScheduler::Options &Opts,
839 ParsingCallbacks &Callbacks)
840 : IdleASTs(LRUCache), HeaderIncluders(HeaderIncluders), RunSync(RunSync),
841 UpdateDebounce(Opts.UpdateDebounce), FileName(FileName),
842 ContextProvider(Opts.ContextProvider), CDB(CDB), Callbacks(Callbacks),
843 Barrier(Barrier), Done(false), Status(FileName, Callbacks),
844 PreamblePeer(FileName, Callbacks, Opts.StorePreamblesInMemory, RunSync,
845 Opts.PreambleThrottler, Status, HeaderIncluders, *this) {
846 // Set a fallback command because compile command can be accessed before
847 // `Inputs` is initialized. Other fields are only used after initialization
848 // from client inputs.
849 FileInputs.CompileCommand = CDB.getFallbackCommand(File: FileName);
850}
851
852ASTWorker::~ASTWorker() {
853 // Make sure we remove the cached AST, if any.
854 IdleASTs.take(K: this);
855#ifndef NDEBUG
856 std::lock_guard<std::mutex> Lock(Mutex);
857 assert(Done && "handle was not destroyed");
858 assert(Requests.empty() && !CurrentRequest &&
859 "unprocessed requests when destroying ASTWorker");
860#endif
861}
862
863void ASTWorker::update(ParseInputs Inputs, WantDiagnostics WantDiags,
864 bool ContentChanged) {
865 llvm::StringLiteral TaskName = "Update";
866 auto Task = [=]() mutable {
867 // Get the actual command as `Inputs` does not have a command.
868 // FIXME: some build systems like Bazel will take time to preparing
869 // environment to build the file, it would be nice if we could emit a
870 // "PreparingBuild" status to inform users, it is non-trivial given the
871 // current implementation.
872 auto Cmd = CDB.getCompileCommand(File: FileName);
873 // If we don't have a reliable command for this file, it may be a header.
874 // Try to find a file that includes it, to borrow its command.
875 if (!Cmd || !isReliable(Cmd: *Cmd)) {
876 std::string ProxyFile = HeaderIncluders.get(Header: FileName);
877 if (!ProxyFile.empty()) {
878 auto ProxyCmd = CDB.getCompileCommand(File: ProxyFile);
879 if (!ProxyCmd || !isReliable(Cmd: *ProxyCmd)) {
880 // This command is supposed to be reliable! It's probably gone.
881 HeaderIncluders.remove(MainFile: ProxyFile);
882 } else {
883 // We have a reliable command for an including file, use it.
884 Cmd = tooling::transferCompileCommand(std::move(*ProxyCmd), Filename: FileName);
885 }
886 }
887 }
888 if (Cmd)
889 Inputs.CompileCommand = std::move(*Cmd);
890 else
891 Inputs.CompileCommand = CDB.getFallbackCommand(File: FileName);
892
893 bool InputsAreTheSame =
894 std::tie(args&: FileInputs.CompileCommand, args&: FileInputs.Contents) ==
895 std::tie(args&: Inputs.CompileCommand, args&: Inputs.Contents);
896 // Cached AST is invalidated.
897 if (!InputsAreTheSame) {
898 IdleASTs.take(K: this);
899 RanASTCallback = false;
900 }
901
902 // Update current inputs so that subsequent reads can see them.
903 {
904 std::lock_guard<std::mutex> Lock(Mutex);
905 FileInputs = Inputs;
906 }
907
908 log(Fmt: "ASTWorker building file {0} version {1} with command {2}\n[{3}]\n{4}",
909 Vals: FileName, Vals&: Inputs.Version, Vals&: Inputs.CompileCommand.Heuristic,
910 Vals&: Inputs.CompileCommand.Directory,
911 Vals: printArgv(Args: Inputs.CompileCommand.CommandLine));
912
913 StoreDiags CompilerInvocationDiagConsumer;
914 std::vector<std::string> CC1Args;
915 std::unique_ptr<CompilerInvocation> Invocation = buildCompilerInvocation(
916 Inputs, D&: CompilerInvocationDiagConsumer, CC1Args: &CC1Args);
917 // Log cc1 args even (especially!) if creating invocation failed.
918 if (!CC1Args.empty())
919 vlog(Fmt: "Driver produced command: cc1 {0}", Vals: printArgv(Args: CC1Args));
920 std::vector<Diag> CompilerInvocationDiags =
921 CompilerInvocationDiagConsumer.take();
922 if (!Invocation) {
923 elog(Fmt: "Could not build CompilerInvocation for file {0}", Vals: FileName);
924 // Remove the old AST if it's still in cache.
925 IdleASTs.take(K: this);
926 RanASTCallback = false;
927 // Report the diagnostics we collected when parsing the command line.
928 Callbacks.onFailedAST(Path: FileName, Version: Inputs.Version,
929 Diags: std::move(CompilerInvocationDiags),
930 Publish: [&](llvm::function_ref<void()> Publish) {
931 // Ensure we only publish results from the worker
932 // if the file was not removed, making sure there
933 // are not race conditions.
934 std::lock_guard<std::mutex> Lock(PublishMu);
935 if (CanPublishResults)
936 Publish();
937 });
938 // Note that this might throw away a stale preamble that might still be
939 // useful, but this is how we communicate a build error.
940 LatestPreamble.emplace();
941 // Make sure anyone waiting for the preamble gets notified it could not be
942 // built.
943 PreambleCV.notify_all();
944 return;
945 }
946
947 // Inform preamble peer, before attempting to build diagnostics so that they
948 // can be built concurrently.
949 PreamblePeer.update(CI: std::make_unique<CompilerInvocation>(args&: *Invocation),
950 PI: Inputs, CIDiags: CompilerInvocationDiags, WantDiags);
951
952 // Emit diagnostics from (possibly) stale preamble while waiting for a
953 // rebuild. Newly built preamble cannot emit diagnostics before this call
954 // finishes (ast callbacks are called from astpeer thread), hence we
955 // guarantee eventual consistency.
956 if (LatestPreamble && WantDiags != WantDiagnostics::No)
957 generateDiagnostics(Invocation: std::move(Invocation), Inputs: std::move(Inputs),
958 CIDiags: std::move(CompilerInvocationDiags));
959
960 std::unique_lock<std::mutex> Lock(Mutex);
961 PreambleCV.wait(lock&: Lock, p: [this] {
962 // Block until we reiceve a preamble request, unless a preamble already
963 // exists, as patching an empty preamble would imply rebuilding it from
964 // scratch.
965 // We block here instead of the consumer to prevent any deadlocks. Since
966 // LatestPreamble is only populated by ASTWorker thread.
967 return LatestPreamble || !PreambleRequests.empty() || Done;
968 });
969 };
970 startTask(Name: TaskName, Task: std::move(Task), Update: UpdateType{.Diagnostics: WantDiags, .ContentChanged: ContentChanged},
971 TUScheduler::NoInvalidation);
972}
973
974void ASTWorker::runWithAST(
975 llvm::StringRef Name,
976 llvm::unique_function<void(llvm::Expected<InputsAndAST>)> Action,
977 TUScheduler::ASTActionInvalidation Invalidation) {
978 // Tracks ast cache accesses for read operations.
979 static constexpr trace::Metric ASTAccessForRead(
980 "ast_access_read", trace::Metric::Counter, "result");
981 auto Task = [=, Action = std::move(Action)]() mutable {
982 if (auto Reason = isCancelled())
983 return Action(llvm::make_error<CancelledError>(Args&: Reason));
984 std::optional<std::unique_ptr<ParsedAST>> AST =
985 IdleASTs.take(K: this, AccessMetric: &ASTAccessForRead);
986 if (!AST) {
987 StoreDiags CompilerInvocationDiagConsumer;
988 std::unique_ptr<CompilerInvocation> Invocation =
989 buildCompilerInvocation(Inputs: FileInputs, D&: CompilerInvocationDiagConsumer);
990 // Try rebuilding the AST.
991 vlog(Fmt: "ASTWorker rebuilding evicted AST to run {0}: {1} version {2}", Vals&: Name,
992 Vals: FileName, Vals&: FileInputs.Version);
993 // FIXME: We might need to build a patched ast once preamble thread starts
994 // running async. Currently getPossiblyStalePreamble below will always
995 // return a compatible preamble as ASTWorker::update blocks.
996 std::optional<ParsedAST> NewAST;
997 if (Invocation) {
998 NewAST = ParsedAST::build(Filename: FileName, Inputs: FileInputs, CI: std::move(Invocation),
999 CompilerInvocationDiags: CompilerInvocationDiagConsumer.take(),
1000 Preamble: getPossiblyStalePreamble());
1001 ++ASTBuildCount;
1002 }
1003 AST = NewAST ? std::make_unique<ParsedAST>(args: std::move(*NewAST)) : nullptr;
1004 }
1005 // Make sure we put the AST back into the LRU cache.
1006 auto _ = llvm::make_scope_exit(
1007 F: [&AST, this]() { IdleASTs.put(K: this, V: std::move(*AST)); });
1008 // Run the user-provided action.
1009 if (!*AST)
1010 return Action(error(EC: llvm::errc::invalid_argument, Fmt: "invalid AST"));
1011 vlog(Fmt: "ASTWorker running {0} on version {2} of {1}", Vals&: Name, Vals: FileName,
1012 Vals&: FileInputs.Version);
1013 Action(InputsAndAST{.Inputs: FileInputs, .AST: **AST});
1014 };
1015 startTask(Name, Task: std::move(Task), /*Update=*/std::nullopt, Invalidation);
1016}
1017
1018/// To be called from ThreadCrashReporter's signal handler.
1019static void crashDumpCompileCommand(llvm::raw_ostream &OS,
1020 const tooling::CompileCommand &Command) {
1021 OS << " Filename: " << Command.Filename << "\n";
1022 OS << " Directory: " << Command.Directory << "\n";
1023 OS << " Command Line:";
1024 for (auto &Arg : Command.CommandLine) {
1025 OS << " " << Arg;
1026 }
1027 OS << "\n";
1028}
1029
1030/// To be called from ThreadCrashReporter's signal handler.
1031static void crashDumpFileContents(llvm::raw_ostream &OS,
1032 const std::string &Contents) {
1033 // Avoid flooding the terminal with source code by default, but allow clients
1034 // to opt in. Use an env var to preserve backwards compatibility of the
1035 // command line interface, while allowing it to be set outside the clangd
1036 // launch site for more flexibility.
1037 if (getenv(name: "CLANGD_CRASH_DUMP_SOURCE")) {
1038 OS << " Contents:\n";
1039 OS << Contents << "\n";
1040 }
1041}
1042
1043/// To be called from ThreadCrashReporter's signal handler.
1044static void crashDumpParseInputs(llvm::raw_ostream &OS,
1045 const ParseInputs &FileInputs) {
1046 auto &Command = FileInputs.CompileCommand;
1047 crashDumpCompileCommand(OS, Command);
1048 OS << " Version: " << FileInputs.Version << "\n";
1049 crashDumpFileContents(OS, Contents: FileInputs.Contents);
1050}
1051
1052void PreambleThread::build(Request Req) {
1053 assert(Req.CI && "Got preamble request with null compiler invocation");
1054 const ParseInputs &Inputs = Req.Inputs;
1055 bool ReusedPreamble = false;
1056
1057 Status.update(Mutator: [&](TUStatus &Status) {
1058 Status.PreambleActivity = PreambleAction::Building;
1059 });
1060 auto _ = llvm::make_scope_exit(F: [this, &Req, &ReusedPreamble] {
1061 ASTPeer.updatePreamble(CI: std::move(Req.CI), PI: std::move(Req.Inputs),
1062 Preamble: LatestBuild, CIDiags: std::move(Req.CIDiags),
1063 WantDiags: std::move(Req.WantDiags));
1064 if (!ReusedPreamble)
1065 Callbacks.onPreamblePublished(File: FileName);
1066 });
1067
1068 if (!LatestBuild || Inputs.ForceRebuild) {
1069 vlog(Fmt: "Building first preamble for {0} version {1}", Vals: FileName,
1070 Vals: Inputs.Version);
1071 } else if (isPreambleCompatible(Preamble: *LatestBuild, Inputs, FileName, CI: *Req.CI)) {
1072 vlog(Fmt: "Reusing preamble version {0} for version {1} of {2}",
1073 Vals: LatestBuild->Version, Vals: Inputs.Version, Vals: FileName);
1074 ReusedPreamble = true;
1075 return;
1076 } else {
1077 vlog(Fmt: "Rebuilding invalidated preamble for {0} version {1} (previous was "
1078 "version {2})",
1079 Vals: FileName, Vals: Inputs.Version, Vals: LatestBuild->Version);
1080 }
1081
1082 ThreadCrashReporter ScopedReporter([&Inputs]() {
1083 llvm::errs() << "Signalled while building preamble\n";
1084 crashDumpParseInputs(OS&: llvm::errs(), FileInputs: Inputs);
1085 });
1086
1087 PreambleBuildStats Stats;
1088 bool IsFirstPreamble = !LatestBuild;
1089 LatestBuild = clang::clangd::buildPreamble(
1090 FileName, CI: *Req.CI, Inputs, StoreInMemory,
1091 PreambleCallback: [&](CapturedASTCtx ASTCtx,
1092 std::shared_ptr<const include_cleaner::PragmaIncludes> PI) {
1093 Callbacks.onPreambleAST(Path: FileName, Version: Inputs.Version, Ctx: std::move(ASTCtx),
1094 std::move(PI));
1095 },
1096 Stats: &Stats);
1097 if (!LatestBuild)
1098 return;
1099 reportPreambleBuild(Stats, IsFirstPreamble);
1100 if (isReliable(Cmd: LatestBuild->CompileCommand))
1101 HeaderIncluders.update(MainFile: FileName, Headers: LatestBuild->Includes.allHeaders());
1102}
1103
1104void ASTWorker::updatePreamble(std::unique_ptr<CompilerInvocation> CI,
1105 ParseInputs PI,
1106 std::shared_ptr<const PreambleData> Preamble,
1107 std::vector<Diag> CIDiags,
1108 WantDiagnostics WantDiags) {
1109 llvm::StringLiteral TaskName = "Build AST";
1110 // Store preamble and build diagnostics with new preamble if requested.
1111 auto Task = [this, Preamble = std::move(Preamble), CI = std::move(CI),
1112 CIDiags = std::move(CIDiags),
1113 WantDiags = std::move(WantDiags)]() mutable {
1114 // Update the preamble inside ASTWorker queue to ensure atomicity. As a task
1115 // running inside ASTWorker assumes internals won't change until it
1116 // finishes.
1117 if (!LatestPreamble || Preamble != *LatestPreamble) {
1118 ++PreambleBuildCount;
1119 // Cached AST is no longer valid.
1120 IdleASTs.take(K: this);
1121 RanASTCallback = false;
1122 std::lock_guard<std::mutex> Lock(Mutex);
1123 // LatestPreamble might be the last reference to old preamble, do not
1124 // trigger destructor while holding the lock.
1125 if (LatestPreamble)
1126 std::swap(a&: *LatestPreamble, b&: Preamble);
1127 else
1128 LatestPreamble = std::move(Preamble);
1129 }
1130 // Notify anyone waiting for a preamble.
1131 PreambleCV.notify_all();
1132 // Give up our ownership to old preamble before starting expensive AST
1133 // build.
1134 Preamble.reset();
1135 // We only need to build the AST if diagnostics were requested.
1136 if (WantDiags == WantDiagnostics::No)
1137 return;
1138 // Since the file may have been edited since we started building this
1139 // preamble, we use the current contents of the file instead. This provides
1140 // more up-to-date diagnostics, and avoids diagnostics going backwards (we
1141 // may have already emitted staler-preamble diagnostics for the new
1142 // version).
1143 // We still have eventual consistency: at some point updatePreamble() will
1144 // catch up to the current file.
1145 // Report diagnostics with the new preamble to ensure progress. Otherwise
1146 // diagnostics might get stale indefinitely if user keeps invalidating the
1147 // preamble.
1148 generateDiagnostics(Invocation: std::move(CI), Inputs: FileInputs, CIDiags: std::move(CIDiags));
1149 };
1150 if (RunSync) {
1151 runTask(Name: TaskName, Task);
1152 return;
1153 }
1154 {
1155 std::lock_guard<std::mutex> Lock(Mutex);
1156 PreambleRequests.push_back(x: {.Action: std::move(Task), .Name: std::string(TaskName),
1157 .AddTime: steady_clock::now(), .Ctx: Context::current().clone(),
1158 .QueueCtx: std::nullopt, .Update: std::nullopt,
1159 .InvalidationPolicy: TUScheduler::NoInvalidation, .Invalidate: nullptr});
1160 }
1161 PreambleCV.notify_all();
1162 RequestsCV.notify_all();
1163}
1164
1165void ASTWorker::updateASTSignals(ParsedAST &AST) {
1166 auto Signals = std::make_shared<const ASTSignals>(args: ASTSignals::derive(AST));
1167 // Existing readers of ASTSignals will have their copy preserved until the
1168 // read is completed. The last reader deletes the old ASTSignals.
1169 {
1170 std::lock_guard<std::mutex> Lock(Mutex);
1171 std::swap(a&: LatestASTSignals, b&: Signals);
1172 }
1173}
1174
1175void ASTWorker::generateDiagnostics(
1176 std::unique_ptr<CompilerInvocation> Invocation, ParseInputs Inputs,
1177 std::vector<Diag> CIDiags) {
1178 // Tracks ast cache accesses for publishing diags.
1179 static constexpr trace::Metric ASTAccessForDiag(
1180 "ast_access_diag", trace::Metric::Counter, "result");
1181 assert(Invocation);
1182 assert(LatestPreamble);
1183 // No need to rebuild the AST if we won't send the diagnostics.
1184 {
1185 std::lock_guard<std::mutex> Lock(PublishMu);
1186 if (!CanPublishResults)
1187 return;
1188 }
1189 // Used to check whether we can update AST cache.
1190 bool InputsAreLatest =
1191 std::tie(args&: FileInputs.CompileCommand, args&: FileInputs.Contents) ==
1192 std::tie(args&: Inputs.CompileCommand, args&: Inputs.Contents);
1193 // Take a shortcut and don't report the diagnostics, since they should be the
1194 // same. All the clients should handle the lack of OnUpdated() call anyway to
1195 // handle empty result from buildAST.
1196 // FIXME: the AST could actually change if non-preamble includes changed,
1197 // but we choose to ignore it.
1198 if (InputsAreLatest && RanASTCallback)
1199 return;
1200
1201 // Get the AST for diagnostics, either build it or use the cached one.
1202 std::string TaskName = llvm::formatv(Fmt: "Build AST ({0})", Vals&: Inputs.Version);
1203 Status.update(Mutator: [&](TUStatus &Status) {
1204 Status.ASTActivity.K = ASTAction::Building;
1205 Status.ASTActivity.Name = std::move(TaskName);
1206 });
1207 // We might be able to reuse the last we've built for a read request.
1208 // FIXME: It might be better to not reuse this AST. That way queued AST builds
1209 // won't be required for diags.
1210 std::optional<std::unique_ptr<ParsedAST>> AST =
1211 IdleASTs.take(K: this, AccessMetric: &ASTAccessForDiag);
1212 if (!AST || !InputsAreLatest) {
1213 auto RebuildStartTime = DebouncePolicy::clock::now();
1214 std::optional<ParsedAST> NewAST = ParsedAST::build(
1215 Filename: FileName, Inputs, CI: std::move(Invocation), CompilerInvocationDiags: CIDiags, Preamble: *LatestPreamble);
1216 auto RebuildDuration = DebouncePolicy::clock::now() - RebuildStartTime;
1217 ++ASTBuildCount;
1218 // Try to record the AST-build time, to inform future update debouncing.
1219 // This is best-effort only: if the lock is held, don't bother.
1220 std::unique_lock<std::mutex> Lock(Mutex, std::try_to_lock);
1221 if (Lock.owns_lock()) {
1222 // Do not let RebuildTimes grow beyond its small-size (i.e.
1223 // capacity).
1224 if (RebuildTimes.size() == RebuildTimes.capacity())
1225 RebuildTimes.erase(CI: RebuildTimes.begin());
1226 RebuildTimes.push_back(Elt: RebuildDuration);
1227 Lock.unlock();
1228 }
1229 Status.update(Mutator: [&](TUStatus &Status) {
1230 Status.Details.ReuseAST = false;
1231 Status.Details.BuildFailed = !NewAST;
1232 });
1233 AST = NewAST ? std::make_unique<ParsedAST>(args: std::move(*NewAST)) : nullptr;
1234 } else {
1235 log(Fmt: "Skipping rebuild of the AST for {0}, inputs are the same.", Vals: FileName);
1236 Status.update(Mutator: [](TUStatus &Status) {
1237 Status.Details.ReuseAST = true;
1238 Status.Details.BuildFailed = false;
1239 });
1240 }
1241
1242 // Publish diagnostics.
1243 auto RunPublish = [&](llvm::function_ref<void()> Publish) {
1244 // Ensure we only publish results from the worker if the file was not
1245 // removed, making sure there are not race conditions.
1246 std::lock_guard<std::mutex> Lock(PublishMu);
1247 if (CanPublishResults)
1248 Publish();
1249 };
1250 if (*AST) {
1251 trace::Span Span("Running main AST callback");
1252 Callbacks.onMainAST(Path: FileName, AST&: **AST, Publish: RunPublish);
1253 updateASTSignals(AST&: **AST);
1254 } else {
1255 // Failed to build the AST, at least report diagnostics from the
1256 // command line if there were any.
1257 // FIXME: we might have got more errors while trying to build the
1258 // AST, surface them too.
1259 Callbacks.onFailedAST(Path: FileName, Version: Inputs.Version, Diags: CIDiags, Publish: RunPublish);
1260 }
1261
1262 // AST might've been built for an older version of the source, as ASTWorker
1263 // queue raced ahead while we were waiting on the preamble. In that case the
1264 // queue can't reuse the AST.
1265 if (InputsAreLatest) {
1266 RanASTCallback = *AST != nullptr;
1267 IdleASTs.put(K: this, V: std::move(*AST));
1268 }
1269}
1270
1271std::shared_ptr<const PreambleData> ASTWorker::getPossiblyStalePreamble(
1272 std::shared_ptr<const ASTSignals> *ASTSignals) const {
1273 std::lock_guard<std::mutex> Lock(Mutex);
1274 if (ASTSignals)
1275 *ASTSignals = LatestASTSignals;
1276 return LatestPreamble ? *LatestPreamble : nullptr;
1277}
1278
1279void ASTWorker::waitForFirstPreamble() const {
1280 std::unique_lock<std::mutex> Lock(Mutex);
1281 PreambleCV.wait(lock&: Lock, p: [this] { return LatestPreamble || Done; });
1282}
1283
1284tooling::CompileCommand ASTWorker::getCurrentCompileCommand() const {
1285 std::unique_lock<std::mutex> Lock(Mutex);
1286 return FileInputs.CompileCommand;
1287}
1288
1289TUScheduler::FileStats ASTWorker::stats() const {
1290 TUScheduler::FileStats Result;
1291 Result.ASTBuilds = ASTBuildCount;
1292 Result.PreambleBuilds = PreambleBuildCount;
1293 // Note that we don't report the size of ASTs currently used for processing
1294 // the in-flight requests. We used this information for debugging purposes
1295 // only, so this should be fine.
1296 Result.UsedBytesAST = IdleASTs.getUsedBytes(K: this);
1297 if (auto Preamble = getPossiblyStalePreamble())
1298 Result.UsedBytesPreamble = Preamble->Preamble.getSize();
1299 return Result;
1300}
1301
1302bool ASTWorker::isASTCached() const { return IdleASTs.getUsedBytes(K: this) != 0; }
1303
1304void ASTWorker::stop() {
1305 {
1306 std::lock_guard<std::mutex> Lock(PublishMu);
1307 CanPublishResults = false;
1308 }
1309 {
1310 std::lock_guard<std::mutex> Lock(Mutex);
1311 assert(!Done && "stop() called twice");
1312 Done = true;
1313 }
1314 PreamblePeer.stop();
1315 // We are no longer going to build any preambles, let the waiters know that.
1316 PreambleCV.notify_all();
1317 Status.stop();
1318 RequestsCV.notify_all();
1319}
1320
1321void ASTWorker::runTask(llvm::StringRef Name, llvm::function_ref<void()> Task) {
1322 ThreadCrashReporter ScopedReporter([this, Name]() {
1323 llvm::errs() << "Signalled during AST worker action: " << Name << "\n";
1324 crashDumpParseInputs(OS&: llvm::errs(), FileInputs);
1325 });
1326 trace::Span Tracer(Name);
1327 WithContext WithProvidedContext(ContextProvider(FileName));
1328 Task();
1329}
1330
1331void ASTWorker::startTask(llvm::StringRef Name,
1332 llvm::unique_function<void()> Task,
1333 std::optional<UpdateType> Update,
1334 TUScheduler::ASTActionInvalidation Invalidation) {
1335 if (RunSync) {
1336 assert(!Done && "running a task after stop()");
1337 runTask(Name, Task);
1338 return;
1339 }
1340
1341 {
1342 std::lock_guard<std::mutex> Lock(Mutex);
1343 assert(!Done && "running a task after stop()");
1344 // Cancel any requests invalidated by this request.
1345 if (Update && Update->ContentChanged) {
1346 for (auto &R : llvm::reverse(C&: Requests)) {
1347 if (R.InvalidationPolicy == TUScheduler::InvalidateOnUpdate)
1348 R.Invalidate();
1349 if (R.Update && R.Update->ContentChanged)
1350 break; // Older requests were already invalidated by the older update.
1351 }
1352 }
1353
1354 // Allow this request to be cancelled if invalidated.
1355 Context Ctx = Context::current().derive(Key: FileBeingProcessed, Value: FileName);
1356 Canceler Invalidate = nullptr;
1357 if (Invalidation) {
1358 WithContext WC(std::move(Ctx));
1359 std::tie(args&: Ctx, args&: Invalidate) = cancelableTask(
1360 /*Reason=*/static_cast<int>(ErrorCode::ContentModified));
1361 }
1362 // Trace the time the request spends in the queue, and the requests that
1363 // it's going to wait for.
1364 std::optional<Context> QueueCtx;
1365 if (trace::enabled()) {
1366 // Tracers that follow threads and need strict nesting will see a tiny
1367 // instantaneous event "we're enqueueing", and sometime later it runs.
1368 WithContext WC(Ctx.clone());
1369 trace::Span Tracer("Queued:" + Name);
1370 if (Tracer.Args) {
1371 if (CurrentRequest)
1372 SPAN_ATTACH(Tracer, "CurrentRequest", CurrentRequest->Name);
1373 llvm::json::Array PreambleRequestsNames;
1374 for (const auto &Req : PreambleRequests)
1375 PreambleRequestsNames.push_back(E: Req.Name);
1376 SPAN_ATTACH(Tracer, "PreambleRequestsNames",
1377 std::move(PreambleRequestsNames));
1378 llvm::json::Array RequestsNames;
1379 for (const auto &Req : Requests)
1380 RequestsNames.push_back(E: Req.Name);
1381 SPAN_ATTACH(Tracer, "RequestsNames", std::move(RequestsNames));
1382 }
1383 // For tracers that follow contexts, keep the trace span's context alive
1384 // until we dequeue the request, so they see the full duration.
1385 QueueCtx = Context::current().clone();
1386 }
1387 Requests.push_back(x: {.Action: std::move(Task), .Name: std::string(Name), .AddTime: steady_clock::now(),
1388 .Ctx: std::move(Ctx), .QueueCtx: std::move(QueueCtx), .Update: Update,
1389 .InvalidationPolicy: Invalidation, .Invalidate: std::move(Invalidate)});
1390 }
1391 RequestsCV.notify_all();
1392}
1393
1394void ASTWorker::run() {
1395 clang::noteBottomOfStack();
1396 while (true) {
1397 {
1398 std::unique_lock<std::mutex> Lock(Mutex);
1399 assert(!CurrentRequest && "A task is already running, multiple workers?");
1400 for (auto Wait = scheduleLocked(); !Wait.expired();
1401 Wait = scheduleLocked()) {
1402 assert(PreambleRequests.empty() &&
1403 "Preamble updates should be scheduled immediately");
1404 if (Done) {
1405 if (Requests.empty())
1406 return;
1407 // Even though Done is set, finish pending requests.
1408 break; // However, skip delays to shutdown fast.
1409 }
1410
1411 // Tracing: we have a next request, attribute this sleep to it.
1412 std::optional<WithContext> Ctx;
1413 std::optional<trace::Span> Tracer;
1414 if (!Requests.empty()) {
1415 Ctx.emplace(args: Requests.front().Ctx.clone());
1416 Tracer.emplace(args: "Debounce");
1417 SPAN_ATTACH(*Tracer, "next_request", Requests.front().Name);
1418 if (!(Wait == Deadline::infinity())) {
1419 Status.update(Mutator: [&](TUStatus &Status) {
1420 Status.ASTActivity.K = ASTAction::Queued;
1421 Status.ASTActivity.Name = Requests.front().Name;
1422 });
1423 SPAN_ATTACH(*Tracer, "sleep_ms",
1424 std::chrono::duration_cast<std::chrono::milliseconds>(
1425 Wait.time() - steady_clock::now())
1426 .count());
1427 }
1428 }
1429
1430 wait(Lock, CV&: RequestsCV, D: Wait);
1431 }
1432 // Any request in ReceivedPreambles is at least as old as the
1433 // Requests.front(), so prefer them first to preserve LSP order.
1434 if (!PreambleRequests.empty()) {
1435 CurrentRequest = std::move(PreambleRequests.front());
1436 PreambleRequests.pop_front();
1437 } else {
1438 CurrentRequest = std::move(Requests.front());
1439 Requests.pop_front();
1440 }
1441 } // unlock Mutex
1442
1443 // Inform tracing that the request was dequeued.
1444 CurrentRequest->QueueCtx.reset();
1445
1446 // It is safe to perform reads to CurrentRequest without holding the lock as
1447 // only writer is also this thread.
1448 {
1449 std::unique_lock<Semaphore> Lock(Barrier, std::try_to_lock);
1450 if (!Lock.owns_lock()) {
1451 Status.update(Mutator: [&](TUStatus &Status) {
1452 Status.ASTActivity.K = ASTAction::Queued;
1453 Status.ASTActivity.Name = CurrentRequest->Name;
1454 });
1455 Lock.lock();
1456 }
1457 WithContext Guard(std::move(CurrentRequest->Ctx));
1458 Status.update(Mutator: [&](TUStatus &Status) {
1459 Status.ASTActivity.K = ASTAction::RunningAction;
1460 Status.ASTActivity.Name = CurrentRequest->Name;
1461 });
1462 runTask(Name: CurrentRequest->Name, Task: CurrentRequest->Action);
1463 }
1464
1465 bool IsEmpty = false;
1466 {
1467 std::lock_guard<std::mutex> Lock(Mutex);
1468 CurrentRequest.reset();
1469 IsEmpty = Requests.empty() && PreambleRequests.empty();
1470 }
1471 if (IsEmpty) {
1472 Status.update(Mutator: [&](TUStatus &Status) {
1473 Status.ASTActivity.K = ASTAction::Idle;
1474 Status.ASTActivity.Name = "";
1475 });
1476 }
1477 RequestsCV.notify_all();
1478 }
1479}
1480
1481Deadline ASTWorker::scheduleLocked() {
1482 // Process new preambles immediately.
1483 if (!PreambleRequests.empty())
1484 return Deadline::zero();
1485 if (Requests.empty())
1486 return Deadline::infinity(); // Wait for new requests.
1487 // Handle cancelled requests first so the rest of the scheduler doesn't.
1488 for (auto I = Requests.begin(), E = Requests.end(); I != E; ++I) {
1489 if (!isCancelled(Ctx: I->Ctx)) {
1490 // Cancellations after the first read don't affect current scheduling.
1491 if (I->Update == std::nullopt)
1492 break;
1493 continue;
1494 }
1495 // Cancelled reads are moved to the front of the queue and run immediately.
1496 if (I->Update == std::nullopt) {
1497 Request R = std::move(*I);
1498 Requests.erase(position: I);
1499 Requests.push_front(x: std::move(R));
1500 return Deadline::zero();
1501 }
1502 // Cancelled updates are downgraded to auto-diagnostics, and may be elided.
1503 if (I->Update->Diagnostics == WantDiagnostics::Yes)
1504 I->Update->Diagnostics = WantDiagnostics::Auto;
1505 }
1506
1507 while (shouldSkipHeadLocked()) {
1508 vlog(Fmt: "ASTWorker skipping {0} for {1}", Vals&: Requests.front().Name, Vals: FileName);
1509 Requests.pop_front();
1510 }
1511 assert(!Requests.empty() && "skipped the whole queue");
1512 // Some updates aren't dead yet, but never end up being used.
1513 // e.g. the first keystroke is live until obsoleted by the second.
1514 // We debounce "maybe-unused" writes, sleeping in case they become dead.
1515 // But don't delay reads (including updates where diagnostics are needed).
1516 for (const auto &R : Requests)
1517 if (R.Update == std::nullopt ||
1518 R.Update->Diagnostics == WantDiagnostics::Yes)
1519 return Deadline::zero();
1520 // Front request needs to be debounced, so determine when we're ready.
1521 Deadline D(Requests.front().AddTime + UpdateDebounce.compute(History: RebuildTimes));
1522 return D;
1523}
1524
1525// Returns true if Requests.front() is a dead update that can be skipped.
1526bool ASTWorker::shouldSkipHeadLocked() const {
1527 assert(!Requests.empty());
1528 auto Next = Requests.begin();
1529 auto Update = Next->Update;
1530 if (!Update) // Only skip updates.
1531 return false;
1532 ++Next;
1533 // An update is live if its AST might still be read.
1534 // That is, if it's not immediately followed by another update.
1535 if (Next == Requests.end() || !Next->Update)
1536 return false;
1537 // The other way an update can be live is if its diagnostics might be used.
1538 switch (Update->Diagnostics) {
1539 case WantDiagnostics::Yes:
1540 return false; // Always used.
1541 case WantDiagnostics::No:
1542 return true; // Always dead.
1543 case WantDiagnostics::Auto:
1544 // Used unless followed by an update that generates diagnostics.
1545 for (; Next != Requests.end(); ++Next)
1546 if (Next->Update && Next->Update->Diagnostics != WantDiagnostics::No)
1547 return true; // Prefer later diagnostics.
1548 return false;
1549 }
1550 llvm_unreachable("Unknown WantDiagnostics");
1551}
1552
1553bool ASTWorker::blockUntilIdle(Deadline Timeout) const {
1554 auto WaitUntilASTWorkerIsIdle = [&] {
1555 std::unique_lock<std::mutex> Lock(Mutex);
1556 return wait(Lock, CV&: RequestsCV, D: Timeout, F: [&] {
1557 return PreambleRequests.empty() && Requests.empty() && !CurrentRequest;
1558 });
1559 };
1560 // Make sure ASTWorker has processed all requests, which might issue new
1561 // updates to PreamblePeer.
1562 if (!WaitUntilASTWorkerIsIdle())
1563 return false;
1564 // Now that ASTWorker processed all requests, ensure PreamblePeer has served
1565 // all update requests. This might create new PreambleRequests for the
1566 // ASTWorker.
1567 if (!PreamblePeer.blockUntilIdle(Timeout))
1568 return false;
1569 assert(Requests.empty() &&
1570 "No new normal tasks can be scheduled concurrently with "
1571 "blockUntilIdle(): ASTWorker isn't threadsafe");
1572 // Finally make sure ASTWorker has processed all of the preamble updates.
1573 return WaitUntilASTWorkerIsIdle();
1574}
1575
1576// Render a TUAction to a user-facing string representation.
1577// TUAction represents clangd-internal states, we don't intend to expose them
1578// to users (say C++ programmers) directly to avoid confusion, we use terms that
1579// are familiar by C++ programmers.
1580std::string renderTUAction(const PreambleAction PA, const ASTAction &AA) {
1581 llvm::SmallVector<std::string, 2> Result;
1582 switch (PA) {
1583 case PreambleAction::Building:
1584 Result.push_back(Elt: "parsing includes");
1585 break;
1586 case PreambleAction::Queued:
1587 Result.push_back(Elt: "includes are queued");
1588 break;
1589 case PreambleAction::Idle:
1590 // We handle idle specially below.
1591 break;
1592 }
1593 switch (AA.K) {
1594 case ASTAction::Queued:
1595 Result.push_back(Elt: "file is queued");
1596 break;
1597 case ASTAction::RunningAction:
1598 Result.push_back(Elt: "running " + AA.Name);
1599 break;
1600 case ASTAction::Building:
1601 Result.push_back(Elt: "parsing main file");
1602 break;
1603 case ASTAction::Idle:
1604 // We handle idle specially below.
1605 break;
1606 }
1607 if (Result.empty())
1608 return "idle";
1609 return llvm::join(R&: Result, Separator: ", ");
1610}
1611
1612} // namespace
1613
1614unsigned getDefaultAsyncThreadsCount() {
1615 return llvm::heavyweight_hardware_concurrency().compute_thread_count();
1616}
1617
1618FileStatus TUStatus::render(PathRef File) const {
1619 FileStatus FStatus;
1620 FStatus.uri = URIForFile::canonicalize(AbsPath: File, /*TUPath=*/File);
1621 FStatus.state = renderTUAction(PA: PreambleActivity, AA: ASTActivity);
1622 return FStatus;
1623}
1624
1625struct TUScheduler::FileData {
1626 /// Latest inputs, passed to TUScheduler::update().
1627 std::string Contents;
1628 ASTWorkerHandle Worker;
1629};
1630
1631TUScheduler::TUScheduler(const GlobalCompilationDatabase &CDB,
1632 const Options &Opts,
1633 std::unique_ptr<ParsingCallbacks> Callbacks)
1634 : CDB(CDB), Opts(Opts),
1635 Callbacks(Callbacks ? std::move(Callbacks)
1636 : std::make_unique<ParsingCallbacks>()),
1637 Barrier(Opts.AsyncThreadsCount), QuickRunBarrier(Opts.AsyncThreadsCount),
1638 IdleASTs(
1639 std::make_unique<ASTCache>(args: Opts.RetentionPolicy.MaxRetainedASTs)),
1640 HeaderIncluders(std::make_unique<HeaderIncluderCache>()) {
1641 // Avoid null checks everywhere.
1642 if (!Opts.ContextProvider) {
1643 this->Opts.ContextProvider = [](llvm::StringRef) {
1644 return Context::current().clone();
1645 };
1646 }
1647 if (0 < Opts.AsyncThreadsCount) {
1648 PreambleTasks.emplace();
1649 WorkerThreads.emplace();
1650 }
1651}
1652
1653TUScheduler::~TUScheduler() {
1654 // Notify all workers that they need to stop.
1655 Files.clear();
1656
1657 // Wait for all in-flight tasks to finish.
1658 if (PreambleTasks)
1659 PreambleTasks->wait();
1660 if (WorkerThreads)
1661 WorkerThreads->wait();
1662}
1663
1664bool TUScheduler::blockUntilIdle(Deadline D) const {
1665 for (auto &File : Files)
1666 if (!File.getValue()->Worker->blockUntilIdle(Timeout: D))
1667 return false;
1668 if (PreambleTasks)
1669 if (!PreambleTasks->wait(D))
1670 return false;
1671 return true;
1672}
1673
1674bool TUScheduler::update(PathRef File, ParseInputs Inputs,
1675 WantDiagnostics WantDiags) {
1676 std::unique_ptr<FileData> &FD = Files[File];
1677 bool NewFile = FD == nullptr;
1678 bool ContentChanged = false;
1679 if (!FD) {
1680 // Create a new worker to process the AST-related tasks.
1681 ASTWorkerHandle Worker = ASTWorker::create(
1682 FileName: File, CDB, IdleASTs&: *IdleASTs, HeaderIncluders&: *HeaderIncluders,
1683 Tasks: WorkerThreads ? &*WorkerThreads : nullptr, Barrier, Opts, Callbacks&: *Callbacks);
1684 FD = std::unique_ptr<FileData>(
1685 new FileData{.Contents: Inputs.Contents, .Worker: std::move(Worker)});
1686 ContentChanged = true;
1687 } else if (FD->Contents != Inputs.Contents) {
1688 ContentChanged = true;
1689 FD->Contents = Inputs.Contents;
1690 }
1691 FD->Worker->update(Inputs: std::move(Inputs), WantDiags, ContentChanged);
1692 // There might be synthetic update requests, don't change the LastActiveFile
1693 // in such cases.
1694 if (ContentChanged)
1695 LastActiveFile = File.str();
1696 return NewFile;
1697}
1698
1699void TUScheduler::remove(PathRef File) {
1700 bool Removed = Files.erase(Key: File);
1701 if (!Removed)
1702 elog(Fmt: "Trying to remove file from TUScheduler that is not tracked: {0}",
1703 Vals&: File);
1704 // We don't call HeaderIncluders.remove(File) here.
1705 // If we did, we'd avoid potentially stale header/mainfile associations.
1706 // However, it would mean that closing a mainfile could invalidate the
1707 // preamble of several open headers.
1708}
1709
1710void TUScheduler::run(llvm::StringRef Name, llvm::StringRef Path,
1711 llvm::unique_function<void()> Action) {
1712 runWithSemaphore(Name, Path, Action: std::move(Action), Sem&: Barrier);
1713}
1714
1715void TUScheduler::runQuick(llvm::StringRef Name, llvm::StringRef Path,
1716 llvm::unique_function<void()> Action) {
1717 // Use QuickRunBarrier to serialize quick tasks: we are ignoring
1718 // the parallelism level set by the user, don't abuse it
1719 runWithSemaphore(Name, Path, Action: std::move(Action), Sem&: QuickRunBarrier);
1720}
1721
1722void TUScheduler::runWithSemaphore(llvm::StringRef Name, llvm::StringRef Path,
1723 llvm::unique_function<void()> Action,
1724 Semaphore &Sem) {
1725 if (Path.empty())
1726 Path = LastActiveFile;
1727 else
1728 LastActiveFile = Path.str();
1729 if (!PreambleTasks) {
1730 WithContext WithProvidedContext(Opts.ContextProvider(Path));
1731 return Action();
1732 }
1733 PreambleTasks->runAsync(Name, Action: [this, &Sem, Ctx = Context::current().clone(),
1734 Path(Path.str()),
1735 Action = std::move(Action)]() mutable {
1736 std::lock_guard<Semaphore> BarrierLock(Sem);
1737 WithContext WC(std::move(Ctx));
1738 WithContext WithProvidedContext(Opts.ContextProvider(Path));
1739 Action();
1740 });
1741}
1742
1743void TUScheduler::runWithAST(
1744 llvm::StringRef Name, PathRef File,
1745 llvm::unique_function<void(llvm::Expected<InputsAndAST>)> Action,
1746 TUScheduler::ASTActionInvalidation Invalidation) {
1747 auto It = Files.find(Key: File);
1748 if (It == Files.end()) {
1749 Action(llvm::make_error<LSPError>(
1750 Args: "trying to get AST for non-added document", Args: ErrorCode::InvalidParams));
1751 return;
1752 }
1753 LastActiveFile = File.str();
1754
1755 It->second->Worker->runWithAST(Name, Action: std::move(Action), Invalidation);
1756}
1757
1758void TUScheduler::runWithPreamble(llvm::StringRef Name, PathRef File,
1759 PreambleConsistency Consistency,
1760 Callback<InputsAndPreamble> Action) {
1761 auto It = Files.find(Key: File);
1762 if (It == Files.end()) {
1763 Action(llvm::make_error<LSPError>(
1764 Args: "trying to get preamble for non-added document",
1765 Args: ErrorCode::InvalidParams));
1766 return;
1767 }
1768 LastActiveFile = File.str();
1769
1770 if (!PreambleTasks) {
1771 trace::Span Tracer(Name);
1772 SPAN_ATTACH(Tracer, "file", File);
1773 std::shared_ptr<const ASTSignals> Signals;
1774 std::shared_ptr<const PreambleData> Preamble =
1775 It->second->Worker->getPossiblyStalePreamble(ASTSignals: &Signals);
1776 WithContext WithProvidedContext(Opts.ContextProvider(File));
1777 Action(InputsAndPreamble{.Contents: It->second->Contents,
1778 .Command: It->second->Worker->getCurrentCompileCommand(),
1779 .Preamble: Preamble.get(), .Signals: Signals.get()});
1780 return;
1781 }
1782
1783 std::shared_ptr<const ASTWorker> Worker = It->second->Worker.lock();
1784 auto Task = [Worker, Consistency, Name = Name.str(), File = File.str(),
1785 Contents = It->second->Contents,
1786 Command = Worker->getCurrentCompileCommand(),
1787 Ctx = Context::current().derive(Key: FileBeingProcessed,
1788 Value: std::string(File)),
1789 Action = std::move(Action), this]() mutable {
1790 clang::noteBottomOfStack();
1791 ThreadCrashReporter ScopedReporter([&Name, &Contents, &Command]() {
1792 llvm::errs() << "Signalled during preamble action: " << Name << "\n";
1793 crashDumpCompileCommand(OS&: llvm::errs(), Command);
1794 crashDumpFileContents(OS&: llvm::errs(), Contents);
1795 });
1796 std::shared_ptr<const PreambleData> Preamble;
1797 if (Consistency == PreambleConsistency::Stale) {
1798 // Wait until the preamble is built for the first time, if preamble
1799 // is required. This avoids extra work of processing the preamble
1800 // headers in parallel multiple times.
1801 Worker->waitForFirstPreamble();
1802 }
1803 std::shared_ptr<const ASTSignals> Signals;
1804 Preamble = Worker->getPossiblyStalePreamble(ASTSignals: &Signals);
1805
1806 std::lock_guard<Semaphore> BarrierLock(Barrier);
1807 WithContext Guard(std::move(Ctx));
1808 trace::Span Tracer(Name);
1809 SPAN_ATTACH(Tracer, "file", File);
1810 WithContext WithProvidedContext(Opts.ContextProvider(File));
1811 Action(InputsAndPreamble{.Contents: Contents, .Command: Command, .Preamble: Preamble.get(), .Signals: Signals.get()});
1812 };
1813
1814 PreambleTasks->runAsync(Name: "task:" + llvm::sys::path::filename(path: File),
1815 Action: std::move(Task));
1816}
1817
1818llvm::StringMap<TUScheduler::FileStats> TUScheduler::fileStats() const {
1819 llvm::StringMap<TUScheduler::FileStats> Result;
1820 for (const auto &PathAndFile : Files)
1821 Result.try_emplace(Key: PathAndFile.first(),
1822 Args: PathAndFile.second->Worker->stats());
1823 return Result;
1824}
1825
1826std::vector<Path> TUScheduler::getFilesWithCachedAST() const {
1827 std::vector<Path> Result;
1828 for (auto &&PathAndFile : Files) {
1829 if (!PathAndFile.second->Worker->isASTCached())
1830 continue;
1831 Result.push_back(x: std::string(PathAndFile.first()));
1832 }
1833 return Result;
1834}
1835
1836DebouncePolicy::clock::duration
1837DebouncePolicy::compute(llvm::ArrayRef<clock::duration> History) const {
1838 assert(Min <= Max && "Invalid policy");
1839 if (History.empty())
1840 return Max; // Arbitrary.
1841
1842 // Base the result on the median rebuild.
1843 // nth_element needs a mutable array, take the chance to bound the data size.
1844 History = History.take_back(N: 15);
1845 llvm::SmallVector<clock::duration, 15> Recent(History);
1846 auto *Median = Recent.begin() + Recent.size() / 2;
1847 std::nth_element(first: Recent.begin(), nth: Median, last: Recent.end());
1848
1849 clock::duration Target =
1850 std::chrono::duration_cast<clock::duration>(d: RebuildRatio * *Median);
1851 if (Target > Max)
1852 return Max;
1853 if (Target < Min)
1854 return Min;
1855 return Target;
1856}
1857
1858DebouncePolicy DebouncePolicy::fixed(clock::duration T) {
1859 DebouncePolicy P;
1860 P.Min = P.Max = T;
1861 return P;
1862}
1863
1864void TUScheduler::profile(MemoryTree &MT) const {
1865 for (const auto &Elem : fileStats()) {
1866 MT.detail(Name: Elem.first())
1867 .child(Name: "preamble")
1868 .addUsage(Increment: Opts.StorePreamblesInMemory ? Elem.second.UsedBytesPreamble
1869 : 0);
1870 MT.detail(Name: Elem.first()).child(Name: "ast").addUsage(Increment: Elem.second.UsedBytesAST);
1871 MT.child(Name: "header_includer_cache").addUsage(Increment: HeaderIncluders->getUsedBytes());
1872 }
1873}
1874} // namespace clangd
1875} // namespace clang
1876

Provided by KDAB

Privacy Policy
Learn to use CMake with our Intro Training
Find out more

source code of clang-tools-extra/clangd/TUScheduler.cpp