Skip to content

Commit

Permalink
Added PropertyStatistics structure for tracking nulls
Browse files Browse the repository at this point in the history
  • Loading branch information
benjaminwinger committed Aug 16, 2023
1 parent 85dd412 commit 732229d
Show file tree
Hide file tree
Showing 20 changed files with 338 additions and 93 deletions.
29 changes: 26 additions & 3 deletions src/include/storage/copier/column_chunk.h
Original file line number Diff line number Diff line change
Expand Up @@ -183,19 +183,40 @@ class BoolColumnChunk : public ColumnChunk {
arrow::Array* array, common::offset_t startPosInChunk, uint32_t numValuesToAppend) final;

void append(ColumnChunk* other, common::offset_t startPosInOtherChunk,
common::offset_t startPosInChunk, uint32_t numValuesToAppend) final;
common::offset_t startPosInChunk, uint32_t numValuesToAppend) override;
};

class NullColumnChunk : public BoolColumnChunk {
public:
NullColumnChunk() : BoolColumnChunk(nullptr /*copyDescription*/, false /*hasNullChunk*/) {}
// Maybe this should be combined with BoolColumnChunk if the only difference is these functions?
inline bool isNull(common::offset_t pos) const { return getValue<bool>(pos); }
inline void setNull(common::offset_t pos, bool isNull) { setValue(isNull, pos); }
inline void setNull(common::offset_t pos, bool isNull) {
setValue(isNull, pos);
if (isNull) {
mayHaveNullValue = true;
}
}

inline bool mayHaveNull() const { return mayHaveNullValue; }

void resize(uint64_t numValues) final;

inline void resetNullBuffer() { memset(buffer.get(), 0 /* non null */, bufferSize); }
inline void resetNullBuffer() {
memset(buffer.get(), 0 /* non null */, bufferSize);
mayHaveNullValue = false;
}

void copyFromBuffer(uint64_t* srcBuffer, uint64_t srcOffset, uint64_t dstOffset,
uint64_t numBits, bool invert = false) {
if (common::NullMask::copyNullMask(

Check warning on line 212 in src/include/storage/copier/column_chunk.h

View check run for this annotation

Codecov / codecov/patch

src/include/storage/copier/column_chunk.h#L212

Added line #L212 was not covered by tests
srcBuffer, srcOffset, (uint64_t*)buffer.get(), dstOffset, numBits, invert)) {
mayHaveNullValue = true;

Check warning on line 214 in src/include/storage/copier/column_chunk.h

View check run for this annotation

Codecov / codecov/patch

src/include/storage/copier/column_chunk.h#L214

Added line #L214 was not covered by tests
}
}

void append(ColumnChunk* other, common::offset_t startPosInOtherChunk,
common::offset_t startPosInChunk, uint32_t numValuesToAppend) final;

protected:
inline uint64_t numBytesForValues(common::offset_t numValues) const {
Expand All @@ -208,6 +229,8 @@ class NullColumnChunk : public BoolColumnChunk {
// Each byte defaults to 0, indicating everything is non-null
buffer = std::make_unique<uint8_t[]>(bufferSize);
}

bool mayHaveNullValue = false;
};

class FixedListColumnChunk : public ColumnChunk {
Expand Down
18 changes: 11 additions & 7 deletions src/include/storage/store/node_column.h
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@
#include "storage/copier/column_chunk.h"
#include "storage/storage_structure/disk_array.h"
#include "storage/storage_structure/storage_structure.h"
#include "storage/store/property_statistics.h"

namespace kuzu {
namespace transaction {
Expand Down Expand Up @@ -57,10 +58,11 @@ class NodeColumn {
public:
NodeColumn(const catalog::Property& property, BMFileHandle* dataFH, BMFileHandle* metadataFH,
BufferManager* bufferManager, WAL* wal, transaction::Transaction* transaction,
bool requireNullColumn = true);
RWPropertyStats propertyStatistics, bool requireNullColumn = true);
NodeColumn(common::LogicalType dataType, const catalog::MetadataDAHInfo& metaDAHeaderInfo,
BMFileHandle* dataFH, BMFileHandle* metadataFH, BufferManager* bufferManager, WAL* wal,
transaction::Transaction* transaction, bool requireNullColumn);
transaction::Transaction* transaction, RWPropertyStats PropertyStatistics,
bool requireNullColumn);
virtual ~NodeColumn() = default;

// Expose for feature store
Expand Down Expand Up @@ -140,13 +142,15 @@ class NodeColumn {
std::vector<std::unique_ptr<NodeColumn>> childrenColumns;
read_node_column_func_t readNodeColumnFunc;
write_node_column_func_t writeNodeColumnFunc;
RWPropertyStats propertyStatistics;
};

class BoolNodeColumn : public NodeColumn {
public:
BoolNodeColumn(const catalog::MetadataDAHInfo& metaDAHeaderInfo, BMFileHandle* dataFH,
BMFileHandle* metadataFH, BufferManager* bufferManager, WAL* wal,
transaction::Transaction* transaction, bool requireNullColumn = true);
transaction::Transaction* transaction, RWPropertyStats propertyStatistics,
bool requireNullColumn = true);

void batchLookup(transaction::Transaction* transaction, const common::offset_t* nodeOffsets,
size_t size, uint8_t* result) final;
Expand All @@ -158,7 +162,7 @@ class NullNodeColumn : public NodeColumn {
public:
NullNodeColumn(common::page_idx_t metaDAHPageIdx, BMFileHandle* dataFH,
BMFileHandle* metadataFH, BufferManager* bufferManager, WAL* wal,
transaction::Transaction* transaction);
transaction::Transaction* transaction, RWPropertyStats propertyStatistics);

void scan(transaction::Transaction* transaction, common::ValueVector* nodeIDVector,
common::ValueVector* resultVector) final;
Expand Down Expand Up @@ -190,14 +194,14 @@ class SerialNodeColumn : public NodeColumn {
struct NodeColumnFactory {
static inline std::unique_ptr<NodeColumn> createNodeColumn(const catalog::Property& property,
BMFileHandle* dataFH, BMFileHandle* metadataFH, BufferManager* bufferManager, WAL* wal,
transaction::Transaction* transaction) {
transaction::Transaction* transaction, RWPropertyStats propertyStatistics) {
return createNodeColumn(*property.getDataType(), *property.getMetadataDAHInfo(), dataFH,
metadataFH, bufferManager, wal, transaction);
metadataFH, bufferManager, wal, transaction, propertyStatistics);
}
static std::unique_ptr<NodeColumn> createNodeColumn(const common::LogicalType& dataType,
const catalog::MetadataDAHInfo& metaDAHeaderInfo, BMFileHandle* dataFH,
BMFileHandle* metadataFH, BufferManager* bufferManager, WAL* wal,
transaction::Transaction* transaction);
transaction::Transaction* transaction, RWPropertyStats propertyStatistics);
};

} // namespace storage
Expand Down
29 changes: 16 additions & 13 deletions src/include/storage/store/nodes_statistics_and_deleted_ids.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,19 +11,17 @@ namespace storage {
class NodeStatisticsAndDeletedIDs : public TableStatistics {

public:
NodeStatisticsAndDeletedIDs(common::table_id_t tableID, common::offset_t maxNodeOffset)
NodeStatisticsAndDeletedIDs(common::table_id_t tableID, common::offset_t maxNodeOffset,
TablePropertyStats&& propertyStatistics)
: NodeStatisticsAndDeletedIDs(tableID, maxNodeOffset,
std::vector<
common::offset_t>() /* no deleted node offsets during initial loading */) {}
std::vector<common::offset_t>() /* no deleted node offsets during initial loading */,
std::move(propertyStatistics)) {}

NodeStatisticsAndDeletedIDs(common::table_id_t tableID, common::offset_t maxNodeOffset,
const std::vector<common::offset_t>& deletedNodeOffsets);
const std::vector<common::offset_t>& deletedNodeOffsets,
TablePropertyStats&& propertyStatistics);

NodeStatisticsAndDeletedIDs(const NodeStatisticsAndDeletedIDs& other)
: TableStatistics{other.getNumTuples()}, tableID{other.tableID},
adjListsAndColumns{other.adjListsAndColumns},
hasDeletedNodesPerMorsel{other.hasDeletedNodesPerMorsel},
deletedNodeOffsetsPerMorsel{other.deletedNodeOffsetsPerMorsel} {}
NodeStatisticsAndDeletedIDs(const NodeStatisticsAndDeletedIDs& other) = default;

inline common::offset_t getMaxNodeOffset() {
return getMaxNodeOffsetFromNumTuples(getNumTuples());
Expand All @@ -48,7 +46,7 @@ class NodeStatisticsAndDeletedIDs : public TableStatistics {

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

static inline uint64_t geNumTuplesFromMaxNodeOffset(common::offset_t maxNodeOffset) {
static inline uint64_t getNumTuplesFromMaxNodeOffset(common::offset_t maxNodeOffset) {
return (maxNodeOffset == UINT64_MAX) ? 0ull : maxNodeOffset + 1ull;
}

Expand Down Expand Up @@ -86,7 +84,7 @@ class NodesStatisticsAndDeletedIDs : public TablesStatistics {
logger->info("Initialized {}.", "NodesStatisticsAndDeletedIDs");
}

// Should be used ony by tests;
// Should be used only by tests;
explicit NodesStatisticsAndDeletedIDs(
std::unordered_map<common::table_id_t, std::unique_ptr<NodeStatisticsAndDeletedIDs>>&
nodesStatisticsAndDeletedIDs);
Expand Down Expand Up @@ -179,10 +177,14 @@ class NodesStatisticsAndDeletedIDs : public TablesStatistics {

inline std::unique_ptr<TableStatistics> constructTableStatistic(
catalog::TableSchema* tableSchema) override {
TablePropertyStats propertyStats;
for (auto property : tableSchema->getProperties()) {
propertyStats[property->getPropertyID()] = std::make_unique<PropertyStatistics>();
}
// We use UINT64_MAX to represent an empty nodeTable which doesn't contain
// any nodes.
return std::make_unique<NodeStatisticsAndDeletedIDs>(
tableSchema->tableID, UINT64_MAX /* maxNodeOffset */);
tableSchema->tableID, UINT64_MAX /* maxNodeOffset */, std::move(propertyStats));
}

inline std::unique_ptr<TableStatistics> constructTableStatistic(
Expand All @@ -197,7 +199,8 @@ class NodesStatisticsAndDeletedIDs : public TablesStatistics {
}

std::unique_ptr<TableStatistics> deserializeTableStatistics(uint64_t numTuples,
uint64_t& offset, common::FileInfo* fileInfo, uint64_t tableID) override;
TablePropertyStats&& propertyStats, uint64_t& offset, common::FileInfo* fileInfo,
uint64_t tableID) override;

void serializeTableStatistics(
TableStatistics* tableStatistics, uint64_t& offset, common::FileInfo* fileInfo) override;
Expand Down
47 changes: 47 additions & 0 deletions src/include/storage/store/property_statistics.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
#pragma once

#include "common/types/types.h"

namespace kuzu {
namespace storage {

class PropertyStatistics {
public:
PropertyStatistics() = default;
explicit PropertyStatistics(bool mayHaveNullValue) : mayHaveNullValue{mayHaveNullValue} {}

inline bool mayHaveNull() const { return mayHaveNullValue; }

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

inline void setHasNull() { mayHaveNullValue = true; }

private:
// Stores whether or not the property is known to have contained a null value
// If false, the property is guaranteed to not contain any nulls
bool mayHaveNullValue = false;
};

class TablesStatistics;

// Accessor used by NodeColumn, so that it doesn't need to handle the TableStatistics directly
class RWPropertyStats {
public:
RWPropertyStats(TablesStatistics* tablesStatistics, common::table_id_t tableID,
common::property_id_t propertyID);

static RWPropertyStats empty() { return RWPropertyStats(nullptr, 0, 0); }

bool mayHaveNull();
void setHasNull();

private:
TablesStatistics* tablesStatistics;
common::table_id_t tableID;
common::property_id_t propertyID;
};

} // namespace storage
} // namespace kuzu
13 changes: 8 additions & 5 deletions src/include/storage/store/rels_statistics.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,11 @@ class RelStatistics : public TableStatistics {
friend class RelsStatistics;

public:
RelStatistics() : TableStatistics{0 /* numTuples */}, nextRelOffset{0} {}
RelStatistics(uint64_t numRels, common::offset_t nextRelOffset)
: TableStatistics{numRels}, nextRelOffset{nextRelOffset} {}
RelStatistics(const catalog::TableSchema& tableSchema)
: TableStatistics{tableSchema}, nextRelOffset{0} {}
RelStatistics(
uint64_t numRels, TablePropertyStats&& propertyStats, common::offset_t nextRelOffset)
: TableStatistics{numRels, std::move(propertyStats)}, nextRelOffset{nextRelOffset} {}

inline common::offset_t getNextRelOffset() const { return nextRelOffset; }

Expand Down Expand Up @@ -64,7 +66,7 @@ class RelsStatistics : public TablesStatistics {

inline std::unique_ptr<TableStatistics> constructTableStatistic(
catalog::TableSchema* tableSchema) override {
return std::make_unique<RelStatistics>();
return std::make_unique<RelStatistics>(*tableSchema);
}

inline std::unique_ptr<TableStatistics> constructTableStatistic(
Expand All @@ -84,7 +86,8 @@ class RelsStatistics : public TablesStatistics {
}

std::unique_ptr<TableStatistics> deserializeTableStatistics(uint64_t numTuples,
uint64_t& offset, common::FileInfo* fileInfo, uint64_t tableID) override;
TablePropertyStats&& propertyStats, uint64_t& offset, common::FileInfo* fileInfo,
uint64_t tableID) override;

void serializeTableStatistics(
TableStatistics* tableStatistics, uint64_t& offset, common::FileInfo* fileInfo) override;
Expand Down
3 changes: 2 additions & 1 deletion src/include/storage/store/string_node_column.h
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
#pragma once

#include "node_column.h"
#include "storage/store/table_statistics.h"

namespace kuzu {
namespace storage {
Expand All @@ -14,7 +15,7 @@ class StringNodeColumn : public NodeColumn {
public:
StringNodeColumn(common::LogicalType dataType, const catalog::MetadataDAHInfo& metaDAHeaderInfo,
BMFileHandle* dataFH, BMFileHandle* metadataFH, BufferManager* bufferManager, WAL* wal,
transaction::Transaction* transaction);
transaction::Transaction* transaction, RWPropertyStats propertyStatistics);

void scan(transaction::Transaction* transaction, common::node_group_idx_t nodeGroupIdx,
common::offset_t startOffsetInGroup, common::offset_t endOffsetInGroup,
Expand Down
3 changes: 2 additions & 1 deletion src/include/storage/store/struct_node_column.h
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
#include "node_column.h"
#include "storage/store/table_statistics.h"

namespace kuzu {
namespace storage {
Expand All @@ -7,7 +8,7 @@ class StructNodeColumn : public NodeColumn {
public:
StructNodeColumn(common::LogicalType dataType, const catalog::MetadataDAHInfo& metaDAHeaderInfo,
BMFileHandle* dataFH, BMFileHandle* metadataFH, BufferManager* bufferManager, WAL* wal,
transaction::Transaction* transaction);
transaction::Transaction* transaction, RWPropertyStats propertyStatistics);

void scan(transaction::Transaction* transaction, common::node_group_idx_t nodeGroupIdx,
common::offset_t startOffsetInGroup, common::offset_t endOffsetInGroup,
Expand Down
Loading

0 comments on commit 732229d

Please sign in to comment.