Merge pull request #52320 from ClickHouse/create-index-unique-noop

This commit is contained in:
Ilya Yatsishin 2023-08-05 10:45:06 +02:00 committed by GitHub
commit 390c97cdae
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 120 additions and 13 deletions

View File

@ -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/ ||:

View File

@ -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.

View File

@ -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)
{

View File

@ -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 ";

View File

@ -20,6 +20,7 @@ public:
ASTPtr index_decl;
bool if_not_exists{false};
bool unique{false};
String getID(char delim) const override;

View File

@ -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)

View File

@ -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

View File

@ -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);

View File

@ -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",
)
)

View File

@ -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

View File

@ -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

View File

@ -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}")

View File

@ -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")