diff --git a/src/communication/rpc/messages-inl.hpp b/src/communication/rpc/messages-inl.hpp
index cc8841568..e33212ec7 100644
--- a/src/communication/rpc/messages-inl.hpp
+++ b/src/communication/rpc/messages-inl.hpp
@@ -10,6 +10,7 @@
 #include "distributed/index_rpc_messages.hpp"
 #include "distributed/plan_rpc_messages.hpp"
 #include "distributed/pull_produce_rpc_messages.hpp"
+#include "distributed/storage_gc_rpc_messages.hpp"
 #include "distributed/transactional_cache_cleaner_rpc_messages.hpp"
 #include "distributed/updates_rpc_messages.hpp"
 #include "stats/stats_rpc_messages.hpp"
@@ -117,6 +118,11 @@ BOOST_CLASS_EXPORT(distributed::RemoveInEdgeData);
 BOOST_CLASS_EXPORT(distributed::RemoveInEdgeReq);
 BOOST_CLASS_EXPORT(distributed::RemoveInEdgeRes);
 
+// Storage Gc.
+BOOST_CLASS_EXPORT(distributed::GcClearedStatusReq);
+BOOST_CLASS_EXPORT(distributed::GcClearedStatusRes);
+
 // Transactional Cache Cleaner.
 BOOST_CLASS_EXPORT(distributed::WaitOnTransactionEndReq);
 BOOST_CLASS_EXPORT(distributed::WaitOnTransactionEndRes);
+
diff --git a/src/data_structures/bitset/dynamic_bitset.hpp b/src/data_structures/bitset/dynamic_bitset.hpp
index 6570731f3..51be6cb9c 100644
--- a/src/data_structures/bitset/dynamic_bitset.hpp
+++ b/src/data_structures/bitset/dynamic_bitset.hpp
@@ -148,6 +148,36 @@ class DynamicBitset {
     return chunk.clear(k, n);
   }
 
