1
1
Fork 0
mirror of https://github.com/NixOS/nix.git synced 2025-11-09 03:56:01 +01:00

Merge pull request #13879 from xokdvium/static-alloc-symbol-ids

libexpr: Statically allocate commonly used symbols
This commit is contained in:
Jörg Thalheim 2025-08-31 13:40:55 +02:00 committed by GitHub
commit 1f7d43e5bd
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
23 changed files with 313 additions and 255 deletions

View file

@ -105,8 +105,8 @@ DerivedPathsWithInfo InstallableFlake::toDerivedPaths()
std::optional<NixInt::Inner> priority;
if (attr->maybeGetAttr(state->sOutputSpecified)) {
} else if (auto aMeta = attr->maybeGetAttr(state->sMeta)) {
if (attr->maybeGetAttr(state->s.outputSpecified)) {
} else if (auto aMeta = attr->maybeGetAttr(state->s.meta)) {
if (auto aPriority = aMeta->maybeGetAttr("priority"))
priority = aPriority->getInt().value;
}
@ -119,12 +119,12 @@ DerivedPathsWithInfo InstallableFlake::toDerivedPaths()
overloaded{
[&](const ExtendedOutputsSpec::Default & d) -> OutputsSpec {
StringSet outputsToInstall;
if (auto aOutputSpecified = attr->maybeGetAttr(state->sOutputSpecified)) {
if (auto aOutputSpecified = attr->maybeGetAttr(state->s.outputSpecified)) {
if (aOutputSpecified->getBool()) {
if (auto aOutputName = attr->maybeGetAttr("outputName"))
outputsToInstall = {aOutputName->getString()};
}
} else if (auto aMeta = attr->maybeGetAttr(state->sMeta)) {
} else if (auto aMeta = attr->maybeGetAttr(state->s.meta)) {
if (auto aOutputsToInstall = aMeta->maybeGetAttr("outputsToInstall"))
for (auto & s : aOutputsToInstall->getListOfStrings())
outputsToInstall.insert(s);

View file

@ -393,7 +393,7 @@ TEST_F(ValuePrintingTests, ansiColorsDerivation)
vDerivation.mkString("derivation");
BindingsBuilder builder(state, state.allocBindings(10));
builder.insert(state.sType, &vDerivation);
builder.insert(state.s.type, &vDerivation);
Value vAttrs;
vAttrs.mkAttrs(builder.finish());
@ -438,8 +438,8 @@ TEST_F(ValuePrintingTests, ansiColorsDerivationError)
vDerivation.mkString("derivation");
BindingsBuilder builder(state, state.allocBindings(10));
builder.insert(state.sType, &vDerivation);
builder.insert(state.sDrvPath, &vError);
builder.insert(state.s.type, &vDerivation);
builder.insert(state.s.drvPath, &vError);
Value vAttrs;
vAttrs.mkAttrs(builder.finish());

View file

@ -330,7 +330,7 @@ AttrCursor::AttrCursor(
AttrKey AttrCursor::getKey()
{
if (!parent)
return {0, root->state.sEpsilon};
return {0, root->state.s.epsilon};
if (!parent->first->cachedValue) {
parent->first->cachedValue = root->db->getAttr(parent->first->getKey());
assert(parent->first->cachedValue);
@ -702,7 +702,7 @@ bool AttrCursor::isDerivation()
StorePath AttrCursor::forceDerivation()
{
auto aDrvPath = getAttr(root->state.sDrvPath);
auto aDrvPath = getAttr(root->state.s.drvPath);
auto drvPath = root->state.store->parseStorePath(aDrvPath->getString());
drvPath.requireDerivation();
if (!root->state.store->isValidPath(drvPath) && !settings.readOnlyMode) {

View file

@ -185,7 +185,7 @@ FrameInfo SampleStack::getPrimOpFrameInfo(const PrimOp & primOp, std::span<Value
/* Error context strings don't actually matter, since we ignore all eval errors. */
state.forceAttrs(*args[0], pos, "");
auto attrs = args[0]->attrs();
auto nameAttr = state.getAttr(state.sName, attrs, "");
auto nameAttr = state.getAttr(state.s.name, attrs, "");
auto drvName = std::string(state.forceStringNoCtx(*nameAttr->value, pos, ""));
return DerivationStrictFrameInfo{.callPos = pos, .drvName = std::move(drvName)};
} catch (...) {
@ -211,7 +211,7 @@ FrameInfo SampleStack::getFrameInfoFromValueAndPos(const Value & v, std::span<Va
/* Resolve primOp eagerly. Must not hold on to a reference to a Value. */
return PrimOpFrameInfo{.expr = v.primOpAppPrimOp(), .callPos = pos};
else if (state.isFunctor(v)) {
const auto functor = v.attrs()->get(state.sFunctor);
const auto functor = v.attrs()->get(state.s.functor);
if (auto pos_ = posCache.lookup(pos); std::holds_alternative<std::monostate>(pos_.origin))
/* HACK: In case callsite position is unresolved. */
return FunctorFrameInfo{.pos = functor->pos};

View file

@ -203,124 +203,65 @@ EvalState::EvalState(
std::shared_ptr<Store> buildStore)
: fetchSettings{fetchSettings}
, settings{settings}
, sWith(symbols.create("<with>"))
, sOutPath(symbols.create("outPath"))
, sDrvPath(symbols.create("drvPath"))
, sType(symbols.create("type"))
, sMeta(symbols.create("meta"))
, sName(symbols.create("name"))
, sValue(symbols.create("value"))
, sSystem(symbols.create("system"))
, sOverrides(symbols.create("__overrides"))
, sOutputs(symbols.create("outputs"))
, sOutputName(symbols.create("outputName"))
, sIgnoreNulls(symbols.create("__ignoreNulls"))
, sFile(symbols.create("file"))
, sLine(symbols.create("line"))
, sColumn(symbols.create("column"))
, sFunctor(symbols.create("__functor"))
, sToString(symbols.create("__toString"))
, sRight(symbols.create("right"))
, sWrong(symbols.create("wrong"))
, sStructuredAttrs(symbols.create("__structuredAttrs"))
, sJson(symbols.create("__json"))
, sAllowedReferences(symbols.create("allowedReferences"))
, sAllowedRequisites(symbols.create("allowedRequisites"))
, sDisallowedReferences(symbols.create("disallowedReferences"))
, sDisallowedRequisites(symbols.create("disallowedRequisites"))
, sMaxSize(symbols.create("maxSize"))
, sMaxClosureSize(symbols.create("maxClosureSize"))
, sBuilder(symbols.create("builder"))
, sArgs(symbols.create("args"))
, sContentAddressed(symbols.create("__contentAddressed"))
, sImpure(symbols.create("__impure"))
, sOutputHash(symbols.create("outputHash"))
, sOutputHashAlgo(symbols.create("outputHashAlgo"))
, sOutputHashMode(symbols.create("outputHashMode"))
, sRecurseForDerivations(symbols.create("recurseForDerivations"))
, sDescription(symbols.create("description"))
, sSelf(symbols.create("self"))
, sEpsilon(symbols.create(""))
, sStartSet(symbols.create("startSet"))
, sOperator(symbols.create("operator"))
, sKey(symbols.create("key"))
, sPath(symbols.create("path"))
, sPrefix(symbols.create("prefix"))
, sOutputSpecified(symbols.create("outputSpecified"))
, exprSymbols{
.sub = symbols.create("__sub"),
.lessThan = symbols.create("__lessThan"),
.mul = symbols.create("__mul"),
.div = symbols.create("__div"),
.or_ = symbols.create("or"),
.findFile = symbols.create("__findFile"),
.nixPath = symbols.create("__nixPath"),
.body = symbols.create("body"),
}
, symbols(StaticEvalSymbols::staticSymbolTable())
, repair(NoRepair)
, emptyBindings(0)
, storeFS(
makeMountedSourceAccessor(
{
{CanonPath::root, makeEmptySourceAccessor()},
/* In the pure eval case, we can simply require
valid paths. However, in the *impure* eval
case this gets in the way of the union
mechanism, because an invalid access in the
upper layer will *not* be caught by the union
source accessor, but instead abort the entire
lookup.
, storeFS(makeMountedSourceAccessor({
{CanonPath::root, makeEmptySourceAccessor()},
/* In the pure eval case, we can simply require
valid paths. However, in the *impure* eval
case this gets in the way of the union
mechanism, because an invalid access in the
upper layer will *not* be caught by the union
source accessor, but instead abort the entire
lookup.
This happens when the store dir in the
ambient file system has a path (e.g. because
another Nix store there), but the relocated
store does not.
This happens when the store dir in the
ambient file system has a path (e.g. because
another Nix store there), but the relocated
store does not.
TODO make the various source accessors doing
access control all throw the same type of
exception, and make union source accessor
catch it, so we don't need to do this hack.
*/
{CanonPath(store->storeDir), store->getFSAccessor(settings.pureEval)},
}))
, rootFS(
({
/* In pure eval mode, we provide a filesystem that only
contains the Nix store.
TODO make the various source accessors doing
access control all throw the same type of
exception, and make union source accessor
catch it, so we don't need to do this hack.
*/
{CanonPath(store->storeDir), store->getFSAccessor(settings.pureEval)},
}))
, rootFS(({
/* In pure eval mode, we provide a filesystem that only
contains the Nix store.
If we have a chroot store and pure eval is not enabled,
use a union accessor to make the chroot store available
at its logical location while still having the
underlying directory available. This is necessary for
instance if we're evaluating a file from the physical
/nix/store while using a chroot store. */
auto accessor = getFSSourceAccessor();
If we have a chroot store and pure eval is not enabled,
use a union accessor to make the chroot store available
at its logical location while still having the
underlying directory available. This is necessary for
instance if we're evaluating a file from the physical
/nix/store while using a chroot store. */
auto accessor = getFSSourceAccessor();
auto realStoreDir = dirOf(store->toRealPath(StorePath::dummy));
if (settings.pureEval || store->storeDir != realStoreDir) {
accessor = settings.pureEval
? storeFS
: makeUnionSourceAccessor({accessor, storeFS});
}
auto realStoreDir = dirOf(store->toRealPath(StorePath::dummy));
if (settings.pureEval || store->storeDir != realStoreDir) {
accessor = settings.pureEval ? storeFS : makeUnionSourceAccessor({accessor, storeFS});
}
/* Apply access control if needed. */
if (settings.restrictEval || settings.pureEval)
accessor = AllowListSourceAccessor::create(accessor, {}, {},
[&settings](const CanonPath & path) -> RestrictedPathError {
auto modeInformation = settings.pureEval
? "in pure evaluation mode (use '--impure' to override)"
: "in restricted mode";
throw RestrictedPathError("access to absolute path '%1%' is forbidden %2%", path, modeInformation);
});
/* Apply access control if needed. */
if (settings.restrictEval || settings.pureEval)
accessor = AllowListSourceAccessor::create(
accessor, {}, {}, [&settings](const CanonPath & path) -> RestrictedPathError {
auto modeInformation = settings.pureEval ? "in pure evaluation mode (use '--impure' to override)"
: "in restricted mode";
throw RestrictedPathError("access to absolute path '%1%' is forbidden %2%", path, modeInformation);
});
accessor;
}))
accessor;
}))
, corepkgsFS(make_ref<MemorySourceAccessor>())
, internalFS(make_ref<MemorySourceAccessor>())
, derivationInternal{corepkgsFS->addFile(
CanonPath("derivation-internal.nix"),
CanonPath("derivation-internal.nix"),
#include "primops/derivation.nix.gen.hh"
)}
)}
, store(store)
, buildStore(buildStore ? buildStore : store)
, inputCache(fetchers::InputCache::create())
@ -654,7 +595,7 @@ std::optional<EvalState::Doc> EvalState::getDoc(Value & v)
}
if (isFunctor(v)) {
try {
Value & functor = *v.attrs()->find(sFunctor)->value;
Value & functor = *v.attrs()->find(s.functor)->value;
Value * vp[] = {&v};
Value partiallyApplied;
// The first parameter is not user-provided, and may be
@ -978,8 +919,8 @@ void EvalState::mkPos(Value & v, PosIdx p)
auto origin = positions.originOf(p);
if (auto path = std::get_if<SourcePath>(&origin)) {
auto attrs = buildBindings(3);
attrs.alloc(sFile).mkString(path->path.abs());
makePositionThunks(*this, p, attrs.alloc(sLine), attrs.alloc(sColumn));
attrs.alloc(s.file).mkString(path->path.abs());
makePositionThunks(*this, p, attrs.alloc(s.line), attrs.alloc(s.column));
v.mkAttrs(attrs);
} else
v.mkNull();
@ -1245,7 +1186,7 @@ void ExprAttrs::eval(EvalState & state, Env & env, Value & v)
dynamicEnv = &env2;
Env * inheritEnv = inheritFromExprs ? buildInheritFromEnv(state, env2) : nullptr;
AttrDefs::iterator overrides = attrs.find(state.sOverrides);
AttrDefs::iterator overrides = attrs.find(state.s.overrides);
bool hasOverrides = overrides != attrs.end();
/* The recursive attributes are evaluated in the new
@ -1717,7 +1658,7 @@ void EvalState::callFunction(Value & fun, std::span<Value *> args, Value & vRes,
}
}
else if (vCur.type() == nAttrs && (functor = vCur.attrs()->get(sFunctor))) {
else if (vCur.type() == nAttrs && (functor = vCur.attrs()->get(s.functor))) {
/* 'vCur' may be allocated on the stack of the calling
function, but for functors we may keep a reference, so
heap-allocate a copy and use that instead. */
@ -1779,7 +1720,7 @@ void EvalState::autoCallFunction(const Bindings & args, Value & fun, Value & res
forceValue(fun, pos);
if (fun.type() == nAttrs) {
auto found = fun.attrs()->find(sFunctor);
auto found = fun.attrs()->find(s.functor);
if (found != fun.attrs()->end()) {
Value * v = allocValue();
callFunction(*found->value, fun, *v, pos);
@ -2241,7 +2182,7 @@ Bindings::const_iterator EvalState::getAttr(Symbol attrSym, const Bindings * att
bool EvalState::isFunctor(const Value & fun) const
{
return fun.type() == nAttrs && fun.attrs()->find(sFunctor) != fun.attrs()->end();
return fun.type() == nAttrs && fun.attrs()->find(s.functor) != fun.attrs()->end();
}
void EvalState::forceFunction(Value & v, const PosIdx pos, std::string_view errorCtx)
@ -2310,7 +2251,7 @@ bool EvalState::isDerivation(Value & v)
{
if (v.type() != nAttrs)
return false;
auto i = v.attrs()->get(sType);
auto i = v.attrs()->get(s.type);
if (!i)
return false;
forceValue(*i->value, i->pos);
@ -2322,7 +2263,7 @@ bool EvalState::isDerivation(Value & v)
std::optional<std::string>
EvalState::tryAttrsToString(const PosIdx pos, Value & v, NixStringContext & context, bool coerceMore, bool copyToStore)
{
auto i = v.attrs()->find(sToString);
auto i = v.attrs()->find(s.toString);
if (i != v.attrs()->end()) {
Value v1;
callFunction(*i->value, v, v1, pos);
@ -2368,7 +2309,7 @@ BackedStringView EvalState::coerceToString(
auto maybeString = tryAttrsToString(pos, v, context, coerceMore, copyToStore);
if (maybeString)
return std::move(*maybeString);
auto i = v.attrs()->find(sOutPath);
auto i = v.attrs()->find(s.outPath);
if (i == v.attrs()->end()) {
error<TypeError>(
"cannot coerce %1% to a string: %2%", showType(v), ValuePrinter(*this, v, errorPrintOptions))
@ -2475,7 +2416,7 @@ SourcePath EvalState::coerceToPath(const PosIdx pos, Value & v, NixStringContext
/* Similarly, handle __toString where the result may be a path
value. */
if (v.type() == nAttrs) {
auto i = v.attrs()->find(sToString);
auto i = v.attrs()->find(s.toString);
if (i != v.attrs()->end()) {
Value v1;
callFunction(*i->value, v, v1, pos);
@ -2665,8 +2606,8 @@ void EvalState::assertEqValues(Value & v1, Value & v2, const PosIdx pos, std::st
case nAttrs: {
if (isDerivation(v1) && isDerivation(v2)) {
auto i = v1.attrs()->get(sOutPath);
auto j = v2.attrs()->get(sOutPath);
auto i = v1.attrs()->get(s.outPath);
auto j = v2.attrs()->get(s.outPath);
if (i && j) {
try {
assertEqValues(*i->value, *j->value, pos, errorCtx);
@ -2819,8 +2760,8 @@ bool EvalState::eqValues(Value & v1, Value & v2, const PosIdx pos, std::string_v
/* If both sets denote a derivation (type = "derivation"),
then compare their outPaths. */
if (isDerivation(v1) && isDerivation(v2)) {
auto i = v1.attrs()->get(sOutPath);
auto j = v2.attrs()->get(sOutPath);
auto i = v1.attrs()->get(s.outPath);
auto j = v2.attrs()->get(s.outPath);
if (i && j)
return eqValues(*i->value, *j->value, pos, errorCtx);
}
@ -3196,8 +3137,7 @@ Expr * EvalState::parse(
docComments = &it->second;
}
auto result = parseExprFromBuf(
text, length, origin, basePath, symbols, settings, positions, *docComments, rootFS, exprSymbols);
auto result = parseExprFromBuf(text, length, origin, basePath, symbols, settings, positions, *docComments, rootFS);
result->bindVars(*this, staticEnv);

View file

@ -45,7 +45,7 @@ PackageInfo::PackageInfo(EvalState & state, ref<Store> store, const std::string
std::string PackageInfo::queryName() const
{
if (name == "" && attrs) {
auto i = attrs->find(state->sName);
auto i = attrs->find(state->s.name);
if (i == attrs->end())
state->error<TypeError>("derivation name missing").debugThrow();
name = state->forceStringNoCtx(*i->value, noPos, "while evaluating the 'name' attribute of a derivation");
@ -56,7 +56,7 @@ std::string PackageInfo::queryName() const
std::string PackageInfo::querySystem() const
{
if (system == "" && attrs) {
auto i = attrs->find(state->sSystem);
auto i = attrs->find(state->s.system);
system =
i == attrs->end()
? "unknown"
@ -68,7 +68,7 @@ std::string PackageInfo::querySystem() const
std::optional<StorePath> PackageInfo::queryDrvPath() const
{
if (!drvPath && attrs) {
if (auto i = attrs->get(state->sDrvPath)) {
if (auto i = attrs->get(state->s.drvPath)) {
NixStringContext context;
auto found = state->coerceToStorePath(
i->pos, *i->value, context, "while evaluating the 'drvPath' attribute of a derivation");
@ -95,7 +95,7 @@ StorePath PackageInfo::requireDrvPath() const
StorePath PackageInfo::queryOutPath() const
{
if (!outPath && attrs) {
auto i = attrs->find(state->sOutPath);
auto i = attrs->find(state->s.outPath);
NixStringContext context;
if (i != attrs->end())
outPath = state->coerceToStorePath(
@ -111,7 +111,7 @@ PackageInfo::Outputs PackageInfo::queryOutputs(bool withPaths, bool onlyOutputsT
if (outputs.empty()) {
/* Get the outputs list. */
const Attr * i;
if (attrs && (i = attrs->get(state->sOutputs))) {
if (attrs && (i = attrs->get(state->s.outputs))) {
state->forceList(*i->value, i->pos, "while evaluating the 'outputs' attribute of a derivation");
/* For each output... */
@ -127,7 +127,7 @@ PackageInfo::Outputs PackageInfo::queryOutputs(bool withPaths, bool onlyOutputsT
state->forceAttrs(*out->value, i->pos, "while evaluating an output of a derivation");
/* And evaluate its outPath attribute. */
auto outPath = out->value->attrs()->get(state->sOutPath);
auto outPath = out->value->attrs()->get(state->s.outPath);
if (!outPath)
continue; // FIXME: throw error?
NixStringContext context;
@ -146,7 +146,7 @@ PackageInfo::Outputs PackageInfo::queryOutputs(bool withPaths, bool onlyOutputsT
return outputs;
const Attr * i;
if (attrs && (i = attrs->get(state->sOutputSpecified))
if (attrs && (i = attrs->get(state->s.outputSpecified))
&& state->forceBool(*i->value, i->pos, "while evaluating the 'outputSpecified' attribute of a derivation")) {
Outputs result;
auto out = outputs.find(queryOutputName());
@ -181,7 +181,7 @@ PackageInfo::Outputs PackageInfo::queryOutputs(bool withPaths, bool onlyOutputsT
std::string PackageInfo::queryOutputName() const
{
if (outputName == "" && attrs) {
auto i = attrs->get(state->sOutputName);
auto i = attrs->get(state->s.outputName);
outputName =
i ? state->forceStringNoCtx(*i->value, noPos, "while evaluating the output name of a derivation") : "";
}
@ -194,7 +194,7 @@ const Bindings * PackageInfo::getMeta()
return meta;
if (!attrs)
return 0;
auto a = attrs->get(state->sMeta);
auto a = attrs->get(state->s.meta);
if (!a)
return 0;
state->forceAttrs(*a->value, a->pos, "while evaluating the 'meta' attribute of a derivation");
@ -221,7 +221,7 @@ bool PackageInfo::checkMeta(Value & v)
return false;
return true;
} else if (v.type() == nAttrs) {
if (v.attrs()->get(state->sOutPath))
if (v.attrs()->get(state->s.outPath))
return false;
for (auto & i : *v.attrs())
if (!checkMeta(*i.value))
@ -411,7 +411,7 @@ static void getDerivations(
should we recurse into it? => Only if it has a
`recurseForDerivations = true' attribute. */
if (i->value->type() == nAttrs) {
auto j = i->value->attrs()->get(state.sRecurseForDerivations);
auto j = i->value->attrs()->get(state.s.recurseForDerivations);
if (j
&& state.forceBool(
*j->value, j->pos, "while evaluating the attribute `recurseForDerivations`"))

View file

@ -213,23 +213,100 @@ struct DebugTrace
}
};
struct StaticEvalSymbols
{
Symbol with, outPath, drvPath, type, meta, name, value, system, overrides, outputs, outputName, ignoreNulls, file,
line, column, functor, toString, right, wrong, structuredAttrs, json, allowedReferences, allowedRequisites,
disallowedReferences, disallowedRequisites, maxSize, maxClosureSize, builder, args, contentAddressed, impure,
outputHash, outputHashAlgo, outputHashMode, recurseForDerivations, description, self, epsilon, startSet,
operator_, key, path, prefix, outputSpecified;
Expr::AstSymbols exprSymbols;
static constexpr auto preallocate()
{
StaticSymbolTable alloc;
StaticEvalSymbols staticSymbols = {
.with = alloc.create("<with>"),
.outPath = alloc.create("outPath"),
.drvPath = alloc.create("drvPath"),
.type = alloc.create("type"),
.meta = alloc.create("meta"),
.name = alloc.create("name"),
.value = alloc.create("value"),
.system = alloc.create("system"),
.overrides = alloc.create("__overrides"),
.outputs = alloc.create("outputs"),
.outputName = alloc.create("outputName"),
.ignoreNulls = alloc.create("__ignoreNulls"),
.file = alloc.create("file"),
.line = alloc.create("line"),
.column = alloc.create("column"),
.functor = alloc.create("__functor"),
.toString = alloc.create("__toString"),
.right = alloc.create("right"),
.wrong = alloc.create("wrong"),
.structuredAttrs = alloc.create("__structuredAttrs"),
.json = alloc.create("__json"),
.allowedReferences = alloc.create("allowedReferences"),
.allowedRequisites = alloc.create("allowedRequisites"),
.disallowedReferences = alloc.create("disallowedReferences"),
.disallowedRequisites = alloc.create("disallowedRequisites"),
.maxSize = alloc.create("maxSize"),
.maxClosureSize = alloc.create("maxClosureSize"),
.builder = alloc.create("builder"),
.args = alloc.create("args"),
.contentAddressed = alloc.create("__contentAddressed"),
.impure = alloc.create("__impure"),
.outputHash = alloc.create("outputHash"),
.outputHashAlgo = alloc.create("outputHashAlgo"),
.outputHashMode = alloc.create("outputHashMode"),
.recurseForDerivations = alloc.create("recurseForDerivations"),
.description = alloc.create("description"),
.self = alloc.create("self"),
.epsilon = alloc.create(""),
.startSet = alloc.create("startSet"),
.operator_ = alloc.create("operator"),
.key = alloc.create("key"),
.path = alloc.create("path"),
.prefix = alloc.create("prefix"),
.outputSpecified = alloc.create("outputSpecified"),
.exprSymbols = {
.sub = alloc.create("__sub"),
.lessThan = alloc.create("__lessThan"),
.mul = alloc.create("__mul"),
.div = alloc.create("__div"),
.or_ = alloc.create("or"),
.findFile = alloc.create("__findFile"),
.nixPath = alloc.create("__nixPath"),
.body = alloc.create("body"),
}};
return std::pair{staticSymbols, alloc};
}
static consteval StaticEvalSymbols create()
{
return preallocate().first;
}
static constexpr StaticSymbolTable staticSymbolTable()
{
return preallocate().second;
}
};
class EvalState : public std::enable_shared_from_this<EvalState>
{
public:
static constexpr StaticEvalSymbols s = StaticEvalSymbols::create();
const fetchers::Settings & fetchSettings;
const EvalSettings & settings;
SymbolTable symbols;
PosTable positions;
const Symbol sWith, sOutPath, sDrvPath, sType, sMeta, sName, sValue, sSystem, sOverrides, sOutputs, sOutputName,
sIgnoreNulls, sFile, sLine, sColumn, sFunctor, sToString, sRight, sWrong, sStructuredAttrs, sJson,
sAllowedReferences, sAllowedRequisites, sDisallowedReferences, sDisallowedRequisites, sMaxSize, sMaxClosureSize,
sBuilder, sArgs, sContentAddressed, sImpure, sOutputHash, sOutputHashAlgo, sOutputHashMode,
sRecurseForDerivations, sDescription, sSelf, sEpsilon, sStartSet, sOperator, sKey, sPath, sPrefix,
sOutputSpecified;
const Expr::AstSymbols exprSymbols;
/**
* If set, force copying files to the Nix store even if they
* already exist there.

View file

@ -88,7 +88,7 @@ struct ParserState
SourcePath basePath;
PosTable::Origin origin;
const ref<SourceAccessor> rootFS;
const Expr::AstSymbols & s;
static constexpr Expr::AstSymbols s = StaticEvalSymbols::create().exprSymbols;
const EvalSettings & settings;
void dupAttr(const AttrPath & attrPath, const PosIdx pos, const PosIdx prevPos);

View file

@ -28,6 +28,8 @@ public:
}
};
class StaticSymbolTable;
/**
* Symbols have the property that they can be compared efficiently
* (using an equality test), because the symbol table stores only one
@ -37,36 +39,29 @@ class Symbol
{
friend class SymbolStr;
friend class SymbolTable;
friend class StaticSymbolTable;
private:
uint32_t id;
explicit Symbol(uint32_t id) noexcept
explicit constexpr Symbol(uint32_t id) noexcept
: id(id)
{
}
public:
Symbol() noexcept
constexpr Symbol() noexcept
: id(0)
{
}
[[gnu::always_inline]]
explicit operator bool() const noexcept
constexpr explicit operator bool() const noexcept
{
return id > 0;
}
auto operator<=>(const Symbol other) const noexcept
{
return id <=> other.id;
}
bool operator==(const Symbol other) const noexcept
{
return id == other.id;
}
constexpr auto operator<=>(const Symbol & other) const noexcept = default;
friend class std::hash<Symbol>;
};
@ -210,6 +205,39 @@ public:
};
};
class SymbolTable;
/**
* Convenience class to statically assign symbol identifiers at compile-time.
*/
class StaticSymbolTable
{
static constexpr std::size_t maxSize = 1024;
struct StaticSymbolInfo
{
std::string_view str;
Symbol sym;
};
std::array<StaticSymbolInfo, maxSize> symbols;
std::size_t size = 0;
public:
constexpr StaticSymbolTable() = default;
constexpr Symbol create(std::string_view str)
{
/* No need to check bounds because out of bounds access is
a compilation error. */
auto sym = Symbol(size + 1); //< +1 because Symbol with id = 0 is reserved
symbols[size++] = {str, sym};
return sym;
}
void copyIntoSymbolTable(SymbolTable & symtab) const;
};
/**
* Symbol table used by the parser and evaluator to represent and look
* up identifiers and attributes efficiently.
@ -232,6 +260,10 @@ private:
boost::unordered_flat_set<SymbolStr, SymbolStr::Hash, SymbolStr::Equal> symbols{SymbolStr::chunkSize};
public:
SymbolTable(const StaticSymbolTable & staticSymtab)
{
staticSymtab.copyIntoSymbolTable(*this);
}
/**
* Converts a string into a symbol.
@ -276,6 +308,16 @@ public:
}
};
inline void StaticSymbolTable::copyIntoSymbolTable(SymbolTable & symtab) const
{
for (std::size_t i = 0; i < size; ++i) {
auto [str, staticSym] = symbols[i];
auto sym = symtab.create(str);
if (sym != staticSym) [[unlikely]]
unreachable();
}
}
} // namespace nix
template<>

View file

@ -68,8 +68,7 @@ Expr * parseExprFromBuf(
const EvalSettings & settings,
PosTable & positions,
DocCommentMap & docComments,
const ref<SourceAccessor> rootFS,
const Expr::AstSymbols & astSymbols);
const ref<SourceAccessor> rootFS);
}
@ -542,8 +541,7 @@ Expr * parseExprFromBuf(
const EvalSettings & settings,
PosTable & positions,
DocCommentMap & docComments,
const ref<SourceAccessor> rootFS,
const Expr::AstSymbols & astSymbols)
const ref<SourceAccessor> rootFS)
{
yyscan_t scanner;
LexerState lexerState {
@ -558,7 +556,6 @@ Expr * parseExprFromBuf(
.basePath = basePath,
.origin = lexerState.origin,
.rootFS = rootFS,
.s = astSymbols,
.settings = settings,
};

View file

@ -214,20 +214,20 @@ void derivationToValue(
auto path2 = path.path.abs();
Derivation drv = state.store->readDerivation(storePath);
auto attrs = state.buildBindings(3 + drv.outputs.size());
attrs.alloc(state.sDrvPath)
attrs.alloc(state.s.drvPath)
.mkString(
path2,
{
NixStringContextElem::DrvDeep{.drvPath = storePath},
});
attrs.alloc(state.sName).mkString(drv.env["name"]);
attrs.alloc(state.s.name).mkString(drv.env["name"]);
auto list = state.buildList(drv.outputs.size());
for (const auto & [i, o] : enumerate(drv.outputs)) {
mkOutputString(state, attrs, storePath, o);
(list[i] = state.allocValue())->mkString(o.first);
}
attrs.alloc(state.sOutputs).mkList(list);
attrs.alloc(state.s.outputs).mkList(list);
auto w = state.allocValue();
w->mkAttrs(attrs);
@ -731,7 +731,7 @@ static void prim_genericClosure(EvalState & state, const PosIdx pos, Value ** ar
/* Get the start set. */
auto startSet = state.getAttr(
state.sStartSet, args[0]->attrs(), "in the attrset passed as argument to builtins.genericClosure");
state.s.startSet, args[0]->attrs(), "in the attrset passed as argument to builtins.genericClosure");
state.forceList(
*startSet->value,
@ -749,7 +749,7 @@ static void prim_genericClosure(EvalState & state, const PosIdx pos, Value ** ar
/* Get the operator. */
auto op = state.getAttr(
state.sOperator, args[0]->attrs(), "in the attrset passed as argument to builtins.genericClosure");
state.s.operator_, args[0]->attrs(), "in the attrset passed as argument to builtins.genericClosure");
state.forceFunction(
*op->value, noPos, "while evaluating the 'operator' attribute passed as argument to builtins.genericClosure");
@ -771,7 +771,7 @@ static void prim_genericClosure(EvalState & state, const PosIdx pos, Value ** ar
"while evaluating one of the elements generated by (or initially passed to) builtins.genericClosure");
auto key = state.getAttr(
state.sKey,
state.s.key,
e->attrs(),
"in one of the attrsets generated by (or initially passed to) builtins.genericClosure");
state.forceValue(*key->value, noPos);
@ -1076,11 +1076,11 @@ static void prim_tryEval(EvalState & state, const PosIdx pos, Value ** args, Val
try {
state.forceValue(*args[0], pos);
attrs.insert(state.sValue, args[0]);
attrs.insert(state.s.value, args[0]);
attrs.insert(state.symbols.create("success"), &state.vTrue);
} catch (AssertionError & e) {
// `value = false;` is unfortunate but removing it is a breaking change.
attrs.insert(state.sValue, &state.vFalse);
attrs.insert(state.s.value, &state.vFalse);
attrs.insert(state.symbols.create("success"), &state.vFalse);
}
@ -1292,7 +1292,8 @@ static void prim_derivationStrict(EvalState & state, const PosIdx pos, Value **
auto attrs = args[0]->attrs();
/* Figure out the name first (for stack backtraces). */
auto nameAttr = state.getAttr(state.sName, attrs, "in the attrset passed as argument to builtins.derivationStrict");
auto nameAttr =
state.getAttr(state.s.name, attrs, "in the attrset passed as argument to builtins.derivationStrict");
std::string_view drvName;
try {
@ -1366,7 +1367,7 @@ static void derivationStrictInternal(EvalState & state, std::string_view drvName
using nlohmann::json;
std::optional<StructuredAttrs> jsonObject;
auto pos = v.determinePos(noPos);
auto attr = attrs->find(state.sStructuredAttrs);
auto attr = attrs->find(state.s.structuredAttrs);
if (attr != attrs->end()
&& state.forceBool(
*attr->value,
@ -1377,7 +1378,7 @@ static void derivationStrictInternal(EvalState & state, std::string_view drvName
/* Check whether null attributes should be ignored. */
bool ignoreNulls = false;
attr = attrs->find(state.sIgnoreNulls);
attr = attrs->find(state.s.ignoreNulls);
if (attr != attrs->end())
ignoreNulls = state.forceBool(
*attr->value,
@ -1401,7 +1402,7 @@ static void derivationStrictInternal(EvalState & state, std::string_view drvName
outputs.insert("out");
for (auto & i : attrs->lexicographicOrder(state.symbols)) {
if (i->name == state.sIgnoreNulls)
if (i->name == state.s.ignoreNulls)
continue;
auto key = state.symbols[i->name];
vomit("processing attribute '%1%'", key);
@ -1453,19 +1454,19 @@ static void derivationStrictInternal(EvalState & state, std::string_view drvName
continue;
}
if (i->name == state.sContentAddressed && state.forceBool(*i->value, pos, context_below)) {
if (i->name == state.s.contentAddressed && state.forceBool(*i->value, pos, context_below)) {
contentAddressed = true;
experimentalFeatureSettings.require(Xp::CaDerivations);
}
else if (i->name == state.sImpure && state.forceBool(*i->value, pos, context_below)) {
else if (i->name == state.s.impure && state.forceBool(*i->value, pos, context_below)) {
isImpure = true;
experimentalFeatureSettings.require(Xp::ImpureDerivations);
}
/* The `args' attribute is special: it supplies the
command-line arguments to the builder. */
else if (i->name == state.sArgs) {
else if (i->name == state.s.args) {
state.forceList(*i->value, pos, context_below);
for (auto elem : i->value->listView()) {
auto s = state
@ -1482,22 +1483,22 @@ static void derivationStrictInternal(EvalState & state, std::string_view drvName
if (jsonObject) {
if (i->name == state.sStructuredAttrs)
if (i->name == state.s.structuredAttrs)
continue;
jsonObject->structuredAttrs.emplace(key, printValueAsJSON(state, true, *i->value, pos, context));
if (i->name == state.sBuilder)
if (i->name == state.s.builder)
drv.builder = state.forceString(*i->value, context, pos, context_below);
else if (i->name == state.sSystem)
else if (i->name == state.s.system)
drv.platform = state.forceStringNoCtx(*i->value, pos, context_below);
else if (i->name == state.sOutputHash)
else if (i->name == state.s.outputHash)
outputHash = state.forceStringNoCtx(*i->value, pos, context_below);
else if (i->name == state.sOutputHashAlgo)
else if (i->name == state.s.outputHashAlgo)
outputHashAlgo = parseHashAlgoOpt(state.forceStringNoCtx(*i->value, pos, context_below));
else if (i->name == state.sOutputHashMode)
else if (i->name == state.s.outputHashMode)
handleHashMode(state.forceStringNoCtx(*i->value, pos, context_below));
else if (i->name == state.sOutputs) {
else if (i->name == state.s.outputs) {
/* Require outputs to be a list of strings. */
state.forceList(*i->value, pos, context_below);
Strings ss;
@ -1506,51 +1507,51 @@ static void derivationStrictInternal(EvalState & state, std::string_view drvName
handleOutputs(ss);
}
if (i->name == state.sAllowedReferences)
if (i->name == state.s.allowedReferences)
warn(
"In a derivation named '%s', 'structuredAttrs' disables the effect of the derivation attribute 'allowedReferences'; use 'outputChecks.<output>.allowedReferences' instead",
drvName);
if (i->name == state.sAllowedRequisites)
if (i->name == state.s.allowedRequisites)
warn(
"In a derivation named '%s', 'structuredAttrs' disables the effect of the derivation attribute 'allowedRequisites'; use 'outputChecks.<output>.allowedRequisites' instead",
drvName);
if (i->name == state.sDisallowedReferences)
if (i->name == state.s.disallowedReferences)
warn(
"In a derivation named '%s', 'structuredAttrs' disables the effect of the derivation attribute 'disallowedReferences'; use 'outputChecks.<output>.disallowedReferences' instead",
drvName);
if (i->name == state.sDisallowedRequisites)
if (i->name == state.s.disallowedRequisites)
warn(
"In a derivation named '%s', 'structuredAttrs' disables the effect of the derivation attribute 'disallowedRequisites'; use 'outputChecks.<output>.disallowedRequisites' instead",
drvName);
if (i->name == state.sMaxSize)
if (i->name == state.s.maxSize)
warn(
"In a derivation named '%s', 'structuredAttrs' disables the effect of the derivation attribute 'maxSize'; use 'outputChecks.<output>.maxSize' instead",
drvName);
if (i->name == state.sMaxClosureSize)
if (i->name == state.s.maxClosureSize)
warn(
"In a derivation named '%s', 'structuredAttrs' disables the effect of the derivation attribute 'maxClosureSize'; use 'outputChecks.<output>.maxClosureSize' instead",
drvName);
} else {
auto s = state.coerceToString(pos, *i->value, context, context_below, true).toOwned();
if (i->name == state.sJson) {
if (i->name == state.s.json) {
warn(
"In derivation '%s': setting structured attributes via '__json' is deprecated, and may be disallowed in future versions of Nix. Set '__structuredAttrs = true' instead.",
drvName);
drv.structuredAttrs = StructuredAttrs::parse(s);
} else {
drv.env.emplace(key, s);
if (i->name == state.sBuilder)
if (i->name == state.s.builder)
drv.builder = std::move(s);
else if (i->name == state.sSystem)
else if (i->name == state.s.system)
drv.platform = std::move(s);
else if (i->name == state.sOutputHash)
else if (i->name == state.s.outputHash)
outputHash = std::move(s);
else if (i->name == state.sOutputHashAlgo)
else if (i->name == state.s.outputHashAlgo)
outputHashAlgo = parseHashAlgoOpt(s);
else if (i->name == state.sOutputHashMode)
else if (i->name == state.s.outputHashMode)
handleHashMode(s);
else if (i->name == state.sOutputs)
else if (i->name == state.s.outputs)
handleOutputs(tokenizeString<Strings>(s));
}
}
@ -1722,7 +1723,7 @@ static void derivationStrictInternal(EvalState & state, std::string_view drvName
}
auto result = state.buildBindings(1 + drv.outputs.size());
result.alloc(state.sDrvPath)
result.alloc(state.s.drvPath)
.mkString(
drvPathS,
{
@ -2006,14 +2007,14 @@ static void prim_findFile(EvalState & state, const PosIdx pos, Value ** args, Va
state.forceAttrs(*v2, pos, "while evaluating an element of the list passed to builtins.findFile");
std::string prefix;
auto i = v2->attrs()->find(state.sPrefix);
auto i = v2->attrs()->find(state.s.prefix);
if (i != v2->attrs()->end())
prefix = state.forceStringNoCtx(
*i->value,
pos,
"while evaluating the `prefix` attribute of an element of the list passed to builtins.findFile");
i = state.getAttr(state.sPath, v2->attrs(), "in an element of the __nixPath");
i = state.getAttr(state.s.path, v2->attrs(), "in an element of the __nixPath");
NixStringContext context;
auto path =
@ -2786,7 +2787,7 @@ static void prim_path(EvalState & state, const PosIdx pos, Value ** args, Value
if (n == "path")
path.emplace(state.coerceToPath(
attr.pos, *attr.value, context, "while evaluating the 'path' attribute passed to 'builtins.path'"));
else if (attr.name == state.sName)
else if (attr.name == state.s.name)
name = state.forceStringNoCtx(
*attr.value, attr.pos, "while evaluating the `name` attribute passed to builtins.path");
else if (n == "filter")
@ -3105,7 +3106,7 @@ static void prim_listToAttrs(EvalState & state, const PosIdx pos, Value ** args,
for (const auto & [n, v2] : enumerate(listView)) {
state.forceAttrs(*v2, pos, "while evaluating an element of the list passed to builtins.listToAttrs");
auto j = state.getAttr(state.sName, v2->attrs(), "in a {name=...; value=...;} pair");
auto j = state.getAttr(state.s.name, v2->attrs(), "in a {name=...; value=...;} pair");
auto name = state.forceStringNoCtx(
*j->value,
@ -3132,7 +3133,7 @@ static void prim_listToAttrs(EvalState & state, const PosIdx pos, Value ** args,
// Note that .value is actually a Value * *; see earlier comments
Value * v2 = *std::bit_cast<ElemPtr>(attr.value);
auto j = state.getAttr(state.sValue, v2->attrs(), "in a {name=...; value=...;} pair");
auto j = state.getAttr(state.s.value, v2->attrs(), "in a {name=...; value=...;} pair");
prev = attr.name;
bindings.push_back({prev, j->value, j->pos});
}
@ -3948,13 +3949,13 @@ static void prim_partition(EvalState & state, const PosIdx pos, Value ** args, V
auto rlist = state.buildList(rsize);
if (rsize)
memcpy(rlist.elems, right.data(), sizeof(Value *) * rsize);
attrs.alloc(state.sRight).mkList(rlist);
attrs.alloc(state.s.right).mkList(rlist);
auto wsize = wrong.size();
auto wlist = state.buildList(wsize);
if (wsize)
memcpy(wlist.elems, wrong.data(), sizeof(Value *) * wsize);
attrs.alloc(state.sWrong).mkList(wlist);
attrs.alloc(state.s.wrong).mkList(wlist);
v.mkAttrs(attrs);
}
@ -4873,7 +4874,7 @@ static void prim_parseDrvName(EvalState & state, const PosIdx pos, Value ** args
state.forceStringNoCtx(*args[0], pos, "while evaluating the first argument passed to builtins.parseDrvName");
DrvName parsed(name);
auto attrs = state.buildBindings(2);
attrs.alloc(state.sName).mkString(parsed.name);
attrs.alloc(state.s.name).mkString(parsed.name);
attrs.alloc("version").mkString(parsed.version);
v.mkAttrs(attrs);
}

View file

@ -219,7 +219,7 @@ static void prim_getContext(EvalState & state, const PosIdx pos, Value ** args,
auto list = state.buildList(info.second.outputs.size());
for (const auto & [i, output] : enumerate(info.second.outputs))
(list[i] = state.allocValue())->mkString(output);
infoAttrs.alloc(state.sOutputs).mkList(list);
infoAttrs.alloc(state.s.outputs).mkList(list);
}
attrs.alloc(state.store->printStorePath(info.first)).mkAttrs(infoAttrs);
}
@ -300,7 +300,7 @@ static void prim_appendContext(EvalState & state, const PosIdx pos, Value ** arg
}
}
if (auto attr = i.value->attrs()->get(state.sOutputs)) {
if (auto attr = i.value->attrs()->get(state.s.outputs)) {
state.forceList(*attr->value, attr->pos, "while evaluating the `outputs` attribute of a string context");
if (attr->value->listSize() && !isDerivation(name)) {
state

View file

@ -84,7 +84,7 @@ static void prim_fetchMercurial(EvalState & state, const PosIdx pos, Value ** ar
auto [storePath, input2] = input.fetchToStore(state.store);
auto attrs2 = state.buildBindings(8);
state.mkStorePathString(storePath, attrs2.alloc(state.sOutPath));
state.mkStorePathString(storePath, attrs2.alloc(state.s.outPath));
if (input2.getRef())
attrs2.alloc("branch").mkString(*input2.getRef());
// Backward compatibility: set 'rev' to

View file

@ -29,7 +29,7 @@ void emitTreeAttrs(
{
auto attrs = state.buildBindings(100);
state.mkStorePathString(storePath, attrs.alloc(state.sOutPath));
state.mkStorePathString(storePath, attrs.alloc(state.s.outPath));
// FIXME: support arbitrary input attributes.
@ -95,7 +95,7 @@ static void fetchTree(
fetchers::Attrs attrs;
if (auto aType = args[0]->attrs()->get(state.sType)) {
if (auto aType = args[0]->attrs()->get(state.s.type)) {
if (type)
state.error<EvalError>("unexpected argument 'type'").atPos(pos).debugThrow();
type = state.forceStringNoCtx(
@ -106,7 +106,7 @@ static void fetchTree(
attrs.emplace("type", type.value());
for (auto & attr : *args[0]->attrs()) {
if (attr.name == state.sType)
if (attr.name == state.s.type)
continue;
state.forceValue(*attr.value, attr.pos);
if (attr.value->type() == nPath || attr.value->type() == nString) {

View file

@ -272,7 +272,7 @@ private:
void printDerivation(Value & v)
{
std::optional<StorePath> storePath;
if (auto i = v.attrs()->get(state.sDrvPath)) {
if (auto i = v.attrs()->get(state.s.drvPath)) {
NixStringContext context;
storePath =
state.coerceToStorePath(i->pos, *i->value, context, "while evaluating the drvPath of a derivation");

View file

@ -53,7 +53,7 @@ json printValueAsJSON(
out = *maybeString;
break;
}
if (auto i = v.attrs()->get(state.sOutPath))
if (auto i = v.attrs()->get(state.s.outPath))
return printValueAsJSON(state, strict, *i->value, i->pos, context, copyToStore);
else {
out = json::object();

View file

@ -98,14 +98,14 @@ static void printValueAsXML(
XMLAttrs xmlAttrs;
Path drvPath;
if (auto a = v.attrs()->get(state.sDrvPath)) {
if (auto a = v.attrs()->get(state.s.drvPath)) {
if (strict)
state.forceValue(*a->value, a->pos);
if (a->value->type() == nString)
xmlAttrs["drvPath"] = drvPath = a->value->c_str();
}
if (auto a = v.attrs()->get(state.sOutPath)) {
if (auto a = v.attrs()->get(state.s.outPath)) {
if (strict)
state.forceValue(*a->value, a->pos);
if (a->value->type() == nString)

View file

@ -232,7 +232,7 @@ static Flake readFlake(
.path = flakePath,
};
if (auto description = vInfo.attrs()->get(state.sDescription)) {
if (auto description = vInfo.attrs()->get(state.s.description)) {
expectType(state, nString, *description->value, description->pos);
flake.description = description->value->c_str();
}
@ -253,7 +253,7 @@ static Flake readFlake(
if (outputs->value->isLambda() && outputs->value->lambda().fun->hasFormals()) {
for (auto & formal : outputs->value->lambda().fun->formals->formals) {
if (formal.name != state.sSelf)
if (formal.name != state.s.self)
flake.inputs.emplace(
state.symbols[formal.name],
FlakeInput{.ref = parseFlakeRef(state.fetchSettings, std::string(state.symbols[formal.name]))});
@ -305,7 +305,8 @@ static Flake readFlake(
}
for (auto & attr : *vInfo.attrs()) {
if (attr.name != state.sDescription && attr.name != sInputs && attr.name != sOutputs && attr.name != sNixConfig)
if (attr.name != state.s.description && attr.name != sInputs && attr.name != sOutputs
&& attr.name != sNixConfig)
throw Error(
"flake '%s' has an unsupported attribute '%s', at %s",
resolvedRef,

View file

@ -103,11 +103,11 @@ UnresolvedApp InstallableValue::toApp(EvalState & state)
else if (type == "derivation") {
auto drvPath = cursor->forceDerivation();
auto outPath = cursor->getAttr(state.sOutPath)->getString();
auto outputName = cursor->getAttr(state.sOutputName)->getString();
auto name = cursor->getAttr(state.sName)->getString();
auto outPath = cursor->getAttr(state.s.outPath)->getString();
auto outputName = cursor->getAttr(state.s.outputName)->getString();
auto name = cursor->getAttr(state.s.name)->getString();
auto aPname = cursor->maybeGetAttr("pname");
auto aMeta = cursor->maybeGetAttr(state.sMeta);
auto aMeta = cursor->maybeGetAttr(state.s.meta);
auto aMainProgram = aMeta ? aMeta->maybeGetAttr("mainProgram") : nullptr;
auto mainProgram = aMainProgram ? aMainProgram->getString() : aPname ? aPname->getString() : DrvName(name).name;
auto program = outPath + "/bin/" + mainProgram;

View file

@ -100,7 +100,7 @@ struct CmdBundle : InstallableValueCommand
if (!evalState->isDerivation(*vRes))
throw Error("the bundler '%s' does not produce a derivation", bundler.what());
auto attr1 = vRes->attrs()->get(evalState->sDrvPath);
auto attr1 = vRes->attrs()->get(evalState->s.drvPath);
if (!attr1)
throw Error("the bundler '%s' does not produce a derivation", bundler.what());
@ -109,7 +109,7 @@ struct CmdBundle : InstallableValueCommand
drvPath.requireDerivation();
auto attr2 = vRes->attrs()->get(evalState->sOutPath);
auto attr2 = vRes->attrs()->get(evalState->s.outPath);
if (!attr2)
throw Error("the bundler '%s' does not produce a derivation", bundler.what());
@ -123,7 +123,7 @@ struct CmdBundle : InstallableValueCommand
});
if (!outLink) {
auto * attr = vRes->attrs()->get(evalState->sName);
auto * attr = vRes->attrs()->get(evalState->s.name);
if (!attr)
throw Error("attribute 'name' missing");
outLink = evalState->forceStringNoCtx(*attr->value, attr->pos, "");

View file

@ -1232,12 +1232,12 @@ struct CmdFlakeShow : FlakeCommand, MixJSON
};
auto showDerivation = [&]() {
auto name = visitor.getAttr(state->sName)->getString();
auto name = visitor.getAttr(state->s.name)->getString();
if (json) {
std::optional<std::string> description;
if (auto aMeta = visitor.maybeGetAttr(state->sMeta)) {
if (auto aDescription = aMeta->maybeGetAttr(state->sDescription))
if (auto aMeta = visitor.maybeGetAttr(state->s.meta)) {
if (auto aDescription = aMeta->maybeGetAttr(state->s.description))
description = aDescription->getString();
}
j.emplace("type", "derivation");
@ -1365,8 +1365,8 @@ struct CmdFlakeShow : FlakeCommand, MixJSON
|| (attrPath.size() == 3 && attrPathS[0] == "apps")) {
auto aType = visitor.maybeGetAttr("type");
std::optional<std::string> description;
if (auto aMeta = visitor.maybeGetAttr(state->sMeta)) {
if (auto aDescription = aMeta->maybeGetAttr(state->sDescription))
if (auto aMeta = visitor.maybeGetAttr(state->s.meta)) {
if (auto aDescription = aMeta->maybeGetAttr(state->s.description))
description = aDescription->getString();
}
if (!aType || aType->getString() != "app")

View file

@ -56,21 +56,21 @@ bool createUserEnv(
auto attrs = state.buildBindings(7 + outputs.size());
attrs.alloc(state.sType).mkString("derivation");
attrs.alloc(state.sName).mkString(i.queryName());
attrs.alloc(state.s.type).mkString("derivation");
attrs.alloc(state.s.name).mkString(i.queryName());
auto system = i.querySystem();
if (!system.empty())
attrs.alloc(state.sSystem).mkString(system);
attrs.alloc(state.sOutPath).mkString(state.store->printStorePath(i.queryOutPath()));
attrs.alloc(state.s.system).mkString(system);
attrs.alloc(state.s.outPath).mkString(state.store->printStorePath(i.queryOutPath()));
if (drvPath)
attrs.alloc(state.sDrvPath).mkString(state.store->printStorePath(*drvPath));
attrs.alloc(state.s.drvPath).mkString(state.store->printStorePath(*drvPath));
// Copy each output meant for installation.
auto outputsList = state.buildList(outputs.size());
for (const auto & [m, j] : enumerate(outputs)) {
(outputsList[m] = state.allocValue())->mkString(j.first);
auto outputAttrs = state.buildBindings(2);
outputAttrs.alloc(state.sOutPath).mkString(state.store->printStorePath(*j.second));
outputAttrs.alloc(state.s.outPath).mkString(state.store->printStorePath(*j.second));
attrs.alloc(j.first).mkAttrs(outputAttrs);
/* This is only necessary when installing store paths, e.g.,
@ -80,7 +80,7 @@ bool createUserEnv(
references.insert(*j.second);
}
attrs.alloc(state.sOutputs).mkList(outputsList);
attrs.alloc(state.s.outputs).mkList(outputsList);
// Copy the meta attributes.
auto meta = state.buildBindings(metaNames.size());
@ -91,7 +91,7 @@ bool createUserEnv(
meta.insert(state.symbols.create(j), v);
}
attrs.alloc(state.sMeta).mkAttrs(meta);
attrs.alloc(state.s.meta).mkAttrs(meta);
(list[n] = state.allocValue())->mkAttrs(attrs);
@ -141,10 +141,10 @@ bool createUserEnv(
debug("evaluating user environment builder");
state.forceValue(topLevel, topLevel.determinePos(noPos));
NixStringContext context;
auto & aDrvPath(*topLevel.attrs()->find(state.sDrvPath));
auto & aDrvPath(*topLevel.attrs()->find(state.s.drvPath));
auto topLevelDrv = state.coerceToStorePath(aDrvPath.pos, *aDrvPath.value, context, "");
topLevelDrv.requireDerivation();
auto & aOutPath(*topLevel.attrs()->find(state.sOutPath));
auto & aOutPath(*topLevel.attrs()->find(state.s.outPath));
auto topLevelOut = state.coerceToStorePath(aOutPath.pos, *aOutPath.value, context, "");
/* Realise the resulting store expression. */

View file

@ -108,10 +108,10 @@ struct CmdSearch : InstallableValueCommand, MixJSON
};
if (cursor.isDerivation()) {
DrvName name(cursor.getAttr(state->sName)->getString());
DrvName name(cursor.getAttr(state->s.name)->getString());
auto aMeta = cursor.maybeGetAttr(state->sMeta);
auto aDescription = aMeta ? aMeta->maybeGetAttr(state->sDescription) : nullptr;
auto aMeta = cursor.maybeGetAttr(state->s.meta);
auto aDescription = aMeta ? aMeta->maybeGetAttr(state->s.description) : nullptr;
auto description = aDescription ? aDescription->getString() : "";
std::replace(description.begin(), description.end(), '\n', ' ');
auto attrPath2 = concatStringsSep(".", attrPathS);
@ -176,7 +176,7 @@ struct CmdSearch : InstallableValueCommand, MixJSON
recurse();
else if (attrPathS[0] == "legacyPackages" && attrPath.size() > 2) {
auto attr = cursor.maybeGetAttr(state->sRecurseForDerivations);
auto attr = cursor.maybeGetAttr(state->s.recurseForDerivations);
if (attr && attr->getBool())
recurse();
}