mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +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 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 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;
|
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 <Storages/IStorage.h>
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
#include <Common/filesystemHelpers.h>
|
#include <Common/filesystemHelpers.h>
|
||||||
#include <Formats/FormatFactory.h>
|
|
||||||
|
|
||||||
#include <filesystem>
|
#include <filesystem>
|
||||||
|
|
||||||
|
|
||||||
namespace fs = std::filesystem;
|
namespace fs = std::filesystem;
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -77,8 +75,10 @@ bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, Cont
|
|||||||
/// Check access for file before checking its existence.
|
/// Check access for file before checking its existence.
|
||||||
if (check_path && !fileOrSymlinkPathStartsWith(table_path, user_files_path))
|
if (check_path && !fileOrSymlinkPathStartsWith(table_path, user_files_path))
|
||||||
{
|
{
|
||||||
/// Access denied is thrown regardless of 'throw_on_error'
|
if (throw_on_error)
|
||||||
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path);
|
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path);
|
||||||
|
else
|
||||||
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Check if the corresponding file exists.
|
/// Check if the corresponding file exists.
|
||||||
@ -131,22 +131,17 @@ bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_)
|
|||||||
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.
|
/// Check if table exists in loaded tables map.
|
||||||
if (auto table = tryGetTableFromCache(name))
|
if (auto table = tryGetTableFromCache(name))
|
||||||
return table;
|
return table;
|
||||||
|
|
||||||
auto table_path = getTablePath(name);
|
auto table_path = getTablePath(name);
|
||||||
if (!checkTableFilePath(table_path, context_, throw_on_error))
|
checkTableFilePath(table_path, context_, /* throw_on_error */true);
|
||||||
return {};
|
|
||||||
|
|
||||||
String format = FormatFactory::instance().getFormatFromFileName(table_path, throw_on_error);
|
|
||||||
if (format.empty())
|
|
||||||
return {};
|
|
||||||
|
|
||||||
/// If the file exists, create a new table using TableFunctionFile and return it.
|
/// 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_);
|
auto table_function = TableFunctionFactory::instance().get(args, context_);
|
||||||
if (!table_function)
|
if (!table_function)
|
||||||
@ -163,7 +158,7 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont
|
|||||||
StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_) const
|
StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_) const
|
||||||
{
|
{
|
||||||
/// getTableImpl can throw exceptions, do not catch them to show correct error to user.
|
/// 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;
|
return storage;
|
||||||
|
|
||||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
|
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
|
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
|
bool DatabaseFilesystem::empty() const
|
||||||
|
@ -48,7 +48,7 @@ public:
|
|||||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override;
|
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override;
|
||||||
|
|
||||||
protected:
|
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;
|
StoragePtr tryGetTableFromCache(const std::string & name) const;
|
||||||
|
|
||||||
|
@ -336,6 +336,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
|||||||
return db_and_table;
|
return db_and_table;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
if (table_id.database_name == TEMPORARY_DATABASE)
|
if (table_id.database_name == TEMPORARY_DATABASE)
|
||||||
{
|
{
|
||||||
/// For temporary tables UUIDs are set in Context::resolveStorageID(...).
|
/// For temporary tables UUIDs are set in Context::resolveStorageID(...).
|
||||||
@ -368,24 +369,8 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
|||||||
database = it->second;
|
database = it->second;
|
||||||
}
|
}
|
||||||
|
|
||||||
StoragePtr table;
|
auto table = database->tryGetTable(table_id.table_name, context_);
|
||||||
if (exception)
|
if (!table && 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())
|
|
||||||
exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs()));
|
exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs()));
|
||||||
|
|
||||||
if (!table)
|
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' ||:
|
""" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||:
|
||||||
|
|
||||||
# FILE_DOESNT_EXIST: unknown file
|
# 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
|
# 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' ||:
|
${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
|
16 17 18
|
||||||
0 0 0
|
0 0 0
|
||||||
Test 2: check exceptions
|
Test 2: check exceptions
|
||||||
BAD_ARGUMENTS
|
OK
|
||||||
BAD_ARGUMENTS
|
OK
|
||||||
BAD_ARGUMENTS
|
|
||||||
BAD_ARGUMENTS
|
|
||||||
|
@ -46,12 +46,12 @@ DROP DATABASE IF EXISTS test3;
|
|||||||
CREATE DATABASE test3 ENGINE = S3;
|
CREATE DATABASE test3 ENGINE = S3;
|
||||||
USE test3;
|
USE test3;
|
||||||
SELECT * FROM \"http://localhost:11111/test/a.myext\"
|
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 """
|
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
||||||
USE test3;
|
USE test3;
|
||||||
SELECT * FROM \"abacaba\"
|
SELECT * FROM \"abacaba\"
|
||||||
""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS"
|
""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK"
|
||||||
|
|
||||||
# Cleanup
|
# Cleanup
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
||||||
|
@ -4,8 +4,9 @@ test1
|
|||||||
1 2 3
|
1 2 3
|
||||||
test2
|
test2
|
||||||
Test 2: check exceptions
|
Test 2: check exceptions
|
||||||
BAD_ARGUMENTS
|
OK0
|
||||||
BAD_ARGUMENTS
|
OK1
|
||||||
BAD_ARGUMENTS
|
OK2
|
||||||
CANNOT_EXTRACT_TABLE_STRUCTURE
|
OK3
|
||||||
BAD_ARGUMENTS
|
OK4
|
||||||
|
OK5
|
||||||
|
@ -1,8 +1,6 @@
|
|||||||
#!/usr/bin/env bash
|
#!/usr/bin/env bash
|
||||||
# Tags: no-fasttest, use-hdfs, no-parallel
|
# Tags: no-fasttest, use-hdfs, no-parallel
|
||||||
|
|
||||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
|
|
||||||
|
|
||||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
# shellcheck source=../shell_config.sh
|
# shellcheck source=../shell_config.sh
|
||||||
. "$CURDIR"/../shell_config.sh
|
. "$CURDIR"/../shell_config.sh
|
||||||
@ -38,19 +36,19 @@ echo "Test 2: check exceptions"
|
|||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
||||||
DROP DATABASE IF EXISTS test3;
|
DROP DATABASE IF EXISTS test3;
|
||||||
CREATE DATABASE test3 ENGINE = HDFS('abacaba');
|
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 """
|
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
||||||
DROP DATABASE IF EXISTS test4;
|
DROP DATABASE IF EXISTS test4;
|
||||||
CREATE DATABASE test4 ENGINE = HDFS;
|
CREATE DATABASE test4 ENGINE = HDFS;
|
||||||
USE test4;
|
USE test4;
|
||||||
SELECT * FROM \"abacaba/file.tsv\"
|
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} -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 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS"
|
${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 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE"
|
${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 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS"
|
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK5"
|
||||||
|
|
||||||
|
|
||||||
# Cleanup
|
# Cleanup
|
||||||
|
Loading…
Reference in New Issue
Block a user