+  /**
+   * Deletes all blocks which contain all positions lower than pos, assumes that
+   * there doesn't exist a pointer to those blocks, i.e. it's safe to delete
+   * them
+   */
+  void delete_prefix(size_t pos) {
+    // Never delete head as that might invalidate the whole structure which
+    // depends on head being available
+    Chunk *last = head_.load();
+    Chunk *chunk = last->next_;
+
+    // High is exclusive endpoint of interval
+    while (chunk != nullptr && chunk->high() > pos) {
+      last = chunk;
+      chunk = chunk->next_;
+    }
+
+    if (chunk != nullptr) {
+      // Unlink from last
+      last->next_ = nullptr;
+      // Deletes chunks
+      Chunk *next;
+      while (chunk) {
+        next = chunk->next_;
+        delete chunk;
+        chunk = next;
+      }
+    }
+  }
+
  private:
   // Finds the chunk to which k-th bit belongs fails if k is out of bounds.
   const Chunk &FindChunk(size_t &k) const {
diff --git a/src/database/graph_db.cpp b/src/database/graph_db.cpp
index a6cc70362..e792de9b6 100644
--- a/src/database/graph_db.cpp
+++ b/src/database/graph_db.cpp
@@ -2,6 +2,9 @@
 
 #include "communication/rpc/server.hpp"
 #include "database/graph_db.hpp"
+#include "database/storage_gc_master.hpp"
+#include "database/storage_gc_single_node.hpp"
+#include "database/storage_gc_worker.hpp"
 #include "distributed/coordination_master.hpp"
 #include "distributed/coordination_worker.hpp"
 #include "distributed/data_manager.hpp"
@@ -97,7 +100,7 @@ class SingleNode : public PrivateBase {
   IMPL_GETTERS
 
   tx::SingleNodeEngine tx_engine_{&wal_};
-  StorageGc storage_gc_{storage_, tx_engine_, config_.gc_cycle_sec};
+  StorageGcSingleNode storage_gc_{storage_, tx_engine_, config_.gc_cycle_sec};
   TypemapPack<SingleNodeConcurrentIdMapper> typemap_pack_;
   database::SingleNodeCounters counters_;
   std::vector<int> GetWorkerIds() const override { return {0}; }
@@ -161,8 +164,9 @@ class Master : public PrivateBase {
       config_.master_endpoint, static_cast<size_t>(config_.rpc_num_workers)};
   tx::MasterEngine tx_engine_{server_, rpc_worker_clients_, &wal_};
   distributed::MasterCoordination coordination_{server_};
+  StorageGcMaster storage_gc_{storage_, tx_engine_, config_.gc_cycle_sec,
+                              server_, coordination_};
   distributed::RpcWorkerClients rpc_worker_clients_{coordination_};
-  StorageGc storage_gc_{storage_, tx_engine_, config_.gc_cycle_sec};
   TypemapPack<MasterConcurrentIdMapper> typemap_pack_{server_};
   database::MasterCounters counters_{server_};
   distributed::DataRpcServer data_server_{*this, server_};
@@ -206,7 +210,9 @@ class Worker : public PrivateBase {
                                                 config_.master_endpoint};
   distributed::RpcWorkerClients rpc_worker_clients_{coordination_};
   tx::WorkerEngine tx_engine_{rpc_worker_clients_.GetClientPool(0)};
-  StorageGc storage_gc_{storage_, tx_engine_, config_.gc_cycle_sec};
+  StorageGcWorker storage_gc_{storage_, tx_engine_, config_.gc_cycle_sec,
+                              rpc_worker_clients_.GetClientPool(0),
+                              config_.worker_id};
   TypemapPack<WorkerConcurrentIdMapper> typemap_pack_{
       rpc_worker_clients_.GetClientPool(0)};
   database::WorkerCounters counters_{rpc_worker_clients_.GetClientPool(0)};
diff --git a/src/database/graph_db.hpp b/src/database/graph_db.hpp
index 4db0756f9..4f129de0f 100644
--- a/src/database/graph_db.hpp
+++ b/src/database/graph_db.hpp
@@ -25,7 +25,7 @@ class UpdatesRpcServer;
 class UpdatesRpcClients;
 class DataManager;
 class IndexRpcClients;
-}
+}  // namespace distributed
 
 namespace database {
 
diff --git a/src/database/storage_gc.hpp b/src/database/storage_gc.hpp
index 7fdd6db5f..3e20e0f1b 100644
--- a/src/database/storage_gc.hpp
+++ b/src/database/storage_gc.hpp
@@ -1,6 +1,7 @@
 #pragma once
 
 #include <chrono>
+#include <queue>
 
 #include "data_structures/concurrent/concurrent_map.hpp"
 #include "database/storage.hpp"
@@ -39,8 +40,8 @@ class StorageGc {
    * tx::Engine. If `pause_sec` is greater then zero, then GC gets triggered
    * periodically. */
   StorageGc(Storage &storage, tx::Engine &tx_engine, int pause_sec)
-      : storage_(storage),
-        tx_engine_(tx_engine),
+      : tx_engine_(tx_engine),
+        storage_(storage),
         vertices_(storage.vertices_),
         edges_(storage.edges_) {
     if (pause_sec > 0)
@@ -63,17 +64,19 @@ class StorageGc {
   StorageGc &operator=(const StorageGc &) = delete;
   StorageGc &operator=(StorageGc &&) = delete;
 
+  virtual void CollectCommitLogGarbage(tx::transaction_id_t oldest_active) = 0;
+
   void CollectGarbage() {
     // main garbage collection logic
     // see wiki documentation for logic explanation
     LOG(INFO) << "Garbage collector started";
-    const auto snapshot = tx_engine_.GlobalGcSnapshot();
+    const auto snapshot_gc = tx_engine_.GlobalGcSnapshot();
     {
       // This can be run concurrently
       utils::Timer x;
 
-      vertices_.gc_.Run(snapshot, tx_engine_);
-      edges_.gc_.Run(snapshot, tx_engine_);
+      vertices_.gc_.Run(snapshot_gc, tx_engine_);
+      edges_.gc_.Run(snapshot_gc, tx_engine_);
 
       VLOG(21) << "Garbage collector mvcc phase time: " << x.Elapsed().count();
     }
@@ -83,8 +86,8 @@ class StorageGc {
     {
       // This can be run concurrently
       utils::Timer x;
-      storage_.labels_index_.Refresh(snapshot, tx_engine_);
-      storage_.label_property_index_.Refresh(snapshot, tx_engine_);
+      storage_.labels_index_.Refresh(snapshot_gc, tx_engine_);
+      storage_.label_property_index_.Refresh(snapshot_gc, tx_engine_);
       VLOG(21) << "Garbage collector index phase time: " << x.Elapsed().count();
     }
     {
@@ -95,17 +98,19 @@ class StorageGc {
       // to those records. New snapshot can be used, different than one used for
       // first two phases of gc.
       utils::Timer x;
-      const auto snapshot = tx_engine_.GlobalGcSnapshot();
-      edges_.record_deleter_.FreeExpiredObjects(snapshot.back());
-      vertices_.record_deleter_.FreeExpiredObjects(snapshot.back());
-      edges_.version_list_deleter_.FreeExpiredObjects(snapshot.back());
-      vertices_.version_list_deleter_.FreeExpiredObjects(snapshot.back());
+      const auto snapshot_gc = tx_engine_.GlobalGcSnapshot();
+      edges_.record_deleter_.FreeExpiredObjects(snapshot_gc.back());
+      vertices_.record_deleter_.FreeExpiredObjects(snapshot_gc.back());
+      edges_.version_list_deleter_.FreeExpiredObjects(snapshot_gc.back());
+      vertices_.version_list_deleter_.FreeExpiredObjects(snapshot_gc.back());
       VLOG(21) << "Garbage collector deferred deletion phase time: "
                << x.Elapsed().count();
     }
 
-    LOG(INFO) << "Garbage collector finished";
-    VLOG(21) << "gc snapshot: " << snapshot;
+    CollectCommitLogGarbage(snapshot_gc.back());
+    gc_txid_ranges_.emplace(snapshot_gc.back(), tx_engine_.GlobalLast());
+
+    VLOG(21) << "gc snapshot: " << snapshot_gc;
     VLOG(21) << "edge_record_deleter_ size: " << edges_.record_deleter_.Count();
     VLOG(21) << "vertex record deleter_ size: "
              << vertices_.record_deleter_.Count();
@@ -115,13 +120,37 @@ class StorageGc {
              << vertices_.version_list_deleter_.Count();
     VLOG(21) << "vertices_ size: " << storage_.vertices_.access().size();
     VLOG(21) << "edges_ size: " << storage_.edges_.access().size();
+    LOG(INFO) << "Garbage collector finished.";
   }
 
+ protected:
+  // Find the largest transaction from which everything older is safe to
+  // delete, ones for which the hints have been set in the gc phase, and no
+  // alive transaction from the time before the hints were set is still alive
+  // (otherwise that transaction could still be waiting for a resolution of
+  // the query to the commit log about some old transaction)
+  std::experimental::optional<tx::transaction_id_t> GetClogSafeTransaction(
+      tx::transaction_id_t oldest_active) {
+    std::experimental::optional<tx::transaction_id_t> safe_to_delete;
+    while (!gc_txid_ranges_.empty() &&
+           gc_txid_ranges_.front().second < oldest_active) {
+      safe_to_delete = gc_txid_ranges_.front().first;
+      gc_txid_ranges_.pop();
+    }
+    return safe_to_delete;
+  }
+
+  tx::Engine &tx_engine_;
+
  private:
   Storage &storage_;
-  tx::Engine &tx_engine_;
   MvccDeleter<Vertex> vertices_;
   MvccDeleter<Edge> edges_;
   Scheduler scheduler_;
+
+  // History of <oldest active transaction, next transaction to be ran> ranges
+  // that gc operated on at some previous time - used to clear commit log
+  std::queue<std::pair<tx::transaction_id_t, tx::transaction_id_t>>
+      gc_txid_ranges_;
 };
 }  // namespace database
diff --git a/src/database/storage_gc_master.hpp b/src/database/storage_gc_master.hpp
new file mode 100644
index 000000000..da2a48b9f
--- /dev/null
+++ b/src/database/storage_gc_master.hpp
@@ -0,0 +1,59 @@
+#pragma once
+
+#include <mutex>
+
+#include "database/storage_gc.hpp"
+#include "distributed/coordination_master.hpp"
+#include "distributed/storage_gc_rpc_messages.hpp"
+
+namespace database {
+class StorageGcMaster : public StorageGc {
+ public:
+  using StorageGc::StorageGc;
+  StorageGcMaster(Storage &storage, tx::Engine &tx_engine, int pause_sec,
+                  communication::rpc::Server &rpc_server,
+                  distributed::MasterCoordination &coordination)
+      : StorageGc(storage, tx_engine, pause_sec),
+        rpc_server_(rpc_server),
+        coordination_(coordination) {
+    rpc_server_.Register<distributed::RanLocalGcRpc>(
+        [this](const distributed::GcClearedStatusReq &req) {
+          std::unique_lock<std::mutex> lock(worker_safe_transaction_mutex_);
+          worker_safe_transaction_[req.worker_id] = req.local_oldest_active;
+          return std::make_unique<distributed::GcClearedStatusRes>();
+        });
+  }
+
+  void CollectCommitLogGarbage(tx::transaction_id_t oldest_active) final {
+    // Workers are sending information when it's safe to delete every
+    // transaction older than oldest_active from their perspective i.e. there
+    // won't exist another transaction in the future with id larger than or
+    // equal to oldest_active that might trigger a query into a commit log about
+    // the state of transactions which we are deleting
+    auto safe_transaction = GetClogSafeTransaction(oldest_active);
+    if (safe_transaction) {
+      tx::transaction_id_t min_safe = *safe_transaction;
+      {
+        std::unique_lock<std::mutex> lock(worker_safe_transaction_mutex_);
+        for (auto worker_id : coordination_.GetWorkerIds()) {
+          // Skip itself
+          if (worker_id == 0) continue;
+          min_safe = std::min(min_safe, worker_safe_transaction_[worker_id]);
+        }
+      }
+      // All workers reported back at least once
+      if (min_safe > 0) {
+        tx_engine_.GarbageCollectCommitLog(min_safe);
+        LOG(INFO) << "Clearing master commit log with tx:" << min_safe;
+      }
+    }
+  }
+
+  communication::rpc::Server &rpc_server_;
+  distributed::MasterCoordination &coordination_;
+  // Mapping of worker ids and oldest active transaction which is safe for
+  // deletion from worker perspective
+  std::unordered_map<int, tx::transaction_id_t> worker_safe_transaction_;
+  std::mutex worker_safe_transaction_mutex_;
+};
+}  // namespace database
diff --git a/src/database/storage_gc_single_node.hpp b/src/database/storage_gc_single_node.hpp
new file mode 100644
index 000000000..39b9e1d1b
--- /dev/null
+++ b/src/database/storage_gc_single_node.hpp
@@ -0,0 +1,15 @@
+#pragma once
+
+#include "database/storage_gc.hpp"
+
+namespace database {
+class StorageGcSingleNode : public StorageGc {
+ public:
+  using StorageGc::StorageGc;
+
+  void CollectCommitLogGarbage(tx::transaction_id_t oldest_active) final {
+    auto safe_to_delete = GetClogSafeTransaction(oldest_active);
+    if (safe_to_delete) tx_engine_.GarbageCollectCommitLog(*safe_to_delete);
+  }
+};
+}  // namespace database
diff --git a/src/database/storage_gc_worker.hpp b/src/database/storage_gc_worker.hpp
new file mode 100644
index 000000000..a88b201f0
--- /dev/null
+++ b/src/database/storage_gc_worker.hpp
@@ -0,0 +1,31 @@
+#pragma once
+
+#include "communication/rpc/client_pool.hpp"
+#include "database/storage_gc.hpp"
+#include "distributed/storage_gc_rpc_messages.hpp"
+
+#include "transactions/transaction.hpp"
+
+namespace database {
+class StorageGcWorker : public StorageGc {
+ public:
+  StorageGcWorker(Storage &storage, tx::Engine &tx_engine, int pause_sec,
+                  communication::rpc::ClientPool &master_client_pool,
+                  int worker_id)
+      : StorageGc(storage, tx_engine, pause_sec),
+        master_client_pool_(master_client_pool),
+        worker_id_(worker_id) {}
+
+  void CollectCommitLogGarbage(tx::transaction_id_t oldest_active) final {
+    auto safe_to_delete = GetClogSafeTransaction(oldest_active);
+    if (safe_to_delete) {
+      master_client_pool_.Call<distributed::RanLocalGcRpc>(*safe_to_delete,
+                                                           worker_id_);
+      tx_engine_.GarbageCollectCommitLog(*safe_to_delete);
+    }
+  }
+
+  communication::rpc::ClientPool &master_client_pool_;
+  int worker_id_;
+};
+}  // namespace database
diff --git a/src/distributed/storage_gc_rpc_messages.hpp b/src/distributed/storage_gc_rpc_messages.hpp
new file mode 100644
index 000000000..b244fd671
--- /dev/null
+++ b/src/distributed/storage_gc_rpc_messages.hpp
@@ -0,0 +1,39 @@
+#pragma once
+
+#include "boost/serialization/access.hpp"
+#include "boost/serialization/base_object.hpp"
+
+#include "communication/rpc/messages.hpp"
+#include "io/network/endpoint.hpp"
+#include "transactions/transaction.hpp"
+
+namespace distributed {
+
+using communication::rpc::Message;
+using Endpoint = io::network::Endpoint;
+
+struct GcClearedStatusReq : public Message {
+  GcClearedStatusReq() {}
+  GcClearedStatusReq(tx::transaction_id_t local_oldest_active, int worker_id)
+      : local_oldest_active(local_oldest_active), worker_id(worker_id) {}
+
+  tx::transaction_id_t local_oldest_active;
+  int worker_id;
+
+ private:
+  friend class boost::serialization::access;
+
+  template <class TArchive>
+  void serialize(TArchive &ar, unsigned int) {
+    ar &boost::serialization::base_object<Message>(*this);
+    ar &local_oldest_active;
+    ar &worker_id;
+  }
+};
+
+RPC_NO_MEMBER_MESSAGE(GcClearedStatusRes);
+
+using RanLocalGcRpc =
+    communication::rpc::RequestResponse<GcClearedStatusReq, GcClearedStatusRes>;
+
+}  // namespace distributed
diff --git a/src/transactions/commit_log.hpp b/src/transactions/commit_log.hpp
index 25c809a67..5f4b6eb5f 100644
--- a/src/transactions/commit_log.hpp
+++ b/src/transactions/commit_log.hpp
@@ -11,6 +11,7 @@ namespace tx {
 // this class and this class doesn't acquire any lock on method calls.
 class CommitLog {
  public:
+  static constexpr int kBitsetBlockSize = 32768;
   CommitLog() = default;
   CommitLog(const CommitLog &) = delete;
   CommitLog(CommitLog &&) = delete;
@@ -33,6 +34,10 @@ class CommitLog {
 
   void set_aborted(transaction_id_t id) { log.set(2 * id + 1); }
 
+  // Clears the commit log from bits associated with transactions with an id
+  // lower than `id`.
+  void garbage_collect_older(transaction_id_t id) { log.delete_prefix(2 * id); }
+
   class Info {
    public:
     Info() {}  // Needed for serialization.
@@ -66,7 +71,7 @@ class CommitLog {
   Info fetch_info(transaction_id_t id) const { return Info{log.at(2 * id, 2)}; }
 
  private:
-  DynamicBitset<uint8_t, 32768> log;
+  DynamicBitset<uint8_t, kBitsetBlockSize> log;
 };
 
 }  // namespace tx
diff --git a/src/transactions/engine.hpp b/src/transactions/engine.hpp
index 4e69f4c40..621989f4b 100644
--- a/src/transactions/engine.hpp
+++ b/src/transactions/engine.hpp
@@ -90,6 +90,9 @@ class Engine {
    * the given id. */
   virtual void EnsureNextIdGreater(transaction_id_t tx_id) = 0;
 
+  /** Garbage collects transactions older than tx_id from commit log. */
+  virtual void GarbageCollectCommitLog(transaction_id_t tx_id) = 0;
+
   auto &local_lock_graph() { return local_lock_graph_; }
   const auto &local_lock_graph() const { return local_lock_graph_; }
 
diff --git a/src/transactions/engine_single_node.cpp b/src/transactions/engine_single_node.cpp
index 2c63cca61..e1b0d0f94 100644
--- a/src/transactions/engine_single_node.cpp
+++ b/src/transactions/engine_single_node.cpp
@@ -110,6 +110,10 @@ transaction_id_t SingleNodeEngine::LocalOldestActive() const {
   return active_.empty() ? counter_ + 1 : active_.front();
 }
 
+void SingleNodeEngine::GarbageCollectCommitLog(transaction_id_t tx_id) {
+  clog_.garbage_collect_older(tx_id);
+}
+
 void SingleNodeEngine::LocalForEachActiveTransaction(
     std::function<void(Transaction &)> f) {
   std::lock_guard<SpinLock> guard(lock_);
diff --git a/src/transactions/engine_single_node.hpp b/src/transactions/engine_single_node.hpp
index f41ca2be7..34519c3c5 100644
--- a/src/transactions/engine_single_node.hpp
+++ b/src/transactions/engine_single_node.hpp
@@ -1,5 +1,6 @@
 #pragma once
 
+#include <atomic>
 #include <experimental/optional>
 #include <unordered_map>
 
@@ -43,6 +44,7 @@ class SingleNodeEngine : public Engine {
       std::function<void(Transaction &)> f) override;
   Transaction *RunningTransaction(transaction_id_t tx_id) override;
   void EnsureNextIdGreater(transaction_id_t tx_id) override;
+  void GarbageCollectCommitLog(transaction_id_t tx_id) override;
 
  private:
   transaction_id_t counter_{0};
diff --git a/src/transactions/engine_worker.cpp b/src/transactions/engine_worker.cpp
index 5ad728cd5..43776329b 100644
--- a/src/transactions/engine_worker.cpp
+++ b/src/transactions/engine_worker.cpp
@@ -187,4 +187,8 @@ void WorkerEngine::UpdateOldestActive(const Snapshot &snapshot,
 void WorkerEngine::EnsureNextIdGreater(transaction_id_t tx_id) {
   master_client_pool_.Call<EnsureNextIdGreaterRpc>(tx_id);
 }
+
+void WorkerEngine::GarbageCollectCommitLog(transaction_id_t tx_id) {
+  clog_.garbage_collect_older(tx_id);
+}
 }  // namespace tx
diff --git a/src/transactions/engine_worker.hpp b/src/transactions/engine_worker.hpp
index 20fcb912d..5c3c3f383 100644
--- a/src/transactions/engine_worker.hpp
+++ b/src/transactions/engine_worker.hpp
@@ -43,6 +43,7 @@ class WorkerEngine : public Engine {
                                   const Snapshot &snapshot);
 
   void EnsureNextIdGreater(transaction_id_t tx_id) override;
+  void GarbageCollectCommitLog(tx::transaction_id_t tx_id) override;
 
   /// Clears the cache of local transactions that have expired. The signature of
   /// this method is dictated by `distributed::TransactionalCacheCleaner`.
diff --git a/tests/unit/distributed_gc.cpp b/tests/unit/distributed_gc.cpp
new file mode 100644
index 000000000..53fd5b5c0
--- /dev/null
+++ b/tests/unit/distributed_gc.cpp
@@ -0,0 +1,78 @@
+#include <gtest/gtest.h>
+
+#include "distributed_common.hpp"
+
+TEST_F(DistributedGraphDbTest, GarbageCollect) {
+  database::GraphDbAccessor dba{master()};
+  auto tx = dba.transaction_id();
+  dba.Commit();
+
+  // Create multiple transactions so that the commit log can be cleared
+  for (int i = 0; i < tx::CommitLog::kBitsetBlockSize; ++i) {
+    database::GraphDbAccessor dba{master()};
+  }
+
+  master().CollectGarbage();
+  worker(1).CollectGarbage();
+  worker(2).CollectGarbage();
+  EXPECT_EQ(master().tx_engine().Info(tx).is_committed(), true);
+
+  database::GraphDbAccessor dba2{master()};
+  auto tx_last = dba2.transaction_id();
+  dba2.Commit();
+
+  worker(1).CollectGarbage();
+  worker(2).CollectGarbage();
+  master().CollectGarbage();
+
+  EXPECT_DEATH(master().tx_engine().Info(tx), "chunk is nullptr");
+  EXPECT_DEATH(worker(1).tx_engine().Info(tx), "chunk is nullptr");
+  EXPECT_DEATH(worker(2).tx_engine().Info(tx), "chunk is nullptr");
+  EXPECT_EQ(master().tx_engine().Info(tx_last).is_committed(), true);
+  EXPECT_EQ(worker(1).tx_engine().Info(tx_last).is_committed(), true);
+  EXPECT_EQ(worker(2).tx_engine().Info(tx_last).is_committed(), true);
+}
+
+TEST_F(DistributedGraphDbTest, GarbageCollectBlocked) {
+  database::GraphDbAccessor dba{master()};
+  auto tx = dba.transaction_id();
+  dba.Commit();
+
+  // Block garbage collection because this is a still alive transaction on the
+  // worker
+  database::GraphDbAccessor dba3{worker(1)};
+
+  // Create multiple transactions so that the commit log can be cleared
+  for (int i = 0; i < tx::CommitLog::kBitsetBlockSize; ++i) {
+    database::GraphDbAccessor dba{master()};
+  }
+
+  // Query for a large id so that the commit log new block is created
+  master().tx_engine().Info(tx::CommitLog::kBitsetBlockSize);
+
+  master().CollectGarbage();
+  worker(1).CollectGarbage();
+  worker(2).CollectGarbage();
+  EXPECT_EQ(master().tx_engine().Info(tx).is_committed(), true);
+
+  database::GraphDbAccessor dba2{master()};
+  auto tx_last = dba2.transaction_id();
+  dba2.Commit();
+
+  worker(1).CollectGarbage();
+  worker(2).CollectGarbage();
+  master().CollectGarbage();
+
+  EXPECT_EQ(master().tx_engine().Info(tx).is_committed(), true);
+  EXPECT_EQ(worker(1).tx_engine().Info(tx).is_committed(), true);
+  EXPECT_EQ(worker(2).tx_engine().Info(tx).is_committed(), true);
+  EXPECT_EQ(master().tx_engine().Info(tx_last).is_committed(), true);
+  EXPECT_EQ(worker(1).tx_engine().Info(tx_last).is_committed(), true);
+  EXPECT_EQ(worker(2).tx_engine().Info(tx_last).is_committed(), true);
+}
+
+int main(int argc, char **argv) {
+  ::testing::FLAGS_gtest_death_test_style = "threadsafe";
+  ::testing::InitGoogleTest(&argc, argv);
+  return RUN_ALL_TESTS();
+}
diff --git a/tests/unit/dynamic_bitset.cpp b/tests/unit/dynamic_bitset.cpp
index 388a643a7..4a9617607 100644
--- a/tests/unit/dynamic_bitset.cpp
+++ b/tests/unit/dynamic_bitset.cpp
@@ -87,4 +87,45 @@ TEST(DynamicBitset, ConstBitset) {
   dbs.set(17);
   const_accepting(dbs);
 }
+
+TEST(DynamicBitSet, PrefixDeleteDontDeleteHead) {
+  DynamicBitset<uint8_t, 8> dbs;
+  dbs.set(7, 1);
+  dbs.delete_prefix(8);
+  EXPECT_EQ(dbs.at(7), 1);
 }
+
+// Checks that the block is not deleted when off by one error in interval
+// endpoint
+TEST(DynamicBitSet, PrefixDeleteDeleteOneBlockOffByOne) {
+  DynamicBitset<uint8_t, 8> dbs;
+  dbs.set(7, 1);
+  // Extends number of blocks
+  dbs.set(10, 1);
+  dbs.delete_prefix(7);
+  EXPECT_EQ(dbs.at(7), 1);
+}
+
+TEST(DynamicBitSet, DeletePrefixDeleteOneBlock) {
+  DynamicBitset<uint8_t, 8> dbs;
+  dbs.set(7, 1);
+  // Extends number of blocks
+  dbs.set(10, 1);
+  dbs.delete_prefix(8);
+  EXPECT_DEATH(dbs.at(7), "chunk is nullptr");
+  EXPECT_EQ(dbs.at(10), 1);
+}
+
+TEST(DynamicBitSet, DeletePrefixDeleteMultipleBlocks) {
+  DynamicBitset<uint8_t, 8> dbs;
+  dbs.set(7, 1);
+  dbs.set(15, 1);
+  dbs.set(23, 1);
+  dbs.set(31, 1);
+  dbs.delete_prefix(30);
+  EXPECT_DEATH(dbs.at(7), "chunk is nullptr");
+  EXPECT_DEATH(dbs.at(15), "chunk is nullptr");
+  EXPECT_DEATH(dbs.at(23), "chunk is nullptr");
+  EXPECT_EQ(dbs.at(31), 1);
+}
+}  // namespace