mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Remove useless stuff
This commit is contained in:
parent
318fbe884a
commit
9b435388de
@ -501,7 +501,8 @@ void S3ObjectStorage::applyNewSettings(
|
||||
}
|
||||
|
||||
auto current_settings = s3_settings.get();
|
||||
if (options.allow_client_change && (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3))
|
||||
if (options.allow_client_change
|
||||
&& (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3))
|
||||
{
|
||||
auto new_client = getClient(uri, *modified_settings, context, for_disk_s3);
|
||||
client.set(std::move(new_client));
|
||||
|
@ -1,9 +1,6 @@
|
||||
#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
|
||||
@ -16,10 +13,6 @@ 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)
|
||||
|
@ -69,27 +69,6 @@ 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 full_key = prefix.empty() ? key : (prefix + "." + key);
|
||||
|
||||
if (config.hasProperty(full_key))
|
||||
{
|
||||
std::string value = config.getString(full_key);
|
||||
LOG_DEBUG(&Poco::Logger::get(log_name), "{} = {}", full_key, value);
|
||||
}
|
||||
|
||||
// Recursively print sub-configurations
|
||||
printConfiguration(config, full_key, log_name);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
StorageObjectStorage::StorageObjectStorage(
|
||||
ConfigurationPtr configuration_,
|
||||
ObjectStoragePtr object_storage_,
|
||||
@ -110,17 +89,14 @@ StorageObjectStorage::StorageObjectStorage(
|
||||
, distributed_processing(distributed_processing_)
|
||||
, log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName())))
|
||||
{
|
||||
// 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 || columns_.empty()
|
||||
|| (configuration->format
|
||||
== "auto")) // If we don't have format or schema yet, we can't ignore failed configuration update, because relevant configuration is crucial for format and schema inference
|
||||
// If we don't have format or schema yet, we can't ignore failed configuration update, because relevant configuration is crucial for format and schema inference
|
||||
if (mode <= LoadingStrictnessLevel::CREATE || columns_.empty() || (configuration->format == "auto"))
|
||||
{
|
||||
throw;
|
||||
}
|
||||
|
@ -5,5 +5,4 @@
|
||||
<path>cache1</path>
|
||||
</cache1>
|
||||
</filesystem_caches>
|
||||
<!-- <async_load_databases>false</async_load_databases> -->
|
||||
</clickhouse>
|
||||
|
@ -5,7 +5,6 @@ import os
|
||||
import time
|
||||
import uuid
|
||||
from datetime import datetime
|
||||
from logging import log
|
||||
|
||||
import pyspark
|
||||
import pytest
|
||||
@ -857,20 +856,14 @@ 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}"
|
||||
)
|
||||
|
||||
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