mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Merge pull request #52320 from ClickHouse/create-index-unique-noop
This commit is contained in:
commit
390c97cdae
@ -96,5 +96,4 @@ rg -Fa "Fatal" /var/log/clickhouse-server/clickhouse-server.log ||:
|
||||
zstd < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst &
|
||||
|
||||
# Compressed (FIXME: remove once only github actions will be left)
|
||||
rm /var/log/clickhouse-server/clickhouse-server.log
|
||||
mv /var/log/clickhouse-server/stderr.log /test_output/ ||:
|
||||
|
@ -783,6 +783,7 @@ class IColumn;
|
||||
M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \
|
||||
M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \
|
||||
M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\
|
||||
M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0)\
|
||||
// End of COMMON_SETTINGS
|
||||
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS.
|
||||
|
||||
|
@ -16,6 +16,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_IS_READ_ONLY;
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
@ -24,6 +25,15 @@ BlockIO InterpreterCreateIndexQuery::execute()
|
||||
auto current_context = getContext();
|
||||
const auto & create_index = query_ptr->as<ASTCreateIndexQuery &>();
|
||||
|
||||
if (create_index.unique)
|
||||
{
|
||||
if (!current_context->getSettingsRef().create_index_ignore_unique)
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CREATE UNIQUE INDEX is not supported."
|
||||
" SET create_index_ignore_unique=1 to ignore this UNIQUE keyword.");
|
||||
}
|
||||
|
||||
}
|
||||
// Noop if allow_create_index_without_type = true. throw otherwise
|
||||
if (!create_index.index_decl->as<ASTIndexDeclaration>()->type)
|
||||
{
|
||||
|
@ -38,7 +38,7 @@ void ASTCreateIndexQuery::formatQueryImpl(const FormatSettings & settings, Forma
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
|
||||
|
||||
settings.ostr << "CREATE INDEX " << (if_not_exists ? "IF NOT EXISTS " : "");
|
||||
settings.ostr << "CREATE " << (unique ? "UNIQUE " : "") << "INDEX " << (if_not_exists ? "IF NOT EXISTS " : "");
|
||||
index_name->formatImpl(settings, state, frame);
|
||||
settings.ostr << " ON ";
|
||||
|
||||
|
@ -20,6 +20,7 @@ public:
|
||||
ASTPtr index_decl;
|
||||
|
||||
bool if_not_exists{false};
|
||||
bool unique{false};
|
||||
|
||||
String getID(char delim) const override;
|
||||
|
||||
|
@ -80,6 +80,7 @@ bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect
|
||||
node = query;
|
||||
|
||||
ParserKeyword s_create("CREATE");
|
||||
ParserKeyword s_unique("UNIQUE");
|
||||
ParserKeyword s_index("INDEX");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserKeyword s_on("ON");
|
||||
@ -91,10 +92,14 @@ bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect
|
||||
|
||||
String cluster_str;
|
||||
bool if_not_exists = false;
|
||||
bool unique = false;
|
||||
|
||||
if (!s_create.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (s_unique.ignore(pos, expected))
|
||||
unique = true;
|
||||
|
||||
if (!s_index.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
@ -131,6 +136,7 @@ bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect
|
||||
query->children.push_back(index_decl);
|
||||
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->unique = unique;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
if (query->database)
|
||||
|
@ -6,7 +6,7 @@ namespace DB
|
||||
{
|
||||
|
||||
/** Query like this:
|
||||
* CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value
|
||||
* CREATE [UNIQUE] INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value
|
||||
*/
|
||||
|
||||
class ParserCreateIndexQuery : public IParserBase
|
||||
|
@ -0,0 +1,3 @@
|
||||
SET allow_create_index_without_type=1;
|
||||
SET create_index_ignore_unique=1;
|
||||
CREATE UNIQUE INDEX idx_tab2_0 ON tab2 (col1);
|
@ -62,7 +62,8 @@ def default_clickhouse_odbc_conn_str():
|
||||
return str(
|
||||
OdbcConnectingArgs.create_from_kw(
|
||||
dsn="ClickHouse DSN (ANSI)",
|
||||
Url="http://localhost:8123/query?default_format=ODBCDriver2&default_table_engine=MergeTree&union_default_mode=DISTINCT&group_by_use_nulls=1&join_use_nulls=1&allow_create_index_without_type=1",
|
||||
Timeout="300",
|
||||
Url="http://localhost:8123/query?default_format=ODBCDriver2&default_table_engine=MergeTree&union_default_mode=DISTINCT&group_by_use_nulls=1&join_use_nulls=1&allow_create_index_without_type=1&create_index_ignore_unique=1",
|
||||
)
|
||||
)
|
||||
|
||||
|
@ -186,10 +186,10 @@ def mode_check_statements(parser):
|
||||
|
||||
out_stages_dir = os.path.join(out_dir, f"{args.mode}-stages")
|
||||
|
||||
complete_sqlite_dir = os.path.join(out_stages_dir, "complete-sqlite")
|
||||
complete_sqlite_dir = os.path.join(out_stages_dir, "statements-sqlite")
|
||||
os.makedirs(complete_sqlite_dir, exist_ok=True)
|
||||
|
||||
reports["complete-sqlite"] = run_all_tests_in_parallel(
|
||||
reports["statements-sqlite"] = run_all_tests_in_parallel(
|
||||
setup_kwargs=as_kwargs(
|
||||
engine=Engines.SQLITE,
|
||||
),
|
||||
@ -224,6 +224,64 @@ def mode_check_statements(parser):
|
||||
parser.set_defaults(func=calle)
|
||||
|
||||
|
||||
def mode_check_complete(parser):
|
||||
parser.add_argument("--input-dir", metavar="DIR", required=True)
|
||||
parser.add_argument("--out-dir", metavar="DIR", required=True)
|
||||
|
||||
def calle(args):
|
||||
input_dir = os.path.realpath(args.input_dir)
|
||||
out_dir = os.path.realpath(args.out_dir)
|
||||
|
||||
if not os.path.exists(input_dir):
|
||||
raise FileNotFoundError(
|
||||
input_dir, f"check statements: no such file or directory {input_dir}"
|
||||
)
|
||||
|
||||
if not os.path.isdir(input_dir):
|
||||
raise NotADirectoryError(
|
||||
input_dir, f"check statements:: not a dir {input_dir}"
|
||||
)
|
||||
|
||||
reports = dict()
|
||||
|
||||
out_stages_dir = os.path.join(out_dir, f"{args.mode}-stages")
|
||||
|
||||
complete_sqlite_dir = os.path.join(out_stages_dir, "complete-sqlite")
|
||||
os.makedirs(complete_sqlite_dir, exist_ok=True)
|
||||
|
||||
reports["complete-sqlite"] = run_all_tests_in_parallel(
|
||||
setup_kwargs=as_kwargs(
|
||||
engine=Engines.SQLITE,
|
||||
),
|
||||
runner_kwargs=as_kwargs(
|
||||
verify_mode=False,
|
||||
stop_at_statement_error=True,
|
||||
),
|
||||
input_dir=input_dir,
|
||||
output_dir=complete_sqlite_dir,
|
||||
)
|
||||
|
||||
verify_clickhouse_dir = os.path.join(out_stages_dir, "complete-clickhouse")
|
||||
os.makedirs(verify_clickhouse_dir, exist_ok=True)
|
||||
|
||||
reports["complete-clickhouse"] = run_all_tests_in_parallel(
|
||||
setup_kwargs=as_kwargs(
|
||||
engine=Engines.ODBC,
|
||||
conn_str=default_clickhouse_odbc_conn_str(),
|
||||
),
|
||||
runner_kwargs=as_kwargs(
|
||||
verify_mode=True,
|
||||
stop_at_statement_error=True,
|
||||
),
|
||||
input_dir=complete_sqlite_dir,
|
||||
output_dir=verify_clickhouse_dir,
|
||||
)
|
||||
|
||||
statements_report(reports, out_dir, args.mode)
|
||||
|
||||
parser.set_defaults(func=calle)
|
||||
|
||||
|
||||
def make_actual_report(reports):
|
||||
return {stage: report.get_map() for stage, report in reports.items()}
|
||||
|
||||
@ -399,16 +457,22 @@ def parse_args():
|
||||
)
|
||||
|
||||
subparsers = parser.add_subparsers(dest="mode")
|
||||
mode_check_complete(
|
||||
subparsers.add_parser(
|
||||
"complete-test",
|
||||
help="Run all tests. Check that all statements and queries are passed",
|
||||
)
|
||||
)
|
||||
mode_check_statements(
|
||||
subparsers.add_parser(
|
||||
"statements-test",
|
||||
help="Run all test. Check that all statements are passed",
|
||||
help="Run all tests. Check that all statements are passed",
|
||||
)
|
||||
)
|
||||
mode_self_test(
|
||||
subparsers.add_parser(
|
||||
"self-test",
|
||||
help="Run all test. Check that all statements are passed",
|
||||
help="Run all tests. Check that all statements are passed",
|
||||
)
|
||||
)
|
||||
args = parser.parse_args()
|
||||
|
File diff suppressed because one or more lines are too long
@ -142,4 +142,13 @@ SELECT number+1 from system.numbers LIMIT 20
|
||||
----
|
||||
20 values hashing to 52c46dff81346ead02fcf6245c762b1a
|
||||
|
||||
# Debug how incorrect result type parses
|
||||
statement ok
|
||||
CREATE TABLE tab0(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col2 TEXT, col3 INTEGER, col4 FLOAT, col5 TEXT)
|
||||
|
||||
statement ok
|
||||
INSERT INTO tab0 VALUES(0,535,860.48,'uxbns',253,640.58,'jvqkl')
|
||||
|
||||
skipif ClickHouse
|
||||
query I rowsort label-20
|
||||
SELECT + col2 AS col5 FROM tab0 WHERE NOT ( col0 ) * - - col4 IS NULL
|
||||
|
@ -9,7 +9,13 @@ from enum import Enum
|
||||
from hashlib import md5
|
||||
from functools import reduce
|
||||
|
||||
from exceptions import Error, ProgramError, ErrorWithParent, DataResultDiffer
|
||||
from exceptions import (
|
||||
Error,
|
||||
ProgramError,
|
||||
ErrorWithParent,
|
||||
DataResultDiffer,
|
||||
QueryExecutionError,
|
||||
)
|
||||
|
||||
|
||||
logger = logging.getLogger("parser")
|
||||
@ -480,6 +486,7 @@ class QueryResult:
|
||||
for row in rows:
|
||||
res_row = []
|
||||
for c, t in zip(row, types):
|
||||
logger.debug(f"Builging row. c:{c} t:{t}")
|
||||
if c is None:
|
||||
res_row.append("NULL")
|
||||
continue
|
||||
@ -490,7 +497,12 @@ class QueryResult:
|
||||
else:
|
||||
res_row.append(str(c))
|
||||
elif t == "I":
|
||||
res_row.append(str(int(c)))
|
||||
try:
|
||||
res_row.append(str(int(c)))
|
||||
except ValueError as ex:
|
||||
raise QueryExecutionError(
|
||||
f"Got non-integer result '{c}' for I type."
|
||||
)
|
||||
elif t == "R":
|
||||
res_row.append(f"{c:.3f}")
|
||||
|
||||
|
@ -361,7 +361,7 @@ class TestRunner:
|
||||
continue
|
||||
|
||||
if block.get_block_type() == test_parser.BlockType.control:
|
||||
clogger.debug("Skip control block", name_pos)
|
||||
clogger.debug("Skip control block %s", name_pos)
|
||||
block.dump_to(out_stream)
|
||||
continue
|
||||
|
||||
@ -374,13 +374,14 @@ class TestRunner:
|
||||
continue
|
||||
|
||||
request = block.get_request()
|
||||
exec_res = execute_request(request, self.connection)
|
||||
|
||||
if block.get_block_type() in self.skip_request_types:
|
||||
clogger.debug("Runtime skip block for %s", self.dbms_name)
|
||||
block.dump_to(out_stream)
|
||||
continue
|
||||
|
||||
exec_res = execute_request(request, self.connection)
|
||||
|
||||
if block.get_block_type() == test_parser.BlockType.statement:
|
||||
try:
|
||||
clogger.debug("this is statement")
|
||||
|
Loading…
Reference in New Issue
Block a user