mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Revert "Remove try/catch from DatabaseFilesystem"
This commit is contained in:
parent
34b8872118
commit
0b312f541a
@ -77,6 +77,7 @@ public:
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override;
|
||||
|
||||
|
||||
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
};
|
||||
|
||||
|
@ -11,11 +11,9 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace DB
|
||||
@ -77,8 +75,10 @@ bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, Cont
|
||||
/// Check access for file before checking its existence.
|
||||
if (check_path && !fileOrSymlinkPathStartsWith(table_path, user_files_path))
|
||||
{
|
||||
/// Access denied is thrown regardless of 'throw_on_error'
|
||||
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path);
|
||||
if (throw_on_error)
|
||||
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Check if the corresponding file exists.
|
||||
@ -128,25 +128,20 @@ bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_)
|
||||
if (tryGetTableFromCache(name))
|
||||
return true;
|
||||
|
||||
return checkTableFilePath(getTablePath(name), context_, /* throw_on_error */ false);
|
||||
return checkTableFilePath(getTablePath(name), context_, /* throw_on_error */false);
|
||||
}
|
||||
|
||||
StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_, bool throw_on_error) const
|
||||
StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_) const
|
||||
{
|
||||
/// Check if table exists in loaded tables map.
|
||||
if (auto table = tryGetTableFromCache(name))
|
||||
return table;
|
||||
|
||||
auto table_path = getTablePath(name);
|
||||
if (!checkTableFilePath(table_path, context_, throw_on_error))
|
||||
return {};
|
||||
|
||||
String format = FormatFactory::instance().getFormatFromFileName(table_path, throw_on_error);
|
||||
if (format.empty())
|
||||
return {};
|
||||
checkTableFilePath(table_path, context_, /* throw_on_error */true);
|
||||
|
||||
/// If the file exists, create a new table using TableFunctionFile and return it.
|
||||
auto args = makeASTFunction("file", std::make_shared<ASTLiteral>(table_path), std::make_shared<ASTLiteral>(format));
|
||||
auto args = makeASTFunction("file", std::make_shared<ASTLiteral>(table_path));
|
||||
|
||||
auto table_function = TableFunctionFactory::instance().get(args, context_);
|
||||
if (!table_function)
|
||||
@ -163,7 +158,7 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont
|
||||
StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_) const
|
||||
{
|
||||
/// getTableImpl can throw exceptions, do not catch them to show correct error to user.
|
||||
if (auto storage = getTableImpl(name, context_, true))
|
||||
if (auto storage = getTableImpl(name, context_))
|
||||
return storage;
|
||||
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
|
||||
@ -172,7 +167,20 @@ StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_
|
||||
|
||||
StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const
|
||||
{
|
||||
return getTableImpl(name, context_, false);
|
||||
try
|
||||
{
|
||||
return getTableImpl(name, context_);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// Ignore exceptions thrown by TableFunctionFile, which indicate that there is no table
|
||||
/// see tests/02722_database_filesystem.sh for more details.
|
||||
if (e.code() == ErrorCodes::FILE_DOESNT_EXIST)
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
bool DatabaseFilesystem::empty() const
|
||||
|
@ -48,7 +48,7 @@ public:
|
||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override;
|
||||
|
||||
protected:
|
||||
StoragePtr getTableImpl(const String & name, ContextPtr context, bool throw_on_error) const;
|
||||
StoragePtr getTableImpl(const String & name, ContextPtr context) const;
|
||||
|
||||
StoragePtr tryGetTableFromCache(const std::string & name) const;
|
||||
|
||||
|
@ -336,6 +336,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
return db_and_table;
|
||||
}
|
||||
|
||||
|
||||
if (table_id.database_name == TEMPORARY_DATABASE)
|
||||
{
|
||||
/// For temporary tables UUIDs are set in Context::resolveStorageID(...).
|
||||
@ -368,24 +369,8 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
database = it->second;
|
||||
}
|
||||
|
||||
StoragePtr table;
|
||||
if (exception)
|
||||
{
|
||||
try
|
||||
{
|
||||
table = database->getTable(table_id.table_name, context_);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
exception->emplace(e);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
table = database->tryGetTable(table_id.table_name, context_);
|
||||
}
|
||||
|
||||
if (!table && exception && !exception->has_value())
|
||||
auto table = database->tryGetTable(table_id.table_name, context_);
|
||||
if (!table && exception)
|
||||
exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs()));
|
||||
|
||||
if (!table)
|
||||
|
@ -61,7 +61,7 @@ CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir');
|
||||
""" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||:
|
||||
|
||||
# FILE_DOESNT_EXIST: unknown file
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 107" > /dev/null && echo "OK" || echo 'FAIL' ||:
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 60" > /dev/null && echo "OK" || echo 'FAIL' ||:
|
||||
|
||||
# BAD_ARGUMENTS: Cannot determine the file format by it's extension
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.myext\`;" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||:
|
||||
|
@ -17,7 +17,5 @@ test1
|
||||
16 17 18
|
||||
0 0 0
|
||||
Test 2: check exceptions
|
||||
BAD_ARGUMENTS
|
||||
BAD_ARGUMENTS
|
||||
BAD_ARGUMENTS
|
||||
BAD_ARGUMENTS
|
||||
OK
|
||||
OK
|
||||
|
@ -46,12 +46,12 @@ DROP DATABASE IF EXISTS test3;
|
||||
CREATE DATABASE test3 ENGINE = S3;
|
||||
USE test3;
|
||||
SELECT * FROM \"http://localhost:11111/test/a.myext\"
|
||||
""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS"
|
||||
""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
||||
USE test3;
|
||||
SELECT * FROM \"abacaba\"
|
||||
""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS"
|
||||
""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK"
|
||||
|
||||
# Cleanup
|
||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
||||
|
@ -4,8 +4,9 @@ test1
|
||||
1 2 3
|
||||
test2
|
||||
Test 2: check exceptions
|
||||
BAD_ARGUMENTS
|
||||
BAD_ARGUMENTS
|
||||
BAD_ARGUMENTS
|
||||
CANNOT_EXTRACT_TABLE_STRUCTURE
|
||||
BAD_ARGUMENTS
|
||||
OK0
|
||||
OK1
|
||||
OK2
|
||||
OK3
|
||||
OK4
|
||||
OK5
|
||||
|
@ -1,8 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, use-hdfs, no-parallel
|
||||
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
@ -38,19 +36,19 @@ echo "Test 2: check exceptions"
|
||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
||||
DROP DATABASE IF EXISTS test3;
|
||||
CREATE DATABASE test3 ENGINE = HDFS('abacaba');
|
||||
""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS"
|
||||
""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK0"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
||||
DROP DATABASE IF EXISTS test4;
|
||||
CREATE DATABASE test4 ENGINE = HDFS;
|
||||
USE test4;
|
||||
SELECT * FROM \"abacaba/file.tsv\"
|
||||
""" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE"
|
||||
""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK1"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK2"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK3"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK4"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK5"
|
||||
|
||||
|
||||
# Cleanup
|
||||
|
Loading…
Reference in New Issue
Block a user