Skip to content

Commit

Permalink
refactoring write-recording/rollback and rogging/recovery of Update: …
Browse files Browse the repository at this point in the history
…WIP (2).
  • Loading branch information
ryogrid committed Jun 10, 2024
1 parent b2eb4bc commit 7cec517
Show file tree
Hide file tree
Showing 26 changed files with 57 additions and 48 deletions.
2 changes: 1 addition & 1 deletion lib/catalog/statistics.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ import (
"math"
)

var AbortedError error = errors.New("aborted")
var AbortedError = errors.New("aborted")

type distinctCounter struct {
max *types.Value
Expand Down
2 changes: 1 addition & 1 deletion lib/catalog/table_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ func (c *Catalog) GetAllTables() []*TableMetadata {
ret := make([]*TableMetadata, 0)
c.tableIdsMutex.Lock()
defer c.tableIdsMutex.Unlock()
for key, _ := range c.tableIds {
for key := range c.tableIds {
ret = append(ret, c.tableIds[key])
}
return ret
Expand Down
2 changes: 1 addition & 1 deletion lib/common/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ const (
// number for calculate log buffer size (number of page size)
LogBufferSizeBase = 128
// size of a log buffer in byte
LogBufferSize = ((LogBufferSizeBase + 1) * PageSize)
LogBufferSize = (LogBufferSizeBase + 1) * PageSize
// size of hash bucket
BucketSizeOfHashIndex = 10
// probability used for determin node level on SkipList
Expand Down
2 changes: 1 addition & 1 deletion lib/container/skip_list/skip_list.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ func (sl *SkipList) FindNode(key *types.Value, opType SkipListOpType) (isSuccess
// entry of corners is corner node or target node
corners := make([]skip_list_page.SkipListCornerInfo, skip_list_page.MAX_FOWARD_LIST_LEN)
var curr *skip_list_page.SkipListBlockPage = nil
for ii := (skip_list_page.MAX_FOWARD_LIST_LEN - 1); ii >= 0; ii-- {
for ii := skip_list_page.MAX_FOWARD_LIST_LEN - 1; ii >= 0; ii-- {
//fmt.Printf("level %d\n", i)
for {
if pred == sl.startNode && pred.GetForwardEntry(ii) == sl.SentinelNodeID {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ type workArray struct {
func (arr *workArray) GetNewWork(threadNum int32) (work []*opTypeAndVal, done bool) {
arr.mutex.Lock()
defer arr.mutex.Unlock()
splitedWorkNum := (WORK_NUM / threadNum)
splitedWorkNum := WORK_NUM / threadNum

if arr.pos+splitedWorkNum <= WORK_NUM {
retArr := arr.arr[arr.pos : arr.pos+splitedWorkNum]
Expand Down
10 changes: 5 additions & 5 deletions lib/execution/executors/aggregation_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -167,8 +167,8 @@ func (aht *SimpleAggregationHashTable) InsertCombine(agg_key *plans.AggregateKey

/* return iterator to the start of the hash table */
func (aht *SimpleAggregationHashTable) Begin() *AggregateHTIterator {
var agg_key_list []*plans.AggregateKey = make([]*plans.AggregateKey, 0)
var agg_val_list []*plans.AggregateValue = make([]*plans.AggregateValue, 0)
var agg_key_list = make([]*plans.AggregateKey, 0)
var agg_val_list = make([]*plans.AggregateValue, 0)
for hval, val := range aht.ht_val {
agg_key_list = append(agg_key_list, aht.ht_key[hval])
agg_val_list = append(agg_val_list, val)
Expand Down Expand Up @@ -240,7 +240,7 @@ func (e *AggregationExecutor) Next() (*tuple.Tuple, Done, error) {
if e.aht_iterator_.IsEnd() {
return nil, true, nil
}
var values []types.Value = make([]types.Value, 0)
var values = make([]types.Value, 0)
for i := 0; i < len(e.exprs_); i++ {
values = append(values, e.exprs_[i].EvaluateAggregate(e.aht_iterator_.Key().Group_bys_, e.aht_iterator_.Val().Aggregates_))
}
Expand All @@ -251,7 +251,7 @@ func (e *AggregationExecutor) Next() (*tuple.Tuple, Done, error) {

/** return the tuple as an AggregateKey */
func (e *AggregationExecutor) MakeKey(tuple_ *tuple.Tuple) *plans.AggregateKey {
var keys []*types.Value = make([]*types.Value, 0)
var keys = make([]*types.Value, 0)
for _, expr := range e.plan_.GetGroupBys() {
tmp_val := expr.Evaluate(tuple_, e.child_[0].GetOutputSchema())
keys = append(keys, &tmp_val)
Expand All @@ -261,7 +261,7 @@ func (e *AggregationExecutor) MakeKey(tuple_ *tuple.Tuple) *plans.AggregateKey {

/** return the tuple as an AggregateValue */
func (e *AggregationExecutor) MakeVal(tuple_ *tuple.Tuple) *plans.AggregateValue {
var vals []*types.Value = make([]*types.Value, 0)
var vals = make([]*types.Value, 0)
for _, expr := range e.plan_.GetAggregates() {
tmp_val := expr.Evaluate(tuple_, e.child_[0].GetOutputSchema())
vals = append(vals, &tmp_val)
Expand Down
2 changes: 1 addition & 1 deletion lib/execution/executors/executor_test/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1869,7 +1869,7 @@ func TestSimpleGroupByAggregation(t *testing.T) {
executor := executionEngine.CreateExecutor(agg_plan, exec_ctx)
executor.Init()

var encountered map[int32]int32 = make(map[int32]int32, 0)
var encountered = make(map[int32]int32, 0)
for tuple_, done, _ := executor.Next(); !done; tuple_, done, _ = executor.Next() {
// Should have countA > 100
countA := tuple_.GetValue(agg_schema, agg_schema.GetColIndex("countA")).ToInteger()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1021,8 +1021,8 @@ func testParallelTxnsQueryingSkipListIndexUsedColumns[T int32 | float32 | string
common.ShPrintf(common.DEBUGGING, "Select(success) op start.\n")
diffToMakeNoExist := int32(10)
rangeSelectRetry:
var rangeStartKey T = samehada_util.ChoiceKeyFromMap(insVals)
var rangeEndKey T = samehada_util.ChoiceKeyFromMap(insVals)
var rangeStartKey = samehada_util.ChoiceKeyFromMap(insVals)
var rangeEndKey = samehada_util.ChoiceKeyFromMap(insVals)
for rangeEndKey < rangeStartKey {
goto rangeSelectRetry
}
Expand Down Expand Up @@ -1126,7 +1126,7 @@ func testParallelTxnsQueryingSkipListIndexUsedColumns[T int32 | float32 | string
// detect tuple which has illegal value at first column (unknown key based value)
// -- make map having values which should be in DB
okValMap := make(map[T]T, 0)
for baseKey, _ := range insVals {
for baseKey := range insVals {
for ii := int32(0); ii < stride; ii++ {
okVal := samehada_util.StrideAdd(samehada_util.StrideMul(baseKey, stride), ii).(T)
okValMap[okVal] = okVal
Expand Down Expand Up @@ -1193,7 +1193,7 @@ func testParallelTxnsQueryingSkipListIndexUsedColumns[T int32 | float32 | string
// detect tuple which has illegal value at second column (unknown key based value)
// -- make map having values which should be in DB
okValMap2 := make(map[int32]int32, 0)
for baseKey, _ := range insVals {
for baseKey := range insVals {
for ii := int32(0); ii < stride; ii++ {
okValBasedKey := samehada_util.StrideAdd(samehada_util.StrideMul(baseKey, stride), ii).(T)
okVal := getInt32ValCorrespondToPassVal(okValBasedKey)
Expand Down
2 changes: 1 addition & 1 deletion lib/execution/executors/orderby_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ func (e *OrderbyExecutor) Init() {
fmt.Printf("inserted_tuple_cnt %d\n", inserted_tuple_cnt)
// arrange tuple array (apply sort result)
tuple_cnt := len(e.sort_tuples_)
var tmp_tuples []*tuple.Tuple = make([]*tuple.Tuple, tuple_cnt)
var tmp_tuples = make([]*tuple.Tuple, tuple_cnt)
idx_of_orig_idx := len(e.plan_.GetColIdxs())
for idx := 0; idx < tuple_cnt; idx++ {
tmp_tuples[idx] = e.sort_tuples_[sort_values[idx][idx_of_orig_idx].ToInteger()]
Expand Down
2 changes: 1 addition & 1 deletion lib/execution/executors/update_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func (e *UpdateExecutor) Next() (*tuple.Tuple, Done, error) {
values := e.plan.GetRawValues()
new_tuple := tuple.NewTupleFromSchema(values, e.child.GetTableMetaData().Schema())

var is_updated bool = false
var is_updated = false
var new_rid *page.RID = nil
var updateErr error = nil
var updateTuple *tuple.Tuple
Expand Down
2 changes: 1 addition & 1 deletion lib/execution/expression/expression_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ func GetExpTreeStr(node interface{}) string {

childTraverse := func(exp Expression) string {
var idx uint32 = 0
var tmpStr string = ""
var tmpStr = ""
for exp.GetChildAt(idx) != nil && idx < math.MaxInt32 {
child := exp.GetChildAt(idx)
tmpStr += GetExpTreeStr(child)
Expand Down
2 changes: 1 addition & 1 deletion lib/planner/optimizer/optimizer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -414,7 +414,7 @@ func TestSimplePlanOptimization(t *testing.T) {
testingpkg.Assert(t, execRslt != nil, "execRslt == nil")
printBestPlan(patternName, queryStr, solution)
fmt.Print("values of first row: ")
for idx, _ := range queryInfo.SelectFields_ {
for idx := range queryInfo.SelectFields_ {
val := execRslt[0].GetValue(solution.OutputSchema(), uint32(idx))
fmt.Printf("%s(%d), ", val.ToString(), val.ValueType())
}
Expand Down
6 changes: 3 additions & 3 deletions lib/planner/optimizer/selinger_optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,8 +3,8 @@ package optimizer
import (
"errors"
mapset "github.com/deckarep/golang-set/v2"
stack "github.com/golang-collections/collections/stack"
pair "github.com/notEpsilon/go-pair"
"github.com/golang-collections/collections/stack"
"github.com/notEpsilon/go-pair"
"github.com/ryogrid/SamehadaDB/lib/catalog"
"github.com/ryogrid/SamehadaDB/lib/common"
"github.com/ryogrid/SamehadaDB/lib/execution/expression"
Expand Down Expand Up @@ -172,7 +172,7 @@ func (so *SelingerOptimizer) findBestScan(outNeededCols []*column.Column, where
candidates := availableKeyIndex()
// Prepare all range of candidates
ranges := make(map[int]*Range, 0)
for key, _ := range candidates {
for key := range candidates {
ranges[key] = NewRange(sc.GetColumn(uint32(key)).GetType())
}

Expand Down
4 changes: 2 additions & 2 deletions lib/planner/simple_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ func (pner *SimplePlanner) MakeSelectPlanWithJoin() (error, plans.Plan) {
var outSchemaL *schema.Schema
var scanPlanL plans.Plan
{
var columns []*column.Column = make([]*column.Column, 0)
var columns = make([]*column.Column, 0)
for _, col := range tgtTblColumnsL {
columns = append(columns, column.NewColumn(col.GetColumnName(), col.GetType(), false, index_constants.INDEX_KIND_INVALID, types.PageID(-1), col.GetExpr()))
}
Expand Down Expand Up @@ -361,7 +361,7 @@ func (pner *SimplePlanner) MakeUpdatePlan() (error, plans.Plan) {
// second, construc values list includes dummy value (not update target)
updateVals := make([]types.Value, tgtTblSchema.GetColumnCount())
// fill all elems with dummy
for idx, _ := range tgtTblSchema.GetColumns() {
for idx := range tgtTblSchema.GetColumns() {
updateVals[idx] = types.NewNull()
}
// overwrite elem which is update target
Expand Down
9 changes: 6 additions & 3 deletions lib/recovery/log_recovery/log_recovery.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,19 +66,22 @@ func (log_recovery *LogRecovery) DeserializeLogRecord(data []byte, log_record *r
if log_record.Log_record_type == recovery.INSERT {
binary.Read(bytes.NewBuffer(data[pos:]), binary.LittleEndian, &log_record.Insert_rid)
pos += uint32(unsafe.Sizeof(log_record.Insert_rid))
// we have provided serialize function for tuple class
log_record.Insert_tuple.DeserializeFrom(data[pos:])
} else if log_record.Log_record_type == recovery.APPLYDELETE ||
log_record.Log_record_type == recovery.MARKDELETE ||
log_record.Log_record_type == recovery.ROLLBACKDELETE {
binary.Read(bytes.NewBuffer(data[pos:]), binary.LittleEndian, &log_record.Delete_rid)
pos += uint32(unsafe.Sizeof(log_record.Delete_rid))
// we have provided serialize function for tuple class
log_record.Delete_tuple.DeserializeFrom(data[pos:])
} else if log_record.Log_record_type == recovery.UPDATE {
binary.Read(bytes.NewBuffer(data[pos:]), binary.LittleEndian, &log_record.Update_rid)
pos += uint32(unsafe.Sizeof(log_record.Update_rid))
// we have provided serialize function for tuple class
log_record.Old_tuple.DeserializeFrom(data[pos:])
pos += log_record.Old_tuple.Size() + uint32(tuple.TupleSizeOffsetInLogrecord)
log_record.New_tuple.DeserializeFrom(data[pos:])
} else if log_record.Log_record_type == recovery.FINALIZE_UPDATE {
binary.Read(bytes.NewBuffer(data[pos:]), binary.LittleEndian, &log_record.Update_rid)
pos += uint32(unsafe.Sizeof(log_record.Update_rid))
log_record.Old_tuple.DeserializeFrom(data[pos:])
pos += log_record.Old_tuple.Size() + uint32(tuple.TupleSizeOffsetInLogrecord)
log_record.New_tuple.DeserializeFrom(data[pos:])
Expand Down
2 changes: 1 addition & 1 deletion lib/samehada/request_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ func (reqManager *RequestManager) executeQuedTxns() {
// caller must having lock of queMutex
func (reqManager *RequestManager) handleAbortedByCCTxn(result *reqResult) {
// insert aborted request to head of que
reqManager.execQue = append([]*queryRequest{&queryRequest{result.reqId, result.query, result.callerCh}}, reqManager.execQue...)
reqManager.execQue = append([]*queryRequest{{result.reqId, result.query, result.callerCh}}, reqManager.execQue...)
//fmt.Println("add que aborted req")
}

Expand Down
4 changes: 2 additions & 2 deletions lib/samehada/samehada.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,10 +186,10 @@ func (sdb *SamehadaDB) ExecuteSQL(sqlStr string) (error, [][]interface{}) {
return ret.err, ret.result
}

var PlanCreationErr error = errors.New("plan creation error")
var PlanCreationErr = errors.New("plan creation error")

// temporal error
var QueryAbortedErr error = errors.New("query aborted")
var QueryAbortedErr = errors.New("query aborted")

func (sdb *SamehadaDB) ExecuteSQLRetValues(sqlStr string) (error, [][]*types.Value) {
qi, err := parser.ProcessSQLStr(&sqlStr)
Expand Down
4 changes: 2 additions & 2 deletions lib/samehada/samehada_util/samehada_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ func ChoiceKeyFromMap[T int32 | float32 | string, V int32 | float32 | string | b
index := rand.Intn(l)

var ans T
for k, _ := range m {
for k := range m {
if index == i {
ans = k
break
Expand Down Expand Up @@ -218,7 +218,7 @@ const SIGN_MASK_SMALL byte = 0x80

// true = big endian, false = little endian
func getEndian() (ret bool) {
var i int = 0x1
var i = 0x1
bs := (*[4]byte)(unsafe.Pointer(&i))
if bs[0] == 0 {
return true
Expand Down
2 changes: 1 addition & 1 deletion lib/storage/access/table_heap.go
Original file line number Diff line number Diff line change
Expand Up @@ -210,7 +210,7 @@ func (t *TableHeap) UpdateTuple(tuple_ *tuple.Tuple, update_col_idxs []int, sche
// add appropriate transaction's write set of Update.
// when txn is ABORTED state case, data is not updated. so adding a write set entry is not needed
// when err == ErrNotEnoughSpace route and old tuple delete is only succeeded, DELETE write set entry is added above (no come here)
if !isRollback && is_updated && txn.GetState() != ABORTED {
if is_updated && txn.GetState() != ABORTED {
if isUpdateWithDelInsert {
// adding write record of UPDATE is not needed
} else {
Expand Down
12 changes: 9 additions & 3 deletions lib/storage/access/table_page.go
Original file line number Diff line number Diff line change
Expand Up @@ -210,9 +210,9 @@ func (tp *TablePage) UpdateTuple(new_tuple *tuple.Tuple, update_col_idxs []int,
} else {
// update specifed columns only case

var update_tuple_values []types.Value = make([]types.Value, 0)
var update_tuple_values = make([]types.Value, 0)
matched_cnt := int(0)
for idx, _ := range schema_.GetColumns() {
for idx := range schema_.GetColumns() {
if matched_cnt < len(update_col_idxs) && idx == update_col_idxs[matched_cnt] {
update_tuple_values = append(update_tuple_values, new_tuple.GetValue(schema_, uint32(idx)))
matched_cnt++
Expand Down Expand Up @@ -247,8 +247,10 @@ func (tp *TablePage) UpdateTuple(new_tuple *tuple.Tuple, update_col_idxs []int,
// and move other tuples data
copy(tp.GetData()[free_space_pointer+tuple_size-update_tuple.Size():], tp.GetData()[free_space_pointer:tuple_offset])
tp.SetFreeSpacePointer(free_space_pointer + tuple_size - update_tuple.Size())
copy(tp.GetData()[tuple_offset+tuple_size-update_tuple.Size():], update_tuple.Data()[:update_tuple.Size()])
//copy(tp.GetData()[tuple_offset+tuple_size-update_tuple.Size():], update_tuple.Data()[:update_tuple.Size()])
copy(tp.GetData()[tuple_offset:], update_tuple.Data()[:update_tuple.Size()])
tp.SetTupleSize(slot_num, update_tuple.Size())
tp.SetTupleOffsetAtSlot(rid.GetSlotNum(), tuple_offset+(tuple_size-update_tuple.Size()))
} else {
// occupy the same memory space as the old_tuple until transaction finish

Expand All @@ -275,6 +277,7 @@ func (tp *TablePage) UpdateTuple(new_tuple *tuple.Tuple, update_col_idxs []int,

// called only at commit or redo
func (tp *TablePage) FinalizeUpdateTuple(rid *page.RID, old_tuple *tuple.Tuple, update_tuple *tuple.Tuple, txn *Transaction, log_manager *recovery.LogManager) {
// needless
if update_tuple.Size() > old_tuple.Size() {
// finalize is not needed
return
Expand All @@ -300,6 +303,9 @@ func (tp *TablePage) FinalizeUpdateTuple(rid *page.RID, old_tuple *tuple.Tuple,
new_slot_offset := slot_offset + slide_size
tp.SetTupleOffsetAtSlot(rid.GetSlotNum(), new_slot_offset)

free_space_pointer := tp.GetFreeSpacePointer()
tp.SetFreeSpacePointer(free_space_pointer + slide_size)

// move tuple data to collect location
copy(tp.GetData()[new_slot_offset:], update_tuple.Data())

Expand Down
6 changes: 3 additions & 3 deletions lib/storage/access/transaction_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ type TransactionManager struct {
mutex *sync.Mutex
}

var txn_map map[types.TxnID]*Transaction = make(map[types.TxnID]*Transaction)
var txn_map = make(map[types.TxnID]*Transaction)

func NewTransactionManager(lock_manager *LockManager, log_manager *recovery.LogManager) *TransactionManager {
return &TransactionManager{0, lock_manager, log_manager, common.NewRWLatch(), new(sync.Mutex)}
Expand All @@ -33,7 +33,7 @@ func NewTransactionManager(lock_manager *LockManager, log_manager *recovery.LogM
func (transaction_manager *TransactionManager) Begin(txn *Transaction) *Transaction {
// Acquire the global transaction latch in shared mode.
transaction_manager.global_txn_latch.RLock()
var txn_ret *Transaction = txn
var txn_ret = txn

if txn_ret == nil {
transaction_manager.mutex.Lock()
Expand Down Expand Up @@ -254,7 +254,7 @@ func (transaction_manager *TransactionManager) ResumeTransactions() {
}

func (transaction_manager *TransactionManager) releaseLocks(txn *Transaction) {
var lock_set []page.RID = make([]page.RID, 0)
var lock_set = make([]page.RID, 0)
lock_set = append(lock_set, txn.GetExclusiveLockSet()...)
lock_set = append(lock_set, txn.GetSharedLockSet()...)
transaction_manager.lock_manager.Unlock(txn, lock_set)
Expand Down
4 changes: 2 additions & 2 deletions lib/storage/buffer/buffer_pool_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -320,7 +320,7 @@ func (b *BufferPoolManager) DeallocatePage(pageID types.PageID) error {
func (b *BufferPoolManager) FlushAllPages() {
pageIDs := make([]types.PageID, 0)
b.mutex.Lock()
for pageID, _ := range b.pageTable {
for pageID := range b.pageTable {
pageIDs = append(pageIDs, pageID)
}
b.mutex.Unlock()
Expand All @@ -334,7 +334,7 @@ func (b *BufferPoolManager) FlushAllPages() {
func (b *BufferPoolManager) FlushAllDirtyPages() bool {
pageIDs := make([]types.PageID, 0)
b.mutex.Lock()
for pageID, _ := range b.pageTable {
for pageID := range b.pageTable {
if frameID, ok := b.pageTable[pageID]; ok {
pg := b.pages[frameID]
pg.RLatch()
Expand Down
2 changes: 1 addition & 1 deletion lib/storage/buffer/clock_replacer.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ func (c *ClockReplacer) Victim() *FrameID {
}

var victimFrameID *FrameID
currentNode := (*c.clockHand)
currentNode := *c.clockHand
for {
if currentNode.value {
currentNode.value = false
Expand Down
2 changes: 1 addition & 1 deletion lib/storage/index/linear_probe_hash_table_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ import (
"github.com/ryogrid/SamehadaDB/lib/samehada/samehada_util"
"github.com/ryogrid/SamehadaDB/lib/types"

hash "github.com/ryogrid/SamehadaDB/lib/container/hash"
"github.com/ryogrid/SamehadaDB/lib/container/hash"
"github.com/ryogrid/SamehadaDB/lib/storage/buffer"
"github.com/ryogrid/SamehadaDB/lib/storage/page"
"github.com/ryogrid/SamehadaDB/lib/storage/table/schema"
Expand Down
4 changes: 2 additions & 2 deletions lib/storage/page/hash_table_block_page.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,8 +48,8 @@ func (page *HashTableBlockPage) Insert(index uint64, key uint64, value uint64) b
}

page.array[index] = HashTablePair{key, value}
page.occuppied[index/8] |= (1 << (index % 8))
page.readable[index/8] |= (1 << (index % 8))
page.occuppied[index/8] |= 1 << (index % 8)
page.readable[index/8] |= 1 << (index % 8)
return true
}

Expand Down
Loading

0 comments on commit 7cec517

Please sign in to comment.