Skip to content

Commit

Permalink
implementing bree index: wrote btree index except MIN and MAX key pas…
Browse files Browse the repository at this point in the history
…sing. testcase has not been passed.
  • Loading branch information
ryogrid committed Aug 23, 2024
1 parent 2966f4f commit c31f0b2
Show file tree
Hide file tree
Showing 7 changed files with 59 additions and 23 deletions.
2 changes: 1 addition & 1 deletion lib/catalog/catalog_test/table_catalog_reload_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ func TestTableCatalogReload(t *testing.T) {

samehada_instance_new := samehada.NewSamehadaInstance(t.Name(), common.BufferPoolMaxFrameNumForTest)
txn_new := samehada_instance_new.GetTransactionManager().Begin(nil)
catalog_recov := catalog.RecoveryCatalogFromCatalogPage(samehada_instance_new.GetBufferPoolManager(), samehada_instance_new.GetLogManager(), samehada_instance_new.GetLockManager(), txn_new)
catalog_recov := catalog.RecoveryCatalogFromCatalogPage(samehada_instance_new.GetBufferPoolManager(), samehada_instance_new.GetLogManager(), samehada_instance_new.GetLockManager(), txn_new, true)

columnToCheck := catalog_recov.GetTableByOID(1).Schema().GetColumn(1)

Expand Down
5 changes: 3 additions & 2 deletions lib/catalog/table_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ func BootstrapCatalog(bpm *buffer.BufferPoolManager, log_manager *recovery.LogMa
}

// RecoveryCatalogFromCatalogPage get all information about tables and columns from disk and put it on memory
func RecoveryCatalogFromCatalogPage(bpm *buffer.BufferPoolManager, log_manager *recovery.LogManager, lock_manager *access.LockManager, txn *access.Transaction) *Catalog {
func RecoveryCatalogFromCatalogPage(bpm *buffer.BufferPoolManager, log_manager *recovery.LogManager, lock_manager *access.LockManager, txn *access.Transaction, isGracefulShutdown bool) *Catalog {
tableCatalogHeapIt := access.InitTableHeap(bpm, TableCatalogPageId, log_manager, lock_manager).Iterator(txn)

tableIds := make(map[uint32]*TableMetadata)
Expand Down Expand Up @@ -106,6 +106,7 @@ func RecoveryCatalogFromCatalogPage(bpm *buffer.BufferPoolManager, log_manager *
access.InitTableHeap(bpm, types.PageID(firstPage), log_manager, lock_manager),
uint32(oid),
log_manager,
isGracefulShutdown,
)

tableIds[uint32(oid)] = tableMetadata
Expand Down Expand Up @@ -171,7 +172,7 @@ func (c *Catalog) CreateTable(name string, schema_ *schema.Schema, txn *access.T
// attach table name as prefix to all columns name
attachTableNameToColumnsName(schema_, name_)

tableMetadata := NewTableMetadata(schema_, name_, tableHeap, oid, c.Log_manager)
tableMetadata := NewTableMetadata(schema_, name_, tableHeap, oid, c.Log_manager, true)

c.tableIdsMutex.Lock()
c.tableIds[oid] = tableMetadata
Expand Down
17 changes: 12 additions & 5 deletions lib/catalog/table_metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ type TableMetadata struct {
oid uint32
}

func NewTableMetadata(schema *schema.Schema, name string, table *access.TableHeap, oid uint32, log_manager *recovery.LogManager) *TableMetadata {
func NewTableMetadata(schema *schema.Schema, name string, table *access.TableHeap, oid uint32, log_manager *recovery.LogManager, isGracefulShutdown bool) *TableMetadata {
ret := new(TableMetadata)
ret.schema = schema
ret.name = name
Expand All @@ -42,6 +42,7 @@ func NewTableMetadata(schema *schema.Schema, name string, table *access.TableHea
// one page can store 512 key/value pair
im := index.NewIndexMetadata(column_.GetColumnName()+"_index", name, schema, []uint32{uint32(idx)})
hIdx := index.NewLinearProbeHashTableIndex(im, table.GetBufferPoolManager(), uint32(idx), common.BucketSizeOfHashIndex, column_.IndexHeaderPageId())

indexes = append(indexes, hIdx)
// at first allocation of pages for index, column's indexHeaderPageID is -1 at above code (column_.IndexHeaderPageId() == -1)
// because first allocation occurs when table creation is processed (not launched DB instace from existing db file which has difinition of this table)
Expand All @@ -64,12 +65,18 @@ func NewTableMetadata(schema *schema.Schema, name string, table *access.TableHea
indexes = append(indexes, slIdx)
//column_.SetIndexHeaderPageId(slIdx.GetHeaderPageId())
case index_constants.INDEX_KIND_BTREE:
// currently, BTree Index always use new pages even if relaunch
im := index.NewIndexMetadata(column_.GetColumnName()+"_index", name, schema, []uint32{uint32(idx)})
// TODO: (SDB) need to add index headae ID argument like HashIndex (NewTableMetadata)
slIdx := index.NewBTreeIndex(im, table.GetBufferPoolManager(), uint32(idx), log_manager)
// TODO: (SDB) need to avoid reuse of page zero when system shutdown was not graceful
var pageZeroId *int32 = nil
if column_.IndexHeaderPageId() != -1 && isGracefulShutdown {
pageZeroId = new(int32)
*pageZeroId = int32(column_.IndexHeaderPageId())
}

indexes = append(indexes, slIdx)
btrIdx := index.NewBTreeIndex(im, table.GetBufferPoolManager(), uint32(idx), log_manager, pageZeroId)

indexes = append(indexes, btrIdx)
column_.SetIndexHeaderPageId(btrIdx.GetHeaderPageId())
default:
panic("illegal index kind!")
}
Expand Down
8 changes: 4 additions & 4 deletions lib/execution/executors/executor_test/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2221,7 +2221,7 @@ func TestInsertAndSpecifiedColumnUpdatePageMoveRecovery(t *testing.T) {
log_mgr)

txn = txn_mgr.Begin(nil)
c = catalog.RecoveryCatalogFromCatalogPage(bpm, log_mgr, lock_mgr, txn)
c = catalog.RecoveryCatalogFromCatalogPage(bpm, log_mgr, lock_mgr, txn, true)
tableMetadata = c.GetTableByName("test_1")

executorContext = executors.NewExecutorContext(c, bpm, txn)
Expand Down Expand Up @@ -2396,7 +2396,7 @@ func TestInsertAndSpecifiedColumnUpdatePageMoveOccurOnRecovery(t *testing.T) {
log_mgr)

txn = txn_mgr.Begin(nil)
c = catalog.RecoveryCatalogFromCatalogPage(bpm, log_mgr, lock_mgr, txn)
c = catalog.RecoveryCatalogFromCatalogPage(bpm, log_mgr, lock_mgr, txn, true)
tableMetadata = c.GetTableByName("test_1")

executorContext = executors.NewExecutorContext(c, bpm, txn)
Expand Down Expand Up @@ -2809,7 +2809,7 @@ func TestDeallocatedPageReuseAfterRelaunchGraceful(t *testing.T) {
log_mgr)

txn = txn_mgr.Begin(nil)
c = catalog.RecoveryCatalogFromCatalogPage(bpm, log_mgr, lock_mgr, txn)
c = catalog.RecoveryCatalogFromCatalogPage(bpm, log_mgr, lock_mgr, txn, true)
tableMetadata = c.GetTableByName("test_1")

executorContext = executors.NewExecutorContext(c, bpm, txn)
Expand Down Expand Up @@ -2964,7 +2964,7 @@ func TestDeallocatedPageReuseAfterRelaunchByCrash(t *testing.T) {
log_mgr)

txn = txn_mgr.Begin(nil)
c = catalog.RecoveryCatalogFromCatalogPage(bpm, log_mgr, lock_mgr, txn)
c = catalog.RecoveryCatalogFromCatalogPage(bpm, log_mgr, lock_mgr, txn, false)
tableMetadata = c.GetTableByName("test_1")

executorContext = executors.NewExecutorContext(c, bpm, txn)
Expand Down
6 changes: 5 additions & 1 deletion lib/samehada/samehada.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,10 @@ func reconstructIndexDataOfATbl(t *catalog.TableMetadata, c *catalog.Catalog, dm
// do nothing here
// (Since SkipList index can't reuse past allocated pages, data clear of allocated pages
// are not needed...)
case index_constants.INDEX_KIND_BTREE:
// do nothing here
// (Since BTree index can't reuse past allocated pages, data clear of allocated pages
// are not needed...)
default:
panic("invalid index kind!")
}
Expand Down Expand Up @@ -147,7 +151,7 @@ func NewSamehadaDB(dbName string, memKBytes int) *SamehadaDB {
}
shi.log_manager.Flush()

c = catalog.RecoveryCatalogFromCatalogPage(shi.GetBufferPoolManager(), shi.GetLogManager(), shi.GetLockManager(), txn)
c = catalog.RecoveryCatalogFromCatalogPage(shi.GetBufferPoolManager(), shi.GetLogManager(), shi.GetLockManager(), txn, isGracefulShutdown)

// if last shutdown is not gracefully done, all index data should be reconstructed
if !isGracefulShutdown {
Expand Down
42 changes: 33 additions & 9 deletions lib/storage/index/btree_index.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package index

import (
"encoding/binary"
"github.com/ryogrid/SamehadaDB/lib/common"
"github.com/ryogrid/SamehadaDB/lib/container/btree"
"github.com/ryogrid/SamehadaDB/lib/recovery"
Expand All @@ -15,6 +16,26 @@ import (
"sync"
)

type BtreeIndexIterator struct {
itr *blink_tree.BLTreeItr
valType types.TypeID
}

func NewBtreeIndexIterator(itr *blink_tree.BLTreeItr, valType types.TypeID) *BtreeIndexIterator {
return &BtreeIndexIterator{itr, valType}
}

func (btreeItr *BtreeIndexIterator) Next() (done bool, err error, key *types.Value, rid *page.RID) {
ok, keyBytes, packedRID := btreeItr.itr.Next()
if ok == false {
return true, nil, nil, &page.RID{-1, 0}
}
uintRID := binary.BigEndian.Uint64(packedRID)
unpackedRID := samehada_util.UnpackUint64toRID(uintRID)
decodedKey := samehada_util.ExtractOrgKeyFromDicOrderComparableEncodedVarchar(samehada_util.GetPonterOfValue(types.NewVarchar(string(keyBytes))), btreeItr.valType)
return false, nil, decodedKey, &unpackedRID
}

type BTreeIndex struct {
container *blink_tree.BLTree
metadata *IndexMetadata
Expand Down Expand Up @@ -53,7 +74,11 @@ func (btidx *BTreeIndex) insertEntryInner(key *tuple.Tuple, rid page.RID, txn in
btidx.updateMtx.RLock()
defer btidx.updateMtx.RUnlock()
}
btidx.container.Insert(convedKeyVal, samehada_util.PackRIDtoUint64(&rid))

packedRID := samehada_util.PackRIDtoUint64(&rid)
var valBuf [8]byte
binary.BigEndian.PutUint64(valBuf[:], packedRID)
btidx.container.InsertKey(convedKeyVal.SerializeOnlyVal(), 0, valBuf, true)
}

func (btidx *BTreeIndex) InsertEntry(key *tuple.Tuple, rid page.RID, txn interface{}) {
Expand All @@ -70,10 +95,7 @@ func (btidx *BTreeIndex) deleteEntryInner(key *tuple.Tuple, rid page.RID, txn in
btidx.updateMtx.RLock()
defer btidx.updateMtx.RUnlock()
}
isSuccess := btidx.container.Remove(convedKeyVal, 0)
if isSuccess == false {
//panic(fmt.Sprintf("BTreeIndex::deleteEntryInner: %v %v\n", convedKeyVal.ToIFValue(), rid))
}
btidx.container.DeleteKey(convedKeyVal.SerializeOnlyVal(), 0)
}

func (btidx *BTreeIndex) DeleteEntry(key *tuple.Tuple, rid page.RID, txn interface{}) {
Expand All @@ -88,11 +110,12 @@ func (btidx *BTreeIndex) ScanKey(key *tuple.Tuple, txn interface{}) []page.RID {

btidx.updateMtx.RLock()
// Attention: returned itr's containing keys are string type Value which is constructed with byte arr of concatenated original key and value
rangeItr := btidx.container.Iterator(smallestKeyVal, biggestKeyVal)
rangeItr := btidx.container.GetRangeItr(smallestKeyVal.SerializeOnlyVal(), biggestKeyVal.SerializeOnlyVal())

retArr := make([]page.RID, 0)
for done, _, _, rid := rangeItr.Next(); !done; done, _, _, rid = rangeItr.Next() {
retArr = append(retArr, *rid)
for ok, _, packedRID := rangeItr.Next(); ok; ok, _, packedRID = rangeItr.Next() {
uintRID := binary.BigEndian.Uint64(packedRID)
retArr = append(retArr, samehada_util.UnpackUint64toRID(uintRID))
}
btidx.updateMtx.RUnlock()

Expand All @@ -112,6 +135,7 @@ func (btidx *BTreeIndex) UpdateEntry(oldKey *tuple.Tuple, oldRID page.RID, newKe
// Attention: returned itr's containing keys are string type Value which is constructed with byte arr of concatenated original key and value
func (btidx *BTreeIndex) GetRangeScanIterator(start_key *tuple.Tuple, end_key *tuple.Tuple, transaction interface{}) IndexRangeScanIterator {
tupleSchema_ := btidx.GetTupleSchema()
// TODO: (SDB) need to handle start_key or/and end_key is nil case
var smallestKeyVal *types.Value = nil
if start_key != nil {
orgStartKeyVal := start_key.GetValue(tupleSchema_, btidx.col_idx)
Expand All @@ -126,7 +150,7 @@ func (btidx *BTreeIndex) GetRangeScanIterator(start_key *tuple.Tuple, end_key *t

btidx.updateMtx.RLock()
defer btidx.updateMtx.RUnlock()
return btidx.container.Iterator(smallestKeyVal, biggestKeyVal)
return NewBtreeIndexIterator(btidx.container.GetRangeItr(smallestKeyVal.SerializeOnlyVal(), biggestKeyVal.SerializeOnlyVal()), btidx.metadata.tuple_schema.GetColumn(btidx.col_idx).GetType())
}

// Return the metadata object associated with the index
Expand Down
2 changes: 1 addition & 1 deletion lib/storage/index/index_test/hash_table_index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ func TestRecounstructionOfHashIndex(t *testing.T) {
dman := shi.GetDiskManager()
dman.GCLogFile()
shi.GetLogManager().SetNextLSN(greatestLSN + 1)
c = catalog.RecoveryCatalogFromCatalogPage(shi.GetBufferPoolManager(), shi.GetLogManager(), shi.GetLockManager(), txn)
c = catalog.RecoveryCatalogFromCatalogPage(shi.GetBufferPoolManager(), shi.GetLogManager(), shi.GetLockManager(), txn, true)

// reconstruct all index data of all column
tableMetadata = c.GetTableByName("test_1")
Expand Down

0 comments on commit c31f0b2

Please sign in to comment.