ClickHouse/src/Common/ShellCommand.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

354 lines
11 KiB
C++
Raw Normal View History

2015-12-13 08:51:28 +00:00
#include <sys/types.h>
#include <sys/wait.h>
#include <dlfcn.h>
2021-03-03 22:52:31 +00:00
#include <unistd.h>
#include <csignal>
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
2021-10-02 07:13:14 +00:00
#include <base/errnoToString.h>
#include <Common/Exception.h>
#include <Common/ShellCommand.h>
#include <Common/PipeFDs.h>
#include <IO/WriteHelpers.h>
2020-11-10 18:22:26 +00:00
#include <IO/Operators.h>
#include <Common/waitForPid.h>
2021-03-03 22:52:31 +00:00
2015-12-13 08:51:28 +00:00
2019-07-05 13:48:47 +00:00
namespace
{
/// By these return codes from the child process, we learn (for sure) about errors when creating it.
enum class ReturnCodes : int
{
2021-08-25 19:30:22 +00:00
CANNOT_DUP_STDIN = 0x55555555, /// The value is not important, but it is chosen so that it's rare to conflict with the program return code.
CANNOT_DUP_STDOUT = 0x55555556,
CANNOT_DUP_STDERR = 0x55555557,
CANNOT_EXEC = 0x55555558,
CANNOT_DUP_READ_DESCRIPTOR = 0x55555559,
CANNOT_DUP_WRITE_DESCRIPTOR = 0x55555560,
2019-07-05 13:48:47 +00:00
};
}
2015-12-13 08:51:28 +00:00
namespace ProfileEvents
{
2022-04-05 09:24:33 +00:00
extern const Event ExecuteShellCommand;
}
namespace DB
{
2019-02-03 09:57:12 +00:00
namespace ErrorCodes
2015-12-13 08:51:28 +00:00
{
2019-02-03 21:30:45 +00:00
extern const int CANNOT_DLSYM;
extern const int CANNOT_FORK;
extern const int CANNOT_WAITPID;
extern const int CHILD_WAS_NOT_EXITED_NORMALLY;
extern const int CANNOT_CREATE_CHILD_PROCESS;
2015-12-13 08:51:28 +00:00
}
2021-08-28 19:47:59 +00:00
ShellCommand::ShellCommand(pid_t pid_, int & in_fd_, int & out_fd_, int & err_fd_, const ShellCommand::Config & config_)
2021-08-25 19:30:22 +00:00
: in(in_fd_)
2019-08-03 11:02:40 +00:00
, out(out_fd_)
2020-09-30 00:00:45 +00:00
, err(err_fd_)
2021-08-25 19:30:22 +00:00
, pid(pid_)
2021-08-28 19:47:59 +00:00
, config(config_)
2020-09-30 00:00:45 +00:00
{
}
Poco::Logger * ShellCommand::getLogger()
{
return &Poco::Logger::get("ShellCommand");
}
ShellCommand::~ShellCommand()
{
2021-03-03 19:34:25 +00:00
if (wait_called)
return;
2021-08-28 19:47:59 +00:00
if (config.terminate_in_destructor_strategy.terminate_in_destructor)
{
2021-08-28 19:47:59 +00:00
size_t try_wait_timeout = config.terminate_in_destructor_strategy.wait_for_normal_exit_before_termination_seconds;
2021-03-04 11:58:36 +00:00
bool process_terminated_normally = tryWaitProcessWithTimeout(try_wait_timeout);
if (process_terminated_normally)
return;
2021-03-04 11:58:36 +00:00
2022-12-28 19:07:26 +00:00
LOG_TRACE(getLogger(), "Will kill shell command pid {} with signal {}", pid, config.terminate_in_destructor_strategy.termination_signal);
2022-12-28 19:07:26 +00:00
int retcode = kill(pid, config.terminate_in_destructor_strategy.termination_signal);
if (retcode != 0)
2022-12-28 19:07:26 +00:00
LOG_WARNING(getLogger(), "Cannot kill shell command pid {}, error: '{}'", pid, errnoToString());
}
2021-03-03 19:34:25 +00:00
else
2020-09-07 03:11:35 +00:00
{
try
{
tryWait();
}
catch (...)
{
tryLogCurrentException(getLogger());
}
}
}
2015-12-13 08:51:28 +00:00
2021-03-04 11:58:36 +00:00
bool ShellCommand::tryWaitProcessWithTimeout(size_t timeout_in_seconds)
2021-03-03 19:34:25 +00:00
{
LOG_TRACE(getLogger(), "Try wait for shell command pid {} with timeout {}", pid, timeout_in_seconds);
2021-03-03 19:34:25 +00:00
2021-03-04 11:58:36 +00:00
wait_called = true;
2021-03-03 19:34:25 +00:00
2021-03-06 18:45:51 +00:00
in.close();
out.close();
err.close();
2021-03-03 19:34:25 +00:00
2023-08-21 11:35:09 +00:00
for (auto & [_, fd] : write_fds)
fd.close();
for (auto & [_, fd] : read_fds)
fd.close();
return waitForPid(pid, timeout_in_seconds);
2021-03-03 19:34:25 +00:00
}
void ShellCommand::logCommand(const char * filename, char * const argv[])
{
2020-11-10 18:22:26 +00:00
WriteBufferFromOwnString args;
2019-09-27 12:22:27 +00:00
for (int i = 0; argv != nullptr && argv[i] != nullptr; ++i)
{
if (i > 0)
2020-05-23 20:13:34 +00:00
args << ", ";
2019-12-09 14:49:21 +00:00
2019-09-27 12:22:27 +00:00
/// NOTE: No escaping is performed.
2020-05-23 20:13:34 +00:00
args << "'" << argv[i] << "'";
}
2020-05-23 22:24:01 +00:00
LOG_TRACE(ShellCommand::getLogger(), "Will start shell command '{}' with arguments {}", filename, args.str());
}
2020-09-07 03:11:35 +00:00
std::unique_ptr<ShellCommand> ShellCommand::executeImpl(
2021-03-03 19:34:25 +00:00
const char * filename,
char * const argv[],
2021-08-25 19:30:22 +00:00
const Config & config)
2015-12-13 08:51:28 +00:00
{
logCommand(filename, argv);
ProfileEvents::increment(ProfileEvents::ExecuteShellCommand);
2021-12-19 05:53:35 +00:00
#if !defined(USE_MUSL)
/** Here it is written that with a normal call `vfork`, there is a chance of deadlock in multithreaded programs,
2020-02-13 14:25:17 +00:00
* because of the resolving of symbols in the shared library
2015-12-13 08:51:28 +00:00
* http://www.oracle.com/technetwork/server-storage/solaris10/subprocess-136439.html
* Therefore, separate the resolving of the symbol from the call.
2015-12-13 08:51:28 +00:00
*/
static void * real_vfork = dlsym(RTLD_DEFAULT, "vfork");
2021-12-19 05:53:35 +00:00
#else
/// If we use Musl with static linking, there is no dlsym and no issue with vfork.
static void * real_vfork = reinterpret_cast<void *>(&vfork);
#endif
2015-12-13 08:51:28 +00:00
if (!real_vfork)
2023-12-15 18:25:49 +00:00
throw ErrnoException(ErrorCodes::CANNOT_DLSYM, "Cannot find symbol vfork in myself");
2015-12-13 08:51:28 +00:00
PipeFDs pipe_stdin;
PipeFDs pipe_stdout;
PipeFDs pipe_stderr;
2015-12-13 08:51:28 +00:00
2021-08-28 19:47:59 +00:00
std::vector<std::unique_ptr<PipeFDs>> read_pipe_fds;
std::vector<std::unique_ptr<PipeFDs>> write_pipe_fds;
for (size_t i = 0; i < config.read_fds.size(); ++i)
read_pipe_fds.emplace_back(std::make_unique<PipeFDs>());
for (size_t i = 0; i < config.write_fds.size(); ++i)
write_pipe_fds.emplace_back(std::make_unique<PipeFDs>());
2015-12-13 08:51:28 +00:00
pid_t pid = reinterpret_cast<pid_t(*)()>(real_vfork)();
2021-08-28 19:47:59 +00:00
if (pid == -1)
2023-12-15 18:25:49 +00:00
throw ErrnoException(ErrorCodes::CANNOT_FORK, "Cannot vfork");
2015-12-13 08:51:28 +00:00
if (0 == pid)
{
2017-03-25 20:12:56 +00:00
/// We are in the freshly created process.
2015-12-13 08:51:28 +00:00
/// Why `_exit` and not `exit`? Because `exit` calls `atexit` and destructors of thread local storage.
/// And there is a lot of garbage (including, for example, mutex is blocked). And this can not be done after `vfork` - deadlock happens.
2015-12-13 08:51:28 +00:00
2017-03-25 20:12:56 +00:00
/// Replace the file descriptors with the ends of our pipes.
if (STDIN_FILENO != dup2(pipe_stdin.fds_rw[0], STDIN_FILENO))
_exit(static_cast<int>(ReturnCodes::CANNOT_DUP_STDIN));
2015-12-13 08:51:28 +00:00
2021-08-25 19:30:22 +00:00
if (!config.pipe_stdin_only)
2017-03-12 21:10:52 +00:00
{
if (STDOUT_FILENO != dup2(pipe_stdout.fds_rw[1], STDOUT_FILENO))
_exit(static_cast<int>(ReturnCodes::CANNOT_DUP_STDOUT));
2015-12-13 08:51:28 +00:00
if (STDERR_FILENO != dup2(pipe_stderr.fds_rw[1], STDERR_FILENO))
_exit(static_cast<int>(ReturnCodes::CANNOT_DUP_STDERR));
2017-03-12 21:10:52 +00:00
}
2015-12-13 08:51:28 +00:00
2021-08-28 19:47:59 +00:00
for (size_t i = 0; i < config.read_fds.size(); ++i)
{
auto & fds = *read_pipe_fds[i];
auto fd = config.read_fds[i];
if (fd != dup2(fds.fds_rw[1], fd))
_exit(static_cast<int>(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR));
2021-08-28 19:47:59 +00:00
}
for (size_t i = 0; i < config.write_fds.size(); ++i)
{
auto & fds = *write_pipe_fds[i];
auto fd = config.write_fds[i];
2021-08-25 19:30:22 +00:00
2021-08-28 19:47:59 +00:00
if (fd != dup2(fds.fds_rw[0], fd))
_exit(static_cast<int>(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR));
2021-08-28 19:47:59 +00:00
}
2021-08-25 19:30:22 +00:00
// Reset the signal mask: it may be non-empty and will be inherited
// by the child process, which might not expect this.
sigset_t mask;
sigemptyset(&mask);
sigprocmask(0, nullptr, &mask); // NOLINT(concurrency-mt-unsafe)
sigprocmask(SIG_UNBLOCK, &mask, nullptr); // NOLINT(concurrency-mt-unsafe)
2015-12-13 11:35:48 +00:00
execv(filename, argv);
/// If the process is running, then `execv` does not return here.
2015-12-13 08:51:28 +00:00
_exit(static_cast<int>(ReturnCodes::CANNOT_EXEC));
2015-12-13 08:51:28 +00:00
}
2020-09-07 03:11:35 +00:00
std::unique_ptr<ShellCommand> res(new ShellCommand(
2021-08-25 19:30:22 +00:00
pid,
pipe_stdin.fds_rw[1],
pipe_stdout.fds_rw[0],
pipe_stderr.fds_rw[0],
2021-08-28 19:47:59 +00:00
config));
2021-08-25 19:30:22 +00:00
2021-08-28 19:47:59 +00:00
for (size_t i = 0; i < config.read_fds.size(); ++i)
{
auto & fds = *read_pipe_fds[i];
auto fd = config.read_fds[i];
res->read_fds.emplace(fd, fds.fds_rw[0]);
}
2021-08-25 19:30:22 +00:00
2021-08-28 19:47:59 +00:00
for (size_t i = 0; i < config.write_fds.size(); ++i)
{
auto & fds = *write_pipe_fds[i];
auto fd = config.write_fds[i];
res->write_fds.emplace(fd, fds.fds_rw[1]);
}
2015-12-13 08:51:28 +00:00
2020-05-23 22:24:01 +00:00
LOG_TRACE(getLogger(), "Started shell command '{}' with pid {}", filename, pid);
2015-12-13 08:51:28 +00:00
return res;
}
2021-08-25 19:30:22 +00:00
std::unique_ptr<ShellCommand> ShellCommand::execute(const ShellCommand::Config & config)
2015-12-13 08:51:28 +00:00
{
2021-08-30 18:41:36 +00:00
auto config_copy = config;
config_copy.command = "/bin/sh";
config_copy.arguments = {"-c", config.command};
2021-08-25 19:30:22 +00:00
2021-08-30 18:41:36 +00:00
for (const auto & argument : config.arguments)
config_copy.arguments.emplace_back(argument);
2015-12-13 08:51:28 +00:00
2021-08-30 18:41:36 +00:00
return executeDirect(config_copy);
2015-12-13 08:51:28 +00:00
}
2021-08-25 19:30:22 +00:00
std::unique_ptr<ShellCommand> ShellCommand::executeDirect(const ShellCommand::Config & config)
2015-12-13 08:51:28 +00:00
{
2021-08-25 19:30:22 +00:00
const auto & path = config.command;
const auto & arguments = config.arguments;
2015-12-13 08:51:28 +00:00
size_t argv_sum_size = path.size() + 1;
for (const auto & arg : arguments)
argv_sum_size += arg.size() + 1;
std::vector<char *> argv(arguments.size() + 2);
std::vector<char> argv_data(argv_sum_size);
2023-06-01 17:00:47 +00:00
WriteBufferFromPointer writer(argv_data.data(), argv_sum_size);
2015-12-13 08:51:28 +00:00
argv[0] = writer.position();
writer.write(path.data(), path.size() + 1);
for (size_t i = 0, size = arguments.size(); i < size; ++i)
{
argv[i + 1] = writer.position();
writer.write(arguments[i].data(), arguments[i].size() + 1);
}
2023-06-01 17:00:47 +00:00
writer.finalize();
2015-12-13 08:51:28 +00:00
argv[arguments.size() + 1] = nullptr;
2021-08-25 19:30:22 +00:00
return executeImpl(path.data(), argv.data(), config);
2015-12-13 08:51:28 +00:00
}
int ShellCommand::tryWait()
{
wait_called = true;
in.close();
out.close();
err.close();
2023-08-21 11:35:09 +00:00
for (auto & [_, fd] : write_fds)
fd.close();
for (auto & [_, fd] : read_fds)
fd.close();
2020-05-23 22:24:01 +00:00
LOG_TRACE(getLogger(), "Will wait for shell command pid {}", pid);
2015-12-13 08:51:28 +00:00
int status = 0;
2021-03-09 10:13:19 +00:00
while (waitpid(pid, &status, 0) < 0)
{
if (errno != EINTR)
2023-12-15 18:25:49 +00:00
throw ErrnoException(ErrorCodes::CANNOT_WAITPID, "Cannot waitpid");
2021-03-09 10:13:19 +00:00
}
2015-12-13 08:51:28 +00:00
2020-05-23 22:24:01 +00:00
LOG_TRACE(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status);
2015-12-13 08:51:28 +00:00
if (WIFEXITED(status))
return WEXITSTATUS(status);
if (WIFSIGNALED(status))
throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was terminated by signal {}", toString(WTERMSIG(status)));
2015-12-13 08:51:28 +00:00
if (WIFSTOPPED(status))
throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was stopped by signal {}", toString(WSTOPSIG(status)));
2015-12-13 08:51:28 +00:00
throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was not exited normally by unknown reason");
2015-12-13 08:51:28 +00:00
}
void ShellCommand::wait()
{
int retcode = tryWait();
2015-12-13 08:51:28 +00:00
if (retcode != EXIT_SUCCESS)
{
switch (retcode)
{
case static_cast<int>(ReturnCodes::CANNOT_DUP_STDIN):
throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdin of child process");
case static_cast<int>(ReturnCodes::CANNOT_DUP_STDOUT):
throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdout of child process");
case static_cast<int>(ReturnCodes::CANNOT_DUP_STDERR):
throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stderr of child process");
case static_cast<int>(ReturnCodes::CANNOT_EXEC):
throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot execv in child process");
case static_cast<int>(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR):
throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 read descriptor of child process");
case static_cast<int>(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR):
throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 write descriptor of child process");
2015-12-13 08:51:28 +00:00
default:
throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was exited with return code {}", toString(retcode));
2015-12-13 08:51:28 +00:00
}
}
}
}