Skip to content

Commit 90b1318

Browse files
authored
Merge pull request ClickHouse#79018 from ClickHouse/revert-76467-refresh-mergetree
Revert "Support for refresh in readonly MergeTree tables"
2 parents acdb7f9 + 5d082a3 commit 90b1318

28 files changed

+105
-346
lines changed

src/Core/SettingsChangesHistory.cpp

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -703,7 +703,6 @@ const VersionToSettingsChangesMap & getMergeTreeSettingsChangesHistory()
703703
{
704704
addSettingsChanges(merge_tree_settings_changes_history, "25.4",
705705
{
706-
{"refresh_parts_interval", 0, 0, "A new setting"},
707706
{"max_merge_delayed_streams_for_parallel_write", 1000, 100, "New setting"},
708707
{"max_postpone_time_for_failed_replicated_fetches_ms", 1ULL * 60 * 1000, 1ULL * 60 * 1000, "Added new setting to enable postponing fetch tasks in the replication queue."},
709708
{"max_postpone_time_for_failed_replicated_merges_ms", 1ULL * 60 * 1000, 1ULL * 60 * 1000, "Added new setting to enable postponing merge tasks in the replication queue."},

src/Databases/DatabaseLazy.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -326,7 +326,7 @@ try
326326

327327
if (!it->second.table || isSharedPtrUnique(it->second.table))
328328
{
329-
LOG_DEBUG(log, "Removing table {} from cache.", backQuote(it->first));
329+
LOG_DEBUG(log, "Drop table {} from cache.", backQuote(it->first));
330330
it->second.table.reset();
331331
expired_tables.erase(it->second.expiration_iterator);
332332
it->second.expiration_iterator = cache_expiration_queue.end();

src/Disks/IDisk.h

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -461,10 +461,9 @@ class IDisk : public Space
461461
virtual void startupImpl(ContextPtr) {}
462462

463463
/// If the state can be changed under the hood and become outdated in memory, perform a reload if necessary.
464-
/// but don't do it more frequently than the specified parameter.
465464
/// Note: for performance reasons, it's allowed to assume that only some subset of changes are possible
466465
/// (those that MergeTree tables can make).
467-
virtual void refresh(UInt64 /* not_sooner_than_milliseconds */)
466+
virtual void refresh()
468467
{
469468
/// The default no-op implementation when the state in memory cannot be out of sync of the actual state.
470469
}

src/Disks/ObjectStorages/DiskObjectStorage.h

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -146,9 +146,9 @@ friend class DiskObjectStorageRemoteMetadataRestoreHelper;
146146

147147
void startupImpl(ContextPtr context) override;
148148

149-
void refresh(UInt64 not_sooner_than_milliseconds) override
149+
void refresh() override
150150
{
151-
metadata_storage->refresh(not_sooner_than_milliseconds);
151+
metadata_storage->refresh();
152152
}
153153

154154
ReservationPtr reserve(UInt64 bytes) override;

src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -766,11 +766,11 @@ std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorageTransaction::writeFile
766766
if (metadata_helper)
767767
{
768768
if (!object_key.hasPrefix())
769-
throw Exception(ErrorCodes::LOGICAL_ERROR, "Metadata helper is not supported with absolute paths");
769+
throw Exception(ErrorCodes::LOGICAL_ERROR, "metadata helper is not supported with absolute paths");
770770

771-
auto revision = ++metadata_helper->revision_counter;
772-
object_attributes =
773-
{
771+
auto revision = metadata_helper->revision_counter + 1;
772+
metadata_helper->revision_counter++;
773+
object_attributes = {
774774
{"path", path}
775775
};
776776

@@ -782,7 +782,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorageTransaction::writeFile
782782
/// Does metadata_storage support empty files without actual blobs in the object_storage?
783783
const bool do_not_write_empty_blob = metadata_storage.supportsEmptyFilesWithoutBlobs();
784784

785-
/// Seems ok
785+
/// seems ok
786786
auto object = StoredObject(object_key.serialize(), path);
787787
std::function<void(size_t count)> create_metadata_callback;
788788

@@ -862,6 +862,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorageTransaction::writeFile
862862
operations_to_execute.emplace_back(std::move(write_operation));
863863
}
864864

865+
865866
auto impl = object_storage.writeObject(
866867
object,
867868
/// We always use mode Rewrite because we simulate append using metadata and different files

src/Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.cpp

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -19,15 +19,15 @@ FlatDirectoryStructureKeyGenerator::FlatDirectoryStructureKeyGenerator(
1919
ObjectStorageKey FlatDirectoryStructureKeyGenerator::generate(const String & path, bool is_directory, const std::optional<String> & key_prefix) const
2020
{
2121
if (is_directory)
22-
chassert(path.empty() || path.ends_with('/'));
22+
chassert(path.ends_with('/'));
2323

24-
const auto fs_path = std::filesystem::path(path);
25-
std::filesystem::path directory = fs_path.parent_path();
24+
const auto p = std::filesystem::path(path);
25+
auto directory = p.parent_path();
2626

2727
std::optional<std::filesystem::path> remote_path;
2828
{
2929
const auto ptr = path_map.lock();
30-
auto res = ptr->getRemotePathInfoIfExists(fs_path);
30+
auto res = ptr->getRemotePathInfoIfExists(p);
3131
if (res)
3232
return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, res->path);
3333

@@ -41,7 +41,7 @@ ObjectStorageKey FlatDirectoryStructureKeyGenerator::generate(const String & pat
4141
: directory;
4242

4343
if (!is_directory)
44-
key /= fs_path.filename();
44+
key /= p.filename();
4545

4646
return ObjectStorageKey::createAsRelative(key_prefix.has_value() ? *key_prefix : storage_key_prefix, key);
4747
}

src/Disks/ObjectStorages/IMetadataStorage.h

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -256,11 +256,10 @@ class IMetadataStorage : private boost::noncopyable
256256
/// This method is overridden for specific metadata implementations in ClickHouse Cloud.
257257
}
258258

259-
/// If the state can be changed under the hood and become outdated in memory, perform a reload if necessary,
260-
/// but don't do it more frequently than the specified parameter.
259+
/// If the state can be changed under the hood and become outdated in memory, perform a reload if necessary.
261260
/// Note: for performance reasons, it's allowed to assume that only some subset of changes are possible
262261
/// (those that MergeTree tables can make).
263-
virtual void refresh(UInt64 /* not_sooner_than_milliseconds */)
262+
virtual void refresh()
264263
{
265264
/// The default no-op implementation when the state in memory cannot be out of sync of the actual state.
266265
}

src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h

Lines changed: 10 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,6 @@ class InMemoryDirectoryPathMap
3939
struct RemotePathInfo
4040
{
4141
std::string path;
42-
std::string etag;
4342
time_t last_modified = 0;
4443
FileNames files;
4544
};
@@ -56,49 +55,28 @@ class InMemoryDirectoryPathMap
5655
return remote_directories.contains(remote_path);
5756
}
5857

59-
bool existsRemotePathUnchanged(const std::string & remote_path, const std::string & etag) const
60-
{
61-
std::lock_guard lock(mutex);
62-
auto it = remote_directories.find(remote_path);
63-
return it != remote_directories.end() && it->second->second.etag == etag;
64-
}
65-
6658
bool existsLocalPath(const std::string & local_path) const
6759
{
6860
std::lock_guard lock(mutex);
6961
return map.contains(local_path);
7062
}
7163

72-
void addOrReplacePath(std::string path, RemotePathInfo info)
64+
auto addPathIfNotExists(std::string path, RemotePathInfo info)
7365
{
7466
std::string remote_path = info.path;
7567
std::lock_guard lock(mutex);
7668

7769
size_t num_files = info.files.size();
70+
auto res = map.emplace(std::move(path), std::move(info));
7871

79-
/// If the logical path already exists, skip it.
80-
if (map.contains(path))
81-
return;
82-
83-
/// If the path was differently named before.
84-
auto old_it = remote_directories.find(info.path);
85-
if (old_it != remote_directories.end())
72+
if (res.second)
8673
{
87-
metric_files.sub(old_it->second->second.files.size());
88-
metric_directories.sub(1);
89-
90-
map.erase(old_it->second->first);
91-
remote_directories.erase(old_it);
74+
remote_directories.emplace(remote_path);
75+
metric_directories.add(1);
76+
metric_files.add(num_files);
9277
}
9378

94-
auto res = map.emplace(std::move(path), std::move(info));
95-
96-
if (!res.second)
97-
return;
98-
99-
remote_directories.emplace(remote_path, &*res.first);
100-
metric_directories.add(1);
101-
metric_files.add(num_files);
79+
return res;
10280
}
10381

10482
bool existsFile(const std::string & local_path) const
@@ -241,14 +219,11 @@ class InMemoryDirectoryPathMap
241219
mutable std::mutex mutex;
242220

243221
/// A mapping from logical filesystem path to the storage path.
244-
using LogicalToPhysicalMap = std::map<std::filesystem::path, RemotePathInfo, PathComparator>;
245-
LogicalToPhysicalMap TSA_GUARDED_BY(mutex) map;
246-
247-
/// A mapping from the storage path to info. Note: std::map has pointers to its nodes stable.
248-
using PhysicalPaths = std::map<std::string, LogicalToPhysicalMap::const_pointer>;
222+
using Map = std::map<std::filesystem::path, RemotePathInfo, PathComparator>;
223+
Map TSA_GUARDED_BY(mutex) map;
249224

250225
/// A set of known storage paths (randomly-assigned names).
251-
PhysicalPaths TSA_GUARDED_BY(mutex) remote_directories;
226+
FileNames TSA_GUARDED_BY(mutex) remote_directories;
252227

253228
CurrentMetrics::Increment metric_directories;
254229
CurrentMetrics::Increment metric_files;

src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp

Lines changed: 5 additions & 48 deletions
Original file line numberDiff line numberDiff line change
@@ -21,21 +21,18 @@ namespace ErrorCodes
2121
extern const int NOT_IMPLEMENTED;
2222
extern const int BAD_ARGUMENTS;
2323
extern const int CANNOT_UNLINK;
24-
extern const int CANNOT_RMDIR;
25-
extern const int READONLY;
2624
}
2725

28-
LocalObjectStorage::LocalObjectStorage(LocalObjectStorageSettings settings_)
29-
: settings(std::move(settings_))
26+
LocalObjectStorage::LocalObjectStorage(String key_prefix_)
27+
: key_prefix(std::move(key_prefix_))
3028
, log(getLogger("LocalObjectStorage"))
3129
{
3230
if (auto block_device_id = tryGetBlockDeviceId("/"); block_device_id.has_value())
3331
description = *block_device_id;
3432
else
3533
description = "/";
3634

37-
if (!settings.read_only)
38-
fs::create_directories(settings.key_prefix);
35+
fs::create_directories(key_prefix);
3936
}
4037

4138
bool LocalObjectStorage::exists(const StoredObject & object) const
@@ -72,8 +69,6 @@ std::unique_ptr<WriteBufferFromFileBase> LocalObjectStorage::writeObject( /// NO
7269
size_t buf_size,
7370
const WriteSettings & /* write_settings */)
7471
{
75-
throwIfReadonly();
76-
7772
if (mode != WriteMode::Rewrite)
7873
throw Exception(ErrorCodes::BAD_ARGUMENTS, "LocalObjectStorage doesn't support append to files");
7974

@@ -88,52 +83,28 @@ std::unique_ptr<WriteBufferFromFileBase> LocalObjectStorage::writeObject( /// NO
8883

8984
void LocalObjectStorage::removeObject(const StoredObject & object) const
9085
{
91-
throwIfReadonly();
92-
9386
/// For local object storage files are actually removed when "metadata" is removed.
9487
if (!exists(object))
9588
return;
9689

9790
if (0 != unlink(object.remote_path.data()))
9891
ErrnoException::throwFromPath(ErrorCodes::CANNOT_UNLINK, object.remote_path, "Cannot unlink file {}", object.remote_path);
99-
100-
/// Remove empty directories.
101-
fs::path dir = fs::path(object.remote_path).parent_path();
102-
fs::path root = fs::weakly_canonical(settings.key_prefix);
103-
while (dir.has_parent_path() && dir.has_relative_path() && dir != root && pathStartsWith(dir, root))
104-
{
105-
LOG_TEST(log, "Removing empty directory {}, has_parent_path: {}, has_relative_path: {}, root: {}, starts with root: {}",
106-
std::string(dir), dir.has_parent_path(), dir.has_relative_path(), std::string(root), pathStartsWith(dir, root));
107-
108-
std::string dir_str = dir;
109-
if (0 != rmdir(dir_str.data()))
110-
{
111-
if (errno == ENOTDIR || errno == ENOTEMPTY)
112-
break;
113-
ErrnoException::throwFromPath(ErrorCodes::CANNOT_RMDIR, dir_str, "Cannot remove directory {}", dir_str);
114-
}
115-
116-
dir = dir.parent_path();
117-
}
11892
}
11993

12094
void LocalObjectStorage::removeObjects(const StoredObjects & objects) const
12195
{
122-
throwIfReadonly();
12396
for (const auto & object : objects)
12497
removeObject(object);
12598
}
12699

127100
void LocalObjectStorage::removeObjectIfExists(const StoredObject & object)
128101
{
129-
throwIfReadonly();
130102
if (exists(object))
131103
removeObject(object);
132104
}
133105

134106
void LocalObjectStorage::removeObjectsIfExist(const StoredObjects & objects)
135107
{
136-
throwIfReadonly();
137108
for (const auto & object : objects)
138109
removeObjectIfExists(object);
139110
}
@@ -142,21 +113,14 @@ ObjectMetadata LocalObjectStorage::getObjectMetadata(const std::string & path) c
142113
{
143114
ObjectMetadata object_metadata;
144115
LOG_TEST(log, "Getting metadata for path: {}", path);
145-
146-
auto time = fs::last_write_time(path);
147-
148116
object_metadata.size_bytes = fs::file_size(path);
149-
object_metadata.etag = std::to_string(std::chrono::duration_cast<std::chrono::nanoseconds>(time.time_since_epoch()).count());
150117
object_metadata.last_modified = Poco::Timestamp::fromEpochTime(
151-
std::chrono::duration_cast<std::chrono::seconds>(time.time_since_epoch()).count());
118+
std::chrono::duration_cast<std::chrono::seconds>(fs::last_write_time(path).time_since_epoch()).count());
152119
return object_metadata;
153120
}
154121

155122
void LocalObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t/* max_keys */) const
156123
{
157-
if (!fs::is_directory(path))
158-
return;
159-
160124
for (const auto & entry : fs::directory_iterator(path))
161125
{
162126
if (entry.is_directory())
@@ -183,7 +147,6 @@ void LocalObjectStorage::copyObject( // NOLINT
183147
const WriteSettings & write_settings,
184148
std::optional<ObjectAttributes> /* object_to_attributes */)
185149
{
186-
throwIfReadonly();
187150
auto in = readObject(object_from, read_settings);
188151
auto out = writeObject(object_to, WriteMode::Rewrite, /* attributes= */ {}, /* buf_size= */ DBMS_DEFAULT_BUFFER_SIZE, write_settings);
189152
copyData(*in, *out);
@@ -198,12 +161,6 @@ void LocalObjectStorage::startup()
198161
{
199162
}
200163

201-
void LocalObjectStorage::throwIfReadonly() const
202-
{
203-
if (settings.read_only)
204-
throw Exception(ErrorCodes::READONLY, "Local object storage `{}` is readonly", getName());
205-
}
206-
207164
std::unique_ptr<IObjectStorage> LocalObjectStorage::cloneObjectStorage(
208165
const std::string & /* new_namespace */,
209166
const Poco::Util::AbstractConfiguration & /* config */,
@@ -216,7 +173,7 @@ ObjectStorageKey
216173
LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional<std::string> & /* key_prefix */) const
217174
{
218175
constexpr size_t key_name_total_size = 32;
219-
return ObjectStorageKey::createAsRelative(settings.key_prefix, getRandomASCIIString(key_name_total_size));
176+
return ObjectStorageKey::createAsRelative(key_prefix, getRandomASCIIString(key_name_total_size));
220177
}
221178

222179
}

src/Disks/ObjectStorages/Local/LocalObjectStorage.h

Lines changed: 5 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -4,42 +4,28 @@
44

55
#include <Disks/ObjectStorages/IObjectStorage.h>
66

7-
87
namespace Poco
98
{
10-
class Logger;
9+
class Logger;
1110
}
1211

1312
namespace DB
1413
{
1514

16-
struct LocalObjectStorageSettings
17-
{
18-
LocalObjectStorageSettings(String key_prefix_, bool read_only_)
19-
: key_prefix(key_prefix_), read_only(read_only_)
20-
{
21-
}
22-
23-
String key_prefix;
24-
bool read_only = false;
25-
};
26-
2715
/// Treat local disk as an object storage (for interface compatibility).
2816
class LocalObjectStorage : public IObjectStorage
2917
{
3018
public:
31-
explicit LocalObjectStorage(LocalObjectStorageSettings settings_);
19+
explicit LocalObjectStorage(String key_prefix_);
3220

3321
std::string getName() const override { return "LocalObjectStorage"; }
3422

3523
ObjectStorageType getType() const override { return ObjectStorageType::Local; }
3624

37-
std::string getCommonKeyPrefix() const override { return settings.key_prefix; }
25+
std::string getCommonKeyPrefix() const override { return key_prefix; }
3826

3927
std::string getDescription() const override { return description; }
4028

41-
bool isReadOnly() const override { return settings.read_only; }
42-
4329
bool exists(const StoredObject & object) const override;
4430

4531
std::unique_ptr<ReadBufferFromFileBase> readObject( /// NOLINT
@@ -95,11 +81,10 @@ class LocalObjectStorage : public IObjectStorage
9581

9682
private:
9783
void removeObject(const StoredObject & object) const;
98-
void removeObjects(const StoredObjects & objects) const;
9984

100-
void throwIfReadonly() const;
85+
void removeObjects(const StoredObjects & objects) const;
10186

102-
LocalObjectStorageSettings settings;
87+
String key_prefix;
10388
LoggerPtr log;
10489
std::string description;
10590
};

0 commit comments

Comments
 (0)