Merge remote-tracking branch 'rschu1ze/master' into fix-norm_distance

This commit is contained in:
Robert Schulze 2024-01-20 02:28:47 +00:00
commit 3f0fbfd885
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
25 changed files with 295 additions and 174 deletions

View File

@ -56,6 +56,9 @@ echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/sys
# Install previous release packages
install_packages previous_release_package_folder
# Save old settings from system table for settings changes check
clickhouse-local -q "select * from system.settings format Native" > old_settings.native
# Initial run without S3 to create system.*_log on local file system to make it
# available for dump via clickhouse-local
configure
@ -152,6 +155,63 @@ install_packages package_folder
export ZOOKEEPER_FAULT_INJECTION=1
configure
# Check that all new/changed setting were added in settings changes history.
# Some settings can be different for builds with sanitizers, so we check
# settings changes only for non-sanitizer builds.
IS_SANITIZED=$(clickhouse-local --query "SELECT value LIKE '%-fsanitize=%' FROM system.build_options WHERE name = 'CXX_FLAGS'")
if [ "${IS_SANITIZED}" -eq "0" ]
then
clickhouse-local -q "select * from system.settings format Native" > new_settings.native
clickhouse-local -nmq "
CREATE TABLE old_settings AS file('old_settings.native');
CREATE TABLE new_settings AS file('new_settings.native');
SELECT
name,
new_settings.value AS new_value,
old_settings.value AS old_value
FROM new_settings
LEFT JOIN old_settings ON new_settings.name = old_settings.name
WHERE (new_settings.value != old_settings.value) AND (name NOT IN (
SELECT arrayJoin(tupleElement(changes, 'name'))
FROM system.settings_changes
WHERE version = extract(version(), '^(?:\\d+\\.\\d+)')
))
SETTINGS join_use_nulls = 1
INTO OUTFILE 'changed_settings.txt'
FORMAT PrettyCompactNoEscapes;
SELECT name
FROM new_settings
WHERE (name NOT IN (
SELECT name
FROM old_settings
)) AND (name NOT IN (
SELECT arrayJoin(tupleElement(changes, 'name'))
FROM system.settings_changes
WHERE version = extract(version(), '^(?:\\d+\\.\\d+)')
))
INTO OUTFILE 'new_settings.txt'
FORMAT PrettyCompactNoEscapes;
"
if [ -s changed_settings.txt ]
then
mv changed_settings.txt /test_output/
echo -e "Changed settings are not reflected in settings changes history (see changed_settings.txt)$FAIL$(head_escaped /test_output/changed_settings.txt)" >> /test_output/test_results.tsv
else
echo -e "There are no changed settings or they are reflected in settings changes history$OK" >> /test_output/test_results.tsv
fi
if [ -s new_settings.txt ]
then
mv new_settings.txt /test_output/
echo -e "New settings are not reflected in settings changes history (see new_settings.txt)$FAIL$(head_escaped /test_output/new_settings.txt)" >> /test_output/test_results.tsv
else
echo -e "There are no new settings or they are reflected in settings changes history$OK" >> /test_output/test_results.tsv
fi
fi
# Just in case previous version left some garbage in zk
sudo cat /etc/clickhouse-server/config.d/lost_forever_check.xml \
| sed "s|>1<|>0<|g" \
@ -257,6 +317,8 @@ clickhouse-local --structure "test String, res String, time Nullable(Float32), d
(test like '%Fatal message%') DESC,
(test like '%Error message%') DESC,
(test like '%previous release%') DESC,
(test like '%Changed settings%') DESC,
(test like '%New settings%') DESC,
rowNumberInAllBlocks()
LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv || echo "failure\tCannot parse test_results.tsv" > /test_output/check_status.tsv
[ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv

View File

@ -108,7 +108,7 @@ String toString(TargetArch arch);
#define AVX512VBMI_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi")))
#define AVX512BW_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw")))
#define AVX512_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f")))
#define AVX2_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2")))
#define AVX2_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,bmi2")))
#define AVX_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx"))
#define SSE42_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt")))
#define DEFAULT_FUNCTION_SPECIFIC_ATTRIBUTE
@ -122,7 +122,7 @@ String toString(TargetArch arch);
# define BEGIN_AVX512F_SPECIFIC_CODE \
_Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f\"))),apply_to=function)")
# define BEGIN_AVX2_SPECIFIC_CODE \
_Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2\"))),apply_to=function)")
_Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,bmi2\"))),apply_to=function)")
# define BEGIN_AVX_SPECIFIC_CODE \
_Pragma("clang attribute push(__attribute__((target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx\"))),apply_to=function)")
# define BEGIN_SSE42_SPECIFIC_CODE \
@ -140,7 +140,7 @@ String toString(TargetArch arch);
#define AVX512VBMI_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,avx512vl,avx512vbmi,tune=native")))
#define AVX512BW_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,avx512bw,tune=native")))
#define AVX512_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,tune=native")))
#define AVX2_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,tune=native")))
#define AVX2_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,bmi2,tune=native")))
#define AVX_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt,avx,tune=native")))
#define SSE42_FUNCTION_SPECIFIC_ATTRIBUTE __attribute__((target("sse,sse2,sse3,ssse3,sse4,popcnt",tune=native)))
#define DEFAULT_FUNCTION_SPECIFIC_ATTRIBUTE
@ -159,7 +159,7 @@ String toString(TargetArch arch);
_Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,avx512f,tune=native\")")
# define BEGIN_AVX2_SPECIFIC_CODE \
_Pragma("GCC push_options") \
_Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,tune=native\")")
_Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,avx2,bmi2,tune=native\")")
# define BEGIN_AVX_SPECIFIC_CODE \
_Pragma("GCC push_options") \
_Pragma("GCC target(\"sse,sse2,sse3,ssse3,sse4,popcnt,avx,tune=native\")")

