aboutsummaryrefslogtreecommitdiff
path: root/src/libstore
diff options
context:
space:
mode:
Diffstat (limited to 'src/libstore')
-rw-r--r--src/libstore/build/derivation-goal.cc1
-rw-r--r--src/libstore/build/local-derivation-goal.cc5
-rw-r--r--src/libstore/ca-specific-schema.sql11
-rw-r--r--src/libstore/daemon.cc25
-rw-r--r--src/libstore/gc.cc4
-rw-r--r--src/libstore/local-fs-store.hh5
-rw-r--r--src/libstore/local-store.cc110
-rw-r--r--src/libstore/local-store.hh5
-rw-r--r--src/libstore/misc.cc39
-rw-r--r--src/libstore/optimise-store.cc2
-rw-r--r--src/libstore/parsed-derivations.cc2
-rw-r--r--src/libstore/realisation.cc48
-rw-r--r--src/libstore/realisation.hh11
-rw-r--r--src/libstore/remote-store.cc23
-rw-r--r--src/libstore/store-api.cc38
-rw-r--r--src/libstore/store-api.hh9
-rw-r--r--src/libstore/worker-protocol.hh2
17 files changed, 290 insertions, 50 deletions
diff --git a/src/libstore/build/derivation-goal.cc b/src/libstore/build/derivation-goal.cc
index 9100d3333..8c9ef0101 100644
--- a/src/libstore/build/derivation-goal.cc
+++ b/src/libstore/build/derivation-goal.cc
@@ -927,6 +927,7 @@ void DerivationGoal::resolvedFinished() {
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);
} else {
diff --git a/src/libstore/build/local-derivation-goal.cc b/src/libstore/build/local-derivation-goal.cc
index 28981a1a1..279139020 100644
--- a/src/libstore/build/local-derivation-goal.cc
+++ b/src/libstore/build/local-derivation-goal.cc
@@ -292,7 +292,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 &&
@@ -417,7 +417,7 @@ void LocalDerivationGoal::startBuilder()
}
auto & localStore = getLocalStore();
- if (localStore.storeDir != localStore.realStoreDir) {
+ if (localStore.storeDir != localStore.realStoreDir.get()) {
#if __linux__
useChroot = true;
#else
@@ -2487,6 +2487,7 @@ void LocalDerivationGoal::registerOutputs()
assert(newInfo.ca);
} else {
auto destPath = worker.store.toRealPath(finalDestPath);
+ deletePath(destPath);
movePath(actualPath, destPath);
actualPath = destPath;
}
diff --git a/src/libstore/ca-specific-schema.sql b/src/libstore/ca-specific-schema.sql
index 20ee046a1..08af0cc1f 100644
--- a/src/libstore/ca-specific-schema.sql
+++ b/src/libstore/ca-specific-schema.sql
@@ -3,10 +3,19 @@
-- is enabled
create table if not exists Realisations (
+ id integer primary key autoincrement not null,
drvPath text not null,
outputName text not null, -- symbolic output id, usually "out"
outputPath integer not null,
signatures text, -- space-separated list
- primary key (drvPath, outputName),
foreign key (outputPath) references ValidPaths(id) on delete cascade
);
+
+create index if not exists IndexRealisations on Realisations(drvPath, outputName);
+
+create table if not exists RealisationsRefs (
+ referrer integer not null,
+ realisationReference integer,
+ foreign key (referrer) references Realisations(id) on delete cascade,
+ foreign key (realisationReference) references Realisations(id) on delete restrict
+);
diff --git a/src/libstore/daemon.cc b/src/libstore/daemon.cc
index 72b3e3e13..e06fb9ce2 100644
--- a/src/libstore/daemon.cc
+++ b/src/libstore/daemon.cc
@@ -885,10 +885,15 @@ static void performOp(TunnelLogger * logger, ref<Store> store,
case wopRegisterDrvOutput: {
logger->startWork();
- auto outputId = DrvOutput::parse(readString(from));
- auto outputPath = StorePath(readString(from));
- store->registerDrvOutput(Realisation{
- .id = outputId, .outPath = outputPath});
+ if (GET_PROTOCOL_MINOR(clientVersion) < 31) {
+ auto outputId = DrvOutput::parse(readString(from));
+ auto outputPath = StorePath(readString(from));
+ store->registerDrvOutput(Realisation{
+ .id = outputId, .outPath = outputPath});
+ } else {
+ auto realisation = worker_proto::read(*store, from, Phantom<Realisation>());
+ store->registerDrvOutput(realisation);
+ }
logger->stopWork();
break;
}
@@ -898,9 +903,15 @@ static void performOp(TunnelLogger * logger, ref<Store> store,
auto outputId = DrvOutput::parse(readString(from));
auto info = store->queryRealisation(outputId);
logger->stopWork();
- std::set<StorePath> outPaths;
- if (info) outPaths.insert(info->outPath);
- worker_proto::write(*store, to, outPaths);
+ if (GET_PROTOCOL_MINOR(clientVersion) < 31) {
+ std::set<StorePath> outPaths;
+ if (info) outPaths.insert(info->outPath);
+ worker_proto::write(*store, to, outPaths);
+ } else {
+ std::set<Realisation> realisations;
+ if (info) realisations.insert(*info);
+ worker_proto::write(*store, to, realisations);
+ }
break;
}
diff --git a/src/libstore/gc.cc b/src/libstore/gc.cc
index bc692ca42..5a62c6529 100644
--- a/src/libstore/gc.cc
+++ b/src/libstore/gc.cc
@@ -775,7 +775,7 @@ void LocalStore::collectGarbage(const GCOptions & options, GCResults & results)
try {
- AutoCloseDir dir(opendir(realStoreDir.c_str()));
+ AutoCloseDir dir(opendir(realStoreDir.get().c_str()));
if (!dir) throw SysError("opening directory '%1%'", realStoreDir);
/* Read the store and immediately delete all paths that
@@ -856,7 +856,7 @@ void LocalStore::autoGC(bool sync)
return std::stoll(readFile(*fakeFreeSpaceFile));
struct statvfs st;
- if (statvfs(realStoreDir.c_str(), &st))
+ if (statvfs(realStoreDir.get().c_str(), &st))
throw SysError("getting filesystem info about '%s'", realStoreDir);
return (uint64_t) st.f_bavail * st.f_frsize;
diff --git a/src/libstore/local-fs-store.hh b/src/libstore/local-fs-store.hh
index 55941b771..f8b19d00d 100644
--- a/src/libstore/local-fs-store.hh
+++ b/src/libstore/local-fs-store.hh
@@ -18,6 +18,9 @@ struct LocalFSStoreConfig : virtual StoreConfig
const PathSetting logDir{(StoreConfig*) this, false,
rootDir != "" ? rootDir + "/nix/var/log/nix" : settings.nixLogDir,
"log", "directory where Nix will store state"};
+ const PathSetting realStoreDir{(StoreConfig*) this, false,
+ rootDir != "" ? rootDir + "/nix/store" : storeDir, "real",
+ "physical path to the Nix store"};
};
class LocalFSStore : public virtual LocalFSStoreConfig, public virtual Store
@@ -34,7 +37,7 @@ public:
/* Register a permanent GC root. */
Path addPermRoot(const StorePath & storePath, const Path & gcRoot);
- virtual Path getRealStoreDir() { return storeDir; }
+ virtual Path getRealStoreDir() { return realStoreDir; }
Path toRealPath(const Path & storePath) override
{
diff --git a/src/libstore/local-store.cc b/src/libstore/local-store.cc
index efea7bd23..c2256635a 100644
--- a/src/libstore/local-store.cc
+++ b/src/libstore/local-store.cc
@@ -59,6 +59,8 @@ struct LocalStore::State::Stmts {
SQLiteStmt QueryAllRealisedOutputs;
SQLiteStmt QueryPathFromHashPart;
SQLiteStmt QueryValidPaths;
+ SQLiteStmt QueryRealisationReferences;
+ SQLiteStmt AddRealisationReference;
};
int getSchema(Path schemaPath)
@@ -76,7 +78,7 @@ int getSchema(Path schemaPath)
void migrateCASchema(SQLite& db, Path schemaPath, AutoCloseFD& lockFd)
{
- const int nixCASchemaVersion = 1;
+ const int nixCASchemaVersion = 2;
int curCASchema = getSchema(schemaPath);
if (curCASchema != nixCASchemaVersion) {
if (curCASchema > nixCASchemaVersion) {
@@ -94,7 +96,39 @@ void migrateCASchema(SQLite& db, Path schemaPath, AutoCloseFD& lockFd)
#include "ca-specific-schema.sql.gen.hh"
;
db.exec(schema);
+ curCASchema = nixCASchemaVersion;
}
+
+ if (curCASchema < 2) {
+ SQLiteTxn txn(db);
+ // Ugly little sql dance to add a new `id` column and make it the primary key
+ db.exec(R"(
+ create table Realisations2 (
+ id integer primary key autoincrement not null,
+ drvPath text not null,
+ outputName text not null, -- symbolic output id, usually "out"
+ outputPath integer not null,
+ signatures text, -- space-separated list
+ foreign key (outputPath) references ValidPaths(id) on delete cascade
+ );
+ insert into Realisations2 (drvPath, outputName, outputPath, signatures)
+ select drvPath, outputName, outputPath, signatures from Realisations;
+ drop table Realisations;
+ alter table Realisations2 rename to Realisations;
+ )");
+ db.exec(R"(
+ create index if not exists IndexRealisations on Realisations(drvPath, outputName);
+
+ create table if not exists RealisationsRefs (
+ referrer integer not null,
+ realisationReference integer,
+ foreign key (referrer) references Realisations(id) on delete cascade,
+ foreign key (realisationReference) references Realisations(id) on delete restrict
+ );
+ )");
+ txn.commit();
+ }
+
writeFile(schemaPath, fmt("%d", nixCASchemaVersion));
lockFile(lockFd.get(), ltRead, true);
}
@@ -106,9 +140,6 @@ LocalStore::LocalStore(const Params & params)
, LocalStoreConfig(params)
, Store(params)
, LocalFSStore(params)
- , realStoreDir_{this, false, rootDir != "" ? rootDir + "/nix/store" : storeDir, "real",
- "physical path to the Nix store"}
- , realStoreDir(realStoreDir_)
, dbDir(stateDir + "/db")
, linksDir(realStoreDir + "/.links")
, reservedPath(dbDir + "/reserved")
@@ -153,13 +184,13 @@ LocalStore::LocalStore(const Params & params)
printError("warning: the group '%1%' specified in 'build-users-group' does not exist", settings.buildUsersGroup);
else {
struct stat st;
- if (stat(realStoreDir.c_str(), &st))
+ if (stat(realStoreDir.get().c_str(), &st))
throw SysError("getting attributes of path '%1%'", realStoreDir);
if (st.st_uid != 0 || st.st_gid != gr->gr_gid || (st.st_mode & ~S_IFMT) != perm) {
- if (chown(realStoreDir.c_str(), 0, gr->gr_gid) == -1)
+ if (chown(realStoreDir.get().c_str(), 0, gr->gr_gid) == -1)
throw SysError("changing ownership of path '%1%'", realStoreDir);
- if (chmod(realStoreDir.c_str(), perm) == -1)
+ if (chmod(realStoreDir.get().c_str(), perm) == -1)
throw SysError("changing permissions on path '%1%'", realStoreDir);
}
}
@@ -316,7 +347,7 @@ LocalStore::LocalStore(const Params & params)
)");
state->stmts->QueryRealisedOutput.create(state->db,
R"(
- select Output.path, Realisations.signatures from Realisations
+ select Realisations.id, Output.path, Realisations.signatures from Realisations
inner join ValidPaths as Output on Output.id = Realisations.outputPath
where drvPath = ? and outputName = ?
;
@@ -328,6 +359,19 @@ LocalStore::LocalStore(const Params & params)
where drvPath = ?
;
)");
+ state->stmts->QueryRealisationReferences.create(state->db,
+ R"(
+ select drvPath, outputName from Realisations
+ join RealisationsRefs on realisationReference = Realisations.id
+ where referrer = ?;
+ )");
+ state->stmts->AddRealisationReference.create(state->db,
+ R"(
+ insert or replace into RealisationsRefs (referrer, realisationReference)
+ values (
+ ?,
+ (select id from Realisations where drvPath = ? and outputName = ?));
+ )");
}
}
@@ -437,14 +481,14 @@ void LocalStore::makeStoreWritable()
if (getuid() != 0) return;
/* Check if /nix/store is on a read-only mount. */
struct statvfs stat;
- if (statvfs(realStoreDir.c_str(), &stat) != 0)
+ if (statvfs(realStoreDir.get().c_str(), &stat) != 0)
throw SysError("getting info about the Nix store mount point");
if (stat.f_flag & ST_RDONLY) {
if (unshare(CLONE_NEWNS) == -1)
throw SysError("setting up a private mount namespace");
- if (mount(0, realStoreDir.c_str(), "none", MS_REMOUNT | MS_BIND, 0) == -1)
+ if (mount(0, realStoreDir.get().c_str(), "none", MS_REMOUNT | MS_BIND, 0) == -1)
throw SysError("remounting %1% writable", realStoreDir);
}
#endif
@@ -664,14 +708,22 @@ void LocalStore::registerDrvOutput(const Realisation & info, CheckSigsFlag check
void LocalStore::registerDrvOutput(const Realisation & info)
{
settings.requireExperimentalFeature("ca-derivations");
- auto state(_state.lock());
retrySQLite<void>([&]() {
+ auto state(_state.lock());
state->stmts->RegisterRealisedOutput.use()
(info.id.strHash())
(info.id.outputName)
(printStorePath(info.outPath))
(concatStringsSep(" ", info.signatures))
.exec();
+ uint64_t myId = state->db.getLastInsertedRowId();
+ for (auto & [outputId, _] : info.dependentRealisations) {
+ state->stmts->AddRealisationReference.use()
+ (myId)
+ (outputId.strHash())
+ (outputId.outputName)
+ .exec();
+ }
});
}
@@ -1687,14 +1739,38 @@ std::optional<const Realisation> LocalStore::queryRealisation(
typedef std::optional<const Realisation> Ret;
return retrySQLite<Ret>([&]() -> Ret {
auto state(_state.lock());
- auto use(state->stmts->QueryRealisedOutput.use()(id.strHash())(
- id.outputName));
- if (!use.next())
+ auto useQueryRealisedOutput(state->stmts->QueryRealisedOutput.use()
+ (id.strHash())
+ (id.outputName));
+ if (!useQueryRealisedOutput.next())
return std::nullopt;
- auto outputPath = parseStorePath(use.getStr(0));
- auto signatures = tokenizeString<StringSet>(use.getStr(1));
+ auto realisationDbId = useQueryRealisedOutput.getInt(0);
+ auto outputPath = parseStorePath(useQueryRealisedOutput.getStr(1));
+ auto signatures =
+ tokenizeString<StringSet>(useQueryRealisedOutput.getStr(2));
+
+ std::map<DrvOutput, StorePath> dependentRealisations;
+ auto useRealisationRefs(
+ state->stmts->QueryRealisationReferences.use()
+ (realisationDbId));
+ while (useRealisationRefs.next()) {
+ auto depHash = useRealisationRefs.getStr(0);
+ auto depOutputName = useRealisationRefs.getStr(1);
+ auto useQueryRealisedOutput(state->stmts->QueryRealisedOutput.use()
+ (depHash)
+ (depOutputName));
+ assert(useQueryRealisedOutput.next());
+ auto outputPath = parseStorePath(useQueryRealisedOutput.getStr(1));
+ auto depId = DrvOutput { Hash::parseAnyPrefixed(depHash), depOutputName };
+ dependentRealisations.insert({depId, outputPath});
+ }
+
return Ret{Realisation{
- .id = id, .outPath = outputPath, .signatures = signatures}};
+ .id = id,
+ .outPath = outputPath,
+ .signatures = signatures,
+ .dependentRealisations = dependentRealisations,
+ }};
});
}
diff --git a/src/libstore/local-store.hh b/src/libstore/local-store.hh
index c0b8e0da6..15c7fc306 100644
--- a/src/libstore/local-store.hh
+++ b/src/libstore/local-store.hh
@@ -83,9 +83,6 @@ private:
public:
- PathSetting realStoreDir_;
-
- const Path realStoreDir;
const Path dbDir;
const Path linksDir;
const Path reservedPath;
@@ -279,8 +276,6 @@ private:
void signPathInfo(ValidPathInfo & info);
void signRealisation(Realisation &);
- Path getRealStoreDir() override { return realStoreDir; }
-
void createUser(const std::string & userName, uid_t userId) override;
// XXX: Make a generic `Store` method
diff --git a/src/libstore/misc.cc b/src/libstore/misc.cc
index 48442563f..b4929b445 100644
--- a/src/libstore/misc.cc
+++ b/src/libstore/misc.cc
@@ -254,5 +254,44 @@ StorePaths Store::topoSortPaths(const StorePathSet & paths)
}});
}
+std::map<DrvOutput, StorePath> drvOutputReferences(
+ const std::set<Realisation> & inputRealisations,
+ const StorePathSet & pathReferences)
+{
+ std::map<DrvOutput, StorePath> res;
+
+ for (const auto & input : inputRealisations) {
+ if (pathReferences.count(input.outPath)) {
+ res.insert({input.id, input.outPath});
+ }
+ }
+
+ return res;
+}
+std::map<DrvOutput, StorePath> drvOutputReferences(
+ Store & store,
+ const Derivation & drv,
+ const StorePath & outputPath)
+{
+ std::set<Realisation> inputRealisations;
+
+ for (const auto& [inputDrv, outputNames] : drv.inputDrvs) {
+ auto outputHashes =
+ staticOutputHashes(store, store.readDerivation(inputDrv));
+ for (const auto& outputName : outputNames) {
+ auto thisRealisation = store.queryRealisation(
+ DrvOutput{outputHashes.at(outputName), outputName});
+ if (!thisRealisation)
+ throw Error(
+ "output '%s' of derivation '%s' isn’t built", outputName,
+ store.printStorePath(inputDrv));
+ inputRealisations.insert(*thisRealisation);
+ }
+ }
+
+ auto info = store.queryPathInfo(outputPath);
+
+ return drvOutputReferences(Realisation::closure(store, inputRealisations), info->references);
+}
}
diff --git a/src/libstore/optimise-store.cc b/src/libstore/optimise-store.cc
index 78d587139..d95e54af1 100644
--- a/src/libstore/optimise-store.cc
+++ b/src/libstore/optimise-store.cc
@@ -198,7 +198,7 @@ void LocalStore::optimisePath_(Activity * act, OptimiseStats & stats,
/* Make the containing directory writable, but only if it's not
the store itself (we don't want or need to mess with its
permissions). */
- bool mustToggle = dirOf(path) != realStoreDir;
+ bool mustToggle = dirOf(path) != realStoreDir.get();
if (mustToggle) makeWritable(dirOf(path));
/* When we're done, make the directory read-only again and reset
diff --git a/src/libstore/parsed-derivations.cc b/src/libstore/parsed-derivations.cc
index c5c3ae3dc..5e383a9a4 100644
--- a/src/libstore/parsed-derivations.cc
+++ b/src/libstore/parsed-derivations.cc
@@ -91,6 +91,8 @@ StringSet ParsedDerivation::getRequiredSystemFeatures() const
StringSet res;
for (auto & i : getStringsAttr("requiredSystemFeatures").value_or(Strings()))
res.insert(i);
+ if (!derivationHasKnownOutputPaths(drv.type()))
+ res.insert("ca-derivations");
return res;
}
diff --git a/src/libstore/realisation.cc b/src/libstore/realisation.cc
index 638065547..0d9d4b433 100644
--- a/src/libstore/realisation.cc
+++ b/src/libstore/realisation.cc
@@ -1,5 +1,6 @@
#include "realisation.hh"
#include "store-api.hh"
+#include "closure.hh"
#include <nlohmann/json.hpp>
namespace nix {
@@ -21,11 +22,52 @@ std::string DrvOutput::to_string() const {
return strHash() + "!" + outputName;
}
+std::set<Realisation> Realisation::closure(Store & store, const std::set<Realisation> & startOutputs)
+{
+ std::set<Realisation> res;
+ Realisation::closure(store, startOutputs, res);
+ return res;
+}
+
+void Realisation::closure(Store & store, const std::set<Realisation> & startOutputs, std::set<Realisation> & res)
+{
+ auto getDeps = [&](const Realisation& current) -> std::set<Realisation> {
+ std::set<Realisation> res;
+ for (auto& [currentDep, _] : current.dependentRealisations) {
+ if (auto currentRealisation = store.queryRealisation(currentDep))
+ res.insert(*currentRealisation);
+ else
+ throw Error(
+ "Unrealised derivation '%s'", currentDep.to_string());
+ }
+ return res;
+ };
+
+ computeClosure<Realisation>(
+ startOutputs, res,
+ [&](const Realisation& current,
+ std::function<void(std::promise<std::set<Realisation>>&)>
+ processEdges) {
+ std::promise<std::set<Realisation>> promise;
+ try {
+ auto res = getDeps(current);
+ promise.set_value(res);
+ } catch (...) {
+ promise.set_exception(std::current_exception());
+ }
+ return processEdges(promise);
+ });
+}
+
nlohmann::json Realisation::toJSON() const {
+ auto jsonDependentRealisations = nlohmann::json::object();
+ for (auto & [depId, depOutPath] : dependentRealisations)
+ jsonDependentRealisations.emplace(depId.to_string(), depOutPath.to_string());
return nlohmann::json{
{"id", id.to_string()},
{"outPath", outPath.to_string()},
{"signatures", signatures},
+ {"dependentRealisations", jsonDependentRealisations},
};
}
@@ -51,10 +93,16 @@ Realisation Realisation::fromJSON(
if (auto signaturesIterator = json.find("signatures"); signaturesIterator != json.end())
signatures.insert(signaturesIterator->begin(), signaturesIterator->end());
+ std::map <DrvOutput, StorePath> dependentRealisations;
+ if (auto jsonDependencies = json.find("dependentRealisations"); jsonDependencies != json.end())
+ for (auto & [jsonDepId, jsonDepOutPath] : jsonDependencies->get<std::map<std::string, std::string>>())
+ dependentRealisations.insert({DrvOutput::parse(jsonDepId), StorePath(jsonDepOutPath)});
+
return Realisation{
.id = DrvOutput::parse(getField("id")),
.outPath = StorePath(getField("outPath")),
.signatures = signatures,
+ .dependentRealisations = dependentRealisations,
};
}
diff --git a/src/libstore/realisation.hh b/src/libstore/realisation.hh
index f5049c9e9..7fdb65acd 100644
--- a/src/libstore/realisation.hh
+++ b/src/libstore/realisation.hh
@@ -28,6 +28,14 @@ struct Realisation {
StringSet signatures;
+ /**
+ * The realisations that are required for the current one to be valid.
+ *
+ * When importing this realisation, the store will first check that all its
+ * dependencies exist, and map to the correct output path
+ */
+ std::map<DrvOutput, StorePath> dependentRealisations;
+
nlohmann::json toJSON() const;
static Realisation fromJSON(const nlohmann::json& json, const std::string& whence);
@@ -36,6 +44,9 @@ struct Realisation {
bool checkSignature(const PublicKeys & publicKeys, const std::string & sig) const;
size_t checkSignatures(const PublicKeys & publicKeys) const;
+ static std::set<Realisation> closure(Store &, const std::set<Realisation> &);
+ static void closure(Store &, const std::set<Realisation> &, std::set<Realisation>& res);
+
StorePath getPath() const { return outPath; }
GENERATE_CMP(Realisation, me->id, me->outPath);
diff --git a/src/libstore/remote-store.cc b/src/libstore/remote-store.cc
index d9b6e9488..aec243637 100644
--- a/src/libstore/remote-store.cc
+++ b/src/libstore/remote-store.cc
@@ -653,8 +653,12 @@ void RemoteStore::registerDrvOutput(const Realisation & info)
{
auto conn(getConnection());
conn->to << wopRegisterDrvOutput;
- conn->to << info.id.to_string();
- conn->to << std::string(info.outPath.to_string());
+ if (GET_PROTOCOL_MINOR(conn->daemonVersion) < 31) {
+ conn->to << info.id.to_string();
+ conn->to << std::string(info.outPath.to_string());
+ } else {
+ worker_proto::write(*this, conn->to, info);
+ }
conn.processStderr();
}
@@ -664,10 +668,17 @@ std::optional<const Realisation> RemoteStore::queryRealisation(const DrvOutput &
conn->to << wopQueryRealisation;
conn->to << id.to_string();
conn.processStderr();
- auto outPaths = worker_proto::read(*this, conn->from, Phantom<std::set<StorePath>>{});
- if (outPaths.empty())
- return std::nullopt;
- return {Realisation{.id = id, .outPath = *outPaths.begin()}};
+ if (GET_PROTOCOL_MINOR(conn->daemonVersion) < 31) {
+ auto outPaths = worker_proto::read(*this, conn->from, Phantom<std::set<StorePath>>{});
+ if (outPaths.empty())
+ return std::nullopt;
+ return {Realisation{.id = id, .outPath = *outPaths.begin()}};
+ } else {
+ auto realisations = worker_proto::read(*this, conn->from, Phantom<std::set<Realisation>>{});
+ if (realisations.empty())
+ return std::nullopt;
+ return *realisations.begin();
+ }
}
static void writeDerivedPaths(RemoteStore & store, ConnectionHandle & conn, const std::vector<DerivedPath> & reqs)
diff --git a/src/libstore/store-api.cc b/src/libstore/store-api.cc
index 93fcb068f..6736adb24 100644
--- a/src/libstore/store-api.cc
+++ b/src/libstore/store-api.cc
@@ -337,6 +337,13 @@ ValidPathInfo Store::addToStoreSlow(std::string_view name, const Path & srcPath,
return info;
}
+StringSet StoreConfig::getDefaultSystemFeatures()
+{
+ auto res = settings.systemFeatures.get();
+ if (settings.isExperimentalFeatureEnabled("ca-derivations"))
+ res.insert("ca-derivations");
+ return res;
+}
Store::Store(const Params & params)
: StoreConfig(params)
@@ -780,20 +787,39 @@ std::map<StorePath, StorePath> copyPaths(ref<Store> srcStore, ref<Store> dstStor
RepairFlag repair, CheckSigsFlag checkSigs, SubstituteFlag substitute)
{
StorePathSet storePaths;
- std::set<Realisation> realisations;
+ std::set<Realisation> toplevelRealisations;
for (auto & path : paths) {
storePaths.insert(path.path());
if (auto realisation = std::get_if<Realisation>(&path.raw)) {
settings.requireExperimentalFeature("ca-derivations");
- realisations.insert(*realisation);
+ toplevelRealisations.insert(*realisation);
}
}
auto pathsMap = copyPaths(srcStore, dstStore, storePaths, repair, checkSigs, substitute);
+
+ ThreadPool pool;
+
try {
- for (auto & realisation : realisations) {
- dstStore->registerDrvOutput(realisation, checkSigs);
- }
- } catch (MissingExperimentalFeature & e) {
+ // Copy the realisation closure
+ processGraph<Realisation>(
+ pool, Realisation::closure(*srcStore, toplevelRealisations),
+ [&](const Realisation& current) -> std::set<Realisation> {
+ std::set<Realisation> children;
+ for (const auto& [drvOutput, _] : current.dependentRealisations) {
+ auto currentChild = srcStore->queryRealisation(drvOutput);
+ if (!currentChild)
+ throw Error(
+ "Incomplete realisation closure: '%s' is a "
+ "dependency of '%s' but isn’t registered",
+ drvOutput.to_string(), current.id.to_string());
+ children.insert(*currentChild);
+ }
+ return children;
+ },
+ [&](const Realisation& current) -> void {
+ dstStore->registerDrvOutput(current, checkSigs);
+ });
+ } catch (MissingExperimentalFeature& e) {
// Don't fail if the remote doesn't support CA derivations is it might
// not be within our control to change that, and we might still want
// to at least copy the output paths.
diff --git a/src/libstore/store-api.hh b/src/libstore/store-api.hh
index f66298991..9657d2adf 100644
--- a/src/libstore/store-api.hh
+++ b/src/libstore/store-api.hh
@@ -180,6 +180,8 @@ struct StoreConfig : public Config
StoreConfig() = delete;
+ StringSet getDefaultSystemFeatures();
+
virtual ~StoreConfig() { }
virtual const std::string name() = 0;
@@ -196,7 +198,7 @@ struct StoreConfig : public Config
Setting<bool> wantMassQuery{this, false, "want-mass-query", "whether this substituter can be queried efficiently for path validity"};
- Setting<StringSet> systemFeatures{this, settings.systemFeatures,
+ Setting<StringSet> systemFeatures{this, getDefaultSystemFeatures(),
"system-features",
"Optional features that the system this store builds on implements (like \"kvm\")."};
@@ -864,4 +866,9 @@ std::pair<std::string, Store::Params> splitUriAndParams(const std::string & uri)
std::optional<ContentAddress> getDerivationCA(const BasicDerivation & drv);
+std::map<DrvOutput, StorePath> drvOutputReferences(
+ Store & store,
+ const Derivation & drv,
+ const StorePath & outputPath);
+
}
diff --git a/src/libstore/worker-protocol.hh b/src/libstore/worker-protocol.hh
index fdd692cf0..e89183d40 100644
--- a/src/libstore/worker-protocol.hh
+++ b/src/libstore/worker-protocol.hh
@@ -9,7 +9,7 @@ namespace nix {
#define WORKER_MAGIC_1 0x6e697863
#define WORKER_MAGIC_2 0x6478696f
-#define PROTOCOL_VERSION (1 << 8 | 30)
+#define PROTOCOL_VERSION (1 << 8 | 31)
#define GET_PROTOCOL_MAJOR(x) ((x) & 0xff00)
#define GET_PROTOCOL_MINOR(x) ((x) & 0x00ff)