mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' into fastops
This commit is contained in:
commit
a25890c038
@ -9,10 +9,11 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
|
||||
#include <boost/filesystem.hpp>
|
||||
#include <filesystem>
|
||||
|
||||
#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_,
|
||||
|
@ -3,10 +3,11 @@
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <boost/filesystem.hpp>
|
||||
#include "applySubstitutions.h"
|
||||
#include <filesystem>
|
||||
#include <iostream>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
@ -39,7 +40,7 @@ void extractSettings(
|
||||
}
|
||||
|
||||
|
||||
namespace fs = boost::filesystem;
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
PerformanceTestInfo::PerformanceTestInfo(
|
||||
XMLConfigurationPtr config,
|
||||
|
@ -4,11 +4,11 @@
|
||||
#include <regex>
|
||||
#include <thread>
|
||||
#include <memory>
|
||||
#include <filesystem>
|
||||
|
||||
#include <port/unistd.h>
|
||||
#include <sys/stat.h>
|
||||
|
||||
#include <boost/filesystem.hpp>
|
||||
#include <boost/program_options.hpp>
|
||||
|
||||
#include <Poco/AutoPtr.h>
|
||||
@ -36,7 +36,7 @@
|
||||
#include "ReportBuilder.h"
|
||||
|
||||
|
||||
namespace fs = boost::filesystem;
|
||||
namespace fs = std::filesystem;
|
||||
namespace po = boost::program_options;
|
||||
|
||||
namespace DB
|
||||
|
@ -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();
|
||||
|
@ -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<void *>(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<decltype(StackTrace::toStringImpl), &StackTrace::toStringImpl> 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 "<Empty trace>";
|
||||
@ -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<decltype(toStringImpl), &toStringImpl> func_cached;
|
||||
return func_cached(frames, offset, size);
|
||||
}
|
||||
|
@ -34,28 +34,17 @@ public:
|
||||
/// Creates empty object for deferred initialization
|
||||
StackTrace(NoCapture);
|
||||
|
||||
/// Fills stack trace frames with provided sequence
|
||||
template <typename Iterator>
|
||||
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);
|
||||
|
@ -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<uintptr_t>(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);
|
||||
}
|
||||
}
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <Common/CompactArray.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <boost/filesystem.hpp>
|
||||
#include <filesystem>
|
||||
#include <string>
|
||||
#include <iostream>
|
||||
#include <fstream>
|
||||
@ -15,7 +15,7 @@
|
||||
#include <cstdlib>
|
||||
#include <port/unistd.h>
|
||||
|
||||
namespace fs = boost::filesystem;
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
std::string createTmpPath(const std::string & filename)
|
||||
{
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <boost/filesystem.hpp>
|
||||
#include <filesystem>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
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 <typename Impl>
|
||||
@ -34,10 +34,10 @@ public:
|
||||
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
return std::make_shared<FilesystemImpl<Impl>>(boost::filesystem::space(context.getConfigRef().getString("path")));
|
||||
return std::make_shared<FilesystemImpl<Impl>>(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;
|
||||
};
|
||||
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <IO/ReadBufferAIO.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <boost/filesystem.hpp>
|
||||
#include <filesystem>
|
||||
#include <vector>
|
||||
#include <iostream>
|
||||
#include <fstream>
|
||||
@ -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;
|
||||
|
@ -1,8 +1,7 @@
|
||||
#include <IO/WriteBufferAIO.h>
|
||||
#include <Core/Defines.h>
|
||||
|
||||
#include <boost/filesystem.hpp>
|
||||
|
||||
#include <filesystem>
|
||||
#include <iostream>
|
||||
#include <fstream>
|
||||
#include <streambuf>
|
||||
@ -11,7 +10,7 @@
|
||||
namespace
|
||||
{
|
||||
|
||||
namespace fs = boost::filesystem;
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
void run();
|
||||
[[noreturn]] void die(const std::string & msg);
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Common/Config/ConfigProcessor.h>
|
||||
#include <Interpreters/UsersManager.h>
|
||||
#include <boost/filesystem.hpp>
|
||||
#include <filesystem>
|
||||
#include <vector>
|
||||
#include <string>
|
||||
#include <tuple>
|
||||
@ -14,7 +14,7 @@
|
||||
namespace
|
||||
{
|
||||
|
||||
namespace fs = boost::filesystem;
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
struct TestEntry
|
||||
{
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -47,7 +47,7 @@
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
#include <memory>
|
||||
#include <boost/filesystem.hpp>
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
|
28
dbms/src/Storages/StorageValues.cpp
Normal file
28
dbms/src/Storages/StorageValues.cpp
Normal file
@ -0,0 +1,28 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/StorageValues.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
|
||||
|
||||
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<OneBlockInputStream>(res_block));
|
||||
}
|
||||
|
||||
}
|
36
dbms/src/Storages/StorageValues.h
Normal file
36
dbms/src/Storages/StorageValues.h
Normal file
@ -0,0 +1,36 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/* One block storage used for values table function
|
||||
* It's structure is similar to IStorageSystemOneBlock
|
||||
*/
|
||||
class StorageValues : public ext::shared_ptr_helper<StorageValues>, 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_);
|
||||
};
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,14 +1,17 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/ITableFunctionFileLike.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Storages/StorageFile.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
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<ASTLiteral &>().value.safeGet<String>();
|
||||
std::string structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
// Create sample block
|
||||
std::vector<std::string> 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 &>(context), table_name);
|
||||
|
||||
storage->startup();
|
||||
|
@ -1,11 +1,10 @@
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Common/IFactoryWithAliases.h>
|
||||
#include <Common/NamePrompter.h>
|
||||
|
||||
#include <ext/singleton.h>
|
||||
@ -16,51 +17,47 @@ namespace DB
|
||||
|
||||
class Context;
|
||||
|
||||
using TableFunctionCreator = std::function<TableFunctionPtr()>;
|
||||
|
||||
/** Lets you get a table function by its name.
|
||||
*/
|
||||
class TableFunctionFactory final: public ext::singleton<TableFunctionFactory>, public IHints<1, TableFunctionFactory>
|
||||
class TableFunctionFactory final: public ext::singleton<TableFunctionFactory>, public IFactoryWithAliases<TableFunctionCreator>
|
||||
{
|
||||
public:
|
||||
using Creator = std::function<TableFunctionPtr()>;
|
||||
|
||||
using TableFunctions = std::unordered_map<std::string, Creator>;
|
||||
/// 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 <typename Function>
|
||||
void registerFunction()
|
||||
void registerFunction(CaseSensitiveness case_sensitiveness = CaseSensitive)
|
||||
{
|
||||
auto creator = [] () -> TableFunctionPtr
|
||||
{
|
||||
return std::make_shared<Function>();
|
||||
};
|
||||
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<String> getAllRegisteredNames() const override
|
||||
{
|
||||
std::vector<String> 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<std::string, Creator>;
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
||||
|
95
dbms/src/TableFunctions/TableFunctionValues.cpp
Normal file
95
dbms/src/TableFunctions/TableFunctionValues.cpp
Normal file
@ -0,0 +1,95 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Storages/StorageValues.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionValues.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
|
||||
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<Tuple>().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<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
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<TableFunctionValues>(TableFunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
20
dbms/src/TableFunctions/TableFunctionValues.h
Normal file
20
dbms/src/TableFunctions/TableFunctionValues.h
Normal file
@ -0,0 +1,20 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
|
||||
}
|
44
dbms/src/TableFunctions/parseColumnsListForTableFunction.cpp
Normal file
44
dbms/src/TableFunctions/parseColumnsListForTableFunction.cpp
Normal file
@ -0,0 +1,44 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||
|
||||
|
||||
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<ASTExpressionList *>(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));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
11
dbms/src/TableFunctions/parseColumnsListForTableFunction.h
Normal file
11
dbms/src/TableFunctions/parseColumnsListForTableFunction.h
Normal file
@ -0,0 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Block.h>
|
||||
|
||||
|
||||
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);
|
||||
|
||||
}
|
@ -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);
|
||||
|
13
dbms/tests/queries/0_stateless/00975_values_list.reference
Normal file
13
dbms/tests/queries/0_stateless/00975_values_list.reference
Normal file
@ -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
|
14
dbms/tests/queries/0_stateless/00975_values_list.sql
Normal file
14
dbms/tests/queries/0_stateless/00975_values_list.sql
Normal file
@ -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;
|
Loading…
Reference in New Issue
Block a user