View File

@ -26,9 +26,8 @@ class IColumn;
* `flags` can be either 0 or IMPORTANT.
* A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions.
*
* When adding new settings that control some backward incompatible changes or when changing some settings values,
* consider adding them to settings changes history in SettingsChangesHistory.h for special `compatibility` setting
* to work correctly.
* When adding new or changing existing settings add them to settings changes history in SettingsChangesHistory.h
* for tracking settings changes in different versions and for special `compatibility` setting to work correctly.
*/
#define COMMON_SETTINGS(M, ALIAS) \

View File

@ -77,18 +77,30 @@ namespace SettingsChangesHistory
/// History of settings changes that controls some backward incompatible changes
/// across all ClickHouse versions. It maps ClickHouse version to settings changes that were done
/// in this version. Settings changes is a vector of structs {setting_name, previous_value, new_value}
/// in this version. This history contains both changes to existing settings and newly added settings.
/// Settings changes is a vector of structs {setting_name, previous_value, new_value}.
/// For newly added setting choose the most appropriate previous_value (for example, if new setting
/// controls new feature and it's 'true' by default, use 'false' as previous_value).
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
{
{"24.1", {{"print_pretty_type_names", false, true, "Better user experience."},
{"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"},
{"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}}},
{"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"},
{"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"},
{"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"},
{"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"},
{"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"},
{"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"},
{"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"},
{"enable_vertical_final", false, true, "Use vertical final by default"},
{"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"},
{"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"},
{"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}}},
{"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."},
{"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"},
{"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"},
{"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"},
{"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}}},
{"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}},
{"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"},
{"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"},
{"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"},

View File

@ -56,7 +56,7 @@ static std::optional<Exception> checkTupleNames(const Strings & names)
return Exception(ErrorCodes::BAD_ARGUMENTS, "Names of tuple elements cannot be empty");
if (!names_set.insert(name).second)
return Exception(ErrorCodes::DUPLICATE_COLUMN, "Names of tuple elements must be unique");
return Exception(ErrorCodes::DUPLICATE_COLUMN, "Names of tuple elements must be unique. Duplicate name: {}", name);
}
return {};

View File

@ -100,7 +100,8 @@ ExternalUserDefinedExecutableFunctionsLoader::ExternalUserDefinedExecutableFunct
{
setConfigSettings({"function", "name", "database", "uuid"});
enableAsyncLoading(false);
enablePeriodicUpdates(true);
if (getContext()->getApplicationType() == Context::ApplicationType::SERVER)
enablePeriodicUpdates(true);
enableAlwaysLoadEverything(true);
}

View File

@ -224,6 +224,8 @@ public:
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Seek is not supported when reading from archive");
}
bool checkIfActuallySeekable() override { return false; }
off_t getPosition() override
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive");

View File

@ -15,6 +15,7 @@ namespace ErrorCodes
extern const int CANNOT_UNPACK_ARCHIVE;
extern const int LOGICAL_ERROR;
extern const int SEEK_POSITION_OUT_OF_BOUND;
extern const int CANNOT_SEEK_THROUGH_FILE;
}
using RawHandle = unzFile;
@ -285,23 +286,27 @@ public:
if (new_pos > static_cast<off_t>(file_info.uncompressed_size))
throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound");
if (file_info.compression_method == MZ_COMPRESS_METHOD_STORE)
{
/// unzSeek64() works only for non-compressed files.
checkResult(unzSeek64(raw_handle, off, whence));
return unzTell64(raw_handle);
}
/// unzSeek64() works only for non-compressed files.
///
/// We used to have a fallback here, where we would:
/// * ignore() to "seek" forward,
/// * unzCloseCurrentFile(raw_handle) + unzOpenCurrentFile(raw_handle) to seek to the
/// beginning of the file.
/// But the close+open didn't work: after closing+reopening once, the second
/// unzCloseCurrentFile() was failing with MZ_CRC_ERROR in mz_zip_entry_read_close(). Maybe
/// it's a bug in minizip where some state was inadvertently left over after close+reopen.
/// Didn't investigate because re-reading the whole file should be avoided anyway.
if (file_info.compression_method != MZ_COMPRESS_METHOD_STORE)
throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Seek in compressed archive is not supported.");
/// As a last try we go slow way, we're going to simply ignore all data before the new position.
if (new_pos < current_pos)
{
checkResult(unzCloseCurrentFile(raw_handle));
checkResult(unzOpenCurrentFile(raw_handle));
current_pos = 0;
}
checkResult(unzSeek64(raw_handle, off, whence));
return unzTell64(raw_handle);
}
ignore(new_pos - current_pos);
return new_pos;
bool checkIfActuallySeekable() override
{
/// The library doesn't support seeking in compressed files.
return handle.getFileInfo().compression_method == MZ_COMPRESS_METHOD_STORE;
}
off_t getPosition() override

View File

@ -13,7 +13,6 @@ namespace DB
/// the head of the queue, and the record with the highest priority is stored at the tail.
class LRUFileCachePriority final : public IFileCachePriority
{
friend class OvercommitFileCachePriority;
protected:
struct State
{

View File

@ -71,7 +71,7 @@ KeyMetadata::KeyMetadata(
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create key metadata with internal user id");
if (!user_.weight.has_value())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create key metadata withouot user weight");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create key metadata without user weight");
chassert(!created_base_directory || fs::exists(getPath()));
}

View File

@ -27,7 +27,8 @@ ExternalDictionariesLoader::ExternalDictionariesLoader(ContextPtr global_context
{
setConfigSettings({"dictionary", "name", "database", "uuid"});
enableAsyncLoading(true);
enablePeriodicUpdates(true);
if (getContext()->getApplicationType() == Context::ApplicationType::SERVER)
enablePeriodicUpdates(true);
}
ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(

View File

@ -1283,6 +1283,81 @@ inline void combineFiltersImpl(UInt8 * first_begin, const UInt8 * first_end, con
}
)
/* The BMI2 intrinsic, _pdep_u64 (unsigned __int64 a, unsigned __int64 mask), works
* by copying contiguous low-order bits from unsigned 64-bit integer a to destination
* at the corresponding bit locations specified by mask. To implement the column
* combination with the intrinsic, 8 contiguous bytes would be loaded from second_begin
* as a UInt64 and act the first operand, meanwhile the mask should be constructed from
* first_begin so that the bytes to be replaced (non-zero elements) are mapped to 0xFF
* at the exact bit locations and 0x00 otherwise.
*
* The construction of mask employs the SSE intrinsic, mm_cmpeq_epi8(__m128i a, __m128i
* b), which compares packed 8-bit integers in first_begin and packed 0s and outputs
* 0xFF for equality and 0x00 for inequality. The result's negation then creates the
* desired bit masks for _pdep_u64.
*
* The below example visualizes how this optimization applies to the combination of
* two quadwords from first_begin and second_begin.
*
* Addr high low
* <----------------------------------------
* first_begin............................0x00 0x11 0x12 0x00 0x00 0x13 0x14 0x15
* | mm_cmpeq_epi8(src, 0) | | | | | | | |
* v v v v v v v v v
* inv_mask..............................0xFF 0x00 0x00 0xFF 0xFF 0x00 0x00 0x00
* | (negation) | | | | | | | |
* v v v v v v v v v
* mask-------------------------+......0x00 0xFF 0xFF 0x00 0x00 0xFF 0xFF 0xFF
* | | | | | |
* v v v v v v
* dst = pdep_u64(second_begin, mask)..0x00 0x05 0x04 0x00 0x00 0x03 0x02 0x01
* ^ ^ ^ ^ ^ ^
* | | | | | |
* | | +---------+ | | |
* +------------------+ +---------+ | | | |
* | | | | | |
* second_begin...........................0x00 0x00 0x00 0x05 0x04 0x03 0x02 0x01
*
* References:
* 1. https://www.felixcloutier.com/x86/pdep
* 2. https://www.felixcloutier.com/x86/pcmpeqb:pcmpeqw:pcmpeqd
*/
DECLARE_AVX2_SPECIFIC_CODE(
inline void combineFiltersImpl(UInt8 * first_begin, const UInt8 * first_end, const UInt8 * second_begin)
{
constexpr size_t XMM_VEC_SIZE_IN_BYTES = 16;
const __m128i zero16 = _mm_setzero_si128();
while (first_begin + XMM_VEC_SIZE_IN_BYTES <= first_end)
{
__m128i src = _mm_loadu_si128(reinterpret_cast<__m128i *>(first_begin));
__m128i inv_mask = _mm_cmpeq_epi8(src, zero16);
UInt64 masks[] = {
~static_cast<UInt64>(_mm_extract_epi64(inv_mask, 0)),
~static_cast<UInt64>(_mm_extract_epi64(inv_mask, 1)),
};
for (const auto & mask: masks)
{
UInt64 dst = _pdep_u64(unalignedLoad<UInt64>(second_begin), mask);
unalignedStore<UInt64>(first_begin, dst);
first_begin += sizeof(UInt64);
second_begin += std::popcount(mask) / 8;
}
}
for (/* empty */; first_begin < first_end; ++first_begin)
{
if (*first_begin)
{
*first_begin = *second_begin++;
}
}
}
)
/// Second filter size must be equal to number of 1s in the first filter.
/// The result has size equal to first filter size and contains 1s only where both filters contain 1s.
static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second)
@ -1330,6 +1405,10 @@ static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second)
{
TargetSpecific::AVX512VBMI2::combineFiltersImpl(first_data.begin(), first_data.end(), second_data);
}
else if (isArchSupported(TargetArch::AVX2))
{
TargetSpecific::AVX2::combineFiltersImpl(first_data.begin(), first_data.end(), second_data);
}
else
#endif
{

View File

@ -31,6 +31,7 @@ public:
struct Arguments
{
const String & engine_name;
/// Mutable to allow replacing constant expressions with literals, and other transformations.
ASTs & engine_args;
ASTStorage * storage_def;
const ASTCreateQuery & query;

View File

@ -1488,25 +1488,13 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format', 'compression')
/// with optional headers() function
if (engine_args.empty() || engine_args.size() > 6)
size_t count = StorageURL::evalArgsAndCollectHeaders(engine_args, configuration.headers_from_ast, local_context);
if (count == 0 || count > 6)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage S3 requires 1 to 5 arguments: "
"url, [NOSIGN | access_key_id, secret_access_key], name of used format and [compression_method]");
auto * header_it = StorageURL::collectHeaders(engine_args, configuration.headers_from_ast, local_context);
if (header_it != engine_args.end())
engine_args.erase(header_it);
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context);
/// Size -> argument indexes
static std::unordered_map<size_t, std::unordered_map<std::string_view, size_t>> size_to_engine_args
{
{1, {{}}},
{6, {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"compression_method", 5}}}
};
std::unordered_map<std::string_view, size_t> engine_args_to_idx;
bool no_sign_request = false;
@ -1514,7 +1502,7 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context
/// - s3(source, format)
/// - s3(source, NOSIGN)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not.
if (engine_args.size() == 2)
if (count == 2)
{
auto second_arg = checkAndGetLiteralArgument<String>(engine_args[1], "format/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
@ -1524,10 +1512,10 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context
}
/// For 3 arguments we support 2 possible variants:
/// - s3(source, format, compression_method)
/// - s3(source, access_key_id, access_key_id)
/// - s3(source, access_key_id, secret_access_key)
/// - s3(source, NOSIGN, format)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or format name.
else if (engine_args.size() == 3)
else if (count == 3)
{
auto second_arg = checkAndGetLiteralArgument<String>(engine_args[1], "format/access_key_id/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
@ -1545,7 +1533,7 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context
/// - s3(source, access_key_id, secret_access_key, format)
/// - s3(source, NOSIGN, format, compression_method)
/// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN or not.
else if (engine_args.size() == 4)
else if (count == 4)
{
auto second_arg = checkAndGetLiteralArgument<String>(engine_args[1], "access_key_id/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
@ -1569,7 +1557,7 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context
/// For 5 arguments we support 2 possible variants:
/// - s3(source, access_key_id, secret_access_key, session_token, format)
/// - s3(source, access_key_id, secret_access_key, format, compression)
else if (engine_args.size() == 5)
else if (count == 5)
{
auto fourth_arg = checkAndGetLiteralArgument<String>(engine_args[3], "session_token/format");
if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg))
@ -1581,9 +1569,9 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context
engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}};
}
}
else
else if (count == 6)
{
engine_args_to_idx = size_to_engine_args[engine_args.size()];
engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"compression_method", 5}};
}
/// This argument is always the first

