Merge branch 'master' into s390x_reinterpretas_fix

This commit is contained in:
Suzy Wang 2023-05-11 14:41:57 -04:00 committed by GitHub
commit 70db49cdeb
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
34 changed files with 969 additions and 926 deletions

2
contrib/libfiu vendored

@ -1 +1 @@
Subproject commit 7ae4328050ccad8867a05a37af941886f717b6fc
Subproject commit b85edbde4cf974b1b40d27828a56f0505f4e2ee5

View File

@ -1636,7 +1636,7 @@ For not replicated tables see [non_replicated_deduplication_window](merge-tree-s
### async_insert {#async-insert}
Enables or disables asynchronous inserts. This makes sense only for insertion over HTTP protocol. Note that deduplication isn't working for such inserts.
Enables or disables asynchronous inserts. Note that deduplication isn't working for such inserts.
If enabled, the data is combined into batches before the insertion into tables, so it is possible to do small and frequent insertions into ClickHouse (up to 15000 queries per second) without buffer tables.

View File

@ -1395,8 +1395,8 @@ try
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(config(), socket, listen_host, port);
socket.setReceiveTimeout(config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC));
socket.setSendTimeout(config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC));
socket.setReceiveTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0));
socket.setSendTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0));
return ProtocolServerAdapter(
listen_host,
port_name,
@ -1418,8 +1418,8 @@ try
#if USE_SSL
Poco::Net::SecureServerSocket socket;
auto address = socketBindListen(config(), socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC));
socket.setSendTimeout(config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC));
socket.setReceiveTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0));
socket.setSendTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0));
return ProtocolServerAdapter(
listen_host,
secure_port_name,

View File

@ -214,14 +214,14 @@ int IBridge::main(const std::vector<std::string> & /*args*/)
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, hostname, port, log);
socket.setReceiveTimeout(http_timeout);
socket.setSendTimeout(http_timeout);
socket.setReceiveTimeout(Poco::Timespan(http_timeout, 0));
socket.setSendTimeout(Poco::Timespan(http_timeout, 0));
Poco::ThreadPool server_pool(3, max_server_connections);
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
http_params->setTimeout(http_timeout);
http_params->setKeepAliveTimeout(keep_alive_timeout);
http_params->setTimeout(Poco::Timespan(http_timeout, 0));
http_params->setKeepAliveTimeout(Poco::Timespan(keep_alive_timeout, 0));
auto shared_context = Context::createShared();
auto context = Context::createGlobal(shared_context.get());

View File

