Revert "Remove try/catch from DatabaseFilesystem"

This commit is contained in:
Alexander Tokmakov 2023-08-04 13:29:28 +03:00 committed by GitHub
parent 34b8872118
commit 0b312f541a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 45 additions and 54 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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