View File

@ -1324,7 +1324,7 @@ FormatSettings StorageURL::getFormatSettingsFromArgs(const StorageFactory::Argum
return format_settings;
}
ASTs::iterator StorageURL::collectHeaders(
size_t StorageURL::evalArgsAndCollectHeaders(
ASTs & url_function_args, HTTPHeaderEntries & header_entries, ContextPtr context)
{
ASTs::iterator headers_it = url_function_args.end();
@ -1382,7 +1382,11 @@ ASTs::iterator StorageURL::collectHeaders(
(*arg_it) = evaluateConstantExpressionOrIdentifierAsLiteral((*arg_it), context);
}
return headers_it;
if (headers_it == url_function_args.end())
return url_function_args.size();
std::rotate(headers_it, std::next(headers_it), url_function_args.end());
return url_function_args.size() - 1;
}
void StorageURL::processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection)
@ -1412,21 +1416,19 @@ StorageURL::Configuration StorageURL::getConfiguration(ASTs & args, ContextPtr l
if (auto named_collection = tryGetNamedCollectionWithOverrides(args, local_context))
{
StorageURL::processNamedCollectionResult(configuration, *named_collection);
collectHeaders(args, configuration.headers, local_context);
evalArgsAndCollectHeaders(args, configuration.headers, local_context);
}
else
{
if (args.empty() || args.size() > 3)
size_t count = evalArgsAndCollectHeaders(args, configuration.headers, local_context);
if (count == 0 || count > 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, bad_arguments_error_message);
auto * header_it = collectHeaders(args, configuration.headers, local_context);
if (header_it != args.end())
args.erase(header_it);
configuration.url = checkAndGetLiteralArgument<String>(args[0], "url");
if (args.size() > 1)
if (count > 1)
configuration.format = checkAndGetLiteralArgument<String>(args[1], "format");
if (args.size() == 3)
if (count == 3)
configuration.compression_method = checkAndGetLiteralArgument<String>(args[2], "compression_method");
}

View File

@ -294,7 +294,10 @@ public:
static Configuration getConfiguration(ASTs & args, ContextPtr context);
static ASTs::iterator collectHeaders(ASTs & url_function_args, HTTPHeaderEntries & header_entries, ContextPtr context);
/// Does evaluateConstantExpressionOrIdentifierAsLiteral() on all arguments.
/// If `headers(...)` argument is present, parses it and moves it to the end of the array.
/// Returns number of arguments excluding `headers(...)`.
static size_t evalArgsAndCollectHeaders(ASTs & url_function_args, HTTPHeaderEntries & header_entries, ContextPtr context);
static void processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection);
};

