mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-11 17:02:25 +00:00
Merge branch 'master' into system-query-metric-log
This commit is contained in:
commit
10b4f722ae
@ -432,6 +432,14 @@ QueryTreeNodePtr IdentifierResolver::tryResolveTableIdentifierFromDatabaseCatalo
|
||||
else
|
||||
storage = DatabaseCatalog::instance().tryGetTable(storage_id, context);
|
||||
|
||||
if (!storage && storage_id.hasUUID())
|
||||
{
|
||||
// If `storage_id` has UUID, it is possible that the UUID is removed from `DatabaseCatalog` after `context->resolveStorageID(storage_id)`
|
||||
// We try to get the table with the database name and the table name.
|
||||
auto database = DatabaseCatalog::instance().tryGetDatabase(storage_id.getDatabaseName());
|
||||
if (database)
|
||||
storage = database->tryGetTable(table_name, context);
|
||||
}
|
||||
if (!storage)
|
||||
return {};
|
||||
|
||||
|
@ -452,7 +452,7 @@
|
||||
M(553, LZMA_STREAM_ENCODER_FAILED) \
|
||||
M(554, LZMA_STREAM_DECODER_FAILED) \
|
||||
M(555, ROCKSDB_ERROR) \
|
||||
M(556, SYNC_MYSQL_USER_ACCESS_ERROR)\
|
||||
M(556, SYNC_MYSQL_USER_ACCESS_ERROR) \
|
||||
M(557, UNKNOWN_UNION) \
|
||||
M(558, EXPECTED_ALL_OR_DISTINCT) \
|
||||
M(559, INVALID_GRPC_QUERY_INFO) \
|
||||
@ -578,7 +578,7 @@
|
||||
M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \
|
||||
M(698, INVALID_REDIS_STORAGE_TYPE) \
|
||||
M(699, INVALID_REDIS_TABLE_STRUCTURE) \
|
||||
M(700, USER_SESSION_LIMIT_EXCEEDED) \
|
||||
M(700, USER_SESSION_LIMIT_EXCEEDED) \
|
||||
M(701, CLUSTER_DOESNT_EXIST) \
|
||||
M(702, CLIENT_INFO_DOES_NOT_MATCH) \
|
||||
M(703, INVALID_IDENTIFIER) \
|
||||
@ -610,15 +610,16 @@
|
||||
M(729, ILLEGAL_TIME_SERIES_TAGS) \
|
||||
M(730, REFRESH_FAILED) \
|
||||
M(731, QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE) \
|
||||
\
|
||||
M(733, TABLE_IS_BEING_RESTARTED) \
|
||||
\
|
||||
M(900, DISTRIBUTED_CACHE_ERROR) \
|
||||
M(901, CANNOT_USE_DISTRIBUTED_CACHE) \
|
||||
\
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
M(1001, STD_EXCEPTION) \
|
||||
M(1002, UNKNOWN_EXCEPTION) \
|
||||
/* See END */
|
||||
/* See END */
|
||||
|
||||
#ifdef APPLY_FOR_EXTERNAL_ERROR_CODES
|
||||
#define APPLY_FOR_ERROR_CODES(M) APPLY_FOR_BUILTIN_ERROR_CODES(M) APPLY_FOR_EXTERNAL_ERROR_CODES(M)
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/atomicRename.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -1663,7 +1664,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
if (need_add_to_database && !database)
|
||||
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist", backQuoteIfNeed(database_name));
|
||||
|
||||
if (create.replace_table)
|
||||
if (create.replace_table
|
||||
|| (create.replace_view && (database->getEngineName() == "Atomic" || database->getEngineName() == "Replicated")))
|
||||
{
|
||||
chassert(!ddl_guard);
|
||||
return doCreateOrReplaceTable(create, properties, mode);
|
||||
@ -1973,15 +1975,19 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
|
||||
|
||||
|
||||
UInt64 name_hash = sipHash64(create.getDatabase() + create.getTable());
|
||||
UInt16 random_suffix = thread_local_rng();
|
||||
String random_suffix;
|
||||
if (auto txn = current_context->getZooKeeperMetadataTransaction())
|
||||
{
|
||||
/// Avoid different table name on database replicas
|
||||
random_suffix = sipHash64(txn->getTaskZooKeeperPath());
|
||||
UInt16 hashed_zk_path = sipHash64(txn->getTaskZooKeeperPath());
|
||||
random_suffix = getHexUIntLowercase(hashed_zk_path);
|
||||
}
|
||||
create.setTable(fmt::format("_tmp_replace_{}_{}",
|
||||
getHexUIntLowercase(name_hash),
|
||||
getHexUIntLowercase(random_suffix)));
|
||||
else
|
||||
{
|
||||
random_suffix = getRandomASCIIString(/*length=*/4);
|
||||
}
|
||||
|
||||
create.setTable(fmt::format("_tmp_replace_{}_{}", getHexUIntLowercase(name_hash), random_suffix));
|
||||
|
||||
ast_drop->setTable(create.getTable());
|
||||
ast_drop->is_dictionary = create.is_dictionary;
|
||||
@ -2024,16 +2030,16 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
|
||||
|
||||
auto ast_rename = std::make_shared<ASTRenameQuery>(ASTRenameQuery::Elements{std::move(elem)});
|
||||
ast_rename->dictionary = create.is_dictionary;
|
||||
if (create.create_or_replace)
|
||||
if (create.create_or_replace || create.replace_view)
|
||||
{
|
||||
/// CREATE OR REPLACE TABLE
|
||||
/// CREATE OR REPLACE TABLE/VIEW
|
||||
/// Will execute ordinary RENAME instead of EXCHANGE if the target table does not exist
|
||||
ast_rename->rename_if_cannot_exchange = true;
|
||||
ast_rename->exchange = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// REPLACE TABLE
|
||||
/// REPLACE TABLE/VIEW
|
||||
/// Will execute EXCHANGE query and fail if the target table does not exist
|
||||
ast_rename->exchange = true;
|
||||
}
|
||||
|
@ -85,7 +85,8 @@ public:
|
||||
void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override;
|
||||
|
||||
/// Check access right, validate definer statement and replace `CURRENT USER` with actual name.
|
||||
static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_materialized_view = false, bool skip_check_permissions = false);
|
||||
static void processSQLSecurityOption(
|
||||
ContextPtr context_, ASTSQLSecurity & sql_security, bool is_materialized_view = false, bool skip_check_permissions = false);
|
||||
|
||||
private:
|
||||
struct TableProperties
|
||||
|
@ -31,6 +31,7 @@ namespace ErrorCodes
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int DEADLOCK_AVOIDED;
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
extern const int TABLE_IS_BEING_RESTARTED;
|
||||
}
|
||||
|
||||
IStorage::IStorage(StorageID storage_id_, std::unique_ptr<StorageInMemoryMetadata> metadata_)
|
||||
@ -66,12 +67,13 @@ RWLockImpl::LockHolder IStorage::tryLockTimed(
|
||||
TableLockHolder IStorage::lockForShare(const String & query_id, const std::chrono::milliseconds & acquire_timeout)
|
||||
{
|
||||
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
||||
|
||||
if (is_dropped || is_detached)
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
auto table_id = getStorageID();
|
||||
if (!table_id.hasUUID() && (is_dropped || is_detached))
|
||||
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped or detached", table_id.database_name, table_id.table_name);
|
||||
}
|
||||
|
||||
if (is_being_restarted)
|
||||
throw Exception(
|
||||
ErrorCodes::TABLE_IS_BEING_RESTARTED, "Table {}.{} is being restarted", table_id.database_name, table_id.table_name);
|
||||
return result;
|
||||
}
|
||||
|
||||
@ -79,12 +81,10 @@ TableLockHolder IStorage::tryLockForShare(const String & query_id, const std::ch
|
||||
{
|
||||
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
||||
|
||||
if (is_dropped || is_detached)
|
||||
{
|
||||
// Table was dropped while acquiring the lock
|
||||
auto table_id = getStorageID();
|
||||
if (is_being_restarted || (!table_id.hasUUID() && (is_dropped || is_detached)))
|
||||
// Table was dropped or is being restarted while acquiring the lock
|
||||
result = nullptr;
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -53,7 +53,7 @@ export -f thread_restart;
|
||||
|
||||
TIMEOUT=15
|
||||
|
||||
timeout $TIMEOUT bash -c thread_ddl 2>&1| grep -Fa "Exception: " | grep -Fv -e "TABLE_IS_DROPPED" -e "UNKNOWN_TABLE" -e "DATABASE_NOT_EMPTY" &
|
||||
timeout $TIMEOUT bash -c thread_ddl 2>&1| grep -Fa "Exception: " | grep -Fv -e "TABLE_IS_DROPPED" -e "UNKNOWN_TABLE" -e "DATABASE_NOT_EMPTY" -e "TABLE_IS_BEING_RESTARTED" &
|
||||
timeout $TIMEOUT bash -c thread_insert 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread_restart 2>&1| grep -Fa "Exception: " | grep -Fv -e "is currently dropped or renamed" -e "is being dropped or detached" &
|
||||
|
||||
|
@ -0,0 +1,52 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
# with Atomic engine
|
||||
$CLICKHOUSE_CLIENT --query "CREATE DATABASE IF NOT EXISTS ${CLICKHOUSE_DATABASE}_db ENGINE=Atomic"
|
||||
|
||||
function create_or_replace_view_thread
|
||||
{
|
||||
for _ in {1..20}; do
|
||||
$CLICKHOUSE_CLIENT --query "CREATE OR REPLACE VIEW ${CLICKHOUSE_DATABASE}_db.test_view AS SELECT 'abcdef'" > /dev/null
|
||||
done
|
||||
}
|
||||
export -f create_or_replace_view_thread;
|
||||
|
||||
function select_view_thread
|
||||
{
|
||||
for _ in {1..20}; do
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM ${CLICKHOUSE_DATABASE}_db.test_view" > /dev/null
|
||||
done
|
||||
}
|
||||
export -f select_view_thread;
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "CREATE OR REPLACE VIEW ${CLICKHOUSE_DATABASE}_db.test_view AS SELECT 'abcdef'" > /dev/null
|
||||
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
|
||||
wait
|
@ -0,0 +1,52 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
# with Replicated engine
|
||||
$CLICKHOUSE_CLIENT --query "CREATE DATABASE IF NOT EXISTS ${CLICKHOUSE_DATABASE}_db ENGINE=Replicated('/test/clickhouse/db/${CLICKHOUSE_DATABASE}_db', 's1', 'r1')"
|
||||
|
||||
function create_or_replace_view_thread
|
||||
{
|
||||
for _ in {1..15}; do
|
||||
$CLICKHOUSE_CLIENT --query "CREATE OR REPLACE VIEW ${CLICKHOUSE_DATABASE}_db.test_view AS SELECT 'abcdef'" > /dev/null
|
||||
done
|
||||
}
|
||||
export -f create_or_replace_view_thread;
|
||||
|
||||
function select_view_thread
|
||||
{
|
||||
for _ in {1..15}; do
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM ${CLICKHOUSE_DATABASE}_db.test_view" > /dev/null
|
||||
done
|
||||
}
|
||||
export -f select_view_thread;
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "CREATE OR REPLACE VIEW ${CLICKHOUSE_DATABASE}_db.test_view AS SELECT 'abcdef'" > /dev/null
|
||||
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
bash -c select_view_thread &
|
||||
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
bash -c create_or_replace_view_thread &
|
||||
|
||||
wait
|
@ -0,0 +1,24 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
# with Atomic engine
|
||||
$CLICKHOUSE_CLIENT --query "CREATE DATABASE IF NOT EXISTS ${CLICKHOUSE_DATABASE}_db ENGINE=Atomic"
|
||||
|
||||
function create_or_replace_table_thread
|
||||
{
|
||||
for _ in {1..20}; do
|
||||
$CLICKHOUSE_CLIENT --query "CREATE OR REPLACE TABLE ${CLICKHOUSE_DATABASE}_db.test_table (x Int) ENGINE=Memory" > /dev/null
|
||||
done
|
||||
}
|
||||
export -f create_or_replace_table_thread;
|
||||
|
||||
for _ in {1..20}; do
|
||||
bash -c create_or_replace_table_thread &
|
||||
done
|
||||
|
||||
wait
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS ${CLICKHOUSE_DATABASE}_db SYNC";
|
@ -0,0 +1,25 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
# with Replicated engine
|
||||
$CLICKHOUSE_CLIENT --query "CREATE DATABASE IF NOT EXISTS ${CLICKHOUSE_DATABASE}_db ENGINE=Replicated('/test/clickhouse/db/${CLICKHOUSE_DATABASE}_db', 's1', 'r1')"
|
||||
|
||||
function create_or_replace_table_thread
|
||||
{
|
||||
for _ in {1..15}; do
|
||||
$CLICKHOUSE_CLIENT --query "CREATE OR REPLACE TABLE ${CLICKHOUSE_DATABASE}_db.test_table (x Int) ENGINE=Memory" > /dev/null
|
||||
done
|
||||
}
|
||||
export -f create_or_replace_table_thread;
|
||||
|
||||
for _ in {1..15}; do
|
||||
bash -c create_or_replace_table_thread &
|
||||
done
|
||||
|
||||
wait
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS ${CLICKHOUSE_DATABASE}_db SYNC";
|
Loading…
Reference in New Issue
Block a user