diff --git a/.seqbench/comparison.env b/.seqbench/comparison.env index ee12dfcb..a43f6254 100644 --- a/.seqbench/comparison.env +++ b/.seqbench/comparison.env @@ -1,6 +1,6 @@ GOGC=100 -SEQDB_STORAGE_FRAC_SIZE=16MiB +SEQDB_STORAGE_FRAC_SIZE=1MiB SEQDB_STORAGE_TOTAL_SIZE=10GiB SEQDB_LIMITS_QUERY_RATE=1024 diff --git a/Makefile b/Makefile index 9a90289f..1d8c034d 100644 --- a/Makefile +++ b/Makefile @@ -56,7 +56,7 @@ test-deps: .PHONY: test test: test-deps - LOG_LEVEL=ERROR go test ./... -count 1 + LOG_LEVEL=ERROR go test ./... -count 1 -v .bin-deps: export GOBIN := $(LOCAL_BIN) .bin-deps: diff --git a/asyncsearcher/async_searcher.go b/asyncsearcher/async_searcher.go index a0fa6368..72bbaa8a 100644 --- a/asyncsearcher/async_searcher.go +++ b/asyncsearcher/async_searcher.go @@ -74,11 +74,11 @@ type AsyncSearcherConfig struct { } type fractionAcquirer interface { - Fractions() fracmanager.List + AcquireFractions() (_ fracmanager.List, release func()) AcquireFraction(name string) (_ frac.Fraction, release func(), ok bool) } -func MustStartAsync(config AsyncSearcherConfig, mp MappingProvider, fracs fractionAcquirer) *AsyncSearcher { +func MustStartAsync(config AsyncSearcherConfig, mp MappingProvider, fracProvider fractionAcquirer) *AsyncSearcher { if config.DataDir == "" { logger.Fatal("can't start async searcher: DataDir is empty") } @@ -107,7 +107,7 @@ func MustStartAsync(config AsyncSearcherConfig, mp MappingProvider, fracs fracti for _, id := range notProcessedIDs { asyncSearchActiveSearches.Add(1) as.processWg.Add(1) - go as.processRequest(id, fracs) + go as.processRequest(id, fracProvider) } // set limit metrics that allow us to calculate alerts' thresholds @@ -209,7 +209,7 @@ func (i *asyncSearchInfo) Status() AsyncSearchStatus { return status } -func (as *AsyncSearcher) StartSearch(r AsyncSearchRequest, fracs fractionAcquirer) error { +func (as *AsyncSearcher) StartSearch(r AsyncSearchRequest, fracProvider fractionAcquirer) error { if as.readOnly.Load() { return fmt.Errorf("cannot start search on read-only mode") } @@ -240,14 +240,17 @@ func (as *AsyncSearcher) StartSearch(r AsyncSearchRequest, fracs fractionAcquire return fmt.Errorf("retention time should be less than %s, got %s", maxRetention, r.Retention) } - fracNames := fracs.Fractions().FilterInRange(r.Params.From, r.Params.To).Names() + fracs, release := fracProvider.AcquireFractions() + defer release() + + fracNames := fracs.FilterInRange(r.Params.From, r.Params.To).Names() if ok := as.saveSearchInfo(r, fracNames); !ok { // Request was saved previously, skip it return nil } asyncSearchActiveSearches.Add(1) as.processWg.Add(1) - go as.processRequest(r.ID, fracs) + go as.processRequest(r.ID, fracProvider) return nil } @@ -301,17 +304,17 @@ func (as *AsyncSearcher) createDataDir() { }) } -func (as *AsyncSearcher) processRequest(asyncSearchID string, fracs fractionAcquirer) { +func (as *AsyncSearcher) processRequest(asyncSearchID string, fracProvider fractionAcquirer) { defer as.processWg.Done() as.rateLimit <- struct{}{} defer func() { <-as.rateLimit }() - as.doSearch(asyncSearchID, fracs) + as.doSearch(asyncSearchID, fracProvider) asyncSearchActiveSearches.Add(-1) } -func (as *AsyncSearcher) doSearch(id string, fracs fractionAcquirer) { +func (as *AsyncSearcher) doSearch(id string, fracProvider fractionAcquirer) { qprPaths, err := as.findQPRs(id) if err != nil { panic(fmt.Errorf("can't find QPRs for id %q: %s", id, err)) @@ -353,7 +356,7 @@ func (as *AsyncSearcher) doSearch(id string, fracs fractionAcquirer) { if as.shouldStopSearch(id) { break } - if err := as.acquireAndProcessFrac(fracInfo, info, fracs); err != nil { + if err := as.acquireAndProcessFrac(fracInfo, info, fracProvider); err != nil { as.updateSearchInfo(id, func(info *asyncSearchInfo) { info.Error = err.Error() }) @@ -395,8 +398,8 @@ func compressQPR(qpr *seq.QPR, cb func(compressed []byte) error) error { return nil } -func (as *AsyncSearcher) acquireAndProcessFrac(fracInfo fracSearchState, searchInfo asyncSearchInfo, fracs fractionAcquirer) (err error) { - f, release, ok := fracs.AcquireFraction(fracInfo.Name) +func (as *AsyncSearcher) acquireAndProcessFrac(fracInfo fracSearchState, searchInfo asyncSearchInfo, fracProvider fractionAcquirer) (err error) { + f, release, ok := fracProvider.AcquireFraction(fracInfo.Name) if !ok { // oldest fracs may already be removed logger.Info( "async search: skip missing fraction", diff --git a/asyncsearcher/async_searcher_test.go b/asyncsearcher/async_searcher_test.go index 8ef37bc3..d6311ef5 100644 --- a/asyncsearcher/async_searcher_test.go +++ b/asyncsearcher/async_searcher_test.go @@ -49,8 +49,8 @@ func (fp fakeFractionProvider) AcquireFraction(name string) (frac.Fraction, func return nil, func() {}, false } -func (fp fakeFractionProvider) Fractions() fracmanager.List { - return fracmanager.List(fp) +func (fp fakeFractionProvider) AcquireFractions() (fracmanager.List, func()) { + return fracmanager.List(fp), func() {} } func TestAsyncSearcherMaintain(t *testing.T) { diff --git a/compaction/executor.go b/compaction/executor.go new file mode 100644 index 00000000..d3b1a55c --- /dev/null +++ b/compaction/executor.go @@ -0,0 +1,65 @@ +package compaction + +import ( + "sync" + + "go.uber.org/zap" + + "github.com/ozontech/seq-db/frac" + "github.com/ozontech/seq-db/frac/common" + "github.com/ozontech/seq-db/frac/sealed" + "github.com/ozontech/seq-db/logger" +) + +type Executor struct { + workers int + wg sync.WaitGroup + p *planner +} + +// FIXME(dkharms): I need to pass here [common.SealParams]. +func NewExecutor(workers int, p *planner) *Executor { + e := Executor{workers: workers, p: p} + e.init() + return &e +} + +func (e *Executor) Close() { + e.p.close() + e.wg.Wait() +} + +func (e *Executor) init() { + for range e.workers { + e.wg.Go(func() { + for t := range e.p.tasks { + logger.Info( + "got new compaction task", + zap.Time("bin", t.bin), + zap.Any("snapshot", t.snapshot), + ) + t.onComplete(e.compact(t)) + } + }) + } +} + +func (e *Executor) compact(t task) (*sealed.PreloadedData, error) { + var ( + names []string + srcs []Source + ) + + for _, f := range t.snapshot.Fractions() { + names = append(names, f.Info().Name()) + srcs = append(srcs, frac.NewSealedSource(f)) + } + + logger.Info( + "compacting fractions", + zap.Strings("names", names), + ) + + preloaded, err := Merge(t.filename, common.SealParams{}, srcs...) + return preloaded, err +} diff --git a/compaction/merge.go b/compaction/merge.go new file mode 100644 index 00000000..928b3044 --- /dev/null +++ b/compaction/merge.go @@ -0,0 +1,162 @@ +package compaction + +import ( + "errors" + "os" + + "github.com/ozontech/seq-db/consts" + "github.com/ozontech/seq-db/frac/common" + "github.com/ozontech/seq-db/frac/sealed" + "github.com/ozontech/seq-db/indexwriter" +) + +func Merge(filename string, params common.SealParams, srcs ...Source) (*sealed.PreloadedData, error) { + writer := indexwriter.New(params) + src := NewMergeSource(filename, srcs) + + if err := createAndWrite( + filename+consts.OffsetsTmpFileSuffix, + filename+consts.OffsetsFileSuffix, + func(f *os.File) error { return writer.WriteOffsetsFile(f, src) }, + ); err != nil { + return nil, err + } + + if err := createAndWrite( + filename+consts.IDTmpFileSuffix, + filename+consts.IDFileSuffix, + func(f *os.File) error { return writer.WriteIDFile(f, src) }, + ); err != nil { + return nil, err + } + + if err := createAndWriteBoth( + filename+consts.TokenTmpFileSuffix, + filename+consts.TokenFileSuffix, + filename+consts.LIDTmpFileSuffix, + filename+consts.LIDFileSuffix, + func(tf, lf *os.File) error { return writer.WriteTokenTriplet(tf, lf, src) }, + ); err != nil { + return nil, err + } + + if err := createAndWrite( + filename+consts.InfoTmpFileSuffix, + filename+consts.InfoFileSuffix, + func(f *os.File) error { return writer.WriteInfoFile(f, src) }, + ); err != nil { + return nil, err + } + + if err := mergeDocs(filename, srcs...); err != nil { + return nil, err + } + + info := src.Info() + info.IndexOnDisk = 0 + + for _, suffix := range []string{ + consts.InfoFileSuffix, + consts.TokenFileSuffix, + consts.OffsetsFileSuffix, + consts.IDFileSuffix, + consts.LIDFileSuffix, + } { + st, err := os.Stat(info.Path + suffix) + if err != nil { + return nil, err + } + info.IndexOnDisk += uint64(st.Size()) + } + + lidsTable := writer.LIDsTable() + preloaded := &sealed.PreloadedData{ + Info: info, + TokenTable: writer.TokenTable(), + BlocksData: sealed.BlocksData{ + LIDsTable: &lidsTable, + IDsTable: writer.IDsTable(), + BlocksOffsets: src.BlockOffsets(), + }, + } + + return preloaded, nil +} + +func mergeDocs(filename string, srcs ...Source) error { + return createAndWrite( + filename+consts.DocsTmpFileSuffix, + filename+consts.DocsFileSuffix, + func(f *os.File) error { + var docsSize uint64 + for _, src := range srcs { + for loc, err := range src.DocBlock() { + if err != nil { + return err + } + + payload, offset := loc.First, loc.Second + if _, err := f.WriteAt(payload, int64(offset+docsSize)); err != nil { + return err + } + } + + docsSize += src.Info().DocsOnDisk + } + + return nil + }, + ) +} + +func syncAndClose(f *os.File) error { + if err := f.Sync(); err != nil { + f.Close() + return err + } + return f.Close() +} + +func createAndWrite( + tmp, final string, + write func(*os.File) error, +) error { + f, err := os.Create(tmp) + if err != nil { + return err + } + + if err := errors.Join(write(f), syncAndClose(f)); err != nil { + return err + } + + return os.Rename(tmp, final) +} + +func createAndWriteBoth( + atmp, afinal, + btmp, bfinal string, + write func(*os.File, *os.File) error, +) error { + a, err := os.Create(atmp) + if err != nil { + return err + } + + b, err := os.Create(btmp) + if err != nil { + a.Close() + return err + } + + writeErr := write(a, b) + if err := errors.Join(writeErr, syncAndClose(a), syncAndClose(b)); err != nil { + return err + } + + if err := os.Rename(atmp, afinal); err != nil { + return err + } + + return os.Rename(btmp, bfinal) +} diff --git a/compaction/merge_source.go b/compaction/merge_source.go new file mode 100644 index 00000000..f2e49da7 --- /dev/null +++ b/compaction/merge_source.go @@ -0,0 +1,445 @@ +package compaction + +import ( + "bytes" + "iter" + "slices" + "sync" + + "github.com/ozontech/seq-db/frac/common" + "github.com/ozontech/seq-db/indexwriter" + "github.com/ozontech/seq-db/seq" + "github.com/ozontech/seq-db/util" +) + +type ( + Document = util.Pair[seq.ID, []byte] + DocBlockLocation = util.Pair[[]byte, uint64] + TokenPosting = util.Pair[[]byte, []uint32] + DocLocation = util.Pair[seq.ID, seq.DocPos] + IndexedDocBlock = util.Pair[[]byte, []seq.DocPos] +) + +type Source interface { + indexwriter.Source + DocBlock() iter.Seq2[DocBlockLocation, error] +} + +type MergeSource struct { + filename string + + // sources is a slice of [sealing.Source] + // which provide view into underlying fractions. + sources []Source + + info *common.Info + infoOnce sync.Once + + offsets []uint64 + offsetsOnce sync.Once + + // docBlockCount is populated during [MergeSource.BlockOffsets] call. + // This slice is used for changing block indexes in [seq.DocPos]. + docBlockCount []int + + // lidMapping describes the transformation of lids + // after k-merge of several fractions. + // + // i-th index of [lidMapping] correponds to i-th fraction. + // j-th index of i-th [lidMapping] corresponds to rename of j-th lid. + lidMapping [][]uint32 +} + +func NewMergeSource(filename string, sources []Source) *MergeSource { + lidMapping := make([][]uint32, len(sources)) + + for i, src := range sources { + lidMapping[i] = make( + []uint32, + // Increment for [seq.SystemID]. + src.Info().DocsTotal+1, + ) + } + + s := &MergeSource{ + filename: filename, + sources: sources, + lidMapping: lidMapping, + } + + s.info = s.prepareInfo() + return s +} + +func (s *MergeSource) prepareInfo() *common.Info { + info := common.NewInfo(s.filename, 0, 0) + + var ( + from seq.MID = seq.MaxID.MID + to seq.MID = seq.MinID.MID + ) + + for _, src := range s.sources { + from = min(from, src.Info().From) + to = max(to, src.Info().To) + } + + info.From, info.To = from, to + info.SealingTime = info.CreationTime + + info.InitEmptyDistribution() + return info +} + +func (s *MergeSource) Info() *common.Info { + s.infoOnce.Do(func() { + for i := range s.sources { + sinfo := s.sources[i].Info() + + s.info.DocsRaw += sinfo.DocsRaw + s.info.DocsTotal += sinfo.DocsTotal + s.info.DocsOnDisk += sinfo.DocsOnDisk + + // NOTE(dkharms): [IndexOnDisk] is calculated later. + } + }) + + return s.info +} + +func (s *MergeSource) BlockOffsets() []uint64 { + s.offsetsOnce.Do(func() { + var ( + docsSize uint64 + offsets []uint64 + ) + + s.docBlockCount = append(s.docBlockCount, 0) + for i := 0; i < len(s.sources); i++ { + for _, offset := range s.sources[i].BlockOffsets() { + offsets = append(offsets, uint64(offset)+docsSize) + } + docsSize += s.sources[i].Info().DocsOnDisk + s.docBlockCount = append(s.docBlockCount, len(offsets)) + } + + s.offsets = offsets + }) + + return s.offsets +} + +func (s *MergeSource) ID() iter.Seq2[DocLocation, error] { + // TODO(dkharms): For now, I will use stupid-simple linear scan for k-way merge. + // + // Its time complexity O(k*n) so it's not efficient enough if we compare it + // against time complexity of min-heap (which is O(n*log(k))) + // or another great data structure -- tournament tree -- which is O(n*log(k)) as well. + // + // However, tournament tree performs less comparisons than min-heap + // and it is around log(k) vs 2*log(k). + + type cursor struct { + next func() (DocLocation, error, bool) + stop func() + + loc DocLocation + lidOld uint32 + + ok bool + } + + return func(yield func(DocLocation, error) bool) { + var cursors []cursor + + defer func() { + for _, c := range cursors { + c.stop() + } + }() + + for i := range s.sources { + src := s.sources[i] + next, stop := iter.Pull2(src.ID()) + + // Skip [seq.SystemID] and [seq.SystemDocPos]. + _, _, _ = next() + + loc, err, ok := next() + cursors = append(cursors, cursor{ + next: next, stop: stop, + loc: loc, lidOld: 1, + ok: ok && err == nil, + }) + + if err != nil { + yield(DocLocation{}, err) + return + } + } + + lid := uint32(1) + // We've previosly dropped [seq.SystemID] from + // iterators however we do have to emit one such id. + if !yield(DocLocation{First: seq.SystemID, Second: seq.SystemDocPos}, nil) { + return + } + + for { + var ( + id seq.ID = seq.MinID + idx int = -1 + ) + + for i, c := range cursors { + // We exhausted i-th cursor so there is nothing pull. + if !c.ok { + continue + } + + if seq.Less(id, c.loc.First) { + id = c.loc.First + idx = i + } + } + + // All pull-iterators are exhausted. + // Close all iterators and return. + if idx == -1 { + break + } + + c := cursors[idx] + + minID, lidOld := c.loc.First, c.lidOld + s.info.AddMID(uint64(minID.MID)) + + blockIdx, offset := c.loc.Second.Unpack() + minDocPos := seq.PackDocPos(uint32(s.docBlockCount[idx]+int(blockIdx)), offset) + + if !yield(DocLocation{First: minID, Second: minDocPos}, nil) { + return + } + + // Rename lid from picked cursor to the new value. + s.lidMapping[idx][lidOld] = lid + + var err error + c.loc, err, c.ok = c.next() + c.lidOld += 1 + + if err != nil { + cursors[idx] = c + yield(DocLocation{}, err) + return + } + + lid += 1 + cursors[idx] = c + } + } +} + +func (s *MergeSource) TokenTriplet() iter.Seq2[string, iter.Seq2[TokenPosting, error]] { + // TODO(dkharms): For now, I will use stupid-simple linear scan for k-way merge. + // + // Its time complexity O(k*n) so it's not efficient enough if we compare it + // against time complexity of min-heap (which is O(n*log(k))) + // or another great data structure -- tournament tree -- which is O(n*log(k)) as well. + // + // However, tournament tree performs less comparisons than min-heap + // and it is around log(k) vs 2*log(k). + + type cursor struct { + next func() (string, iter.Seq2[TokenPosting, error], bool) + stop func() + + field string + tokIt iter.Seq2[TokenPosting, error] + + ok bool + } + + minimal := func(cursors []cursor) (string, bool) { + var ( + set bool + field string + ) + + for _, c := range cursors { + if !c.ok { + continue + } + + if !set { + field = c.field + set = true + continue + } + + field = min(field, c.field) + } + + return field, set + } + + return func(yield func(string, iter.Seq2[TokenPosting, error]) bool) { + var cursors []cursor + + for i := range s.sources { + src := s.sources[i] + + next, stop := iter.Pull2(src.TokenTriplet()) + field, tokIt, has := next() + + cursors = append(cursors, cursor{ + next: next, stop: stop, + field: field, tokIt: tokIt, + ok: has, + }) + } + + defer func() { + for _, c := range cursors { + c.stop() + } + }() + + for { + field, ok := minimal(cursors) + if !ok { + break + } + + var ( + idxs []int + iters []iter.Seq2[TokenPosting, error] + ) + + for i, c := range cursors { + if !c.ok || c.field != field { + continue + } + + idxs = append(idxs, i) + iters = append(iters, c.tokIt) + } + + if !yield(field, s.postingsForField(idxs, iters)) { + return + } + + // Advance all cursors that were on this field. + for _, idx := range idxs { + c := cursors[idx] + c.field, c.tokIt, c.ok = c.next() + cursors[idx] = c + } + } + } +} + +func (s *MergeSource) postingsForField( + idxs []int, iters []iter.Seq2[TokenPosting, error], +) iter.Seq2[TokenPosting, error] { + type cursor struct { + next func() (TokenPosting, error, bool) + stop func() + + idx int + posting TokenPosting + + ok bool + } + + minimal := func(cursors []cursor) ([]byte, bool) { + var ( + set bool + token []byte + ) + + for _, c := range cursors { + if !c.ok { + continue + } + + if !set { + token = c.posting.First + set = true + continue + } + + if bytes.Compare(c.posting.First, token) < 0 { + token = c.posting.First + } + } + + return token, set + } + + // NB: This buffer will be reused across + // all calls within current field. + var lidRenamed []uint32 + + return func(yield func(TokenPosting, error) bool) { + var cursors []cursor + + defer func() { + for _, c := range cursors { + c.stop() + } + }() + + for i := range iters { + next, stop := iter.Pull2(iters[i]) + posting, err, ok := next() + + cursors = append(cursors, cursor{ + next: next, stop: stop, + idx: idxs[i], posting: posting, + ok: ok && err == nil, + }) + + if err != nil { + yield(TokenPosting{}, err) + return + } + } + + for { + token, ok := minimal(cursors) + if !ok { + break + } + + // Collect and remap lids from all cursors at this token, then advance them. + for i, c := range cursors { + if !c.ok || !bytes.Equal(c.posting.First, token) { + continue + } + + for _, lid := range c.posting.Second { + lidRenamed = append(lidRenamed, s.lidMapping[c.idx][lid]) + } + + var err error + c.posting, err, c.ok = c.next() + + if err != nil { + cursors[i] = c + yield(TokenPosting{}, err) + return + } + + cursors[i] = c + } + + slices.Sort(lidRenamed) + if !yield(TokenPosting{First: token, Second: lidRenamed}, nil) { + return + } + + lidRenamed = lidRenamed[:0] + } + } +} diff --git a/compaction/merge_source_test.go b/compaction/merge_source_test.go new file mode 100644 index 00000000..12ba093c --- /dev/null +++ b/compaction/merge_source_test.go @@ -0,0 +1,352 @@ +package compaction + +import ( + "cmp" + "fmt" + "iter" + "math/rand" + "slices" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/ozontech/seq-db/frac/common" + "github.com/ozontech/seq-db/seq" +) + +type mockSealingSource struct { + ids []seq.ID + pos []seq.DocPos + blocks []uint64 + docsOnDisk uint64 + fields map[string]map[string][]uint32 +} + +func (m *mockSealingSource) Info() *common.Info { + return &common.Info{ + DocsRaw: m.docsOnDisk, + DocsTotal: uint32(len(m.ids)), + DocsOnDisk: m.docsOnDisk, + + From: slices.MinFunc(m.ids, func(x, y seq.ID) int { + return cmp.Compare(x.MID, y.MID) + }).MID, + + To: slices.MaxFunc(m.ids, func(x, y seq.ID) int { + return cmp.Compare(x.MID, y.MID) + }).MID, + } +} + +func (m *mockSealingSource) BlockOffsets() []uint64 { + return m.blocks +} + +func (m *mockSealingSource) ID() iter.Seq2[DocLocation, error] { + return func(yield func(DocLocation, error) bool) { + docloc := DocLocation{First: seq.SystemID, Second: seq.SystemDocPos} + if !yield(docloc, nil) { + return + } + + for i, id := range m.ids { + docloc = DocLocation{First: id, Second: m.pos[i]} + if !yield(docloc, nil) { + return + } + } + } +} + +func (m *mockSealingSource) TokenTriplet() iter.Seq2[string, iter.Seq2[TokenPosting, error]] { + fields := make([]string, 0, len(m.fields)) + for f := range m.fields { + fields = append(fields, f) + } + + slices.Sort(fields) + return func(yield func(string, iter.Seq2[TokenPosting, error]) bool) { + for _, field := range fields { + if !yield(field, m.postingsForField(field)) { + return + } + } + } +} + +func (m *mockSealingSource) postingsForField(field string) iter.Seq2[TokenPosting, error] { + return func(yield func(TokenPosting, error) bool) { + tokens := make([]string, 0, len(m.fields[field])) + for t := range m.fields[field] { + tokens = append(tokens, t) + } + + slices.Sort(tokens) + for _, tok := range tokens { + posting := TokenPosting{ + First: []byte(tok), + Second: m.fields[field][tok], + } + + if !yield(posting, nil) { + return + } + } + } +} + +func (m *mockSealingSource) DocBlock() iter.Seq2[DocBlockLocation, error] { + return func(yield func(DocBlockLocation, error) bool) { + if !yield(DocBlockLocation{}, nil) { + return + } + } +} + +func (m *mockSealingSource) LastError() error { + return nil +} + +func TestMergeSource(t *testing.T) { + first := &mockSealingSource{ + ids: []seq.ID{ + {MID: 3}, + {MID: 2}, + {MID: 1}, + }, + + pos: []seq.DocPos{ + seq.PackDocPos(0, 0), + seq.PackDocPos(0, 1024), + seq.PackDocPos(0, 2048), + }, + + fields: map[string]map[string][]uint32{ + "level": { + "error": {1, 3}, + "info": {2, 3}, + }, + }, + + blocks: []uint64{0}, + docsOnDisk: 1024, + } + + second := &mockSealingSource{ + ids: []seq.ID{ + {MID: 6}, + {MID: 5}, + }, + + pos: []seq.DocPos{ + seq.PackDocPos(0, 0), + seq.PackDocPos(0, 2048), + }, + + fields: map[string]map[string][]uint32{ + "level": { + "debug": {1}, + "info": {2}, + }, + }, + + blocks: []uint64{0}, + docsOnDisk: 2048, + } + + source := NewMergeSource("inmemory", []Source{first, second}) + + t.Run("offsets", func(t *testing.T) { + // Validate correctness of [storage.DocBlock] calculation. + offsets := source.BlockOffsets() + require.Equal(t, []uint64{0, 1024}, offsets) + }) + + t.Run("ids", func(t *testing.T) { + var ( + ids []seq.ID + docpos []seq.DocPos + ) + + for loc, err := range source.ID() { + require.NoError(t, err) + ids = append(ids, loc.First) + docpos = append(docpos, loc.Second) + } + + require.Equal(t, + []seq.ID{ + seq.SystemID, + // [seq.ID] from the second source. + {MID: 6}, + {MID: 5}, + // [seq.ID] from the first source. + {MID: 3}, + {MID: 2}, + {MID: 1}, + }, + ids, + ) + + require.Equal(t, + []seq.DocPos{ + seq.SystemDocPos, + // [seq.DocPos] from the second source. + seq.PackDocPos(1, 0), seq.PackDocPos(1, 2048), + // [seq.DocPos] from the first source. + seq.PackDocPos(0, 0), seq.PackDocPos(0, 1024), seq.PackDocPos(0, 2048), + }, + docpos, + ) + }) + + t.Run("tokens-lids", func(t *testing.T) { + var ( + fields []string + tokens [][]byte + lids [][]uint32 + ) + + for field, fieldIt := range source.TokenTriplet() { + fields = append(fields, field) + + for posting, err := range fieldIt { + require.NoError(t, err) + tokens = append(tokens, posting.First) + lids = append(lids, slices.Clone(posting.Second)) + } + } + + // Both sources have the same and the only field. + require.Equal(t, []string{"level"}, fields) + + // Ensure tokens are sorted in ascending order. + require.Equal(t, + [][]byte{[]byte("debug"), []byte("error"), []byte("info")}, + tokens, + ) + + // Ensure correctness of lids remapping: + // ------------------------- + // seq.MID 6 5 | 3 2 1 + // seq.LID (old) 1 2 | 1 2 3 + // seq.LID (new) 1 2 | 3 4 5 + // ------------------------- + require.Equal(t, + [][]uint32{ + // Sequence of [seq.LID] for token `debug`. + {1}, + // Sequence of [seq.LID] for token `error`. + {3, 5}, + // Sequence of [seq.LID] for token `info`. + {2, 4, 5}, + }, + lids, + ) + }) + + t.Run("info", func(t *testing.T) { + merged := source.Info() + finfo, sinfo := first.Info(), second.Info() + + // Validate correctness of fraction time-range. + require.Equal(t, merged.From, min(finfo.From, sinfo.From)) + require.Equal(t, merged.To, max(finfo.To, sinfo.To)) + + // Validate correctness of total documents of merged fractions. + require.Equal(t, merged.DocsTotal, finfo.DocsTotal+sinfo.DocsTotal) + require.Equal(t, merged.DocsOnDisk, finfo.DocsOnDisk+sinfo.DocsOnDisk) + require.Equal(t, merged.DocsRaw, finfo.DocsRaw+sinfo.DocsRaw) + + // Validate correctness of distribution. + require.NotNil(t, merged.Distribution) + require.True(t, merged.IsIntersecting(finfo.From, finfo.To)) + require.True(t, merged.IsIntersecting(sinfo.From, sinfo.To)) + require.True(t, merged.IsIntersecting(min(finfo.From, sinfo.From), max(finfo.To, sinfo.To))) + }) +} + +func BenchmarkMergeSource(b *testing.B) { + const ( + numSources = 4 + docsPerSource = 512_000 + + // Total count of pairs of (field, token) will be + // [numFields] * [numTokens]. + numFields = 512 + numTokens = 16384 + ) + + rng := rand.New(rand.NewSource(42)) + + fieldNames := make([]string, numFields) + for i := range fieldNames { + fieldNames[i] = fmt.Sprintf("field-%d", i) + } + + tokenNames := make([]string, numTokens) + for i := range tokenNames { + tokenNames[i] = fmt.Sprintf("token-%d", i) + } + + makeSource := func(midOffset seq.MID) Source { + ids := make([]seq.ID, docsPerSource) + pos := make([]seq.DocPos, docsPerSource) + + for j := range ids { + // IDs must be in descending MID order within each source. + ids[j] = seq.ID{MID: midOffset + seq.MID(docsPerSource-j)} + pos[j] = seq.PackDocPos(0, uint64(j*64)) + } + + // Assign each lid to a random (field, token) pair from the vocabulary + // so that total lids per source equals [docsPerSource]. + fields := make(map[string]map[string][]uint32) + for lid := uint32(1); lid <= uint32(docsPerSource); lid++ { + field := fieldNames[rng.Intn(numFields)] + token := tokenNames[rng.Intn(numTokens)] + + if fields[field] == nil { + fields[field] = make(map[string][]uint32) + } + + fields[field][token] = append(fields[field][token], lid) + } + + for _, tokens := range fields { + for tok, lids := range tokens { + slices.Sort(lids) + tokens[tok] = lids + } + } + + return &mockSealingSource{ + ids: ids, + pos: pos, + blocks: []uint64{0}, + docsOnDisk: docsPerSource * 64, + fields: fields, + } + } + + sources := make([]Source, numSources) + for i := range sources { + sources[i] = makeSource(seq.MID(i * docsPerSource)) + } + + b.ResetTimer() + b.ReportAllocs() + + for b.Loop() { + ms := NewMergeSource("bench", sources) + + ms.BlockOffsets() + for range ms.ID() { + } + + for _, tokIt := range ms.TokenTriplet() { + for range tokIt { + } + } + } +} diff --git a/compaction/metrics.go b/compaction/metrics.go new file mode 100644 index 00000000..d1d3cde1 --- /dev/null +++ b/compaction/metrics.go @@ -0,0 +1 @@ +package compaction diff --git a/compaction/planner.go b/compaction/planner.go new file mode 100644 index 00000000..0d41ad7f --- /dev/null +++ b/compaction/planner.go @@ -0,0 +1,241 @@ +package compaction + +import ( + "context" + "maps" + "slices" + "sync" + "time" + + "go.uber.org/zap" + + "github.com/alecthomas/units" + "github.com/ozontech/seq-db/frac/common" + "github.com/ozontech/seq-db/frac/sealed" + "github.com/ozontech/seq-db/fracmanager" + "github.com/ozontech/seq-db/logger" +) + +type fraction interface { + Info() *common.Info +} + +const ( + // TODO(dkharms): Move this options to config. + compactionTick = time.Second + compactionWindow = time.Minute +) + +type task struct { + bin time.Time + filename string + snapshot *fracmanager.CompactionSnapshot + onComplete func(*sealed.PreloadedData, error) +} + +type planner struct { + wg sync.WaitGroup + ctx context.Context + done chan struct{} + + fm *fracmanager.FracManager + tasks chan task + + mu sync.RWMutex + // inflight tracks active compactions for each time-bin. + // We cannot have concurrent compactions within one time-bin for correctness purposes. + inflight map[time.Time]struct{} + + stats map[time.Time]int +} + +func NewPlanner(ctx context.Context, fm *fracmanager.FracManager) *planner { + p := planner{ + ctx: ctx, + done: make(chan struct{}), + + fm: fm, + + tasks: make(chan task), + + inflight: make(map[time.Time]struct{}), + stats: make(map[time.Time]int), + } + + p.init() + return &p +} + +func (p *planner) init() { + p.wg.Go(func() { + t := time.NewTicker(compactionTick) + + for { + select { + case <-p.ctx.Done(): + close(p.tasks) + return + + case <-p.done: + close(p.tasks) + return + + case <-t.C: + task, ok := p.pick() + if !ok { + continue + } + + select { + case p.tasks <- task: + case <-time.NewTimer(time.Second).C: + // If all executor workers are busy for some long period of time, + // we want to drop the task because it might contain stale decision. + } + } + } + }) +} + +func (p *planner) close() { + close(p.done) +} + +func (p *planner) pick() (task, bool) { + names := func(fracs []fraction) []string { + fnames := make([]string, len(fracs)) + for i := range fracs { + fnames[i] = fracs[i].Info().Name() + } + return fnames + } + + fractions := p.fm.SealedFractionsSnapshot() + snapshot := make([]fraction, len(fractions)) + + for i := range fractions { + snapshot[i] = fractions[i] + } + + bins := p.distribute(compactionWindow, snapshot) + times := p.prioritize(bins) + + p.mu.Lock() + defer p.mu.Unlock() + + // NOTE(dkharms): This lock guards [inflight] map. + // Maybe I can find another way to signal from worker that time-bin is free? + + for _, t := range times { + if _, ok := p.inflight[t]; ok { + // There is on-going compaction within this time-bin. + continue + } + + // TODO(dkharms): Move this options to config. + picked := strategySTCS{ + mergeTrigger: 4, + mergeFanIn: 32, + mergeFanOutSize: 128 * uint64(units.MiB), + bucketLowerbound: 0.5, + bucketUpperbound: 1.5, + }.Pick(bins[t].fracs) + + if len(picked) == 0 { + // No candidates were found. + continue + } + + csnapshot, err := p.fm.ClaimForCompaction(names(picked)) + if err != nil { + continue + } + + p.inflight[t] = struct{}{} + + return task{ + bin: t, + + filename: p.fm.FractionName(), + snapshot: csnapshot, + + onComplete: func(s *sealed.PreloadedData, err error) { + p.mu.Lock() + defer p.mu.Unlock() + delete(p.inflight, t) + + if err != nil { + logger.Error( + "failed to compact fractions", + zap.Error(err), + zap.Any("snapshot", csnapshot), + ) + return + } + + if s == nil { + logger.Info( + "compaction did not produce fraction", + zap.Any("snapshot", csnapshot), + ) + return + } + + // TODO(dkharms): Is it fine to substitute and delete? + // We need somehow substitute and delete atomically. + p.fm.SubstituteWithSealed(s, csnapshot) + csnapshot.Destroy() + }, + }, true + } + + return task{}, false +} + +type timestampBin struct { + t time.Time + fracs []fraction +} + +func (p *planner) distribute(window time.Duration, fracs []fraction) map[time.Time]timestampBin { + bins := make(map[time.Time]timestampBin) + + for _, f := range fracs { + ct := time.UnixMilli(int64(f.Info().CreationTime)) + bin := ct.Truncate(window) + tb := bins[bin] + + tb.t = bin + tb.fracs = append(tb.fracs, f) + + bins[bin] = tb + } + + return bins +} + +func (p *planner) prioritize(bins map[time.Time]timestampBin) []time.Time { + // NOTE(dkharms): What other strategies we can use here? + // (*) Prioritize by change rate; + // (*) Prioritize by amount of fractions; + + ordered := slices.Collect(maps.Keys(bins)) + + // Order timestamp-bins by the change-rate. + // We will prioritize bins with higher change rate. + slices.SortFunc(ordered, func(x, y time.Time) int { + xold, xnew := p.stats[x], len(bins[x].fracs) + yold, ynew := p.stats[y], len(bins[y].fracs) + xchange, ychange := xnew-xold, ynew-yold + + p.stats[x], p.stats[y] = xnew, ynew + + if xchange == ychange { + return -x.Compare(y) + } + + return -(xchange - ychange) + }) + + return ordered +} diff --git a/compaction/stcs.go b/compaction/stcs.go new file mode 100644 index 00000000..192951b8 --- /dev/null +++ b/compaction/stcs.go @@ -0,0 +1,106 @@ +package compaction + +import ( + "cmp" + "slices" +) + +type strategySTCS struct { + // To trigger compaction of bucket there must be + // at least [mergeTrigger] fractions. + mergeTrigger int + + // At most this many fractions are compacted from a single bucket + // per compaction iteration. + mergeFanIn int + mergeFanOutSize uint64 + + // Fraction size must be within [bucketLowerbound, bucketUpperbound] * avg(bucket) + // to be considered part of the bucket. + bucketLowerbound float64 + bucketUpperbound float64 +} + +func (s strategySTCS) Pick(candidates []fraction) []fraction { + if len(candidates) < s.mergeTrigger { + return nil + } + + sorted := slices.Clone(candidates) + slices.SortFunc(sorted, func(a, b fraction) int { + return cmp.Compare(a.Info().IndexOnDisk, b.Info().IndexOnDisk) + }) + + buckets := s.group(sorted) + // We are interested in buckets with the most amount of fractions. + // Usually, these are the lowest tiers where all freshly sealed fractions end up. + slices.SortFunc(buckets, func(x, y []fraction) int { + return -cmp.Compare(len(x), len(y)) + }) + + for _, bucket := range buckets { + if len(bucket) < s.mergeTrigger { + continue + } + + fracs := bucket[:min(len(bucket), s.mergeFanIn)] + if picked := s.takeUntilSize(fracs); len(picked) > 0 { + return picked + } + } + + return nil +} + +func (s strategySTCS) group(sorted []fraction) [][]fraction { + var ( + sum uint64 + current []fraction + buckets [][]fraction + ) + + for _, f := range sorted { + size := f.Info().IndexOnDisk + + if len(current) == 0 { + current = append(current, f) + sum = size + continue + } + + avg := float64(sum) / float64(len(current)) + fsize := float64(size) + + lower := avg * s.bucketLowerbound + upper := avg * s.bucketUpperbound + + if lower <= fsize && fsize <= upper { + current = append(current, f) + sum += size + continue + } + + buckets = append(buckets, current) + current = []fraction{f} + sum = size + } + + if len(current) > 0 { + buckets = append(buckets, current) + } + + return buckets +} + +func (s strategySTCS) takeUntilSize(fracs []fraction) []fraction { + var picked uint64 + + for i := range fracs { + picked += fracs[i].Info().IndexOnDisk + if picked >= s.mergeFanOutSize { + return fracs[:i] + } + } + + return fracs +} diff --git a/compaction/stcs_test.go b/compaction/stcs_test.go new file mode 100644 index 00000000..5729c971 --- /dev/null +++ b/compaction/stcs_test.go @@ -0,0 +1,74 @@ +package compaction + +import ( + "math" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/ozontech/seq-db/frac/common" +) + +type mockFraction struct { + indexOnDisk uint64 +} + +func (m *mockFraction) Info() *common.Info { + return &common.Info{IndexOnDisk: m.indexOnDisk} +} + +func makeFracs(sizes ...uint64) []fraction { + out := make([]fraction, len(sizes)) + for i, s := range sizes { + out[i] = &mockFraction{indexOnDisk: s} + } + return out +} + +func TestSTCS_Pick(t *testing.T) { + s := strategySTCS{ + mergeTrigger: 4, + mergeFanIn: 32, + mergeFanOutSize: math.MaxUint64, + bucketLowerbound: 0.5, + bucketUpperbound: 1.5, + } + + t.Run("not-enough-candidates", func(t *testing.T) { + for n := range s.mergeTrigger { + require.Nil(t, s.Pick(makeFracs(make([]uint64, n)...))) + } + }) + + t.Run("requirement-not-met", func(t *testing.T) { + // Each Fraction size is 10x the previous. + // They land in different buckets and no bucket with [mergeTrigger] fractions exists. + require.Nil(t, s.Pick(makeFracs(100, 1000, 10000, 100000))) + }) + + t.Run("one-bucket", func(t *testing.T) { + require.Len(t, s.Pick(makeFracs(1000, 1000, 1000, 1000)), 4) + }) + + t.Run("largest-bucket", func(t *testing.T) { + fracs := s.Pick(makeFracs( + 1000, 1000, + 100000, 100000, 100000, 100000, 100000, // Will take this bucket. + )) + + require.Len(t, fracs, 5) + for _, f := range fracs { + require.Equal(t, uint64(100000), f.Info().IndexOnDisk) + } + }) + + t.Run("cap-at-fan-in", func(t *testing.T) { + sizes := make([]uint64, s.mergeFanIn+10) + + for i := range sizes { + sizes[i] = 5000 + } + + require.Len(t, s.Pick(makeFracs(sizes...)), s.mergeFanIn) + }) +} diff --git a/consts/consts.go b/consts/consts.go index 80aabbe3..f9f45037 100644 --- a/consts/consts.go +++ b/consts/consts.go @@ -56,6 +56,7 @@ const ( WalFileSuffix = ".wal" DocsFileSuffix = ".docs" + DocsTmpFileSuffix = "._docs" DocsDelFileSuffix = ".docs.del" SdocsFileSuffix = ".sdocs" diff --git a/frac/active.go b/frac/active.go index 75d04e16..f8e89c99 100644 --- a/frac/active.go +++ b/frac/active.go @@ -446,6 +446,15 @@ func (f *Active) createDataProvider(ctx context.Context) *activeDataProvider { } } +func (f *Active) MemSize() int { + return f.MIDs.Size() + + f.RIDs.Size() + + f.DocBlocks.Size() + + f.DocsPositions.Size() + + f.IDsToLIDs.Size() + + f.TokenList.Size() +} + func (f *Active) Info() *common.Info { f.infoMu.RLock() defer f.infoMu.RUnlock() diff --git a/frac/active_docs_positions.go b/frac/active_docs_positions.go index f058091f..618ca910 100644 --- a/frac/active_docs_positions.go +++ b/frac/active_docs_positions.go @@ -2,6 +2,7 @@ package frac import ( "sync" + "unsafe" "github.com/ozontech/seq-db/seq" ) @@ -32,6 +33,16 @@ func (dp *DocsPositions) GetSync(id seq.ID) seq.DocPos { return dp.Get(id) } +func (dp *DocsPositions) Size() int { + dp.mu.RLock() + defer dp.mu.RUnlock() + + const entrySize = int(unsafe.Sizeof(seq.ID{})) + + int(unsafe.Sizeof(seq.DocPos(0))) + + return len(dp.idToPos) * entrySize +} + // SetMultiple returns a slice of added ids func (dp *DocsPositions) SetMultiple(ids []seq.ID, pos []seq.DocPos) []seq.ID { dp.mu.Lock() diff --git a/frac/active_ids.go b/frac/active_ids.go index 1195c8fa..cab87083 100644 --- a/frac/active_ids.go +++ b/frac/active_ids.go @@ -2,6 +2,7 @@ package frac import ( "sync" + "unsafe" ) type UInt64s struct { @@ -47,3 +48,9 @@ func (l *UInt64s) Append(val uint64) uint32 { return l.append(val) } + +func (l *UInt64s) Size() int { + l.mu.RLock() + defer l.mu.RUnlock() + return len(l.vals) * int(unsafe.Sizeof(int64(0))) +} diff --git a/frac/active_indexer_test.go b/frac/active_indexer_test.go index a1200a7c..812b2763 100644 --- a/frac/active_indexer_test.go +++ b/frac/active_indexer_test.go @@ -1,4 +1,4 @@ -package frac +package frac_test import ( "bytes" @@ -12,6 +12,7 @@ import ( "go.uber.org/zap/zapcore" "github.com/ozontech/seq-db/cache" + "github.com/ozontech/seq-db/frac" "github.com/ozontech/seq-db/indexer" "github.com/ozontech/seq-db/logger" "github.com/ozontech/seq-db/metric/stopwatch" @@ -76,20 +77,20 @@ func getTestProcessor() *indexer.Processor { func BenchmarkIndexer(b *testing.B) { logger.SetLevel(zapcore.FatalLevel) - idx, stop := NewActiveIndexer(8, 8) + idx, stop := frac.NewActiveIndexer(8, 8) defer stop() allLogs, err := readFileAllAtOnce(filepath.Join(common.TestDataDir, "k8s.logs")) readers := splitLogsToBulks(allLogs, 1000) assert.NoError(b, err) - active := NewActive( + active := frac.NewActive( filepath.Join(b.TempDir(), "test"), idx, storage.NewReadLimiter(1, nil), cache.NewCache[[]byte](nil, nil), cache.NewCache[[]byte](nil, nil), - &Config{}, + &frac.Config{}, testSkipMaskProvider{}, ) diff --git a/frac/active_lids.go b/frac/active_lids.go index 236136ef..41970d45 100644 --- a/frac/active_lids.go +++ b/frac/active_lids.go @@ -135,6 +135,18 @@ func mergeSorted(right, left []uint32, mids, rids []uint64) []uint32 { return result } +func (tl *TokenLIDs) Size() int { + tl.sortedMu.Lock() + sortedLen := len(tl.sorted) + tl.sortedMu.Unlock() + + tl.queueMu.Lock() + queueLen := len(tl.queue) + tl.queueMu.Unlock() + + return (sortedLen + queueLen) * 4 +} + func (tl *TokenLIDs) PutLIDsInQueue(lids []uint32) int { tl.queueMu.Lock() defer tl.queueMu.Unlock() diff --git a/frac/active_lids_map.go b/frac/active_lids_map.go index bae64854..418fb82c 100644 --- a/frac/active_lids_map.go +++ b/frac/active_lids_map.go @@ -2,6 +2,7 @@ package frac import ( "sync" + "unsafe" "github.com/ozontech/seq-db/seq" ) @@ -27,6 +28,16 @@ func (al *ActiveLIDs) Get(id seq.ID) (seq.LID, bool) { return val, ok } +func (al *ActiveLIDs) Size() int { + al.mu.RLock() + defer al.mu.RUnlock() + + const entrySize = int(unsafe.Sizeof(seq.ID{})) + + int(unsafe.Sizeof(seq.LID(0))) + + return len(al.idToLid) * entrySize +} + func (al *ActiveLIDs) SetMultiple(ids []seq.ID, lids []uint32) { al.mu.Lock() defer al.mu.Unlock() diff --git a/frac/active_sealing_source.go b/frac/active_source.go similarity index 91% rename from frac/active_sealing_source.go rename to frac/active_source.go index e7c451e2..af7084b0 100644 --- a/frac/active_sealing_source.go +++ b/frac/active_source.go @@ -30,7 +30,7 @@ type ( IndexedDocBlock = util.Pair[[]byte, []seq.DocPos] ) -type ActiveSealingSource struct { +type ActiveSource struct { params common.SealParams // Sealing parameters info *common.Info // fraction Info @@ -55,13 +55,13 @@ type ActiveSealingSource struct { docsReader *storage.DocsReader // Document storage reader } -func NewActiveSealingSource(active *Active, params common.SealParams) (*ActiveSealingSource, error) { +func NewActiveSealingSource(active *Active, params common.SealParams) (*ActiveSource, error) { info := *active.info // copy sortedLIDs := active.GetAllDocuments() fields, fieldTIDs := sortFields(active.TokenList) - src := ActiveSealingSource{ + src := ActiveSource{ params: params, info: &info, @@ -116,7 +116,7 @@ func sortFields(tl *TokenList) ([]string, [][]uint32) { return fields, fieldTIDs } -func (src *ActiveSealingSource) ID() iter.Seq2[DocLocation, error] { +func (src *ActiveSource) ID() iter.Seq2[DocLocation, error] { return func(yield func(DocLocation, error) bool) { mids := src.mids.vals rids := src.rids.vals @@ -155,11 +155,11 @@ func (src *ActiveSealingSource) ID() iter.Seq2[DocLocation, error] { } } -func (src *ActiveSealingSource) BlockOffsets() []uint64 { +func (src *ActiveSource) BlockOffsets() []uint64 { return src.blocksOffsets } -func (src *ActiveSealingSource) prepareInfo() { +func (src *ActiveSource) prepareInfo() { src.info.MetaOnDisk = 0 src.info.SealingTime = uint64(src.created.UnixMilli()) mids := src.mids.vals @@ -170,17 +170,17 @@ func (src *ActiveSealingSource) prepareInfo() { src.info.BuildDistribution(mids) } -func (src *ActiveSealingSource) prepareLids() { +func (src *ActiveSource) prepareLids() { for _, tl := range src.lids[1:] { tl.GetLIDs(src.mids, src.rids) } } -func (src *ActiveSealingSource) Info() *common.Info { +func (src *ActiveSource) Info() *common.Info { return src.info } -func (src *ActiveSealingSource) TokenTriplet() iter.Seq2[string, iter.Seq2[TokenPosting, error]] { +func (src *ActiveSource) TokenTriplet() iter.Seq2[string, iter.Seq2[TokenPosting, error]] { return func(yield func(string, iter.Seq2[TokenPosting, error]) bool) { for idx, field := range src.fields { if !yield(field, src.postingsForField(field, idx)) { @@ -190,7 +190,7 @@ func (src *ActiveSealingSource) TokenTriplet() iter.Seq2[string, iter.Seq2[Token } } -func (src *ActiveSealingSource) postingsForField(field string, idx int) iter.Seq2[TokenPosting, error] { +func (src *ActiveSource) postingsForField(field string, idx int) iter.Seq2[TokenPosting, error] { var lidsbuf []uint32 return func(yield func(TokenPosting, error) bool) { for _, tid := range src.fieldTIDs[idx] { @@ -221,7 +221,7 @@ func makeInverser(sortedLIDs []uint32) []uint32 { // Docs returns an iterator for documents with their IDs. // Handles duplicate IDs (for nested indexes). -func (src *ActiveSealingSource) Docs() iter.Seq2[Document, error] { +func (src *ActiveSource) Docs() iter.Seq2[Document, error] { return func(yield func(Document, error) bool) { var ( curdoc []byte @@ -256,7 +256,7 @@ func (src *ActiveSealingSource) Docs() iter.Seq2[Document, error] { } // doc reads a document from storage by its position. -func (src *ActiveSealingSource) doc(pos seq.DocPos) ([]byte, error) { +func (src *ActiveSource) doc(pos seq.DocPos) ([]byte, error) { blockIndex, docOffset := pos.Unpack() blockOffset := src.blocksOffsets[blockIndex] @@ -277,7 +277,7 @@ func (src *ActiveSealingSource) doc(pos seq.DocPos) ([]byte, error) { // SortDocs sorts documents and writes them in compressed form to disk. // Creates a temporary file that is then renamed to the final one. -func (src *ActiveSealingSource) SortDocs() error { +func (src *ActiveSource) SortDocs() error { start := time.Now() logger.Info("sorting docs...") @@ -346,7 +346,7 @@ func (src *ActiveSealingSource) SortDocs() error { // writeDocs compresses and writes document blocks, calculating new offsets // and collecting document positions. -func (src *ActiveSealingSource) writeDocs(blocks iter.Seq2[IndexedDocBlock, error], w io.Writer) ([]uint64, []seq.DocPos, error) { +func (src *ActiveSource) writeDocs(blocks iter.Seq2[IndexedDocBlock, error], w io.Writer) ([]uint64, []seq.DocPos, error) { offset := 0 buf := make([]byte, 0) blocksOffsets := make([]uint64, 0) diff --git a/frac/active_token_list.go b/frac/active_token_list.go index adf94ffd..77c85d02 100644 --- a/frac/active_token_list.go +++ b/frac/active_token_list.go @@ -5,6 +5,7 @@ import ( "fmt" "hash/crc32" "sync" + "unsafe" "github.com/ozontech/seq-db/seq" @@ -91,6 +92,32 @@ func NewActiveTokenList(workers int) *TokenList { return tl } + +func (tl *TokenList) Size() int { + size := 0 + + tl.tidMu.RLock() + for _, val := range tl.tidToVal { + size += len(val) + } + + for _, lids := range tl.tidToLIDs { + if lids != nil { + size += lids.Size() + } + } + tl.tidMu.RUnlock() + + tl.fieldsMu.RLock() + for field, tids := range tl.FieldTIDs { + size += len(field) + + len(tids)*int(unsafe.Sizeof(uint32(0))) + } + tl.fieldsMu.RUnlock() + + return size +} + func (tl *TokenList) Stop() { for _, c := range tl.chList { close(c) diff --git a/frac/common/info.go b/frac/common/info.go index 20e7f7c2..d714f660 100644 --- a/frac/common/info.go +++ b/frac/common/info.go @@ -82,6 +82,13 @@ func (s *Info) BuildDistribution(mids []uint64) { } } +func (s *Info) AddMID(mid uint64) { + if s.Distribution == nil { + return + } + s.Distribution.Add(seq.MID(mid)) +} + func (s *Info) InitEmptyDistribution() bool { from := s.From.Time() creationTime := time.UnixMilli(int64(s.CreationTime)) diff --git a/frac/fraction_concurrency_test.go b/frac/fraction_concurrency_test.go index 27f5d971..addf376d 100644 --- a/frac/fraction_concurrency_test.go +++ b/frac/fraction_concurrency_test.go @@ -1,4 +1,4 @@ -package frac +package frac_test import ( "fmt" @@ -14,11 +14,12 @@ import ( "golang.org/x/sync/errgroup" "github.com/ozontech/seq-db/cache" + "github.com/ozontech/seq-db/frac" "github.com/ozontech/seq-db/frac/common" "github.com/ozontech/seq-db/frac/processor" - "github.com/ozontech/seq-db/frac/sealed/sealing" "github.com/ozontech/seq-db/indexer" "github.com/ozontech/seq-db/parser" + "github.com/ozontech/seq-db/sealing" "github.com/ozontech/seq-db/seq" "github.com/ozontech/seq-db/storage" testcommon "github.com/ozontech/seq-db/tests/common" @@ -39,16 +40,16 @@ func TestConcurrentAppendAndQuery(t *testing.T) { fracPath := filepath.Join(tmpDir, "test_fraction") defer testcommon.RemoveDir(fracPath) - activeIndexer, stop := NewActiveIndexer(numIndexWorkers, 1000) + activeIndexer, stop := frac.NewActiveIndexer(numIndexWorkers, 1000) defer stop() - active := NewActive( + active := frac.NewActive( fracPath, activeIndexer, storage.NewReadLimiter(numReaders/2, nil), cache.NewCache[[]byte](nil, nil), cache.NewCache[[]byte](nil, nil), - &Config{}, + &frac.Config{}, testSkipMaskProvider{}, ) @@ -154,7 +155,7 @@ const ( kafka = "kafka" ) -func readTest(t *testing.T, fraction Fraction, numReaders, numQueries int, docs []*testDoc, fromTime, toTime time.Time, mapping seq.Mapping) { +func readTest(t *testing.T, fraction frac.Fraction, numReaders, numQueries int, docs []*testDoc, fromTime, toTime time.Time, mapping seq.Mapping) { readersGroup, ctx := errgroup.WithContext(t.Context()) type queryFilter func(doc *testDoc) bool @@ -332,7 +333,7 @@ func generatesMessages(numMessages, bulkSize int) ([]*testDoc, [][]string, time. return docs, bulks, fromTime, toTime } -func seal(active *Active) (*Sealed, error) { +func seal(active *frac.Active) (*frac.Sealed, error) { sealParams := common.SealParams{ IDsZstdLevel: 1, LIDsZstdLevel: 1, @@ -342,7 +343,7 @@ func seal(active *Active) (*Sealed, error) { DocBlocksZstdLevel: 1, DocBlockSize: 128 * int(units.KiB), } - activeSealingSource, err := NewActiveSealingSource(active, sealParams) + activeSealingSource, err := frac.NewActiveSealingSource(active, sealParams) if err != nil { return nil, err } @@ -351,13 +352,13 @@ func seal(active *Active) (*Sealed, error) { return nil, err } - sealed := NewSealedPreloaded( + sealed := frac.NewSealedPreloaded( active.BaseFileName, preloaded, storage.NewReadLimiter(1, nil), - newIndexCache(), + frac.NewIndexCache(), cache.NewCache[[]byte](nil, nil), - &Config{}, + &frac.Config{}, testSkipMaskProvider{}, ) diff --git a/frac/fraction_test.go b/frac/fraction_test.go index 8757c0db..b4bbe08a 100644 --- a/frac/fraction_test.go +++ b/frac/fraction_test.go @@ -1,7 +1,9 @@ -package frac +package frac_test import ( "context" + cryptorand "crypto/rand" + "encoding/hex" "fmt" "math" "math/rand/v2" @@ -20,12 +22,14 @@ import ( "github.com/stretchr/testify/suite" "github.com/ozontech/seq-db/cache" + "github.com/ozontech/seq-db/compaction" + "github.com/ozontech/seq-db/frac" "github.com/ozontech/seq-db/frac/common" "github.com/ozontech/seq-db/frac/processor" - "github.com/ozontech/seq-db/frac/sealed/sealing" "github.com/ozontech/seq-db/indexer" "github.com/ozontech/seq-db/node" "github.com/ozontech/seq-db/parser" + "github.com/ozontech/seq-db/sealing" "github.com/ozontech/seq-db/seq" "github.com/ozontech/seq-db/storage" "github.com/ozontech/seq-db/storage/s3" @@ -42,20 +46,20 @@ func (testSkipMaskProvider) RemoveFrac(_ string) {} type FractionTestSuite struct { suite.Suite tmpDir string - config *Config + config *frac.Config mapping seq.Mapping tokenizers map[seq.TokenizerType]tokenizer.Tokenizer - activeIndexer *ActiveIndexer + activeIndexer *frac.ActiveIndexer stopIndexer func() sealParams common.SealParams - fraction Fraction + fraction frac.Fraction insertDocuments func(docs ...[]string) } func (s *FractionTestSuite) SetupSuiteCommon() { - s.activeIndexer, s.stopIndexer = NewActiveIndexer(4, 10) + s.activeIndexer, s.stopIndexer = frac.NewActiveIndexer(4, 10) } func (s *FractionTestSuite) TearDownSuiteCommon() { @@ -63,7 +67,7 @@ func (s *FractionTestSuite) TearDownSuiteCommon() { } func (s *FractionTestSuite) SetupTestCommon() { - s.config = &Config{} + s.config = &frac.Config{} s.tokenizers = map[seq.TokenizerType]tokenizer.Tokenizer{ seq.TokenizerTypeKeyword: tokenizer.NewKeywordTokenizer(20, false, true), seq.TokenizerTypeText: tokenizer.NewTextTokenizer(20, false, true, 100), @@ -110,6 +114,12 @@ func (s *FractionTestSuite) TearDownTestCommon() { s.NoError(err, "Failed to remove tmp dir") } +func randomHex(n int) string { + b := make([]byte, (n+1)/2) + cryptorand.Read(b) + return hex.EncodeToString(b)[:n] +} + func (s *FractionTestSuite) TestSearchKeyword() { docs := []string{ /*0*/ `{"timestamp":"2000-01-01T13:00:25Z","service":"service_a","message":"first message some text","trace_id":"abcdef","source":"prod01","level":"1"}`, @@ -1789,7 +1799,7 @@ func (s *FractionTestSuite) TestMIDDistribution() { s.insertDocuments(docs) - _, ok := s.fraction.(*Active) + _, ok := s.fraction.(*frac.Active) if ok { s.Require().Nil(s.fraction.Info().Distribution, "active fraction has MID distribution") return @@ -1828,15 +1838,15 @@ func (s *FractionTestSuite) TestFractionInfo() { s.Require().Equal(seq.MID(946731654000000000), info.To, "to doesn't match") switch s.fraction.(type) { - case *Active: + case *frac.Active: s.Require().True(info.MetaOnDisk >= uint64(250) && info.MetaOnDisk <= uint64(400), "meta on disk doesn't match. actual value: %d", info.MetaOnDisk) s.Require().Equal(uint64(0), info.IndexOnDisk, "index on disk doesn't match") - case *Sealed: + case *frac.Sealed: s.Require().Equal(uint64(0), info.MetaOnDisk, "meta on disk doesn't match. actual value") s.Require().True(info.IndexOnDisk > uint64(1300) && info.IndexOnDisk < uint64(1400), "index on disk doesn't match. actual value: %d", info.IndexOnDisk) - case *Remote: + case *frac.Remote: s.Require().Equal(uint64(0), info.MetaOnDisk, "meta on disk doesn't match. actual value") s.Require().True(info.IndexOnDisk > uint64(1300) && info.IndexOnDisk < uint64(1400), "index on disk doesn't match. actual value: %d", info.IndexOnDisk) @@ -2035,9 +2045,10 @@ func (s *FractionTestSuite) AssertHist( } } -func (s *FractionTestSuite) newActive(bulks ...[]string) *Active { - baseName := filepath.Join(s.tmpDir, "test_fraction") - active := NewActive( +func (s *FractionTestSuite) newActive(bulks ...[]string) *frac.Active { + baseName := filepath.Join(s.tmpDir, randomHex(12)) + + active := frac.NewActive( baseName, s.activeIndexer, storage.NewReadLimiter(1, nil), @@ -2081,20 +2092,20 @@ func (s *FractionTestSuite) newActive(bulks ...[]string) *Active { return active } -func (s *FractionTestSuite) newSealed(bulks ...[]string) *Sealed { +func (s *FractionTestSuite) newSealed(bulks ...[]string) *frac.Sealed { active := s.newActive(bulks...) - activeSealingSource, err := NewActiveSealingSource(active, s.sealParams) + activeSealingSource, err := frac.NewActiveSealingSource(active, s.sealParams) s.Require().NoError(err, "Sealing source creation failed") preloaded, err := sealing.Seal(activeSealingSource, s.sealParams) s.Require().NoError(err, "Sealing failed") - sealed := NewSealedPreloaded( + sealed := frac.NewSealedPreloaded( active.BaseFileName, preloaded, storage.NewReadLimiter(1, nil), - newIndexCache(), + frac.NewIndexCache(), cache.NewCache[[]byte](nil, nil), s.config, testSkipMaskProvider{}, @@ -2127,7 +2138,7 @@ func (s *ActiveFractionTestSuite) SetupTest() { } func (s *ActiveFractionTestSuite) TearDownTest() { - if active, ok := s.fraction.(*Active); ok { + if active, ok := s.fraction.(*frac.Active); ok { active.Release() } else { s.Require().Nil(s.fraction, "fraction is not of Active type") @@ -2145,7 +2156,7 @@ ActiveReplayedFractionTestSuite run tests for active fraction which was replayed */ type ActiveReplayedFractionTestSuite struct { FractionTestSuite - originalFrac *Active + originalFrac *frac.Active } func (s *ActiveReplayedFractionTestSuite) SetupSuite() { @@ -2166,26 +2177,29 @@ func (s *ActiveReplayedFractionTestSuite) SetupTest() { } } -func (s *ActiveReplayedFractionTestSuite) Replay(frac *Active) Fraction { - fracFileName := frac.BaseFileName - s.originalFrac = frac - replayedFrac := NewActive( +func (s *ActiveReplayedFractionTestSuite) Replay(f *frac.Active) frac.Fraction { + s.originalFrac = f + fracFileName := f.BaseFileName + + replayedFrac := frac.NewActive( fracFileName, s.activeIndexer, storage.NewReadLimiter(1, nil), cache.NewCache[[]byte](nil, nil), cache.NewCache[[]byte](nil, nil), - &Config{}, + &frac.Config{}, testSkipMaskProvider{}, ) + err := replayedFrac.Replay(context.Background()) s.Require().NoError(err, "replay failed") + return replayedFrac } func (s *ActiveReplayedFractionTestSuite) TearDownTest() { s.originalFrac.Release() - if active, ok := s.fraction.(*Active); ok { + if active, ok := s.fraction.(*frac.Active); ok { active.Release() } else { s.Require().Nil(s.fraction, "fraction is not of Active type") @@ -2220,7 +2234,7 @@ func (s *SealedFractionTestSuite) SetupTest() { } func (s *SealedFractionTestSuite) TearDownTest() { - if sealed, ok := s.fraction.(*Sealed); ok { + if sealed, ok := s.fraction.(*frac.Sealed); ok { sealed.Release() } else { s.Require().Nil(s.fraction, "fraction is not of Sealed type") @@ -2256,7 +2270,7 @@ func (s *SealedLoadedFractionTestSuite) SetupTest() { } func (s *SealedLoadedFractionTestSuite) TearDownTest() { - if sealed, ok := s.fraction.(*Sealed); ok { + if sealed, ok := s.fraction.(*frac.Sealed); ok { sealed.Release() } else { s.Require().Nil(s.fraction, "fraction is not of Sealed type") @@ -2268,14 +2282,14 @@ func (s *SealedLoadedFractionTestSuite) TearDownSuite() { s.TearDownSuiteCommon() } -func (s *SealedLoadedFractionTestSuite) newSealedLoaded(bulks ...[]string) *Sealed { +func (s *SealedLoadedFractionTestSuite) newSealedLoaded(bulks ...[]string) *frac.Sealed { sealed := s.newSealed(bulks...) sealed.Release() - sealed = NewSealed( + sealed = frac.NewSealed( sealed.BaseFileName, storage.NewReadLimiter(1, nil), - newIndexCache(), + frac.NewIndexCache(), cache.NewCache[[]byte](nil, nil), nil, s.config, @@ -2332,13 +2346,13 @@ func (s *RemoteFractionTestSuite) SetupTest() { s.Require().NoError(err, "offload failed") s.Require().True(offloaded, "didn't offload frac") - remoteFrac := NewRemote( + remoteFrac := frac.NewRemote( context.Background(), sealed.BaseFileName, storage.NewReadLimiter(1, nil), - newIndexCache(), + frac.NewIndexCache(), cache.NewCache[[]byte](nil, nil), - sealed.info, + sealed.Info(), s.config, s3cli, testSkipMaskProvider{}, @@ -2350,7 +2364,7 @@ func (s *RemoteFractionTestSuite) SetupTest() { } func (s *RemoteFractionTestSuite) TearDownTest() { - if remote, ok := s.fraction.(*Remote); ok { + if remote, ok := s.fraction.(*frac.Remote); ok { remote.Suicide() } else { s.Require().Nil(s.fraction, "fraction is not of Remote type") @@ -2364,6 +2378,113 @@ func (s *RemoteFractionTestSuite) TearDownSuite() { s.s3server.Close() } +type CompactedFractionTestSuite struct { + FractionTestSuite +} + +func (s *CompactedFractionTestSuite) SetupSuite() { + s.SetupSuiteCommon() +} + +func (s *CompactedFractionTestSuite) SetupTest() { + s.SetupTestCommon() + + s.insertDocuments = func(bulks ...[]string) { + if s.fraction != nil { + s.Require().Fail("can insert docs only once") + } + s.fraction = s.newCompacted(bulks...) + } +} + +func (s *CompactedFractionTestSuite) TearDownTest() { + if sealed, ok := s.fraction.(*frac.Sealed); ok { + sealed.Release() + } else { + s.Require().Nil(s.fraction, "fraction is not of Sealed type") + } + s.TearDownTestCommon() +} + +func (s *CompactedFractionTestSuite) TearDownSuite() { + s.TearDownSuiteCommon() +} + +// newCompacted flattens all bulks into one doc list, splits it in half, +// seals each half as a separate fraction, and merges them with compaction.Merge. +func (s *CompactedFractionTestSuite) newCompacted(bulks ...[]string) *frac.Sealed { + // Flatten all documents because we are going to reorganize it. + var docs []string + for _, b := range bulks { + docs = append(docs, b...) + } + + var ( + reorganized [][]string + bulkSize = max(len(docs)/32, 1) + ) + + for i := 0; i < len(docs); i += bulkSize { + reorganized = append( + reorganized, + docs[i:min(i+bulkSize, len(docs))], + ) + } + + merged := s.newSealed(reorganized[0]) + for i, bulk := range reorganized[1:] { + current := s.newSealed(bulk) + + mergedBase := filepath.Join( + s.tmpDir, + fmt.Sprintf("merged-%d", i), + ) + + preloaded, err := compaction.Merge( + mergedBase, s.sealParams, + frac.NewSealedSource(merged), + frac.NewSealedSource(current), + ) + + s.Require().NoError(err) + merged = frac.NewSealedPreloaded( + mergedBase, + preloaded, + storage.NewReadLimiter(1, nil), + frac.NewIndexCache(), + cache.NewCache[[]byte](nil, nil), + s.config, + testSkipMaskProvider{}, + ) + } + + return merged +} + +// TestFractionInfo overrides the base test because DocsOnDisk is larger in a +// merged fraction (sum of two source docs files) and MIDsDistribution is not +// populated by compaction.Merge. +func (s *CompactedFractionTestSuite) TestFractionInfo() { + docs := []string{ + `{"timestamp":"2000-01-01T13:00:25Z","service":"service_a","message":"first message some text", "container":"gateway"}`, + `{"timestamp":"2000-01-01T13:00:32Z","service":"service_b","message":"second message other text", "container":"kube-proxy"}`, + `{"timestamp":"2000-01-01T13:00:43Z","service":"service_c","message":"third message other text", "container":"gateway"}`, + `{"timestamp":"2000-01-01T13:00:53Z","service":"service_a","message":"fourth message some text", "container":"kube-proxy"}`, + `{"timestamp":"2000-01-01T13:00:54Z","service":"service_c","message":"apple","container":"kube-scheduler"}`, + } + + s.insertDocuments(docs) + + info := s.fraction.Info() + + s.Require().Equal(uint32(5), info.DocsTotal, "doc total doesn't match") + s.Require().Equal(uint64(583), info.DocsRaw, "doc raw doesn't match") + s.Require().Equal(seq.MID(946731625000000000), info.From, "from doesn't match") + s.Require().Equal(seq.MID(946731654000000000), info.To, "to doesn't match") + s.Require().Equal(uint64(0), info.MetaOnDisk, "meta on disk doesn't match") + s.Require().True(info.IndexOnDisk > 0, "index on disk should be non-zero") +} + func TestActiveFractionTestSuite(t *testing.T) { suite.Run(t, new(ActiveFractionTestSuite)) } @@ -2383,3 +2504,7 @@ func TestSealedLoadedFractionTestSuite(t *testing.T) { func TestRemoteFractionTestSuite(t *testing.T) { suite.Run(t, new(RemoteFractionTestSuite)) } + +func TestCompactedFractionTestSuite(t *testing.T) { + suite.Run(t, new(CompactedFractionTestSuite)) +} diff --git a/frac/index_cache.go b/frac/index_cache.go index 043e8c5c..f270f209 100644 --- a/frac/index_cache.go +++ b/frac/index_cache.go @@ -7,7 +7,7 @@ import ( "github.com/ozontech/seq-db/frac/sealed/token" ) -func newIndexCache() *IndexCache { +func NewIndexCache() *IndexCache { return &IndexCache{ LegacyRegistry: cache.NewCache[[]byte](nil, nil), diff --git a/frac/sealed.go b/frac/sealed.go index c18f9a62..e5f8a555 100644 --- a/frac/sealed.go +++ b/frac/sealed.go @@ -392,6 +392,7 @@ func (f *Sealed) Release() { func (f *Sealed) Suicide() { f.Release() + // Rename docs atomically first — this commits the intent to delete. oldPath := f.BaseFileName + consts.DocsFileSuffix newPath := f.BaseFileName + consts.DocsDelFileSuffix diff --git a/frac/sealed_source.go b/frac/sealed_source.go new file mode 100644 index 00000000..3cafcccb --- /dev/null +++ b/frac/sealed_source.go @@ -0,0 +1,159 @@ +package frac + +import ( + "iter" + "slices" + + "github.com/ozontech/seq-db/frac/common" + "github.com/ozontech/seq-db/frac/sealed/lids" + "github.com/ozontech/seq-db/frac/sealed/seqids" + "github.com/ozontech/seq-db/frac/sealed/token" + "github.com/ozontech/seq-db/indexwriter" + "github.com/ozontech/seq-db/seq" + "github.com/ozontech/seq-db/storage" + "github.com/ozontech/seq-db/util" +) + +type DocBlockLocation = util.Pair[[]byte, uint64] + +// SealedSource implements [indexwriter.Source] for a sealed fraction. +// Used as input to [compaction.MergeSource] when compacting multiple fractions. +type SealedSource struct { + f *Sealed + + idsProvider *seqids.Provider + lidsLoader *lids.Loader + + tokenBlockLoader *token.BlockLoader + tokenTableLoader *token.TableLoader +} + +func NewSealedSource(f *Sealed) *SealedSource { + f.init(true) + return &SealedSource{ + f: f, + idsProvider: seqids.NewProvider( + &f.idReader, + f.indexCache.MIDs, + f.indexCache.RIDs, + f.indexCache.Params, + &f.blocksData.IDsTable, + f.info.BinaryDataVer, + ), + lidsLoader: lids.NewLoader(&f.lidReader, f.indexCache.LIDs), + tokenBlockLoader: token.NewBlockLoader(f.BaseFileName, &f.tokenReader, f.indexCache.Tokens), + tokenTableLoader: token.NewTableLoader(f.BaseFileName, f.IsLegacy, &f.tokenReader, f.indexCache.TokenTable), + } +} + +func (s *SealedSource) Info() *common.Info { + return s.f.info +} + +func (s *SealedSource) BlockOffsets() []uint64 { + return s.f.blocksData.BlocksOffsets +} + +func (s *SealedSource) ID() iter.Seq2[indexwriter.DocLocation, error] { + return func(yield func(indexwriter.DocLocation, error) bool) { + for lid := uint32(0); lid < s.f.blocksData.IDsTable.IDsTotal; lid++ { + mid, err := s.idsProvider.MID(seq.LID(lid)) + if err != nil { + yield(indexwriter.DocLocation{}, err) + return + } + + rid, err := s.idsProvider.RID(seq.LID(lid)) + if err != nil { + yield(indexwriter.DocLocation{}, err) + return + } + + pos, err := s.idsProvider.DocPos(seq.LID(lid)) + if err != nil { + yield(indexwriter.DocLocation{}, err) + return + } + + if !yield(indexwriter.DocLocation{First: seq.ID{MID: mid, RID: rid}, Second: pos}, nil) { + return + } + } + } +} + +func (s *SealedSource) TokenTriplet() iter.Seq2[string, iter.Seq2[indexwriter.TokenPosting, error]] { + tokenTable := s.tokenTableLoader.Load() + + fields := make([]string, 0, len(tokenTable)) + for field := range tokenTable { + fields = append(fields, field) + } + + slices.Sort(fields) + return func(yield func(string, iter.Seq2[indexwriter.TokenPosting, error]) bool) { + for _, field := range fields { + if !yield(field, s.postingsForField(field)) { + return + } + } + } +} + +func (s *SealedSource) postingsForField(field string) iter.Seq2[indexwriter.TokenPosting, error] { + lidsTable := s.f.blocksData.LIDsTable + tokenTable := s.tokenTableLoader.Load() + + var lidsBuf []uint32 + return func(yield func(indexwriter.TokenPosting, error) bool) { + for _, entry := range tokenTable[field].Entries { + block := s.tokenBlockLoader.Load(entry.BlockIndex) + + for tid := entry.StartTID; tid < entry.StartTID+entry.ValCount; tid++ { + lidsBuf = lidsBuf[:0] + + tokenVal := block.GetToken(entry.GetIndexInTokensBlock(tid)) + firstBlock := lidsTable.GetFirstBlockIndexForTID(tid) + lastBlock := lidsTable.GetLastBlockIndexForTID(tid) + + for bi := firstBlock; bi <= lastBlock; bi++ { + lidBlock, err := s.lidsLoader.GetLIDsBlock(bi) + if err != nil { + yield(indexwriter.TokenPosting{}, err) + return + } + + chunkIdx := lidsTable.GetChunkIndex(bi, tid) + lidsBuf = append(lidsBuf, lidBlock.LIDs[lidBlock.Offsets[chunkIdx]:lidBlock.Offsets[chunkIdx+1]]...) + } + + if !yield(indexwriter.TokenPosting{First: tokenVal, Second: lidsBuf}, nil) { + return + } + } + } + } +} + +func (s *SealedSource) DocBlock() iter.Seq2[DocBlockLocation, error] { + return func(yield func(DocBlockLocation, error) bool) { + // We do not want to cache payload of DocBlock because + // it will just pollute cache and cause unnecessary evictions. + r := storage.NewDocBlocksReader(s.f.readLimiter, s.f.docsFile) + + for _, offset := range s.f.blocksData.BlocksOffsets { + // Read DocBlock payload (including its header) but do not decompress it. + // Caller of [SealedSource.DocBlock] will decide whether it requires decompressed data. + payload, _, err := r.ReadDocBlock(int64(offset)) + if err != nil { + yield(DocBlockLocation{}, err) + return + } + + loc := DocBlockLocation{First: payload, Second: offset} + if !yield(loc, nil) { + return + } + } + } +} diff --git a/fracmanager/fracmanager.go b/fracmanager/fracmanager.go index 6b5b7c87..9ac3c642 100644 --- a/fracmanager/fracmanager.go +++ b/fracmanager/fracmanager.go @@ -11,6 +11,7 @@ import ( "github.com/ozontech/seq-db/config" "github.com/ozontech/seq-db/consts" "github.com/ozontech/seq-db/frac" + "github.com/ozontech/seq-db/frac/sealed" "github.com/ozontech/seq-db/logger" "github.com/ozontech/seq-db/storage" "github.com/ozontech/seq-db/storage/s3" @@ -76,18 +77,19 @@ func New(ctx context.Context, cfg *Config, s3cli *s3.Client, skipMaskProvider sk cancel() wg.Wait() - // freeze active fraction to prevent new writes - active := lc.registry.Active() - if err := active.Finalize(); err != nil { + // finalize appender to prevent new writes + appender := lc.registry.appender() + if err := appender.finalize(); err != nil { logger.Fatal("shutdown fraction freezing error", zap.Error(err)) } - active.WaitWriteIdle() + appender.waitWriteIdle() stopIdx() lc.SyncInfoCache() - sealOnShutdown(active.instance, provider, cfg.MinSealFracSize) + // Seal active fraction + sealOnShutdown(appender.Active, provider, cfg.MinSealFracSize) logger.Info("fracmanager's workers are stopped", zap.Int64("took_ms", time.Since(n).Milliseconds())) } @@ -95,16 +97,59 @@ func New(ctx context.Context, cfg *Config, s3cli *s3.Client, skipMaskProvider sk return &fm, stop, nil } +type CompactionSnapshot struct { + claimed []*refCountedSealed +} + +func (cs *CompactionSnapshot) Fractions() []*frac.Sealed { + result := make([]*frac.Sealed, len(cs.claimed)) + for i, f := range cs.claimed { + result[i] = f.Sealed + } + return result +} + +func (cs *CompactionSnapshot) Destroy() { + for _, f := range cs.claimed { + f.Destroy() + } +} + +func (fm *FracManager) FractionName() string { + filePath := fileBasePattern + fm.lc.provider.nextFractionID() + baseFilePath := filepath.Join(fm.lc.provider.config.DataDir, filePath) + return baseFilePath +} + +func (fm *FracManager) SealedFractionsSnapshot() []*frac.Sealed { + return fm.lc.registry.sealedSnapshot() +} + +func (fm *FracManager) ClaimForCompaction(names []string) (*CompactionSnapshot, error) { + claimed, err := fm.lc.registry.claimForCompaction(names) + if err != nil { + return nil, err + } + return &CompactionSnapshot{claimed: claimed}, nil +} + +func (fm *FracManager) SubstituteWithSealed(produced *sealed.PreloadedData, snapshot *CompactionSnapshot) { + fm.lc.registry.substituteWithSealed( + fm.lc.provider.NewSealedPreloaded(produced.Info.Path, produced), + snapshot.claimed..., + ) +} + func (fm *FracManager) AcquireFraction(name string) (frac.Fraction, func(), bool) { - return fm.lc.registry.AcquireFraction(name) + return fm.lc.registry.acquireOneFraction(name) } -func (fm *FracManager) Fractions() List { - return fm.lc.registry.AllFractions() +func (fm *FracManager) AcquireFractions() (List, func()) { + return fm.lc.registry.acquireAllFractions() } func (fm *FracManager) Oldest() uint64 { - return fm.lc.registry.OldestTotal() + return fm.lc.registry.oldestTotal() } func (fm *FracManager) Flags() *StateManager { @@ -120,7 +165,7 @@ func (fm *FracManager) Append(ctx context.Context, docs storage.DocBlock, metas return ctx.Err() default: // Try to append data to the currently active fraction - err := fm.lc.registry.Active().Append(docs, metas) + err := fm.lc.registry.appender().append(docs, metas) if err != nil { logger.Info("append fail", zap.Error(err)) if err == ErrFractionNotWritable { @@ -166,7 +211,7 @@ func startStatsWorker(ctx context.Context, reg *fractionRegistry, wg *sync.WaitG logger.Info("stats loop is started") // Run stats collection every 10 seconds util.RunEvery(ctx.Done(), time.Second*10, func() { - stats := reg.Stats() + stats := reg.statistics() stats.Log() // Log statistics stats.SetMetrics() // Update Prometheus metrics }) diff --git a/fracmanager/fracmanager_for_tests.go b/fracmanager/fracmanager_for_tests.go index ab7cd851..39349289 100644 --- a/fracmanager/fracmanager_for_tests.go +++ b/fracmanager/fracmanager_for_tests.go @@ -3,7 +3,7 @@ package fracmanager import "sync" func (fm *FracManager) WaitIdleForTests() { - fm.lc.registry.Active().WaitWriteIdle() + fm.lc.registry.appender().waitWriteIdle() } func (fm *FracManager) SealForcedForTests() { diff --git a/fracmanager/fracmanager_test.go b/fracmanager/fracmanager_test.go index 663dedec..64c264b6 100644 --- a/fracmanager/fracmanager_test.go +++ b/fracmanager/fracmanager_test.go @@ -61,30 +61,32 @@ func TestSealingOnShutdown(t *testing.T) { cfg.MinSealFracSize = 0 // to ensure that the frac will not be sealed on shutdown cfg, fm, stop := setupFracManager(t, cfg) appendDocsToFracManager(t, fm, 10) - activeName := fm.Fractions()[0].Info().Name() + + activeName := fm.lc.registry.snapshot.fractions[0].Info().Name() + stop() // second start cfg.MinSealFracSize = 1 // to ensure that the frac will be sealed on shutdown cfg, fm, stop = setupFracManager(t, cfg) - assert.Equal(t, 1, len(fm.Fractions()), "should have one fraction") - assert.Equal(t, activeName, fm.Fractions()[0].Info().Name(), "fraction should have the same name") - _, ok := fm.Fractions()[0].(*fractionProxy).impl.(*frac.Active) + allFractions := fm.lc.registry.snapshot.fractions + assert.Equal(t, 1, len(allFractions), "should have one fraction") + assert.Equal(t, activeName, allFractions[0].Info().Name(), "fraction should have the same name") + _, ok := allFractions[0].(*syncAppender) assert.True(t, ok, "fraction should be active") - stop() // third start _, fm, stop = setupFracManager(t, cfg) - assert.Equal(t, 2, len(fm.Fractions()), "should have 2 fraction: new active and old sealed") - _, ok = fm.Fractions()[0].(*fractionProxy).impl.(*frac.Sealed) + allFractions = fm.lc.registry.snapshot.fractions + assert.Equal(t, 2, len(allFractions), "should have 2 fraction: new active and old sealed") + _, ok = allFractions[0].(*refCountedSealed) assert.True(t, ok, "first fraction should be sealed") - assert.Equal(t, activeName, fm.Fractions()[0].Info().Name(), "sealed fraction should have the same name") - assert.Equal(t, uint32(0), fm.Fractions()[1].Info().DocsTotal, "active fraction should be empty") - _, ok = fm.Fractions()[1].(*fractionProxy).impl.(*frac.Active) + assert.Equal(t, activeName, allFractions[0].Info().Name(), "sealed fraction should have the same name") + assert.Equal(t, uint32(0), allFractions[1].Info().DocsTotal, "active fraction should be empty") + _, ok = allFractions[1].(*syncAppender) assert.True(t, ok, "new fraction should be active") - stop() } diff --git a/fracmanager/fracs_stats.go b/fracmanager/fracs_stats.go index 968b8b41..ee255543 100644 --- a/fracmanager/fracs_stats.go +++ b/fracmanager/fracs_stats.go @@ -76,6 +76,7 @@ type registryStats struct { active fracsStats // Statistics for active fraction sealing fracsStats // Statistics for fractions in the sealing process sealed fracsStats // Statistics for fractions on sealed disk + compacting fracsStats // Statistics for fractions participating in compaction offloading fracsStats // Statistics for fractions in the offloading process remotes fracsStats // Statistics for fractions in remote storage } @@ -84,6 +85,7 @@ func (s *registryStats) Log() { s.active.Log("active") s.sealing.Log("sealing") s.sealed.Log("sealed") + s.compacting.Log("compacting") s.offloading.Log("offloading") s.remotes.Log("remotes") } @@ -92,6 +94,11 @@ func (s *registryStats) SetMetrics() { s.active.SetMetrics(dataSizeTotal, "active") s.sealing.SetMetrics(dataSizeTotal, "sealing") s.sealed.SetMetrics(dataSizeTotal, "sealed") + s.compacting.SetMetrics(dataSizeTotal, "compacting") s.offloading.SetMetrics(dataSizeTotal, "offloading") s.remotes.SetMetrics(dataSizeTotal, "remotes") } + +func (s registryStats) TotalSizeOnDiskLocal() uint64 { + return s.sealing.totalSizeOnDisk + s.sealed.totalSizeOnDisk + s.compacting.totalSizeOnDisk +} diff --git a/fracmanager/fraction_provider.go b/fracmanager/fraction_provider.go index 66e6477b..556eb2f1 100644 --- a/fracmanager/fraction_provider.go +++ b/fracmanager/fraction_provider.go @@ -5,17 +5,21 @@ import ( "io" "math/rand" "path/filepath" + "sync" "time" "github.com/oklog/ulid/v2" + "go.uber.org/zap" "github.com/ozontech/seq-db/frac" "github.com/ozontech/seq-db/frac/common" "github.com/ozontech/seq-db/frac/sealed" - "github.com/ozontech/seq-db/frac/sealed/sealing" + "github.com/ozontech/seq-db/logger" "github.com/ozontech/seq-db/node" + "github.com/ozontech/seq-db/sealing" "github.com/ozontech/seq-db/storage" "github.com/ozontech/seq-db/storage/s3" + "github.com/ozontech/seq-db/util" ) const fileBasePattern = "seq-db-" @@ -34,8 +38,10 @@ type fractionProvider struct { cacheProvider *CacheMaintainer // Cache provider for data access optimization activeIndexer *frac.ActiveIndexer // Indexer for active fractions readLimiter *storage.ReadLimiter // Read rate limiter - ulidEntropy io.Reader // Entropy source for ULID generation skipMaskProvider skipMaskProvider + + mu sync.Mutex + ulidEntropy io.Reader // Entropy source for ULID generation } func newFractionProvider( @@ -110,6 +116,8 @@ func (fp *fractionProvider) NewRemote(ctx context.Context, name string, cachedIn // IMPORTANT: This method is not thread-safe. When used in concurrent environments, // external synchronization must be provided to avoid ID collisions func (fp *fractionProvider) nextFractionID() string { + fp.mu.Lock() + defer fp.mu.Unlock() return ulid.MustNew(ulid.Timestamp(time.Now()), fp.ulidEntropy).String() } @@ -123,8 +131,11 @@ func (fp *fractionProvider) CreateActive() *frac.Active { // Seal converts an active fraction to a sealed one // Process includes sorting, indexing, and data optimization for reading -func (fp *fractionProvider) Seal(active *frac.Active) (*frac.Sealed, error) { - src, err := frac.NewActiveSealingSource(active, fp.config.SealParams) +func (fp *fractionProvider) Seal(a *frac.Active) (*frac.Sealed, error) { + sealsTotal.Inc() + now := time.Now() + + src, err := frac.NewActiveSealingSource(a, fp.config.SealParams) if err != nil { return nil, err } @@ -133,9 +144,19 @@ func (fp *fractionProvider) Seal(active *frac.Active) (*frac.Sealed, error) { return nil, err } - sealedFrac := fp.NewSealedPreloaded(active.BaseFileName, preloaded) - fp.skipMaskProvider.RefreshFrac(sealedFrac) - return sealedFrac, nil + s := fp.NewSealedPreloaded(a.BaseFileName, preloaded) + fp.skipMaskProvider.RefreshFrac(s) + + sealingTime := time.Since(now) + sealsDoneSeconds.Observe(sealingTime.Seconds()) + + logger.Info( + "fraction sealed", + zap.String("fraction", filepath.Base(s.BaseFileName)), + zap.Float64("time_spent_s", util.DurationToUnit(sealingTime, "s")), + ) + + return s, nil } // Offload uploads fraction to S3 storage and returns a remote fraction diff --git a/fracmanager/fraction_registry.go b/fracmanager/fraction_registry.go index 0f79c28d..be77be20 100644 --- a/fracmanager/fraction_registry.go +++ b/fracmanager/fraction_registry.go @@ -15,138 +15,157 @@ import ( // fractionRegistry manages fraction queues at different lifecycle stages. // Tracks fractions through different stages: active → sealing → sealed → offloading → remote -// Ensures correct state transitions while maintaining chronological order. // The entire structure is thread-safe due to internal synchronization. -// Lifecycle: created once, persists through application lifetime. type fractionRegistry struct { mu sync.RWMutex // main mutex for protecting registry state - // lifecycle queues (FIFO order, oldest at lower indexes) - sealing []*activeProxy // fractions being sealed (0-5 typical) - sealed []*sealedProxy // local sealed fractions (can be thousands) - offloading []*sealedProxy // fractions being offloaded (0-5 typical) - remotes []*remoteProxy // offloaded fractions (can be thousands) + sealing map[string]*syncAppender // fractions being sealed (0-5 typical) + sealed PartitionedCollection[*refCountedSealed] // local sealed fractions (can be thousands) + compacting map[string]*refCountedSealed // fractions participating in compaction + offloading PartitionedCollection[*refCountedSealed] // fractions being offloaded (0-5 typical) + remotes PartitionedCollection[*refCountedRemote] // offloaded fractions (can be thousands) - stats registryStats // size statistics for monitoring - oldestTotal uint64 // creation time of oldest fraction in all list including remote - oldestLocal uint64 // creation time of oldest fraction in local or offloading queues + stats registryStats // size statistics for monitoring - muAll sync.RWMutex // protects active, all, and oldestTotal fields - active *activeProxy // currently active writable fraction - all []frac.Fraction // all fractions in creation order (read-only view) - allMap map[string]frac.Fraction + muAppender sync.RWMutex + sappender *syncAppender // currently active writable fraction + + muSnapshot sync.RWMutex + snapshot fractionsSnapshot // all fractions } // NewFractionRegistry creates and initializes a new fraction registry instance. // Populates the registry with existing active, sealed and remote fractions. -// Rebuilds the complete fractions list in chronological order. func NewFractionRegistry(active *frac.Active, sealed []*frac.Sealed, remotes []*frac.Remote) (*fractionRegistry, error) { if active == nil { return nil, errors.New("active fraction must be specified") } - r := fractionRegistry{ - active: &activeProxy{ - proxy: &fractionProxy{impl: active}, - instance: active, - }, + creationTime := func(f frac.Fraction) uint64 { return f.Info().CreationTime } + + lastDocTime := func(f frac.Fraction) uint64 { + aligned := f.Info().To.Time(). + Add(-time.Nanosecond). + Truncate(time.Minute). + Add(time.Minute) + return uint64(aligned.UnixMilli()) + } + + reg := fractionRegistry{ + sappender: &syncAppender{refCountedActive: refCountedActive{Active: active}}, + + sealing: map[string]*syncAppender{}, + sealed: NewPartitionedCollection(func(rcs *refCountedSealed) uint64 { return creationTime(rcs) }), + compacting: map[string]*refCountedSealed{}, + offloading: NewPartitionedCollection(func(rcs *refCountedSealed) uint64 { return lastDocTime(rcs) }), + remotes: NewPartitionedCollection(func(rcr *refCountedRemote) uint64 { return lastDocTime(rcr) }), } // initialize local sealed fractions - for _, sealed := range sealed { - r.stats.sealed.Add(sealed.Info()) - r.sealed = append(r.sealed, &sealedProxy{ - proxy: &fractionProxy{impl: sealed}, - instance: sealed, - }) + for _, s := range sealed { + reg.stats.sealed.Add(s.Info()) + reg.sealed.Add(s.Info().Name(), &refCountedSealed{Sealed: s}) } // initialize remote fractions - for _, remote := range remotes { - r.stats.remotes.Add(remote.Info()) - r.remotes = append(r.remotes, &remoteProxy{ - proxy: &fractionProxy{impl: remote}, - instance: remote, - }) + for _, r := range remotes { + reg.stats.remotes.Add(r.Info()) + reg.remotes.Add(r.Info().Name(), &refCountedRemote{Remote: r}) } - r.updateOldestLocal() - r.rebuildAllFractions() + reg.rebuildSnapshot() - return &r, nil + return ®, nil } -// Active returns the currently active writable fraction. -func (r *fractionRegistry) Active() *activeProxy { - r.muAll.RLock() - defer r.muAll.RUnlock() - return r.active +// appender returns the currently active writable fraction. +func (r *fractionRegistry) appender() *syncAppender { + r.muAppender.RLock() + defer r.muAppender.RUnlock() + return r.sappender } -func (r *fractionRegistry) AcquireFraction(name string) (frac.Fraction, func(), bool) { - r.muAll.RLock() - defer r.muAll.RUnlock() +func (r *fractionRegistry) acquireOneFraction(name string) (frac.Fraction, func(), bool) { + r.muSnapshot.RLock() + defer r.muSnapshot.RUnlock() - f, ok := r.allMap[name] - return f, func() {}, ok + return r.snapshot.AcquireOne(name) } -// AllFractions returns a read-only view of all fractions in creation order. -func (r *fractionRegistry) AllFractions() []frac.Fraction { - r.muAll.RLock() - defer r.muAll.RUnlock() - return r.all +// acquireAllFractions returns a read-only view of all fractions +func (r *fractionRegistry) acquireAllFractions() ([]frac.Fraction, func()) { + r.muSnapshot.RLock() + defer r.muSnapshot.RUnlock() + + return r.snapshot.AcquireAll() } -// Stats returns current size statistics of the registry. -func (r *fractionRegistry) Stats() registryStats { +// statistics returns current size statistics of the registry. +func (r *fractionRegistry) statistics() registryStats { r.mu.RLock() s := r.stats - i := r.active.instance.Info() + i := r.sappender.Info() r.mu.RUnlock() s.active.Set(i) return s } -// OldestTotal returns the creation time of the oldest fraction in the registry. -func (r *fractionRegistry) OldestTotal() uint64 { - r.muAll.RLock() - defer r.muAll.RUnlock() - return r.oldestTotal +// oldestTotal returns the creation time of the oldest fraction in the registry. +func (r *fractionRegistry) oldestTotal() uint64 { + r.muSnapshot.RLock() + defer r.muSnapshot.RUnlock() + return r.snapshot.oldestTotal } -// OldestLocal returns the creation time of the oldest local fraction in the registry. -func (r *fractionRegistry) OldestLocal() uint64 { - r.mu.RLock() - defer r.mu.RUnlock() - return r.oldestLocal +// oldestLocal returns the creation time of the oldest local fraction in the registry. +func (r *fractionRegistry) oldestLocal() uint64 { + r.muSnapshot.RLock() + defer r.muSnapshot.RUnlock() + return r.snapshot.oldestLocal } -// RotateIfFull completes the current active fraction and starts a new one. +type activeProvider interface { + CreateActive() *frac.Active +} + +func (r *fractionRegistry) setAppender(appender *syncAppender) { + r.muAppender.Lock() + defer r.muAppender.Unlock() + + r.sappender = appender + + r.muSnapshot.Lock() + defer r.muSnapshot.Unlock() + + r.snapshot.AddActive(appender) +} + +// rotateIfFull completes the current active fraction and starts a new one. // Moves previous active fraction to sealing queue. -// Updates statistics and maintains chronological order. -// Should be called when creating a new fraction. -func (r *fractionRegistry) RotateIfFull(maxSize uint64, newActive func() *activeProxy) (*activeProxy, func(), error) { +// Should be called when the current active fraction reaches size limit and needs to be rotated +func (r *fractionRegistry) rotateIfFull(maxSize uint64, ap activeProvider) (*refCountedActive, func(), error) { r.mu.Lock() defer r.mu.Unlock() - if r.active.instance.Info().DocsOnDisk <= maxSize { + if uint64(r.sappender.MemSize()) <= maxSize { return nil, nil, nil } - old := r.active - r.sealing = append(r.sealing, old) - r.addActive(newActive()) + old := r.sappender + + r.sealing[old.Info().Name()] = old - if err := old.Finalize(); err != nil { - return old, nil, err + r.setAppender(&syncAppender{refCountedActive: refCountedActive{Active: ap.CreateActive()}}) + + if err := old.finalize(); err != nil { + return nil, nil, err } - curInfo := old.instance.Info() + curInfo := old.Info() r.stats.sealing.Add(curInfo) - r.active.Suspend(old.Suspended()) + r.sappender.suspend(old.isSuspended()) wg := sync.WaitGroup{} wg.Add(1) @@ -155,8 +174,8 @@ func (r *fractionRegistry) RotateIfFull(maxSize uint64, newActive func() *active go func() { defer wg.Done() - old.WaitWriteIdle() // can be long enough - finalInfo := old.instance.Info() + old.waitWriteIdle() // can be long enough + finalInfo := old.Info() r.mu.Lock() defer r.mu.Unlock() @@ -167,14 +186,14 @@ func (r *fractionRegistry) RotateIfFull(maxSize uint64, newActive func() *active r.stats.sealing.Add(finalInfo) }() - return old, wg.Wait, nil + return &old.refCountedActive, wg.Wait, nil } -func (r *fractionRegistry) SuspendIfOverCapacity(maxQueue, maxSize uint64) { +func (r *fractionRegistry) suspendIfOverCapacity(maxQueue, maxSize uint64) { r.mu.Lock() defer r.mu.Unlock() - suspended := r.active.Suspended() + suspended := r.sappender.isSuspended() if maxQueue > 0 && r.stats.sealing.count >= int(maxQueue) { if !suspended { @@ -182,7 +201,7 @@ func (r *fractionRegistry) SuspendIfOverCapacity(maxQueue, maxSize uint64) { zap.String("reason", "sealing queue size exceeded"), zap.Uint64("limit", maxQueue), zap.Int("queue_size", r.stats.sealing.count)) - r.active.Suspend(true) + r.sappender.suspend(true) } return } @@ -195,7 +214,7 @@ func (r *fractionRegistry) SuspendIfOverCapacity(maxQueue, maxSize uint64) { zap.String("reason", "occupied space limit exceeded"), zap.Float64("queue_size_limit_gb", util.Float64ToPrec(util.SizeToUnit(maxSize, "gb"), 2)), zap.Float64("occupied_space_gb", util.Float64ToPrec(util.SizeToUnit(du, "gb"), 2))) - r.active.Suspend(true) + r.sappender.suspend(true) } return } @@ -206,66 +225,69 @@ func (r *fractionRegistry) SuspendIfOverCapacity(maxQueue, maxSize uint64) { zap.Float64("occupied_space_gb", util.Float64ToPrec(util.SizeToUnit(du, "gb"), 2)), zap.Uint64("sealing_queue_size_limit", maxQueue), zap.Int("queue_size", r.stats.sealing.count)) - r.active.Suspend(false) + r.sappender.suspend(false) } } func (r *fractionRegistry) diskUsage() uint64 { - return r.active.instance.Info().FullSize() + + return r.sappender.Info().FullSize() + r.stats.sealed.totalSizeOnDisk + r.stats.sealing.totalSizeOnDisk + + r.stats.compacting.totalSizeOnDisk + r.stats.offloading.totalSizeOnDisk } -// addActive sets a new active fraction and updates the complete fractions list. -func (r *fractionRegistry) addActive(a *activeProxy) { - r.muAll.Lock() - defer r.muAll.Unlock() +// evictLocalForDelete removes oldest local fractions to free disk space. +// Returns evicted fractions or error if insufficient space is released. +func (r *fractionRegistry) evictLocalForDelete(sizeLimit uint64) (evicted []*refCountedSealed, err error) { + r.mu.Lock() + defer r.mu.Unlock() - r.active = a - r.all = append(r.all, a.proxy) - r.allMap[a.instance.Info().Name()] = a.proxy -} + if evicted, err = r.evictLocal(sizeLimit); err != nil { + return nil, err + } -// trimAll removes the oldest fractions from the complete fractions list. -// Used when fractions are evicted or deleted from the system. -func (r *fractionRegistry) trimAll(count int) { - r.muAll.Lock() - defer r.muAll.Unlock() + r.rebuildSnapshot() - for _, f := range r.all[:count] { - delete(r.allMap, f.Info().Name()) - } - r.all = r.all[count:] - r.updateOldestTotal() + return evicted, nil } -// EvictLocal removes oldest local fractions to free disk space. -// If shouldOffload is true, moves fractions to offloading queue instead of deleting. +// evictLocalForOffload removes oldest local fractions to moves it to offloading queue. // Returns evicted fractions or error if insufficient space is released. -func (r *fractionRegistry) EvictLocal(shouldOffload bool, sizeLimit uint64) ([]*sealedProxy, error) { +func (r *fractionRegistry) evictLocalForOffload(sizeLimit uint64) ([]*refCountedSealed, error) { r.mu.Lock() defer r.mu.Unlock() - var ( - count int - releasingSize uint64 - ) + evicted, err := r.evictLocal(sizeLimit) + if err != nil { + return nil, err + } + + for _, sealed := range evicted { + r.offloading.Add(sealed.Info().Name(), sealed) + r.stats.offloading.Add(sealed.Info()) + } + + return evicted, nil +} + +func (r *fractionRegistry) evictLocal(sizeLimit uint64) ([]*refCountedSealed, error) { + var releasingSize uint64 // calculate total used disk space - totalUsedSize := r.stats.sealed.totalSizeOnDisk + - r.stats.sealing.totalSizeOnDisk + - r.active.instance.Info().FullSize() + totalUsedSize := r.stats.TotalSizeOnDiskLocal() + r.sappender.Info().FullSize() + + var evicted []*refCountedSealed + for r.sealed.Len() > 0 && totalUsedSize-releasingSize > sizeLimit { + for _, s := range r.sealed.GetByPartition(r.sealed.MinPartition()) { + info := s.Info() + releasingSize += info.FullSize() - // determine how many oldest fractions need to be removed to meet size limit - for _, item := range r.sealed { - if totalUsedSize-releasingSize <= sizeLimit { - break + r.stats.sealed.Sub(info) + r.sealed.Del(info.Name()) + + evicted = append(evicted, s) } - info := item.instance.Info() - releasingSize += info.FullSize() - r.stats.sealed.Sub(info) - count++ } // check if enough space will be freed @@ -275,28 +297,13 @@ func (r *fractionRegistry) EvictLocal(shouldOffload bool, sizeLimit uint64) ([]* (totalUsedSize-releasingSize)-sizeLimit, totalUsedSize, releasingSize, sizeLimit) } - // extract fractions to evict - evicted := r.sealed[:count] - r.sealed = r.sealed[count:] - - // either offload or completely remove the fractions - if shouldOffload { - for _, item := range evicted { - r.offloading = append(r.offloading, item) - r.stats.offloading.Add(item.instance.Info()) - } - } else { - r.trimAll(count) // permanently remove - r.updateOldestLocal() // oldest local can be changed here - } - return evicted, nil } -// EvictRemote removes oldest remote fractions based on retention policy. +// evictRemote removes oldest remote fractions based on retention policy. // Fractions older than retention period are permanently deleted. // Returns removed fractions or empty slice if nothing to remove. -func (r *fractionRegistry) EvictRemote(retention time.Duration) []*remoteProxy { +func (r *fractionRegistry) evictRemote(retention time.Duration) []*refCountedRemote { if retention == 0 { return nil } @@ -304,28 +311,24 @@ func (r *fractionRegistry) EvictRemote(retention time.Duration) []*remoteProxy { r.mu.Lock() defer r.mu.Unlock() - count := 0 - // find fractions older than retention period - for _, item := range r.remotes { - info := item.instance.Info() - if time.Since(time.UnixMilli(int64(info.CreationTime))) <= retention { - break // stop at first fraction within retention + evicted := []*refCountedRemote{} + for r.remotes.Len() > 0 && time.Since(time.UnixMilli(int64(r.remotes.MinPartition()))) > retention { + for _, remote := range r.remotes.GetByPartition(r.remotes.MinPartition()) { + info := remote.Info() + r.stats.remotes.Sub(info) + evicted = append(evicted, remote) + r.remotes.Del(info.Name()) } - r.stats.remotes.Sub(info) - count++ } - evicted := r.remotes[:count] - r.remotes = r.remotes[count:] - r.trimAll(count) // remove from complete list + r.rebuildSnapshot() return evicted } -// EvictOverflowed removes oldest fractions from offloading queue when it exceeds size limit. -// Selects fractions that haven't finished offloading yet to minimize data loss. +// evictOverflowed removes oldest fractions from offloading queue when it exceeds size limit. // Used when offloading queue grows too large due to slow remote storage performance. -func (r *fractionRegistry) EvictOverflowed(sizeLimit uint64) []*sealedProxy { +func (r *fractionRegistry) evictOverflowed(sizeLimit uint64) (evicted []*refCountedSealed) { if sizeLimit == 0 { return nil } @@ -338,168 +341,151 @@ func (r *fractionRegistry) EvictOverflowed(sizeLimit uint64) []*sealedProxy { return nil } - count := 0 - evicted := []*sealedProxy{} +loop: // filter fractions - for _, item := range r.offloading { - // keep items that are within limits or already offloaded - if r.stats.offloading.totalSizeOnDisk <= sizeLimit || item.remote != nil { - r.offloading[count] = item - count++ - continue + for r.offloading.Len() > 0 { + for _, s := range r.offloading.GetByPartition(r.offloading.MinPartition()) { + evicted = append(evicted, s) + r.stats.offloading.Sub(s.Info()) + r.offloading.Del(s.Info().Name()) + if r.stats.offloading.totalSizeOnDisk <= sizeLimit { + break loop + } } - evicted = append(evicted, item) - r.stats.offloading.Sub(item.instance.Info()) } - r.offloading = r.offloading[:count] - r.rebuildAllFractions() + r.rebuildSnapshot() return evicted } -// PromoteToSealed moves fractions from sealing to local queue when sealing completes. -// Maintains strict ordering - younger fractions wait for older ones to seal first. -func (r *fractionRegistry) PromoteToSealed(active *activeProxy, sealed *frac.Sealed) { +// promoteToSealed moves fractions from sealing to local queue when sealing completes. +func (r *fractionRegistry) promoteToSealed(active *refCountedActive, sealed ...*frac.Sealed) { r.mu.Lock() defer r.mu.Unlock() - active.sealed = sealed + for _, f := range sealed { + info := f.Info() + r.sealed.Add(info.Name(), &refCountedSealed{Sealed: f}) + r.stats.sealed.Add(info) + } - promotedCount := 0 - // process sealing queue in order, promoting completed fractions - for _, item := range r.sealing { - if item.sealed == nil { - break // maintain order - wait for previous fractions to complete - } - promotedCount++ - r.sealed = append(r.sealed, &sealedProxy{ - proxy: item.proxy, - instance: item.sealed, - }) - r.stats.sealed.Add(item.sealed.Info()) - r.stats.sealing.Sub(item.instance.Info()) + r.stats.sealing.Sub(active.Info()) + delete(r.sealing, active.Info().Name()) + + r.rebuildSnapshot() +} + +func (r *fractionRegistry) substituteWithSealed(produced *frac.Sealed, consumed ...*refCountedSealed) { + r.mu.Lock() + defer r.mu.Unlock() + + for _, f := range consumed { + info := f.Info() + r.stats.compacting.Sub(info) + delete(r.compacting, info.Name()) } - // remove promoted fractions from sealing queue - r.sealing = r.sealing[promotedCount:] + info := produced.Info() + r.stats.sealed.Add(info) + r.sealed.Add(info.Name(), &refCountedSealed{Sealed: produced}) + + r.rebuildSnapshot() } -// PromoteToRemote moves fractions from offloading to remote queue when offloading completes. +// promoteToRemote moves fractions from offloading to remote queue when offloading completes. // Special case: handles fractions that don't require offloading (remote == nil). -// Maintains strict ordering - younger fractions wait for older ones to offload. -func (r *fractionRegistry) PromoteToRemote(sealed *sealedProxy, remote *frac.Remote) { +func (r *fractionRegistry) promoteToRemote(sealed *refCountedSealed, remote *frac.Remote) { r.mu.Lock() defer r.mu.Unlock() - sealed.remote = remote - - // special case: remote == nil means fraction doesn't require offloading - if remote == nil { - r.removeFromOffloading(sealed) + if remote != nil { + r.remotes.Add(remote.Info().Name(), &refCountedRemote{Remote: remote}) + r.stats.remotes.Add(remote.Info()) } - promotedCount := 0 - // process offloading queue in order, promoting completed fractions - for _, item := range r.offloading { - if item.remote == nil { - break // maintain order - wait for previous fractions to complete - } - promotedCount++ - r.remotes = append(r.remotes, &remoteProxy{ - proxy: item.proxy, - instance: item.remote, - }) - - r.stats.remotes.Add(item.remote.Info()) - r.stats.offloading.Sub(item.instance.Info()) - } - if promotedCount > 0 { - // remove promoted fractions from offloading queue - r.offloading = r.offloading[promotedCount:] - r.updateOldestLocal() + r.offloading.Del(sealed.Info().Name()) + r.stats.offloading.Sub(sealed.Info()) + + r.rebuildSnapshot() +} + +func (r *fractionRegistry) sealedSnapshot() []*frac.Sealed { + r.mu.RLock() + defer r.mu.RUnlock() + + result := make([]*frac.Sealed, 0, r.sealed.Len()) + for s := range r.sealed.All() { + result = append(result, s.Sealed) } + + return result } -// removeFromOffloading removes a specific fraction from offloading queue. -// O(n) operation that rebuilds the all fractions list. -func (r *fractionRegistry) removeFromOffloading(sealed *sealedProxy) { - count := 0 - // filter out the target fraction - for _, item := range r.offloading { - if sealed != item { - r.offloading[count] = item - count++ +func (r *fractionRegistry) claimForCompaction(names []string) ([]*refCountedSealed, error) { + r.mu.Lock() + defer r.mu.Unlock() + + for _, name := range names { + // NOTE(dkharms): If offloading pressure is high on the oldest fractions, + // compaction may repeatedly fail to claim them and get into livelock. + if _, ok := r.sealed.Get(name); !ok { + return nil, fmt.Errorf( + "fraction %q is not available for compaction", + name, + ) } } - if count == len(r.offloading) { // not found to remove (can be removed earlier in EvictOverflowed) - return - } + claimed := make([]*refCountedSealed, 0, len(names)) + for _, name := range names { + s, _ := r.sealed.Get(name) - r.offloading = r.offloading[:count] - r.stats.offloading.Sub(sealed.instance.Info()) + r.sealed.Del(name) + r.stats.sealed.Sub(s.Info()) - // oldest local can be changed here - r.updateOldestLocal() + r.compacting[name] = s + r.stats.compacting.Add(s.Info()) + + claimed = append(claimed, s) + } - // rebuild complete list since we modified the middle of the queue - r.rebuildAllFractions() + r.rebuildSnapshot() + return claimed, nil } -// rebuildAllFractions reconstructs the all fractions list in correct chronological order. -// Order: remote (oldest) → offloading → sealed → sealing → active (newest) -// Expensive O(n) operation used when direct list modification is insufficient. -func (r *fractionRegistry) rebuildAllFractions() { - all := make([]frac.Fraction, 0, len(r.all)) - allMap := make(map[string]frac.Fraction, len(r.all)) +// rebuildSnapshot reconstructs the all fractions list +func (r *fractionRegistry) rebuildSnapshot() { + capacity := r.remotes.Len() + r.offloading.Len() + + r.sealed.Len() + len(r.compacting) + len(r.sealing) + 1 - add := func(f frac.Fraction) { - all = append(all, f) - allMap[f.Info().Name()] = f - } + // allocate extra capacity to accommodate appender rotation that may occur during snapshot lifetime + all := newFractionsSnapshot(capacity + 1) - // collect fractions in correct chronological order: from oldest (remote) to newest (active) - for _, remote := range r.remotes { - add(remote.proxy) - } - for _, offloaded := range r.offloading { - add(offloaded.proxy) + for r := range r.remotes.All() { + all.AddRemote(r) } - for _, sealed := range r.sealed { - add(sealed.proxy) + + for o := range r.offloading.All() { + all.AddSealed(o) } - for _, active := range r.sealing { - add(active.proxy) + + for s := range r.sealed.All() { + all.AddSealed(s) } - add(r.active.proxy) + for _, c := range r.compacting { + all.AddSealed(c) + } - r.muAll.Lock() - defer r.muAll.Unlock() + for _, a := range r.sealing { + all.AddActive(a) + } - r.all = all - r.allMap = allMap - r.updateOldestTotal() -} + all.AddActive(r.sappender) -// updateOldestTotal recalculates the creation time of the oldest fraction. -// Called after modifications of the complete fractions list. -func (r *fractionRegistry) updateOldestTotal() { - r.oldestTotal = r.all[0].Info().CreationTime -} + r.muSnapshot.Lock() + defer r.muSnapshot.Unlock() -// updateOldestLocal recalculates the creation time of the oldest local fraction. -// Priority order: offloading queue → sealed queue → sealing queue → active fraction. -// Called after modifications -func (r *fractionRegistry) updateOldestLocal() { - if len(r.offloading) > 0 { - r.oldestLocal = r.offloading[0].proxy.Info().CreationTime - } else if len(r.sealed) > 0 { - r.oldestLocal = r.sealed[0].proxy.Info().CreationTime - } else if len(r.sealing) > 0 { - r.oldestLocal = r.sealing[0].proxy.Info().CreationTime - } else { - r.oldestLocal = r.active.proxy.Info().CreationTime - } + r.snapshot = all } diff --git a/fracmanager/fractions_snapshot.go b/fracmanager/fractions_snapshot.go new file mode 100644 index 00000000..9d561f63 --- /dev/null +++ b/fracmanager/fractions_snapshot.go @@ -0,0 +1,136 @@ +package fracmanager + +import ( + "math" + "sync" + + "github.com/ozontech/seq-db/frac" +) + +// RefCounter provides reference counting capability. +type RefCounter interface { + Inc() + Dec() +} + +// fractionsSnapshot represents a point-in-time view of multiple fractions +// with associated reference counters to keep them alive. +type fractionsSnapshot struct { + counters []RefCounter // Reference counters to keep fractions alive + fractions []frac.Fraction // The actual fractions in chronological order + names map[string]int + oldestLocal uint64 + oldestTotal uint64 +} + +func newFractionsSnapshot(capacity int) fractionsSnapshot { + return fractionsSnapshot{ + counters: make([]RefCounter, 0, capacity), + fractions: make([]frac.Fraction, 0, capacity), + names: make(map[string]int, capacity), + oldestLocal: math.MaxUint64, + oldestTotal: math.MaxUint64, + } +} + +func (fs *fractionsSnapshot) AddActive(a *syncAppender) { + fs.names[a.Info().Name()] = len(fs.fractions) + + fs.counters = append(fs.counters, a) + fs.fractions = append(fs.fractions, a) + + fs.oldestLocal = min(fs.oldestLocal, a.Info().CreationTime) + fs.oldestTotal = min(fs.oldestTotal, fs.oldestLocal) +} + +func (fs *fractionsSnapshot) AddSealed(s *refCountedSealed) { + fs.names[s.Info().Name()] = len(fs.fractions) + + fs.counters = append(fs.counters, s) + fs.fractions = append(fs.fractions, s) + + fs.oldestLocal = min(fs.oldestLocal, s.Info().CreationTime) + fs.oldestTotal = min(fs.oldestTotal, fs.oldestLocal) +} + +func (fs *fractionsSnapshot) AddRemote(r *refCountedRemote) { + fs.names[r.Info().Name()] = len(fs.fractions) + + fs.counters = append(fs.counters, r) + fs.fractions = append(fs.fractions, r) + + fs.oldestTotal = min(fs.oldestTotal, r.Info().CreationTime) +} + +// AcquireAll returns the fractions and a release function. +// Caller must call the release function when done to decrement reference counts. +func (fs *fractionsSnapshot) AcquireAll() ([]frac.Fraction, func()) { + for _, c := range fs.counters { + c.Inc() + } + + counters := fs.counters // make copy of counters + return fs.fractions, func() { + for _, c := range counters { + c.Dec() + } + } +} + +func (fs *fractionsSnapshot) AcquireOne(name string) (frac.Fraction, func(), bool) { + i, ok := fs.names[name] + if !ok { + return nil, func() {}, false + } + + c := fs.counters[i] + f := fs.fractions[i] + + c.Inc() + return f, c.Dec, true +} + +type refCounterWg struct { + wg sync.WaitGroup +} + +func (p *refCounterWg) Inc() { p.wg.Add(1) } + +func (p *refCounterWg) Dec() { p.wg.Done() } + +// refCountedActive wraps frac.Active with reference counting. +// Destroy releases the underlying Active after all references are gone. +type refCountedActive struct { + refCounterWg + *frac.Active +} + +// Destroy waits for all references to be released and then releases the Active. +func (p *refCountedActive) Destroy() { + p.wg.Wait() + p.Release() +} + +// refCountedSealed wraps frac.Sealed with reference counting. +type refCountedSealed struct { + refCounterWg + *frac.Sealed +} + +// Destroy waits for all references to be released and then destroys the Sealed. +func (p *refCountedSealed) Destroy() { + p.wg.Wait() + p.Suicide() +} + +// refCountedRemote wraps frac.Remote with reference counting. +type refCountedRemote struct { + refCounterWg + *frac.Remote +} + +// Destroy waits for all references to be released and then destroys the Remote. +func (p *refCountedRemote) Destroy() { + p.wg.Wait() + p.Suicide() +} diff --git a/fracmanager/lifecycle_manager.go b/fracmanager/lifecycle_manager.go index cd1c4bd3..e98c5871 100644 --- a/fracmanager/lifecycle_manager.go +++ b/fracmanager/lifecycle_manager.go @@ -2,7 +2,6 @@ package fracmanager import ( "context" - "path/filepath" "sync" "time" @@ -10,7 +9,6 @@ import ( "github.com/ozontech/seq-db/frac" "github.com/ozontech/seq-db/logger" - "github.com/ozontech/seq-db/util" ) // lifecycleManager manages the complete lifecycle of fractions. @@ -23,7 +21,7 @@ type lifecycleManager struct { registry *fractionRegistry // fraction state registry tasks *TaskManager // Background offloading tasks - sealingWg sync.WaitGroup + sealingWg sync.WaitGroup // todo: get rid after removing SealAll in tests } func newLifecycleManager( @@ -44,7 +42,7 @@ func newLifecycleManager( // Maintain performs periodic lifecycle management tasks. // It coordinates rotation, offloading, cleanup based on configuration. func (lc *lifecycleManager) Maintain(ctx context.Context, cfg *Config, wg *sync.WaitGroup) { - lc.registry.SuspendIfOverCapacity(cfg.SealingQueueLen, cfg.SuspendThreshold()) + lc.registry.suspendIfOverCapacity(cfg.SealingQueueLen, cfg.SuspendThreshold()) lc.rotate(cfg.FracSize, wg) if cfg.OffloadingEnabled { @@ -67,41 +65,14 @@ func (lc *lifecycleManager) SyncInfoCache() { } } -// seal converts an active fraction to sealed state. -// It freezes writes, waits for pending operations, then seals the fraction. -func (lc *lifecycleManager) seal(active *activeProxy) error { - sealsTotal.Inc() - now := time.Now() - sealed, err := lc.provider.Seal(active.instance) - if err != nil { - return err - } - sealingTime := time.Since(now) - sealsDoneSeconds.Observe(sealingTime.Seconds()) - - logger.Info( - "fraction sealed", - zap.String("fraction", filepath.Base(sealed.BaseFileName)), - zap.Float64("time_spent_s", util.DurationToUnit(sealingTime, "s")), - ) - - lc.infoCache.Add(sealed.Info()) - lc.registry.PromoteToSealed(active, sealed) - active.proxy.Redirect(sealed) - active.instance.Release() - return nil -} - // rotate checks if active fraction needs rotation based on size limit. // Creates new active fraction and starts sealing the previous one. func (lc *lifecycleManager) rotate(maxSize uint64, wg *sync.WaitGroup) { - activeToSeal, waitBeforeSealing, err := lc.registry.RotateIfFull(maxSize, func() *activeProxy { - return newActiveProxy(lc.provider.CreateActive()) - }) + active, waitBeforeSealing, err := lc.registry.rotateIfFull(maxSize, lc.provider) if err != nil { logger.Fatal("active fraction rotation error", zap.Error(err)) } - if activeToSeal == nil { + if active == nil { return } @@ -112,37 +83,39 @@ func (lc *lifecycleManager) rotate(maxSize uint64, wg *sync.WaitGroup) { defer lc.sealingWg.Done() waitBeforeSealing() - if err := lc.seal(activeToSeal); err != nil { + sealed, err := lc.provider.Seal(active.Active) + if err != nil { logger.Fatal("sealing error", zap.Error(err)) } + + lc.infoCache.Add(sealed.Info()) + lc.registry.promoteToSealed(active, sealed) + active.Destroy() }() } // offloadLocal starts offloading of local fractions to remote storage. // Selects fractions based on disk space usage and retention policy. func (lc *lifecycleManager) offloadLocal(ctx context.Context, sizeLimit uint64, retryDelay time.Duration, wg *sync.WaitGroup) { - toOffload, err := lc.registry.EvictLocal(true, sizeLimit) + toOffload, err := lc.registry.evictLocalForOffload(sizeLimit) if err != nil { logger.Fatal("error releasing old fractions:", zap.Error(err)) } - for _, sealed := range toOffload { + for _, frac := range toOffload { wg.Add(1) - _, err := lc.tasks.Run(sealed.instance.BaseFileName, ctx, func(ctx context.Context) { + _, err := lc.tasks.Run(frac.BaseFileName, ctx, func(ctx context.Context) { defer wg.Done() - remote := lc.offloadWithRetry(ctx, sealed.instance, retryDelay) + remote := lc.offloadWithRetry(ctx, frac.Sealed, retryDelay) - lc.registry.PromoteToRemote(sealed, remote) + lc.registry.promoteToRemote(frac, remote) if remote == nil { - sealed.proxy.Redirect(emptyFraction{}) - lc.infoCache.Remove(sealed.instance.Info().Name()) - } else { - sealed.proxy.Redirect(remote) + lc.infoCache.Remove(frac.Info().Name()) } // free up local resources - sealed.instance.Suicide() + frac.Destroy() maintenanceTruncateTotal.Add(1) }) if err != nil { @@ -208,59 +181,58 @@ func (lc *lifecycleManager) tryOffload(ctx context.Context, sealed *frac.Sealed) // cleanRemote deletes outdated remote fractions based on retention policy. func (lc *lifecycleManager) cleanRemote(retention time.Duration, wg *sync.WaitGroup) { - toDelete := lc.registry.EvictRemote(retention) - wg.Add(1) - go func() { - defer wg.Done() - for _, remote := range toDelete { - remote.proxy.Redirect(emptyFraction{}) - lc.infoCache.Remove(remote.instance.Info().Name()) - remote.instance.Suicide() - } - }() + toDelete := lc.registry.evictRemote(retention) + wg.Add(len(toDelete)) + for _, remote := range toDelete { + go func() { + defer wg.Done() + lc.infoCache.Remove(remote.Info().Name()) + remote.Destroy() + }() + } } // cleanLocal deletes outdated local fractions when offloading is disabled. func (lc *lifecycleManager) cleanLocal(sizeLimit uint64, wg *sync.WaitGroup) { - toDelete, err := lc.registry.EvictLocal(false, sizeLimit) + toDelete, err := lc.registry.evictLocalForDelete(sizeLimit) if err != nil { logger.Fatal("error releasing old fractions:", zap.Error(err)) } + if len(toDelete) > 0 && !lc.flags.IsCapacityExceeded() { if err := lc.flags.setCapacityExceeded(true); err != nil { logger.Fatal("can't set capacity_exceeded flag", zap.Error(err)) } } - wg.Add(1) - go func() { - defer wg.Done() - for _, sealed := range toDelete { - sealed.proxy.Redirect(emptyFraction{}) - lc.infoCache.Remove(sealed.instance.Info().Name()) - sealed.instance.Suicide() + wg.Add(len(toDelete)) + for _, frac := range toDelete { + go func() { + defer wg.Done() + lc.infoCache.Remove(frac.Info().Name()) + frac.Destroy() maintenanceTruncateTotal.Add(1) - } - }() + }() + } } // updateOldestMetric updates the prometheus metric with oldest fraction timestamp. func (lc *lifecycleManager) updateOldestMetric() { - oldestFracTime.WithLabelValues("remote").Set((time.Duration(lc.registry.OldestTotal()) * time.Millisecond).Seconds()) - oldestFracTime.WithLabelValues("local").Set((time.Duration(lc.registry.OldestLocal()) * time.Millisecond).Seconds()) + oldestFracTime.WithLabelValues("remote").Set((time.Duration(lc.registry.oldestTotal()) * time.Millisecond).Seconds()) + oldestFracTime.WithLabelValues("local").Set((time.Duration(lc.registry.oldestLocal()) * time.Millisecond).Seconds()) } // removeOverflowed removes fractions from offloading queue that exceed size limit // Stops ongoing offloading tasks and cleans up both local and remote resources. func (lc *lifecycleManager) removeOverflowed(sizeLimit uint64, wg *sync.WaitGroup) { - evicted := lc.registry.EvictOverflowed(sizeLimit) - for _, item := range evicted { + evicted := lc.registry.evictOverflowed(sizeLimit) + for _, sealed := range evicted { wg.Add(1) go func() { defer wg.Done() // Cancel the offloading task - this operation may take significant time // hence executed in a separate goroutine to avoid blocking - lc.tasks.Cancel(item.instance.BaseFileName) + lc.tasks.Cancel(sealed.BaseFileName) }() } } diff --git a/fracmanager/lifecycle_manager_test.go b/fracmanager/lifecycle_manager_test.go index abd180e2..bebc2c1f 100644 --- a/fracmanager/lifecycle_manager_test.go +++ b/fracmanager/lifecycle_manager_test.go @@ -1,14 +1,20 @@ package fracmanager import ( + "math" "math/rand" "path/filepath" "sync" "testing" + "time" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/ozontech/seq-db/consts" + "github.com/ozontech/seq-db/frac/processor" + "github.com/ozontech/seq-db/parser" + "github.com/ozontech/seq-db/seq" ) func setupLifecycle(t testing.TB, cfg *Config) (*lifecycleManager, func()) { @@ -31,21 +37,18 @@ func TestFracInfoCache(t *testing.T) { lc, tearDown := setupLifecycle(t, nil) defer tearDown() - var total uint64 - fillRotateAndCheck := func(names map[string]struct{}) { - active := lc.registry.Active() - appendDocsToActive(t, active.instance, 10+rand.Intn(10)) + appender := lc.registry.appender() + appendDocsToActive(t, appender.Active, 10+rand.Intn(10)) wg := sync.WaitGroup{} lc.rotate(0, &wg) wg.Wait() - info := active.proxy.Info() + info := appender.Info() _, ok := lc.infoCache.Get(info.Name()) assert.True(t, ok) - total += info.FullSize() names[info.Name()] = struct{}{} } @@ -53,12 +56,13 @@ func TestFracInfoCache(t *testing.T) { for range 10 { fillRotateAndCheck(first) } - halfSize := total + halfSize := lc.registry.statistics().TotalSizeOnDiskLocal() second := map[string]struct{}{} for range 10 { fillRotateAndCheck(second) } + total := lc.registry.statistics().TotalSizeOnDiskLocal() wg := sync.WaitGroup{} lc.cleanLocal(total-halfSize, &wg) @@ -80,18 +84,14 @@ func TestCapacityExceeded(t *testing.T) { defer tearDown() const fracsCount = 10 - var total uint64 fillAndRotate := func() { - active := lc.registry.Active() - appendDocsToActive(t, active.instance, 10+rand.Intn(10)) + appender := lc.registry.appender() + appendDocsToActive(t, appender.Active, 10+rand.Intn(10)) wg := sync.WaitGroup{} lc.rotate(0, &wg) wg.Wait() - - info := active.proxy.Info() - total += info.FullSize() } assert.False(t, lc.flags.IsCapacityExceeded(), "expect data dir is empty") @@ -102,17 +102,19 @@ func TestCapacityExceeded(t *testing.T) { } assert.False(t, lc.flags.IsCapacityExceeded(), "there should be no deletions and the flag is false") + total := lc.registry.statistics().TotalSizeOnDiskLocal() + wg := sync.WaitGroup{} lc.cleanLocal(total, &wg) wg.Wait() - assert.Equal(t, fracsCount, lc.registry.Stats().sealed.count, "as much as was added, so much should be") + assert.Equal(t, fracsCount, lc.registry.statistics().sealed.count, "as much as was added, so much should be") assert.False(t, lc.flags.IsCapacityExceeded(), "there should still be no deletions, and the flag is false") lc.cleanLocal(total-1, &wg) wg.Wait() - assert.Equal(t, fracsCount-1, lc.registry.Stats().sealed.count, "expect one less") + assert.Equal(t, fracsCount-1, lc.registry.statistics().sealed.count, "expect one less") assert.True(t, lc.flags.IsCapacityExceeded(), "the flag must be true now") } @@ -121,40 +123,115 @@ func TestOldestMetrics(t *testing.T) { defer tearDown() const fracsCount = 10 - var total uint64 - fillAndRotate := func() { - active := lc.registry.Active() - appendDocsToActive(t, active.instance, 10+rand.Intn(10)) + appender := lc.registry.appender() + appendDocsToActive(t, appender.Active, 10+rand.Intn(10)) wg := sync.WaitGroup{} lc.rotate(0, &wg) wg.Wait() - - info := active.proxy.Info() - total += info.FullSize() } - firstFracTime := lc.registry.Active().proxy.Info().CreationTime + firstFracTime := lc.registry.appender().Info().CreationTime for range fracsCount { fillAndRotate() } // Check state after initial rotations - assert.Equal(t, firstFracTime, lc.registry.OldestTotal(), "should point to the very first fraction when all data is local") - assert.Equal(t, firstFracTime, lc.registry.OldestLocal(), "should point to the first fraction when nothing is offloaded") + assert.Equal(t, firstFracTime, lc.registry.oldestTotal(), "should point to the very first fraction when all data is local") + assert.Equal(t, firstFracTime, lc.registry.oldestLocal(), "should point to the first fraction when nothing is offloaded") + + halfSize := lc.registry.statistics().TotalSizeOnDiskLocal() - halfSize := total - halfwayFracTime := lc.registry.Active().proxy.Info().CreationTime + halfwayFracTime := lc.registry.appender().Info().CreationTime for range fracsCount { fillAndRotate() } + total := lc.registry.statistics().TotalSizeOnDiskLocal() + wg := sync.WaitGroup{} lc.offloadLocal(t.Context(), total-halfSize, 0, &wg) wg.Wait() // Check state after offloading assert.NotEqual(t, firstFracTime, halfwayFracTime, "expect different creation times") - assert.Equal(t, firstFracTime, lc.registry.OldestTotal(), "should still reference the first fraction after offload") - assert.Equal(t, halfwayFracTime, lc.registry.OldestLocal(), "should point to the oldest remaining local fraction after offload") + assert.Equal(t, firstFracTime, lc.registry.oldestTotal(), "should still reference the first fraction after offload") + assert.Equal(t, halfwayFracTime, lc.registry.oldestLocal(), "should point to the oldest remaining local fraction after offload") +} + +func TestPendingDestroy(t *testing.T) { + lc, tearDown := setupLifecycle(t, nil) + defer tearDown() + + const ( + fracsCount = 10 + docsPerFrac = 10 + ) + // appending docs to `fracsCount` fractions where the last is active and the rest are sealed + wg := sync.WaitGroup{} + for range fracsCount - 1 { + appendDocsToActive(t, lc.registry.appender().Active, docsPerFrac) + lc.rotate(0, &wg) + } + appendDocsToActive(t, lc.registry.appender().Active, docsPerFrac) + + // wait sealing complete + wg.Wait() + + // take all fracs to search + fractions1, release1 := lc.registry.acquireAllFractions() + + // delete all sealing fracs + lc.cleanLocal(lc.registry.appender().Info().FullSize(), &wg) + + var ( + beforeRelease time.Time + afterCleanup time.Time + ) + + cleanup := sync.WaitGroup{} + cleanup.Add(1) + go func() { + // cleanup is pending, so run it in a goroutine + // waiting for cleanup to finish + defer cleanup.Done() + wg.Wait() + afterCleanup = time.Now() + }() + + queryAst, err := parser.ParseSeqQL("*", seq.Mapping{}) + require.NoError(t, err, "failed to parse query") + params := processor.SearchParams{ + AST: queryAst.Root, + From: seq.MID(0), + To: seq.MID(math.MaxUint64), + Limit: math.MaxInt32, + } + + for _, f := range fractions1 { + qpr, err := f.Search(t.Context(), params) + assert.NoError(t, err, "failed to search") + assert.Equal(t, docsPerFrac, len(qpr.IDs)) + } + + beforeRelease = time.Now() + release1() + + cleanup.Wait() + assert.Less(t, beforeRelease, afterCleanup, "we expect cleanup to happen after release") + + fractions2, release2 := lc.registry.acquireAllFractions() + + assert.Len(t, fractions2, 1, "only one active fraction should remain") + singleName := fractions2[0].Info().Name() + + for _, f := range fractions1 { + if f.Info().Name() == singleName { + continue + } + assert.Panics(t, func() { + _, _ = f.Search(t.Context(), params) + }, "searching by destroyed faction is expected to trigger a panic") + } + release2() } diff --git a/fracmanager/partitioned_collection.go b/fracmanager/partitioned_collection.go new file mode 100644 index 00000000..7f37f045 --- /dev/null +++ b/fracmanager/partitioned_collection.go @@ -0,0 +1,117 @@ +package fracmanager + +import ( + "iter" + + "github.com/ozontech/seq-db/util" +) + +// PartitionedCollection manages a collection of objects grouped into partitions by a user‑defined value. +// Each partition is identified by a uint64. +type PartitionedCollection[T any] struct { + getPartition func(T) uint64 // function to extract partition ID from object + byKey map[string]T // primary index: key -> object + byPartition map[uint64]map[string]T // partition ID -> map[key]object + minPartition *util.MinHeap[uint64] // min‑heap of partition IDs for O(1) MinPartition +} + +// NewPartitionedCollection creates a new empty PartitionedCollection. +func NewPartitionedCollection[T any](getPartition func(T) uint64) PartitionedCollection[T] { + return PartitionedCollection[T]{ + getPartition: getPartition, + byKey: make(map[string]T), + byPartition: make(map[uint64]map[string]T), + minPartition: util.NewMinHeap[uint64](), + } +} + +// Add inserts a new object into the collection. +func (c *PartitionedCollection[T]) Add(key string, obj T) { + if _, ok := c.byKey[key]; ok { + return + } + + partitionID := c.getPartition(obj) + if _, ok := c.byPartition[partitionID]; !ok { + c.minPartition.Push(partitionID) + c.byPartition[partitionID] = make(map[string]T) + } + c.byPartition[partitionID][key] = obj + c.byKey[key] = obj +} + +// Delete removes an object from the collection by its key. +// Does nothing if the key doesn't exist. +func (c *PartitionedCollection[T]) Del(key string) { + obj, ok := c.byKey[key] + if !ok { + return + } + + partitionID := c.getPartition(obj) + delete(c.byPartition[partitionID], key) + if len(c.byPartition[partitionID]) == 0 { + c.minPartition.Remove(partitionID) + delete(c.byPartition, partitionID) + } + delete(c.byKey, key) +} + +// MinPartition returns the smallest partition ID among all stored objects. +// Returns 0 if the collection is empty. +func (c *PartitionedCollection[T]) MinPartition() uint64 { + if val, ok := c.minPartition.Min(); ok { + return val + } + return 0 +} + +// GetByPartition returns all objects in the specified partition. +func (c *PartitionedCollection[T]) GetByPartition(partitionID uint64) []T { + partitionMap, ok := c.byPartition[partitionID] + if !ok { + return nil + } + res := make([]T, 0, len(partitionMap)) + for _, obj := range partitionMap { + res = append(res, obj) + } + return res +} + +// Get retrieves an object by its key. +// Returns the object and true if found, zero value and false otherwise. +func (c *PartitionedCollection[T]) Get(key string) (T, bool) { + obj, ok := c.byKey[key] + return obj, ok +} + +// All returns all objects in the collection. +// The order is not guaranteed. +func (c *PartitionedCollection[T]) All() iter.Seq[T] { + return func(yield func(T) bool) { + for _, obj := range c.byKey { + if !yield(obj) { + return + } + } + } +} + +// Len returns the number of objects in the collection. +func (c *PartitionedCollection[T]) Len() int { + return len(c.byKey) +} + +// GetAllPartitions returns a map of all partitions in the collection. +func (c *PartitionedCollection[T]) GetAllPartitions() map[uint64][]T { + result := make(map[uint64][]T, len(c.byPartition)) + for partitionID, objects := range c.byPartition { + partition := make([]T, 0, len(objects)) + for _, obj := range objects { + partition = append(partition, obj) + } + result[partitionID] = partition + } + return result +} diff --git a/fracmanager/proxy_frac.go b/fracmanager/proxy_frac.go deleted file mode 100644 index 6d4df41f..00000000 --- a/fracmanager/proxy_frac.go +++ /dev/null @@ -1,201 +0,0 @@ -package fracmanager - -import ( - "context" - "errors" - "math" - "sync" - "time" - - "go.uber.org/zap" - - "github.com/ozontech/seq-db/frac" - "github.com/ozontech/seq-db/frac/common" - "github.com/ozontech/seq-db/frac/processor" - "github.com/ozontech/seq-db/logger" - "github.com/ozontech/seq-db/metric" - "github.com/ozontech/seq-db/seq" - "github.com/ozontech/seq-db/util" -) - -var ( - _ frac.Fraction = (*fractionProxy)(nil) - _ frac.Fraction = (*emptyFraction)(nil) - - ErrFractionNotWritable = errors.New("fraction is not writable") - ErrFractionSuspended = errors.New("write operations temporarily suspended - database capacity exceeded") -) - -// fractionProxy provides thread-safe access to a fraction with atomic replacement -// Used to switch fraction implementations (active → sealed → remote) without blocking readers. -// Lifecycle: Created for each fraction, persists through state transitions. -type fractionProxy struct { - mu sync.RWMutex - impl frac.Fraction // Current fraction implementation -} - -func (p *fractionProxy) Redirect(f frac.Fraction) { - p.mu.Lock() - defer p.mu.Unlock() - p.impl = f -} - -func (p *fractionProxy) Info() *common.Info { - p.mu.RLock() - defer p.mu.RUnlock() - return p.impl.Info() -} - -func (p *fractionProxy) IsIntersecting(from, to seq.MID) bool { - p.mu.RLock() - defer p.mu.RUnlock() - return p.impl.IsIntersecting(from, to) -} - -func (p *fractionProxy) Contains(mid seq.MID) bool { - p.mu.RLock() - defer p.mu.RUnlock() - return p.impl.Contains(mid) -} - -func (p *fractionProxy) Fetch(ctx context.Context, ids []seq.ID) ([][]byte, error) { - p.mu.RLock() - defer p.mu.RUnlock() - return p.impl.Fetch(ctx, ids) -} - -func (p *fractionProxy) Search(ctx context.Context, params processor.SearchParams) (*seq.QPR, error) { - p.mu.RLock() - defer p.mu.RUnlock() - return p.impl.Search(ctx, params) -} - -func (p *fractionProxy) FindLIDs(ctx context.Context, ids []seq.ID) ([]seq.LID, error) { - return p.impl.FindLIDs(ctx, ids) -} - -// activeProxy manages an active (writable) fraction -// Tracks pending write operations and provides freeze capability. -// Lifecycle: Created when fraction becomes active, destroyed after sealing. -type activeProxy struct { - proxy *fractionProxy // Thread-safe fraction access - instance *frac.Active // Actual active fraction instance - sealed *frac.Sealed // Sealed version (set after sealing) - - mu sync.RWMutex // Protects readonly state - wg sync.WaitGroup // Tracks pending write operations - - finalized bool // Whether fraction is frozen for writes - suspended bool // Temporarily suspended for writes -} - -func newActiveProxy(active *frac.Active) *activeProxy { - return &activeProxy{ - proxy: &fractionProxy{impl: active}, - instance: active, - } -} - -// Append adds documents to the active fraction -func (p *activeProxy) Append(docs, meta []byte) error { - p.mu.RLock() - if p.finalized { - p.mu.RUnlock() - return ErrFractionNotWritable - } - if p.suspended { - p.mu.RUnlock() - return ErrFractionSuspended - } - p.wg.Add(1) // Important: wg.Add() inside lock to prevent race with WaitWriteIdle() - p.mu.RUnlock() - - return p.instance.Append(docs, meta, &p.wg) -} - -// WaitWriteIdle waits for all pending write operations to complete -// Used before sealing to ensure data consistency. -func (p *activeProxy) WaitWriteIdle() { - start := time.Now() - logger.Info("waiting fraction to stop write...", zap.String("name", p.instance.BaseFileName)) - p.wg.Wait() - waitTime := util.DurationToUnit(time.Since(start), "s") - logger.Info("write is stopped", - zap.String("name", p.instance.BaseFileName), - zap.Float64("time_wait_s", waitTime)) -} - -func (p *activeProxy) Suspended() bool { - p.mu.Lock() - defer p.mu.Unlock() - - return p.suspended -} - -func (p *activeProxy) Suspend(value bool) { - p.mu.Lock() - p.suspended = value - p.mu.Unlock() -} - -// Finalize marks the fraction as read-only and prevents new writes from starting after finalize. -func (p *activeProxy) Finalize() error { - p.mu.Lock() - defer p.mu.Unlock() - - if p.finalized { - return errors.New("fraction is already finalized") - } - p.finalized = true - - return nil -} - -// sealedProxy represents a sealed fraction that may be offloaded -// Tracks both local sealed instance and remote version if offloaded. -type sealedProxy struct { - proxy *fractionProxy // Thread-safe fraction access - instance *frac.Sealed // Local sealed fraction - remote *frac.Remote // Remote version (if offloaded) -} - -// remoteProxy represents an offloaded fraction -type remoteProxy struct { - proxy *fractionProxy // Thread-safe fraction access - instance *frac.Remote // Remote fraction instance -} - -// emptyFraction represents a missing or deleted fraction -// Returns empty results for all operations. -// Used as placeholder when fraction is removed but references still exist. -type emptyFraction struct { -} - -func (emptyFraction) Info() *common.Info { - return &common.Info{ - Path: "empty", - From: math.MaxUint64, - To: 0, - } -} - -func (emptyFraction) IsIntersecting(_, _ seq.MID) bool { - return false -} - -func (emptyFraction) Contains(mid seq.MID) bool { - return false -} - -func (emptyFraction) Fetch(ctx context.Context, ids []seq.ID) ([][]byte, error) { - return nil, nil -} - -func (emptyFraction) Search(_ context.Context, params processor.SearchParams) (*seq.QPR, error) { - metric.CountersTotal.WithLabelValues("empty_data_provider").Inc() - return &seq.QPR{Aggs: make([]seq.AggregatableSamples, len(params.AggQ))}, nil -} - -func (emptyFraction) FindLIDs(_ context.Context, _ []seq.ID) ([]seq.LID, error) { - return nil, nil -} diff --git a/fracmanager/sealer_test.go b/fracmanager/sealer_test.go index f85c3f8f..51c16b6b 100644 --- a/fracmanager/sealer_test.go +++ b/fracmanager/sealer_test.go @@ -19,8 +19,8 @@ import ( "github.com/ozontech/seq-db/frac" "github.com/ozontech/seq-db/frac/common" "github.com/ozontech/seq-db/frac/sealed" - "github.com/ozontech/seq-db/frac/sealed/sealing" "github.com/ozontech/seq-db/indexer" + "github.com/ozontech/seq-db/sealing" "github.com/ozontech/seq-db/seq" testscommon "github.com/ozontech/seq-db/tests/common" ) diff --git a/fracmanager/sync_appender.go b/fracmanager/sync_appender.go new file mode 100644 index 00000000..1acb15a3 --- /dev/null +++ b/fracmanager/sync_appender.go @@ -0,0 +1,84 @@ +package fracmanager + +import ( + "errors" + "sync" + "time" + + "go.uber.org/zap" + + "github.com/ozontech/seq-db/logger" + "github.com/ozontech/seq-db/util" +) + +var ( + ErrFractionNotWritable = errors.New("fraction is not writable") + ErrFractionSuspended = errors.New("write operations temporarily suspended - database capacity exceeded") +) + +type syncAppender struct { + refCountedActive // Actual active fraction instance + + mu sync.RWMutex // Protects readonly state + wg sync.WaitGroup // Tracks pending write operations + + finalized bool // Whether fraction is frozen for writes + suspended bool // Temporarily suspended for writes +} + +// append adds documents to the active fraction +func (a *syncAppender) append(docs, meta []byte) error { + a.mu.RLock() + if a.finalized { + a.mu.RUnlock() + return ErrFractionNotWritable + } + if a.suspended { + a.mu.RUnlock() + return ErrFractionSuspended + } + a.wg.Add(1) // Important: wg.Add() inside lock to prevent race with WaitWriteIdle() + a.mu.RUnlock() + + return a.refCountedActive.Append(docs, meta, &a.wg) +} + +func (a *syncAppender) isSuspended() bool { + a.mu.Lock() + defer a.mu.Unlock() + + return a.suspended +} + +func (a *syncAppender) suspend(value bool) { + a.mu.Lock() + a.suspended = value + a.mu.Unlock() +} + +// waitWriteIdle waits for all pending write operations to complete +// Used before sealing to ensure data consistency. +func (a *syncAppender) waitWriteIdle() { + start := time.Now() + logger.Info("waiting fraction to stop write...", zap.String("name", a.BaseFileName)) + a.wg.Wait() + waitTime := util.DurationToUnit(time.Since(start), "s") + logger.Info( + "write is stopped", + zap.String("name", a.BaseFileName), + zap.Float64("time_wait_s", waitTime), + ) +} + +// finalize marks the fraction as read-only and prevents new writes from starting after finalize. +func (a *syncAppender) finalize() error { + a.mu.Lock() + if a.finalized { + a.mu.Unlock() + return errors.New("fraction is already finalized") + } + a.finalized = true + a.mu.Unlock() + + return nil +} diff --git a/frac/sealed/sealing/blocks_builder.go b/indexwriter/blocks.go similarity index 53% rename from frac/sealed/sealing/blocks_builder.go rename to indexwriter/blocks.go index fc069cbf..3064491b 100644 --- a/frac/sealed/sealing/blocks_builder.go +++ b/indexwriter/blocks.go @@ -1,4 +1,4 @@ -package sealing +package indexwriter import ( "encoding/binary" @@ -11,53 +11,47 @@ import ( "github.com/ozontech/seq-db/util" ) -type ( - TokenBlock = util.Pair[tokensSealBlock, []token.FieldTable] -) +type tokenFieldBlock = util.Pair[unpackedTokenBlock, []token.FieldTable] -// tokensExt represents the token ID range contained in a block. -type tokensExt struct { +// tokenExt represents the token ID range contained in a block. +type tokenExt struct { minTID uint32 // First token ID in the block maxTID uint32 // Last token ID in the block } -// tokensSealBlock represents a sealed block containing token data with metadata. -type tokensSealBlock struct { - ext tokensExt // Tokens block metadata for registry marking +// unpackedTokenBlock represents a sealed block containing token data with metadata. +type unpackedTokenBlock struct { + ext tokenExt // Tokens block metadata for registry marking payload token.Block // Actual token data payload } -// lidsExt represents the range and continuation status of LID blocks. -type lidsExt struct { +// lidExt represents the range and continuation status of LID blocks. +type lidExt struct { minTID uint32 // First token ID in the LID block maxTID uint32 // Last token ID in the LID block isContinued bool // Whether LID sequence continues in next block } -// lidsSealBlock represents a sealed block containing LID (Local ID) data. -type lidsSealBlock struct { - ext lidsExt // LIDs block metadata for registry marking +// unpackedLIDBlock represents a sealed block containing LID (Local ID) data. +type unpackedLIDBlock struct { + ext lidExt // LIDs block metadata for registry marking payload lids.Block // LID data payload } -// idsSealBlock represents a sealed block containing various identifier types. -type idsSealBlock struct { +// unpackedIDBlock represents a sealed block containing various identifier types. +type unpackedIDBlock struct { mids seqids.BlockMIDs rids seqids.BlockRIDs params seqids.BlockParams } -// blocksBuilder constructs sealed blocks from various data sources. -// Provides error tracking and consistency validation during block construction. -type blocksBuilder struct{} - -func (bb *blocksBuilder) BuildTokenBlocks( +func tokenBlock( it iter.Seq2[string, iter.Seq2[TokenPosting, error]], accumulate func([]uint32) error, blockCapacity int, -) iter.Seq2[TokenBlock, error] { - return func(yield func(TokenBlock, error) bool) { +) iter.Seq2[tokenFieldBlock, error] { + return func(yield func(tokenFieldBlock, error) bool) { var ( - block tokensSealBlock + block unpackedTokenBlock blockIdx uint32 blockSize int ) @@ -86,7 +80,7 @@ func (bb *blocksBuilder) BuildTokenBlocks( emitFieldEntry() block.ext.maxTID = currentTID - pair := TokenBlock{First: block, Second: pendingTable} + pair := tokenFieldBlock{First: block, Second: pendingTable} if !yield(pair, nil) { return false } @@ -105,15 +99,15 @@ func (bb *blocksBuilder) BuildTokenBlocks( } block.ext.minTID = 1 - for field, tokenIterator := range it { + for field, tokIt := range it { emitFieldEntry() fieldName = field fieldEntryStartTID = currentTID + 1 - for pair, err := range tokenIterator { + for pair, err := range tokIt { if err != nil { - yield(TokenBlock{}, err) + yield(tokenFieldBlock{}, err) return } @@ -131,7 +125,7 @@ func (bb *blocksBuilder) BuildTokenBlocks( block.payload.Payload = append(block.payload.Payload, tok...) if err := accumulate(tlids); err != nil { - yield(TokenBlock{}, err) + yield(tokenFieldBlock{}, err) return } @@ -148,7 +142,7 @@ func (bb *blocksBuilder) BuildTokenBlocks( func newTokenTableEntry( entryStartTID, entryEndTID uint32, - blockIndex uint32, block tokensSealBlock, + blockIndex uint32, block unpackedTokenBlock, ) *token.TableEntry { // Convert global TIDs to block-local indices firstIndex := entryStartTID - block.ext.minTID @@ -168,15 +162,15 @@ func newTokenTableEntry( } } -// seqBlockID accumulates scalar (ID, position) pairs into sealed ID blocks. +// idBlock accumulates scalar (ID, position) pairs into sealed ID blocks. // A new block is yielded every `blockCapacity` IDs. -func seqBlockID(ids iter.Seq2[DocLocation, error], blockCapacity int) iter.Seq2[idsSealBlock, error] { - return func(yield func(idsSealBlock, error) bool) { - var block idsSealBlock +func idBlock(ids iter.Seq2[DocLocation, error], blockCapacity int) iter.Seq2[unpackedIDBlock, error] { + return func(yield func(unpackedIDBlock, error) bool) { + var block unpackedIDBlock for pair, err := range ids { if err != nil { - yield(idsSealBlock{}, err) + yield(unpackedIDBlock{}, err) return } @@ -202,84 +196,24 @@ func seqBlockID(ids iter.Seq2[DocLocation, error], blockCapacity int) iter.Seq2[ } } -type lidAccumulator struct { - blockCapacity int - onBlock func(lidsSealBlock) error - - currentTID uint32 - currentBlock lidsSealBlock - - isEndOfToken bool - isContinued bool -} - -func newLIDAccumulator( - blockCapacity int, - onBlock func(lidsSealBlock) error, -) *lidAccumulator { - a := &lidAccumulator{ - blockCapacity: blockCapacity, - onBlock: onBlock, +// collapseOrderedFieldsTables merges FieldTables with the same field name. +// Assumes input is sorted by Field. +func collapseOrderedFieldsTables(src []token.FieldTable) []token.FieldTable { + if len(src) == 0 { + return nil } - a.currentBlock.ext.minTID = 1 - a.currentBlock.payload = lids.Block{ - LIDs: make([]uint32, 0, blockCapacity), - Offsets: []uint32{0}, - } - - return a -} - -// Add processes LIDs of one token (must be called in TID order). -// -// For each block that fills up, `onBlock` is called immediately -// before the backing arrays are reset, so `onBlock` may read the -// block data but must not retain references to it. -func (a *lidAccumulator) Add(lidsbuf []uint32) error { - a.currentTID++ - - for _, lid := range lidsbuf { - if len(a.currentBlock.payload.LIDs) == a.blockCapacity { - if err := a.onBlock(a.finalizeBlock()); err != nil { - return err - } - - a.currentBlock.ext.minTID = a.currentTID - a.currentBlock.payload.LIDs = a.currentBlock.payload.LIDs[:0] - a.currentBlock.payload.Offsets = a.currentBlock.payload.Offsets[:1] + current := src[0] + var dst []token.FieldTable + for _, ft := range src[1:] { + if current.Field == ft.Field { + current.Entries = append(current.Entries, ft.Entries...) + continue } - a.isEndOfToken = false - a.currentBlock.ext.maxTID = a.currentTID - a.currentBlock.payload.LIDs = append(a.currentBlock.payload.LIDs, lid) - } - - a.isEndOfToken = true - a.currentBlock.payload.Offsets = append( - a.currentBlock.payload.Offsets, - uint32(len(a.currentBlock.payload.LIDs)), - ) - - return nil -} - -func (a *lidAccumulator) Finalize() error { - return a.onBlock(a.finalizeBlock()) -} - -func (a *lidAccumulator) finalizeBlock() lidsSealBlock { - if !a.isEndOfToken { - a.currentBlock.payload.Offsets = append( - a.currentBlock.payload.Offsets, - uint32(len(a.currentBlock.payload.LIDs)), - ) + dst = append(dst, current) + current = ft } - result := a.currentBlock - result.payload.IsLastLID = a.isEndOfToken - result.ext.isContinued = a.isContinued - - a.isContinued = !a.isEndOfToken - return result + return append(dst, current) } diff --git a/frac/sealed/sealing/blocks_builder_test.go b/indexwriter/blocks_test.go similarity index 84% rename from frac/sealed/sealing/blocks_builder_test.go rename to indexwriter/blocks_test.go index d6bca144..bb513130 100644 --- a/frac/sealed/sealing/blocks_builder_test.go +++ b/indexwriter/blocks_test.go @@ -1,4 +1,4 @@ -package sealing +package indexwriter import ( "iter" @@ -7,27 +7,20 @@ import ( "github.com/stretchr/testify/assert" - "github.com/ozontech/seq-db/frac/common" "github.com/ozontech/seq-db/frac/sealed/lids" "github.com/ozontech/seq-db/frac/sealed/token" "github.com/ozontech/seq-db/seq" ) -var _ Source = (*mockSource)(nil) - type mockSource struct { - info common.Info - tokens [][]byte - fields []string - fieldMaxTIDs []uint32 - ids []seq.ID - pos []seq.DocPos - tokenLIDs [][]uint32 - blocksOffsets []uint64 + tokens [][]byte + fields []string + fieldMaxTIDs []uint32 + ids []seq.ID + pos []seq.DocPos + tokenLIDs [][]uint32 } -func (m *mockSource) Info() *common.Info { return &m.info } - func (m *mockSource) TokenTriplet() iter.Seq2[string, iter.Seq2[TokenPosting, error]] { return func(yield func(string, iter.Seq2[TokenPosting, error]) bool) { start := 0 @@ -48,8 +41,7 @@ func (m *mockSource) tokensForField(start, end int) iter.Seq2[TokenPosting, erro if j < len(m.tokenLIDs) { lidsbuf = m.tokenLIDs[j] } - pair := TokenPosting{First: m.tokens[j], Second: lidsbuf} - if !yield(pair, nil) { + if !yield(TokenPosting{First: m.tokens[j], Second: lidsbuf}, nil) { return } } @@ -66,8 +58,6 @@ func (m *mockSource) ID() iter.Seq2[DocLocation, error] { } } -func (m *mockSource) BlockOffsets() []uint64 { return m.blocksOffsets } - func TestBlocksBuilder_BuildTokenBlocks(t *testing.T) { src := mockSource{ tokens: [][]byte{ @@ -114,10 +104,10 @@ func TestBlocksBuilder_BuildTokenBlocks(t *testing.T) { const blockSize = 24 const lidBlockCap = 3 - var lidBlocks []lidsSealBlock + var lidBlocks []unpackedLIDBlock lidAccumulator := newLIDAccumulator( lidBlockCap, - func(block lidsSealBlock) error { + func(block unpackedLIDBlock) error { block.payload.LIDs = slices.Clone(block.payload.LIDs) block.payload.Offsets = slices.Clone(block.payload.Offsets) lidBlocks = append(lidBlocks, block) @@ -125,12 +115,9 @@ func TestBlocksBuilder_BuildTokenBlocks(t *testing.T) { }, ) - var bb blocksBuilder - tokenBlocks := bb.BuildTokenBlocks( + tokenBlocksIter := tokenBlock( src.TokenTriplet(), - func(lids []uint32) error { - return lidAccumulator.Add(lids) - }, + lidAccumulator.add, blockSize, ) @@ -142,7 +129,7 @@ func TestBlocksBuilder_BuildTokenBlocks(t *testing.T) { blockIndex := 0 allFieldsTables := []token.FieldTable{} - for pair, err := range tokenBlocks { + for pair, err := range tokenBlocksIter { assert.NoError(t, err) block, fieldsTables := pair.First, pair.Second assert.Equal(t, expectedSizes[blockIndex], block.payload.Len()) @@ -249,31 +236,31 @@ func TestBlocksBuilder_BuildTokenBlocks(t *testing.T) { }, } assert.Equal(t, actualTokenTable.FieldsTables, expectedTokenTable.FieldsTables) - assert.NoError(t, lidAccumulator.Finalize()) + assert.NoError(t, lidAccumulator.finalize()) - expectedLIDBlocks := []lidsSealBlock{ + expectedLIDBlocks := []unpackedLIDBlock{ { - ext: lidsExt{minTID: 1, maxTID: 1, isContinued: false}, + ext: lidExt{minTID: 1, maxTID: 1, isContinued: false}, payload: lids.Block{LIDs: []uint32{10, 20, 30}, Offsets: []uint32{0, 3}, IsLastLID: false}, }, { - ext: lidsExt{minTID: 1, maxTID: 3, isContinued: true}, + ext: lidExt{minTID: 1, maxTID: 3, isContinued: true}, payload: lids.Block{LIDs: []uint32{40, 2, 3}, Offsets: []uint32{0, 1, 2, 3}, IsLastLID: true}, }, { - ext: lidsExt{minTID: 4, maxTID: 6, isContinued: false}, + ext: lidExt{minTID: 4, maxTID: 6, isContinued: false}, payload: lids.Block{LIDs: []uint32{4, 5, 6}, Offsets: []uint32{0, 1, 2, 3}, IsLastLID: true}, }, { - ext: lidsExt{minTID: 7, maxTID: 9, isContinued: false}, + ext: lidExt{minTID: 7, maxTID: 9, isContinued: false}, payload: lids.Block{LIDs: []uint32{7, 8, 9}, Offsets: []uint32{0, 1, 2, 3}, IsLastLID: true}, }, { - ext: lidsExt{minTID: 10, maxTID: 12, isContinued: false}, + ext: lidExt{minTID: 10, maxTID: 12, isContinued: false}, payload: lids.Block{LIDs: []uint32{10, 11, 12}, Offsets: []uint32{0, 1, 2, 3}, IsLastLID: true}, }, { - ext: lidsExt{minTID: 13, maxTID: 14, isContinued: false}, + ext: lidExt{minTID: 13, maxTID: 14, isContinued: false}, payload: lids.Block{LIDs: []uint32{13, 14}, Offsets: []uint32{0, 1, 2}, IsLastLID: true}, }, } @@ -313,7 +300,7 @@ func TestBlocksBuilder_IDsBlocks(t *testing.T) { i := 0 ids := []seq.ID{} pos := []seq.DocPos{} - for block, err := range seqBlockID(src.ID(), 3) { + for block, err := range idBlock(src.ID(), 3) { assert.NoError(t, err) assert.Equal(t, expectedSizes[i], len(block.mids.Values)) diff --git a/frac/sealed/sealing/index.go b/indexwriter/index.go similarity index 72% rename from frac/sealed/sealing/index.go rename to indexwriter/index.go index 5c23842a..0e51e5cb 100644 --- a/frac/sealed/sealing/index.go +++ b/indexwriter/index.go @@ -1,7 +1,8 @@ -package sealing +package indexwriter import ( "io" + "iter" "github.com/ozontech/seq-db/consts" "github.com/ozontech/seq-db/frac/common" @@ -11,9 +12,34 @@ import ( "github.com/ozontech/seq-db/frac/sealed/token" "github.com/ozontech/seq-db/seq" "github.com/ozontech/seq-db/storage" + "github.com/ozontech/seq-db/util" "github.com/ozontech/seq-db/zstd" ) +type ( + DocLocation = util.Pair[seq.ID, seq.DocPos] + TokenPosting = util.Pair[[]byte, []uint32] +) + +// Source defines the data required to write all index files for a fraction. +type Source interface { + // Info returns metadata describing this source. + Info() *common.Info + + // ID returns an iterator over stored document identifiers paired with + // their positions, in descending [seq.ID] order. + ID() iter.Seq2[DocLocation, error] + + // BlockOffsets returns byte offsets to each document block + // within this source's `.docs` file. + BlockOffsets() []uint64 + + // TokenTriplet iterates over fields in lexicographic order. + // For each field, it yields tokens (lexicographically sorted) + // paired with the local document ID list for that token. + TokenTriplet() iter.Seq2[string, iter.Seq2[TokenPosting, error]] +} + // indexBlock is one compressed (or not) block with its registry metadata. type indexBlock struct { codec storage.Codec @@ -23,11 +49,11 @@ type indexBlock struct { ext2 uint64 } -func (i indexBlock) Bin(pos int64) (storage.IndexBlockHeader, []byte) { +func (i indexBlock) bin(pos int64) (storage.IndexBlockHeader, []byte) { return storage.NewIndexBlockHeader(pos, i.ext1, i.ext2, uint32(len(i.payload)), i.rawLen, i.codec), i.payload } -type IndexSealer struct { +type IndexWriter struct { params common.SealParams buf1 []byte @@ -38,28 +64,28 @@ type IndexSealer struct { tokenTable token.Table } -func NewIndexSealer(params common.SealParams) *IndexSealer { - return &IndexSealer{ +func New(params common.SealParams) *IndexWriter { + return &IndexWriter{ params: params, buf1: make([]byte, 0, consts.RegularBlockSize), buf2: make([]byte, 0, consts.RegularBlockSize), } } -func (s *IndexSealer) LIDsTable() lids.Table { +func (s *IndexWriter) LIDsTable() lids.Table { return s.lidsTable } -func (s *IndexSealer) TokenTable() token.Table { +func (s *IndexWriter) TokenTable() token.Table { return s.tokenTable } -func (s *IndexSealer) IDsTable() seqids.Table { +func (s *IndexWriter) IDsTable() seqids.Table { return s.idsTable } // WriteOffsetsFile writes the .offsets file containing a single BlockOffsets block. -func (s *IndexSealer) WriteOffsetsFile(ws io.WriteSeeker, src Source) error { +func (s *IndexWriter) WriteOffsetsFile(ws io.WriteSeeker, src Source) error { w, err := newWriter(ws) if err != nil { return err @@ -78,14 +104,14 @@ func (s *IndexSealer) WriteOffsetsFile(ws io.WriteSeeker, src Source) error { return w.finalize() } -func (s *IndexSealer) WriteIDFile(ws io.WriteSeeker, src Source) error { +func (s *IndexWriter) WriteIDFile(ws io.WriteSeeker, src Source) error { w, err := newWriter(ws) if err != nil { return err } defer w.release() - for block, err := range seqBlockID(src.ID(), consts.IDsPerBlock) { + for block, err := range idBlock(src.ID(), consts.IDsPerBlock) { if err != nil { return err } @@ -106,7 +132,7 @@ func (s *IndexSealer) WriteIDFile(ws io.WriteSeeker, src Source) error { return w.finalize() } -func (s *IndexSealer) WriteTokenTriplet(tws, lws io.WriteSeeker, src Source) error { +func (s *IndexWriter) WriteTokenTriplet(tws, lws io.WriteSeeker, src Source) error { tw, err := newWriter(tws) if err != nil { return err @@ -119,19 +145,15 @@ func (s *IndexSealer) WriteTokenTriplet(tws, lws io.WriteSeeker, src Source) err } defer lw.release() - var ( - bb blocksBuilder - allFieldsTables []token.FieldTable - ) - lidAccumulator := newLIDAccumulator( consts.LIDBlockCap, - func(block lidsSealBlock) error { + func(block unpackedLIDBlock) error { return lw.writeBlock(blockTypeLID, s.packLIDsBlock(block)) }, ) - for pair, err := range bb.BuildTokenBlocks(src.TokenTriplet(), lidAccumulator.Add, consts.RegularBlockSize) { + var allFieldsTables []token.FieldTable + for pair, err := range tokenBlock(src.TokenTriplet(), lidAccumulator.add, consts.RegularBlockSize) { if err != nil { return err } @@ -150,15 +172,15 @@ func (s *IndexSealer) WriteTokenTriplet(tws, lws io.WriteSeeker, src Source) err return s.finalizeTokenFile(tw, allFieldsTables) } -func (s *IndexSealer) finalizeLIDFile(w *writer, lidAccumulator *lidAccumulator) error { - if err := lidAccumulator.Finalize(); err != nil { +func (s *IndexWriter) finalizeLIDFile(w *writer, lidAccumulator *lidAccumulator) error { + if err := lidAccumulator.finalize(); err != nil { return err } return w.finalize() } -func (s *IndexSealer) finalizeTokenFile(w *writer, allFieldsTables []token.FieldTable) error { +func (s *IndexWriter) finalizeTokenFile(w *writer, allFieldsTables []token.FieldTable) error { // Emit section separator. if err := w.writeEmptyBlock(); err != nil { return err @@ -172,39 +194,17 @@ func (s *IndexSealer) finalizeTokenFile(w *writer, allFieldsTables []token.Field return w.finalize() } -func (s *IndexSealer) WriteInfoFile(ws io.Writer, src Source) error { +func (s *IndexWriter) WriteInfoFile(ws io.Writer, src Source) error { block := sealed.BlockInfo{Info: src.Info()} _, err := ws.Write(s.packInfoBlock(block).payload) return err } -// collapseOrderedFieldsTables merges FieldTables with the same field name. -// Assumes input is sorted by Field. -func collapseOrderedFieldsTables(src []token.FieldTable) []token.FieldTable { - if len(src) == 0 { - return nil - } - - current := src[0] - var dst []token.FieldTable - for _, ft := range src[1:] { - if current.Field == ft.Field { - current.Entries = append(current.Entries, ft.Entries...) - continue - } - - dst = append(dst, current) - current = ft - } - - return append(dst, current) -} - func newIndexBlock(raw []byte) indexBlock { return indexBlock{codec: storage.CodecNo, rawLen: uint32(len(raw)), payload: raw} } -func (s *IndexSealer) newIndexBlockZSTD(raw []byte, level int) indexBlock { +func (s *IndexWriter) newIndexBlockZSTD(raw []byte, level int) indexBlock { s.buf2 = zstd.CompressLevel(raw, s.buf2[:0], level) if len(s.buf2) < len(raw) { return indexBlock{codec: storage.CodecZSTD, rawLen: uint32(len(raw)), payload: s.buf2} @@ -213,13 +213,13 @@ func (s *IndexSealer) newIndexBlockZSTD(raw []byte, level int) indexBlock { } // packInfoBlock packs fraction information into an index block. -func (s *IndexSealer) packInfoBlock(block sealed.BlockInfo) indexBlock { +func (s *IndexWriter) packInfoBlock(block sealed.BlockInfo) indexBlock { s.buf1 = block.Pack(s.buf1[:0]) return newIndexBlock(s.buf1) // Info block is typically small, no compression } // packTokenBlock packs token data into a compressed index block. -func (s *IndexSealer) packTokenBlock(block tokensSealBlock) indexBlock { +func (s *IndexWriter) packTokenBlock(block unpackedTokenBlock) indexBlock { s.buf1 = block.payload.Pack(s.buf1[:0]) // Pack token data b := s.newIndexBlockZSTD(s.buf1, s.params.TokenListZstdLevel) // Store TID range in extended metadata @@ -228,7 +228,7 @@ func (s *IndexSealer) packTokenBlock(block tokensSealBlock) indexBlock { } // packTokenTableBlock packs the token table into a compressed index block. -func (s *IndexSealer) packTokenTableBlock(tokenTableBlock token.TableBlock) indexBlock { +func (s *IndexWriter) packTokenTableBlock(tokenTableBlock token.TableBlock) indexBlock { s.tokenTable = token.TableFromBlocks([]token.TableBlock{tokenTableBlock}) // Store for PreloadedData // Packing block @@ -237,7 +237,7 @@ func (s *IndexSealer) packTokenTableBlock(tokenTableBlock token.TableBlock) inde } // packBlocksOffsetsBlock packs document block offsets into a compressed index block. -func (s *IndexSealer) packBlocksOffsetsBlock(block sealed.BlockOffsets) indexBlock { +func (s *IndexWriter) packBlocksOffsetsBlock(block sealed.BlockOffsets) indexBlock { // Update IDs table for PreloadedData s.idsTable.IDsTotal = block.IDsTotal // Total number of IDs s.idsTable.IDBlocksTotal = uint32(len(block.Offsets)) // Number of ID blocks @@ -249,7 +249,7 @@ func (s *IndexSealer) packBlocksOffsetsBlock(block sealed.BlockOffsets) indexBlo } // packMIDsBlock packs MIDs into a compressed index block. -func (s *IndexSealer) packMIDsBlock(block idsSealBlock) indexBlock { +func (s *IndexWriter) packMIDsBlock(block unpackedIDBlock) indexBlock { // Get the last ID in the block (smallest due to descending order) last := len(block.mids.Values) - 1 @@ -272,14 +272,14 @@ func (s *IndexSealer) packMIDsBlock(block idsSealBlock) indexBlock { } // packRIDsBlock packs RIDs into a compressed index block. -func (s *IndexSealer) packRIDsBlock(block idsSealBlock) indexBlock { +func (s *IndexWriter) packRIDsBlock(block unpackedIDBlock) indexBlock { s.buf1 = block.rids.Pack(s.buf1[:0]) b := s.newIndexBlockZSTD(s.buf1, s.params.IDsZstdLevel) return b } // packPosBlock packs document positions into a compressed index block. -func (s *IndexSealer) packPosBlock(block idsSealBlock) indexBlock { +func (s *IndexWriter) packPosBlock(block unpackedIDBlock) indexBlock { s.buf1 = block.params.Pack(s.buf1[:0]) b := s.newIndexBlockZSTD(s.buf1, s.params.IDsZstdLevel) return b @@ -287,7 +287,7 @@ func (s *IndexSealer) packPosBlock(block idsSealBlock) indexBlock { // packLIDsBlock packs Local IDs (LIDs) into a compressed index block. // Also updates LIDs table for preloaded data access. -func (s *IndexSealer) packLIDsBlock(block lidsSealBlock) indexBlock { +func (s *IndexWriter) packLIDsBlock(block unpackedLIDBlock) indexBlock { var ext1 uint64 if block.ext.isContinued { // todo: Legacy continuation flag ext1 = 1 diff --git a/indexwriter/lid_accumulator.go b/indexwriter/lid_accumulator.go new file mode 100644 index 00000000..311311ef --- /dev/null +++ b/indexwriter/lid_accumulator.go @@ -0,0 +1,85 @@ +package indexwriter + +import "github.com/ozontech/seq-db/frac/sealed/lids" + +type lidAccumulator struct { + blockCapacity int + onBlock func(unpackedLIDBlock) error + + currentTID uint32 + currentBlock unpackedLIDBlock + + isEndOfToken bool + isContinued bool +} + +func newLIDAccumulator( + blockCapacity int, + onBlock func(unpackedLIDBlock) error, +) *lidAccumulator { + a := &lidAccumulator{ + blockCapacity: blockCapacity, + onBlock: onBlock, + } + + a.currentBlock.ext.minTID = 1 + a.currentBlock.payload = lids.Block{ + LIDs: make([]uint32, 0, blockCapacity), + Offsets: []uint32{0}, + } + + return a +} + +// add processes LIDs of one token (must be called in TID order). +// +// For each block that fills up, `onBlock` is called immediately +// before the backing arrays are reset, so `onBlock` may read the +// block data but must not retain references to it. +func (a *lidAccumulator) add(lidsbuf []uint32) error { + a.currentTID++ + + for _, lid := range lidsbuf { + if len(a.currentBlock.payload.LIDs) == a.blockCapacity { + if err := a.onBlock(a.finalizeBlock()); err != nil { + return err + } + + a.currentBlock.ext.minTID = a.currentTID + a.currentBlock.payload.LIDs = a.currentBlock.payload.LIDs[:0] + a.currentBlock.payload.Offsets = a.currentBlock.payload.Offsets[:1] + } + + a.isEndOfToken = false + a.currentBlock.ext.maxTID = a.currentTID + a.currentBlock.payload.LIDs = append(a.currentBlock.payload.LIDs, lid) + } + + a.isEndOfToken = true + a.currentBlock.payload.Offsets = append( + a.currentBlock.payload.Offsets, + uint32(len(a.currentBlock.payload.LIDs)), + ) + + return nil +} + +func (a *lidAccumulator) finalize() error { + return a.onBlock(a.finalizeBlock()) +} + +func (a *lidAccumulator) finalizeBlock() unpackedLIDBlock { + if !a.isEndOfToken { + a.currentBlock.payload.Offsets = append( + a.currentBlock.payload.Offsets, + uint32(len(a.currentBlock.payload.LIDs)), + ) + } + + result := a.currentBlock + result.payload.IsLastLID = a.isEndOfToken + result.ext.isContinued = a.isContinued + + a.isContinued = !a.isEndOfToken + return result +} diff --git a/frac/sealed/sealing/writer.go b/indexwriter/writer.go similarity index 96% rename from frac/sealed/sealing/writer.go rename to indexwriter/writer.go index 1a147e4e..7746c1db 100644 --- a/frac/sealed/sealing/writer.go +++ b/indexwriter/writer.go @@ -1,4 +1,4 @@ -package sealing +package indexwriter import ( "bytes" @@ -73,7 +73,7 @@ func newWriter(ws io.WriteSeeker) (*writer, error) { } func (w *writer) writeBlock(btype string, block indexBlock) error { - header, payload := block.Bin(int64(w.pos)) + header, payload := block.bin(int64(w.pos)) if _, err := w.wpayload.Write(payload); err != nil { return err } @@ -92,7 +92,7 @@ func (w *writer) writeBlock(btype string, block indexBlock) error { } func (w *writer) writeEmptyBlock() error { - header, _ := indexBlock{}.Bin(int64(w.pos)) + header, _ := indexBlock{}.bin(int64(w.pos)) w.wheader.Write(header) return nil } diff --git a/frac/sealed/sealing/sealer.go b/sealing/sealer.go similarity index 61% rename from frac/sealed/sealing/sealer.go rename to sealing/sealer.go index 57863d82..0c21ffc4 100644 --- a/frac/sealed/sealing/sealer.go +++ b/sealing/sealer.go @@ -2,41 +2,19 @@ package sealing import ( "errors" - "iter" "os" "path/filepath" "github.com/ozontech/seq-db/consts" "github.com/ozontech/seq-db/frac/common" "github.com/ozontech/seq-db/frac/sealed" - "github.com/ozontech/seq-db/seq" + "github.com/ozontech/seq-db/indexwriter" "github.com/ozontech/seq-db/util" ) -type ( - DocLocation = util.Pair[seq.ID, seq.DocPos] - TokenPosting = util.Pair[[]byte, []uint32] -) - -// Source interface defines the contract for data sources that can be sealed. -// Provides access to all necessary data components for index creation -type Source interface { - // Info returns metadata describing this source. - Info() *common.Info - - // ID returns an iterator over stored document identifiers paired with - // their positions, in descending [seq.ID] order. - ID() iter.Seq2[DocLocation, error] - - // BlockOffsets returns byte offsets to each document block - // within this source's `.docs` file. - BlockOffsets() []uint64 - - // TokenTriplet iterates over fields in lexicographic order. - // For each field, it yields tokens (lexicographically sorted) - // paired with the local document ID list for that token. - TokenTriplet() iter.Seq2[string, iter.Seq2[TokenPosting, error]] -} +// Source defines the contract for data sources that can be sealed. +// Provides access to all necessary data components for index creation. +type Source = indexwriter.Source // Seal writes five index files (.info, .token, .offsets, .id, .lid) for the fraction // and returns PreloadedData for fast initialization of the sealed fraction. @@ -47,12 +25,11 @@ func Seal(src Source, params common.SealParams) (*sealed.PreloadedData, error) { return nil, errors.New("sealing of an empty active fraction is not supported") } - sealer := NewIndexSealer(params) - + w := indexwriter.New(params) if err := createAndWrite( info.Path+consts.OffsetsTmpFileSuffix, info.Path+consts.OffsetsFileSuffix, - func(f *os.File) error { return sealer.WriteOffsetsFile(f, src) }, + func(f *os.File) error { return w.WriteOffsetsFile(f, src) }, ); err != nil { return nil, err } @@ -60,7 +37,7 @@ func Seal(src Source, params common.SealParams) (*sealed.PreloadedData, error) { if err := createAndWrite( info.Path+consts.IDTmpFileSuffix, info.Path+consts.IDFileSuffix, - func(f *os.File) error { return sealer.WriteIDFile(f, src) }, + func(f *os.File) error { return w.WriteIDFile(f, src) }, ); err != nil { return nil, err } @@ -68,7 +45,7 @@ func Seal(src Source, params common.SealParams) (*sealed.PreloadedData, error) { if err := createAndWriteBoth( info.Path+consts.TokenTmpFileSuffix, info.Path+consts.TokenFileSuffix, info.Path+consts.LIDTmpFileSuffix, info.Path+consts.LIDFileSuffix, - func(tokenF, lidF *os.File) error { return sealer.WriteTokenTriplet(tokenF, lidF, src) }, + func(tokenF, lidF *os.File) error { return w.WriteTokenTriplet(tokenF, lidF, src) }, ); err != nil { return nil, err } @@ -76,7 +53,7 @@ func Seal(src Source, params common.SealParams) (*sealed.PreloadedData, error) { if err := createAndWrite( info.Path+consts.InfoTmpFileSuffix, info.Path+consts.InfoFileSuffix, - func(f *os.File) error { return sealer.WriteInfoFile(f, src) }, + func(f *os.File) error { return w.WriteInfoFile(f, src) }, ); err != nil { return nil, err } @@ -100,13 +77,13 @@ func Seal(src Source, params common.SealParams) (*sealed.PreloadedData, error) { } info.IndexOnDisk = totalSize - lidsTable := sealer.LIDsTable() + lidsTable := w.LIDsTable() preloaded := &sealed.PreloadedData{ Info: info, - TokenTable: sealer.TokenTable(), + TokenTable: w.TokenTable(), BlocksData: sealed.BlocksData{ - IDsTable: sealer.IDsTable(), + IDsTable: w.IDsTable(), LIDsTable: &lidsTable, BlocksOffsets: src.BlockOffsets(), }, @@ -123,10 +100,7 @@ func syncAndClose(f *os.File) error { return f.Close() } -func createAndWrite( - tmp, final string, - write func(*os.File) error, -) error { +func createAndWrite(tmp, final string, write func(*os.File) error) error { f, err := os.Create(tmp) if err != nil { return err @@ -140,16 +114,16 @@ func createAndWrite( } func createAndWriteBoth( - tmpa, finala, - tmpb, finalb string, + atmp, afinal, + btmp, bfinal string, write func(*os.File, *os.File) error, ) error { - a, err := os.Create(tmpa) + a, err := os.Create(atmp) if err != nil { return err } - b, err := os.Create(tmpb) + b, err := os.Create(btmp) if err != nil { a.Close() return err @@ -160,9 +134,9 @@ func createAndWriteBoth( return err } - if err := os.Rename(tmpa, finala); err != nil { + if err := os.Rename(atmp, afinal); err != nil { return err } - return os.Rename(tmpb, finalb) + return os.Rename(btmp, bfinal) } diff --git a/seq/seq.go b/seq/seq.go index adae4265..d3557a16 100644 --- a/seq/seq.go +++ b/seq/seq.go @@ -11,9 +11,13 @@ import ( ) var ( - SystemMID MID = math.MaxUint64 - SystemRID RID = math.MaxUint64 - SystemID ID = ID{SystemMID, SystemRID} + SystemMID MID = math.MaxUint64 + SystemRID RID = math.MaxUint64 + + SystemID ID = ID{SystemMID, SystemRID} + MinID ID = ID{0, 0} + MaxID ID = SystemID + SystemDocPos DocPos = DocPos(0) ) diff --git a/skipmaskmanager/skip_mask_manager.go b/skipmaskmanager/skip_mask_manager.go index 79f6680d..40f48480 100644 --- a/skipmaskmanager/skip_mask_manager.go +++ b/skipmaskmanager/skip_mask_manager.go @@ -45,7 +45,7 @@ type MappingProvider interface { } type fractionAcquirer interface { - Fractions() fracmanager.List + AcquireFractions() (fracmanager.List, func()) AcquireFraction(name string) (_ frac.Fraction, release func(), ok bool) } @@ -142,7 +142,7 @@ func New( // - Begins asynchronous processing of all skip mask queries // // This method must be called before using the manager. -func (smm *SkipMaskManager) Start(fracs fractionAcquirer) { +func (smm *SkipMaskManager) Start(fracProvider fractionAcquirer) { smm.createDataDir() err := smm.loadSkipMasks() @@ -150,7 +150,10 @@ func (smm *SkipMaskManager) Start(fracs fractionAcquirer) { logger.Fatal("failed to load previous skip masks", zap.Error(err)) } - err = smm.buildQueue(fracs.Fractions()) + fracs, release := fracProvider.AcquireFractions() + defer release() + + err = smm.buildQueue(fracs) if err != nil { logger.Fatal("failed to build skip mask manager queue", zap.Error(err)) } @@ -171,7 +174,7 @@ func (smm *SkipMaskManager) Start(fracs fractionAcquirer) { } sm.ast = ast - smm.processSkipMask(sm, fracs) + smm.processSkipMask(sm, fracProvider) } }() } @@ -439,7 +442,7 @@ func (smm *SkipMaskManager) buildQueue(fracs fracmanager.List) error { // It processes each fraction with a .queue file, running search queries in parallel // (limited by the rate limiter). Each successful search writes results to a .skipmask // file. The skip mask status is set to Done when all fractions are processed. -func (smm *SkipMaskManager) processSkipMask(skipMask *SkipMask, fracs fractionAcquirer) { +func (smm *SkipMaskManager) processSkipMask(skipMask *SkipMask, fracProvider fractionAcquirer) { skipMaskDes, err := os.ReadDir(skipMask.dirPath) if err != nil { panic(fmt.Errorf("BUG: reading directory must be successful: %s", err)) @@ -457,7 +460,7 @@ func (smm *SkipMaskManager) processSkipMask(skipMask *SkipMask, fracs fractionAc defer skipMask.processWg.Done() defer func() { <-smm.rateLimit }() - f, release, ok := fracs.AcquireFraction(fracNameFromFilePath(name)) + f, release, ok := fracProvider.AcquireFraction(fracNameFromFilePath(name)) if !ok { // skip missing fracs return } diff --git a/storeapi/grpc_fetch.go b/storeapi/grpc_fetch.go index d640618c..9eb14147 100644 --- a/storeapi/grpc_fetch.go +++ b/storeapi/grpc_fetch.go @@ -68,7 +68,10 @@ func (g *GrpcV1) doFetch(ctx context.Context, req *storeapi.FetchRequest, stream dp := acquireDocFieldsFilter(req.FieldsFilter) defer releaseDocFieldsFilter(dp) - docsStream := newDocsStream(ctx, ids, g.fetchData.docFetcher, g.fracManager.Fractions()) + fracs, release := g.fracManager.AcquireFractions() + defer release() + + docsStream := newDocsStream(ctx, ids, g.fetchData.docFetcher, fracs) for _, id := range ids { workTime := time.Now() doc, err := docsStream.Next() diff --git a/storeapi/grpc_search.go b/storeapi/grpc_search.go index 9eb89e73..e5eedd98 100644 --- a/storeapi/grpc_search.go +++ b/storeapi/grpc_search.go @@ -189,18 +189,13 @@ func (g *GrpcV1) doSearch( } searchTr := tr.NewChild("search iteratively") - qpr, err := g.searchData.searcher.SearchDocs( - ctx, - g.fracManager.Fractions(), - searchParams, - tr, - ) + qpr, err := g.searchDocs(ctx, searchParams, tr) searchTr.Done() + if err != nil { if code, ok := parseStoreError(err); ok { return &storeapi.SearchResponse{Code: code}, nil } - return nil, err } @@ -229,6 +224,13 @@ func (g *GrpcV1) doSearch( return buildSearchResponse(qpr), nil } +func (g *GrpcV1) searchDocs(ctx context.Context, sp processor.SearchParams, tr *querytracer.Tracer) (*seq.QPR, error) { + fracs, release := g.fracManager.AcquireFractions() + defer release() + + return g.searchData.searcher.SearchDocs(ctx, fracs, sp, tr) +} + func (g *GrpcV1) parseQuery(query string) (*parser.ASTNode, error) { seqql, err := parser.ParseSeqQL(query, g.mappingProvider.GetMapping()) if err != nil { diff --git a/storeapi/store.go b/storeapi/store.go index dd53079e..c3ee0aca 100644 --- a/storeapi/store.go +++ b/storeapi/store.go @@ -8,6 +8,7 @@ import ( "go.uber.org/atomic" + "github.com/ozontech/seq-db/compaction" "github.com/ozontech/seq-db/consts" "github.com/ozontech/seq-db/fracmanager" "github.com/ozontech/seq-db/logger" @@ -31,6 +32,7 @@ type Store struct { fracManagerStop func() SkipMaskManager *skipmaskmanager.SkipMaskManager + Executor *compaction.Executor isStopped atomic.Bool } @@ -66,12 +68,14 @@ func NewStore( } skipMaskManager := skipmaskmanager.New(ctx, c.SkipMaskManagerConfig, skipMaskParams, mappingProvider) - fracManager, stop, err := fracmanager.New(ctx, &c.FracManager, s3cli, skipMaskManager) if err != nil { return nil, fmt.Errorf("loading fractions error: %w", err) } + planner := compaction.NewPlanner(ctx, fracManager) + executor := compaction.NewExecutor(10, planner) + skipMaskManager.Start(fracManager) return &Store{ @@ -82,6 +86,7 @@ func NewStore( FracManager: fracManager, fracManagerStop: stop, SkipMaskManager: skipMaskManager, + Executor: executor, isStopped: atomic.Bool{}, }, nil } @@ -107,6 +112,7 @@ func (s *Store) Stop() { s.grpcServer.Stop(ctx) s.fracManagerStop() s.SkipMaskManager.Stop() + s.Executor.Close() logger.Info("store stopped") } diff --git a/util/min_heap.go b/util/min_heap.go new file mode 100644 index 00000000..70b92fcf --- /dev/null +++ b/util/min_heap.go @@ -0,0 +1,117 @@ +package util + +import ( + "cmp" + "container/heap" +) + +// MinHeap is a min‑heap for any comparable type. +// Maintains both a heap structure and a map for fast lookup of items. +type MinHeap[T cmp.Ordered] struct { + items []*heapItem[T] // Heap elements + indexMap map[T]*heapItem[T] // Value → item mapping for O(1) lookup +} + +// heapItem represents an element in the heap. +type heapItem[T comparable] struct { + value T // Stored value + index int // Current index in the heap +} + +// NewMinHeap creates and initializes a new MinHeap instance. +func NewMinHeap[T cmp.Ordered]() *MinHeap[T] { + h := &MinHeap[T]{ + items: make([]*heapItem[T], 0), + indexMap: make(map[T]*heapItem[T]), + } + heap.Init((*heapWrapper[T])(h)) + return h +} + +// Push adds a value to the heap if it doesn't already exist (no duplicates). +func (h *MinHeap[T]) Push(value T) { + if _, ok := h.indexMap[value]; !ok { + item := &heapItem[T]{ + value: value, + index: -1, + } + h.indexMap[value] = item + heap.Push((*heapWrapper[T])(h), item) + } +} + +// Remove deletes one occurrence of the specified value from the heap. +// Does nothing if the value doesn't exist. +func (h *MinHeap[T]) Remove(value T) { + item, ok := h.indexMap[value] + if !ok { + return + } + heap.Remove((*heapWrapper[T])(h), item.index) + delete(h.indexMap, value) +} + +// PopMin removes and returns the minimum value from the heap. +// Returns (zero value, false) if the heap is empty. +func (h *MinHeap[T]) PopMin() (T, bool) { + var zero T + if len(h.items) == 0 { + return zero, false + } + item := h.items[0] + value := item.value + heap.Pop((*heapWrapper[T])(h)) + return value, true +} + +// Min returns the minimum value in the heap without removing it. +// Returns (zero value, false) if the heap is empty. +func (h *MinHeap[T]) Min() (T, bool) { + var zero T + if len(h.items) == 0 { + return zero, false + } + return h.items[0].value, true +} + +// Len returns the current number of elements in the heap. +func (h *MinHeap[T]) Len() int { + return len(h.items) +} + +// heapWrapper is a type alias for MinHeap to implement heap.Interface. +type heapWrapper[T cmp.Ordered] MinHeap[T] + +// Len is part of heap.Interface — returns the number of elements. +func (hw *heapWrapper[T]) Len() int { + return len(hw.items) +} + +// Less is part of heap.Interface — defines min‑heap order (smaller values first). +func (hw *heapWrapper[T]) Less(i, j int) bool { + return hw.items[i].value < hw.items[j].value +} + +// Swap is part of heap.Interface — swaps elements and updates their indices. +func (hw *heapWrapper[T]) Swap(i, j int) { + hw.items[i], hw.items[j] = hw.items[j], hw.items[i] + hw.items[i].index = i + hw.items[j].index = j +} + +// Push is part of heap.Interface — adds a new element to the heap. +func (hw *heapWrapper[T]) Push(x any) { + item := x.(*heapItem[T]) + item.index = len(hw.items) + hw.items = append(hw.items, item) +} + +// Pop is part of heap.Interface — removes and returns the last element. +func (hw *heapWrapper[T]) Pop() any { + old := hw.items + n := len(old) - 1 + item := old[n] + item.index = -1 + hw.items = old[0:n] + return item +}