Fix for alter update and IN operator

This commit is contained in:
Ivan Lezhankin 2021-04-14 18:35:52 +03:00
parent a73483751f
commit 00e8571088
12 changed files with 33 additions and 17 deletions

View File

@ -102,7 +102,6 @@ private:
tryVisit<ASTSubquery>(table_expression.subquery);
}
/// @note It expects that only table (not column) identifiers are visited.
void visit(const ASTTableIdentifier & identifier, ASTPtr & ast) const
{
if (!identifier.compound())
@ -134,6 +133,11 @@ private:
{
if (is_operator_in && i == 1)
{
/// XXX: for some unknown reason this place assumes that argument can't be an alias,
/// like in the similar code in `MarkTableIdentifierVisitor`.
if (auto * identifier = child->children[i]->as<ASTIdentifier>())
child->children[i] = identifier->createTable();
/// Second argument of the "in" function (or similar) may be a table name or a subselect.
/// Rewrite the table name or descend into subselect.
if (!tryVisit<ASTTableIdentifier>(child->children[i]))

View File

@ -65,8 +65,7 @@ BlockIO InterpreterAlterQuery::execute()
auto alter_lock = table->lockForAlter(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout);
auto metadata_snapshot = table->getInMemoryMetadataPtr();
/// Add default database to table identifiers that we can encounter in e.g. default expressions,
/// mutation expression, etc.
/// Add default database to table identifiers that we can encounter in e.g. default expressions, mutation expression, etc.
AddDefaultDatabaseVisitor visitor(table_id.getDatabaseName());
ASTPtr command_list_ptr = alter.command_list->ptr();
visitor.visit(command_list_ptr);

View File

@ -1,12 +1,13 @@
#include <Poco/String.h>
#include <IO/WriteBufferFromOStream.h>
#include <Interpreters/misc.h>
#include <Interpreters/MarkTableIdentifiersVisitor.h>
#include <IO/WriteBufferFromOStream.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/misc.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
namespace DB
{

View File

@ -271,7 +271,7 @@ MutationsInterpreter::MutationsInterpreter(
: storage(std::move(storage_))
, metadata_snapshot(metadata_snapshot_)
, commands(std::move(commands_))
, context(context_)
, context(Context::createCopy(context_))
, can_execute(can_execute_)
, select_limits(SelectQueryOptions().analyze(!can_execute).ignoreLimits())
{

View File

@ -106,8 +106,7 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
settings.ostr << (settings.hilite ? hilite_none : "");
};
/// It could be compound but short
if (!isShort())
if (compound())
{
for (size_t i = 0, j = 0, size = name_parts.size(); i < size; ++i)
{

View File

@ -30,3 +30,5 @@ INSERT INTO geo VALUES ([[[(0, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4),
INSERT INTO geo VALUES ([[[(1, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], [[(-10, -10), (-10, -9), (-9, 10)]]], 2);
INSERT INTO geo VALUES ([[[(2, 0), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], [[(-10, -10), (-10, -9), (-9, 10)]]], 3);
SELECT wkt(p) FROM geo ORDER BY id;
DROP TABLE geo;

View File

@ -6,3 +6,5 @@ drop table if exists polygon_01302;
create table polygon_01302 (x Array(Array(Array(Tuple(Float64, Float64)))), y Array(Array(Array(Tuple(Float64, Float64))))) engine=Memory();
insert into polygon_01302 values ([[[(23.725750, 37.971536)]]], [[[(4.3826169, 50.8119483)]]]);
select polygonsDistanceSpherical(x, y) from polygon_01302;
drop table polygon_01302;

View File

@ -7,3 +7,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} -q "create table insert_big_json(a String, b String) engine=MergeTree() order by tuple()";
python3 -c "[print('{{\"a\":\"{}\", \"b\":\"{}\"'.format('clickhouse'* 1000000, 'dbms' * 1000000)) for i in range(10)]; [print('{{\"a\":\"{}\", \"b\":\"{}\"}}'.format('clickhouse'* 100000, 'dbms' * 100000)) for i in range(10)]" 2>/dev/null | ${CLICKHOUSE_CLIENT} --input_format_parallel_parsing=1 --max_memory_usage=0 -q "insert into insert_big_json FORMAT JSONEachRow" 2>&1 | grep -q "min_chunk_bytes_for_parallel_parsing" && echo "Ok." || echo "FAIL" ||:
${CLICKHOUSE_CLIENT} -q "drop table insert_big_json"

View File

@ -23,3 +23,5 @@ ${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'"
${CLICKHOUSE_CLIENT} -q "SELECT name, polygonAreaSpherical(p) from country_rings"
${CLICKHOUSE_CLIENT} -q "SELECT '-------------------------------------'"
${CLICKHOUSE_CLIENT} -q "drop table if exists country_rings;"
${CLICKHOUSE_CLIENT} -q "drop table country_polygons";

View File

@ -1,3 +1,3 @@
create table dist_01756 (dummy UInt8) ENGINE = Distributed('test_cluster_two_shards', 'system', 'one', dummy);
select ignore(1), * from dist_01756 where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1
select ignore(1), * from dist_01756 where 0 settings optimize_skip_unused_shards=1, force_optimize_skip_unused_shards=1;
drop table dist_01756;

View File

@ -65,3 +65,4 @@ SELECT tuple(inf, inf) as key, dictGet('01760_db.dict_array', 'name', key); --{s
DROP DICTIONARY 01760_db.dict_array;
DROP TABLE 01760_db.points;
DROP TABLE 01760_db.polygons;
DROP DATABASE 01760_db;

View File

@ -52,6 +52,7 @@ SKIP_LIST = [
"01304_direct_io",
"01306_benchmark_json",
"01035_lc_empty_part_bug", # FLAKY
"01175_distributed_ddl_output_mode_long", # tcp port in reference
"01320_create_sync_race_condition_zookeeper",
"01355_CSV_input_format_allow_errors",
"01370_client_autocomplete_word_break_characters", # expect-test
@ -76,13 +77,16 @@ SKIP_LIST = [
"01599_multiline_input_and_singleline_comments", # expect-test
"01601_custom_tld",
"01610_client_spawn_editor", # expect-test
"01658_read_file_to_stringcolumn",
"01674_unicode_asan",
"01676_clickhouse_client_autocomplete", # expect-test (partially)
"01683_text_log_deadlock", # secure tcp
"01684_ssd_cache_dictionary_simple_key",
"01746_executable_pool_dictionary",
"01747_executable_pool_dictionary_implicit_key.sql",
"01747_join_view_filter_dictionary",
"01748_dictionary_table_dot",
"01780_clickhouse_dictionary_source_loop",
]