mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge branch 'master' into fix-hash-empty-tuple
This commit is contained in:
commit
083fd34552
@ -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
|
||||
```
|
||||
|
@ -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");
|
||||
|
||||
|
@ -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,
|
||||
|
@ -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 =
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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) \
|
||||
|
@ -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."}
|
||||
}},
|
||||
|
@ -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;
|
||||
|
@ -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(
|
||||
|
@ -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",
|
||||
)
|
||||
|
||||
|
@ -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>
|
||||
|
@ -0,0 +1,2 @@
|
||||
2020-01-01
|
||||
2020-01-02
|
@ -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
|
Loading…
Reference in New Issue
Block a user