Skip to content

Commit

Permalink
Merge pull request #2074 from kuzudb/refactor-storage-1
Browse files Browse the repository at this point in the history
Move metadataDAHInfo from catalog to statistics
  • Loading branch information
ray6080 committed Sep 24, 2023
2 parents 4f8b9c2 + 380e8a9 commit 57d37b2
Show file tree
Hide file tree
Showing 43 changed files with 359 additions and 375 deletions.
2 changes: 1 addition & 1 deletion CMakeLists.txt
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
cmake_minimum_required(VERSION 3.11)

project(Kuzu VERSION 0.0.8.10 LANGUAGES CXX)
project(Kuzu VERSION 0.0.8.11 LANGUAGES CXX)

find_package(Threads REQUIRED)

Expand Down
19 changes: 5 additions & 14 deletions src/catalog/catalog.cpp
Original file line number Diff line number Diff line change
@@ -1,7 +1,6 @@
#include "catalog/catalog.h"

#include "catalog/node_table_schema.h"
#include "catalog/rdf_graph_schema.h"
#include "catalog/rel_table_group_schema.h"
#include "catalog/rel_table_schema.h"
#include "common/ser_deser.h"
Expand Down Expand Up @@ -44,9 +43,7 @@ ExpressionType Catalog::getFunctionType(const std::string& name) const {

table_id_t Catalog::addNodeTableSchema(const binder::BoundCreateTableInfo& info) {
initCatalogContentForWriteTrxIfNecessary();
auto tableID = catalogContentForWriteTrx->addNodeTableSchema(info);
wal->logNodeTableRecord(tableID);
return tableID;
return catalogContentForWriteTrx->addNodeTableSchema(info);
}

table_id_t Catalog::addRelTableSchema(const binder::BoundCreateTableInfo& info) {
Expand All @@ -71,11 +68,7 @@ common::table_id_t Catalog::addRelTableGroupSchema(const binder::BoundCreateTabl

common::table_id_t Catalog::addRdfGraphSchema(const binder::BoundCreateTableInfo& info) {
initCatalogContentForWriteTrxIfNecessary();
auto tableID = catalogContentForWriteTrx->addRdfGraphSchema(info);
auto rdfGraphSchema = (RdfGraphSchema*)catalogContentForWriteTrx->getTableSchema(tableID);
wal->logRdfGraphRecord(
tableID, rdfGraphSchema->getNodeTableID(), rdfGraphSchema->getRelTableID());
return tableID;
return catalogContentForWriteTrx->addRdfGraphSchema(info);
}

void Catalog::dropTableSchema(table_id_t tableID) {
Expand All @@ -89,13 +82,11 @@ void Catalog::renameTable(table_id_t tableID, const std::string& newName) {
catalogContentForWriteTrx->renameTable(tableID, newName);
}

void Catalog::addNodeProperty(table_id_t tableID, const std::string& propertyName,
std::unique_ptr<LogicalType> dataType, std::unique_ptr<MetadataDAHInfo> metadataDAHInfo) {
void Catalog::addNodeProperty(
table_id_t tableID, const std::string& propertyName, std::unique_ptr<LogicalType> dataType) {
initCatalogContentForWriteTrxIfNecessary();
catalogContentForWriteTrx->getTableSchema(tableID)->addNodeProperty(
propertyName, std::move(dataType), std::move(metadataDAHInfo));
wal->logAddPropertyRecord(
tableID, catalogContentForWriteTrx->getTableSchema(tableID)->getPropertyID(propertyName));
propertyName, std::move(dataType));
}

void Catalog::addRelProperty(
Expand Down
20 changes: 1 addition & 19 deletions src/catalog/property.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -7,27 +7,11 @@ using namespace kuzu::common;
namespace kuzu {
namespace catalog {

void MetadataDAHInfo::serialize(FileInfo* fileInfo, uint64_t& offset) const {
SerDeser::serializeValue(dataDAHPageIdx, fileInfo, offset);
SerDeser::serializeValue(nullDAHPageIdx, fileInfo, offset);
SerDeser::serializeVectorOfPtrs(childrenInfos, fileInfo, offset);
}

std::unique_ptr<MetadataDAHInfo> MetadataDAHInfo::deserialize(
FileInfo* fileInfo, uint64_t& offset) {
auto metadataDAHInfo = std::make_unique<MetadataDAHInfo>();
SerDeser::deserializeValue(metadataDAHInfo->dataDAHPageIdx, fileInfo, offset);
SerDeser::deserializeValue(metadataDAHInfo->nullDAHPageIdx, fileInfo, offset);
SerDeser::deserializeVectorOfPtrs(metadataDAHInfo->childrenInfos, fileInfo, offset);
return metadataDAHInfo;
}

void Property::serialize(FileInfo* fileInfo, uint64_t& offset) const {
SerDeser::serializeValue(name, fileInfo, offset);
dataType->serialize(fileInfo, offset);
SerDeser::serializeValue(propertyID, fileInfo, offset);
SerDeser::serializeValue(tableID, fileInfo, offset);
metadataDAHInfo->serialize(fileInfo, offset);
}

std::unique_ptr<Property> Property::deserialize(FileInfo* fileInfo, uint64_t& offset) {
Expand All @@ -38,9 +22,7 @@ std::unique_ptr<Property> Property::deserialize(FileInfo* fileInfo, uint64_t& of
auto dataType = LogicalType::deserialize(fileInfo, offset);
SerDeser::deserializeValue(propertyID, fileInfo, offset);
SerDeser::deserializeValue(tableID, fileInfo, offset);
auto metadataDAHInfo = MetadataDAHInfo::deserialize(fileInfo, offset);
return std::make_unique<Property>(
name, std::move(dataType), propertyID, tableID, std::move(metadataDAHInfo));
return std::make_unique<Property>(name, std::move(dataType), propertyID, tableID);
}

std::vector<std::unique_ptr<Property>> Property::copy(
Expand Down
3 changes: 1 addition & 2 deletions src/include/catalog/catalog.h
Original file line number Diff line number Diff line change
Expand Up @@ -56,8 +56,7 @@ class Catalog {
void renameTable(common::table_id_t tableID, const std::string& newName);

void addNodeProperty(common::table_id_t tableID, const std::string& propertyName,
std::unique_ptr<common::LogicalType> dataType,
std::unique_ptr<MetadataDAHInfo> metadataDAHInfo);
std::unique_ptr<common::LogicalType> dataType);
void addRelProperty(common::table_id_t tableID, const std::string& propertyName,
std::unique_ptr<common::LogicalType> dataType);

Expand Down
46 changes: 3 additions & 43 deletions src/include/catalog/property.h
Original file line number Diff line number Diff line change
Expand Up @@ -5,33 +5,6 @@
namespace kuzu {
namespace catalog {

// DAH is the abbreviation for Disk Array Header.
class MetadataDAHInfo {
public:
MetadataDAHInfo() : MetadataDAHInfo{common::INVALID_PAGE_IDX, common::INVALID_PAGE_IDX} {}
MetadataDAHInfo(common::page_idx_t dataDAHPageIdx)
: MetadataDAHInfo{dataDAHPageIdx, common::INVALID_PAGE_IDX} {}
MetadataDAHInfo(common::page_idx_t dataDAHPageIdx, common::page_idx_t nullDAHPageIdx)
: dataDAHPageIdx{dataDAHPageIdx}, nullDAHPageIdx{nullDAHPageIdx} {}

inline std::unique_ptr<MetadataDAHInfo> copy() {
auto result = std::make_unique<MetadataDAHInfo>(dataDAHPageIdx, nullDAHPageIdx);
result->childrenInfos.resize(childrenInfos.size());
for (size_t i = 0; i < childrenInfos.size(); ++i) {
result->childrenInfos[i] = childrenInfos[i]->copy();
}
return result;
}

void serialize(common::FileInfo* fileInfo, uint64_t& offset) const;
static std::unique_ptr<MetadataDAHInfo> deserialize(
common::FileInfo* fileInfo, uint64_t& offset);

common::page_idx_t dataDAHPageIdx = common::INVALID_PAGE_IDX;
common::page_idx_t nullDAHPageIdx = common::INVALID_PAGE_IDX;
std::vector<std::unique_ptr<MetadataDAHInfo>> childrenInfos;
};

class Property {
public:
// TODO: these should be guarded as reserved property names.
Expand All @@ -46,14 +19,9 @@ class Property {
common::INVALID_TABLE_ID} {}

Property(std::string name, std::unique_ptr<common::LogicalType> dataType,
common::property_id_t propertyID, common::table_id_t tableID,
std::unique_ptr<MetadataDAHInfo> metadataDAHInfo = nullptr)
common::property_id_t propertyID, common::table_id_t tableID)
: name{std::move(name)}, dataType{std::move(dataType)},
propertyID{propertyID}, tableID{tableID}, metadataDAHInfo{std::move(metadataDAHInfo)} {
if (this->metadataDAHInfo == nullptr) {
this->metadataDAHInfo = std::make_unique<MetadataDAHInfo>();
}
}
propertyID{propertyID}, tableID{tableID} {}

inline std::string getName() const { return name; }

Expand All @@ -63,24 +31,17 @@ class Property {

inline common::table_id_t getTableID() const { return tableID; }

inline MetadataDAHInfo* getMetadataDAHInfo() const { return metadataDAHInfo.get(); }

inline void setPropertyID(common::property_id_t propertyID_) { this->propertyID = propertyID_; }

inline void setTableID(common::table_id_t tableID_) { this->tableID = tableID_; }

inline void setMetadataDAHInfo(std::unique_ptr<MetadataDAHInfo> metadataDAHInfo_) {
this->metadataDAHInfo = std::move(metadataDAHInfo_);
}

inline void rename(std::string newName) { this->name = std::move(newName); }

void serialize(common::FileInfo* fileInfo, uint64_t& offset) const;
static std::unique_ptr<Property> deserialize(common::FileInfo* fileInfo, uint64_t& offset);

inline std::unique_ptr<Property> copy() const {
return std::make_unique<Property>(
name, dataType->copy(), propertyID, tableID, metadataDAHInfo->copy());
return std::make_unique<Property>(name, dataType->copy(), propertyID, tableID);
}

static std::vector<std::unique_ptr<Property>> copy(
Expand All @@ -91,7 +52,6 @@ class Property {
std::unique_ptr<common::LogicalType> dataType;
common::property_id_t propertyID;
common::table_id_t tableID;
std::unique_ptr<MetadataDAHInfo> metadataDAHInfo;
};

} // namespace catalog
Expand Down
9 changes: 4 additions & 5 deletions src/include/catalog/table_schema.h
Original file line number Diff line number Diff line change
Expand Up @@ -51,11 +51,10 @@ class TableSchema {

std::vector<Property*> getProperties() const;

inline void addNodeProperty(std::string propertyName,
std::unique_ptr<common::LogicalType> dataType,
std::unique_ptr<MetadataDAHInfo> metadataDAHInfo) {
properties.push_back(std::make_unique<Property>(std::move(propertyName),
std::move(dataType), increaseNextPropertyID(), tableID, std::move(metadataDAHInfo)));
inline void addNodeProperty(
std::string propertyName, std::unique_ptr<common::LogicalType> dataType) {
properties.push_back(std::make_unique<Property>(
std::move(propertyName), std::move(dataType), increaseNextPropertyID(), tableID));
}
inline void addRelProperty(
std::string propertyName, std::unique_ptr<common::LogicalType> dataType) {
Expand Down
21 changes: 15 additions & 6 deletions src/include/processor/operator/ddl/drop_property.h
Original file line number Diff line number Diff line change
Expand Up @@ -9,20 +9,29 @@ namespace processor {
class DropProperty : public DDL {
public:
DropProperty(catalog::Catalog* catalog, common::table_id_t tableID,
common::property_id_t propertyID, const DataPos& outputPos, uint32_t id,
const std::string& paramsString)
common::property_id_t propertyID, const DataPos& outputPos,
storage::StorageManager& storageManager, uint32_t id, const std::string& paramsString)
: DDL{PhysicalOperatorType::DROP_PROPERTY, catalog, outputPos, id, paramsString},
tableID{tableID}, propertyID{propertyID} {}

void executeDDLInternal() override { catalog->dropProperty(tableID, propertyID); }
storageManager{storageManager}, tableID{tableID}, propertyID{propertyID} {}

void executeDDLInternal() override {
auto tableSchema = catalog->getReadOnlyVersion()->getTableSchema(tableID);
catalog->dropProperty(tableID, propertyID);
if (tableSchema->tableType == common::TableType::NODE) {
auto nodesStats = storageManager.getNodesStore().getNodesStatisticsAndDeletedIDs();
nodesStats->removeMetadataDAHInfo(tableID, tableSchema->getColumnID(propertyID));
}
}

std::string getOutputMsg() override { return {"Drop succeed."}; }

std::unique_ptr<PhysicalOperator> clone() override {
return make_unique<DropProperty>(catalog, tableID, propertyID, outputPos, id, paramsString);
return make_unique<DropProperty>(
catalog, tableID, propertyID, outputPos, storageManager, id, paramsString);
}

protected:
storage::StorageManager& storageManager;
common::table_id_t tableID;
common::property_id_t propertyID;
};
Expand Down
67 changes: 55 additions & 12 deletions src/include/storage/stats/nodes_statistics_and_deleted_ids.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,10 +10,17 @@ namespace kuzu {
namespace storage {

class NodeTableStatsAndDeletedIDs : public TableStatistics {

public:
explicit NodeTableStatsAndDeletedIDs(const catalog::TableSchema& schema)
: TableStatistics{schema}, tableID{schema.tableID} {}
NodeTableStatsAndDeletedIDs(BMFileHandle* metadataFH, const catalog::TableSchema& schema,
BufferManager* bufferManager, WAL* wal)
: TableStatistics{schema}, tableID{schema.tableID} {
metadataDAHInfos.clear();
metadataDAHInfos.reserve(schema.getNumProperties());
for (auto property : schema.getProperties()) {
metadataDAHInfos.push_back(TablesStatistics::createMetadataDAHInfo(
*property->getDataType(), *metadataFH, bufferManager, wal));
}
}

NodeTableStatsAndDeletedIDs(common::table_id_t tableID, common::offset_t maxNodeOffset,
const std::vector<common::offset_t>& deletedNodeOffsets)
Expand All @@ -30,7 +37,17 @@ class NodeTableStatsAndDeletedIDs : public TableStatistics {
std::unordered_map<common::property_id_t, std::unique_ptr<PropertyStatistics>>&&
propertyStatistics);

NodeTableStatsAndDeletedIDs(const NodeTableStatsAndDeletedIDs& other) = default;
NodeTableStatsAndDeletedIDs(const NodeTableStatsAndDeletedIDs& other)
: TableStatistics{other}, tableID{other.tableID},
adjListsAndColumns{other.adjListsAndColumns},
hasDeletedNodesPerMorsel{other.hasDeletedNodesPerMorsel},
deletedNodeOffsetsPerMorsel{other.deletedNodeOffsetsPerMorsel} {
metadataDAHInfos.clear();
metadataDAHInfos.reserve(other.metadataDAHInfos.size());
for (auto& metadataDAHInfo : other.metadataDAHInfos) {
metadataDAHInfos.push_back(metadataDAHInfo->copy());
}
}

inline common::offset_t getMaxNodeOffset() {
return getMaxNodeOffsetFromNumTuples(getNumTuples());
Expand All @@ -53,7 +70,7 @@ class NodeTableStatsAndDeletedIDs : public TableStatistics {

void setNumTuples(uint64_t numTuples) override;

std::vector<common::offset_t> getDeletedNodeOffsets();
std::vector<common::offset_t> getDeletedNodeOffsets() const;

static inline uint64_t getNumTuplesFromMaxNodeOffset(common::offset_t maxNodeOffset) {
return (maxNodeOffset == UINT64_MAX) ? 0ull : maxNodeOffset + 1ull;
Expand All @@ -63,10 +80,26 @@ class NodeTableStatsAndDeletedIDs : public TableStatistics {
return numTuples == 0 ? UINT64_MAX : numTuples - 1;
}

inline void addMetadataDAHInfoForColumn(std::unique_ptr<MetadataDAHInfo> metadataDAHInfo) {
metadataDAHInfos.push_back(std::move(metadataDAHInfo));
}
inline void removeMetadataDAHInfoForColumn(common::column_id_t columnID) {
assert(columnID < metadataDAHInfos.size());
metadataDAHInfos.erase(metadataDAHInfos.begin() + columnID);
}
inline MetadataDAHInfo* getMetadataDAHInfo(common::column_id_t columnID) {
assert(columnID < metadataDAHInfos.size());
return metadataDAHInfos[columnID].get();
}

void serializeInternal(common::FileInfo* fileInfo, uint64_t& offset) final;
static std::unique_ptr<NodeTableStatsAndDeletedIDs> deserialize(common::table_id_t tableID,
common::offset_t maxNodeOffset, common::FileInfo* fileInfo, uint64_t& offset);

std::unique_ptr<TableStatistics> copy() final {
return std::make_unique<NodeTableStatsAndDeletedIDs>(*this);
}

private:
void errorIfNodeHasEdges(common::offset_t nodeOffset);

Expand All @@ -75,6 +108,7 @@ class NodeTableStatsAndDeletedIDs : public TableStatistics {

private:
common::table_id_t tableID;
std::vector<std::unique_ptr<MetadataDAHInfo>> metadataDAHInfos;
// Note: This is initialized explicitly through a call to setAdjListsAndColumns after
// construction.
std::pair<std::vector<AdjLists*>, std::vector<Column*>> adjListsAndColumns;
Expand All @@ -85,16 +119,15 @@ class NodeTableStatsAndDeletedIDs : public TableStatistics {
// Manages the disk image of the maxNodeOffsets and deleted node IDs (per node table).
// Note: This class is *not* thread-safe.
class NodesStatisticsAndDeletedIDs : public TablesStatistics {

public:
// Should only be used by saveInitialNodesStatisticsAndDeletedIDsToFile to start a database
// from an empty directory.
NodesStatisticsAndDeletedIDs() : TablesStatistics{} {};
NodesStatisticsAndDeletedIDs() : TablesStatistics{nullptr} {};
// Should be used when an already loaded database is started from a directory.
explicit NodesStatisticsAndDeletedIDs(
const std::string& directory, common::DBFileType dbFileType = common::DBFileType::ORIGINAL)
: TablesStatistics{} {
readFromFile(directory, dbFileType);
explicit NodesStatisticsAndDeletedIDs(BMFileHandle* metadataFH, BufferManager* bufferManager,
WAL* wal, common::DBFileType dbFileType = common::DBFileType::ORIGINAL)
: TablesStatistics{metadataFH}, bufferManager{bufferManager}, wal{wal} {
readFromFile(wal->getDirectory(), dbFileType);
}

// Should be used only by tests;
Expand Down Expand Up @@ -180,10 +213,16 @@ class NodesStatisticsAndDeletedIDs : public TablesStatistics {

void addNodeStatisticsAndDeletedIDs(catalog::NodeTableSchema* tableSchema);

void addMetadataDAHInfo(common::table_id_t tableID, const common::LogicalType& dataType);
void removeMetadataDAHInfo(common::table_id_t tableID, common::column_id_t columnID);
MetadataDAHInfo* getMetadataDAHInfo(transaction::Transaction* transaction,
common::table_id_t tableID, common::column_id_t columnID);

protected:
inline std::unique_ptr<TableStatistics> constructTableStatistic(
catalog::TableSchema* tableSchema) override {
return std::make_unique<NodeTableStatsAndDeletedIDs>(*tableSchema);
return std::make_unique<NodeTableStatsAndDeletedIDs>(
metadataFH, *tableSchema, bufferManager, wal);
}

inline std::unique_ptr<TableStatistics> constructTableStatistic(
Expand All @@ -196,6 +235,10 @@ class NodesStatisticsAndDeletedIDs : public TablesStatistics {
const std::string& directory, common::DBFileType dbFileType) override {
return StorageUtils::getNodesStatisticsAndDeletedIDsFilePath(directory, dbFileType);
}

private:
BufferManager* bufferManager;
WAL* wal;
};

} // namespace storage
Expand Down
Loading

0 comments on commit 57d37b2

Please sign in to comment.