ClickHouse/src/Functions/FunctionFile.cpp

168 lines
6.0 KiB
C++
Raw Normal View History

2022-07-14 09:49:30 +00:00
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
2022-07-14 09:49:30 +00:00
#include <Columns/ColumnConst.h>
#include <Columns/IColumn.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeString.h>
2022-07-14 09:49:30 +00:00
#include <DataTypes/DataTypeNullable.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/copyData.h>
2021-01-16 03:27:31 +00:00
#include <Interpreters/Context.h>
#include <unistd.h>
2021-04-28 20:48:34 +00:00
#include <filesystem>
2021-04-28 20:48:34 +00:00
namespace fs = std::filesystem;
namespace DB
{
2021-02-16 12:37:49 +00:00
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int NOT_IMPLEMENTED;
2022-07-14 09:49:30 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2021-02-16 12:37:49 +00:00
extern const int DATABASE_ACCESS_DENIED;
}
/// A function to read file as a string.
2021-06-01 12:20:52 +00:00
class FunctionFile : public IFunction, WithContext
2021-02-16 12:37:49 +00:00
{
public:
static constexpr auto name = "file";
2021-06-01 12:20:52 +00:00
static FunctionPtr create(ContextPtr context_) { return std::make_shared<FunctionFile>(context_); }
explicit FunctionFile(ContextPtr context_) : WithContext(context_) {}
2021-02-16 12:37:49 +00:00
2022-07-14 09:49:30 +00:00
bool isVariadic() const override { return true; }
2021-02-16 12:37:49 +00:00
String getName() const override { return name; }
2022-07-14 09:49:30 +00:00
size_t getNumberOfArguments() const override { return 0; }
2021-06-22 16:21:23 +00:00
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
2021-02-16 12:37:49 +00:00
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
2022-07-15 10:08:14 +00:00
if (arguments.empty() || arguments.size() > 2)
2022-07-14 09:49:30 +00:00
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 1 or 2",
getName(), toString(arguments.size()));
2021-02-16 12:37:49 +00:00
if (!isString(arguments[0].type))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is only implemented for type String", getName());
2022-07-14 09:49:30 +00:00
if (arguments.size() == 2)
{
if (arguments[1].type->onlyNull())
return makeNullable(std::make_shared<DataTypeString>());
if (!isString(arguments[1].type))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} only accepts String or Null as second argument", getName());
}
2021-02-16 12:37:49 +00:00
return std::make_shared<DataTypeString>();
}
2022-07-14 09:49:30 +00:00
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
2022-07-19 09:15:42 +00:00
bool useDefaultImplementationForNulls() const override { return false; }
2021-02-16 12:37:49 +00:00
bool useDefaultImplementationForConstants() const override { return true; }
2022-07-14 09:49:30 +00:00
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
2021-02-16 12:37:49 +00:00
const ColumnPtr column = arguments[0].column;
const ColumnString * column_src = checkAndGetColumn<ColumnString>(column.get());
if (!column_src)
2021-02-16 12:37:49 +00:00
throw Exception(
fmt::format("Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()),
ErrorCodes::ILLEGAL_COLUMN);
2022-07-14 09:49:30 +00:00
String default_result;
ColumnUInt8::MutablePtr col_null_map_to;
ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr;
if (arguments.size() == 2)
{
if (result_type->isNullable())
{
col_null_map_to = ColumnUInt8::create(input_rows_count, false);
vec_null_map_to = &col_null_map_to->getData();
}
else
{
const auto & default_column = arguments[1].column;
const ColumnConst * default_col = checkAndGetColumn<ColumnConst>(default_column.get());
if (!default_col)
throw Exception(
"Illegal column " + arguments[1].column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
default_result = default_col->getValue<String>();
}
}
2021-02-16 12:37:49 +00:00
auto result = ColumnString::create();
auto & res_chars = result->getChars();
auto & res_offsets = result->getOffsets();
2021-02-16 12:37:49 +00:00
res_offsets.resize(input_rows_count);
2021-01-14 12:09:13 +00:00
fs::path user_files_absolute_path = fs::canonical(fs::path(getContext()->getUserFilesPath()));
std::string user_files_absolute_path_string = user_files_absolute_path.string();
// If run in Local mode, no need for path checking.
bool need_check = getContext()->getApplicationType() != Context::ApplicationType::LOCAL;
2021-02-16 12:37:49 +00:00
for (size_t row = 0; row < input_rows_count; ++row)
{
std::string_view filename = column_src->getDataAt(row).toView();
fs::path file_path(filename.data(), filename.data() + filename.size());
2021-02-16 12:37:49 +00:00
2021-04-28 20:48:34 +00:00
if (file_path.is_relative())
file_path = user_files_absolute_path / file_path;
2021-02-16 12:37:49 +00:00
/// Do not use fs::canonical or fs::weakly_canonical.
/// Otherwise it will not allow to work with symlinks in `user_files_path` directory.
file_path = fs::absolute(file_path).lexically_normal();
2021-02-16 12:37:49 +00:00
2022-07-14 09:49:30 +00:00
try
{
if (need_check && file_path.string().find(user_files_absolute_path_string) != 0)
throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "File is not inside {}", user_files_absolute_path.string());
ReadBufferFromFile in(file_path);
WriteBufferFromVector out(res_chars, AppendModeTag{});
copyData(in, out);
out.finalize();
}
catch (...)
{
if (arguments.size() == 1)
throw;
if (vec_null_map_to)
(*vec_null_map_to)[row] = true;
else
res_chars.insert(default_result.data(), default_result.data() + default_result.size());
}
2021-02-16 12:37:49 +00:00
res_chars.push_back(0);
res_offsets[row] = res_chars.size();
}
2022-07-14 09:49:30 +00:00
if (vec_null_map_to)
return ColumnNullable::create(std::move(result), std::move(col_null_map_to));
2021-02-16 12:37:49 +00:00
return result;
}
};
REGISTER_FUNCTION(File)
2021-02-16 12:37:49 +00:00
{
factory.registerFunction<FunctionFile>();
}
}