ClickHouse/src/Storages/StorageFile.cpp

821 lines
30 KiB
C++
Raw Normal View History

#include <Storages/StorageFile.h>
#include <Storages/StorageFactory.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTIdentifier.h>
#include <IO/ReadBufferFromFile.h>
2021-07-18 12:55:24 +00:00
#include <IO/ReadBufferFromFileDescriptor.h>
2019-11-20 14:48:01 +00:00
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeString.h>
#include <DataStreams/IBlockOutputStream.h>
2021-07-23 14:25:35 +00:00
#include <Processors/Sinks/SinkToStorage.h>
2021-07-21 16:13:17 +00:00
#include <Processors/Transforms/AddingDefaultsTransform.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
2019-07-21 13:15:04 +00:00
#include <Common/parseGlobs.h>
2021-03-30 21:28:23 +00:00
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageInMemoryMetadata.h>
2021-07-06 10:23:39 +00:00
#include <sys/stat.h>
#include <fcntl.h>
2020-01-05 02:57:09 +00:00
#include <unistd.h>
2019-07-21 13:15:04 +00:00
#include <re2/re2.h>
2019-08-27 15:20:31 +00:00
#include <filesystem>
2020-01-04 14:45:11 +00:00
#include <Storages/Distributed/DirectoryMonitor.h>
#include <Processors/Sources/SourceWithProgress.h>
2020-05-18 10:00:22 +00:00
#include <Processors/Formats/InputStreamFromInputFormat.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Pipe.h>
2021-07-20 18:18:43 +00:00
#include <Processors/Executors/PullingPipelineExecutor.h>
2021-04-26 13:34:44 +00:00
2019-07-21 13:15:04 +00:00
2019-08-27 15:20:31 +00:00
namespace fs = std::filesystem;
2019-07-21 13:15:04 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
2021-08-22 17:21:49 +00:00
extern const int CANNOT_FSTAT;
2020-01-05 02:57:09 +00:00
extern const int CANNOT_TRUNCATE_FILE;
extern const int DATABASE_ACCESS_DENIED;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int UNKNOWN_IDENTIFIER;
extern const int INCORRECT_FILE_NAME;
extern const int FILE_DOESNT_EXIST;
extern const int TIMEOUT_EXCEEDED;
extern const int INCOMPATIBLE_COLUMNS;
2021-07-09 09:20:11 +00:00
extern const int CANNOT_STAT;
}
2019-08-02 15:00:12 +00:00
namespace
{
2019-09-22 22:13:42 +00:00
2019-08-10 16:00:01 +00:00
/* Recursive directory listing with matched paths as a result.
* Have the same method in StorageHDFS.
*/
2021-04-26 13:34:44 +00:00
std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match, size_t & total_bytes_to_read)
2019-08-02 15:00:12 +00:00
{
2019-08-30 15:19:05 +00:00
const size_t first_glob = for_match.find_first_of("*?{");
2019-08-05 23:10:19 +00:00
2019-08-30 15:19:05 +00:00
const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/');
const std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/'
2019-08-08 14:26:02 +00:00
2019-08-30 15:19:05 +00:00
const size_t next_slash = suffix_with_globs.find('/', 1);
2020-01-05 20:11:26 +00:00
auto regexp = makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash));
re2::RE2 matcher(regexp);
2019-08-05 23:10:19 +00:00
2019-08-08 14:26:02 +00:00
std::vector<std::string> result;
2019-08-30 15:19:05 +00:00
const std::string prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs);
2021-04-27 00:05:43 +00:00
if (!fs::exists(prefix_without_globs))
2019-08-10 16:00:01 +00:00
return result;
2021-04-27 00:05:43 +00:00
2019-08-30 15:19:05 +00:00
const fs::directory_iterator end;
for (fs::directory_iterator it(prefix_without_globs); it != end; ++it)
2019-08-02 15:00:12 +00:00
{
2019-08-30 15:19:05 +00:00
const std::string full_path = it->path().string();
const size_t last_slash = full_path.rfind('/');
const String file_name = full_path.substr(last_slash);
const bool looking_for_directory = next_slash != std::string::npos;
2019-08-10 16:00:01 +00:00
/// Condition is_directory means what kind of path is it in current iteration of ls
2019-08-30 15:19:05 +00:00
if (!fs::is_directory(it->path()) && !looking_for_directory)
2019-08-02 15:00:12 +00:00
{
2019-08-08 14:26:02 +00:00
if (re2::RE2::FullMatch(file_name, matcher))
2019-08-02 15:00:12 +00:00
{
2021-04-26 13:34:44 +00:00
total_bytes_to_read += fs::file_size(it->path());
2019-08-02 15:00:12 +00:00
result.push_back(it->path().string());
}
}
2019-08-30 15:19:05 +00:00
else if (fs::is_directory(it->path()) && looking_for_directory)
2019-08-02 15:00:12 +00:00
{
2019-08-08 14:26:02 +00:00
if (re2::RE2::FullMatch(file_name, matcher))
2019-08-02 15:00:12 +00:00
{
2019-09-23 14:50:33 +00:00
/// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check.
Strings result_part = listFilesWithRegexpMatching(fs::path(full_path) / "", suffix_with_globs.substr(next_slash), total_bytes_to_read);
2019-08-02 15:00:12 +00:00
std::move(result_part.begin(), result_part.end(), std::back_inserter(result));
}
}
}
return result;
}
2020-03-09 01:03:43 +00:00
std::string getTablePath(const std::string & table_dir_path, const std::string & format_name)
{
2019-10-25 19:07:47 +00:00
return table_dir_path + "/data." + escapeForFileName(format_name);
}
2018-04-19 04:39:16 +00:00
/// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..').
void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_dir_path, const std::string & table_path)
{
if (context_global->getApplicationType() != Context::ApplicationType::SERVER)
return;
2019-12-29 07:03:39 +00:00
/// "/dev/null" is allowed for perf testing
if (!startsWith(table_path, db_dir_path) && table_path != "/dev/null")
2020-01-05 20:11:26 +00:00
throw Exception("File is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED);
2021-04-27 00:05:43 +00:00
if (fs::exists(table_path) && fs::is_directory(table_path))
2020-01-05 20:11:26 +00:00
throw Exception("File must not be a directory", ErrorCodes::INCORRECT_FILE_NAME);
}
2019-09-06 18:29:41 +00:00
}
2021-04-26 13:34:44 +00:00
Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read)
{
2021-05-24 16:03:09 +00:00
fs::path user_files_absolute_path = fs::weakly_canonical(user_files_path);
2021-04-27 00:05:43 +00:00
fs::path fs_table_path(table_path);
if (fs_table_path.is_relative())
fs_table_path = user_files_absolute_path / fs_table_path;
Strings paths;
2021-05-24 16:03:09 +00:00
const String path = fs::weakly_canonical(fs_table_path);
if (path.find_first_of("*?{") == std::string::npos)
2021-04-26 13:34:44 +00:00
{
2021-04-27 09:54:12 +00:00
std::error_code error;
if (fs::exists(path))
total_bytes_to_read += fs::file_size(path, error);
paths.push_back(path);
2021-04-26 13:34:44 +00:00
}
else
2021-04-26 13:34:44 +00:00
paths = listFilesWithRegexpMatching("/", path, total_bytes_to_read);
for (const auto & cur_path : paths)
checkCreationIsAllowed(context, user_files_absolute_path, cur_path);
return paths;
}
2021-03-31 14:21:19 +00:00
bool StorageFile::isColumnOriented() const
{
return format_name != "Distributed" && FormatFactory::instance().checkIfFormatIsColumnOriented(format_name);
}
2019-10-30 14:17:55 +00:00
StorageFile::StorageFile(int table_fd_, CommonArguments args)
: StorageFile(args)
{
2021-08-22 17:21:49 +00:00
struct stat buf;
int res = fstat(table_fd_, &buf);
if (-1 == res)
throwFromErrno("Cannot execute fstat", res, ErrorCodes::CANNOT_FSTAT);
total_bytes_to_read = buf.st_size;
if (args.getContext()->getApplicationType() == Context::ApplicationType::SERVER)
2019-10-30 14:17:55 +00:00
throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED);
if (args.format_name == "Distributed")
throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME);
2019-08-24 21:20:20 +00:00
2019-10-30 14:17:55 +00:00
is_db_table = false;
use_table_fd = true;
table_fd = table_fd_;
}
StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args)
2019-10-30 14:17:55 +00:00
: StorageFile(args)
{
is_db_table = false;
2021-04-26 13:34:44 +00:00
paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read);
2020-01-04 18:05:42 +00:00
if (args.format_name == "Distributed")
{
if (paths.empty())
throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME);
auto & first_path = paths[0];
2021-07-20 18:18:43 +00:00
Block header = StorageDistributedDirectoryMonitor::createSourceFromFile(first_path)->getOutputs().front().getHeader();
StorageInMemoryMetadata storage_metadata;
auto columns = ColumnsDescription(header.getNamesAndTypesList());
if (!args.columns.empty() && columns != args.columns)
throw Exception("Table structure and file structure are different", ErrorCodes::INCOMPATIBLE_COLUMNS);
storage_metadata.setColumns(columns);
setInMemoryMetadata(storage_metadata);
2020-01-04 18:05:42 +00:00
}
2019-10-30 14:17:55 +00:00
}
2019-10-30 14:17:55 +00:00
StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArguments args)
: StorageFile(args)
{
if (relative_table_dir_path.empty())
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
if (args.format_name == "Distributed")
throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME);
2021-05-08 10:59:55 +00:00
String table_dir_path = fs::path(base_path) / relative_table_dir_path / "";
2021-04-27 00:05:43 +00:00
fs::create_directories(table_dir_path);
2019-10-30 14:17:55 +00:00
paths = {getTablePath(table_dir_path, format_name)};
2021-08-22 18:41:09 +00:00
if (fs::exists(paths[0]))
total_bytes_to_read = fs::file_size(paths[0]);
}
2019-10-30 14:17:55 +00:00
StorageFile::StorageFile(CommonArguments args)
2020-04-27 13:55:30 +00:00
: IStorage(args.table_id)
2019-12-04 16:06:55 +00:00
, format_name(args.format_name)
, format_settings(args.format_settings)
2019-12-04 16:06:55 +00:00
, compression_method(args.compression_method)
, base_path(args.getContext()->getPath())
2019-10-30 14:17:55 +00:00
{
2020-06-19 15:39:41 +00:00
StorageInMemoryMetadata storage_metadata;
2020-01-04 18:37:31 +00:00
if (args.format_name != "Distributed")
2020-06-19 15:39:41 +00:00
storage_metadata.setColumns(args.columns);
2020-01-04 18:37:31 +00:00
2020-06-19 15:39:41 +00:00
storage_metadata.setConstraints(args.constraints);
2021-04-23 12:18:23 +00:00
storage_metadata.setComment(args.comment);
2020-06-19 15:39:41 +00:00
setInMemoryMetadata(storage_metadata);
2019-10-30 14:17:55 +00:00
}
static std::chrono::seconds getLockTimeout(ContextPtr context)
{
const Settings & settings = context->getSettingsRef();
Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds();
if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout)
lock_timeout = settings.max_execution_time.totalSeconds();
return std::chrono::seconds{lock_timeout};
}
2021-03-30 17:57:21 +00:00
using StorageFilePtr = std::shared_ptr<StorageFile>;
class StorageFileSource : public SourceWithProgress
{
public:
struct FilesInfo
{
std::vector<std::string> files;
std::atomic<size_t> next_file_to_read = 0;
bool need_path_column = false;
bool need_file_column = false;
};
using FilesInfoPtr = std::shared_ptr<FilesInfo>;
static Block getHeader(const StorageMetadataPtr & metadata_snapshot, bool need_path_column, bool need_file_column)
{
auto header = metadata_snapshot->getSampleBlock();
/// Note: AddingDefaultsBlockInputStream doesn't change header.
if (need_path_column)
header.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_path"});
if (need_file_column)
header.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_file"});
return header;
}
2021-03-30 17:57:21 +00:00
static Block getBlockForSource(
const StorageFilePtr & storage,
const StorageMetadataPtr & metadata_snapshot,
const ColumnsDescription & columns_description,
const FilesInfoPtr & files_info)
{
2021-03-31 14:21:19 +00:00
if (storage->isColumnOriented())
return metadata_snapshot->getSampleBlockForColumns(
columns_description.getNamesOfPhysical(), storage->getVirtuals(), storage->getStorageID());
2021-03-30 21:25:37 +00:00
else
2021-03-30 17:57:21 +00:00
return getHeader(metadata_snapshot, files_info->need_path_column, files_info->need_file_column);
}
StorageFileSource(
std::shared_ptr<StorageFile> storage_,
const StorageMetadataPtr & metadata_snapshot_,
ContextPtr context_,
UInt64 max_block_size_,
FilesInfoPtr files_info_,
2020-10-02 12:38:50 +00:00
ColumnsDescription columns_description_)
2021-03-30 17:57:21 +00:00
: SourceWithProgress(getBlockForSource(storage_, metadata_snapshot_, columns_description_, files_info_))
, storage(std::move(storage_))
, metadata_snapshot(metadata_snapshot_)
, files_info(std::move(files_info_))
2020-10-02 12:38:50 +00:00
, columns_description(std::move(columns_description_))
, context(context_)
, max_block_size(max_block_size_)
{
if (!storage->use_table_fd)
{
shared_lock = std::shared_lock(storage->rwlock, getLockTimeout(context));
if (!shared_lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
}
}
String getName() const override
{
return storage->getName();
}
Chunk generate() override
{
while (!finished_generate)
{
/// Open file lazily on first read. This is needed to avoid too many open files from different streams.
if (!reader)
{
if (!storage->use_table_fd)
{
auto current_file = files_info->next_file_to_read.fetch_add(1);
if (current_file >= files_info->files.size())
return {};
current_path = files_info->files[current_file];
/// Special case for distributed format. Defaults are not needed here.
if (storage->format_name == "Distributed")
{
pipeline = std::make_unique<QueryPipelineBuilder>();
2021-07-20 18:18:43 +00:00
pipeline->init(Pipe(StorageDistributedDirectoryMonitor::createSourceFromFile(current_path)));
reader = std::make_unique<PullingPipelineExecutor>(*pipeline);
continue;
}
}
std::unique_ptr<ReadBuffer> nested_buffer;
CompressionMethod method;
struct stat file_stat{};
if (storage->use_table_fd)
{
/// Check if file descriptor allows random reads (and reading it twice).
if (0 != fstat(storage->table_fd, &file_stat))
throwFromErrno("Cannot stat table file descriptor, inside " + storage->getName(), ErrorCodes::CANNOT_STAT);
if (S_ISREG(file_stat.st_mode))
nested_buffer = std::make_unique<ReadBufferFromFileDescriptorPRead>(storage->table_fd);
else
nested_buffer = std::make_unique<ReadBufferFromFileDescriptor>(storage->table_fd);
method = chooseCompressionMethod("", storage->compression_method);
}
else
{
/// Check if file descriptor allows random reads (and reading it twice).
if (0 != stat(current_path.c_str(), &file_stat))
throwFromErrno("Cannot stat file " + current_path, ErrorCodes::CANNOT_STAT);
if (S_ISREG(file_stat.st_mode))
nested_buffer = std::make_unique<ReadBufferFromFilePRead>(current_path, context->getSettingsRef().max_read_buffer_size);
else
nested_buffer = std::make_unique<ReadBufferFromFile>(current_path, context->getSettingsRef().max_read_buffer_size);
method = chooseCompressionMethod(current_path, storage->compression_method);
}
/// For clickhouse-local add progress callback to display progress bar.
if (context->getApplicationType() == Context::ApplicationType::LOCAL)
{
auto & in = static_cast<ReadBufferFromFileDescriptor &>(*nested_buffer);
in.setProgressCallback(context);
}
2020-08-03 17:38:11 +00:00
read_buf = wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method);
2021-03-30 17:57:21 +00:00
auto get_block_for_format = [&]() -> Block
{
2021-03-31 14:21:19 +00:00
if (storage->isColumnOriented())
2021-03-30 17:57:21 +00:00
return metadata_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical());
return metadata_snapshot->getSampleBlock();
};
auto format = FormatFactory::instance().getInput(
storage->format_name, *read_buf, get_block_for_format(), context, max_block_size, storage->format_settings);
2020-05-18 10:00:22 +00:00
pipeline = std::make_unique<QueryPipelineBuilder>();
2021-07-20 18:18:43 +00:00
pipeline->init(Pipe(format));
2020-10-02 12:38:50 +00:00
if (columns_description.hasDefaults())
2021-07-20 18:18:43 +00:00
{
pipeline->addSimpleTransform([&](const Block & header)
{
return std::make_shared<AddingDefaultsTransform>(header, columns_description, *format, context);
});
}
2021-07-20 18:18:43 +00:00
reader = std::make_unique<PullingPipelineExecutor>(*pipeline);
}
2021-07-20 18:18:43 +00:00
Chunk chunk;
if (reader->pull(chunk))
{
2021-07-20 18:18:43 +00:00
//Columns columns = res.getColumns();
UInt64 num_rows = chunk.getNumRows();
/// Enrich with virtual columns.
if (files_info->need_path_column)
{
auto column = DataTypeString().createColumnConst(num_rows, current_path);
2021-07-20 18:18:43 +00:00
chunk.addColumn(column->convertToFullColumnIfConst());
}
if (files_info->need_file_column)
{
size_t last_slash_pos = current_path.find_last_of('/');
auto file_name = current_path.substr(last_slash_pos + 1);
auto column = DataTypeString().createColumnConst(num_rows, std::move(file_name));
2021-07-20 18:18:43 +00:00
chunk.addColumn(column->convertToFullColumnIfConst());
}
2021-07-20 18:18:43 +00:00
return chunk;
}
/// Read only once for file descriptor.
if (storage->use_table_fd)
finished_generate = true;
/// Close file prematurely if stream was ended.
reader.reset();
2021-07-20 18:18:43 +00:00
pipeline.reset();
read_buf.reset();
}
return {};
}
2021-04-26 13:34:44 +00:00
private:
std::shared_ptr<StorageFile> storage;
StorageMetadataPtr metadata_snapshot;
FilesInfoPtr files_info;
String current_path;
Block sample_block;
std::unique_ptr<ReadBuffer> read_buf;
std::unique_ptr<QueryPipelineBuilder> pipeline;
2021-07-20 18:18:43 +00:00
std::unique_ptr<PullingPipelineExecutor> reader;
2020-10-02 12:38:50 +00:00
ColumnsDescription columns_description;
ContextPtr context; /// TODO Untangle potential issues with context lifetime.
UInt64 max_block_size;
bool finished_generate = false;
std::shared_lock<std::shared_timed_mutex> shared_lock;
};
2020-08-03 13:54:14 +00:00
Pipe StorageFile::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
SelectQueryInfo & /*query_info*/,
ContextPtr context,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
unsigned num_streams)
{
2019-07-21 13:15:04 +00:00
BlockInputStreams blocks_input;
2019-09-06 18:29:41 +00:00
if (use_table_fd) /// need to call ctr BlockInputStream
paths = {""}; /// when use fd, paths are empty
else
{
2021-04-27 00:05:43 +00:00
if (paths.size() == 1 && !fs::exists(paths[0]))
{
if (context->getSettingsRef().engine_file_empty_if_not_exists)
return Pipe(std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID())));
else
throw Exception("File " + paths[0] + " doesn't exist", ErrorCodes::FILE_DOESNT_EXIST);
}
}
auto files_info = std::make_shared<StorageFileSource::FilesInfo>();
files_info->files = paths;
for (const auto & column : column_names)
{
if (column == "_path")
files_info->need_path_column = true;
if (column == "_file")
files_info->need_file_column = true;
}
2020-01-04 14:45:11 +00:00
auto this_ptr = std::static_pointer_cast<StorageFile>(shared_from_this());
2020-01-04 14:45:11 +00:00
2020-02-14 10:22:05 +00:00
if (num_streams > paths.size())
num_streams = paths.size();
Pipes pipes;
pipes.reserve(num_streams);
/// Set total number of bytes to process. For progress bar.
auto progress_callback = context->getFileProgressCallback();
if (context->getApplicationType() == Context::ApplicationType::LOCAL && progress_callback)
progress_callback(FileProgress(0, total_bytes_to_read));
2021-04-26 13:34:44 +00:00
for (size_t i = 0; i < num_streams; ++i)
{
2021-03-30 17:57:21 +00:00
const auto get_columns_for_format = [&]() -> ColumnsDescription
{
2021-03-31 14:21:19 +00:00
if (isColumnOriented())
return ColumnsDescription{
metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getNamesAndTypesList()};
2021-03-30 17:57:21 +00:00
else
return metadata_snapshot->getColumns();
};
pipes.emplace_back(std::make_shared<StorageFileSource>(
2021-03-30 17:57:21 +00:00
this_ptr, metadata_snapshot, context, max_block_size, files_info, get_columns_for_format()));
}
2020-08-06 12:24:05 +00:00
return Pipe::unitePipes(std::move(pipes));
}
2021-07-26 14:47:29 +00:00
class StorageFileSink final : public SinkToStorage
{
public:
2021-07-23 14:25:35 +00:00
explicit StorageFileSink(
StorageFile & storage_,
const StorageMetadataPtr & metadata_snapshot_,
std::unique_lock<std::shared_timed_mutex> && lock_,
2019-10-30 14:17:55 +00:00
const CompressionMethod compression_method,
ContextPtr context,
const std::optional<FormatSettings> & format_settings,
int & flags)
2021-07-26 10:08:40 +00:00
: SinkToStorage(metadata_snapshot_->getSampleBlock())
2021-07-23 14:25:35 +00:00
, storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, lock(std::move(lock_))
{
if (!lock)
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
2020-07-07 11:45:20 +00:00
std::unique_ptr<WriteBufferFromFileDescriptor> naked_buffer = nullptr;
if (storage.use_table_fd)
{
naked_buffer = std::make_unique<WriteBufferFromFileDescriptor>(storage.table_fd, DBMS_DEFAULT_BUFFER_SIZE);
}
else
{
2019-09-06 18:29:41 +00:00
if (storage.paths.size() != 1)
throw Exception("Table '" + storage.getStorageID().getNameForLogs() + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED);
flags |= O_WRONLY | O_APPEND | O_CREAT;
naked_buffer = std::make_unique<WriteBufferFromFile>(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, flags);
}
2020-07-07 11:45:20 +00:00
/// In case of CSVWithNames we have already written prefix.
if (naked_buffer->size())
prefix_written = true;
write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3);
2020-12-30 03:07:30 +00:00
writer = FormatFactory::instance().getOutputStreamParallelIfPossible(storage.format_name,
*write_buf, metadata_snapshot->getSampleBlock(), context,
{}, format_settings);
}
2021-07-23 19:33:59 +00:00
String getName() const override { return "StorageFileSink"; }
2021-07-26 14:47:29 +00:00
void onStart() override
{
2020-07-07 11:45:20 +00:00
if (!prefix_written)
writer->writePrefix();
prefix_written = true;
2021-07-26 14:47:29 +00:00
}
2021-07-23 14:25:35 +00:00
2021-07-26 14:47:29 +00:00
void consume(Chunk chunk) override
{
writer->write(getHeader().cloneWithColumns(chunk.detachColumns()));
}
2021-07-23 14:25:35 +00:00
void onFinish() override
{
writer->writeSuffix();
}
2021-07-23 14:25:35 +00:00
// void flush() override
// {
// writer->flush();
// }
private:
StorageFile & storage;
StorageMetadataPtr metadata_snapshot;
std::unique_lock<std::shared_timed_mutex> lock;
std::unique_ptr<WriteBuffer> write_buf;
BlockOutputStreamPtr writer;
2020-07-07 11:45:20 +00:00
bool prefix_written{false};
};
2021-07-23 14:25:35 +00:00
SinkToStoragePtr StorageFile::write(
2017-12-01 21:13:25 +00:00
const ASTPtr & /*query*/,
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context)
{
2020-01-04 14:45:11 +00:00
if (format_name == "Distributed")
throw Exception("Method write is not implemented for Distributed format", ErrorCodes::NOT_IMPLEMENTED);
int flags = 0;
std::string path;
if (context->getSettingsRef().engine_file_truncate_on_insert)
flags |= O_TRUNC;
if (!paths.empty())
{
path = paths[0];
2021-04-27 00:05:43 +00:00
fs::create_directories(fs::path(path).parent_path());
}
2021-07-23 14:25:35 +00:00
return std::make_shared<StorageFileSink>(
*this,
metadata_snapshot,
std::unique_lock{rwlock, getLockTimeout(context)},
chooseCompressionMethod(path, compression_method),
context,
format_settings,
flags);
}
2020-11-01 17:38:43 +00:00
bool StorageFile::storesDataOnDisk() const
{
return is_db_table;
}
2019-09-06 08:53:32 +00:00
Strings StorageFile::getDataPaths() const
2019-09-04 19:55:56 +00:00
{
2019-09-05 18:09:19 +00:00
if (paths.empty())
2019-12-03 16:25:32 +00:00
throw Exception("Table '" + getStorageID().getNameForLogs() + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED);
2019-09-06 08:53:32 +00:00
return paths;
2019-09-04 19:55:56 +00:00
}
2020-04-07 14:05:51 +00:00
void StorageFile::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
{
if (!is_db_table)
2021-06-28 17:02:22 +00:00
throw Exception("Can't rename table " + getStorageID().getNameForLogs() + " bounded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED);
2019-09-04 11:11:30 +00:00
if (paths.size() != 1)
2019-12-03 16:25:32 +00:00
throw Exception("Can't rename table " + getStorageID().getNameForLogs() + " in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED);
2019-09-04 11:11:30 +00:00
std::string path_new = getTablePath(base_path + new_path_to_table_data, format_name);
if (path_new == paths[0])
return;
2021-04-27 00:05:43 +00:00
fs::create_directories(fs::path(path_new).parent_path());
fs::rename(paths[0], path_new);
2019-09-04 11:11:30 +00:00
paths[0] = std::move(path_new);
2020-04-07 14:05:51 +00:00
renameInMemory(new_table_id);
}
2020-06-18 10:29:13 +00:00
void StorageFile::truncate(
const ASTPtr & /*query*/,
const StorageMetadataPtr & /* metadata_snapshot */,
ContextPtr /* context */,
2020-06-18 16:10:47 +00:00
TableExclusiveLockHolder &)
2020-01-05 02:57:09 +00:00
{
if (paths.size() != 1)
throw Exception("Can't truncate table '" + getStorageID().getNameForLogs() + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED);
2020-01-05 02:57:09 +00:00
if (use_table_fd)
{
if (0 != ::ftruncate(table_fd, 0))
throwFromErrno("Cannot truncate file at fd " + toString(table_fd), ErrorCodes::CANNOT_TRUNCATE_FILE);
}
else
{
2021-04-27 00:05:43 +00:00
if (!fs::exists(paths[0]))
2020-01-05 20:11:26 +00:00
return;
2020-01-05 02:57:09 +00:00
if (0 != ::truncate(paths[0].c_str(), 0))
throwFromErrnoWithPath("Cannot truncate file " + paths[0], paths[0], ErrorCodes::CANNOT_TRUNCATE_FILE);
}
}
void registerStorageFile(StorageFactory & factory)
{
StorageFactory::StorageFeatures storage_features{
.supports_settings = true,
.source_access_type = AccessType::FILE
};
factory.registerStorage(
"File",
[](const StorageFactory::Arguments & factory_args)
{
StorageFile::CommonArguments storage_args
{
WithContext(factory_args.getContext()),
factory_args.table_id,
{},
{},
{},
factory_args.columns,
factory_args.constraints,
2021-04-23 12:18:23 +00:00
factory_args.comment,
};
ASTs & engine_args_ast = factory_args.engine_args;
if (!(engine_args_ast.size() >= 1 && engine_args_ast.size() <= 3)) // NOLINT
throw Exception(
"Storage File requires from 1 to 3 arguments: name of used format, source and compression_method.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
engine_args_ast[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[0], factory_args.getLocalContext());
storage_args.format_name = engine_args_ast[0]->as<ASTLiteral &>().value.safeGet<String>();
// Use format settings from global server context + settings from
// the SETTINGS clause of the create query. Settings from current
// session and user are ignored.
if (factory_args.storage_def->settings)
{
FormatFactorySettings user_format_settings;
2019-10-30 14:17:55 +00:00
// Apply changed settings from global context, but ignore the
// unknown ones, because we only have the format settings here.
const auto & changes = factory_args.getContext()->getSettingsRef().changes();
for (const auto & change : changes)
{
if (user_format_settings.has(change.name))
{
user_format_settings.set(change.name, change.value);
}
}
// Apply changes from SETTINGS clause, with validation.
user_format_settings.applyChanges(
factory_args.storage_def->settings->changes);
storage_args.format_settings = getFormatSettings(
factory_args.getContext(), user_format_settings);
}
else
{
storage_args.format_settings = getFormatSettings(
factory_args.getContext());
}
2019-10-30 14:17:55 +00:00
if (engine_args_ast.size() == 1) /// Table in database
return StorageFile::create(factory_args.relative_data_path, storage_args);
2019-10-30 14:17:55 +00:00
/// Will use FD if engine_args[1] is int literal or identifier with std* name
int source_fd = -1;
String source_path;
if (auto opt_name = tryGetIdentifierName(engine_args_ast[1]))
{
if (*opt_name == "stdin")
source_fd = STDIN_FILENO;
else if (*opt_name == "stdout")
source_fd = STDOUT_FILENO;
else if (*opt_name == "stderr")
source_fd = STDERR_FILENO;
else
throw Exception(
"Unknown identifier '" + *opt_name + "' in second arg of File storage constructor", ErrorCodes::UNKNOWN_IDENTIFIER);
}
else if (const auto * literal = engine_args_ast[1]->as<ASTLiteral>())
{
auto type = literal->value.getType();
if (type == Field::Types::Int64)
source_fd = static_cast<int>(literal->value.get<Int64>());
else if (type == Field::Types::UInt64)
source_fd = static_cast<int>(literal->value.get<UInt64>());
else if (type == Field::Types::String)
source_path = literal->value.get<String>();
else
throw Exception("Second argument must be path or file descriptor", ErrorCodes::BAD_ARGUMENTS);
}
if (engine_args_ast.size() == 3)
{
engine_args_ast[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[2], factory_args.getLocalContext());
storage_args.compression_method = engine_args_ast[2]->as<ASTLiteral &>().value.safeGet<String>();
}
2019-10-30 14:17:55 +00:00
else
storage_args.compression_method = "auto";
if (0 <= source_fd) /// File descriptor
return StorageFile::create(source_fd, storage_args);
else /// User's file
return StorageFile::create(source_path, factory_args.getContext()->getUserFilesPath(), storage_args);
},
storage_features);
}
NamesAndTypesList StorageFile::getVirtuals() const
2020-04-27 13:55:30 +00:00
{
return NamesAndTypesList{
2020-04-27 13:55:30 +00:00
{"_path", std::make_shared<DataTypeString>()},
{"_file", std::make_shared<DataTypeString>()}
};
}
2021-03-30 21:25:37 +00:00
}