diff --git a/doc/manual/src/projects.md b/doc/manual/src/projects.md index a399406d..f7c4975f 100644 --- a/doc/manual/src/projects.md +++ b/doc/manual/src/projects.md @@ -404,3 +404,10 @@ analogous: | `String value` | `gitea_status_repo` | *Name of the `Git checkout` input* | | `String value` | `gitea_http_url` | *Public URL of `gitea`*, optional | +Content-addressed derivations +----------------------------- + +Hydra can to a certain extent use the [`ca-derivations` experimental Nix feature](https://github.com/NixOS/rfcs/pull/62). +To use it, make sure that the Nix version you use is at least as recent as the one used in hydra's flake. + +Be warned that this support is still highly experimental, and anything beyond the basic functionality might be broken at that point. diff --git a/src/hydra-eval-jobs/hydra-eval-jobs.cc b/src/hydra-eval-jobs/hydra-eval-jobs.cc index de7ae7ba..13f611cf 100644 --- a/src/hydra-eval-jobs/hydra-eval-jobs.cc +++ b/src/hydra-eval-jobs/hydra-eval-jobs.cc @@ -174,7 +174,7 @@ static void worker( if (auto drv = getDerivation(state, *v, false)) { - DrvInfo::Outputs outputs = drv->queryOutputs(); + DrvInfo::Outputs outputs = drv->queryOutputs(!settings.isExperimentalFeatureEnabled(Xp::CaDerivations)); if (drv->querySystem() == "unknown") throw EvalError("derivation must have a 'system' attribute"); @@ -231,12 +231,13 @@ static void worker( } nlohmann::json out; - for (auto & j : outputs) - // FIXME: handle CA/impure builds. - if (j.second) - out[j.first] = state.store->printStorePath(*j.second); + if (settings.isExperimentalFeatureEnabled(Xp::CaDerivations)) + for (auto & j : outputs) + out[j.first] = ""; + else + for (auto & j : outputs) + out[j.first] = state.store->printStorePath(*j.second); job["outputs"] = std::move(out); - reply["job"] = std::move(job); } diff --git a/src/hydra-queue-runner/build-remote.cc b/src/hydra-queue-runner/build-remote.cc index 05380681..c74f3d4c 100644 --- a/src/hydra-queue-runner/build-remote.cc +++ b/src/hydra-queue-runner/build-remote.cc @@ -174,6 +174,71 @@ StorePaths reverseTopoSortPaths(const std::map & paths return sorted; } +/** + * Replace the input derivations by their output paths to send a minimal closure + * to the builder. + * + * If we can afford it, resolve it, so that the newly generated derivation still + * has some sensible output paths. + */ +BasicDerivation inlineInputDerivations(Store & store, Derivation & drv, const StorePath & drvPath) +{ + BasicDerivation ret; + auto outputHashes = staticOutputHashes(store, drv); + if (!drv.type().hasKnownOutputPaths()) { + auto maybeBasicDrv = drv.tryResolve(store); + if (!maybeBasicDrv) + throw Error( + "the derivation '%s' can’t be resolved. It’s probably " + "missing some outputs", + store.printStorePath(drvPath)); + ret = *maybeBasicDrv; + } else { + // If the derivation is a real `InputAddressed` derivation, we must + // resolve it manually to keep the original output paths + ret = BasicDerivation(drv); + for (auto & input : drv.inputDrvs) { + auto drv2 = store.readDerivation(input.first); + auto drv2Outputs = drv2.outputsAndOptPaths(store); + for (auto & name : input.second) { + auto inputPath = drv2Outputs.at(name); + ret.inputSrcs.insert(*inputPath.second); + } + } + } + return ret; +} + +/** + * Get the newly built outputs, either from the remote if it supports it, or by + * introspecting the derivation if the remote is too old + */ +DrvOutputs getBuiltOutputs(Store & store, const int remoteVersion, FdSource & from, Derivation & drv) +{ + DrvOutputs builtOutputs; + if (GET_PROTOCOL_MINOR(remoteVersion) >= 6) { + builtOutputs + = worker_proto::read(store, from, Phantom {}); + } else { + // If the remote is too old to handle CA derivations, we can’t get this + // far anyways + assert(drv.type().hasKnownOutputPaths()); + DerivationOutputsAndOptPaths drvOutputs + = drv.outputsAndOptPaths(store); + auto outputHashes = staticOutputHashes(store, drv); + for (auto & [outputName, output] : drvOutputs) { + auto outputPath = output.second; + // We’ve just asserted that the output paths of the derivation + // were known + assert(outputPath); + auto outputHash = outputHashes.at(outputName); + auto drvOutput = DrvOutput { outputHash, outputName }; + builtOutputs.insert( + { drvOutput, Realisation { drvOutput, *outputPath } }); + } + } + return builtOutputs; +} void State::buildRemote(ref destStore, Machine::ptr machine, Step::ptr step, @@ -264,22 +329,7 @@ void State::buildRemote(ref destStore, outputs of the input derivations. */ updateStep(ssSendingInputs); - StorePathSet inputs; - BasicDerivation basicDrv(*step->drv); - - for (auto & p : step->drv->inputSrcs) - inputs.insert(p); - - for (auto & input : step->drv->inputDrvs) { - auto drv2 = localStore->readDerivation(input.first); - for (auto & name : input.second) { - if (auto i = get(drv2.outputs, name)) { - auto outPath = i->path(*localStore, drv2.name, name); - inputs.insert(*outPath); - basicDrv.inputSrcs.insert(*outPath); - } - } - } + BasicDerivation basicDrv = inlineInputDerivations(*localStore, *step->drv, step->drvPath); /* Ensure that the inputs exist in the destination store. This is a no-op for regular stores, but for the binary cache store, @@ -304,10 +354,11 @@ void State::buildRemote(ref destStore, /* Copy the input closure. */ if (machine->isLocalhost()) { StorePathSet closure; - destStore->computeFSClosure(inputs, closure); + destStore->computeFSClosure(basicDrv.inputSrcs, closure); copyPaths(*destStore, *localStore, closure, NoRepair, NoCheckSigs, NoSubstitute); } else { - copyClosureTo(machine->state->sendLock, *destStore, from, to, inputs, true); + copyClosureTo(machine->state->sendLock, *destStore, from, to, step->drv->inputSrcs, true); + copyClosureTo(machine->state->sendLock, *destStore, from, to, basicDrv.inputSrcs, true); } auto now2 = std::chrono::steady_clock::now(); @@ -366,9 +417,6 @@ void State::buildRemote(ref destStore, result.stopTime = stop; } } - if (GET_PROTOCOL_MINOR(remoteVersion) >= 6) { - worker_proto::read(*localStore, from, Phantom {}); - } switch ((BuildResult::Status) res) { case BuildResult::Built: result.stepStatus = bsSuccess; @@ -426,6 +474,11 @@ void State::buildRemote(ref destStore, result.logFile = ""; } + auto builtOutputs = getBuiltOutputs(*localStore, remoteVersion, from, *step->drv); + StorePathSet outputs; + for (auto & [_, realisation] : builtOutputs) + outputs.insert(realisation.outPath); + /* Copy the output paths. */ if (!machine->isLocalhost() || localStore != std::shared_ptr(destStore)) { updateStep(ssReceivingOutputs); @@ -434,12 +487,6 @@ void State::buildRemote(ref destStore, auto now1 = std::chrono::steady_clock::now(); - StorePathSet outputs; - for (auto & i : step->drv->outputsAndOptPaths(*localStore)) { - if (i.second.second) - outputs.insert(*i.second.second); - } - /* Get info about each output path. */ std::map infos; size_t totalNarSize = 0; @@ -477,26 +524,27 @@ void State::buildRemote(ref destStore, for (auto & path : pathsSorted) { auto & info = infos.find(path)->second; - /* Receive the NAR from the remote and add it to the - destination store. Meanwhile, extract all the info from the - NAR that getBuildOutput() needs. */ - auto source2 = sinkToSource([&](Sink & sink) - { - /* Note: we should only send the command to dump the store - path to the remote if the NAR is actually going to get read - by the destination store, which won't happen if this path - is already valid on the destination store. Since this - lambda function only gets executed if someone tries to read - from source2, we will send the command from here rather - than outside the lambda. */ - to << cmdDumpStorePath << localStore->printStorePath(path); - to.flush(); - - TeeSource tee(from, sink); - extractNarData(tee, localStore->printStorePath(path), narMembers); - }); - - destStore->addToStore(info, *source2, NoRepair, NoCheckSigs); + for (auto & store : {&*destStore, &*localStore}) { + /* Receive the NAR from the remote and add it to the + destination store. Meanwhile, extract all the info from the + NAR that getBuildOutput() needs. */ + auto source2 = sinkToSource([&](Sink & sink) + { + /* Note: we should only send the command to dump the store + path to the remote if the NAR is actually going to get read + by the destination store, which won't happen if this path + is already valid on the destination store. Since this + lambda function only gets executed if someone tries to read + from source2, we will send the command from here rather + than outside the lambda. */ + to << cmdDumpStorePath << localStore->printStorePath(path); + to.flush(); + + TeeSource tee(from, sink); + extractNarData(tee, localStore->printStorePath(path), narMembers); + }); + store->addToStore(info, *source2, NoRepair, NoCheckSigs); + } } auto now2 = std::chrono::steady_clock::now(); @@ -504,6 +552,23 @@ void State::buildRemote(ref destStore, result.overhead += std::chrono::duration_cast(now2 - now1).count(); } + /* Register the outputs of the newly built drv */ + if (settings.isExperimentalFeatureEnabled(Xp::CaDerivations)) { + auto outputHashes = staticOutputHashes(*localStore, *step->drv); + for (auto & [outputId, realisation] : builtOutputs) { + // Register the resolved drv output + localStore->registerDrvOutput(realisation); + destStore->registerDrvOutput(realisation); + + // Also register the unresolved one + auto unresolvedRealisation = realisation; + unresolvedRealisation.signatures.clear(); + unresolvedRealisation.id.drvHash = outputHashes.at(outputId.outputName); + localStore->registerDrvOutput(unresolvedRealisation); + destStore->registerDrvOutput(unresolvedRealisation); + } + } + /* Shut down the connection. */ child.to = -1; child.pid.wait(); diff --git a/src/hydra-queue-runner/build-result.cc b/src/hydra-queue-runner/build-result.cc index ea8b4a6a..6dfa280b 100644 --- a/src/hydra-queue-runner/build-result.cc +++ b/src/hydra-queue-runner/build-result.cc @@ -11,18 +11,18 @@ using namespace nix; BuildOutput getBuildOutput( nix::ref store, NarMemberDatas & narMembers, - const Derivation & drv) + const OutputPathMap derivationOutputs) { BuildOutput res; /* Compute the closure size. */ StorePathSet outputs; StorePathSet closure; - for (auto & i : drv.outputsAndOptPaths(*store)) - if (i.second.second) { - store->computeFSClosure(*i.second.second, closure); - outputs.insert(*i.second.second); - } + for (auto& [outputName, outputPath] : derivationOutputs) { + store->computeFSClosure(outputPath, closure); + outputs.insert(outputPath); + res.outputs.insert({outputName, outputPath}); + } for (auto & path : closure) { auto info = store->queryPathInfo(path); res.closureSize += info->narSize; @@ -107,13 +107,12 @@ BuildOutput getBuildOutput( /* If no build products were explicitly declared, then add all outputs as a product of type "nix-build". */ if (!explicitProducts) { - for (auto & [name, output] : drv.outputs) { + for (auto& [name, output] : derivationOutputs) { BuildProduct product; - auto outPath = output.path(*store, drv.name, name); - product.path = store->printStorePath(*outPath); + product.path = store->printStorePath(output); product.type = "nix-build"; product.subtype = name == "out" ? "" : name; - product.name = outPath->name(); + product.name = output.name(); auto file = narMembers.find(product.path); assert(file != narMembers.end()); diff --git a/src/hydra-queue-runner/builder.cc b/src/hydra-queue-runner/builder.cc index 37022522..c365fa79 100644 --- a/src/hydra-queue-runner/builder.cc +++ b/src/hydra-queue-runner/builder.cc @@ -221,7 +221,7 @@ State::StepResult State::doBuildStep(nix::ref destStore, if (result.stepStatus == bsSuccess) { updateStep(ssPostProcessing); - res = getBuildOutput(destStore, narMembers, *step->drv); + res = getBuildOutput(destStore, narMembers, localStore->queryDerivationOutputMap(step->drvPath)); } } @@ -275,9 +275,9 @@ State::StepResult State::doBuildStep(nix::ref destStore, assert(stepNr); - for (auto & i : step->drv->outputsAndOptPaths(*localStore)) { - if (i.second.second) - addRoot(*i.second.second); + for (auto & i : localStore->queryPartialDerivationOutputMap(step->drvPath)) { + if (i.second) + addRoot(*i.second); } /* Register success in the database for all Build objects that diff --git a/src/hydra-queue-runner/hydra-build-result.hh b/src/hydra-queue-runner/hydra-build-result.hh index a3f71ae9..7d47f67c 100644 --- a/src/hydra-queue-runner/hydra-build-result.hh +++ b/src/hydra-queue-runner/hydra-build-result.hh @@ -36,10 +36,12 @@ struct BuildOutput std::list products; + std::map outputs; + std::map metrics; }; BuildOutput getBuildOutput( nix::ref store, NarMemberDatas & narMembers, - const nix::Derivation & drv); + const nix::OutputPathMap derivationOutputs); diff --git a/src/hydra-queue-runner/hydra-queue-runner.cc b/src/hydra-queue-runner/hydra-queue-runner.cc index b84681d5..b3098c3d 100644 --- a/src/hydra-queue-runner/hydra-queue-runner.cc +++ b/src/hydra-queue-runner/hydra-queue-runner.cc @@ -311,10 +311,10 @@ unsigned int State::createBuildStep(pqxx::work & txn, time_t startTime, BuildID if (r.affected_rows() == 0) goto restart; - for (auto & [name, output] : step->drv->outputs) - txn.exec_params0 - ("insert into BuildStepOutputs (build, stepnr, name, path) values ($1, $2, $3, $4)", - buildId, stepNr, name, localStore->printStorePath(*output.path(*localStore, step->drv->name, name))); + for (auto& [name, output] : localStore->queryPartialDerivationOutputMap(step->drvPath)) + txn.exec_params0 + ("insert into BuildStepOutputs (build, stepnr, name, path, contentAddressed) values ($1, $2, $3, $4, $5)", + buildId, stepNr, name, output ? localStore->printStorePath(*output) : "", step->drv->type().isCA()); if (status == bsBusy) txn.exec(fmt("notify step_started, '%d\t%d'", buildId, stepNr)); @@ -351,11 +351,23 @@ void State::finishBuildStep(pqxx::work & txn, const RemoteResult & result, assert(result.logFile.find('\t') == std::string::npos); txn.exec(fmt("notify step_finished, '%d\t%d\t%s'", buildId, stepNr, result.logFile)); + + if (result.stepStatus == bsSuccess) { + // Update the corresponding `BuildStepOutputs` row to add the output path + auto res = txn.exec_params1("select drvPath from BuildSteps where build = $1 and stepnr = $2", buildId, stepNr); + assert(res.size()); + StorePath drvPath = localStore->parseStorePath(res[0].as()); + // If we've finished building, all the paths should be known + for (auto& [name, output] : localStore->queryDerivationOutputMap(drvPath)) + txn.exec_params0 + ("update BuildStepOutputs set path = $4 where build = $1 and stepnr = $2 and name = $3", + buildId, stepNr, name, localStore->printStorePath(output)); + } } int State::createSubstitutionStep(pqxx::work & txn, time_t startTime, time_t stopTime, - Build::ptr build, const StorePath & drvPath, const std::string & outputName, const StorePath & storePath) + Build::ptr build, const StorePath & drvPath, const nix::Derivation drv, const std::string & outputName, const StorePath & storePath) { restart: auto stepNr = allocBuildStep(txn, build->id); @@ -374,9 +386,10 @@ int State::createSubstitutionStep(pqxx::work & txn, time_t startTime, time_t sto if (r.affected_rows() == 0) goto restart; txn.exec_params0 - ("insert into BuildStepOutputs (build, stepnr, name, path) values ($1, $2, $3, $4)", + ("insert into BuildStepOutputs (build, stepnr, name, path, contentAddressed) values ($1, $2, $3, $4, $5)", build->id, stepNr, outputName, - localStore->printStorePath(storePath)); + localStore->printStorePath(storePath), + drv.type().isCA()); return stepNr; } @@ -456,6 +469,15 @@ void State::markSucceededBuild(pqxx::work & txn, Build::ptr build, res.releaseName != "" ? std::make_optional(res.releaseName) : std::nullopt, isCachedBuild ? 1 : 0); + for (auto & [outputName, outputPath] : res.outputs) { + txn.exec_params0 + ("update BuildOutputs set path = $3 where build = $1 and name = $2", + build->id, + outputName, + localStore->printStorePath(outputPath) + ); + } + txn.exec_params0("delete from BuildProducts where build = $1", build->id); unsigned int productNr = 1; diff --git a/src/hydra-queue-runner/queue-monitor.cc b/src/hydra-queue-runner/queue-monitor.cc index 12d55b79..2c538b67 100644 --- a/src/hydra-queue-runner/queue-monitor.cc +++ b/src/hydra-queue-runner/queue-monitor.cc @@ -192,15 +192,14 @@ bool State::getQueuedBuilds(Connection & conn, if (!res[0].is_null()) propagatedFrom = res[0].as(); if (!propagatedFrom) { - for (auto & i : ex.step->drv->outputsAndOptPaths(*localStore)) { - if (i.second.second) { - auto res = txn.exec_params - ("select max(s.build) from BuildSteps s join BuildStepOutputs o on s.build = o.build where path = $1 and startTime != 0 and stopTime != 0 and status = 1", - localStore->printStorePath(*i.second.second)); - if (!res[0][0].is_null()) { - propagatedFrom = res[0][0].as(); - break; - } + for (auto & i : localStore->queryPartialDerivationOutputMap(ex.step->drvPath)) { + auto res = txn.exec_params + ("select max(s.build) from BuildSteps s join BuildStepOutputs o on s.build = o.build where drvPath = $1 and name = $2 and startTime != 0 and stopTime != 0 and status = 1", + localStore->printStorePath(ex.step->drvPath), + i.first); + if (!res[0][0].is_null()) { + propagatedFrom = res[0][0].as(); + break; } } } @@ -236,12 +235,10 @@ bool State::getQueuedBuilds(Connection & conn, /* If we didn't get a step, it means the step's outputs are all valid. So we mark this as a finished, cached build. */ if (!step) { - auto drv = localStore->readDerivation(build->drvPath); - BuildOutput res = getBuildOutputCached(conn, destStore, drv); + BuildOutput res = getBuildOutputCached(conn, destStore, build->drvPath); - for (auto & i : drv.outputsAndOptPaths(*localStore)) - if (i.second.second) - addRoot(*i.second.second); + for (auto & i : localStore->queryDerivationOutputMap(build->drvPath)) + addRoot(i.second); { auto mc = startDbUpdate(); @@ -481,26 +478,40 @@ Step::ptr State::createStep(ref destStore, throw PreviousFailure{step}; /* Are all outputs valid? */ + auto outputHashes = staticOutputHashes(*localStore, *(step->drv)); bool valid = true; - DerivationOutputs missing; - for (auto & i : step->drv->outputs) - if (!destStore->isValidPath(*i.second.path(*localStore, step->drv->name, i.first))) { - valid = false; - missing.insert_or_assign(i.first, i.second); + std::map> missing; + for (auto &[outputName, maybeOutputPath] : + step->drv->outputsAndOptPaths(*destStore)) { + auto outputHash = outputHashes.at(outputName); + if (maybeOutputPath.second) { + if (!destStore->isValidPath(*maybeOutputPath.second)) { + valid = false; + missing.insert({{outputHash, outputName}, maybeOutputPath.second}); } + } else { + settings.requireExperimentalFeature(Xp::CaDerivations); + if (!destStore->queryRealisation(DrvOutput{outputHash, outputName})) { + valid = false; + missing.insert({{outputHash, outputName}, std::nullopt}); + } + } + } /* Try to copy the missing paths from the local store or from substitutes. */ if (!missing.empty()) { size_t avail = 0; - for (auto & i : missing) { - auto path = i.second.path(*localStore, step->drv->name, i.first); - if (/* localStore != destStore && */ localStore->isValidPath(*path)) + for (auto & [i, maybePath] : missing) { + if ((maybePath && localStore->isValidPath(*maybePath))) avail++; - else if (useSubstitutes) { + else if (settings.isExperimentalFeatureEnabled(Xp::CaDerivations) && localStore->queryRealisation(i)) { + maybePath = localStore->queryRealisation(i)->outPath; + avail++; + } else if (useSubstitutes && maybePath) { SubstitutablePathInfos infos; - localStore->querySubstitutablePathInfos({{*path, {}}}, infos); + localStore->querySubstitutablePathInfos({{*maybePath, {}}}, infos); if (infos.size() == 1) avail++; } @@ -508,44 +519,44 @@ Step::ptr State::createStep(ref destStore, if (missing.size() == avail) { valid = true; - for (auto & i : missing) { - auto path = i.second.path(*localStore, step->drv->name, i.first); + for (auto & [i, path] : missing) { + if (path) { + try { + time_t startTime = time(0); + + if (localStore->isValidPath(*path)) + printInfo("copying output ‘%1%’ of ‘%2%’ from local store", + localStore->printStorePath(*path), + localStore->printStorePath(drvPath)); + else { + printInfo("substituting output ‘%1%’ of ‘%2%’", + localStore->printStorePath(*path), + localStore->printStorePath(drvPath)); + localStore->ensurePath(*path); + // FIXME: should copy directly from substituter to destStore. + } - try { - time_t startTime = time(0); + StorePathSet closure; + localStore->computeFSClosure({*path}, closure); + copyPaths(*localStore, *destStore, closure, NoRepair, CheckSigs, NoSubstitute); - if (localStore->isValidPath(*path)) - printInfo("copying output ‘%1%’ of ‘%2%’ from local store", - localStore->printStorePath(*path), - localStore->printStorePath(drvPath)); - else { - printInfo("substituting output ‘%1%’ of ‘%2%’", - localStore->printStorePath(*path), - localStore->printStorePath(drvPath)); - localStore->ensurePath(*path); - // FIXME: should copy directly from substituter to destStore. - } + time_t stopTime = time(0); - copyClosure(*localStore, *destStore, - StorePathSet { *path }, - NoRepair, CheckSigs, NoSubstitute); - - time_t stopTime = time(0); + { + auto mc = startDbUpdate(); + pqxx::work txn(conn); + createSubstitutionStep(txn, startTime, stopTime, build, drvPath, *(step->drv), "out", *path); + txn.commit(); + } - { - auto mc = startDbUpdate(); - pqxx::work txn(conn); - createSubstitutionStep(txn, startTime, stopTime, build, drvPath, "out", *path); - txn.commit(); + } catch (Error & e) { + printError("while copying/substituting output ‘%s’ of ‘%s’: %s", + localStore->printStorePath(*path), + localStore->printStorePath(drvPath), + e.what()); + valid = false; + break; } - - } catch (Error & e) { - printError("while copying/substituting output ‘%s’ of ‘%s’: %s", - localStore->printStorePath(*path), - localStore->printStorePath(drvPath), - e.what()); - valid = false; - break; } } } @@ -640,17 +651,20 @@ void State::processJobsetSharesChange(Connection & conn) } -BuildOutput State::getBuildOutputCached(Connection & conn, nix::ref destStore, const nix::Derivation & drv) +BuildOutput State::getBuildOutputCached(Connection & conn, nix::ref destStore, const nix::StorePath & drvPath) { + + auto derivationOutputs = localStore->queryDerivationOutputMap(drvPath); + { pqxx::work txn(conn); - for (auto & [name, output] : drv.outputsAndOptPaths(*localStore)) { + for (auto & [name, output] : derivationOutputs) { auto r = txn.exec_params ("select id, buildStatus, releaseName, closureSize, size from Builds b " "join BuildOutputs o on b.id = o.build " "where finished = 1 and (buildStatus = 0 or buildStatus = 6) and path = $1", - localStore->printStorePath(*output.second)); + localStore->printStorePath(output)); if (r.empty()) continue; BuildID id = r[0][0].as(); @@ -704,5 +718,5 @@ BuildOutput State::getBuildOutputCached(Connection & conn, nix::ref } NarMemberDatas narMembers; - return getBuildOutput(destStore, narMembers, drv); + return getBuildOutput(destStore, narMembers, derivationOutputs); } diff --git a/src/hydra-queue-runner/state.hh b/src/hydra-queue-runner/state.hh index 55c99afc..1073926e 100644 --- a/src/hydra-queue-runner/state.hh +++ b/src/hydra-queue-runner/state.hh @@ -485,7 +485,7 @@ private: const std::string & machine); int createSubstitutionStep(pqxx::work & txn, time_t startTime, time_t stopTime, - Build::ptr build, const nix::StorePath & drvPath, const std::string & outputName, const nix::StorePath & storePath); + Build::ptr build, const nix::StorePath & drvPath, const nix::Derivation drv, const std::string & outputName, const nix::StorePath & storePath); void updateBuild(pqxx::work & txn, Build::ptr build, BuildStatus status); @@ -501,7 +501,7 @@ private: void processQueueChange(Connection & conn); BuildOutput getBuildOutputCached(Connection & conn, nix::ref destStore, - const nix::Derivation & drv); + const nix::StorePath & drvPath); Step::ptr createStep(nix::ref store, Connection & conn, Build::ptr build, const nix::StorePath & drvPath, diff --git a/src/lib/Hydra/Controller/Build.pm b/src/lib/Hydra/Controller/Build.pm index 18a0eba3..a4b43364 100644 --- a/src/lib/Hydra/Controller/Build.pm +++ b/src/lib/Hydra/Controller/Build.pm @@ -78,9 +78,11 @@ sub build_GET { $c->stash->{template} = 'build.tt'; $c->stash->{isLocalStore} = isLocalStore(); + # XXX: If the derivation is content-addressed then this will always return + # false because `$_->path` will be empty $c->stash->{available} = $c->stash->{isLocalStore} - ? all { isValidPath($_->path) } $build->buildoutputs->all + ? all { $_->path && isValidPath($_->path) } $build->buildoutputs->all : 1; $c->stash->{drvAvailable} = isValidPath $build->drvpath; @@ -113,6 +115,18 @@ sub build_GET { $c->stash->{steps} = [$build->buildsteps->search({}, {order_by => "stepnr desc"})]; + $c->stash->{contentAddressed} = 0; + # Hydra marks single outputs as CA but currently in Nix only derivations + # can be CA (and *all* their outputs are CA). + # So the next check (which assumes that if a step's output is CA then + # all the other outptus and the whole derivation are CA) is safe. + foreach my $step (@{$c->stash->{steps}}) { + if ($step->buildstepoutputs->search({contentaddressed => 1})->count > 0) { + $c->stash->{contentAddressed} = 1; + last; + } + } + $c->stash->{binaryCachePublicUri} = $c->config->{binary_cache_public_uri}; } diff --git a/src/lib/Hydra/Schema/Result/BuildOutputs.pm b/src/lib/Hydra/Schema/Result/BuildOutputs.pm index 9fc4f7c7..3997b497 100644 --- a/src/lib/Hydra/Schema/Result/BuildOutputs.pm +++ b/src/lib/Hydra/Schema/Result/BuildOutputs.pm @@ -49,7 +49,7 @@ __PACKAGE__->table("buildoutputs"); =head2 path data_type: 'text' - is_nullable: 0 + is_nullable: 1 =cut @@ -59,7 +59,7 @@ __PACKAGE__->add_columns( "name", { data_type => "text", is_nullable => 0 }, "path", - { data_type => "text", is_nullable => 0 }, + { data_type => "text", is_nullable => 1 }, ); =head1 PRIMARY KEY @@ -94,8 +94,8 @@ __PACKAGE__->belongs_to( ); -# Created by DBIx::Class::Schema::Loader v0.07049 @ 2021-08-26 12:02:36 -# DO NOT MODIFY THIS OR ANYTHING ABOVE! md5sum:gU+kZ6A0ISKpaXGRGve8mg +# Created by DBIx::Class::Schema::Loader v0.07049 @ 2022-06-30 12:02:32 +# DO NOT MODIFY THIS OR ANYTHING ABOVE! md5sum:Jsabm3YTcI7YvCuNdKP5Ng my %hint = ( columns => [ diff --git a/src/lib/Hydra/Schema/Result/BuildStepOutputs.pm b/src/lib/Hydra/Schema/Result/BuildStepOutputs.pm index 016a35fe..42392190 100644 --- a/src/lib/Hydra/Schema/Result/BuildStepOutputs.pm +++ b/src/lib/Hydra/Schema/Result/BuildStepOutputs.pm @@ -55,6 +55,11 @@ __PACKAGE__->table("buildstepoutputs"); =head2 path data_type: 'text' + is_nullable: 1 + +=head2 contentaddressed + + data_type: 'boolean' is_nullable: 0 =cut @@ -67,7 +72,9 @@ __PACKAGE__->add_columns( "name", { data_type => "text", is_nullable => 0 }, "path", - { data_type => "text", is_nullable => 0 }, + { data_type => "text", is_nullable => 1 }, + "contentaddressed", + { data_type => "boolean", is_nullable => 0 }, ); =head1 PRIMARY KEY @@ -119,8 +126,8 @@ __PACKAGE__->belongs_to( ); -# Created by DBIx::Class::Schema::Loader v0.07049 @ 2021-08-26 12:02:36 -# DO NOT MODIFY THIS OR ANYTHING ABOVE! md5sum:gxp8rOjpRVen4YbIjomHTw +# Created by DBIx::Class::Schema::Loader v0.07049 @ 2022-06-30 12:02:32 +# DO NOT MODIFY THIS OR ANYTHING ABOVE! md5sum:Bad70CRTt7zb2GGuRoQ++Q # You can replace this text with custom code or comments, and it will be preserved on regeneration diff --git a/src/root/build.tt b/src/root/build.tt index 93a02e0f..79b0a1e6 100644 --- a/src/root/build.tt +++ b/src/root/build.tt @@ -20,8 +20,13 @@ END; %] [% BLOCK renderOutputs %] - [% start=1; FOREACH output IN outputs %] - [% IF !start %],
[% END; start=0; output.path %] + [% start=1; FOREACH output IN step.buildstepoutputs %] + [% IF !start %],
[% END; start=0; %] + [% IF step.status != 0 && output.contentaddressed %] + [% output.name %] + [% ELSE %] + [% output.path %] + [% END %] [% END %] [% END %] @@ -40,9 +45,9 @@ END; [% step.stepnr %] [% IF step.type == 0 %] - Build of [% INCLUDE renderOutputs outputs=step.buildstepoutputs %] + Build of [% INCLUDE renderOutputs step=step %] [% ELSE %] - Substitution of [% INCLUDE renderOutputs outputs=step.buildstepoutputs %] + Substitution of [% INCLUDE renderOutputs step=step %] [% END %] @@ -382,9 +387,21 @@ END; [% build.drvpath %] - Output store paths: - [% INCLUDE renderOutputs outputs=build.buildoutputs %] + Content addressed: + + [% IF contentAddressed %] + Yes + [% ELSE %] + No + [% END %] + + [% IF !contentAddressed || step.status == 0 %] + + Output store paths: + [% INCLUDE renderOutputs step=step %] + + [% END %] [% chartsURL = c.uri_for('/job' build.project.name build.jobset.name build.job) _ "#tabs-charts" %] [% IF build.finished && build.closuresize %] diff --git a/src/script/hydra-eval-jobset b/src/script/hydra-eval-jobset index c6f6c275..3e91111a 100755 --- a/src/script/hydra-eval-jobset +++ b/src/script/hydra-eval-jobset @@ -444,7 +444,7 @@ sub checkBuild { # the eval), but they give a factor 1000 speedup on # the Nixpkgs jobset with PostgreSQL. { jobset_id => $jobset->get_column('id'), job => $jobName, - name => $firstOutputName, path => $firstOutputPath }, + name => $firstOutputName, drvPath => $drvPath }, { rows => 1, columns => ['id', 'finished'], join => ['buildoutputs'] }); if (defined $prevBuild) { #print STDERR " already scheduled/built as build ", $prevBuild->id, "\n"; diff --git a/src/sql/hydra.sql b/src/sql/hydra.sql index eaae6da3..02159fe8 100644 --- a/src/sql/hydra.sql +++ b/src/sql/hydra.sql @@ -247,7 +247,7 @@ create trigger BuildBumped after update on Builds for each row create table BuildOutputs ( build integer not null, name text not null, - path text not null, + path text, primary key (build, name), foreign key (build) references Builds(id) on delete cascade ); @@ -300,13 +300,14 @@ create table BuildSteps ( create table BuildStepOutputs ( - build integer not null, - stepnr integer not null, - name text not null, - path text not null, - primary key (build, stepnr, name), - foreign key (build) references Builds(id) on delete cascade, - foreign key (build, stepnr) references BuildSteps(build, stepnr) on delete cascade + build integer not null, + stepnr integer not null, + name text not null, + path text, + contentAddressed boolean not null, + primary key (build, stepnr, name), + foreign key (build) references Builds(id) on delete cascade, + foreign key (build, stepnr) references BuildSteps(build, stepnr) on delete cascade ); diff --git a/t/content-addressed/basic.t b/t/content-addressed/basic.t new file mode 100644 index 00000000..f19e5d6d --- /dev/null +++ b/t/content-addressed/basic.t @@ -0,0 +1,61 @@ +use feature 'unicode_strings'; +use strict; +use warnings; +use Setup; + +my %ctx = test_init( + nix_config => qq| + experimental-features = ca-derivations + |, +); + +require Hydra::Schema; +require Hydra::Model::DB; + +use JSON::MaybeXS; + +use HTTP::Request::Common; +use Test2::V0; +require Catalyst::Test; +Catalyst::Test->import('Hydra'); + +my $db = Hydra::Model::DB->new; +hydra_setup($db); + +my $project = $db->resultset('Projects')->create({name => "tests", displayname => "", owner => "root"}); + +my $jobset = createBaseJobset("content-addressed", "content-addressed.nix", $ctx{jobsdir}); + +ok(evalSucceeds($jobset), "Evaluating jobs/content-addressed.nix should exit with return code 0"); +is(nrQueuedBuildsForJobset($jobset), 4, "Evaluating jobs/content-addressed.nix should result in 4 builds"); + +for my $build (queuedBuildsForJobset($jobset)) { + ok(runBuild($build), "Build '".$build->job."' from jobs/content-addressed.nix should exit with code 0"); + my $newbuild = $db->resultset('Builds')->find($build->id); + is($newbuild->finished, 1, "Build '".$build->job."' from jobs/content-addressed.nix should be finished."); + my $expected = $build->job eq "fails" ? 1 : $build->job =~ /with_failed/ ? 6 : 0; + is($newbuild->buildstatus, $expected, "Build '".$build->job."' from jobs/content-addressed.nix should have buildstatus $expected."); + + my $response = request("/build/".$build->id); + ok($response->is_success, "The 'build' page for build '".$build->job."' should load properly"); + + if ($newbuild->buildstatus == 0) { + my $buildOutputs = $newbuild->buildoutputs; + for my $output ($newbuild->buildoutputs) { + # XXX: This hardcodes /nix/store/. + # It's fine because in practice the nix store for the tests will be of + # the form `/some/thing/nix/store/`, but it would be cleaner if there + # was a way to query Nix for its store dir? + like( + $output->path, qr|/nix/store/|, + "Output '".$output->name."' of build '".$build->job."' should be a valid store path" + ); + } + } + +} + +isnt(<$ctx{deststoredir}/realisations/*>, "", "The destination store should have the realisations of the built derivations registered"); + +done_testing; + diff --git a/t/content-addressed/without-experimental-feature.t b/t/content-addressed/without-experimental-feature.t new file mode 100644 index 00000000..a37d138e --- /dev/null +++ b/t/content-addressed/without-experimental-feature.t @@ -0,0 +1,28 @@ +use feature 'unicode_strings'; +use strict; +use warnings; +use Setup; + +my %ctx = test_init(); + +require Hydra::Schema; +require Hydra::Model::DB; + +use JSON::MaybeXS; + +use HTTP::Request::Common; +use Test2::V0; +require Catalyst::Test; +Catalyst::Test->import('Hydra'); + +my $db = Hydra::Model::DB->new; +hydra_setup($db); + +my $project = $db->resultset('Projects')->create({name => "tests", displayname => "", owner => "root"}); + +my $jobset = createBaseJobset("content-addressed", "content-addressed.nix", $ctx{jobsdir}); + +ok(evalSucceeds($jobset), "Evaluating jobs/content-addressed.nix without the experimental feature should exit with return code 0"); +is(nrQueuedBuildsForJobset($jobset), 0, "Evaluating jobs/content-addressed.nix without the experimental Nix feature should result in 0 build"); + +done_testing; diff --git a/t/jobs/config.nix.in b/t/jobs/config.nix.in index 51b6c06f..41776341 100644 --- a/t/jobs/config.nix.in +++ b/t/jobs/config.nix.in @@ -6,4 +6,9 @@ rec { system = builtins.currentSystem; PATH = path; } // args); + mkContentAddressedDerivation = args: mkDerivation ({ + __contentAddressed = true; + outputHashMode = "recursive"; + outputHashAlgo = "sha256"; + } // args); } diff --git a/t/jobs/content-addressed.nix b/t/jobs/content-addressed.nix new file mode 100644 index 00000000..785e917c --- /dev/null +++ b/t/jobs/content-addressed.nix @@ -0,0 +1,28 @@ +let cfg = import ./config.nix; in +rec { + empty_dir = + cfg.mkContentAddressedDerivation { + name = "empty-dir"; + builder = ./empty-dir-builder.sh; + }; + + fails = + cfg.mkContentAddressedDerivation { + name = "fails"; + builder = ./fail.sh; + }; + + succeed_with_failed = + cfg.mkContentAddressedDerivation { + name = "succeed-with-failed"; + builder = ./succeed-with-failed.sh; + }; + + nonCaDependingOnCA = + cfg.mkDerivation { + name = "non-ca-depending-on-ca"; + builder = ./empty-dir-builder.sh; + FOO = empty_dir; + }; +} + diff --git a/t/lib/HydraTestContext.pm b/t/lib/HydraTestContext.pm index 53eaa0f7..7f254b49 100644 --- a/t/lib/HydraTestContext.pm +++ b/t/lib/HydraTestContext.pm @@ -39,6 +39,8 @@ use Hydra::Helper::Exec; sub new { my ($class, %opts) = @_; + my $deststoredir; + my $dir = File::Temp->newdir(); $ENV{'HYDRA_DATA'} = "$dir/hydra-data"; @@ -79,8 +81,9 @@ sub new { nix_state_dir => $nix_state_dir, nix_log_dir => $nix_log_dir, testdir => abs_path(dirname(__FILE__) . "/.."), - jobsdir => abs_path(dirname(__FILE__) . "/../jobs") - }, $class; + jobsdir => abs_path(dirname(__FILE__) . "/../jobs"), + deststoredir => $deststoredir, + }; if ($opts{'before_init'}) { $opts{'before_init'}->($self); diff --git a/t/queue-runner/notifications.t b/t/queue-runner/notifications.t index 1966cde1..d0e72409 100644 --- a/t/queue-runner/notifications.t +++ b/t/queue-runner/notifications.t @@ -8,7 +8,7 @@ my $binarycachedir = File::Temp->newdir(); my $ctx = test_context( nix_config => qq| - experimental-features = nix-command + experimental-features = nix-command ca-derivations substituters = file://${binarycachedir}?trusted=1 |, hydra_config => q|