aboutsummaryrefslogtreecommitdiff
path: root/src/libstore/build
diff options
context:
space:
mode:
Diffstat (limited to 'src/libstore/build')
-rw-r--r--src/libstore/build/derivation-goal.cc268
-rw-r--r--src/libstore/build/derivation-goal.hh3
-rw-r--r--src/libstore/build/drv-output-substitution-goal.cc122
-rw-r--r--src/libstore/build/drv-output-substitution-goal.hh50
-rw-r--r--src/libstore/build/entry-points.cc30
-rw-r--r--src/libstore/build/goal.cc15
-rw-r--r--src/libstore/build/goal.hh4
-rw-r--r--src/libstore/build/local-derivation-goal.cc352
-rw-r--r--src/libstore/build/local-derivation-goal.hh11
-rw-r--r--src/libstore/build/substitution-goal.cc67
-rw-r--r--src/libstore/build/substitution-goal.hh13
-rw-r--r--src/libstore/build/worker.cc46
-rw-r--r--src/libstore/build/worker.hh20
13 files changed, 610 insertions, 391 deletions
diff --git a/src/libstore/build/derivation-goal.cc b/src/libstore/build/derivation-goal.cc
index c29237f5c..0907120db 100644
--- a/src/libstore/build/derivation-goal.cc
+++ b/src/libstore/build/derivation-goal.cc
@@ -20,6 +20,7 @@
#include <sys/types.h>
#include <sys/socket.h>
#include <sys/un.h>
+#include <sys/wait.h>
#include <netdb.h>
#include <fcntl.h>
#include <termios.h>
@@ -73,7 +74,7 @@ DerivationGoal::DerivationGoal(const StorePath & drvPath,
state = &DerivationGoal::getDerivation;
name = fmt(
"building of '%s' from .drv file",
- StorePathWithOutputs { drvPath, wantedOutputs }.to_string(worker.store));
+ DerivedPath::Built { drvPath, wantedOutputs }.to_string(worker.store));
trace("created");
mcExpectedBuilds = std::make_unique<MaintainCount<uint64_t>>(worker.expectedBuilds);
@@ -94,7 +95,7 @@ DerivationGoal::DerivationGoal(const StorePath & drvPath, const BasicDerivation
state = &DerivationGoal::haveDerivation;
name = fmt(
"building of '%s' from in-memory derivation",
- StorePathWithOutputs { drvPath, drv.outputNames() }.to_string(worker.store));
+ DerivedPath::Built { drvPath, drv.outputNames() }.to_string(worker.store));
trace("created");
mcExpectedBuilds = std::make_unique<MaintainCount<uint64_t>>(worker.expectedBuilds);
@@ -142,7 +143,6 @@ void DerivationGoal::work()
(this->*state)();
}
-
void DerivationGoal::addWantedOutputs(const StringSet & outputs)
{
/* If we already want all outputs, there is nothing to do. */
@@ -165,12 +165,12 @@ void DerivationGoal::getDerivation()
/* The first thing to do is to make sure that the derivation
exists. If it doesn't, it may be created through a
substitute. */
- if (buildMode == bmNormal && worker.store.isValidPath(drvPath)) {
+ if (buildMode == bmNormal && worker.evalStore.isValidPath(drvPath)) {
loadDerivation();
return;
}
- addWaitee(upcast_goal(worker.makeSubstitutionGoal(drvPath)));
+ addWaitee(upcast_goal(worker.makePathSubstitutionGoal(drvPath)));
state = &DerivationGoal::loadDerivation;
}
@@ -188,12 +188,12 @@ void DerivationGoal::loadDerivation()
/* `drvPath' should already be a root, but let's be on the safe
side: if the user forgot to make it a root, we wouldn't want
things being garbage collected while we're busy. */
- worker.store.addTempRoot(drvPath);
+ worker.evalStore.addTempRoot(drvPath);
- assert(worker.store.isValidPath(drvPath));
+ assert(worker.evalStore.isValidPath(drvPath));
/* Get the derivation. */
- drv = std::make_unique<Derivation>(worker.store.derivationFromPath(drvPath));
+ drv = std::make_unique<Derivation>(worker.evalStore.derivationFromPath(drvPath));
haveDerivation();
}
@@ -212,8 +212,8 @@ void DerivationGoal::haveDerivation()
if (i.second.second)
worker.store.addTempRoot(*i.second.second);
- auto outputHashes = staticOutputHashes(worker.store, *drv);
- for (auto &[outputName, outputHash] : outputHashes)
+ auto outputHashes = staticOutputHashes(worker.evalStore, *drv);
+ for (auto & [outputName, outputHash] : outputHashes)
initialOutputs.insert({
outputName,
InitialOutput{
@@ -246,17 +246,22 @@ void DerivationGoal::haveDerivation()
through substitutes. If that doesn't work, we'll build
them. */
if (settings.useSubstitutes && parsedDrv->substitutesAllowed())
- for (auto & [_, status] : initialOutputs) {
+ for (auto & [outputName, status] : initialOutputs) {
if (!status.wanted) continue;
- if (!status.known) {
- warn("do not know how to query for unknown floating content-addressed derivation output yet");
- /* Nothing to wait for; tail call */
- return DerivationGoal::gaveUpOnSubstitution();
- }
- addWaitee(upcast_goal(worker.makeSubstitutionGoal(
- status.known->path,
- buildMode == bmRepair ? Repair : NoRepair,
- getDerivationCA(*drv))));
+ if (!status.known)
+ addWaitee(
+ upcast_goal(
+ worker.makeDrvOutputSubstitutionGoal(
+ DrvOutput{status.outputHash, outputName},
+ buildMode == bmRepair ? Repair : NoRepair
+ )
+ )
+ );
+ else
+ addWaitee(upcast_goal(worker.makePathSubstitutionGoal(
+ status.known->path,
+ buildMode == bmRepair ? Repair : NoRepair,
+ getDerivationCA(*drv))));
}
if (waitees.empty()) /* to prevent hang (no wake-up event) */
@@ -332,12 +337,21 @@ void DerivationGoal::gaveUpOnSubstitution()
for (auto & i : dynamic_cast<Derivation *>(drv.get())->inputDrvs)
addWaitee(worker.makeDerivationGoal(i.first, i.second, buildMode == bmRepair ? bmRepair : bmNormal));
+ /* Copy the input sources from the eval store to the build
+ store. */
+ if (&worker.evalStore != &worker.store) {
+ RealisedPath::Set inputSrcs;
+ for (auto & i : drv->inputSrcs)
+ inputSrcs.insert(i);
+ copyClosure(worker.evalStore, worker.store, inputSrcs);
+ }
+
for (auto & i : drv->inputSrcs) {
if (worker.store.isValidPath(i)) continue;
if (!settings.useSubstitutes)
throw Error("dependency '%s' of '%s' does not exist, and substitution is disabled",
worker.store.printStorePath(i), worker.store.printStorePath(drvPath));
- addWaitee(upcast_goal(worker.makeSubstitutionGoal(i)));
+ addWaitee(upcast_goal(worker.makePathSubstitutionGoal(i)));
}
if (waitees.empty()) /* to prevent hang (no wake-up event) */
@@ -388,7 +402,7 @@ void DerivationGoal::repairClosure()
worker.store.printStorePath(i), worker.store.printStorePath(drvPath));
auto drvPath2 = outputsToDrv.find(i);
if (drvPath2 == outputsToDrv.end())
- addWaitee(upcast_goal(worker.makeSubstitutionGoal(i, Repair)));
+ addWaitee(upcast_goal(worker.makePathSubstitutionGoal(i, Repair)));
else
addWaitee(worker.makeDerivationGoal(drvPath2->second, StringSet(), bmRepair));
}
@@ -473,8 +487,8 @@ void DerivationGoal::inputsRealised()
/* Add the relevant output closures of the input derivation
`i' as input paths. Only add the closures of output paths
that are specified as inputs. */
- assert(worker.store.isValidPath(drvPath));
- auto outputs = worker.store.queryPartialDerivationOutputMap(depDrvPath);
+ assert(worker.evalStore.isValidPath(drvPath));
+ auto outputs = worker.evalStore.queryPartialDerivationOutputMap(depDrvPath);
for (auto & j : wantedDepOutputs) {
if (outputs.count(j) > 0) {
auto optRealizedInput = outputs.at(j);
@@ -539,7 +553,7 @@ void DerivationGoal::tryToBuild()
PathSet lockFiles;
/* FIXME: Should lock something like the drv itself so we don't build same
CA drv concurrently */
- if (dynamic_cast<LocalStore *>(&worker.store))
+ if (dynamic_cast<LocalStore *>(&worker.store)) {
/* If we aren't a local store, we might need to use the local store as
a build remote, but that would cause a deadlock. */
/* FIXME: Make it so we can use ourselves as a build remote even if we
@@ -547,9 +561,15 @@ void DerivationGoal::tryToBuild()
/* FIXME: find some way to lock for scheduling for the other stores so
a forking daemon with --store still won't farm out redundant builds.
*/
- for (auto & i : drv->outputsAndOptPaths(worker.store))
+ for (auto & i : drv->outputsAndOptPaths(worker.store)) {
if (i.second.second)
lockFiles.insert(worker.store.Store::toRealPath(*i.second.second));
+ else
+ lockFiles.insert(
+ worker.store.Store::toRealPath(drvPath) + "." + i.first
+ );
+ }
+ }
if (!outputLocks.lockPaths(lockFiles, "", false)) {
if (!actLock)
@@ -733,6 +753,64 @@ void DerivationGoal::cleanupPostOutputsRegisteredModeNonCheck()
{
}
+void runPostBuildHook(
+ Store & store,
+ Logger & logger,
+ const StorePath & drvPath,
+ StorePathSet outputPaths
+)
+{
+ auto hook = settings.postBuildHook;
+ if (hook == "")
+ return;
+
+ Activity act(logger, lvlInfo, actPostBuildHook,
+ fmt("running post-build-hook '%s'", settings.postBuildHook),
+ Logger::Fields{store.printStorePath(drvPath)});
+ PushActivity pact(act.id);
+ std::map<std::string, std::string> hookEnvironment = getEnv();
+
+ hookEnvironment.emplace("DRV_PATH", store.printStorePath(drvPath));
+ hookEnvironment.emplace("OUT_PATHS", chomp(concatStringsSep(" ", store.printStorePathSet(outputPaths))));
+ hookEnvironment.emplace("NIX_CONFIG", globalConfig.toKeyValue());
+
+ struct LogSink : Sink {
+ Activity & act;
+ std::string currentLine;
+
+ LogSink(Activity & act) : act(act) { }
+
+ void operator() (std::string_view data) override {
+ for (auto c : data) {
+ if (c == '\n') {
+ flushLine();
+ } else {
+ currentLine += c;
+ }
+ }
+ }
+
+ void flushLine() {
+ act.result(resPostBuildLogLine, currentLine);
+ currentLine.clear();
+ }
+
+ ~LogSink() {
+ if (currentLine != "") {
+ currentLine += '\n';
+ flushLine();
+ }
+ }
+ };
+ LogSink sink(act);
+
+ runProgram2({
+ .program = settings.postBuildHook,
+ .environment = hookEnvironment,
+ .standardOut = &sink,
+ .mergeStderrToStdout = true,
+ });
+}
void DerivationGoal::buildDone()
{
@@ -798,57 +876,15 @@ void DerivationGoal::buildDone()
being valid. */
registerOutputs();
- if (settings.postBuildHook != "") {
- Activity act(*logger, lvlInfo, actPostBuildHook,
- fmt("running post-build-hook '%s'", settings.postBuildHook),
- Logger::Fields{worker.store.printStorePath(drvPath)});
- PushActivity pact(act.id);
- StorePathSet outputPaths;
- for (auto i : drv->outputs) {
- outputPaths.insert(finalOutputs.at(i.first));
- }
- std::map<std::string, std::string> hookEnvironment = getEnv();
-
- hookEnvironment.emplace("DRV_PATH", worker.store.printStorePath(drvPath));
- hookEnvironment.emplace("OUT_PATHS", chomp(concatStringsSep(" ", worker.store.printStorePathSet(outputPaths))));
-
- RunOptions opts(settings.postBuildHook, {});
- opts.environment = hookEnvironment;
-
- struct LogSink : Sink {
- Activity & act;
- std::string currentLine;
-
- LogSink(Activity & act) : act(act) { }
-
- void operator() (std::string_view data) override {
- for (auto c : data) {
- if (c == '\n') {
- flushLine();
- } else {
- currentLine += c;
- }
- }
- }
-
- void flushLine() {
- act.result(resPostBuildLogLine, currentLine);
- currentLine.clear();
- }
-
- ~LogSink() {
- if (currentLine != "") {
- currentLine += '\n';
- flushLine();
- }
- }
- };
- LogSink sink(act);
-
- opts.standardOut = &sink;
- opts.mergeStderrToStdout = true;
- runProgram2(opts);
- }
+ StorePathSet outputPaths;
+ for (auto & [_, path] : finalOutputs)
+ outputPaths.insert(path);
+ runPostBuildHook(
+ worker.store,
+ *logger,
+ drvPath,
+ outputPaths
+ );
if (buildMode == bmCheck) {
cleanupPostOutputsRegisteredModeCheck();
@@ -904,6 +940,8 @@ void DerivationGoal::resolvedFinished() {
auto resolvedHashes = staticOutputHashes(worker.store, *resolvedDrv);
+ StorePathSet outputPaths;
+
// `wantedOutputs` might be empty, which means “all the outputs”
auto realWantedOutputs = wantedOutputs;
if (realWantedOutputs.empty())
@@ -920,7 +958,11 @@ void DerivationGoal::resolvedFinished() {
if (realisation) {
auto newRealisation = *realisation;
newRealisation.id = DrvOutput{initialOutputs.at(wantedOutput).outputHash, wantedOutput};
+ newRealisation.signatures.clear();
+ newRealisation.dependentRealisations = drvOutputReferences(worker.store, *drv, realisation->outPath);
+ signRealisation(newRealisation);
worker.store.registerDrvOutput(newRealisation);
+ outputPaths.insert(realisation->outPath);
} else {
// If we don't have a realisation, then it must mean that something
// failed when building the resolved drv
@@ -928,6 +970,13 @@ void DerivationGoal::resolvedFinished() {
}
}
+ runPostBuildHook(
+ worker.store,
+ *logger,
+ drvPath,
+ outputPaths
+ );
+
// This is potentially a bit fishy in terms of error reporting. Not sure
// how to do it in a cleaner way
amDone(nrFailed == 0 ? ecSuccess : ecFailed, ex);
@@ -960,7 +1009,7 @@ HookReply DerivationGoal::tryBuildHook()
return readLine(worker.hook->fromHook.readSide.get());
} catch (Error & e) {
e.addTrace({}, "while reading the response from the build hook");
- throw e;
+ throw;
}
}();
if (handleJSONLogMessage(s, worker.act, worker.hook->activities, true))
@@ -1006,7 +1055,7 @@ HookReply DerivationGoal::tryBuildHook()
machineName = readLine(hook->fromHook.readSide.get());
} catch (Error & e) {
e.addTrace({}, "while reading the machine name from the build hook");
- throw e;
+ throw;
}
/* Tell the hook all the inputs that have to be copied to the
@@ -1040,42 +1089,6 @@ HookReply DerivationGoal::tryBuildHook()
}
-StorePathSet DerivationGoal::exportReferences(const StorePathSet & storePaths)
-{
- StorePathSet paths;
-
- for (auto & storePath : storePaths) {
- if (!inputPaths.count(storePath))
- throw BuildError("cannot export references of path '%s' because it is not in the input closure of the derivation", worker.store.printStorePath(storePath));
-
- worker.store.computeFSClosure({storePath}, paths);
- }
-
- /* If there are derivations in the graph, then include their
- outputs as well. This is useful if you want to do things
- like passing all build-time dependencies of some path to a
- derivation that builds a NixOS DVD image. */
- auto paths2 = paths;
-
- for (auto & j : paths2) {
- if (j.isDerivation()) {
- Derivation drv = worker.store.derivationFromPath(j);
- for (auto & k : drv.outputsAndOptPaths(worker.store)) {
- if (!k.second.second)
- /* FIXME: I am confused why we are calling
- `computeFSClosure` on the output path, rather than
- derivation itself. That doesn't seem right to me, so I
- won't try to implemented this for CA derivations. */
- throw UnimplementedError("exportReferences on CA derivations is not yet implemented");
- worker.store.computeFSClosure(*k.second.second, paths);
- }
- }
- }
-
- return paths;
-}
-
-
void DerivationGoal::registerOutputs()
{
/* When using a build hook, the build hook can register the output
@@ -1243,9 +1256,12 @@ OutputPathMap DerivationGoal::queryDerivationOutputMap()
void DerivationGoal::checkPathValidity()
{
bool checkHash = buildMode == bmRepair;
+ auto wantedOutputsLeft = wantedOutputs;
for (auto & i : queryPartialDerivationOutputMap()) {
InitialOutput & info = initialOutputs.at(i.first);
info.wanted = wantOutput(i.first, wantedOutputs);
+ if (info.wanted)
+ wantedOutputsLeft.erase(i.first);
if (i.second) {
auto outputPath = *i.second;
info.known = {
@@ -1258,15 +1274,33 @@ void DerivationGoal::checkPathValidity()
};
}
if (settings.isExperimentalFeatureEnabled("ca-derivations")) {
- if (auto real = worker.store.queryRealisation(
- DrvOutput{initialOutputs.at(i.first).outputHash, i.first})) {
+ auto drvOutput = DrvOutput{initialOutputs.at(i.first).outputHash, i.first};
+ if (auto real = worker.store.queryRealisation(drvOutput)) {
info.known = {
.path = real->outPath,
.status = PathStatus::Valid,
};
+ } else if (info.known && info.known->status == PathStatus::Valid) {
+ // We know the output because it' a static output of the
+ // derivation, and the output path is valid, but we don't have
+ // its realisation stored (probably because it has been built
+ // without the `ca-derivations` experimental flag)
+ worker.store.registerDrvOutput(
+ Realisation{
+ drvOutput,
+ info.known->path,
+ }
+ );
}
}
}
+ // If we requested all the outputs via the empty set, we are always fine.
+ // If we requested specific elements, the loop above removes all the valid
+ // ones, so any that are left must be invalid.
+ if (!wantedOutputsLeft.empty())
+ throw Error("derivation '%s' does not have wanted outputs %s",
+ worker.store.printStorePath(drvPath),
+ concatStringsSep(", ", quoteStrings(wantedOutputsLeft)));
}
diff --git a/src/libstore/build/derivation-goal.hh b/src/libstore/build/derivation-goal.hh
index c85bcd84f..704b77caf 100644
--- a/src/libstore/build/derivation-goal.hh
+++ b/src/libstore/build/derivation-goal.hh
@@ -180,6 +180,9 @@ struct DerivationGoal : public Goal
/* Open a log file and a pipe to it. */
Path openLogFile();
+ /* Sign the newly built realisation if the store allows it */
+ virtual void signRealisation(Realisation&) {}
+
/* Close the log file. */
void closeLogFile();
diff --git a/src/libstore/build/drv-output-substitution-goal.cc b/src/libstore/build/drv-output-substitution-goal.cc
new file mode 100644
index 000000000..be270d079
--- /dev/null
+++ b/src/libstore/build/drv-output-substitution-goal.cc
@@ -0,0 +1,122 @@
+#include "drv-output-substitution-goal.hh"
+#include "worker.hh"
+#include "substitution-goal.hh"
+
+namespace nix {
+
+DrvOutputSubstitutionGoal::DrvOutputSubstitutionGoal(const DrvOutput& id, Worker & worker, RepairFlag repair, std::optional<ContentAddress> ca)
+ : Goal(worker)
+ , id(id)
+{
+ state = &DrvOutputSubstitutionGoal::init;
+ name = fmt("substitution of '%s'", id.to_string());
+ trace("created");
+}
+
+
+void DrvOutputSubstitutionGoal::init()
+{
+ trace("init");
+
+ /* If the derivation already exists, we’re done */
+ if (worker.store.queryRealisation(id)) {
+ amDone(ecSuccess);
+ return;
+ }
+
+ subs = settings.useSubstitutes ? getDefaultSubstituters() : std::list<ref<Store>>();
+ tryNext();
+}
+
+void DrvOutputSubstitutionGoal::tryNext()
+{
+ trace("Trying next substituter");
+
+ if (subs.size() == 0) {
+ /* None left. Terminate this goal and let someone else deal
+ with it. */
+ debug("drv output '%s' is required, but there is no substituter that can provide it", id.to_string());
+
+ /* Hack: don't indicate failure if there were no substituters.
+ In that case the calling derivation should just do a
+ build. */
+ amDone(substituterFailed ? ecFailed : ecNoSubstituters);
+
+ if (substituterFailed) {
+ worker.failedSubstitutions++;
+ worker.updateProgress();
+ }
+
+ return;
+ }
+
+ auto sub = subs.front();
+ subs.pop_front();
+
+ // FIXME: Make async
+ outputInfo = sub->queryRealisation(id);
+ if (!outputInfo) {
+ tryNext();
+ return;
+ }
+
+ for (const auto & [depId, depPath] : outputInfo->dependentRealisations) {
+ if (depId != id) {
+ if (auto localOutputInfo = worker.store.queryRealisation(depId);
+ localOutputInfo && localOutputInfo->outPath != depPath) {
+ warn(
+ "substituter '%s' has an incompatible realisation for '%s', ignoring.\n"
+ "Local: %s\n"
+ "Remote: %s",
+ sub->getUri(),
+ depId.to_string(),
+ worker.store.printStorePath(localOutputInfo->outPath),
+ worker.store.printStorePath(depPath)
+ );
+ tryNext();
+ return;
+ }
+ addWaitee(worker.makeDrvOutputSubstitutionGoal(depId));
+ }
+ }
+
+ addWaitee(worker.makePathSubstitutionGoal(outputInfo->outPath));
+
+ if (waitees.empty()) outPathValid();
+ else state = &DrvOutputSubstitutionGoal::outPathValid;
+}
+
+void DrvOutputSubstitutionGoal::outPathValid()
+{
+ assert(outputInfo);
+ trace("Output path substituted");
+
+ if (nrFailed > 0) {
+ debug("The output path of the derivation output '%s' could not be substituted", id.to_string());
+ amDone(nrNoSubstituters > 0 || nrIncompleteClosure > 0 ? ecIncompleteClosure : ecFailed);
+ return;
+ }
+
+ worker.store.registerDrvOutput(*outputInfo);
+ finished();
+}
+
+void DrvOutputSubstitutionGoal::finished()
+{
+ trace("finished");
+ amDone(ecSuccess);
+}
+
+string DrvOutputSubstitutionGoal::key()
+{
+ /* "a$" ensures substitution goals happen before derivation
+ goals. */
+ return "a$" + std::string(id.to_string());
+}
+
+void DrvOutputSubstitutionGoal::work()
+{
+ (this->*state)();
+}
+
+}
diff --git a/src/libstore/build/drv-output-substitution-goal.hh b/src/libstore/build/drv-output-substitution-goal.hh
new file mode 100644
index 000000000..63ab53d89
--- /dev/null
+++ b/src/libstore/build/drv-output-substitution-goal.hh
@@ -0,0 +1,50 @@
+#pragma once
+
+#include "store-api.hh"
+#include "goal.hh"
+#include "realisation.hh"
+
+namespace nix {
+
+class Worker;
+
+// Substitution of a derivation output.
+// This is done in three steps:
+// 1. Fetch the output info from a substituter
+// 2. Substitute the corresponding output path
+// 3. Register the output info
+class DrvOutputSubstitutionGoal : public Goal {
+private:
+ // The drv output we're trying to substitue
+ DrvOutput id;
+
+ // The realisation corresponding to the given output id.
+ // Will be filled once we can get it.
+ std::optional<Realisation> outputInfo;
+
+ /* The remaining substituters. */
+ std::list<ref<Store>> subs;
+
+ /* Whether a substituter failed. */
+ bool substituterFailed = false;
+
+public:
+ DrvOutputSubstitutionGoal(const DrvOutput& id, Worker & worker, RepairFlag repair = NoRepair, std::optional<ContentAddress> ca = std::nullopt);
+
+ typedef void (DrvOutputSubstitutionGoal::*GoalState)();
+ GoalState state;
+
+ void init();
+ void tryNext();
+ void outPathValid();
+ void finished();
+
+ void timedOut(Error && ex) override { abort(); };
+
+ string key() override;
+
+ void work() override;
+
+};
+
+}
diff --git a/src/libstore/build/entry-points.cc b/src/libstore/build/entry-points.cc
index 01a564aba..2b77e4354 100644
--- a/src/libstore/build/entry-points.cc
+++ b/src/libstore/build/entry-points.cc
@@ -6,16 +6,20 @@
namespace nix {
-void Store::buildPaths(const std::vector<StorePathWithOutputs> & drvPaths, BuildMode buildMode)
+void Store::buildPaths(const std::vector<DerivedPath> & reqs, BuildMode buildMode, std::shared_ptr<Store> evalStore)
{
- Worker worker(*this);
+ Worker worker(*this, evalStore ? *evalStore : *this);
Goals goals;
- for (auto & path : drvPaths) {
- if (path.path.isDerivation())
- goals.insert(worker.makeDerivationGoal(path.path, path.outputs, buildMode));
- else
- goals.insert(worker.makeSubstitutionGoal(path.path, buildMode == bmRepair ? Repair : NoRepair));
+ for (const auto & br : reqs) {
+ std::visit(overloaded {
+ [&](const DerivedPath::Built & bfd) {
+ goals.insert(worker.makeDerivationGoal(bfd.drvPath, bfd.outputs, buildMode));
+ },
+ [&](const DerivedPath::Opaque & bo) {
+ goals.insert(worker.makePathSubstitutionGoal(bo.path, buildMode == bmRepair ? Repair : NoRepair));
+ },
+ }, br.raw());
}
worker.run(goals);
@@ -31,7 +35,7 @@ void Store::buildPaths(const std::vector<StorePathWithOutputs> & drvPaths, Build
}
if (i->exitCode != Goal::ecSuccess) {
if (auto i2 = dynamic_cast<DerivationGoal *>(i.get())) failed.insert(i2->drvPath);
- else if (auto i2 = dynamic_cast<SubstitutionGoal *>(i.get())) failed.insert(i2->storePath);
+ else if (auto i2 = dynamic_cast<PathSubstitutionGoal *>(i.get())) failed.insert(i2->storePath);
}
}
@@ -47,7 +51,7 @@ void Store::buildPaths(const std::vector<StorePathWithOutputs> & drvPaths, Build
BuildResult Store::buildDerivation(const StorePath & drvPath, const BasicDerivation & drv,
BuildMode buildMode)
{
- Worker worker(*this);
+ Worker worker(*this, *this);
auto goal = worker.makeBasicDerivationGoal(drvPath, drv, {}, buildMode);
BuildResult result;
@@ -89,8 +93,8 @@ void Store::ensurePath(const StorePath & path)
/* If the path is already valid, we're done. */
if (isValidPath(path)) return;
- Worker worker(*this);
- GoalPtr goal = worker.makeSubstitutionGoal(path);
+ Worker worker(*this, *this);
+ GoalPtr goal = worker.makePathSubstitutionGoal(path);
Goals goals = {goal};
worker.run(goals);
@@ -107,8 +111,8 @@ void Store::ensurePath(const StorePath & path)
void LocalStore::repairPath(const StorePath & path)
{
- Worker worker(*this);
- GoalPtr goal = worker.makeSubstitutionGoal(path, Repair);
+ Worker worker(*this, *this);
+ GoalPtr goal = worker.makePathSubstitutionGoal(path, Repair);
Goals goals = {goal};
worker.run(goals);
diff --git a/src/libstore/build/goal.cc b/src/libstore/build/goal.cc
index 2dd7a4d37..7c985128b 100644
--- a/src/libstore/build/goal.cc
+++ b/src/libstore/build/goal.cc
@@ -13,11 +13,9 @@ bool CompareGoalPtrs::operator() (const GoalPtr & a, const GoalPtr & b) const {
void addToWeakGoals(WeakGoals & goals, GoalPtr p)
{
- // FIXME: necessary?
- // FIXME: O(n)
- for (auto & i : goals)
- if (i.lock() == p) return;
- goals.push_back(p);
+ if (goals.find(p) != goals.end())
+ return;
+ goals.insert(p);
}
@@ -46,10 +44,7 @@ void Goal::waiteeDone(GoalPtr waitee, ExitCode result)
/* If we failed and keepGoing is not set, we remove all
remaining waitees. */
for (auto & goal : waitees) {
- WeakGoals waiters2;
- for (auto & j : goal->waiters)
- if (j.lock() != shared_from_this()) waiters2.push_back(j);
- goal->waiters = waiters2;
+ goal->waiters.extract(shared_from_this());
}
waitees.clear();
@@ -78,6 +73,8 @@ void Goal::amDone(ExitCode result, std::optional<Error> ex)
}
waiters.clear();
worker.removeGoal(shared_from_this());
+
+ cleanup();
}
diff --git a/src/libstore/build/goal.hh b/src/libstore/build/goal.hh
index fca4f2d00..192e416d2 100644
--- a/src/libstore/build/goal.hh
+++ b/src/libstore/build/goal.hh
@@ -19,7 +19,7 @@ struct CompareGoalPtrs {
/* Set of goals. */
typedef set<GoalPtr, CompareGoalPtrs> Goals;
-typedef list<WeakGoalPtr> WeakGoals;
+typedef set<WeakGoalPtr, std::owner_less<WeakGoalPtr>> WeakGoals;
/* A map of paths to goals (and the other way around). */
typedef std::map<StorePath, WeakGoalPtr> WeakGoalMap;
@@ -100,6 +100,8 @@ struct Goal : public std::enable_shared_from_this<Goal>
virtual string key() = 0;
void amDone(ExitCode result, std::optional<Error> ex = {});
+
+ virtual void cleanup() { }
};
void addToWeakGoals(WeakGoals & goals, GoalPtr p);
diff --git a/src/libstore/build/local-derivation-goal.cc b/src/libstore/build/local-derivation-goal.cc
index 90731d98d..8d245f84a 100644
--- a/src/libstore/build/local-derivation-goal.cc
+++ b/src/libstore/build/local-derivation-goal.cc
@@ -17,16 +17,14 @@
#include <regex>
#include <queue>
-#include <sys/types.h>
-#include <sys/socket.h>
#include <sys/un.h>
-#include <netdb.h>
#include <fcntl.h>
#include <termios.h>
#include <unistd.h>
#include <sys/mman.h>
#include <sys/utsname.h>
#include <sys/resource.h>
+#include <sys/socket.h>
#if HAVE_STATVFS
#include <sys/statvfs.h>
@@ -34,7 +32,6 @@
/* Includes required for chroot support. */
#if __linux__
-#include <sys/socket.h>
#include <sys/ioctl.h>
#include <net/if.h>
#include <netinet/ip.h>
@@ -70,12 +67,14 @@ void handleDiffHook(
auto diffHook = settings.diffHook;
if (diffHook != "" && settings.runDiffHook) {
try {
- RunOptions diffHookOptions(diffHook,{tryA, tryB, drvPath, tmpDir});
- diffHookOptions.searchPath = true;
- diffHookOptions.uid = uid;
- diffHookOptions.gid = gid;
- diffHookOptions.chdir = "/";
- auto diffRes = runProgram(diffHookOptions);
+ auto diffRes = runProgram(RunOptions {
+ .program = diffHook,
+ .searchPath = true,
+ .args = {tryA, tryB, drvPath, tmpDir},
+ .uid = uid,
+ .gid = gid,
+ .chdir = "/"
+ });
if (!statusOk(diffRes.first))
throw ExecError(diffRes.first,
"diff-hook program '%1%' %2%",
@@ -153,6 +152,7 @@ void LocalDerivationGoal::killChild()
void LocalDerivationGoal::tryLocalBuild() {
unsigned int curBuilds = worker.getNrLocalBuilds();
if (curBuilds >= settings.maxBuildJobs) {
+ state = &DerivationGoal::tryToBuild;
worker.waitForBuildSlot(shared_from_this());
outputLocks.unlock();
return;
@@ -291,7 +291,7 @@ bool LocalDerivationGoal::cleanupDecideWhetherDiskFull()
auto & localStore = getLocalStore();
uint64_t required = 8ULL * 1024 * 1024; // FIXME: make configurable
struct statvfs st;
- if (statvfs(localStore.realStoreDir.c_str(), &st) == 0 &&
+ if (statvfs(localStore.realStoreDir.get().c_str(), &st) == 0 &&
(uint64_t) st.f_bavail * st.f_bsize < required)
diskFull = true;
if (statvfs(tmpDir.c_str(), &st) == 0 &&
@@ -343,23 +343,6 @@ int childEntry(void * arg)
}
-static std::once_flag dns_resolve_flag;
-
-static void preloadNSS() {
- /* builtin:fetchurl can trigger a DNS lookup, which with glibc can trigger a dynamic library load of
- one of the glibc NSS libraries in a sandboxed child, which will fail unless the library's already
- been loaded in the parent. So we force a lookup of an invalid domain to force the NSS machinery to
- load its lookup libraries in the parent before any child gets a chance to. */
- std::call_once(dns_resolve_flag, []() {
- struct addrinfo *res = NULL;
-
- if (getaddrinfo("this.pre-initializes.the.dns.resolvers.invalid.", "http", NULL, &res) != 0) {
- if (res) freeaddrinfo(res);
- }
- });
-}
-
-
static void linkOrCopy(const Path & from, const Path & to)
{
if (link(from.c_str(), to.c_str()) == -1) {
@@ -388,9 +371,6 @@ void LocalDerivationGoal::startBuilder()
settings.thisSystem,
concatStringsSep<StringSet>(", ", worker.store.systemFeatures));
- if (drv->isBuiltin())
- preloadNSS();
-
#if __APPLE__
additionalSandboxProfile = parsedDrv->getStringAttr("__sandboxProfile").value_or("");
#endif
@@ -416,7 +396,7 @@ void LocalDerivationGoal::startBuilder()
}
auto & localStore = getLocalStore();
- if (localStore.storeDir != localStore.realStoreDir) {
+ if (localStore.storeDir != localStore.realStoreDir.get()) {
#if __linux__
useChroot = true;
#else
@@ -517,7 +497,7 @@ void LocalDerivationGoal::startBuilder()
/* Write closure info to <fileName>. */
writeFile(tmpDir + "/" + fileName,
worker.store.makeValidityRegistration(
- exportReferences({storePath}), false, false));
+ worker.store.exportReferences({storePath}, inputPaths), false, false));
}
}
@@ -581,7 +561,9 @@ void LocalDerivationGoal::startBuilder()
throw Error("derivation '%s' requested impure path '%s', but it was not in allowed-impure-host-deps",
worker.store.printStorePath(drvPath), i);
- dirsInChroot[i] = i;
+ /* Allow files in __impureHostDeps to be missing; e.g.
+ macOS 11+ has no /usr/lib/libSystem*.dylib */
+ dirsInChroot[i] = {i, true};
}
#if __linux__
@@ -729,6 +711,7 @@ void LocalDerivationGoal::startBuilder()
if (!builderOut.readSide)
throw SysError("opening pseudoterminal master");
+ // FIXME: not thread-safe, use ptsname_r
std::string slaveName(ptsname(builderOut.readSide.get()));
if (buildUser) {
@@ -772,7 +755,6 @@ void LocalDerivationGoal::startBuilder()
result.startTime = time(0);
/* Fork a child to build the package. */
- ProcessOptions options;
#if __linux__
if (useChroot) {
@@ -815,8 +797,6 @@ void LocalDerivationGoal::startBuilder()
userNamespaceSync.create();
- options.allowVfork = false;
-
Path maxUserNamespaces = "/proc/sys/user/max_user_namespaces";
static bool userNamespacesEnabled =
pathExists(maxUserNamespaces)
@@ -874,7 +854,7 @@ void LocalDerivationGoal::startBuilder()
writeFull(builderOut.writeSide.get(),
fmt("%d %d\n", usingUserNamespace, child));
_exit(0);
- }, options);
+ });
int res = helper.wait();
if (res != 0 && settings.sandboxFallback) {
@@ -938,10 +918,9 @@ void LocalDerivationGoal::startBuilder()
#endif
{
fallback:
- options.allowVfork = !buildUser && !drv->isBuiltin();
pid = startProcess([&]() {
runChild();
- }, options);
+ });
}
/* parent */
@@ -956,9 +935,12 @@ void LocalDerivationGoal::startBuilder()
try {
return readLine(builderOut.readSide.get());
} catch (Error & e) {
- e.addTrace({}, "while waiting for the build environment to initialize (previous messages: %s)",
+ auto status = pid.wait();
+ e.addTrace({}, "while waiting for the build environment for '%s' to initialize (%s, previous messages: %s)",
+ worker.store.printStorePath(drvPath),
+ statusToString(status),
concatStringsSep("|", msgs));
- throw e;
+ throw;
}
}();
if (string(msg, 0, 1) == "\2") break;
@@ -1081,115 +1063,50 @@ void LocalDerivationGoal::initEnv()
}
-static std::regex shVarName("[A-Za-z_][A-Za-z0-9_]*");
-
-
void LocalDerivationGoal::writeStructuredAttrs()
{
- auto structuredAttrs = parsedDrv->getStructuredAttrs();
- if (!structuredAttrs) return;
-
- auto json = *structuredAttrs;
-
- /* Add an "outputs" object containing the output paths. */
- nlohmann::json outputs;
- for (auto & i : drv->outputs) {
- /* The placeholder must have a rewrite, so we use it to cover both the
- cases where we know or don't know the output path ahead of time. */
- outputs[i.first] = rewriteStrings(hashPlaceholder(i.first), inputRewrites);
- }
- json["outputs"] = outputs;
-
- /* Handle exportReferencesGraph. */
- auto e = json.find("exportReferencesGraph");
- if (e != json.end() && e->is_object()) {
- for (auto i = e->begin(); i != e->end(); ++i) {
- std::ostringstream str;
- {
- JSONPlaceholder jsonRoot(str, true);
- StorePathSet storePaths;
- for (auto & p : *i)
- storePaths.insert(worker.store.parseStorePath(p.get<std::string>()));
- worker.store.pathInfoToJSON(jsonRoot,
- exportReferences(storePaths), false, true);
- }
- json[i.key()] = nlohmann::json::parse(str.str()); // urgh
+ if (auto structAttrsJson = parsedDrv->prepareStructuredAttrs(worker.store, inputPaths)) {
+ auto json = structAttrsJson.value();
+ nlohmann::json rewritten;
+ for (auto & [i, v] : json["outputs"].get<nlohmann::json::object_t>()) {
+ /* The placeholder must have a rewrite, so we use it to cover both the
+ cases where we know or don't know the output path ahead of time. */
+ rewritten[i] = rewriteStrings(v, inputRewrites);
}
- }
-
- writeFile(tmpDir + "/.attrs.json", rewriteStrings(json.dump(), inputRewrites));
- chownToBuilder(tmpDir + "/.attrs.json");
-
- /* As a convenience to bash scripts, write a shell file that
- maps all attributes that are representable in bash -
- namely, strings, integers, nulls, Booleans, and arrays and
- objects consisting entirely of those values. (So nested
- arrays or objects are not supported.) */
-
- auto handleSimpleType = [](const nlohmann::json & value) -> std::optional<std::string> {
- if (value.is_string())
- return shellEscape(value);
-
- if (value.is_number()) {
- auto f = value.get<float>();
- if (std::ceil(f) == f)
- return std::to_string(value.get<int>());
- }
-
- if (value.is_null())
- return std::string("''");
-
- if (value.is_boolean())
- return value.get<bool>() ? std::string("1") : std::string("");
-
- return {};
- };
- std::string jsonSh;
+ json["outputs"] = rewritten;
- for (auto i = json.begin(); i != json.end(); ++i) {
+ auto jsonSh = writeStructuredAttrsShell(json);
- if (!std::regex_match(i.key(), shVarName)) continue;
-
- auto & value = i.value();
-
- auto s = handleSimpleType(value);
- if (s)
- jsonSh += fmt("declare %s=%s\n", i.key(), *s);
-
- else if (value.is_array()) {
- std::string s2;
- bool good = true;
-
- for (auto i = value.begin(); i != value.end(); ++i) {
- auto s3 = handleSimpleType(i.value());
- if (!s3) { good = false; break; }
- s2 += *s3; s2 += ' ';
- }
-
- if (good)
- jsonSh += fmt("declare -a %s=(%s)\n", i.key(), s2);
- }
+ writeFile(tmpDir + "/.attrs.sh", rewriteStrings(jsonSh, inputRewrites));
+ chownToBuilder(tmpDir + "/.attrs.sh");
+ env["NIX_ATTRS_SH_FILE"] = tmpDir + "/.attrs.sh";
+ writeFile(tmpDir + "/.attrs.json", rewriteStrings(json.dump(), inputRewrites));
+ chownToBuilder(tmpDir + "/.attrs.json");
+ env["NIX_ATTRS_JSON_FILE"] = tmpDir + "/.attrs.json";
+ }
+}
- else if (value.is_object()) {
- std::string s2;
- bool good = true;
- for (auto i = value.begin(); i != value.end(); ++i) {
- auto s3 = handleSimpleType(i.value());
- if (!s3) { good = false; break; }
- s2 += fmt("[%s]=%s ", shellEscape(i.key()), *s3);
- }
+static StorePath pathPartOfReq(const DerivedPath & req)
+{
+ return std::visit(overloaded {
+ [&](const DerivedPath::Opaque & bo) {
+ return bo.path;
+ },
+ [&](const DerivedPath::Built & bfd) {
+ return bfd.drvPath;
+ },
+ }, req.raw());
+}
- if (good)
- jsonSh += fmt("declare -A %s=(%s)\n", i.key(), s2);
- }
- }
- writeFile(tmpDir + "/.attrs.sh", rewriteStrings(jsonSh, inputRewrites));
- chownToBuilder(tmpDir + "/.attrs.sh");
+bool LocalDerivationGoal::isAllowed(const DerivedPath & req)
+{
+ return this->isAllowed(pathPartOfReq(req));
}
+
struct RestrictedStoreConfig : virtual LocalFSStoreConfig
{
using LocalFSStoreConfig::LocalFSStoreConfig;
@@ -1310,33 +1227,54 @@ struct RestrictedStore : public virtual RestrictedStoreConfig, public virtual Lo
std::optional<const Realisation> queryRealisation(const DrvOutput & id) override
// XXX: This should probably be allowed if the realisation corresponds to
// an allowed derivation
- { throw Error("queryRealisation"); }
+ {
+ if (!goal.isAllowed(id))
+ throw InvalidPath("cannot query an unknown output id '%s' in recursive Nix", id.to_string());
+ return next->queryRealisation(id);
+ }
- void buildPaths(const std::vector<StorePathWithOutputs> & paths, BuildMode buildMode) override
+ void buildPaths(const std::vector<DerivedPath> & paths, BuildMode buildMode, std::shared_ptr<Store> evalStore) override
{
+ assert(!evalStore);
+
if (buildMode != bmNormal) throw Error("unsupported build mode");
StorePathSet newPaths;
+ std::set<Realisation> newRealisations;
- for (auto & path : paths) {
- if (!goal.isAllowed(path.path))
- throw InvalidPath("cannot build unknown path '%s' in recursive Nix", printStorePath(path.path));
+ for (auto & req : paths) {
+ if (!goal.isAllowed(req))
+ throw InvalidPath("cannot build '%s' in recursive Nix because path is unknown", req.to_string(*next));
}
next->buildPaths(paths, buildMode);
for (auto & path : paths) {
- if (!path.path.isDerivation()) continue;
- auto outputs = next->queryDerivationOutputMap(path.path);
- for (auto & output : outputs)
- if (wantOutput(output.first, path.outputs))
- newPaths.insert(output.second);
+ auto p = std::get_if<DerivedPath::Built>(&path);
+ if (!p) continue;
+ auto & bfd = *p;
+ auto drv = readDerivation(bfd.drvPath);
+ auto drvHashes = staticOutputHashes(*this, drv);
+ auto outputs = next->queryDerivationOutputMap(bfd.drvPath);
+ for (auto & [outputName, outputPath] : outputs)
+ if (wantOutput(outputName, bfd.outputs)) {
+ newPaths.insert(outputPath);
+ if (settings.isExperimentalFeatureEnabled("ca-derivations")) {
+ auto thisRealisation = next->queryRealisation(
+ DrvOutput{drvHashes.at(outputName), outputName}
+ );
+ assert(thisRealisation);
+ newRealisations.insert(*thisRealisation);
+ }
+ }
}
StorePathSet closure;
next->computeFSClosure(newPaths, closure);
for (auto & path : closure)
goal.addDependency(path);
+ for (auto & real : Realisation::closure(*next, newRealisations))
+ goal.addedDrvOutputs.insert(real.id);
}
BuildResult buildDerivation(const StorePath & drvPath, const BasicDerivation & drv,
@@ -1358,7 +1296,7 @@ struct RestrictedStore : public virtual RestrictedStoreConfig, public virtual Lo
void addSignatures(const StorePath & storePath, const StringSet & sigs) override
{ unsupported("addSignatures"); }
- void queryMissing(const std::vector<StorePathWithOutputs> & targets,
+ void queryMissing(const std::vector<DerivedPath> & targets,
StorePathSet & willBuild, StorePathSet & willSubstitute, StorePathSet & unknown,
uint64_t & downloadSize, uint64_t & narSize) override
{
@@ -1366,12 +1304,12 @@ struct RestrictedStore : public virtual RestrictedStoreConfig, public virtual Lo
client about what paths will be built/substituted or are
already present. Probably not a big deal. */
- std::vector<StorePathWithOutputs> allowed;
- for (auto & path : targets) {
- if (goal.isAllowed(path.path))
- allowed.emplace_back(path);
+ std::vector<DerivedPath> allowed;
+ for (auto & req : targets) {
+ if (goal.isAllowed(req))
+ allowed.emplace_back(req);
else
- unknown.insert(path.path);
+ unknown.insert(pathPartOfReq(req));
}
next->queryMissing(allowed, willBuild, willSubstitute,
@@ -1712,7 +1650,7 @@ void LocalDerivationGoal::runChild()
/* N.B. it is realistic that these paths might not exist. It
happens when testing Nix building fixed-output derivations
within a pure derivation. */
- for (auto & path : { "/etc/resolv.conf", "/etc/services", "/etc/hosts", "/var/run/nscd/socket" })
+ for (auto & path : { "/etc/resolv.conf", "/etc/services", "/etc/hosts" })
if (pathExists(path))
ss.push_back(path);
}
@@ -1894,7 +1832,7 @@ void LocalDerivationGoal::runChild()
/* Fill in the arguments. */
Strings args;
- const char *builder = "invalid";
+ std::string builder = "invalid";
if (drv->isBuiltin()) {
;
@@ -2020,13 +1958,13 @@ void LocalDerivationGoal::runChild()
}
args.push_back(drv->builder);
} else {
- builder = drv->builder.c_str();
+ builder = drv->builder;
args.push_back(std::string(baseNameOf(drv->builder)));
}
}
#else
else {
- builder = drv->builder.c_str();
+ builder = drv->builder;
args.push_back(std::string(baseNameOf(drv->builder)));
}
#endif
@@ -2082,9 +2020,9 @@ void LocalDerivationGoal::runChild()
posix_spawnattr_setbinpref_np(&attrp, 1, &cpu, NULL);
}
- posix_spawn(NULL, builder, NULL, &attrp, stringsToCharPtrs(args).data(), stringsToCharPtrs(envStrs).data());
+ posix_spawn(NULL, builder.c_str(), NULL, &attrp, stringsToCharPtrs(args).data(), stringsToCharPtrs(envStrs).data());
#else
- execve(builder, stringsToCharPtrs(args).data(), stringsToCharPtrs(envStrs).data());
+ execve(builder.c_str(), stringsToCharPtrs(args).data(), stringsToCharPtrs(envStrs).data());
#endif
throw SysError("executing '%1%'", drv->builder);
@@ -2199,8 +2137,7 @@ void LocalDerivationGoal::registerOutputs()
/* Pass blank Sink as we are not ready to hash data at this stage. */
NullSink blank;
- auto references = worker.store.parseStorePathSet(
- scanForReferences(blank, actualPath, worker.store.printStorePathSet(referenceablePaths)));
+ auto references = scanForReferences(blank, actualPath, referenceablePaths);
outputReferencesIfUnregistered.insert_or_assign(
outputName,
@@ -2214,8 +2151,8 @@ void LocalDerivationGoal::registerOutputs()
/* Since we'll use the already installed versions of these, we
can treat them as leaves and ignore any references they
have. */
- [&](AlreadyRegistered _) { return StringSet {}; },
- [&](PerhapsNeedToRegister refs) {
+ [&](const AlreadyRegistered &) { return StringSet {}; },
+ [&](const PerhapsNeedToRegister & refs) {
StringSet referencedOutputs;
/* FIXME build inverted map up front so no quadratic waste here */
for (auto & r : refs.refs)
@@ -2251,11 +2188,11 @@ void LocalDerivationGoal::registerOutputs()
};
std::optional<StorePathSet> referencesOpt = std::visit(overloaded {
- [&](AlreadyRegistered skippedFinalPath) -> std::optional<StorePathSet> {
+ [&](const AlreadyRegistered & skippedFinalPath) -> std::optional<StorePathSet> {
finish(skippedFinalPath.path);
return std::nullopt;
},
- [&](PerhapsNeedToRegister r) -> std::optional<StorePathSet> {
+ [&](const PerhapsNeedToRegister & r) -> std::optional<StorePathSet> {
return r.refs;
},
}, outputReferencesIfUnregistered.at(outputName));
@@ -2267,7 +2204,7 @@ void LocalDerivationGoal::registerOutputs()
auto rewriteOutput = [&]() {
/* Apply hash rewriting if necessary. */
if (!outputRewrites.empty()) {
- warn("rewriting hashes in '%1%'; cross fingers", actualPath);
+ debug("rewriting hashes in '%1%'; cross fingers", actualPath);
/* FIXME: this is in-memory. */
StringSink sink;
@@ -2276,10 +2213,6 @@ void LocalDerivationGoal::registerOutputs()
sink.s = make_ref<std::string>(rewriteStrings(*sink.s, outputRewrites));
StringSource source(*sink.s);
restorePath(actualPath, source);
-
- /* FIXME: set proper permissions in restorePath() so
- we don't have to do another traversal. */
- canonicalisePathMetaData(actualPath, -1, inodesSeen);
}
};
@@ -2333,32 +2266,19 @@ void LocalDerivationGoal::registerOutputs()
}
auto got = caSink.finish().first;
auto refs = rewriteRefs();
- HashModuloSink narSink { htSHA256, oldHashPart };
- dumpPath(actualPath, narSink);
- auto narHashAndSize = narSink.finish();
- ValidPathInfo newInfo0 {
- worker.store.makeFixedOutputPath(
+
+ auto finalPath = worker.store.makeFixedOutputPath(
outputHash.method,
got,
outputPathName(drv->name, outputName),
refs.second,
- refs.first),
- narHashAndSize.first,
- };
- newInfo0.narSize = narHashAndSize.second;
- newInfo0.ca = FixedOutputHash {
- .method = outputHash.method,
- .hash = got,
- };
- newInfo0.references = refs.second;
- if (refs.first)
- newInfo0.references.insert(newInfo0.path);
- if (scratchPath != newInfo0.path) {
+ refs.first);
+ if (scratchPath != finalPath) {
// Also rewrite the output path
auto source = sinkToSource([&](Sink & nextSink) {
StringSink sink;
dumpPath(actualPath, sink);
- RewritingSink rsink2(oldHashPart, std::string(newInfo0.path.hashPart()), nextSink);
+ RewritingSink rsink2(oldHashPart, std::string(finalPath.hashPart()), nextSink);
rsink2(*sink.s);
rsink2.flush();
});
@@ -2368,12 +2288,27 @@ void LocalDerivationGoal::registerOutputs()
movePath(tmpPath, actualPath);
}
+ HashResult narHashAndSize = hashPath(htSHA256, actualPath);
+ ValidPathInfo newInfo0 {
+ finalPath,
+ narHashAndSize.first,
+ };
+
+ newInfo0.narSize = narHashAndSize.second;
+ newInfo0.ca = FixedOutputHash {
+ .method = outputHash.method,
+ .hash = got,
+ };
+ newInfo0.references = refs.second;
+ if (refs.first)
+ newInfo0.references.insert(newInfo0.path);
+
assert(newInfo0.ca);
return newInfo0;
};
ValidPathInfo newInfo = std::visit(overloaded {
- [&](DerivationOutputInputAddressed output) {
+ [&](const DerivationOutputInputAddressed & output) {
/* input-addressed case */
auto requiredFinalPath = output.path;
/* Preemptively add rewrite rule for final hash, as that is
@@ -2392,14 +2327,14 @@ void LocalDerivationGoal::registerOutputs()
newInfo0.references.insert(newInfo0.path);
return newInfo0;
},
- [&](DerivationOutputCAFixed dof) {
+ [&](const DerivationOutputCAFixed & dof) {
auto newInfo0 = newInfoFromCA(DerivationOutputCAFloating {
.method = dof.hash.method,
.hashType = dof.hash.hash.type,
});
/* Check wanted hash */
- Hash & wanted = dof.hash.hash;
+ const Hash & wanted = dof.hash.hash;
assert(newInfo0.ca);
auto got = getContentAddressHash(*newInfo0.ca);
if (wanted != got) {
@@ -2428,6 +2363,10 @@ void LocalDerivationGoal::registerOutputs()
},
}, output.output);
+ /* FIXME: set proper permissions in restorePath() so
+ we don't have to do another traversal. */
+ canonicalisePathMetaData(actualPath, -1, inodesSeen);
+
/* Calculate where we'll move the output files. In the checking case we
will leave leave them where they are, for now, rather than move to
their usual "final destination" */
@@ -2437,6 +2376,7 @@ void LocalDerivationGoal::registerOutputs()
floating CA derivations and hash-mismatching fixed-output
derivations. */
PathLocks dynamicOutputLock;
+ dynamicOutputLock.setDeletion(true);
auto optFixedPath = output.path(worker.store, drv->name, outputName);
if (!optFixedPath ||
worker.store.printStorePath(*optFixedPath) != finalDestPath)
@@ -2460,6 +2400,7 @@ void LocalDerivationGoal::registerOutputs()
assert(newInfo.ca);
} else {
auto destPath = worker.store.toRealPath(finalDestPath);
+ deletePath(destPath);
movePath(actualPath, destPath);
actualPath = destPath;
}
@@ -2529,7 +2470,13 @@ void LocalDerivationGoal::registerOutputs()
infos.emplace(outputName, std::move(newInfo));
}
- if (buildMode == bmCheck) return;
+ if (buildMode == bmCheck) {
+ // In case of FOD mismatches on `--check` an error must be thrown as this is also
+ // a source for non-determinism.
+ if (delayedException)
+ std::rethrow_exception(delayedException);
+ return;
+ }
/* Apply output checks. */
checkOutputs(infos);
@@ -2615,13 +2562,22 @@ void LocalDerivationGoal::registerOutputs()
but it's fine to do in all cases. */
if (settings.isExperimentalFeatureEnabled("ca-derivations")) {
- for (auto& [outputName, newInfo] : infos)
- worker.store.registerDrvOutput(Realisation{
- .id = DrvOutput{initialOutputs.at(outputName).outputHash, outputName},
- .outPath = newInfo.path});
+ for (auto& [outputName, newInfo] : infos) {
+ auto thisRealisation = Realisation{
+ .id = DrvOutput{initialOutputs.at(outputName).outputHash,
+ outputName},
+ .outPath = newInfo.path};
+ signRealisation(thisRealisation);
+ worker.store.registerDrvOutput(thisRealisation);
+ }
}
}
+void LocalDerivationGoal::signRealisation(Realisation & realisation)
+{
+ getLocalStore().signRealisation(realisation);
+}
+
void LocalDerivationGoal::checkOutputs(const std::map<Path, ValidPathInfo> & outputs)
{
diff --git a/src/libstore/build/local-derivation-goal.hh b/src/libstore/build/local-derivation-goal.hh
index 4bbf27a1b..088a57209 100644
--- a/src/libstore/build/local-derivation-goal.hh
+++ b/src/libstore/build/local-derivation-goal.hh
@@ -108,6 +108,9 @@ struct LocalDerivationGoal : public DerivationGoal
/* Paths that were added via recursive Nix calls. */
StorePathSet addedPaths;
+ /* Realisations that were added via recursive Nix calls. */
+ std::set<DrvOutput> addedDrvOutputs;
+
/* 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
@@ -116,6 +119,12 @@ struct LocalDerivationGoal : public DerivationGoal
{
return inputPaths.count(path) || addedPaths.count(path);
}
+ bool isAllowed(const DrvOutput & id)
+ {
+ return addedDrvOutputs.count(id);
+ }
+
+ bool isAllowed(const DerivedPath & req);
friend struct RestrictedStore;
@@ -161,6 +170,8 @@ struct LocalDerivationGoal : public DerivationGoal
as valid. */
void registerOutputs() override;
+ void signRealisation(Realisation &) override;
+
/* Check that an output meets the requirements specified by the
'outputChecks' attribute (or the legacy
'{allowed,disallowed}{References,Requisites}' attributes). */
diff --git a/src/libstore/build/substitution-goal.cc b/src/libstore/build/substitution-goal.cc
index c4b0de78d..29a8cfb87 100644
--- a/src/libstore/build/substitution-goal.cc
+++ b/src/libstore/build/substitution-goal.cc
@@ -5,40 +5,32 @@
namespace nix {
-SubstitutionGoal::SubstitutionGoal(const StorePath & storePath, Worker & worker, RepairFlag repair, std::optional<ContentAddress> ca)
+PathSubstitutionGoal::PathSubstitutionGoal(const StorePath & storePath, Worker & worker, RepairFlag repair, std::optional<ContentAddress> ca)
: Goal(worker)
, storePath(storePath)
, repair(repair)
, ca(ca)
{
- state = &SubstitutionGoal::init;
+ state = &PathSubstitutionGoal::init;
name = fmt("substitution of '%s'", worker.store.printStorePath(this->storePath));
trace("created");
maintainExpectedSubstitutions = std::make_unique<MaintainCount<uint64_t>>(worker.expectedSubstitutions);
}
-SubstitutionGoal::~SubstitutionGoal()
+PathSubstitutionGoal::~PathSubstitutionGoal()
{
- try {
- if (thr.joinable()) {
- // FIXME: signal worker thread to quit.
- thr.join();
- worker.childTerminated(this);
- }
- } catch (...) {
- ignoreException();
- }
+ cleanup();
}
-void SubstitutionGoal::work()
+void PathSubstitutionGoal::work()
{
(this->*state)();
}
-void SubstitutionGoal::init()
+void PathSubstitutionGoal::init()
{
trace("init");
@@ -59,10 +51,12 @@ void SubstitutionGoal::init()
}
-void SubstitutionGoal::tryNext()
+void PathSubstitutionGoal::tryNext()
{
trace("trying next substituter");
+ cleanup();
+
if (subs.size() == 0) {
/* None left. Terminate this goal and let someone else deal
with it. */
@@ -142,7 +136,7 @@ void SubstitutionGoal::tryNext()
/* Bail out early if this substituter lacks a valid
signature. LocalStore::addToStore() also checks for this, but
only after we've downloaded the path. */
- if (!sub->isTrusted && worker.store.pathInfoIsTrusted(*info))
+ if (!sub->isTrusted && worker.store.pathInfoIsUntrusted(*info))
{
warn("substituter '%s' does not have a valid signature for path '%s'",
sub->getUri(), worker.store.printStorePath(storePath));
@@ -154,16 +148,16 @@ void SubstitutionGoal::tryNext()
paths referenced by this one. */
for (auto & i : info->references)
if (i != storePath) /* ignore self-references */
- addWaitee(worker.makeSubstitutionGoal(i));
+ addWaitee(worker.makePathSubstitutionGoal(i));
if (waitees.empty()) /* to prevent hang (no wake-up event) */
referencesValid();
else
- state = &SubstitutionGoal::referencesValid;
+ state = &PathSubstitutionGoal::referencesValid;
}
-void SubstitutionGoal::referencesValid()
+void PathSubstitutionGoal::referencesValid()
{
trace("all references realised");
@@ -177,12 +171,12 @@ void SubstitutionGoal::referencesValid()
if (i != storePath) /* ignore self-references */
assert(worker.store.isValidPath(i));
- state = &SubstitutionGoal::tryToRun;
+ state = &PathSubstitutionGoal::tryToRun;
worker.wakeUp(shared_from_this());
}
-void SubstitutionGoal::tryToRun()
+void PathSubstitutionGoal::tryToRun()
{
trace("trying to run");
@@ -205,12 +199,12 @@ void SubstitutionGoal::tryToRun()
thr = std::thread([this]() {
try {
/* Wake up the worker loop when we're done. */
- Finally updateStats([this]() { outPipe.writeSide = -1; });
+ Finally updateStats([this]() { outPipe.writeSide.close(); });
Activity act(*logger, actSubstitute, Logger::Fields{worker.store.printStorePath(storePath), sub->getUri()});
PushActivity pact(act.id);
- copyStorePath(ref<Store>(sub), ref<Store>(worker.store.shared_from_this()),
+ copyStorePath(*sub, worker.store,
subPath ? *subPath : storePath, repair, sub->isTrusted ? NoCheckSigs : CheckSigs);
promise.set_value();
@@ -221,11 +215,11 @@ void SubstitutionGoal::tryToRun()
worker.childStarted(shared_from_this(), {outPipe.readSide.get()}, true, false);
- state = &SubstitutionGoal::finished;
+ state = &PathSubstitutionGoal::finished;
}
-void SubstitutionGoal::finished()
+void PathSubstitutionGoal::finished()
{
trace("substitute finished");
@@ -249,7 +243,7 @@ void SubstitutionGoal::finished()
}
/* Try the next substitute. */
- state = &SubstitutionGoal::tryNext;
+ state = &PathSubstitutionGoal::tryNext;
worker.wakeUp(shared_from_this());
return;
}
@@ -278,14 +272,31 @@ void SubstitutionGoal::finished()
}
-void SubstitutionGoal::handleChildOutput(int fd, const string & data)
+void PathSubstitutionGoal::handleChildOutput(int fd, const string & data)
{
}
-void SubstitutionGoal::handleEOF(int fd)
+void PathSubstitutionGoal::handleEOF(int fd)
{
if (fd == outPipe.readSide.get()) worker.wakeUp(shared_from_this());
}
+
+void PathSubstitutionGoal::cleanup()
+{
+ try {
+ if (thr.joinable()) {
+ // FIXME: signal worker thread to quit.
+ thr.join();
+ worker.childTerminated(this);
+ }
+
+ outPipe.close();
+ } catch (...) {
+ ignoreException();
+ }
+}
+
+
}
diff --git a/src/libstore/build/substitution-goal.hh b/src/libstore/build/substitution-goal.hh
index dee2cecbf..70c806d23 100644
--- a/src/libstore/build/substitution-goal.hh
+++ b/src/libstore/build/substitution-goal.hh
@@ -8,13 +8,13 @@ namespace nix {
class Worker;
-struct SubstitutionGoal : public Goal
+struct PathSubstitutionGoal : public Goal
{
/* The store path that should be realised through a substitute. */
StorePath storePath;
/* The path the substituter refers to the path as. This will be
- * different when the stores have different names. */
+ different when the stores have different names. */
std::optional<StorePath> subPath;
/* The remaining substituters. */
@@ -47,14 +47,15 @@ struct SubstitutionGoal : public Goal
std::unique_ptr<MaintainCount<uint64_t>> maintainExpectedSubstitutions,
maintainRunningSubstitutions, maintainExpectedNar, maintainExpectedDownload;
- typedef void (SubstitutionGoal::*GoalState)();
+ typedef void (PathSubstitutionGoal::*GoalState)();
GoalState state;
/* Content address for recomputing store path */
std::optional<ContentAddress> ca;
- SubstitutionGoal(const StorePath & storePath, Worker & worker, RepairFlag repair = NoRepair, std::optional<ContentAddress> ca = std::nullopt);
- ~SubstitutionGoal();
+public:
+ PathSubstitutionGoal(const StorePath & storePath, Worker & worker, RepairFlag repair = NoRepair, std::optional<ContentAddress> ca = std::nullopt);
+ ~PathSubstitutionGoal();
void timedOut(Error && ex) override { abort(); };
@@ -78,6 +79,8 @@ struct SubstitutionGoal : public Goal
/* Callback used by the worker to write to the log. */
void handleChildOutput(int fd, const string & data) override;
void handleEOF(int fd) override;
+
+ void cleanup() override;
};
}
diff --git a/src/libstore/build/worker.cc b/src/libstore/build/worker.cc
index b2223c3b6..55afb5cca 100644
--- a/src/libstore/build/worker.cc
+++ b/src/libstore/build/worker.cc
@@ -1,6 +1,7 @@
#include "machines.hh"
#include "worker.hh"
#include "substitution-goal.hh"
+#include "drv-output-substitution-goal.hh"
#include "local-derivation-goal.hh"
#include "hook-instance.hh"
@@ -8,11 +9,12 @@
namespace nix {
-Worker::Worker(Store & store)
+Worker::Worker(Store & store, Store & evalStore)
: act(*logger, actRealise)
, actDerivations(*logger, actBuilds)
, actSubstitutions(*logger, actCopyPaths)
, store(store)
+ , evalStore(evalStore)
{
/* Debugging: prevent recursive workers. */
nrLocalBuilds = 0;
@@ -78,20 +80,32 @@ std::shared_ptr<DerivationGoal> Worker::makeBasicDerivationGoal(const StorePath
}
-std::shared_ptr<SubstitutionGoal> Worker::makeSubstitutionGoal(const StorePath & path, RepairFlag repair, std::optional<ContentAddress> ca)
+std::shared_ptr<PathSubstitutionGoal> Worker::makePathSubstitutionGoal(const StorePath & path, RepairFlag repair, std::optional<ContentAddress> ca)
{
- std::weak_ptr<SubstitutionGoal> & goal_weak = substitutionGoals[path];
+ std::weak_ptr<PathSubstitutionGoal> & goal_weak = substitutionGoals[path];
auto goal = goal_weak.lock(); // FIXME
if (!goal) {
- goal = std::make_shared<SubstitutionGoal>(path, *this, repair, ca);
+ goal = std::make_shared<PathSubstitutionGoal>(path, *this, repair, ca);
goal_weak = goal;
wakeUp(goal);
}
return goal;
}
-template<typename G>
-static void removeGoal(std::shared_ptr<G> goal, std::map<StorePath, std::weak_ptr<G>> & goalMap)
+std::shared_ptr<DrvOutputSubstitutionGoal> Worker::makeDrvOutputSubstitutionGoal(const DrvOutput& id, RepairFlag repair, std::optional<ContentAddress> ca)
+{
+ std::weak_ptr<DrvOutputSubstitutionGoal> & goal_weak = drvOutputSubstitutionGoals[id];
+ auto goal = goal_weak.lock(); // FIXME
+ if (!goal) {
+ goal = std::make_shared<DrvOutputSubstitutionGoal>(id, *this, repair, ca);
+ goal_weak = goal;
+ wakeUp(goal);
+ }
+ return goal;
+}
+
+template<typename K, typename G>
+static void removeGoal(std::shared_ptr<G> goal, std::map<K, std::weak_ptr<G>> & goalMap)
{
/* !!! inefficient */
for (auto i = goalMap.begin();
@@ -109,10 +123,13 @@ void Worker::removeGoal(GoalPtr goal)
{
if (auto drvGoal = std::dynamic_pointer_cast<DerivationGoal>(goal))
nix::removeGoal(drvGoal, derivationGoals);
- else if (auto subGoal = std::dynamic_pointer_cast<SubstitutionGoal>(goal))
+ else if (auto subGoal = std::dynamic_pointer_cast<PathSubstitutionGoal>(goal))
nix::removeGoal(subGoal, substitutionGoals);
+ else if (auto subGoal = std::dynamic_pointer_cast<DrvOutputSubstitutionGoal>(goal))
+ nix::removeGoal(subGoal, drvOutputSubstitutionGoals);
else
assert(false);
+
if (topGoals.find(goal) != topGoals.end()) {
topGoals.erase(goal);
/* If a top-level goal failed, then kill all other goals
@@ -211,18 +228,18 @@ void Worker::waitForAWhile(GoalPtr goal)
void Worker::run(const Goals & _topGoals)
{
- std::vector<nix::StorePathWithOutputs> topPaths;
+ std::vector<nix::DerivedPath> topPaths;
for (auto & i : _topGoals) {
topGoals.insert(i);
if (auto goal = dynamic_cast<DerivationGoal *>(i.get())) {
- topPaths.push_back({goal->drvPath, goal->wantedOutputs});
- } else if (auto goal = dynamic_cast<SubstitutionGoal *>(i.get())) {
- topPaths.push_back({goal->storePath});
+ topPaths.push_back(DerivedPath::Built{goal->drvPath, goal->wantedOutputs});
+ } else if (auto goal = dynamic_cast<PathSubstitutionGoal *>(i.get())) {
+ topPaths.push_back(DerivedPath::Opaque{goal->storePath});
}
}
- /* Call queryMissing() efficiently query substitutes. */
+ /* Call queryMissing() to efficiently query substitutes. */
StorePathSet willBuild, willSubstitute, unknown;
uint64_t downloadSize, narSize;
store.queryMissing(topPaths, willBuild, willSubstitute, unknown, downloadSize, narSize);
@@ -471,7 +488,10 @@ void Worker::markContentsGood(const StorePath & path)
}
-GoalPtr upcast_goal(std::shared_ptr<SubstitutionGoal> subGoal) {
+GoalPtr upcast_goal(std::shared_ptr<PathSubstitutionGoal> subGoal) {
+ return subGoal;
+}
+GoalPtr upcast_goal(std::shared_ptr<DrvOutputSubstitutionGoal> subGoal) {
return subGoal;
}
diff --git a/src/libstore/build/worker.hh b/src/libstore/build/worker.hh
index 82e711191..6a3b99c02 100644
--- a/src/libstore/build/worker.hh
+++ b/src/libstore/build/worker.hh
@@ -4,6 +4,7 @@
#include "lock.hh"
#include "store-api.hh"
#include "goal.hh"
+#include "realisation.hh"
#include <future>
#include <thread>
@@ -12,18 +13,20 @@ namespace nix {
/* Forward definition. */
struct DerivationGoal;
-struct SubstitutionGoal;
+struct PathSubstitutionGoal;
+class DrvOutputSubstitutionGoal;
/* Workaround for not being able to declare a something like
- class SubstitutionGoal : public Goal;
+ class PathSubstitutionGoal : public Goal;
even when Goal is a complete type.
This is still a static cast. The purpose of exporting it is to define it in
- a place where `SubstitutionGoal` is concrete, and use it in a place where it
+ a place where `PathSubstitutionGoal` is concrete, and use it in a place where it
is opaque. */
-GoalPtr upcast_goal(std::shared_ptr<SubstitutionGoal> subGoal);
+GoalPtr upcast_goal(std::shared_ptr<PathSubstitutionGoal> subGoal);
+GoalPtr upcast_goal(std::shared_ptr<DrvOutputSubstitutionGoal> subGoal);
typedef std::chrono::time_point<std::chrono::steady_clock> steady_time_point;
@@ -72,7 +75,8 @@ private:
/* Maps used to prevent multiple instantiations of a goal for the
same derivation / path. */
std::map<StorePath, std::weak_ptr<DerivationGoal>> derivationGoals;
- std::map<StorePath, std::weak_ptr<SubstitutionGoal>> substitutionGoals;
+ std::map<StorePath, std::weak_ptr<PathSubstitutionGoal>> substitutionGoals;
+ std::map<DrvOutput, std::weak_ptr<DrvOutputSubstitutionGoal>> drvOutputSubstitutionGoals;
/* Goals waiting for busy paths to be unlocked. */
WeakGoals waitingForAnyGoal;
@@ -106,6 +110,7 @@ public:
bool checkMismatch;
Store & store;
+ Store & evalStore;
std::unique_ptr<HookInstance> hook;
@@ -127,7 +132,7 @@ public:
it answers with "decline-permanently", we don't try again. */
bool tryBuildHook = true;
- Worker(Store & store);
+ Worker(Store & store, Store & evalStore);
~Worker();
/* Make a goal (with caching). */
@@ -146,7 +151,8 @@ public:
const StringSet & wantedOutputs, BuildMode buildMode = bmNormal);
/* substitution goal */
- std::shared_ptr<SubstitutionGoal> makeSubstitutionGoal(const StorePath & storePath, RepairFlag repair = NoRepair, std::optional<ContentAddress> ca = std::nullopt);
+ std::shared_ptr<PathSubstitutionGoal> makePathSubstitutionGoal(const StorePath & storePath, RepairFlag repair = NoRepair, std::optional<ContentAddress> ca = std::nullopt);
+ std::shared_ptr<DrvOutputSubstitutionGoal> makeDrvOutputSubstitutionGoal(const DrvOutput & id, RepairFlag repair = NoRepair, std::optional<ContentAddress> ca = std::nullopt);
/* Remove a dead goal. */
void removeGoal(GoalPtr goal);