mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse
This commit is contained in:
commit
813a09f75a
@ -914,8 +914,17 @@ public:
|
||||
scramble.resize(SCRAMBLE_LENGTH + 1, 0);
|
||||
Poco::RandomInputStream generator;
|
||||
|
||||
for (size_t i = 0; i < SCRAMBLE_LENGTH; i++)
|
||||
/** 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.
|
||||
* https://github.com/mysql/mysql-server/blob/8.0/mysys/crypt_genhash_impl.cc#L427
|
||||
*/
|
||||
for (size_t i = 0; i < SCRAMBLE_LENGTH; ++i)
|
||||
{
|
||||
generator >> scramble[i];
|
||||
scramble[i] &= 0x7f;
|
||||
if (scramble[i] == '\0' || scramble[i] == '$')
|
||||
scramble[i] = scramble[i] + 1;
|
||||
}
|
||||
}
|
||||
|
||||
String getName() override
|
||||
@ -993,8 +1002,13 @@ public:
|
||||
scramble.resize(SCRAMBLE_LENGTH + 1, 0);
|
||||
Poco::RandomInputStream generator;
|
||||
|
||||
for (size_t i = 0; i < SCRAMBLE_LENGTH; i++)
|
||||
for (size_t i = 0; i < SCRAMBLE_LENGTH; ++i)
|
||||
{
|
||||
generator >> scramble[i];
|
||||
scramble[i] &= 0x7f;
|
||||
if (scramble[i] == '\0' || scramble[i] == '$')
|
||||
scramble[i] = scramble[i] + 1;
|
||||
}
|
||||
}
|
||||
|
||||
String getName() override
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
@ -166,7 +167,19 @@ private:
|
||||
{
|
||||
if (func.name == "globalIn" || func.name == "globalNotIn")
|
||||
{
|
||||
data.addExternalStorage(func.arguments->children[1]);
|
||||
ASTPtr & ast = func.arguments->children[1];
|
||||
|
||||
/// Literal can use regular IN
|
||||
if (ast->as<ASTLiteral>())
|
||||
{
|
||||
if (func.name == "globalIn")
|
||||
func.name = "in";
|
||||
else
|
||||
func.name = "notIn";
|
||||
return;
|
||||
}
|
||||
|
||||
data.addExternalStorage(ast);
|
||||
data.has_global_subqueries = true;
|
||||
}
|
||||
}
|
||||
|
@ -202,10 +202,10 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const
|
||||
prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, cond_it);
|
||||
total_size_of_moved_conditions += cond_it->columns_size;
|
||||
|
||||
/// Move all other conditions that depend on the same set of columns.
|
||||
/// Move all other viable conditions that depend on the same set of columns.
|
||||
for (auto jt = where_conditions.begin(); jt != where_conditions.end();)
|
||||
{
|
||||
if (jt->columns_size == cond_it->columns_size && jt->identifiers == cond_it->identifiers)
|
||||
if (jt->viable && jt->columns_size == cond_it->columns_size && jt->identifiers == cond_it->identifiers)
|
||||
prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, jt++);
|
||||
else
|
||||
++jt;
|
||||
|
@ -278,15 +278,29 @@ def test_java_client(server_address, java_container):
|
||||
with open(os.path.join(SCRIPT_DIR, 'clients', 'java', '0.reference')) as fp:
|
||||
reference = fp.read()
|
||||
|
||||
# database not exists exception.
|
||||
code, (stdout, stderr) = java_container.exec_run('java JavaConnectorTest --host {host} --port {port} --user user_with_empty_password --database '
|
||||
'abc'.format(host=server_address, port=server_port), demux=True)
|
||||
assert code == 1
|
||||
|
||||
# empty password passed.
|
||||
code, (stdout, stderr) = java_container.exec_run('java JavaConnectorTest --host {host} --port {port} --user user_with_empty_password --database '
|
||||
'default'.format(host=server_address, port=server_port), demux=True)
|
||||
assert code == 0
|
||||
assert stdout == reference
|
||||
|
||||
# non-empty password passed.
|
||||
code, (stdout, stderr) = java_container.exec_run('java JavaConnectorTest --host {host} --port {port} --user default --password 123 --database '
|
||||
'default'.format(host=server_address, port=server_port), demux=True)
|
||||
assert code == 0
|
||||
assert stdout == reference
|
||||
|
||||
# double-sha1 password passed.
|
||||
code, (stdout, stderr) = java_container.exec_run('java JavaConnectorTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database '
|
||||
'default'.format(host=server_address, port=server_port), demux=True)
|
||||
assert code == 0
|
||||
assert stdout == reference
|
||||
|
||||
|
||||
def test_types(server_address):
|
||||
client = pymysql.connections.Connection(host=server_address, user='default', password='123', database='default', port=server_port)
|
||||
|
7
tests/queries/0_stateless/01115_prewhere_array_join.sql
Normal file
7
tests/queries/0_stateless/01115_prewhere_array_join.sql
Normal file
@ -0,0 +1,7 @@
|
||||
DROP TABLE IF EXISTS prewhere;
|
||||
|
||||
CREATE TABLE prewhere (light UInt8, heavy String) ENGINE = MergeTree ORDER BY tuple();
|
||||
INSERT INTO prewhere SELECT 0, randomPrintableASCII(10000) FROM numbers(10000);
|
||||
SELECT arrayJoin([light]) != 0 AS cond, length(heavy) FROM prewhere WHERE light != 0 AND cond != 0;
|
||||
|
||||
DROP TABLE prewhere;
|
@ -99,3 +99,5 @@ merge()
|
||||
distributed_group_by_no_merge
|
||||
33
|
||||
33
|
||||
GLOBAL IN
|
||||
1
|
||||
|
@ -82,6 +82,10 @@ select count() from merge_dist_01223;
|
||||
select 'distributed_group_by_no_merge';
|
||||
select count() from merge_dist_01223 settings distributed_group_by_no_merge=1;
|
||||
|
||||
-- global in
|
||||
select 'GLOBAL IN';
|
||||
select distinct * from dist_01223 where key global in (select toInt32(1));
|
||||
|
||||
drop table merge_dist_01223;
|
||||
drop table dist_01223;
|
||||
drop table dist_layer_01223;
|
||||
|
@ -0,0 +1,6 @@
|
||||
GLOBAL IN
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
GLOBAL NOT IN
|
10
tests/queries/0_stateless/01226_dist_on_dist_global_in.sql
Normal file
10
tests/queries/0_stateless/01226_dist_on_dist_global_in.sql
Normal file
@ -0,0 +1,10 @@
|
||||
SELECT 'GLOBAL IN';
|
||||
select * from remote('localhost', system.one) where dummy global in (0);
|
||||
select * from remote('localhost', system.one) where toUInt64(dummy) global in numbers(1);
|
||||
select * from remote('localhost', system.one) where dummy global in system.one;
|
||||
select * from remote('localhost', system.one) where dummy global in (select 0);
|
||||
SELECT 'GLOBAL NOT IN';
|
||||
select * from remote('localhost', system.one) where dummy global not in (0);
|
||||
select * from remote('localhost', system.one) where toUInt64(dummy) global not in numbers(1);
|
||||
select * from remote('localhost', system.one) where dummy global not in system.one;
|
||||
select * from remote('localhost', system.one) where dummy global not in (select 0);
|
@ -0,0 +1,3 @@
|
||||
2
|
||||
2
|
||||
2
|
@ -0,0 +1,6 @@
|
||||
-- Test from the issue https://github.com/ClickHouse/ClickHouse/issues/2610
|
||||
drop table if exists data_01227;
|
||||
create table data_01227 (key Int) Engine=MergeTree() order by key;
|
||||
insert into data_01227 select * from numbers(10);
|
||||
select * from remote('127.1', currentDatabase(), data_01227) prewhere key global in (select key from data_01227 prewhere key = 2);
|
||||
select * from cluster('test_cluster_two_shards', currentDatabase(), data_01227) prewhere key global in (select key from data_01227 prewhere key = 2);
|
9
tests/queries/1_stateful/00093_prewhere_array_join.sql
Normal file
9
tests/queries/1_stateful/00093_prewhere_array_join.sql
Normal file
@ -0,0 +1,9 @@
|
||||
SELECT arrayJoin([SearchEngineID]) AS search_engine, URL FROM test.hits WHERE SearchEngineID != 0 AND search_engine != 0 FORMAT Null;
|
||||
|
||||
SELECT
|
||||
arrayJoin([0]) AS browser,
|
||||
arrayJoin([SearchEngineID]) AS search_engine,
|
||||
URL
|
||||
FROM test.hits
|
||||
WHERE 1 AND (SearchEngineID != 0) AND (browser != 0) AND (search_engine != 0)
|
||||
FORMAT Null;
|
@ -1,4 +0,0 @@
|
||||
GLOBAL IN distributed_group_by_no_merge
|
||||
1
|
||||
GLOBAL IN
|
||||
1
|
@ -1,18 +0,0 @@
|
||||
create table if not exists data_01224 (key Int) Engine=Memory();
|
||||
create table if not exists dist_layer_01224 as data_01224 Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01224);
|
||||
create table if not exists dist_01224 as data_01224 Engine=Distributed(test_cluster_two_shards, currentDatabase(), dist_layer_01224);
|
||||
|
||||
select * from dist_01224;
|
||||
insert into data_01224 select * from numbers(3);
|
||||
|
||||
-- "Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery"
|
||||
select 'GLOBAL IN distributed_group_by_no_merge';
|
||||
select distinct * from dist_01224 where key global in (1) settings distributed_group_by_no_merge=1;
|
||||
|
||||
-- requires #9923
|
||||
select 'GLOBAL IN';
|
||||
select distinct * from dist_01224 where key global in (1);
|
||||
|
||||
drop table dist_01224;
|
||||
drop table dist_layer_01224;
|
||||
drop table data_01224;
|
Loading…
Reference in New Issue
Block a user