mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #32640 from ClickHouse/fix-function-file
Fix surprisingly bad code in function `file`
This commit is contained in:
commit
fd9d40925a
@ -860,7 +860,7 @@ namespace
|
||||
template <typename NumberType>
|
||||
void toStringAppend(NumberType value, PaddedPODArray<UInt8> & str)
|
||||
{
|
||||
WriteBufferFromVector buf{str, WriteBufferFromVector<PaddedPODArray<UInt8>>::AppendModeTag{}};
|
||||
WriteBufferFromVector buf{str, AppendModeTag{}};
|
||||
writeText(value, buf);
|
||||
}
|
||||
|
||||
|
@ -3,10 +3,13 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <unistd.h>
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace DB
|
||||
@ -16,9 +19,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int DATABASE_ACCESS_DENIED;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
}
|
||||
|
||||
/// A function to read file as a string.
|
||||
@ -30,15 +31,14 @@ public:
|
||||
explicit FunctionFile(ContextPtr context_) : WithContext(context_) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isInjective(const ColumnsWithTypeAndName &) const override { return true; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0].type))
|
||||
throw Exception(getName() + " is only implemented for types String", ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is only implemented for type String", getName());
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
@ -47,80 +47,50 @@ public:
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnPtr column = arguments[0].column;
|
||||
const ColumnString * expected = checkAndGetColumn<ColumnString>(column.get());
|
||||
if (!expected)
|
||||
const ColumnString * column_src = checkAndGetColumn<ColumnString>(column.get());
|
||||
if (!column_src)
|
||||
throw Exception(
|
||||
fmt::format("Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
const ColumnString::Chars & chars = expected->getChars();
|
||||
const ColumnString::Offsets & offsets = expected->getOffsets();
|
||||
|
||||
std::vector<String> checked_filenames(input_rows_count);
|
||||
|
||||
auto result = ColumnString::create();
|
||||
auto & res_chars = result->getChars();
|
||||
auto & res_offsets = result->getOffsets();
|
||||
|
||||
res_offsets.resize(input_rows_count);
|
||||
|
||||
size_t source_offset = 0;
|
||||
size_t result_offset = 0;
|
||||
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;
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
const char * filename = reinterpret_cast<const char *>(&chars[source_offset]);
|
||||
StringRef filename = column_src->getDataAt(row);
|
||||
fs::path file_path(filename.data, filename.data + filename.size);
|
||||
|
||||
fs::path user_files_absolute_path = fs::canonical(fs::path(getContext()->getUserFilesPath()));
|
||||
fs::path file_path(filename);
|
||||
if (file_path.is_relative())
|
||||
file_path = user_files_absolute_path / file_path;
|
||||
fs::path file_absolute_path = fs::canonical(file_path);
|
||||
checkReadIsAllowedOrThrow(user_files_absolute_path.string(), file_absolute_path);
|
||||
|
||||
checked_filenames[row] = file_absolute_path.string();
|
||||
/// 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();
|
||||
|
||||
if (!fs::exists(file_absolute_path))
|
||||
throw Exception(fmt::format("File {} doesn't exist.", file_absolute_path.string()), ErrorCodes::FILE_DOESNT_EXIST);
|
||||
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());
|
||||
|
||||
const auto current_file_size = fs::file_size(file_absolute_path);
|
||||
ReadBufferFromFile in(file_path);
|
||||
WriteBufferFromVector out(res_chars, AppendModeTag{});
|
||||
copyData(in, out);
|
||||
out.finalize();
|
||||
|
||||
result_offset += current_file_size + 1;
|
||||
res_offsets[row] = result_offset;
|
||||
source_offset = offsets[row];
|
||||
}
|
||||
|
||||
res_chars.resize(result_offset);
|
||||
|
||||
size_t prev_offset = 0;
|
||||
|
||||
for (size_t row = 0; row < input_rows_count; ++row)
|
||||
{
|
||||
auto file_absolute_path = checked_filenames[row];
|
||||
ReadBufferFromFile in(file_absolute_path);
|
||||
char * res_buf = reinterpret_cast<char *>(&res_chars[prev_offset]);
|
||||
|
||||
const size_t file_lenght = res_offsets[row] - prev_offset - 1;
|
||||
prev_offset = res_offsets[row];
|
||||
in.readStrict(res_buf, file_lenght);
|
||||
res_buf[file_lenght] = '\0';
|
||||
res_chars.push_back(0);
|
||||
res_offsets[row] = res_chars.size();
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
void checkReadIsAllowedOrThrow(const std::string & user_files_absolute_path, const std::string & file_absolute_path) const
|
||||
{
|
||||
// If run in Local mode, no need for path checking.
|
||||
if (getContext()->getApplicationType() != Context::ApplicationType::LOCAL)
|
||||
if (file_absolute_path.find(user_files_absolute_path) != 0)
|
||||
throw Exception("File is not inside " + user_files_absolute_path, ErrorCodes::DATABASE_ACCESS_DENIED);
|
||||
|
||||
fs::path fs_path(file_absolute_path);
|
||||
if (fs::exists(fs_path) && fs::is_directory(fs_path))
|
||||
throw Exception("File can't be a directory", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -1249,7 +1249,7 @@ public:
|
||||
{
|
||||
ColumnString & col_str = assert_cast<ColumnString &>(dest);
|
||||
auto & chars = col_str.getChars();
|
||||
WriteBufferFromVector<ColumnString::Chars> buf(chars, WriteBufferFromVector<ColumnString::Chars>::AppendModeTag());
|
||||
WriteBufferFromVector<ColumnString::Chars> buf(chars, AppendModeTag());
|
||||
traverse(element, buf);
|
||||
buf.finalize();
|
||||
chars.push_back(0);
|
||||
|
@ -13,6 +13,8 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_WRITE_AFTER_END_OF_BUFFER;
|
||||
}
|
||||
|
||||
struct AppendModeTag {};
|
||||
|
||||
/** Writes data to existing std::vector or similar type. When not enough space, it doubles vector size.
|
||||
*
|
||||
* In destructor, vector is cut to the size of written data.
|
||||
@ -35,7 +37,6 @@ public:
|
||||
}
|
||||
|
||||
/// Append to vector instead of rewrite.
|
||||
struct AppendModeTag {};
|
||||
WriteBufferFromVector(VectorType & vector_, AppendModeTag)
|
||||
: WriteBuffer(nullptr, 0), vector(vector_)
|
||||
{
|
||||
|
@ -325,7 +325,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding(
|
||||
{
|
||||
String serialized_value = " ";
|
||||
{
|
||||
WriteBufferFromString out_serialize(serialized_value, WriteBufferFromString::AppendModeTag());
|
||||
WriteBufferFromString out_serialize(serialized_value, AppendModeTag());
|
||||
serialization.serializeText(column, row_num, out_serialize, format_settings);
|
||||
}
|
||||
|
||||
|
@ -147,8 +147,7 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user
|
||||
Strings paths;
|
||||
/// Do not use fs::canonical or fs::weakly_canonical.
|
||||
/// Otherwise it will not allow to work with symlinks in `user_files_path` directory.
|
||||
String path = fs::absolute(fs_table_path);
|
||||
path = fs::path(path).lexically_normal(); /// Normalize path.
|
||||
String path = fs::absolute(fs_table_path).lexically_normal(); /// Normalize path.
|
||||
if (path.find_first_of("*?{") == std::string::npos)
|
||||
{
|
||||
std::error_code error;
|
||||
|
@ -8,11 +8,11 @@ ccccccccc aaaaaaaaa bbbbbbbbb
|
||||
aaaaaaaaa
|
||||
bbbbbbbbb
|
||||
ccccccccc
|
||||
:233
|
||||
:79
|
||||
:107
|
||||
:74
|
||||
:35
|
||||
:35
|
||||
:35
|
||||
:233
|
||||
695071
|
||||
aaaaaaaaa bbbbbbbbb
|
||||
ccccccccc aaaaaaaaa bbbbbbbbb
|
||||
@ -21,5 +21,9 @@ ccccccccc aaaaaaaaa bbbbbbbbb
|
||||
ccccccccc aaaaaaaaa bbbbbbbbb
|
||||
695071 0
|
||||
:0
|
||||
:233
|
||||
:79
|
||||
:107
|
||||
:74
|
||||
Hello 2
|
||||
World 1
|
||||
Hello 2
|
||||
World 1
|
||||
|
@ -8,16 +8,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
# Data preparation.
|
||||
|
||||
# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as:
|
||||
# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')"
|
||||
user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||
CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||
|
||||
mkdir -p ${user_files_path}/
|
||||
echo -n aaaaaaaaa > ${user_files_path}/a.txt
|
||||
echo -n bbbbbbbbb > ${user_files_path}/b.txt
|
||||
echo -n ccccccccc > ${user_files_path}/c.txt
|
||||
mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/
|
||||
echo -n aaaaaaaaa > ${CLICKHOUSE_USER_FILES_PATH}/a.txt
|
||||
echo -n bbbbbbbbb > ${CLICKHOUSE_USER_FILES_PATH}/b.txt
|
||||
echo -n ccccccccc > ${CLICKHOUSE_USER_FILES_PATH}/c.txt
|
||||
echo -n ccccccccc > /tmp/c.txt
|
||||
mkdir -p ${user_files_path}/dir
|
||||
mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/dir
|
||||
|
||||
|
||||
### 1st TEST in CLIENT mode.
|
||||
@ -26,28 +27,28 @@ ${CLICKHOUSE_CLIENT} --query "create table data (A String, B String) engine=Merg
|
||||
|
||||
|
||||
# Valid cases:
|
||||
${CLICKHOUSE_CLIENT} --query "select file('${user_files_path}/a.txt'), file('${user_files_path}/b.txt');";echo ":"$?
|
||||
${CLICKHOUSE_CLIENT} --query "insert into data select file('${user_files_path}/a.txt'), file('${user_files_path}/b.txt');";echo ":"$?
|
||||
${CLICKHOUSE_CLIENT} --query "insert into data select file('${user_files_path}/a.txt'), file('${user_files_path}/b.txt');";echo ":"$?
|
||||
${CLICKHOUSE_CLIENT} --query "select file('${user_files_path}/c.txt'), * from data";echo ":"$?
|
||||
${CLICKHOUSE_CLIENT} --query "select file('a.txt'), file('b.txt');";echo ":"$?
|
||||
${CLICKHOUSE_CLIENT} --query "insert into data select file('a.txt'), file('b.txt');";echo ":"$?
|
||||
${CLICKHOUSE_CLIENT} --query "insert into data select file('a.txt'), file('b.txt');";echo ":"$?
|
||||
${CLICKHOUSE_CLIENT} --query "select file('c.txt'), * from data";echo ":"$?
|
||||
${CLICKHOUSE_CLIENT} --multiquery --query "
|
||||
create table filenames(name String) engine=MergeTree() order by tuple();
|
||||
insert into filenames values ('a.txt'), ('b.txt'), ('c.txt');
|
||||
select file(name) from filenames format TSV;
|
||||
drop table if exists filenames;
|
||||
create table filenames(name String) engine=MergeTree() order by tuple();
|
||||
insert into filenames values ('a.txt'), ('b.txt'), ('c.txt');
|
||||
select file(name) from filenames format TSV;
|
||||
drop table if exists filenames;
|
||||
"
|
||||
|
||||
# Invalid cases: (Here using sub-shell to catch exception avoiding the test quit)
|
||||
# Test non-exists file
|
||||
echo "clickhouse-client --query "'"select file('"'nonexist.txt'), file('${user_files_path}/b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'nonexist.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
# Test isDir
|
||||
echo "clickhouse-client --query "'"select file('"'${user_files_path}/dir'), file('${user_files_path}/b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'dir'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
# Test path out of the user_files directory. It's not allowed in client mode
|
||||
echo "clickhouse-client --query "'"select file('"'/tmp/c.txt'), file('${user_files_path}/b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'/tmp/c.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
|
||||
# Test relative path consists of ".." whose absolute path is out of the user_files directory.
|
||||
echo "clickhouse-client --query "'"select file('"'${user_files_path}/../../../../../../../../../../../../../../../../../../../tmp/c.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
echo "clickhouse-client --query "'"select file('"'../../../../a.txt'), file('${user_files_path}/b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'../../../../../../../../../../../../../../../../../../../tmp/c.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
echo "${CLICKHOUSE_CLIENT} --query "'"select file('"'../../../../a.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
|
||||
|
||||
### 2nd TEST in LOCAL mode.
|
||||
@ -63,29 +64,36 @@ echo $c_count
|
||||
# Valid cases:
|
||||
# The default dir is the CWD path in LOCAL mode
|
||||
${CLICKHOUSE_LOCAL} --query "
|
||||
drop table if exists data;
|
||||
create table data (A String, B String) engine=MergeTree() order by A;
|
||||
select file('a.txt'), file('b.txt');
|
||||
insert into data select file('a.txt'), file('b.txt');
|
||||
insert into data select file('a.txt'), file('b.txt');
|
||||
select file('c.txt'), * from data;
|
||||
select file('/tmp/c.txt'), * from data;
|
||||
select $c_count, $c_count -length(file('${CURDIR}/01518_nullable_aggregate_states2.reference'))
|
||||
drop table if exists data;
|
||||
create table data (A String, B String) engine=MergeTree() order by A;
|
||||
select file('a.txt'), file('b.txt');
|
||||
insert into data select file('a.txt'), file('b.txt');
|
||||
insert into data select file('a.txt'), file('b.txt');
|
||||
select file('c.txt'), * from data;
|
||||
select file('/tmp/c.txt'), * from data;
|
||||
select $c_count, $c_count -length(file('${CURDIR}/01518_nullable_aggregate_states2.reference'))
|
||||
"
|
||||
echo ":"$?
|
||||
|
||||
|
||||
# Invalid cases: (Here using sub-shell to catch exception avoiding the test quit)
|
||||
# Test non-exists file
|
||||
echo "clickhouse-local --query "'"select file('"'nonexist.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
echo "${CLICKHOUSE_LOCAL} --query "'"select file('"'nonexist.txt'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
|
||||
# Test isDir
|
||||
echo "clickhouse-local --query "'"select file('"'dir'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
echo "${CLICKHOUSE_LOCAL} --query "'"select file('"'dir'), file('b.txt')"'";echo :$?' | bash 2>/dev/null
|
||||
|
||||
# Test that the function is not injective
|
||||
|
||||
echo -n Hello > ${CLICKHOUSE_USER_FILES_PATH}/a
|
||||
echo -n Hello > ${CLICKHOUSE_USER_FILES_PATH}/b
|
||||
echo -n World > ${CLICKHOUSE_USER_FILES_PATH}/c
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT file(arrayJoin(['a', 'b', 'c'])) AS s, count() GROUP BY s ORDER BY s"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT s, count() FROM file('?', TSV, 's String') GROUP BY s ORDER BY s"
|
||||
|
||||
# Restore
|
||||
rm -rf a.txt b.txt c.txt dir
|
||||
rm -rf ${user_files_path}/a.txt
|
||||
rm -rf ${user_files_path}/b.txt
|
||||
rm -rf ${user_files_path}/c.txt
|
||||
rm -rf /tmp/c.txt
|
||||
rm -rf ${user_files_path}/dir
|
||||
rm ${CLICKHOUSE_USER_FILES_PATH}/{a,b,c}.txt
|
||||
rm ${CLICKHOUSE_USER_FILES_PATH}/{a,b,c}
|
||||
rm /tmp/c.txt
|
||||
rm -rf ${CLICKHOUSE_USER_FILES_PATH}/dir
|
||||
|
Loading…
Reference in New Issue
Block a user