diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index a005fcb5fbb..45e9dfa89a7 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -9,10 +9,11 @@ #include #include -#include +#include #include "executeQuery.h" + namespace DB { @@ -48,7 +49,7 @@ void waitQuery(Connection & connection) } } -namespace fs = boost::filesystem; +namespace fs = std::filesystem; PerformanceTest::PerformanceTest( const XMLConfigurationPtr & config_, diff --git a/dbms/programs/performance-test/PerformanceTestInfo.cpp b/dbms/programs/performance-test/PerformanceTestInfo.cpp index f016b257c5f..40a066aa0a7 100644 --- a/dbms/programs/performance-test/PerformanceTestInfo.cpp +++ b/dbms/programs/performance-test/PerformanceTestInfo.cpp @@ -3,10 +3,11 @@ #include #include #include -#include #include "applySubstitutions.h" +#include #include + namespace DB { namespace ErrorCodes @@ -39,7 +40,7 @@ void extractSettings( } -namespace fs = boost::filesystem; +namespace fs = std::filesystem; PerformanceTestInfo::PerformanceTestInfo( XMLConfigurationPtr config, diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp index 91314a0fbff..6ddcc67f48a 100644 --- a/dbms/programs/performance-test/PerformanceTestSuite.cpp +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -4,11 +4,11 @@ #include #include #include +#include #include #include -#include #include #include @@ -36,7 +36,7 @@ #include "ReportBuilder.h" -namespace fs = boost::filesystem; +namespace fs = std::filesystem; namespace po = boost::program_options; namespace DB diff --git a/dbms/src/Common/QueryProfiler.cpp b/dbms/src/Common/QueryProfiler.cpp index 38c223678d7..bcdfcb5ae24 100644 --- a/dbms/src/Common/QueryProfiler.cpp +++ b/dbms/src/Common/QueryProfiler.cpp @@ -85,7 +85,8 @@ namespace constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag 8 * sizeof(char) + // maximum VarUInt length for string size QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length - sizeof(StackTrace) + // collected stack trace + sizeof(UInt8) + // number of stack frames + sizeof(StackTrace::Frames) + // collected stack trace, maximum capacity sizeof(TimerType) + // timer type sizeof(UInt32); // thread_number char buffer[buf_size]; @@ -101,7 +102,13 @@ namespace writeChar(false, out); writeStringBinary(query_id, out); - writePODBinary(stack_trace, out); + + size_t stack_trace_size = stack_trace.getSize(); + size_t stack_trace_offset = stack_trace.getOffset(); + writeIntBinary(UInt8(stack_trace_size - stack_trace_offset), out); + for (size_t i = stack_trace_offset; i < stack_trace_size; ++i) + writePODBinary(stack_trace.getFrames()[i], out); + writePODBinary(timer_type, out); writePODBinary(thread_number, out); out.next(); diff --git a/dbms/src/Common/StackTrace.cpp b/dbms/src/Common/StackTrace.cpp index 30fb66f218e..a39eaf484e4 100644 --- a/dbms/src/Common/StackTrace.cpp +++ b/dbms/src/Common/StackTrace.cpp @@ -155,7 +155,7 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext return error.str(); } -void * getCallerAddress(const ucontext_t & context) +static void * getCallerAddress(const ucontext_t & context) { #if defined(__x86_64__) /// Get the address at the time the signal was raised from the RIP (x86-64) @@ -182,12 +182,25 @@ StackTrace::StackTrace(const ucontext_t & signal_context) { tryCapture(); - if (size == 0) + void * caller_address = getCallerAddress(signal_context); + + if (size == 0 && caller_address) { - /// No stack trace was captured. At least we can try parsing caller address - void * caller_address = getCallerAddress(signal_context); - if (caller_address) - frames[size++] = reinterpret_cast(caller_address); + frames[0] = caller_address; + size = 1; + } + else + { + /// Skip excessive stack frames that we have created while finding stack trace. + + for (size_t i = 0; i < size; ++i) + { + if (frames[i] == caller_address) + { + offset = i; + break; + } + } } } @@ -214,21 +227,18 @@ size_t StackTrace::getSize() const return size; } +size_t StackTrace::getOffset() const +{ + return offset; +} + const StackTrace::Frames & StackTrace::getFrames() const { return frames; } -std::string StackTrace::toString() const -{ - /// Calculation of stack trace text is extremely slow. - /// We use simple cache because otherwise the server could be overloaded by trash queries. - static SimpleCache func_cached; - return func_cached(frames, size); -} - -std::string StackTrace::toStringImpl(const Frames & frames, size_t size) +static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset, size_t size) { if (size == 0) return ""; @@ -238,7 +248,7 @@ std::string StackTrace::toStringImpl(const Frames & frames, size_t size) std::stringstream out; - for (size_t i = 0; i < size; ++i) + for (size_t i = offset; i < size; ++i) { const void * addr = frames[i]; @@ -275,3 +285,12 @@ std::string StackTrace::toStringImpl(const Frames & frames, size_t size) return out.str(); } + +std::string StackTrace::toString() const +{ + /// Calculation of stack trace text is extremely slow. + /// We use simple cache because otherwise the server could be overloaded by trash queries. + + static SimpleCache func_cached; + return func_cached(frames, offset, size); +} diff --git a/dbms/src/Common/StackTrace.h b/dbms/src/Common/StackTrace.h index 997730eb440..40e1bbbe504 100644 --- a/dbms/src/Common/StackTrace.h +++ b/dbms/src/Common/StackTrace.h @@ -34,28 +34,17 @@ public: /// Creates empty object for deferred initialization StackTrace(NoCapture); - /// Fills stack trace frames with provided sequence - template - StackTrace(Iterator it, Iterator end) - { - while (size < capacity && it != end) - { - frames[size++] = *(it++); - } - } - size_t getSize() const; + size_t getOffset() const; const Frames & getFrames() const; std::string toString() const; protected: void tryCapture(); - static std::string toStringImpl(const Frames & frames, size_t size); size_t size = 0; + size_t offset = 0; /// How many frames to skip while displaying. Frames frames{}; }; std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context); - -void * getCallerAddress(const ucontext_t & context); diff --git a/dbms/src/Common/TraceCollector.cpp b/dbms/src/Common/TraceCollector.cpp index 13d2061c810..aad69fa4d40 100644 --- a/dbms/src/Common/TraceCollector.cpp +++ b/dbms/src/Common/TraceCollector.cpp @@ -103,25 +103,28 @@ void TraceCollector::run() break; std::string query_id; - StackTrace stack_trace(NoCapture{}); - TimerType timer_type; - UInt32 thread_number; - readStringBinary(query_id, in); - readPODBinary(stack_trace, in); - readPODBinary(timer_type, in); - readPODBinary(thread_number, in); - const auto size = stack_trace.getSize(); - const auto & frames = stack_trace.getFrames(); + UInt8 size = 0; + readIntBinary(size, in); Array trace; trace.reserve(size); + for (size_t i = 0; i < size; i++) - trace.emplace_back(UInt64(reinterpret_cast(frames[i]))); + { + uintptr_t addr = 0; + readPODBinary(addr, in); + trace.emplace_back(UInt64(addr)); + } + + TimerType timer_type; + readPODBinary(timer_type, in); + + UInt32 thread_number; + readPODBinary(thread_number, in); TraceLogElement element{std::time(nullptr), timer_type, thread_number, query_id, trace}; - trace_log->add(element); } } diff --git a/dbms/src/Common/tests/compact_array.cpp b/dbms/src/Common/tests/compact_array.cpp index 14c6b819c37..3714b6ef176 100644 --- a/dbms/src/Common/tests/compact_array.cpp +++ b/dbms/src/Common/tests/compact_array.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -15,7 +15,7 @@ #include #include -namespace fs = boost::filesystem; +namespace fs = std::filesystem; std::string createTmpPath(const std::string & filename) { diff --git a/dbms/src/Functions/filesystem.cpp b/dbms/src/Functions/filesystem.cpp index 9859b5ed2f2..3aa27a81559 100644 --- a/dbms/src/Functions/filesystem.cpp +++ b/dbms/src/Functions/filesystem.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include namespace DB @@ -11,19 +11,19 @@ namespace DB struct FilesystemAvailable { static constexpr auto name = "filesystemAvailable"; - static boost::uintmax_t get(boost::filesystem::space_info & spaceinfo) { return spaceinfo.available; } + static std::uintmax_t get(std::filesystem::space_info & spaceinfo) { return spaceinfo.available; } }; struct FilesystemFree { static constexpr auto name = "filesystemFree"; - static boost::uintmax_t get(boost::filesystem::space_info & spaceinfo) { return spaceinfo.free; } + static std::uintmax_t get(std::filesystem::space_info & spaceinfo) { return spaceinfo.free; } }; struct FilesystemCapacity { static constexpr auto name = "filesystemCapacity"; - static boost::uintmax_t get(boost::filesystem::space_info & spaceinfo) { return spaceinfo.capacity; } + static std::uintmax_t get(std::filesystem::space_info & spaceinfo) { return spaceinfo.capacity; } }; template @@ -34,10 +34,10 @@ public: static FunctionPtr create(const Context & context) { - return std::make_shared>(boost::filesystem::space(context.getConfigRef().getString("path"))); + return std::make_shared>(std::filesystem::space(context.getConfigRef().getString("path"))); } - explicit FilesystemImpl(boost::filesystem::space_info spaceinfo_) : spaceinfo(spaceinfo_) { } + explicit FilesystemImpl(std::filesystem::space_info spaceinfo_) : spaceinfo(spaceinfo_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -54,7 +54,7 @@ public: } private: - boost::filesystem::space_info spaceinfo; + std::filesystem::space_info spaceinfo; }; diff --git a/dbms/src/IO/tests/read_buffer_aio.cpp b/dbms/src/IO/tests/read_buffer_aio.cpp index 81d04da79f2..adb2f7d5458 100644 --- a/dbms/src/IO/tests/read_buffer_aio.cpp +++ b/dbms/src/IO/tests/read_buffer_aio.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -44,7 +44,7 @@ bool test20(const std::string & filename, const std::string & buf); void run() { - namespace fs = boost::filesystem; + namespace fs = std::filesystem; std::string filename; std::string buf; diff --git a/dbms/src/IO/tests/write_buffer_aio.cpp b/dbms/src/IO/tests/write_buffer_aio.cpp index 4e76a91639c..5794e277848 100644 --- a/dbms/src/IO/tests/write_buffer_aio.cpp +++ b/dbms/src/IO/tests/write_buffer_aio.cpp @@ -1,8 +1,7 @@ #include #include -#include - +#include #include #include #include @@ -11,7 +10,7 @@ namespace { -namespace fs = boost::filesystem; +namespace fs = std::filesystem; void run(); [[noreturn]] void die(const std::string & msg); diff --git a/dbms/src/Interpreters/tests/users.cpp b/dbms/src/Interpreters/tests/users.cpp index 986d97ce43e..ae680ce5e5a 100644 --- a/dbms/src/Interpreters/tests/users.cpp +++ b/dbms/src/Interpreters/tests/users.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -14,7 +14,7 @@ namespace { -namespace fs = boost::filesystem; +namespace fs = std::filesystem; struct TestEntry { diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index bd3c8f671f0..98109ae9893 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -263,7 +263,7 @@ protected: class ParserColumnsOrIndicesDeclarationList : public IParserBase { - protected: +protected: const char * getName() const override { return "columns or indices declaration list"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 7b810484d1c..6155dabd028 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -47,7 +47,7 @@ #include #include -#include +#include namespace DB @@ -120,13 +120,13 @@ UInt64 getMaximumFileNumber(const std::string & dir_path) { UInt64 res = 0; - boost::filesystem::recursive_directory_iterator begin(dir_path); - boost::filesystem::recursive_directory_iterator end; + std::filesystem::recursive_directory_iterator begin(dir_path); + std::filesystem::recursive_directory_iterator end; for (auto it = begin; it != end; ++it) { const auto & file_path = it->path(); - if (it->status().type() != boost::filesystem::regular_file || !endsWith(file_path.filename().string(), ".bin")) + if (!std::filesystem::is_regular_file(*it) || !endsWith(file_path.filename().string(), ".bin")) continue; UInt64 num = 0; @@ -431,10 +431,10 @@ void StorageDistributed::createDirectoryMonitors() Poco::File{path}.createDirectory(); - boost::filesystem::directory_iterator begin(path); - boost::filesystem::directory_iterator end; + std::filesystem::directory_iterator begin(path); + std::filesystem::directory_iterator end; for (auto it = begin; it != end; ++it) - if (it->status().type() == boost::filesystem::directory_file) + if (std::filesystem::is_directory(*it)) requireDirectoryMonitor(it->path().filename().string()); } diff --git a/dbms/src/Storages/StorageValues.cpp b/dbms/src/Storages/StorageValues.cpp new file mode 100644 index 00000000000..bda44569cc6 --- /dev/null +++ b/dbms/src/Storages/StorageValues.cpp @@ -0,0 +1,28 @@ +#include +#include +#include + + +namespace DB +{ + +StorageValues::StorageValues(const std::string & database_name_, const std::string & table_name_, const Block & res_block_) + : database_name(database_name_), table_name(table_name_), res_block(res_block_) +{ + setColumns(ColumnsDescription(res_block.getNamesAndTypesList())); +} + +BlockInputStreams StorageValues::read( + const Names & column_names, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/) +{ + check(column_names); + + return BlockInputStreams(1, std::make_shared(res_block)); +} + +} diff --git a/dbms/src/Storages/StorageValues.h b/dbms/src/Storages/StorageValues.h new file mode 100644 index 00000000000..d26d1b27101 --- /dev/null +++ b/dbms/src/Storages/StorageValues.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include + + +namespace DB +{ +/* One block storage used for values table function + * It's structure is similar to IStorageSystemOneBlock + */ +class StorageValues : public ext::shared_ptr_helper, public IStorage +{ +public: + std::string getName() const override { return "Values"; } + std::string getTableName() const override { return table_name; } + std::string getDatabaseName() const override { return database_name; } + + BlockInputStreams read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + +private: + std::string database_name; + std::string table_name; + Block res_block; + +protected: + StorageValues(const std::string & database_name_, const std::string & table_name_, const Block & res_block_); +}; + +} diff --git a/dbms/src/Storages/System/StorageSystemTableFunctions.cpp b/dbms/src/Storages/System/StorageSystemTableFunctions.cpp index 15067bbc41f..367595e9742 100644 --- a/dbms/src/Storages/System/StorageSystemTableFunctions.cpp +++ b/dbms/src/Storages/System/StorageSystemTableFunctions.cpp @@ -11,10 +11,10 @@ NamesAndTypesList StorageSystemTableFunctions::getNamesAndTypes() void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const { - const auto & functions = TableFunctionFactory::instance().getAllTableFunctions(); - for (const auto & pair : functions) + const auto & functions_names = TableFunctionFactory::instance().getAllRegisteredNames(); + for (const auto & name : functions_names) { - res_columns[0]->insert(pair.first); + res_columns[0]->insert(name); } } diff --git a/dbms/src/TableFunctions/ITableFunctionFileLike.cpp b/dbms/src/TableFunctions/ITableFunctionFileLike.cpp index 7201af6ca06..0413839e819 100644 --- a/dbms/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/dbms/src/TableFunctions/ITableFunctionFileLike.cpp @@ -1,14 +1,17 @@ #include #include +#include + #include #include + #include #include + #include -#include + #include #include -#include namespace DB @@ -21,7 +24,7 @@ namespace ErrorCodes StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { - // Parse args + /// Parse args ASTs & args_func = ast_function->children; if (args_func.size() != 1) @@ -40,26 +43,12 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons std::string format = args[1]->as().value.safeGet(); std::string structure = args[2]->as().value.safeGet(); - // Create sample block - std::vector structure_vals; - boost::split(structure_vals, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on); - - if (structure_vals.size() % 2 != 0) - throw Exception("Odd number of elements in section structure: must be a list of name type pairs", ErrorCodes::LOGICAL_ERROR); + /// Create sample block Block sample_block; - const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); + parseColumnsListFromString(structure, sample_block, context); - for (size_t i = 0, size = structure_vals.size(); i < size; i += 2) - { - ColumnWithTypeAndName column; - column.name = structure_vals[i]; - column.type = data_type_factory.get(structure_vals[i + 1]); - column.column = column.type->createColumn(); - sample_block.insert(std::move(column)); - } - - // Create table + /// Create table StoragePtr storage = getStorage(filename, format, sample_block, const_cast(context), table_name); storage->startup(); diff --git a/dbms/src/TableFunctions/TableFunctionFactory.cpp b/dbms/src/TableFunctions/TableFunctionFactory.cpp index 7edd445379a..7ad98599f76 100644 --- a/dbms/src/TableFunctions/TableFunctionFactory.cpp +++ b/dbms/src/TableFunctions/TableFunctionFactory.cpp @@ -1,11 +1,10 @@ #include #include - #include - #include + namespace DB { @@ -17,11 +16,16 @@ namespace ErrorCodes } -void TableFunctionFactory::registerFunction(const std::string & name, Creator creator) +void TableFunctionFactory::registerFunction(const std::string & name, Creator creator, CaseSensitiveness case_sensitiveness) { - if (!functions.emplace(name, std::move(creator)).second) + if (!table_functions.emplace(name, creator).second) throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + + if (case_sensitiveness == CaseInsensitive + && !case_insensitive_table_functions.emplace(Poco::toLower(name), creator).second) + throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique", + ErrorCodes::LOGICAL_ERROR); } TableFunctionPtr TableFunctionFactory::get( @@ -31,8 +35,8 @@ TableFunctionPtr TableFunctionFactory::get( if (context.getSettings().readonly == 1) /** For example, for readonly = 2 - allowed. */ throw Exception("Table functions are forbidden in readonly mode", ErrorCodes::READONLY); - auto it = functions.find(name); - if (it == functions.end()) + auto res = tryGet(name, context); + if (!res) { auto hints = getHints(name); if (!hints.empty()) @@ -41,12 +45,29 @@ TableFunctionPtr TableFunctionFactory::get( throw Exception("Unknown table function " + name, ErrorCodes::UNKNOWN_FUNCTION); } - return it->second(); + return res; +} + +TableFunctionPtr TableFunctionFactory::tryGet( + const std::string & name_param, + const Context &) const +{ + String name = getAliasToOrName(name_param); + + auto it = table_functions.find(name); + if (table_functions.end() != it) + return it->second(); + + it = case_insensitive_table_functions.find(Poco::toLower(name)); + if (case_insensitive_table_functions.end() != it) + return it->second(); + + return {}; } bool TableFunctionFactory::isTableFunctionName(const std::string & name) const { - return functions.count(name); + return table_functions.count(name); } } diff --git a/dbms/src/TableFunctions/TableFunctionFactory.h b/dbms/src/TableFunctions/TableFunctionFactory.h index acbb6244c4e..ae1c3e997e7 100644 --- a/dbms/src/TableFunctions/TableFunctionFactory.h +++ b/dbms/src/TableFunctions/TableFunctionFactory.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -16,51 +17,47 @@ namespace DB class Context; +using TableFunctionCreator = std::function; /** Lets you get a table function by its name. */ -class TableFunctionFactory final: public ext::singleton, public IHints<1, TableFunctionFactory> +class TableFunctionFactory final: public ext::singleton, public IFactoryWithAliases { public: - using Creator = std::function; - using TableFunctions = std::unordered_map; /// Register a function by its name. /// No locking, you must register all functions before usage of get. - void registerFunction(const std::string & name, Creator creator); + void registerFunction(const std::string & name, Creator creator, CaseSensitiveness case_sensitiveness = CaseSensitive); template - void registerFunction() + void registerFunction(CaseSensitiveness case_sensitiveness = CaseSensitive) { auto creator = [] () -> TableFunctionPtr { return std::make_shared(); }; - registerFunction(Function::name, std::move(creator)); + registerFunction(Function::name, std::move(creator), case_sensitiveness); } /// Throws an exception if not found. - TableFunctionPtr get( - const std::string & name, - const Context & context) const; + TableFunctionPtr get(const std::string & name, const Context & context) const; + + /// Returns nullptr if not found. + TableFunctionPtr tryGet(const std::string & name, const Context & context) const; bool isTableFunctionName(const std::string & name) const; - const TableFunctions & getAllTableFunctions() const - { - return functions; - } - - std::vector getAllRegisteredNames() const override - { - std::vector result; - auto getter = [](const auto & pair) { return pair.first; }; - std::transform(functions.begin(), functions.end(), std::back_inserter(result), getter); - return result; - } - private: - TableFunctions functions; + using TableFunctions = std::unordered_map; + + const TableFunctions & getCreatorMap() const override { return table_functions; } + + const TableFunctions & getCaseInsensitiveCreatorMap() const override { return case_insensitive_table_functions; } + + String getFactoryName() const override { return "TableFunctionFactory"; } + + TableFunctions table_functions; + TableFunctions case_insensitive_table_functions; }; } diff --git a/dbms/src/TableFunctions/TableFunctionValues.cpp b/dbms/src/TableFunctions/TableFunctionValues.cpp new file mode 100644 index 00000000000..326e6eaace7 --- /dev/null +++ b/dbms/src/TableFunctions/TableFunctionValues.cpp @@ -0,0 +1,95 @@ +#include +#include + +#include +#include + +#include +#include +#include + +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args, const Block & sample_block, const Context & context) +{ + if (res_columns.size() == 1) /// Parsing arguments as Fields + { + for (size_t i = 1; i < args.size(); ++i) + { + const auto & [value_field, value_type_ptr] = evaluateConstantExpression(args[i], context); + + Field value = convertFieldToType(value_field, *sample_block.getByPosition(0).type, value_type_ptr.get()); + res_columns[0]->insert(value); + } + } + else /// Parsing arguments as Tuples + { + for (size_t i = 1; i < args.size(); ++i) + { + const auto & [value_field, value_type_ptr] = evaluateConstantExpression(args[i], context); + const TupleBackend & value_tuple = value_field.safeGet().toUnderType(); + + if (value_tuple.size() != sample_block.columns()) + throw Exception("Values size should match with number of columns", ErrorCodes::LOGICAL_ERROR); + + for (size_t j = 0; j < value_tuple.size(); ++j) + { + Field value = convertFieldToType(value_tuple[j], *sample_block.getByPosition(j).type, value_type_ptr.get()); + res_columns[j]->insert(value); + } + } + } +} + +StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + ASTs & args_func = ast_function->children; + + if (args_func.size() != 1) + throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::LOGICAL_ERROR); + + ASTs & args = args_func.at(0)->children; + + if (args.size() < 2) + throw Exception("Table function '" + getName() + "' requires 2 or more arguments: structure and values.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + /// Parsing first argument as table structure and creating a sample block + std::string structure = args[0]->as().value.safeGet(); + + Block sample_block; + parseColumnsListFromString(structure, sample_block, context); + + MutableColumns res_columns = sample_block.cloneEmptyColumns(); + + /// Parsing other arguments as values and inserting them into columns + parseAndInsertValues(res_columns, args, sample_block, context); + + Block res_block = sample_block.cloneWithColumns(std::move(res_columns)); + + auto res = StorageValues::create(getDatabaseName(), table_name, res_block); + res->startup(); + return res; +} + +void registerTableFunctionValues(TableFunctionFactory & factory) +{ + factory.registerFunction(TableFunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/TableFunctions/TableFunctionValues.h b/dbms/src/TableFunctions/TableFunctionValues.h new file mode 100644 index 00000000000..f02dc69162f --- /dev/null +++ b/dbms/src/TableFunctions/TableFunctionValues.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ +/* values(structure, values...) - creates a temporary storage filling columns with values + * values is case-insensitive table function + */ +class TableFunctionValues : public ITableFunction +{ +public: + static constexpr auto name = "values"; + std::string getName() const override { return name; } +private: + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; +}; + + +} diff --git a/dbms/src/TableFunctions/parseColumnsListForTableFunction.cpp b/dbms/src/TableFunctions/parseColumnsListForTableFunction.cpp new file mode 100644 index 00000000000..05d8aa7cddc --- /dev/null +++ b/dbms/src/TableFunctions/parseColumnsListForTableFunction.cpp @@ -0,0 +1,44 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + +void parseColumnsListFromString(const std::string & structure, Block & sample_block, const Context & context) +{ + Expected expected; + + Tokens tokens(structure.c_str(), structure.c_str() + structure.size()); + TokenIterator token_iterator(tokens); + + ParserColumnDeclarationList parser; + ASTPtr columns_list_raw; + + if (!parser.parse(token_iterator, columns_list_raw, expected)) + throw Exception("Cannot parse columns declaration list.", ErrorCodes::SYNTAX_ERROR); + + auto * columns_list = dynamic_cast(columns_list_raw.get()); + if (!columns_list) + throw Exception("Could not cast AST to ASTExpressionList", ErrorCodes::LOGICAL_ERROR); + + ColumnsDescription columns_desc = InterpreterCreateQuery::getColumnsDescription(*columns_list, context); + + for (const auto & [name, type]: columns_desc.getAllPhysical()) + { + ColumnWithTypeAndName column; + column.name = name; + column.type = type; + column.column = type->createColumn(); + sample_block.insert(std::move(column)); + } +} + +} diff --git a/dbms/src/TableFunctions/parseColumnsListForTableFunction.h b/dbms/src/TableFunctions/parseColumnsListForTableFunction.h new file mode 100644 index 00000000000..3e942afa358 --- /dev/null +++ b/dbms/src/TableFunctions/parseColumnsListForTableFunction.h @@ -0,0 +1,11 @@ +#pragma once + +#include + + +namespace DB +{ +/// Parses a common argument for table functions such as table structure given in string +void parseColumnsListFromString(const std::string & structure, Block & sample_block, const Context & context); + +} diff --git a/dbms/src/TableFunctions/registerTableFunctions.cpp b/dbms/src/TableFunctions/registerTableFunctions.cpp index 43583c8e1bb..76eeb23f6cc 100644 --- a/dbms/src/TableFunctions/registerTableFunctions.cpp +++ b/dbms/src/TableFunctions/registerTableFunctions.cpp @@ -13,6 +13,7 @@ void registerTableFunctionNumbers(TableFunctionFactory & factory); void registerTableFunctionCatBoostPool(TableFunctionFactory & factory); void registerTableFunctionFile(TableFunctionFactory & factory); void registerTableFunctionURL(TableFunctionFactory & factory); +void registerTableFunctionValues(TableFunctionFactory & factory); #if USE_HDFS void registerTableFunctionHDFS(TableFunctionFactory & factory); @@ -39,6 +40,7 @@ void registerTableFunctions() registerTableFunctionCatBoostPool(factory); registerTableFunctionFile(factory); registerTableFunctionURL(factory); + registerTableFunctionValues(factory); #if USE_HDFS registerTableFunctionHDFS(factory); diff --git a/dbms/tests/queries/0_stateless/00975_values_list.reference b/dbms/tests/queries/0_stateless/00975_values_list.reference new file mode 100644 index 00000000000..eee9e0a0ca5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_values_list.reference @@ -0,0 +1,13 @@ +1 one +2 two +3 three +1 one +2 two +3 three +2018-01-01 2018-01-01 00:00:00 +abra +cadabra +abracadabra +23 23 23 +24 24 24 +1.6660 a b diff --git a/dbms/tests/queries/0_stateless/00975_values_list.sql b/dbms/tests/queries/0_stateless/00975_values_list.sql new file mode 100644 index 00000000000..ad30cec21e9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_values_list.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS values_list; + +SELECT * FROM VALUES('a UInt64, s String', (1, 'one'), (2, 'two'), (3, 'three')); +CREATE TABLE values_list AS VALUES('a UInt64, s String', (1, 'one'), (2, 'two'), (3, 'three')); +SELECT * FROM values_list; + +SELECT subtractYears(date, 1), subtractYears(date_time, 1) FROM VALUES('date Date, date_time DateTime', (toDate('2019-01-01'), toDateTime('2019-01-01 00:00:00'))); + +SELECT * FROM VALUES('s String', ('abra'), ('cadabra'), ('abracadabra')); + +SELECT * FROM VALUES('n UInt64, s String, ss String', (1 + 22, '23', toString(23)), (toUInt64('24'), '24', concat('2', '4'))); + +SELECT * FROM VALUES('a Decimal(4, 4), b String, c String', (divide(toDecimal32(5, 3), 3), 'a', 'b')); +DROP TABLE values_list;