Skip to content

Commit

Permalink
Store nulls as densely packed bitfields
Browse files Browse the repository at this point in the history
instead of one value per byte
  • Loading branch information
benjaminwinger committed Aug 2, 2023
1 parent baf9e56 commit f7f6de2
Show file tree
Hide file tree
Showing 13 changed files with 121 additions and 39 deletions.
16 changes: 12 additions & 4 deletions src/common/null_mask.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -5,14 +5,13 @@
namespace kuzu {
namespace common {

void NullMask::setNull(uint32_t pos, bool isNull) {
void NullMask::setNull(uint64_t* nullEntries, uint32_t pos, bool isNull) {
auto entryPos = pos >> NUM_BITS_PER_NULL_ENTRY_LOG2;
auto bitPosInEntry = pos - (entryPos << NUM_BITS_PER_NULL_ENTRY_LOG2);
if (isNull) {
data[entryPos] |= NULL_BITMASKS_WITH_SINGLE_ONE[bitPosInEntry];
mayContainNulls = true;
nullEntries[entryPos] |= NULL_BITMASKS_WITH_SINGLE_ONE[bitPosInEntry];
} else {
data[entryPos] &= NULL_BITMASKS_WITH_SINGLE_ZERO[bitPosInEntry];
nullEntries[entryPos] &= NULL_BITMASKS_WITH_SINGLE_ZERO[bitPosInEntry];
}
}

Expand Down Expand Up @@ -87,5 +86,14 @@ void NullMask::resize(uint64_t capacity) {
numNullEntries = capacity;
}

bool NullMask::copyFromNullBits(const uint64_t* srcNullEntries, uint64_t srcOffset,
uint64_t dstOffset, uint64_t numBitsToCopy) {
if (copyNullMask(srcNullEntries, srcOffset, this->data, dstOffset, numBitsToCopy)) {
this->mayContainNulls = true;
return true;
}
return false;
}

} // namespace common
} // namespace kuzu
3 changes: 3 additions & 0 deletions src/common/types/types.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -311,6 +311,9 @@ void LogicalType::setPhysicalType() {
case LogicalTypeID::BOOL: {
physicalType = PhysicalTypeID::BOOL;
} break;
case LogicalTypeID::NULL_: {
physicalType = PhysicalTypeID::NULL_;
} break;
case LogicalTypeID::TIMESTAMP:
case LogicalTypeID::SERIAL:
case LogicalTypeID::INT64: {
Expand Down
5 changes: 5 additions & 0 deletions src/common/vector/value_vector.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,11 @@ bool NodeIDVector::discardNull(ValueVector& vector) {
}
}

bool ValueVector::setNullFromBits(const uint64_t* srcNullEntries, uint64_t srcOffset,
uint64_t dstOffset, uint64_t numBitsToCopy) {
return nullMask->copyFromNullBits(srcNullEntries, srcOffset, dstOffset, numBitsToCopy);
}

template<typename T>
void ValueVector::setValue(uint32_t pos, T val) {
((T*)valueBuffer.get())[pos] = val;
Expand Down
29 changes: 24 additions & 5 deletions src/include/common/null_mask.h
Original file line number Diff line number Diff line change
Expand Up @@ -96,10 +96,15 @@ class NullMask {
mayContainNulls = true;
}

inline void setMayContainNulls() { mayContainNulls = true; }
inline bool hasNoNullsGuarantee() const { return !mayContainNulls; }

void setNull(uint32_t pos, bool isNull);
static void setNull(uint64_t* nullEntries, uint32_t pos, bool isNull);
inline void setNull(uint32_t pos, bool isNull) {
setNull(data, pos, isNull);
if (isNull) {
mayContainNulls = true;
}
}

static inline bool isNull(const uint64_t* nullEntries, uint32_t pos) {
auto [entryPos, bitPosInEntry] = getNullEntryAndBitPos(pos);
Expand All @@ -108,18 +113,32 @@ class NullMask {

inline bool isNull(uint32_t pos) const { return isNull(data, pos); }

inline uint64_t* getData() { return data; }
// const because updates to the data must set mayContainNulls if any value
// becomes non-null
// Modifying the underlying data shuld be done with setNull or copyFromNullData
inline const uint64_t* getData() { return data; }

static inline uint64_t getNumNullEntries(uint64_t numNullBits) {
return (numNullBits >> NUM_BITS_PER_NULL_ENTRY_LOG2) +
((numNullBits - (numNullBits << NUM_BITS_PER_NULL_ENTRY_LOG2)) == 0 ? 0 : 1);
}

// This function returns true if we have copied a nullBit with value 1 (indicate a null
// value) to dstNullEntries.
// @brief Copies bitpacked null flags from one buffer to another
//
// @param srcNullEntries The bit data being copied
// @param srcOffset offset of the first source bit
// @param dstNullEntries The destination buffer of the copy
// @param dstOffset offset of the first destination bit
// @param numBitsToCopy The number of bits to copy from srcNullEntries to dstNullEntries
//
// @return true if we have copied a nullBit with value 1 (indicates a null value) to
// dstNullEntries.
static bool copyNullMask(const uint64_t* srcNullEntries, uint64_t srcOffset,
uint64_t* dstNullEntries, uint64_t dstOffset, uint64_t numBitsToCopy);

bool copyFromNullBits(const uint64_t* srcNullEntries, uint64_t srcOffset, uint64_t dstOffset,
uint64_t numBitsToCopy);

void resize(uint64_t capacity);

private:
Expand Down
2 changes: 2 additions & 0 deletions src/include/common/types/types.h
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@ KUZU_API enum class LogicalTypeID : uint8_t {
SERIAL = 13,

// fixed size types
NULL_ = 21,
BOOL = 22,
INT64 = 23,
INT32 = 24,
Expand Down Expand Up @@ -110,6 +111,7 @@ enum class PhysicalTypeID : uint8_t {
INTERVAL = 7,
INTERNAL_ID = 9,
ARROW_COLUMN = 10,
NULL_ = 11,

// Variable size types.
STRING = 20,
Expand Down
6 changes: 4 additions & 2 deletions src/include/common/vector/value_vector.h
Original file line number Diff line number Diff line change
Expand Up @@ -34,22 +34,24 @@ class ValueVector {

inline void setAllNull() { nullMask->setAllNull(); }
inline void setAllNonNull() { nullMask->setAllNonNull(); }
inline void setMayContainNulls() { nullMask->setMayContainNulls(); }
// On return true, there are no null. On return false, there may or may not be nulls.
inline bool hasNoNullsGuarantee() const { return nullMask->hasNoNullsGuarantee(); }
inline void setRangeNonNull(uint32_t startPos, uint32_t len) {
for (auto i = 0u; i < len; ++i) {
setNull(startPos + i, false);
}
}
inline uint64_t* getNullMaskData() { return nullMask->getData(); }
inline const uint64_t* getNullMaskData() { return nullMask->getData(); }
inline void setNull(uint32_t pos, bool isNull) { nullMask->setNull(pos, isNull); }
inline uint8_t isNull(uint32_t pos) const { return nullMask->isNull(pos); }
inline void setAsSingleNullEntry() {
state->selVector->selectedSize = 1;
setNull(state->selVector->selectedPositions[0], true);
}

bool setNullFromBits(const uint64_t* srcNullEntries, uint64_t srcOffset, uint64_t dstOffset,
uint64_t numBitsToCopy);

inline uint32_t getNumBytesPerValue() const { return numBytesPerValue; }

template<typename T>
Expand Down
19 changes: 16 additions & 3 deletions src/include/storage/copier/column_chunk.h
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ class ColumnChunk {

virtual common::page_idx_t flushBuffer(BMFileHandle* dataFH, common::page_idx_t startPageIdx);

/// Returns the size of the data type in bits
static uint32_t getDataTypeSizeInChunk(common::LogicalType& dataType);

template<typename T>
Expand Down Expand Up @@ -89,6 +90,7 @@ class ColumnChunk {

protected:
common::LogicalType dataType;
uint32_t numBitsPerValue;
uint32_t numBytesPerValue;
uint64_t numBytes;
std::unique_ptr<uint8_t[]> buffer;
Expand All @@ -100,15 +102,26 @@ class ColumnChunk {
class NullColumnChunk : public ColumnChunk {
public:
NullColumnChunk()
: ColumnChunk(common::LogicalType(common::LogicalTypeID::BOOL),
: ColumnChunk(common::LogicalType(common::LogicalTypeID::NULL_),
nullptr /* copyDescription */, false /* hasNullChunk */) {
resetNullBuffer();
}

inline void resetNullBuffer() { memset(buffer.get(), 0 /* non null */, numBytes); }

inline bool isNull(common::offset_t pos) const { return getValue<bool>(pos); }
inline void setNull(common::offset_t pos, bool isNull) { ((bool*)buffer.get())[pos] = isNull; }
inline bool isNull(common::offset_t pos) const {
// Buffer is rounded up to the nearest 8 bytes so that this cast is safe
return common::NullMask::isNull((uint64_t*)buffer.get(), pos);
}
inline void setNull(common::offset_t pos, bool isNull) {
using common::NullMask;
NullMask::setNull(
// Buffer is rounded up to the nearest 8 bytes so that this cast is safe
(uint64_t*)buffer.get(), pos, isNull);
}

void appendColumnChunk(NullColumnChunk* other, common::offset_t startPosInOtherChunk,
common::offset_t startPosInChunk, uint32_t numValuesToAppend);
};

class FixedListColumnChunk : public ColumnChunk {
Expand Down
2 changes: 1 addition & 1 deletion src/include/storage/storage_structure/lists/lists.h
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ struct InMemList {

inline uint8_t* getListData() const { return listData.get(); }
inline bool hasNullBuffer() const { return nullMask != nullptr; }
inline uint64_t* getNullMask() const { return nullMask->getData(); }
inline common::NullMask* getNullMask() const { return nullMask.get(); }

uint64_t numElements;
std::unique_ptr<uint8_t[]> listData;
Expand Down
37 changes: 31 additions & 6 deletions src/storage/copier/column_chunk.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,19 @@ ColumnChunk::ColumnChunk(LogicalType dataType, CopyDescription* copyDescription,

ColumnChunk::ColumnChunk(
LogicalType dataType, offset_t numValues, CopyDescription* copyDescription, bool hasNullChunk)
: dataType{std::move(dataType)}, numBytesPerValue{getDataTypeSizeInChunk(this->dataType)},
numBytes{numBytesPerValue * numValues}, copyDescription{copyDescription} {
: dataType{std::move(dataType)}, numBitsPerValue{getDataTypeSizeInChunk(this->dataType)},
numBytesPerValue{numBitsPerValue / 8}, numBytes{numBitsPerValue * numValues / 8},
copyDescription{copyDescription} {
// TODO(bmwinger): Move logic into NullColumnChunk
// Maybe we need a BaseColumnChunk which doesn't initialize the buffer, so that each subclass
// can create a buffer that meets their requirements. Particularly when adding other compression
// types which will presumably also have different size quirks. But it might be worth making a
// more complex plan for that later.
// buffer size needs to be a multiple of 8 bytes so that NullMask can treat it as a 64-bit
// buffer.
if (this->dataType.getLogicalTypeID() == LogicalTypeID::NULL_) {
numBytes = std::ceil(numBytes / 8.0) * 8;
}
buffer = std::make_unique<uint8_t[]>(numBytes);
if (hasNullChunk) {
nullChunk = std::make_unique<NullColumnChunk>();
Expand Down Expand Up @@ -224,20 +235,34 @@ uint32_t ColumnChunk::getDataTypeSizeInChunk(common::LogicalType& dataType) {
return 0;
}
case LogicalTypeID::STRING: {
return sizeof(ku_string_t);
return sizeof(ku_string_t) * 8;
}
case LogicalTypeID::VAR_LIST: {
return sizeof(ku_list_t);
return sizeof(ku_list_t) * 8;
}
case LogicalTypeID::INTERNAL_ID: {
return sizeof(offset_t);
return sizeof(offset_t) * 8;
}
case LogicalTypeID::NULL_: {
return 1;
}
default: {
return StorageUtils::getDataTypeSize(dataType);
return StorageUtils::getDataTypeSize(dataType) * 8;
}
}
}

// TODO(bmwinger): Eventually, to support bitpacked bools, all these functions will need to be
// updated to support values sizes of less than one byte.
// But for the moment, this is the only generic ColumnChunk function which is needed by
// NullColumnChunk, and it's invoked directly on the nullColumn, so we don't need dynamic dispatch
void NullColumnChunk::appendColumnChunk(NullColumnChunk* other,
common::offset_t startPosInOtherChunk, common::offset_t startPosInChunk,
uint32_t numValuesToAppend) {
NullMask::copyNullMask((uint64_t*)other->buffer.get(), startPosInOtherChunk,
(uint64_t*)buffer.get(), startPosInChunk, numValuesToAppend);
}

void FixedListColumnChunk::append(ColumnChunk* other, common::offset_t startPosInOtherChunk,
common::offset_t startPosInChunk, uint32_t numValuesToAppend) {
auto otherChunk = (FixedListColumnChunk*)other;
Expand Down
7 changes: 3 additions & 4 deletions src/storage/storage_structure/lists/lists.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ void Lists::fillInMemListsFromFrame(InMemList& inMemList, const uint8_t* frame,
if (deletedRelOffsetsInList.empty()) {
memcpy(listData, frameData, numElementsToReadInCurPage * elementSize);
if (inMemList.hasNullBuffer()) {
NullMask::copyNullMask(nullBufferInPage, elemPosInPage, inMemList.getNullMask(),
inMemList.getNullMask()->copyFromNullBits(nullBufferInPage, elemPosInPage,
nextPosToWriteToInMemList, numElementsToReadInCurPage);
}
readPropertyUpdatesToInMemListIfExists(inMemList, updatedPersistentListOffsets,
Expand All @@ -164,9 +164,8 @@ void Lists::fillInMemListsFromFrame(InMemList& inMemList, const uint8_t* frame,
// Otherwise, we can directly read from persistentStore.
memcpy(listData, frameData, elementSize);
if (inMemList.hasNullBuffer()) {
NullMask::copyNullMask(nullBufferInPage, elemPosInPage,
inMemList.getNullMask(), nextPosToWriteToInMemList,
1 /* numBitsToCopy */);
inMemList.getNullMask()->copyFromNullBits(nullBufferInPage, elemPosInPage,
nextPosToWriteToInMemList, 1 /* numBitsToCopy */);
}
}
listData += elementSize;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -247,7 +247,7 @@ void ListsUpdateIterator::writeAtOffset(InMemList& inMemList, page_idx_t pageLis
memcpy(frame + lists->getElemByteOffset(elementOffsetInListPage), dataToFill,
numElementsToWriteToCurrentPage * elementSize);
if (inMemList.hasNullBuffer()) {
NullMask::copyNullMask(inMemList.getNullMask(), numUpdatedElements,
NullMask::copyNullMask(inMemList.getNullMask()->getData(), numUpdatedElements,
(uint64_t*)lists->getNullBufferInPage(frame), elementOffsetInListPage,
numElementsToWriteToCurrentPage);
}
Expand Down
7 changes: 2 additions & 5 deletions src/storage/storage_structure/storage_structure.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -102,11 +102,8 @@ void BaseColumnOrList::readInternalIDsFromAPageBySequentialCopy(Transaction* tra

void BaseColumnOrList::readNullBitsFromAPage(ValueVector* valueVector, const uint8_t* frame,
uint64_t posInPage, uint64_t posInVector, uint64_t numBitsToRead) const {
auto hasNullInSrcNullMask = NullMask::copyNullMask((uint64_t*)getNullBufferInPage(frame),
posInPage, valueVector->getNullMaskData(), posInVector, numBitsToRead);
if (hasNullInSrcNullMask) {
valueVector->setMayContainNulls();
}
valueVector->setNullFromBits(
(uint64_t*)getNullBufferInPage(frame), posInPage, posInVector, numBitsToRead);
}

void BaseColumnOrList::readAPageBySequentialCopy(Transaction* transaction, ValueVector* vector,
Expand Down
25 changes: 17 additions & 8 deletions src/storage/store/node_column.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -43,15 +43,19 @@ void FixedSizedNodeColumnFunc::writeInternalIDValueToPage(

void NullNodeColumnFunc::readValuesFromPage(uint8_t* frame, PageElementCursor& pageCursor,
ValueVector* resultVector, uint32_t posInVector, uint32_t numValuesToRead) {
for (auto i = 0u; i < numValuesToRead; i++) {
bool isNull = *(frame + pageCursor.elemPosInPage + i);
resultVector->setNull(posInVector + i, isNull);
}
// Read bit-packed null flags from the frame into the result vector
// Casting to uint64_t should be safe as long as the page size is a multiple of 8 bytes.
// Otherwise it could read off the end of the page.
resultVector->setNullFromBits(
(uint64_t*)frame, pageCursor.elemPosInPage, posInVector, numValuesToRead);
}

void NullNodeColumnFunc::writeValueToPage(
uint8_t* frame, uint16_t posInFrame, common::ValueVector* vector, uint32_t posInVector) {
*(frame + posInFrame) = vector->isNull(posInVector);
// Casting to uint64_t should be safe as long as the page size is a multiple of 8 bytes.
// Otherwise it could read off the end of the page.
NullMask::setNull(
(uint64_t*)frame, posInFrame, NullMask::isNull(vector->getNullMaskData(), posInVector));
}

NodeColumn::NodeColumn(const Property& property, BMFileHandle* dataFH, BMFileHandle* metadataFH,
Expand All @@ -66,7 +70,7 @@ NodeColumn::NodeColumn(LogicalType dataType, const MetadataDAHInfo& metaDAHeader
dataFH{dataFH}, metadataFH{metadataFH}, bufferManager{bufferManager}, wal{wal} {
metadataDA = std::make_unique<InMemDiskArray<ColumnChunkMetadata>>(*metadataFH,
StorageStructureID::newMetadataID(), metaDAHeaderInfo.dataDAHPageIdx, bufferManager, wal);
numBytesPerFixedSizedValue = ColumnChunk::getDataTypeSizeInChunk(this->dataType);
numBytesPerFixedSizedValue = ColumnChunk::getDataTypeSizeInChunk(this->dataType) / 8;
assert(numBytesPerFixedSizedValue <= BufferPoolConstants::PAGE_4KB_SIZE);
numValuesPerPage =
numBytesPerFixedSizedValue == 0 ?
Expand Down Expand Up @@ -344,10 +348,15 @@ void NodeColumn::rollbackInMemory() {

NullNodeColumn::NullNodeColumn(page_idx_t metaDAHPageIdx, BMFileHandle* dataFH,
BMFileHandle* metadataFH, BufferManager* bufferManager, WAL* wal)
: NodeColumn{LogicalType(LogicalTypeID::BOOL), MetadataDAHInfo{metaDAHPageIdx}, dataFH,
: NodeColumn{LogicalType(LogicalTypeID::NULL_), MetadataDAHInfo{metaDAHPageIdx}, dataFH,
metadataFH, bufferManager, wal, false /* requireNullColumn */} {
readNodeColumnFunc = NullNodeColumnFunc::readValuesFromPage;
writeNodeColumnFunc = NullNodeColumnFunc::writeValueToPage;
// 8 values per byte
numValuesPerPage = PageUtils::getNumElementsInAPage(1, false) * 8;
// Page size must be aligned to 8 byte chunks for the 64-bit NullMask algorithms to work
// without the possibility of memory errors from reading/writing off the end of a page.
assert(PageUtils::getNumElementsInAPage(1, false) % 8 == 0);
}

void NullNodeColumn::scan(
Expand All @@ -371,7 +380,7 @@ page_idx_t NullNodeColumn::append(
void NullNodeColumn::setNull(common::offset_t nodeOffset) {
auto walPageInfo = createWALVersionOfPageForValue(nodeOffset);
try {
*(walPageInfo.frame + walPageInfo.posInPage) = true;
common::NullMask::setNull((uint64_t*)walPageInfo.frame, walPageInfo.posInPage, true);
} catch (Exception& e) {
bufferManager->unpin(*wal->fileHandle, walPageInfo.pageIdxInWAL);
dataFH->releaseWALPageIdxLock(walPageInfo.originalPageIdx);
Expand Down

0 comments on commit f7f6de2

Please sign in to comment.