You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kvrocks.apache.org by hu...@apache.org on 2022/10/11 06:52:59 UTC

[incubator-kvrocks] branch unstable updated: Support the disk usage command (#882)

This is an automated email from the ASF dual-hosted git repository.

hulk pushed a commit to branch unstable
in repository https://gitbox.apache.org/repos/asf/incubator-kvrocks.git


The following commit(s) were added to refs/heads/unstable by this push:
     new 9dbc8c7  Support the disk usage command (#882)
9dbc8c7 is described below

commit 9dbc8c7e2e919689ec5d87db8126c64c242c722b
Author: Ruixiang Tan <81...@qq.com>
AuthorDate: Tue Oct 11 14:52:54 2022 +0800

    Support the disk usage command (#882)
    
    Like Redis key memory usage, we also want to know how many disk bytes were occupied by some key,
    and it's possible since the RocksDB give the way to get approximate bytes by the key range.  After this PR,
    users can use `disk usage user_key` command to get the disk usage of the key.
---
 src/redis_cmd.cc                          |  26 ++++
 src/redis_disk.cc                         | 160 ++++++++++++++++++++++
 src/redis_disk.h                          |  52 ++++++++
 tests/cppunit/disk_test.cc                | 212 ++++++++++++++++++++++++++++++
 tests/gocase/unit/command/command_test.go |   7 +
 tests/gocase/unit/disk/disk_test.go       | 145 ++++++++++++++++++++
 6 files changed, 602 insertions(+)

diff --git a/src/redis_cmd.cc b/src/redis_cmd.cc
index 08fcf71..cf2e99d 100644
--- a/src/redis_cmd.cc
+++ b/src/redis_cmd.cc
@@ -41,6 +41,7 @@
 #include "redis_geo.h"
 #include "redis_hash.h"
 #include "redis_list.h"
+#include "redis_disk.h"
 #include "redis_pubsub.h"
 #include "redis_reply.h"
 #include "redis_set.h"
@@ -3476,6 +3477,30 @@ class CommandInfo : public Commander {
   }
 };
 
+class CommandDisk : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    std::string opname = Util::ToLower(args[1]);
+    if (opname != "usage")
+      return Status(Status::RedisInvalidCmd, "Unknown operation");
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    RedisType type;
+    Redis::Disk disk_db(svr->storage_, conn->GetNamespace());
+    rocksdb::Status s = disk_db.Type(args_[2], &type);
+    if (!s.ok()) return Status(Status::RedisExecErr, s.ToString());
+
+    uint64_t result = 0;
+    s = disk_db.GetKeySize(args_[2], type, &result);
+
+    if (!s.ok()) return Status(Status::RedisExecErr, s.ToString());
+    *output = Redis::Integer(result);
+    return Status::OK();
+  }
+};
+
 class CommandRole : public Commander {
  public:
   Status Execute(Server *svr, Connection *conn, std::string *output) override {
@@ -5874,6 +5899,7 @@ CommandAttributes redisCommandTable[] = {
     ADD_CMD("debug", -2, "read-only exclusive", 0, 0, 0, CommandDebug),
     ADD_CMD("command", -1, "read-only", 0, 0, 0, CommandCommand),
     ADD_CMD("echo", 2, "read-only", 0, 0, 0, CommandEcho),
+    ADD_CMD("disk", 3, "read-only", 0, 0, 0, CommandDisk),
     ADD_CMD("hello", -1,  "read-only ok-loading", 0, 0, 0, CommandHello),
 
     ADD_CMD("ttl", 2, "read-only", 1, 1, 1, CommandTTL),
diff --git a/src/redis_disk.cc b/src/redis_disk.cc
new file mode 100644
index 0000000..02b71d7
--- /dev/null
+++ b/src/redis_disk.cc
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <vector>
+#include <memory>
+#include <utility>
+#include <algorithm>
+#include <string>
+
+#include "db_util.h"
+#include "redis_metadata.h"
+#include "redis_sortedint.h"
+#include "rocksdb/status.h"
+#include "redis_zset.h"
+#include "redis_bitmap.h"
+#include "redis_disk.h"
+#include "status.h"
+
+namespace Redis {
+
+rocksdb::Status Disk::GetApproximateSizes(const Metadata &metadata, const Slice &ns_key,
+                                          rocksdb::ColumnFamilyHandle *column_family,
+                                          uint64_t *key_size, Slice subkeyleft,
+                                          Slice subkeyright) {
+  std::string prefix_key, next_version_prefix_key;
+  InternalKey(ns_key, subkeyleft, metadata.version,
+              storage_->IsSlotIdEncoded()).Encode(&prefix_key);
+  InternalKey(ns_key, subkeyright, metadata.version + 1,
+              storage_->IsSlotIdEncoded()).Encode(&next_version_prefix_key);
+  auto key_range = rocksdb::Range(prefix_key, next_version_prefix_key);
+  uint64_t tmp_size = 0;
+  rocksdb::Status s = db_->GetApproximateSizes(option_, column_family,
+                                               &key_range, 1, &tmp_size);
+  if (!s.ok()) return s;
+  *key_size += tmp_size;
+  return rocksdb::Status::OK();
+}
+
+rocksdb::Status Disk::GetKeySize(const Slice &user_key, RedisType type, uint64_t *key_size) {
+  *key_size = 0;
+  std::string ns_key;
+  AppendNamespacePrefix(user_key, &ns_key);
+  switch (type) {
+    case RedisType::kRedisString:
+      return GetStringSize(ns_key, key_size);
+    case RedisType::kRedisHash:
+      return GetHashSize(ns_key, key_size);
+    case RedisType::kRedisBitmap:
+      return GetBitmapSize(ns_key, key_size);
+    case RedisType::kRedisList:
+      return GetListSize(ns_key, key_size);
+    case RedisType::kRedisSet:
+      return GetSetSize(ns_key, key_size);
+    case RedisType::kRedisSortedint:
+      return GetSortedintSize(ns_key, key_size);
+    case RedisType::kRedisZSet:
+      return GetZsetSize(ns_key, key_size);
+    case RedisType::kRedisStream:
+      return GetStreamSize(ns_key, key_size);
+    case RedisType::kRedisNone:
+      return rocksdb::Status::NotFound("Not found ", user_key);
+  }
+}
+
+rocksdb::Status Disk::GetStringSize(const Slice &ns_key, uint64_t *key_size) {
+  auto key_range = rocksdb::Range(Slice(ns_key), Slice(ns_key.ToString() + static_cast<char>(0)));
+  return db_->GetApproximateSizes(option_, metadata_cf_handle_, &key_range, 1, key_size);
+}
+
+rocksdb::Status Disk::GetHashSize(const Slice &ns_key, uint64_t *key_size) {
+  HashMetadata metadata(false);
+  rocksdb::Status s = Database::GetMetadata(kRedisHash, ns_key, &metadata);
+  if (!s.ok()) return s.IsNotFound() ? rocksdb::Status::OK() : s;
+  return GetApproximateSizes(metadata, ns_key,
+                             storage_->GetCFHandle(Engine::kSubkeyColumnFamilyName),
+                             key_size);
+}
+
+rocksdb::Status Disk::GetSetSize(const Slice &ns_key, uint64_t *key_size) {
+  SetMetadata metadata(false);
+  rocksdb::Status s = Database::GetMetadata(kRedisSet, ns_key, &metadata);
+  if (!s.ok()) return s.IsNotFound() ? rocksdb::Status::OK() : s;
+  return GetApproximateSizes(metadata, ns_key,
+                             storage_->GetCFHandle(Engine::kSubkeyColumnFamilyName),
+                             key_size);
+}
+
+rocksdb::Status Disk::GetListSize(const Slice &ns_key, uint64_t *key_size) {
+  ListMetadata metadata(false);
+  rocksdb::Status s = Database::GetMetadata(kRedisList, ns_key, &metadata);
+  if (!s.ok()) return s.IsNotFound() ? rocksdb::Status::OK() : s;
+  std::string buf;
+  PutFixed64(&buf, metadata.head);
+  return GetApproximateSizes(metadata, ns_key,
+                             storage_->GetCFHandle(Engine::kSubkeyColumnFamilyName),
+                             key_size, buf);
+}
+
+rocksdb::Status Disk::GetZsetSize(const Slice &ns_key, uint64_t *key_size) {
+  ZSetMetadata metadata(false);
+  rocksdb::Status s = Database::GetMetadata(kRedisZSet, ns_key, &metadata);
+  if (!s.ok()) return s.IsNotFound() ? rocksdb::Status::OK() : s;
+  std::string score_bytes;
+  PutDouble(&score_bytes, kMinScore);
+  s = GetApproximateSizes(metadata, ns_key,
+                          storage_->GetCFHandle(Engine::kZSetScoreColumnFamilyName),
+                          key_size, score_bytes, score_bytes);
+  if (!s.ok()) return s;
+  return GetApproximateSizes(metadata, ns_key,
+                             storage_->GetCFHandle(Engine::kSubkeyColumnFamilyName),
+                             key_size);
+}
+
+rocksdb::Status Disk::GetBitmapSize(const Slice &ns_key, uint64_t *key_size) {
+  BitmapMetadata metadata(false);
+  rocksdb::Status s = Database::GetMetadata(kRedisBitmap, ns_key, &metadata);
+  if (!s.ok()) return s.IsNotFound() ? rocksdb::Status::OK() : s;
+  return GetApproximateSizes(metadata, ns_key,
+                             storage_->GetCFHandle(Engine::kSubkeyColumnFamilyName),
+                             key_size, std::to_string(0), std::to_string(0));
+}
+
+rocksdb::Status Disk::GetSortedintSize(const Slice &ns_key, uint64_t *key_size) {
+  SortedintMetadata metadata(false);
+  rocksdb::Status s = Database::GetMetadata(kRedisSortedint, ns_key, &metadata);
+  if (!s.ok()) return s.IsNotFound() ? rocksdb::Status::OK() : s;
+  std::string start_buf;
+  PutFixed64(&start_buf, 0);
+  return GetApproximateSizes(metadata, ns_key,
+                             storage_->GetCFHandle(Engine::kSubkeyColumnFamilyName),
+                             key_size, start_buf, start_buf);
+}
+
+rocksdb::Status Disk::GetStreamSize(const Slice &ns_key, uint64_t *key_size) {
+  StreamMetadata metadata(false);
+  rocksdb::Status s = Database::GetMetadata(kRedisStream, ns_key, &metadata);
+  if (!s.ok()) return s.IsNotFound() ? rocksdb::Status::OK() : s;
+  return GetApproximateSizes(metadata, ns_key,
+                             storage_->GetCFHandle(Engine::kStreamColumnFamilyName),
+                             key_size);
+}
+
+}  // namespace Redis
diff --git a/src/redis_disk.h b/src/redis_disk.h
new file mode 100644
index 0000000..d9b9fe7
--- /dev/null
+++ b/src/redis_disk.h
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#pragma once
+
+#include <string>
+#include "redis_db.h"
+#include "redis_metadata.h"
+
+namespace Redis {
+
+class Disk : public Database {
+ public:
+  explicit Disk(Engine::Storage *storage, const std::string &ns): Database(storage, ns) {
+    option_.include_memtabtles = true;
+    option_.include_files = true;
+  }
+  rocksdb::Status GetApproximateSizes(const Metadata &metadata, const Slice &ns_key,
+                                      rocksdb::ColumnFamilyHandle *column_family,
+                                      uint64_t *key_size, Slice subkeyleft = Slice(),
+                                      Slice subkeyright = Slice());
+  rocksdb::Status GetStringSize(const Slice &ns_key, uint64_t *key_size);
+  rocksdb::Status GetHashSize(const Slice &ns_key, uint64_t *key_size);
+  rocksdb::Status GetSetSize(const Slice &ns_key, uint64_t *key_size);
+  rocksdb::Status GetListSize(const Slice &ns_key, uint64_t *key_size);
+  rocksdb::Status GetZsetSize(const Slice &ns_key, uint64_t *key_size);
+  rocksdb::Status GetBitmapSize(const Slice &ns_key, uint64_t *key_size);
+  rocksdb::Status GetSortedintSize(const Slice &ns_key, uint64_t *key_size);
+  rocksdb::Status GetStreamSize(const Slice &ns_key, uint64_t *key_size);
+  rocksdb::Status GetKeySize(const Slice &user_key, RedisType type, uint64_t *key_size);
+ private:
+  rocksdb::SizeApproximationOptions option_;
+};
+
+}  // namespace Redis
diff --git a/tests/cppunit/disk_test.cc b/tests/cppunit/disk_test.cc
new file mode 100644
index 0000000..5ad7452
--- /dev/null
+++ b/tests/cppunit/disk_test.cc
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <chrono>
+#include <memory>
+#include <string>
+#include <vector>
+#include <gtest/gtest.h>
+#include "redis_metadata.h"
+#include "test_base.h"
+#include "redis_string.h"
+#include "redis_disk.h"
+#include "redis_set.h"
+#include "redis_list.h"
+#include "redis_zset.h"
+#include "redis_bitmap.h"
+#include "redis_sortedint.h"
+#include "redis_stream.h"
+
+class RedisDiskTest : public TestBase {
+protected:
+  explicit RedisDiskTest() : TestBase() {
+    delete storage_;
+    config_->RocksDB.compression = rocksdb::CompressionType::kNoCompression;
+    config_->RocksDB.write_buffer_size = 1;
+    config_->RocksDB.block_size = 100;
+    storage_ = new Engine::Storage(config_);
+    Status s = storage_->Open();
+    if (!s.IsOK()) {
+      std::cout << "Failed to open the storage, encounter error: " << s.Msg() << std::endl;
+      assert(s.IsOK());
+    }
+  }
+  ~RedisDiskTest() = default;
+
+protected:
+  double estimation_factor_ = 0.1;
+};
+
+TEST_F(RedisDiskTest, StringDisk) {
+  key_ = "stringdisk_key";
+  std::unique_ptr<Redis::String> string = Util::MakeUnique<Redis::String>(storage_, "disk_ns_string");
+  std::unique_ptr<Redis::Disk> disk = Util::MakeUnique<Redis::Disk>(storage_, "disk_ns_string");
+  std::vector<int> value_size{1024*1024};
+  EXPECT_TRUE(string->Set(key_, std::string(value_size[0], 'a')).ok());
+  uint64_t result = 0;
+  EXPECT_TRUE(disk->GetKeySize(key_, kRedisString, &result).ok());
+  EXPECT_GE(result,  value_size[0] * estimation_factor_);
+  EXPECT_LE(result,  value_size[0] / estimation_factor_);
+  string->Del(key_);
+}
+
+TEST_F(RedisDiskTest, HashDisk) {
+  std::unique_ptr<Redis::Hash> hash = Util::MakeUnique<Redis::Hash>(storage_, "disk_ns_hash");
+  std::unique_ptr<Redis::Disk> disk = Util::MakeUnique<Redis::Disk>(storage_, "disk_ns_hash");
+  key_ = "hashdisk_key";
+  fields_ = {"hashdisk_kkey1", "hashdisk_kkey2", "hashdisk_kkey3", "hashdisk_kkey4", "hashdisk_kkey5"};
+  values_.resize(5);
+  uint64_t approximate_size = 0;
+  int ret = 0;
+  std::vector<int>value_size{1024, 1024, 1024, 1024, 1024};
+  for(int i = 0; i < int(fields_.size()); i++) {
+    values_[i] = std::string(value_size[i], static_cast<char>('a' + i));
+    approximate_size +=  key_.size() + 8 + fields_[i].size() + values_[i].size();
+    rocksdb::Status s = hash->Set(key_, fields_[i], values_[i], &ret);
+    EXPECT_TRUE(s.ok() && ret == 1);
+  }
+  uint64_t key_size = 0;
+  EXPECT_TRUE(disk->GetKeySize(key_, kRedisHash, &key_size).ok());
+  EXPECT_GE(key_size,  approximate_size * estimation_factor_);
+  EXPECT_LE(key_size,  approximate_size / estimation_factor_);
+  hash->Del(key_);
+}
+
+TEST_F(RedisDiskTest, SetDisk) {
+  std::unique_ptr<Redis::Set> set = Util::MakeUnique<Redis::Set>(storage_, "disk_ns_set");
+  std::unique_ptr<Redis::Disk> disk = Util::MakeUnique<Redis::Disk>(storage_, "disk_ns_set");
+  key_ = "setdisk_key";
+  values_.resize(5);
+  uint64_t approximate_size = 0;
+  int ret =0 ;
+  std::vector<int>value_size{1024, 1024, 1024, 1024, 1024};
+  for(int i = 0; i < int(values_.size()); i++) {
+    values_[i] = std::string(value_size[i], static_cast<char>(i+ 'a'));
+    approximate_size += key_.size() + values_[i].size() + 8;
+  }
+  rocksdb::Status s = set->Add(key_, values_, &ret);
+  EXPECT_TRUE(s.ok() && ret == 5);
+  
+  uint64_t key_size = 0;
+  EXPECT_TRUE(disk->GetKeySize(key_, kRedisSet, &key_size).ok());
+  EXPECT_GE(key_size,  approximate_size * estimation_factor_);
+  EXPECT_LE(key_size,  approximate_size / estimation_factor_);
+
+  set->Del(key_);
+}
+
+
+TEST_F(RedisDiskTest, ListDisk) {
+  std::unique_ptr<Redis::List> list = Util::MakeUnique<Redis::List>(storage_, "disk_ns_list");
+  std::unique_ptr<Redis::Disk> disk = Util::MakeUnique<Redis::Disk>(storage_, "disk_ns_list");
+  key_ = "listdisk_key";
+  values_.resize(5);
+  std::vector<int>value_size{1024, 1024, 1024, 1024, 1024};
+  uint64_t approximate_size = 0;
+  for(int i = 0; i < int(values_.size()); i++) {
+    values_[i] = std::string(value_size[i], static_cast<char>('a' + i));
+    approximate_size += key_.size() + values_[i].size() + 8 + 8;
+  }
+  int ret = 0;
+  rocksdb::Status s = list->Push(key_, values_, false, &ret);
+  EXPECT_TRUE(s.ok() && ret == 5);
+  uint64_t key_size = 0;
+  EXPECT_TRUE(disk->GetKeySize(key_, kRedisList, &key_size).ok());
+  EXPECT_GE(key_size,  approximate_size * estimation_factor_);
+  EXPECT_LE(key_size,  approximate_size / estimation_factor_);
+  list->Del(key_);
+}
+
+TEST_F(RedisDiskTest, ZsetDisk) {
+  std::unique_ptr<Redis::ZSet> zset = Util::MakeUnique<Redis::ZSet>(storage_, "disk_ns_zet");
+  std::unique_ptr<Redis::Disk> disk = Util::MakeUnique<Redis::Disk>(storage_, "disk_ns_zet");
+  key_ = "zsetdisk_key";
+  int ret = 0;
+  uint64_t approximate_size = 0;
+  std::vector<MemberScore> mscores(5);
+  std::vector<int>value_size{1024, 1024, 1024, 1024, 1024};
+  for(int i = 0; i < int(value_size.size()); i++) {
+    mscores[i].member = std::string(value_size[i], static_cast<char>('a' + i));
+    mscores[i].score = 1.0 * value_size[int(values_.size()) - i - 1];
+    approximate_size += (key_.size() + 8 + mscores[i].member.size() + 8) * 2;
+  }
+  rocksdb::Status s = zset->Add(key_, 0, &mscores, &ret);
+  EXPECT_TRUE(s.ok() && ret == 5);
+  uint64_t key_size = 0; 
+  EXPECT_TRUE(disk->GetKeySize(key_, kRedisZSet, &key_size).ok());
+  EXPECT_GE(key_size,  approximate_size * estimation_factor_);
+  EXPECT_LE(key_size,  approximate_size / estimation_factor_);
+  zset->Del(key_);
+}
+
+TEST_F(RedisDiskTest, BitmapDisk) {
+  std::unique_ptr<Redis::Bitmap> bitmap = Util::MakeUnique<Redis::Bitmap>(storage_, "disk_ns_bitmap");
+  std::unique_ptr<Redis::Disk> disk = Util::MakeUnique<Redis::Disk>(storage_, "disk_ns_bitmap");
+  key_ = "bitmapdisk_key";
+  bool bit = false;
+  uint64_t approximate_size = 0;
+  for (int i= 0; i < 1024*8*100000; i += 1024 * 8) {
+    EXPECT_TRUE(bitmap->SetBit(key_, i, true, &bit).ok());
+    approximate_size += key_.size() + 8 + std::to_string(i/1024/8).size();
+  }
+  uint64_t key_size;
+  EXPECT_TRUE(disk->GetKeySize(key_, kRedisBitmap, &key_size).ok());
+  EXPECT_GE(key_size,  approximate_size * estimation_factor_);
+  EXPECT_LE(key_size,  approximate_size / estimation_factor_);
+  bitmap->Del(key_);
+}
+
+TEST_F(RedisDiskTest, SortedintDisk) {
+  std::unique_ptr<Redis::Sortedint> sortedint = Util::MakeUnique<Redis::Sortedint>(storage_, "disk_ns_sortedint");
+  std::unique_ptr<Redis::Disk> disk = Util::MakeUnique<Redis::Disk>(storage_, "disk_ns_sortedint");
+  key_ = "sortedintdisk_key";
+  int ret;
+  uint64_t approximate_size = 0;
+  for(int i = 0; i < 100000; i++) {
+    EXPECT_TRUE(sortedint->Add(key_, std::vector<uint64_t>{uint64_t(i)}, &ret).ok() && ret==1);
+    approximate_size += key_.size() + 8 + 8;
+  }
+  uint64_t key_size;
+  EXPECT_TRUE(disk->GetKeySize(key_, kRedisSortedint, &key_size).ok());
+  EXPECT_GE(key_size,  approximate_size * estimation_factor_);
+  EXPECT_LE(key_size,  approximate_size / estimation_factor_);
+  sortedint->Del(key_);
+}
+
+TEST_F(RedisDiskTest, StreamDisk) {
+  std::unique_ptr<Redis::Stream> stream = Util::MakeUnique<Redis::Stream>(storage_, "disk_ns_stream");
+  std::unique_ptr<Redis::Disk> disk = Util::MakeUnique<Redis::Disk>(storage_, "disk_ns_stream");
+  key_ = "streamdisk_key";
+  Redis::StreamAddOptions options;
+  options.with_entry_id = false;
+  Redis::StreamEntryID id;
+  uint64_t approximate_size = 0;
+  for (int i = 0; i < 100000; i++) {
+    std::vector<std::string> values = {"key" + std::to_string(i), "val" + std::to_string(i)};
+    auto s = stream->Add(key_, options, values, &id);
+    EXPECT_TRUE(s.ok());
+    approximate_size += key_.size() + 8 + 8 + values[0].size() + values[1].size();
+  }
+  uint64_t key_size;
+  EXPECT_TRUE(disk->GetKeySize(key_, kRedisStream, &key_size).ok());
+  EXPECT_GE(key_size,  approximate_size * estimation_factor_);
+  EXPECT_LE(key_size,  approximate_size / estimation_factor_);
+  stream->Del(key_);
+}
\ No newline at end of file
diff --git a/tests/gocase/unit/command/command_test.go b/tests/gocase/unit/command/command_test.go
index a1eb556..f1be98a 100644
--- a/tests/gocase/unit/command/command_test.go
+++ b/tests/gocase/unit/command/command_test.go
@@ -35,6 +35,13 @@ func TestCommand(t *testing.T) {
 	rdb := srv.NewClient()
 	defer func() { require.NoError(t, rdb.Close()) }()
 
+	t.Run("Kvrocks supports 185 commands currently", func(t *testing.T) {
+		r := rdb.Do(ctx, "COMMAND", "COUNT")
+		v, err := r.Int()
+		require.NoError(t, err)
+		require.Equal(t, 185, v)
+	})
+
 	t.Run("acquire GET command info by COMMAND INFO", func(t *testing.T) {
 		r := rdb.Do(ctx, "COMMAND", "INFO", "GET")
 		vs, err := r.Slice()
diff --git a/tests/gocase/unit/disk/disk_test.go b/tests/gocase/unit/disk/disk_test.go
new file mode 100644
index 0000000..c2d86f4
--- /dev/null
+++ b/tests/gocase/unit/disk/disk_test.go
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package disk
+
+import (
+	"context"
+	"strconv"
+	"strings"
+	"testing"
+
+	"github.com/apache/incubator-kvrocks/tests/gocase/util"
+	"github.com/go-redis/redis/v9"
+	"github.com/stretchr/testify/require"
+)
+
+func TestDisk(t *testing.T) {
+	srv := util.StartServer(t, map[string]string{
+		"rocksdb.compression":       "no",
+		"rocksdb.write_buffer_size": "1",
+		"rocksdb.block_size":        "100",
+	})
+	defer srv.Close()
+	ctx := context.Background()
+	rdb := srv.NewClient()
+	defer func() { require.NoError(t, rdb.Close()) }()
+	estimationFactor := 0.1
+	t.Run("Disk usage String", func(t *testing.T) {
+		require.NoError(t, rdb.Set(ctx, "stringkey", strings.Repeat("A", 1024), 0).Err())
+		val, err := rdb.Do(ctx, "Disk", "usage", "stringkey").Int()
+		require.NoError(t, err)
+		require.GreaterOrEqual(t, val, int(estimationFactor*1024))
+		require.LessOrEqual(t, val, int(1024/estimationFactor))
+	})
+
+	t.Run("Disk usage Hash", func(t *testing.T) {
+		approximateSize := 0
+		for i := 0; i < 10000; i++ {
+			require.NoError(t, rdb.HSet(ctx, "hashkey", "key"+strconv.Itoa(i), "value"+strconv.Itoa(i)).Err())
+			approximateSize += len("hashkey") + len("key"+strconv.Itoa(i)) + len("value"+strconv.Itoa(i)) + 8
+		}
+		require.NoError(t, rdb.Set(ctx, "hashkey", strings.Repeat("A", 1024), 0).Err())
+		val, err := rdb.Do(ctx, "Disk", "usage", "hashkey").Int()
+		require.NoError(t, err)
+		require.GreaterOrEqual(t, val, int(float64(approximateSize)*estimationFactor))
+		require.LessOrEqual(t, val, int(float64(approximateSize)/estimationFactor))
+	})
+
+	t.Run("Disk usage Set", func(t *testing.T) {
+		approximateSize := 0
+		for i := 0; i < 10000; i++ {
+			require.NoError(t, rdb.SAdd(ctx, "setkey", i).Err())
+			approximateSize += len(strconv.Itoa(i)) + len("setkey") + 8
+		}
+		val, err := rdb.Do(ctx, "Disk", "usage", "setkey").Int()
+		require.NoError(t, err)
+		require.GreaterOrEqual(t, val, int(float64(approximateSize)*estimationFactor))
+		require.LessOrEqual(t, val, int(float64(approximateSize)/estimationFactor))
+	})
+
+	t.Run("Disk usage List", func(t *testing.T) {
+		approximateSize := 0
+		for i := 0; i < 10000; i++ {
+			require.NoError(t, rdb.LPush(ctx, "listkey", i).Err())
+			approximateSize += len("listkey") + 8 + 8 + len(strconv.Itoa(i))
+		}
+		val, err := rdb.Do(ctx, "Disk", "usage", "listkey").Int()
+		require.NoError(t, err)
+		require.GreaterOrEqual(t, val, int(float64(approximateSize)*estimationFactor))
+		require.LessOrEqual(t, val, int(float64(approximateSize)/estimationFactor))
+	})
+
+	t.Run("Disk usage Zset", func(t *testing.T) {
+		approximateSize := 0
+		for i := 0; i < 10000; i++ {
+			require.NoError(t, rdb.ZAdd(ctx, "zsetkey", redis.Z{Score: float64(i), Member: "x" + strconv.Itoa(i)}).Err())
+			approximateSize += (len("zsetkey") + 8 + len("x"+strconv.Itoa(i))) * 2
+		}
+		val, err := rdb.Do(ctx, "Disk", "usage", "zsetkey").Int()
+		require.NoError(t, err)
+		require.GreaterOrEqual(t, val, int(float64(approximateSize)*estimationFactor))
+		require.LessOrEqual(t, val, int(float64(approximateSize)/estimationFactor))
+	})
+
+	t.Run("Disk usage Bitmap", func(t *testing.T) {
+		approximateSize := 0
+		for i := 0; i < 1024*8*100000; i += 1024 * 8 {
+			require.NoError(t, rdb.SetBit(ctx, "bitmapkey", int64(i), 1).Err())
+			approximateSize += len("bitmapkey") + 8 + len(strconv.Itoa(i/1024*8))
+		}
+		val, err := rdb.Do(ctx, "Disk", "usage", "bitmapkey").Int()
+		require.NoError(t, err)
+		require.GreaterOrEqual(t, val, int(float64(approximateSize)*estimationFactor))
+		require.LessOrEqual(t, val, int(float64(approximateSize)/estimationFactor))
+	})
+
+	t.Run("Disk usage Sortedint", func(t *testing.T) {
+		approximateSize := 0
+		for i := 0; i < 100000; i++ {
+			require.NoError(t, rdb.Do(ctx, "siadd", "sortedintkey", i).Err())
+			approximateSize += len("sortedintkey") + 8 + 8
+		}
+		val, err := rdb.Do(ctx, "Disk", "usage", "sortedintkey").Int()
+		require.NoError(t, err)
+		require.GreaterOrEqual(t, val, int(float64(approximateSize)*estimationFactor))
+		require.LessOrEqual(t, val, int(float64(approximateSize)/estimationFactor))
+	})
+
+	t.Run("Disk usage Stream", func(t *testing.T) {
+		approximateSize := 0
+		for i := 0; i < 100000; i++ {
+			require.NoError(t, rdb.Do(ctx, "xadd", "streamkey", "*", "key"+strconv.Itoa(i), "value"+strconv.Itoa(i)).Err())
+			approximateSize += len("streamkey") + 8 + 8 + len("value"+strconv.Itoa(i)) + len("value"+strconv.Itoa(i))
+		}
+		val, err := rdb.Do(ctx, "Disk", "usage", "streamkey").Int()
+		require.NoError(t, err)
+		require.GreaterOrEqual(t, val, int(float64(approximateSize)*estimationFactor))
+		require.LessOrEqual(t, val, int(float64(approximateSize)/estimationFactor))
+	})
+
+	t.Run("Disk usage with typo ", func(t *testing.T) {
+		require.ErrorContains(t, rdb.Do(ctx, "Disk", "usa", "sortedintkey").Err(), "Unknown operation")
+	})
+
+	t.Run("Disk usage nonexistent key ", func(t *testing.T) {
+		require.ErrorContains(t, rdb.Do(ctx, "Disk", "usage", "nonexistentkey").Err(), "Not found")
+	})
+
+}