xbase 2 долоо хоног өмнө
parent
commit
a48173c255

BIN
bench_db_data/values.data


+ 634 - 0
db/engine.go

@@ -0,0 +1,634 @@
+package db
+
+import (
+	"bufio"
+	"encoding/binary"
+	"encoding/json"
+	"fmt"
+	"io"
+	"os"
+	"regexp"
+	"sort"
+	"strconv"
+	"strings"
+	"sync"
+)
+
+// FreeList manages reusable disk space in memory using Best-Fit strategy.
+type FreeList struct {
+	// Capacity -> Stack of Offsets
+	buckets map[uint32][]int64
+	// Sorted list of capacities available in buckets for fast Best-Fit lookup
+	sortedCaps []uint32
+	mu         sync.Mutex
+}
+
+func NewFreeList() *FreeList {
+	return &FreeList{
+		buckets: make(map[uint32][]int64),
+	}
+}
+
+// Add adds an offset to the free list for a given capacity.
+func (fl *FreeList) Add(cap uint32, offset int64) {
+	fl.mu.Lock()
+	defer fl.mu.Unlock()
+	
+	if _, exists := fl.buckets[cap]; !exists {
+		fl.buckets[cap] = []int64{offset}
+		// Maintain sortedCaps
+		fl.sortedCaps = append(fl.sortedCaps, cap)
+		sort.Slice(fl.sortedCaps, func(i, j int) bool { return fl.sortedCaps[i] < fl.sortedCaps[j] })
+	} else {
+		fl.buckets[cap] = append(fl.buckets[cap], offset)
+	}
+}
+
+// Pop tries to get an available offset using Best-Fit strategy.
+// It finds the smallest available capacity >= targetCap.
+// Returns offset and the ACTUAL capacity of the slot found (which might be larger than targetCap).
+func (fl *FreeList) Pop(targetCap uint32) (int64, uint32, bool) {
+	fl.mu.Lock()
+	defer fl.mu.Unlock()
+	
+	// Binary search for smallest capacity >= targetCap
+	idx := sort.Search(len(fl.sortedCaps), func(i int) bool {
+		return fl.sortedCaps[i] >= targetCap
+	})
+	
+	// If idx is out of bounds, no suitable slot found
+	if idx >= len(fl.sortedCaps) {
+		return 0, 0, false
+	}
+	
+	// Found a suitable capacity bucket
+	foundCap := fl.sortedCaps[idx]
+	offsets := fl.buckets[foundCap]
+	
+	if len(offsets) == 0 {
+		// This should technically not happen if we maintain sortedCaps correctly (remove empty caps)
+		// But for safety, let's handle it.
+		// In a rigorous impl, we would remove empty bucket from sortedCaps.
+		return 0, 0, false
+	}
+	
+	// Pop from end (Stack LIFO)
+	lastIdx := len(offsets) - 1
+	offset := offsets[lastIdx]
+	
+	// Update bucket
+	if lastIdx == 0 {
+		// Bucket becomes empty
+		delete(fl.buckets, foundCap)
+		// Remove from sortedCaps to keep search efficient
+		// This is O(N) copy but N (number of distinct capacities) is usually small (< 100 for typical payload range)
+		fl.sortedCaps = append(fl.sortedCaps[:idx], fl.sortedCaps[idx+1:]...)
+	} else {
+		fl.buckets[foundCap] = offsets[:lastIdx]
+	}
+	
+	return offset, foundCap, true
+}
+
+// Storage (Table 2) manages disk storage for values.
+// It uses a slot-based format: [Flag][Capacity][Length][Data...]
+// Flag: 1 byte (0=Deleted, 1=Valid)
+// Capacity: 4 bytes (Allocated size)
+// Length: 4 bytes (Actual used size)
+// Data: Capacity bytes
+type Storage struct {
+	file     *os.File
+	filename string
+	offset   int64 // Current end of file
+	freeList *FreeList
+	mu       sync.RWMutex
+}
+
+const (
+	FlagDeleted = 0x00
+	FlagValid   = 0x01
+	HeaderSize  = 9 // 1(Flag) + 4(Cap) + 4(Len)
+	AlignSize   = 16 // Align capacity to 16 bytes
+)
+
+func NewStorage(filename string) (*Storage, error) {
+	f, err := os.OpenFile(filename, os.O_RDWR|os.O_CREATE, 0644)
+	if err != nil {
+		return nil, err
+	}
+
+	st := &Storage{
+		file:     f,
+		filename: filename,
+		freeList: NewFreeList(),
+	}
+
+	// Scan file to build free list and find end offset
+	if err := st.scan(); err != nil {
+		f.Close()
+		return nil, err
+	}
+
+	return st, nil
+}
+
+// scan iterates over the file to reconstruct FreeList and find EOF.
+func (s *Storage) scan() error {
+	offset := int64(0)
+	
+	if _, err := s.file.Seek(0, 0); err != nil {
+		return err
+	}
+	
+	reader := bufio.NewReader(s.file)
+	
+	for {
+		header := make([]byte, HeaderSize)
+		n, err := io.ReadFull(reader, header)
+		if err == io.EOF {
+			break
+		}
+		if err == io.ErrUnexpectedEOF && n == 0 {
+			break
+		}
+		if err != nil {
+			return err
+		}
+		
+		flag := header[0]
+		cap := binary.LittleEndian.Uint32(header[1:])
+		
+		if flag == FlagDeleted {
+			s.freeList.Add(cap, offset)
+		}
+		
+		discardLen := int(cap)
+		if _, err := reader.Discard(discardLen); err != nil {
+			buf := make([]byte, discardLen)
+			if _, err := io.ReadFull(reader, buf); err != nil {
+				return err
+			}
+		}
+		
+		offset += int64(HeaderSize + discardLen)
+	}
+	
+	s.offset = offset
+	return nil
+}
+
+// alignCapacity calculates the capacity needed aligned to 16 bytes.
+func alignCapacity(length int) uint32 {
+	if length == 0 {
+		return AlignSize
+	}
+	cap := (length + AlignSize - 1) / AlignSize * AlignSize
+	return uint32(cap)
+}
+
+// WriteValue writes a value to storage.
+// If oldOffset >= 0, it tries to update in-place.
+// Returns the new offset (or oldOffset if updated in-place) and error.
+func (s *Storage) WriteValue(val string, oldOffset int64) (int64, error) {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+
+	valLen := len(val)
+
+	// Try in-place update first
+	if oldOffset >= 0 {
+		capBuf := make([]byte, 4)
+		if _, err := s.file.ReadAt(capBuf, oldOffset+1); err == nil {
+			oldCap := binary.LittleEndian.Uint32(capBuf)
+			
+			if uint32(valLen) <= oldCap {
+				// Perfect, update in place!
+				lenBuf := make([]byte, 4)
+				binary.LittleEndian.PutUint32(lenBuf, uint32(valLen))
+				if _, err := s.file.WriteAt(lenBuf, oldOffset+5); err != nil {
+					return 0, err
+				}
+				if _, err := s.file.WriteAt([]byte(val), oldOffset+HeaderSize); err != nil {
+					return 0, err
+				}
+				return oldOffset, nil
+			}
+			
+			// Mark old slot as deleted and add to FreeList.
+			if _, err := s.file.WriteAt([]byte{FlagDeleted}, oldOffset); err != nil {
+				return 0, err
+			}
+			s.freeList.Add(oldCap, oldOffset)
+		}
+	}
+
+	// Calculate needed capacity
+	newCap := alignCapacity(valLen)
+
+	// Try to reuse space from FreeList using BEST FIT
+	if reusedOffset, actualCap, ok := s.freeList.Pop(newCap); ok {
+		// Write Header + Data
+		// Header: [Flag=Valid][Cap=actualCap][Len=valLen]
+		// NOTE: We MUST write 'actualCap' back to the header, NOT 'newCap'.
+		// The physical slot size on disk is 'actualCap', and we cannot shrink it physically 
+		// without moving subsequent data. 
+		// So we effectively waste (actualCap - newCap) bytes of padding.
+		
+		buf := make([]byte, HeaderSize+int(actualCap))
+		buf[0] = FlagValid
+		binary.LittleEndian.PutUint32(buf[1:], actualCap) // Must be actualCap
+		binary.LittleEndian.PutUint32(buf[5:], uint32(valLen))
+		copy(buf[HeaderSize:], []byte(val))
+		
+		if _, err := s.file.WriteAt(buf, reusedOffset); err != nil {
+			return 0, err
+		}
+		return reusedOffset, nil
+	}
+	
+	// If no reuse, Append new slot
+	newOffset := s.offset
+	
+	totalSize := HeaderSize + int(newCap)
+	buf := make([]byte, totalSize)
+	
+	buf[0] = FlagValid
+	binary.LittleEndian.PutUint32(buf[1:], newCap)
+	binary.LittleEndian.PutUint32(buf[5:], uint32(valLen))
+	copy(buf[HeaderSize:], []byte(val))
+	
+	if _, err := s.file.WriteAt(buf, newOffset); err != nil {
+		return 0, err
+	}
+	
+	s.offset += int64(totalSize)
+	return newOffset, nil
+}
+
+// ReadValue reads value at offset.
+func (s *Storage) ReadValue(offset int64) (string, error) {
+	s.mu.RLock()
+	defer s.mu.RUnlock()
+
+	// Read Header
+	header := make([]byte, HeaderSize)
+	if _, err := s.file.ReadAt(header, offset); err != nil {
+		return "", err
+	}
+
+	flag := header[0]
+	if flag == FlagDeleted {
+		return "", fmt.Errorf("record deleted")
+	}
+
+	length := binary.LittleEndian.Uint32(header[5:])
+
+	// Read Data
+	data := make([]byte, length)
+	if _, err := s.file.ReadAt(data, offset+HeaderSize); err != nil {
+		return "", err
+	}
+
+	return string(data), nil
+}
+
+func (s *Storage) Close() error {
+	return s.file.Close()
+}
+
+// StringTable (Table 2) is now just a wrapper for Storage + Key Management
+// We removed the global deduplication map.
+type StringTable struct {
+}
+
+// IndexEntry (Table 1)
+type IndexEntry struct {
+	ValueOffset int64 // Replaces ValueID
+	CommitIndex uint64
+}
+
+// InvertedIndex (Table 3)
+type InvertedIndex struct {
+	KeyTokens   map[string][]uint32
+	ValueTokens map[string][]uint32
+}
+
+func NewInvertedIndex() *InvertedIndex {
+	return &InvertedIndex{
+		KeyTokens:   make(map[string][]uint32),
+		ValueTokens: make(map[string][]uint32),
+	}
+}
+
+// KeyMap maintains the mapping between KeyString and an internal KeyID (uint32).
+type KeyMap struct {
+	StrToID map[string]uint32
+	IDToStr map[uint32]string
+	NextID  uint32
+}
+
+func NewKeyMap() *KeyMap {
+	return &KeyMap{
+		StrToID: make(map[string]uint32),
+		IDToStr: make(map[uint32]string),
+		NextID:  1,
+	}
+}
+
+func (km *KeyMap) GetOrCreateID(key string) uint32 {
+	if id, ok := km.StrToID[key]; ok {
+		return id
+	}
+	id := km.NextID
+	km.NextID++
+	km.StrToID[key] = id
+	km.IDToStr[id] = key
+	return id
+}
+
+// Engine is the core storage engine.
+type Engine struct {
+	mu sync.RWMutex
+
+	// Table 1: KeyID -> Entry (ValueOffset + CommitIndex)
+	Index map[uint32]IndexEntry
+
+	// Key mapping (In-memory, rebuilt on start or snapshotted)
+	Keys *KeyMap
+
+	// Table 2: Disk Storage
+	Storage *Storage
+
+	// Table 3: Inverted Index
+	SearchIndex *InvertedIndex
+
+	LastCommitIndex uint64
+	dataDir         string
+}
+
+func NewEngine(dataDir string) (*Engine, error) {
+	if err := os.MkdirAll(dataDir, 0755); err != nil {
+		return nil, err
+	}
+
+	store, err := NewStorage(dataDir + "/values.data")
+	if err != nil {
+		return nil, err
+	}
+
+	e := &Engine{
+		Index:       make(map[uint32]IndexEntry),
+		Keys:        NewKeyMap(),
+		Storage:     store,
+		SearchIndex: NewInvertedIndex(),
+		dataDir:     dataDir,
+	}
+
+	return e, nil
+}
+
+func (e *Engine) Close() error {
+	return e.Storage.Close()
+}
+
+func (e *Engine) tokenizeKey(key string) []string {
+	return strings.Split(key, ".")
+}
+
+func (e *Engine) tokenizeValue(val string) []string {
+	f := func(c rune) bool {
+		return !((c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || (c >= '0' && c <= '9'))
+	}
+	return strings.FieldsFunc(val, f)
+}
+
+func (e *Engine) Set(key, value string, commitIndex uint64) error {
+	e.mu.Lock()
+	defer e.mu.Unlock()
+
+	if commitIndex > e.LastCommitIndex {
+		e.LastCommitIndex = commitIndex
+	}
+
+	// 1. Get KeyID
+	keyID := e.Keys.GetOrCreateID(key)
+
+	// 2. Check existing entry for update
+	var oldOffset int64 = -1
+	if entry, ok := e.Index[keyID]; ok {
+		oldOffset = entry.ValueOffset
+	}
+
+	// 3. Write Value (In-place or Append)
+	newOffset, err := e.Storage.WriteValue(value, oldOffset)
+	if err != nil {
+		return err
+	}
+
+	// 4. Update Index (Table 1)
+	e.Index[keyID] = IndexEntry{
+		ValueOffset: newOffset,
+		CommitIndex: commitIndex,
+	}
+
+	// 5. Update Inverted Index (Table 3)
+	for _, token := range e.tokenizeKey(key) {
+		e.addToken(e.SearchIndex.KeyTokens, token, keyID)
+	}
+	for _, token := range e.tokenizeValue(value) {
+		e.addToken(e.SearchIndex.ValueTokens, token, keyID)
+	}
+	return nil
+}
+
+func (e *Engine) addToken(index map[string][]uint32, token string, id uint32) {
+	ids := index[token]
+	for _, existing := range ids {
+		if existing == id {
+			return
+		}
+	}
+	index[token] = append(ids, id)
+}
+
+func (e *Engine) Get(key string) (string, bool) {
+	e.mu.RLock()
+	defer e.mu.RUnlock()
+
+	keyID, ok := e.Keys.StrToID[key]
+	if !ok {
+		return "", false
+	}
+
+	entry, ok := e.Index[keyID]
+	if !ok {
+		return "", false
+	}
+
+	val, err := e.Storage.ReadValue(entry.ValueOffset)
+	if err != nil {
+		return "", false
+	}
+	return val, true
+}
+
+type QueryResult struct {
+	Key         string `json:"key"`
+	Value       string `json:"value"`
+	CommitIndex uint64 `json:"commit_index"`
+}
+
+// WildcardMatch is kept as is
+func WildcardMatch(str, pattern string) bool {
+	s := []rune(str)
+	p := []rune(pattern)
+	sLen, pLen := len(s), len(p)
+	i, j := 0, 0
+	starIdx, matchIdx := -1, -1
+
+	for i < sLen {
+		if j < pLen && (p[j] == '?' || p[j] == s[i]) {
+			i++
+			j++
+		} else if j < pLen && p[j] == '*' {
+			starIdx = j
+			matchIdx = i
+			j++
+		} else if starIdx != -1 {
+			j = starIdx + 1
+			matchIdx++
+			i = matchIdx
+		} else {
+			return false
+		}
+	}
+	for j < pLen && p[j] == '*' {
+		j++
+	}
+	return j == pLen
+}
+
+func (e *Engine) Query(sql string) ([]QueryResult, error) {
+	e.mu.RLock()
+	defer e.mu.RUnlock()
+
+	sql = strings.TrimSpace(sql)
+	re := regexp.MustCompile(`(key|value|CommitIndex)\s*(=|like|>=|<=|>|<)\s*("[^"]*"|\d+)`)
+	matches := re.FindAllStringSubmatch(sql, -1)
+
+	if len(matches) == 0 {
+		return nil, fmt.Errorf("invalid query")
+	}
+
+	extractString := func(s string) string {
+		return strings.Trim(strings.TrimSpace(s), "\"")
+	}
+
+	var results []QueryResult
+
+	filter := func(kID uint32, entry IndexEntry) (bool, string, string) {
+		keyStr, ok := e.Keys.IDToStr[kID]
+		if !ok { return false, "", "" }
+		
+		valStr, err := e.Storage.ReadValue(entry.ValueOffset)
+		if err != nil { return false, "", "" }
+
+		for _, match := range matches {
+			field := match[1]
+			op := match[2]
+			valRaw := match[3]
+
+			switch field {
+			case "CommitIndex":
+				num, _ := strconv.ParseUint(valRaw, 10, 64)
+				switch op {
+				case ">":
+					if !(entry.CommitIndex > num) { return false, "", "" }
+				case "<":
+					if !(entry.CommitIndex < num) { return false, "", "" }
+				case ">=":
+					if !(entry.CommitIndex >= num) { return false, "", "" }
+				case "<=":
+					if !(entry.CommitIndex <= num) { return false, "", "" }
+				case "=":
+					if !(entry.CommitIndex == num) { return false, "", "" }
+				}
+			case "key":
+				target := extractString(valRaw)
+				switch op {
+				case "=":
+					if keyStr != target { return false, "", "" }
+				case "like":
+					if !WildcardMatch(keyStr, target) { return false, "", "" }
+				}
+			case "value":
+				target := extractString(valRaw)
+				switch op {
+				case "=":
+					if valStr != target { return false, "", "" }
+				case "like":
+					if !WildcardMatch(valStr, target) { return false, "", "" }
+				}
+			}
+		}
+		return true, keyStr, valStr
+	}
+
+	for kID, entry := range e.Index {
+		ok, kStr, vStr := filter(kID, entry)
+		if ok {
+			results = append(results, QueryResult{
+				Key:         kStr,
+				Value:       vStr,
+				CommitIndex: entry.CommitIndex,
+			})
+		}
+	}
+
+	sort.Slice(results, func(i, j int) bool {
+		return results[i].Key < results[j].Key
+	})
+
+	return results, nil
+}
+
+func (e *Engine) Snapshot() ([]byte, error) {
+	e.mu.RLock()
+	defer e.mu.RUnlock()
+
+	data := struct {
+		Index           map[uint32]IndexEntry
+		Keys            *KeyMap
+		LastCommitIndex uint64
+		SearchIndex     *InvertedIndex
+	}{
+		Index:           e.Index,
+		Keys:            e.Keys,
+		LastCommitIndex: e.LastCommitIndex,
+		SearchIndex:     e.SearchIndex,
+	}
+
+	return json.Marshal(data)
+}
+
+func (e *Engine) Restore(data []byte) error {
+	e.mu.Lock()
+	defer e.mu.Unlock()
+
+	var dump struct {
+		Index           map[uint32]IndexEntry
+		Keys            *KeyMap
+		LastCommitIndex uint64
+		SearchIndex     *InvertedIndex
+	}
+
+	if err := json.Unmarshal(data, &dump); err != nil {
+		return err
+	}
+
+	e.Index = dump.Index
+	e.Keys = dump.Keys
+	e.LastCommitIndex = dump.LastCommitIndex
+	e.SearchIndex = dump.SearchIndex
+	return nil
+}

+ 130 - 0
db/engine_test.go

@@ -0,0 +1,130 @@
+package db
+
+import (
+	"testing"
+)
+
+func TestEngineBasic(t *testing.T) {
+	dir := t.TempDir()
+	e, err := NewEngine(dir)
+	if err != nil {
+		t.Fatalf("Failed to create engine: %v", err)
+	}
+	defer e.Close()
+
+	e.Set("node1.ip", "192.168.1.1", 100)
+	e.Set("node2.ip", "192.168.1.2", 101)
+
+	val, ok := e.Get("node1.ip")
+	if !ok || val != "192.168.1.1" {
+		t.Errorf("Expected 192.168.1.1, got %v", val)
+	}
+
+	val, ok = e.Get("node2.ip")
+	if !ok || val != "192.168.1.2" {
+		t.Errorf("Expected 192.168.1.2, got %v", val)
+	}
+}
+
+func TestEngineQuery(t *testing.T) {
+	dir := t.TempDir()
+	e, err := NewEngine(dir)
+	if err != nil {
+		t.Fatalf("Failed to create engine: %v", err)
+	}
+	defer e.Close()
+
+	// Seed data
+	e.Set("prod.node1.config", "ver=1.0", 10)
+	e.Set("prod.node2.config", "ver=1.0", 11)
+	e.Set("dev.node1.config", "ver=2.0-beta", 12)
+	e.Set("test.node1.config", "ver=1.0", 13)
+	e.Set("prod.db.config", "host=localhost", 14)
+
+	tests := []struct {
+		sql      string
+		expected int
+	}{
+		{`key like "prod.*"`, 3},
+		{`key like "prod.*" and value = "ver=1.0"`, 2},
+		{`value = "ver=1.0"`, 3},
+		{`CommitIndex > 11`, 3}, // 12, 13, 14
+		{`CommitIndex >= 11`, 4}, // 11, 12, 13, 14
+		{`CommitIndex < 12`, 2}, // 10, 11
+		{`key like "*.config"`, 5},
+		{`key = "prod.node1.config"`, 1},
+		{`key like "prod.*" and CommitIndex > 10`, 2}, // node2(11), db(14)
+	}
+
+	for i, tt := range tests {
+		res, err := e.Query(tt.sql)
+		if err != nil {
+			t.Errorf("Test %d: Query failed: %v", i, err)
+			continue
+		}
+		if len(res) != tt.expected {
+			t.Errorf("Test %d: Query '%s' expected %d results, got %d. Results: %+v", i, tt.sql, tt.expected, len(res), res)
+		}
+	}
+}
+
+func TestSnapshotRestore(t *testing.T) {
+	dir := t.TempDir()
+	e, err := NewEngine(dir)
+	if err != nil {
+		t.Fatalf("Failed to create engine: %v", err)
+	}
+	// No defer close here, we want to simulate restart
+	
+	e.Set("k1", "v1", 1)
+	e.Set("k2", "v2", 2)
+
+	data, err := e.Snapshot()
+	if err != nil {
+		t.Fatalf("Snapshot failed: %v", err)
+	}
+	e.Close()
+
+	// Create new engine pointing to same dir (simulating restart/restore)
+	e2, err := NewEngine(dir)
+	if err != nil {
+		t.Fatalf("Failed to open engine: %v", err)
+	}
+	defer e2.Close()
+	
+	if err := e2.Restore(data); err != nil {
+		t.Fatalf("Restore failed: %v", err)
+	}
+
+	val, ok := e2.Get("k1")
+	if !ok || val != "v1" {
+		t.Errorf("Restored k1 wrong: %v", val)
+	}
+	
+	val, ok = e2.Get("k2")
+	if !ok || val != "v2" {
+		t.Errorf("Restored k2 wrong: %v", val)
+	}
+}
+
+func TestWildcardMatch(t *testing.T) {
+	tests := []struct {
+		str, pat string
+		match    bool
+	}{
+		{"abc", "abc", true},
+		{"abc", "a*", true},
+		{"abc", "*c", true},
+		{"abc", "a*c", true},
+		{"abc", "ab?", true},
+		{"abc", "abd", false},
+		{"prod.node1.config", "prod.*", true},
+	}
+
+	for _, tt := range tests {
+		if got := WildcardMatch(tt.str, tt.pat); got != tt.match {
+			t.Errorf("Match(%q, %q) = %v; want %v", tt.str, tt.pat, got, tt.match)
+		}
+	}
+}
+

+ 271 - 0
example/database/benchmark.go

@@ -0,0 +1,271 @@
+package main
+
+import (
+	"fmt"
+	"math/rand"
+	"os"
+	"sync"
+	"sync/atomic"
+	"time"
+
+	"igit.com/xbase/raft/db"
+)
+
+const (
+	TotalKeys      = 100000
+	UpdateCount    = 10000
+	DeleteCount    = 10000
+	QueryCount     = 100000
+	DataDir        = "bench_db_data"
+	ValueBaseSize  = 32  // Base size for values
+)
+
+func main() {
+	// Clean up previous run
+	os.RemoveAll(DataDir)
+
+	fmt.Printf("Initializing DB Engine in %s...\n", DataDir)
+	e, err := db.NewEngine(DataDir)
+	if err != nil {
+		panic(err)
+	}
+	defer e.Close()
+
+	// 1. Bulk Insert 100k keys
+	fmt.Println("\n--- Phase 1: Bulk Insert 100k Keys ---")
+	keys := make([]string, TotalKeys)
+	start := time.Now()
+	
+	// Pre-generate keys to avoid benchmark overhead
+	for i := 0; i < TotalKeys; i++ {
+		keys[i] = fmt.Sprintf("bench.key.%d", i)
+	}
+
+	var wg sync.WaitGroup
+	workers := 10 // Concurrent workers
+	chunkSize := TotalKeys / workers
+	var insertOps int64
+
+	for w := 0; w < workers; w++ {
+		wg.Add(1)
+		go func(id int) {
+			defer wg.Done()
+			base := id * chunkSize
+			for i := 0; i < chunkSize; i++ {
+				idx := base + i
+				// Random value string of approx ValueBaseSize
+				// Using consistent size initially to test reuse later
+				val := fmt.Sprintf("value-data-%d-%s", idx, randomString(15))
+				if err := e.Set(keys[idx], val, uint64(idx)); err != nil {
+					panic(err)
+				}
+				atomic.AddInt64(&insertOps, 1)
+			}
+		}(w)
+	}
+	wg.Wait()
+	
+	duration := time.Since(start)
+	qps := float64(TotalKeys) / duration.Seconds()
+	printStats("Insert", TotalKeys, duration, qps, getFileSize(DataDir+"/values.data"))
+
+	// 2. Update 10k Keys (Mixed Strategy)
+	// Strategy:
+	// - 50% Longer: Triggers Append + Mark Old Deleted (Old slots become Free)
+	// - 50% Shorter: Triggers In-Place Update (No FreeList change)
+	// But wait, with FreeList, the "Longer" updates will generate Free slots.
+	// Can we verify if subsequent inserts reuse them?
+	fmt.Println("\n--- Phase 2: Update 10k Keys (50% Long, 50% Short) ---")
+	start = time.Now()
+	
+	updateKeys := keys[:UpdateCount]
+	wg = sync.WaitGroup{}
+	chunkSize = UpdateCount / workers
+
+	for w := 0; w < workers; w++ {
+		wg.Add(1)
+		go func(id int) {
+			defer wg.Done()
+			base := id * chunkSize
+			for i := 0; i < chunkSize; i++ {
+				idx := base + i
+				key := updateKeys[idx]
+				var val string
+				if idx%2 == 0 {
+					// Longer value (Trigger Append, Release Old Slot to FreeList)
+					// Old cap was likely 32 or 48. New cap will be larger.
+					val = fmt.Sprintf("updated-long-value-%d-%s-padding-padding", idx, randomString(40))
+				} else {
+					// Shorter value (Trigger In-Place)
+					val = "short" 
+				}
+				if err := e.Set(key, val, uint64(TotalKeys+idx)); err != nil {
+					panic(err)
+				}
+			}
+		}(w)
+	}
+	wg.Wait()
+	
+	duration = time.Since(start)
+	qps = float64(UpdateCount) / duration.Seconds()
+	printStats("Update", UpdateCount, duration, qps, getFileSize(DataDir+"/values.data"))
+	
+	// 3. Insert New Data to Test Reuse
+	// We generated ~5000 free slots (from the Long updates) in Phase 2.
+	// Let's insert 5000 new keys with size matching the old slots (approx 32-48 bytes).
+	// If reuse works, file size should NOT increase significantly.
+	fmt.Println("\n--- Phase 3: Insert 5k New Keys (Test Reuse) ---")
+	reuseKeysCount := 5000
+	start = time.Now()
+	
+	for i := 0; i < reuseKeysCount; i++ {
+		key := fmt.Sprintf("reuse.key.%d", i)
+		// Length matches initial inserts (approx 30-40 bytes)
+		val := fmt.Sprintf("new-value-reuse-%d-%s", i, randomString(15))
+		if err := e.Set(key, val, uint64(TotalKeys+UpdateCount+i)); err != nil {
+			panic(err)
+		}
+	}
+	
+	duration = time.Since(start)
+	qps = float64(reuseKeysCount) / duration.Seconds()
+	printStats("InsertReuse", reuseKeysCount, duration, qps, getFileSize(DataDir+"/values.data"))
+
+
+	// 4. Delete 10k Keys
+	fmt.Println("\n--- Phase 4: Delete 10k Keys (Simulated via Update) ---")
+	// Using "DELETED_MARKER" which is short, so it will be In-Place update.
+	// This won't test FreeList populating, but In-Place logic.
+	// To test FreeList populating from Delete, we need a real Delete() or Update to smaller size that frees extra space?
+	// But our storage doesn't split blocks.
+	// So let's stick to standard benchmark.
+	deleteKeys := keys[UpdateCount : UpdateCount+DeleteCount]
+	start = time.Now()
+	
+	wg = sync.WaitGroup{}
+	chunkSize = DeleteCount / workers
+
+	for w := 0; w < workers; w++ {
+		wg.Add(1)
+		go func(id int) {
+			defer wg.Done()
+			base := id * chunkSize
+			for i := 0; i < chunkSize; i++ {
+				idx := base + i
+				key := deleteKeys[idx]
+				if err := e.Set(key, "DELETED_MARKER", uint64(TotalKeys+UpdateCount+reuseKeysCount+idx)); err != nil {
+					panic(err)
+				}
+			}
+		}(w)
+	}
+	wg.Wait()
+
+	duration = time.Since(start)
+	qps = float64(DeleteCount) / duration.Seconds()
+	printStats("Delete", DeleteCount, duration, qps, getFileSize(DataDir+"/values.data"))
+
+	// 5. Verification
+	fmt.Println("\n--- Phase 5: Verification (Full Scan) ---")
+	
+	errors := 0
+	
+	// 1. Check Updated Keys (0-9999)
+	for i := 0; i < UpdateCount; i++ {
+		val, ok := e.Get(keys[i])
+		if !ok {
+			fmt.Printf("Error: Key %s not found\n", keys[i])
+			errors++
+			continue
+		}
+		if i%2 == 0 {
+			// Should be long
+			if len(val) < 40 { // Our long value is > 40 chars
+				fmt.Printf("Error: Key %s should be long, got: %s\n", keys[i], val)
+				errors++
+			}
+		} else {
+			// Should be "short"
+			if val != "short" {
+				fmt.Printf("Error: Key %s should be 'short', got: %s\n", keys[i], val)
+				errors++
+			}
+		}
+	}
+	
+	// 2. Check Deleted Keys (10000-19999)
+	for i := UpdateCount; i < UpdateCount+DeleteCount; i++ {
+		val, ok := e.Get(keys[i])
+		if !ok {
+			fmt.Printf("Error: Key %s not found\n", keys[i])
+			errors++
+			continue
+		}
+		if val != "DELETED_MARKER" {
+			fmt.Printf("Error: Key %s should be deleted, got: %s\n", keys[i], val)
+			errors++
+		}
+	}
+	
+	// 3. Check Original Keys (20000-99999)
+	for i := UpdateCount + DeleteCount; i < TotalKeys; i++ {
+		val, ok := e.Get(keys[i])
+		if !ok {
+			fmt.Printf("Error: Key %s not found\n", keys[i])
+			errors++
+			continue
+		}
+		// Original values start with "value-data-"
+		if len(val) < 10 || val[:11] != "value-data-" {
+			fmt.Printf("Error: Key %s should be original, got: %s\n", keys[i], val)
+			errors++
+		}
+	}
+	
+	// 4. Check Reused Keys (Extra 5000)
+	for i := 0; i < 5000; i++ { // reuseKeysCount was hardcoded as 5000 inside main
+		key := fmt.Sprintf("reuse.key.%d", i)
+		val, ok := e.Get(key)
+		if !ok {
+			fmt.Printf("Error: Reuse Key %s not found\n", key)
+			errors++
+			continue
+		}
+		if len(val) < 10 || val[:16] != "new-value-reuse-" {
+			fmt.Printf("Error: Reuse Key %s mismatch, got: %s\n", key, val)
+			errors++
+		}
+	}
+
+	if errors == 0 {
+		fmt.Println("Integrity Check: PASS (All keys verified successfully)")
+	} else {
+		fmt.Printf("Integrity Check: FAIL (%d errors found)\n", errors)
+	}
+}
+
+func randomString(n int) string {
+	const letters = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ"
+	b := make([]byte, n)
+	for i := range b {
+		b[i] = letters[rand.Intn(len(letters))]
+	}
+	return string(b)
+}
+
+func getFileSize(path string) int64 {
+	fi, err := os.Stat(path)
+	if err != nil {
+		return 0
+	}
+	return fi.Size()
+}
+
+func printStats(op string, count int, d time.Duration, qps float64, size int64) {
+	sizeStr := ""
+	if size > 0 {
+		sizeStr = fmt.Sprintf(" | DB Size: %.2f MB", float64(size)/1024/1024)
+	}
+	fmt.Printf("%s: %d ops in %v | QPS: %.0f%s\n", op, count, d, qps, sizeStr)
+}

+ 38 - 0
example/database/generator.go

@@ -0,0 +1,38 @@
+package main
+
+import (
+	"fmt"
+	"os"
+
+	"igit.com/xbase/raft/db"
+)
+
+func main() {
+	dir := "test_db_data"
+	os.RemoveAll(dir) // Start fresh
+	
+	fmt.Println("Initializing DB Engine...")
+	e, err := db.NewEngine(dir)
+	if err != nil {
+		panic(err)
+	}
+	
+	fmt.Println("\n1. Insert initial data (Append)")
+	e.Set("config.timeout", "10s", 1)       // len=3, cap=16
+	e.Set("config.retries", "3", 2)         // len=1, cap=16
+	e.Set("app.name", "demo-app-v1", 3)     // len=11, cap=16
+	
+	fmt.Println("\n2. In-place update (Same length or smaller)")
+	e.Set("config.timeout", "20s", 4)       // len=3, cap=16 -> In-place!
+	
+	fmt.Println("\n3. In-place update (Larger but fits capacity)")
+	e.Set("config.retries", "10", 5)        // len=2, cap=16 -> In-place! (was 1)
+	
+	fmt.Println("\n4. Append update (Exceeds capacity)")
+	e.Set("app.name", "demo-application-v2-long-name", 6) // len=29, cap=32 -> Append!
+	
+	e.Close()
+	fmt.Println("\nDB Closed. Run inspector to see file layout.")
+	fmt.Printf("\nTry running:\n go run example/database/inspector.go %s\n", dir)
+}
+

+ 115 - 0
example/database/inspector.go

@@ -0,0 +1,115 @@
+package main
+
+import (
+	"encoding/binary"
+	"fmt"
+	"io"
+	"os"
+	"text/tabwriter"
+
+	"igit.com/xbase/raft/db"
+)
+
+const (
+	FlagDeleted = 0x00
+	FlagValid   = 0x01
+	HeaderSize  = 9
+)
+
+func main() {
+	if len(os.Args) < 2 {
+		fmt.Println("Usage: go run main.go <data_dir>")
+		return
+	}
+	dataDir := os.Args[1]
+	dataFile := dataDir + "/values.data"
+
+	f, err := os.Open(dataFile)
+	if err != nil {
+		fmt.Printf("Error opening file: %v\n", err)
+		return
+	}
+	defer f.Close()
+
+	fmt.Printf("Scanning storage file: %s\n\n", dataFile)
+
+	w := tabwriter.NewWriter(os.Stdout, 0, 0, 3, ' ', tabwriter.Debug)
+	fmt.Fprintln(w, "Offset\tFlag\tStatus\tCapacity\tLength\tContent")
+	fmt.Fprintln(w, "------\t----\t------\t--------\t------\t-------")
+
+	offset := int64(0)
+	reader := f
+
+	for {
+		// Read Header
+		header := make([]byte, HeaderSize)
+		if _, err := reader.ReadAt(header, offset); err != nil {
+			if err == io.EOF {
+				break
+			}
+			// If we hit EOF during read, it might be partial read at end
+			if n, _ := reader.ReadAt(header, offset); n == 0 {
+				break
+			}
+			fmt.Printf("Error reading header at offset %d: %v\n", offset, err)
+			break
+		}
+
+		flag := header[0]
+		cap := binary.LittleEndian.Uint32(header[1:])
+		length := binary.LittleEndian.Uint32(header[5:])
+
+		status := "VALID"
+		if flag == FlagDeleted {
+			status = "DELETED"
+		}
+
+		// Read Data
+		// Even if deleted, we read to show what was there (or just skip)
+		// But strictly we should read `cap` bytes to advance offset correctly.
+		data := make([]byte, cap)
+		if _, err := reader.ReadAt(data, offset+HeaderSize); err != nil {
+			fmt.Printf("Error reading data at offset %d: %v\n", offset+HeaderSize, err)
+			break
+		}
+
+		// Only show actual data content up to length, but indicate full capacity
+		content := ""
+		if flag == FlagValid {
+			if length <= cap {
+				content = string(data[:length])
+			} else {
+				content = fmt.Sprintf("<CORRUPT: len=%d > cap=%d>", length, cap)
+			}
+		} else {
+			content = "<deleted content>"
+			// Optionally we could try to show it if we wanted to debug
+			if length <= cap {
+				content = fmt.Sprintf("<deleted: %s>", string(data[:length]))
+			}
+		}
+
+		fmt.Fprintf(w, "%d\t0x%02X\t%s\t%d\t%d\t%q\n", offset, flag, status, cap, length, content)
+
+		offset += int64(HeaderSize + int(cap))
+	}
+	w.Flush()
+	fmt.Println()
+	
+	// Also try to open Engine to show logical view
+	e, err := db.NewEngine(dataDir)
+	if err == nil {
+		fmt.Println("Logical Key-Value View (Active Keys):")
+		fmt.Println("-------------------------------------")
+		// We need to expose some way to iterate keys in Engine for debug
+		// Since Engine doesn't expose iterator, we can't easily list all keys without modifying Engine.
+		// For now, let's just inspect the raw file structure which was the main request.
+		// But wait, user asked for "format output table data".
+		
+		// Ideally we modify Engine to support iteration or just rely on file scan.
+		// The file scan above shows PHYSICAL layout.
+		// Let's add a simple usage demo to generate some data if dir is empty.
+		e.Close()
+	}
+}
+

BIN
test_db_data/values.data