Fix checks

This commit is contained in:
kssenii 2022-07-14 12:52:30 +02:00
parent a60f1599d0
commit 5459644a37
8 changed files with 36 additions and 61 deletions

View File

@ -440,6 +440,11 @@ bool DiskObjectStorage::supportsCache() const
return object_storage->supportsCache();
}
bool DiskObjectStorage::isReadOnly() const
{
return object_storage->isReadOnly();
}
DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage(const String & name_)
{
return std::make_shared<DiskObjectStorage>(

View File

@ -168,6 +168,8 @@ public:
bool supportsCache() const override;
bool isReadOnly() const override;
void wrapWithCache(FileCachePtr cache, const String & layer_name);
const std::unordered_set<String> & getCacheLayersNames() const override { return cache_layers; }

View File

@ -164,6 +164,8 @@ public:
virtual void removeCacheIfExists(const std::string & /* path */) {}
virtual bool supportsCache() const { return false; }
virtual bool isReadOnly() const { return false; }
};
using ObjectStoragePtr = std::shared_ptr<IObjectStorage>;

View File

@ -11,7 +11,6 @@ namespace DB
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int FS_METADATA_ERROR;
extern const int FILE_DOESNT_EXIST;
extern const int NETWORK_ERROR;
}
@ -174,16 +173,6 @@ void MetadataStorageFromStaticFilesWebServerTransaction::removeRecursive(const s
WebObjectStorage::throwNotAllowed();
}
void MetadataStorageFromStaticFilesWebServerTransaction::createDirectory(const std::string &)
{
WebObjectStorage::throwNotAllowed();
}
void MetadataStorageFromStaticFilesWebServerTransaction::createDirectoryRecursive(const std::string &)
{
WebObjectStorage::throwNotAllowed();
}
void MetadataStorageFromStaticFilesWebServerTransaction::removeDirectory(const std::string &)
{
WebObjectStorage::throwNotAllowed();
@ -214,6 +203,26 @@ void MetadataStorageFromStaticFilesWebServerTransaction::createHardLink(const st
WebObjectStorage::throwNotAllowed();
}
void MetadataStorageFromStaticFilesWebServerTransaction::addBlobToMetadata(const std::string &, const std::string &, uint64_t)
{
WebObjectStorage::throwNotAllowed();
}
void MetadataStorageFromStaticFilesWebServerTransaction::unlinkMetadata(const std::string &)
{
WebObjectStorage::throwNotAllowed();
}
void MetadataStorageFromStaticFilesWebServerTransaction::createDirectory(const std::string &)
{
/// Noop.
}
void MetadataStorageFromStaticFilesWebServerTransaction::createDirectoryRecursive(const std::string &)
{
/// Noop.
}
void MetadataStorageFromStaticFilesWebServerTransaction::createEmptyMetadataFile(const std::string & /* path */)
{
/// Noop.
@ -230,16 +239,6 @@ void MetadataStorageFromStaticFilesWebServerTransaction::commit()
/// Noop.
}
void MetadataStorageFromStaticFilesWebServerTransaction::addBlobToMetadata(const std::string &, const std::string &, uint64_t)
{
WebObjectStorage::throwNotAllowed();
}
void MetadataStorageFromStaticFilesWebServerTransaction::unlinkMetadata(const std::string &)
{
WebObjectStorage::throwNotAllowed();
}
std::unordered_map<String, String> MetadataStorageFromStaticFilesWebServer::getSerializedMetadata(const std::vector<String> &) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getSerializedMetadata is not implemented for MetadataStorageFromStaticFilesWebServer");

View File

@ -1,8 +1,5 @@
#include <Disks/ObjectStorages/Web/WebObjectStorage.h>
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
#include <filesystem>
#include <Common/logger_useful.h>
#include <Common/escapeForFileName.h>
@ -12,12 +9,6 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Disks/IDisk.h>
#include <Disks/ObjectStorages/IObjectStorage.h>
#include <IO/ReadBufferFromFile.h>
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
@ -26,6 +17,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Poco/Exception.h>
#include <filesystem>
namespace fs = std::filesystem;
@ -37,17 +29,10 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
extern const int FILE_DOESNT_EXIST;
extern const int DIRECTORY_DOESNT_EXIST;
extern const int NETWORK_ERROR;
}
namespace ErrorCodes
{
extern const int CANNOT_UNLINK;
}
void WebObjectStorage::initialize(const String & uri_path) const
{
std::vector<String> directories_to_load;

View File

@ -79,11 +79,13 @@ public:
const std::string & config_prefix,
ContextPtr context) override;
bool supportsAppend() const override { return true; }
bool supportsAppend() const override { return false; }
std::string generateBlobNameForPath(const std::string & path) override { return path; }
bool isRemote() const override { return false; }
bool isRemote() const override { return true; }
bool isReadOnly() const override { return true; }
protected:
void initialize(const String & uri_path) const;

View File

@ -1928,25 +1928,6 @@ class ClickHouseCluster:
logging.debug("Waiting for NATS to start up")
time.sleep(1)
def wait_nginx_to_start(self, timeout=60):
self.nginx_ip = self.get_instance_ip(self.nginx_host)
start = time.time()
while time.time() - start < timeout:
try:
self.exec_in_container(
self.nginx_id,
["curl", "-X", "PUT", "-d", "Test", "http://test.com/test.txt"],
)
res = self.exec_in_container(
self.nginx_id, ["curl", "-X", "GET", "http://test.com/test.txt"]
)
assert res == "Test"
print("nginx static files server is available")
return
except Exception as ex:
print("Can't connect to nginx: " + str(ex))
time.sleep(0.5)
def wait_zookeeper_secure_to_start(self, timeout=20):
logging.debug("Wait ZooKeeper Secure to start")
start = time.time()
@ -2433,7 +2414,6 @@ class ClickHouseCluster:
)
self.up_called = True
self.nginx_docker_id = self.get_instance_docker_id("nginx")
self.wait_nginx_to_start()
if self.with_mongo and self.base_mongo_cmd:
logging.debug("Setup Mongo")

View File

@ -13,6 +13,6 @@ $CLICKHOUSE_CLIENT --xyzgarbage 2>&1 | grep -q "UNRECOGNIZED_ARGUMENTS" && echo
cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' xyzgarbage 2>&1 | grep -q "BAD_ARGUMENTS" && echo 'OK' || echo 'FAIL'
cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external -xyzgarbage --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' 2>&1 | grep -q "Bad arguments" && echo 'OK' || echo 'FAIL'
cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external -xyzgarbage --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' 2>&1 | grep -q "UNRECOGNIZED_ARGUMENTS" && echo 'OK' || echo 'FAIL'
cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --xyzgarbage --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' 2>&1 | grep -q "Bad arguments" && echo 'OK' || echo 'FAIL'
cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --xyzgarbage --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' 2>&1 | grep -q "UNRECOGNIZED_ARGUMENTS" && echo 'OK' || echo 'FAIL'