mirror of
https://github.com/NixOS/nix
synced 2025-06-30 07:33:16 +02:00
Merge remote-tracking branch 'origin/recursive-nix'
This commit is contained in:
commit
ac2bc721d8
7 changed files with 585 additions and 80 deletions
|
@ -13,6 +13,7 @@
|
|||
#include "nar-info.hh"
|
||||
#include "parsed-derivations.hh"
|
||||
#include "machines.hh"
|
||||
#include "daemon.hh"
|
||||
|
||||
#include <algorithm>
|
||||
#include <iostream>
|
||||
|
@ -33,6 +34,7 @@
|
|||
#include <sys/select.h>
|
||||
#include <sys/resource.h>
|
||||
#include <sys/socket.h>
|
||||
#include <sys/un.h>
|
||||
#include <fcntl.h>
|
||||
#include <netdb.h>
|
||||
#include <unistd.h>
|
||||
|
@ -765,9 +767,6 @@ private:
|
|||
immediate input paths). */
|
||||
PathSet inputPaths;
|
||||
|
||||
/* Referenceable paths (i.e., input and output paths). */
|
||||
PathSet allPaths;
|
||||
|
||||
/* Outputs that are already valid. If we're repairing, these are
|
||||
the outputs that are valid *and* not corrupt. */
|
||||
PathSet validPaths;
|
||||
|
@ -805,9 +804,13 @@ private:
|
|||
/* Pipe for the builder's standard output/error. */
|
||||
Pipe builderOut;
|
||||
|
||||
/* Pipe for synchronising updates to the builder user namespace. */
|
||||
/* Pipe for synchronising updates to the builder namespaces. */
|
||||
Pipe userNamespaceSync;
|
||||
|
||||
/* The mount namespace of the builder, used to add additional
|
||||
paths to the sandbox as a result of recursive Nix calls. */
|
||||
AutoCloseFD sandboxMountNamespace;
|
||||
|
||||
/* The build hook. */
|
||||
std::unique_ptr<HookInstance> hook;
|
||||
|
||||
|
@ -886,6 +889,29 @@ private:
|
|||
/* The remote machine on which we're building. */
|
||||
std::string machineName;
|
||||
|
||||
/* The recursive Nix daemon socket. */
|
||||
AutoCloseFD daemonSocket;
|
||||
|
||||
/* The daemon main thread. */
|
||||
std::thread daemonThread;
|
||||
|
||||
/* The daemon worker threads. */
|
||||
std::vector<std::thread> daemonWorkerThreads;
|
||||
|
||||
/* Paths that were added via recursive Nix calls. */
|
||||
PathSet addedPaths;
|
||||
|
||||
/* Recursive Nix calls are only allowed to build or realize paths
|
||||
in the original input closure or added via a recursive Nix call
|
||||
(so e.g. you can't do 'nix-store -r /nix/store/<bla>' where
|
||||
/nix/store/<bla> is some arbitrary path in a binary cache). */
|
||||
bool isAllowed(const Path & path)
|
||||
{
|
||||
return inputPaths.count(path) || addedPaths.count(path);
|
||||
}
|
||||
|
||||
friend class RestrictedStore;
|
||||
|
||||
public:
|
||||
DerivationGoal(const Path & drvPath, const StringSet & wantedOutputs,
|
||||
Worker & worker, BuildMode buildMode = bmNormal);
|
||||
|
@ -945,6 +971,14 @@ private:
|
|||
/* Write a JSON file containing the derivation attributes. */
|
||||
void writeStructuredAttrs();
|
||||
|
||||
void startDaemon();
|
||||
|
||||
void stopDaemon();
|
||||
|
||||
/* Add 'path' to the set of paths that may be referenced by the
|
||||
outputs, and make it appear in the sandbox. */
|
||||
void addDependency(const Path & path);
|
||||
|
||||
/* Make a file owned by the builder. */
|
||||
void chownToBuilder(const Path & path);
|
||||
|
||||
|
@ -1046,6 +1080,7 @@ DerivationGoal::~DerivationGoal()
|
|||
/* Careful: we should never ever throw an exception from a
|
||||
destructor. */
|
||||
try { killChild(); } catch (...) { ignoreException(); }
|
||||
try { stopDaemon(); } catch (...) { ignoreException(); }
|
||||
try { deleteTmpDir(false); } catch (...) { ignoreException(); }
|
||||
try { closeLogFile(); } catch (...) { ignoreException(); }
|
||||
}
|
||||
|
@ -1335,12 +1370,6 @@ void DerivationGoal::inputsRealised()
|
|||
/* Gather information necessary for computing the closure and/or
|
||||
running the build hook. */
|
||||
|
||||
/* The outputs are referenceable paths. */
|
||||
for (auto & i : drv->outputs) {
|
||||
debug(format("building path '%1%'") % i.second.path);
|
||||
allPaths.insert(i.second.path);
|
||||
}
|
||||
|
||||
/* Determine the full set of input paths. */
|
||||
|
||||
/* First, the input derivations. */
|
||||
|
@ -1365,8 +1394,6 @@ void DerivationGoal::inputsRealised()
|
|||
|
||||
debug(format("added input paths %1%") % showPaths(inputPaths));
|
||||
|
||||
allPaths.insert(inputPaths.begin(), inputPaths.end());
|
||||
|
||||
/* Is this a fixed-output derivation? */
|
||||
fixedOutput = drv->isFixedOutput();
|
||||
|
||||
|
@ -1530,6 +1557,8 @@ void DerivationGoal::buildDone()
|
|||
uid and then messing around with our output. */
|
||||
Finally releaseBuildUser([&]() { buildUser.reset(); });
|
||||
|
||||
sandboxMountNamespace = -1;
|
||||
|
||||
/* Since we got an EOF on the logger pipe, the builder is presumed
|
||||
to have terminated. In fact, the builder could also have
|
||||
simply have closed its end of the pipe, so just to be sure,
|
||||
|
@ -1561,6 +1590,9 @@ void DerivationGoal::buildDone()
|
|||
root. */
|
||||
if (buildUser) buildUser->kill();
|
||||
|
||||
/* Terminate the recursive Nix daemon. */
|
||||
stopDaemon();
|
||||
|
||||
bool diskFull = false;
|
||||
|
||||
try {
|
||||
|
@ -2213,6 +2245,11 @@ void DerivationGoal::startBuilder()
|
|||
}
|
||||
}
|
||||
|
||||
/* Fire up a Nix daemon to process recursive Nix calls from the
|
||||
builder. */
|
||||
if (parsedDrv->getRequiredSystemFeatures().count("recursive-nix"))
|
||||
startDaemon();
|
||||
|
||||
/* Run the builder. */
|
||||
printMsg(lvlChatty, format("executing builder '%1%'") % drv->builder);
|
||||
|
||||
|
@ -2387,6 +2424,12 @@ void DerivationGoal::startBuilder()
|
|||
writeFile("/proc/" + std::to_string(pid) + "/gid_map",
|
||||
(format("%d %d 1") % sandboxGid % hostGid).str());
|
||||
|
||||
/* Save the mount namespace of the child. We have to do this
|
||||
*before* the child does a chroot. */
|
||||
sandboxMountNamespace = open(fmt("/proc/%d/ns/mnt", (pid_t) pid).c_str(), O_RDONLY);
|
||||
if (sandboxMountNamespace.get() == -1)
|
||||
throw SysError("getting sandbox mount namespace");
|
||||
|
||||
/* Signal the builder that we've updated its user namespace. */
|
||||
writeFull(userNamespaceSync.writeSide.get(), "1");
|
||||
userNamespaceSync.writeSide = -1;
|
||||
|
@ -2629,6 +2672,319 @@ void DerivationGoal::writeStructuredAttrs()
|
|||
}
|
||||
|
||||
|
||||
/* A wrapper around LocalStore that only allows building/querying of
|
||||
paths that are in the input closures of the build or were added via
|
||||
recursive Nix calls. */
|
||||
struct RestrictedStore : public LocalFSStore
|
||||
{
|
||||
ref<LocalStore> next;
|
||||
|
||||
DerivationGoal & goal;
|
||||
|
||||
RestrictedStore(const Params & params, ref<LocalStore> next, DerivationGoal & goal)
|
||||
: Store(params), LocalFSStore(params), next(next), goal(goal)
|
||||
{ }
|
||||
|
||||
Path getRealStoreDir() override
|
||||
{ return next->realStoreDir; }
|
||||
|
||||
std::string getUri() override
|
||||
{ return next->getUri(); }
|
||||
|
||||
PathSet queryAllValidPaths() override
|
||||
{
|
||||
PathSet paths;
|
||||
for (auto & p : goal.inputPaths) paths.insert(p);
|
||||
for (auto & p : goal.addedPaths) paths.insert(p);
|
||||
return paths;
|
||||
}
|
||||
|
||||
void queryPathInfoUncached(const Path & path,
|
||||
Callback<std::shared_ptr<const ValidPathInfo>> callback) noexcept override
|
||||
{
|
||||
if (goal.isAllowed(path)) {
|
||||
try {
|
||||
/* Censor impure information. */
|
||||
auto info = std::make_shared<ValidPathInfo>(*next->queryPathInfo(path));
|
||||
info->deriver.clear();
|
||||
info->registrationTime = 0;
|
||||
info->ultimate = false;
|
||||
info->sigs.clear();
|
||||
callback(info);
|
||||
} catch (InvalidPath &) {
|
||||
callback(nullptr);
|
||||
}
|
||||
} else
|
||||
callback(nullptr);
|
||||
};
|
||||
|
||||
void queryReferrers(const Path & path, PathSet & referrers) override
|
||||
{ }
|
||||
|
||||
PathSet queryDerivationOutputs(const Path & path) override
|
||||
{ throw Error("queryDerivationOutputs"); }
|
||||
|
||||
StringSet queryDerivationOutputNames(const Path & path) override
|
||||
{ throw Error("queryDerivationOutputNames"); }
|
||||
|
||||
Path queryPathFromHashPart(const string & hashPart) override
|
||||
{ throw Error("queryPathFromHashPart"); }
|
||||
|
||||
Path addToStore(const string & name, const Path & srcPath,
|
||||
bool recursive = true, HashType hashAlgo = htSHA256,
|
||||
PathFilter & filter = defaultPathFilter, RepairFlag repair = NoRepair) override
|
||||
{ throw Error("addToStore"); }
|
||||
|
||||
void addToStore(const ValidPathInfo & info, Source & narSource,
|
||||
RepairFlag repair = NoRepair, CheckSigsFlag checkSigs = CheckSigs,
|
||||
std::shared_ptr<FSAccessor> accessor = 0) override
|
||||
{
|
||||
next->addToStore(info, narSource, repair, checkSigs, accessor);
|
||||
goal.addDependency(info.path);
|
||||
}
|
||||
|
||||
Path addToStoreFromDump(const string & dump, const string & name,
|
||||
bool recursive = true, HashType hashAlgo = htSHA256, RepairFlag repair = NoRepair) override
|
||||
{
|
||||
auto path = next->addToStoreFromDump(dump, name, recursive, hashAlgo, repair);
|
||||
goal.addDependency(path);
|
||||
return path;
|
||||
}
|
||||
|
||||
Path addTextToStore(const string & name, const string & s,
|
||||
const PathSet & references, RepairFlag repair = NoRepair) override
|
||||
{
|
||||
auto path = next->addTextToStore(name, s, references, repair);
|
||||
goal.addDependency(path);
|
||||
return path;
|
||||
}
|
||||
|
||||
void narFromPath(const Path & path, Sink & sink) override
|
||||
{
|
||||
if (!goal.isAllowed(path))
|
||||
throw InvalidPath("cannot dump unknown path '%s' in recursive Nix", path);
|
||||
LocalFSStore::narFromPath(path, sink);
|
||||
}
|
||||
|
||||
void ensurePath(const Path & path) override
|
||||
{
|
||||
if (!goal.isAllowed(path))
|
||||
throw InvalidPath("cannot substitute unknown path '%s' in recursive Nix", path);
|
||||
/* Nothing to be done; 'path' must already be valid. */
|
||||
}
|
||||
|
||||
void buildPaths(const PathSet & paths, BuildMode buildMode) override
|
||||
{
|
||||
if (buildMode != bmNormal) throw Error("unsupported build mode");
|
||||
|
||||
PathSet newPaths;
|
||||
|
||||
for (auto & path : paths) {
|
||||
DrvPathWithOutputs i = parseDrvPathWithOutputs(path);
|
||||
if (isDerivation(i.first)) {
|
||||
if (!goal.isAllowed(i.first))
|
||||
throw InvalidPath("cannot build unknown path '%s' in recursive Nix", i.first);
|
||||
auto drv = derivationFromPath(i.first);
|
||||
for (auto & output : drv.outputs)
|
||||
if (wantOutput(output.first, i.second))
|
||||
newPaths.insert(output.second.path);
|
||||
} else if (!goal.isAllowed(path))
|
||||
throw InvalidPath("cannot build unknown path '%s' in recursive Nix", path);
|
||||
}
|
||||
|
||||
next->buildPaths(paths, buildMode);
|
||||
|
||||
PathSet closure;
|
||||
next->computeFSClosure(newPaths, closure);
|
||||
for (auto & path : closure)
|
||||
goal.addDependency(path);
|
||||
}
|
||||
|
||||
BuildResult buildDerivation(const Path & drvPath, const BasicDerivation & drv,
|
||||
BuildMode buildMode = bmNormal) override
|
||||
{ unsupported("buildDerivation"); }
|
||||
|
||||
void addTempRoot(const Path & path)
|
||||
{ }
|
||||
|
||||
void addIndirectRoot(const Path & path)
|
||||
{ }
|
||||
|
||||
Roots findRoots()
|
||||
{ return Roots(); }
|
||||
|
||||
void collectGarbage(const GCOptions & options, GCResults & results)
|
||||
{ }
|
||||
|
||||
void addSignatures(const Path & storePath, const StringSet & sigs)
|
||||
{ unsupported("addSignatures"); }
|
||||
|
||||
void queryMissing(const PathSet & targets,
|
||||
PathSet & willBuild, PathSet & willSubstitute, PathSet & unknown,
|
||||
unsigned long long & downloadSize, unsigned long long & narSize)
|
||||
{
|
||||
/* This is slightly impure since it leaks information to the
|
||||
client about what paths will be built/substituted or are
|
||||
already present. Probably not a big deal. */
|
||||
|
||||
PathSet allowed;
|
||||
for (auto & path : targets) {
|
||||
DrvPathWithOutputs i = parseDrvPathWithOutputs(path);
|
||||
if (goal.isAllowed(i.first))
|
||||
allowed.insert(i.first);
|
||||
else
|
||||
unknown.insert(i.first);
|
||||
}
|
||||
|
||||
next->queryMissing(allowed, willBuild, willSubstitute,
|
||||
unknown, downloadSize, narSize);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void DerivationGoal::startDaemon()
|
||||
{
|
||||
settings.requireExperimentalFeature("recursive-nix");
|
||||
|
||||
Store::Params params;
|
||||
params["path-info-cache-size"] = "0";
|
||||
params["store"] = worker.store.storeDir;
|
||||
params["root"] = worker.store.rootDir;
|
||||
params["state"] = "/no-such-path";
|
||||
params["log"] = "/no-such-path";
|
||||
auto store = make_ref<RestrictedStore>(params,
|
||||
ref<LocalStore>(std::dynamic_pointer_cast<LocalStore>(worker.store.shared_from_this())),
|
||||
*this);
|
||||
|
||||
addedPaths.clear();
|
||||
|
||||
auto socketName = ".nix-socket";
|
||||
Path socketPath = tmpDir + "/" + socketName;
|
||||
env["NIX_REMOTE"] = "unix://" + tmpDirInSandbox + "/" + socketName;
|
||||
|
||||
daemonSocket = createUnixDomainSocket(socketPath, 0600);
|
||||
|
||||
chownToBuilder(socketPath);
|
||||
|
||||
daemonThread = std::thread([this, store]() {
|
||||
|
||||
while (true) {
|
||||
|
||||
/* Accept a connection. */
|
||||
struct sockaddr_un remoteAddr;
|
||||
socklen_t remoteAddrLen = sizeof(remoteAddr);
|
||||
|
||||
AutoCloseFD remote = accept(daemonSocket.get(),
|
||||
(struct sockaddr *) &remoteAddr, &remoteAddrLen);
|
||||
if (!remote) {
|
||||
if (errno == EINTR) continue;
|
||||
if (errno == EINVAL) break;
|
||||
throw SysError("accepting connection");
|
||||
}
|
||||
|
||||
closeOnExec(remote.get());
|
||||
|
||||
debug("received daemon connection");
|
||||
|
||||
auto workerThread = std::thread([this, store, remote{std::move(remote)}]() {
|
||||
FdSource from(remote.get());
|
||||
FdSink to(remote.get());
|
||||
try {
|
||||
daemon::processConnection(store, from, to,
|
||||
daemon::NotTrusted, daemon::Recursive, "nobody", 65535);
|
||||
debug("terminated daemon connection");
|
||||
} catch (SysError &) {
|
||||
ignoreException();
|
||||
}
|
||||
});
|
||||
|
||||
daemonWorkerThreads.push_back(std::move(workerThread));
|
||||
}
|
||||
|
||||
debug("daemon shutting down");
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
void DerivationGoal::stopDaemon()
|
||||
{
|
||||
if (daemonSocket && shutdown(daemonSocket.get(), SHUT_RDWR) == -1)
|
||||
throw SysError("shutting down daemon socket");
|
||||
|
||||
if (daemonThread.joinable())
|
||||
daemonThread.join();
|
||||
|
||||
// FIXME: should prune worker threads more quickly.
|
||||
// FIXME: shutdown the client socket to speed up worker termination.
|
||||
for (auto & thread : daemonWorkerThreads)
|
||||
thread.join();
|
||||
daemonWorkerThreads.clear();
|
||||
|
||||
daemonSocket = -1;
|
||||
}
|
||||
|
||||
|
||||
void DerivationGoal::addDependency(const Path & path)
|
||||
{
|
||||
worker.store.assertStorePath(path);
|
||||
|
||||
if (isAllowed(path)) return;
|
||||
|
||||
addedPaths.insert(path);
|
||||
|
||||
/* If we're doing a sandbox build, then we have to make the path
|
||||
appear in the sandbox. */
|
||||
if (useChroot) {
|
||||
|
||||
debug("materialising '%s' in the sandbox", path);
|
||||
|
||||
#if __linux__
|
||||
|
||||
Path source = worker.store.toRealPath(path);
|
||||
Path target = chrootRootDir + path;
|
||||
debug("bind-mounting %s -> %s", target, source);
|
||||
|
||||
if (pathExists(target))
|
||||
throw Error("store path '%s' already exists in the sandbox", path);
|
||||
|
||||
struct stat st;
|
||||
if (lstat(source.c_str(), &st))
|
||||
throw SysError("getting attributes of path '%s'", source);
|
||||
|
||||
if (S_ISDIR(st.st_mode)) {
|
||||
|
||||
/* Bind-mount the path into the sandbox. This requires
|
||||
entering its mount namespace, which is not possible
|
||||
in multithreaded programs. So we do this in a
|
||||
child process.*/
|
||||
Pid child(startProcess([&]() {
|
||||
|
||||
if (setns(sandboxMountNamespace.get(), 0) == -1)
|
||||
throw SysError("entering sandbox mount namespace");
|
||||
|
||||
createDirs(target);
|
||||
|
||||
if (mount(source.c_str(), target.c_str(), "", MS_BIND, 0) == -1)
|
||||
throw SysError("bind mount from '%s' to '%s' failed", source, target);
|
||||
|
||||
_exit(0);
|
||||
}));
|
||||
|
||||
int status = child.wait();
|
||||
if (status != 0)
|
||||
throw Error("could not add path '%s' to sandbox", path);
|
||||
|
||||
} else
|
||||
linkOrCopy(source, target);
|
||||
|
||||
#else
|
||||
throw Error("don't know how to make path '%s' (produced by a recursive Nix call) appear in the sandbox", path);
|
||||
#endif
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DerivationGoal::chownToBuilder(const Path & path)
|
||||
{
|
||||
if (!buildUser) return;
|
||||
|
@ -2764,15 +3120,30 @@ void DerivationGoal::runChild()
|
|||
outside of the namespace. Making a subtree private is
|
||||
local to the namespace, though, so setting MS_PRIVATE
|
||||
does not affect the outside world. */
|
||||
if (mount(0, "/", 0, MS_REC|MS_PRIVATE, 0) == -1) {
|
||||
throw SysError("unable to make '/' private mount");
|
||||
}
|
||||
if (mount(0, "/", 0, MS_PRIVATE | MS_REC, 0) == -1)
|
||||
throw SysError("unable to make '/' private");
|
||||
|
||||
/* Bind-mount chroot directory to itself, to treat it as a
|
||||
different filesystem from /, as needed for pivot_root. */
|
||||
if (mount(chrootRootDir.c_str(), chrootRootDir.c_str(), 0, MS_BIND, 0) == -1)
|
||||
throw SysError(format("unable to bind mount '%1%'") % chrootRootDir);
|
||||
|
||||
/* Bind-mount the sandbox's Nix store onto itself so that
|
||||
we can mark it as a "shared" subtree, allowing bind
|
||||
mounts made in *this* mount namespace to be propagated
|
||||
into the child namespace created by the
|
||||
unshare(CLONE_NEWNS) call below.
|
||||
|
||||
Marking chrootRootDir as MS_SHARED causes pivot_root()
|
||||
to fail with EINVAL. Don't know why. */
|
||||
Path chrootStoreDir = chrootRootDir + worker.store.storeDir;
|
||||
|
||||
if (mount(chrootStoreDir.c_str(), chrootStoreDir.c_str(), 0, MS_BIND, 0) == -1)
|
||||
throw SysError("unable to bind mount the Nix store", chrootStoreDir);
|
||||
|
||||
if (mount(0, chrootStoreDir.c_str(), 0, MS_SHARED, 0) == -1)
|
||||
throw SysError("unable to make '%s' shared", chrootStoreDir);
|
||||
|
||||
/* Set up a nearly empty /dev, unless the user asked to
|
||||
bind-mount the host /dev. */
|
||||
Strings ss;
|
||||
|
@ -2874,6 +3245,19 @@ void DerivationGoal::runChild()
|
|||
}
|
||||
}
|
||||
|
||||
/* Unshare this mount namespace. This is necessary because
|
||||
pivot_root() below changes the root of the mount
|
||||
namespace. This means that the call to setns() in
|
||||
addDependency() would hide the host's filesystem,
|
||||
making it impossible to bind-mount paths from the host
|
||||
Nix store into the sandbox. Therefore, we save the
|
||||
pre-pivot_root namespace in
|
||||
sandboxMountNamespace. Since we made /nix/store a
|
||||
shared subtree above, this allows addDependency() to
|
||||
make paths appear in the sandbox. */
|
||||
if (unshare(CLONE_NEWNS) == -1)
|
||||
throw SysError("unsharing mount namespace");
|
||||
|
||||
/* Do the chroot(). */
|
||||
if (chdir(chrootRootDir.c_str()) == -1)
|
||||
throw SysError(format("cannot change directory to '%1%'") % chrootRootDir);
|
||||
|
@ -3192,6 +3576,14 @@ void DerivationGoal::registerOutputs()
|
|||
|
||||
std::exception_ptr delayedException;
|
||||
|
||||
/* The paths that can be referenced are the input closures, the
|
||||
output paths, and any paths that have been built via recursive
|
||||
Nix calls. */
|
||||
PathSet referenceablePaths;
|
||||
for (auto & p : inputPaths) referenceablePaths.insert(p);
|
||||
for (auto & i : drv->outputs) referenceablePaths.insert(i.second.path);
|
||||
for (auto & p : addedPaths) referenceablePaths.insert(p);
|
||||
|
||||
/* Check whether the output paths were created, and grep each
|
||||
output path to determine what other paths it references. Also make all
|
||||
output paths read-only. */
|
||||
|
@ -3327,7 +3719,7 @@ void DerivationGoal::registerOutputs()
|
|||
verify later on whether nobody has messed with the store. */
|
||||
debug("scanning for references inside '%1%'", path);
|
||||
HashResult hash;
|
||||
PathSet references = scanForReferences(actualPath, allPaths, hash);
|
||||
PathSet references = scanForReferences(actualPath, referenceablePaths, hash);
|
||||
|
||||
if (buildMode == bmCheck) {
|
||||
if (!worker.store.isValidPath(path)) continue;
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue