aboutsummaryrefslogtreecommitdiff
path: root/src/libstore/gc.cc
diff options
context:
space:
mode:
Diffstat (limited to 'src/libstore/gc.cc')
-rw-r--r--src/libstore/gc.cc767
1 files changed, 379 insertions, 388 deletions
diff --git a/src/libstore/gc.cc b/src/libstore/gc.cc
index bc692ca42..7a414da6b 100644
--- a/src/libstore/gc.cc
+++ b/src/libstore/gc.cc
@@ -10,48 +10,22 @@
#include <regex>
#include <random>
-#include <sys/types.h>
-#include <sys/stat.h>
-#include <sys/statvfs.h>
+#include <climits>
#include <errno.h>
#include <fcntl.h>
+#include <poll.h>
+#include <sys/socket.h>
+#include <sys/stat.h>
+#include <sys/statvfs.h>
+#include <sys/types.h>
+#include <sys/un.h>
#include <unistd.h>
-#include <climits>
namespace nix {
-static string gcLockName = "gc.lock";
-static string gcRootsDir = "gcroots";
-
-
-/* Acquire the global GC lock. This is used to prevent new Nix
- processes from starting after the temporary root files have been
- read. To be precise: when they try to create a new temporary root
- file, they will block until the garbage collector has finished /
- yielded the GC lock. */
-AutoCloseFD LocalStore::openGCLock(LockType lockType)
-{
- Path fnGCLock = (format("%1%/%2%")
- % stateDir % gcLockName).str();
-
- debug(format("acquiring global GC lock '%1%'") % fnGCLock);
-
- AutoCloseFD fdGCLock = open(fnGCLock.c_str(), O_RDWR | O_CREAT | O_CLOEXEC, 0600);
- if (!fdGCLock)
- throw SysError("opening global GC lock '%1%'", fnGCLock);
-
- if (!lockFile(fdGCLock.get(), lockType, false)) {
- printInfo("waiting for the big garbage collector lock...");
- lockFile(fdGCLock.get(), lockType, true);
- }
-
- /* !!! Restrict read permission on the GC root. Otherwise any
- process that can open the file for reading can DoS the
- collector. */
-
- return fdGCLock;
-}
+static std::string gcSocketPath = "/gc-socket/socket";
+static std::string gcRootsDir = "gcroots";
static void makeSymlink(const Path & link, const Path & target)
@@ -71,12 +45,6 @@ static void makeSymlink(const Path & link, const Path & target)
}
-void LocalStore::syncWithGC()
-{
- AutoCloseFD fdGCLock = openGCLock(ltRead);
-}
-
-
void LocalStore::addIndirectRoot(const Path & path)
{
string hash = hashString(htSHA1, path).to_string(Base32, false);
@@ -95,6 +63,12 @@ Path LocalFSStore::addPermRoot(const StorePath & storePath, const Path & _gcRoot
"creating a garbage collector root (%1%) in the Nix store is forbidden "
"(are you running nix-build inside the store?)", gcRoot);
+ /* Register this root with the garbage collector, if it's
+ running. This should be superfluous since the caller should
+ have registered this root yet, but let's be on the safe
+ side. */
+ addTempRoot(storePath);
+
/* Don't clobber the link if it already exists and doesn't
point to the Nix store. */
if (pathExists(gcRoot) && (!isLink(gcRoot) || !isInStore(readLink(gcRoot))))
@@ -102,11 +76,6 @@ Path LocalFSStore::addPermRoot(const StorePath & storePath, const Path & _gcRoot
makeSymlink(gcRoot, printStorePath(storePath));
addIndirectRoot(gcRoot);
- /* Grab the global GC root, causing us to block while a GC is in
- progress. This prevents the set of permanent roots from
- increasing while a GC is in progress. */
- syncWithGC();
-
return gcRoot;
}
@@ -119,8 +88,6 @@ void LocalStore::addTempRoot(const StorePath & path)
if (!state->fdTempRoots) {
while (1) {
- AutoCloseFD fdGCLock = openGCLock(ltRead);
-
if (pathExists(fnTempRoots))
/* It *must* be stale, since there can be no two
processes with the same pid. */
@@ -128,10 +95,8 @@ void LocalStore::addTempRoot(const StorePath & path)
state->fdTempRoots = openLockFile(fnTempRoots, true);
- fdGCLock = -1;
-
- debug(format("acquiring read lock on '%1%'") % fnTempRoots);
- lockFile(state->fdTempRoots.get(), ltRead, true);
+ debug("acquiring write lock on '%s'", fnTempRoots);
+ lockFile(state->fdTempRoots.get(), ltWrite, true);
/* Check whether the garbage collector didn't get in our
way. */
@@ -147,24 +112,55 @@ void LocalStore::addTempRoot(const StorePath & path)
}
- /* Upgrade the lock to a write lock. This will cause us to block
- if the garbage collector is holding our lock. */
- debug(format("acquiring write lock on '%1%'") % fnTempRoots);
- lockFile(state->fdTempRoots.get(), ltWrite, true);
+ if (!state->fdGCLock)
+ state->fdGCLock = openGCLock();
+
+ restart:
+ FdLock gcLock(state->fdGCLock.get(), ltRead, false, "");
+
+ if (!gcLock.acquired) {
+ /* We couldn't get a shared global GC lock, so the garbage
+ collector is running. So we have to connect to the garbage
+ collector and inform it about our root. */
+ if (!state->fdRootsSocket) {
+ auto socketPath = stateDir.get() + gcSocketPath;
+ debug("connecting to '%s'", socketPath);
+ state->fdRootsSocket = createUnixDomainSocket();
+ nix::connect(state->fdRootsSocket.get(), socketPath);
+ }
+
+ try {
+ debug("sending GC root '%s'", printStorePath(path));
+ writeFull(state->fdRootsSocket.get(), printStorePath(path) + "\n", false);
+ char c;
+ readFull(state->fdRootsSocket.get(), &c, 1);
+ assert(c == '1');
+ debug("got ack for GC root '%s'", printStorePath(path));
+ } catch (SysError & e) {
+ /* The garbage collector may have exited, so we need to
+ restart. */
+ if (e.errNo == EPIPE) {
+ debug("GC socket disconnected");
+ state->fdRootsSocket.close();
+ goto restart;
+ }
+ } catch (EndOfFile & e) {
+ debug("GC socket disconnected");
+ state->fdRootsSocket.close();
+ goto restart;
+ }
+ }
+ /* Append the store path to the temporary roots file. */
string s = printStorePath(path) + '\0';
writeFull(state->fdTempRoots.get(), s);
-
- /* Downgrade to a read lock. */
- debug(format("downgrading to read lock on '%1%'") % fnTempRoots);
- lockFile(state->fdTempRoots.get(), ltRead, true);
}
static std::string censored = "{censored}";
-void LocalStore::findTempRoots(FDs & fds, Roots & tempRoots, bool censor)
+void LocalStore::findTempRoots(Roots & tempRoots, bool censor)
{
/* Read the `temproots' directory for per-process temporary root
files. */
@@ -179,35 +175,25 @@ void LocalStore::findTempRoots(FDs & fds, Roots & tempRoots, bool censor)
pid_t pid = std::stoi(i.name);
debug(format("reading temporary root file '%1%'") % path);
- FDPtr fd(new AutoCloseFD(open(path.c_str(), O_CLOEXEC | O_RDWR, 0666)));
- if (!*fd) {
+ AutoCloseFD fd(open(path.c_str(), O_CLOEXEC | O_RDWR, 0666));
+ if (!fd) {
/* It's okay if the file has disappeared. */
if (errno == ENOENT) continue;
throw SysError("opening temporary roots file '%1%'", path);
}
- /* This should work, but doesn't, for some reason. */
- //FDPtr fd(new AutoCloseFD(openLockFile(path, false)));
- //if (*fd == -1) continue;
-
/* Try to acquire a write lock without blocking. This can
only succeed if the owning process has died. In that case
we don't care about its temporary roots. */
- if (lockFile(fd->get(), ltWrite, false)) {
+ if (lockFile(fd.get(), ltWrite, false)) {
printInfo("removing stale temporary roots file '%1%'", path);
unlink(path.c_str());
- writeFull(fd->get(), "d");
+ writeFull(fd.get(), "d");
continue;
}
- /* Acquire a read lock. This will prevent the owning process
- from upgrading to a write lock, therefore it will block in
- addTempRoot(). */
- debug(format("waiting for read lock on '%1%'") % path);
- lockFile(fd->get(), ltRead, true);
-
/* Read the entire file. */
- string contents = readFile(fd->get());
+ string contents = readFile(fd.get());
/* Extract the roots. */
string::size_type pos = 0, end;
@@ -218,8 +204,6 @@ void LocalStore::findTempRoots(FDs & fds, Roots & tempRoots, bool censor)
tempRoots[parseStorePath(root)].emplace(censor ? censored : fmt("{temp:%d}", pid));
pos = end + 1;
}
-
- fds.push_back(fd); /* keep open */
}
}
@@ -304,8 +288,7 @@ Roots LocalStore::findRoots(bool censor)
Roots roots;
findRootsNoTemp(roots, censor);
- FDs fds;
- findTempRoots(fds, roots, censor);
+ findTempRoots(roots, censor);
return roots;
}
@@ -341,6 +324,7 @@ static string quoteRegexChars(const string & raw)
return std::regex_replace(raw, specialRegex, R"(\$&)");
}
+#if __linux__
static void readFileRoots(const char * path, UncheckedRoots & roots)
{
try {
@@ -350,6 +334,7 @@ static void readFileRoots(const char * path, UncheckedRoots & roots)
throw;
}
}
+#endif
void LocalStore::findRuntimeRoots(Roots & roots, bool censor)
{
@@ -431,7 +416,7 @@ void LocalStore::findRuntimeRoots(Roots & roots, bool censor)
}
#endif
-#if defined(__linux__)
+#if __linux__
readFileRoots("/proc/sys/kernel/modprobe", unchecked);
readFileRoots("/proc/sys/kernel/fbsplash", unchecked);
readFileRoots("/proc/sys/kernel/poweroff_cmd", unchecked);
@@ -455,391 +440,397 @@ void LocalStore::findRuntimeRoots(Roots & roots, bool censor)
struct GCLimitReached { };
-struct LocalStore::GCState
-{
- const GCOptions & options;
- GCResults & results;
- StorePathSet roots;
- StorePathSet tempRoots;
- StorePathSet dead;
- StorePathSet alive;
- bool gcKeepOutputs;
- bool gcKeepDerivations;
- uint64_t bytesInvalidated;
- bool moveToTrash = true;
- bool shouldDelete;
- GCState(const GCOptions & options, GCResults & results)
- : options(options), results(results), bytesInvalidated(0) { }
-};
-
-
-bool LocalStore::isActiveTempFile(const GCState & state,
- const Path & path, const string & suffix)
+void LocalStore::collectGarbage(const GCOptions & options, GCResults & results)
{
- return hasSuffix(path, suffix)
- && state.tempRoots.count(parseStorePath(string(path, 0, path.size() - suffix.size())));
-}
+ bool shouldDelete = options.action == GCOptions::gcDeleteDead || options.action == GCOptions::gcDeleteSpecific;
+ bool gcKeepOutputs = settings.gcKeepOutputs;
+ bool gcKeepDerivations = settings.gcKeepDerivations;
+ StorePathSet roots, dead, alive;
-void LocalStore::deleteGarbage(GCState & state, const Path & path)
-{
- uint64_t bytesFreed;
- deletePath(path, bytesFreed);
- state.results.bytesFreed += bytesFreed;
-}
+ struct Shared
+ {
+ // The temp roots only store the hash part to make it easier to
+ // ignore suffixes like '.lock', '.chroot' and '.check'.
+ std::unordered_set<std::string> tempRoots;
+ // Hash part of the store path currently being deleted, if
+ // any.
+ std::optional<std::string> pending;
+ };
-void LocalStore::deletePathRecursive(GCState & state, const Path & path)
-{
- checkInterrupt();
-
- uint64_t size = 0;
-
- auto storePath = maybeParseStorePath(path);
- if (storePath && isValidPath(*storePath)) {
- StorePathSet referrers;
- queryReferrers(*storePath, referrers);
- for (auto & i : referrers)
- if (printStorePath(i) != path) deletePathRecursive(state, printStorePath(i));
- size = queryPathInfo(*storePath)->narSize;
- invalidatePathChecked(*storePath);
- }
+ Sync<Shared> _shared;
- Path realPath = realStoreDir + "/" + std::string(baseNameOf(path));
+ std::condition_variable wakeup;
- struct stat st;
- if (lstat(realPath.c_str(), &st)) {
- if (errno == ENOENT) return;
- throw SysError("getting status of %1%", realPath);
+ /* Using `--ignore-liveness' with `--delete' can have unintended
+ consequences if `keep-outputs' or `keep-derivations' are true
+ (the garbage collector will recurse into deleting the outputs
+ or derivers, respectively). So disable them. */
+ if (options.action == GCOptions::gcDeleteSpecific && options.ignoreLiveness) {
+ gcKeepOutputs = false;
+ gcKeepDerivations = false;
}
- printInfo(format("deleting '%1%'") % path);
-
- state.results.paths.insert(path);
+ if (shouldDelete)
+ deletePath(reservedPath);
- /* If the path is not a regular file or symlink, move it to the
- trash directory. The move is to ensure that later (when we're
- not holding the global GC lock) we can delete the path without
- being afraid that the path has become alive again. Otherwise
- delete it right away. */
- if (state.moveToTrash && S_ISDIR(st.st_mode)) {
- // Estimate the amount freed using the narSize field. FIXME:
- // if the path was not valid, need to determine the actual
- // size.
- try {
- if (chmod(realPath.c_str(), st.st_mode | S_IWUSR) == -1)
- throw SysError("making '%1%' writable", realPath);
- Path tmp = trashDir + "/" + std::string(baseNameOf(path));
- if (rename(realPath.c_str(), tmp.c_str()))
- throw SysError("unable to rename '%1%' to '%2%'", realPath, tmp);
- state.bytesInvalidated += size;
- } catch (SysError & e) {
- if (e.errNo == ENOSPC) {
- printInfo(format("note: can't create move '%1%': %2%") % realPath % e.msg());
- deleteGarbage(state, realPath);
+ /* Acquire the global GC root. Note: we don't use fdGCLock
+ here because then in auto-gc mode, another thread could
+ downgrade our exclusive lock. */
+ auto fdGCLock = openGCLock();
+ FdLock gcLock(fdGCLock.get(), ltWrite, true, "waiting for the big garbage collector lock...");
+
+ /* Start the server for receiving new roots. */
+ auto socketPath = stateDir.get() + gcSocketPath;
+ createDirs(dirOf(socketPath));
+ auto fdServer = createUnixDomainSocket(socketPath, 0666);
+
+ if (fcntl(fdServer.get(), F_SETFL, fcntl(fdServer.get(), F_GETFL) | O_NONBLOCK) == -1)
+ throw SysError("making socket '%1%' non-blocking", socketPath);
+
+ Pipe shutdownPipe;
+ shutdownPipe.create();
+
+ std::thread serverThread([&]() {
+ Sync<std::map<int, std::thread>> connections;
+
+ Finally cleanup([&]() {
+ debug("GC roots server shutting down");
+ while (true) {
+ auto item = remove_begin(*connections.lock());
+ if (!item) break;
+ auto & [fd, thread] = *item;
+ shutdown(fd, SHUT_RDWR);
+ thread.join();
}
- }
- } else
- deleteGarbage(state, realPath);
-
- if (state.results.bytesFreed + state.bytesInvalidated > state.options.maxFreed) {
- printInfo(format("deleted or invalidated more than %1% bytes; stopping") % state.options.maxFreed);
- throw GCLimitReached();
- }
-}
-
-
-bool LocalStore::canReachRoot(GCState & state, StorePathSet & visited, const StorePath & path)
-{
- if (visited.count(path)) return false;
-
- if (state.alive.count(path)) return true;
-
- if (state.dead.count(path)) return false;
-
- if (state.roots.count(path)) {
- debug("cannot delete '%1%' because it's a root", printStorePath(path));
- state.alive.insert(path);
- return true;
- }
-
- visited.insert(path);
-
- if (!isValidPath(path)) return false;
-
- StorePathSet incoming;
-
- /* Don't delete this path if any of its referrers are alive. */
- queryReferrers(path, incoming);
-
- /* If keep-derivations is set and this is a derivation, then
- don't delete the derivation if any of the outputs are alive. */
- if (state.gcKeepDerivations && path.isDerivation()) {
- for (auto & [name, maybeOutPath] : queryPartialDerivationOutputMap(path))
- if (maybeOutPath &&
- isValidPath(*maybeOutPath) &&
- queryPathInfo(*maybeOutPath)->deriver == path
- )
- incoming.insert(*maybeOutPath);
- }
-
- /* If keep-outputs is set, then don't delete this path if there
- are derivers of this path that are not garbage. */
- if (state.gcKeepOutputs) {
- auto derivers = queryValidDerivers(path);
- for (auto & i : derivers)
- incoming.insert(i);
- }
+ });
+
+ while (true) {
+ std::vector<struct pollfd> fds;
+ fds.push_back({.fd = shutdownPipe.readSide.get(), .events = POLLIN});
+ fds.push_back({.fd = fdServer.get(), .events = POLLIN});
+ auto count = poll(fds.data(), fds.size(), -1);
+ assert(count != -1);
+
+ if (fds[0].revents)
+ /* Parent is asking us to quit. */
+ break;
+
+ if (fds[1].revents) {
+ /* Accept a new connection. */
+ assert(fds[1].revents & POLLIN);
+ AutoCloseFD fdClient = accept(fdServer.get(), nullptr, nullptr);
+ if (!fdClient) continue;
+
+ /* Process the connection in a separate thread. */
+ auto fdClient_ = fdClient.get();
+ std::thread clientThread([&, fdClient = std::move(fdClient)]() {
+ Finally cleanup([&]() {
+ auto conn(connections.lock());
+ auto i = conn->find(fdClient.get());
+ if (i != conn->end()) {
+ i->second.detach();
+ conn->erase(i);
+ }
+ });
+
+ while (true) {
+ try {
+ auto path = readLine(fdClient.get());
+ auto storePath = maybeParseStorePath(path);
+ if (storePath) {
+ debug("got new GC root '%s'", path);
+ auto hashPart = std::string(storePath->hashPart());
+ auto shared(_shared.lock());
+ shared->tempRoots.insert(hashPart);
+ /* If this path is currently being
+ deleted, then we have to wait until
+ deletion is finished to ensure that
+ the client doesn't start
+ re-creating it before we're
+ done. FIXME: ideally we would use a
+ FD for this so we don't block the
+ poll loop. */
+ while (shared->pending == hashPart) {
+ debug("synchronising with deletion of path '%s'", path);
+ shared.wait(wakeup);
+ }
+ } else
+ printError("received garbage instead of a root from client");
+ writeFull(fdClient.get(), "1", false);
+ } catch (Error &) { break; }
+ }
+ });
- for (auto & i : incoming)
- if (i != path)
- if (canReachRoot(state, visited, i)) {
- state.alive.insert(path);
- return true;
+ connections.lock()->insert({fdClient_, std::move(clientThread)});
}
+ }
+ });
- return false;
-}
-
-
-void LocalStore::tryToDelete(GCState & state, const Path & path)
-{
- checkInterrupt();
-
- auto realPath = realStoreDir + "/" + std::string(baseNameOf(path));
- if (realPath == linksDir || realPath == trashDir) return;
-
- //Activity act(*logger, lvlDebug, format("considering whether to delete '%1%'") % path);
-
- auto storePath = maybeParseStorePath(path);
-
- if (!storePath || !isValidPath(*storePath)) {
- /* A lock file belonging to a path that we're building right
- now isn't garbage. */
- if (isActiveTempFile(state, path, ".lock")) return;
+ Finally stopServer([&]() {
+ writeFull(shutdownPipe.writeSide.get(), "x", false);
+ wakeup.notify_all();
+ if (serverThread.joinable()) serverThread.join();
+ });
- /* Don't delete .chroot directories for derivations that are
- currently being built. */
- if (isActiveTempFile(state, path, ".chroot")) return;
+ /* Find the roots. Since we've grabbed the GC lock, the set of
+ permanent roots cannot increase now. */
+ printInfo("finding garbage collector roots...");
+ Roots rootMap;
+ if (!options.ignoreLiveness)
+ findRootsNoTemp(rootMap, true);
- /* Don't delete .check directories for derivations that are
- currently being built, because we may need to run
- diff-hook. */
- if (isActiveTempFile(state, path, ".check")) return;
- }
+ for (auto & i : rootMap) roots.insert(i.first);
- StorePathSet visited;
-
- if (storePath && canReachRoot(state, visited, *storePath)) {
- debug("cannot delete '%s' because it's still reachable", path);
- } else {
- /* No path we visited was a root, so everything is garbage.
- But we only delete ‘path’ and its referrers here so that
- ‘nix-store --delete’ doesn't have the unexpected effect of
- recursing into derivations and outputs. */
- for (auto & i : visited)
- state.dead.insert(i);
- if (state.shouldDelete)
- deletePathRecursive(state, path);
+ /* Read the temporary roots created before we acquired the global
+ GC root. Any new roots will be sent to our socket. */
+ Roots tempRoots;
+ findTempRoots(tempRoots, true);
+ for (auto & root : tempRoots) {
+ _shared.lock()->tempRoots.insert(std::string(root.first.hashPart()));
+ roots.insert(root.first);
}
-}
+ /* Helper function that deletes a path from the store and throws
+ GCLimitReached if we've deleted enough garbage. */
+ auto deleteFromStore = [&](std::string_view baseName)
+ {
+ Path path = storeDir + "/" + std::string(baseName);
+ Path realPath = realStoreDir + "/" + std::string(baseName);
-/* Unlink all files in /nix/store/.links that have a link count of 1,
- which indicates that there are no other links and so they can be
- safely deleted. FIXME: race condition with optimisePath(): we
- might see a link count of 1 just before optimisePath() increases
- the link count. */
-void LocalStore::removeUnusedLinks(const GCState & state)
-{
- AutoCloseDir dir(opendir(linksDir.c_str()));
- if (!dir) throw SysError("opening directory '%1%'", linksDir);
-
- int64_t actualSize = 0, unsharedSize = 0;
+ printInfo("deleting '%1%'", path);
- struct dirent * dirent;
- while (errno = 0, dirent = readdir(dir.get())) {
- checkInterrupt();
- string name = dirent->d_name;
- if (name == "." || name == "..") continue;
- Path path = linksDir + "/" + name;
+ results.paths.insert(path);
- auto st = lstat(path);
+ uint64_t bytesFreed;
+ deletePath(realPath, bytesFreed);
+ results.bytesFreed += bytesFreed;
- if (st.st_nlink != 1) {
- actualSize += st.st_size;
- unsharedSize += (st.st_nlink - 1) * st.st_size;
- continue;
+ if (results.bytesFreed > options.maxFreed) {
+ printInfo("deleted more than %d bytes; stopping", options.maxFreed);
+ throw GCLimitReached();
}
+ };
- printMsg(lvlTalkative, format("deleting unused link '%1%'") % path);
-
- if (unlink(path.c_str()) == -1)
- throw SysError("deleting '%1%'", path);
+ std::map<StorePath, StorePathSet> referrersCache;
- state.results.bytesFreed += st.st_size;
- }
+ /* Helper function that visits all paths reachable from `start`
+ via the referrers edges and optionally derivers and derivation
+ output edges. If none of those paths are roots, then all
+ visited paths are garbage and are deleted. */
+ auto deleteReferrersClosure = [&](const StorePath & start) {
+ StorePathSet visited;
+ std::queue<StorePath> todo;
- struct stat st;
- if (stat(linksDir.c_str(), &st) == -1)
- throw SysError("statting '%1%'", linksDir);
- int64_t overhead = st.st_blocks * 512ULL;
+ /* Wake up any GC client waiting for deletion of the paths in
+ 'visited' to finish. */
+ Finally releasePending([&]() {
+ auto shared(_shared.lock());
+ shared->pending.reset();
+ wakeup.notify_all();
+ });
- printInfo("note: currently hard linking saves %.2f MiB",
- ((unsharedSize - actualSize - overhead) / (1024.0 * 1024.0)));
-}
+ auto enqueue = [&](const StorePath & path) {
+ if (visited.insert(path).second)
+ todo.push(path);
+ };
+ enqueue(start);
-void LocalStore::collectGarbage(const GCOptions & options, GCResults & results)
-{
- GCState state(options, results);
- state.gcKeepOutputs = settings.gcKeepOutputs;
- state.gcKeepDerivations = settings.gcKeepDerivations;
+ while (auto path = pop(todo)) {
+ checkInterrupt();
- /* Using `--ignore-liveness' with `--delete' can have unintended
- consequences if `keep-outputs' or `keep-derivations' are true
- (the garbage collector will recurse into deleting the outputs
- or derivers, respectively). So disable them. */
- if (options.action == GCOptions::gcDeleteSpecific && options.ignoreLiveness) {
- state.gcKeepOutputs = false;
- state.gcKeepDerivations = false;
- }
+ /* Bail out if we've previously discovered that this path
+ is alive. */
+ if (alive.count(*path)) {
+ alive.insert(start);
+ return;
+ }
- state.shouldDelete = options.action == GCOptions::gcDeleteDead || options.action == GCOptions::gcDeleteSpecific;
+ /* If we've previously deleted this path, we don't have to
+ handle it again. */
+ if (dead.count(*path)) continue;
- if (state.shouldDelete)
- deletePath(reservedPath);
+ auto markAlive = [&]()
+ {
+ alive.insert(*path);
+ alive.insert(start);
+ try {
+ StorePathSet closure;
+ computeFSClosure(*path, closure);
+ for (auto & p : closure)
+ alive.insert(p);
+ } catch (InvalidPath &) { }
+ };
+
+ /* If this is a root, bail out. */
+ if (roots.count(*path)) {
+ debug("cannot delete '%s' because it's a root", printStorePath(*path));
+ return markAlive();
+ }
- /* Acquire the global GC root. This prevents
- a) New roots from being added.
- b) Processes from creating new temporary root files. */
- AutoCloseFD fdGCLock = openGCLock(ltWrite);
+ if (options.action == GCOptions::gcDeleteSpecific
+ && !options.pathsToDelete.count(*path))
+ return;
- /* Find the roots. Since we've grabbed the GC lock, the set of
- permanent roots cannot increase now. */
- printInfo("finding garbage collector roots...");
- Roots rootMap;
- if (!options.ignoreLiveness)
- findRootsNoTemp(rootMap, true);
+ {
+ auto hashPart = std::string(path->hashPart());
+ auto shared(_shared.lock());
+ if (shared->tempRoots.count(hashPart)) {
+ debug("cannot delete '%s' because it's a temporary root", printStorePath(*path));
+ return markAlive();
+ }
+ shared->pending = hashPart;
+ }
- for (auto & i : rootMap) state.roots.insert(i.first);
+ if (isValidPath(*path)) {
- /* Read the temporary roots. This acquires read locks on all
- per-process temporary root files. So after this point no paths
- can be added to the set of temporary roots. */
- FDs fds;
- Roots tempRoots;
- findTempRoots(fds, tempRoots, true);
- for (auto & root : tempRoots) {
- state.tempRoots.insert(root.first);
- state.roots.insert(root.first);
- }
+ /* Visit the referrers of this path. */
+ auto i = referrersCache.find(*path);
+ if (i == referrersCache.end()) {
+ StorePathSet referrers;
+ queryReferrers(*path, referrers);
+ referrersCache.emplace(*path, std::move(referrers));
+ i = referrersCache.find(*path);
+ }
+ for (auto & p : i->second)
+ enqueue(p);
+
+ /* If keep-derivations is set and this is a
+ derivation, then visit the derivation outputs. */
+ if (gcKeepDerivations && path->isDerivation()) {
+ for (auto & [name, maybeOutPath] : queryPartialDerivationOutputMap(*path))
+ if (maybeOutPath &&
+ isValidPath(*maybeOutPath) &&
+ queryPathInfo(*maybeOutPath)->deriver == *path)
+ enqueue(*maybeOutPath);
+ }
- /* After this point the set of roots or temporary roots cannot
- increase, since we hold locks on everything. So everything
- that is not reachable from `roots' is garbage. */
+ /* If keep-outputs is set, then visit the derivers. */
+ if (gcKeepOutputs) {
+ auto derivers = queryValidDerivers(*path);
+ for (auto & i : derivers)
+ enqueue(i);
+ }
+ }
+ }
- if (state.shouldDelete) {
- if (pathExists(trashDir)) deleteGarbage(state, trashDir);
- try {
- createDirs(trashDir);
- } catch (SysError & e) {
- if (e.errNo == ENOSPC) {
- printInfo("note: can't create trash directory: %s", e.msg());
- state.moveToTrash = false;
+ for (auto & path : topoSortPaths(visited)) {
+ if (!dead.insert(path).second) continue;
+ if (shouldDelete) {
+ invalidatePathChecked(path);
+ deleteFromStore(path.to_string());
+ referrersCache.erase(path);
}
}
- }
+ };
- /* Now either delete all garbage paths, or just the specified
- paths (for gcDeleteSpecific). */
+ /* Synchronisation point for testing, see tests/gc-concurrent.sh. */
+ if (auto p = getEnv("_NIX_TEST_GC_SYNC"))
+ readFile(*p);
+ /* Either delete all garbage paths, or just the specified
+ paths (for gcDeleteSpecific). */
if (options.action == GCOptions::gcDeleteSpecific) {
for (auto & i : options.pathsToDelete) {
- tryToDelete(state, printStorePath(i));
- if (state.dead.find(i) == state.dead.end())
+ deleteReferrersClosure(i);
+ if (!dead.count(i))
throw Error(
- "cannot delete path '%1%' since it is still alive. "
- "To find out why use: "
+ "Cannot delete path '%1%' since it is still alive. "
+ "To find out why, use: "
"nix-store --query --roots",
printStorePath(i));
}
} else if (options.maxFreed > 0) {
- if (state.shouldDelete)
+ if (shouldDelete)
printInfo("deleting garbage...");
else
printInfo("determining live/dead paths...");
try {
-
- AutoCloseDir dir(opendir(realStoreDir.c_str()));
+ AutoCloseDir dir(opendir(realStoreDir.get().c_str()));
if (!dir) throw SysError("opening directory '%1%'", realStoreDir);
- /* Read the store and immediately delete all paths that
- aren't valid. When using --max-freed etc., deleting
- invalid paths is preferred over deleting unreachable
- paths, since unreachable paths could become reachable
- again. We don't use readDirectory() here so that GCing
- can start faster. */
+ /* Read the store and delete all paths that are invalid or
+ unreachable. We don't use readDirectory() here so that
+ GCing can start faster. */
+ auto linksName = baseNameOf(linksDir);
Paths entries;
struct dirent * dirent;
while (errno = 0, dirent = readdir(dir.get())) {
checkInterrupt();
string name = dirent->d_name;
- if (name == "." || name == "..") continue;
- Path path = storeDir + "/" + name;
- auto storePath = maybeParseStorePath(path);
- if (storePath && isValidPath(*storePath))
- entries.push_back(path);
- else
- tryToDelete(state, path);
- }
-
- dir.reset();
-
- /* Now delete the unreachable valid paths. Randomise the
- order in which we delete entries to make the collector
- less biased towards deleting paths that come
- alphabetically first (e.g. /nix/store/000...). This
- matters when using --max-freed etc. */
- vector<Path> entries_(entries.begin(), entries.end());
- std::mt19937 gen(1);
- std::shuffle(entries_.begin(), entries_.end(), gen);
+ if (name == "." || name == ".." || name == linksName) continue;
- for (auto & i : entries_)
- tryToDelete(state, i);
+ if (auto storePath = maybeParseStorePath(storeDir + "/" + name))
+ deleteReferrersClosure(*storePath);
+ else
+ deleteFromStore(name);
+ }
} catch (GCLimitReached & e) {
}
}
- if (state.options.action == GCOptions::gcReturnLive) {
- for (auto & i : state.alive)
- state.results.paths.insert(printStorePath(i));
+ if (options.action == GCOptions::gcReturnLive) {
+ for (auto & i : alive)
+ results.paths.insert(printStorePath(i));
return;
}
- if (state.options.action == GCOptions::gcReturnDead) {
- for (auto & i : state.dead)
- state.results.paths.insert(printStorePath(i));
+ if (options.action == GCOptions::gcReturnDead) {
+ for (auto & i : dead)
+ results.paths.insert(printStorePath(i));
return;
}
- /* Allow other processes to add to the store from here on. */
- fdGCLock = -1;
- fds.clear();
-
- /* Delete the trash directory. */
- printInfo(format("deleting '%1%'") % trashDir);
- deleteGarbage(state, trashDir);
-
- /* Clean up the links directory. */
+ /* Unlink all files in /nix/store/.links that have a link count of 1,
+ which indicates that there are no other links and so they can be
+ safely deleted. FIXME: race condition with optimisePath(): we
+ might see a link count of 1 just before optimisePath() increases
+ the link count. */
if (options.action == GCOptions::gcDeleteDead || options.action == GCOptions::gcDeleteSpecific) {
printInfo("deleting unused links...");
- removeUnusedLinks(state);
+
+ AutoCloseDir dir(opendir(linksDir.c_str()));
+ if (!dir) throw SysError("opening directory '%1%'", linksDir);
+
+ int64_t actualSize = 0, unsharedSize = 0;
+
+ struct dirent * dirent;
+ while (errno = 0, dirent = readdir(dir.get())) {
+ checkInterrupt();
+ string name = dirent->d_name;
+ if (name == "." || name == "..") continue;
+ Path path = linksDir + "/" + name;
+
+ auto st = lstat(path);
+
+ if (st.st_nlink != 1) {
+ actualSize += st.st_size;
+ unsharedSize += (st.st_nlink - 1) * st.st_size;
+ continue;
+ }
+
+ printMsg(lvlTalkative, format("deleting unused link '%1%'") % path);
+
+ if (unlink(path.c_str()) == -1)
+ throw SysError("deleting '%1%'", path);
+
+ results.bytesFreed += st.st_size;
+ }
+
+ struct stat st;
+ if (stat(linksDir.c_str(), &st) == -1)
+ throw SysError("statting '%1%'", linksDir);
+ int64_t overhead = st.st_blocks * 512ULL;
+
+ printInfo("note: currently hard linking saves %.2f MiB",
+ ((unsharedSize - actualSize - overhead) / (1024.0 * 1024.0)));
}
/* While we're at it, vacuum the database. */
@@ -856,7 +847,7 @@ void LocalStore::autoGC(bool sync)
return std::stoll(readFile(*fakeFreeSpaceFile));
struct statvfs st;
- if (statvfs(realStoreDir.c_str(), &st))
+ if (statvfs(realStoreDir.get().c_str(), &st))
throw SysError("getting filesystem info about '%s'", realStoreDir);
return (uint64_t) st.f_bavail * st.f_frsize;