mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
add test & use isRepresentedByNumber
This commit is contained in:
parent
dc61a73a6e
commit
263de501f5
@ -1138,8 +1138,7 @@ struct ToIntMonotonicity
|
|||||||
|
|
||||||
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
|
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
|
||||||
{
|
{
|
||||||
/// If type is string, the function is no monotonic
|
if (!type.isValueRepresentedByNumber())
|
||||||
if (WhichDataType(type).isString())
|
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
size_t size_of_type = type.getSizeOfValueInMemory();
|
size_t size_of_type = type.getSizeOfValueInMemory();
|
||||||
@ -1157,14 +1156,10 @@ struct ToIntMonotonicity
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// If type is same, too. (Enum has separate case, because it is different data type)
|
/// If type is same, too. (Enum has separate case, because it is different data type)
|
||||||
if (checkAndGetDataType<DataTypeNumber<T>>(&type) ||
|
if (checkAndGetDataType<DataTypeNumberBase<T>>(&type) ||
|
||||||
checkAndGetDataType<DataTypeEnum<T>>(&type))
|
checkAndGetDataType<DataTypeEnum<T>>(&type))
|
||||||
return { true, true, true };
|
return { true, true, true };
|
||||||
|
|
||||||
/// In other cases, if range is unbounded, we don't know, whether function is monotonic or not.
|
|
||||||
if (left.isNull() || right.isNull())
|
|
||||||
return {};
|
|
||||||
|
|
||||||
/// If converting from float, for monotonicity, arguments must fit in range of result type.
|
/// If converting from float, for monotonicity, arguments must fit in range of result type.
|
||||||
if (WhichDataType(type).isFloat())
|
if (WhichDataType(type).isFloat())
|
||||||
{
|
{
|
||||||
|
@ -1,2 +1 @@
|
|||||||
1
|
1
|
||||||
1
|
|
||||||
|
@ -1,10 +1,4 @@
|
|||||||
DROP TABLE IF EXISTS test.number_test_table;
|
drop table if exists test.table;
|
||||||
DROP TABLE IF EXISTS test.string_test_table;
|
create table test.table (val Int32) engine = MergeTree order by val;
|
||||||
CREATE TABLE test.number_test_table (val Int32) ENGINE = MergeTree ORDER BY val;
|
insert into test.table values (-2), (0), (2);
|
||||||
CREATE TABLE test.string_test_table (val String) ENGINE = MergeTree ORDER BY val;
|
select count() from test.table where toUInt64(val) == 0;
|
||||||
INSERT INTO test.number_test_table VALUES (-2), (0), (2);
|
|
||||||
INSERT INTO test.string_test_table VALUES ('0'), ('2');
|
|
||||||
SELECT count() FROM test.number_test_table WHERE toUInt64(val) == 0;
|
|
||||||
SELECT count() FROM test.string_test_table WHERE toUInt64(val) == 0;
|
|
||||||
DROP TABLE IF EXISTS test.number_test_table;
|
|
||||||
DROP TABLE IF EXISTS test.string_test_table;
|
|
||||||
|
@ -0,0 +1,26 @@
|
|||||||
|
no monotonic int case: String -> UInt64
|
||||||
|
no monotonic int case: FixedString -> UInt64
|
||||||
|
monotonic int case: Int32 -> Int64
|
||||||
|
monotonic int case: Int32 -> UInt64
|
||||||
|
monotonic int case: Int32 -> Int32
|
||||||
|
monotonic int case: Int32 -> UInt32
|
||||||
|
monotonic int case: Int32 -> Int16
|
||||||
|
monotonic int case: Int32 -> UInt16
|
||||||
|
monotonic int case: UInt32 -> Int64
|
||||||
|
monotonic int case: UInt32 -> UInt64
|
||||||
|
monotonic int case: UInt32 -> Int32
|
||||||
|
monotonic int case: UInt32 -> UInt32
|
||||||
|
monotonic int case: UInt32 -> Int16
|
||||||
|
monotonic int case: UInt32 -> UInt16
|
||||||
|
monotonic int case: Enum16 -> Int32
|
||||||
|
monotonic int case: Enum16 -> UInt32
|
||||||
|
monotonic int case: Enum16 -> Int16
|
||||||
|
monotonic int case: Enum16 -> UInt16
|
||||||
|
monotonic int case: Enum16 -> Int8
|
||||||
|
monotonic int case: Enum16 -> UInt8
|
||||||
|
monotonic int case: Date -> Int32
|
||||||
|
monotonic int case: Date -> UInt32
|
||||||
|
monotonic int case: Date -> Int16
|
||||||
|
monotonic int case: Date -> UInt16
|
||||||
|
monotonic int case: Date -> Int8
|
||||||
|
monotonic int case: Date -> UInt8
|
85
dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh
Executable file
85
dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh
Executable file
@ -0,0 +1,85 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
#--------------------------------------------
|
||||||
|
# Description of test result:
|
||||||
|
# Test the correctness of the optimization
|
||||||
|
# by asserting read marks in the log.
|
||||||
|
# Relation of read marks and optimization:
|
||||||
|
# read marks =
|
||||||
|
# the number of monotonic marks filtered through predicates
|
||||||
|
# + no monotonic marks count
|
||||||
|
#--------------------------------------------
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
. $CURDIR/../shell_config.sh
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES;"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.string_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.fixed_string_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.signed_integer_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.unsigned_integer_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.enum_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.date_test_table;"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.string_test_table (val String) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.fixed_string_test_table (val FixedString(1)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.signed_integer_test_table (val Int32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.unsigned_integer_test_table (val UInt32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.enum_test_table (val Enum16('hello' = 1, 'world' = 2, 'yandex' = 256, 'clickhouse' = 257)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.date_test_table (val Date) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;"
|
||||||
|
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="INSERT INTO test.string_test_table VALUES ('0'), ('2'), ('2');"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="INSERT INTO test.fixed_string_test_table VALUES ('0'), ('2'), ('2');"
|
||||||
|
# 131072 -> 17 bit is 1
|
||||||
|
${CLICKHOUSE_CLIENT} --query="INSERT INTO test.signed_integer_test_table VALUES (-2), (0), (2), (2), (131072), (131073), (131073);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="INSERT INTO test.unsigned_integer_test_table VALUES (0), (2), (2), (131072), (131073), (131073);"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="INSERT INTO test.enum_test_table VALUES ('hello'), ('world'), ('world'), ('yandex'), ('clickhouse'), ('clickhouse');"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="INSERT INTO test.date_test_table VALUES (1), (2), (2), (256), (257), (257);"
|
||||||
|
|
||||||
|
export CLICKHOUSE_CLIENT=`echo ${CLICKHOUSE_CLIENT} |sed 's/'"${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/debug/g'`
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: String -> UInt64"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: FixedString -> UInt64"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int64"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt64"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int32"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt32"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> Int16"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> UInt16"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int64"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt64"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int32"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt32"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> Int16"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> UInt16"
|
||||||
|
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int32"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt32"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int16"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt16"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> Int8"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> UInt8"
|
||||||
|
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int32"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt32"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int16"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt16"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> Int8"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> UInt8"
|
||||||
|
|
||||||
|
export CLICKHOUSE_CLIENT=`echo ${CLICKHOUSE_CLIENT} |sed 's/debug/'"${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g'`
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.string_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.fixed_string_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.signed_integer_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.unsigned_integer_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.enum_test_table;"
|
||||||
|
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.date_test_table;"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} --query="SYSTEM START MERGES;"
|
Loading…
Reference in New Issue
Block a user