mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Fix some bugs
This commit is contained in:
parent
07508cb381
commit
af7aa7de56
@ -1,6 +1,9 @@
|
||||
#include "Common.h"
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Poco/DateTimeFormat.h>
|
||||
#include <Poco/DateTimeFormatter.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace DB
|
||||
@ -13,6 +16,10 @@ std::vector<String> listFiles(
|
||||
{
|
||||
auto key = std::filesystem::path(configuration.getPath()) / prefix;
|
||||
RelativePathsWithMetadata files_with_metadata;
|
||||
// time_t now = time(nullptr);
|
||||
Poco::DateTime now;
|
||||
std::string formatted = Poco::DateTimeFormatter::format(now, Poco::DateTimeFormat::ISO8601_FORMAT);
|
||||
LOG_ERROR(&Poco::Logger::get("Inside listFiles"), "Time of files listing: {}", formatted);
|
||||
object_storage.listObjects(key, files_with_metadata, 0);
|
||||
Strings res;
|
||||
for (const auto & file_with_metadata : files_with_metadata)
|
||||
|
@ -36,7 +36,7 @@ public:
|
||||
|
||||
void update(ObjectStoragePtr object_storage, ContextPtr local_context) override
|
||||
{
|
||||
BaseStorageConfiguration::update(object_storage, local_context);
|
||||
// BaseStorageConfiguration::update(object_storage, local_context);
|
||||
auto new_metadata = DataLakeMetadata::create(object_storage, weak_from_this(), local_context);
|
||||
if (current_metadata && *current_metadata == *new_metadata)
|
||||
return;
|
||||
|
@ -22,6 +22,7 @@
|
||||
#include <Storages/ObjectStorage/Utils.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include "Databases/LoadingStrictnessLevel.h"
|
||||
#include "Storages/ColumnsDescription.h"
|
||||
|
||||
|
||||
@ -68,6 +69,27 @@ String StorageObjectStorage::getPathSample(StorageInMemoryMetadata metadata, Con
|
||||
return "";
|
||||
}
|
||||
|
||||
void printConfiguration(const Poco::Util::AbstractConfiguration & config, std::string log_name, const std::string & prefix = "")
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(prefix, keys);
|
||||
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
std::string fullKey = prefix.empty() ? key : (prefix + "." + key);
|
||||
|
||||
if (config.hasProperty(fullKey))
|
||||
{
|
||||
std::string value = config.getString(fullKey);
|
||||
LOG_DEBUG(&Poco::Logger::get(log_name), "{} = {}", fullKey, value);
|
||||
}
|
||||
|
||||
// Recursively print sub-configurations
|
||||
printConfiguration(config, fullKey, log_name);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
StorageObjectStorage::StorageObjectStorage(
|
||||
ConfigurationPtr configuration_,
|
||||
ObjectStoragePtr object_storage_,
|
||||
@ -77,6 +99,7 @@ StorageObjectStorage::StorageObjectStorage(
|
||||
const ConstraintsDescription & constraints_,
|
||||
const String & comment,
|
||||
std::optional<FormatSettings> format_settings_,
|
||||
LoadingStrictnessLevel mode,
|
||||
bool distributed_processing_,
|
||||
ASTPtr partition_by_)
|
||||
: IStorage(table_id_)
|
||||
@ -87,11 +110,27 @@ StorageObjectStorage::StorageObjectStorage(
|
||||
, distributed_processing(distributed_processing_)
|
||||
, log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName())))
|
||||
{
|
||||
ColumnsDescription columns{columns_};
|
||||
LOG_DEBUG(&Poco::Logger::get("StorageObjectStorage Creation"), "Columns size {}", columns.size());
|
||||
configuration->update(object_storage, context);
|
||||
// LOG_DEBUG(&Poco::Logger::get("StorageObjectStorage Creation"), "Columns size {}", columns.size());
|
||||
printConfiguration(context->getConfigRef(), "Storage create");
|
||||
try
|
||||
{
|
||||
// configuration->update(object_storage, context);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (mode <= LoadingStrictnessLevel::CREATE)
|
||||
{
|
||||
throw;
|
||||
}
|
||||
else
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
std::string sample_path;
|
||||
ColumnsDescription columns{columns_};
|
||||
resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context);
|
||||
configuration->check(context);
|
||||
|
||||
@ -271,6 +310,7 @@ void StorageObjectStorage::read(
|
||||
size_t num_streams)
|
||||
{
|
||||
configuration->update(object_storage, local_context);
|
||||
printConfiguration(local_context->getConfigRef(), "Select query");
|
||||
if (partition_by && configuration->withPartitionWildcard())
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
|
@ -57,6 +57,7 @@ public:
|
||||
const ConstraintsDescription & constraints_,
|
||||
const String & comment,
|
||||
std::optional<FormatSettings> format_settings_,
|
||||
LoadingStrictnessLevel mode,
|
||||
bool distributed_processing_ = false,
|
||||
ASTPtr partition_by_ = nullptr);
|
||||
|
||||
@ -217,6 +218,7 @@ public:
|
||||
|
||||
virtual void update(ObjectStoragePtr object_storage, ContextPtr local_context);
|
||||
|
||||
|
||||
protected:
|
||||
virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0;
|
||||
virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0;
|
||||
|
@ -51,13 +51,14 @@ static std::shared_ptr<StorageObjectStorage> createStorageObjectStorage(
|
||||
|
||||
return std::make_shared<StorageObjectStorage>(
|
||||
configuration,
|
||||
configuration->createObjectStorage(context, /* is_readonly */false),
|
||||
configuration->createObjectStorage(context, /* is_readonly */ false),
|
||||
args.getContext(),
|
||||
args.table_id,
|
||||
args.columns,
|
||||
args.constraints,
|
||||
args.comment,
|
||||
format_settings,
|
||||
args.mode,
|
||||
/* distributed_processing */ false,
|
||||
partition_by);
|
||||
}
|
||||
|
@ -117,8 +117,9 @@ StoragePtr TableFunctionObjectStorage<Definition, Configuration>::executeImpl(
|
||||
columns,
|
||||
ConstraintsDescription{},
|
||||
String{},
|
||||
/* format_settings */std::nullopt,
|
||||
/* distributed_processing */false,
|
||||
/* format_settings */ std::nullopt,
|
||||
/* mode */ LoadingStrictnessLevel::CREATE,
|
||||
/* distributed_processing */ false,
|
||||
nullptr);
|
||||
|
||||
storage->startup();
|
||||
|
@ -41,9 +41,10 @@ StoragePtr TableFunctionObjectStorageCluster<Definition, Configuration>::execute
|
||||
StorageID(Base::getDatabaseName(), table_name),
|
||||
columns,
|
||||
ConstraintsDescription{},
|
||||
/* comment */String{},
|
||||
/* format_settings */std::nullopt, /// No format_settings
|
||||
/* distributed_processing */true,
|
||||
/* comment */ String{},
|
||||
/* format_settings */ std::nullopt, /// No format_settings
|
||||
/* mode */ LoadingStrictnessLevel::CREATE,
|
||||
/* distributed_processing */ true,
|
||||
/*partition_by_=*/nullptr);
|
||||
}
|
||||
else
|
||||
|
@ -5,4 +5,5 @@
|
||||
<path>cache1</path>
|
||||
</cache1>
|
||||
</filesystem_caches>
|
||||
<!-- <async_load_databases>false</async_load_databases> -->
|
||||
</clickhouse>
|
||||
|
@ -6,6 +6,8 @@ import time
|
||||
import uuid
|
||||
from datetime import datetime
|
||||
|
||||
from logging import log
|
||||
|
||||
import pyspark
|
||||
import pytest
|
||||
from azure.storage.blob import BlobServiceClient
|
||||
@ -856,14 +858,20 @@ def test_restart_broken_s3(started_cluster):
|
||||
)
|
||||
minio_client.remove_bucket(bucket)
|
||||
|
||||
print("Before restart: ", datetime.now())
|
||||
|
||||
instance.restart_clickhouse()
|
||||
|
||||
assert "NoSuchBucket" in instance.query_and_get_error(
|
||||
f"SELECT count() FROM {TABLE_NAME}"
|
||||
)
|
||||
# assert "NoSuchBucket" in instance.query_and_get_error(
|
||||
# f"SELECT count() FROM {TABLE_NAME}"
|
||||
# )
|
||||
|
||||
time.sleep(10)
|
||||
|
||||
minio_client.make_bucket(bucket)
|
||||
|
||||
print("Before successful select: ", datetime.now())
|
||||
|
||||
files = default_upload_directory(
|
||||
started_cluster,
|
||||
"s3",
|
||||
|
Loading…
Reference in New Issue
Block a user