View File

@ -67,23 +67,11 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
else
{
auto * header_it = StorageURL::collectHeaders(args, configuration.headers_from_ast, context);
if (header_it != args.end())
args.erase(header_it);
size_t count = StorageURL::evalArgsAndCollectHeaders(args, configuration.headers_from_ast, context);
if (args.empty() || args.size() > 7)
if (count == 0 || count > 7)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature());
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
/// Size -> argument indexes
static std::unordered_map<size_t, std::unordered_map<std::string_view, size_t>> size_to_args
{
{1, {{}}},
{7, {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}}
};
std::unordered_map<std::string_view, size_t> args_to_idx;
bool no_sign_request = false;
@ -92,7 +80,7 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
/// - s3(source, format)
/// - s3(source, NOSIGN)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not.
if (args.size() == 2)
if (count == 2)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
@ -102,10 +90,10 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
}
/// For 3 arguments we support 3 possible variants:
/// - s3(source, format, structure)
/// - s3(source, access_key_id, access_key_id)
/// - s3(source, access_key_id, secret_access_key)
/// - s3(source, NOSIGN, format)
/// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not.
else if (args.size() == 3)
else if (count == 3)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/access_key_id/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
@ -120,11 +108,11 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
}
/// For 4 arguments we support 4 possible variants:
/// - s3(source, format, structure, compression_method),
/// - s3(source, access_key_id, access_key_id, format),
/// - s3(source, access_key_id, access_key_id, session_token)
/// - s3(source, access_key_id, secret_access_key, format),
/// - s3(source, access_key_id, secret_access_key, session_token)
/// - s3(source, NOSIGN, format, structure)
/// We can distinguish them by looking at the 2-nd and 4-th argument: check if it's a format name or not.
else if (args.size() == 4)
else if (count == 4)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/access_key_id/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
@ -150,12 +138,12 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
}
}
/// For 5 arguments we support 3 possible variants:
/// - s3(source, access_key_id, access_key_id, format, structure)
/// - s3(source, access_key_id, access_key_id, session_token, format)
/// - s3(source, access_key_id, secret_access_key, format, structure)
/// - s3(source, access_key_id, secret_access_key, session_token, format)
/// - s3(source, NOSIGN, format, structure, compression_method)
/// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or no,
/// and by the 4-th argument, check if it's a format name or not
else if (args.size() == 5)
else if (count == 5)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "NOSIGN/access_key_id");
if (boost::iequals(second_arg, "NOSIGN"))
@ -177,10 +165,10 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
}
}
// For 6 arguments we support 2 possible variants:
/// - s3(source, access_key_id, access_key_id, format, structure, compression_method)
/// - s3(source, access_key_id, access_key_id, session_token, format, structure)
/// - s3(source, access_key_id, secret_access_key, format, structure, compression_method)
/// - s3(source, access_key_id, secret_access_key, session_token, format, structure)
/// We can distinguish them by looking at the 4-th argument: check if it's a format name or not
else if (args.size() == 6)
else if (count == 6)
{
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/session_token");
if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg))
@ -192,9 +180,9 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}};
}
}
else
else if (count == 7)
{
args_to_idx = size_to_args[args.size()];
args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}};
}
/// This argument is always the first
@ -262,24 +250,16 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String &
}
else
{
/// If arguments contain headers, just remove it and add to the end of arguments later
/// (header argument can be at any position).
HTTPHeaderEntries tmp_headers;
auto * headers_it = StorageURL::collectHeaders(args, tmp_headers, context);
ASTPtr headers_ast;
if (headers_it != args.end())
{
headers_ast = *headers_it;
args.erase(headers_it);
}
size_t count = StorageURL::evalArgsAndCollectHeaders(args, tmp_headers, context);
if (args.empty() || args.size() > getMaxNumberOfArguments())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function, got {}", getMaxNumberOfArguments(), args.size());
if (count == 0 || count > getMaxNumberOfArguments())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function, got {}", getMaxNumberOfArguments(), count);
auto structure_literal = std::make_shared<ASTLiteral>(structure);
/// s3(s3_url)
if (args.size() == 1)
if (count == 1)
{
/// Add format=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
@ -287,7 +267,7 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String &
}
/// s3(s3_url, format) or s3(s3_url, NOSIGN)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not.
else if (args.size() == 2)
else if (count == 2)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
/// If there is NOSIGN, add format=auto before structure.
@ -296,10 +276,10 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String &
args.push_back(structure_literal);
}
/// s3(source, format, structure) or
/// s3(source, access_key_id, access_key_id) or
/// s3(source, access_key_id, secret_access_key) or
/// s3(source, NOSIGN, format)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither.
else if (args.size() == 3)
else if (count == 3)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
@ -308,7 +288,7 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String &
}
else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg))
{
args.back() = structure_literal;
args[count - 1] = structure_literal;
}
else
{
@ -318,48 +298,45 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String &
}
}
/// s3(source, format, structure, compression_method) or
/// s3(source, access_key_id, access_key_id, format) or
/// s3(source, access_key_id, secret_access_key, format) or
/// s3(source, NOSIGN, format, structure)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither.
else if (args.size() == 4)
else if (count == 4)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
{
args.back() = structure_literal;
args[count - 1] = structure_literal;
}
else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg))
{
args[args.size() - 2] = structure_literal;
args[count - 2] = structure_literal;
}
else
{
args.push_back(structure_literal);
}
}
/// s3(source, access_key_id, access_key_id, format, structure) or
/// s3(source, access_key_id, secret_access_key, format, structure) or
/// s3(source, NOSIGN, format, structure, compression_method)
/// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or not.
else if (args.size() == 5)
else if (count == 5)
{
auto sedond_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
if (boost::iequals(sedond_arg, "NOSIGN"))
{
args[args.size() - 2] = structure_literal;
args[count - 2] = structure_literal;
}
else
{
args.back() = structure_literal;
args[count - 1] = structure_literal;
}
}
/// s3(source, access_key_id, access_key_id, format, structure, compression)
else if (args.size() == 6)
/// s3(source, access_key_id, secret_access_key, format, structure, compression)
else if (count == 6)
{
args[args.size() - 2] = structure_literal;
args[count - 2] = structure_literal;
}
if (headers_ast)
args.push_back(headers_ast);
}
}

