2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/StorageFile.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/ReadBufferFromFile.h>
|
|
|
|
#include <IO/WriteBufferFromFile.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
2019-01-23 14:48:50 +00:00
|
|
|
#include <DataStreams/IBlockInputStream.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataStreams/IBlockOutputStream.h>
|
2018-07-11 12:05:04 +00:00
|
|
|
#include <DataStreams/AddingDefaultsBlockInputStream.h>
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/escapeForFileName.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2019-07-21 13:15:04 +00:00
|
|
|
#include <Common/parseGlobs.h>
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2016-10-25 13:49:07 +00:00
|
|
|
#include <fcntl.h>
|
|
|
|
|
2018-04-06 09:53:29 +00:00
|
|
|
#include <Poco/Path.h>
|
2018-04-10 07:09:50 +00:00
|
|
|
#include <Poco/File.h>
|
2017-12-18 02:37:08 +00:00
|
|
|
|
2019-07-21 13:15:04 +00:00
|
|
|
#include <re2/re2.h>
|
2019-08-27 15:20:31 +00:00
|
|
|
#include <filesystem>
|
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
|
|
|
|
2016-10-18 14:18:37 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-10-28 17:38:32 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR;
|
|
|
|
extern const int CANNOT_SEEK_THROUGH_FILE;
|
|
|
|
extern const int DATABASE_ACCESS_DENIED;
|
2017-12-30 00:36:06 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int UNKNOWN_IDENTIFIER;
|
2017-11-03 19:53:10 +00:00
|
|
|
extern const int INCORRECT_FILE_NAME;
|
2018-04-10 08:54:31 +00:00
|
|
|
extern const int FILE_DOESNT_EXIST;
|
2017-11-03 19:53:10 +00:00
|
|
|
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
2018-08-10 04:02:56 +00:00
|
|
|
}
|
2016-10-28 17:38:32 +00:00
|
|
|
|
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.
|
|
|
|
*/
|
2019-10-25 19:07:47 +00:00
|
|
|
static std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match)
|
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);
|
2019-08-08 14:26:02 +00:00
|
|
|
re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash)));
|
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);
|
|
|
|
if (!fs::exists(fs::path(prefix_without_globs.data())))
|
2019-08-10 16:00:01 +00:00
|
|
|
{
|
|
|
|
return result;
|
|
|
|
}
|
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
|
|
|
{
|
|
|
|
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.
|
2019-09-22 22:13:42 +00:00
|
|
|
Strings result_part = listFilesWithRegexpMatching(full_path + "/", suffix_with_globs.substr(next_slash));
|
2019-08-02 15:00:12 +00:00
|
|
|
std::move(result_part.begin(), result_part.end(), std::back_inserter(result));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return result;
|
|
|
|
}
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2019-10-25 19:07:47 +00:00
|
|
|
static std::string getTablePath(const std::string & table_dir_path, const std::string & format_name)
|
2016-10-18 14:18:37 +00:00
|
|
|
{
|
2019-10-25 19:07:47 +00:00
|
|
|
return table_dir_path + "/data." + escapeForFileName(format_name);
|
2016-11-11 17:01:02 +00:00
|
|
|
}
|
|
|
|
|
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 '..').
|
2019-11-07 14:52:12 +00:00
|
|
|
static void checkCreationIsAllowed(const Context & context_global, const std::string & db_dir_path, const std::string & table_path)
|
2016-11-11 17:01:02 +00:00
|
|
|
{
|
2018-04-06 09:53:29 +00:00
|
|
|
if (context_global.getApplicationType() != Context::ApplicationType::SERVER)
|
|
|
|
return;
|
|
|
|
|
2019-09-06 18:29:41 +00:00
|
|
|
if (!startsWith(table_path, db_dir_path))
|
2018-04-06 10:35:52 +00:00
|
|
|
throw Exception("Part path " + table_path + " is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED);
|
2018-04-10 07:09:50 +00:00
|
|
|
|
|
|
|
Poco::File table_path_poco_file = Poco::File(table_path);
|
|
|
|
if (!table_path_poco_file.exists())
|
2018-04-19 04:39:48 +00:00
|
|
|
throw Exception("File " + table_path + " is not exist", ErrorCodes::FILE_DOESNT_EXIST);
|
2018-04-10 07:09:50 +00:00
|
|
|
else if (table_path_poco_file.isDirectory())
|
|
|
|
throw Exception("File " + table_path + " must not be a directory", ErrorCodes::INCORRECT_FILE_NAME);
|
2016-10-18 14:18:37 +00:00
|
|
|
}
|
2019-09-06 18:29:41 +00:00
|
|
|
}
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2019-10-30 14:17:55 +00:00
|
|
|
StorageFile::StorageFile(int table_fd_, CommonArguments args)
|
|
|
|
: StorageFile(args)
|
2016-10-18 14:18:37 +00:00
|
|
|
{
|
2019-10-30 14:17:55 +00:00
|
|
|
if (context_global.getApplicationType() == Context::ApplicationType::SERVER)
|
|
|
|
throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED);
|
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_;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-10-30 14:17:55 +00:00
|
|
|
/// Save initial offset, it will be used for repeating SELECTs
|
|
|
|
/// If FD isn't seekable (lseek returns -1), then the second and subsequent SELECTs will fail.
|
|
|
|
table_fd_init_offset = lseek(table_fd, 0, SEEK_CUR);
|
|
|
|
}
|
2017-11-03 19:53:10 +00:00
|
|
|
|
2019-10-30 14:17:55 +00:00
|
|
|
StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_absolute_path, CommonArguments args)
|
|
|
|
: StorageFile(args)
|
|
|
|
{
|
|
|
|
is_db_table = false;
|
|
|
|
Poco::Path poco_path = Poco::Path(table_path_);
|
|
|
|
if (poco_path.isRelative())
|
|
|
|
poco_path = Poco::Path(user_files_absolute_path, poco_path);
|
|
|
|
|
|
|
|
const std::string path = poco_path.absolute().toString();
|
|
|
|
paths = listFilesWithRegexpMatching("/", path);
|
|
|
|
for (const auto & cur_path : paths)
|
|
|
|
checkCreationIsAllowed(context_global, user_files_absolute_path, cur_path);
|
|
|
|
}
|
2018-04-06 09:53:29 +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);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-10-30 14:17:55 +00:00
|
|
|
String table_dir_path = context_global.getPath() + relative_table_dir_path + "/";
|
|
|
|
Poco::File(table_dir_path).createDirectories();
|
|
|
|
paths = {getTablePath(table_dir_path, format_name)};
|
2016-10-18 14:18:37 +00:00
|
|
|
}
|
|
|
|
|
2019-10-30 14:17:55 +00:00
|
|
|
StorageFile::StorageFile(CommonArguments args)
|
|
|
|
: table_name(args.table_name), database_name(args.database_name), format_name(args.format_name), context_global(args.context)
|
|
|
|
{
|
|
|
|
setColumns(args.columns);
|
|
|
|
setConstraints(args.constraints);
|
|
|
|
}
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2019-01-23 14:48:50 +00:00
|
|
|
class StorageFileBlockInputStream : public IBlockInputStream
|
2016-10-18 14:18:37 +00:00
|
|
|
{
|
|
|
|
public:
|
2019-08-08 19:16:17 +00:00
|
|
|
StorageFileBlockInputStream(StorageFile & storage_, const Context & context, UInt64 max_block_size, std::string file_path)
|
2017-07-28 17:34:02 +00:00
|
|
|
: storage(storage_)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
if (storage.use_table_fd)
|
|
|
|
{
|
2019-01-27 00:38:30 +00:00
|
|
|
unique_lock = std::unique_lock(storage.rwlock);
|
2017-07-28 17:34:02 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// We could use common ReadBuffer and WriteBuffer in storage to leverage cache
|
|
|
|
/// and add ability to seek unseekable files, but cache sync isn't supported.
|
|
|
|
|
|
|
|
if (storage.table_fd_was_used) /// We need seek to initial position
|
|
|
|
{
|
|
|
|
if (storage.table_fd_init_offset < 0)
|
|
|
|
throw Exception("File descriptor isn't seekable, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
|
|
|
|
2019-01-22 19:56:53 +00:00
|
|
|
/// ReadBuffer's seek() doesn't make sense, since cache is empty
|
2017-04-01 07:20:54 +00:00
|
|
|
if (lseek(storage.table_fd, storage.table_fd_init_offset, SEEK_SET) < 0)
|
|
|
|
throwFromErrno("Cannot seek file descriptor, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
|
|
|
}
|
|
|
|
|
|
|
|
storage.table_fd_was_used = true;
|
|
|
|
read_buf = std::make_unique<ReadBufferFromFileDescriptor>(storage.table_fd);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2019-01-27 00:38:30 +00:00
|
|
|
shared_lock = std::shared_lock(storage.rwlock);
|
2019-08-08 19:16:17 +00:00
|
|
|
read_buf = std::make_unique<ReadBufferFromFile>(file_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
return storage.getName();
|
|
|
|
}
|
|
|
|
|
|
|
|
Block readImpl() override
|
|
|
|
{
|
|
|
|
return reader->read();
|
|
|
|
}
|
|
|
|
|
2018-06-03 20:39:06 +00:00
|
|
|
Block getHeader() const override { return reader->getHeader(); }
|
2018-01-06 18:10:44 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void readPrefixImpl() override
|
|
|
|
{
|
|
|
|
reader->readPrefix();
|
|
|
|
}
|
|
|
|
|
|
|
|
void readSuffixImpl() override
|
|
|
|
{
|
|
|
|
reader->readSuffix();
|
|
|
|
}
|
2016-10-18 14:18:37 +00:00
|
|
|
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
StorageFile & storage;
|
|
|
|
Block sample_block;
|
|
|
|
std::unique_ptr<ReadBufferFromFileDescriptor> read_buf;
|
|
|
|
BlockInputStreamPtr reader;
|
2019-01-27 00:38:30 +00:00
|
|
|
|
|
|
|
std::shared_lock<std::shared_mutex> shared_lock;
|
|
|
|
std::unique_lock<std::shared_mutex> unique_lock;
|
2016-10-18 14:18:37 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
BlockInputStreams StorageFile::read(
|
2017-12-01 21:13:25 +00:00
|
|
|
const Names & /*column_names*/,
|
|
|
|
const SelectQueryInfo & /*query_info*/,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context,
|
2018-09-08 11:29:23 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
2019-02-18 23:38:44 +00:00
|
|
|
size_t max_block_size,
|
2017-12-01 21:13:25 +00:00
|
|
|
unsigned /*num_streams*/)
|
2016-10-18 14:18:37 +00:00
|
|
|
{
|
2019-08-03 11:02:40 +00:00
|
|
|
const ColumnsDescription & columns_ = getColumns();
|
|
|
|
auto column_defaults = columns_.getDefaults();
|
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
|
2019-09-04 11:11:30 +00:00
|
|
|
blocks_input.reserve(paths.size());
|
|
|
|
for (const auto & file_path : paths)
|
2019-07-21 13:15:04 +00:00
|
|
|
{
|
2019-08-08 19:16:17 +00:00
|
|
|
BlockInputStreamPtr cur_block = std::make_shared<StorageFileBlockInputStream>(*this, context, max_block_size, file_path);
|
2019-08-05 23:10:19 +00:00
|
|
|
blocks_input.push_back(column_defaults.empty() ? cur_block : std::make_shared<AddingDefaultsBlockInputStream>(cur_block, column_defaults, context));
|
2019-07-21 13:15:04 +00:00
|
|
|
}
|
|
|
|
return blocks_input;
|
2016-10-18 14:18:37 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
class StorageFileBlockOutputStream : public IBlockOutputStream
|
|
|
|
{
|
|
|
|
public:
|
2017-09-07 21:04:48 +00:00
|
|
|
explicit StorageFileBlockOutputStream(StorageFile & storage_)
|
2017-04-01 07:20:54 +00:00
|
|
|
: storage(storage_), lock(storage.rwlock)
|
|
|
|
{
|
|
|
|
if (storage.use_table_fd)
|
|
|
|
{
|
|
|
|
/** NOTE: Using real file binded to FD may be misleading:
|
|
|
|
* SELECT *; INSERT insert_data; SELECT *; last SELECT returns initil_fd_data + insert_data
|
|
|
|
* INSERT data; SELECT *; last SELECT returns only insert_data
|
|
|
|
*/
|
|
|
|
storage.table_fd_was_used = true;
|
|
|
|
write_buf = std::make_unique<WriteBufferFromFileDescriptor>(storage.table_fd);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2019-09-06 18:29:41 +00:00
|
|
|
if (storage.paths.size() != 1)
|
2019-09-20 11:26:00 +00:00
|
|
|
throw Exception("Table '" + storage.table_name + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED);
|
2019-09-04 11:11:30 +00:00
|
|
|
write_buf = std::make_unique<WriteBufferFromFile>(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-02-19 00:45:32 +00:00
|
|
|
Block getHeader() const override { return storage.getSampleBlock(); }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void write(const Block & block) override
|
|
|
|
{
|
|
|
|
writer->write(block);
|
|
|
|
}
|
|
|
|
|
|
|
|
void writePrefix() override
|
|
|
|
{
|
|
|
|
writer->writePrefix();
|
|
|
|
}
|
|
|
|
|
|
|
|
void writeSuffix() override
|
|
|
|
{
|
|
|
|
writer->writeSuffix();
|
|
|
|
}
|
|
|
|
|
|
|
|
void flush() override
|
|
|
|
{
|
|
|
|
writer->flush();
|
|
|
|
}
|
2016-10-18 14:18:37 +00:00
|
|
|
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
StorageFile & storage;
|
2017-07-28 17:34:02 +00:00
|
|
|
std::unique_lock<std::shared_mutex> lock;
|
2017-04-01 07:20:54 +00:00
|
|
|
std::unique_ptr<WriteBufferFromFileDescriptor> write_buf;
|
|
|
|
BlockOutputStreamPtr writer;
|
2016-10-18 14:18:37 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
BlockOutputStreamPtr StorageFile::write(
|
2017-12-01 21:13:25 +00:00
|
|
|
const ASTPtr & /*query*/,
|
2019-02-27 18:26:24 +00:00
|
|
|
const Context & /*context*/)
|
2016-10-18 14:18:37 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return std::make_shared<StorageFileBlockOutputStream>(*this);
|
2016-10-18 14:18:37 +00:00
|
|
|
}
|
|
|
|
|
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-09-04 19:55:56 +00:00
|
|
|
throw Exception("Table '" + table_name + "' 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
|
|
|
}
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2019-11-11 14:28:28 +00:00
|
|
|
void StorageFile::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
2016-10-18 14:18:37 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!is_db_table)
|
|
|
|
throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED);
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2019-09-04 11:11:30 +00:00
|
|
|
if (paths.size() != 1)
|
|
|
|
throw Exception("Can't rename table '" + table_name + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED);
|
|
|
|
|
2017-07-28 17:34:02 +00:00
|
|
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2019-11-11 14:28:28 +00:00
|
|
|
std::string path_new = getTablePath(context_global.getPath() + new_path_to_table_data, format_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
Poco::File(Poco::Path(path_new).parent()).createDirectories();
|
2019-09-04 11:11:30 +00:00
|
|
|
Poco::File(paths[0]).renameTo(path_new);
|
2016-10-18 14:18:37 +00:00
|
|
|
|
2019-09-04 11:11:30 +00:00
|
|
|
paths[0] = std::move(path_new);
|
2019-07-09 15:40:21 +00:00
|
|
|
table_name = new_table_name;
|
|
|
|
database_name = new_database_name;
|
2016-10-18 14:18:37 +00:00
|
|
|
}
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
|
|
|
|
void registerStorageFile(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("File", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
ASTs & engine_args = args.engine_args;
|
|
|
|
|
|
|
|
if (!(engine_args.size() == 1 || engine_args.size() == 2))
|
|
|
|
throw Exception(
|
|
|
|
"Storage File requires 1 or 2 arguments: name of used format and source.",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context);
|
2019-03-15 17:09:14 +00:00
|
|
|
String format_name = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2019-10-30 14:17:55 +00:00
|
|
|
StorageFile::CommonArguments common_args{args.database_name, args.table_name, format_name,
|
2019-11-07 14:52:12 +00:00
|
|
|
args.columns, args.constraints, args.context};
|
2019-10-30 14:17:55 +00:00
|
|
|
|
|
|
|
if (engine_args.size() == 1) /// Table in database
|
|
|
|
return StorageFile::create(args.relative_data_path, common_args);
|
|
|
|
|
|
|
|
/// Will use FD if engine_args[1] is int literal or identifier with std* name
|
2017-12-30 00:36:06 +00:00
|
|
|
int source_fd = -1;
|
|
|
|
String source_path;
|
2019-10-30 14:17:55 +00:00
|
|
|
if (auto opt_name = tryGetIdentifierName(engine_args[1]))
|
2017-12-30 00:36:06 +00:00
|
|
|
{
|
2019-10-30 14:17:55 +00:00
|
|
|
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[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);
|
2017-12-30 00:36:06 +00:00
|
|
|
}
|
|
|
|
|
2019-10-30 14:17:55 +00:00
|
|
|
if (0 <= source_fd) /// File descriptor
|
|
|
|
return StorageFile::create(source_fd, common_args);
|
|
|
|
else /// User's file
|
|
|
|
return StorageFile::create(source_path, args.context.getUserFilesPath(), common_args);
|
2017-12-30 00:36:06 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2016-10-18 14:18:37 +00:00
|
|
|
}
|