Collapse a job's "parent stuff" into a new type job_lineage_t

Currently a job needs to know three things about its "parents:"

1. Any IO redirections for the block or function containing this job
2. The pgid for the parent job
3. Whether the parent job has been fully constructed (to defer self-disown)

These are all tracked in somewhat separate awkward ways. Collapse them
into a single new type job_lineage_t.
This commit is contained in:
ridiculousfish 2019-12-08 13:03:42 -08:00
parent e3b8203187
commit f136d634eb
7 changed files with 79 additions and 76 deletions

View File

@ -246,28 +246,27 @@ static bool resolve_file_redirections_to_fds(const io_chain_t &in_chain, const w
return success;
}
/// Morph an io redirection chain into redirections suitable for passing to eval, call eval, and
/// clean up morphed redirections.
/// Morph an io redirection chain into redirections suitable for passing to eval, and then call
/// eval.
///
/// \param parsed_source the parsed source code containing the node to evaluate
/// \param node the node to evaluate
/// \param ios the io redirections to be performed on this block
template <typename T>
void internal_exec_helper(parser_t &parser, parsed_source_ref_t parsed_source, tnode_t<T> node,
const io_chain_t &ios, std::shared_ptr<job_t> parent_job) {
job_lineage_t lineage) {
assert(parsed_source && node && "exec_helper missing source or without node");
io_chain_t morphed_chain;
std::vector<autoclose_fd_t> opened_fds;
if (!resolve_file_redirections_to_fds(ios, parser.vars().get_pwd_slash(), &morphed_chain,
&opened_fds)) {
if (!resolve_file_redirections_to_fds(lineage.block_io, parser.vars().get_pwd_slash(),
&morphed_chain, &opened_fds)) {
parser.set_last_statuses(statuses_t::just(STATUS_EXEC_FAIL));
return;
}
parser.eval_node(parsed_source, node, morphed_chain, TOP, parent_job);
morphed_chain.clear();
lineage.block_io = std::move(morphed_chain);
parser.eval_node(parsed_source, node, TOP, std::move(lineage));
job_reap(parser, false);
}
@ -817,23 +816,28 @@ static void function_restore_environment(parser_t &parser, const block_t *block)
// This accepts a place to execute as \p parser, and a parent job as \p parent, and then executes
// the result, returning a status.
// This is factored out in this funny way in preparation for concurrent execution.
using proc_performer_t =
std::function<proc_status_t(parser_t &parser, std::shared_ptr<job_t> parent)>;
using proc_performer_t = std::function<proc_status_t(parser_t &parser)>;
// \return a function which may be to run the given process \p.
// May return an empty std::function in the rare case that the to-be called fish function no longer
// exists. This is just a dumb artifact of the fact that we only capture the functions name, not its
// properties, when creating the job; thus a race could delete the function before we fetch its
// properties.
static proc_performer_t get_performer_for_process(process_t *p, const io_chain_t &io_chain) {
static proc_performer_t get_performer_for_process(process_t *p, const job_t *job,
const io_chain_t &io_chain) {
assert((p->type == process_type_t::function || p->type == process_type_t::block_node) &&
"Unexpected process type");
// Construct a lineage, starting from the job's lineage.
job_lineage_t lineage = job->lineage();
lineage.block_io = io_chain;
lineage.parent_pgid = (job->pgid == INVALID_PID ? none() : maybe_t<pid_t>(job->pgid));
if (p->type == process_type_t::block_node) {
const parsed_source_ref_t &source = p->block_node_source;
tnode_t<grammar::statement> node = p->internal_block_node;
assert(source && node && "Process is missing node info");
return [=](parser_t &parser, std::shared_ptr<job_t> parent) {
internal_exec_helper(parser, source, node, io_chain, parent);
return [=](parser_t &parser) {
internal_exec_helper(parser, source, node, lineage);
int status = parser.get_last_status();
// FIXME: setting the status this way is dangerous nonsense, we need to decode the
// status properly if it was a signal.
@ -847,11 +851,11 @@ static proc_performer_t get_performer_for_process(process_t *p, const io_chain_t
return proc_performer_t{};
}
auto argv = move_to_sharedptr(p->get_argv_array().to_list());
return [=](parser_t &parser, std::shared_ptr<job_t> parent) {
return [=](parser_t &parser) {
const auto &ld = parser.libdata();
auto saved_exec_count = ld.exec_count;
const block_t *fb = function_prepare_environment(parser, *argv, *props);
internal_exec_helper(parser, props->parsed_source, props->body_node, io_chain, parent);
internal_exec_helper(parser, props->parsed_source, props->body_node, lineage);
function_restore_environment(parser, fb);
// If the function did not execute anything, treat it as success.
@ -893,8 +897,8 @@ static bool exec_block_or_func_process(parser_t &parser, std::shared_ptr<job_t>
// Get the process performer, and just execute it directly.
// Do it in this scoped way so that the performer function can be eagerly deallocating releasing
// its captured io chain.
if (proc_performer_t performer = get_performer_for_process(p, io_chain)) {
p->status = performer(parser, j);
if (proc_performer_t performer = get_performer_for_process(p, j.get(), io_chain)) {
p->status = performer(parser);
} else {
return false;
}
@ -1130,11 +1134,10 @@ bool exec_job(parser_t &parser, shared_ptr<job_t> j) {
// Check to see if we should reclaim the foreground pgrp after the job finishes or stops.
const bool reclaim_foreground_pgrp = (tcgetpgrp(STDIN_FILENO) == getpgrp());
const std::shared_ptr<job_t> parent_job = j->get_parent();
// Perhaps inherit our parent's pgid and job control flag.
if (parent_job && parent_job->pgid != INVALID_PID) {
j->pgid = parent_job->pgid;
// If our lineage indicates a pgid, share it.
if (auto parent_pgid = j->lineage().parent_pgid) {
assert(*parent_pgid != INVALID_PID && "parent pgid should be none, not INVALID_PID");
j->pgid = *parent_pgid;
j->mut_flags().job_control = true;
}

View File

@ -87,8 +87,8 @@ static std::shared_ptr<io_data_t> get_stderr_merge() {
}
parse_execution_context_t::parse_execution_context_t(parsed_source_ref_t pstree, parser_t *p,
std::shared_ptr<job_t> parent)
: pstree(std::move(pstree)), parser(p), parent_job(std::move(parent)) {}
job_lineage_t lineage)
: pstree(std::move(pstree)), parser(p), lineage(std::move(lineage)) {}
// Utilities
@ -1306,7 +1306,7 @@ parse_execution_result_t parse_execution_context_t::run_1_job(tnode_t<g::job> jo
ld.is_subshell || ld.is_block || ld.is_event || !parser->is_interactive();
props.from_event_handler = ld.is_event;
shared_ptr<job_t> job = std::make_shared<job_t>(acquire_job_id(), props, block_io, parent_job);
shared_ptr<job_t> job = std::make_shared<job_t>(acquire_job_id(), props, this->lineage);
job->tmodes = tmodes;
job->mut_flags().foreground = !job_node_is_background(job_node);
@ -1434,12 +1434,9 @@ parse_execution_result_t parse_execution_context_t::run_job_list(tnode_t<Type> j
}
parse_execution_result_t parse_execution_context_t::eval_node(tnode_t<g::statement> statement,
const block_t *associated_block,
const io_chain_t &io) {
const block_t *associated_block) {
assert(statement && "Empty node in eval_node");
assert(statement.matches_node_tree(tree()) && "statement has unexpected tree");
// Apply this block IO for the duration of this function.
scoped_push<io_chain_t> block_io_push(&block_io, io);
enum parse_execution_result_t status = parse_execution_success;
if (auto block = statement.try_get_child<g::block_statement, 0>()) {
status = this->run_block_statement(block, associated_block);
@ -1456,13 +1453,11 @@ parse_execution_result_t parse_execution_context_t::eval_node(tnode_t<g::stateme
}
parse_execution_result_t parse_execution_context_t::eval_node(tnode_t<g::job_list> job_list,
const block_t *associated_block,
const io_chain_t &io) {
const block_t *associated_block) {
// Apply this block IO for the duration of this function.
assert(job_list && "Empty node in eval_node");
assert(job_list.matches_node_tree(tree()) && "job_list has unexpected tree");
assert(associated_block && "Null block");
scoped_push<io_chain_t> block_io_push(&block_io, io);
// Check for infinite recursion: a function which immediately calls itself..
wcstring func_name;

View File

@ -29,15 +29,14 @@ enum parse_execution_result_t {
class parse_execution_context_t {
private:
parsed_source_ref_t pstree;
io_chain_t block_io;
parser_t *const parser;
// The currently executing job node, used to indicate the line number.
tnode_t<grammar::job> executing_job_node{};
// Cached line number information.
size_t cached_lineno_offset = 0;
int cached_lineno_count = 0;
// The parent job for any jobs created by this context.
const std::shared_ptr<job_t> parent_job;
// The lineage for any jobs created by this context.
const job_lineage_t lineage;
// No copying allowed.
parse_execution_context_t(const parse_execution_context_t &) = delete;
parse_execution_context_t &operator=(const parse_execution_context_t &) = delete;
@ -144,8 +143,7 @@ class parse_execution_context_t {
int line_offset_of_character_at_offset(size_t offset);
public:
parse_execution_context_t(parsed_source_ref_t pstree, parser_t *p,
std::shared_ptr<job_t> parent);
parse_execution_context_t(parsed_source_ref_t pstree, parser_t *p, job_lineage_t lineage);
/// Returns the current line number, indexed from 1. Not const since it touches
/// cached_lineno_offset.
@ -163,9 +161,9 @@ class parse_execution_context_t {
/// Start executing at the given node. Returns 0 if there was no error, 1 if there was an
/// error.
parse_execution_result_t eval_node(tnode_t<grammar::statement> statement,
const block_t *associated_block, const io_chain_t &io);
const block_t *associated_block);
parse_execution_result_t eval_node(tnode_t<grammar::job_list> job_list,
const block_t *associated_block, const io_chain_t &io);
const block_t *associated_block);
};
#endif

View File

@ -635,16 +635,18 @@ int parser_t::eval(wcstring cmd, const io_chain_t &io, enum block_type_t block_t
int parser_t::eval(parsed_source_ref_t ps, const io_chain_t &io, enum block_type_t block_type) {
assert(block_type == TOP || block_type == SUBST);
if (!ps->tree.empty()) {
job_lineage_t lineage;
lineage.block_io = io;
// Execute the first node.
tnode_t<grammar::job_list> start{&ps->tree, &ps->tree.front()};
return this->eval_node(ps, start, io, block_type, nullptr /* parent */);
return this->eval_node(ps, start, block_type, std::move(lineage));
}
return 0;
}
template <typename T>
int parser_t::eval_node(parsed_source_ref_t ps, tnode_t<T> node, const io_chain_t &io,
block_type_t block_type, std::shared_ptr<job_t> parent_job) {
int parser_t::eval_node(parsed_source_ref_t ps, tnode_t<T> node, block_type_t block_type,
job_lineage_t lineage) {
static_assert(
std::is_same<T, grammar::statement>::value || std::is_same<T, grammar::job_list>::value,
"Unexpected node type");
@ -672,9 +674,9 @@ int parser_t::eval_node(parsed_source_ref_t ps, tnode_t<T> node, const io_chain_
// Create and set a new execution context.
using exc_ctx_ref_t = std::unique_ptr<parse_execution_context_t>;
scoped_push<exc_ctx_ref_t> exc(&execution_context,
make_unique<parse_execution_context_t>(ps, this, parent_job));
int result = execution_context->eval_node(node, scope_block, io);
scoped_push<exc_ctx_ref_t> exc(
&execution_context, make_unique<parse_execution_context_t>(ps, this, std::move(lineage)));
int result = execution_context->eval_node(node, scope_block);
exc.restore();
this->pop_block(scope_block);
@ -684,11 +686,9 @@ int parser_t::eval_node(parsed_source_ref_t ps, tnode_t<T> node, const io_chain_
// Explicit instantiations. TODO: use overloads instead?
template int parser_t::eval_node(parsed_source_ref_t, tnode_t<grammar::statement>,
const io_chain_t &, enum block_type_t,
std::shared_ptr<job_t> parent_job);
template int parser_t::eval_node(parsed_source_ref_t, tnode_t<grammar::job_list>,
const io_chain_t &, enum block_type_t,
std::shared_ptr<job_t> parent_job);
enum block_type_t, job_lineage_t lineage);
template int parser_t::eval_node(parsed_source_ref_t, tnode_t<grammar::job_list>, enum block_type_t,
job_lineage_t lineage);
void parser_t::get_backtrace(const wcstring &src, const parse_error_list_t &errors,
wcstring &output) const {

View File

@ -265,8 +265,8 @@ class parser_t : public std::enable_shared_from_this<parser_t> {
/// Evaluates a node.
/// The node type must be grammar::statement or grammar::job_list.
template <typename T>
int eval_node(parsed_source_ref_t ps, tnode_t<T> node, const io_chain_t &io,
block_type_t block_type, std::shared_ptr<job_t> parent_job);
int eval_node(parsed_source_ref_t ps, tnode_t<T> node, block_type_t block_type,
job_lineage_t lineage);
/// Evaluate line as a list of parameters, i.e. tokenize it and perform parameter expansion and
/// cmdsubst execution on the tokens. Errors are ignored. If a parser is provided, it is used

View File

@ -168,14 +168,7 @@ bool job_t::should_report_process_exits() const {
return false;
}
bool job_t::job_chain_is_fully_constructed() const {
const job_t *cursor = this;
while (cursor) {
if (!cursor->is_constructed()) return false;
cursor = cursor->get_parent().get();
}
return true;
}
bool job_t::job_chain_is_fully_constructed() const { return *lineage().root_constructed; }
bool job_t::signal(int signal) {
// Presumably we are distinguishing between the two cases below because we do
@ -275,15 +268,14 @@ void process_t::check_generations_before_launch() {
gens_ = topic_monitor_t::principal().current_generations();
}
job_t::job_t(job_id_t job_id, const properties_t &props, io_chain_t bio,
std::shared_ptr<job_t> parent)
: properties(props), block_io(std::move(bio)), parent_job(std::move(parent)), job_id(job_id) {}
job_t::job_t(job_id_t job_id, const properties_t &props, job_lineage_t lineage)
: properties(props), job_lineage(std::move(lineage)), job_id(job_id) {}
job_t::~job_t() { release_job_id(job_id); }
/// Return all the IO redirections. Start with the block IO, then walk over the processes.
io_chain_t job_t::all_io_redirections() const {
io_chain_t result = this->block_io;
io_chain_t result = this->block_io_chain();
for (const process_ptr_t &p : this->processes) {
result.append(p->io_chain());
}

View File

@ -18,6 +18,7 @@
#include "common.h"
#include "event.h"
#include "global_safety.h"
#include "io.h"
#include "parse_tree.h"
#include "tnode.h"
@ -263,6 +264,25 @@ typedef int job_id_t;
job_id_t acquire_job_id(void);
void release_job_id(job_id_t jid);
/// Information about where a job comes from.
/// This should be safe to copy across threads; in particular that means this cannot contain a
/// job_t.
struct job_lineage_t {
/// The pgid of the parental job.
/// If our job is "nested" as part of a function or block execution, and that function or block
/// is part of a pipeline, then this may be set.
maybe_t<pid_t> parent_pgid{};
/// The IO chain associated with any block containing this job.
/// For example, in `begin; foo ; end < file.txt` this would have the 'file.txt' IO.
io_chain_t block_io{};
/// A shared pointer indicating that the entire tree of jobs is safe to disown.
/// This is set by the "root" job after it is constructed.
std::shared_ptr<relaxed_atomic_bool_t> root_constructed{
std::make_shared<relaxed_atomic_bool_t>(false)};
};
/// A struct represeting a job. A job is basically a pipeline of one or more processes and a couple
/// of flags.
class job_t {
@ -289,20 +309,15 @@ class job_t {
/// messages about job status on the terminal.
wcstring command_str;
// The IO chain associated with the block.
const io_chain_t block_io;
// The parent job. If we were created as a nested job due to execution of a block or function in
// a pipeline, then this refers to the job corresponding to that pipeline. Otherwise it is null.
const std::shared_ptr<job_t> parent_job;
// The lineage associated with the job.
const job_lineage_t job_lineage;
// No copying.
job_t(const job_t &rhs) = delete;
void operator=(const job_t &) = delete;
public:
job_t(job_id_t job_id, const properties_t &props, io_chain_t bio,
std::shared_ptr<job_t> parent);
job_t(job_id_t job_id, const properties_t &props, job_lineage_t lineage);
~job_t();
/// Returns the command as a wchar_t *. */
@ -401,9 +416,12 @@ class job_t {
/// \return if this job should own the terminal when it runs.
bool should_claim_terminal() const { return properties.wants_terminal && is_foreground(); }
/// \return the job lineage.
const job_lineage_t &lineage() const { return job_lineage; }
/// Returns the block IO redirections associated with the job. These are things like the IO
/// redirections associated with the begin...end statement.
const io_chain_t &block_io_chain() const { return this->block_io; }
const io_chain_t &block_io_chain() const { return lineage().block_io; }
/// Fetch all the IO redirections associated with the job.
io_chain_t all_io_redirections() const;
@ -428,9 +446,6 @@ class job_t {
/// This implements some historical behavior which has not been justified.
bool should_report_process_exits() const;
/// \return the parent job, or nullptr.
const std::shared_ptr<job_t> get_parent() const { return parent_job; }
/// \return whether this job and its parent chain are fully constructed.
bool job_chain_is_fully_constructed() const;