mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Fix for alter update and IN operator
This commit is contained in:
parent
a73483751f
commit
00e8571088
@ -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]))
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
{
|
||||
|
||||
|
@ -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())
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -1,9 +1,11 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${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" ||:
|
||||
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"
|
||||
|
@ -22,4 +22,6 @@ ${CLICKHOUSE_CLIENT} -q "SELECT name, polygonPerimeterSpherical(p) from country_
|
||||
${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 if exists country_rings;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "drop table country_polygons";
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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",
|
||||
]
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user