aboutsummaryrefslogtreecommitdiff
path: root/src/libstore/build.cc
diff options
context:
space:
mode:
Diffstat (limited to 'src/libstore/build.cc')
-rw-r--r--src/libstore/build.cc459
1 files changed, 249 insertions, 210 deletions
diff --git a/src/libstore/build.cc b/src/libstore/build.cc
index ae7ba6549..e911b8f17 100644
--- a/src/libstore/build.cc
+++ b/src/libstore/build.cc
@@ -86,7 +86,7 @@ struct HookInstance;
/* A pointer to a goal. */
-class Goal;
+struct Goal;
class DerivationGoal;
typedef std::shared_ptr<Goal> GoalPtr;
typedef std::weak_ptr<Goal> WeakGoalPtr;
@@ -104,9 +104,8 @@ typedef std::map<StorePath, WeakGoalPtr> WeakGoalMap;
-class Goal : public std::enable_shared_from_this<Goal>
+struct Goal : public std::enable_shared_from_this<Goal>
{
-public:
enum struct ExitCode {
Busy,
Success,
@@ -115,8 +114,6 @@ public:
IncompleteClosure,
};
-protected:
-
/* Backlink to the worker. */
Worker & worker;
@@ -144,6 +141,9 @@ protected:
/* Whether the goal is finished. */
ExitCode exitCode;
+ /* Exception containing an error message, if any. */
+ std::optional<Error> ex;
+
Goal(Worker & worker) : worker(worker)
{
nrFailed = nrNoSubstituters = nrIncompleteClosure = 0;
@@ -155,7 +155,6 @@ protected:
trace("goal destroyed");
}
-public:
virtual void work() = 0;
void addWaitee(GoalPtr waitee);
@@ -179,21 +178,14 @@ public:
return name;
}
- ExitCode getExitCode()
- {
- return exitCode;
- }
-
/* Callback in case of a timeout. It should wake up its waiters,
get rid of any running child processes that are being monitored
by the worker (important!), etc. */
- virtual void timedOut() = 0;
+ virtual void timedOut(Error && ex) = 0;
virtual string key() = 0;
-protected:
-
- virtual void amDone(ExitCode result);
+ void amDone(ExitCode result, std::optional<Error> ex = {});
};
@@ -309,7 +301,7 @@ public:
/* Make a goal (with caching). */
GoalPtr makeDerivationGoal(const StorePath & drvPath, const StringSet & wantedOutputs, BuildMode buildMode = bmNormal);
- std::shared_ptr<DerivationGoal> makeBasicDerivationGoal(StorePath && drvPath,
+ std::shared_ptr<DerivationGoal> makeBasicDerivationGoal(const StorePath & drvPath,
const BasicDerivation & drv, BuildMode buildMode = bmNormal);
GoalPtr makeSubstitutionGoal(const StorePath & storePath, RepairFlag repair = NoRepair);
@@ -361,7 +353,7 @@ public:
contents. */
bool pathContentsGood(const StorePath & path);
- void markContentsGood(StorePath && path);
+ void markContentsGood(const StorePath & path);
void updateProgress()
{
@@ -398,8 +390,7 @@ void Goal::waiteeDone(GoalPtr waitee, ExitCode result)
assert(waitees.find(waitee) != waitees.end());
waitees.erase(waitee);
- trace(format("waitee '%1%' done; %2% left") %
- waitee->name % waitees.size());
+ trace(fmt("waitee '%s' done; %d left", waitee->name, waitees.size()));
if (result == ExitCode::Failed || result == ExitCode::NoSubstituters || result == ExitCode::IncompleteClosure) ++nrFailed;
@@ -424,12 +415,20 @@ void Goal::waiteeDone(GoalPtr waitee, ExitCode result)
}
-void Goal::amDone(ExitCode result)
+void Goal::amDone(ExitCode result, std::optional<Error> ex)
{
trace("done");
assert(exitCode == ExitCode::Busy);
assert(result == ExitCode::Success || result == ExitCode::Failed || result == ExitCode::NoSubstituters || result == ExitCode::IncompleteClosure);
exitCode = result;
+
+ if (ex) {
+ if (!waiters.empty())
+ logError(ex->info());
+ else
+ this->ex = std::move(*ex);
+ }
+
for (auto & i : waiters) {
GoalPtr goal = i.lock();
if (goal) goal->waiteeDone(shared_from_this(), result);
@@ -459,7 +458,7 @@ static void commonChildInit(Pipe & logPipe)
that e.g. ssh cannot open /dev/tty) and it doesn't receive
terminal signals. */
if (setsid() == -1)
- throw SysError(format("creating a new session"));
+ throw SysError("creating a new session");
/* Dup the write side of the logger pipe into stderr. */
if (dup2(logPipe.writeSide.get(), STDERR_FILENO) == -1)
@@ -472,7 +471,7 @@ static void commonChildInit(Pipe & logPipe)
/* Reroute stdin to /dev/null. */
int fdDevNull = open(pathNullDevice.c_str(), O_RDWR);
if (fdDevNull == -1)
- throw SysError(format("cannot open '%1%'") % pathNullDevice);
+ throw SysError("cannot open '%1%'", pathNullDevice);
if (dup2(fdDevNull, STDIN_FILENO) == -1)
throw SysError("cannot dup null device into stdin");
close(fdDevNull);
@@ -494,12 +493,18 @@ void handleDiffHook(
auto diffRes = runProgram(diffHookOptions);
if (!statusOk(diffRes.first))
- throw ExecError(diffRes.first, fmt("diff-hook program '%1%' %2%", diffHook, statusToString(diffRes.first)));
+ throw ExecError(diffRes.first,
+ "diff-hook program '%1%' %2%",
+ diffHook,
+ statusToString(diffRes.first));
if (diffRes.second != "")
printError(chomp(diffRes.second));
} catch (Error & error) {
- printError("diff hook execution failed: %s", error.what());
+ ErrorInfo ei = error.info();
+ ei.hint = hintfmt("diff hook execution failed: %s",
+ (error.info().hint.has_value() ? error.info().hint->str() : ""));
+ logError(ei);
}
}
}
@@ -548,37 +553,37 @@ bool UserLock::findFreeUser() {
/* Get the members of the build-users-group. */
struct group * gr = getgrnam(settings.buildUsersGroup.get().c_str());
if (!gr)
- throw Error(format("the group '%1%' specified in 'build-users-group' does not exist")
- % settings.buildUsersGroup);
+ throw Error("the group '%1%' specified in 'build-users-group' does not exist",
+ settings.buildUsersGroup);
gid = gr->gr_gid;
/* Copy the result of getgrnam. */
Strings users;
for (char * * p = gr->gr_mem; *p; ++p) {
- debug(format("found build user '%1%'") % *p);
+ debug("found build user '%1%'", *p);
users.push_back(*p);
}
if (users.empty())
- throw Error(format("the build users group '%1%' has no members")
- % settings.buildUsersGroup);
+ throw Error("the build users group '%1%' has no members",
+ settings.buildUsersGroup);
/* Find a user account that isn't currently in use for another
build. */
for (auto & i : users) {
- debug(format("trying user '%1%'") % i);
+ debug("trying user '%1%'", i);
struct passwd * pw = getpwnam(i.c_str());
if (!pw)
- throw Error(format("the user '%1%' in the group '%2%' does not exist")
- % i % settings.buildUsersGroup);
+ throw Error("the user '%1%' in the group '%2%' does not exist",
+ i, settings.buildUsersGroup);
fnUserLock = (format("%1%/userpool/%2%") % settings.nixStateDir % pw->pw_uid).str();
AutoCloseFD fd = open(fnUserLock.c_str(), O_RDWR | O_CREAT | O_CLOEXEC, 0600);
if (!fd)
- throw SysError(format("opening user lock '%1%'") % fnUserLock);
+ throw SysError("opening user lock '%1%'", fnUserLock);
if (lockFile(fd.get(), ltWrite, false)) {
fdUserLock = std::move(fd);
@@ -587,8 +592,8 @@ bool UserLock::findFreeUser() {
/* Sanity check... */
if (uid == getuid() || uid == geteuid())
- throw Error(format("the Nix user should not be a member of '%1%'")
- % settings.buildUsersGroup);
+ throw Error("the Nix user should not be a member of '%1%'",
+ settings.buildUsersGroup);
#if __linux__
/* Get the list of supplementary groups of this build user. This
@@ -598,7 +603,7 @@ bool UserLock::findFreeUser() {
int err = getgrouplist(pw->pw_name, pw->pw_gid,
supplementaryGIDs.data(), &ngroups);
if (err == -1)
- throw Error(format("failed to get list of supplementary groups for '%1%'") % pw->pw_name);
+ throw Error("failed to get list of supplementary groups for '%1%'", pw->pw_name);
supplementaryGIDs.resize(ngroups);
#endif
@@ -607,6 +612,7 @@ bool UserLock::findFreeUser() {
return true;
}
}
+
return false;
}
@@ -868,6 +874,9 @@ private:
std::unique_ptr<Activity> act;
+ /* Activity that denotes waiting for a lock. */
+ std::unique_ptr<Activity> actLock;
+
std::map<ActivityId, Activity> builderActivities;
/* The remote machine on which we're building. */
@@ -897,16 +906,16 @@ private:
friend struct RestrictedStore;
public:
- DerivationGoal(StorePath && drvPath, const StringSet & wantedOutputs,
+ DerivationGoal(const StorePath & drvPath, const StringSet & wantedOutputs,
Worker & worker, BuildMode buildMode = bmNormal);
- DerivationGoal(StorePath && drvPath, const BasicDerivation & drv,
+ DerivationGoal(const StorePath & drvPath, const BasicDerivation & drv,
Worker & worker, BuildMode buildMode = bmNormal);
~DerivationGoal();
/* Whether we need to perform hash rewriting if there are valid output paths. */
bool needsHashRewrite();
- void timedOut() override;
+ void timedOut(Error && ex) override;
string key() override
{
@@ -921,7 +930,7 @@ public:
StorePath getDrvPath()
{
- return drvPath.clone();
+ return drvPath;
}
/* Add wanted outputs to an already existing derivation goal. */
@@ -1005,14 +1014,11 @@ private:
void repairClosure();
- void amDone(ExitCode result) override
- {
- Goal::amDone(result);
- }
-
void started();
- void done(BuildResult::Status status, const string & msg = "");
+ void done(
+ BuildResult::Status status,
+ std::optional<Error> ex = {});
StorePathSet exportReferences(const StorePathSet & storePaths);
};
@@ -1021,11 +1027,11 @@ private:
const Path DerivationGoal::homeDir = "/homeless-shelter";
-DerivationGoal::DerivationGoal(StorePath && drvPath, const StringSet & wantedOutputs,
+DerivationGoal::DerivationGoal(const StorePath & drvPath, const StringSet & wantedOutputs,
Worker & worker, BuildMode buildMode)
: Goal(worker)
, useDerivation(true)
- , drvPath(std::move(drvPath))
+ , drvPath(drvPath)
, wantedOutputs(wantedOutputs)
, buildMode(buildMode)
{
@@ -1038,11 +1044,11 @@ DerivationGoal::DerivationGoal(StorePath && drvPath, const StringSet & wantedOut
}
-DerivationGoal::DerivationGoal(StorePath && drvPath, const BasicDerivation & drv,
+DerivationGoal::DerivationGoal(const StorePath & drvPath, const BasicDerivation & drv,
Worker & worker, BuildMode buildMode)
: Goal(worker)
, useDerivation(false)
- , drvPath(std::move(drvPath))
+ , drvPath(drvPath)
, buildMode(buildMode)
{
this->drv = std::make_unique<BasicDerivation>(BasicDerivation(drv));
@@ -1106,10 +1112,10 @@ void DerivationGoal::killChild()
}
-void DerivationGoal::timedOut()
+void DerivationGoal::timedOut(Error && ex)
{
killChild();
- done(BuildResult::TimedOut);
+ done(BuildResult::TimedOut, ex);
}
@@ -1157,8 +1163,7 @@ void DerivationGoal::loadDerivation()
trace("loading derivation");
if (nrFailed != 0) {
- printError("cannot build missing derivation '%s'", worker.store.printStorePath(drvPath));
- done(BuildResult::MiscFailure);
+ done(BuildResult::MiscFailure, Error("cannot build missing derivation '%s'", worker.store.printStorePath(drvPath)));
return;
}
@@ -1194,7 +1199,13 @@ void DerivationGoal::haveDerivation()
return;
}
- parsedDrv = std::make_unique<ParsedDerivation>(drvPath.clone(), *drv);
+ parsedDrv = std::make_unique<ParsedDerivation>(drvPath, *drv);
+
+ if (parsedDrv->contentAddressed()) {
+ settings.requireExperimentalFeature("ca-derivations");
+ throw Error("ca-derivations isn't implemented yet");
+ }
+
/* We are first going to try to create the invalid output paths
through substitutes. If that doesn't work, we'll build
@@ -1302,14 +1313,18 @@ void DerivationGoal::repairClosure()
if (i.isDerivation()) {
Derivation drv = worker.store.derivationFromPath(i);
for (auto & j : drv.outputs)
- outputsToDrv.insert_or_assign(j.second.path.clone(), i.clone());
+ outputsToDrv.insert_or_assign(j.second.path, i);
}
/* Check each path (slow!). */
for (auto & i : outputClosure) {
if (worker.pathContentsGood(i)) continue;
- printError("found corrupted or missing path '%s' in the output closure of '%s'",
- worker.store.printStorePath(i), worker.store.printStorePath(drvPath));
+ logError({
+ .name = "Corrupt path in closure",
+ .hint = hintfmt(
+ "found corrupted or missing path '%s' in the output closure of '%s'",
+ worker.store.printStorePath(i), worker.store.printStorePath(drvPath))
+ });
auto drvPath2 = outputsToDrv.find(i);
if (drvPath2 == outputsToDrv.end())
addWaitee(worker.makeSubstitutionGoal(i, Repair));
@@ -1343,9 +1358,9 @@ void DerivationGoal::inputsRealised()
if (nrFailed != 0) {
if (!useDerivation)
throw Error("some dependencies of '%s' are missing", worker.store.printStorePath(drvPath));
- printError("cannot build derivation '%s': %s dependencies couldn't be built",
- worker.store.printStorePath(drvPath), nrFailed);
- done(BuildResult::DependencyFailed);
+ done(BuildResult::DependencyFailed, Error(
+ "%s dependencies of derivation '%s' failed to build",
+ nrFailed, worker.store.printStorePath(drvPath)));
return;
}
@@ -1427,10 +1442,15 @@ void DerivationGoal::tryToBuild()
lockFiles.insert(worker.store.Store::toRealPath(outPath));
if (!outputLocks.lockPaths(lockFiles, "", false)) {
+ if (!actLock)
+ actLock = std::make_unique<Activity>(*logger, Verbosity::Warn, ActivityType::BuildWaiting,
+ fmt("waiting for lock on %s", yellowtxt(showPaths(lockFiles))));
worker.waitForAWhile(shared_from_this());
return;
}
+ actLock.reset();
+
/* Now check again whether the outputs are valid. This is because
another process may have started building in parallel. After
it has finished and released the locks, we can (and should)
@@ -1446,7 +1466,7 @@ void DerivationGoal::tryToBuild()
return;
}
- missingPaths = cloneStorePathSet(drv->outputPaths());
+ missingPaths = drv->outputPaths();
if (buildMode != bmCheck)
for (auto & i : validPaths) missingPaths.erase(i);
@@ -1483,6 +1503,7 @@ void DerivationGoal::tryToBuild()
case rpAccept:
/* Yes, it has started doing so. Wait until we get
EOF from the hook. */
+ actLock.reset();
result.startTime = time(0); // inexact
state = &DerivationGoal::buildDone;
started();
@@ -1490,6 +1511,9 @@ void DerivationGoal::tryToBuild()
case rpPostpone:
/* Not now; wait until at least one child finishes or
the wake-up timeout expires. */
+ if (!actLock)
+ actLock = std::make_unique<Activity>(*logger, Verbosity::Warn, ActivityType::BuildWaiting,
+ fmt("waiting for a machine to build '%s'", yellowtxt(worker.store.printStorePath(drvPath))));
worker.waitForAWhile(shared_from_this());
outputLocks.unlock();
return;
@@ -1499,6 +1523,8 @@ void DerivationGoal::tryToBuild()
}
}
+ actLock.reset();
+
/* Make sure that we are allowed to start a build. If this
derivation prefers to be done locally, do it even if
maxBuildJobs is 0. */
@@ -1526,7 +1552,9 @@ void DerivationGoal::tryLocalBuild() {
uid. */
buildUser->kill();
} else {
- debug("waiting for build users");
+ if (!actLock)
+ actLock = std::make_unique<Activity>(*logger, Verbosity::Warn, ActivityType::BuildWaiting,
+ fmt("waiting for UID to build '%s'", yellowtxt(worker.store.printStorePath(drvPath))));
worker.waitForAWhile(shared_from_this());
return;
}
@@ -1537,17 +1565,18 @@ void DerivationGoal::tryLocalBuild() {
#endif
}
+ actLock.reset();
+
try {
/* Okay, we have to build. */
startBuilder();
} catch (BuildError & e) {
- printError(e.msg());
outputLocks.unlock();
buildUser.reset();
worker.permanentFailure = true;
- done(BuildResult::InputRejected, e.msg());
+ done(BuildResult::InputRejected, e);
return;
}
@@ -1659,10 +1688,10 @@ void DerivationGoal::buildDone()
}
auto msg = fmt("builder for '%s' %s",
- worker.store.printStorePath(drvPath),
+ yellowtxt(worker.store.printStorePath(drvPath)),
statusToString(status));
- if (!settings.verboseBuild && !logTail.empty()) {
+ if (!logger->isVerbose() && !logTail.empty()) {
msg += (format("; last %d log lines:") % logTail.size()).str();
for (auto & line : logTail)
msg += "\n " + line;
@@ -1711,11 +1740,7 @@ void DerivationGoal::buildDone()
}
void flushLine() {
- if (settings.verboseBuild) {
- printError("post-build-hook: " + currentLine);
- } else {
- act.result(ResultType::PostBuildLogLine, currentLine);
- }
+ act.result(ResultType::PostBuildLogLine, currentLine);
currentLine.clear();
}
@@ -1764,8 +1789,6 @@ void DerivationGoal::buildDone()
outputLocks.unlock();
} catch (BuildError & e) {
- printError(e.msg());
-
outputLocks.unlock();
BuildResult::Status st = BuildResult::MiscFailure;
@@ -1784,7 +1807,7 @@ void DerivationGoal::buildDone()
BuildResult::PermanentFailure;
}
- done(st, e.msg());
+ done(st, e);
return;
}
@@ -1827,7 +1850,7 @@ HookReply DerivationGoal::tryBuildHook()
}
}
- debug(format("hook reply is '%1%'") % reply);
+ debug("hook reply is '%1%'", reply);
if (reply == "decline")
return rpDecline;
@@ -1843,8 +1866,12 @@ HookReply DerivationGoal::tryBuildHook()
} catch (SysError & e) {
if (e.errNo == EPIPE) {
- printError("build hook died unexpectedly: %s",
- chomp(drainFD(worker.hook->fromHook.readSide.get())));
+ logError({
+ .name = "Build hook died",
+ .hint = hintfmt(
+ "build hook died unexpectedly: %s",
+ chomp(drainFD(worker.hook->fromHook.readSide.get())))
+ });
worker.hook = 0;
return rpDecline;
} else
@@ -1900,14 +1927,14 @@ StorePathSet DerivationGoal::exportReferences(const StorePathSet & 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(singleton(storePath), paths);
+ 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 = cloneStorePathSet(paths);
+ auto paths2 = paths;
for (auto & j : paths2) {
if (j.isDerivation()) {
@@ -2024,7 +2051,7 @@ void DerivationGoal::startBuilder()
string s = get(drv->env, "exportReferencesGraph").value_or("");
Strings ss = tokenizeString<Strings>(s);
if (ss.size() % 2 != 0)
- throw BuildError(format("odd number of tokens in 'exportReferencesGraph': '%1%'") % s);
+ throw BuildError("odd number of tokens in 'exportReferencesGraph': '%1%'", s);
for (Strings::iterator i = ss.begin(); i != ss.end(); ) {
string fileName = *i++;
static std::regex regex("[A-Za-z_][A-Za-z0-9_.-]*");
@@ -2036,7 +2063,7 @@ void DerivationGoal::startBuilder()
/* Write closure info to <fileName>. */
writeFile(tmpDir + "/" + fileName,
worker.store.makeValidityRegistration(
- exportReferences(singleton(storePath)), false, false));
+ exportReferences({storePath}), false, false));
}
}
@@ -2073,7 +2100,7 @@ void DerivationGoal::startBuilder()
worker.store.computeFSClosure(worker.store.parseStorePath(worker.store.toStorePath(i.second.source)), closure);
} catch (InvalidPath & e) {
} catch (Error & e) {
- throw Error(format("while processing 'sandbox-paths': %s") % e.what());
+ throw Error("while processing 'sandbox-paths': %s", e.what());
}
for (auto & i : closure) {
auto p = worker.store.printStorePath(i);
@@ -2120,10 +2147,10 @@ void DerivationGoal::startBuilder()
printMsg(Verbosity::Chatty, format("setting up chroot environment in '%1%'") % chrootRootDir);
if (mkdir(chrootRootDir.c_str(), 0750) == -1)
- throw SysError(format("cannot create '%1%'") % chrootRootDir);
+ throw SysError("cannot create '%1%'", chrootRootDir);
if (buildUser && chown(chrootRootDir.c_str(), 0, buildUser->getGID()) == -1)
- throw SysError(format("cannot change ownership of '%1%'") % chrootRootDir);
+ throw SysError("cannot change ownership of '%1%'", chrootRootDir);
/* Create a writable /tmp in the chroot. Many builders need
this. (Of course they should really respect $TMPDIR
@@ -2167,7 +2194,7 @@ void DerivationGoal::startBuilder()
chmod_(chrootStoreDir, 01775);
if (buildUser && chown(chrootStoreDir.c_str(), 0, buildUser->getGID()) == -1)
- throw SysError(format("cannot change ownership of '%1%'") % chrootStoreDir);
+ throw SysError("cannot change ownership of '%1%'", chrootStoreDir);
for (auto & i : inputPaths) {
auto p = worker.store.printStorePath(i);
@@ -2200,7 +2227,7 @@ void DerivationGoal::startBuilder()
if (needsHashRewrite()) {
if (pathExists(homeDir))
- throw Error(format("home directory '%1%' exists; please remove it to assure purity of builds without sandboxing") % homeDir);
+ throw Error("home directory '%1%' exists; please remove it to assure purity of builds without sandboxing", homeDir);
/* We're not doing a chroot build, but we have some valid
output paths. Since we can't just overwrite or delete
@@ -2221,7 +2248,7 @@ void DerivationGoal::startBuilder()
for (auto & i : missingPaths)
if (worker.store.isValidPath(i) && pathExists(worker.store.printStorePath(i))) {
addHashRewrite(i);
- redirectedBadOutputs.insert(i.clone());
+ redirectedBadOutputs.insert(i);
}
}
@@ -2245,8 +2272,7 @@ void DerivationGoal::startBuilder()
if (line == "extra-sandbox-paths" || line == "extra-chroot-dirs") {
state = stExtraChrootDirs;
} else {
- throw Error(format("unknown pre-build hook command '%1%'")
- % line);
+ throw Error("unknown pre-build hook command '%1%'", line);
}
} else if (state == stExtraChrootDirs) {
if (line == "") {
@@ -2268,7 +2294,7 @@ void DerivationGoal::startBuilder()
startDaemon();
/* Run the builder. */
- printMsg(Verbosity::Chatty, format("executing builder '%1%'") % drv->builder);
+ printMsg(Verbosity::Chatty, "executing builder '%1%'", drv->builder);
/* Create the log file. */
Path logFile = openLogFile();
@@ -2717,8 +2743,8 @@ struct RestrictedStore : public LocalFSStore
StorePathSet queryAllValidPaths() override
{
StorePathSet paths;
- for (auto & p : goal.inputPaths) paths.insert(p.clone());
- for (auto & p : goal.addedPaths) paths.insert(p.clone());
+ for (auto & p : goal.inputPaths) paths.insert(p);
+ for (auto & p : goal.addedPaths) paths.insert(p);
return paths;
}
@@ -2747,9 +2773,6 @@ struct RestrictedStore : public LocalFSStore
StorePathSet queryDerivationOutputs(const StorePath & path) override
{ throw Error("queryDerivationOutputs"); }
- StringSet queryDerivationOutputNames(const StorePath & path) override
- { throw Error("queryDerivationOutputNames"); }
-
std::optional<StorePath> queryPathFromHashPart(const std::string & hashPart) override
{ throw Error("queryPathFromHashPart"); }
@@ -2809,7 +2832,7 @@ struct RestrictedStore : public LocalFSStore
auto drv = derivationFromPath(path.path);
for (auto & output : drv.outputs)
if (wantOutput(output.first, path.outputs))
- newPaths.insert(output.second.path.clone());
+ newPaths.insert(output.second.path);
} else if (!goal.isAllowed(path.path))
throw InvalidPath("cannot build unknown path '%s' in recursive Nix", printStorePath(path.path));
}
@@ -2854,7 +2877,7 @@ struct RestrictedStore : public LocalFSStore
if (goal.isAllowed(path.path))
allowed.emplace_back(path);
else
- unknown.insert(path.path.clone());
+ unknown.insert(path.path);
}
next->queryMissing(allowed, willBuild, willSubstitute,
@@ -2949,7 +2972,7 @@ void DerivationGoal::addDependency(const StorePath & path)
{
if (isAllowed(path)) return;
- addedPaths.insert(path.clone());
+ addedPaths.insert(path);
/* If we're doing a sandbox build, then we have to make the path
appear in the sandbox. */
@@ -3009,7 +3032,7 @@ void DerivationGoal::chownToBuilder(const Path & path)
{
if (!buildUser) return;
if (chown(path.c_str(), buildUser->getUID(), buildUser->getGID()) == -1)
- throw SysError(format("cannot change ownership of '%1%'") % path);
+ throw SysError("cannot change ownership of '%1%'", path);
}
@@ -3146,7 +3169,7 @@ void DerivationGoal::runChild()
/* 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);
+ throw SysError("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
@@ -3208,7 +3231,7 @@ void DerivationGoal::runChild()
filesystem that we want in the chroot
environment. */
auto doBind = [&](const Path & source, const Path & target, bool optional = false) {
- debug(format("bind mounting '%1%' to '%2%'") % source % target);
+ debug("bind mounting '%1%' to '%2%'", source, target);
struct stat st;
if (stat(source.c_str(), &st) == -1) {
if (optional && errno == ENOENT)
@@ -3280,16 +3303,16 @@ void DerivationGoal::runChild()
/* Do the chroot(). */
if (chdir(chrootRootDir.c_str()) == -1)
- throw SysError(format("cannot change directory to '%1%'") % chrootRootDir);
+ throw SysError("cannot change directory to '%1%'", chrootRootDir);
if (mkdir("real-root", 0) == -1)
throw SysError("cannot create real-root directory");
if (pivot_root(".", "real-root") == -1)
- throw SysError(format("cannot pivot old root directory onto '%1%'") % (chrootRootDir + "/real-root"));
+ throw SysError("cannot pivot old root directory onto '%1%'", (chrootRootDir + "/real-root"));
if (chroot(".") == -1)
- throw SysError(format("cannot change root directory to '%1%'") % chrootRootDir);
+ throw SysError("cannot change root directory to '%1%'", chrootRootDir);
if (umount2("real-root", MNT_DETACH) == -1)
throw SysError("cannot unmount real root filesystem");
@@ -3310,7 +3333,7 @@ void DerivationGoal::runChild()
#endif
if (chdir(tmpDirInSandbox.c_str()) == -1)
- throw SysError(format("changing into '%1%'") % tmpDir);
+ throw SysError("changing into '%1%'", tmpDir);
/* Close all other file descriptors. */
closeMostFDs({STDIN_FILENO, STDOUT_FILENO, STDERR_FILENO});
@@ -3449,9 +3472,9 @@ void DerivationGoal::runChild()
sandboxProfile += "(allow file-read* file-write* process-exec\n";
for (auto & i : dirsInChroot) {
if (i.first != i.second.source)
- throw Error(format(
- "can't map '%1%' to '%2%': mismatched impure paths not supported on Darwin")
- % i.first % i.second.source);
+ throw Error(
+ "can't map '%1%' to '%2%': mismatched impure paths not supported on Darwin",
+ i.first, i.second.source);
string path = i.first;
struct stat st;
@@ -3542,7 +3565,7 @@ void DerivationGoal::runChild()
else if (drv->builder == "builtin:unpack-channel")
builtinUnpackChannel(drv2);
else
- throw Error(format("unsupported builtin function '%1%'") % string(drv->builder, 8));
+ throw Error("unsupported builtin function '%1%'", string(drv->builder, 8));
_exit(0);
} catch (std::exception & e) {
writeFull(STDERR_FILENO, "error: " + string(e.what()) + "\n");
@@ -3552,7 +3575,7 @@ void DerivationGoal::runChild()
execve(builder, stringsToCharPtrs(args).data(), stringsToCharPtrs(envStrs).data());
- throw SysError(format("executing '%1%'") % drv->builder);
+ throw SysError("executing '%1%'", drv->builder);
} catch (std::exception & e) {
writeFull(STDERR_FILENO, "\1while setting up the build environment: " + string(e.what()) + "\n");
@@ -3571,7 +3594,7 @@ StorePathSet parseReferenceSpecifiers(Store & store, const BasicDerivation & drv
if (store.isStorePath(i))
result.insert(store.parseStorePath(i));
else if (drv.outputs.count(i))
- result.insert(drv.outputs.find(i)->second.path.clone());
+ result.insert(drv.outputs.find(i)->second.path);
else throw BuildError("derivation contains an illegal reference specifier '%s'", i);
}
return result;
@@ -3585,7 +3608,7 @@ static void moveCheckToStore(const Path & src, const Path & dst)
directory's parent link ".."). */
struct stat st;
if (lstat(src.c_str(), &st) == -1) {
- throw SysError(format("getting attributes of path '%1%'") % src);
+ throw SysError("getting attributes of path '%1%'", src);
}
bool changePerm = (geteuid() && S_ISDIR(st.st_mode) && !(st.st_mode & S_IWUSR));
@@ -3594,7 +3617,7 @@ static void moveCheckToStore(const Path & src, const Path & dst)
chmod_(src, st.st_mode | S_IWUSR);
if (rename(src.c_str(), dst.c_str()))
- throw SysError(format("renaming '%1%' to '%2%'") % src % dst);
+ throw SysError("renaming '%1%' to '%2%'", src, dst);
if (changePerm)
chmod_(dst, st.st_mode);
@@ -3629,9 +3652,9 @@ void DerivationGoal::registerOutputs()
output paths, and any paths that have been built via recursive
Nix calls. */
StorePathSet referenceablePaths;
- for (auto & p : inputPaths) referenceablePaths.insert(p.clone());
- for (auto & i : drv->outputs) referenceablePaths.insert(i.second.path.clone());
- for (auto & p : addedPaths) referenceablePaths.insert(p.clone());
+ 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
@@ -3660,7 +3683,7 @@ void DerivationGoal::registerOutputs()
replaceValidPath(path, actualPath);
else
if (buildMode != bmCheck && rename(actualPath.c_str(), worker.store.toRealPath(path).c_str()) == -1)
- throw SysError(format("moving build output '%1%' from the sandbox to the Nix store") % path);
+ throw SysError("moving build output '%1%' from the sandbox to the Nix store", path);
}
if (buildMode != bmCheck) actualPath = worker.store.toRealPath(path);
}
@@ -3681,13 +3704,16 @@ void DerivationGoal::registerOutputs()
user. */
if ((!S_ISLNK(st.st_mode) && (st.st_mode & (S_IWGRP | S_IWOTH))) ||
(buildUser && st.st_uid != buildUser->getUID()))
- throw BuildError(format("suspicious ownership or permission on '%1%'; rejecting this build output") % path);
+ throw BuildError("suspicious ownership or permission on '%1%'; rejecting this build output", path);
#endif
/* Apply hash rewriting if necessary. */
bool rewritten = false;
if (!outputRewrites.empty()) {
- printError(format("warning: rewriting hashes in '%1%'; cross fingers") % path);
+ logWarning({
+ .name = "Rewriting hashes",
+ .hint = hintfmt("rewriting hashes in '%1%'; cross fingers", path)
+ });
/* Canonicalise first. This ensures that the path we're
rewriting doesn't contain a hard link to /etc/shadow or
@@ -3719,8 +3745,9 @@ void DerivationGoal::registerOutputs()
/* The output path should be a regular file without execute permission. */
if (!S_ISREG(st.st_mode) || (st.st_mode & S_IXUSR) != 0)
throw BuildError(
- format("output path '%1%' should be a non-executable regular file "
- "since recursive hashing is not enabled (outputHashMode=flat)") % path);
+ "output path '%1%' should be a non-executable regular file "
+ "since recursive hashing is not enabled (outputHashMode=flat)",
+ path);
}
/* Check the hash. In hash mode, move the path produced by
@@ -3738,7 +3765,7 @@ void DerivationGoal::registerOutputs()
worker.hashMismatch = true;
delayedException = std::make_exception_ptr(
BuildError("hash mismatch in fixed-output derivation '%s':\n wanted: %s\n got: %s",
- worker.store.printStorePath(dest), h.to_string(Base::SRI), h2.to_string(Base::SRI)));
+ worker.store.printStorePath(dest), h.to_string(Base::SRI, true), h2.to_string(Base::SRI, true)));
Path actualDest = worker.store.Store::toRealPath(dest);
@@ -3826,7 +3853,7 @@ void DerivationGoal::registerOutputs()
info.narHash = hash.first;
info.narSize = hash.second;
info.references = std::move(references);
- info.deriver = drvPath.clone();
+ info.deriver = drvPath;
info.ultimate = true;
info.ca = ca;
worker.store.signPathInfo(info);
@@ -3850,10 +3877,10 @@ void DerivationGoal::registerOutputs()
result.isNonDeterministic = true;
Path prev = worker.store.printStorePath(i->second.path) + checkSuffix;
bool prevExists = keepPreviousRound && pathExists(prev);
- auto msg = prevExists
- ? fmt("output '%s' of '%s' differs from '%s' from previous round",
+ hintformat hint = prevExists
+ ? hintfmt("output '%s' of '%s' differs from '%s' from previous round",
worker.store.printStorePath(i->second.path), worker.store.printStorePath(drvPath), prev)
- : fmt("output '%s' of '%s' differs from previous round",
+ : hintfmt("output '%s' of '%s' differs from previous round",
worker.store.printStorePath(i->second.path), worker.store.printStorePath(drvPath));
handleDiffHook(
@@ -3863,9 +3890,13 @@ void DerivationGoal::registerOutputs()
worker.store.printStorePath(drvPath), tmpDir);
if (settings.enforceDeterminism)
- throw NotDeterministic(msg);
+ throw NotDeterministic(hint);
+
+ logError({
+ .name = "Output determinism error",
+ .hint = hint
+ });
- printError(msg);
curRound = nrRounds; // we know enough, bail out early
}
}
@@ -3937,23 +3968,23 @@ void DerivationGoal::checkOutputs(const std::map<Path, ValidPathInfo> & outputs)
uint64_t closureSize = 0;
StorePathSet pathsDone;
std::queue<StorePath> pathsLeft;
- pathsLeft.push(path.clone());
+ pathsLeft.push(path);
while (!pathsLeft.empty()) {
- auto path = pathsLeft.front().clone();
+ auto path = pathsLeft.front();
pathsLeft.pop();
- if (!pathsDone.insert(path.clone()).second) continue;
+ if (!pathsDone.insert(path).second) continue;
auto i = outputsByPath.find(worker.store.printStorePath(path));
if (i != outputsByPath.end()) {
closureSize += i->second.narSize;
for (auto & ref : i->second.references)
- pathsLeft.push(ref.clone());
+ pathsLeft.push(ref);
} else {
auto info = worker.store.queryPathInfo(path);
closureSize += info->narSize;
for (auto & ref : info->references)
- pathsLeft.push(ref.clone());
+ pathsLeft.push(ref);
}
}
@@ -3980,8 +4011,8 @@ void DerivationGoal::checkOutputs(const std::map<Path, ValidPathInfo> & outputs)
auto spec = parseReferenceSpecifiers(worker.store, *drv, *value);
auto used = recursive
- ? cloneStorePathSet(getClosure(info.path).first)
- : cloneStorePathSet(info.references);
+ ? getClosure(info.path).first
+ : info.references;
if (recursive && checks.ignoreSelfRefs)
used.erase(info.path);
@@ -3991,10 +4022,10 @@ void DerivationGoal::checkOutputs(const std::map<Path, ValidPathInfo> & outputs)
for (auto & i : used)
if (allowed) {
if (!spec.count(i))
- badPaths.insert(i.clone());
+ badPaths.insert(i);
} else {
if (spec.count(i))
- badPaths.insert(i.clone());
+ badPaths.insert(i);
}
if (!badPaths.empty()) {
@@ -4083,7 +4114,7 @@ Path DerivationGoal::openLogFile()
settings.compressLog ? ".bz2" : "");
fdLogFile = open(logFileName.c_str(), O_CREAT | O_WRONLY | O_TRUNC | O_CLOEXEC, 0666);
- if (!fdLogFile) throw SysError(format("creating log file '%1%'") % logFileName);
+ if (!fdLogFile) throw SysError("creating log file '%1%'", logFileName);
logFileSink = std::make_shared<FdSink>(fdLogFile.get());
@@ -4129,11 +4160,11 @@ void DerivationGoal::handleChildOutput(int fd, const string & data)
{
logSize += data.size();
if (settings.maxLogSize && logSize > settings.maxLogSize) {
- printError(
- format("%1% killed after writing more than %2% bytes of log output")
- % getName() % settings.maxLogSize);
killChild();
- done(BuildResult::LogLimitExceeded);
+ done(
+ BuildResult::LogLimitExceeded,
+ Error("%s killed after writing more than %d bytes of log output",
+ getName(), settings.maxLogSize));
return;
}
@@ -4175,13 +4206,8 @@ void DerivationGoal::flushLine()
;
else {
- if (settings.verboseBuild &&
- (settings.printRepeatedBuilds || curRound == 1))
- printError(currentLogLine);
- else {
- logTail.push_back(currentLogLine);
- if (logTail.size() > settings.logLines) logTail.pop_front();
- }
+ logTail.push_back(currentLogLine);
+ if (logTail.size() > settings.logLines) logTail.pop_front();
act->result(ResultType::BuildLogLine, currentLogLine);
}
@@ -4199,7 +4225,7 @@ StorePathSet DerivationGoal::checkPathValidity(bool returnValid, bool checkHash)
bool good =
worker.store.isValidPath(i.second.path) &&
(!checkHash || worker.pathContentsGood(i.second.path));
- if (good == returnValid) result.insert(i.second.path.clone());
+ if (good == returnValid) result.insert(i.second.path);
}
return result;
}
@@ -4215,15 +4241,16 @@ void DerivationGoal::addHashRewrite(const StorePath & path)
deletePath(worker.store.printStorePath(p));
inputRewrites[h1] = h2;
outputRewrites[h2] = h1;
- redirectedOutputs.insert_or_assign(path.clone(), std::move(p));
+ redirectedOutputs.insert_or_assign(path, std::move(p));
}
-void DerivationGoal::done(BuildResult::Status status, const string & msg)
+void DerivationGoal::done(BuildResult::Status status, std::optional<Error> ex)
{
result.status = status;
- result.errorMsg = msg;
- amDone(result.success() ? ExitCode::Success : ExitCode::Failed);
+ if (ex)
+ result.errorMsg = ex->what();
+ amDone(result.success() ? ExitCode::Success : ExitCode::Failed, ex);
if (result.status == BuildResult::TimedOut)
worker.timedOut = true;
if (result.status == BuildResult::PermanentFailure)
@@ -4289,10 +4316,10 @@ private:
GoalState state;
public:
- SubstitutionGoal(StorePath && storePath, Worker & worker, RepairFlag repair = NoRepair);
+ SubstitutionGoal(const StorePath & storePath, Worker & worker, RepairFlag repair = NoRepair);
~SubstitutionGoal();
- void timedOut() override { abort(); };
+ void timedOut(Error && ex) override { abort(); };
string key() override
{
@@ -4315,18 +4342,13 @@ public:
void handleChildOutput(int fd, const string & data) override;
void handleEOF(int fd) override;
- StorePath getStorePath() { return storePath.clone(); }
-
- void amDone(ExitCode result) override
- {
- Goal::amDone(result);
- }
+ StorePath getStorePath() { return storePath; }
};
-SubstitutionGoal::SubstitutionGoal(StorePath && storePath, Worker & worker, RepairFlag repair)
+SubstitutionGoal::SubstitutionGoal(const StorePath & storePath, Worker & worker, RepairFlag repair)
: Goal(worker)
- , storePath(std::move(storePath))
+ , storePath(storePath)
, repair(repair)
{
state = &SubstitutionGoal::init;
@@ -4421,7 +4443,7 @@ void SubstitutionGoal::tryNext()
throw;
} catch (Error & e) {
if (settings.tryFallback) {
- printError(e.what());
+ logError(e.info());
tryNext();
return;
}
@@ -4447,8 +4469,11 @@ void SubstitutionGoal::tryNext()
&& !sub->isTrusted
&& !info->checkSignatures(worker.store, worker.store.getPublicKeys()))
{
- printError("warning: substituter '%s' does not have a valid signature for path '%s'",
- sub->getUri(), worker.store.printStorePath(storePath));
+ logWarning({
+ .name = "Invalid path signature",
+ .hint = hintfmt("substituter '%s' does not have a valid signature for path '%s'",
+ sub->getUri(), worker.store.printStorePath(storePath))
+ });
tryNext();
return;
}
@@ -4557,7 +4582,7 @@ void SubstitutionGoal::finished()
return;
}
- worker.markContentsGood(storePath.clone());
+ worker.markContentsGood(storePath);
printMsg(Verbosity::Chatty, "substitution of path '%s' succeeded", worker.store.printStorePath(storePath));
@@ -4591,7 +4616,6 @@ void SubstitutionGoal::handleEOF(int fd)
if (fd == outPipe.readSide.get()) worker.wakeUp(shared_from_this());
}
-
//////////////////////////////////////////////////////////////////////
@@ -4628,10 +4652,10 @@ Worker::~Worker()
GoalPtr Worker::makeDerivationGoal(const StorePath & path,
const StringSet & wantedOutputs, BuildMode buildMode)
{
- GoalPtr goal = derivationGoals[path.clone()].lock(); // FIXME
+ GoalPtr goal = derivationGoals[path].lock(); // FIXME
if (!goal) {
- goal = std::make_shared<DerivationGoal>(path.clone(), wantedOutputs, *this, buildMode);
- derivationGoals.insert_or_assign(path.clone(), goal);
+ goal = std::make_shared<DerivationGoal>(path, wantedOutputs, *this, buildMode);
+ derivationGoals.insert_or_assign(path, goal);
wakeUp(goal);
} else
(dynamic_cast<DerivationGoal *>(goal.get()))->addWantedOutputs(wantedOutputs);
@@ -4639,10 +4663,10 @@ GoalPtr Worker::makeDerivationGoal(const StorePath & path,
}
-std::shared_ptr<DerivationGoal> Worker::makeBasicDerivationGoal(StorePath && drvPath,
+std::shared_ptr<DerivationGoal> Worker::makeBasicDerivationGoal(const StorePath & drvPath,
const BasicDerivation & drv, BuildMode buildMode)
{
- auto goal = std::make_shared<DerivationGoal>(std::move(drvPath), drv, *this, buildMode);
+ auto goal = std::make_shared<DerivationGoal>(drvPath, drv, *this, buildMode);
wakeUp(goal);
return goal;
}
@@ -4650,10 +4674,10 @@ std::shared_ptr<DerivationGoal> Worker::makeBasicDerivationGoal(StorePath && drv
GoalPtr Worker::makeSubstitutionGoal(const StorePath & path, RepairFlag repair)
{
- GoalPtr goal = substitutionGoals[path.clone()].lock(); // FIXME
+ GoalPtr goal = substitutionGoals[path].lock(); // FIXME
if (!goal) {
- goal = std::make_shared<SubstitutionGoal>(path.clone(), *this, repair);
- substitutionGoals.insert_or_assign(path.clone(), goal);
+ goal = std::make_shared<SubstitutionGoal>(path, *this, repair);
+ substitutionGoals.insert_or_assign(path, goal);
wakeUp(goal);
}
return goal;
@@ -4682,7 +4706,7 @@ void Worker::removeGoal(GoalPtr goal)
topGoals.erase(goal);
/* If a top-level goal failed, then kill all other goals
(unless keepGoing was set). */
- if (goal->getExitCode() == Goal::ExitCode::Failed && !settings.keepGoing)
+ if (goal->exitCode == Goal::ExitCode::Failed && !settings.keepGoing)
topGoals.clear();
}
@@ -4808,9 +4832,9 @@ void Worker::run(const Goals & _topGoals)
if (!children.empty() || !waitingForAWhile.empty())
waitForInput();
else {
- if (awake.empty() && 0 == settings.maxBuildJobs) throw Error(
- "unable to start any build; either increase '--max-jobs' "
- "or enable remote builds");
+ if (awake.empty() && 0 == settings.maxBuildJobs)
+ throw Error("unable to start any build; either increase '--max-jobs' "
+ "or enable remote builds");
assert(!awake.empty());
}
}
@@ -4823,7 +4847,6 @@ void Worker::run(const Goals & _topGoals)
assert(!settings.keepGoing || children.empty());
}
-
void Worker::waitForInput()
{
printMsg(Verbosity::Vomit, "waiting for children");
@@ -4861,8 +4884,6 @@ void Worker::waitForInput()
up after a few seconds at most. */
if (!waitingForAWhile.empty()) {
useTimeout = true;
- if (lastWokenUp == steady_time_point::min())
- printError("waiting for locks, build slots or build users...");
if (lastWokenUp == steady_time_point::min() || lastWokenUp > before) lastWokenUp = before;
timeout = std::max(1L,
(long) std::chrono::duration_cast<std::chrono::seconds>(
@@ -4911,15 +4932,15 @@ void Worker::waitForInput()
// FIXME: is there a cleaner way to handle pt close
// than EIO? Is this even standard?
if (rd == 0 || (rd == -1 && errno == EIO)) {
- debug(format("%1%: got EOF") % goal->getName());
+ debug("%1%: got EOF", goal->getName());
goal->handleEOF(k);
j->fds.erase(k);
} else if (rd == -1) {
if (errno != EINTR)
throw SysError("%s: read failed", goal->getName());
} else {
- printMsg(Verbosity::Vomit, format("%1%: read %2% bytes")
- % goal->getName() % rd);
+ printMsg(Verbosity::Vomit, "%1%: read %2% bytes",
+ goal->getName(), rd);
string data((char *) buffer.data(), rd);
j->lastOutput = after;
goal->handleChildOutput(k, data);
@@ -4927,26 +4948,24 @@ void Worker::waitForInput()
}
}
- if (goal->getExitCode() == Goal::ExitCode::Busy &&
+ if (goal->exitCode == Goal::ExitCode::Busy &&
0 != settings.maxSilentTime &&
j->respectTimeouts &&
after - j->lastOutput >= std::chrono::seconds(settings.maxSilentTime))
{
- printError(
- format("%1% timed out after %2% seconds of silence")
- % goal->getName() % settings.maxSilentTime);
- goal->timedOut();
+ goal->timedOut(Error(
+ "%1% timed out after %2% seconds of silence",
+ goal->getName(), settings.maxSilentTime));
}
- else if (goal->getExitCode() == Goal::ExitCode::Busy &&
+ else if (goal->exitCode == Goal::ExitCode::Busy &&
0 != settings.buildTimeout &&
j->respectTimeouts &&
after - j->timeStarted >= std::chrono::seconds(settings.buildTimeout))
{
- printError(
- format("%1% timed out after %2% seconds")
- % goal->getName() % settings.buildTimeout);
- goal->timedOut();
+ goal->timedOut(Error(
+ "%1% timed out after %2% seconds",
+ goal->getName(), settings.buildTimeout));
}
}
@@ -5003,15 +5022,19 @@ bool Worker::pathContentsGood(const StorePath & path)
Hash nullHash(HashType::SHA256);
res = info->narHash == nullHash || info->narHash == current.first;
}
- pathContentsGoodCache.insert_or_assign(path.clone(), res);
- if (!res) printError("path '%s' is corrupted or missing!", store.printStorePath(path));
+ pathContentsGoodCache.insert_or_assign(path, res);
+ if (!res)
+ logError({
+ .name = "Corrupted path",
+ .hint = hintfmt("path '%s' is corrupted or missing!", store.printStorePath(path))
+ });
return res;
}
-void Worker::markContentsGood(StorePath && path)
+void Worker::markContentsGood(const StorePath & path)
{
- pathContentsGoodCache.insert_or_assign(std::move(path), true);
+ pathContentsGoodCache.insert_or_assign(path, true);
}
@@ -5048,24 +5071,35 @@ void LocalStore::buildPaths(const std::vector<StorePathWithOutputs> & drvPaths,
worker.run(goals);
StorePathSet failed;
+ std::optional<Error> ex;
for (auto & i : goals) {
- if (i->getExitCode() != Goal::ExitCode::Success) {
+ if (i->ex) {
+ if (ex)
+ logError(i->ex->info());
+ else
+ ex = i->ex;
+ }
+ if (i->exitCode != Goal::ExitCode::Success) {
DerivationGoal * i2 = dynamic_cast<DerivationGoal *>(i.get());
if (i2) failed.insert(i2->getDrvPath());
else failed.insert(dynamic_cast<SubstitutionGoal *>(i.get())->getStorePath());
}
}
- if (!failed.empty())
+ if (failed.size() == 1 && ex) {
+ ex->status = worker.exitStatus();
+ throw *ex;
+ } else if (!failed.empty()) {
+ if (ex) logError(ex->info());
throw Error(worker.exitStatus(), "build of %s failed", showPaths(failed));
+ }
}
-
BuildResult LocalStore::buildDerivation(const StorePath & drvPath, const BasicDerivation & drv,
BuildMode buildMode)
{
Worker worker(*this);
- auto goal = worker.makeBasicDerivationGoal(drvPath.clone(), drv, buildMode);
+ auto goal = worker.makeBasicDerivationGoal(drvPath, drv, buildMode);
BuildResult result;
@@ -5086,7 +5120,7 @@ void LocalStore::ensurePath(const StorePath & path)
/* If the path is already valid, we're done. */
if (isValidPath(path)) return;
- primeCache(*this, {StorePathWithOutputs(path)});
+ primeCache(*this, {{path}});
Worker worker(*this);
GoalPtr goal = worker.makeSubstitutionGoal(path);
@@ -5094,8 +5128,13 @@ void LocalStore::ensurePath(const StorePath & path)
worker.run(goals);
- if (goal->getExitCode() != Goal::ExitCode::Success)
- throw Error(worker.exitStatus(), "path '%s' does not exist and cannot be created", printStorePath(path));
+ if (goal->exitCode != Goal::ExitCode::Success) {
+ if (goal->ex) {
+ goal->ex->status = worker.exitStatus();
+ throw *goal->ex;
+ } else
+ throw Error(worker.exitStatus(), "path '%s' does not exist and cannot be created", printStorePath(path));
+ }
}
@@ -5107,7 +5146,7 @@ void LocalStore::repairPath(const StorePath & path)
worker.run(goals);
- if (goal->getExitCode() != Goal::ExitCode::Success) {
+ if (goal->exitCode != Goal::ExitCode::Success) {
/* Since substituting the path didn't work, if we have a valid
deriver, then rebuild the deriver. */
auto info = queryPathInfo(path);