mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Merge branch 'master' into synchronous_mutations
This commit is contained in:
commit
9a005d27ea
@ -21,8 +21,7 @@ set(OPENSSL_MODULESDIR "/usr/local/lib/ossl-modules" CACHE PATH "Set the default
|
||||
add_definitions(-DOPENSSLDIR="${OPENSSLDIR}" -DENGINESDIR="${OPENSSL_ENGINESDIR}" -DMODULESDIR="${OPENSSL_MODULESDIR}")
|
||||
|
||||
if(ARCH_AMD64)
|
||||
# -DSHA256_ASM was removed because it doesn't work with asynchronous unwind (query profiler) for unknown reason.
|
||||
add_definitions(-DAES_ASM -DBSAES_ASM -DECP_NISTZ256_ASM -DGHASH_ASM -DKECCAK1600_ASM -DMD5_ASM -DOPENSSL_BN_ASM_GF2m -DOPENSSL_BN_ASM_MONT -DOPENSSL_BN_ASM_MONT5 -DOPENSSL_CPUID_OBJ -DOPENSSL_IA32_SSE2 -DPOLY1305_ASM -DSHA1_ASM -DSHA512_ASM -DVPAES_ASM -DWHIRLPOOL_ASM -DX25519_ASM -DOPENSSL_USE_NODELETE -DL_ENDIAN)
|
||||
add_definitions(-DAES_ASM -DBSAES_ASM -DECP_NISTZ256_ASM -DGHASH_ASM -DKECCAK1600_ASM -DMD5_ASM -DOPENSSL_BN_ASM_GF2m -DOPENSSL_BN_ASM_MONT -DOPENSSL_BN_ASM_MONT5 -DOPENSSL_CPUID_OBJ -DOPENSSL_IA32_SSE2 -DPOLY1305_ASM -DSHA1_ASM -DSHA256_ASM -DSHA512_ASM -DVPAES_ASM -DWHIRLPOOL_ASM -DX25519_ASM -DOPENSSL_USE_NODELETE -DL_ENDIAN)
|
||||
elseif(ARCH_AARCH64)
|
||||
add_definitions(-DECP_NISTZ256_ASM -DKECCAK1600_ASM -DOPENSSL_BN_ASM_MONT -DOPENSSL_CPUID_OBJ -DPOLY1305_ASM -DSHA1_ASM -DSHA256_ASM -DSHA512_ASM -DVPAES_ASM -DOPENSSL_USE_NODELETE -DL_ENDIAN)
|
||||
endif()
|
||||
@ -35,7 +34,11 @@ endif ()
|
||||
if (ARCH_AMD64)
|
||||
macro(perl_generate_asm FILE_IN FILE_OUT)
|
||||
add_custom_command(OUTPUT ${FILE_OUT}
|
||||
COMMAND /usr/bin/env perl ${FILE_IN} ${FILE_OUT})
|
||||
COMMAND /usr/bin/env perl ${FILE_IN} ${FILE_OUT}
|
||||
# ASM code has broken unwind tables (CFI), strip them.
|
||||
# Otherwise asynchronous unwind (that we use for query profiler)
|
||||
# will lead to segfault while trying to interpret wrong "CFA expression".
|
||||
COMMAND sed -i -e '/^\.cfi_/d' ${FILE_OUT})
|
||||
endmacro()
|
||||
perl_generate_asm(${OPENSSL_SOURCE_DIR}/crypto/aes/asm/aes-x86_64.pl ${OPENSSL_BINARY_DIR}/crypto/aes/aes-x86_64.s)
|
||||
perl_generate_asm(${OPENSSL_SOURCE_DIR}/crypto/aes/asm/aesni-mb-x86_64.pl ${OPENSSL_BINARY_DIR}/crypto/aes/aesni-mb-x86_64.s)
|
||||
@ -71,6 +74,7 @@ elseif (ARCH_AARCH64)
|
||||
macro(perl_generate_asm FILE_IN FILE_OUT)
|
||||
add_custom_command(OUTPUT ${FILE_OUT}
|
||||
COMMAND /usr/bin/env perl ${FILE_IN} "linux64" ${FILE_OUT})
|
||||
# Hope that the ASM code for AArch64 doesn't have broken CFI. Otherwise, add the same sed as for x86_64.
|
||||
endmacro()
|
||||
perl_generate_asm(${OPENSSL_SOURCE_DIR}/crypto/aes/asm/aesv8-armx.pl ${OPENSSL_BINARY_DIR}/crypto/aes/aesv8-armx.S)
|
||||
perl_generate_asm(${OPENSSL_SOURCE_DIR}/crypto/aes/asm/vpaes-armv8.pl ${OPENSSL_BINARY_DIR}/crypto/aes/vpaes-armv8.S)
|
||||
@ -911,10 +915,7 @@ if (ARCH_AMD64)
|
||||
${OPENSSL_BINARY_DIR}/crypto/sha/sha1-mb-x86_64.s
|
||||
${OPENSSL_BINARY_DIR}/crypto/sha/sha1-x86_64.s
|
||||
${OPENSSL_BINARY_DIR}/crypto/sha/sha256-mb-x86_64.s
|
||||
|
||||
#${OPENSSL_BINARY_DIR}/crypto/sha/sha256-x86_64.s
|
||||
${OPENSSL_SOURCE_DIR}/crypto/sha/sha256.c
|
||||
|
||||
${OPENSSL_BINARY_DIR}/crypto/sha/sha256-x86_64.s
|
||||
${OPENSSL_BINARY_DIR}/crypto/sha/sha512-x86_64.s
|
||||
${OPENSSL_BINARY_DIR}/crypto/whrlpool/wp-x86_64.s)
|
||||
elseif (ARCH_AARCH64)
|
||||
|
@ -443,7 +443,7 @@ void LocalServer::init(int argc, char ** argv)
|
||||
exit(0);
|
||||
}
|
||||
|
||||
if (options.count("help"))
|
||||
if (options.empty() || options.count("help"))
|
||||
{
|
||||
std::cout << getHelpHeader() << "\n";
|
||||
std::cout << description << "\n";
|
||||
|
@ -375,6 +375,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
Poco::File(user_files_path).createDirectories();
|
||||
}
|
||||
|
||||
{
|
||||
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", path + "dictionaries_lib/");
|
||||
global_context->setDictionariesLibPath(dictionaries_lib_path);
|
||||
Poco::File(dictionaries_lib_path).createDirectories();
|
||||
}
|
||||
|
||||
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
|
||||
throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
|
@ -919,10 +919,15 @@ void TCPHandler::receiveQuery()
|
||||
}
|
||||
|
||||
/// Per query settings.
|
||||
Settings & settings = query_context->getSettingsRef();
|
||||
Settings custom_settings{};
|
||||
auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS
|
||||
: SettingsBinaryFormat::OLD;
|
||||
settings.deserialize(*in, settings_format);
|
||||
custom_settings.deserialize(*in, settings_format);
|
||||
auto settings_changes = custom_settings.changes();
|
||||
query_context->checkSettingsConstraints(settings_changes);
|
||||
query_context->applySettingsChanges(settings_changes);
|
||||
|
||||
Settings & settings = query_context->getSettingsRef();
|
||||
|
||||
/// Sync timeouts on client and server during current query to avoid dangling queries on server
|
||||
/// NOTE: We use settings.send_timeout for the receive timeout and vice versa (change arguments ordering in TimeoutSetter),
|
||||
|
@ -263,7 +263,7 @@ size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
|
||||
return getNullValueIndex();
|
||||
|
||||
if (size_of_value_if_fixed)
|
||||
return uniqueInsertData(&x.get<char>(), size_of_value_if_fixed);
|
||||
return uniqueInsertData(&x.reinterpret<char>(), size_of_value_if_fixed);
|
||||
|
||||
auto & val = x.get<String>();
|
||||
return uniqueInsertData(val.data(), val.size());
|
||||
|
@ -188,7 +188,7 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \
|
||||
\
|
||||
M(SettingBool, input_format_values_interpret_expressions, true, "For Values format: if field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \
|
||||
M(SettingBool, input_format_values_deduce_templates_of_expressions, false, "For Values format: if field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
|
||||
M(SettingBool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \
|
||||
M(SettingBool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \
|
||||
\
|
||||
M(SettingBool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \
|
||||
|
@ -15,7 +15,7 @@ list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp Dictionar
|
||||
list(REMOVE_ITEM clickhouse_dictionaries_headers DictionaryFactory.h DictionarySourceFactory.h DictionaryStructure.h getDictionaryConfigurationFromAST.h)
|
||||
|
||||
add_library(clickhouse_dictionaries ${clickhouse_dictionaries_sources})
|
||||
target_link_libraries(clickhouse_dictionaries PRIVATE dbms clickhouse_common_io ${BTRIE_LIBRARIES})
|
||||
target_link_libraries(clickhouse_dictionaries PRIVATE dbms clickhouse_common_io string_utils ${BTRIE_LIBRARIES})
|
||||
|
||||
if(Poco_SQL_FOUND AND NOT USE_INTERNAL_POCO_LIBRARY)
|
||||
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${Poco_SQL_INCLUDE_DIR})
|
||||
|
@ -210,7 +210,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool /* check_config */) -> DictionarySourcePtr
|
||||
{
|
||||
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse", sample_block, context);
|
||||
};
|
||||
|
@ -27,7 +27,8 @@ DictionaryPtr DictionaryFactory::create(
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Context & context) const
|
||||
const Context & context,
|
||||
bool check_source_config) const
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
const auto & layout_prefix = config_prefix + ".layout";
|
||||
@ -38,7 +39,7 @@ DictionaryPtr DictionaryFactory::create(
|
||||
|
||||
const DictionaryStructure dict_struct{config, config_prefix + ".structure"};
|
||||
|
||||
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context);
|
||||
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context, check_source_config);
|
||||
|
||||
const auto & layout_type = keys.front();
|
||||
|
||||
@ -57,7 +58,17 @@ DictionaryPtr DictionaryFactory::create(
|
||||
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, const Context & context) const
|
||||
{
|
||||
auto configurationFromAST = getDictionaryConfigurationFromAST(ast);
|
||||
return DictionaryFactory::create(name, *configurationFromAST, "dictionary", context);
|
||||
return DictionaryFactory::create(name, *configurationFromAST, "dictionary", context, true);
|
||||
}
|
||||
|
||||
bool DictionaryFactory::isComplex(const std::string & layout_type) const
|
||||
{
|
||||
auto found = layout_complexity.find(layout_type);
|
||||
|
||||
if (found != layout_complexity.end())
|
||||
return found->second;
|
||||
|
||||
throw Exception{"Unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
|
||||
}
|
||||
|
||||
|
||||
|
@ -28,12 +28,16 @@ public:
|
||||
|
||||
static DictionaryFactory & instance();
|
||||
|
||||
/// Create dictionary from AbstractConfiguration parsed
|
||||
/// from xml-file on filesystem.
|
||||
DictionaryPtr create(
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Context & context) const;
|
||||
const Context & context,
|
||||
bool check_source_config = false) const;
|
||||
|
||||
/// Create dictionary from DDL-query
|
||||
DictionaryPtr create(const std::string & name,
|
||||
const ASTCreateQuery & ast,
|
||||
const Context & context) const;
|
||||
@ -45,7 +49,7 @@ public:
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr)>;
|
||||
|
||||
bool isComplex(const std::string & layout_type) const { return layout_complexity.at(layout_type); }
|
||||
bool isComplex(const std::string & layout_type) const;
|
||||
|
||||
void registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex);
|
||||
|
||||
|
@ -80,7 +80,8 @@ DictionarySourcePtr DictionarySourceFactory::create(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Context & context) const
|
||||
const Context & context,
|
||||
bool check_config) const
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(config_prefix, keys);
|
||||
@ -95,7 +96,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
|
||||
{
|
||||
const auto & create_source = found->second;
|
||||
auto sample_block = createSampleBlock(dict_struct);
|
||||
return create_source(dict_struct, config, config_prefix, sample_block, context);
|
||||
return create_source(dict_struct, config, config_prefix, sample_block, context, check_config);
|
||||
}
|
||||
|
||||
throw Exception{name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
|
||||
|
@ -31,7 +31,8 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context)>;
|
||||
const Context & context,
|
||||
bool check_config)>;
|
||||
|
||||
DictionarySourceFactory();
|
||||
|
||||
@ -42,7 +43,8 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Context & context) const;
|
||||
const Context & context,
|
||||
bool check_config) const;
|
||||
|
||||
private:
|
||||
using SourceRegistry = std::unordered_map<std::string, Creator>;
|
||||
|
@ -20,6 +20,10 @@ namespace DB
|
||||
{
|
||||
static const UInt64 max_block_size = 8192;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int DICTIONARY_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
@ -217,12 +221,21 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{"Dictionary source of type `executable` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
return std::make_unique<ExecutableDictionarySource>(dict_struct, config, config_prefix + ".executable", sample_block, context);
|
||||
/// Executable dictionaries may execute arbitrary commands.
|
||||
/// It's OK for dictionaries created by administrator from xml-file, but
|
||||
/// maybe dangerous for dictionaries created from DDL-queries.
|
||||
if (check_config)
|
||||
throw Exception("Dictionaries with Executable dictionary source is not allowed", ErrorCodes::DICTIONARY_ACCESS_DENIED);
|
||||
|
||||
return std::make_unique<ExecutableDictionarySource>(
|
||||
dict_struct, config, config_prefix + ".executable",
|
||||
sample_block, context);
|
||||
};
|
||||
factory.registerSource("executable", createTableSource);
|
||||
}
|
||||
|
@ -11,16 +11,31 @@ namespace DB
|
||||
{
|
||||
static const UInt64 max_block_size = 8192;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
|
||||
FileDictionarySource::FileDictionarySource(
|
||||
const std::string & filename_, const std::string & format_, Block & sample_block_, const Context & context_)
|
||||
: filename{filename_}, format{format_}, sample_block{sample_block_}, context(context_)
|
||||
const std::string & filepath_, const std::string & format_,
|
||||
Block & sample_block_, const Context & context_, bool check_config)
|
||||
: filepath{filepath_}
|
||||
, format{format_}
|
||||
, sample_block{sample_block_}
|
||||
, context(context_)
|
||||
{
|
||||
if (check_config)
|
||||
{
|
||||
const String user_files_path = context.getUserFilesPath();
|
||||
if (!startsWith(filepath, user_files_path))
|
||||
throw Exception("File path " + filepath + " is not inside " + user_files_path, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
FileDictionarySource::FileDictionarySource(const FileDictionarySource & other)
|
||||
: filename{other.filename}
|
||||
: filepath{other.filepath}
|
||||
, format{other.format}
|
||||
, sample_block{other.sample_block}
|
||||
, context(other.context)
|
||||
@ -31,7 +46,7 @@ FileDictionarySource::FileDictionarySource(const FileDictionarySource & other)
|
||||
|
||||
BlockInputStreamPtr FileDictionarySource::loadAll()
|
||||
{
|
||||
auto in_ptr = std::make_unique<ReadBufferFromFile>(filename);
|
||||
auto in_ptr = std::make_unique<ReadBufferFromFile>(filepath);
|
||||
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
|
||||
last_modification = getLastModification();
|
||||
|
||||
@ -41,13 +56,13 @@ BlockInputStreamPtr FileDictionarySource::loadAll()
|
||||
|
||||
std::string FileDictionarySource::toString() const
|
||||
{
|
||||
return "File: " + filename + ' ' + format;
|
||||
return "File: " + filepath + ' ' + format;
|
||||
}
|
||||
|
||||
|
||||
Poco::Timestamp FileDictionarySource::getLastModification() const
|
||||
{
|
||||
return Poco::File{filename}.getLastModified();
|
||||
return Poco::File{filepath}.getLastModified();
|
||||
}
|
||||
|
||||
void registerDictionarySourceFile(DictionarySourceFactory & factory)
|
||||
@ -56,15 +71,16 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
const auto filename = config.getString(config_prefix + ".file.path");
|
||||
const auto filepath = config.getString(config_prefix + ".file.path");
|
||||
const auto format = config.getString(config_prefix + ".file.format");
|
||||
|
||||
return std::make_unique<FileDictionarySource>(filename, format, sample_block, context);
|
||||
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context, check_config);
|
||||
};
|
||||
|
||||
factory.registerSource("file", createTableSource);
|
||||
|
@ -13,7 +13,8 @@ class Context;
|
||||
class FileDictionarySource final : public IDictionarySource
|
||||
{
|
||||
public:
|
||||
FileDictionarySource(const std::string & filename_, const std::string & format_, Block & sample_block_, const Context & context_);
|
||||
FileDictionarySource(const std::string & filepath_, const std::string & format_,
|
||||
Block & sample_block_, const Context & context_, bool check_config);
|
||||
|
||||
FileDictionarySource(const FileDictionarySource & other);
|
||||
|
||||
@ -47,7 +48,7 @@ public:
|
||||
private:
|
||||
Poco::Timestamp getLastModification() const;
|
||||
|
||||
const std::string filename;
|
||||
const std::string filepath;
|
||||
const std::string format;
|
||||
Block sample_block;
|
||||
const Context & context;
|
||||
|
@ -25,7 +25,8 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
const Context & context_)
|
||||
const Context & context_,
|
||||
bool check_config)
|
||||
: log(&Logger::get("HTTPDictionarySource"))
|
||||
, update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
@ -36,6 +37,10 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
, context(context_)
|
||||
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context))
|
||||
{
|
||||
|
||||
if (check_config)
|
||||
context.getRemoteHostFilter().checkURL(Poco::URI(url));
|
||||
|
||||
const auto & credentials_prefix = config_prefix + ".credentials";
|
||||
|
||||
if (config.has(credentials_prefix))
|
||||
@ -187,12 +192,15 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
return std::make_unique<HTTPDictionarySource>(dict_struct, config, config_prefix + ".http", sample_block, context);
|
||||
return std::make_unique<HTTPDictionarySource>(
|
||||
dict_struct, config, config_prefix + ".http",
|
||||
sample_block, context, check_config);
|
||||
};
|
||||
factory.registerSource("http", createTableSource);
|
||||
}
|
||||
|
@ -25,7 +25,8 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
const Context & context_);
|
||||
const Context & context_,
|
||||
bool check_config);
|
||||
|
||||
HTTPDictionarySource(const HTTPDictionarySource & other);
|
||||
HTTPDictionarySource & operator=(const HTTPDictionarySource &) = delete;
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <ext/bit_cast.h>
|
||||
#include <ext/range.h>
|
||||
#include <ext/scope_guard.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include "DictionarySourceFactory.h"
|
||||
#include "DictionaryStructure.h"
|
||||
#include "LibraryDictionarySourceExternal.h"
|
||||
@ -20,6 +21,7 @@ namespace ErrorCodes
|
||||
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int EXTERNAL_LIBRARY_ERROR;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
|
||||
@ -122,17 +124,28 @@ LibraryDictionarySource::LibraryDictionarySource(
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix_,
|
||||
Block & sample_block_)
|
||||
Block & sample_block_,
|
||||
const Context & context,
|
||||
bool check_config)
|
||||
: log(&Logger::get("LibraryDictionarySource"))
|
||||
, dict_struct{dict_struct_}
|
||||
, config_prefix{config_prefix_}
|
||||
, path{config.getString(config_prefix + ".path", "")}
|
||||
, sample_block{sample_block_}
|
||||
{
|
||||
|
||||
if (check_config)
|
||||
{
|
||||
const String dictionaries_lib_path = context.getDictionariesLibPath();
|
||||
if (!startsWith(path, dictionaries_lib_path))
|
||||
throw Exception("LibraryDictionarySource: Library path " + dictionaries_lib_path + " is not inside " + dictionaries_lib_path, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
}
|
||||
|
||||
if (!Poco::File(path).exists())
|
||||
throw Exception(
|
||||
"LibraryDictionarySource: Can't load lib " + toString() + ": " + Poco::File(path).path() + " - File doesn't exist",
|
||||
ErrorCodes::FILE_DOESNT_EXIST);
|
||||
|
||||
description.init(sample_block);
|
||||
library = std::make_shared<SharedLibrary>(path, RTLD_LAZY
|
||||
#if defined(RTLD_DEEPBIND) && !defined(ADDRESS_SANITIZER) // Does not exists in FreeBSD. Cannot work with Address Sanitizer.
|
||||
@ -286,9 +299,10 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context &) -> DictionarySourcePtr
|
||||
const Context & context,
|
||||
bool check_config) -> DictionarySourcePtr
|
||||
{
|
||||
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block);
|
||||
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);
|
||||
};
|
||||
factory.registerSource("library", createTableSource);
|
||||
}
|
||||
|
@ -32,7 +32,9 @@ public:
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix_,
|
||||
Block & sample_block_);
|
||||
Block & sample_block_,
|
||||
const Context & context,
|
||||
bool check_config);
|
||||
|
||||
LibraryDictionarySource(const LibraryDictionarySource & other);
|
||||
LibraryDictionarySource & operator=(const LibraryDictionarySource &) = delete;
|
||||
|
@ -15,7 +15,8 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & /* context */) -> DictionarySourcePtr {
|
||||
const Context & /* context */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
#if USE_POCO_MONGODB
|
||||
return std::make_unique<MongoDBDictionarySource>(dict_struct, config, config_prefix + ".mongodb", sample_block);
|
||||
#else
|
||||
|
@ -20,7 +20,8 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & /* context */) -> DictionarySourcePtr {
|
||||
const Context & /* context */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
#if USE_MYSQL
|
||||
return std::make_unique<MySQLDictionarySource>(dict_struct, config, config_prefix + ".mysql", sample_block);
|
||||
#else
|
||||
|
@ -15,7 +15,8 @@ namespace DB
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & /* context */) -> DictionarySourcePtr {
|
||||
const Context & /* context */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
#if USE_POCO_REDIS
|
||||
return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block);
|
||||
#else
|
||||
|
@ -238,7 +238,8 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context) -> DictionarySourcePtr {
|
||||
const Context & context,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
|
||||
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(
|
||||
context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string"));
|
||||
@ -262,7 +263,8 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & /* config */,
|
||||
const std::string & /* config_prefix */,
|
||||
Block & /* sample_block */,
|
||||
const Context & /* context */) -> DictionarySourcePtr {
|
||||
const Context & /* context */,
|
||||
bool /* check_config */) -> DictionarySourcePtr {
|
||||
throw Exception{"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
// BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string"));
|
||||
|
@ -126,7 +126,11 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const Colum
|
||||
|
||||
/// Const Nullable that are NULL.
|
||||
if (elem.column->onlyNull())
|
||||
return block.getByPosition(result).type->createColumnConst(input_rows_count, Null());
|
||||
{
|
||||
auto result_type = block.getByPosition(result).type;
|
||||
assert(result_type->isNullable());
|
||||
return result_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
}
|
||||
|
||||
if (isColumnConst(*elem.column))
|
||||
continue;
|
||||
@ -281,7 +285,13 @@ bool ExecutableFunctionAdaptor::defaultImplementationForNulls(
|
||||
|
||||
if (null_presence.has_null_constant)
|
||||
{
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, Null());
|
||||
auto & result_column = block.getByPosition(result).column;
|
||||
auto result_type = block.getByPosition(result).type;
|
||||
// Default implementation for nulls returns null result for null arguments,
|
||||
// so the result type must be nullable.
|
||||
assert(result_type->isNullable());
|
||||
|
||||
result_column = result_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -10,20 +10,8 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
extern "C"
|
||||
{
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wdocumentation"
|
||||
#endif
|
||||
|
||||
#include <h3api.h>
|
||||
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic pop
|
||||
#endif
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -86,10 +74,10 @@ public:
|
||||
const UInt8 res = col_res->getUInt(row);
|
||||
|
||||
GeoCoord coord;
|
||||
coord.lon = H3_EXPORT(degsToRads)(lon);
|
||||
coord.lat = H3_EXPORT(degsToRads)(lat);
|
||||
coord.lon = degsToRads(lon);
|
||||
coord.lat = degsToRads(lat);
|
||||
|
||||
H3Index hindex = H3_EXPORT(geoToH3)(&coord, res);
|
||||
H3Index hindex = geoToH3(&coord, res);
|
||||
|
||||
dst_data[row] = hindex;
|
||||
}
|
||||
|
67
dbms/src/Functions/h3EdgeAngle.cpp
Normal file
67
dbms/src/Functions/h3EdgeAngle.cpp
Normal file
@ -0,0 +1,67 @@
|
||||
#include "config_functions.h"
|
||||
#if USE_H3
|
||||
# include <Columns/ColumnsNumber.h>
|
||||
# include <DataTypes/DataTypesNumber.h>
|
||||
# include <Functions/FunctionFactory.h>
|
||||
# include <Functions/IFunction.h>
|
||||
# include <Common/typeid_cast.h>
|
||||
# include <ext/range.h>
|
||||
|
||||
# include <h3api.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class FunctionH3EdgeAngle : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "h3EdgeAngle";
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3EdgeAngle>(); }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
auto arg = arguments[0].get();
|
||||
if (!WhichDataType(arg).isUInt8())
|
||||
throw Exception(
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be UInt8",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
|
||||
|
||||
auto dst = ColumnVector<Float64>::create();
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : ext::range(0, input_rows_count))
|
||||
{
|
||||
const int resolution = col_hindex->getUInt(row);
|
||||
|
||||
// Numerical constant is 180 degrees / pi / Earth radius, Earth radius is from h3 sources
|
||||
Float64 res = 8.99320592271288084e-6 * edgeLengthM(resolution);
|
||||
|
||||
dst_data[row] = res;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionH3EdgeAngle(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionH3EdgeAngle>();
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
71
dbms/src/Functions/h3EdgeLengthM.cpp
Normal file
71
dbms/src/Functions/h3EdgeLengthM.cpp
Normal file
@ -0,0 +1,71 @@
|
||||
#include "config_functions.h"
|
||||
#if USE_H3
|
||||
# include <Columns/ColumnsNumber.h>
|
||||
# include <DataTypes/DataTypesNumber.h>
|
||||
# include <Functions/FunctionFactory.h>
|
||||
# include <Functions/IFunction.h>
|
||||
# include <Common/typeid_cast.h>
|
||||
# include <ext/range.h>
|
||||
|
||||
# include <h3api.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
// Average metric edge length of H3 hexagon. The edge length `e` for given resolution `res` can
|
||||
// be used for converting metric search radius `radius` to hexagon search ring size `k` that is
|
||||
// used by `H3kRing` function. For small enough search area simple flat approximation can be used,
|
||||
// i.e. the smallest `k` that satisfies relation `3 k^2 - 3 k + 1 >= (radius / e)^2` should be
|
||||
// chosen
|
||||
class FunctionH3EdgeLengthM : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "h3EdgeLengthM";
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3EdgeLengthM>(); }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
auto arg = arguments[0].get();
|
||||
if (!WhichDataType(arg).isUInt8())
|
||||
throw Exception(
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be UInt8",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
|
||||
|
||||
auto dst = ColumnVector<Float64>::create();
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : ext::range(0, input_rows_count))
|
||||
{
|
||||
const int resolution = col_hindex->getUInt(row);
|
||||
|
||||
Float64 res = edgeLengthM(resolution);
|
||||
|
||||
dst_data[row] = res;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionH3EdgeLengthM(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionH3EdgeLengthM>();
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
66
dbms/src/Functions/h3GetResolution.cpp
Normal file
66
dbms/src/Functions/h3GetResolution.cpp
Normal file
@ -0,0 +1,66 @@
|
||||
#include "config_functions.h"
|
||||
#if USE_H3
|
||||
# include <Columns/ColumnsNumber.h>
|
||||
# include <DataTypes/DataTypesNumber.h>
|
||||
# include <Functions/FunctionFactory.h>
|
||||
# include <Functions/IFunction.h>
|
||||
# include <Common/typeid_cast.h>
|
||||
# include <ext/range.h>
|
||||
|
||||
# include <h3api.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class FunctionH3GetResolution : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "h3GetResolution";
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3GetResolution>(); }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
auto arg = arguments[0].get();
|
||||
if (!WhichDataType(arg).isUInt64())
|
||||
throw Exception(
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be UInt64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
|
||||
|
||||
auto dst = ColumnVector<UInt8>::create();
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : ext::range(0, input_rows_count))
|
||||
{
|
||||
const UInt64 hindex = col_hindex->getUInt(row);
|
||||
|
||||
UInt8 res = h3GetResolution(hindex);
|
||||
|
||||
dst_data[row] = res;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionH3GetResolution(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionH3GetResolution>();
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
66
dbms/src/Functions/h3IsValid.cpp
Normal file
66
dbms/src/Functions/h3IsValid.cpp
Normal file
@ -0,0 +1,66 @@
|
||||
#include "config_functions.h"
|
||||
#if USE_H3
|
||||
# include <Columns/ColumnsNumber.h>
|
||||
# include <DataTypes/DataTypesNumber.h>
|
||||
# include <Functions/FunctionFactory.h>
|
||||
# include <Functions/IFunction.h>
|
||||
# include <Common/typeid_cast.h>
|
||||
# include <ext/range.h>
|
||||
|
||||
# include <h3api.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class FunctionH3IsValid : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "h3IsValid";
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3IsValid>(); }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
auto arg = arguments[0].get();
|
||||
if (!WhichDataType(arg).isUInt64())
|
||||
throw Exception(
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be UInt64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
|
||||
|
||||
auto dst = ColumnVector<UInt8>::create();
|
||||
auto & dst_data = dst->getData();
|
||||
dst_data.resize(input_rows_count);
|
||||
|
||||
for (const auto row : ext::range(0, input_rows_count))
|
||||
{
|
||||
const UInt64 hindex = col_hindex->getUInt(row);
|
||||
|
||||
UInt8 is_valid = h3IsValid(hindex) == 0 ? 0 : 1;
|
||||
|
||||
dst_data[row] = is_valid;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionH3IsValid(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionH3IsValid>();
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
93
dbms/src/Functions/h3kRing.cpp
Normal file
93
dbms/src/Functions/h3kRing.cpp
Normal file
@ -0,0 +1,93 @@
|
||||
#include "config_functions.h"
|
||||
#if USE_H3
|
||||
# include <vector>
|
||||
# include <Columns/ColumnArray.h>
|
||||
# include <Columns/ColumnsNumber.h>
|
||||
# include <DataTypes/DataTypeArray.h>
|
||||
# include <DataTypes/DataTypesNumber.h>
|
||||
# include <DataTypes/IDataType.h>
|
||||
# include <Functions/FunctionFactory.h>
|
||||
# include <Functions/IFunction.h>
|
||||
# include <Common/typeid_cast.h>
|
||||
# include <ext/range.h>
|
||||
|
||||
# include <h3api.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class FunctionH3KRing : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "h3kRing";
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionH3KRing>(); }
|
||||
|
||||
std::string getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
auto arg = arguments[0].get();
|
||||
if (!WhichDataType(arg).isUInt64())
|
||||
throw Exception(
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(1) + " of function " + getName() + ". Must be UInt64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
arg = arguments[1].get();
|
||||
if (!isInteger(arg))
|
||||
throw Exception(
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(2) + " of function " + getName() + ". Must be integer",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const auto col_hindex = block.getByPosition(arguments[0]).column.get();
|
||||
const auto col_k = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
auto dst = ColumnArray::create(ColumnUInt64::create());
|
||||
auto & dst_data = dst->getData();
|
||||
auto & dst_offsets = dst->getOffsets();
|
||||
dst_offsets.resize(input_rows_count);
|
||||
auto current_offset = 0;
|
||||
|
||||
std::vector<H3Index> hindex_vec;
|
||||
|
||||
for (const auto row : ext::range(0, input_rows_count))
|
||||
{
|
||||
const H3Index origin_hindex = col_hindex->getUInt(row);
|
||||
const int k = col_k->getInt(row);
|
||||
|
||||
const auto vec_size = maxKringSize(k);
|
||||
hindex_vec.resize(vec_size);
|
||||
kRing(origin_hindex, k, hindex_vec.data());
|
||||
|
||||
dst_data.reserve(dst_data.size() + vec_size);
|
||||
for (auto hindex : hindex_vec)
|
||||
{
|
||||
if (hindex != 0)
|
||||
{
|
||||
++current_offset;
|
||||
dst_data.insert(hindex);
|
||||
}
|
||||
}
|
||||
dst_offsets[row] = current_offset;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(dst);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionH3KRing(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionH3KRing>();
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
@ -18,7 +18,7 @@ namespace ErrorCodes
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
DateTime64::NativeType nowSubsecond(UInt32 scale)
|
||||
static Field nowSubsecond(UInt32 scale)
|
||||
{
|
||||
const Int32 fractional_scale = 9;
|
||||
timespec spec;
|
||||
@ -36,7 +36,8 @@ DateTime64::NativeType nowSubsecond(UInt32 scale)
|
||||
else if (adjust_scale > 0)
|
||||
components.fractional /= intExp10(adjust_scale);
|
||||
|
||||
return DecimalUtils::decimalFromComponents<DateTime64>(components, scale).value;
|
||||
return DecimalField(DecimalUtils::decimalFromComponents<DateTime64>(components, scale),
|
||||
scale);
|
||||
}
|
||||
|
||||
class FunctionNow64 : public IFunction
|
||||
|
@ -14,6 +14,11 @@ void registerFunctionGeohashesInBox(FunctionFactory & factory);
|
||||
|
||||
#if USE_H3
|
||||
void registerFunctionGeoToH3(FunctionFactory &);
|
||||
void registerFunctionH3EdgeAngle(FunctionFactory &);
|
||||
void registerFunctionH3EdgeLengthM(FunctionFactory &);
|
||||
void registerFunctionH3GetResolution(FunctionFactory &);
|
||||
void registerFunctionH3IsValid(FunctionFactory &);
|
||||
void registerFunctionH3KRing(FunctionFactory &);
|
||||
#endif
|
||||
|
||||
void registerFunctionsGeo(FunctionFactory & factory)
|
||||
@ -27,6 +32,11 @@ void registerFunctionsGeo(FunctionFactory & factory)
|
||||
|
||||
#if USE_H3
|
||||
registerFunctionGeoToH3(factory);
|
||||
registerFunctionH3EdgeAngle(factory);
|
||||
registerFunctionH3EdgeLengthM(factory);
|
||||
registerFunctionH3GetResolution(factory);
|
||||
registerFunctionH3IsValid(factory);
|
||||
registerFunctionH3KRing(factory);
|
||||
#endif
|
||||
}
|
||||
|
||||
|
@ -796,7 +796,11 @@ private:
|
||||
table_num_to_num = std::make_unique<NumToNum>();
|
||||
auto & table = *table_num_to_num;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
table[from[i].get<UInt64>()] = (*used_to)[i].get<UInt64>();
|
||||
{
|
||||
// Field may be of Float type, but for the purpose of bitwise
|
||||
// equality we can treat them as UInt64, hence the reinterpret().
|
||||
table[from[i].reinterpret<UInt64>()] = (*used_to)[i].reinterpret<UInt64>();
|
||||
}
|
||||
}
|
||||
else if (from[0].getType() != Field::Types::String && to[0].getType() == Field::Types::String)
|
||||
{
|
||||
@ -806,7 +810,7 @@ private:
|
||||
{
|
||||
const String & str_to = to[i].get<const String &>();
|
||||
StringRef ref{string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
|
||||
table[from[i].get<UInt64>()] = ref;
|
||||
table[from[i].reinterpret<UInt64>()] = ref;
|
||||
}
|
||||
}
|
||||
else if (from[0].getType() == Field::Types::String && to[0].getType() != Field::Types::String)
|
||||
@ -817,7 +821,7 @@ private:
|
||||
{
|
||||
const String & str_from = from[i].get<const String &>();
|
||||
StringRef ref{string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
|
||||
table[ref] = (*used_to)[i].get<UInt64>();
|
||||
table[ref] = (*used_to)[i].reinterpret<UInt64>();
|
||||
}
|
||||
}
|
||||
else if (from[0].getType() == Field::Types::String && to[0].getType() == Field::Types::String)
|
||||
|
@ -127,6 +127,7 @@ struct ContextShared
|
||||
String tmp_path; /// The path to the temporary files that occur when processing the request.
|
||||
String flags_path; /// Path to the directory with some control flags for server maintenance.
|
||||
String user_files_path; /// Path to the directory with user provided files, usable by 'file' table function.
|
||||
String dictionaries_lib_path; /// Path to the directory with user provided binaries and libraries for external dictionaries.
|
||||
ConfigurationPtr config; /// Global configuration settings.
|
||||
|
||||
Databases databases; /// List of databases and tables in them.
|
||||
@ -544,6 +545,12 @@ String Context::getUserFilesPath() const
|
||||
return shared->user_files_path;
|
||||
}
|
||||
|
||||
String Context::getDictionariesLibPath() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
return shared->dictionaries_lib_path;
|
||||
}
|
||||
|
||||
void Context::setPath(const String & path)
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -558,6 +565,9 @@ void Context::setPath(const String & path)
|
||||
|
||||
if (shared->user_files_path.empty())
|
||||
shared->user_files_path = shared->path + "user_files/";
|
||||
|
||||
if (shared->dictionaries_lib_path.empty())
|
||||
shared->dictionaries_lib_path = shared->path + "dictionaries_lib/";
|
||||
}
|
||||
|
||||
void Context::setTemporaryPath(const String & path)
|
||||
@ -578,6 +588,12 @@ void Context::setUserFilesPath(const String & path)
|
||||
shared->user_files_path = path;
|
||||
}
|
||||
|
||||
void Context::setDictionariesLibPath(const String & path)
|
||||
{
|
||||
auto lock = getLock();
|
||||
shared->dictionaries_lib_path = path;
|
||||
}
|
||||
|
||||
void Context::setConfig(const ConfigurationPtr & config)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
@ -191,11 +191,13 @@ public:
|
||||
String getTemporaryPath() const;
|
||||
String getFlagsPath() const;
|
||||
String getUserFilesPath() const;
|
||||
String getDictionariesLibPath() const;
|
||||
|
||||
void setPath(const String & path);
|
||||
void setTemporaryPath(const String & path);
|
||||
void setFlagsPath(const String & path);
|
||||
void setUserFilesPath(const String & path);
|
||||
void setDictionariesLibPath(const String & path);
|
||||
|
||||
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
|
||||
|
||||
|
@ -15,9 +15,13 @@ ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_)
|
||||
|
||||
|
||||
ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
|
||||
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config) const
|
||||
const std::string & name, const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & key_in_config, const std::string & repository_name) const
|
||||
{
|
||||
return DictionaryFactory::instance().create(name, config, key_in_config, context);
|
||||
/// For dictionaries from databases (created with DDL qureies) we have to perform
|
||||
/// additional checks, so we identify them here.
|
||||
bool dictionary_from_database = !repository_name.empty();
|
||||
return DictionaryFactory::instance().create(name, config, key_in_config, context, dictionary_from_database);
|
||||
}
|
||||
|
||||
void ExternalDictionariesLoader::addConfigRepository(
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
|
||||
protected:
|
||||
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & key_in_config) const override;
|
||||
const std::string & key_in_config, const std::string & repository_name) const override;
|
||||
|
||||
friend class StorageSystemDictionaries;
|
||||
friend class DatabaseDictionary;
|
||||
|
@ -1257,7 +1257,7 @@ ExternalLoader::LoadablePtr ExternalLoader::createObject(
|
||||
if (previous_version)
|
||||
return previous_version->clone();
|
||||
|
||||
return create(name, *config.config, config.key_in_config);
|
||||
return create(name, *config.config, config.key_in_config, config.repository_name);
|
||||
}
|
||||
|
||||
std::vector<std::pair<String, Int8>> ExternalLoader::getStatusEnumAllPossibleValues()
|
||||
|
@ -173,7 +173,7 @@ public:
|
||||
void reloadConfig(const String & repository_name, const String & path) const;
|
||||
|
||||
protected:
|
||||
virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const = 0;
|
||||
virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config, const String & repository_name) const = 0;
|
||||
|
||||
private:
|
||||
struct ObjectConfig;
|
||||
|
@ -18,7 +18,8 @@ ExternalModelsLoader::ExternalModelsLoader(Context & context_)
|
||||
}
|
||||
|
||||
std::shared_ptr<const IExternalLoadable> ExternalModelsLoader::create(
|
||||
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) const
|
||||
const std::string & name, const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix, const std::string & /* repository_name */) const
|
||||
{
|
||||
String type = config.getString(config_prefix + ".type");
|
||||
ExternalLoadableLifetime lifetime(config, config_prefix + ".lifetime");
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
|
||||
protected:
|
||||
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & key_in_config) const override;
|
||||
const std::string & key_in_config, const std::string & repository_name) const override;
|
||||
|
||||
friend class StorageSystemModels;
|
||||
private:
|
||||
|
@ -99,9 +99,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
const auto & query = query_ptr->as<ASTInsertQuery &>();
|
||||
checkAccess(query);
|
||||
|
||||
BlockIO res;
|
||||
StoragePtr table = getTable(query);
|
||||
res.pipeline.addStorageHolder(table);
|
||||
|
||||
auto table_lock = table->lockStructureForShare(true, context.getInitialQueryId());
|
||||
|
||||
@ -137,7 +135,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
out_wrapper->setProcessListElement(context.getProcessListElement());
|
||||
out = std::move(out_wrapper);
|
||||
|
||||
res.out = std::move(out);
|
||||
BlockIO res;
|
||||
|
||||
/// What type of query: INSERT or INSERT SELECT?
|
||||
if (query.select)
|
||||
@ -145,13 +143,13 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
/// Passing 1 as subquery_depth will disable limiting size of intermediate result.
|
||||
InterpreterSelectWithUnionQuery interpreter_select{query.select, context, SelectQueryOptions(QueryProcessingStage::Complete, 1)};
|
||||
|
||||
res.in = interpreter_select.execute().in;
|
||||
|
||||
res.in = std::make_shared<ConvertingBlockInputStream>(context, res.in, res.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Position);
|
||||
res.in = std::make_shared<NullAndDoCopyBlockInputStream>(res.in, res.out);
|
||||
|
||||
/// BlockIO may hold StoragePtrs to temporary tables
|
||||
res = interpreter_select.execute();
|
||||
res.out = nullptr;
|
||||
|
||||
res.in = std::make_shared<ConvertingBlockInputStream>(context, res.in, out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Position);
|
||||
res.in = std::make_shared<NullAndDoCopyBlockInputStream>(res.in, out);
|
||||
|
||||
if (!allow_materialized)
|
||||
{
|
||||
Block in_header = res.in->getHeader();
|
||||
@ -163,9 +161,12 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
else if (query.data && !query.has_tail) /// can execute without additional data
|
||||
{
|
||||
res.in = std::make_shared<InputStreamFromASTInsertQuery>(query_ptr, nullptr, query_sample_block, context, nullptr);
|
||||
res.in = std::make_shared<NullAndDoCopyBlockInputStream>(res.in, res.out);
|
||||
res.out = nullptr;
|
||||
res.in = std::make_shared<NullAndDoCopyBlockInputStream>(res.in, out);
|
||||
}
|
||||
else
|
||||
res.out = std::move(out);
|
||||
|
||||
res.pipeline.addStorageHolder(table);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
@ -84,8 +84,16 @@ void NO_INLINE Set::insertFromBlockImplCase(
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
if constexpr (has_null_map)
|
||||
{
|
||||
if ((*null_map)[i])
|
||||
{
|
||||
if constexpr (build_filter)
|
||||
{
|
||||
(*out_filter)[i] = false;
|
||||
}
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
[[maybe_unused]] auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool);
|
||||
|
||||
|
@ -113,9 +113,7 @@ void checkCreationIsAllowed(Context & context_global, const std::string & db_dir
|
||||
throw Exception("Part path " + table_path + " is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED);
|
||||
|
||||
Poco::File table_path_poco_file = Poco::File(table_path);
|
||||
if (!table_path_poco_file.exists())
|
||||
throw Exception("File " + table_path + " is not exist", ErrorCodes::FILE_DOESNT_EXIST);
|
||||
else if (table_path_poco_file.isDirectory())
|
||||
if (table_path_poco_file.exists() && table_path_poco_file.isDirectory())
|
||||
throw Exception("File " + table_path + " must not be a directory", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
}
|
||||
}
|
||||
@ -150,7 +148,12 @@ StorageFile::StorageFile(
|
||||
poco_path = Poco::Path(db_dir_path_abs, poco_path);
|
||||
|
||||
const std::string path = poco_path.absolute().toString();
|
||||
paths = listFilesWithRegexpMatching("/", path);
|
||||
if (path.find_first_of("*?{") == std::string::npos)
|
||||
{
|
||||
paths.push_back(path);
|
||||
}
|
||||
else
|
||||
paths = listFilesWithRegexpMatching("/", path);
|
||||
for (const auto & cur_path : paths)
|
||||
checkCreationIsAllowed(context_global, db_dir_path_abs, cur_path);
|
||||
is_db_table = false;
|
||||
@ -264,6 +267,11 @@ BlockInputStreams StorageFile::read(
|
||||
BlockInputStreams blocks_input;
|
||||
if (use_table_fd) /// need to call ctr BlockInputStream
|
||||
paths = {""}; /// when use fd, paths are empty
|
||||
else
|
||||
{
|
||||
if (paths.size() == 1 && !Poco::File(paths[0]).exists())
|
||||
throw Exception("File " + paths[0] + " doesn't exist", ErrorCodes::FILE_DOESNT_EXIST);
|
||||
}
|
||||
blocks_input.reserve(paths.size());
|
||||
for (const auto & file_path : paths)
|
||||
{
|
||||
|
@ -16,4 +16,8 @@
|
||||
<path>./clickhouse/</path>
|
||||
<users_config>users.xml</users_config>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||
|
||||
<remote_url_allow_hosts>
|
||||
<host>node1</host>
|
||||
</remote_url_allow_hosts>
|
||||
</yandex>
|
||||
|
@ -182,3 +182,35 @@ def test_conflicting_name(started_cluster):
|
||||
|
||||
# old version still works
|
||||
node3.query("select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))") == '17\n'
|
||||
|
||||
def test_http_dictionary_restrictions(started_cluster):
|
||||
try:
|
||||
node3.query("""
|
||||
CREATE DICTIONARY test.restricted_http_dictionary (
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(HTTP(URL 'http://somehost.net' FORMAT TabSeparated))
|
||||
LIFETIME(1)
|
||||
""")
|
||||
node3.query("SELECT dictGetString('test.restricted_http_dictionary', 'value', toUInt64(1))")
|
||||
except QueryRuntimeException as ex:
|
||||
assert 'is not allowed in config.xml' in str(ex)
|
||||
|
||||
def test_file_dictionary_restrictions(started_cluster):
|
||||
try:
|
||||
node3.query("""
|
||||
CREATE DICTIONARY test.restricted_file_dictionary (
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(FILE(PATH '/usr/bin/cat' FORMAT TabSeparated))
|
||||
LIFETIME(1)
|
||||
""")
|
||||
node3.query("SELECT dictGetString('test.restricted_file_dictionary', 'value', toUInt64(1))")
|
||||
except QueryRuntimeException as ex:
|
||||
assert 'is not inside' in str(ex)
|
||||
|
@ -10,7 +10,12 @@ path_to_userfiles_from_defaut_config = "/var/lib/clickhouse/user_files/" # sho
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
yield cluster
|
||||
|
||||
except Exception as ex:
|
||||
print(ex)
|
||||
raise ex
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
@ -57,6 +62,7 @@ def test_linear_structure(start_cluster):
|
||||
|
||||
test_requests = [("file{0..9}", "10"),
|
||||
("file?", "10"),
|
||||
("nothing*", "0"),
|
||||
("file{0..9}{0..9}{0..9}", "10"),
|
||||
("file???", "10"),
|
||||
("file*", "20"),
|
||||
@ -92,7 +98,7 @@ def test_deep_structure(start_cluster):
|
||||
for i in range(10):
|
||||
for j in range(10):
|
||||
for k in range(10):
|
||||
files.append("directory1/big_dir/file"+str(i)+str(j)+str(k))
|
||||
files.append("directory1/big_dir/file" + str(i) + str(j) + str(k))
|
||||
|
||||
for dir in dirs:
|
||||
files.append(dir+"file")
|
||||
@ -119,3 +125,5 @@ def test_table_function(start_cluster):
|
||||
node.exec_in_container(['bash', '-c', 'touch {}some/path/to/data.CSV'.format(path_to_userfiles_from_defaut_config)])
|
||||
node.query("insert into table function file('some/path/to/data.CSV', CSV, 'n UInt8, s String') select number, concat('str_', toString(number)) from numbers(100000)")
|
||||
assert node.query("select count() from file('some/path/to/data.CSV', CSV, 'n UInt8, s String')").rstrip() == '100000'
|
||||
node.query("insert into table function file('nonexist.csv', 'CSV', 'val1 UInt32') values (1)")
|
||||
assert node.query("select * from file('nonexist.csv', 'CSV', 'val1 UInt32')").rstrip()== '1'
|
||||
|
@ -109,6 +109,8 @@ def test_table_function(started_cluster):
|
||||
" UNION ALL SELECT count() as c FROM {} WHERE id % 3 == 2)".format(table_function, table_function,
|
||||
table_function)).rstrip() == '10000'
|
||||
assert node1.query("SELECT sum(`money`) FROM {}".format(table_function)).rstrip() == '30000'
|
||||
node1.query("INSERT INTO {} SELECT id + 100000, name, age, money FROM {}".format('TABLE FUNCTION ' + table_function, table_function))
|
||||
assert node1.query("SELECT sum(`money`) FROM {}".format(table_function)).rstrip() == '60000'
|
||||
conn.close()
|
||||
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
<type>loop</type>
|
||||
|
||||
<settings>
|
||||
<input_format_values_deduce_templates_of_expressions>1</input_format_values_deduce_templates_of_expressions>
|
||||
<input_format_values_accurate_types_of_literals>1</input_format_values_accurate_types_of_literals>
|
||||
</settings>
|
||||
<!--
|
||||
|
@ -1,2 +1,2 @@
|
||||
1
|
||||
['123']
|
||||
['123','Hello world!','123']
|
||||
|
@ -5,7 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'DROP TABLE IF EXISTS bad_arrays'
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'CREATE TABLE bad_arrays (a Array(String)) ENGINE = Memory'
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'INSERT INTO bad_arrays VALUES ([123])' 2>&1 | grep -c 'Exception'
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "INSERT INTO bad_arrays VALUES (['123', concat('Hello', ' world!'), toString(123)])"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT * FROM bad_arrays'
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "INSERT INTO bad_arrays VALUES ([123]), (['123', concat('Hello', ' world!'), toString(123)])"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT * FROM bad_arrays ORDER BY a'
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'DROP TABLE bad_arrays'
|
||||
|
@ -84,8 +84,8 @@ INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0.0, 0.0, 0.0, 0.0, 0
|
||||
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0);
|
||||
|
||||
INSERT INTO decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990');
|
||||
INSERT INTO decimal (a) VALUES ('-9x'); -- { clientError 72 }
|
||||
INSERT INTO decimal (a) VALUES ('0x1'); -- { clientError 72 }
|
||||
INSERT INTO decimal (a) VALUES ('-9x'); -- { clientError 6 }
|
||||
INSERT INTO decimal (a) VALUES ('0x1'); -- { clientError 6 }
|
||||
|
||||
INSERT INTO decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38');
|
||||
INSERT INTO decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');
|
||||
|
@ -3,7 +3,7 @@ DROP TABLE IF EXISTS arraytest;
|
||||
CREATE TABLE arraytest ( created_date Date DEFAULT toDate(created_at), created_at DateTime DEFAULT now(), strings Array(String) DEFAULT emptyArrayString()) ENGINE = MergeTree(created_date, cityHash64(created_at), (created_date, cityHash64(created_at)), 8192);
|
||||
|
||||
INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', 'ccccc']);
|
||||
INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', null]); -- { clientError 53 }
|
||||
INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', null]); -- { clientError 349 }
|
||||
|
||||
SELECT strings from arraytest;
|
||||
|
||||
|
@ -7,7 +7,6 @@ CREATE TABLE values_template_nullable (d Date, s Nullable(String), u Nullable(UI
|
||||
CREATE TABLE values_template_fallback (n UInt8) ENGINE = Memory;
|
||||
|
||||
SET input_format_values_interpret_expressions = 0;
|
||||
SET input_format_values_deduce_templates_of_expressions = 1;
|
||||
|
||||
--(1, lower(replaceAll(_STR_1, 'o', 'a')), _NUM_1 + _NUM_2 + _NUM_3, round(_NUM_4 / _NUM_5), _NUM_6 * CAST(_STR_7, 'Int8'), _ARR_8);
|
||||
-- _NUM_1: UInt64 -> Int64 -> UInt64
|
||||
|
@ -50,6 +50,6 @@ echo 'Values'
|
||||
echo '(NULL, '\''1'\'', (null), '\''2019-07-22'\'', ([10, 20, 30]), (NuLl)),
|
||||
(1, '\''world'\'', (3), '\''2019-07-23'\'', (NULL), (('\''tuple'\'', 3.14))),
|
||||
(2, null, (123), null, ([]), (('\''test'\'', 2.71828))),
|
||||
(3, null, (null), null, (null), (null))' | $CLICKHOUSE_CLIENT --input_format_null_as_default=1 --input_format_values_deduce_templates_of_expressions=1 --query="INSERT INTO null_as_default VALUES";
|
||||
(3, null, (null), null, (null), (null))' | $CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO null_as_default VALUES";
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i";
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE null_as_default";
|
||||
|
@ -0,0 +1,3 @@
|
||||
1
|
||||
10
|
||||
15
|
@ -0,0 +1,3 @@
|
||||
SELECT h3GetResolution(581276613233082367);
|
||||
SELECT h3GetResolution(621807531097128959);
|
||||
SELECT h3GetResolution(644325529233966508);
|
@ -0,0 +1,4 @@
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
4
dbms/tests/queries/0_stateless/01041_h3_is_valid.sql
Normal file
4
dbms/tests/queries/0_stateless/01041_h3_is_valid.sql
Normal file
@ -0,0 +1,4 @@
|
||||
SELECT h3IsValid(581276613233082367);
|
||||
SELECT h3IsValid(621807531097128959);
|
||||
SELECT h3IsValid(Cast(0, 'UInt64'));
|
||||
SELECT h3IsValid(100000000000000000);
|
3
dbms/tests/queries/0_stateless/01042_h3_k_ring.reference
Normal file
3
dbms/tests/queries/0_stateless/01042_h3_k_ring.reference
Normal file
@ -0,0 +1,3 @@
|
||||
[581250224954015743,581259021047037951,581267817140060159,581276613233082367,581500913605148671,581518505791193087,581764796395814911]
|
||||
[581276613233082367]
|
||||
[581276613233082367]
|
3
dbms/tests/queries/0_stateless/01042_h3_k_ring.sql
Normal file
3
dbms/tests/queries/0_stateless/01042_h3_k_ring.sql
Normal file
@ -0,0 +1,3 @@
|
||||
SELECT arraySort(h3kRing(581276613233082367, 1));
|
||||
SELECT h3kRing(581276613233082367, 0);
|
||||
SELECT h3kRing(581276613233082367, -1);
|
@ -0,0 +1,2 @@
|
||||
418676.0055
|
||||
22606.3794
|
@ -0,0 +1,2 @@
|
||||
SELECT h3EdgeLengthM(1);
|
||||
SELECT h3EdgeLengthM(4);
|
@ -0,0 +1,2 @@
|
||||
0.0005927224846720883
|
||||
0
|
2
dbms/tests/queries/0_stateless/01044_h3_edge_angle.sql
Normal file
2
dbms/tests/queries/0_stateless/01044_h3_edge_angle.sql
Normal file
@ -0,0 +1,2 @@
|
||||
SELECT h3EdgeAngle(10);
|
||||
SELECT h3EdgeLengthM(2) * 180 / pi() / 6371007.180918475 - h3EdgeAngle(2);
|
@ -0,0 +1 @@
|
||||
Ok.
|
@ -0,0 +1,21 @@
|
||||
DROP DATABASE IF EXISTS dictdb;
|
||||
|
||||
CREATE DATABASE dictdb ENGINE=Ordinary;
|
||||
|
||||
CREATE DICTIONARY dictdb.restricted_dict (
|
||||
key UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key
|
||||
SOURCE(EXECUTABLE(COMMAND 'echo -E "1\thello"' FORMAT TabSeparated))
|
||||
LIFETIME(MIN 0 MAX 1)
|
||||
LAYOUT(CACHE(SIZE_IN_CELLS 10));
|
||||
|
||||
-- because of lazy load we can check only in dictGet query
|
||||
select dictGetString('dictdb.restricted_dict', 'value', toUInt64(1)); -- {serverError 482}
|
||||
|
||||
select 'Ok.';
|
||||
|
||||
DROP DICTIONARY IF EXISTS dictdb.restricted_dict;
|
||||
|
||||
DROP DATABASE IF EXISTS dictdb;
|
13
docs/en/interfaces/third-party/gui.md
vendored
13
docs/en/interfaces/third-party/gui.md
vendored
@ -113,4 +113,17 @@ Features:
|
||||
- Refactorings.
|
||||
- Search and Navigation.
|
||||
|
||||
### Looker
|
||||
[Looker](https://looker.com) is a data platform and business intelligence tool with support for 50+ database dialects including ClickHouse. Looker is available as a SaaS platform and self-hosted. Users
|
||||
can use Looker via the browser to explore data, build visualizations and dashboards, schedule reports, and share their
|
||||
insights with colleagues. Looker provides a rich set of tools to embed these features in other applications, and an API
|
||||
to integrate data with other applications.
|
||||
|
||||
Features:
|
||||
|
||||
- Designed around ease of use and self-service for end users.
|
||||
- Easy and agile development using LookML, a language which supports currated
|
||||
[Data Modeling](https://looker.com/platform/data-modeling) to support report writers and end users.
|
||||
- Powerful workflow integration via Looker's [Data Actions](https://looker.com/platform/actions).
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/en/interfaces/third-party/gui/) <!--hide-->
|
||||
|
@ -28,10 +28,10 @@ Users are recorded in the `users` section. Here is a fragment of the `users.xml`
|
||||
Each list item has one of the following forms:
|
||||
<ip> The IP address or subnet mask. For example: 198.51.100.0/24 or 2001:DB8::/32.
|
||||
<host> Host name. For example: example01. A DNS query is made for verification, and all addresses obtained are compared with the address of the customer.
|
||||
<host_regexp> Regular expression for host names. For example, ^example\d\d-\d\d-\d\.yandex\.ru$
|
||||
<host_regexp> Regular expression for host names. For example, ^example\d\d-\d\d-\d\.host\.ru$
|
||||
To check it, a DNS PTR request is made for the client's address and a regular expression is applied to the result.
|
||||
Then another DNS query is made for the result of the PTR query, and all received address are compared to the client address.
|
||||
We strongly recommend that the regex ends with \.yandex\.ru$.
|
||||
We strongly recommend that the regex ends with \.host\.ru$.
|
||||
|
||||
If you are installing ClickHouse yourself, specify here:
|
||||
<networks>
|
||||
@ -104,5 +104,4 @@ The user can get a list of all databases and tables in them by using `SHOW` quer
|
||||
|
||||
Database access is not related to the [readonly](settings/permissions_for_queries.md#settings_readonly) setting. You can't grant full access to one database and `readonly` access to another one.
|
||||
|
||||
|
||||
[Original article](https://clickhouse.yandex/docs/en/operations/access_rights/) <!--hide-->
|
||||
|
@ -223,7 +223,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), (
|
||||
- if `input_format_values_interpret_expressions=0` and `format_values_deduce_templates_of_expressions=1` expressions in the first, second and third rows will be parsed using template `lower(String)` and interpreted together, expression is the forth row will be parsed with another template (`upper(String)`)
|
||||
- if `input_format_values_interpret_expressions=1` and `format_values_deduce_templates_of_expressions=1` - the same as in previous case, but also allows fallback to interpreting expressions separately if it's not possible to deduce template.
|
||||
|
||||
This feature is experimental, disabled by default.
|
||||
Enabled by default.
|
||||
|
||||
## input_format_values_accurate_types_of_literals {#settings-input_format_values_accurate_types_of_literals}
|
||||
This setting is used only when `input_format_values_deduce_templates_of_expressions = 1`. It can happen, that expressions for some column have the same structure, but contain numeric literals of different types, e.g
|
||||
|
@ -64,13 +64,13 @@ Each element of the list can have one of the following forms:
|
||||
|
||||
- `<host>` — Hostname.
|
||||
|
||||
Example: `server01.yandex.ru`.
|
||||
Example: `example01.host.ru`.
|
||||
|
||||
To check access, a DNS query is performed, and all returned IP addresses are compared to the peer address.
|
||||
|
||||
- `<host_regexp>` — Regular expression for hostnames.
|
||||
|
||||
Example, `^server\d\d-\d\d-\d\.yandex\.ru$`
|
||||
Example, `^example\d\d-\d\d-\d\.host\.ru$`
|
||||
|
||||
To check access, a [DNS PTR query](https://en.wikipedia.org/wiki/Reverse_DNS_lookup) is performed for the peer address and then the specified regexp is applied. Then, another DNS query is performed for the results of the PTR query and all the received addresses are compared to the peer address. We strongly recommend that regexp ends with $.
|
||||
|
||||
|
@ -72,6 +72,8 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64
|
||||
## Details of Implementation
|
||||
|
||||
- Multiple `SELECT` queries can be performed concurrently, but `INSERT` queries will wait each other.
|
||||
- Supported creating new file by `INSERT` query.
|
||||
- If file exists, `INSERT` would append new values in it.
|
||||
- Not supported:
|
||||
- `ALTER`
|
||||
- `SELECT ... SAMPLE`
|
||||
|
@ -22,7 +22,14 @@ Compressed data for `INSERT` and `ALTER` queries is replicated (for more informa
|
||||
- The `DROP TABLE` query deletes the replica located on the server where the query is run.
|
||||
- The `RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas.
|
||||
|
||||
To use replication, set the addresses of the ZooKeeper cluster in the config file. Example:
|
||||
ClickHouse uses [Apache ZooKeeper](https://zookeeper.apache.org) for storing replicas meta information. Use ZooKeeper version 3.4.5 or newer.
|
||||
|
||||
To use replication, set parameters in the [zookeeper](../server_settings/settings.md#server-settings_zookeeper) server configuration section.
|
||||
|
||||
!!! attention "Attention"
|
||||
Don't neglect the securiry setting. ClickHouse supports the `digest` [ACL scheme](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) of the ZooKeeper security subsystem.
|
||||
|
||||
Example of setting the addresses of the ZooKeeper cluster:
|
||||
|
||||
```xml
|
||||
<zookeeper>
|
||||
@ -40,9 +47,7 @@ To use replication, set the addresses of the ZooKeeper cluster in the config fil
|
||||
</node>
|
||||
</zookeeper>
|
||||
```
|
||||
|
||||
Use ZooKeeper version 3.4.5 or later.
|
||||
|
||||
|
||||
You can specify any existing ZooKeeper cluster and the system will use a directory on it for its own data (the directory is specified when creating a replicatable table).
|
||||
|
||||
If ZooKeeper isn't set in the config file, you can't create replicated tables, and any existing replicated tables will be read-only.
|
||||
|
@ -28,10 +28,10 @@
|
||||
Каждый элемент списка имеет одну из следующих форм:
|
||||
<ip> IP-адрес или маска подсети. Например, 198.51.100.0/24 или 2001:DB8::/32.
|
||||
<host> Имя хоста. Например: example01. Для проверки делается DNS-запрос, и все полученные адреса сравниваются с адресом клиента.
|
||||
<host_regexp> Регулярное выражение для имён хостов. Например, ^example\d\d-\d\d-\d\.yandex\.ru$
|
||||
<host_regexp> Регулярное выражение для имён хостов. Например, ^example\d\d-\d\d-\d\.host\.ru$
|
||||
Для проверки, для адреса клиента делается DNS PTR-запрос и к результату применяется регулярное выражение.
|
||||
Потом для результата PTR-запроса делается снова DNS-запрос, и все полученные адреса сравниваются с адресом клиента.
|
||||
Настоятельно рекомендуется, чтобы регулярное выражение заканчивалось на \.yandex\.ru$.
|
||||
Настоятельно рекомендуется, чтобы регулярное выражение заканчивалось на \.host\.ru$.
|
||||
|
||||
Если вы устанавливаете ClickHouse самостоятельно, укажите здесь:
|
||||
<networks>
|
||||
|
@ -63,13 +63,13 @@
|
||||
|
||||
- `<host>` — Имя хоста.
|
||||
|
||||
Пример: `server01.yandex.ru`.
|
||||
Пример: `example01.host.ru`.
|
||||
|
||||
Для проверки доступа выполняется DNS-запрос, и все возвращенные IP-адреса сравниваются с адресом клиента.
|
||||
|
||||
- `<host_regexp>` — Регулярное выражение для имен хостов.
|
||||
|
||||
Пример, `^server\d\d-\d\d-\d\.yandex\.ru$`
|
||||
Пример, `^example\d\d-\d\d-\d\.host\.ru$`
|
||||
|
||||
Для проверки доступа выполняется [DNS запрос PTR](https://en.wikipedia.org/wiki/Reverse_DNS_lookup) для адреса клиента, а затем применяется заданное регулярное выражение. Затем, для результатов запроса PTR выполняется другой DNS-запрос и все полученные адреса сравниваются с адресом клиента. Рекомендуем завершать регулярное выражение символом $.
|
||||
|
||||
|
@ -69,6 +69,8 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64
|
||||
## Детали реализации
|
||||
|
||||
- Поддерживается одновременное выполнение множества запросов `SELECT`, запросы `INSERT` могут выполняться только последовательно.
|
||||
- Поддерживается создание ещё не существующего файла при запросе `INSERT`.
|
||||
- Для существующих файлов `INSERT` записывает в конец файла.
|
||||
- Не поддерживается:
|
||||
- использование операций `ALTER` и `SELECT...SAMPLE`;
|
||||
- индексы;
|
||||
|
@ -22,7 +22,14 @@
|
||||
- `DROP TABLE` удаляет реплику, расположенную на том сервере, где выполняется запрос.
|
||||
- Запрос `RENAME` переименовывает таблицу на одной реплик. Другими словами, реплицируемые таблицы на разных репликах могут называться по-разному.
|
||||
|
||||
Чтобы использовать репликацию, укажите в конфигурационном файле адреса ZooKeeper кластера. Пример:
|
||||
ClickHouse хранит метаинформацию о репликах в [Apache ZooKeeper](https://zookeeper.apache.org). Используйте ZooKeeper 3.4.5 или новее.
|
||||
|
||||
Для использовании репликации, установите параметры в секции [zookeeper](../server_settings/settings.md#server-settings_zookeeper) конфигурации сервера.
|
||||
|
||||
!!! attention "Внимание"
|
||||
Не пренебрегайте настройками безопасности. ClickHouse поддерживает [ACL схему](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) `digest` подсистемы безопасности ZooKeeper.
|
||||
|
||||
Пример указания адресов кластера ZooKeeper:
|
||||
|
||||
```xml
|
||||
<zookeeper>
|
||||
@ -41,8 +48,6 @@
|
||||
</zookeeper>
|
||||
```
|
||||
|
||||
Используйте ZooKeeper версии 3.4.5 или более новый.
|
||||
|
||||
Можно указать любой имеющийся у вас ZooKeeper-кластер - система будет использовать в нём одну директорию для своих данных (директория указывается при создании реплицируемой таблицы).
|
||||
|
||||
Если в конфигурационном файле не настроен ZooKeeper, то вы не сможете создать реплицируемые таблицы, а уже имеющиеся реплицируемые таблицы будут доступны в режиме только на чтение.
|
||||
|
@ -35,6 +35,38 @@ SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673)
|
||||
└───────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## greatCircleAngle
|
||||
|
||||
Вычисляет угловое расстояние на сфере по [формуле большого круга](https://en.wikipedia.org/wiki/Great-circle_distance).
|
||||
|
||||
```sql
|
||||
greatCircleDistance(lon1Deg, lat1Deg, lon2Deg, lat2Deg)
|
||||
```
|
||||
|
||||
**Входные параметры**
|
||||
|
||||
- `lon1Deg` — долгота первой точки в градусах.
|
||||
- `lat1Deg` — широта первой точки в градусах.
|
||||
- `lon2Deg` — долгота второй точки в градусах.
|
||||
- `lat2Deg` — широта второй точки в градусах.
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
Длина дуги большого круга между двумя точками в градусах.
|
||||
|
||||
**Пример**
|
||||
|
||||
```sql
|
||||
SELECT greatCircleAngle(0, 0, 45, 0) AS arc
|
||||
```
|
||||
|
||||
```text
|
||||
┌─arc─┐
|
||||
│ 45 │
|
||||
└─────┘
|
||||
```
|
||||
|
||||
|
||||
## pointInEllipses
|
||||
|
||||
Проверяет, принадлежит ли точка хотя бы одному из эллипсов.
|
||||
@ -157,6 +189,121 @@ SELECT geohashDecode('ezs42') AS res
|
||||
└─────────────────────────────────┘
|
||||
```
|
||||
|
||||
## h3IsValid {#h3isvalid}
|
||||
|
||||
Проверяет корректность H3-индекса.
|
||||
|
||||
```sql
|
||||
h3IsValid(h3index)
|
||||
```
|
||||
|
||||
**Входные значения**
|
||||
|
||||
- `h3index` — идентификатор шестиугольника. Тип данных — [UInt64](../../data_types/int_uint.md).
|
||||
|
||||
**Возвращаемые значения**
|
||||
|
||||
- 0 — число не является H3-индексом
|
||||
- 1 — число является H3-индексом
|
||||
|
||||
Тип — [UInt8](../../data_types/int_uint.md).
|
||||
|
||||
**Пример**
|
||||
|
||||
```sql
|
||||
SELECT h3IsValid(630814730351855103) as h3IsValid
|
||||
```
|
||||
```text
|
||||
┌─h3IsValid─┐
|
||||
│ 1 │
|
||||
└───────────┘
|
||||
```
|
||||
|
||||
## h3GetResolution
|
||||
|
||||
Извлекает разрешение H3-индекса.
|
||||
|
||||
```sql
|
||||
h3GetResolution(h3index)
|
||||
```
|
||||
|
||||
**Входные значения**
|
||||
|
||||
- `h3index` — идентификатор шестиугольника. Тип данных — [UInt64](../../data_types/int_uint.md).
|
||||
|
||||
**Возвращаемые значения**
|
||||
|
||||
- Разрешение сетки, от 0 до 15.
|
||||
- Для несуществующего идентификатора может быть возвращено произвольное значение, используйте [h3IsValid](#h3isvalid) для проверки идентификаторов
|
||||
|
||||
Тип — [UInt8](../../data_types/int_uint.md).
|
||||
|
||||
**Пример**
|
||||
|
||||
```sql
|
||||
SELECT h3GetResolution(639821929606596015) as resolution
|
||||
```
|
||||
```text
|
||||
┌─resolution─┐
|
||||
│ 14 │
|
||||
└────────────┘
|
||||
```
|
||||
|
||||
## h3EdgeAngle
|
||||
|
||||
Информирует о среднем размере стороны шестигранника H3 в градусах
|
||||
|
||||
```sql
|
||||
h3EdgeAngle(resolution)
|
||||
```
|
||||
|
||||
**Входные значения**
|
||||
|
||||
- `resolution` — требуемое разрешение индекса. Тип данных — [UInt8](../../data_types/int_uint.md). Диапазон возможных значений — `[0, 15]`.
|
||||
|
||||
**Возвращаемые значения**
|
||||
|
||||
Средняя длина стороны многоугольника H3 в градусах, тип — [Float64](../../data_types/float.md).
|
||||
|
||||
**Пример**
|
||||
|
||||
```sql
|
||||
SELECT h3EdgeAngle(10) as edgeAngle
|
||||
```
|
||||
```text
|
||||
┌─────────h3EdgeAngle(10)─┐
|
||||
│ 0.0005927224846720883 │
|
||||
└───────────────────────┘
|
||||
```
|
||||
|
||||
|
||||
## h3EdgeLengthM
|
||||
|
||||
Информирует о среднем размере стороны шестигранника H3 в метрах
|
||||
|
||||
```sql
|
||||
h3EdgeLengthM(resolution)
|
||||
```
|
||||
|
||||
**Входные значения**
|
||||
|
||||
- `resolution` — требуемое разрешение индекса. Тип данных — [UInt8](../../data_types/int_uint.md). Диапазон возможных значений — `[0, 15]`.
|
||||
|
||||
**Возвращаемые значения**
|
||||
|
||||
Средняя длина стороны многоугольника H3 в метрах, тип — [Float64](../../data_types/float.md).
|
||||
|
||||
**Пример**
|
||||
|
||||
```sql
|
||||
SELECT h3EdgeLengthM(15) as edgeLengthM
|
||||
```
|
||||
```text
|
||||
┌─edgeLengthM─┐
|
||||
│ 0.509713273 │
|
||||
└─────────────┘
|
||||
```
|
||||
|
||||
## geoToH3
|
||||
|
||||
Получает H3 индекс точки `(lon, lat)` с заданным разрешением
|
||||
@ -189,4 +336,38 @@ SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index
|
||||
└────────────────────┘
|
||||
```
|
||||
|
||||
## h3kRing
|
||||
|
||||
Возвращает H3-индексы шестиугольников в радиусе `k` от данного в произвольном порядке
|
||||
|
||||
```sql
|
||||
h3kRing(h3index, k)
|
||||
```
|
||||
**Входные значения**
|
||||
|
||||
- `h3index` — идентификатор шестиугольника. Тип данных — [UInt64](../../data_types/int_uint.md).
|
||||
- `k` — радиус. Тип данных — [целое число](../../data_types/int_uint.md)
|
||||
|
||||
**Возвращаемые значения**
|
||||
|
||||
[Массив](../../data_types/array.md) из H3-индексов типа [UInt64](../../data_types/int_uint.md).
|
||||
|
||||
**Пример**
|
||||
|
||||
```sql
|
||||
SELECT arrayJoin(h3kRing(644325529233966508, 1)) AS h3index
|
||||
```
|
||||
```text
|
||||
┌────────────h3index─┐
|
||||
│ 644325529233966508 │
|
||||
│ 644325529233966497 │
|
||||
│ 644325529233966510 │
|
||||
│ 644325529233966504 │
|
||||
│ 644325529233966509 │
|
||||
│ 644325529233966355 │
|
||||
│ 644325529233966354 │
|
||||
└────────────────────┘
|
||||
```
|
||||
|
||||
|
||||
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/geo/) <!--hide-->
|
||||
|
Loading…
Reference in New Issue
Block a user