Skip to content

Commit

Permalink
Merge pull request #5044 from Lazin/feature/improve-si-cache
Browse files Browse the repository at this point in the history
cloud_storage: Improve SI cache
  • Loading branch information
Lazin committed Jun 24, 2022
2 parents 68b545b + 11dea2b commit 0d9e424
Show file tree
Hide file tree
Showing 16 changed files with 585 additions and 93 deletions.
1 change: 1 addition & 0 deletions src/v/cloud_storage/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ v_cc_library(
NAME cloud_storage
SRCS
cache_service.cc
access_time_tracker.cc
cache_probe.cc
topic_manifest.cc
partition_manifest.cc
Expand Down
116 changes: 116 additions & 0 deletions src/v/cloud_storage/access_time_tracker.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* Copyright 2022 Redpanda Data, Inc.
*
* Licensed as a Redpanda Enterprise file under the Redpanda Community
* License (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md
*/

#include "cloud_storage/access_time_tracker.h"

#include "serde/serde.h"
#include "units.h"

#include <seastar/core/coroutine.hh>
#include <seastar/core/smp.hh>

#include <absl/container/btree_map.h>

#include <exception>
#include <variant>

namespace absl {

template<class Key, class Value>
void write(iobuf& out, const btree_map<Key, Value>& btree) {
using serde::write;
write(out, static_cast<uint64_t>(btree.size()));
for (auto it : btree) {
write(out, it.first);
write(out, it.second);
}
}

template<class Key, class Value>
void read_nested(
iobuf_parser& in,
btree_map<Key, Value>& btree,
size_t const bytes_left_limit) {
using serde::read_nested;
uint64_t sz;
read_nested(in, sz, bytes_left_limit);
for (auto i = 0UL; i < sz; i++) {
Key key;
Value value;
read_nested(in, key, bytes_left_limit);
read_nested(in, value, bytes_left_limit);
btree.insert({key, value});
}
}

} // namespace absl

namespace cloud_storage {

void access_time_tracker::add_timestamp(
std::string_view key, std::chrono::system_clock::time_point ts) {
uint32_t seconds = std::chrono::time_point_cast<std::chrono::seconds>(ts)
.time_since_epoch()
.count();
uint32_t hash = xxhash_32(key.data(), key.size());
_table.data[hash] = seconds;
_dirty = true;
}

void access_time_tracker::remove_timestamp(std::string_view key) noexcept {
try {
uint32_t hash = xxhash_32(key.data(), key.size());
_table.data.erase(hash);
_dirty = true;
} catch (...) {
vassert(
false,
"Can't remove key {} from access_time_tracker, exception: {}",
key,
std::current_exception());
}
}

void access_time_tracker::remove_others(const access_time_tracker& t) {
table_t tmp;
for (auto it : _table.data) {
if (t._table.data.contains(it.first)) {
tmp.data.insert(it);
}
}
_table = std::move(tmp);
}

std::optional<std::chrono::system_clock::time_point>
access_time_tracker::estimate_timestamp(std::string_view key) const {
uint32_t hash = xxhash_32(key.data(), key.size());
auto it = _table.data.find(hash);
if (it == _table.data.end()) {
return std::nullopt;
}
auto seconds = std::chrono::seconds(it->second);
std::chrono::system_clock::time_point ts(seconds);
return ts;
}

iobuf access_time_tracker::to_iobuf() {
_dirty = false;
return serde::to_iobuf(_table);
}

void access_time_tracker::from_iobuf(iobuf b) {
iobuf_parser parser(std::move(b));
_table = serde::read<table_t>(parser);
_dirty = false;
}

bool access_time_tracker::is_dirty() const { return _dirty; }

} // namespace cloud_storage
72 changes: 72 additions & 0 deletions src/v/cloud_storage/access_time_tracker.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* Copyright 2022 Redpanda Data, Inc.
*
* Licensed as a Redpanda Enterprise file under the Redpanda Community
* License (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md
*/

#pragma once

#include "bytes/iobuf.h"
#include "hashing/xx.h"
#include "seastarx.h"
#include "serde/envelope.h"

#include <seastar/core/future.hh>

#include <absl/container/btree_map.h>

#include <chrono>
#include <string_view>

namespace cloud_storage {

/// Access time tracker maintains map from filename hash to
/// the timestamp that represents the time when the file was
/// accessed last.
///
/// It is possible to have conflicts. In case of conflict
/// 'add_timestamp' method will overwrite another key. For that
/// key we will observe larger access time. When one of the
/// conflicted entries will be deleted another will be deleted
/// as well. This is OK because the code in the
/// 'cloud_storage/cache_service' is ready for that.
class access_time_tracker {
using timestamp_t = uint32_t;
struct table_t
: serde::envelope<table_t, serde::version<0>, serde::compat_version<0>> {
absl::btree_map<uint32_t, timestamp_t> data;
};

public:
/// Add access time to the container.
void add_timestamp(
std::string_view key, std::chrono::system_clock::time_point ts);

/// Remove key from the container.
void remove_timestamp(std::string_view) noexcept;

/// Return access time estimate (it can differ if there is a conflict
/// on file name hash).
std::optional<std::chrono::system_clock::time_point>
estimate_timestamp(std::string_view key) const;

iobuf to_iobuf();
void from_iobuf(iobuf b);

/// Returns true if tracker has new data which wasn't serialized
/// to disk.
bool is_dirty() const;

/// Remove every key which isn't present in 't'
void remove_others(const access_time_tracker& t);

private:
table_t _table;
bool _dirty{false};
};

} // namespace cloud_storage
Loading

0 comments on commit 0d9e424

Please sign in to comment.