@ -386,8 +386,6 @@ protected:
FallbackSearcher fallback_searcher;
public:
using Searcher = FallbackSearcher;
/** haystack_size_hint - the expected total size of the haystack for `search` calls. Optional (zero means unspecified).
* If you specify it small enough, the fallback algorithm will be used,
* since it is considered that it's useless to waste time initializing the hash table.
@ -729,7 +727,7 @@ public:
using Volnitsky = VolnitskyBase<true, true, ASCIICaseSensitiveStringSearcher>;
using VolnitskyUTF8 = VolnitskyBase<true, false, ASCIICaseSensitiveStringSearcher>; /// exactly same as Volnitsky
using VolnitskyUTF8 = VolnitskyBase<true, false, UTF8CaseSensitiveStringSearcher>;
using VolnitskyCaseInsensitive = VolnitskyBase<false, true, ASCIICaseInsensitiveStringSearcher>; /// ignores non-ASCII bytes
using VolnitskyCaseInsensitiveUTF8 = VolnitskyBase<false, false, UTF8CaseInsensitiveStringSearcher>;
@ -737,7 +735,7 @@ using VolnitskyCaseSensitiveToken = VolnitskyBase<true, true, ASCIICaseSensitive
using VolnitskyCaseInsensitiveToken = VolnitskyBase<false, true, ASCIICaseInsensitiveTokenSearcher>;
using MultiVolnitsky = MultiVolnitskyBase<true, true, ASCIICaseSensitiveStringSearcher>;
using MultiVolnitskyUTF8 = MultiVolnitskyBase<true, false, ASCIICaseSensitiveStringSearcher>;
using MultiVolnitskyUTF8 = MultiVolnitskyBase<true, false, UTF8CaseSensitiveStringSearcher>;
using MultiVolnitskyCaseInsensitive = MultiVolnitskyBase<false, true, ASCIICaseInsensitiveStringSearcher>;
using MultiVolnitskyCaseInsensitiveUTF8 = MultiVolnitskyBase<false, false, UTF8CaseInsensitiveStringSearcher>;

View File

@ -17,18 +17,18 @@ namespace Format
{
using IndexPositions = PODArrayWithStackMemory<UInt64, 64>;
static inline void parseNumber(const String & description, UInt64 l, UInt64 r, UInt64 & res, UInt64 argument_number)
static inline UInt64 parseNumber(const String & description, UInt64 l, UInt64 r, UInt64 argument_number)
{
res = 0;
UInt64 res = 0;
for (UInt64 pos = l; pos < r; ++pos)
{
if (!isNumericASCII(description[pos]))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not a number in curly braces at position {}", std::to_string(pos));
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not a number in curly braces at position {}", pos);
res = res * 10 + description[pos] - '0';
if (res >= argument_number)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Too big number for arguments, must be at most {}",
argument_number - 1);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Too big number for arguments, must be at most {}", argument_number - 1);
}
return res;
}
static inline void init(
@ -132,8 +132,7 @@ namespace Format
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot switch from automatic field numbering to manual field specification");
is_plain_numbering = false;
UInt64 arg;
parseNumber(pattern, last_open, i, arg, argument_number);
UInt64 arg = parseNumber(pattern, last_open, i, argument_number);
if (arg >= argument_number)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument is too big for formatting. Note that indexing starts from zero");

View File

@ -31,7 +31,7 @@ namespace Authentication
static const size_t SCRAMBLE_LENGTH = 20;
/** Generate a random string using ASCII characters but avoid separator character,
* produce pseudo random numbers between with about 7 bit worth of entropty between 1-127.
* produce pseudo random numbers between with about 7 bit worth of entropy between 1-127.
* https://github.com/mysql/mysql-server/blob/8.0/mysys/crypt_genhash_impl.cc#L427
*/
static String generateScramble()

View File

@ -61,26 +61,26 @@ public:
offsets_to.resize(input_rows_count);
const IColumn & length_column = *arguments[0].column;
size_t summary_utf8_len = 0;
const IColumn & col_length = *arguments[0].column;
size_t total_codepoints = 0;
for (size_t row_num = 0; row_num < input_rows_count; ++row_num)
{
size_t utf8_len = length_column.getUInt(row_num);
summary_utf8_len += utf8_len;
size_t codepoints = col_length.getUInt(row_num);
total_codepoints += codepoints;
}
/* As we generate only assigned planes, the mathematical expectation of the number of bytes
* per generated code point ~= 3.85. So, reserving for coefficient 4 will not be an overhead
*/
if (summary_utf8_len > (1 << 29))
if (total_codepoints > (1 << 29))
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size in function {}", getName());
size_t size_in_bytes_with_margin = summary_utf8_len * 4 + input_rows_count;
data_to.resize(size_in_bytes_with_margin);
pcg64_fast rng(randomSeed()); // TODO It is inefficient. We should use SIMD PRNG instead.
size_t max_byte_size = total_codepoints * 4 + input_rows_count;
data_to.resize(max_byte_size);
const auto generate_code_point = [](UInt32 rand) -> UInt32 {
const auto generate_code_point = [](UInt32 rand)
{
/// We want to generate number in [0x0, 0x70000) and shift it if need
/// Generate highest byte in [0, 6]
@ -104,43 +104,41 @@ public:
return code_point;
};
pcg64_fast rng(randomSeed());
IColumn::Offset offset = 0;
for (size_t row_num = 0; row_num < input_rows_count; ++row_num)
{
size_t utf8_len = length_column.getUInt(row_num);
size_t codepoints = col_length.getUInt(row_num);
auto * pos = data_to.data() + offset;
size_t last_writen_bytes = 0;
size_t i = 0;
for (; i < utf8_len; i += 2)
for (size_t i = 0; i < codepoints; i +=2)
{
UInt64 rand = rng();
UInt64 rand = rng(); /// that's the bottleneck
UInt32 code_point1 = generate_code_point(static_cast<UInt32>(rand));
UInt32 code_point2 = generate_code_point(static_cast<UInt32>(rand >> 32u));
/// We have padding in column buffers that we can overwrite.
size_t length1 = UTF8::convertCodePointToUTF8(code_point1, pos, sizeof(int));
assert(length1 <= 4);
pos += length1;
size_t bytes1 = UTF8::convertCodePointToUTF8(code_point1, pos, 4);
chassert(bytes1 <= 4);
pos += bytes1;
size_t length2 = UTF8::convertCodePointToUTF8(code_point2, pos, sizeof(int));
assert(length2 <= 4);
last_writen_bytes = length2;
pos += last_writen_bytes;
}
offset = pos - data_to.data() + 1;
if (i > utf8_len)
{
offset -= last_writen_bytes;
if (i + 1 != codepoints)
{
UInt32 code_point2 = generate_code_point(static_cast<UInt32>(rand >> 32u));
size_t bytes2 = UTF8::convertCodePointToUTF8(code_point2, pos, 4);
chassert(bytes2 <= 4);
pos += bytes2;
}
}
*pos = 0;
++pos;
offset = pos - data_to.data();
offsets_to[row_num] = offset;
}
/// Put zero bytes in between.
auto * pos = data_to.data();
for (size_t row_num = 0; row_num < input_rows_count; ++row_num)
pos[offsets_to[row_num] - 1] = 0;
data_to.resize(offset);
return col_to;
}

