diff --git a/lib/catalog/catalog_test/table_catalog_reload_test.go b/lib/catalog/catalog_test/table_catalog_reload_test.go index 957923f2..efe81eef 100644 --- a/lib/catalog/catalog_test/table_catalog_reload_test.go +++ b/lib/catalog/catalog_test/table_catalog_reload_test.go @@ -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) diff --git a/lib/catalog/table_catalog.go b/lib/catalog/table_catalog.go index 1a7bc0fe..4fb5bdb1 100644 --- a/lib/catalog/table_catalog.go +++ b/lib/catalog/table_catalog.go @@ -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) @@ -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 @@ -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 diff --git a/lib/catalog/table_metadata.go b/lib/catalog/table_metadata.go index 3ad4c142..08d6bdf4 100644 --- a/lib/catalog/table_metadata.go +++ b/lib/catalog/table_metadata.go @@ -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 @@ -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) @@ -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!") } diff --git a/lib/execution/executors/executor_test/executor_test.go b/lib/execution/executors/executor_test/executor_test.go index 33734ac8..bb2da542 100644 --- a/lib/execution/executors/executor_test/executor_test.go +++ b/lib/execution/executors/executor_test/executor_test.go @@ -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) @@ -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) @@ -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) @@ -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) diff --git a/lib/samehada/samehada.go b/lib/samehada/samehada.go index cfa0bbfd..f80fbd8d 100644 --- a/lib/samehada/samehada.go +++ b/lib/samehada/samehada.go @@ -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!") } @@ -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 { diff --git a/lib/storage/index/btree_index.go b/lib/storage/index/btree_index.go index efbc1ba9..a294f3c8 100644 --- a/lib/storage/index/btree_index.go +++ b/lib/storage/index/btree_index.go @@ -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" @@ -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 @@ -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{}) { @@ -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{}) { @@ -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() @@ -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) @@ -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 diff --git a/lib/storage/index/index_test/hash_table_index_test.go b/lib/storage/index/index_test/hash_table_index_test.go index b871276b..dc9b2316 100644 --- a/lib/storage/index/index_test/hash_table_index_test.go +++ b/lib/storage/index/index_test/hash_table_index_test.go @@ -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")