View File

@ -57,16 +57,24 @@ void TableFunctionURL::parseArgumentsImpl(ASTs & args, const ContextPtr & contex
if (format == "auto")
format = FormatFactory::instance().getFormatFromFileName(Poco::URI(filename).getPath(), true);
StorageURL::collectHeaders(args, configuration.headers, context);
StorageURL::evalArgsAndCollectHeaders(args, configuration.headers, context);
}
else
{
auto * headers_it = StorageURL::collectHeaders(args, configuration.headers, context);
size_t count = StorageURL::evalArgsAndCollectHeaders(args, configuration.headers, context);
/// ITableFunctionFileLike cannot parse headers argument, so remove it.
if (headers_it != args.end())
args.erase(headers_it);
ASTPtr headers_ast;
if (count != args.size())
{
chassert(count + 1 == args.size());
headers_ast = args.back();
args.pop_back();
}
ITableFunctionFileLike::parseArgumentsImpl(args, context);
if (headers_ast)
args.push_back(headers_ast);
}
}
@ -82,15 +90,15 @@ void TableFunctionURL::addColumnsStructureToArguments(ASTs & args, const String
}
else
{
/// If arguments contain headers, just remove it and add to the end of arguments later
/// (header argument can be at any position).
/// If arguments contain headers, just remove it and add to the end of arguments later.
HTTPHeaderEntries tmp_headers;
auto * headers_it = StorageURL::collectHeaders(args, tmp_headers, context);
size_t count = StorageURL::evalArgsAndCollectHeaders(args, tmp_headers, context);
ASTPtr headers_ast;
if (headers_it != args.end())
if (count != args.size())
{
headers_ast = *headers_it;
args.erase(headers_it);
chassert(count + 1 == args.size());
headers_ast = args.back();
args.pop_back();
}
ITableFunctionFileLike::addColumnsStructureToArguments(args, desired_structure, context);

View File

@ -1,41 +1,11 @@
-- negative tests
-- const and non-const inputs
Row 1:
──────
arr: [1,2,3,4,5]
len: 1
arrayShingles([1, 2, 3, 4, 5], 1): [[1],[2],[3],[4],[5]]
arrayShingles(materialize([1, 2, 3, 4, 5]), 1): [[1],[2],[3],[4],[5]]
Row 1:
──────
arr: [1,2,3,4,5]
len: 3
arrayShingles([1, 2, 3, 4, 5], 3): [[1,2,3],[2,3,4],[3,4,5]]
arrayShingles(materialize([1, 2, 3, 4, 5]), 3): [[1,2,3],[2,3,4],[3,4,5]]
Row 1:
──────
arr: [1,2,3,4,5]
len: 5
arrayShingles([1, 2, 3, 4, 5], 5): [[1,2,3,4,5]]
arrayShingles(materialize([1, 2, 3, 4, 5]), 5): [[1,2,3,4,5]]
Row 1:
──────
arr: ['ab','c','de','','hi']
len: 1
arrayShingles(['ab', 'c', 'de', '', 'hi'], 1): [['ab'],['c'],['de'],[''],['hi']]
arrayShingles(materialize(['ab', 'c', 'de', '', 'hi']), 1): [['ab'],['c'],['de'],[''],['hi']]
Row 1:
──────
arr: ['ab','c','de','','hi']
len: 3
arrayShingles(['ab', 'c', 'de', '', 'hi'], 3): [['ab','c','de'],['c','de',''],['de','','hi']]
arrayShingles(materialize(['ab', 'c', 'de', '', 'hi']), 3): [['ab','c','de'],['c','de',''],['de','','hi']]
Row 1:
──────
arr: ['ab','c','de','','hi']
len: 5
arrayShingles(['ab', 'c', 'de', '', 'hi'], 5): [['ab','c','de','','hi']]
arrayShingles(materialize(['ab', 'c', 'de', '', 'hi']), 5): [['ab','c','de','','hi']]
[1,2,3,4,5] 1 [[1],[2],[3],[4],[5]] [[1],[2],[3],[4],[5]]
[1,2,3,4,5] 3 [[1,2,3],[2,3,4],[3,4,5]] [[1,2,3],[2,3,4],[3,4,5]]
[1,2,3,4,5] 5 [[1,2,3,4,5]] [[1,2,3,4,5]]
['ab','c','de','','hi'] 1 [['ab'],['c'],['de'],[''],['hi']] [['ab'],['c'],['de'],[''],['hi']]
['ab','c','de','','hi'] 3 [['ab','c','de'],['c','de',''],['de','','hi']] [['ab','c','de'],['c','de',''],['de','','hi']]
['ab','c','de','','hi'] 5 [['ab','c','de','','hi']] [['ab','c','de','','hi']]
-- special cases
[[2],[1]]
[[2],[1]]

View File

@ -10,13 +10,13 @@ SELECT arrayShingles([1, 2, 3, 4, 5], 6); -- { serverError BAD_ARGUMENTS }
SELECT arrayShingles([], 1); -- { serverError BAD_ARGUMENTS }
SELECT '-- const and non-const inputs';
SELECT [1, 2, 3, 4, 5] AS arr, 1 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len) FORMAT Vertical;
SELECT [1, 2, 3, 4, 5] AS arr, 3 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len) FORMAT Vertical;
SELECT [1, 2 ,3, 4, 5] AS arr, 5 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len) FORMAT Vertical;
SELECT [1, 2, 3, 4, 5] AS arr, 1 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len);
SELECT [1, 2, 3, 4, 5] AS arr, 3 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len);
SELECT [1, 2 ,3, 4, 5] AS arr, 5 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len);
SELECT ['ab', 'c', 'de', '', 'hi'] AS arr, 1 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len) FORMAT Vertical;
SELECT ['ab', 'c', 'de', '', 'hi'] AS arr, 3 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len) FORMAT Vertical;
SELECT ['ab', 'c', 'de', '', 'hi'] AS arr, 5 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len) FORMAT Vertical;
SELECT ['ab', 'c', 'de', '', 'hi'] AS arr, 1 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len);
SELECT ['ab', 'c', 'de', '', 'hi'] AS arr, 3 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len);
SELECT ['ab', 'c', 'de', '', 'hi'] AS arr, 5 AS len, arrayShingles(arr, len), arrayShingles(materialize(arr), len);
SELECT '-- special cases';
SELECT arrayShingles([toNullable(2), toNullable(1)], 1);

View File

@ -0,0 +1 @@
CREATE TABLE default.a\n(\n `x` Int64\n)\nENGINE = URL(\'https://example.com/\', \'CSV\', headers(\'foo\' = \'bar\'))

View File

@ -0,0 +1,2 @@
create table a (x Int64) engine URL('https://example.com/', CSV, headers('foo' = 'bar'));
show create a;

View File

@ -0,0 +1 @@
10551038310762432828

View File

@ -0,0 +1,8 @@
#!/usr/bin/env bash
# Tags: no-fasttest
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_LOCAL -q "select sum(cityHash64(*)) from file('$CURDIR/data_parquet/02969.zip :: u.parquet') settings max_threads=4, max_read_buffer_size=1000"

Binary file not shown.