View File

@ -2140,8 +2140,12 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
}
auto conjunction = getConjunctionNodes(predicate, allowed_nodes);
if (conjunction.rejected.size() == 1 && WhichDataType{removeNullable(conjunction.rejected.front()->result_type)}.isFloat())
if (conjunction.rejected.size() == 1 && !conjunction.rejected.front()->result_type->equals(*predicate->result_type)
&& conjunction.allowed.front()->type == ActionType::COLUMN)
{
// No further optimization can be done
return nullptr;
}
auto actions = cloneActionsForConjunction(conjunction.allowed, all_inputs);
if (!actions)
@ -2191,55 +2195,26 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
else
{
/// Predicate is conjunction, where both allowed and rejected sets are not empty.
/// Replace this node to conjunction of rejected predicates.
NodeRawConstPtrs new_children = std::move(conjunction.rejected);
if (new_children.size() == 1)
if (new_children.size() == 1 && new_children.front()->result_type->equals(*predicate->result_type))
{
/// Rejected set has only one predicate.
if (new_children.front()->result_type->equals(*predicate->result_type))
{
/// If it's type is same, just add alias.
Node node;
node.type = ActionType::ALIAS;
node.result_name = predicate->result_name;
node.result_type = predicate->result_type;
node.children.swap(new_children);
*predicate = std::move(node);
}
else if (!WhichDataType{removeNullable(new_children.front()->result_type)}.isFloat())
{
/// If type is different, cast column.
/// This case is possible, cause AND can use any numeric type as argument.
/// But casting floats to UInt8 or Bool produces different results.
/// so we can't apply this optimization to them.
Node node;
node.type = ActionType::COLUMN;
node.result_name = predicate->result_type->getName();
node.column = DataTypeString().createColumnConst(0, node.result_name);
node.result_type = std::make_shared<DataTypeString>();
const auto * right_arg = &nodes.emplace_back(std::move(node));
const auto * left_arg = new_children.front();
predicate->children = {left_arg, right_arg};
auto arguments = prepareFunctionArguments(predicate->children);
FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver<CastType::nonAccurate>::createImpl();
predicate->function_base = func_builder_cast->build(arguments);
predicate->function = predicate->function_base->prepare(arguments);
}
/// Rejected set has only one predicate. And the type is the same as the result_type.
/// Just add alias.
Node node;
node.type = ActionType::ALIAS;
node.result_name = predicate->result_name;
node.result_type = predicate->result_type;
node.children.swap(new_children);
*predicate = std::move(node);
}
else
{
/// Predicate is function AND, which still have more then one argument.
/// Or there is only one argument that is a float and we can't just
/// remove the AND.
/// Predicate is function AND, which still have more then one argument
/// or it has one argument of the wrong type.
/// Just update children and rebuild it.
predicate->children.swap(new_children);
if (WhichDataType{removeNullable(predicate->children.front()->result_type)}.isFloat())
if (new_children.size() == 1)
{
Node node;
node.type = ActionType::COLUMN;
@ -2247,8 +2222,9 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
node.column = DataTypeUInt8().createColumnConst(0, 1u);
node.result_type = std::make_shared<DataTypeUInt8>();
const auto * const_col = &nodes.emplace_back(std::move(node));
predicate->children.emplace_back(const_col);
new_children.emplace_back(const_col);
}
predicate->children.swap(new_children);
auto arguments = prepareFunctionArguments(predicate->children);
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());

View File

@ -79,7 +79,13 @@ namespace JoinStuff
{
assert(flags[nullptr].size() <= size);
need_flags = true;
flags[nullptr] = std::vector<std::atomic_bool>(size);
// For one disjunct clause case, we don't need to reinit each time we call addJoinedBlock.
// and there is no value inserted in this JoinUsedFlags before addJoinedBlock finish.
// So we reinit only when the hash table is rehashed to a larger size.
if (flags.empty() || flags[nullptr].size() < size) [[unlikely]]
{
flags[nullptr] = std::vector<std::atomic_bool>(size);
}
}
}

