This is an automated email from the ASF dual-hosted git repository.
alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git
The following commit(s) were added to refs/heads/master by this push:
new 74a1d77 [util] change return type of Cache::Allocate()
74a1d77 is described below
commit 74a1d7706d99db2d9a14ed5d7c64afbcef853b20
Author: Alexey Serbin <alexey@apache.org>
AuthorDate: Wed Mar 27 18:23:09 2019 -0700
[util] change return type of Cache::Allocate()
This patch changes the signature of Cache::Allocate() and
Cache::Insert() methods to be explicit on the ownership of the memory
allocated behind Cache::PendingHandle.
Change-Id: I2b0fa72060456b167b61a816f8abb9cd4c8f56b6
Reviewed-on: http://gerrit.cloudera.org:8080/12876
Reviewed-by: Adar Dembo <adar@cloudera.com>
Tested-by: Kudu Jenkins
---
src/kudu/cfile/block_cache.cc | 6 ++---
src/kudu/cfile/block_cache.h | 29 +++++++++++-----------
src/kudu/codegen/code_cache.cc | 11 +++++----
src/kudu/tools/kudu-tool-test.cc | 2 +-
src/kudu/util/cache-bench.cc | 9 ++++---
src/kudu/util/cache-test.cc | 9 ++++---
src/kudu/util/cache.cc | 23 ++++++++++--------
src/kudu/util/cache.h | 52 ++++++++++++++++++++++++++--------------
src/kudu/util/file_cache.cc | 10 ++++----
src/kudu/util/nvm_cache.cc | 23 +++++++++---------
10 files changed, 96 insertions(+), 78 deletions(-)
diff --git a/src/kudu/cfile/block_cache.cc b/src/kudu/cfile/block_cache.cc
index bddaa08..b5b5c2c 100644
--- a/src/kudu/cfile/block_cache.cc
+++ b/src/kudu/cfile/block_cache.cc
@@ -149,7 +149,7 @@ BlockCache::PendingEntry BlockCache::Allocate(const CacheKey& key,
size_t block_
bool BlockCache::Lookup(const CacheKey& key, Cache::CacheBehavior behavior,
BlockCacheHandle *handle) {
- Cache::Handle *h = cache_->Lookup(Slice(reinterpret_cast<const uint8_t*>(&key),
+ Cache::Handle* h = cache_->Lookup(Slice(reinterpret_cast<const uint8_t*>(&key),
sizeof(key)), behavior);
if (h != nullptr) {
handle->SetHandle(cache_.get(), h);
@@ -158,8 +158,8 @@ bool BlockCache::Lookup(const CacheKey& key, Cache::CacheBehavior
behavior,
}
void BlockCache::Insert(BlockCache::PendingEntry* entry, BlockCacheHandle* inserted) {
- Cache::Handle *h = cache_->Insert(entry->handle_, /* eviction_callback= */ nullptr);
- entry->handle_ = nullptr;
+ Cache::Handle* h = cache_->Insert(std::move(entry->handle_),
+ /* eviction_callback= */ nullptr);
inserted->SetHandle(cache_.get(), h);
}
diff --git a/src/kudu/cfile/block_cache.h b/src/kudu/cfile/block_cache.h
index 381758b..ba700c1 100644
--- a/src/kudu/cfile/block_cache.h
+++ b/src/kudu/cfile/block_cache.h
@@ -81,9 +81,13 @@ class BlockCache {
// cache insertion path.
class PendingEntry {
public:
- PendingEntry() : cache_(nullptr), handle_(nullptr) {}
- PendingEntry(Cache* cache, Cache::PendingHandle* handle)
- : cache_(cache), handle_(handle) {
+ PendingEntry()
+ : cache_(nullptr),
+ handle_(Cache::UniquePendingHandle(nullptr,
+ Cache::PendingHandleDeleter(nullptr))) {
+ }
+ PendingEntry(Cache* cache, Cache::UniquePendingHandle handle)
+ : cache_(cache), handle_(std::move(handle)) {
}
PendingEntry(PendingEntry&& other) noexcept : PendingEntry() {
*this = std::move(other);
@@ -102,22 +106,22 @@ class BlockCache {
// Return true if this is a valid pending entry.
bool valid() const {
- return handle_ != nullptr;
+ return static_cast<bool>(handle_);
}
// Return the pointer into which the value should be written.
uint8_t* val_ptr() {
- return cache_->MutableValue(handle_);
+ return cache_->MutableValue(handle_.get());
}
private:
friend class BlockCache;
Cache* cache_;
- Cache::PendingHandle* handle_;
+ Cache::UniquePendingHandle handle_;
};
- static BlockCache *GetSingleton() {
+ static BlockCache* GetSingleton() {
return Singleton<BlockCache>::get();
}
@@ -230,19 +234,14 @@ class BlockCacheHandle {
inline BlockCache::PendingEntry& BlockCache::PendingEntry::operator=(
BlockCache::PendingEntry&& other) noexcept {
reset();
- cache_ = other.cache_;
- handle_ = other.handle_;
- other.cache_ = nullptr;
- other.handle_ = nullptr;
+ std::swap(cache_, other.cache_);
+ handle_ = std::move(other.handle_);
return *this;
}
inline void BlockCache::PendingEntry::reset() {
- if (cache_ && handle_) {
- cache_->Free(handle_);
- }
cache_ = nullptr;
- handle_ = nullptr;
+ handle_.reset();
}
} // namespace cfile
diff --git a/src/kudu/codegen/code_cache.cc b/src/kudu/codegen/code_cache.cc
index 1f07e5d..393659c 100644
--- a/src/kudu/codegen/code_cache.cc
+++ b/src/kudu/codegen/code_cache.cc
@@ -18,6 +18,8 @@
#include "kudu/codegen/code_cache.h"
#include <cstring>
+#include <memory>
+#include <utility>
#include <glog/logging.h>
@@ -76,16 +78,17 @@ Status CodeCache::AddEntry(const scoped_refptr<JITWrapper>&
value) {
// We CHECK_NOTNULL because this is always a DRAM-based cache, and if allocation
// failed, we'd just crash the process.
- Cache::PendingHandle* pending = CHECK_NOTNULL(
- cache_->Allocate(Slice(key), val_len, /*charge = */1));
- memcpy(cache_->MutableValue(pending), &val, val_len);
+ auto pending(cache_->Allocate(Slice(key), val_len, /*charge = */1));
+ CHECK(pending);
+ memcpy(cache_->MutableValue(pending.get()), &val, val_len);
// Because Cache only accepts void* values, we store just the JITWrapper*
// and increase its ref count.
value->AddRef();
// Insert into cache and release the handle (we have a local copy of a refptr).
- Cache::Handle* inserted = DCHECK_NOTNULL(cache_->Insert(pending, eviction_callback_.get()));
+ Cache::Handle* inserted = DCHECK_NOTNULL(cache_->Insert(
+ std::move(pending), eviction_callback_.get()));
cache_->Release(inserted);
return Status::OK();
}
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 871b2fa..8ed3c3b 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -522,7 +522,7 @@ class ToolTest : public KuduTest {
&stdout));
// Check total count.
- int64_t total = max(args.max_value - args.min_value + 1, 0L);
+ int64_t total = max<int64_t>(args.max_value - args.min_value + 1, 0);
if (args.mode == TableCopyMode::COPY_SCHEMA_ONLY) {
ASSERT_STR_NOT_CONTAINS(stdout, "Total count ");
} else {
diff --git a/src/kudu/util/cache-bench.cc b/src/kudu/util/cache-bench.cc
index 4a8e2e5..6e95327 100644
--- a/src/kudu/util/cache-bench.cc
+++ b/src/kudu/util/cache-bench.cc
@@ -15,10 +15,9 @@
// specific language governing permissions and limitations
// under the License.
-#include <string.h>
-
#include <atomic>
#include <cstdint>
+#include <cstring>
#include <memory>
#include <ostream>
#include <string>
@@ -120,9 +119,9 @@ class CacheBench : public KuduTest,
if (h) {
hits++;
} else {
- Cache::PendingHandle* ph = cache_->Allocate(
- key_slice, /* val_len=*/kEntrySize, /* charge=*/kEntrySize);
- h = cache_->Insert(ph, nullptr);
+ auto ph(cache_->Allocate(
+ key_slice, /* val_len=*/kEntrySize, /* charge=*/kEntrySize));
+ h = cache_->Insert(std::move(ph), nullptr);
}
cache_->Release(h);
diff --git a/src/kudu/util/cache-test.cc b/src/kudu/util/cache-test.cc
index 5c2b526..a2c401d 100644
--- a/src/kudu/util/cache-test.cc
+++ b/src/kudu/util/cache-test.cc
@@ -91,11 +91,10 @@ class CacheBaseTest : public KuduTest,
void Insert(int key, int value, int charge = 1) {
std::string key_str = EncodeInt(key);
std::string val_str = EncodeInt(value);
- Cache::PendingHandle* handle = CHECK_NOTNULL(
- cache_->Allocate(key_str, val_str.size(), charge));
- memcpy(cache_->MutableValue(handle), val_str.data(), val_str.size());
-
- cache_->Release(cache_->Insert(handle, this));
+ auto handle(cache_->Allocate(key_str, val_str.size(), charge));
+ CHECK(handle);
+ memcpy(cache_->MutableValue(handle.get()), val_str.data(), val_str.size());
+ cache_->Release(cache_->Insert(std::move(handle), this));
}
void Erase(int key) {
diff --git a/src/kudu/util/cache.cc b/src/kudu/util/cache.cc
index eb01289..44d0845 100644
--- a/src/kudu/util/cache.cc
+++ b/src/kudu/util/cache.cc
@@ -540,9 +540,9 @@ class ShardedCache : public Cache {
STLDeleteElements(&shards_);
}
- Handle* Insert(PendingHandle* handle,
+ Handle* Insert(UniquePendingHandle handle,
Cache::EvictionCallback* eviction_callback) override {
- RLHandle* h = reinterpret_cast<RLHandle*>(DCHECK_NOTNULL(handle));
+ RLHandle* h = reinterpret_cast<RLHandle*>(DCHECK_NOTNULL(handle.release()));
return shards_[Shard(h->hash)]->Insert(h, eviction_callback);
}
Handle* Lookup(const Slice& key, CacheBehavior caching) override {
@@ -576,23 +576,26 @@ class ShardedCache : public Cache {
}
}
- PendingHandle* Allocate(Slice key, int val_len, int charge) override {
+ UniquePendingHandle Allocate(Slice key, int val_len, int charge) override {
int key_len = key.size();
DCHECK_GE(key_len, 0);
DCHECK_GE(val_len, 0);
int key_len_padded = KUDU_ALIGN_UP(key_len, sizeof(void*));
- uint8_t* buf = new uint8_t[sizeof(RLHandle)
- + key_len_padded + val_len // the kv_data VLA data
- - 1 // (the VLA has a 1-byte placeholder)
- ];
- RLHandle* handle = reinterpret_cast<RLHandle*>(buf);
+ UniquePendingHandle h(reinterpret_cast<PendingHandle*>(
+ new uint8_t[sizeof(RLHandle)
+ + key_len_padded + val_len // the kv_data VLA data
+ - 1 // (the VLA has a 1-byte placeholder)
+ ]),
+ PendingHandleDeleter(this));
+ RLHandle* handle = reinterpret_cast<RLHandle*>(h.get());
handle->key_length = key_len;
handle->val_length = val_len;
- handle->charge = (charge == kAutomaticCharge) ? kudu_malloc_usable_size(buf) : charge;
+ handle->charge = (charge == kAutomaticCharge) ? kudu_malloc_usable_size(h.get())
+ : charge;
handle->hash = HashSlice(key);
memcpy(handle->kv_data, key.data(), key_len);
- return reinterpret_cast<PendingHandle*>(handle);
+ return h;
}
void Free(PendingHandle* h) override {
diff --git a/src/kudu/util/cache.h b/src/kudu/util/cache.h
index 7ba1ae5..a35aa3d 100644
--- a/src/kudu/util/cache.h
+++ b/src/kudu/util/cache.h
@@ -103,6 +103,24 @@ class Cache {
};
typedef std::unique_ptr<Handle, HandleDeleter> UniqueHandle;
+ // Opaque handle to an entry which is being prepared to be added to the cache.
+ struct PendingHandle { };
+
+ class PendingHandleDeleter {
+ public:
+ explicit PendingHandleDeleter(Cache* c)
+ : c_(c) {
+ }
+
+ void operator()(Cache::PendingHandle* h) const {
+ c_->Free(h);
+ }
+
+ private:
+ Cache* c_;
+ };
+ typedef std::unique_ptr<PendingHandle, PendingHandleDeleter> UniquePendingHandle;
+
// Passing EXPECT_IN_CACHE will increment the hit/miss metrics that track the number of
times
// blocks were requested that the users were hoping to get the block from the cache, along
with
// with the basic metrics.
@@ -150,20 +168,15 @@ class Cache {
//
// For example:
//
- // PendingHandle* ph = cache_->Allocate("my entry", value_size, charge);
- // if (!ReadDataFromDisk(cache_->MutableValue(ph)).ok()) {
- // cache_->Free(ph);
+ // auto ph(cache_->Allocate("my entry", value_size, charge));
+ // if (!ReadDataFromDisk(cache_->MutableValue(ph.get())).ok()) {
// ... error handling ...
// return;
// }
- // Handle* h = cache_->Insert(ph, my_eviction_callback);
+ // Handle* h = cache_->Insert(std::move(ph), my_eviction_callback);
// ...
// cache_->Release(h);
- // Opaque handle to an entry which is being prepared to be added to
- // the cache.
- struct PendingHandle { };
-
// Indicates that the charge of an item in the cache should be calculated
// based on its memory consumption.
static constexpr int kAutomaticCharge = -1;
@@ -177,21 +190,23 @@ class Cache {
// If 'charge' is not 'kAutomaticCharge', then the cache capacity will be charged
// the explicit amount. This is useful when caching items that are small but need to
// maintain a bounded count (eg file descriptors) rather than caring about their actual
- // memory usage.
+ // memory usage. It is also useful when caching items for whom calculating
+ // memory usage is a complex affair (i.e. items containing pointers to
+ // additional heap allocations).
//
// Note that this does not mutate the cache itself: lookups will
// not be able to find the provided key until it is inserted.
//
- // It is possible that this will return NULL if the cache is above its capacity
- // and eviction fails to free up enough space for the requested allocation.
+ // It is possible that this will return a nullptr wrapped in a std::unique_ptr
+ // if the cache is above its capacity and eviction fails to free up enough
+ // space for the requested allocation.
//
- // NOTE: the returned memory is not automatically freed by the cache: the
- // caller must either free it using Free(), or insert it using Insert().
- virtual PendingHandle* Allocate(Slice key, int val_len, int charge) = 0;
+ // The returned handle owns the allocated memory.
+ virtual UniquePendingHandle Allocate(Slice key, int val_len, int charge) = 0;
- // Default 'charge' should be kAutomaticCharge.
- // (default arguments on virtual functions are prohibited)
- PendingHandle* Allocate(Slice key, int val_len) {
+ // Default 'charge' should be kAutomaticCharge
+ // (default arguments on virtual functions are prohibited).
+ UniquePendingHandle Allocate(Slice key, int val_len) {
return Allocate(key, val_len, kAutomaticCharge);
}
@@ -209,7 +224,8 @@ class Cache {
//
// If 'eviction_callback' is non-NULL, then it will be called when the
// entry is later evicted or when the cache shuts down.
- virtual Handle* Insert(PendingHandle* pending, EvictionCallback* eviction_callback) = 0;
+ virtual Handle* Insert(UniquePendingHandle pending,
+ EvictionCallback* eviction_callback) = 0;
// Free 'ptr', which must have been previously allocated using 'Allocate'.
virtual void Free(PendingHandle* ptr) = 0;
diff --git a/src/kudu/util/file_cache.cc b/src/kudu/util/file_cache.cc
index b9294d0..effe23c 100644
--- a/src/kudu/util/file_cache.cc
+++ b/src/kudu/util/file_cache.cc
@@ -122,13 +122,11 @@ class BaseDescriptor {
// The allocated charge is always one byte. This is incorrect with respect
// to memory tracking, but it's necessary if the cache capacity is to be
// equivalent to the max number of fds.
- Cache::PendingHandle* pending = CHECK_NOTNULL(cache()->Allocate(
- filename(), sizeof(file_ptr), 1));
- memcpy(cache()->MutableValue(pending),
- &file_ptr,
- sizeof(file_ptr));
+ auto pending(cache()->Allocate(filename(), sizeof(file_ptr), 1));
+ CHECK(pending);
+ memcpy(cache()->MutableValue(pending.get()), &file_ptr, sizeof(file_ptr));
return ScopedOpenedDescriptor<FileType>(this, Cache::UniqueHandle(
- cache()->Insert(pending, file_cache_->eviction_cb_.get()),
+ cache()->Insert(std::move(pending), file_cache_->eviction_cb_.get()),
Cache::HandleDeleter(cache())));
}
diff --git a/src/kudu/util/nvm_cache.cc b/src/kudu/util/nvm_cache.cc
index bf450b9..20d086b 100644
--- a/src/kudu/util/nvm_cache.cc
+++ b/src/kudu/util/nvm_cache.cc
@@ -494,9 +494,9 @@ class ShardedLRUCache : public Cache {
vmem_delete(vmp_);
}
- virtual Handle* Insert(PendingHandle* handle,
+ virtual Handle* Insert(UniquePendingHandle handle,
Cache::EvictionCallback* eviction_callback) OVERRIDE {
- LRUHandle* h = reinterpret_cast<LRUHandle*>(DCHECK_NOTNULL(handle));
+ LRUHandle* h = reinterpret_cast<LRUHandle*>(DCHECK_NOTNULL(handle.release()));
return shards_[Shard(h->hash)]->Insert(h, eviction_callback);
}
virtual Handle* Lookup(const Slice& key, CacheBehavior caching) OVERRIDE {
@@ -524,20 +524,21 @@ class ShardedLRUCache : public Cache {
cache->SetMetrics(metrics_.get());
}
}
- virtual PendingHandle* Allocate(Slice key, int val_len, int charge) OVERRIDE {
+ virtual UniquePendingHandle Allocate(Slice key, int val_len, int charge) OVERRIDE {
int key_len = key.size();
DCHECK_GE(key_len, 0);
DCHECK_GE(val_len, 0);
- LRUHandle* handle = nullptr;
// Try allocating from each of the shards -- if vmem is tight,
// this can cause eviction, so we might have better luck in different
// shards.
for (NvmLRUCache* cache : shards_) {
- uint8_t* buf = static_cast<uint8_t*>(cache->AllocateAndRetry(
- sizeof(LRUHandle) + key_len + val_len));
- if (buf) {
- handle = reinterpret_cast<LRUHandle*>(buf);
+ UniquePendingHandle ph(static_cast<PendingHandle*>(
+ cache->AllocateAndRetry(sizeof(LRUHandle) + key_len + val_len)),
+ Cache::PendingHandleDeleter(this));
+ if (ph) {
+ LRUHandle* handle = reinterpret_cast<LRUHandle*>(ph.get());
+ uint8_t* buf = reinterpret_cast<uint8_t*>(ph.get());
handle->kv_data = &buf[sizeof(LRUHandle)];
handle->val_length = val_len;
handle->key_length = key_len;
@@ -545,11 +546,11 @@ class ShardedLRUCache : public Cache {
vmem_malloc_usable_size(vmp_, buf) : charge;
handle->hash = HashSlice(key);
memcpy(handle->kv_data, key.data(), key.size());
- return reinterpret_cast<PendingHandle*>(handle);
+ return ph;
}
}
- // TODO: increment a metric here on allocation failure.
- return nullptr;
+ // TODO(unknown): increment a metric here on allocation failure.
+ return UniquePendingHandle(nullptr, Cache::PendingHandleDeleter(this));
}
virtual void Free(PendingHandle* ph) OVERRIDE {
|