Merge branch 'master' into fix-hash-empty-tuple

This commit is contained in:
Alexey Milovidov 2024-07-17 03:50:17 +02:00
commit 083fd34552
14 changed files with 38 additions and 14 deletions

View File

@ -11,7 +11,7 @@ This is for the case when you have Linux machine and want to use it to build `cl
The cross-build for RISC-V 64 is based on the [Build instructions](../development/build.md), follow them first.
## Install Clang-16
## Install Clang-18
Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup or do
```

View File

@ -12,6 +12,7 @@
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
namespace DB
{
@ -616,6 +617,7 @@ private:
bool is_any_nullable = false;
Tuple args;
args.reserve(equals_functions.size());
DataTypes tuple_element_types;
/// first we create tuple from RHS of equals functions
for (const auto & equals : equals_functions)
{
@ -628,16 +630,18 @@ private:
if (const auto * rhs_literal = equals_arguments[1]->as<ConstantNode>())
{
args.push_back(rhs_literal->getValue());
tuple_element_types.push_back(rhs_literal->getResultType());
}
else
{
const auto * lhs_literal = equals_arguments[0]->as<ConstantNode>();
assert(lhs_literal);
args.push_back(lhs_literal->getValue());
tuple_element_types.push_back(lhs_literal->getResultType());
}
}
auto rhs_node = std::make_shared<ConstantNode>(std::move(args));
auto rhs_node = std::make_shared<ConstantNode>(std::move(args), std::make_shared<DataTypeTuple>(std::move(tuple_element_types)));
auto in_function = std::make_shared<FunctionNode>("in");

View File

@ -55,9 +55,9 @@ namespace
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
settings.auth_settings.region,
context->getRemoteHostFilter(),
static_cast<unsigned>(global_settings.s3_max_redirects),
static_cast<unsigned>(global_settings.s3_retry_attempts),
global_settings.enable_s3_requests_logging,
static_cast<unsigned>(local_settings.s3_max_redirects),
static_cast<unsigned>(local_settings.backup_restore_s3_retry_attempts),
local_settings.enable_s3_requests_logging,
/* for_disk_s3 = */ false,
request_settings.get_request_throttler,
request_settings.put_request_throttler,

View File

@ -80,6 +80,7 @@
#include <Common/config_version.h>
#include "config.h"
namespace fs = std::filesystem;
using namespace std::literals;
@ -2565,12 +2566,12 @@ void ClientBase::runInteractive()
word_break_characters,
highlight_callback);
#else
(void)word_break_characters;
LineReader lr(
history_file,
getClientConfiguration().has("multiline"),
query_extenders,
query_delimiters,
word_break_characters);
query_delimiters);
#endif
static const std::initializer_list<std::pair<String, String>> backslash_aliases =

View File

@ -46,8 +46,7 @@ public:
Patterns delimiters,
std::istream & input_stream_ = std::cin,
std::ostream & output_stream_ = std::cout,
int in_fd_ = STDIN_FILENO
);
int in_fd_ = STDIN_FILENO);
virtual ~LineReader() = default;

View File

@ -362,6 +362,9 @@ ReplxxLineReader::ReplxxLineReader(
rx.bind_key(Replxx::KEY::control('N'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_NEXT, code); });
rx.bind_key(Replxx::KEY::control('P'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_PREVIOUS, code); });
/// We don't want the default, "suspend" behavior, it confuses people.
rx.bind_key_internal(replxx::Replxx::KEY::control('Z'), "insert_character");
auto commit_action = [this](char32_t code)
{
/// If we allow multiline and there is already something in the input, start a newline.

View File

@ -502,6 +502,7 @@ class IColumn;
M(UInt64, backup_restore_keeper_value_max_size, 1048576, "Maximum size of data of a [Zoo]Keeper's node during backup", 0) \
M(UInt64, backup_restore_batch_size_for_keeper_multiread, 10000, "Maximum size of batch for multiread request to [Zoo]Keeper during backup or restore", 0) \
M(UInt64, backup_restore_batch_size_for_keeper_multi, 1000, "Maximum size of batch for multi request to [Zoo]Keeper during backup or restore", 0) \
M(UInt64, backup_restore_s3_retry_attempts, 1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore.", 0) \
M(UInt64, max_backup_bandwidth, 0, "The maximum read speed in bytes per second for particular backup on server. Zero means unlimited.", 0) \
\
M(Bool, log_profile_events, true, "Log query performance statistics into the query_log, query_thread_log and query_views_log.", 0) \

View File

@ -77,6 +77,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"},
{"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"},
{"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."},
{"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."},
{"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."},
{"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}
}},

View File

@ -162,7 +162,7 @@ public:
class RetryStrategy : public Aws::Client::RetryStrategy
{
public:
explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 90000);
explicit RetryStrategy(uint32_t maxRetries_ = 10, uint32_t scaleFactor_ = 25, uint32_t maxDelayMs_ = 5000);
/// NOLINTNEXTLINE(google-runtime-int)
bool ShouldRetry(const Aws::Client::AWSError<Aws::Client::CoreErrors>& error, long attemptedRetries) const override;

View File

@ -58,14 +58,14 @@ def decompress_fast(archive_path: Path, result_path: Optional[Path] = None) -> N
archive_path,
result_path,
)
program_part = "--use-compress-program='zstd --threads=0'"
program_part = "--use-compress-program='zstd --threads=0 -d'"
elif PIGZ.exists():
logging.info(
"pigz found, will compress and decompress faster ('%s' -> '%s')",
archive_path,
result_path,
)
program_part = "--use-compress-program='pigz'"
program_part = "--use-compress-program='pigz -d'"
else:
program_part = "-z"
logging.info(

View File

@ -3386,13 +3386,13 @@ def parse_args():
parser.add_argument(
"--replace-replicated-with-shared",
action="store_true",
default=os.environ.get("USE_META_IN_KEEPER_FOR_MERGE_TREE", False),
default=os.environ.get("REPLACE_RMT_WITH_SMT", False),
help="Replace ReplicatedMergeTree engine with SharedMergeTree",
)
parser.add_argument(
"--replace-non-replicated-with-shared",
action="store_true",
default=False,
default=os.environ.get("REPLACE_MT_WITH_SMT", False),
help="Replace ordinary MergeTree engine with SharedMergeTree",
)

View File

@ -2,6 +2,7 @@
<profiles>
<default>
<s3_retry_attempts>5</s3_retry_attempts>
<backup_restore_s3_retry_attempts>5</backup_restore_s3_retry_attempts>
</default>
</profiles>
<users>

View File

@ -0,0 +1,2 @@
2020-01-01
2020-01-02

View File

@ -0,0 +1,12 @@
SET allow_experimental_analyzer=1;
CREATE TABLE foo (i Date) ENGINE MergeTree ORDER BY i;
INSERT INTO foo VALUES ('2020-01-01');
INSERT INTO foo VALUES ('2020-01-02');
SET optimize_min_equality_disjunction_chain_length = 3;
SELECT *
FROM foo
WHERE (foo.i = parseDateTimeBestEffort('2020-01-01'))
OR (foo.i = parseDateTimeBestEffort('2020-01-02'))
OR (foo.i = parseDateTimeBestEffort('2020-01-03'))
ORDER BY foo.i ASC