View File

@ -293,7 +293,7 @@ Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length)
if (handshake_length == Coordination::CLIENT_HANDSHAKE_LENGTH_WITH_READONLY)
Coordination::read(readonly, *in);
return Poco::Timespan(0, timeout_ms * 1000);
return Poco::Timespan(timeout_ms * 1000);
}
@ -342,8 +342,8 @@ void KeeperTCPHandler::runImpl()
int32_t handshake_length = header;
auto client_timeout = receiveHandshake(handshake_length);
if (client_timeout == 0)
client_timeout = Coordination::DEFAULT_SESSION_TIMEOUT_MS;
if (client_timeout.totalMilliseconds() == 0)
client_timeout = Poco::Timespan(Coordination::DEFAULT_SESSION_TIMEOUT_MS * Poco::Timespan::MILLISECONDS);
session_timeout = std::max(client_timeout, min_session_timeout);
session_timeout = std::min(session_timeout, max_session_timeout);
}

View File

@ -173,8 +173,8 @@ private:
/// Connection settings, which are extracted from a context.
bool send_exception_with_stack_trace = true;
Poco::Timespan send_timeout = DBMS_DEFAULT_SEND_TIMEOUT_SEC;
Poco::Timespan receive_timeout = DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC;
Poco::Timespan send_timeout = Poco::Timespan(DBMS_DEFAULT_SEND_TIMEOUT_SEC, 0);
Poco::Timespan receive_timeout = Poco::Timespan(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, 0);
UInt64 poll_interval = DBMS_DEFAULT_POLL_INTERVAL;
UInt64 idle_connection_timeout = 3600;
UInt64 interactive_delay = 100000;

View File

@ -724,7 +724,7 @@ void StorageRabbitMQ::read(
uint64_t max_execution_time_ms = rabbitmq_settings->rabbitmq_flush_interval_ms.changed
? rabbitmq_settings->rabbitmq_flush_interval_ms
: static_cast<UInt64>(Poco::Timespan(getContext()->getSettingsRef().stream_flush_interval_ms).milliseconds());
: static_cast<UInt64>(getContext()->getSettingsRef().stream_flush_interval_ms.totalMilliseconds());
for (size_t i = 0; i < num_created_consumers; ++i)
{
@ -1053,7 +1053,7 @@ bool StorageRabbitMQ::tryStreamToViews()
uint64_t max_execution_time_ms = rabbitmq_settings->rabbitmq_flush_interval_ms.changed
? rabbitmq_settings->rabbitmq_flush_interval_ms
: static_cast<UInt64>(Poco::Timespan(getContext()->getSettingsRef().stream_flush_interval_ms).milliseconds());
: static_cast<UInt64>(getContext()->getSettingsRef().stream_flush_interval_ms.totalMilliseconds());
for (size_t i = 0; i < num_created_consumers; ++i)
{

View File

@ -1986,9 +1986,9 @@ void registerStorageDistributed(StorageFactory & factory)
if (!distributed_settings.monitor_split_batch_on_failure.changed)
distributed_settings.monitor_split_batch_on_failure = context->getSettingsRef().distributed_directory_monitor_split_batch_on_failure;
if (!distributed_settings.monitor_sleep_time_ms.changed)
distributed_settings.monitor_sleep_time_ms = Poco::Timespan(context->getSettingsRef().distributed_directory_monitor_sleep_time_ms);
distributed_settings.monitor_sleep_time_ms = context->getSettingsRef().distributed_directory_monitor_sleep_time_ms;
if (!distributed_settings.monitor_max_sleep_time_ms.changed)
distributed_settings.monitor_max_sleep_time_ms = Poco::Timespan(context->getSettingsRef().distributed_directory_monitor_max_sleep_time_ms);
distributed_settings.monitor_max_sleep_time_ms = context->getSettingsRef().distributed_directory_monitor_max_sleep_time_ms;
return std::make_shared<StorageDistributed>(
args.table_id,

View File

@ -1 +1 @@
requests
requests<2.30

View File

@ -1,3 +1,3 @@
requests
requests<2.30
PyJWT
cryptography==37.0.4
cryptography<38

View File

@ -1,3 +1,3 @@
requests
requests<2.30
PyJWT
cryptography==37.0.4
cryptography<38

View File

@ -1,3 +1,3 @@
requests
requests<2.30
PyJWT
cryptography==37.0.4
cryptography<38

View File

@ -5,7 +5,7 @@ WORKDIR=$(dirname "$0")
WORKDIR=$(readlink -f "${WORKDIR}")
cd "$WORKDIR"
PY_VERSION=3.9
PY_VERSION=3.10
PY_EXEC="python${PY_VERSION}"
DOCKER_IMAGE="python:${PY_VERSION}-slim"
LAMBDA_NAME=$(basename "$WORKDIR")

View File

@ -1 +1 @@
requests
requests<2.30

View File

@ -1,3 +1,3 @@
requests
requests<2.30
PyJWT
cryptography==37.0.4
cryptography<38

View File

@ -12,12 +12,11 @@ import boto3 # type: ignore
SUSPICIOUS_CHANGED_FILES_NUMBER = 200
SUSPICIOUS_PATTERNS = [
"tests/ci/*",
"docs/tools/*",
".github/*",
"utils/release/*",
"docker/*",
"release",
"docs/tools/*",
"packages/*",
"tests/ci/*",
]
# Number of retries for API calls.
@ -313,11 +312,12 @@ def check_suspicious_changed_files(changed_files):
)
return True
print("No changed files match suspicious patterns, run will be approved")
print("No changed files match suspicious patterns, run could be approved")
return False
def approve_run(workflow_description: WorkflowDescription, token: str) -> None:
print("Approving run")
url = f"{workflow_description.api_url}/approve"
_exec_post_with_retry(url, token)
@ -480,6 +480,11 @@ def main(event):
approve_run(workflow_description, token)
return
labels = {label["name"] for label in pull_request["labels"]}
if "can be tested" not in labels:
print("Label 'can be tested' is required for untrusted users")
return
changed_files = get_changed_files_for_pull_request(pull_request, token)
print(f"Totally have {len(changed_files)} changed files in PR:", changed_files)
if check_suspicious_changed_files(changed_files):

View File

@ -1,3 +1,3 @@
requests
requests<2.30
PyJWT
cryptography==37.0.4
cryptography<38

View File

@ -251,14 +251,20 @@ def send_event_workflow_job(workflow_job: WorkflowJob) -> None:
clickhouse_client.insert_event_into(**kwargs)
def handler(event: dict, _: Any) -> dict:
def handler(event: dict, context: Any) -> dict:
if event["isBase64Encoded"]:
event_data = json.loads(b64decode(event["body"]))
else:
event_data = json.loads(event["body"])
repo = event_data["repository"]
wf_job = event_data["workflow_job"]
try:
wf_job = event_data["workflow_job"]
except KeyError:
logging.error("The event does not contain valid workflow_jobs data")
logging.error("The event data: %s", event)
logging.error("The context data: %s", context)
workflow_job = WorkflowJob(
wf_job["id"],
wf_job["run_id"],

View File

@ -1 +1 @@
requests
requests<2.30

View File

@ -0,0 +1,6 @@
<test>
<create_query>CREATE TABLE test_join_used_flags (i64 Int64, i32 Int32) ENGINE = Memory</create_query>
<fill_query>INSERT INTO test_join_used_flags SELECT number AS i64, rand32() AS i32 FROM numbers(20000000)</fill_query>
<query>SELECT l.i64, r.i64, l.i32, r.i32 FROM test_join_used_flags l RIGHT JOIN test_join_used_flags r USING i64 format Null</query>
<drop_query>DROP TABLE IF EXISTS test_join_used_flags</drop_query>
</test>

View File

@ -2,3 +2,4 @@
String
1
99

View File

@ -3,3 +3,4 @@ SELECT lengthUTF8(randomStringUTF8(100));
SELECT toTypeName(randomStringUTF8(10));
SELECT isValidUTF8(randomStringUTF8(100000));
SELECT randomStringUTF8(0);
SELECT lengthUTF8(lowerUTF8(randomStringUTF8(99))); -- bug #49672: msan assert

View File

@ -53,7 +53,7 @@ Filter column: notEquals(y, 0)
9 10
> one condition of filter should be pushed down after aggregating, other condition is casted
Filter column
FUNCTION _CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4))
FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2
Aggregating
Filter column: notEquals(y, 0)
0 1

View File

@ -56,7 +56,7 @@ $CLICKHOUSE_CLIENT -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION _CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4))"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 1, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4)) UInt8 : 2"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y

View File

@ -68,13 +68,13 @@ $CLICKHOUSE_CLIENT -n --query="
insert into user_agents select ua from input('ua String') FORMAT LineAsString" < $CURDIR/data_ua_parser/useragents.txt
$CLICKHOUSE_CLIENT -n --query="
select device,
select ua, device,
concat(tupleElement(browser, 1), ' ', tupleElement(browser, 2), '.', tupleElement(browser, 3)) as browser ,
concat(tupleElement(os, 1), ' ', tupleElement(os, 2), '.', tupleElement(os, 3), '.', tupleElement(os, 4)) as os
from (
select dictGet('regexp_os', ('os_replacement', 'os_v1_replacement', 'os_v2_replacement', 'os_v3_replacement'), ua) os,
select ua, dictGet('regexp_os', ('os_replacement', 'os_v1_replacement', 'os_v2_replacement', 'os_v3_replacement'), ua) os,
dictGet('regexp_browser', ('family_replacement', 'v1_replacement', 'v2_replacement'), ua) as browser,
dictGet('regexp_device', 'device_replacement', ua) device from user_agents);
dictGet('regexp_device', 'device_replacement', ua) device from user_agents) order by ua;
"
$CLICKHOUSE_CLIENT -n --query="

View File

@ -0,0 +1,5 @@
=
1554690688
=
1554690688
=

View File

@ -0,0 +1,42 @@
DROP TABLE IF EXISTS t1;
CREATE TABLE t1 (c0 Int32, PRIMARY KEY (c0)) ENGINE=MergeTree;
INSERT INTO t1 VALUES (1554690688);
select '=';
SELECT MIN(t1.c0)
FROM t1
GROUP BY
(-sign(cos(t1.c0))) * (-max2(t1.c0, t1.c0 / t1.c0)),
t1.c0 * t1.c0,
sign(-exp(-t1.c0))
HAVING -(-(MIN(t1.c0) + MIN(t1.c0))) AND (pow('{b' > '-657301241', log(-1004522121)) IS NOT NULL)
UNION ALL
SELECT MIN(t1.c0)
FROM t1
GROUP BY
(-sign(cos(t1.c0))) * (-max2(t1.c0, t1.c0 / t1.c0)),
t1.c0 * t1.c0,
sign(-exp(-t1.c0))
HAVING NOT (-(-(MIN(t1.c0) + MIN(t1.c0))) AND (pow('{b' > '-657301241', log(-1004522121)) IS NOT NULL))
UNION ALL
SELECT MIN(t1.c0)
FROM t1
GROUP BY
(-sign(cos(t1.c0))) * (-max2(t1.c0, t1.c0 / t1.c0)),
t1.c0 * t1.c0,
sign(-exp(-t1.c0))
HAVING (-(-(MIN(t1.c0) + MIN(t1.c0))) AND (pow('{b' > '-657301241', log(-1004522121)) IS NOT NULL)) IS NULL
SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0;
select '=';
SELECT MIN(t1.c0)
FROM t1
GROUP BY t1.c0
HAVING and(MIN(t1.c0) + MIN(t1.c0), 1)
SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0;
select '=';
DROP TABLE IF EXISTS t1;