From 86c5e586e9f615dbb82ed2dbcdba8df711f93ad9 Mon Sep 17 00:00:00 2001 From: steiler Date: Thu, 2 Oct 2025 22:04:01 +0200 Subject: [PATCH 01/44] Importer: Use a WorkerPool to import data into the tree --- pkg/tree/entry.go | 2 + pkg/tree/parallelImporter.go | 146 +++++++++++++++++++ pkg/tree/sharedEntryAttributes.go | 224 +++++++++++++++--------------- pkg/utils/pool.go | 154 ++++++++++++++++++++ 4 files changed, 415 insertions(+), 111 deletions(-) create mode 100644 pkg/tree/parallelImporter.go create mode 100644 pkg/utils/pool.go diff --git a/pkg/tree/entry.go b/pkg/tree/entry.go index 2f2bbf83..47339774 100644 --- a/pkg/tree/entry.go +++ b/pkg/tree/entry.go @@ -115,6 +115,7 @@ type Entry interface { // - shouldDelete() returns false, because no explicit delete should be issued for them. canDelete() bool GetChilds(DescendMethod) EntryMap + GetChild(name string) (Entry, bool) // entry, exists FilterChilds(keys map[string]string) ([]Entry, error) // ToJson returns the Tree contained structure as JSON // use e.g. json.MarshalIndent() on the returned struct @@ -166,6 +167,7 @@ type LeafVariantEntries interface { DeleteByOwner(owner string) *LeafEntry AddExplicitDeleteEntry(owner string, priority int32) *LeafEntry GetByOwner(owner string) *LeafEntry + Add(l *LeafEntry) } type DescendMethod int diff --git a/pkg/tree/parallelImporter.go b/pkg/tree/parallelImporter.go new file mode 100644 index 00000000..465291ea --- /dev/null +++ b/pkg/tree/parallelImporter.go @@ -0,0 +1,146 @@ +package tree + +import ( + "context" + "fmt" + "runtime" + "slices" + + "github.com/sdcio/data-server/pkg/tree/importer" + "github.com/sdcio/data-server/pkg/tree/types" + "github.com/sdcio/data-server/pkg/utils" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" + "google.golang.org/protobuf/types/known/emptypb" +) + +type importTask struct { + entry Entry + importerElement importer.ImportConfigAdapterElement + intentName string + intentPrio int32 + insertFlags *types.UpdateInsertFlags + treeContext *TreeContext +} + +func (s *sharedEntryAttributes) ImportConfig( + ctx context.Context, + importerElement importer.ImportConfigAdapterElement, + intentName string, + intentPrio int32, + insertFlags *types.UpdateInsertFlags, +) error { + p := utils.NewWorkerPool[importTask](ctx, runtime.NumCPU(), 900000) + + p.Start(importHandler) + + // seed root + if err := p.Submit(importTask{entry: s, importerElement: importerElement, intentName: intentName, intentPrio: intentPrio, insertFlags: insertFlags, treeContext: s.treeContext}); err != nil { + p.Close() + return err + } + + // signal we are done seeding external tasks (workers may still submit) + p.CloseForSubmit() + + // wait for the import to finish (or error) + return p.Wait() +} + +func importHandler(ctx context.Context, task importTask, submit func(importTask) error) error { + switch x := task.entry.GetSchema().GetSchema().(type) { + case *sdcpb.SchemaElem_Container, nil: + // keyed container: handle keys sequentially + if len(task.entry.GetSchema().GetContainer().GetKeys()) > 0 { + var exists bool + var actual Entry = task.entry + var keyChild Entry + + keys := task.entry.GetSchemaKeys() + slices.Sort(keys) + for _, k := range keys { + ktrans := task.importerElement.GetElement(k) + if ktrans == nil { + return fmt.Errorf("unable to find key attribute %s under %s", k, task.entry.SdcpbPath().ToXPath(false)) + } + kv, err := ktrans.GetKeyValue() + if err != nil { + return err + } + if keyChild, exists = actual.GetChild(kv); !exists { + keyChild, err = newEntry(ctx, actual, kv, task.treeContext) + if err != nil { + return err + } + } + actual = keyChild + } + // submit resolved entry with same adapter element + return submit(importTask{entry: actual, importerElement: task.importerElement, intentName: task.intentName, intentPrio: task.intentPrio, insertFlags: task.insertFlags, treeContext: task.treeContext}) + } + + // presence container or children + elems := task.importerElement.GetElements() + if len(elems) == 0 { + schem := task.entry.GetSchema().GetContainer() + if schem != nil && schem.IsPresence { + tv := &sdcpb.TypedValue{Value: &sdcpb.TypedValue_EmptyVal{EmptyVal: &emptypb.Empty{}}} + upd := types.NewUpdate(task.entry.SdcpbPath(), tv, task.intentPrio, task.intentName, 0) + task.entry.GetLeafVariantEntries().Add(NewLeafEntry(upd, task.insertFlags, task.entry)) + } + return nil + } + + // submit each child (no external locking per your guarantee) + for _, childElt := range elems { + child, exists := task.entry.GetChild(childElt.GetName()) + if !exists { + var err error + child, err = newEntry(ctx, task.entry, childElt.GetName(), task.treeContext) + if err != nil { + return fmt.Errorf("error inserting %s at %s: %w", childElt.GetName(), task.entry.SdcpbPath().ToXPath(false), err) + } + } + if err := submit(importTask{entry: child, importerElement: childElt, intentName: task.intentName, intentPrio: task.intentPrio, insertFlags: task.insertFlags, treeContext: task.treeContext}); err != nil { + return err + } + } + return nil + + case *sdcpb.SchemaElem_Field: + tv, err := task.importerElement.GetTVValue(x.Field.GetType()) + if err != nil { + return err + } + upd := types.NewUpdate(task.entry.SdcpbPath(), tv, task.intentPrio, task.intentName, 0) + task.entry.GetLeafVariantEntries().Add(NewLeafEntry(upd, task.insertFlags, task.entry)) + return nil + + case *sdcpb.SchemaElem_Leaflist: + var scalarArr *sdcpb.ScalarArray + mustAdd := false + le := task.entry.GetLeafVariantEntries().GetByOwner(task.intentName) + if le != nil { + scalarArr = le.Value().GetLeaflistVal() + } else { + le = NewLeafEntry(nil, task.insertFlags, task.entry) + mustAdd = true + scalarArr = &sdcpb.ScalarArray{Element: []*sdcpb.TypedValue{}} + } + + tv, err := task.importerElement.GetTVValue(x.Leaflist.GetType()) + if err != nil { + return err + } + if tv.GetLeaflistVal() == nil { + scalarArr.Element = append(scalarArr.Element, tv) + tv = &sdcpb.TypedValue{Value: &sdcpb.TypedValue_LeaflistVal{LeaflistVal: scalarArr}} + } + le.Update = types.NewUpdate(task.entry.SdcpbPath(), tv, task.intentPrio, task.intentName, 0) + if mustAdd { + task.entry.GetLeafVariantEntries().Add(le) + } + return nil + default: + return nil + } +} diff --git a/pkg/tree/sharedEntryAttributes.go b/pkg/tree/sharedEntryAttributes.go index 1b3b35fb..268b5c13 100644 --- a/pkg/tree/sharedEntryAttributes.go +++ b/pkg/tree/sharedEntryAttributes.go @@ -13,13 +13,11 @@ import ( "unicode/utf8" "github.com/sdcio/data-server/pkg/config" - "github.com/sdcio/data-server/pkg/tree/importer" "github.com/sdcio/data-server/pkg/tree/types" "github.com/sdcio/data-server/pkg/utils" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "github.com/sdcio/sdc-protos/tree_persist" log "github.com/sirupsen/logrus" - "google.golang.org/protobuf/types/known/emptypb" ) // sharedEntryAttributes contains the attributes shared by Entry and RootEntry @@ -1223,118 +1221,118 @@ func (s *sharedEntryAttributes) validatePattern(resultChan chan<- *types.Validat } } -func (s *sharedEntryAttributes) ImportConfig(ctx context.Context, t importer.ImportConfigAdapterElement, intentName string, intentPrio int32, insertFlags *types.UpdateInsertFlags) error { - var err error - - switch x := s.schema.GetSchema().(type) { - case *sdcpb.SchemaElem_Container, nil: - switch { - case len(s.schema.GetContainer().GetKeys()) > 0: - - var exists bool - var actualEntry Entry = s - var keyChild Entry - schemaKeys := s.GetSchemaKeys() - slices.Sort(schemaKeys) - for _, schemaKey := range schemaKeys { - - keyTransf := t.GetElement(schemaKey) - if keyTransf == nil { - return fmt.Errorf("unable to find key attribute %s under %s", schemaKey, s.SdcpbPath().ToXPath(false)) - } - keyElemValue, err := keyTransf.GetKeyValue() - if err != nil { - return err - } - // if the child does not exist, create it - if keyChild, exists = actualEntry.GetChilds(DescendMethodAll)[keyElemValue]; !exists { - keyChild, err = newEntry(ctx, actualEntry, keyElemValue, s.treeContext) - if err != nil { - return err - } - } - actualEntry = keyChild - } - err = actualEntry.ImportConfig(ctx, t, intentName, intentPrio, insertFlags) - if err != nil { - return err - } - default: - if len(t.GetElements()) == 0 { - // it might be a presence container - schem := s.schema.GetContainer() - if schem == nil { - return nil - } - if schem.IsPresence { - tv := &sdcpb.TypedValue{Value: &sdcpb.TypedValue_EmptyVal{EmptyVal: &emptypb.Empty{}}} - upd := types.NewUpdate(s.SdcpbPath(), tv, intentPrio, intentName, 0) - s.leafVariants.Add(NewLeafEntry(upd, insertFlags, s)) - } - } - - for _, elem := range t.GetElements() { - var child Entry - var exists bool - - // if the child does not exist, create it - if child, exists = s.getChildren()[elem.GetName()]; !exists { - child, err = newEntry(ctx, s, elem.GetName(), s.treeContext) - if err != nil { - return fmt.Errorf("error trying to insert %s at path %s: %w", elem.GetName(), s.SdcpbPath().ToXPath(false), err) - } - } - err = child.ImportConfig(ctx, elem, intentName, intentPrio, insertFlags) - if err != nil { - return err - } - } - } - case *sdcpb.SchemaElem_Field: - // // if it is as leafref we need to figure out the type of the references field. - fieldType := x.Field.GetType() - // if x.Field.GetType().Type == "leafref" { - // s.treeContext.treeSchemaCacheClient.GetSchema(ctx,) - // } - - tv, err := t.GetTVValue(fieldType) - if err != nil { - return err - } - upd := types.NewUpdate(s.SdcpbPath(), tv, intentPrio, intentName, 0) - - s.leafVariants.Add(NewLeafEntry(upd, insertFlags, s)) - - case *sdcpb.SchemaElem_Leaflist: - var scalarArr *sdcpb.ScalarArray - mustAdd := false - le := s.leafVariants.GetByOwner(intentName) - if le != nil { - scalarArr = le.Value().GetLeaflistVal() - } else { - le = NewLeafEntry(nil, insertFlags, s) - mustAdd = true - scalarArr = &sdcpb.ScalarArray{Element: []*sdcpb.TypedValue{}} - } +// func (s *sharedEntryAttributes) ImportConfig(ctx context.Context, t importer.ImportConfigAdapterElement, intentName string, intentPrio int32, insertFlags *types.UpdateInsertFlags) error { +// var err error + +// switch x := s.schema.GetSchema().(type) { +// case *sdcpb.SchemaElem_Container, nil: +// switch { +// case len(s.schema.GetContainer().GetKeys()) > 0: + +// var exists bool +// var actualEntry Entry = s +// var keyChild Entry +// schemaKeys := s.GetSchemaKeys() +// slices.Sort(schemaKeys) +// for _, schemaKey := range schemaKeys { + +// keyTransf := t.GetElement(schemaKey) +// if keyTransf == nil { +// return fmt.Errorf("unable to find key attribute %s under %s", schemaKey, s.SdcpbPath().ToXPath(false)) +// } +// keyElemValue, err := keyTransf.GetKeyValue() +// if err != nil { +// return err +// } +// // if the child does not exist, create it +// if keyChild, exists = actualEntry.GetChilds(DescendMethodAll)[keyElemValue]; !exists { +// keyChild, err = newEntry(ctx, actualEntry, keyElemValue, s.treeContext) +// if err != nil { +// return err +// } +// } +// actualEntry = keyChild +// } +// err = actualEntry.ImportConfig(ctx, t, intentName, intentPrio, insertFlags) +// if err != nil { +// return err +// } +// default: +// if len(t.GetElements()) == 0 { +// // it might be a presence container +// schem := s.schema.GetContainer() +// if schem == nil { +// return nil +// } +// if schem.IsPresence { +// tv := &sdcpb.TypedValue{Value: &sdcpb.TypedValue_EmptyVal{EmptyVal: &emptypb.Empty{}}} +// upd := types.NewUpdate(s.SdcpbPath(), tv, intentPrio, intentName, 0) +// s.leafVariants.Add(NewLeafEntry(upd, insertFlags, s)) +// } +// } + +// for _, elem := range t.GetElements() { +// var child Entry +// var exists bool + +// // if the child does not exist, create it +// if child, exists = s.getChildren()[elem.GetName()]; !exists { +// child, err = newEntry(ctx, s, elem.GetName(), s.treeContext) +// if err != nil { +// return fmt.Errorf("error trying to insert %s at path %s: %w", elem.GetName(), s.SdcpbPath().ToXPath(false), err) +// } +// } +// err = child.ImportConfig(ctx, elem, intentName, intentPrio, insertFlags) +// if err != nil { +// return err +// } +// } +// } +// case *sdcpb.SchemaElem_Field: +// // // if it is as leafref we need to figure out the type of the references field. +// fieldType := x.Field.GetType() +// // if x.Field.GetType().Type == "leafref" { +// // s.treeContext.treeSchemaCacheClient.GetSchema(ctx,) +// // } + +// tv, err := t.GetTVValue(fieldType) +// if err != nil { +// return err +// } +// upd := types.NewUpdate(s.SdcpbPath(), tv, intentPrio, intentName, 0) + +// s.leafVariants.Add(NewLeafEntry(upd, insertFlags, s)) + +// case *sdcpb.SchemaElem_Leaflist: +// var scalarArr *sdcpb.ScalarArray +// mustAdd := false +// le := s.leafVariants.GetByOwner(intentName) +// if le != nil { +// scalarArr = le.Value().GetLeaflistVal() +// } else { +// le = NewLeafEntry(nil, insertFlags, s) +// mustAdd = true +// scalarArr = &sdcpb.ScalarArray{Element: []*sdcpb.TypedValue{}} +// } - tv, err := t.GetTVValue(x.Leaflist.GetType()) - if err != nil { - return err - } +// tv, err := t.GetTVValue(x.Leaflist.GetType()) +// if err != nil { +// return err +// } - // the proto implementation will return leaflist tvs - if tv.GetLeaflistVal() == nil { - scalarArr.Element = append(scalarArr.Element, tv) - tv = &sdcpb.TypedValue{Value: &sdcpb.TypedValue_LeaflistVal{LeaflistVal: scalarArr}} - } +// // the proto implementation will return leaflist tvs +// if tv.GetLeaflistVal() == nil { +// scalarArr.Element = append(scalarArr.Element, tv) +// tv = &sdcpb.TypedValue{Value: &sdcpb.TypedValue_LeaflistVal{LeaflistVal: scalarArr}} +// } - le.Update = types.NewUpdate(s.SdcpbPath(), tv, intentPrio, intentName, 0) - if mustAdd { - s.leafVariants.Add(le) - } - } - return nil -} +// le.Update = types.NewUpdate(s.SdcpbPath(), tv, intentPrio, intentName, 0) +// if mustAdd { +// s.leafVariants.Add(le) +// } +// } +// return nil +// } // validateMandatory validates that all the mandatory attributes, // defined by the schema are present either in the tree or in the index. @@ -1532,6 +1530,10 @@ func (s *sharedEntryAttributes) populateChoiceCaseResolvers(_ context.Context) e return nil } +func (s *sharedEntryAttributes) GetChild(name string) (Entry, bool) { + return s.childs.GetEntry(name) +} + func (s *sharedEntryAttributes) GetChilds(d DescendMethod) EntryMap { if s.schema == nil { return s.childs.GetAll() diff --git a/pkg/utils/pool.go b/pkg/utils/pool.go new file mode 100644 index 00000000..a700bca0 --- /dev/null +++ b/pkg/utils/pool.go @@ -0,0 +1,154 @@ +package utils + +import ( + "context" + "errors" + "runtime" + "sync" + "sync/atomic" +) + +// Pool[T] is a channel-driven worker pool that tracks pending tasks via a WaitGroup. +// Submit increments the pending WaitGroup before enqueuing; workers call pending.Done() +// after processing a task. Caller must call CloseForSubmit() once when external seeding is finished. +type Pool[T any] struct { + tasks chan T + workerCount int + + ctx context.Context + cancel context.CancelFunc + + workersWg sync.WaitGroup // wait for worker goroutines to exit + pending sync.WaitGroup // counts submitted-but-not-done tasks + + closeOnce sync.Once + + firstErr atomic.Pointer[error] + + closedForSubmit atomic.Bool +} + +// NewWorkerPool creates a new Pool. If workerCount <= 0 it defaults to runtime.NumCPU(). +// buf is the internal task-channel buffer size (if <=0 defaults to workerCount). +func NewWorkerPool[T any](parent context.Context, workerCount, buf int) *Pool[T] { + if workerCount <= 0 { + workerCount = runtime.NumCPU() + } + if buf <= 0 { + buf = workerCount + } + ctx, cancel := context.WithCancel(parent) + return &Pool[T]{ + tasks: make(chan T, buf), + workerCount: workerCount, + ctx: ctx, + cancel: cancel, + } +} + +// Submit enqueues a task. It increments the pending WaitGroup BEFORE attempting to send. +// If ctx is already cancelled, Submit returns ctx.Err() and does NOT increment pending. +func (p *Pool[T]) Submit(item T) error { + // fast-fail if canceled + if p.ctx.Err() != nil { + return p.ctx.Err() + } + + // account for this task first + p.pending.Add(1) + + // try to send; if ctx cancelled while sending, balance the pending counter + select { + case p.tasks <- item: + return nil + case <-p.ctx.Done(): + p.pending.Done() // balance + return p.ctx.Err() + } + +} + +// Start spawns workerCount workers that call handler(ctx, item, submit). +// Handler should process the item and return an error if it wants to abort the whole pool. +// Handler may call submit(...) to add child tasks (workers are allowed to submit). +func (p *Pool[T]) Start(handler func(ctx context.Context, item T, submit func(T) error) error) { + // spawn workers + p.workersWg.Add(p.workerCount) + for i := 0; i < p.workerCount; i++ { + go func() { + defer p.workersWg.Done() + for item := range p.tasks { + // if canceled, mark task done and continue so pending can reach zero and close can proceed + if p.ctx.Err() != nil { + p.pending.Done() + continue + } + // run handler (handler may call p.Submit) + if err := handler(p.ctx, item, func(it T) error { return p.Submit(it) }); err != nil { + // store first error and cancel + e := err + p.firstErr.CompareAndSwap(nil, &e) + p.cancel() + // still mark this task done + p.pending.Done() + // continue draining (workers will see ctx canceled and just Done() remaining items) + continue + } + // normal completion of this task + p.pending.Done() + } + }() + } + + // close the tasks channel once pending is zero and CloseForSubmit() has been called + go func() { + // Wait until CloseForSubmit is called (external seeding done) + // After CloseForSubmit, this goroutine waits for pending.Wait() to reach zero, + // then closes the tasks channel exactly once so workers can exit. + for !p.closedForSubmit.Load() { + // spin-wait/check; small sleep could be added but not necessary if CloseForSubmit will be called soon + // We could also use a conditional variable, but this is sufficient and simple. + // Alternatively, CloseForSubmit could spawn the pending-wait goroutine directly (done below). + // Here, just yield + if p.ctx.Err() != nil { + // aborted externally; close tasks to let workers exit + p.closeOnce.Do(func() { close(p.tasks) }) + return + } + } + + // when CloseForSubmit has been called, wait for pending to drain + p.pending.Wait() + p.closeOnce.Do(func() { close(p.tasks) }) + }() +} + +// CloseForSubmit indicates the caller will not submit more external (caller-side) tasks. +// Workers may still call Submit to add child tasks. When pending reaches zero, the pool +// closes the tasks channel to stop workers. +func (p *Pool[T]) CloseForSubmit() { + p.closedForSubmit.Store(true) + // if there are already no pending tasks, close immediately + go func() { + p.pending.Wait() + p.closeOnce.Do(func() { close(p.tasks) }) + }() +} + +// Wait blocks until all workers have exited and returns the first error (if any). +func (p *Pool[T]) Wait() error { + p.workersWg.Wait() + if e := p.firstErr.Load(); e != nil && *e != nil { + return *e + } + if p.ctx.Err() != nil && !errors.Is(p.ctx.Err(), context.Canceled) { + return p.ctx.Err() + } + return nil +} + +// Close cancels the pool and closes the internal tasks channel (force-stop). +func (p *Pool[T]) Close() { + p.cancel() + p.closeOnce.Do(func() { close(p.tasks) }) +} From 75441f5dcc50ee500becfd929e9e0c7589bf869f Mon Sep 17 00:00:00 2001 From: steiler Date: Mon, 6 Oct 2025 11:57:12 +0200 Subject: [PATCH 02/44] fix data race --- pkg/tree/parallelImporter.go | 4 +- pkg/tree/sharedEntryAttributes.go | 2 + pkg/utils/pool.go | 341 ++++++++++++++++++++++++------ 3 files changed, 283 insertions(+), 64 deletions(-) diff --git a/pkg/tree/parallelImporter.go b/pkg/tree/parallelImporter.go index 465291ea..6854b2db 100644 --- a/pkg/tree/parallelImporter.go +++ b/pkg/tree/parallelImporter.go @@ -29,13 +29,12 @@ func (s *sharedEntryAttributes) ImportConfig( intentPrio int32, insertFlags *types.UpdateInsertFlags, ) error { - p := utils.NewWorkerPool[importTask](ctx, runtime.NumCPU(), 900000) + p := utils.NewWorkerPool[importTask](ctx, runtime.NumCPU()) p.Start(importHandler) // seed root if err := p.Submit(importTask{entry: s, importerElement: importerElement, intentName: intentName, intentPrio: intentPrio, insertFlags: insertFlags, treeContext: s.treeContext}); err != nil { - p.Close() return err } @@ -75,6 +74,7 @@ func importHandler(ctx context.Context, task importTask, submit func(importTask) actual = keyChild } // submit resolved entry with same adapter element + // return importHandler(ctx, importTask{entry: actual, importerElement: task.importerElement, intentName: task.intentName, intentPrio: task.intentPrio, insertFlags: task.insertFlags, treeContext: task.treeContext}, submit) return submit(importTask{entry: actual, importerElement: task.importerElement, intentName: task.intentName, intentPrio: task.intentPrio, insertFlags: task.insertFlags, treeContext: task.treeContext}) } diff --git a/pkg/tree/sharedEntryAttributes.go b/pkg/tree/sharedEntryAttributes.go index 268b5c13..3a7c703d 100644 --- a/pkg/tree/sharedEntryAttributes.go +++ b/pkg/tree/sharedEntryAttributes.go @@ -1607,6 +1607,8 @@ func (s *sharedEntryAttributes) StringIndent(result []string) []string { // SdcpbPath returns the sdcpb.Path, with its elements and keys based on the local schema func (s *sharedEntryAttributes) SdcpbPath() *sdcpb.Path { + s.cacheMutex.Lock() + defer s.cacheMutex.Unlock() if s.pathCache != nil { return s.pathCache } diff --git a/pkg/utils/pool.go b/pkg/utils/pool.go index a700bca0..1b0fcd6c 100644 --- a/pkg/utils/pool.go +++ b/pkg/utils/pool.go @@ -8,64 +8,75 @@ import ( "sync/atomic" ) -// Pool[T] is a channel-driven worker pool that tracks pending tasks via a WaitGroup. -// Submit increments the pending WaitGroup before enqueuing; workers call pending.Done() -// after processing a task. Caller must call CloseForSubmit() once when external seeding is finished. +// Pool[T] is a worker pool backed by WorkerPoolQueue. +// It uses an atomic inflight counter + cond to avoid deadlocks between closing the queue +// and tracking outstanding work. type Pool[T any] struct { - tasks chan T + tasks *WorkerPoolQueue[T] workerCount int ctx context.Context cancel context.CancelFunc workersWg sync.WaitGroup // wait for worker goroutines to exit - pending sync.WaitGroup // counts submitted-but-not-done tasks closeOnce sync.Once firstErr atomic.Pointer[error] closedForSubmit atomic.Bool + + // inflight counter and condition for waiting until work drains + inflight int64 + inflightMu sync.Mutex + inflightC *sync.Cond } // NewWorkerPool creates a new Pool. If workerCount <= 0 it defaults to runtime.NumCPU(). -// buf is the internal task-channel buffer size (if <=0 defaults to workerCount). -func NewWorkerPool[T any](parent context.Context, workerCount, buf int) *Pool[T] { +func NewWorkerPool[T any](parent context.Context, workerCount int) *Pool[T] { if workerCount <= 0 { workerCount = runtime.NumCPU() } - if buf <= 0 { - buf = workerCount - } ctx, cancel := context.WithCancel(parent) - return &Pool[T]{ - tasks: make(chan T, buf), + p := &Pool[T]{ + tasks: NewWorkerPoolQueue[T](), workerCount: workerCount, ctx: ctx, cancel: cancel, } + p.inflightC = sync.NewCond(&p.inflightMu) + return p +} + +// addInflight increments inflight and must be called when a task is known submitted. +func (p *Pool[T]) addInflight(delta int64) { + atomic.AddInt64(&p.inflight, delta) + if atomic.LoadInt64(&p.inflight) == 0 { + // wake any waiter (lock to satisfy cond's invariant) + p.inflightMu.Lock() + p.inflightC.Broadcast() + p.inflightMu.Unlock() + } } -// Submit enqueues a task. It increments the pending WaitGroup BEFORE attempting to send. -// If ctx is already cancelled, Submit returns ctx.Err() and does NOT increment pending. +// Submit enqueues a task. It increments the inflight counter BEFORE attempting to enqueue. +// If ctx is already cancelled, Submit returns ctx.Err() and does NOT increment inflight. func (p *Pool[T]) Submit(item T) error { // fast-fail if canceled - if p.ctx.Err() != nil { - return p.ctx.Err() + if err := p.ctx.Err(); err != nil { + return err } - // account for this task first - p.pending.Add(1) + // increment inflight first + p.addInflight(1) - // try to send; if ctx cancelled while sending, balance the pending counter - select { - case p.tasks <- item: - return nil - case <-p.ctx.Done(): - p.pending.Done() // balance - return p.ctx.Err() + // try to put into queue + if err := p.tasks.Put(item); err != nil { + // queue closed (or otherwise failed) -> unaccount the inflight and wake waiters if needed + p.addInflight(-1) + return err } - + return nil } // Start spawns workerCount workers that call handler(ctx, item, submit). @@ -77,62 +88,88 @@ func (p *Pool[T]) Start(handler func(ctx context.Context, item T, submit func(T) for i := 0; i < p.workerCount; i++ { go func() { defer p.workersWg.Done() - for item := range p.tasks { - // if canceled, mark task done and continue so pending can reach zero and close can proceed + for { + item, ok := p.tasks.Get() + if !ok { + // queue closed and drained -> exit worker + return + } + + // If ctx canceled, we must still decrement inflight for this item and skip handler. if p.ctx.Err() != nil { - p.pending.Done() + p.addInflight(-1) continue } + // run handler (handler may call p.Submit) if err := handler(p.ctx, item, func(it T) error { return p.Submit(it) }); err != nil { - // store first error and cancel - e := err - p.firstErr.CompareAndSwap(nil, &e) + // store first error safely (allocate on heap) + ep := new(error) + *ep = err + p.firstErr.CompareAndSwap(nil, ep) + + // cancel pool so other workers see ctx canceled p.cancel() - // still mark this task done - p.pending.Done() - // continue draining (workers will see ctx canceled and just Done() remaining items) + + // decrement inflight for this item + p.addInflight(-1) + + // force-close the queue and abandon queued items (so we won't wait forever) + p.forceClose() + + // continue so other workers can observe ctx and drain/exit continue } - // normal completion of this task - p.pending.Done() + + // normal completion of this task: decrement inflight + p.addInflight(-1) } }() } - // close the tasks channel once pending is zero and CloseForSubmit() has been called + // monitor goroutine: when CloseForSubmit has been called, wait until both inflight==0 and queue empty, + // then close the queue so workers exit. Also handle ctx cancellation (force-close). go func() { - // Wait until CloseForSubmit is called (external seeding done) - // After CloseForSubmit, this goroutine waits for pending.Wait() to reach zero, - // then closes the tasks channel exactly once so workers can exit. - for !p.closedForSubmit.Load() { - // spin-wait/check; small sleep could be added but not necessary if CloseForSubmit will be called soon - // We could also use a conditional variable, but this is sufficient and simple. - // Alternatively, CloseForSubmit could spawn the pending-wait goroutine directly (done below). - // Here, just yield + for { + // graceful path: wait for CloseForSubmit flag then wait for work to drain + if p.closedForSubmit.Load() { + // wait until inflight==0 AND tasks.Len()==0 + p.inflightMu.Lock() + for { + if atomic.LoadInt64(&p.inflight) == 0 && p.tasks.Len() == 0 { + break + } + p.inflightC.Wait() + // loop and re-check + } + p.inflightMu.Unlock() + + // Now safe to close queue: there is no inflight and no queued items + p.closeOnce.Do(func() { p.tasks.Close() }) + return + } + + // if ctx canceled -> force-close path if p.ctx.Err() != nil { - // aborted externally; close tasks to let workers exit - p.closeOnce.Do(func() { close(p.tasks) }) + p.forceClose() return } - } - // when CloseForSubmit has been called, wait for pending to drain - p.pending.Wait() - p.closeOnce.Do(func() { close(p.tasks) }) + // avoid busy spin + runtime.Gosched() + } }() } // CloseForSubmit indicates the caller will not submit more external (caller-side) tasks. -// Workers may still call Submit to add child tasks. When pending reaches zero, the pool -// closes the tasks channel to stop workers. +// Workers may still call Submit to add child tasks. When inflight reaches zero and queue is empty, +// the pool will close tasks so workers exit. func (p *Pool[T]) CloseForSubmit() { p.closedForSubmit.Store(true) - // if there are already no pending tasks, close immediately - go func() { - p.pending.Wait() - p.closeOnce.Do(func() { close(p.tasks) }) - }() + // kick the monitor by signaling condition in case inflight==0 already + p.inflightMu.Lock() + p.inflightC.Broadcast() + p.inflightMu.Unlock() } // Wait blocks until all workers have exited and returns the first error (if any). @@ -147,8 +184,188 @@ func (p *Pool[T]) Wait() error { return nil } -// Close cancels the pool and closes the internal tasks channel (force-stop). -func (p *Pool[T]) Close() { +// forceClose performs a one-time forced shutdown: cancel context, close queue and +// subtract any queued-but-unprocessed items from inflight so waiters don't block forever. +func (p *Pool[T]) forceClose() { p.cancel() - p.closeOnce.Do(func() { close(p.tasks) }) + p.closeOnce.Do(func() { + // first capture queued items + queued := p.tasks.Len() + if queued > 0 { + // reduce inflight by queued. Use atomic and then broadcast condition. + // Ensure we don't go negative. + for { + cur := atomic.LoadInt64(&p.inflight) + // clamp + var toSub int64 = int64(queued) + if toSub > cur { + toSub = cur + } + if toSub == 0 { + break + } + if atomic.CompareAndSwapInt64(&p.inflight, cur, cur-toSub) { + p.inflightMu.Lock() + p.inflightC.Broadcast() + p.inflightMu.Unlock() + break + } + // retry on CAS failure + } + } + // now close the queue to wake Get() waiters + p.tasks.Close() + }) +} + +var ErrClosed = errors.New("queue closed") + +type node[T any] struct { + val T + next atomic.Pointer[node[T]] +} + +// WorkerPoolQueue is a dual-lock MPMC linked-list queue. +// Node.next is an atomic.Pointer to avoid races between producers (writing next) +// and consumers (reading next) when they hold different locks. +type WorkerPoolQueue[T any] struct { + head *node[T] // sentinel + tail *node[T] // last node + headMu sync.Mutex // protects head and cond / waiting + tailMu sync.Mutex // protects tail pointer + cond *sync.Cond // tied to headMu + closed atomic.Bool // closed flag (atomic) + size int64 // approximate size +} + +func NewWorkerPoolQueue[T any]() *WorkerPoolQueue[T] { + s := &node[T]{} + q := &WorkerPoolQueue[T]{head: s, tail: s} + q.cond = sync.NewCond(&q.headMu) + return q +} + +func (q *WorkerPoolQueue[T]) Put(v T) error { + // Quick closed check + if q.closed.Load() { + return ErrClosed + } + + n := &node[T]{} + n.val = v + + for { + // Fast path: append under tailMu if queue is non-empty. + q.tailMu.Lock() + if q.closed.Load() { + q.tailMu.Unlock() + return ErrClosed + } + if q.tail != q.head { // non-empty + // use atomic store for tail.next + q.tail.next.Store(n) + q.tail = n + atomic.AddInt64(&q.size, 1) + q.tailMu.Unlock() + // wake a waiter (ok to call Signal without headMu) + q.cond.Signal() + return nil + } + // maybe empty: release tailMu and take both locks in canonical order + q.tailMu.Unlock() + + q.headMu.Lock() + q.tailMu.Lock() + + // re-check closed & emptiness + if q.closed.Load() { + q.tailMu.Unlock() + q.headMu.Unlock() + return ErrClosed + } + if q.tail != q.head { + // someone appended in the meantime: release and retry + q.tailMu.Unlock() + q.headMu.Unlock() + continue + } + + // safe to append: we hold headMu+tailMu and queue is empty + q.tail.next.Store(n) + q.tail = n + atomic.AddInt64(&q.size, 1) + // signal while holding headMu (canonical) + q.cond.Signal() + q.tailMu.Unlock() + q.headMu.Unlock() + return nil + } +} + +func (q *WorkerPoolQueue[T]) Get() (T, bool) { + q.headMu.Lock() + // wait while empty and not closed + for q.head.next.Load() == nil && !q.closed.Load() { + q.cond.Wait() + } + + // empty + closed => done + if q.head.next.Load() == nil { + q.headMu.Unlock() + var zero T + return zero, false + } + + // pop head.next (atomic load) + n := q.head.next.Load() + // set head.next = n.next (atomic load of next) + next := n.next.Load() + q.head.next.Store(next) + + // If queue became empty after pop, reset tail -> head sentinel if safe. + if q.head.next.Load() == nil { + q.tailMu.Lock() + // compare pointer identity under tailMu + if q.tail == n { + q.tail = q.head + } + q.tailMu.Unlock() + } + + q.headMu.Unlock() + atomic.AddInt64(&q.size, -1) + return n.val, true +} + +func (q *WorkerPoolQueue[T]) TryGet() (T, bool) { + q.headMu.Lock() + if q.head.next.Load() == nil { + q.headMu.Unlock() + var zero T + return zero, false + } + n := q.head.next.Load() + next := n.next.Load() + q.head.next.Store(next) + if q.head.next.Load() == nil { + q.tailMu.Lock() + if q.tail == n { + q.tail = q.head + } + q.tailMu.Unlock() + } + q.headMu.Unlock() + atomic.AddInt64(&q.size, -1) + return n.val, true +} + +func (q *WorkerPoolQueue[T]) Len() int { + return int(atomic.LoadInt64(&q.size)) +} + +func (q *WorkerPoolQueue[T]) Close() { + q.closed.Store(true) + q.headMu.Lock() + q.cond.Broadcast() + q.headMu.Unlock() } From 34537ba90e6ac08145c3cec711d043b35366082c Mon Sep 17 00:00:00 2001 From: steiler Date: Mon, 6 Oct 2025 20:02:47 +0200 Subject: [PATCH 03/44] workerPoolQueue with single lock for head and tail --- pkg/utils/pool.go | 150 ++++++++--------------- pkg/utils/workerpoolqueue_stress_test.go | 86 +++++++++++++ 2 files changed, 137 insertions(+), 99 deletions(-) create mode 100644 pkg/utils/workerpoolqueue_stress_test.go diff --git a/pkg/utils/pool.go b/pkg/utils/pool.go index 1b0fcd6c..2d7b8e8e 100644 --- a/pkg/utils/pool.go +++ b/pkg/utils/pool.go @@ -220,141 +220,93 @@ func (p *Pool[T]) forceClose() { var ErrClosed = errors.New("queue closed") +// noCopy may be embedded into structs which must not be copied after first use. +// go vet will warn on accidental copies (it looks for Lock methods). +type noCopy struct{} + +func (*noCopy) Lock() {} + +// node for single-lock queue (plain pointer; protected by mu) type node[T any] struct { val T - next atomic.Pointer[node[T]] + next *node[T] } -// WorkerPoolQueue is a dual-lock MPMC linked-list queue. -// Node.next is an atomic.Pointer to avoid races between producers (writing next) -// and consumers (reading next) when they hold different locks. +// WorkerPoolQueue is a simple, single-mutex MPMC queue. +// This is easier to reason about than a two-lock variant and avoids lost-wakeup races. type WorkerPoolQueue[T any] struct { - head *node[T] // sentinel - tail *node[T] // last node - headMu sync.Mutex // protects head and cond / waiting - tailMu sync.Mutex // protects tail pointer - cond *sync.Cond // tied to headMu - closed atomic.Bool // closed flag (atomic) - size int64 // approximate size + noCopy noCopy + + mu sync.Mutex + cond *sync.Cond + head *node[T] // sentinel + tail *node[T] + closed bool + size int64 // track queued count (atomic operations used for Len to avoid taking mu) } +// NewWorkerPoolQueue constructs a new queue. func NewWorkerPoolQueue[T any]() *WorkerPoolQueue[T] { s := &node[T]{} q := &WorkerPoolQueue[T]{head: s, tail: s} - q.cond = sync.NewCond(&q.headMu) + q.cond = sync.NewCond(&q.mu) return q } func (q *WorkerPoolQueue[T]) Put(v T) error { - // Quick closed check - if q.closed.Load() { + q.mu.Lock() + defer q.mu.Unlock() + if q.closed { return ErrClosed } - - n := &node[T]{} - n.val = v - - for { - // Fast path: append under tailMu if queue is non-empty. - q.tailMu.Lock() - if q.closed.Load() { - q.tailMu.Unlock() - return ErrClosed - } - if q.tail != q.head { // non-empty - // use atomic store for tail.next - q.tail.next.Store(n) - q.tail = n - atomic.AddInt64(&q.size, 1) - q.tailMu.Unlock() - // wake a waiter (ok to call Signal without headMu) - q.cond.Signal() - return nil - } - // maybe empty: release tailMu and take both locks in canonical order - q.tailMu.Unlock() - - q.headMu.Lock() - q.tailMu.Lock() - - // re-check closed & emptiness - if q.closed.Load() { - q.tailMu.Unlock() - q.headMu.Unlock() - return ErrClosed - } - if q.tail != q.head { - // someone appended in the meantime: release and retry - q.tailMu.Unlock() - q.headMu.Unlock() - continue - } - - // safe to append: we hold headMu+tailMu and queue is empty - q.tail.next.Store(n) - q.tail = n - atomic.AddInt64(&q.size, 1) - // signal while holding headMu (canonical) - q.cond.Signal() - q.tailMu.Unlock() - q.headMu.Unlock() - return nil - } + n := &node[T]{val: v} + q.tail.next = n + q.tail = n + atomic.AddInt64(&q.size, 1) + // signal one waiter (consumer checks under mu) + q.cond.Signal() + return nil } func (q *WorkerPoolQueue[T]) Get() (T, bool) { - q.headMu.Lock() + q.mu.Lock() // wait while empty and not closed - for q.head.next.Load() == nil && !q.closed.Load() { + for q.head.next == nil && !q.closed { q.cond.Wait() } // empty + closed => done - if q.head.next.Load() == nil { - q.headMu.Unlock() + if q.head.next == nil { + q.mu.Unlock() var zero T return zero, false } - // pop head.next (atomic load) - n := q.head.next.Load() - // set head.next = n.next (atomic load of next) - next := n.next.Load() - q.head.next.Store(next) - - // If queue became empty after pop, reset tail -> head sentinel if safe. - if q.head.next.Load() == nil { - q.tailMu.Lock() - // compare pointer identity under tailMu - if q.tail == n { - q.tail = q.head - } - q.tailMu.Unlock() + // pop head.next + n := q.head.next + q.head.next = n.next + if q.head.next == nil { + q.tail = q.head } + q.mu.Unlock() - q.headMu.Unlock() atomic.AddInt64(&q.size, -1) return n.val, true } func (q *WorkerPoolQueue[T]) TryGet() (T, bool) { - q.headMu.Lock() - if q.head.next.Load() == nil { - q.headMu.Unlock() + q.mu.Lock() + if q.head.next == nil { + q.mu.Unlock() var zero T return zero, false } - n := q.head.next.Load() - next := n.next.Load() - q.head.next.Store(next) - if q.head.next.Load() == nil { - q.tailMu.Lock() - if q.tail == n { - q.tail = q.head - } - q.tailMu.Unlock() + n := q.head.next + q.head.next = n.next + if q.head.next == nil { + q.tail = q.head } - q.headMu.Unlock() + q.mu.Unlock() atomic.AddInt64(&q.size, -1) return n.val, true } @@ -364,8 +316,8 @@ func (q *WorkerPoolQueue[T]) Len() int { } func (q *WorkerPoolQueue[T]) Close() { - q.closed.Store(true) - q.headMu.Lock() + q.mu.Lock() + q.closed = true q.cond.Broadcast() - q.headMu.Unlock() + q.mu.Unlock() } diff --git a/pkg/utils/workerpoolqueue_stress_test.go b/pkg/utils/workerpoolqueue_stress_test.go new file mode 100644 index 00000000..1f34e8cc --- /dev/null +++ b/pkg/utils/workerpoolqueue_stress_test.go @@ -0,0 +1,86 @@ +package utils + +import ( + "fmt" + "sync" + "sync/atomic" + "testing" + "time" +) + +// Debug stress test: tracks Put errors and queue length to find where items disappear. +func TestWorkerPoolQueue_StressDebug(t *testing.T) { + q := NewWorkerPoolQueue[int]() + const producers = 8 + const consumers = 32 + const perProducer = 10000 + total := int64(producers * perProducer) + + var produced int64 // attempts (kept for reference) + var putErrors int64 // Put returned ErrClosed or other error + var enqueuedReported int64 // q.Len() snapshot after producers done + var consumed int64 + + var wg sync.WaitGroup + + // Consumers + wg.Add(consumers) + for i := 0; i < consumers; i++ { + go func(id int) { + defer wg.Done() + for { + v, ok := q.Get() + if !ok { + return + } + _ = v + atomic.AddInt64(&consumed, 1) + } + }(i) + } + + // Producers + var pwg sync.WaitGroup + pwg.Add(producers) + for p := 0; p < producers; p++ { + go func(base int) { + defer pwg.Done() + for i := 0; i < perProducer; i++ { + atomic.AddInt64(&produced, 1) + if err := q.Put(base*perProducer + i); err != nil { + // record Put errors + atomic.AddInt64(&putErrors, 1) + } + } + }(p) + } + + // wait for producers to finish + pwg.Wait() + + // snapshot how many the queue reports as enqueued + enqueuedReported = int64(q.Len()) + + // let consumers run for a bit to drain + time.Sleep(200 * time.Millisecond) + + // close queue so remaining consumers exit and we can get final counts + q.Close() + wg.Wait() + + // final snapshot + finalLen := int64(q.Len()) + p := atomic.LoadInt64(&produced) + pe := atomic.LoadInt64(&putErrors) + c := atomic.LoadInt64(&consumed) + + fmt.Printf("DEBUG: producedAttempts=%d putErrors=%d enqueuedReported(after producers)=%d finalReportedLen=%d consumed=%d totalExpected=%d\n", + p, pe, enqueuedReported, finalLen, c, total) + + if pe != 0 { + t.Fatalf("Put returned errors: %d (see debug print)", pe) + } + if c != total { + t.Fatalf("consumed %d, want %d (see debug print)", c, total) + } +} From 3ce615944e42cbcb1de2bbfe1f5f19a403e1fad0 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 8 Oct 2025 11:40:54 +0200 Subject: [PATCH 04/44] Reorg the Syncing --- mocks/mocktarget/target.go | 139 ++----- pkg/datastore/datastore_rpc.go | 256 +------------ pkg/datastore/deviations.go | 162 ++++++++ pkg/datastore/intent_rpc.go | 4 +- pkg/datastore/sync.go | 128 +++++++ pkg/datastore/target/gnmi/get.go | 147 ++++++++ pkg/datastore/target/{ => gnmi}/gnmi.go | 367 ++++++------------- pkg/datastore/target/gnmi/gnmisync.go | 6 + pkg/datastore/target/gnmi/once.go | 80 ++++ pkg/datastore/target/gnmi/stream.go | 82 +++++ pkg/datastore/target/gnmi/utils/utils.go | 62 ++++ pkg/datastore/target/nc.go | 58 +-- pkg/datastore/target/{ => noop}/noop.go | 11 +- pkg/datastore/target/target.go | 84 ++--- pkg/datastore/target/types/runningstore.go | 13 + pkg/datastore/target/types/syncresult.go | 17 + pkg/datastore/target/types/targetsource.go | 20 + pkg/datastore/target/types/targetstatus.go | 22 ++ pkg/datastore/types/target_source_replace.go | 6 +- pkg/server/datastore.go | 6 +- 20 files changed, 955 insertions(+), 715 deletions(-) create mode 100644 pkg/datastore/deviations.go create mode 100644 pkg/datastore/sync.go create mode 100644 pkg/datastore/target/gnmi/get.go rename pkg/datastore/target/{ => gnmi}/gnmi.go (50%) create mode 100644 pkg/datastore/target/gnmi/gnmisync.go create mode 100644 pkg/datastore/target/gnmi/once.go create mode 100644 pkg/datastore/target/gnmi/stream.go create mode 100644 pkg/datastore/target/gnmi/utils/utils.go rename pkg/datastore/target/{ => noop}/noop.go (85%) create mode 100644 pkg/datastore/target/types/runningstore.go create mode 100644 pkg/datastore/target/types/syncresult.go create mode 100644 pkg/datastore/target/types/targetsource.go create mode 100644 pkg/datastore/target/types/targetstatus.go diff --git a/mocks/mocktarget/target.go b/mocks/mocktarget/target.go index 920135b7..6f7ffbeb 100644 --- a/mocks/mocktarget/target.go +++ b/mocks/mocktarget/target.go @@ -13,9 +13,8 @@ import ( context "context" reflect "reflect" - etree "github.com/beevik/etree" config "github.com/sdcio/data-server/pkg/config" - target "github.com/sdcio/data-server/pkg/datastore/target" + types "github.com/sdcio/data-server/pkg/datastore/target/types" schema_server "github.com/sdcio/sdc-protos/sdcpb" gomock "go.uber.org/mock/gomock" ) @@ -44,6 +43,25 @@ func (m *MockTarget) EXPECT() *MockTargetMockRecorder { return m.recorder } +// AddSyncs mocks base method. +func (m *MockTarget) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) error { + m.ctrl.T.Helper() + varargs := []any{ctx} + for _, a := range sps { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "AddSyncs", varargs...) + ret0, _ := ret[0].(error) + return ret0 +} + +// AddSyncs indicates an expected call of AddSyncs. +func (mr *MockTargetMockRecorder) AddSyncs(ctx any, sps ...any) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]any{ctx}, sps...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddSyncs", reflect.TypeOf((*MockTarget)(nil).AddSyncs), varargs...) +} + // Close mocks base method. func (m *MockTarget) Close() error { m.ctrl.T.Helper() @@ -74,7 +92,7 @@ func (mr *MockTargetMockRecorder) Get(ctx, req any) *gomock.Call { } // Set mocks base method. -func (m *MockTarget) Set(ctx context.Context, source target.TargetSource) (*schema_server.SetDataResponse, error) { +func (m *MockTarget) Set(ctx context.Context, source types.TargetSource) (*schema_server.SetDataResponse, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Set", ctx, source) ret0, _ := ret[0].(*schema_server.SetDataResponse) @@ -89,10 +107,10 @@ func (mr *MockTargetMockRecorder) Set(ctx, source any) *gomock.Call { } // Status mocks base method. -func (m *MockTarget) Status() *target.TargetStatus { +func (m *MockTarget) Status() *types.TargetStatus { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Status") - ret0, _ := ret[0].(*target.TargetStatus) + ret0, _ := ret[0].(*types.TargetStatus) return ret0 } @@ -101,114 +119,3 @@ func (mr *MockTargetMockRecorder) Status() *gomock.Call { mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Status", reflect.TypeOf((*MockTarget)(nil).Status)) } - -// Sync mocks base method. -func (m *MockTarget) Sync(ctx context.Context, syncConfig *config.Sync, syncCh chan *target.SyncUpdate) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Sync", ctx, syncConfig, syncCh) -} - -// Sync indicates an expected call of Sync. -func (mr *MockTargetMockRecorder) Sync(ctx, syncConfig, syncCh any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Sync", reflect.TypeOf((*MockTarget)(nil).Sync), ctx, syncConfig, syncCh) -} - -// MockTargetSource is a mock of TargetSource interface. -type MockTargetSource struct { - ctrl *gomock.Controller - recorder *MockTargetSourceMockRecorder - isgomock struct{} -} - -// MockTargetSourceMockRecorder is the mock recorder for MockTargetSource. -type MockTargetSourceMockRecorder struct { - mock *MockTargetSource -} - -// NewMockTargetSource creates a new mock instance. -func NewMockTargetSource(ctrl *gomock.Controller) *MockTargetSource { - mock := &MockTargetSource{ctrl: ctrl} - mock.recorder = &MockTargetSourceMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *MockTargetSource) EXPECT() *MockTargetSourceMockRecorder { - return m.recorder -} - -// ToJson mocks base method. -func (m *MockTargetSource) ToJson(onlyNewOrUpdated bool) (any, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ToJson", onlyNewOrUpdated) - ret0, _ := ret[0].(any) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ToJson indicates an expected call of ToJson. -func (mr *MockTargetSourceMockRecorder) ToJson(onlyNewOrUpdated any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ToJson", reflect.TypeOf((*MockTargetSource)(nil).ToJson), onlyNewOrUpdated) -} - -// ToJsonIETF mocks base method. -func (m *MockTargetSource) ToJsonIETF(onlyNewOrUpdated bool) (any, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ToJsonIETF", onlyNewOrUpdated) - ret0, _ := ret[0].(any) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ToJsonIETF indicates an expected call of ToJsonIETF. -func (mr *MockTargetSourceMockRecorder) ToJsonIETF(onlyNewOrUpdated any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ToJsonIETF", reflect.TypeOf((*MockTargetSource)(nil).ToJsonIETF), onlyNewOrUpdated) -} - -// ToProtoDeletes mocks base method. -func (m *MockTargetSource) ToProtoDeletes(ctx context.Context) ([]*schema_server.Path, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ToProtoDeletes", ctx) - ret0, _ := ret[0].([]*schema_server.Path) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ToProtoDeletes indicates an expected call of ToProtoDeletes. -func (mr *MockTargetSourceMockRecorder) ToProtoDeletes(ctx any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ToProtoDeletes", reflect.TypeOf((*MockTargetSource)(nil).ToProtoDeletes), ctx) -} - -// ToProtoUpdates mocks base method. -func (m *MockTargetSource) ToProtoUpdates(ctx context.Context, onlyNewOrUpdated bool) ([]*schema_server.Update, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ToProtoUpdates", ctx, onlyNewOrUpdated) - ret0, _ := ret[0].([]*schema_server.Update) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ToProtoUpdates indicates an expected call of ToProtoUpdates. -func (mr *MockTargetSourceMockRecorder) ToProtoUpdates(ctx, onlyNewOrUpdated any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ToProtoUpdates", reflect.TypeOf((*MockTargetSource)(nil).ToProtoUpdates), ctx, onlyNewOrUpdated) -} - -// ToXML mocks base method. -func (m *MockTargetSource) ToXML(onlyNewOrUpdated, honorNamespace, operationWithNamespace, useOperationRemove bool) (*etree.Document, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ToXML", onlyNewOrUpdated, honorNamespace, operationWithNamespace, useOperationRemove) - ret0, _ := ret[0].(*etree.Document) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ToXML indicates an expected call of ToXML. -func (mr *MockTargetSourceMockRecorder) ToXML(onlyNewOrUpdated, honorNamespace, operationWithNamespace, useOperationRemove any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ToXML", reflect.TypeOf((*MockTargetSource)(nil).ToXML), onlyNewOrUpdated, honorNamespace, operationWithNamespace, useOperationRemove) -} diff --git a/pkg/datastore/datastore_rpc.go b/pkg/datastore/datastore_rpc.go index fe2c3aa6..117862d0 100644 --- a/pkg/datastore/datastore_rpc.go +++ b/pkg/datastore/datastore_rpc.go @@ -17,25 +17,21 @@ package datastore import ( "context" "errors" - "strings" "sync" "time" sdcpb "github.com/sdcio/sdc-protos/sdcpb" log "github.com/sirupsen/logrus" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/peer" - status "google.golang.org/grpc/status" "github.com/sdcio/data-server/pkg/cache" "github.com/sdcio/data-server/pkg/config" schemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" "github.com/sdcio/data-server/pkg/datastore/target" + targettypes "github.com/sdcio/data-server/pkg/datastore/target/types" "github.com/sdcio/data-server/pkg/datastore/types" "github.com/sdcio/data-server/pkg/schema" "github.com/sdcio/data-server/pkg/tree" - treetypes "github.com/sdcio/data-server/pkg/tree/types" ) type Datastore struct { @@ -51,9 +47,6 @@ type Datastore struct { // schemaClient sdcpb.SchemaServerClient schemaClient schemaClient.SchemaClientBound - // sync channel, to be passed to the SBI Sync method - synCh chan *target.SyncUpdate - // stop cancel func cfn context.CancelFunc @@ -106,9 +99,6 @@ func New(ctx context.Context, c *config.DatastoreConfig, sc schema.Client, cc ca } ds.transactionManager = types.NewTransactionManager(NewDatastoreRollbackAdapter(ds)) - if c.Sync != nil { - ds.synCh = make(chan *target.SyncUpdate, c.Sync.Buffer) - } ctx, cancel := context.WithCancel(ctx) ds.cfn = cancel @@ -126,10 +116,7 @@ func New(ctx context.Context, c *config.DatastoreConfig, sc schema.Client, cc ca log.Errorf("failed to create SBI for target %s: %v", ds.Config().Name, err) return } - // start syncing goroutine - if c.Sync != nil { - go ds.Sync(ctx) - } + // start deviation goroutine ds.DeviationMgr(ctx, c.Deviation) }() @@ -159,7 +146,7 @@ CREATE: func (d *Datastore) connectSBI(ctx context.Context, opts ...grpc.DialOption) error { var err error - d.sbi, err = target.New(ctx, d.config.Name, d.config.SBI, d.schemaClient, opts...) + d.sbi, err = target.New(ctx, d.config.Name, d.config.SBI, d.schemaClient, d, d.config.Sync.Config, opts...) if err == nil { return nil } @@ -173,7 +160,7 @@ func (d *Datastore) connectSBI(ctx context.Context, opts ...grpc.DialOption) err case <-ctx.Done(): return ctx.Err() case <-ticker.C: - d.sbi, err = target.New(ctx, d.config.Name, d.config.SBI, d.schemaClient, opts...) + d.sbi, err = target.New(ctx, d.config.Name, d.config.SBI, d.schemaClient, d, d.config.Sync.Config, opts...) if err != nil { log.Errorf("failed to create DS %s target: %v", d.config.Name, err) continue @@ -199,9 +186,9 @@ func (d *Datastore) Delete(ctx context.Context) error { return d.cacheClient.InstanceDelete(ctx) } -func (d *Datastore) ConnectionState() *target.TargetStatus { +func (d *Datastore) ConnectionState() *targettypes.TargetStatus { if d.sbi == nil { - return target.NewTargetStatus(target.TargetStatusNotConnected) + return targettypes.NewTargetStatus(targettypes.TargetStatusNotConnected) } return d.sbi.Status() } @@ -221,237 +208,6 @@ func (d *Datastore) Stop() error { return nil } -func (d *Datastore) Sync(ctx context.Context) { - go d.sbi.Sync(ctx, - d.config.Sync, - d.synCh, - ) - - var err error - var startTs int64 - - d.syncTreeCandidate, err = tree.NewTreeRoot(ctx, tree.NewTreeContext(d.schemaClient, tree.RunningIntentName)) - if err != nil { - log.Errorf("creating a new synctree candidate: %v", err) - return - } - - for { - select { - case <-ctx.Done(): - if !errors.Is(ctx.Err(), context.Canceled) { - log.Errorf("datastore %s sync stopped: %v", d.Name(), ctx.Err()) - } - return - case syncup := <-d.synCh: - switch { - case syncup.Start: - log.Debugf("%s: sync start", d.Name()) - startTs = time.Now().Unix() - - case syncup.End: - log.Debugf("%s: sync end", d.Name()) - - startTs = 0 - - d.syncTreeMutex.Lock() - d.syncTree = d.syncTreeCandidate - d.syncTreeMutex.Unlock() - - // create new syncTreeCandidat - d.syncTreeCandidate, err = tree.NewTreeRoot(ctx, tree.NewTreeContext(d.schemaClient, tree.RunningIntentName)) - if err != nil { - log.Errorf("creating a new synctree candidate: %v", err) - return - } - - // export and write to cache - runningExport, err := d.syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) - if err != nil { - log.Error(err) - continue - } - err = d.cacheClient.IntentModify(ctx, runningExport) - if err != nil { - log.Errorf("issue modifying running cache content: %v", err) - continue - } - default: - if startTs == 0 { - startTs = time.Now().Unix() - } - err := d.writeToSyncTreeCandidate(ctx, syncup.Update.GetUpdate(), startTs) - if err != nil { - log.Errorf("failed to write to sync tree: %v", err) - } - } - } - } -} - -func (d *Datastore) writeToSyncTreeCandidate(ctx context.Context, updates []*sdcpb.Update, ts int64) error { - upds, err := treetypes.ExpandAndConvertIntent(ctx, d.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, updates, ts) - if err != nil { - return err - } - - // fmt.Println(upds.String()) - for idx, upd := range upds { - _ = idx - _, err := d.syncTreeCandidate.AddUpdateRecursive(ctx, upd.Path(), upd, treetypes.NewUpdateInsertFlags()) - if err != nil { - return err - } - } - return nil -} - -func (d *Datastore) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdcpb.DataServer_WatchDeviationsServer) error { - d.m.Lock() - defer d.m.Unlock() - - ctx := stream.Context() - p, ok := peer.FromContext(ctx) - if !ok { - return status.Errorf(codes.InvalidArgument, "missing peer info") - } - pName := p.Addr.String() - - d.deviationClients[pName] = stream - return nil -} - -func (d *Datastore) StopDeviationsWatch(peer string) { - d.m.Lock() - defer d.m.Unlock() - delete(d.deviationClients, peer) - log.Debugf("deviation watcher %s removed", peer) -} - -func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) { - log.Infof("%s: starting deviationMgr...", d.Name()) - ticker := time.NewTicker(c.Interval) - defer func() { - ticker.Stop() - }() - for { - select { - case <-ctx.Done(): - return - case <-ticker.C: - d.m.RLock() - deviationClientNames := make([]string, 0, len(d.deviationClients)) - deviationClients := map[string]sdcpb.DataServer_WatchDeviationsServer{} - for clientIdentifier, devStream := range d.deviationClients { - deviationClients[clientIdentifier] = devStream - deviationClientNames = append(deviationClientNames, clientIdentifier) - } - d.m.RUnlock() - if len(deviationClients) == 0 { - log.Debugf("no deviation clients present %s", d.config.Name) - continue - } - log.Debugf("deviations clients for %s: [ %s ]", d.config.Name, strings.Join(deviationClientNames, ", ")) - for clientIdentifier, dc := range deviationClients { - err := dc.Send(&sdcpb.WatchDeviationResponse{ - Name: d.config.Name, - Event: sdcpb.DeviationEvent_START, - }) - if err != nil { - log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) - } - } - deviationChan, err := d.calculateDeviations(ctx) - if err != nil { - log.Error(err) - continue - } - d.SendDeviations(deviationChan, deviationClients) - for clientIdentifier, dc := range deviationClients { - err := dc.Send(&sdcpb.WatchDeviationResponse{ - Name: d.config.Name, - Event: sdcpb.DeviationEvent_END, - }) - if err != nil { - log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) - } - } - } - } -} - -func (d *Datastore) SendDeviations(ch <-chan *treetypes.DeviationEntry, deviationClients map[string]sdcpb.DataServer_WatchDeviationsServer) { - wg := &sync.WaitGroup{} - for { - select { - case de, ok := <-ch: - if !ok { - wg.Wait() - return - } - wg.Add(1) - go func(de DeviationEntry, dcs map[string]sdcpb.DataServer_WatchDeviationsServer) { - for clientIdentifier, dc := range dcs { - err := dc.Send(&sdcpb.WatchDeviationResponse{ - Name: d.config.Name, - Intent: de.IntentName(), - Event: sdcpb.DeviationEvent_UPDATE, - Reason: sdcpb.DeviationReason(de.Reason()), - Path: de.Path(), - ExpectedValue: de.ExpectedValue(), - CurrentValue: de.CurrentValue(), - }) - if err != nil { - log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) - } - } - wg.Done() - }(de, deviationClients) - } - } -} - -type DeviationEntry interface { - IntentName() string - Reason() treetypes.DeviationReason - Path() *sdcpb.Path - CurrentValue() *sdcpb.TypedValue - ExpectedValue() *sdcpb.TypedValue -} - -func (d *Datastore) calculateDeviations(ctx context.Context) (<-chan *treetypes.DeviationEntry, error) { - deviationChan := make(chan *treetypes.DeviationEntry, 10) - - d.syncTreeMutex.RLock() - deviationTree, err := d.syncTree.DeepCopy(ctx) - if err != nil { - return nil, err - } - d.syncTreeMutex.RUnlock() - - addedIntentNames, err := d.LoadAllButRunningIntents(ctx, deviationTree, true) - if err != nil { - return nil, err - } - - // Send IntentExists - for _, n := range addedIntentNames { - deviationChan <- treetypes.NewDeviationEntry(n, treetypes.DeviationReasonIntentExists, nil) - } - - err = deviationTree.FinishInsertionPhase(ctx) - if err != nil { - return nil, err - } - - go func() { - deviationTree.GetDeviations(deviationChan) - close(deviationChan) - }() - - return deviationChan, nil -} - func (d *Datastore) BlameConfig(ctx context.Context, includeDefaults bool) (*sdcpb.BlameTreeElement, error) { // create a new TreeRoot by copying the syncTree d.syncTreeMutex.Lock() diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go new file mode 100644 index 00000000..f332a938 --- /dev/null +++ b/pkg/datastore/deviations.go @@ -0,0 +1,162 @@ +package datastore + +import ( + "context" + "strings" + "sync" + "time" + + "github.com/sdcio/data-server/pkg/config" + treetypes "github.com/sdcio/data-server/pkg/tree/types" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" + log "github.com/sirupsen/logrus" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/peer" + status "google.golang.org/grpc/status" +) + +func (d *Datastore) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdcpb.DataServer_WatchDeviationsServer) error { + d.m.Lock() + defer d.m.Unlock() + + ctx := stream.Context() + p, ok := peer.FromContext(ctx) + if !ok { + return status.Errorf(codes.InvalidArgument, "missing peer info") + } + pName := p.Addr.String() + + d.deviationClients[pName] = stream + return nil +} + +func (d *Datastore) StopDeviationsWatch(peer string) { + d.m.Lock() + defer d.m.Unlock() + delete(d.deviationClients, peer) + log.Debugf("deviation watcher %s removed", peer) +} + +func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) { + log.Infof("%s: starting deviationMgr...", d.Name()) + ticker := time.NewTicker(c.Interval) + defer func() { + ticker.Stop() + }() + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + d.m.RLock() + deviationClientNames := make([]string, 0, len(d.deviationClients)) + deviationClients := map[string]sdcpb.DataServer_WatchDeviationsServer{} + for clientIdentifier, devStream := range d.deviationClients { + deviationClients[clientIdentifier] = devStream + deviationClientNames = append(deviationClientNames, clientIdentifier) + } + d.m.RUnlock() + if len(deviationClients) == 0 { + log.Debugf("no deviation clients present %s", d.config.Name) + continue + } + log.Debugf("deviations clients for %s: [ %s ]", d.config.Name, strings.Join(deviationClientNames, ", ")) + for clientIdentifier, dc := range deviationClients { + err := dc.Send(&sdcpb.WatchDeviationResponse{ + Name: d.config.Name, + Event: sdcpb.DeviationEvent_START, + }) + if err != nil { + log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) + } + } + deviationChan, err := d.calculateDeviations(ctx) + if err != nil { + log.Error(err) + continue + } + d.SendDeviations(deviationChan, deviationClients) + for clientIdentifier, dc := range deviationClients { + err := dc.Send(&sdcpb.WatchDeviationResponse{ + Name: d.config.Name, + Event: sdcpb.DeviationEvent_END, + }) + if err != nil { + log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) + } + } + } + } +} + +func (d *Datastore) SendDeviations(ch <-chan *treetypes.DeviationEntry, deviationClients map[string]sdcpb.DataServer_WatchDeviationsServer) { + wg := &sync.WaitGroup{} + for { + select { + case de, ok := <-ch: + if !ok { + wg.Wait() + return + } + wg.Add(1) + go func(de DeviationEntry, dcs map[string]sdcpb.DataServer_WatchDeviationsServer) { + for clientIdentifier, dc := range dcs { + err := dc.Send(&sdcpb.WatchDeviationResponse{ + Name: d.config.Name, + Intent: de.IntentName(), + Event: sdcpb.DeviationEvent_UPDATE, + Reason: sdcpb.DeviationReason(de.Reason()), + Path: de.Path(), + ExpectedValue: de.ExpectedValue(), + CurrentValue: de.CurrentValue(), + }) + if err != nil { + log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) + } + } + wg.Done() + }(de, deviationClients) + } + } +} + +type DeviationEntry interface { + IntentName() string + Reason() treetypes.DeviationReason + Path() *sdcpb.Path + CurrentValue() *sdcpb.TypedValue + ExpectedValue() *sdcpb.TypedValue +} + +func (d *Datastore) calculateDeviations(ctx context.Context) (<-chan *treetypes.DeviationEntry, error) { + deviationChan := make(chan *treetypes.DeviationEntry, 10) + + d.syncTreeMutex.RLock() + deviationTree, err := d.syncTree.DeepCopy(ctx) + if err != nil { + return nil, err + } + d.syncTreeMutex.RUnlock() + + addedIntentNames, err := d.LoadAllButRunningIntents(ctx, deviationTree, true) + if err != nil { + return nil, err + } + + // Send IntentExists + for _, n := range addedIntentNames { + deviationChan <- treetypes.NewDeviationEntry(n, treetypes.DeviationReasonIntentExists, nil) + } + + err = deviationTree.FinishInsertionPhase(ctx) + if err != nil { + return nil, err + } + + go func() { + deviationTree.GetDeviations(deviationChan) + close(deviationChan) + }() + + return deviationChan, nil +} diff --git a/pkg/datastore/intent_rpc.go b/pkg/datastore/intent_rpc.go index 2664d56f..bdc53546 100644 --- a/pkg/datastore/intent_rpc.go +++ b/pkg/datastore/intent_rpc.go @@ -23,7 +23,7 @@ import ( sdcpb "github.com/sdcio/sdc-protos/sdcpb" log "github.com/sirupsen/logrus" - "github.com/sdcio/data-server/pkg/datastore/target" + targettypes "github.com/sdcio/data-server/pkg/datastore/target/types" "github.com/sdcio/data-server/pkg/tree" "github.com/sdcio/data-server/pkg/tree/importer/proto" "github.com/sdcio/data-server/pkg/tree/types" @@ -31,7 +31,7 @@ import ( var ErrIntentNotFound = errors.New("intent not found") -func (d *Datastore) applyIntent(ctx context.Context, source target.TargetSource) (*sdcpb.SetDataResponse, error) { +func (d *Datastore) applyIntent(ctx context.Context, source targettypes.TargetSource) (*sdcpb.SetDataResponse, error) { var err error var rsp *sdcpb.SetDataResponse diff --git a/pkg/datastore/sync.go b/pkg/datastore/sync.go new file mode 100644 index 00000000..cf9a9654 --- /dev/null +++ b/pkg/datastore/sync.go @@ -0,0 +1,128 @@ +package datastore + +import ( + "context" + + "github.com/sdcio/data-server/pkg/tree" + "github.com/sdcio/data-server/pkg/tree/importer/proto" + treetypes "github.com/sdcio/data-server/pkg/tree/types" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" + "github.com/sdcio/sdc-protos/tree_persist" +) + +// func (d *Datastore) SyncOld(ctx context.Context) { +// go d.sbi.Sync(ctx, +// d.config.Sync, +// ) + +// var err error +// var startTs int64 + +// d.syncTreeCandidate, err = tree.NewTreeRoot(ctx, tree.NewTreeContext(d.schemaClient, tree.RunningIntentName)) +// if err != nil { +// log.Errorf("creating a new synctree candidate: %v", err) +// return +// } + +// for { +// select { +// case <-ctx.Done(): +// if !errors.Is(ctx.Err(), context.Canceled) { +// log.Errorf("datastore %s sync stopped: %v", d.Name(), ctx.Err()) +// } +// return +// case syncup := <-d.synCh: +// switch { +// case syncup.Start: +// log.Debugf("%s: sync start", d.Name()) +// startTs = time.Now().Unix() + +// case syncup.End: +// log.Debugf("%s: sync end", d.Name()) + +// startTs = 0 + +// d.syncTreeMutex.Lock() +// d.syncTree = d.syncTreeCandidate +// d.syncTreeMutex.Unlock() + +// // create new syncTreeCandidat +// d.syncTreeCandidate, err = tree.NewTreeRoot(ctx, tree.NewTreeContext(d.schemaClient, tree.RunningIntentName)) +// if err != nil { +// log.Errorf("creating a new synctree candidate: %v", err) +// return +// } + +// // export and write to cache +// runningExport, err := d.syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) +// if err != nil { +// log.Error(err) +// continue +// } +// err = d.cacheClient.IntentModify(ctx, runningExport) +// if err != nil { +// log.Errorf("issue modifying running cache content: %v", err) +// continue +// } +// default: +// if startTs == 0 { +// startTs = time.Now().Unix() +// } +// err := d.writeToSyncTreeCandidate(ctx, syncup.Update.GetUpdate(), startTs) +// if err != nil { +// log.Errorf("failed to write to sync tree: %v", err) +// } +// } +// } +// } +// } + +func (d *Datastore) writeToSyncTreeCandidate(ctx context.Context, updates []*sdcpb.Update, ts int64) error { + upds, err := treetypes.ExpandAndConvertIntent(ctx, d.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, updates, ts) + if err != nil { + return err + } + + for idx, upd := range upds { + _ = idx + _, err := d.syncTreeCandidate.AddUpdateRecursive(ctx, upd.Path(), upd, treetypes.NewUpdateInsertFlags()) + if err != nil { + return err + } + } + return nil +} + +func (d *Datastore) ApplyToRunning(ctx context.Context, i *tree_persist.Intent) error { + + i.IntentName = tree.RunningIntentName + i.Priority = tree.RunningValuesPrio + i.Deviation = false + + importer := proto.NewProtoTreeImporter(i) + + // need to reset the explicit deletes, they carry the actual deletes that we need to delete. + // the imported would otherwise add explicit deletes for these. + deletes := i.ExplicitDeletes + i.ExplicitDeletes = nil + + d.syncTreeMutex.Lock() + defer d.syncTreeMutex.Unlock() + for _, delete := range deletes { + err := d.syncTree.DeleteBranch(ctx, delete, i.IntentName) + if err != nil { + return err + } + } + + return d.syncTree.ImportConfig(ctx, &sdcpb.Path{}, importer, i.GetIntentName(), i.GetPriority(), treetypes.NewUpdateInsertFlags()) +} + +func (d *Datastore) NewEmptyTree(ctx context.Context) (*tree.RootEntry, error) { + tc := tree.NewTreeContext(d.schemaClient, tree.RunningIntentName) + newTree, err := tree.NewTreeRoot(ctx, tc) + if err != nil { + return nil, err + } + return newTree, nil +} diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go new file mode 100644 index 00000000..634abbdc --- /dev/null +++ b/pkg/datastore/target/gnmi/get.go @@ -0,0 +1,147 @@ +package gnmi + +import ( + "context" + "time" + + "github.com/sdcio/data-server/pkg/config" + "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" + "github.com/sdcio/data-server/pkg/datastore/target/types" + "github.com/sdcio/data-server/pkg/tree" + treetypes "github.com/sdcio/data-server/pkg/tree/types" + dsutils "github.com/sdcio/data-server/pkg/utils" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" + log "github.com/sirupsen/logrus" +) + +type GetSync struct { + config *config.SyncProtocol + target GetTarget + cancel context.CancelFunc + runningStore types.RunningStore + ctx context.Context + schemaClient dsutils.SchemaClientBound +} + +func NewGetSync(ctx context.Context, target GetTarget, c *config.SyncProtocol, runningStore types.RunningStore, schemaClient dsutils.SchemaClientBound) *GetSync { + ctx, cancel := context.WithCancel(ctx) + + return &GetSync{ + config: c, + target: target, + cancel: cancel, + runningStore: runningStore, + ctx: ctx, + schemaClient: schemaClient, + } +} + +func (s *GetSync) syncConfig() (*sdcpb.GetDataRequest, error) { + // iterate syncConfig + paths := make([]*sdcpb.Path, 0, len(s.config.Paths)) + // iterate referenced paths + for _, p := range s.config.Paths { + path, err := sdcpb.ParsePath(p) + if err != nil { + return nil, err + } + // add the parsed path + paths = append(paths, path) + } + + req := &sdcpb.GetDataRequest{ + Name: s.config.Name, + Path: paths, + DataType: sdcpb.DataType_CONFIG, + Encoding: sdcpb.Encoding(utils.ParseSdcpbEncoding(s.config.Encoding)), + } + return req, nil +} + +func (s *GetSync) Stop() error { + // TODO + s.cancel() + return nil +} + +func (s *GetSync) Start() error { + req, err := s.syncConfig() + if err != nil { + return err + } + + go s.internalGetSync(req) + + go func() { + ticker := time.NewTicker(s.config.Interval) + defer ticker.Stop() + for { + select { + case <-s.ctx.Done(): + return + case <-ticker.C: + s.internalGetSync(req) + } + } + }() + + return nil +} + +func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { + // execute gnmi get + resp, err := s.target.Get(s.ctx, req) + if err != nil { + log.Errorf("sync error: %v", err) + return + } + syncTree, err := s.runningStore.NewEmptyTree(s.ctx) + if err != nil { + log.Errorf("sync newemptytree error: %v", err) + return + } + deletes := []*sdcpb.Path{} + + ts := time.Now() + uif := treetypes.NewUpdateInsertFlags() + + // process Noifications + for _, noti := range resp.Notification { + // updates + upds, err := treetypes.ExpandAndConvertIntent(s.ctx, s.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, noti.Update, ts.Unix()) + if err != nil { + log.Errorf("sync expanding error: %v", err) + return + } + + for idx2, upd := range upds { + _ = idx2 + _, err = syncTree.AddUpdateRecursive(s.ctx, upd.Path(), upd, uif) + if err != nil { + log.Errorf("sync process notifications error: %v", err) + return + } + + } + // deletes + deletes = append(deletes, noti.GetDelete()...) + } + + result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) + if err != nil { + log.Errorf("sync tree export error: %v", err) + return + } + // add also the deletes to the export + result.ExplicitDeletes = deletes + + err = s.runningStore.ApplyToRunning(s.ctx, result) + if err != nil { + log.Errorf("sync import to running error: %v", err) + return + } +} + +type GetTarget interface { + Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) +} diff --git a/pkg/datastore/target/gnmi.go b/pkg/datastore/target/gnmi/gnmi.go similarity index 50% rename from pkg/datastore/target/gnmi.go rename to pkg/datastore/target/gnmi/gnmi.go index e5abbed4..4563c4af 100644 --- a/pkg/datastore/target/gnmi.go +++ b/pkg/datastore/target/gnmi/gnmi.go @@ -12,31 +12,32 @@ // See the License for the specific language governing permissions and // limitations under the License. -package target +package gnmi import ( "context" "encoding/json" - "errors" "fmt" - "strconv" "strings" "time" "github.com/AlekSi/pointer" "github.com/openconfig/gnmi/proto/gnmi" - gapi "github.com/openconfig/gnmic/pkg/api" gtarget "github.com/openconfig/gnmic/pkg/api/target" "github.com/openconfig/gnmic/pkg/api/types" + "github.com/sdcio/data-server/pkg/config" + gnmiutils "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" + targetTypes "github.com/sdcio/data-server/pkg/datastore/target/types" + "github.com/sdcio/data-server/pkg/utils" + dsutils "github.com/sdcio/data-server/pkg/utils" sdcpb "github.com/sdcio/sdc-protos/sdcpb" log "github.com/sirupsen/logrus" "google.golang.org/grpc" "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/keepalive" + "google.golang.org/protobuf/encoding/prototext" "google.golang.org/protobuf/proto" - - "github.com/sdcio/data-server/pkg/config" - "github.com/sdcio/data-server/pkg/utils" ) const ( @@ -44,12 +45,15 @@ const ( ) type gnmiTarget struct { - target *gtarget.Target - encodings map[gnmi.Encoding]struct{} - cfg *config.SBI + target *gtarget.Target + encodings map[gnmi.Encoding]struct{} + cfg *config.SBI + syncs map[string]GnmiSync + runningStore targetTypes.RunningStore + schemaClient dsutils.SchemaClientBound } -func newGNMITarget(ctx context.Context, name string, cfg *config.SBI, opts ...grpc.DialOption) (*gnmiTarget, error) { +func NewTarget(ctx context.Context, name string, cfg *config.SBI, runningStore targetTypes.RunningStore, schemaClient dsutils.SchemaClientBound, opts ...grpc.DialOption) (*gnmiTarget, error) { tc := &types.TargetConfig{ Name: name, Address: fmt.Sprintf("%s:%d", cfg.Address, cfg.Port), @@ -70,10 +74,20 @@ func newGNMITarget(ctx context.Context, name string, cfg *config.SBI, opts ...gr tc.Insecure = pointer.ToBool(true) } gt := &gnmiTarget{ - target: gtarget.NewTarget(tc), - encodings: make(map[gnmi.Encoding]struct{}), - cfg: cfg, + target: gtarget.NewTarget(tc), + encodings: make(map[gnmi.Encoding]struct{}), + cfg: cfg, + syncs: map[string]GnmiSync{}, + runningStore: runningStore, + schemaClient: schemaClient, } + + opts = append(opts, grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: 10 * time.Second, // send pings every 10s if no activity + Timeout: 2 * time.Second, // wait 2s for ping ack before considering dead + PermitWithoutStream: true, // send pings even without active RPCs + })) + err := gt.target.CreateGNMIClient(ctx, opts...) if err != nil { return nil, err @@ -87,39 +101,15 @@ func newGNMITarget(ctx context.Context, name string, cfg *config.SBI, opts ...gr gt.encodings[enc] = struct{}{} } - if _, exists := gt.encodings[gnmi.Encoding(encoding(cfg.GnmiOptions.Encoding))]; !exists { + if _, exists := gt.encodings[gnmi.Encoding(gnmiutils.ParseGnmiEncoding(cfg.GnmiOptions.Encoding))]; !exists { return nil, fmt.Errorf("encoding %q not supported", cfg.GnmiOptions.Encoding) } return gt, nil } -// sdcpbDataTypeToGNMIType helper to convert the sdcpb data type to the gnmi data type -func sdcpbDataTypeToGNMIType(x sdcpb.DataType) (gnmi.GetRequest_DataType, error) { - switch x { - case sdcpb.DataType_ALL: - return gnmi.GetRequest_ALL, nil - case sdcpb.DataType_CONFIG: - return gnmi.GetRequest_CONFIG, nil - case sdcpb.DataType_STATE: - return gnmi.GetRequest_STATE, nil - } - return 9999, fmt.Errorf("unable to convert sdcpb DataType %s to gnmi DataType", x) -} - -// sdcpbEncodingToGNMIENcoding helper to convert sdcpb encoding to gnmi encoding -func sdcpbEncodingToGNMIENcoding(x sdcpb.Encoding) (gnmi.Encoding, error) { - switch x { - case sdcpb.Encoding_JSON: - return gnmi.Encoding_JSON, nil - case sdcpb.Encoding_JSON_IETF: - return gnmi.Encoding_JSON_IETF, nil - case sdcpb.Encoding_PROTO: - return gnmi.Encoding_PROTO, nil - case sdcpb.Encoding_STRING: - return gnmi.Encoding_ASCII, nil - } - return 9999, fmt.Errorf("unable to convert sdcpb encoding %s to gnmi encoding", x) +func (t *gnmiTarget) Subscribe(ctx context.Context, req *gnmi.SubscribeRequest, subscriptionName string) { + t.Subscribe(ctx, req, subscriptionName) } func (t *gnmiTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) { @@ -132,13 +122,13 @@ func (t *gnmiTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb } // convert sdcpb data type to gnmi data type - gnmiReq.Type, err = sdcpbDataTypeToGNMIType(req.GetDataType()) + gnmiReq.Type, err = gnmiutils.SdcpbDataTypeToGNMIType(req.GetDataType()) if err != nil { return nil, err } // convert sdcpb encoding to gnmi encoding - gnmiReq.Encoding, err = sdcpbEncodingToGNMIENcoding(req.Encoding) + gnmiReq.Encoding, err = gnmiutils.SdcpbEncodingToGNMIENcoding(req.Encoding) if err != nil { return nil, err } @@ -158,7 +148,7 @@ func (t *gnmiTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb return schemaRsp, nil } -func (t *gnmiTarget) Set(ctx context.Context, source TargetSource) (*sdcpb.SetDataResponse, error) { +func (t *gnmiTarget) Set(ctx context.Context, source targetTypes.TargetSource) (*sdcpb.SetDataResponse, error) { var upds []*sdcpb.Update var deletes []*sdcpb.Path var err error @@ -239,8 +229,8 @@ func (t *gnmiTarget) Set(ctx context.Context, source TargetSource) (*sdcpb.SetDa return schemaSetRsp, nil } -func (t *gnmiTarget) Status() *TargetStatus { - result := NewTargetStatus(TargetStatusNotConnected) +func (t *gnmiTarget) Status() *targetTypes.TargetStatus { + result := targetTypes.NewTargetStatus(targetTypes.TargetStatusNotConnected) if t == nil || t.target == nil { result.Details = "connection not initialized" @@ -248,76 +238,106 @@ func (t *gnmiTarget) Status() *TargetStatus { } switch t.target.ConnState() { case connectivity.Ready.String(), connectivity.Idle.String(): - result.Status = TargetStatusConnected + result.Status = targetTypes.TargetStatusConnected result.Details = t.target.ConnState() case connectivity.Connecting.String(), connectivity.Shutdown.String(), connectivity.TransientFailure.String(): - result.Status = TargetStatusNotConnected + result.Status = targetTypes.TargetStatusNotConnected result.Details = t.target.ConnState() } return result } -func (t *gnmiTarget) Sync(octx context.Context, syncConfig *config.Sync, syncCh chan *SyncUpdate) { - if t != nil && t.target != nil && t.target.Config != nil { - log.Infof("starting target %s sync", t.target.Config.Name) - } - var cancel context.CancelFunc - var ctx context.Context - var err error -START: - if cancel != nil { - cancel() - } - ctx, cancel = context.WithCancel(octx) - defer cancel() - - // todo: do not run read subscriptions for GET - for _, gnmiSync := range syncConfig.Config { - switch gnmiSync.Mode { +func (t *gnmiTarget) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) error { + var g GnmiSync + for _, sp := range sps { + switch sp.Mode { case "once": - err = t.periodicSync(ctx, gnmiSync) + g = NewOnceSync(ctx, t, sp, t.runningStore) case "get": - err = t.getSync(ctx, gnmiSync, syncCh) + g = NewGetSync(ctx, t, sp, t.runningStore, t.schemaClient) default: - err = t.streamSync(ctx, gnmiSync) + g = NewStreamSync(ctx, t, sp, t.runningStore) } + t.syncs[sp.Name] = g + err := g.Start() if err != nil { - log.Errorf("target=%s: failed to sync: %v", t.target.Config.Name, err) - time.Sleep(syncRetryWaitTime) - goto START + return err } } + return nil +} - defer t.target.StopSubscriptions() - - rspch, errCh := t.target.ReadSubscriptions() - for { - select { - case <-ctx.Done(): - if !errors.Is(ctx.Err(), context.Canceled) { - log.Errorf("datastore %s sync stopped: %v", t.target.Config.Name, ctx.Err()) - } - return - case rsp := <-rspch: - switch r := rsp.Response.Response.(type) { - case *gnmi.SubscribeResponse_Update: - syncCh <- &SyncUpdate{ - Store: rsp.SubscriptionName, - Update: utils.ToSchemaNotification(r.Update), - } - } - case err := <-errCh: - if err.Err != nil { - t.target.StopSubscriptions() - log.Errorf("%s: sync subscription failed: %v", t.target.Config.Name, err) - time.Sleep(time.Second) - goto START - } +func (t *gnmiTarget) Sync() error { + for _, v := range t.syncs { + err := v.Start() + if err != nil { + return err } } + return nil } +// func (t *gnmiTarget) Sync(octx context.Context, syncConfig *config.Sync) { +// if t != nil && t.target != nil && t.target.Config != nil { +// log.Infof("starting target %s sync", t.target.Config.Name) +// } +// var cancel context.CancelFunc +// var ctx context.Context +// var err error +// START: +// if cancel != nil { +// cancel() +// } +// ctx, cancel = context.WithCancel(octx) +// defer cancel() + +// // todo: do not run read subscriptions for GET +// for _, gnmiSync := range syncConfig.Config { +// switch gnmiSync.Mode { +// case "once": +// err = t.periodicSync(ctx, gnmiSync) +// case "get": +// err = t.getSync(ctx, gnmiSync) +// default: +// err = t.streamSync(ctx, gnmiSync) +// } +// if err != nil { +// log.Errorf("target=%s: failed to sync: %v", t.target.Config.Name, err) +// time.Sleep(syncRetryWaitTime) +// goto START +// } +// } + +// defer t.target.StopSubscriptions() + +// rspch, errCh := t.target.ReadSubscriptions() +// for { +// select { +// case <-ctx.Done(): +// if !errors.Is(ctx.Err(), context.Canceled) { +// log.Errorf("datastore %s sync stopped: %v", t.target.Config.Name, ctx.Err()) +// } +// return +// case rsp := <-rspch: +// switch r := rsp.Response.Response.(type) { +// case *gnmi.SubscribeResponse_Update: +// syncCh <- &SyncUpdate{ +// Store: rsp.SubscriptionName, +// Update: utils.ToSchemaNotification(r.Update), +// } +// } +// case err := <-errCh: +// if err.Err != nil { +// t.target.StopSubscriptions() +// log.Errorf("%s: sync subscription failed: %v", t.target.Config.Name, err) +// time.Sleep(time.Second) +// goto START +// } +// } +// } +// } + func (t *gnmiTarget) Close() error { if t == nil { return nil @@ -328,161 +348,6 @@ func (t *gnmiTarget) Close() error { return t.target.Close() } -func sdcpbEncoding(e string) int { - enc, ok := sdcpb.Encoding_value[strings.ToUpper(e)] - if ok { - return int(enc) - } - en, err := strconv.Atoi(e) - if err != nil { - return 0 - } - return en -} - -func encoding(e string) int { - enc, ok := gnmi.Encoding_value[strings.ToUpper(e)] - if ok { - return int(enc) - } - en, err := strconv.Atoi(e) - if err != nil { - return 0 - } - return en -} - -func (t *gnmiTarget) getSync(ctx context.Context, gnmiSync *config.SyncProtocol, syncCh chan *SyncUpdate) error { - // iterate syncConfig - paths := make([]*sdcpb.Path, 0, len(gnmiSync.Paths)) - // iterate referenced paths - for _, p := range gnmiSync.Paths { - path, err := sdcpb.ParsePath(p) - if err != nil { - return err - } - // add the parsed path - paths = append(paths, path) - } - - req := &sdcpb.GetDataRequest{ - Name: gnmiSync.Name, - Path: paths, - DataType: sdcpb.DataType_CONFIG, - Encoding: sdcpb.Encoding(sdcpbEncoding(gnmiSync.Encoding)), - } - - go t.internalGetSync(ctx, req, syncCh) - - go func() { - ticker := time.NewTicker(gnmiSync.Interval) - defer ticker.Stop() - for { - select { - case <-ctx.Done(): - return - case <-ticker.C: - t.internalGetSync(ctx, req, syncCh) - } - } - }() - - return nil -} - -func (t *gnmiTarget) internalGetSync(ctx context.Context, req *sdcpb.GetDataRequest, syncCh chan *SyncUpdate) { - // execute gnmi get - resp, err := t.Get(ctx, req) - if err != nil { - log.Errorf("sync error: %v", err) - return - } - - // push notifications into syncCh - syncCh <- &SyncUpdate{ - Start: true, - } - notificationsCount := 0 - - for idx, n := range resp.GetNotification() { - _ = idx - syncCh <- &SyncUpdate{ - Update: n, - } - notificationsCount++ - } - log.Debugf("%s: synced %d notifications", t.target.Config.Name, notificationsCount) - syncCh <- &SyncUpdate{ - End: true, - } -} - -func (t *gnmiTarget) periodicSync(ctx context.Context, gnmiSync *config.SyncProtocol) error { - opts := make([]gapi.GNMIOption, 0) - subscriptionOpts := make([]gapi.GNMIOption, 0) - for _, p := range gnmiSync.Paths { - subscriptionOpts = append(subscriptionOpts, gapi.Path(p)) - } - opts = append(opts, - gapi.EncodingCustom(encoding(gnmiSync.Encoding)), - gapi.SubscriptionListModeONCE(), - gapi.Subscription(subscriptionOpts...), - gapi.DataTypeCONFIG(), - ) - subReq, err := gapi.NewSubscribeRequest(opts...) - if err != nil { - return err - } - // initial subscribe ONCE - go t.target.Subscribe(ctx, subReq, gnmiSync.Name) - // periodic subscribe ONCE - go func(gnmiSync *config.SyncProtocol) { - ticker := time.NewTicker(gnmiSync.Interval) - defer ticker.Stop() - for { - select { - case <-ctx.Done(): - return - case <-ticker.C: - t.target.Subscribe(ctx, subReq, gnmiSync.Name) - } - } - }(gnmiSync) - return nil -} - -func (t *gnmiTarget) streamSync(ctx context.Context, gnmiSync *config.SyncProtocol) error { - opts := make([]gapi.GNMIOption, 0) - subscriptionOpts := make([]gapi.GNMIOption, 0) - for _, p := range gnmiSync.Paths { - subscriptionOpts = append(subscriptionOpts, gapi.Path(p)) - } - switch gnmiSync.Mode { - case "sample": - subscriptionOpts = append(subscriptionOpts, gapi.SubscriptionModeSAMPLE()) - case "on-change": - subscriptionOpts = append(subscriptionOpts, gapi.SubscriptionModeON_CHANGE()) - } - - if gnmiSync.Interval > 0 { - subscriptionOpts = append(subscriptionOpts, gapi.SampleInterval(gnmiSync.Interval)) - } - opts = append(opts, - gapi.EncodingCustom(encoding(gnmiSync.Encoding)), - gapi.SubscriptionListModeSTREAM(), - gapi.Subscription(subscriptionOpts...), - gapi.DataTypeCONFIG(), - ) - subReq, err := gapi.NewSubscribeRequest(opts...) - if err != nil { - return err - - } - log.Infof("sync %q: subRequest: %v", gnmiSync.Name, subReq) - go t.target.Subscribe(ctx, subReq, gnmiSync.Name) - return nil -} - func (t *gnmiTarget) convertKeyUpdates(upd *sdcpb.Update) *gnmi.Update { if !pathIsKeyAsLeaf(upd.GetPath()) { return &gnmi.Update{ diff --git a/pkg/datastore/target/gnmi/gnmisync.go b/pkg/datastore/target/gnmi/gnmisync.go new file mode 100644 index 00000000..bd73e388 --- /dev/null +++ b/pkg/datastore/target/gnmi/gnmisync.go @@ -0,0 +1,6 @@ +package gnmi + +type GnmiSync interface { + Start() error + Stop() error +} diff --git a/pkg/datastore/target/gnmi/once.go b/pkg/datastore/target/gnmi/once.go new file mode 100644 index 00000000..a73cc4fa --- /dev/null +++ b/pkg/datastore/target/gnmi/once.go @@ -0,0 +1,80 @@ +package gnmi + +import ( + "context" + "time" + + "github.com/openconfig/gnmi/proto/gnmi" + gapi "github.com/openconfig/gnmic/pkg/api" + "github.com/sdcio/data-server/pkg/config" + "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" + "github.com/sdcio/data-server/pkg/datastore/target/types" +) + +type OnceSync struct { + config *config.SyncProtocol + target SyncTarget + cancel context.CancelFunc + runningStore types.RunningStore + ctx context.Context +} + +func NewOnceSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore) *OnceSync { + ctx, cancel := context.WithCancel(ctx) + return &OnceSync{ + config: c, + target: target, + cancel: cancel, + runningStore: runningStore, + } +} + +func (s *OnceSync) syncConfig() (*gnmi.SubscribeRequest, error) { + opts := make([]gapi.GNMIOption, 0) + subscriptionOpts := make([]gapi.GNMIOption, 0) + for _, p := range s.config.Paths { + subscriptionOpts = append(subscriptionOpts, gapi.Path(p)) + } + opts = append(opts, + gapi.EncodingCustom(utils.ParseGnmiEncoding(s.config.Encoding)), + gapi.SubscriptionListModeONCE(), + gapi.Subscription(subscriptionOpts...), + gapi.DataTypeCONFIG(), + ) + subReq, err := gapi.NewSubscribeRequest(opts...) + if err != nil { + return nil, err + } + return subReq, nil +} + +func (s *OnceSync) Start() error { + + subReq, err := s.syncConfig() + if err != nil { + return err + } + + // initial subscribe ONCE + go s.target.Subscribe(s.ctx, subReq, s.config.Name) + // periodic subscribe ONCE + go func(interval time.Duration, name string) { + ticker := time.NewTicker(interval) + defer ticker.Stop() + for { + select { + case <-s.ctx.Done(): + return + case <-ticker.C: + s.target.Subscribe(s.ctx, subReq, name) + } + } + }(s.config.Interval, s.config.Name) + return nil +} + +func (s *OnceSync) Stop() error { + // TODO + s.cancel() + return nil +} diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go new file mode 100644 index 00000000..eb101dbb --- /dev/null +++ b/pkg/datastore/target/gnmi/stream.go @@ -0,0 +1,82 @@ +package gnmi + +import ( + "context" + + "github.com/openconfig/gnmi/proto/gnmi" + gapi "github.com/openconfig/gnmic/pkg/api" + "github.com/sdcio/data-server/pkg/config" + "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" + "github.com/sdcio/data-server/pkg/datastore/target/types" + log "github.com/sirupsen/logrus" +) + +type StreamSync struct { + config *config.SyncProtocol + target SyncTarget + cancel context.CancelFunc + runningStore types.RunningStore + ctx context.Context +} + +func NewStreamSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore) *StreamSync { + ctx, cancel := context.WithCancel(ctx) + + return &StreamSync{ + config: c, + target: target, + cancel: cancel, + runningStore: runningStore, + } +} + +func (s *StreamSync) syncConfig() (*gnmi.SubscribeRequest, error) { + + opts := make([]gapi.GNMIOption, 0) + subscriptionOpts := make([]gapi.GNMIOption, 0) + for _, p := range s.config.Paths { + subscriptionOpts = append(subscriptionOpts, gapi.Path(p)) + } + switch s.config.Mode { + case "sample": + subscriptionOpts = append(subscriptionOpts, gapi.SubscriptionModeSAMPLE()) + case "on-change": + subscriptionOpts = append(subscriptionOpts, gapi.SubscriptionModeON_CHANGE()) + } + + if s.config.Interval > 0 { + subscriptionOpts = append(subscriptionOpts, gapi.SampleInterval(s.config.Interval)) + } + opts = append(opts, + gapi.EncodingCustom(utils.ParseGnmiEncoding(s.config.Encoding)), + gapi.SubscriptionListModeSTREAM(), + gapi.Subscription(subscriptionOpts...), + gapi.DataTypeCONFIG(), + ) + subReq, err := gapi.NewSubscribeRequest(opts...) + if err != nil { + return nil, err + } + return subReq, nil +} + +func (s *StreamSync) Stop() error { + s.cancel() + return nil +} + +func (s *StreamSync) Start() error { + + subReq, err := s.syncConfig() + if err != nil { + return err + } + + log.Infof("sync %q: subRequest: %v", s.config.Name, subReq) + go s.target.Subscribe(s.ctx, subReq, s.config.Name) + return nil +} + +type SyncTarget interface { + Subscribe(ctx context.Context, req *gnmi.SubscribeRequest, subscriptionName string) +} diff --git a/pkg/datastore/target/gnmi/utils/utils.go b/pkg/datastore/target/gnmi/utils/utils.go new file mode 100644 index 00000000..2d1b4129 --- /dev/null +++ b/pkg/datastore/target/gnmi/utils/utils.go @@ -0,0 +1,62 @@ +package utils + +import ( + "fmt" + "strconv" + "strings" + + "github.com/openconfig/gnmi/proto/gnmi" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" +) + +// sdcpbDataTypeToGNMIType helper to convert the sdcpb data type to the gnmi data type +func SdcpbDataTypeToGNMIType(x sdcpb.DataType) (gnmi.GetRequest_DataType, error) { + switch x { + case sdcpb.DataType_ALL: + return gnmi.GetRequest_ALL, nil + case sdcpb.DataType_CONFIG: + return gnmi.GetRequest_CONFIG, nil + case sdcpb.DataType_STATE: + return gnmi.GetRequest_STATE, nil + } + return 9999, fmt.Errorf("unable to convert sdcpb DataType %s to gnmi DataType", x) +} + +// sdcpbEncodingToGNMIENcoding helper to convert sdcpb encoding to gnmi encoding +func SdcpbEncodingToGNMIENcoding(x sdcpb.Encoding) (gnmi.Encoding, error) { + switch x { + case sdcpb.Encoding_JSON: + return gnmi.Encoding_JSON, nil + case sdcpb.Encoding_JSON_IETF: + return gnmi.Encoding_JSON_IETF, nil + case sdcpb.Encoding_PROTO: + return gnmi.Encoding_PROTO, nil + case sdcpb.Encoding_STRING: + return gnmi.Encoding_ASCII, nil + } + return 9999, fmt.Errorf("unable to convert sdcpb encoding %s to gnmi encoding", x) +} + +func ParseGnmiEncoding(e string) int { + enc, ok := gnmi.Encoding_value[strings.ToUpper(e)] + if ok { + return int(enc) + } + en, err := strconv.Atoi(e) + if err != nil { + return 0 + } + return en +} + +func ParseSdcpbEncoding(e string) int { + enc, ok := sdcpb.Encoding_value[strings.ToUpper(e)] + if ok { + return int(enc) + } + en, err := strconv.Atoi(e) + if err != nil { + return 0 + } + return en +} diff --git a/pkg/datastore/target/nc.go b/pkg/datastore/target/nc.go index d06075f3..667d8ed1 100644 --- a/pkg/datastore/target/nc.go +++ b/pkg/datastore/target/nc.go @@ -30,6 +30,7 @@ import ( schemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" "github.com/sdcio/data-server/pkg/datastore/target/netconf" "github.com/sdcio/data-server/pkg/datastore/target/netconf/driver/scrapligo" + "github.com/sdcio/data-server/pkg/datastore/target/types" ) type ncTarget struct { @@ -62,7 +63,7 @@ func newNCTarget(_ context.Context, name string, cfg *config.SBI, schemaClient s func (t *ncTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) { if !t.Status().IsConnected() { - return nil, fmt.Errorf("%s", TargetStatusNotConnected) + return nil, fmt.Errorf("%s", types.TargetStatusNotConnected) } source := "running" @@ -121,9 +122,9 @@ func (t *ncTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.G return result, nil } -func (t *ncTarget) Set(ctx context.Context, source TargetSource) (*sdcpb.SetDataResponse, error) { +func (t *ncTarget) Set(ctx context.Context, source types.TargetSource) (*sdcpb.SetDataResponse, error) { if !t.Status().IsConnected() { - return nil, fmt.Errorf("%s", TargetStatusNotConnected) + return nil, fmt.Errorf("%s", types.TargetStatusNotConnected) } switch t.sbiConfig.NetconfOptions.CommitDatastore { @@ -136,26 +137,26 @@ func (t *ncTarget) Set(ctx context.Context, source TargetSource) (*sdcpb.SetData return nil, fmt.Errorf("unknown commit-datastore: %s", t.sbiConfig.NetconfOptions.CommitDatastore) } -func (t *ncTarget) Status() *TargetStatus { - result := NewTargetStatus(TargetStatusNotConnected) +func (t *ncTarget) Status() *types.TargetStatus { + result := types.NewTargetStatus(types.TargetStatusNotConnected) if t == nil || t.driver == nil { result.Details = "connection not initialized" return result } if t.driver.IsAlive() { - result.Status = TargetStatusConnected + result.Status = types.TargetStatusConnected } return result } -func (t *ncTarget) Sync(ctx context.Context, syncConfig *config.Sync, syncCh chan *SyncUpdate) { +func (t *ncTarget) Sync(ctx context.Context, syncConfig *config.Sync) { log.Infof("starting target %s [%s] sync", t.name, t.sbiConfig.Address) for _, ncc := range syncConfig.Config { // periodic get log.Debugf("target %s, starting sync: %s, Interval: %s, Paths: [ \"%s\" ]", t.name, ncc.Name, ncc.Interval.String(), strings.Join(ncc.Paths, "\", \"")) go func(ncSync *config.SyncProtocol) { - t.internalSync(ctx, ncSync, true, syncCh) + t.internalSync(ctx, ncSync, true) ticker := time.NewTicker(ncSync.Interval) defer ticker.Stop() for { @@ -163,7 +164,7 @@ func (t *ncTarget) Sync(ctx context.Context, syncConfig *config.Sync, syncCh cha case <-ctx.Done(): return case <-ticker.C: - t.internalSync(ctx, ncSync, false, syncCh) + t.internalSync(ctx, ncSync, false) } } }(ncc) @@ -175,7 +176,7 @@ func (t *ncTarget) Sync(ctx context.Context, syncConfig *config.Sync, syncCh cha } } -func (t *ncTarget) internalSync(ctx context.Context, sc *config.SyncProtocol, force bool, syncCh chan *SyncUpdate) { +func (t *ncTarget) internalSync(ctx context.Context, sc *config.SyncProtocol, force bool) { if !t.Status().IsConnected() { return } @@ -209,22 +210,23 @@ func (t *ncTarget) internalSync(ctx context.Context, sc *config.SyncProtocol, fo } return } - // push notifications into syncCh - syncCh <- &SyncUpdate{ - Start: true, - Force: force, - } - notificationsCount := 0 - for _, n := range resp.GetNotification() { - syncCh <- &SyncUpdate{ - Update: n, - } - notificationsCount++ - } - log.Debugf("%s: sync-ed %d notifications", t.name, notificationsCount) - syncCh <- &SyncUpdate{ - End: true, - } + _ = resp + // // push notifications into syncCh + // syncCh <- &SyncUpdate{ + // Start: true, + // Force: force, + // } + // notificationsCount := 0 + // for _, n := range resp.GetNotification() { + // syncCh <- &SyncUpdate{ + // Update: n, + // } + // notificationsCount++ + // } + // log.Debugf("%s: sync-ed %d notifications", t.name, notificationsCount) + // syncCh <- &SyncUpdate{ + // End: true, + // } } func (t *ncTarget) Close() error { @@ -259,7 +261,7 @@ func (t *ncTarget) reconnect() { } } -func (t *ncTarget) setRunning(source TargetSource) (*sdcpb.SetDataResponse, error) { +func (t *ncTarget) setRunning(source types.TargetSource) (*sdcpb.SetDataResponse, error) { xtree, err := source.ToXML(true, t.sbiConfig.NetconfOptions.IncludeNS, t.sbiConfig.NetconfOptions.OperationWithNamespace, t.sbiConfig.NetconfOptions.UseOperationRemove) if err != nil { @@ -319,7 +321,7 @@ func filterRPCErrors(xml *etree.Document, severity string) ([]string, error) { return result, nil } -func (t *ncTarget) setCandidate(source TargetSource) (*sdcpb.SetDataResponse, error) { +func (t *ncTarget) setCandidate(source types.TargetSource) (*sdcpb.SetDataResponse, error) { xtree, err := source.ToXML(true, t.sbiConfig.NetconfOptions.IncludeNS, t.sbiConfig.NetconfOptions.OperationWithNamespace, t.sbiConfig.NetconfOptions.UseOperationRemove) if err != nil { return nil, err diff --git a/pkg/datastore/target/noop.go b/pkg/datastore/target/noop/noop.go similarity index 85% rename from pkg/datastore/target/noop.go rename to pkg/datastore/target/noop/noop.go index 68a781ba..c27c7bf7 100644 --- a/pkg/datastore/target/noop.go +++ b/pkg/datastore/target/noop/noop.go @@ -22,6 +22,7 @@ import ( log "github.com/sirupsen/logrus" "github.com/sdcio/data-server/pkg/config" + "github.com/sdcio/data-server/pkg/datastore/target/types" ) type noopTarget struct { @@ -53,7 +54,7 @@ func (t *noopTarget) Get(_ context.Context, req *sdcpb.GetDataRequest) (*sdcpb.G return result, nil } -func (t *noopTarget) Set(ctx context.Context, source TargetSource) (*sdcpb.SetDataResponse, error) { +func (t *noopTarget) Set(ctx context.Context, source types.TargetSource) (*sdcpb.SetDataResponse, error) { upds, err := source.ToProtoUpdates(ctx, true) if err != nil { @@ -86,13 +87,13 @@ func (t *noopTarget) Set(ctx context.Context, source TargetSource) (*sdcpb.SetDa return result, nil } -func (t *noopTarget) Status() *TargetStatus { - return &TargetStatus{ - Status: TargetStatusConnected, +func (t *noopTarget) Status() *types.TargetStatus { + return &types.TargetStatus{ + Status: types.TargetStatusConnected, } } -func (t *noopTarget) Sync(ctx context.Context, _ *config.Sync, syncCh chan *SyncUpdate) { +func (t *noopTarget) Sync(ctx context.Context, _ *config.Sync) { log.Infof("starting target %s sync", t.name) } diff --git a/pkg/datastore/target/target.go b/pkg/datastore/target/target.go index 29b90177..e1c76962 100644 --- a/pkg/datastore/target/target.go +++ b/pkg/datastore/target/target.go @@ -18,12 +18,13 @@ import ( "context" "fmt" - "github.com/beevik/etree" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "google.golang.org/grpc" "github.com/sdcio/data-server/pkg/config" schemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" + "github.com/sdcio/data-server/pkg/datastore/target/gnmi" + "github.com/sdcio/data-server/pkg/datastore/target/types" ) const ( @@ -32,70 +33,39 @@ const ( targetTypeGNMI = "gnmi" ) -type TargetStatus struct { - Status TargetConnectionStatus - Details string -} - -func NewTargetStatus(status TargetConnectionStatus) *TargetStatus { - return &TargetStatus{ - Status: status, - } -} -func (ts *TargetStatus) IsConnected() bool { - return ts.Status == TargetStatusConnected -} - -type TargetConnectionStatus string - -const ( - TargetStatusConnected TargetConnectionStatus = "connected" - TargetStatusNotConnected TargetConnectionStatus = "not connected" -) - type Target interface { Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) - Set(ctx context.Context, source TargetSource) (*sdcpb.SetDataResponse, error) - Sync(ctx context.Context, syncConfig *config.Sync, syncCh chan *SyncUpdate) - Status() *TargetStatus + Set(ctx context.Context, source types.TargetSource) (*sdcpb.SetDataResponse, error) + AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) error + Status() *types.TargetStatus Close() error } -func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaClient.SchemaClientBound, opts ...grpc.DialOption) (Target, error) { +func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaClient.SchemaClientBound, runningStore types.RunningStore, syncConfigs []*config.SyncProtocol, opts ...grpc.DialOption) (Target, error) { + var t Target + var err error + + targetContext := context.Background() + switch cfg.Type { case targetTypeGNMI: - return newGNMITarget(ctx, name, cfg, opts...) - case targetTypeNETCONF: - return newNCTarget(ctx, name, cfg, schemaClient) - case targetTypeNOOP, "": - return newNoopTarget(ctx, name) + t, err = gnmi.NewTarget(targetContext, name, cfg, runningStore, schemaClient, opts...) + if err != nil { + return nil, err + } + // case targetTypeNETCONF: + // return newNCTarget(ctx, name, cfg, schemaClient) + // case targetTypeNOOP, "": + // return newNoopTarget(ctx, name) + default: + return nil, fmt.Errorf("unknown DS target type %q", cfg.Type) + } - return nil, fmt.Errorf("unknown DS target type %q", cfg.Type) -} -type SyncUpdate struct { - // identifies the store this updates needs to be written to if Sync.Validate == false - Store string - // The received update - Update *sdcpb.Notification - // if true indicates the start of cache pruning - Start bool - // if true and start is true indicates first sync iteration, - // it overrides any ongoing pruning in the cache. - Force bool - // if true indicates the end of a sync iteration. - // triggers the pruning on the cache side. - End bool -} + err = t.AddSyncs(targetContext, syncConfigs...) + if err != nil { + return nil, err + } -type TargetSource interface { - // ToJson returns the Tree contained structure as JSON - // use e.g. json.MarshalIndent() on the returned struct - ToJson(onlyNewOrUpdated bool) (any, error) - // ToJsonIETF returns the Tree contained structure as JSON_IETF - // use e.g. json.MarshalIndent() on the returned struct - ToJsonIETF(onlyNewOrUpdated bool) (any, error) - ToXML(onlyNewOrUpdated bool, honorNamespace bool, operationWithNamespace bool, useOperationRemove bool) (*etree.Document, error) - ToProtoUpdates(ctx context.Context, onlyNewOrUpdated bool) ([]*sdcpb.Update, error) - ToProtoDeletes(ctx context.Context) ([]*sdcpb.Path, error) + return t, nil } diff --git a/pkg/datastore/target/types/runningstore.go b/pkg/datastore/target/types/runningstore.go new file mode 100644 index 00000000..bf1967d1 --- /dev/null +++ b/pkg/datastore/target/types/runningstore.go @@ -0,0 +1,13 @@ +package types + +import ( + "context" + + "github.com/sdcio/data-server/pkg/tree" + "github.com/sdcio/sdc-protos/tree_persist" +) + +type RunningStore interface { + ApplyToRunning(ctx context.Context, i *tree_persist.Intent) error + NewEmptyTree(ctx context.Context) (*tree.RootEntry, error) +} diff --git a/pkg/datastore/target/types/syncresult.go b/pkg/datastore/target/types/syncresult.go new file mode 100644 index 00000000..a951ad9a --- /dev/null +++ b/pkg/datastore/target/types/syncresult.go @@ -0,0 +1,17 @@ +package types + +import ( + sdcpb "github.com/sdcio/sdc-protos/sdcpb" +) + +type SyncResult struct { + deltes []*sdcpb.Path +} + +func NewSyncResult() *SyncResult { + return &SyncResult{} +} + +func (r *SyncResult) AddDeletes(d ...*sdcpb.Path) { + r.deltes = append(r.deltes, d...) +} diff --git a/pkg/datastore/target/types/targetsource.go b/pkg/datastore/target/types/targetsource.go new file mode 100644 index 00000000..69d66d95 --- /dev/null +++ b/pkg/datastore/target/types/targetsource.go @@ -0,0 +1,20 @@ +package types + +import ( + "context" + + "github.com/beevik/etree" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" +) + +type TargetSource interface { + // ToJson returns the Tree contained structure as JSON + // use e.g. json.MarshalIndent() on the returned struct + ToJson(onlyNewOrUpdated bool) (any, error) + // ToJsonIETF returns the Tree contained structure as JSON_IETF + // use e.g. json.MarshalIndent() on the returned struct + ToJsonIETF(onlyNewOrUpdated bool) (any, error) + ToXML(onlyNewOrUpdated bool, honorNamespace bool, operationWithNamespace bool, useOperationRemove bool) (*etree.Document, error) + ToProtoUpdates(ctx context.Context, onlyNewOrUpdated bool) ([]*sdcpb.Update, error) + ToProtoDeletes(ctx context.Context) ([]*sdcpb.Path, error) +} diff --git a/pkg/datastore/target/types/targetstatus.go b/pkg/datastore/target/types/targetstatus.go new file mode 100644 index 00000000..249f8767 --- /dev/null +++ b/pkg/datastore/target/types/targetstatus.go @@ -0,0 +1,22 @@ +package types + +type TargetStatus struct { + Status TargetConnectionStatus + Details string +} + +func NewTargetStatus(status TargetConnectionStatus) *TargetStatus { + return &TargetStatus{ + Status: status, + } +} +func (ts *TargetStatus) IsConnected() bool { + return ts.Status == TargetStatusConnected +} + +type TargetConnectionStatus string + +const ( + TargetStatusConnected TargetConnectionStatus = "connected" + TargetStatusNotConnected TargetConnectionStatus = "not connected" +) diff --git a/pkg/datastore/types/target_source_replace.go b/pkg/datastore/types/target_source_replace.go index 627e4e70..eb79b5d7 100644 --- a/pkg/datastore/types/target_source_replace.go +++ b/pkg/datastore/types/target_source_replace.go @@ -4,7 +4,7 @@ import ( "context" "github.com/beevik/etree" - "github.com/sdcio/data-server/pkg/datastore/target" + "github.com/sdcio/data-server/pkg/datastore/target/types" "github.com/sdcio/data-server/pkg/utils" sdcpb "github.com/sdcio/sdc-protos/sdcpb" ) @@ -16,11 +16,11 @@ import ( // - ToXML(...) returns the TargetSource generated etree.Document, but sets the // replace flag on the root element type TargetSourceReplace struct { - target.TargetSource + types.TargetSource } // NewTargetSourceReplace constructor for TargetSourceReplace -func NewTargetSourceReplace(ts target.TargetSource) *TargetSourceReplace { +func NewTargetSourceReplace(ts types.TargetSource) *TargetSourceReplace { return &TargetSourceReplace{ ts, } diff --git a/pkg/server/datastore.go b/pkg/server/datastore.go index b8b04a61..41cc220e 100644 --- a/pkg/server/datastore.go +++ b/pkg/server/datastore.go @@ -29,7 +29,7 @@ import ( "github.com/sdcio/data-server/pkg/config" "github.com/sdcio/data-server/pkg/datastore" - "github.com/sdcio/data-server/pkg/datastore/target" + targettypes "github.com/sdcio/data-server/pkg/datastore/target/types" ) // datastore @@ -252,9 +252,9 @@ func (s *Server) datastoreToRsp(ctx context.Context, ds *datastore.Datastore) (* } // map datastore sbi conn state to sdcpb.TargetStatus switch ds.ConnectionState().Status { - case target.TargetStatusConnected: + case targettypes.TargetStatusConnected: rsp.Target.Status = sdcpb.TargetStatus_CONNECTED - case target.TargetStatusNotConnected: + case targettypes.TargetStatusNotConnected: rsp.Target.Status = sdcpb.TargetStatus_NOT_CONNECTED default: rsp.Target.Status = sdcpb.TargetStatus_UNKNOWN From 272f30e960a49e4c750f3014179e102d216869be Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 8 Oct 2025 12:26:44 +0200 Subject: [PATCH 05/44] updare --- pkg/datastore/sync.go | 83 ------------------------------- pkg/datastore/target/gnmi/gnmi.go | 2 +- 2 files changed, 1 insertion(+), 84 deletions(-) diff --git a/pkg/datastore/sync.go b/pkg/datastore/sync.go index cf9a9654..657f0aca 100644 --- a/pkg/datastore/sync.go +++ b/pkg/datastore/sync.go @@ -10,89 +10,6 @@ import ( "github.com/sdcio/sdc-protos/tree_persist" ) -// func (d *Datastore) SyncOld(ctx context.Context) { -// go d.sbi.Sync(ctx, -// d.config.Sync, -// ) - -// var err error -// var startTs int64 - -// d.syncTreeCandidate, err = tree.NewTreeRoot(ctx, tree.NewTreeContext(d.schemaClient, tree.RunningIntentName)) -// if err != nil { -// log.Errorf("creating a new synctree candidate: %v", err) -// return -// } - -// for { -// select { -// case <-ctx.Done(): -// if !errors.Is(ctx.Err(), context.Canceled) { -// log.Errorf("datastore %s sync stopped: %v", d.Name(), ctx.Err()) -// } -// return -// case syncup := <-d.synCh: -// switch { -// case syncup.Start: -// log.Debugf("%s: sync start", d.Name()) -// startTs = time.Now().Unix() - -// case syncup.End: -// log.Debugf("%s: sync end", d.Name()) - -// startTs = 0 - -// d.syncTreeMutex.Lock() -// d.syncTree = d.syncTreeCandidate -// d.syncTreeMutex.Unlock() - -// // create new syncTreeCandidat -// d.syncTreeCandidate, err = tree.NewTreeRoot(ctx, tree.NewTreeContext(d.schemaClient, tree.RunningIntentName)) -// if err != nil { -// log.Errorf("creating a new synctree candidate: %v", err) -// return -// } - -// // export and write to cache -// runningExport, err := d.syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) -// if err != nil { -// log.Error(err) -// continue -// } -// err = d.cacheClient.IntentModify(ctx, runningExport) -// if err != nil { -// log.Errorf("issue modifying running cache content: %v", err) -// continue -// } -// default: -// if startTs == 0 { -// startTs = time.Now().Unix() -// } -// err := d.writeToSyncTreeCandidate(ctx, syncup.Update.GetUpdate(), startTs) -// if err != nil { -// log.Errorf("failed to write to sync tree: %v", err) -// } -// } -// } -// } -// } - -func (d *Datastore) writeToSyncTreeCandidate(ctx context.Context, updates []*sdcpb.Update, ts int64) error { - upds, err := treetypes.ExpandAndConvertIntent(ctx, d.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, updates, ts) - if err != nil { - return err - } - - for idx, upd := range upds { - _ = idx - _, err := d.syncTreeCandidate.AddUpdateRecursive(ctx, upd.Path(), upd, treetypes.NewUpdateInsertFlags()) - if err != nil { - return err - } - } - return nil -} - func (d *Datastore) ApplyToRunning(ctx context.Context, i *tree_persist.Intent) error { i.IntentName = tree.RunningIntentName diff --git a/pkg/datastore/target/gnmi/gnmi.go b/pkg/datastore/target/gnmi/gnmi.go index 4563c4af..02c28561 100644 --- a/pkg/datastore/target/gnmi/gnmi.go +++ b/pkg/datastore/target/gnmi/gnmi.go @@ -109,7 +109,7 @@ func NewTarget(ctx context.Context, name string, cfg *config.SBI, runningStore t } func (t *gnmiTarget) Subscribe(ctx context.Context, req *gnmi.SubscribeRequest, subscriptionName string) { - t.Subscribe(ctx, req, subscriptionName) + t.target.Subscribe(ctx, req, subscriptionName) } func (t *gnmiTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) { From 6a5116a20d2dbeddb1aa9025e58f78329489d170 Mon Sep 17 00:00:00 2001 From: steiler Date: Mon, 13 Oct 2025 12:43:36 +0200 Subject: [PATCH 06/44] update --- main.go | 7 +++ pkg/datastore/target/gnmi/get.go | 57 +++++++++++++++---------- pkg/datastore/target/gnmi/gnmi.go | 6 +-- pkg/datastore/target/gnmi/stream.go | 66 ++++++++++++++++++++++++++--- pkg/utils/converter.go | 9 ++++ pkg/utils/notification.go | 7 +-- 6 files changed, 118 insertions(+), 34 deletions(-) diff --git a/main.go b/main.go index 5c8533aa..d5672d57 100644 --- a/main.go +++ b/main.go @@ -27,6 +27,9 @@ import ( log "github.com/sirupsen/logrus" "github.com/spf13/pflag" + "net/http" + _ "net/http/pprof" + "github.com/sdcio/data-server/pkg/config" "github.com/sdcio/data-server/pkg/dslog" "github.com/sdcio/data-server/pkg/server" @@ -53,6 +56,10 @@ func main() { return } + go func() { + log.Println(http.ListenAndServe("localhost:6060", nil)) + }() + log.SetFormatter(&log.TextFormatter{FullTimestamp: true}) log.SetLevel(log.InfoLevel) if debug { diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index 634abbdc..a5734158 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -100,31 +100,12 @@ func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { log.Errorf("sync newemptytree error: %v", err) return } - deletes := []*sdcpb.Path{} - - ts := time.Now() - uif := treetypes.NewUpdateInsertFlags() // process Noifications - for _, noti := range resp.Notification { - // updates - upds, err := treetypes.ExpandAndConvertIntent(s.ctx, s.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, noti.Update, ts.Unix()) - if err != nil { - log.Errorf("sync expanding error: %v", err) - return - } - - for idx2, upd := range upds { - _ = idx2 - _, err = syncTree.AddUpdateRecursive(s.ctx, upd.Path(), upd, uif) - if err != nil { - log.Errorf("sync process notifications error: %v", err) - return - } - - } - // deletes - deletes = append(deletes, noti.GetDelete()...) + deletes, err := processNotifications(s.ctx, resp.GetNotification(), s.schemaClient, syncTree) + if err != nil { + log.Errorf("sync process notifications error: %v", err) + return } result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) @@ -145,3 +126,33 @@ func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { type GetTarget interface { Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) } + +func processNotifications(ctx context.Context, n []*sdcpb.Notification, schemaClient dsutils.SchemaClientBound, syncTree *tree.RootEntry) ([]*sdcpb.Path, error) { + + ts := time.Now().Unix() + uif := treetypes.NewUpdateInsertFlags() + + deletes := []*sdcpb.Path{} + + for _, noti := range n { + // updates + upds, err := treetypes.ExpandAndConvertIntent(ctx, schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, noti.Update, ts) + if err != nil { + log.Errorf("sync expanding error: %v", err) + return nil, err + } + + for idx2, upd := range upds { + _ = idx2 + _, err = syncTree.AddUpdateRecursive(ctx, upd.Path(), upd, uif) + if err != nil { + log.Errorf("sync process notifications error: %v", err) + return nil, err + } + + } + // deletes + deletes = append(deletes, noti.GetDelete()...) + } + return deletes, nil +} diff --git a/pkg/datastore/target/gnmi/gnmi.go b/pkg/datastore/target/gnmi/gnmi.go index 02c28561..cf64c2de 100644 --- a/pkg/datastore/target/gnmi/gnmi.go +++ b/pkg/datastore/target/gnmi/gnmi.go @@ -108,8 +108,8 @@ func NewTarget(ctx context.Context, name string, cfg *config.SBI, runningStore t return gt, nil } -func (t *gnmiTarget) Subscribe(ctx context.Context, req *gnmi.SubscribeRequest, subscriptionName string) { - t.target.Subscribe(ctx, req, subscriptionName) +func (t *gnmiTarget) Subscribe(ctx context.Context, req *gnmi.SubscribeRequest, subscriptionName string) (chan *gnmi.SubscribeResponse, chan error) { + return t.target.SubscribeStreamChan(ctx, req, subscriptionName) } func (t *gnmiTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) { @@ -257,7 +257,7 @@ func (t *gnmiTarget) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) case "get": g = NewGetSync(ctx, t, sp, t.runningStore, t.schemaClient) default: - g = NewStreamSync(ctx, t, sp, t.runningStore) + g = NewStreamSync(ctx, t, sp, t.runningStore, t.schemaClient) } t.syncs[sp.Name] = g err := g.Start() diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index eb101dbb..863e34a6 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -8,18 +8,22 @@ import ( "github.com/sdcio/data-server/pkg/config" "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" "github.com/sdcio/data-server/pkg/datastore/target/types" + "github.com/sdcio/data-server/pkg/tree" + dsutils "github.com/sdcio/data-server/pkg/utils" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" log "github.com/sirupsen/logrus" ) type StreamSync struct { + ctx context.Context config *config.SyncProtocol target SyncTarget cancel context.CancelFunc runningStore types.RunningStore - ctx context.Context + schemaClient dsutils.SchemaClientBound } -func NewStreamSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore) *StreamSync { +func NewStreamSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore, schemaClient dsutils.SchemaClientBound) *StreamSync { ctx, cancel := context.WithCancel(ctx) return &StreamSync{ @@ -27,6 +31,8 @@ func NewStreamSync(ctx context.Context, target SyncTarget, c *config.SyncProtoco target: target, cancel: cancel, runningStore: runningStore, + schemaClient: schemaClient, + ctx: ctx, } } @@ -51,7 +57,6 @@ func (s *StreamSync) syncConfig() (*gnmi.SubscribeRequest, error) { gapi.EncodingCustom(utils.ParseGnmiEncoding(s.config.Encoding)), gapi.SubscriptionListModeSTREAM(), gapi.Subscription(subscriptionOpts...), - gapi.DataTypeCONFIG(), ) subReq, err := gapi.NewSubscribeRequest(opts...) if err != nil { @@ -73,10 +78,61 @@ func (s *StreamSync) Start() error { } log.Infof("sync %q: subRequest: %v", s.config.Name, subReq) - go s.target.Subscribe(s.ctx, subReq, s.config.Name) + + respChan, errChan := s.target.Subscribe(s.ctx, subReq, s.config.Name) + + go func() { + syncTree, err := s.runningStore.NewEmptyTree(s.ctx) + if err != nil { + log.Errorf("sync newemptytree error: %v", err) + return + } + for { + select { + case <-s.ctx.Done(): + return + case err = <-errChan: + if err != nil { + log.Errorf("Error stream sync: %s", err) + return + } + case resp, ok := <-respChan: + if !ok { + return + } + _ = resp + switch r := resp.GetResponse().(type) { + case *gnmi.SubscribeResponse_Update: + sn := dsutils.ToSchemaNotification(r.Update) + processNotifications(s.ctx, []*sdcpb.Notification{sn}, s.schemaClient, syncTree) + case *gnmi.SubscribeResponse_SyncResponse: + log.Info("SyncResponse flag received") + result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) + if err != nil { + log.Errorf("sync tree export error: %v", err) + return + } + + err = s.runningStore.ApplyToRunning(s.ctx, result) + if err != nil { + log.Errorf("sync import to running error: %v", err) + return + } + syncTree, err = s.runningStore.NewEmptyTree(s.ctx) + if err != nil { + log.Errorf("sync newemptytree error: %v", err) + return + } + case *gnmi.SubscribeResponse_Error: + log.Error(r.Error.Message) + } + } + } + }() + return nil } type SyncTarget interface { - Subscribe(ctx context.Context, req *gnmi.SubscribeRequest, subscriptionName string) + Subscribe(ctx context.Context, req *gnmi.SubscribeRequest, subscriptionName string) (chan *gnmi.SubscribeResponse, chan error) } diff --git a/pkg/utils/converter.go b/pkg/utils/converter.go index 363244d1..9684d793 100644 --- a/pkg/utils/converter.go +++ b/pkg/utils/converter.go @@ -54,6 +54,11 @@ func (c *Converter) ExpandUpdate(ctx context.Context, upd *sdcpb.Update) ([]*sdc return nil, err } + // skip state + if rsp.GetSchema().IsState() { + return nil, nil + } + switch rsp := rsp.GetSchema().Schema.(type) { case *sdcpb.SchemaElem_Container: if upd.Value == nil { @@ -120,6 +125,10 @@ func (c *Converter) ExpandUpdate(ctx context.Context, upd *sdcpb.Update) ([]*sdc var err error var jsonValue []byte + if upd.GetValue() == nil { + log.Errorf("Error - Path: %s - TypedValue == nil", upd.Path.ToXPath(false)) + return nil, nil + } switch upd.GetValue().Value.(type) { case *sdcpb.TypedValue_JsonVal: jsonValue = upd.GetValue().GetJsonVal() diff --git a/pkg/utils/notification.go b/pkg/utils/notification.go index 8bf3bf47..3b67a290 100644 --- a/pkg/utils/notification.go +++ b/pkg/utils/notification.go @@ -48,9 +48,10 @@ func FromGNMIPath(pre, p *gnmi.Path) *sdcpb.Path { return nil } r := &sdcpb.Path{ - Origin: pre.GetOrigin(), - Elem: make([]*sdcpb.PathElem, 0, len(pre.GetElem())+len(p.GetElem())), - Target: pre.GetTarget(), + Origin: pre.GetOrigin(), + Elem: make([]*sdcpb.PathElem, 0, len(pre.GetElem())+len(p.GetElem())), + Target: pre.GetTarget(), + IsRootBased: true, } for _, pe := range pre.GetElem() { r.Elem = append(r.Elem, &sdcpb.PathElem{ From 1c375388f5c6bd68225f5b3299c41c622c37ec39 Mon Sep 17 00:00:00 2001 From: steiler Date: Tue, 14 Oct 2025 11:01:10 +0200 Subject: [PATCH 07/44] update --- pkg/datastore/target/gnmi/get.go | 21 ++++--- pkg/datastore/target/gnmi/stream.go | 86 +++++++++++++++++++++++------ pkg/tree/entry.go | 1 + pkg/tree/sharedEntryAttributes.go | 10 +++- pkg/utils/notification.go | 4 ++ 5 files changed, 95 insertions(+), 27 deletions(-) diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index a5734158..32bedc7b 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -2,6 +2,7 @@ package gnmi import ( "context" + "sync" "time" "github.com/sdcio/data-server/pkg/config" @@ -101,8 +102,10 @@ func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { return } + syncTreeMutex := &sync.Mutex{} + // process Noifications - deletes, err := processNotifications(s.ctx, resp.GetNotification(), s.schemaClient, syncTree) + deletes, err := processNotifications(s.ctx, resp.GetNotification(), s.schemaClient, syncTree, syncTreeMutex) if err != nil { log.Errorf("sync process notifications error: %v", err) return @@ -127,32 +130,34 @@ type GetTarget interface { Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) } -func processNotifications(ctx context.Context, n []*sdcpb.Notification, schemaClient dsutils.SchemaClientBound, syncTree *tree.RootEntry) ([]*sdcpb.Path, error) { +func processNotifications(ctx context.Context, n []*sdcpb.Notification, schemaClient dsutils.SchemaClientBound, syncTree *tree.RootEntry, m *sync.Mutex) ([]*sdcpb.Path, error) { ts := time.Now().Unix() uif := treetypes.NewUpdateInsertFlags() deletes := []*sdcpb.Path{} + m.Lock() + defer m.Unlock() + for _, noti := range n { // updates upds, err := treetypes.ExpandAndConvertIntent(ctx, schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, noti.Update, ts) if err != nil { log.Errorf("sync expanding error: %v", err) - return nil, err + continue } + // deletes + deletes = append(deletes, noti.GetDelete()...) + for idx2, upd := range upds { _ = idx2 _, err = syncTree.AddUpdateRecursive(ctx, upd.Path(), upd, uif) if err != nil { - log.Errorf("sync process notifications error: %v", err) - return nil, err + log.Errorf("sync process notifications error: %v, continuing", err) } - } - // deletes - deletes = append(deletes, noti.GetDelete()...) } return deletes, nil } diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index 863e34a6..8f94193c 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -2,6 +2,9 @@ package gnmi import ( "context" + "errors" + "sync" + "time" "github.com/openconfig/gnmi/proto/gnmi" gapi "github.com/openconfig/gnmic/pkg/api" @@ -81,8 +84,48 @@ func (s *StreamSync) Start() error { respChan, errChan := s.target.Subscribe(s.ctx, subReq, s.config.Name) + syncStartTime := time.Now() + + syncToRunning := func(syncTree *tree.RootEntry, m *sync.Mutex, logCount bool) (*tree.RootEntry, error) { + m.Lock() + defer m.Unlock() + + startTime := time.Now() + result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) + if err != nil { + if errors.Is(err, tree.ErrorIntentNotPresent) { + log.Info("sync no config chnages.") + // all good no data present + return syncTree, nil + } + log.Errorf("sync tree export error: %v", err) + return s.runningStore.NewEmptyTree(s.ctx) + } + + if logCount { + log.Infof("Syncing %d elements", result.Root.CountTerminals()) + } + + log.Infof("TreeExport to proto took: %s", time.Since(startTime)) + startTime = time.Now() + + err = s.runningStore.ApplyToRunning(s.ctx, result) + if err != nil { + log.Errorf("sync import to running error: %v", err) + return s.runningStore.NewEmptyTree(s.ctx) + } + log.Infof("Import to SyncTree took: %s", time.Since(startTime)) + return s.runningStore.NewEmptyTree(s.ctx) + } + go func() { + ticker := time.NewTicker(5 * time.Second) + defer ticker.Stop() + // disable ticker until after the initial full sync is done + tickerActive := false syncTree, err := s.runningStore.NewEmptyTree(s.ctx) + syncTreeMutex := &sync.Mutex{} + if err != nil { log.Errorf("sync newemptytree error: %v", err) return @@ -91,38 +134,49 @@ func (s *StreamSync) Start() error { select { case <-s.ctx.Done(): return - case err = <-errChan: + case err, ok := <-errChan: + if !ok { + return + } if err != nil { log.Errorf("Error stream sync: %s", err) return } + + case <-ticker.C: + if !tickerActive { + log.Info("Skipping a sync tick - initial sync not finished yet") + continue + } + log.Info("SyncRunning due to ticker") + syncTree, err = syncToRunning(syncTree, syncTreeMutex, true) + if err != nil { + // TODO + log.Errorf("syncToRunning Error %v", err) + } case resp, ok := <-respChan: if !ok { return } - _ = resp switch r := resp.GetResponse().(type) { case *gnmi.SubscribeResponse_Update: sn := dsutils.ToSchemaNotification(r.Update) - processNotifications(s.ctx, []*sdcpb.Notification{sn}, s.schemaClient, syncTree) - case *gnmi.SubscribeResponse_SyncResponse: - log.Info("SyncResponse flag received") - result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) - if err != nil { - log.Errorf("sync tree export error: %v", err) - return - } - err = s.runningStore.ApplyToRunning(s.ctx, result) + _, err = processNotifications(s.ctx, []*sdcpb.Notification{sn}, s.schemaClient, syncTree, syncTreeMutex) if err != nil { - log.Errorf("sync import to running error: %v", err) - return + log.Errorf("error processing Notifications: %s", err) + continue } - syncTree, err = s.runningStore.NewEmptyTree(s.ctx) + case *gnmi.SubscribeResponse_SyncResponse: + log.Info("SyncResponse flag received") + log.Infof("Duration since sync Start: %s", time.Since(syncStartTime)) + syncTree, err = syncToRunning(syncTree, syncTreeMutex, false) if err != nil { - log.Errorf("sync newemptytree error: %v", err) - return + // TODO + log.Errorf("syncToRunning Error %v", err) } + // activate ticker processing + tickerActive = true case *gnmi.SubscribeResponse_Error: log.Error(r.Error.Message) } diff --git a/pkg/tree/entry.go b/pkg/tree/entry.go index 2f2bbf83..ed7a05b4 100644 --- a/pkg/tree/entry.go +++ b/pkg/tree/entry.go @@ -115,6 +115,7 @@ type Entry interface { // - shouldDelete() returns false, because no explicit delete should be issued for them. canDelete() bool GetChilds(DescendMethod) EntryMap + GetChild(name string) (Entry, bool) // entry, exists FilterChilds(keys map[string]string) ([]Entry, error) // ToJson returns the Tree contained structure as JSON // use e.g. json.MarshalIndent() on the returned struct diff --git a/pkg/tree/sharedEntryAttributes.go b/pkg/tree/sharedEntryAttributes.go index 1b3b35fb..38f7ee32 100644 --- a/pkg/tree/sharedEntryAttributes.go +++ b/pkg/tree/sharedEntryAttributes.go @@ -1247,7 +1247,7 @@ func (s *sharedEntryAttributes) ImportConfig(ctx context.Context, t importer.Imp return err } // if the child does not exist, create it - if keyChild, exists = actualEntry.GetChilds(DescendMethodAll)[keyElemValue]; !exists { + if keyChild, exists = actualEntry.GetChild(keyElemValue); !exists { keyChild, err = newEntry(ctx, actualEntry, keyElemValue, s.treeContext) if err != nil { return err @@ -1278,7 +1278,7 @@ func (s *sharedEntryAttributes) ImportConfig(ctx context.Context, t importer.Imp var exists bool // if the child does not exist, create it - if child, exists = s.getChildren()[elem.GetName()]; !exists { + if child, exists = s.GetChild(elem.GetName()); !exists { child, err = newEntry(ctx, s, elem.GetName(), s.treeContext) if err != nil { return fmt.Errorf("error trying to insert %s at path %s: %w", elem.GetName(), s.SdcpbPath().ToXPath(false), err) @@ -1532,6 +1532,10 @@ func (s *sharedEntryAttributes) populateChoiceCaseResolvers(_ context.Context) e return nil } +func (s *sharedEntryAttributes) GetChild(name string) (Entry, bool) { + return s.childs.GetEntry(name) +} + func (s *sharedEntryAttributes) GetChilds(d DescendMethod) EntryMap { if s.schema == nil { return s.childs.GetAll() @@ -1781,7 +1785,7 @@ func (s *sharedEntryAttributes) AddUpdateRecursive(ctx context.Context, path *sd var x Entry = s var exists bool for name := range relPath.GetElem()[0].PathElemNames() { - if e, exists = x.GetChilds(DescendMethodAll)[name]; !exists { + if e, exists = x.GetChild(name); !exists { newE, err := newEntry(ctx, x, name, s.treeContext) if err != nil { return nil, err diff --git a/pkg/utils/notification.go b/pkg/utils/notification.go index 3b67a290..29016ef7 100644 --- a/pkg/utils/notification.go +++ b/pkg/utils/notification.go @@ -134,6 +134,10 @@ func FromGNMITypedValue(v *gnmi.TypedValue) *sdcpb.TypedValue { return &sdcpb.TypedValue{ Value: &sdcpb.TypedValue_DoubleVal{DoubleVal: float64(v.GetFloatVal())}, } + case *gnmi.TypedValue_DoubleVal: + return &sdcpb.TypedValue{ + Value: &sdcpb.TypedValue_DoubleVal{DoubleVal: v.GetDoubleVal()}, + } default: log.Errorf("FromGNMITypedValue unhandled type: %T: %v", v, v) return nil From 68d50bdce2a150bcfada4338f062f6af4cf5fd75 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 15 Oct 2025 11:39:29 +0200 Subject: [PATCH 08/44] pool: stop busy wait --- pkg/utils/pool.go | 24 +++++++++++------------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/pkg/utils/pool.go b/pkg/utils/pool.go index 2d7b8e8e..62786b7c 100644 --- a/pkg/utils/pool.go +++ b/pkg/utils/pool.go @@ -130,33 +130,31 @@ func (p *Pool[T]) Start(handler func(ctx context.Context, item T, submit func(T) // monitor goroutine: when CloseForSubmit has been called, wait until both inflight==0 and queue empty, // then close the queue so workers exit. Also handle ctx cancellation (force-close). go func() { + // We'll wait on the condition variable instead of busy looping. + p.inflightMu.Lock() + defer p.inflightMu.Unlock() for { - // graceful path: wait for CloseForSubmit flag then wait for work to drain + // If CloseForSubmit was called, wait for inflight==0 and queue empty then close queue. if p.closedForSubmit.Load() { - // wait until inflight==0 AND tasks.Len()==0 - p.inflightMu.Lock() - for { - if atomic.LoadInt64(&p.inflight) == 0 && p.tasks.Len() == 0 { - break - } + for atomic.LoadInt64(&p.inflight) != 0 || p.tasks.Len() != 0 { p.inflightC.Wait() - // loop and re-check } - p.inflightMu.Unlock() - // Now safe to close queue: there is no inflight and no queued items p.closeOnce.Do(func() { p.tasks.Close() }) return } - // if ctx canceled -> force-close path + // If ctx canceled -> force-close path. if p.ctx.Err() != nil { + // we hold inflightMu; unlock before calling forceClose (which may broadcast/use locks). + p.inflightMu.Unlock() p.forceClose() return } - // avoid busy spin - runtime.Gosched() + // Wait to be signalled when either inflight changes or CloseForSubmit is called. + p.inflightC.Wait() + // loop and recheck conditions } }() } From 25ff1fde46c9545ab39a386ef28adc7405222c56 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 15 Oct 2025 11:40:01 +0200 Subject: [PATCH 09/44] gnmi: sync on-change delete working --- client/cmd/data_getIntent.go | 27 +++++- client/cmd/root.go | 1 + pkg/datastore/deviations.go | 30 +++---- pkg/datastore/sync.go | 23 +++-- pkg/datastore/target/gnmi/stream.go | 126 +++++++++++++++++++++------- 5 files changed, 149 insertions(+), 58 deletions(-) diff --git a/client/cmd/data_getIntent.go b/client/cmd/data_getIntent.go index f88636b0..c155f289 100644 --- a/client/cmd/data_getIntent.go +++ b/client/cmd/data_getIntent.go @@ -15,8 +15,13 @@ package cmd import ( + "bytes" "context" + "encoding/json" "fmt" + "os" + "time" + "unsafe" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "github.com/spf13/cobra" @@ -42,18 +47,32 @@ var dataGetIntentCmd = &cobra.Command{ if err != nil { return err } - fmt.Println("request:") - fmt.Println(prototext.Format(req)) + fmt.Fprintln(os.Stderr, "request:") + fmt.Fprintln(os.Stderr, prototext.Format(req)) + startTime := time.Now() rsp, err := dataClient.GetIntent(ctx, req) if err != nil { return err } - fmt.Println("response:") - fmt.Println(prototext.Format(rsp)) + fmt.Fprintf(os.Stderr, "GetIntent took: %s\n", time.Since(startTime)) + + var buf bytes.Buffer + if err := json.Indent(&buf, rsp.GetBlob(), "", " "); err != nil { + return err + } + + if _, err := os.Stdout.Write(buf.Bytes()); err != nil { + return err + } + return nil }, } +func BytesToString(b []byte) string { + return unsafe.String(&b[0], len(b)) +} + func init() { dataCmd.AddCommand(dataGetIntentCmd) dataGetIntentCmd.Flags().StringVarP(&intentName, "intent", "", "", "intent name") diff --git a/client/cmd/root.go b/client/cmd/root.go index 69aa2b2e..29f56f3c 100644 --- a/client/cmd/root.go +++ b/client/cmd/root.go @@ -62,6 +62,7 @@ func createDataClient(ctx context.Context, addr string) (sdcpb.DataServerClient, grpc.WithTransportCredentials( insecure.NewCredentials(), ), + grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(200*1024*1024)), ) if err != nil { return nil, err diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index f332a938..3d7abd48 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -70,21 +70,21 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) } } - deviationChan, err := d.calculateDeviations(ctx) - if err != nil { - log.Error(err) - continue - } - d.SendDeviations(deviationChan, deviationClients) - for clientIdentifier, dc := range deviationClients { - err := dc.Send(&sdcpb.WatchDeviationResponse{ - Name: d.config.Name, - Event: sdcpb.DeviationEvent_END, - }) - if err != nil { - log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) - } - } + // deviationChan, err := d.calculateDeviations(ctx) + // if err != nil { + // log.Error(err) + // continue + // } + // d.SendDeviations(deviationChan, deviationClients) + // for clientIdentifier, dc := range deviationClients { + // err := dc.Send(&sdcpb.WatchDeviationResponse{ + // Name: d.config.Name, + // Event: sdcpb.DeviationEvent_END, + // }) + // if err != nil { + // log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) + // } + // } } } } diff --git a/pkg/datastore/sync.go b/pkg/datastore/sync.go index 657f0aca..a6b86dec 100644 --- a/pkg/datastore/sync.go +++ b/pkg/datastore/sync.go @@ -8,6 +8,7 @@ import ( treetypes "github.com/sdcio/data-server/pkg/tree/types" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "github.com/sdcio/sdc-protos/tree_persist" + log "github.com/sirupsen/logrus" ) func (d *Datastore) ApplyToRunning(ctx context.Context, i *tree_persist.Intent) error { @@ -16,23 +17,29 @@ func (d *Datastore) ApplyToRunning(ctx context.Context, i *tree_persist.Intent) i.Priority = tree.RunningValuesPrio i.Deviation = false - importer := proto.NewProtoTreeImporter(i) + d.syncTreeMutex.Lock() + defer d.syncTreeMutex.Unlock() + for _, delete := range i.ExplicitDeletes { + //TODO this will most likely give us errors in case optimisticWriteback already deleted the entries. + err := d.syncTree.DeleteBranch(ctx, delete, i.IntentName) + if err != nil { + log.Errorf("error deleting paths from datastore sync tree: %v", err) + return err + } + } // need to reset the explicit deletes, they carry the actual deletes that we need to delete. // the imported would otherwise add explicit deletes for these. - deletes := i.ExplicitDeletes i.ExplicitDeletes = nil - d.syncTreeMutex.Lock() - defer d.syncTreeMutex.Unlock() - for _, delete := range deletes { - err := d.syncTree.DeleteBranch(ctx, delete, i.IntentName) + if i.GetRoot() != nil { + importer := proto.NewProtoTreeImporter(i) + err := d.syncTree.ImportConfig(ctx, &sdcpb.Path{}, importer, i.GetIntentName(), i.GetPriority(), treetypes.NewUpdateInsertFlags()) if err != nil { return err } } - - return d.syncTree.ImportConfig(ctx, &sdcpb.Path{}, importer, i.GetIntentName(), i.GetPriority(), treetypes.NewUpdateInsertFlags()) + return nil } func (d *Datastore) NewEmptyTree(ctx context.Context) (*tree.RootEntry, error) { diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index 8f94193c..77c13df1 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -3,6 +3,7 @@ package gnmi import ( "context" "errors" + "runtime" "sync" "time" @@ -12,6 +13,7 @@ import ( "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" "github.com/sdcio/data-server/pkg/datastore/target/types" "github.com/sdcio/data-server/pkg/tree" + treetypes "github.com/sdcio/data-server/pkg/tree/types" dsutils "github.com/sdcio/data-server/pkg/utils" sdcpb "github.com/sdcio/sdc-protos/sdcpb" log "github.com/sirupsen/logrus" @@ -86,6 +88,35 @@ func (s *StreamSync) Start() error { syncStartTime := time.Now() + wpool := dsutils.NewWorkerPool[*gnmi.Notification](s.ctx, runtime.NumCPU()) + + updChan := make(chan *NotificationData, 20) + + wpoolHandler := func(ctx context.Context, item *gnmi.Notification, submit func(*gnmi.Notification) error) error { + sn := dsutils.ToSchemaNotification(item) + // updates + upds, err := treetypes.ExpandAndConvertIntent(s.ctx, s.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, sn.GetUpdate(), item.GetTimestamp()) + if err != nil { + log.Errorf("sync expanding error: %v", err) + } + + deletes := sdcpb.NewPathSet() + if len(item.GetDelete()) > 0 { + for _, del := range item.GetDelete() { + deletes.AddPath(dsutils.FromGNMIPath(item.GetPrefix(), del)) + } + } + + updChan <- &NotificationData{ + updates: upds, + deletes: deletes, + } + + return nil + } + + wpool.Start(wpoolHandler) + syncToRunning := func(syncTree *tree.RootEntry, m *sync.Mutex, logCount bool) (*tree.RootEntry, error) { m.Lock() defer m.Unlock() @@ -94,7 +125,7 @@ func (s *StreamSync) Start() error { result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) if err != nil { if errors.Is(err, tree.ErrorIntentNotPresent) { - log.Info("sync no config chnages.") + log.Info("sync no config changes") // all good no data present return syncTree, nil } @@ -103,7 +134,7 @@ func (s *StreamSync) Start() error { } if logCount { - log.Infof("Syncing %d elements", result.Root.CountTerminals()) + log.Infof("Syncing: %d elements, %d deletes ", result.GetRoot().CountTerminals(), len(result.GetExplicitDeletes())) } log.Infof("TreeExport to proto took: %s", time.Since(startTime)) @@ -118,18 +149,14 @@ func (s *StreamSync) Start() error { return s.runningStore.NewEmptyTree(s.ctx) } - go func() { - ticker := time.NewTicker(5 * time.Second) - defer ticker.Stop() - // disable ticker until after the initial full sync is done - tickerActive := false - syncTree, err := s.runningStore.NewEmptyTree(s.ctx) - syncTreeMutex := &sync.Mutex{} + syncResponse := make(chan struct{}) + go func() { if err != nil { log.Errorf("sync newemptytree error: %v", err) return } + defer wpool.CloseForSubmit() for { select { case <-s.ctx.Done(): @@ -142,27 +169,13 @@ func (s *StreamSync) Start() error { log.Errorf("Error stream sync: %s", err) return } - - case <-ticker.C: - if !tickerActive { - log.Info("Skipping a sync tick - initial sync not finished yet") - continue - } - log.Info("SyncRunning due to ticker") - syncTree, err = syncToRunning(syncTree, syncTreeMutex, true) - if err != nil { - // TODO - log.Errorf("syncToRunning Error %v", err) - } case resp, ok := <-respChan: if !ok { return } switch r := resp.GetResponse().(type) { case *gnmi.SubscribeResponse_Update: - sn := dsutils.ToSchemaNotification(r.Update) - - _, err = processNotifications(s.ctx, []*sdcpb.Notification{sn}, s.schemaClient, syncTree, syncTreeMutex) + err := wpool.Submit(resp.GetUpdate()) if err != nil { log.Errorf("error processing Notifications: %s", err) continue @@ -170,13 +183,8 @@ func (s *StreamSync) Start() error { case *gnmi.SubscribeResponse_SyncResponse: log.Info("SyncResponse flag received") log.Infof("Duration since sync Start: %s", time.Since(syncStartTime)) - syncTree, err = syncToRunning(syncTree, syncTreeMutex, false) - if err != nil { - // TODO - log.Errorf("syncToRunning Error %v", err) - } - // activate ticker processing - tickerActive = true + syncResponse <- struct{}{} + case *gnmi.SubscribeResponse_Error: log.Error(r.Error.Message) } @@ -184,9 +192,65 @@ func (s *StreamSync) Start() error { } }() + go func(cUS <-chan *NotificationData, syncResponse <-chan struct{}) { + + syncTree, err := s.runningStore.NewEmptyTree(s.ctx) + if err != nil { + log.Errorf("error creating new sync tree: %v", err) + return + } + syncTreeMutex := &sync.Mutex{} + + ticker := time.NewTicker(5 * time.Second) + defer ticker.Stop() + // disable ticker until after the initial full sync is done + tickerActive := false + + uif := treetypes.NewUpdateInsertFlags() + + for { + select { + case <-s.ctx.Done(): + return + case noti, ok := <-cUS: + if !ok { + return + } + err := syncTree.AddUpdatesRecursive(s.ctx, noti.updates, uif) + if err != nil { + log.Errorf("error adding to sync tree: %v", err) + } + syncTree.AddExplicitDeletes(tree.RunningIntentName, tree.RunningValuesPrio, noti.deletes) + case <-syncResponse: + syncTree, err = syncToRunning(syncTree, syncTreeMutex, true) + if err != nil { + // TODO + log.Errorf("syncToRunning Error %v", err) + } + tickerActive = true + case <-ticker.C: + if !tickerActive { + log.Info("Skipping a sync tick - initial sync not finished yet") + continue + } + log.Info("SyncRunning due to ticker") + syncTree, err = syncToRunning(syncTree, syncTreeMutex, true) + if err != nil { + // TODO + log.Errorf("syncToRunning Error %v", err) + } + } + } + }(updChan, syncResponse) + return nil } type SyncTarget interface { Subscribe(ctx context.Context, req *gnmi.SubscribeRequest, subscriptionName string) (chan *gnmi.SubscribeResponse, chan error) } + +type NotificationData struct { + updates treetypes.UpdateSlice + deletes *sdcpb.PathSet +} From 7f331baa0f101bfecfa2cf4863a3fc02e073fe2f Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 15 Oct 2025 11:54:05 +0200 Subject: [PATCH 10/44] utils: adjust gnmi to sdcpb Notification test --- pkg/utils/notification_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/utils/notification_test.go b/pkg/utils/notification_test.go index 235345d1..66733221 100644 --- a/pkg/utils/notification_test.go +++ b/pkg/utils/notification_test.go @@ -46,6 +46,7 @@ func TestFromGNMIPath(t *testing.T) { Elem: []*sdcpb.PathElem{ {Name: "foo"}, }, + IsRootBased: true, }, }, { @@ -64,6 +65,7 @@ func TestFromGNMIPath(t *testing.T) { {Name: "foo"}, {Name: "bar"}, }, + IsRootBased: true, }, }, { @@ -82,6 +84,7 @@ func TestFromGNMIPath(t *testing.T) { {Name: "foo", Key: map[string]string{"k": "v"}}, {Name: "bar"}, }, + IsRootBased: true, }, }, { @@ -100,6 +103,7 @@ func TestFromGNMIPath(t *testing.T) { {Name: "foo", Key: map[string]string{"k1": "v1", "k2": "v2"}}, {Name: "bar"}, }, + IsRootBased: true, }, }, } From d35d91eeb8d0164d7b4f9bcada38be72401dd0a6 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 15 Oct 2025 15:21:22 +0200 Subject: [PATCH 11/44] reorg --- pkg/datastore/deviations.go | 30 +- pkg/datastore/target/gnmi/stream.go | 3 +- pkg/pool/pool.go | 219 ++++++++++++ pkg/pool/queue.go | 108 ++++++ pkg/pool/virtual_pool.go | 318 ++++++++++++++++++ pkg/pool/virtual_pool_test.go | 131 ++++++++ .../workerpoolqueue_stress_test.go | 2 +- pkg/tree/parallelImporter.go | 4 +- 8 files changed, 796 insertions(+), 19 deletions(-) create mode 100644 pkg/pool/pool.go create mode 100644 pkg/pool/queue.go create mode 100644 pkg/pool/virtual_pool.go create mode 100644 pkg/pool/virtual_pool_test.go rename pkg/{utils => pool}/workerpoolqueue_stress_test.go (99%) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 3d7abd48..f332a938 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -70,21 +70,21 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) } } - // deviationChan, err := d.calculateDeviations(ctx) - // if err != nil { - // log.Error(err) - // continue - // } - // d.SendDeviations(deviationChan, deviationClients) - // for clientIdentifier, dc := range deviationClients { - // err := dc.Send(&sdcpb.WatchDeviationResponse{ - // Name: d.config.Name, - // Event: sdcpb.DeviationEvent_END, - // }) - // if err != nil { - // log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) - // } - // } + deviationChan, err := d.calculateDeviations(ctx) + if err != nil { + log.Error(err) + continue + } + d.SendDeviations(deviationChan, deviationClients) + for clientIdentifier, dc := range deviationClients { + err := dc.Send(&sdcpb.WatchDeviationResponse{ + Name: d.config.Name, + Event: sdcpb.DeviationEvent_END, + }) + if err != nil { + log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) + } + } } } } diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index 77c13df1..4b3818e1 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -12,6 +12,7 @@ import ( "github.com/sdcio/data-server/pkg/config" "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" "github.com/sdcio/data-server/pkg/datastore/target/types" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/tree" treetypes "github.com/sdcio/data-server/pkg/tree/types" dsutils "github.com/sdcio/data-server/pkg/utils" @@ -88,7 +89,7 @@ func (s *StreamSync) Start() error { syncStartTime := time.Now() - wpool := dsutils.NewWorkerPool[*gnmi.Notification](s.ctx, runtime.NumCPU()) + wpool := pool.NewWorkerPool[*gnmi.Notification](s.ctx, runtime.NumCPU()) updChan := make(chan *NotificationData, 20) diff --git a/pkg/pool/pool.go b/pkg/pool/pool.go new file mode 100644 index 00000000..f6a959ff --- /dev/null +++ b/pkg/pool/pool.go @@ -0,0 +1,219 @@ +package pool + +import ( + "context" + "errors" + "runtime" + "sync" + "sync/atomic" +) + +var ErrClosed = errors.New("queue closed") + +// Pool[T] is a worker pool backed by WorkerPoolQueue. +// It uses an atomic inflight counter + cond to avoid deadlocks between closing the queue +// and tracking outstanding work. +type Pool[T any] struct { + tasks *WorkerPoolQueue[T] + workerCount int + + ctx context.Context + cancel context.CancelFunc + + workersWg sync.WaitGroup // wait for worker goroutines to exit + + closeOnce sync.Once + + firstErr atomic.Pointer[error] + + closedForSubmit atomic.Bool + + // inflight counter and condition for waiting until work drains + inflight int64 + inflightMu sync.Mutex + inflightC *sync.Cond +} + +// NewWorkerPool creates a new Pool. If workerCount <= 0 it defaults to runtime.NumCPU(). +func NewWorkerPool[T any](parent context.Context, workerCount int) *Pool[T] { + if workerCount <= 0 { + workerCount = runtime.NumCPU() + } + ctx, cancel := context.WithCancel(parent) + p := &Pool[T]{ + tasks: NewWorkerPoolQueue[T](), + workerCount: workerCount, + ctx: ctx, + cancel: cancel, + } + p.inflightC = sync.NewCond(&p.inflightMu) + return p +} + +// addInflight increments inflight and must be called when a task is known submitted. +func (p *Pool[T]) addInflight(delta int64) { + atomic.AddInt64(&p.inflight, delta) + if atomic.LoadInt64(&p.inflight) == 0 { + // wake any waiter (lock to satisfy cond's invariant) + p.inflightMu.Lock() + p.inflightC.Broadcast() + p.inflightMu.Unlock() + } +} + +// Submit enqueues a task. It increments the inflight counter BEFORE attempting to enqueue. +// If ctx is already cancelled, Submit returns ctx.Err() and does NOT increment inflight. +func (p *Pool[T]) Submit(item T) error { + // fast-fail if canceled + if err := p.ctx.Err(); err != nil { + return err + } + + // increment inflight first + p.addInflight(1) + + // try to put into queue + if err := p.tasks.Put(item); err != nil { + // queue closed (or otherwise failed) -> unaccount the inflight and wake waiters if needed + p.addInflight(-1) + return err + } + return nil +} + +// Start spawns workerCount workers that call handler(ctx, item, submit). +// Handler should process the item and return an error if it wants to abort the whole pool. +// Handler may call submit(...) to add child tasks (workers are allowed to submit). +func (p *Pool[T]) Start(handler func(ctx context.Context, item T, submit func(T) error) error) { + // spawn workers + p.workersWg.Add(p.workerCount) + for i := 0; i < p.workerCount; i++ { + go func() { + defer p.workersWg.Done() + for { + item, ok := p.tasks.Get() + if !ok { + // queue closed and drained -> exit worker + return + } + + // If ctx canceled, we must still decrement inflight for this item and skip handler. + if p.ctx.Err() != nil { + p.addInflight(-1) + continue + } + + // run handler (handler may call p.Submit) + if err := handler(p.ctx, item, func(it T) error { return p.Submit(it) }); err != nil { + // store first error safely (allocate on heap) + ep := new(error) + *ep = err + p.firstErr.CompareAndSwap(nil, ep) + + // cancel pool so other workers see ctx canceled + p.cancel() + + // decrement inflight for this item + p.addInflight(-1) + + // force-close the queue and abandon queued items (so we won't wait forever) + p.forceClose() + + // continue so other workers can observe ctx and drain/exit + continue + } + + // normal completion of this task: decrement inflight + p.addInflight(-1) + } + }() + } + + // monitor goroutine: when CloseForSubmit has been called, wait until both inflight==0 and queue empty, + // then close the queue so workers exit. Also handle ctx cancellation (force-close). + go func() { + // We'll wait on the condition variable instead of busy looping. + p.inflightMu.Lock() + defer p.inflightMu.Unlock() + for { + // If CloseForSubmit was called, wait for inflight==0 and queue empty then close queue. + if p.closedForSubmit.Load() { + for atomic.LoadInt64(&p.inflight) != 0 || p.tasks.Len() != 0 { + p.inflightC.Wait() + } + // Now safe to close queue: there is no inflight and no queued items + p.closeOnce.Do(func() { p.tasks.Close() }) + return + } + + // If ctx canceled -> force-close path. + if p.ctx.Err() != nil { + // we hold inflightMu; unlock before calling forceClose (which may broadcast/use locks). + p.inflightMu.Unlock() + p.forceClose() + return + } + + // Wait to be signalled when either inflight changes or CloseForSubmit is called. + p.inflightC.Wait() + // loop and recheck conditions + } + }() +} + +// CloseForSubmit indicates the caller will not submit more external (caller-side) tasks. +// Workers may still call Submit to add child tasks. When inflight reaches zero and queue is empty, +// the pool will close tasks so workers exit. +func (p *Pool[T]) CloseForSubmit() { + p.closedForSubmit.Store(true) + // kick the monitor by signaling condition in case inflight==0 already + p.inflightMu.Lock() + p.inflightC.Broadcast() + p.inflightMu.Unlock() +} + +// Wait blocks until all workers have exited and returns the first error (if any). +func (p *Pool[T]) Wait() error { + p.workersWg.Wait() + if e := p.firstErr.Load(); e != nil && *e != nil { + return *e + } + if p.ctx.Err() != nil && !errors.Is(p.ctx.Err(), context.Canceled) { + return p.ctx.Err() + } + return nil +} + +// forceClose performs a one-time forced shutdown: cancel context, close queue and +// subtract any queued-but-unprocessed items from inflight so waiters don't block forever. +func (p *Pool[T]) forceClose() { + p.cancel() + p.closeOnce.Do(func() { + // first capture queued items + queued := p.tasks.Len() + if queued > 0 { + // reduce inflight by queued. Use atomic and then broadcast condition. + // Ensure we don't go negative. + for { + cur := atomic.LoadInt64(&p.inflight) + // clamp + var toSub int64 = int64(queued) + if toSub > cur { + toSub = cur + } + if toSub == 0 { + break + } + if atomic.CompareAndSwapInt64(&p.inflight, cur, cur-toSub) { + p.inflightMu.Lock() + p.inflightC.Broadcast() + p.inflightMu.Unlock() + break + } + // retry on CAS failure + } + } + // now close the queue to wake Get() waiters + p.tasks.Close() + }) +} diff --git a/pkg/pool/queue.go b/pkg/pool/queue.go new file mode 100644 index 00000000..6055a755 --- /dev/null +++ b/pkg/pool/queue.go @@ -0,0 +1,108 @@ +package pool + +import ( + "sync" + "sync/atomic" +) + +// noCopy may be embedded into structs which must not be copied after first use. +// go vet will warn on accidental copies (it looks for Lock methods). +type noCopy struct{} + +func (*noCopy) Lock() {} + +// node for single-lock queue (plain pointer; protected by mu) +type node[T any] struct { + val T + next *node[T] +} + +// WorkerPoolQueue is a simple, single-mutex MPMC queue. +// This is easier to reason about than a two-lock variant and avoids lost-wakeup races. +type WorkerPoolQueue[T any] struct { + noCopy noCopy + + mu sync.Mutex + cond *sync.Cond + head *node[T] // sentinel + tail *node[T] + closed bool + size int64 // track queued count (atomic operations used for Len to avoid taking mu) +} + +// NewWorkerPoolQueue constructs a new queue. +func NewWorkerPoolQueue[T any]() *WorkerPoolQueue[T] { + s := &node[T]{} + q := &WorkerPoolQueue[T]{head: s, tail: s} + q.cond = sync.NewCond(&q.mu) + return q +} + +func (q *WorkerPoolQueue[T]) Put(v T) error { + q.mu.Lock() + defer q.mu.Unlock() + if q.closed { + return ErrClosed + } + n := &node[T]{val: v} + q.tail.next = n + q.tail = n + atomic.AddInt64(&q.size, 1) + // signal one waiter (consumer checks under mu) + q.cond.Signal() + return nil +} + +func (q *WorkerPoolQueue[T]) Get() (T, bool) { + q.mu.Lock() + // wait while empty and not closed + for q.head.next == nil && !q.closed { + q.cond.Wait() + } + + // empty + closed => done + if q.head.next == nil { + q.mu.Unlock() + var zero T + return zero, false + } + + // pop head.next + n := q.head.next + q.head.next = n.next + if q.head.next == nil { + q.tail = q.head + } + q.mu.Unlock() + + atomic.AddInt64(&q.size, -1) + return n.val, true +} + +func (q *WorkerPoolQueue[T]) TryGet() (T, bool) { + q.mu.Lock() + if q.head.next == nil { + q.mu.Unlock() + var zero T + return zero, false + } + n := q.head.next + q.head.next = n.next + if q.head.next == nil { + q.tail = q.head + } + q.mu.Unlock() + atomic.AddInt64(&q.size, -1) + return n.val, true +} + +func (q *WorkerPoolQueue[T]) Len() int { + return int(atomic.LoadInt64(&q.size)) +} + +func (q *WorkerPoolQueue[T]) Close() { + q.mu.Lock() + q.closed = true + q.cond.Broadcast() + q.mu.Unlock() +} diff --git a/pkg/pool/virtual_pool.go b/pkg/pool/virtual_pool.go new file mode 100644 index 00000000..0dabb61a --- /dev/null +++ b/pkg/pool/virtual_pool.go @@ -0,0 +1,318 @@ +// Copyright 2024 Nokia +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pool + +import ( + "context" + "errors" + "sync" + "sync/atomic" +) + +// --- Task helpers (lightweight) --- + +// Task is a unit of work executed by the shared worker pool. +// submit allows spawning child tasks into the same logical/virtual pool. +type Task interface { + Run(ctx context.Context, submit func(Task) error) error +} + +// TaskFunc convenience adapter so closures are easy to submit. +type TaskFunc func(ctx context.Context, submit func(Task) error) error + +func (f TaskFunc) Run(ctx context.Context, submit func(Task) error) error { + if f == nil { + return nil + } + return f(ctx, submit) +} + +// --- ErrorCollector (per-virtual tolerant mode) --- + +// ErrorCollector collects errors for a virtual pool. +// It stores a snapshotable slice and provides a live channel for streaming. +type ErrorCollector struct { + mu sync.Mutex + errs []error + Ch chan error +} + +func newErrorCollector(buf int) *ErrorCollector { + if buf <= 0 { + buf = 1024 + } + return &ErrorCollector{ + Ch: make(chan error, buf), + } +} + +func (ec *ErrorCollector) add(err error) { + if err == nil { + return + } + ec.mu.Lock() + ec.errs = append(ec.errs, err) + ec.mu.Unlock() + + select { + case ec.Ch <- err: + default: + // drop if full + } +} + +// Errors returns a snapshot of collected errors. +func (ec *ErrorCollector) Errors() []error { + ec.mu.Lock() + defer ec.mu.Unlock() + out := make([]error, len(ec.errs)) + copy(out, ec.errs) + return out +} + +// close channel when done +func (ec *ErrorCollector) close() { + close(ec.Ch) +} + +// --- Virtual pool system --- + +var ErrVirtualPoolClosed = errors.New("virtual pool closed for submit") + +// VirtualMode controls virtual pool failure semantics. +type VirtualMode int + +const ( + // VirtualFailFast: first error stops executing further tasks for this virtual pool. + VirtualFailFast VirtualMode = iota + // VirtualTolerant: errors are collected, tasks continue. + VirtualTolerant +) + +// SharedTaskPool manages a shared worker pool (reusing Pool[Task]) and provides +// creation of VirtualPools that submit into the shared pool. +type SharedTaskPool struct { + inner *Pool[Task] + + mu sync.RWMutex + vmap map[string]*VirtualPool +} + +// NewSharedTaskPool constructs a shared pool; caller should call Start() to begin workers. +func NewSharedTaskPool(parent context.Context, workerCount int) *SharedTaskPool { + inner := NewWorkerPool[Task](parent, workerCount) + // Start inner with a handler that executes Task.Run but never returns an error to + // avoid aborting the shared pool. Per-virtual semantics are enforced by VirtualPool. + inner.Start(func(ctx context.Context, item Task, submit func(Task) error) error { + _ = item.Run(ctx, submit) + return nil + }) + + return &SharedTaskPool{ + inner: inner, + vmap: make(map[string]*VirtualPool), + } +} + +// CloseForSubmit proxies to underlying pool when caller is done submitting to all virtuals. +func (s *SharedTaskPool) CloseForSubmit() { + s.inner.CloseForSubmit() +} + +// Wait proxies to underlying pool wait. Note: this waits for all tasks on the shared pool. +func (s *SharedTaskPool) Wait() error { + return s.inner.Wait() +} + +// NewVirtualPool creates and registers a virtual pool on top of the shared pool. +// id is an arbitrary identifier (must be unique per SharedTaskPool). +// mode controls failure semantics. buf controls error channel buffer for tolerant mode. +func (s *SharedTaskPool) NewVirtualPool(id string, mode VirtualMode, buf int) *VirtualPool { + v := &VirtualPool{ + id: id, + parent: s, + mode: mode, + ec: nil, + closed: atomic.Bool{}, + firstErr: atomic.Pointer[error]{}, + } + if mode == VirtualTolerant { + v.ec = newErrorCollector(buf) + } + s.mu.Lock() + s.vmap[id] = v + s.mu.Unlock() + return v +} + +// UnregisterVirtualPool removes the virtual pool registration (does not wait or close inner). +func (s *SharedTaskPool) UnregisterVirtualPool(id string) { + s.mu.Lock() + delete(s.vmap, id) + s.mu.Unlock() +} + +// submitWrapped submits a virtualTask into the shared pool. +func (s *SharedTaskPool) submitWrapped(vt *virtualTask) error { + return s.inner.Submit(vt) +} + +// --- VirtualPool types --- + +// VirtualPool represents a logical pool view that reuses shared workers. +// It enforces per-virtual behaviour like fail-fast or tolerant error collection. +type VirtualPool struct { + id string + parent *SharedTaskPool + mode VirtualMode + + ec *ErrorCollector // non-nil for VirtualTolerant + + closed atomic.Bool // closed for new submissions + // firstErr used for fail-fast + firstErr atomic.Pointer[error] + // per-virtual inflight counter (matches lifecycle of tasks submitted by this virtual) + inflight int64 + // ensure collector channel closed only once + collectorOnce sync.Once +} + +// virtualTask wraps a Task with its owning VirtualPool reference. +type virtualTask struct { + vp *VirtualPool + task Task +} + +func (vt *virtualTask) Run(ctx context.Context, submit func(Task) error) error { + // If virtual is closed due to fail-fast, skip executing the task. + if vt.vp.isFailed() { + return nil + } + + // build a submit wrapper so child tasks submitted by this task remain in the same virtual pool. + submitWrapper := func(t Task) error { + return vt.vp.Submit(t) + } + + // Execute the actual task. + err := vt.task.Run(ctx, submitWrapper) + + // handle per-virtual error semantics + if err != nil { + switch vt.vp.mode { + case VirtualFailFast: + vt.vp.recordFirstError(err) + // mark closed so subsequent tasks from this virtual are skipped + vt.vp.closed.Store(true) + case VirtualTolerant: + if vt.vp.ec != nil { + vt.vp.ec.add(err) + } + } + } + + // return nil to shared pool so shared pool doesn't abort + // decrement inflight and possibly close collector if virtual is closed and no more inflight + if remaining := atomic.AddInt64(&vt.vp.inflight, -1); remaining == 0 && vt.vp.closed.Load() { + // close collector once + vt.vp.collectorOnce.Do(func() { + if vt.vp.ec != nil { + vt.vp.ec.close() + } + }) + } + return nil +} + +// --- VirtualPool API --- + +// Submit enqueues a Task into this virtual pool. +// It wraps the Task into a virtualTask that remembers the virtual identity. +func (v *VirtualPool) Submit(t Task) error { + // fast-fail if virtual pool closed for submit + if v.closed.Load() { + return ErrVirtualPoolClosed + } + // If already failed (fail-fast), disallow further submissions. + if v.isFailed() { + return ErrVirtualPoolClosed + } + // increment per-virtual inflight (will be decremented by worker after run) + atomic.AddInt64(&v.inflight, 1) + vt := &virtualTask{vp: v, task: t} + if err := v.parent.submitWrapped(vt); err != nil { + // submission failed: revert inflight + atomic.AddInt64(&v.inflight, -1) + return err + } + return nil +} + +// SubmitFunc convenience to submit a TaskFunc. +func (v *VirtualPool) SubmitFunc(f TaskFunc) error { return v.Submit(f) } + +// CloseForSubmit marks this virtual pool as no longer accepting top-level submissions. +// Note: this does not close the shared pool; caller is responsible for closing the shared pool +// when all virtual pools are done (call SharedTaskPool.CloseForSubmit()). +func (v *VirtualPool) CloseForSubmit() { + v.closed.Store(true) + // if nothing inflight, close collector now + if atomic.LoadInt64(&v.inflight) == 0 { + v.collectorOnce.Do(func() { + if v.ec != nil { + v.ec.close() + } + }) + } +} + +// isFailed returns true if this virtual pool encountered a fail-fast error. +func (v *VirtualPool) isFailed() bool { + if p := v.firstErr.Load(); p != nil && *p != nil { + return true + } + return false +} + +func (v *VirtualPool) recordFirstError(err error) { + ep := new(error) + *ep = err + v.firstErr.CompareAndSwap(nil, ep) // set only first +} + +// FirstError returns the first encountered error for fail-fast virtual pools, or nil. +func (v *VirtualPool) FirstError() error { + if p := v.firstErr.Load(); p != nil && *p != nil { + return *p + } + return nil +} + +// Errors returns a snapshot of collected errors for tolerant virtual pools. +// For fail-fast virtual pools this returns nil. +func (v *VirtualPool) Errors() []error { + if v.ec == nil { + return nil + } + return v.ec.Errors() +} + +// ErrorChan returns the live channel of errors for tolerant mode, or nil for fail-fast mode. +func (v *VirtualPool) ErrorChan() <-chan error { + if v.ec == nil { + return nil + } + return v.ec.Ch +} diff --git a/pkg/pool/virtual_pool_test.go b/pkg/pool/virtual_pool_test.go new file mode 100644 index 00000000..afedec09 --- /dev/null +++ b/pkg/pool/virtual_pool_test.go @@ -0,0 +1,131 @@ +// Copyright 2024 Nokia +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pool + +import ( + "context" + "sync/atomic" + "testing" + "time" +) + +// incrTask is a simple Task that increments a counter and optionally spawns nested child tasks. +type incrTask struct { + counter *int64 + nested int + fail bool +} + +func (t *incrTask) Run(ctx context.Context, submit func(Task) error) error { + atomic.AddInt64(t.counter, 1) + if t.fail { + return &poolTestError{msg: "fail"} + } + for i := 0; i < t.nested; i++ { + if err := submit(&incrTask{counter: t.counter}); err != nil { + return err + } + } + return nil +} + +type poolTestError struct{ msg string } + +func (e *poolTestError) Error() string { return e.msg } + +func TestVirtualPools_TolerantAndFailFast(t *testing.T) { + ctx := context.Background() + sp := NewSharedTaskPool(ctx, 4) + + // create virtual pools + vt := sp.NewVirtualPool("tolerant", VirtualTolerant, 16) + vf := sp.NewVirtualPool("fail", VirtualFailFast, 0) + + // submit tasks: tolerant pool will collect errors, fail pool will stop after first error + var cntT int64 + var cntF int64 + + // tolerant: submit 3 tasks, one of them fails + if err := vt.Submit(&incrTask{counter: &cntT, nested: 0}); err != nil { + t.Fatal(err) + } + if err := vt.Submit(&incrTask{counter: &cntT, nested: 0, fail: true}); err != nil { + t.Fatal(err) + } + if err := vt.Submit(&incrTask{counter: &cntT, nested: 1}); err != nil { + t.Fatal(err) + } + + // fail-fast: submit tasks where second fails + if err := vf.Submit(&incrTask{counter: &cntF, nested: 0}); err != nil { + t.Fatal(err) + } + if err := vf.Submit(&incrTask{counter: &cntF, nested: 0, fail: true}); err != nil { + t.Fatal(err) + } + // this submission after the failing one should be accepted but skipped when running + if err := vf.Submit(&incrTask{counter: &cntF, nested: 0}); err != nil { + t.Fatal(err) + } + + // close virtuals for submit + vt.CloseForSubmit() + vf.CloseForSubmit() + + // close shared pool for submit and wait + sp.CloseForSubmit() + + // drain tolerant error channel live + var seenErrs int32 + done := make(chan struct{}) + go func() { + for e := range vt.ErrorChan() { + if e != nil { + atomic.AddInt32(&seenErrs, 1) + } + } + close(done) + }() + + if err := sp.Wait(); err != nil { + t.Fatalf("shared wait error: %v", err) + } + + // tolerant collector channel is closed automatically when virtual is closed and inflight reaches zero + // wait for drain goroutine + select { + case <-done: + case <-time.After(2 * time.Second): + t.Fatalf("timeout waiting for tolerant drain") + } + + // tolerant: expect 3 increments (one of them had nested:1 so total 3) + if got := atomic.LoadInt64(&cntT); got != 3 { + t.Fatalf("tolerant counter expected 3 got %d", got) + } + + // tolerant: expect at least one error recorded + if len(vt.Errors()) == 0 { + t.Fatalf("expected tolerant errors, got none") + } + + // fail-fast: at least one task should have executed and the virtual should have recorded a first error + if got := atomic.LoadInt64(&cntF); got < 1 || got > 3 { + t.Fatalf("fail-fast counter expected between 1 and 3 got %d", got) + } + if vf.FirstError() == nil { + t.Fatalf("expected fail-fast virtual to record a first error") + } +} diff --git a/pkg/utils/workerpoolqueue_stress_test.go b/pkg/pool/workerpoolqueue_stress_test.go similarity index 99% rename from pkg/utils/workerpoolqueue_stress_test.go rename to pkg/pool/workerpoolqueue_stress_test.go index 1f34e8cc..f8b36f1f 100644 --- a/pkg/utils/workerpoolqueue_stress_test.go +++ b/pkg/pool/workerpoolqueue_stress_test.go @@ -1,4 +1,4 @@ -package utils +package pool import ( "fmt" diff --git a/pkg/tree/parallelImporter.go b/pkg/tree/parallelImporter.go index 6854b2db..e7d56a80 100644 --- a/pkg/tree/parallelImporter.go +++ b/pkg/tree/parallelImporter.go @@ -6,9 +6,9 @@ import ( "runtime" "slices" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/tree/importer" "github.com/sdcio/data-server/pkg/tree/types" - "github.com/sdcio/data-server/pkg/utils" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "google.golang.org/protobuf/types/known/emptypb" ) @@ -29,7 +29,7 @@ func (s *sharedEntryAttributes) ImportConfig( intentPrio int32, insertFlags *types.UpdateInsertFlags, ) error { - p := utils.NewWorkerPool[importTask](ctx, runtime.NumCPU()) + p := pool.NewWorkerPool[importTask](ctx, runtime.NumCPU()) p.Start(importHandler) From b48339c7e7496bf06b0c149887a215dca9b50be2 Mon Sep 17 00:00:00 2001 From: steiler Date: Fri, 17 Oct 2025 11:36:55 +0200 Subject: [PATCH 12/44] deviation clients context canceled --- pkg/datastore/deviations.go | 51 +++++++++++++++++++------------------ 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index f332a938..3297c39c 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -91,33 +91,34 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) func (d *Datastore) SendDeviations(ch <-chan *treetypes.DeviationEntry, deviationClients map[string]sdcpb.DataServer_WatchDeviationsServer) { wg := &sync.WaitGroup{} - for { - select { - case de, ok := <-ch: - if !ok { - wg.Wait() - return - } - wg.Add(1) - go func(de DeviationEntry, dcs map[string]sdcpb.DataServer_WatchDeviationsServer) { - for clientIdentifier, dc := range dcs { - err := dc.Send(&sdcpb.WatchDeviationResponse{ - Name: d.config.Name, - Intent: de.IntentName(), - Event: sdcpb.DeviationEvent_UPDATE, - Reason: sdcpb.DeviationReason(de.Reason()), - Path: de.Path(), - ExpectedValue: de.ExpectedValue(), - CurrentValue: de.CurrentValue(), - }) - if err != nil { - log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) - } + for de := range ch { + wg.Add(1) + go func(de DeviationEntry, dcs map[string]sdcpb.DataServer_WatchDeviationsServer) { + for clientIdentifier, dc := range dcs { + select { + // skip deviation clients with closed context + case <-dc.Context().Done(): + continue + default: } - wg.Done() - }(de, deviationClients) - } + + err := dc.Send(&sdcpb.WatchDeviationResponse{ + Name: d.config.Name, + Intent: de.IntentName(), + Event: sdcpb.DeviationEvent_UPDATE, + Reason: sdcpb.DeviationReason(de.Reason()), + Path: de.Path(), + ExpectedValue: de.ExpectedValue(), + CurrentValue: de.CurrentValue(), + }) + if err != nil { + log.Errorf("error sending deviation to %s: %v", clientIdentifier, err) + } + } + wg.Done() + }(de, deviationClients) } + wg.Wait() } type DeviationEntry interface { From 5ffbdf7a2a6cb78e3e11f743a8d4d708748092d8 Mon Sep 17 00:00:00 2001 From: steiler Date: Mon, 20 Oct 2025 11:09:08 +0200 Subject: [PATCH 13/44] worker index --- pkg/pool/pool.go | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/pkg/pool/pool.go b/pkg/pool/pool.go index f6a959ff..627e17f7 100644 --- a/pkg/pool/pool.go +++ b/pkg/pool/pool.go @@ -50,6 +50,19 @@ func NewWorkerPool[T any](parent context.Context, workerCount int) *Pool[T] { return p } +// workerIndexKey is used to store the worker index in the worker context. +type workerIndexKey struct{} + +// WorkerIndexFromContext returns the worker index stored in ctx, if present. +func WorkerIndexFromContext(ctx context.Context) (int, bool) { + if v := ctx.Value(workerIndexKey{}); v != nil { + if idx, ok := v.(int); ok { + return idx, true + } + } + return 0, false +} + // addInflight increments inflight and must be called when a task is known submitted. func (p *Pool[T]) addInflight(delta int64) { atomic.AddInt64(&p.inflight, delta) @@ -88,8 +101,13 @@ func (p *Pool[T]) Start(handler func(ctx context.Context, item T, submit func(T) // spawn workers p.workersWg.Add(p.workerCount) for i := 0; i < p.workerCount; i++ { + idx := i go func() { defer p.workersWg.Done() + + // attach worker index to this worker's context so handlers/tasks can discover it + workerCtx := context.WithValue(p.ctx, workerIndexKey{}, idx) + for { item, ok := p.tasks.Get() if !ok { @@ -104,7 +122,7 @@ func (p *Pool[T]) Start(handler func(ctx context.Context, item T, submit func(T) } // run handler (handler may call p.Submit) - if err := handler(p.ctx, item, func(it T) error { return p.Submit(it) }); err != nil { + if err := handler(workerCtx, item, func(it T) error { return p.Submit(it) }); err != nil { // store first error safely (allocate on heap) ep := new(error) *ep = err From 122d637c26a2f0cc4fe0e898e8bc1f8b19ec6c41 Mon Sep 17 00:00:00 2001 From: steiler Date: Fri, 24 Oct 2025 11:57:44 +0200 Subject: [PATCH 14/44] update --- pkg/datastore/sync.go | 22 +- pkg/datastore/target/gnmi/get.go | 36 +-- pkg/datastore/target/gnmi/gnmi.go | 6 +- pkg/datastore/target/gnmi/stream.go | 8 +- pkg/datastore/target/types/runningstore.go | 5 +- pkg/pool/virtual_pool.go | 86 +++++- pkg/pool/virtual_pool_test.go | 79 ++++- pkg/tree/sharedEntryAttributes.go | 7 + pkg/tree/visitor_explicit_delete.go | 5 + pkg/tree/visitor_mark_owner_delete.go | 5 + pkg/utils/pool.go | 321 --------------------- 11 files changed, 214 insertions(+), 366 deletions(-) delete mode 100644 pkg/utils/pool.go diff --git a/pkg/datastore/sync.go b/pkg/datastore/sync.go index a6b86dec..5e911e79 100644 --- a/pkg/datastore/sync.go +++ b/pkg/datastore/sync.go @@ -4,37 +4,27 @@ import ( "context" "github.com/sdcio/data-server/pkg/tree" - "github.com/sdcio/data-server/pkg/tree/importer/proto" + "github.com/sdcio/data-server/pkg/tree/importer" treetypes "github.com/sdcio/data-server/pkg/tree/types" sdcpb "github.com/sdcio/sdc-protos/sdcpb" - "github.com/sdcio/sdc-protos/tree_persist" log "github.com/sirupsen/logrus" ) -func (d *Datastore) ApplyToRunning(ctx context.Context, i *tree_persist.Intent) error { - - i.IntentName = tree.RunningIntentName - i.Priority = tree.RunningValuesPrio - i.Deviation = false +func (d *Datastore) ApplyToRunning(ctx context.Context, deletes []*sdcpb.Path, importer importer.ImportConfigAdapter) error { d.syncTreeMutex.Lock() defer d.syncTreeMutex.Unlock() - for _, delete := range i.ExplicitDeletes { + for _, delete := range deletes { //TODO this will most likely give us errors in case optimisticWriteback already deleted the entries. - err := d.syncTree.DeleteBranch(ctx, delete, i.IntentName) + err := d.syncTree.DeleteBranch(ctx, delete, tree.RunningIntentName) if err != nil { log.Errorf("error deleting paths from datastore sync tree: %v", err) return err } } - // need to reset the explicit deletes, they carry the actual deletes that we need to delete. - // the imported would otherwise add explicit deletes for these. - i.ExplicitDeletes = nil - - if i.GetRoot() != nil { - importer := proto.NewProtoTreeImporter(i) - err := d.syncTree.ImportConfig(ctx, &sdcpb.Path{}, importer, i.GetIntentName(), i.GetPriority(), treetypes.NewUpdateInsertFlags()) + if importer != nil { + err := d.syncTree.ImportConfig(ctx, &sdcpb.Path{}, importer, tree.RunningIntentName, tree.RunningValuesPrio, treetypes.NewUpdateInsertFlags()) if err != nil { return err } diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index 32bedc7b..f32fc44c 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -9,6 +9,7 @@ import ( "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" "github.com/sdcio/data-server/pkg/datastore/target/types" "github.com/sdcio/data-server/pkg/tree" + "github.com/sdcio/data-server/pkg/tree/importer/proto" treetypes "github.com/sdcio/data-server/pkg/tree/types" dsutils "github.com/sdcio/data-server/pkg/utils" sdcpb "github.com/sdcio/sdc-protos/sdcpb" @@ -22,11 +23,23 @@ type GetSync struct { runningStore types.RunningStore ctx context.Context schemaClient dsutils.SchemaClientBound + paths []*sdcpb.Path } -func NewGetSync(ctx context.Context, target GetTarget, c *config.SyncProtocol, runningStore types.RunningStore, schemaClient dsutils.SchemaClientBound) *GetSync { +func NewGetSync(ctx context.Context, target GetTarget, c *config.SyncProtocol, runningStore types.RunningStore, schemaClient dsutils.SchemaClientBound) (*GetSync, error) { ctx, cancel := context.WithCancel(ctx) + paths := make([]*sdcpb.Path, 0, len(c.Paths)) + // iterate referenced paths + for _, p := range c.Paths { + path, err := sdcpb.ParsePath(p) + if err != nil { + return nil, err + } + // add the parsed path + paths = append(paths, path) + } + return &GetSync{ config: c, target: target, @@ -34,25 +47,16 @@ func NewGetSync(ctx context.Context, target GetTarget, c *config.SyncProtocol, r runningStore: runningStore, ctx: ctx, schemaClient: schemaClient, - } + paths: paths, + }, nil } func (s *GetSync) syncConfig() (*sdcpb.GetDataRequest, error) { // iterate syncConfig - paths := make([]*sdcpb.Path, 0, len(s.config.Paths)) - // iterate referenced paths - for _, p := range s.config.Paths { - path, err := sdcpb.ParsePath(p) - if err != nil { - return nil, err - } - // add the parsed path - paths = append(paths, path) - } req := &sdcpb.GetDataRequest{ Name: s.config.Name, - Path: paths, + Path: s.paths, DataType: sdcpb.DataType_CONFIG, Encoding: sdcpb.Encoding(utils.ParseSdcpbEncoding(s.config.Encoding)), } @@ -105,7 +109,7 @@ func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { syncTreeMutex := &sync.Mutex{} // process Noifications - deletes, err := processNotifications(s.ctx, resp.GetNotification(), s.schemaClient, syncTree, syncTreeMutex) + _, err = processNotifications(s.ctx, resp.GetNotification(), s.schemaClient, syncTree, syncTreeMutex) if err != nil { log.Errorf("sync process notifications error: %v", err) return @@ -116,10 +120,8 @@ func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { log.Errorf("sync tree export error: %v", err) return } - // add also the deletes to the export - result.ExplicitDeletes = deletes - err = s.runningStore.ApplyToRunning(s.ctx, result) + err = s.runningStore.ApplyToRunning(s.ctx, s.paths, proto.NewProtoTreeImporter(result)) if err != nil { log.Errorf("sync import to running error: %v", err) return diff --git a/pkg/datastore/target/gnmi/gnmi.go b/pkg/datastore/target/gnmi/gnmi.go index cf64c2de..8f5c74b3 100644 --- a/pkg/datastore/target/gnmi/gnmi.go +++ b/pkg/datastore/target/gnmi/gnmi.go @@ -250,12 +250,16 @@ func (t *gnmiTarget) Status() *targetTypes.TargetStatus { func (t *gnmiTarget) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) error { var g GnmiSync + var err error for _, sp := range sps { switch sp.Mode { case "once": g = NewOnceSync(ctx, t, sp, t.runningStore) case "get": - g = NewGetSync(ctx, t, sp, t.runningStore, t.schemaClient) + g, err = NewGetSync(ctx, t, sp, t.runningStore, t.schemaClient) + if err != nil { + return err + } default: g = NewStreamSync(ctx, t, sp, t.runningStore, t.schemaClient) } diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index 4b3818e1..59561c41 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -14,6 +14,7 @@ import ( "github.com/sdcio/data-server/pkg/datastore/target/types" "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/tree" + "github.com/sdcio/data-server/pkg/tree/importer/proto" treetypes "github.com/sdcio/data-server/pkg/tree/types" dsutils "github.com/sdcio/data-server/pkg/utils" sdcpb "github.com/sdcio/sdc-protos/sdcpb" @@ -133,7 +134,10 @@ func (s *StreamSync) Start() error { log.Errorf("sync tree export error: %v", err) return s.runningStore.NewEmptyTree(s.ctx) } - + // extract the explicit deletes + deletes := result.ExplicitDeletes + // set them to nil + result.ExplicitDeletes = nil if logCount { log.Infof("Syncing: %d elements, %d deletes ", result.GetRoot().CountTerminals(), len(result.GetExplicitDeletes())) } @@ -141,7 +145,7 @@ func (s *StreamSync) Start() error { log.Infof("TreeExport to proto took: %s", time.Since(startTime)) startTime = time.Now() - err = s.runningStore.ApplyToRunning(s.ctx, result) + err = s.runningStore.ApplyToRunning(s.ctx, deletes, proto.NewProtoTreeImporter(result)) if err != nil { log.Errorf("sync import to running error: %v", err) return s.runningStore.NewEmptyTree(s.ctx) diff --git a/pkg/datastore/target/types/runningstore.go b/pkg/datastore/target/types/runningstore.go index bf1967d1..5b9836fd 100644 --- a/pkg/datastore/target/types/runningstore.go +++ b/pkg/datastore/target/types/runningstore.go @@ -4,10 +4,11 @@ import ( "context" "github.com/sdcio/data-server/pkg/tree" - "github.com/sdcio/sdc-protos/tree_persist" + "github.com/sdcio/data-server/pkg/tree/importer" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" ) type RunningStore interface { - ApplyToRunning(ctx context.Context, i *tree_persist.Intent) error + ApplyToRunning(ctx context.Context, deletes []*sdcpb.Path, importer importer.ImportConfigAdapter) error NewEmptyTree(ctx context.Context) (*tree.RootEntry, error) } diff --git a/pkg/pool/virtual_pool.go b/pkg/pool/virtual_pool.go index 0dabb61a..da389942 100644 --- a/pkg/pool/virtual_pool.go +++ b/pkg/pool/virtual_pool.go @@ -16,6 +16,8 @@ package pool import ( "context" + "crypto/rand" + "encoding/hex" "errors" "sync" "sync/atomic" @@ -140,19 +142,42 @@ func (s *SharedTaskPool) Wait() error { // id is an arbitrary identifier (must be unique per SharedTaskPool). // mode controls failure semantics. buf controls error channel buffer for tolerant mode. func (s *SharedTaskPool) NewVirtualPool(id string, mode VirtualMode, buf int) *VirtualPool { + // ensure unique id in the shared pool's map. If the requested id is already + // registered, append a short random hex postfix so multiple callers can + // create virtual pools with the same base name without colliding. + s.mu.Lock() + finalID := id + if _, exists := s.vmap[finalID]; exists { + // generate short random postfix until unique + for { + // 4 bytes -> 8 hex chars + b := make([]byte, 4) + if _, err := rand.Read(b); err != nil { + // fallback to using a timestamp-like postfix if crypto fails + finalID = finalID + "-r" + break + } + suf := hex.EncodeToString(b) + candidate := id + "-" + suf + if _, ok := s.vmap[candidate]; !ok { + finalID = candidate + break + } + } + } v := &VirtualPool{ - id: id, + id: finalID, parent: s, mode: mode, ec: nil, closed: atomic.Bool{}, firstErr: atomic.Pointer[error]{}, + done: make(chan struct{}), } if mode == VirtualTolerant { v.ec = newErrorCollector(buf) } - s.mu.Lock() - s.vmap[id] = v + s.vmap[v.id] = v s.mu.Unlock() return v } @@ -187,6 +212,10 @@ type VirtualPool struct { inflight int64 // ensure collector channel closed only once collectorOnce sync.Once + // ensure done channel closed only once (for Wait) + waitOnce sync.Once + // done is closed when the virtual pool is closed for submit and inflight reaches zero + done chan struct{} } // virtualTask wraps a Task with its owning VirtualPool reference. @@ -198,12 +227,25 @@ type virtualTask struct { func (vt *virtualTask) Run(ctx context.Context, submit func(Task) error) error { // If virtual is closed due to fail-fast, skip executing the task. if vt.vp.isFailed() { + // decrement inflight for skipped task and possibly close collector/done + if remaining := atomic.AddInt64(&vt.vp.inflight, -1); remaining == 0 && vt.vp.closed.Load() { + vt.vp.collectorOnce.Do(func() { + if vt.vp.ec != nil { + vt.vp.ec.close() + } + }) + vt.vp.waitOnce.Do(func() { + close(vt.vp.done) + }) + } return nil } // build a submit wrapper so child tasks submitted by this task remain in the same virtual pool. + // Use an internal submit variant so nested submissions from running tasks are allowed + // even after CloseForSubmit() has been called externally. submitWrapper := func(t Task) error { - return vt.vp.Submit(t) + return vt.vp.submitInternal(t) } // Execute the actual task. @@ -232,6 +274,10 @@ func (vt *virtualTask) Run(ctx context.Context, submit func(Task) error) error { vt.vp.ec.close() } }) + // signal Wait() callers that virtual is drained + vt.vp.waitOnce.Do(func() { + close(vt.vp.done) + }) } return nil } @@ -263,6 +309,27 @@ func (v *VirtualPool) Submit(t Task) error { // SubmitFunc convenience to submit a TaskFunc. func (v *VirtualPool) SubmitFunc(f TaskFunc) error { return v.Submit(f) } +// submitInternal is used by running tasks to submit child tasks into the same virtual. +// It bypasses the external CloseForSubmit guard so internal (nested) submissions can +// continue even after CloseForSubmit() has been called. However, fail-fast semantics +// still apply: if the virtual has recorded a first error, nested submissions are +// rejected. +func (v *VirtualPool) submitInternal(t Task) error { + // If already failed (fail-fast), disallow further submissions. + if v.isFailed() { + return ErrVirtualPoolClosed + } + // increment per-virtual inflight (will be decremented by worker after run) + atomic.AddInt64(&v.inflight, 1) + vt := &virtualTask{vp: v, task: t} + if err := v.parent.submitWrapped(vt); err != nil { + // submission failed: revert inflight + atomic.AddInt64(&v.inflight, -1) + return err + } + return nil +} + // CloseForSubmit marks this virtual pool as no longer accepting top-level submissions. // Note: this does not close the shared pool; caller is responsible for closing the shared pool // when all virtual pools are done (call SharedTaskPool.CloseForSubmit()). @@ -275,9 +342,20 @@ func (v *VirtualPool) CloseForSubmit() { v.ec.close() } }) + // signal Wait() callers that virtual is drained + v.waitOnce.Do(func() { + close(v.done) + }) } } +// Wait blocks until this virtual pool has been closed for submit and all inflight tasks +// (including queued tasks) have completed. Call this after CloseForSubmit when you +// want to wait for the virtual's queue to drain. +func (v *VirtualPool) Wait() { + <-v.done +} + // isFailed returns true if this virtual pool encountered a fail-fast error. func (v *VirtualPool) isFailed() bool { if p := v.firstErr.Load(); p != nil && *p != nil { diff --git a/pkg/pool/virtual_pool_test.go b/pkg/pool/virtual_pool_test.go index afedec09..ff5bf03a 100644 --- a/pkg/pool/virtual_pool_test.go +++ b/pkg/pool/virtual_pool_test.go @@ -111,9 +111,9 @@ func TestVirtualPools_TolerantAndFailFast(t *testing.T) { t.Fatalf("timeout waiting for tolerant drain") } - // tolerant: expect 3 increments (one of them had nested:1 so total 3) - if got := atomic.LoadInt64(&cntT); got != 3 { - t.Fatalf("tolerant counter expected 3 got %d", got) + // tolerant: expect 4 increments (one of the tasks had nested:1 which is now allowed even after CloseForSubmit) + if got := atomic.LoadInt64(&cntT); got != 4 { + t.Fatalf("tolerant counter expected 4 got %d", got) } // tolerant: expect at least one error recorded @@ -129,3 +129,76 @@ func TestVirtualPools_TolerantAndFailFast(t *testing.T) { t.Fatalf("expected fail-fast virtual to record a first error") } } + +// Test that Wait blocks until the virtual pool is drained (tolerant mode). +func TestVirtualPool_Wait_Tolerant(t *testing.T) { + ctx := context.Background() + sp := NewSharedTaskPool(ctx, 2) + vt := sp.NewVirtualPool("wait-tolerant", VirtualTolerant, 4) + + var cnt int64 + // submit a few tasks + if err := vt.Submit(&incrTask{counter: &cnt}); err != nil { + t.Fatal(err) + } + if err := vt.Submit(&incrTask{counter: &cnt}); err != nil { + t.Fatal(err) + } + + // stop accepting submissions + vt.CloseForSubmit() + // Wait should block until inflight==0 and then return + go func() { + // close shared when we're done submitting all virtuals + sp.CloseForSubmit() + }() + + // Wait on virtual specifically + vt.Wait() + + // ensure error channel closed (collector closed) + select { + case _, ok := <-vt.ErrorChan(): + if ok { + t.Fatalf("expected error channel to be closed") + } + default: + // channel may be drained; ensure snapshot is usable + } + + if got := atomic.LoadInt64(&cnt); got != 2 { + t.Fatalf("expected 2 increments, got %d", got) + } +} + +// Test that Wait unblocks for a fail-fast virtual even if some queued tasks are skipped. +func TestVirtualPool_Wait_FailFast(t *testing.T) { + ctx := context.Background() + sp := NewSharedTaskPool(ctx, 2) + vf := sp.NewVirtualPool("wait-fail", VirtualFailFast, 0) + + var cnt int64 + // submit a task that will fail and another that would be skipped when run + if err := vf.Submit(&incrTask{counter: &cnt}); err != nil { + t.Fatal(err) + } + if err := vf.Submit(&incrTask{counter: &cnt, fail: true}); err != nil { + t.Fatal(err) + } + if err := vf.Submit(&incrTask{counter: &cnt}); err != nil { + t.Fatal(err) + } + + vf.CloseForSubmit() + sp.CloseForSubmit() + + // Wait should return once virtual is drained (some tasks may be skipped but inflight reaches 0) + vf.Wait() + + if vf.FirstError() == nil { + t.Fatalf("expected first error on fail-fast virtual") + } + if got := atomic.LoadInt64(&cnt); got < 1 { + t.Fatalf("expected at least one task executed, got %d", got) + } +} diff --git a/pkg/tree/sharedEntryAttributes.go b/pkg/tree/sharedEntryAttributes.go index 410485d1..d84ac74a 100644 --- a/pkg/tree/sharedEntryAttributes.go +++ b/pkg/tree/sharedEntryAttributes.go @@ -832,6 +832,10 @@ func (s *sharedEntryAttributes) DeleteBranch(ctx context.Context, path *sdcpb.Pa return err } + if entry == nil { + return nil + } + // need to remove the leafvariants down from entry. // however if the path points to a key, which is in fact getting deleted // we also need to remove the key, which is the parent. Thats why we do it in this loop @@ -842,6 +846,9 @@ func (s *sharedEntryAttributes) DeleteBranch(ctx context.Context, path *sdcpb.Pa // forward the entry pointer to the parent // depending on the remains var the DeleteSubtree is again called on that parent entry entry = entry.GetParent() + if entry == nil { + return nil + } // calling DeleteSubtree with the empty string, because it should not delete the owner from the higher level keys, // but what it will also do is delete possibly dangling key elements in the tree entry.deleteCanDeleteChilds(true) diff --git a/pkg/tree/visitor_explicit_delete.go b/pkg/tree/visitor_explicit_delete.go index fc96ae67..4991b68c 100644 --- a/pkg/tree/visitor_explicit_delete.go +++ b/pkg/tree/visitor_explicit_delete.go @@ -2,6 +2,7 @@ package tree import ( "context" + "sync" "github.com/sdcio/data-server/pkg/utils" ) @@ -13,6 +14,7 @@ type ExplicitDeleteVisitor struct { // created entries for further stat calculation relatedLeafVariants LeafVariantSlice + rlvMutex *sync.Mutex } var _ EntryVisitor = (*ExplicitDeleteVisitor)(nil) @@ -22,6 +24,7 @@ func NewExplicitDeleteVisitor(owner string, priority int32) *ExplicitDeleteVisit priority: priority, owner: owner, relatedLeafVariants: []*LeafEntry{}, + rlvMutex: &sync.Mutex{}, } } @@ -35,7 +38,9 @@ func (edv *ExplicitDeleteVisitor) Visit(ctx context.Context, e Entry) error { } else { le = e.GetLeafVariantEntries().AddExplicitDeleteEntry(edv.owner, edv.priority) } + edv.rlvMutex.Lock() edv.relatedLeafVariants = append(edv.relatedLeafVariants, le) + edv.rlvMutex.Unlock() return nil } diff --git a/pkg/tree/visitor_mark_owner_delete.go b/pkg/tree/visitor_mark_owner_delete.go index 7e62f398..ca0b5e2d 100644 --- a/pkg/tree/visitor_mark_owner_delete.go +++ b/pkg/tree/visitor_mark_owner_delete.go @@ -2,6 +2,7 @@ package tree import ( "context" + "sync" ) type MarkOwnerDeleteVisitor struct { @@ -9,6 +10,7 @@ type MarkOwnerDeleteVisitor struct { owner string onlyIntended bool leafVariantsMatched LeafVariantSlice + lvMutex *sync.Mutex } var _ EntryVisitor = (*MarkOwnerDeleteVisitor)(nil) @@ -18,13 +20,16 @@ func NewMarkOwnerDeleteVisitor(owner string, onlyIntended bool) *MarkOwnerDelete owner: owner, onlyIntended: onlyIntended, leafVariantsMatched: LeafVariantSlice{}, + lvMutex: &sync.Mutex{}, } } func (o *MarkOwnerDeleteVisitor) Visit(ctx context.Context, e Entry) error { le := e.GetLeafVariantEntries().MarkOwnerForDeletion(o.owner, o.onlyIntended) if le != nil { + o.lvMutex.Lock() o.leafVariantsMatched = append(o.leafVariantsMatched, le) + o.lvMutex.Unlock() } return nil } diff --git a/pkg/utils/pool.go b/pkg/utils/pool.go deleted file mode 100644 index 62786b7c..00000000 --- a/pkg/utils/pool.go +++ /dev/null @@ -1,321 +0,0 @@ -package utils - -import ( - "context" - "errors" - "runtime" - "sync" - "sync/atomic" -) - -// Pool[T] is a worker pool backed by WorkerPoolQueue. -// It uses an atomic inflight counter + cond to avoid deadlocks between closing the queue -// and tracking outstanding work. -type Pool[T any] struct { - tasks *WorkerPoolQueue[T] - workerCount int - - ctx context.Context - cancel context.CancelFunc - - workersWg sync.WaitGroup // wait for worker goroutines to exit - - closeOnce sync.Once - - firstErr atomic.Pointer[error] - - closedForSubmit atomic.Bool - - // inflight counter and condition for waiting until work drains - inflight int64 - inflightMu sync.Mutex - inflightC *sync.Cond -} - -// NewWorkerPool creates a new Pool. If workerCount <= 0 it defaults to runtime.NumCPU(). -func NewWorkerPool[T any](parent context.Context, workerCount int) *Pool[T] { - if workerCount <= 0 { - workerCount = runtime.NumCPU() - } - ctx, cancel := context.WithCancel(parent) - p := &Pool[T]{ - tasks: NewWorkerPoolQueue[T](), - workerCount: workerCount, - ctx: ctx, - cancel: cancel, - } - p.inflightC = sync.NewCond(&p.inflightMu) - return p -} - -// addInflight increments inflight and must be called when a task is known submitted. -func (p *Pool[T]) addInflight(delta int64) { - atomic.AddInt64(&p.inflight, delta) - if atomic.LoadInt64(&p.inflight) == 0 { - // wake any waiter (lock to satisfy cond's invariant) - p.inflightMu.Lock() - p.inflightC.Broadcast() - p.inflightMu.Unlock() - } -} - -// Submit enqueues a task. It increments the inflight counter BEFORE attempting to enqueue. -// If ctx is already cancelled, Submit returns ctx.Err() and does NOT increment inflight. -func (p *Pool[T]) Submit(item T) error { - // fast-fail if canceled - if err := p.ctx.Err(); err != nil { - return err - } - - // increment inflight first - p.addInflight(1) - - // try to put into queue - if err := p.tasks.Put(item); err != nil { - // queue closed (or otherwise failed) -> unaccount the inflight and wake waiters if needed - p.addInflight(-1) - return err - } - return nil -} - -// Start spawns workerCount workers that call handler(ctx, item, submit). -// Handler should process the item and return an error if it wants to abort the whole pool. -// Handler may call submit(...) to add child tasks (workers are allowed to submit). -func (p *Pool[T]) Start(handler func(ctx context.Context, item T, submit func(T) error) error) { - // spawn workers - p.workersWg.Add(p.workerCount) - for i := 0; i < p.workerCount; i++ { - go func() { - defer p.workersWg.Done() - for { - item, ok := p.tasks.Get() - if !ok { - // queue closed and drained -> exit worker - return - } - - // If ctx canceled, we must still decrement inflight for this item and skip handler. - if p.ctx.Err() != nil { - p.addInflight(-1) - continue - } - - // run handler (handler may call p.Submit) - if err := handler(p.ctx, item, func(it T) error { return p.Submit(it) }); err != nil { - // store first error safely (allocate on heap) - ep := new(error) - *ep = err - p.firstErr.CompareAndSwap(nil, ep) - - // cancel pool so other workers see ctx canceled - p.cancel() - - // decrement inflight for this item - p.addInflight(-1) - - // force-close the queue and abandon queued items (so we won't wait forever) - p.forceClose() - - // continue so other workers can observe ctx and drain/exit - continue - } - - // normal completion of this task: decrement inflight - p.addInflight(-1) - } - }() - } - - // monitor goroutine: when CloseForSubmit has been called, wait until both inflight==0 and queue empty, - // then close the queue so workers exit. Also handle ctx cancellation (force-close). - go func() { - // We'll wait on the condition variable instead of busy looping. - p.inflightMu.Lock() - defer p.inflightMu.Unlock() - for { - // If CloseForSubmit was called, wait for inflight==0 and queue empty then close queue. - if p.closedForSubmit.Load() { - for atomic.LoadInt64(&p.inflight) != 0 || p.tasks.Len() != 0 { - p.inflightC.Wait() - } - // Now safe to close queue: there is no inflight and no queued items - p.closeOnce.Do(func() { p.tasks.Close() }) - return - } - - // If ctx canceled -> force-close path. - if p.ctx.Err() != nil { - // we hold inflightMu; unlock before calling forceClose (which may broadcast/use locks). - p.inflightMu.Unlock() - p.forceClose() - return - } - - // Wait to be signalled when either inflight changes or CloseForSubmit is called. - p.inflightC.Wait() - // loop and recheck conditions - } - }() -} - -// CloseForSubmit indicates the caller will not submit more external (caller-side) tasks. -// Workers may still call Submit to add child tasks. When inflight reaches zero and queue is empty, -// the pool will close tasks so workers exit. -func (p *Pool[T]) CloseForSubmit() { - p.closedForSubmit.Store(true) - // kick the monitor by signaling condition in case inflight==0 already - p.inflightMu.Lock() - p.inflightC.Broadcast() - p.inflightMu.Unlock() -} - -// Wait blocks until all workers have exited and returns the first error (if any). -func (p *Pool[T]) Wait() error { - p.workersWg.Wait() - if e := p.firstErr.Load(); e != nil && *e != nil { - return *e - } - if p.ctx.Err() != nil && !errors.Is(p.ctx.Err(), context.Canceled) { - return p.ctx.Err() - } - return nil -} - -// forceClose performs a one-time forced shutdown: cancel context, close queue and -// subtract any queued-but-unprocessed items from inflight so waiters don't block forever. -func (p *Pool[T]) forceClose() { - p.cancel() - p.closeOnce.Do(func() { - // first capture queued items - queued := p.tasks.Len() - if queued > 0 { - // reduce inflight by queued. Use atomic and then broadcast condition. - // Ensure we don't go negative. - for { - cur := atomic.LoadInt64(&p.inflight) - // clamp - var toSub int64 = int64(queued) - if toSub > cur { - toSub = cur - } - if toSub == 0 { - break - } - if atomic.CompareAndSwapInt64(&p.inflight, cur, cur-toSub) { - p.inflightMu.Lock() - p.inflightC.Broadcast() - p.inflightMu.Unlock() - break - } - // retry on CAS failure - } - } - // now close the queue to wake Get() waiters - p.tasks.Close() - }) -} - -var ErrClosed = errors.New("queue closed") - -// noCopy may be embedded into structs which must not be copied after first use. -// go vet will warn on accidental copies (it looks for Lock methods). -type noCopy struct{} - -func (*noCopy) Lock() {} - -// node for single-lock queue (plain pointer; protected by mu) -type node[T any] struct { - val T - next *node[T] -} - -// WorkerPoolQueue is a simple, single-mutex MPMC queue. -// This is easier to reason about than a two-lock variant and avoids lost-wakeup races. -type WorkerPoolQueue[T any] struct { - noCopy noCopy - - mu sync.Mutex - cond *sync.Cond - head *node[T] // sentinel - tail *node[T] - closed bool - size int64 // track queued count (atomic operations used for Len to avoid taking mu) -} - -// NewWorkerPoolQueue constructs a new queue. -func NewWorkerPoolQueue[T any]() *WorkerPoolQueue[T] { - s := &node[T]{} - q := &WorkerPoolQueue[T]{head: s, tail: s} - q.cond = sync.NewCond(&q.mu) - return q -} - -func (q *WorkerPoolQueue[T]) Put(v T) error { - q.mu.Lock() - defer q.mu.Unlock() - if q.closed { - return ErrClosed - } - n := &node[T]{val: v} - q.tail.next = n - q.tail = n - atomic.AddInt64(&q.size, 1) - // signal one waiter (consumer checks under mu) - q.cond.Signal() - return nil -} - -func (q *WorkerPoolQueue[T]) Get() (T, bool) { - q.mu.Lock() - // wait while empty and not closed - for q.head.next == nil && !q.closed { - q.cond.Wait() - } - - // empty + closed => done - if q.head.next == nil { - q.mu.Unlock() - var zero T - return zero, false - } - - // pop head.next - n := q.head.next - q.head.next = n.next - if q.head.next == nil { - q.tail = q.head - } - q.mu.Unlock() - - atomic.AddInt64(&q.size, -1) - return n.val, true -} - -func (q *WorkerPoolQueue[T]) TryGet() (T, bool) { - q.mu.Lock() - if q.head.next == nil { - q.mu.Unlock() - var zero T - return zero, false - } - n := q.head.next - q.head.next = n.next - if q.head.next == nil { - q.tail = q.head - } - q.mu.Unlock() - atomic.AddInt64(&q.size, -1) - return n.val, true -} - -func (q *WorkerPoolQueue[T]) Len() int { - return int(atomic.LoadInt64(&q.size)) -} - -func (q *WorkerPoolQueue[T]) Close() { - q.mu.Lock() - q.closed = true - q.cond.Broadcast() - q.mu.Unlock() -} From 5548bea7e5e30dd2c59d0bf2d81dc1a145b97676 Mon Sep 17 00:00:00 2001 From: steiler Date: Mon, 27 Oct 2025 13:52:10 +0100 Subject: [PATCH 15/44] fix parallel import Leaflist --- pkg/tree/parallelImporter.go | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/pkg/tree/parallelImporter.go b/pkg/tree/parallelImporter.go index e7d56a80..9aa2eca4 100644 --- a/pkg/tree/parallelImporter.go +++ b/pkg/tree/parallelImporter.go @@ -3,8 +3,8 @@ package tree import ( "context" "fmt" - "runtime" "slices" + "sync" "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/tree/importer" @@ -20,6 +20,7 @@ type importTask struct { intentPrio int32 insertFlags *types.UpdateInsertFlags treeContext *TreeContext + leafListLock *sync.Map } func (s *sharedEntryAttributes) ImportConfig( @@ -29,12 +30,12 @@ func (s *sharedEntryAttributes) ImportConfig( intentPrio int32, insertFlags *types.UpdateInsertFlags, ) error { - p := pool.NewWorkerPool[importTask](ctx, runtime.NumCPU()) + p := pool.NewWorkerPool[importTask](ctx, 1) p.Start(importHandler) // seed root - if err := p.Submit(importTask{entry: s, importerElement: importerElement, intentName: intentName, intentPrio: intentPrio, insertFlags: insertFlags, treeContext: s.treeContext}); err != nil { + if err := p.Submit(importTask{entry: s, importerElement: importerElement, intentName: intentName, intentPrio: intentPrio, insertFlags: insertFlags, treeContext: s.treeContext, leafListLock: &sync.Map{}}); err != nil { return err } @@ -75,7 +76,7 @@ func importHandler(ctx context.Context, task importTask, submit func(importTask) } // submit resolved entry with same adapter element // return importHandler(ctx, importTask{entry: actual, importerElement: task.importerElement, intentName: task.intentName, intentPrio: task.intentPrio, insertFlags: task.insertFlags, treeContext: task.treeContext}, submit) - return submit(importTask{entry: actual, importerElement: task.importerElement, intentName: task.intentName, intentPrio: task.intentPrio, insertFlags: task.insertFlags, treeContext: task.treeContext}) + return submit(importTask{entry: actual, importerElement: task.importerElement, intentName: task.intentName, intentPrio: task.intentPrio, insertFlags: task.insertFlags, treeContext: task.treeContext, leafListLock: task.leafListLock}) } // presence container or children @@ -100,7 +101,7 @@ func importHandler(ctx context.Context, task importTask, submit func(importTask) return fmt.Errorf("error inserting %s at %s: %w", childElt.GetName(), task.entry.SdcpbPath().ToXPath(false), err) } } - if err := submit(importTask{entry: child, importerElement: childElt, intentName: task.intentName, intentPrio: task.intentPrio, insertFlags: task.insertFlags, treeContext: task.treeContext}); err != nil { + if err := submit(importTask{entry: child, importerElement: childElt, intentName: task.intentName, intentPrio: task.intentPrio, insertFlags: task.insertFlags, treeContext: task.treeContext, leafListLock: task.leafListLock}); err != nil { return err } } @@ -116,6 +117,12 @@ func importHandler(ctx context.Context, task importTask, submit func(importTask) return nil case *sdcpb.SchemaElem_Leaflist: + llm, loaded := task.leafListLock.LoadOrStore(task.entry.SdcpbPath().ToXPath(false), &sync.Mutex{}) + _ = loaded + llMutex := llm.(*sync.Mutex) + llMutex.Lock() + defer llMutex.Unlock() + var scalarArr *sdcpb.ScalarArray mustAdd := false le := task.entry.GetLeafVariantEntries().GetByOwner(task.intentName) From 3a46884d83a6c6170c3260c93c1ae9d0f0c0b45e Mon Sep 17 00:00:00 2001 From: steiler Date: Mon, 27 Oct 2025 13:53:41 +0100 Subject: [PATCH 16/44] parallel blame --- pkg/tree/visitor_blame_config.go | 128 +++++++++++++++++--------- pkg/tree/visitor_blame_config_test.go | 13 +-- 2 files changed, 91 insertions(+), 50 deletions(-) diff --git a/pkg/tree/visitor_blame_config.go b/pkg/tree/visitor_blame_config.go index 8a4f23c3..1a30a40d 100644 --- a/pkg/tree/visitor_blame_config.go +++ b/pkg/tree/visitor_blame_config.go @@ -2,81 +2,121 @@ package tree import ( "context" - "slices" - "strings" + "sync" + "github.com/sdcio/data-server/pkg/pool" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "google.golang.org/protobuf/proto" ) -type BlameConfigVisitor struct { - BaseVisitor - stack []*sdcpb.BlameTreeElement - includeDefaults bool +func BlameConfig(ctx context.Context, e Entry, includeDefaults bool, pool pool.VirtualPoolI) (*sdcpb.BlameTreeElement, error) { + dropChan := make(chan *DropBlameChild, 10) + + wg := &sync.WaitGroup{} + wg.Add(1) + // execute the deletes in a seperate single channel + go func(dC <-chan *DropBlameChild) { + for elem := range dC { + elem.Exec() + } + wg.Done() + }(dropChan) + + blameTask := NewBlameConfigTask(e, dropChan, includeDefaults) + err := pool.Submit(blameTask) + if err != nil { + return nil, err + } + // close pool for additional external submission + pool.CloseForSubmit() + // wait for the pool to run dry + pool.Wait() + // close the dropChan channel + close(dropChan) + wg.Wait() + + return blameTask.self, pool.FirstError() } -var _ EntryVisitor = (*BlameConfigVisitor)(nil) +type BlameConfigTask struct { + includeDefaults bool + parent *sdcpb.BlameTreeElement + self *sdcpb.BlameTreeElement + selfEntry Entry + dropChan chan<- *DropBlameChild +} -func NewBlameConfigVisitor(includeDefaults bool) *BlameConfigVisitor { - return &BlameConfigVisitor{ - stack: []*sdcpb.BlameTreeElement{}, +func NewBlameConfigTask(e Entry, dropChan chan<- *DropBlameChild, includeDefaults bool) *BlameConfigTask { + return &BlameConfigTask{ includeDefaults: includeDefaults, + parent: nil, + self: &sdcpb.BlameTreeElement{}, + selfEntry: e, + dropChan: dropChan, } } -func (b *BlameConfigVisitor) Visit(ctx context.Context, e Entry) error { - name := e.PathName() - if e.IsRoot() { - name = "root" +func (t *BlameConfigTask) Run(ctx context.Context, submit func(pool.Task) error) error { + t.self.Name = t.selfEntry.PathName() + if t.selfEntry.IsRoot() { + t.self.Name = "root" } - result := sdcpb.NewBlameTreeElement(name) - skipAdd := false // process Value - highestLe := e.GetLeafVariantEntries().GetHighestPrecedence(false, true, true) + highestLe := t.selfEntry.GetLeafVariantEntries().GetHighestPrecedence(false, true, true) if highestLe != nil { - if highestLe.Update.Owner() != DefaultsIntentName || b.includeDefaults { - result.SetValue(highestLe.Update.Value()).SetOwner(highestLe.Update.Owner()) + if highestLe.Update.Owner() != DefaultsIntentName || t.includeDefaults { + t.self.SetValue(highestLe.Update.Value()).SetOwner(highestLe.Update.Owner()) // check if running equals the expected - runningLe := e.GetLeafVariantEntries().GetRunning() + runningLe := t.selfEntry.GetLeafVariantEntries().GetRunning() if runningLe != nil { if !proto.Equal(runningLe.Update.Value(), highestLe.Update.Value()) { - result.SetDeviationValue(runningLe.Value()) + t.self.SetDeviationValue(runningLe.Value()) } } } else { // if it is default but no default is meant to be returned - skipAdd = true + t.dropChan <- &DropBlameChild{parent: t.parent, dropElem: t.self} } } - // add to the result tree as a child - if !skipAdd && len(b.stack) > 0 { - b.stack[len(b.stack)-1].AddChild(result) - } - // add to the stack as last element - b.stack = append(b.stack, result) - return nil -} -func (b *BlameConfigVisitor) Up() { - // sort to make te output stable - slices.SortFunc(b.stack[len(b.stack)-1].Childs, func(a *sdcpb.BlameTreeElement, b *sdcpb.BlameTreeElement) int { - return strings.Compare(a.GetName(), b.GetName()) - }) - // remove the last elem from stack - if len(b.stack) > 1 { - b.stack = b.stack[:len(b.stack)-1] + for _, childEntry := range t.selfEntry.GetChilds(DescendMethodActiveChilds) { + child := &sdcpb.BlameTreeElement{Name: childEntry.PathName()} + t.self.AddChild(child) + + // create a new task for each child + task := &BlameConfigTask{ + includeDefaults: t.includeDefaults, + parent: t.self, + self: child, + selfEntry: childEntry, + dropChan: t.dropChan, + } + // submit the task + if err := submit(task); err != nil { + return err + } } + + return nil } -func (o *BlameConfigVisitor) DescendMethod() DescendMethod { - return DescendMethodAll +type DropBlameChild struct { + parent *sdcpb.BlameTreeElement + dropElem *sdcpb.BlameTreeElement } -func (b *BlameConfigVisitor) GetResult() *sdcpb.BlameTreeElement { - if len(b.stack) == 1 { - return b.stack[0] +func (d *DropBlameChild) Exec() { + // from parent drop the child dropElem + index := -1 + for i, child := range d.parent.GetChilds() { + if child == d.dropElem { + index = i + break + } + } + if index != -1 { + d.parent.Childs = append(d.parent.Childs[:index], d.parent.Childs[index+1:]...) } - return nil } diff --git a/pkg/tree/visitor_blame_config_test.go b/pkg/tree/visitor_blame_config_test.go index 5d283fae..4b937904 100644 --- a/pkg/tree/visitor_blame_config_test.go +++ b/pkg/tree/visitor_blame_config_test.go @@ -6,6 +6,7 @@ import ( "github.com/google/go-cmp/cmp" "github.com/openconfig/ygot/ygot" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/utils/testhelper" sdcio_schema "github.com/sdcio/data-server/tests/sdcioygot" sdcpb "github.com/sdcio/sdc-protos/sdcpb" @@ -150,16 +151,16 @@ func Test_sharedEntryAttributes_BlameConfig(t *testing.T) { t.Run(tt.name, func(t *testing.T) { treeRoot := tt.r(t) - bcv := NewBlameConfigVisitor(tt.includeDefaults) + sharedPool := pool.NewSharedTaskPool(ctx, 10) - err := treeRoot.Walk(ctx, bcv) - if (err != nil) != tt.wantErr { - t.Errorf("NewBlameConfigVisitor error = %v, wantErr %v", err, tt.wantErr) + vPool := sharedPool.NewVirtualPool("blame", pool.VirtualFailFast, 1) + + got, err := BlameConfig(ctx, treeRoot.sharedEntryAttributes, tt.includeDefaults, vPool) + if err != nil { + t.Errorf("BlameConfig() error %s", err) return } - got := bcv.GetResult() - t.Log(got.ToString()) want := &sdcpb.BlameTreeElement{} From f8ad2329c679e5ed87274418056f1c4043fc3e9a Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 5 Nov 2025 11:23:27 +0100 Subject: [PATCH 17/44] update --- pkg/datastore/datastore_rpc.go | 16 +-- pkg/datastore/target/gnmi/get.go | 1 + pkg/datastore/transaction_rpc.go | 14 ++- pkg/datastore/tree_operation_test.go | 16 ++- pkg/pool/virtual_pool.go | 40 +----- pkg/pool/virtual_pool_iface.go | 24 ++++ pkg/pool/virtual_pool_test.go | 8 +- pkg/tree/entry_test.go | 26 +++- ...me_config.go => processor_blame_config.go} | 62 +++++++--- ...test.go => processor_blame_config_test.go} | 9 +- pkg/tree/processor_mark_owner_delete.go | 116 ++++++++++++++++++ pkg/tree/root_entry.go | 1 + pkg/tree/sharedEntryAttributes_test.go | 10 +- pkg/tree/visitor_explicit_delete.go | 61 +++++++++ pkg/tree/visitor_mark_owner_delete.go | 45 ------- pkg/tree/xml_test.go | 10 +- 16 files changed, 329 insertions(+), 130 deletions(-) create mode 100644 pkg/pool/virtual_pool_iface.go rename pkg/tree/{visitor_blame_config.go => processor_blame_config.go} (67%) rename pkg/tree/{visitor_blame_config_test.go => processor_blame_config_test.go} (97%) create mode 100644 pkg/tree/processor_mark_owner_delete.go delete mode 100644 pkg/tree/visitor_mark_owner_delete.go diff --git a/pkg/datastore/datastore_rpc.go b/pkg/datastore/datastore_rpc.go index 117862d0..9944b9c8 100644 --- a/pkg/datastore/datastore_rpc.go +++ b/pkg/datastore/datastore_rpc.go @@ -17,6 +17,7 @@ package datastore import ( "context" "errors" + "runtime" "sync" "time" @@ -30,6 +31,7 @@ import ( "github.com/sdcio/data-server/pkg/datastore/target" targettypes "github.com/sdcio/data-server/pkg/datastore/target/types" "github.com/sdcio/data-server/pkg/datastore/types" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/schema" "github.com/sdcio/data-server/pkg/tree" ) @@ -70,6 +72,8 @@ type Datastore struct { // owned by sync syncTreeCandidate *tree.RootEntry + + taskPool *pool.SharedTaskPool } // New creates a new datastore, its schema server client and initializes the SBI target @@ -96,6 +100,7 @@ func New(ctx context.Context, c *config.DatastoreConfig, sc schema.Client, cc ca currentIntentsDeviations: make(map[string][]*sdcpb.WatchDeviationResponse), syncTree: syncTreeRoot, syncTreeMutex: &sync.RWMutex{}, + taskPool: pool.NewSharedTaskPool(ctx, runtime.NumCPU()), } ds.transactionManager = types.NewTransactionManager(NewDatastoreRollbackAdapter(ds)) @@ -222,13 +227,10 @@ func (d *Datastore) BlameConfig(ctx context.Context, includeDefaults bool) (*sdc return nil, err } - // calculate the Blame - bcv := tree.NewBlameConfigVisitor(includeDefaults) - err = root.Walk(ctx, bcv) - if err != nil { - return nil, err - } - bte := bcv.GetResult() + blamePool := d.taskPool.NewVirtualPool(pool.VirtualFailFast, 1) + bcp := tree.NewBlameConfigProcessor(tree.NewBlameConfigProcessorConfig(includeDefaults)) + + bte, err := bcp.Run(ctx, root.GetRoot(), blamePool) // set the root level elements name to the target name bte.Name = d.config.Name diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index f32fc44c..74725fbd 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -34,6 +34,7 @@ func NewGetSync(ctx context.Context, target GetTarget, c *config.SyncProtocol, r for _, p := range c.Paths { path, err := sdcpb.ParsePath(p) if err != nil { + cancel() return nil, err } // add the parsed path diff --git a/pkg/datastore/transaction_rpc.go b/pkg/datastore/transaction_rpc.go index 5d0a2299..60be2a71 100644 --- a/pkg/datastore/transaction_rpc.go +++ b/pkg/datastore/transaction_rpc.go @@ -8,6 +8,7 @@ import ( "time" "github.com/sdcio/data-server/pkg/datastore/types" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/tree" treeproto "github.com/sdcio/data-server/pkg/tree/importer/proto" treetypes "github.com/sdcio/data-server/pkg/tree/types" @@ -213,11 +214,20 @@ func (d *Datastore) lowlevelTransactionSet(ctx context.Context, transaction *typ oldIntentContent := lvs.ToUpdateSlice() - marksOwnerDeleteVisitor := tree.NewMarkOwnerDeleteVisitor(intent.GetName(), intent.GetOnlyIntended()) - err := root.Walk(ctx, marksOwnerDeleteVisitor) + deleteVisitorPool := d.taskPool.NewVirtualPool(pool.VirtualFailFast, 1) + ownerDeleteMarker := tree.NewOwnerDeleteMarker(tree.NewOwnerDeleteMarkerTaskConfig(intent.GetName(), intent.GetOnlyIntended())) + + err := ownerDeleteMarker.Run(root.GetRoot(), deleteVisitorPool) if err != nil { return nil, err } + + // marksOwnerDeleteVisitor := tree.NewMarkOwnerDeleteVisitor(intent.GetName(), intent.GetOnlyIntended()) + // err := root.Walk(ctx, marksOwnerDeleteVisitor) + // if err != nil { + // return nil, err + // } + // clear the owners existing explicit delete entries, retrieving the old entries for storing in the transaction for possible rollback oldExplicitDeletes := root.RemoveExplicitDeletes(intent.GetName()) diff --git a/pkg/datastore/tree_operation_test.go b/pkg/datastore/tree_operation_test.go index 7ce2f6e8..d33b16c0 100644 --- a/pkg/datastore/tree_operation_test.go +++ b/pkg/datastore/tree_operation_test.go @@ -18,12 +18,14 @@ import ( "context" "encoding/json" "fmt" + "runtime" "strings" "testing" "github.com/openconfig/ygot/ygot" "github.com/sdcio/data-server/pkg/config" schemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/tree" jsonImporter "github.com/sdcio/data-server/pkg/tree/importer/json" "github.com/sdcio/data-server/pkg/tree/types" @@ -1320,12 +1322,22 @@ func TestDatastore_populateTree(t *testing.T) { t.Error(err) } - marksOwnerDeleteVisitor := tree.NewMarkOwnerDeleteVisitor(tt.intentName, false) - err = root.Walk(ctx, marksOwnerDeleteVisitor) + sharedTaskPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + deleteVisitorPool := sharedTaskPool.NewVirtualPool(pool.VirtualFailFast, 1) + ownerDeleteMarker := tree.NewOwnerDeleteMarker(tree.NewOwnerDeleteMarkerTaskConfig(tt.intentName, false)) + + err = ownerDeleteMarker.Run(root.GetRoot(), deleteVisitorPool) if err != nil { t.Error(err) + return } + // marksOwnerDeleteVisitor := tree.NewMarkOwnerDeleteVisitor(tt.intentName, false) + // err = root.Walk(ctx, marksOwnerDeleteVisitor) + // if err != nil { + // t.Error(err) + // } + newFlag := types.NewUpdateInsertFlags().SetNewFlag() err = root.ImportConfig(ctx, tt.intentReqPath, jsonImporter.NewJsonTreeImporter(jsonConfAny), tt.intentName, tt.intentPrio, newFlag) diff --git a/pkg/pool/virtual_pool.go b/pkg/pool/virtual_pool.go index da389942..3b249071 100644 --- a/pkg/pool/virtual_pool.go +++ b/pkg/pool/virtual_pool.go @@ -16,8 +16,6 @@ package pool import ( "context" - "crypto/rand" - "encoding/hex" "errors" "sync" "sync/atomic" @@ -108,8 +106,7 @@ const ( type SharedTaskPool struct { inner *Pool[Task] - mu sync.RWMutex - vmap map[string]*VirtualPool + mu sync.RWMutex } // NewSharedTaskPool constructs a shared pool; caller should call Start() to begin workers. @@ -124,7 +121,6 @@ func NewSharedTaskPool(parent context.Context, workerCount int) *SharedTaskPool return &SharedTaskPool{ inner: inner, - vmap: make(map[string]*VirtualPool), } } @@ -141,32 +137,12 @@ func (s *SharedTaskPool) Wait() error { // NewVirtualPool creates and registers a virtual pool on top of the shared pool. // id is an arbitrary identifier (must be unique per SharedTaskPool). // mode controls failure semantics. buf controls error channel buffer for tolerant mode. -func (s *SharedTaskPool) NewVirtualPool(id string, mode VirtualMode, buf int) *VirtualPool { +func (s *SharedTaskPool) NewVirtualPool(mode VirtualMode, buf int) *VirtualPool { // ensure unique id in the shared pool's map. If the requested id is already // registered, append a short random hex postfix so multiple callers can // create virtual pools with the same base name without colliding. - s.mu.Lock() - finalID := id - if _, exists := s.vmap[finalID]; exists { - // generate short random postfix until unique - for { - // 4 bytes -> 8 hex chars - b := make([]byte, 4) - if _, err := rand.Read(b); err != nil { - // fallback to using a timestamp-like postfix if crypto fails - finalID = finalID + "-r" - break - } - suf := hex.EncodeToString(b) - candidate := id + "-" + suf - if _, ok := s.vmap[candidate]; !ok { - finalID = candidate - break - } - } - } + v := &VirtualPool{ - id: finalID, parent: s, mode: mode, ec: nil, @@ -177,18 +153,9 @@ func (s *SharedTaskPool) NewVirtualPool(id string, mode VirtualMode, buf int) *V if mode == VirtualTolerant { v.ec = newErrorCollector(buf) } - s.vmap[v.id] = v - s.mu.Unlock() return v } -// UnregisterVirtualPool removes the virtual pool registration (does not wait or close inner). -func (s *SharedTaskPool) UnregisterVirtualPool(id string) { - s.mu.Lock() - delete(s.vmap, id) - s.mu.Unlock() -} - // submitWrapped submits a virtualTask into the shared pool. func (s *SharedTaskPool) submitWrapped(vt *virtualTask) error { return s.inner.Submit(vt) @@ -199,7 +166,6 @@ func (s *SharedTaskPool) submitWrapped(vt *virtualTask) error { // VirtualPool represents a logical pool view that reuses shared workers. // It enforces per-virtual behaviour like fail-fast or tolerant error collection. type VirtualPool struct { - id string parent *SharedTaskPool mode VirtualMode diff --git a/pkg/pool/virtual_pool_iface.go b/pkg/pool/virtual_pool_iface.go new file mode 100644 index 00000000..ea2095be --- /dev/null +++ b/pkg/pool/virtual_pool_iface.go @@ -0,0 +1,24 @@ +package pool + +// VirtualPoolI defines the public behaviour used by callers of VirtualPool. +// It intentionally exposes only the stable, public methods used by consumers +// (submission, lifecycle control and error inspection). +type VirtualPoolI interface { + // Submit enqueues a Task into this virtual pool. + Submit(Task) error + // SubmitFunc convenience to submit a TaskFunc. + SubmitFunc(TaskFunc) error + // CloseForSubmit marks this virtual pool as no longer accepting top-level submissions. + CloseForSubmit() + // Wait blocks until the virtual has been closed for submit and all inflight tasks have completed. + Wait() + // FirstError returns the first encountered error for fail-fast virtual pools, or nil. + FirstError() error + // Errors returns a snapshot of collected errors for tolerant virtual pools. + Errors() []error + // ErrorChan returns the live channel of errors for tolerant mode, or nil for fail-fast mode. + ErrorChan() <-chan error +} + +// Ensure VirtualPool implements the interface. +var _ VirtualPoolI = (*VirtualPool)(nil) diff --git a/pkg/pool/virtual_pool_test.go b/pkg/pool/virtual_pool_test.go index ff5bf03a..de0081d6 100644 --- a/pkg/pool/virtual_pool_test.go +++ b/pkg/pool/virtual_pool_test.go @@ -50,8 +50,8 @@ func TestVirtualPools_TolerantAndFailFast(t *testing.T) { sp := NewSharedTaskPool(ctx, 4) // create virtual pools - vt := sp.NewVirtualPool("tolerant", VirtualTolerant, 16) - vf := sp.NewVirtualPool("fail", VirtualFailFast, 0) + vt := sp.NewVirtualPool(VirtualTolerant, 16) + vf := sp.NewVirtualPool(VirtualFailFast, 0) // submit tasks: tolerant pool will collect errors, fail pool will stop after first error var cntT int64 @@ -134,7 +134,7 @@ func TestVirtualPools_TolerantAndFailFast(t *testing.T) { func TestVirtualPool_Wait_Tolerant(t *testing.T) { ctx := context.Background() sp := NewSharedTaskPool(ctx, 2) - vt := sp.NewVirtualPool("wait-tolerant", VirtualTolerant, 4) + vt := sp.NewVirtualPool(VirtualTolerant, 4) var cnt int64 // submit a few tasks @@ -175,7 +175,7 @@ func TestVirtualPool_Wait_Tolerant(t *testing.T) { func TestVirtualPool_Wait_FailFast(t *testing.T) { ctx := context.Background() sp := NewSharedTaskPool(ctx, 2) - vf := sp.NewVirtualPool("wait-fail", VirtualFailFast, 0) + vf := sp.NewVirtualPool(VirtualFailFast, 0) var cnt int64 // submit a task that will fail and another that would be skipped when run diff --git a/pkg/tree/entry_test.go b/pkg/tree/entry_test.go index fe163c8a..99a7b400 100644 --- a/pkg/tree/entry_test.go +++ b/pkg/tree/entry_test.go @@ -3,12 +3,15 @@ package tree import ( "context" "reflect" + "runtime" "slices" "strings" "testing" "github.com/google/go-cmp/cmp" "github.com/sdcio/data-server/pkg/config" + "github.com/sdcio/data-server/pkg/pool" + "github.com/sdcio/data-server/pkg/tree/types" "github.com/sdcio/data-server/pkg/utils/testhelper" sdcpb "github.com/sdcio/sdc-protos/sdcpb" @@ -519,10 +522,14 @@ func Test_Entry_Three(t *testing.T) { // indicate that the intent is receiving an update // therefor invalidate all the present entries of the owner / intent - marksOwnerDeleteVisitor := NewMarkOwnerDeleteVisitor(owner1, false) - err = root.Walk(ctx, marksOwnerDeleteVisitor) + sharedTaskPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + deleteVisitorPool := sharedTaskPool.NewVirtualPool(pool.VirtualFailFast, 1) + ownerDeleteMarker := NewOwnerDeleteMarker(NewOwnerDeleteMarkerTaskConfig(owner1, false)) + + err = ownerDeleteMarker.Run(root.GetRoot(), deleteVisitorPool) if err != nil { t.Error(err) + return } // add incomming set intent reques data @@ -762,10 +769,14 @@ func Test_Entry_Four(t *testing.T) { // indicate that the intent is receiving an update // therefor invalidate all the present entries of the owner / intent - marksOwnerDeleteVisitor := NewMarkOwnerDeleteVisitor(owner1, false) - err = root.Walk(ctx, marksOwnerDeleteVisitor) + sharedTaskPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + deleteVisitorPool := sharedTaskPool.NewVirtualPool(pool.VirtualFailFast, 1) + ownerDeleteMarker := NewOwnerDeleteMarker(NewOwnerDeleteMarkerTaskConfig(owner1, false)) + + err = ownerDeleteMarker.Run(root.GetRoot(), deleteVisitorPool) if err != nil { t.Error(err) + return } // add incomming set intent reques data @@ -1104,8 +1115,11 @@ func Test_Entry_Delete_Aggregation(t *testing.T) { } } - marksOwnerDeleteVisitor := NewMarkOwnerDeleteVisitor(owner1, false) - err = root.Walk(ctx, marksOwnerDeleteVisitor) + sharedTaskPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + deleteVisitorPool := sharedTaskPool.NewVirtualPool(pool.VirtualFailFast, 1) + ownerDeleteMarker := NewOwnerDeleteMarker(NewOwnerDeleteMarkerTaskConfig(owner1, false)) + + err = ownerDeleteMarker.Run(root.GetRoot(), deleteVisitorPool) if err != nil { t.Error(err) return diff --git a/pkg/tree/visitor_blame_config.go b/pkg/tree/processor_blame_config.go similarity index 67% rename from pkg/tree/visitor_blame_config.go rename to pkg/tree/processor_blame_config.go index 1a30a40d..90063420 100644 --- a/pkg/tree/visitor_blame_config.go +++ b/pkg/tree/processor_blame_config.go @@ -9,7 +9,27 @@ import ( "google.golang.org/protobuf/proto" ) -func BlameConfig(ctx context.Context, e Entry, includeDefaults bool, pool pool.VirtualPoolI) (*sdcpb.BlameTreeElement, error) { +type BlameConfigProcessor struct { + config *BlameConfigProcessorConfig +} + +func NewBlameConfigProcessor(config *BlameConfigProcessorConfig) *BlameConfigProcessor { + return &BlameConfigProcessor{ + config: config, + } +} + +type BlameConfigProcessorConfig struct { + includeDefaults bool +} + +func NewBlameConfigProcessorConfig(includeDefaults bool) *BlameConfigProcessorConfig { + return &BlameConfigProcessorConfig{ + includeDefaults: includeDefaults, + } +} + +func (p *BlameConfigProcessor) Run(ctx context.Context, e Entry, pool pool.VirtualPoolI) (*sdcpb.BlameTreeElement, error) { dropChan := make(chan *DropBlameChild, 10) wg := &sync.WaitGroup{} @@ -22,7 +42,7 @@ func BlameConfig(ctx context.Context, e Entry, includeDefaults bool, pool pool.V wg.Done() }(dropChan) - blameTask := NewBlameConfigTask(e, dropChan, includeDefaults) + blameTask := NewBlameConfigTask(e, dropChan, p.config) err := pool.Submit(blameTask) if err != nil { return nil, err @@ -39,24 +59,28 @@ func BlameConfig(ctx context.Context, e Entry, includeDefaults bool, pool pool.V } type BlameConfigTask struct { - includeDefaults bool - parent *sdcpb.BlameTreeElement - self *sdcpb.BlameTreeElement - selfEntry Entry - dropChan chan<- *DropBlameChild + config *BlameConfigProcessorConfig + parent *sdcpb.BlameTreeElement + self *sdcpb.BlameTreeElement + selfEntry Entry + dropChan chan<- *DropBlameChild } -func NewBlameConfigTask(e Entry, dropChan chan<- *DropBlameChild, includeDefaults bool) *BlameConfigTask { +func NewBlameConfigTask(e Entry, dropChan chan<- *DropBlameChild, c *BlameConfigProcessorConfig) *BlameConfigTask { return &BlameConfigTask{ - includeDefaults: includeDefaults, - parent: nil, - self: &sdcpb.BlameTreeElement{}, - selfEntry: e, - dropChan: dropChan, + config: c, + parent: nil, + self: &sdcpb.BlameTreeElement{}, + selfEntry: e, + dropChan: dropChan, } } func (t *BlameConfigTask) Run(ctx context.Context, submit func(pool.Task) error) error { + if ctx.Err() != nil { + return ctx.Err() + } + t.self.Name = t.selfEntry.PathName() if t.selfEntry.IsRoot() { t.self.Name = "root" @@ -65,7 +89,7 @@ func (t *BlameConfigTask) Run(ctx context.Context, submit func(pool.Task) error) // process Value highestLe := t.selfEntry.GetLeafVariantEntries().GetHighestPrecedence(false, true, true) if highestLe != nil { - if highestLe.Update.Owner() != DefaultsIntentName || t.includeDefaults { + if highestLe.Update.Owner() != DefaultsIntentName || t.config.includeDefaults { t.self.SetValue(highestLe.Update.Value()).SetOwner(highestLe.Update.Owner()) // check if running equals the expected @@ -87,11 +111,11 @@ func (t *BlameConfigTask) Run(ctx context.Context, submit func(pool.Task) error) // create a new task for each child task := &BlameConfigTask{ - includeDefaults: t.includeDefaults, - parent: t.self, - self: child, - selfEntry: childEntry, - dropChan: t.dropChan, + config: t.config, + parent: t.self, + self: child, + selfEntry: childEntry, + dropChan: t.dropChan, } // submit the task if err := submit(task); err != nil { diff --git a/pkg/tree/visitor_blame_config_test.go b/pkg/tree/processor_blame_config_test.go similarity index 97% rename from pkg/tree/visitor_blame_config_test.go rename to pkg/tree/processor_blame_config_test.go index 4b937904..ea7ca80c 100644 --- a/pkg/tree/visitor_blame_config_test.go +++ b/pkg/tree/processor_blame_config_test.go @@ -2,6 +2,7 @@ package tree import ( "context" + "runtime" "testing" "github.com/google/go-cmp/cmp" @@ -151,11 +152,11 @@ func Test_sharedEntryAttributes_BlameConfig(t *testing.T) { t.Run(tt.name, func(t *testing.T) { treeRoot := tt.r(t) - sharedPool := pool.NewSharedTaskPool(ctx, 10) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + vPool := sharedPool.NewVirtualPool(pool.VirtualFailFast, 10) - vPool := sharedPool.NewVirtualPool("blame", pool.VirtualFailFast, 1) - - got, err := BlameConfig(ctx, treeRoot.sharedEntryAttributes, tt.includeDefaults, vPool) + bp := NewBlameConfigProcessor(NewBlameConfigProcessorConfig(tt.includeDefaults)) + got, err := bp.Run(ctx, treeRoot.sharedEntryAttributes, vPool) if err != nil { t.Errorf("BlameConfig() error %s", err) return diff --git a/pkg/tree/processor_mark_owner_delete.go b/pkg/tree/processor_mark_owner_delete.go new file mode 100644 index 00000000..1453d4e7 --- /dev/null +++ b/pkg/tree/processor_mark_owner_delete.go @@ -0,0 +1,116 @@ +package tree + +import ( + "context" + "sync" + + "github.com/sdcio/data-server/pkg/pool" +) + +type MarkOwnerDeleteProcessor struct { + config *OwnerDeleteMarkerTaskConfig + matches *Collector[*LeafEntry] +} + +func NewOwnerDeleteMarker(c *OwnerDeleteMarkerTaskConfig) *MarkOwnerDeleteProcessor { + return &MarkOwnerDeleteProcessor{ + config: c, + matches: NewCollector[*LeafEntry](20), + } +} + +func (o *MarkOwnerDeleteProcessor) Run(e Entry, pool pool.VirtualPoolI) error { + + err := pool.Submit(newOwnerDeleteMarkerTask(o.config, e, o.matches)) + if err != nil { + return err + } + // close pool for additional external submission + pool.CloseForSubmit() + // wait for the pool to run dry + pool.Wait() + + return nil + +} + +type OwnerDeleteMarkerTaskConfig struct { + owner string + onlyIntended bool +} + +func NewOwnerDeleteMarkerTaskConfig(owner string, onlyIntended bool) *OwnerDeleteMarkerTaskConfig { + return &OwnerDeleteMarkerTaskConfig{ + owner: owner, + onlyIntended: onlyIntended, + } +} + +type ownerDeleteMarkerTask struct { + config *OwnerDeleteMarkerTaskConfig + matches *Collector[*LeafEntry] + e Entry +} + +func newOwnerDeleteMarkerTask(c *OwnerDeleteMarkerTaskConfig, e Entry, matches *Collector[*LeafEntry]) *ownerDeleteMarkerTask { + return &ownerDeleteMarkerTask{ + config: c, + e: e, + matches: matches, + } +} + +func (x ownerDeleteMarkerTask) Run(ctx context.Context, submit func(pool.Task) error) error { + if ctx.Err() != nil { + return ctx.Err() + } + le := x.e.GetLeafVariantEntries().MarkOwnerForDeletion(x.config.owner, x.config.onlyIntended) + if le != nil { + x.matches.Append(le) + } + for _, c := range x.e.GetChilds(DescendMethodAll) { + submit(newOwnerDeleteMarkerTask(x.config, c, x.matches)) + } + return nil +} + +// Collector is a concurrent-safe, append-only collector for values of type T. +type Collector[T any] struct { + mu sync.Mutex + out []T +} + +// NewCollector creates a Collector with a preallocated capacity. +// Pass 0 if you don't want to preallocate. +func NewCollector[T any](cap int) *Collector[T] { + if cap < 0 { + cap = 0 + } + return &Collector[T]{out: make([]T, 0, cap)} +} + +// Append appends one element to the collector. +func (c *Collector[T]) Append(x T) { + c.mu.Lock() + c.out = append(c.out, x) + c.mu.Unlock() +} + +// AppendAll appends all elements from the provided slice. +// This is slightly more efficient than calling Append in a loop. +func (c *Collector[T]) AppendAll(xs []T) { + if len(xs) == 0 { + return + } + c.mu.Lock() + c.out = append(c.out, xs...) + c.mu.Unlock() +} + +// Len returns the current number of elements. +func (c *Collector[T]) Len() int { + c.mu.Lock() + n := len(c.out) + c.mu.Unlock() + return n +} diff --git a/pkg/tree/root_entry.go b/pkg/tree/root_entry.go index c3765242..71361aa4 100644 --- a/pkg/tree/root_entry.go +++ b/pkg/tree/root_entry.go @@ -243,6 +243,7 @@ func (r *RootEntry) DeleteBranchPaths(ctx context.Context, deletes types.DeleteE } func (r *RootEntry) FinishInsertionPhase(ctx context.Context) error { + edvs := ExplicitDeleteVisitors{} // apply the explicit deletes diff --git a/pkg/tree/sharedEntryAttributes_test.go b/pkg/tree/sharedEntryAttributes_test.go index b597c049..af8e7882 100644 --- a/pkg/tree/sharedEntryAttributes_test.go +++ b/pkg/tree/sharedEntryAttributes_test.go @@ -4,6 +4,7 @@ import ( "context" "encoding/json" "fmt" + "runtime" "slices" "strings" "sync" @@ -13,6 +14,7 @@ import ( "github.com/openconfig/ygot/ygot" "github.com/sdcio/data-server/pkg/config" schemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" + "github.com/sdcio/data-server/pkg/pool" jsonImporter "github.com/sdcio/data-server/pkg/tree/importer/json" "github.com/sdcio/data-server/pkg/tree/importer/proto" "github.com/sdcio/data-server/pkg/tree/types" @@ -1126,10 +1128,14 @@ func Test_sharedEntryAttributes_ReApply(t *testing.T) { } // mark owner delete - marksOwnerDeleteVisitor := NewMarkOwnerDeleteVisitor(owner1, false) - err = root.Walk(ctx, marksOwnerDeleteVisitor) + sharedTaskPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + deleteVisitorPool := sharedTaskPool.NewVirtualPool(pool.VirtualFailFast, 1) + ownerDeleteMarker := NewOwnerDeleteMarker(NewOwnerDeleteMarkerTaskConfig(owner1, false)) + + err = ownerDeleteMarker.Run(root.GetRoot(), deleteVisitorPool) if err != nil { t.Error(err) + return } err = newRoot.AddUpdatesRecursive(ctx, updSlice, flagsNew) diff --git a/pkg/tree/visitor_explicit_delete.go b/pkg/tree/visitor_explicit_delete.go index 4991b68c..1c1f5c83 100644 --- a/pkg/tree/visitor_explicit_delete.go +++ b/pkg/tree/visitor_explicit_delete.go @@ -62,3 +62,64 @@ func (e ExplicitDeleteVisitors) Stats() map[string]int { return v.GetExplicitDeleteCreationCount() }) } + +// type ExplicitDeleteTask struct { +// e Entry +// owner string +// priority int32 +// } + +// func NewExplicitDeleteTask(e Entry, owner string, priority int32) *ExplicitDeleteTask { +// return &ExplicitDeleteTask{ +// e: e, +// owner: owner, +// priority: priority, +// } +// } + +// func (edt *ExplicitDeleteTask) Run(ctx context.Context, submit func(pool.Task) error) error { + +// if !edt.e.HoldsLeafvariants() { +// return nil +// } +// le := edt.e.GetLeafVariantEntries().GetByOwner(edt.owner) +// if le != nil { +// le.MarkExpliciteDelete() +// } else { +// le = edt.e.GetLeafVariantEntries().AddExplicitDeleteEntry(edt.owner, edt.priority) +// } + +// for _, c := range edt.e.GetChilds(DescendMethodAll) { +// deleteTask := NewExplicitDeleteTask(c, edt.owner, edt.priority) +// err := submit(deleteTask) +// if err != nil { +// return err +// } +// } +// return nil + +// } + +// func ExplicitDelete(ctx context.Context, e Entry, dps *DeletePathSet, pool pool.VirtualPoolI) error { +// for dp := range dps.Items() { +// for pi := range dp.PathItems() { +// edp, err := e.NavigateSdcpbPath(ctx, pi) +// if err != nil { +// return err +// } + +// deleteTask := NewExplicitDeleteTask(edp, dp.GetOwner(), dp.GetPrio()) + +// err = pool.Submit(deleteTask) +// if err != nil { +// return err +// } +// } +// } + +// // close pool for additional external submission +// pool.CloseForSubmit() +// // wait for the pool to run dry +// pool.Wait() +// return nil +// } diff --git a/pkg/tree/visitor_mark_owner_delete.go b/pkg/tree/visitor_mark_owner_delete.go deleted file mode 100644 index ca0b5e2d..00000000 --- a/pkg/tree/visitor_mark_owner_delete.go +++ /dev/null @@ -1,45 +0,0 @@ -package tree - -import ( - "context" - "sync" -) - -type MarkOwnerDeleteVisitor struct { - BaseVisitor - owner string - onlyIntended bool - leafVariantsMatched LeafVariantSlice - lvMutex *sync.Mutex -} - -var _ EntryVisitor = (*MarkOwnerDeleteVisitor)(nil) - -func NewMarkOwnerDeleteVisitor(owner string, onlyIntended bool) *MarkOwnerDeleteVisitor { - return &MarkOwnerDeleteVisitor{ - owner: owner, - onlyIntended: onlyIntended, - leafVariantsMatched: LeafVariantSlice{}, - lvMutex: &sync.Mutex{}, - } -} - -func (o *MarkOwnerDeleteVisitor) Visit(ctx context.Context, e Entry) error { - le := e.GetLeafVariantEntries().MarkOwnerForDeletion(o.owner, o.onlyIntended) - if le != nil { - o.lvMutex.Lock() - o.leafVariantsMatched = append(o.leafVariantsMatched, le) - o.lvMutex.Unlock() - } - return nil -} - -// GetHitCount returns the number of entries marked for deletion -func (o *MarkOwnerDeleteVisitor) GetHitCount() int { - return len(o.leafVariantsMatched) -} - -// GetMatches return all the altered LeafVariants -func (o *MarkOwnerDeleteVisitor) GetMatches() LeafVariantSlice { - return o.leafVariantsMatched -} diff --git a/pkg/tree/xml_test.go b/pkg/tree/xml_test.go index 88fe3430..53a42e69 100644 --- a/pkg/tree/xml_test.go +++ b/pkg/tree/xml_test.go @@ -3,11 +3,13 @@ package tree import ( "context" "fmt" + "runtime" "slices" "testing" "github.com/google/go-cmp/cmp" "github.com/openconfig/ygot/ygot" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/utils" "github.com/sdcio/data-server/pkg/utils/testhelper" sdcio_schema "github.com/sdcio/data-server/tests/sdcioygot" @@ -488,10 +490,14 @@ func TestToXMLTable(t *testing.T) { fmt.Println(root.String()) if tt.newConfig != nil { - marksOwnerDeleteVisitor := NewMarkOwnerDeleteVisitor(owner, false) - err = root.Walk(ctx, marksOwnerDeleteVisitor) + sharedTaskPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + deleteVisitorPool := sharedTaskPool.NewVirtualPool(pool.VirtualFailFast, 1) + ownerDeleteMarker := NewOwnerDeleteMarker(NewOwnerDeleteMarkerTaskConfig(owner, false)) + + err = ownerDeleteMarker.Run(root.GetRoot(), deleteVisitorPool) if err != nil { t.Error(err) + return } newUpds, err := tt.newConfig(ctx, converter) From 23e2f9d625897811b76027dd8ff32017357ce946 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 5 Nov 2025 15:28:49 +0100 Subject: [PATCH 18/44] update --- pkg/datastore/datastore_rpc.go | 4 +- pkg/datastore/deviations.go | 16 +- pkg/datastore/sync.go | 5 +- pkg/datastore/target/gnmi/gnmi.go | 33 +- pkg/datastore/target/gnmi/once.go | 5 +- pkg/datastore/target/gnmi/stream.go | 312 ++++++++++-------- pkg/datastore/target/target.go | 5 +- pkg/datastore/transaction_rpc.go | 4 +- pkg/datastore/tree_operation_test.go | 3 +- .../tree_operation_validation_test.go | 5 +- pkg/pool/virtual_pool.go | 2 +- pkg/pool/virtual_pool_iface.go | 32 +- pkg/tree/entry.go | 2 +- pkg/tree/entry_test.go | 18 +- pkg/tree/processor_validate.go | 61 ++++ pkg/tree/root_entry.go | 13 +- pkg/tree/sharedEntryAttributes.go | 24 +- pkg/tree/sharedEntryAttributes_test.go | 17 +- pkg/tree/types/validation_stats.go | 13 +- pkg/tree/validation_range_test.go | 12 +- 20 files changed, 345 insertions(+), 241 deletions(-) create mode 100644 pkg/tree/processor_validate.go diff --git a/pkg/datastore/datastore_rpc.go b/pkg/datastore/datastore_rpc.go index 9944b9c8..53b7c4db 100644 --- a/pkg/datastore/datastore_rpc.go +++ b/pkg/datastore/datastore_rpc.go @@ -151,7 +151,7 @@ CREATE: func (d *Datastore) connectSBI(ctx context.Context, opts ...grpc.DialOption) error { var err error - d.sbi, err = target.New(ctx, d.config.Name, d.config.SBI, d.schemaClient, d, d.config.Sync.Config, opts...) + d.sbi, err = target.New(ctx, d.config.Name, d.config.SBI, d.schemaClient, d, d.config.Sync.Config, d.taskPool, opts...) if err == nil { return nil } @@ -165,7 +165,7 @@ func (d *Datastore) connectSBI(ctx context.Context, opts ...grpc.DialOption) err case <-ctx.Done(): return ctx.Err() case <-ticker.C: - d.sbi, err = target.New(ctx, d.config.Name, d.config.SBI, d.schemaClient, d, d.config.Sync.Config, opts...) + d.sbi, err = target.New(ctx, d.config.Name, d.config.SBI, d.schemaClient, d, d.config.Sync.Config, d.taskPool, opts...) if err != nil { log.Errorf("failed to create DS %s target: %v", d.config.Name, err) continue diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 3297c39c..70ad71f7 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -7,6 +7,7 @@ import ( "time" "github.com/sdcio/data-server/pkg/config" + "github.com/sdcio/data-server/pkg/pool" treetypes "github.com/sdcio/data-server/pkg/tree/types" sdcpb "github.com/sdcio/sdc-protos/sdcpb" log "github.com/sirupsen/logrus" @@ -91,17 +92,15 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) func (d *Datastore) SendDeviations(ch <-chan *treetypes.DeviationEntry, deviationClients map[string]sdcpb.DataServer_WatchDeviationsServer) { wg := &sync.WaitGroup{} + vPool := d.taskPool.NewVirtualPool(pool.VirtualTolerant, 1) + for de := range ch { wg.Add(1) - go func(de DeviationEntry, dcs map[string]sdcpb.DataServer_WatchDeviationsServer) { - for clientIdentifier, dc := range dcs { - select { - // skip deviation clients with closed context - case <-dc.Context().Done(): + vPool.SubmitFunc(func(ctx context.Context, _ func(pool.Task) error) error { + for clientIdentifier, dc := range deviationClients { + if dc.Context().Err() != nil { continue - default: } - err := dc.Send(&sdcpb.WatchDeviationResponse{ Name: d.config.Name, Intent: de.IntentName(), @@ -116,7 +115,8 @@ func (d *Datastore) SendDeviations(ch <-chan *treetypes.DeviationEntry, deviatio } } wg.Done() - }(de, deviationClients) + return nil + }) } wg.Wait() } diff --git a/pkg/datastore/sync.go b/pkg/datastore/sync.go index 5e911e79..a4e99c16 100644 --- a/pkg/datastore/sync.go +++ b/pkg/datastore/sync.go @@ -15,11 +15,10 @@ func (d *Datastore) ApplyToRunning(ctx context.Context, deletes []*sdcpb.Path, i d.syncTreeMutex.Lock() defer d.syncTreeMutex.Unlock() for _, delete := range deletes { - //TODO this will most likely give us errors in case optimisticWriteback already deleted the entries. err := d.syncTree.DeleteBranch(ctx, delete, tree.RunningIntentName) if err != nil { - log.Errorf("error deleting paths from datastore sync tree: %v", err) - return err + log.Warnf("failed deleting path (%s) from datastore sync tree: %v", delete.ToXPath(false), err) + continue } } diff --git a/pkg/datastore/target/gnmi/gnmi.go b/pkg/datastore/target/gnmi/gnmi.go index 8f5c74b3..d725d3c9 100644 --- a/pkg/datastore/target/gnmi/gnmi.go +++ b/pkg/datastore/target/gnmi/gnmi.go @@ -28,6 +28,7 @@ import ( "github.com/sdcio/data-server/pkg/config" gnmiutils "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" targetTypes "github.com/sdcio/data-server/pkg/datastore/target/types" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/utils" dsutils "github.com/sdcio/data-server/pkg/utils" sdcpb "github.com/sdcio/sdc-protos/sdcpb" @@ -45,15 +46,16 @@ const ( ) type gnmiTarget struct { - target *gtarget.Target - encodings map[gnmi.Encoding]struct{} - cfg *config.SBI - syncs map[string]GnmiSync - runningStore targetTypes.RunningStore - schemaClient dsutils.SchemaClientBound + target *gtarget.Target + encodings map[gnmi.Encoding]struct{} + cfg *config.SBI + syncs map[string]GnmiSync + runningStore targetTypes.RunningStore + schemaClient dsutils.SchemaClientBound + taskpoolFactory pool.VirtualPoolFactory } -func NewTarget(ctx context.Context, name string, cfg *config.SBI, runningStore targetTypes.RunningStore, schemaClient dsutils.SchemaClientBound, opts ...grpc.DialOption) (*gnmiTarget, error) { +func NewTarget(ctx context.Context, name string, cfg *config.SBI, runningStore targetTypes.RunningStore, schemaClient dsutils.SchemaClientBound, taskpoolFactory pool.VirtualPoolFactory, opts ...grpc.DialOption) (*gnmiTarget, error) { tc := &types.TargetConfig{ Name: name, Address: fmt.Sprintf("%s:%d", cfg.Address, cfg.Port), @@ -74,12 +76,13 @@ func NewTarget(ctx context.Context, name string, cfg *config.SBI, runningStore t tc.Insecure = pointer.ToBool(true) } gt := &gnmiTarget{ - target: gtarget.NewTarget(tc), - encodings: make(map[gnmi.Encoding]struct{}), - cfg: cfg, - syncs: map[string]GnmiSync{}, - runningStore: runningStore, - schemaClient: schemaClient, + target: gtarget.NewTarget(tc), + encodings: make(map[gnmi.Encoding]struct{}), + cfg: cfg, + syncs: map[string]GnmiSync{}, + runningStore: runningStore, + schemaClient: schemaClient, + taskpoolFactory: taskpoolFactory, } opts = append(opts, grpc.WithKeepaliveParams(keepalive.ClientParameters{ @@ -254,14 +257,14 @@ func (t *gnmiTarget) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) for _, sp := range sps { switch sp.Mode { case "once": - g = NewOnceSync(ctx, t, sp, t.runningStore) + g = NewOnceSync(ctx, t, sp, t.runningStore, t.taskpoolFactory) case "get": g, err = NewGetSync(ctx, t, sp, t.runningStore, t.schemaClient) if err != nil { return err } default: - g = NewStreamSync(ctx, t, sp, t.runningStore, t.schemaClient) + g = NewStreamSync(ctx, t, sp, t.runningStore, t.schemaClient, t.taskpoolFactory) } t.syncs[sp.Name] = g err := g.Start() diff --git a/pkg/datastore/target/gnmi/once.go b/pkg/datastore/target/gnmi/once.go index a73cc4fa..cc11ce32 100644 --- a/pkg/datastore/target/gnmi/once.go +++ b/pkg/datastore/target/gnmi/once.go @@ -9,6 +9,7 @@ import ( "github.com/sdcio/data-server/pkg/config" "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" "github.com/sdcio/data-server/pkg/datastore/target/types" + "github.com/sdcio/data-server/pkg/pool" ) type OnceSync struct { @@ -17,15 +18,17 @@ type OnceSync struct { cancel context.CancelFunc runningStore types.RunningStore ctx context.Context + vpoolFactory pool.VirtualPoolFactory } -func NewOnceSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore) *OnceSync { +func NewOnceSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore, vpoolFactory pool.VirtualPoolFactory) *OnceSync { ctx, cancel := context.WithCancel(ctx) return &OnceSync{ config: c, target: target, cancel: cancel, runningStore: runningStore, + vpoolFactory: vpoolFactory, } } diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index 59561c41..d095fcbf 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -3,7 +3,6 @@ package gnmi import ( "context" "errors" - "runtime" "sync" "time" @@ -28,9 +27,10 @@ type StreamSync struct { cancel context.CancelFunc runningStore types.RunningStore schemaClient dsutils.SchemaClientBound + vpoolFactory pool.VirtualPoolFactory } -func NewStreamSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore, schemaClient dsutils.SchemaClientBound) *StreamSync { +func NewStreamSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore, schemaClient dsutils.SchemaClientBound, vpoolFactory pool.VirtualPoolFactory) *StreamSync { ctx, cancel := context.WithCancel(ctx) return &StreamSync{ @@ -40,6 +40,7 @@ func NewStreamSync(ctx context.Context, target SyncTarget, c *config.SyncProtoco runningStore: runningStore, schemaClient: schemaClient, ctx: ctx, + vpoolFactory: vpoolFactory, } } @@ -79,176 +80,152 @@ func (s *StreamSync) Stop() error { func (s *StreamSync) Start() error { + updChan := make(chan *NotificationData, 20) + + syncResponse := make(chan struct{}) + subReq, err := s.syncConfig() if err != nil { return err } - log.Infof("sync %q: subRequest: %v", s.config.Name, subReq) - - respChan, errChan := s.target.Subscribe(s.ctx, subReq, s.config.Name) - - syncStartTime := time.Now() - - wpool := pool.NewWorkerPool[*gnmi.Notification](s.ctx, runtime.NumCPU()) - - updChan := make(chan *NotificationData, 20) - - wpoolHandler := func(ctx context.Context, item *gnmi.Notification, submit func(*gnmi.Notification) error) error { - sn := dsutils.ToSchemaNotification(item) - // updates - upds, err := treetypes.ExpandAndConvertIntent(s.ctx, s.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, sn.GetUpdate(), item.GetTimestamp()) - if err != nil { - log.Errorf("sync expanding error: %v", err) - } - - deletes := sdcpb.NewPathSet() - if len(item.GetDelete()) > 0 { - for _, del := range item.GetDelete() { - deletes.AddPath(dsutils.FromGNMIPath(item.GetPrefix(), del)) - } - } + // start the gnmi subscribe request, that also used the pool for + go s.gnmiSubscribe(subReq, updChan, syncResponse) + // + go s.buildTreeSyncWithDatastore(updChan, syncResponse) - updChan <- &NotificationData{ - updates: upds, - deletes: deletes, - } + return nil +} - return nil +func (s *StreamSync) buildTreeSyncWithDatastore(cUS <-chan *NotificationData, syncResponse <-chan struct{}) { + syncTree, err := s.runningStore.NewEmptyTree(s.ctx) + if err != nil { + log.Errorf("error creating new sync tree: %v", err) + return } + syncTreeMutex := &sync.Mutex{} - wpool.Start(wpoolHandler) + ticker := time.NewTicker(5 * time.Second) + defer ticker.Stop() + // disable ticker until after the initial full sync is done + tickerActive := false - syncToRunning := func(syncTree *tree.RootEntry, m *sync.Mutex, logCount bool) (*tree.RootEntry, error) { - m.Lock() - defer m.Unlock() + uif := treetypes.NewUpdateInsertFlags() - startTime := time.Now() - result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) - if err != nil { - if errors.Is(err, tree.ErrorIntentNotPresent) { - log.Info("sync no config changes") - // all good no data present - return syncTree, nil + for { + select { + case <-s.ctx.Done(): + return + case noti, ok := <-cUS: + if !ok { + return + } + err := syncTree.AddUpdatesRecursive(s.ctx, noti.updates, uif) + if err != nil { + log.Errorf("error adding to sync tree: %v", err) + } + syncTree.AddExplicitDeletes(tree.RunningIntentName, tree.RunningValuesPrio, noti.deletes) + case <-syncResponse: + syncTree, err = s.syncToRunning(syncTree, syncTreeMutex, true) + tickerActive = true + if err != nil { + // TODO + log.Errorf("syncToRunning Error %v", err) + } + case <-ticker.C: + if !tickerActive { + log.Info("Skipping a sync tick - initial sync not finished yet") + continue + } + log.Info("SyncRunning due to ticker") + syncTree, err = s.syncToRunning(syncTree, syncTreeMutex, true) + if err != nil { + // TODO + log.Errorf("syncToRunning Error %v", err) } - log.Errorf("sync tree export error: %v", err) - return s.runningStore.NewEmptyTree(s.ctx) - } - // extract the explicit deletes - deletes := result.ExplicitDeletes - // set them to nil - result.ExplicitDeletes = nil - if logCount { - log.Infof("Syncing: %d elements, %d deletes ", result.GetRoot().CountTerminals(), len(result.GetExplicitDeletes())) } + } +} - log.Infof("TreeExport to proto took: %s", time.Since(startTime)) - startTime = time.Now() +func (s *StreamSync) gnmiSubscribe(subReq *gnmi.SubscribeRequest, updChan chan<- *NotificationData, syncResponse chan<- struct{}) { + var err error + log.Infof("sync %q: subRequest: %v", s.config.Name, subReq) - err = s.runningStore.ApplyToRunning(s.ctx, deletes, proto.NewProtoTreeImporter(result)) - if err != nil { - log.Errorf("sync import to running error: %v", err) - return s.runningStore.NewEmptyTree(s.ctx) - } - log.Infof("Import to SyncTree took: %s", time.Since(startTime)) - return s.runningStore.NewEmptyTree(s.ctx) - } + respChan, errChan := s.target.Subscribe(s.ctx, subReq, s.config.Name) - syncResponse := make(chan struct{}) + taskPool := s.vpoolFactory.NewVirtualPool(pool.VirtualTolerant, 10) + defer taskPool.CloseForSubmit() + taskParams := NewNotificationProcessorTaskParameters(updChan, s.schemaClient) - go func() { - if err != nil { - log.Errorf("sync newemptytree error: %v", err) + syncStartTime := time.Now() + for { + select { + case <-s.ctx.Done(): return - } - defer wpool.CloseForSubmit() - for { - select { - case <-s.ctx.Done(): + case err, ok := <-errChan: + if !ok { return - case err, ok := <-errChan: - if !ok { - return - } + } + if err != nil { + log.Errorf("Error stream sync: %s", err) + return + } + case resp, ok := <-respChan: + if !ok { + return + } + switch r := resp.GetResponse().(type) { + case *gnmi.SubscribeResponse_Update: + err = taskPool.Submit(newNotificationProcessorTask(resp.GetUpdate(), taskParams)) if err != nil { - log.Errorf("Error stream sync: %s", err) - return - } - case resp, ok := <-respChan: - if !ok { - return - } - switch r := resp.GetResponse().(type) { - case *gnmi.SubscribeResponse_Update: - err := wpool.Submit(resp.GetUpdate()) - if err != nil { - log.Errorf("error processing Notifications: %s", err) - continue - } - case *gnmi.SubscribeResponse_SyncResponse: - log.Info("SyncResponse flag received") - log.Infof("Duration since sync Start: %s", time.Since(syncStartTime)) - syncResponse <- struct{}{} - - case *gnmi.SubscribeResponse_Error: - log.Error(r.Error.Message) + log.Errorf("error processing Notifications: %s", err) + continue } + case *gnmi.SubscribeResponse_SyncResponse: + log.Info("SyncResponse flag received") + log.Infof("Duration since sync Start: %s", time.Since(syncStartTime)) + syncResponse <- struct{}{} + + case *gnmi.SubscribeResponse_Error: + log.Error(r.Error.Message) } } - }() + } +} - go func(cUS <-chan *NotificationData, syncResponse <-chan struct{}) { +func (s *StreamSync) syncToRunning(syncTree *tree.RootEntry, m *sync.Mutex, logCount bool) (*tree.RootEntry, error) { + m.Lock() + defer m.Unlock() - syncTree, err := s.runningStore.NewEmptyTree(s.ctx) - if err != nil { - log.Errorf("error creating new sync tree: %v", err) - return + startTime := time.Now() + result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) + if err != nil { + if errors.Is(err, tree.ErrorIntentNotPresent) { + log.Info("sync no config changes") + // all good no data present + return syncTree, nil } - syncTreeMutex := &sync.Mutex{} - - ticker := time.NewTicker(5 * time.Second) - defer ticker.Stop() - // disable ticker until after the initial full sync is done - tickerActive := false - - uif := treetypes.NewUpdateInsertFlags() + log.Errorf("sync tree export error: %v", err) + return s.runningStore.NewEmptyTree(s.ctx) + } + // extract the explicit deletes + deletes := result.ExplicitDeletes + // set them to nil + result.ExplicitDeletes = nil + if logCount { + log.Infof("Syncing: %d elements, %d deletes ", result.GetRoot().CountTerminals(), len(result.GetExplicitDeletes())) + } - for { - select { - case <-s.ctx.Done(): - return - case noti, ok := <-cUS: - if !ok { - return - } - err := syncTree.AddUpdatesRecursive(s.ctx, noti.updates, uif) - if err != nil { - log.Errorf("error adding to sync tree: %v", err) - } - syncTree.AddExplicitDeletes(tree.RunningIntentName, tree.RunningValuesPrio, noti.deletes) - case <-syncResponse: - syncTree, err = syncToRunning(syncTree, syncTreeMutex, true) - if err != nil { - // TODO - log.Errorf("syncToRunning Error %v", err) - } - tickerActive = true - case <-ticker.C: - if !tickerActive { - log.Info("Skipping a sync tick - initial sync not finished yet") - continue - } - log.Info("SyncRunning due to ticker") - syncTree, err = syncToRunning(syncTree, syncTreeMutex, true) - if err != nil { - // TODO - log.Errorf("syncToRunning Error %v", err) - } - } - } - }(updChan, syncResponse) + log.Infof("TreeExport to proto took: %s", time.Since(startTime)) + startTime = time.Now() - return nil + err = s.runningStore.ApplyToRunning(s.ctx, deletes, proto.NewProtoTreeImporter(result)) + if err != nil { + log.Errorf("sync import to running error: %v", err) + return s.runningStore.NewEmptyTree(s.ctx) + } + log.Infof("Import to SyncTree took: %s", time.Since(startTime)) + return s.runningStore.NewEmptyTree(s.ctx) } type SyncTarget interface { @@ -259,3 +236,50 @@ type NotificationData struct { updates treetypes.UpdateSlice deletes *sdcpb.PathSet } + +type notificationProcessorTask struct { + item *gnmi.Notification + params *NotificationProcessorTaskParameters +} + +type NotificationProcessorTaskParameters struct { + notificationResult chan<- *NotificationData + schemaClientBound dsutils.SchemaClientBound +} + +func NewNotificationProcessorTaskParameters(notificationResult chan<- *NotificationData, scb dsutils.SchemaClientBound) *NotificationProcessorTaskParameters { + return &NotificationProcessorTaskParameters{ + notificationResult: notificationResult, + schemaClientBound: scb, + } +} + +func newNotificationProcessorTask(item *gnmi.Notification, params *NotificationProcessorTaskParameters) *notificationProcessorTask { + return ¬ificationProcessorTask{ + item: item, + params: params, + } +} + +func (t *notificationProcessorTask) Run(ctx context.Context, _ func(pool.Task) error) error { + sn := dsutils.ToSchemaNotification(t.item) + // updates + upds, err := treetypes.ExpandAndConvertIntent(ctx, t.params.schemaClientBound, tree.RunningIntentName, tree.RunningValuesPrio, sn.GetUpdate(), t.item.GetTimestamp()) + if err != nil { + log.Errorf("sync expanding error: %v", err) + } + + deletes := sdcpb.NewPathSet() + if len(t.item.GetDelete()) > 0 { + for _, del := range t.item.GetDelete() { + deletes.AddPath(dsutils.FromGNMIPath(t.item.GetPrefix(), del)) + } + } + + t.params.notificationResult <- &NotificationData{ + updates: upds, + deletes: deletes, + } + + return nil +} diff --git a/pkg/datastore/target/target.go b/pkg/datastore/target/target.go index e1c76962..ee400516 100644 --- a/pkg/datastore/target/target.go +++ b/pkg/datastore/target/target.go @@ -25,6 +25,7 @@ import ( schemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" "github.com/sdcio/data-server/pkg/datastore/target/gnmi" "github.com/sdcio/data-server/pkg/datastore/target/types" + "github.com/sdcio/data-server/pkg/pool" ) const ( @@ -41,7 +42,7 @@ type Target interface { Close() error } -func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaClient.SchemaClientBound, runningStore types.RunningStore, syncConfigs []*config.SyncProtocol, opts ...grpc.DialOption) (Target, error) { +func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaClient.SchemaClientBound, runningStore types.RunningStore, syncConfigs []*config.SyncProtocol, taskpoolFactory pool.VirtualPoolFactory, opts ...grpc.DialOption) (Target, error) { var t Target var err error @@ -49,7 +50,7 @@ func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaC switch cfg.Type { case targetTypeGNMI: - t, err = gnmi.NewTarget(targetContext, name, cfg, runningStore, schemaClient, opts...) + t, err = gnmi.NewTarget(targetContext, name, cfg, runningStore, schemaClient, taskpoolFactory, opts...) if err != nil { return nil, err } diff --git a/pkg/datastore/transaction_rpc.go b/pkg/datastore/transaction_rpc.go index 60be2a71..f26c660c 100644 --- a/pkg/datastore/transaction_rpc.go +++ b/pkg/datastore/transaction_rpc.go @@ -105,7 +105,7 @@ func (d *Datastore) replaceIntent(ctx context.Context, transaction *types.Transa log.TraceFn(func() []interface{} { return []interface{}{root.String()} }) // perform validation - validationResult, validationStats := root.Validate(ctx, d.config.Validation) + validationResult, validationStats := root.Validate(ctx, d.config.Validation, d.taskPool) validationResult.ErrorsStr() if validationResult.HasErrors() { return nil, validationResult.JoinErrors() @@ -264,7 +264,7 @@ func (d *Datastore) lowlevelTransactionSet(ctx context.Context, transaction *typ log.Debug(root.String()) // perform validation - validationResult, validationStats := root.Validate(ctx, d.config.Validation) + validationResult, validationStats := root.Validate(ctx, d.config.Validation, d.taskPool) log.Debugf("Transaction: %s - Validation Stats: %s", transaction.GetTransactionId(), validationStats.String()) diff --git a/pkg/datastore/tree_operation_test.go b/pkg/datastore/tree_operation_test.go index d33b16c0..b238d0e6 100644 --- a/pkg/datastore/tree_operation_test.go +++ b/pkg/datastore/tree_operation_test.go @@ -1352,7 +1352,8 @@ func TestDatastore_populateTree(t *testing.T) { } fmt.Println(root.String()) - validationResult, _ := root.Validate(ctx, validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(ctx, validationConfig, sharedPool) fmt.Printf("Validation Errors:\n%v\n", strings.Join(validationResult.ErrorsStr(), "\n")) fmt.Printf("Tree:%s\n", root.String()) diff --git a/pkg/datastore/tree_operation_validation_test.go b/pkg/datastore/tree_operation_validation_test.go index b8b2140c..ed93d911 100644 --- a/pkg/datastore/tree_operation_validation_test.go +++ b/pkg/datastore/tree_operation_validation_test.go @@ -18,12 +18,14 @@ import ( "context" "encoding/json" "fmt" + "runtime" "slices" "testing" "github.com/openconfig/ygot/ygot" "github.com/sdcio/data-server/pkg/cache" schemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/tree" json_importer "github.com/sdcio/data-server/pkg/tree/importer/json" "github.com/sdcio/data-server/pkg/tree/types" @@ -205,7 +207,8 @@ func TestDatastore_validateTree(t *testing.T) { t.Error(err) } - validationResult, _ := root.Validate(ctx, validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(ctx, validationConfig, sharedPool) t.Log(root.String()) diff --git a/pkg/pool/virtual_pool.go b/pkg/pool/virtual_pool.go index 3b249071..822f5286 100644 --- a/pkg/pool/virtual_pool.go +++ b/pkg/pool/virtual_pool.go @@ -137,7 +137,7 @@ func (s *SharedTaskPool) Wait() error { // NewVirtualPool creates and registers a virtual pool on top of the shared pool. // id is an arbitrary identifier (must be unique per SharedTaskPool). // mode controls failure semantics. buf controls error channel buffer for tolerant mode. -func (s *SharedTaskPool) NewVirtualPool(mode VirtualMode, buf int) *VirtualPool { +func (s *SharedTaskPool) NewVirtualPool(mode VirtualMode, buf int) VirtualPoolI { // ensure unique id in the shared pool's map. If the requested id is already // registered, append a short random hex postfix so multiple callers can // create virtual pools with the same base name without colliding. diff --git a/pkg/pool/virtual_pool_iface.go b/pkg/pool/virtual_pool_iface.go index ea2095be..396a34c2 100644 --- a/pkg/pool/virtual_pool_iface.go +++ b/pkg/pool/virtual_pool_iface.go @@ -4,21 +4,25 @@ package pool // It intentionally exposes only the stable, public methods used by consumers // (submission, lifecycle control and error inspection). type VirtualPoolI interface { - // Submit enqueues a Task into this virtual pool. - Submit(Task) error - // SubmitFunc convenience to submit a TaskFunc. - SubmitFunc(TaskFunc) error - // CloseForSubmit marks this virtual pool as no longer accepting top-level submissions. - CloseForSubmit() - // Wait blocks until the virtual has been closed for submit and all inflight tasks have completed. - Wait() - // FirstError returns the first encountered error for fail-fast virtual pools, or nil. - FirstError() error - // Errors returns a snapshot of collected errors for tolerant virtual pools. - Errors() []error - // ErrorChan returns the live channel of errors for tolerant mode, or nil for fail-fast mode. - ErrorChan() <-chan error + // Submit enqueues a Task into this virtual pool. + Submit(Task) error + // SubmitFunc convenience to submit a TaskFunc. + SubmitFunc(TaskFunc) error + // CloseForSubmit marks this virtual pool as no longer accepting top-level submissions. + CloseForSubmit() + // Wait blocks until the virtual has been closed for submit and all inflight tasks have completed. + Wait() + // FirstError returns the first encountered error for fail-fast virtual pools, or nil. + FirstError() error + // Errors returns a snapshot of collected errors for tolerant virtual pools. + Errors() []error + // ErrorChan returns the live channel of errors for tolerant mode, or nil for fail-fast mode. + ErrorChan() <-chan error } // Ensure VirtualPool implements the interface. var _ VirtualPoolI = (*VirtualPool)(nil) + +type VirtualPoolFactory interface { + NewVirtualPool(mode VirtualMode, buf int) VirtualPoolI +} diff --git a/pkg/tree/entry.go b/pkg/tree/entry.go index 47339774..922fb8d7 100644 --- a/pkg/tree/entry.go +++ b/pkg/tree/entry.go @@ -68,7 +68,7 @@ type Entry interface { // Walk takes the EntryVisitor and applies it to every Entry in the tree Walk(ctx context.Context, v EntryVisitor) error // Validate kicks off validation - Validate(ctx context.Context, resultChan chan<- *types.ValidationResultEntry, stats *types.ValidationStats, vCfg *config.Validation) + ValidateLevel(ctx context.Context, resultChan chan<- *types.ValidationResultEntry, stats *types.ValidationStats, vCfg *config.Validation) // validateMandatory the Mandatory schema field validateMandatory(ctx context.Context, resultChan chan<- *types.ValidationResultEntry, stats *types.ValidationStats) // validateMandatoryWithKeys is an internally used function that us called by validateMandatory in case diff --git a/pkg/tree/entry_test.go b/pkg/tree/entry_test.go index 99a7b400..3c5a56d7 100644 --- a/pkg/tree/entry_test.go +++ b/pkg/tree/entry_test.go @@ -907,7 +907,8 @@ func Test_Validation_Leaflist_Min_Max(t *testing.T) { t.Log(root.String()) - validationResult, _ := root.Validate(context.TODO(), validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(context.TODO(), validationConfig, sharedPool) // check if errors are received // If so, join them and return the cumulated errors @@ -955,7 +956,8 @@ func Test_Validation_Leaflist_Min_Max(t *testing.T) { } } - validationResult, _ := root.Validate(context.TODO(), validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(context.TODO(), validationConfig, sharedPool) // check if errors are received // If so, join them and return the cumulated errors @@ -1009,7 +1011,8 @@ func Test_Validation_Leaflist_Min_Max(t *testing.T) { } } - validationResult, _ := root.Validate(context.TODO(), validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(context.TODO(), validationConfig, sharedPool) // check if errors are received // If so, join them and return the cumulated errors @@ -1581,7 +1584,8 @@ func Test_Validation_String_Pattern(t *testing.T) { t.Error(err) } - validationResult, _ := root.Validate(context.TODO(), validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(context.TODO(), validationConfig, sharedPool) // check if errors are received // If so, join them and return the cumulated errors @@ -1618,7 +1622,8 @@ func Test_Validation_String_Pattern(t *testing.T) { t.Error(err) } - validationResult, _ := root.Validate(context.TODO(), validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(context.TODO(), validationConfig, sharedPool) // check if errors are received // If so, join them and return the cumulated errors @@ -1714,7 +1719,8 @@ func Test_Validation_Deref(t *testing.T) { t.Error(err) } - validationResult, _ := root.Validate(context.TODO(), validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(context.TODO(), validationConfig, sharedPool) // check if errors are received // If so, join them and return the cumulated errors diff --git a/pkg/tree/processor_validate.go b/pkg/tree/processor_validate.go new file mode 100644 index 00000000..0fa2067e --- /dev/null +++ b/pkg/tree/processor_validate.go @@ -0,0 +1,61 @@ +package tree + +import ( + "context" + + "github.com/sdcio/data-server/pkg/config" + "github.com/sdcio/data-server/pkg/pool" + "github.com/sdcio/data-server/pkg/tree/types" +) + +type ValidateProcessor struct { + parameters *ValidateProcessorParameters +} + +func NewValidateProcessor(parameters *ValidateProcessorParameters) *ValidateProcessor { + return &ValidateProcessor{ + parameters: parameters, + } +} + +func (p *ValidateProcessor) Run(taskpoolFactory pool.VirtualPoolFactory, e Entry) { + taskpool := taskpoolFactory.NewVirtualPool(pool.VirtualTolerant, 0) + taskpool.Submit(newValidateTask(e, p.parameters)) +} + +type ValidateProcessorParameters struct { + resultChan chan<- *types.ValidationResultEntry + stats *types.ValidationStats + vCfg *config.Validation +} + +func NewValidateProcessorConfig(resultChan chan<- *types.ValidationResultEntry, stats *types.ValidationStats, vCfg *config.Validation) *ValidateProcessorParameters { + return &ValidateProcessorParameters{ + resultChan: resultChan, + stats: stats, + vCfg: vCfg, + } +} + +type validateTask struct { + e Entry + parameters *ValidateProcessorParameters +} + +func newValidateTask(e Entry, parameters *ValidateProcessorParameters) *validateTask { + return &validateTask{ + e: e, + parameters: parameters, + } +} + +func (t *validateTask) Run(ctx context.Context, submit func(pool.Task) error) error { + // validate the mandatory statement on this entry + if t.e.remainsToExist() { + t.e.ValidateLevel(ctx, t.parameters.resultChan, t.parameters.stats, t.parameters.vCfg) + for _, c := range t.e.GetChilds(DescendMethodActiveChilds) { + submit(newValidateTask(c, t.parameters)) + } + } + return nil +} diff --git a/pkg/tree/root_entry.go b/pkg/tree/root_entry.go index 71361aa4..f07e2bce 100644 --- a/pkg/tree/root_entry.go +++ b/pkg/tree/root_entry.go @@ -8,6 +8,7 @@ import ( "sync" "github.com/sdcio/data-server/pkg/config" + "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/tree/importer" "github.com/sdcio/data-server/pkg/tree/types" "github.com/sdcio/data-server/pkg/utils" @@ -104,18 +105,12 @@ func (r *RootEntry) AddExplicitDeletes(intentName string, priority int32, pathse r.explicitDeletes.Add(intentName, priority, pathset) } -func (r *RootEntry) Validate(ctx context.Context, vCfg *config.Validation) (types.ValidationResults, *types.ValidationStats) { +func (r *RootEntry) Validate(ctx context.Context, vCfg *config.Validation, taskpoolFactory pool.VirtualPoolFactory) (types.ValidationResults, *types.ValidationStats) { // perform validation // we use a channel and cumulate all the errors validationResultEntryChan := make(chan *types.ValidationResultEntry, 10) validationStats := types.NewValidationStats() - // start validation in a seperate goroutine - go func() { - r.sharedEntryAttributes.Validate(ctx, validationResultEntryChan, validationStats, vCfg) - close(validationResultEntryChan) - }() - // create a ValidationResult struct validationResult := types.ValidationResults{} @@ -129,6 +124,10 @@ func (r *RootEntry) Validate(ctx context.Context, vCfg *config.Validation) (type syncWait.Done() }() + validationProcessor := NewValidateProcessor(NewValidateProcessorConfig(validationResultEntryChan, validationStats, vCfg)) + validationProcessor.Run(taskpoolFactory, r.sharedEntryAttributes) + close(validationResultEntryChan) + syncWait.Wait() return validationResult, validationStats } diff --git a/pkg/tree/sharedEntryAttributes.go b/pkg/tree/sharedEntryAttributes.go index d84ac74a..f9b86ce5 100644 --- a/pkg/tree/sharedEntryAttributes.go +++ b/pkg/tree/sharedEntryAttributes.go @@ -962,11 +962,9 @@ func (s *sharedEntryAttributes) getHighestPrecedenceValueOfBranch(filter Highest // Validate is the highlevel function to perform validation. // it will multiplex all the different Validations that need to happen -func (s *sharedEntryAttributes) Validate(ctx context.Context, resultChan chan<- *types.ValidationResultEntry, stats *types.ValidationStats, vCfg *config.Validation) { - +func (s *sharedEntryAttributes) ValidateLevel(ctx context.Context, resultChan chan<- *types.ValidationResultEntry, stats *types.ValidationStats, vCfg *config.Validation) { // validate the mandatory statement on this entry if s.remainsToExist() { - // TODO: Validate Enums if !vCfg.DisabledValidators.Mandatory { s.validateMandatory(ctx, resultChan, stats) @@ -992,26 +990,6 @@ func (s *sharedEntryAttributes) Validate(ctx context.Context, resultChan chan<- if !vCfg.DisabledValidators.MaxElements { s.validateMinMaxElements(resultChan, stats) } - - // recurse the call to the child elements - wg := sync.WaitGroup{} - defer wg.Wait() - for _, c := range s.GetChilds(DescendMethodActiveChilds) { - if c.canDeleteBranch(false) { - // skip validation of branches that can be deleted - continue - } - wg.Add(1) - valFunc := func(x Entry) { - x.Validate(ctx, resultChan, stats, vCfg) - wg.Done() - } - if !vCfg.DisableConcurrency { - go valFunc(c) - } else { - valFunc(c) - } - } } } diff --git a/pkg/tree/sharedEntryAttributes_test.go b/pkg/tree/sharedEntryAttributes_test.go index af8e7882..9ea307d0 100644 --- a/pkg/tree/sharedEntryAttributes_test.go +++ b/pkg/tree/sharedEntryAttributes_test.go @@ -669,7 +669,9 @@ func Test_sharedEntryAttributes_MustCount(t *testing.T) { valConfig.DisabledValidators.DisableAll() valConfig.DisabledValidators.MustStatement = false - result, _ := root.Validate(ctx, valConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + + result, _ := root.Validate(ctx, valConfig, sharedPool) t.Log(strings.Join(result.ErrorsStr(), "\n")) @@ -788,8 +790,9 @@ func Test_sharedEntryAttributes_MustCountDoubleKey(t *testing.T) { valConfig := validationConfig.DeepCopy() valConfig.DisabledValidators.DisableAll() valConfig.DisabledValidators.MustStatement = false + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) - result, _ := root.Validate(ctx, valConfig) + result, _ := root.Validate(ctx, valConfig, sharedPool) t.Log(strings.Join(result.ErrorsStr(), "\n")) @@ -998,7 +1001,9 @@ func Test_sharedEntryAttributes_validateMandatory(t *testing.T) { validationConfig.DisabledValidators.DisableAll() validationConfig.DisabledValidators.Mandatory = false - validationResults, _ := root.Validate(ctx, validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + + validationResults, _ := root.Validate(ctx, validationConfig, sharedPool) results := []string{} for _, e := range validationResults { @@ -1289,7 +1294,8 @@ func Test_sharedEntryAttributes_validateMinMaxElements(t *testing.T) { valConfig.DisabledValidators.DisableAll() valConfig.DisabledValidators.MaxElements = false - result, _ := root.Validate(ctx, valConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + result, _ := root.Validate(ctx, valConfig, sharedPool) t.Log(strings.Join(result.ErrorsStr(), "\n")) @@ -1457,7 +1463,8 @@ func Test_sharedEntryAttributes_validateMinMaxElementsDoubleKey(t *testing.T) { valConfig.DisabledValidators.DisableAll() valConfig.DisabledValidators.MaxElements = false - result, _ := root.Validate(ctx, valConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + result, _ := root.Validate(ctx, valConfig, sharedPool) t.Log(strings.Join(result.ErrorsStr(), "\n")) diff --git a/pkg/tree/types/validation_stats.go b/pkg/tree/types/validation_stats.go index 1679fc4b..b4e2a182 100644 --- a/pkg/tree/types/validation_stats.go +++ b/pkg/tree/types/validation_stats.go @@ -3,6 +3,7 @@ package types import ( "fmt" "strings" + "sync" "sync/atomic" ) @@ -59,12 +60,14 @@ func (s StatType) String() string { } type ValidationStats struct { - Counter map[StatType]*uint32 `json:"counters"` + Counter map[StatType]*uint32 `json:"counters"` + muCounter *sync.Mutex } func NewValidationStats() *ValidationStats { result := &ValidationStats{ - Counter: map[StatType]*uint32{}, + Counter: map[StatType]*uint32{}, + muCounter: &sync.Mutex{}, } for _, t := range AllStatTypes { result.Counter[t] = new(uint32) @@ -73,6 +76,8 @@ func NewValidationStats() *ValidationStats { } func (v *ValidationStats) Add(t StatType, i uint32) { + v.muCounter.Lock() + defer v.muCounter.Unlock() if counter, ok := v.Counter[t]; ok { atomic.AddUint32(counter, i) } @@ -80,6 +85,8 @@ func (v *ValidationStats) Add(t StatType, i uint32) { // String returns a string representation of all counters func (v *ValidationStats) String() string { + v.muCounter.Lock() + defer v.muCounter.Unlock() result := make([]string, 0, len(v.Counter)) for typ, count := range v.Counter { val := atomic.LoadUint32(count) @@ -90,6 +97,8 @@ func (v *ValidationStats) String() string { // GetCounter returns a snapshot of the counters as a plain map func (v *ValidationStats) GetCounter() map[StatType]uint32 { + v.muCounter.Lock() + defer v.muCounter.Unlock() snapshot := make(map[StatType]uint32, len(v.Counter)) for typ, count := range v.Counter { snapshot[typ] = atomic.LoadUint32(count) diff --git a/pkg/tree/validation_range_test.go b/pkg/tree/validation_range_test.go index 4b9c6c19..bed8495a 100644 --- a/pkg/tree/validation_range_test.go +++ b/pkg/tree/validation_range_test.go @@ -3,10 +3,12 @@ package tree import ( "context" "encoding/json" + "runtime" "strings" "testing" "github.com/openconfig/ygot/ygot" + "github.com/sdcio/data-server/pkg/pool" json_importer "github.com/sdcio/data-server/pkg/tree/importer/json" "github.com/sdcio/data-server/pkg/tree/types" "github.com/sdcio/data-server/pkg/utils/testhelper" @@ -76,7 +78,9 @@ func TestValidate_Range_SDC_Schema(t *testing.T) { } valConf := validationConfig.DeepCopy() - validationResult, _ := root.Validate(ctx, valConf) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + + validationResult, _ := root.Validate(ctx, valConf, sharedPool) t.Logf("Validation Errors:\n%s", strings.Join(validationResult.ErrorsStr(), "\n")) t.Log(root.String()) @@ -188,7 +192,8 @@ func TestValidate_RangesSigned(t *testing.T) { t.Error(err) } - validationResult, _ := root.Validate(ctx, validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(ctx, validationConfig, sharedPool) t.Logf("Validation Errors:\n%s", strings.Join(validationResult.ErrorsStr(), "\n")) t.Log(root.String()) @@ -322,7 +327,8 @@ func TestValidate_RangesUnSigned(t *testing.T) { } // run validation - validationResults, _ := root.Validate(ctx, validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResults, _ := root.Validate(ctx, validationConfig, sharedPool) t.Logf("Validation Errors:\n%s", strings.Join(validationResults.ErrorsStr(), "\n")) t.Log(root.String()) From c4977cbd5a1986e731fcee927f4475d17cf36c70 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 5 Nov 2025 16:29:56 +0100 Subject: [PATCH 19/44] update --- pkg/tree/processor_validate.go | 5 +++++ pkg/tree/validation_entry_leafref.go | 5 ++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/pkg/tree/processor_validate.go b/pkg/tree/processor_validate.go index 0fa2067e..a8a018c5 100644 --- a/pkg/tree/processor_validate.go +++ b/pkg/tree/processor_validate.go @@ -21,6 +21,8 @@ func NewValidateProcessor(parameters *ValidateProcessorParameters) *ValidateProc func (p *ValidateProcessor) Run(taskpoolFactory pool.VirtualPoolFactory, e Entry) { taskpool := taskpoolFactory.NewVirtualPool(pool.VirtualTolerant, 0) taskpool.Submit(newValidateTask(e, p.parameters)) + taskpool.CloseForSubmit() + taskpool.Wait() } type ValidateProcessorParameters struct { @@ -50,6 +52,9 @@ func newValidateTask(e Entry, parameters *ValidateProcessorParameters) *validate } func (t *validateTask) Run(ctx context.Context, submit func(pool.Task) error) error { + if ctx.Err() != nil { + return nil + } // validate the mandatory statement on this entry if t.e.remainsToExist() { t.e.ValidateLevel(ctx, t.parameters.resultChan, t.parameters.stats, t.parameters.vCfg) diff --git a/pkg/tree/validation_entry_leafref.go b/pkg/tree/validation_entry_leafref.go index f0f2bd09..3367270f 100644 --- a/pkg/tree/validation_entry_leafref.go +++ b/pkg/tree/validation_entry_leafref.go @@ -221,8 +221,11 @@ func (s *sharedEntryAttributes) validateLeafRefs(ctx context.Context, resultChan if highest != nil { owner = highest.Owner() } + + pathStr := s.SdcpbPath().ToXPath(false) // if required, issue error - resultChan <- types.NewValidationResultEntry(owner, fmt.Errorf("missing leaf reference: failed resolving leafref %s for %s: %v", lref, s.SdcpbPath().ToXPath(false), err), types.ValidationResultEntryTypeError) + valRes := types.NewValidationResultEntry(owner, fmt.Errorf("missing leaf reference: failed resolving leafref %s for %s: %v", lref, pathStr, err), types.ValidationResultEntryTypeError) + resultChan <- valRes return } From 1c6a642c516820e90eaaeb1a9f25067e9b2f7173 Mon Sep 17 00:00:00 2001 From: steiler Date: Thu, 6 Nov 2025 11:17:45 +0100 Subject: [PATCH 20/44] fix get sync --- pkg/datastore/target/gnmi/get.go | 61 +++++++++++++++----------------- pkg/datastore/target/target.go | 1 - 2 files changed, 29 insertions(+), 33 deletions(-) diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index 74725fbd..2b40a7a7 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -17,13 +17,15 @@ import ( ) type GetSync struct { - config *config.SyncProtocol - target GetTarget - cancel context.CancelFunc - runningStore types.RunningStore - ctx context.Context - schemaClient dsutils.SchemaClientBound - paths []*sdcpb.Path + config *config.SyncProtocol + target GetTarget + cancel context.CancelFunc + runningStore types.RunningStore + ctx context.Context + schemaClient dsutils.SchemaClientBound + paths []*sdcpb.Path + syncTree *tree.RootEntry + syncTreeMutex *sync.Mutex } func NewGetSync(ctx context.Context, target GetTarget, c *config.SyncProtocol, runningStore types.RunningStore, schemaClient dsutils.SchemaClientBound) (*GetSync, error) { @@ -42,13 +44,14 @@ func NewGetSync(ctx context.Context, target GetTarget, c *config.SyncProtocol, r } return &GetSync{ - config: c, - target: target, - cancel: cancel, - runningStore: runningStore, - ctx: ctx, - schemaClient: schemaClient, - paths: paths, + config: c, + target: target, + cancel: cancel, + runningStore: runningStore, + ctx: ctx, + schemaClient: schemaClient, + paths: paths, + syncTreeMutex: &sync.Mutex{}, }, nil } @@ -95,28 +98,30 @@ func (s *GetSync) Start() error { } func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { + s.syncTreeMutex.Lock() + defer s.syncTreeMutex.Unlock() + // execute gnmi get resp, err := s.target.Get(s.ctx, req) if err != nil { log.Errorf("sync error: %v", err) return } - syncTree, err := s.runningStore.NewEmptyTree(s.ctx) + + s.syncTree, err = s.runningStore.NewEmptyTree(s.ctx) if err != nil { log.Errorf("sync newemptytree error: %v", err) return } - syncTreeMutex := &sync.Mutex{} - // process Noifications - _, err = processNotifications(s.ctx, resp.GetNotification(), s.schemaClient, syncTree, syncTreeMutex) + err = s.processNotifications(resp.GetNotification()) if err != nil { log.Errorf("sync process notifications error: %v", err) return } - result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) + result, err := s.syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) if err != nil { log.Errorf("sync tree export error: %v", err) return @@ -133,34 +138,26 @@ type GetTarget interface { Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) } -func processNotifications(ctx context.Context, n []*sdcpb.Notification, schemaClient dsutils.SchemaClientBound, syncTree *tree.RootEntry, m *sync.Mutex) ([]*sdcpb.Path, error) { - +func (s *GetSync) processNotifications(n []*sdcpb.Notification) error { ts := time.Now().Unix() uif := treetypes.NewUpdateInsertFlags() - deletes := []*sdcpb.Path{} - - m.Lock() - defer m.Unlock() - for _, noti := range n { // updates - upds, err := treetypes.ExpandAndConvertIntent(ctx, schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, noti.Update, ts) + upds, err := treetypes.ExpandAndConvertIntent(s.ctx, s.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, noti.Update, ts) if err != nil { log.Errorf("sync expanding error: %v", err) continue } - // deletes - deletes = append(deletes, noti.GetDelete()...) - for idx2, upd := range upds { _ = idx2 - _, err = syncTree.AddUpdateRecursive(ctx, upd.Path(), upd, uif) + _, err = s.syncTree.AddUpdateRecursive(s.ctx, upd.Path(), upd, uif) if err != nil { log.Errorf("sync process notifications error: %v, continuing", err) } } } - return deletes, nil + + return nil } diff --git a/pkg/datastore/target/target.go b/pkg/datastore/target/target.go index ee400516..9466db48 100644 --- a/pkg/datastore/target/target.go +++ b/pkg/datastore/target/target.go @@ -60,7 +60,6 @@ func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaC // return newNoopTarget(ctx, name) default: return nil, fmt.Errorf("unknown DS target type %q", cfg.Type) - } err = t.AddSyncs(targetContext, syncConfigs...) From 01bad77a2b8dc00a4d227eaa85801e77130e7739 Mon Sep 17 00:00:00 2001 From: steiler Date: Thu, 6 Nov 2025 12:46:35 +0100 Subject: [PATCH 21/44] fix context cancelation --- pkg/datastore/datastore_rpc.go | 1 + pkg/datastore/target/gnmi/get.go | 5 +++++ pkg/datastore/target/gnmi/stream.go | 1 + pkg/datastore/target/target.go | 6 ++---- 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/pkg/datastore/datastore_rpc.go b/pkg/datastore/datastore_rpc.go index 53b7c4db..18bb2942 100644 --- a/pkg/datastore/datastore_rpc.go +++ b/pkg/datastore/datastore_rpc.go @@ -119,6 +119,7 @@ func New(ctx context.Context, c *config.DatastoreConfig, sc schema.Client, cc ca } if err != nil { log.Errorf("failed to create SBI for target %s: %v", ds.Config().Name, err) + cancel() return } diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index 2b40a7a7..4075c330 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -79,6 +79,11 @@ func (s *GetSync) Start() error { return err } + if s.ctx.Err() != nil { + // stopping sync due to context canceled + return nil + } + go s.internalGetSync(req) go func() { diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index d095fcbf..5213b74e 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -115,6 +115,7 @@ func (s *StreamSync) buildTreeSyncWithDatastore(cUS <-chan *NotificationData, sy for { select { case <-s.ctx.Done(): + log.Debugf(">>>>>>>>>>>>>>>>>> Sync %s context done", s.config.Name) return case noti, ok := <-cUS: if !ok { diff --git a/pkg/datastore/target/target.go b/pkg/datastore/target/target.go index 9466db48..2b6b9e92 100644 --- a/pkg/datastore/target/target.go +++ b/pkg/datastore/target/target.go @@ -46,11 +46,9 @@ func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaC var t Target var err error - targetContext := context.Background() - switch cfg.Type { case targetTypeGNMI: - t, err = gnmi.NewTarget(targetContext, name, cfg, runningStore, schemaClient, taskpoolFactory, opts...) + t, err = gnmi.NewTarget(ctx, name, cfg, runningStore, schemaClient, taskpoolFactory, opts...) if err != nil { return nil, err } @@ -62,7 +60,7 @@ func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaC return nil, fmt.Errorf("unknown DS target type %q", cfg.Type) } - err = t.AddSyncs(targetContext, syncConfigs...) + err = t.AddSyncs(ctx, syncConfigs...) if err != nil { return nil, err } From fe1b4fab8e41448a2cdbaf25277bb6c0171edb1d Mon Sep 17 00:00:00 2001 From: steiler Date: Mon, 8 Dec 2025 13:08:31 +0100 Subject: [PATCH 22/44] mod update --- go.mod | 2 ++ 1 file changed, 2 insertions(+) diff --git a/go.mod b/go.mod index 3b8946a9..510c85fb 100644 --- a/go.mod +++ b/go.mod @@ -4,6 +4,8 @@ go 1.23.9 replace github.com/openconfig/goyang v1.6.0 => github.com/sdcio/goyang v1.6.2-2 +replace github.com/sdcio/sdc-protos => /home/mava/projects/sdc-protos + require ( github.com/AlekSi/pointer v1.2.0 github.com/beevik/etree v1.6.0 From 238cff119104c1ca41553853a4262e76142e8c39 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 10 Dec 2025 09:38:23 +0100 Subject: [PATCH 23/44] implement netconf --- pkg/datastore/datastore_rpc.go | 4 +- pkg/datastore/target/gnmi/get.go | 8 +- pkg/datastore/target/gnmi/gnmi.go | 13 +- pkg/datastore/target/gnmi/gnmisync.go | 1 + pkg/datastore/target/gnmi/once.go | 5 + pkg/datastore/target/gnmi/stream.go | 4 + pkg/datastore/target/{ => netconf}/nc.go | 163 +++++++----------- pkg/datastore/target/{ => netconf}/nc_test.go | 15 +- pkg/datastore/target/netconf/sync.go | 123 +++++++++++++ pkg/datastore/target/noop/noop.go | 24 ++- pkg/datastore/target/target.go | 18 +- pkg/server/datastore.go | 2 +- pkg/server/server.go | 6 +- 13 files changed, 243 insertions(+), 143 deletions(-) rename pkg/datastore/target/{ => netconf}/nc.go (70%) rename pkg/datastore/target/{ => netconf}/nc_test.go (96%) create mode 100644 pkg/datastore/target/netconf/sync.go diff --git a/pkg/datastore/datastore_rpc.go b/pkg/datastore/datastore_rpc.go index 18bb2942..981e8f92 100644 --- a/pkg/datastore/datastore_rpc.go +++ b/pkg/datastore/datastore_rpc.go @@ -199,7 +199,7 @@ func (d *Datastore) ConnectionState() *targettypes.TargetStatus { return d.sbi.Status() } -func (d *Datastore) Stop() error { +func (d *Datastore) Stop(ctx context.Context) error { if d == nil { return nil } @@ -207,7 +207,7 @@ func (d *Datastore) Stop() error { if d.sbi == nil { return nil } - err := d.sbi.Close() + err := d.sbi.Close(ctx) if err != nil { log.Errorf("datastore %s failed to close the target connection: %v", d.Name(), err) } diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index 4075c330..fd6ff77b 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -12,6 +12,7 @@ import ( "github.com/sdcio/data-server/pkg/tree/importer/proto" treetypes "github.com/sdcio/data-server/pkg/tree/types" dsutils "github.com/sdcio/data-server/pkg/utils" + "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" log "github.com/sirupsen/logrus" ) @@ -68,11 +69,16 @@ func (s *GetSync) syncConfig() (*sdcpb.GetDataRequest, error) { } func (s *GetSync) Stop() error { - // TODO + log := logger.FromContext(s.ctx) + log.Info("Stopping Sync", "sync", s.config.Name) s.cancel() return nil } +func (s *GetSync) Name() string { + return s.config.Name +} + func (s *GetSync) Start() error { req, err := s.syncConfig() if err != nil { diff --git a/pkg/datastore/target/gnmi/gnmi.go b/pkg/datastore/target/gnmi/gnmi.go index d725d3c9..bed5986f 100644 --- a/pkg/datastore/target/gnmi/gnmi.go +++ b/pkg/datastore/target/gnmi/gnmi.go @@ -267,17 +267,8 @@ func (t *gnmiTarget) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) g = NewStreamSync(ctx, t, sp, t.runningStore, t.schemaClient, t.taskpoolFactory) } t.syncs[sp.Name] = g - err := g.Start() - if err != nil { - return err - } - } - return nil -} -func (t *gnmiTarget) Sync() error { - for _, v := range t.syncs { - err := v.Start() + err := g.Start() if err != nil { return err } @@ -345,7 +336,7 @@ func (t *gnmiTarget) Sync() error { // } // } -func (t *gnmiTarget) Close() error { +func (t *gnmiTarget) Close(ctx context.Context) error { if t == nil { return nil } diff --git a/pkg/datastore/target/gnmi/gnmisync.go b/pkg/datastore/target/gnmi/gnmisync.go index bd73e388..8774b87f 100644 --- a/pkg/datastore/target/gnmi/gnmisync.go +++ b/pkg/datastore/target/gnmi/gnmisync.go @@ -3,4 +3,5 @@ package gnmi type GnmiSync interface { Start() error Stop() error + Name() string } diff --git a/pkg/datastore/target/gnmi/once.go b/pkg/datastore/target/gnmi/once.go index cc11ce32..83b7ab31 100644 --- a/pkg/datastore/target/gnmi/once.go +++ b/pkg/datastore/target/gnmi/once.go @@ -29,9 +29,14 @@ func NewOnceSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, cancel: cancel, runningStore: runningStore, vpoolFactory: vpoolFactory, + ctx: ctx, } } +func (s *OnceSync) Name() string { + return s.config.Name +} + func (s *OnceSync) syncConfig() (*gnmi.SubscribeRequest, error) { opts := make([]gapi.GNMIOption, 0) subscriptionOpts := make([]gapi.GNMIOption, 0) diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index 5213b74e..22a21d3c 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -78,6 +78,10 @@ func (s *StreamSync) Stop() error { return nil } +func (s *StreamSync) Name() string { + return s.config.Name +} + func (s *StreamSync) Start() error { updChan := make(chan *NotificationData, 20) diff --git a/pkg/datastore/target/nc.go b/pkg/datastore/target/netconf/nc.go similarity index 70% rename from pkg/datastore/target/nc.go rename to pkg/datastore/target/netconf/nc.go index 667d8ed1..4686fa13 100644 --- a/pkg/datastore/target/nc.go +++ b/pkg/datastore/target/netconf/nc.go @@ -12,11 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -package target +package netconf import ( "context" - "errors" "fmt" "strings" "sync" @@ -28,29 +27,35 @@ import ( "github.com/sdcio/data-server/pkg/config" schemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" - "github.com/sdcio/data-server/pkg/datastore/target/netconf" "github.com/sdcio/data-server/pkg/datastore/target/netconf/driver/scrapligo" + nctypes "github.com/sdcio/data-server/pkg/datastore/target/netconf/types" "github.com/sdcio/data-server/pkg/datastore/target/types" + "github.com/sdcio/data-server/pkg/tree/importer" + "github.com/sdcio/data-server/pkg/tree/importer/xml" ) type ncTarget struct { name string - driver netconf.Driver + driver Driver m *sync.Mutex + syncs map[string]NetconfSync schemaClient schemaClient.SchemaClientBound sbiConfig *config.SBI - xml2sdcpbAdapter *netconf.XML2sdcpbConfigAdapter + xml2sdcpbAdapter *XML2sdcpbConfigAdapter + runningStore types.RunningStore } -func newNCTarget(_ context.Context, name string, cfg *config.SBI, schemaClient schemaClient.SchemaClientBound) (*ncTarget, error) { +func NewNCTarget(_ context.Context, name string, cfg *config.SBI, runningStore types.RunningStore, schemaClient schemaClient.SchemaClientBound) (*ncTarget, error) { t := &ncTarget{ name: name, m: new(sync.Mutex), schemaClient: schemaClient, sbiConfig: cfg, - xml2sdcpbAdapter: netconf.NewXML2sdcpbConfigAdapter(schemaClient), + xml2sdcpbAdapter: NewXML2sdcpbConfigAdapter(schemaClient), + syncs: map[string]NetconfSync{}, + runningStore: runningStore, } var err error // create a new NETCONF driver @@ -61,15 +66,42 @@ func newNCTarget(_ context.Context, name string, cfg *config.SBI, schemaClient s return t, nil } -func (t *ncTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) { +func (t *ncTarget) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) error { + for _, sp := range sps { + ncSync, err := NewNetconfSyncImpl(ctx, t.name, t, sp, t.runningStore) + if err != nil { + return err + } + t.syncs[sp.Name] = ncSync + + err = ncSync.Start() + if err != nil { + return err + } + } + return nil +} + +func (t *ncTarget) GetImportAdapter(ctx context.Context, req *sdcpb.GetDataRequest) (importer.ImportConfigAdapter, error) { + ncResponse, err := t.internalGet(ctx, req) + if err != nil { + return nil, err + } + + cmlImport := xml.NewXmlTreeImporter(ncResponse.Doc.Root()) + + return cmlImport, nil +} + +func (t *ncTarget) internalGet(ctx context.Context, req *sdcpb.GetDataRequest) (*nctypes.NetconfResponse, error) { if !t.Status().IsConnected() { return nil, fmt.Errorf("%s", types.TargetStatusNotConnected) } source := "running" // init a new XMLConfigBuilder for the pathfilter - pathfilterXmlBuilder := netconf.NewXMLConfigBuilder(t.schemaClient, - &netconf.XMLConfigBuilderOpts{ + pathfilterXmlBuilder := NewXMLConfigBuilder(t.schemaClient, + &XMLConfigBuilderOpts{ HonorNamespace: t.sbiConfig.NetconfOptions.IncludeNS, OperationWithNamespace: t.sbiConfig.NetconfOptions.OperationWithNamespace, UseOperationRemove: t.sbiConfig.NetconfOptions.UseOperationRemove, @@ -94,20 +126,21 @@ func (t *ncTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.G ncResponse, err := t.driver.GetConfig(source, filterDoc) if err != nil { if strings.Contains(err.Error(), "EOF") { - t.Close() + t.Close(ctx) go t.reconnect() } return nil, err } - log.Debugf("%s: netconf response:\n%s", t.name, ncResponse.DocAsString()) + return ncResponse, err +} - // cmlImport := xml.NewXmlTreeImporter(ncResponse.Doc.Root()) - - // treeCacheSchemaClient := tree.NewTreeSchemaCacheClient(t.name, nil, d.getValidationClient()) - // tc := tree.NewTreeContext(treeCacheSchemaClient, tree.RunningIntentName) +func (t *ncTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) { - // NewTreeRoot + ncResponse, err := t.internalGet(ctx, req) + if err != nil { + return nil, err + } // start transformation, which yields the sdcpb_Notification noti, err := t.xml2sdcpbAdapter.Transform(ctx, ncResponse.Doc) @@ -129,9 +162,9 @@ func (t *ncTarget) Set(ctx context.Context, source types.TargetSource) (*sdcpb.S switch t.sbiConfig.NetconfOptions.CommitDatastore { case "running": - return t.setRunning(source) + return t.setRunning(ctx, source) case "candidate": - return t.setCandidate(source) + return t.setCandidate(ctx, source) } // should not get here if the config validation happened. return nil, fmt.Errorf("unknown commit-datastore: %s", t.sbiConfig.NetconfOptions.CommitDatastore) @@ -149,87 +182,7 @@ func (t *ncTarget) Status() *types.TargetStatus { return result } -func (t *ncTarget) Sync(ctx context.Context, syncConfig *config.Sync) { - log.Infof("starting target %s [%s] sync", t.name, t.sbiConfig.Address) - - for _, ncc := range syncConfig.Config { - // periodic get - log.Debugf("target %s, starting sync: %s, Interval: %s, Paths: [ \"%s\" ]", t.name, ncc.Name, ncc.Interval.String(), strings.Join(ncc.Paths, "\", \"")) - go func(ncSync *config.SyncProtocol) { - t.internalSync(ctx, ncSync, true) - ticker := time.NewTicker(ncSync.Interval) - defer ticker.Stop() - for { - select { - case <-ctx.Done(): - return - case <-ticker.C: - t.internalSync(ctx, ncSync, false) - } - } - }(ncc) - } - - <-ctx.Done() - if !errors.Is(ctx.Err(), context.Canceled) { - log.Errorf("datastore %s sync stopped: %v", t.name, ctx.Err()) - } -} - -func (t *ncTarget) internalSync(ctx context.Context, sc *config.SyncProtocol, force bool) { - if !t.Status().IsConnected() { - return - } - // iterate syncConfig - paths := make([]*sdcpb.Path, 0, len(sc.Paths)) - // iterate referenced paths - for _, p := range sc.Paths { - path, err := sdcpb.ParsePath(p) - if err != nil { - log.Errorf("failed Parsing Path %q, %v", p, err) - return - } - // add the parsed path - paths = append(paths, path) - } - - // init a DataRequest - req := &sdcpb.GetDataRequest{ - Name: sc.Name, - Path: paths, - DataType: sdcpb.DataType_CONFIG, - } - - // execute netconf get - resp, err := t.Get(ctx, req) - if err != nil { - log.Errorf("failed getting config from target %v: %T | %v", t.name, err, err) - if strings.Contains(err.Error(), "EOF") { - t.Close() - go t.reconnect() - } - return - } - _ = resp - // // push notifications into syncCh - // syncCh <- &SyncUpdate{ - // Start: true, - // Force: force, - // } - // notificationsCount := 0 - // for _, n := range resp.GetNotification() { - // syncCh <- &SyncUpdate{ - // Update: n, - // } - // notificationsCount++ - // } - // log.Debugf("%s: sync-ed %d notifications", t.name, notificationsCount) - // syncCh <- &SyncUpdate{ - // End: true, - // } -} - -func (t *ncTarget) Close() error { +func (t *ncTarget) Close(ctx context.Context) error { if t == nil { return nil } @@ -261,7 +214,7 @@ func (t *ncTarget) reconnect() { } } -func (t *ncTarget) setRunning(source types.TargetSource) (*sdcpb.SetDataResponse, error) { +func (t *ncTarget) setRunning(ctx context.Context, source types.TargetSource) (*sdcpb.SetDataResponse, error) { xtree, err := source.ToXML(true, t.sbiConfig.NetconfOptions.IncludeNS, t.sbiConfig.NetconfOptions.OperationWithNamespace, t.sbiConfig.NetconfOptions.UseOperationRemove) if err != nil { @@ -287,7 +240,7 @@ func (t *ncTarget) setRunning(source types.TargetSource) (*sdcpb.SetDataResponse if err != nil { log.Errorf("datastore %s failed edit-config: %v", t.name, err) if strings.Contains(err.Error(), "EOF") { - t.Close() + t.Close(ctx) go t.reconnect() return nil, err } @@ -321,7 +274,7 @@ func filterRPCErrors(xml *etree.Document, severity string) ([]string, error) { return result, nil } -func (t *ncTarget) setCandidate(source types.TargetSource) (*sdcpb.SetDataResponse, error) { +func (t *ncTarget) setCandidate(ctx context.Context, source types.TargetSource) (*sdcpb.SetDataResponse, error) { xtree, err := source.ToXML(true, t.sbiConfig.NetconfOptions.IncludeNS, t.sbiConfig.NetconfOptions.OperationWithNamespace, t.sbiConfig.NetconfOptions.UseOperationRemove) if err != nil { return nil, err @@ -346,7 +299,7 @@ func (t *ncTarget) setCandidate(source types.TargetSource) (*sdcpb.SetDataRespon if err != nil { log.Errorf("datastore %s failed edit-config: %v", t.name, err) if strings.Contains(err.Error(), "EOF") { - t.Close() + t.Close(ctx) go t.reconnect() return nil, err } @@ -367,7 +320,7 @@ func (t *ncTarget) setCandidate(source types.TargetSource) (*sdcpb.SetDataRespon err = t.driver.Commit() if err != nil { if strings.Contains(err.Error(), "EOF") { - t.Close() + t.Close(ctx) go t.reconnect() } return nil, err diff --git a/pkg/datastore/target/nc_test.go b/pkg/datastore/target/netconf/nc_test.go similarity index 96% rename from pkg/datastore/target/nc_test.go rename to pkg/datastore/target/netconf/nc_test.go index 17a76d65..e426f397 100644 --- a/pkg/datastore/target/nc_test.go +++ b/pkg/datastore/target/netconf/nc_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package target +package netconf import ( "context" @@ -26,21 +26,16 @@ import ( "github.com/sdcio/data-server/mocks/mockschemaclientbound" "github.com/sdcio/data-server/pkg/config" SchemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" - "github.com/sdcio/data-server/pkg/datastore/target/netconf" "github.com/sdcio/data-server/pkg/datastore/target/netconf/types" "github.com/sdcio/data-server/pkg/utils/testhelper" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "go.uber.org/mock/gomock" ) -var ( - TestCtx = context.TODO() -) - func Test_ncTarget_Get(t *testing.T) { type fields struct { name string - getDriver func(*gomock.Controller, *testing.T) netconf.Driver + getDriver func(*gomock.Controller, *testing.T) Driver getSchemaClient func(*gomock.Controller, *testing.T) SchemaClient.SchemaClientBound sbiConfig *config.SBI } @@ -58,7 +53,7 @@ func Test_ncTarget_Get(t *testing.T) { { name: "One", fields: fields{ - getDriver: func(c *gomock.Controller, t *testing.T) netconf.Driver { + getDriver: func(c *gomock.Controller, t *testing.T) Driver { d := mocknetconf.NewMockDriver(c) responseDoc := etree.NewDocument() err := responseDoc.ReadFromString("eth01500") @@ -242,7 +237,7 @@ func Test_ncTarget_Get(t *testing.T) { driver: tt.fields.getDriver(mockCtrl, t), schemaClient: sc, sbiConfig: tt.fields.sbiConfig, - xml2sdcpbAdapter: netconf.NewXML2sdcpbConfigAdapter(sc), + xml2sdcpbAdapter: NewXML2sdcpbConfigAdapter(sc), } got, err := tr.Get(tt.args.ctx, tt.args.req) if (err != nil) != tt.wantErr { @@ -290,7 +285,7 @@ func TestLeafList(t *testing.T) { }, } - xmlBuilder := netconf.NewXMLConfigBuilder(scb, &netconf.XMLConfigBuilderOpts{ + xmlBuilder := NewXMLConfigBuilder(scb, &XMLConfigBuilderOpts{ UseOperationRemove: true, HonorNamespace: true, }) diff --git a/pkg/datastore/target/netconf/sync.go b/pkg/datastore/target/netconf/sync.go new file mode 100644 index 00000000..98b0ddea --- /dev/null +++ b/pkg/datastore/target/netconf/sync.go @@ -0,0 +1,123 @@ +package netconf + +import ( + "context" + "time" + + "github.com/sdcio/data-server/pkg/config" + "github.com/sdcio/data-server/pkg/datastore/target/types" + "github.com/sdcio/data-server/pkg/tree/importer" + "github.com/sdcio/logger" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" +) + +type NetconfSync interface { + Start() error + Stop() error +} + +type NetconfSyncImpl struct { + config *config.SyncProtocol + cancel context.CancelFunc + ctx context.Context + target GetXMLImporter + paths []*sdcpb.Path + targetName string + runningStore types.RunningStore +} + +func NewNetconfSyncImpl(ctx context.Context, targetName string, target GetXMLImporter, c *config.SyncProtocol, runningStore types.RunningStore) (*NetconfSyncImpl, error) { + ctx, cancel := context.WithCancel(ctx) + + // add the sync name to the logger values + log := logger.FromContext(ctx).WithValues("sync", c.Name) + ctx = logger.IntoContext(ctx, log) + + paths := make([]*sdcpb.Path, 0, len(c.Paths)) + // iterate referenced paths + for _, p := range c.Paths { + path, err := sdcpb.ParsePath(p) + if err != nil { + cancel() + return nil, err + } + // add the parsed path + paths = append(paths, path) + } + + return &NetconfSyncImpl{ + config: c, + cancel: cancel, + ctx: ctx, + targetName: targetName, + runningStore: runningStore, + target: target, + }, nil +} + +func (s *NetconfSyncImpl) Start() error { + log := logger.FromContext(s.ctx) + log.Info("Starting Sync") + + req, err := s.syncConfig() + if err != nil { + return err + } + + if s.ctx.Err() != nil { + // stopping sync due to context canceled + return nil + } + + go s.internalSync(req) + + go func() { + ticker := time.NewTicker(s.config.Interval) + defer ticker.Stop() + for { + select { + case <-s.ctx.Done(): + return + case <-ticker.C: + err := s.internalSync(req) + if err != nil { + log.Error(err, "failed syncing") + } + } + } + }() + + return nil + +} + +func (s *NetconfSyncImpl) syncConfig() (*sdcpb.GetDataRequest, error) { + // init a DataRequest + req := &sdcpb.GetDataRequest{ + Name: s.targetName, + Path: s.paths, + DataType: sdcpb.DataType_CONFIG, + } + return req, nil +} + +func (s *NetconfSyncImpl) internalSync(req *sdcpb.GetDataRequest) error { + // execute netconf get + importer, err := s.target.GetImportAdapter(s.ctx, req) + if err != nil { + return err + } + + return s.runningStore.ApplyToRunning(s.ctx, s.paths, importer) +} + +func (s *NetconfSyncImpl) Stop() error { + log := logger.FromContext(s.ctx) + log.Info("Stopping Sync") + s.cancel() + return nil +} + +type GetXMLImporter interface { + GetImportAdapter(ctx context.Context, req *sdcpb.GetDataRequest) (importer.ImportConfigAdapter, error) +} diff --git a/pkg/datastore/target/noop/noop.go b/pkg/datastore/target/noop/noop.go index c27c7bf7..1f35c541 100644 --- a/pkg/datastore/target/noop/noop.go +++ b/pkg/datastore/target/noop/noop.go @@ -12,14 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -package target +package noop import ( "context" + "encoding/json" "time" + logf "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" - log "github.com/sirupsen/logrus" "github.com/sdcio/data-server/pkg/config" "github.com/sdcio/data-server/pkg/datastore/target/types" @@ -29,13 +30,25 @@ type noopTarget struct { name string } -func newNoopTarget(_ context.Context, name string) (*noopTarget, error) { +func NewNoopTarget(_ context.Context, name string) (*noopTarget, error) { nt := &noopTarget{ name: name, } return nt, nil } +func (t *noopTarget) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) error { + log := logf.FromContext(ctx) + for _, sp := range sps { + jConf, err := json.Marshal(sp) + if err != nil { + return err + } + log.Info("Sync added", "Config", jConf) + } + return nil +} + func (t *noopTarget) Get(_ context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) { result := &sdcpb.GetDataResponse{ Notification: make([]*sdcpb.Notification, 0, len(req.GetPath())), @@ -94,7 +107,8 @@ func (t *noopTarget) Status() *types.TargetStatus { } func (t *noopTarget) Sync(ctx context.Context, _ *config.Sync) { - log.Infof("starting target %s sync", t.name) + log := logf.FromContext(ctx) + log.Info("starting sync", "target", t.name) } -func (t *noopTarget) Close() error { return nil } +func (t *noopTarget) Close(ctx context.Context) error { return nil } diff --git a/pkg/datastore/target/target.go b/pkg/datastore/target/target.go index 2b6b9e92..aced9696 100644 --- a/pkg/datastore/target/target.go +++ b/pkg/datastore/target/target.go @@ -24,6 +24,8 @@ import ( "github.com/sdcio/data-server/pkg/config" schemaClient "github.com/sdcio/data-server/pkg/datastore/clients/schema" "github.com/sdcio/data-server/pkg/datastore/target/gnmi" + "github.com/sdcio/data-server/pkg/datastore/target/netconf" + "github.com/sdcio/data-server/pkg/datastore/target/noop" "github.com/sdcio/data-server/pkg/datastore/target/types" "github.com/sdcio/data-server/pkg/pool" ) @@ -39,7 +41,7 @@ type Target interface { Set(ctx context.Context, source types.TargetSource) (*sdcpb.SetDataResponse, error) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) error Status() *types.TargetStatus - Close() error + Close(ctx context.Context) error } func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaClient.SchemaClientBound, runningStore types.RunningStore, syncConfigs []*config.SyncProtocol, taskpoolFactory pool.VirtualPoolFactory, opts ...grpc.DialOption) (Target, error) { @@ -52,10 +54,16 @@ func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaC if err != nil { return nil, err } - // case targetTypeNETCONF: - // return newNCTarget(ctx, name, cfg, schemaClient) - // case targetTypeNOOP, "": - // return newNoopTarget(ctx, name) + case targetTypeNETCONF: + t, err = netconf.NewNCTarget(ctx, name, cfg, runningStore, schemaClient) + if err != nil { + return nil, err + } + case targetTypeNOOP, "": + t, err = noop.NewNoopTarget(ctx, name) + if err != nil { + return nil, err + } default: return nil, fmt.Errorf("unknown DS target type %q", cfg.Type) } diff --git a/pkg/server/datastore.go b/pkg/server/datastore.go index 41cc220e..89cc7446 100644 --- a/pkg/server/datastore.go +++ b/pkg/server/datastore.go @@ -199,7 +199,7 @@ func (s *Server) DeleteDataStore(ctx context.Context, req *sdcpb.DeleteDataStore return nil, err } - err = ds.Stop() + err = ds.Stop(s.ctx) if err != nil { log.Errorf("failed to stop datastore %s: %v", name, err) } diff --git a/pkg/server/server.go b/pkg/server/server.go index 98855291..3ae31ee8 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -79,9 +79,9 @@ func NewDatastoreMap() *DatastoreMap { datastores: map[string]*datastore.Datastore{}, } } -func (d *DatastoreMap) StopAll() { +func (d *DatastoreMap) StopAll(ctx context.Context) { for _, ds := range d.datastores { - ds.Stop() + ds.Stop(ctx) } } @@ -238,7 +238,7 @@ func (s *Server) ServeHTTP() { func (s *Server) Stop() { s.srv.Stop() - s.datastores.StopAll() + s.datastores.StopAll(s.ctx) s.cfn() } From 7aa5a8d62823e9ba1e6564b95c5a6e6b16ba6ea9 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 10 Dec 2025 10:51:26 +0100 Subject: [PATCH 24/44] fix leaflist import --- pkg/tree/parallelImporter.go | 30 +++++++++++++++++++++++++----- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/pkg/tree/parallelImporter.go b/pkg/tree/parallelImporter.go index 9aa2eca4..096f5332 100644 --- a/pkg/tree/parallelImporter.go +++ b/pkg/tree/parallelImporter.go @@ -47,6 +47,10 @@ func (s *sharedEntryAttributes) ImportConfig( } func importHandler(ctx context.Context, task importTask, submit func(importTask) error) error { + + elem := task.entry.PathName() + _ = elem + switch x := task.entry.GetSchema().GetSchema().(type) { case *sdcpb.SchemaElem_Container, nil: // keyed container: handle keys sequentially @@ -117,16 +121,32 @@ func importHandler(ctx context.Context, task importTask, submit func(importTask) return nil case *sdcpb.SchemaElem_Leaflist: - llm, loaded := task.leafListLock.LoadOrStore(task.entry.SdcpbPath().ToXPath(false), &sync.Mutex{}) - _ = loaded - llMutex := llm.(*sync.Mutex) + // for the leaflist, since in XML the leaf list elements are independet elements, we need to make + // sure that the first element is basically resetting the leaflist and all consecutive elemts are then + // added to the already resettet leaflist. + // strategy here is to create a mutex lock it and try to store it in the leafListLock map. + // if the mutex was then stored, we're the first goroutine and need to reset. If we get a different mutex back + // and the the loaded var is set to true, we should not reset the list and trxy to lock the returned mutex. + + // create a mutex and lock it + llMutex := &sync.Mutex{} llMutex.Lock() + + // try storing it or load it from leafListLock + llm, loaded := task.leafListLock.LoadOrStore(task.entry.SdcpbPath().ToXPath(false), llMutex) + + // if it was loaded, we need to lock the loaded mutex + if loaded { + llMutex = llm.(*sync.Mutex) + llMutex.Lock() + } defer llMutex.Unlock() var scalarArr *sdcpb.ScalarArray mustAdd := false - le := task.entry.GetLeafVariantEntries().GetByOwner(task.intentName) - if le != nil { + var le *LeafEntry + if loaded { + le = task.entry.GetLeafVariantEntries().GetByOwner(task.intentName) scalarArr = le.Value().GetLeaflistVal() } else { le = NewLeafEntry(nil, task.insertFlags, task.entry) From 319e2b7f90212e3c8c3c1b4537421d13e6dfea44 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 10 Dec 2025 11:05:34 +0100 Subject: [PATCH 25/44] enable ConfigSubscription --- pkg/datastore/target/gnmi/once.go | 11 ++++++++++- pkg/datastore/target/gnmi/stream.go | 10 ++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/pkg/datastore/target/gnmi/once.go b/pkg/datastore/target/gnmi/once.go index 83b7ab31..4afdc46d 100644 --- a/pkg/datastore/target/gnmi/once.go +++ b/pkg/datastore/target/gnmi/once.go @@ -5,6 +5,7 @@ import ( "time" "github.com/openconfig/gnmi/proto/gnmi" + "github.com/openconfig/gnmi/proto/gnmi_ext" gapi "github.com/openconfig/gnmic/pkg/api" "github.com/sdcio/data-server/pkg/config" "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" @@ -47,7 +48,15 @@ func (s *OnceSync) syncConfig() (*gnmi.SubscribeRequest, error) { gapi.EncodingCustom(utils.ParseGnmiEncoding(s.config.Encoding)), gapi.SubscriptionListModeONCE(), gapi.Subscription(subscriptionOpts...), - gapi.DataTypeCONFIG(), + gapi.Extension(&gnmi_ext.Extension{ + Ext: &gnmi_ext.Extension_ConfigSubscription{ + ConfigSubscription: &gnmi_ext.ConfigSubscription{ + Action: &gnmi_ext.ConfigSubscription_Start{ + Start: &gnmi_ext.ConfigSubscriptionStart{}, + }, + }, + }, + }), ) subReq, err := gapi.NewSubscribeRequest(opts...) if err != nil { diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index 22a21d3c..79de2f09 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -7,6 +7,7 @@ import ( "time" "github.com/openconfig/gnmi/proto/gnmi" + "github.com/openconfig/gnmi/proto/gnmi_ext" gapi "github.com/openconfig/gnmic/pkg/api" "github.com/sdcio/data-server/pkg/config" "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" @@ -65,6 +66,15 @@ func (s *StreamSync) syncConfig() (*gnmi.SubscribeRequest, error) { gapi.EncodingCustom(utils.ParseGnmiEncoding(s.config.Encoding)), gapi.SubscriptionListModeSTREAM(), gapi.Subscription(subscriptionOpts...), + gapi.Extension(&gnmi_ext.Extension{ + Ext: &gnmi_ext.Extension_ConfigSubscription{ + ConfigSubscription: &gnmi_ext.ConfigSubscription{ + Action: &gnmi_ext.ConfigSubscription_Start{ + Start: &gnmi_ext.ConfigSubscriptionStart{}, + }, + }, + }, + }), ) subReq, err := gapi.NewSubscribeRequest(opts...) if err != nil { From a83db31e22fcfffc1104670d222ff3e6ba39b9be Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 10 Dec 2025 11:53:20 +0100 Subject: [PATCH 26/44] logging --- pkg/datastore/target/gnmi/once.go | 5 ++++- pkg/datastore/target/gnmi/stream.go | 3 +++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/pkg/datastore/target/gnmi/once.go b/pkg/datastore/target/gnmi/once.go index 4afdc46d..6293c193 100644 --- a/pkg/datastore/target/gnmi/once.go +++ b/pkg/datastore/target/gnmi/once.go @@ -11,6 +11,7 @@ import ( "github.com/sdcio/data-server/pkg/datastore/target/gnmi/utils" "github.com/sdcio/data-server/pkg/datastore/target/types" "github.com/sdcio/data-server/pkg/pool" + "github.com/sdcio/logger" ) type OnceSync struct { @@ -91,7 +92,9 @@ func (s *OnceSync) Start() error { } func (s *OnceSync) Stop() error { - // TODO + log := logger.FromContext(s.ctx) + log.Info("Stopping Sync", "sync", s.config.Name) + s.cancel() return nil } diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index 79de2f09..f83e2c5b 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -17,6 +17,7 @@ import ( "github.com/sdcio/data-server/pkg/tree/importer/proto" treetypes "github.com/sdcio/data-server/pkg/tree/types" dsutils "github.com/sdcio/data-server/pkg/utils" + "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" log "github.com/sirupsen/logrus" ) @@ -84,6 +85,8 @@ func (s *StreamSync) syncConfig() (*gnmi.SubscribeRequest, error) { } func (s *StreamSync) Stop() error { + log := logger.FromContext(s.ctx) + log.Info("Stopping Sync", "sync", s.config.Name) s.cancel() return nil } From 6a37bb380a8cf6bf78357fe1d81e9facc6620188 Mon Sep 17 00:00:00 2001 From: steiler Date: Wed, 10 Dec 2025 17:00:08 +0100 Subject: [PATCH 27/44] adjustments after merger --- pkg/datastore/datastore_rpc.go | 4 ++ pkg/datastore/deviations.go | 3 +- pkg/datastore/sync.go | 6 ++- pkg/datastore/target/gnmi/get.go | 27 ++++++---- pkg/datastore/target/gnmi/gnmi.go | 81 +++++------------------------ pkg/datastore/target/gnmi/once.go | 6 +++ pkg/datastore/target/gnmi/stream.go | 52 ++++++++++-------- pkg/datastore/target/netconf/nc.go | 5 +- pkg/datastore/transaction_rpc.go | 6 ++- pkg/server/server.go | 5 +- pkg/tree/entry_test.go | 3 +- pkg/tree/parallelImporter.go | 10 ++-- pkg/utils/converter.go | 4 +- 13 files changed, 94 insertions(+), 118 deletions(-) diff --git a/pkg/datastore/datastore_rpc.go b/pkg/datastore/datastore_rpc.go index 3964df49..6804c855 100644 --- a/pkg/datastore/datastore_rpc.go +++ b/pkg/datastore/datastore_rpc.go @@ -21,6 +21,7 @@ import ( "sync" "time" + "github.com/sdcio/logger" logf "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "google.golang.org/grpc" @@ -233,6 +234,9 @@ func (d *Datastore) Stop(ctx context.Context) error { } func (d *Datastore) BlameConfig(ctx context.Context, includeDefaults bool) (*sdcpb.BlameTreeElement, error) { + log := logger.FromContext(ctx).WithName("BlameConfig") + ctx = logger.IntoContext(ctx, log) + // create a new TreeRoot by copying the syncTree d.syncTreeMutex.Lock() root, err := d.syncTree.DeepCopy(ctx) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index e7185f6c..47152273 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -38,8 +38,7 @@ func (d *Datastore) StopDeviationsWatch(peer string) { } func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) { - log := logf.FromContext(ctx) - log = log.WithValues("target-name", d.config.Name) + log := logf.FromContext(ctx).WithName("DeviationManager").WithValues("target-name", d.config.Name) ctx = logf.IntoContext(ctx, log) log.Info("starting deviation manager") diff --git a/pkg/datastore/sync.go b/pkg/datastore/sync.go index a4e99c16..e9cb3626 100644 --- a/pkg/datastore/sync.go +++ b/pkg/datastore/sync.go @@ -6,18 +6,20 @@ import ( "github.com/sdcio/data-server/pkg/tree" "github.com/sdcio/data-server/pkg/tree/importer" treetypes "github.com/sdcio/data-server/pkg/tree/types" + "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" - log "github.com/sirupsen/logrus" ) func (d *Datastore) ApplyToRunning(ctx context.Context, deletes []*sdcpb.Path, importer importer.ImportConfigAdapter) error { + log := logger.FromContext(ctx) + d.syncTreeMutex.Lock() defer d.syncTreeMutex.Unlock() for _, delete := range deletes { err := d.syncTree.DeleteBranch(ctx, delete, tree.RunningIntentName) if err != nil { - log.Warnf("failed deleting path (%s) from datastore sync tree: %v", delete.ToXPath(false), err) + log.V(logger.VWarn).Error(err, "failed deleting path from datastore sync tree", "path", delete.ToXPath(false)) continue } } diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index fd6ff77b..28ac8d1f 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -14,7 +14,6 @@ import ( dsutils "github.com/sdcio/data-server/pkg/utils" "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" - log "github.com/sirupsen/logrus" ) type GetSync struct { @@ -32,6 +31,10 @@ type GetSync struct { func NewGetSync(ctx context.Context, target GetTarget, c *config.SyncProtocol, runningStore types.RunningStore, schemaClient dsutils.SchemaClientBound) (*GetSync, error) { ctx, cancel := context.WithCancel(ctx) + // add the sync name to the logger values + log := logger.FromContext(ctx).WithValues("sync", c.Name) + ctx = logger.IntoContext(ctx, log) + paths := make([]*sdcpb.Path, 0, len(c.Paths)) // iterate referenced paths for _, p := range c.Paths { @@ -80,6 +83,9 @@ func (s *GetSync) Name() string { } func (s *GetSync) Start() error { + log := logger.FromContext(s.ctx) + log.Info("Starting Sync") + req, err := s.syncConfig() if err != nil { return err @@ -109,38 +115,40 @@ func (s *GetSync) Start() error { } func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { + log := logger.FromContext(s.ctx) + s.syncTreeMutex.Lock() defer s.syncTreeMutex.Unlock() // execute gnmi get resp, err := s.target.Get(s.ctx, req) if err != nil { - log.Errorf("sync error: %v", err) + log.Error(err, "error performing gnmi get from target") return } s.syncTree, err = s.runningStore.NewEmptyTree(s.ctx) if err != nil { - log.Errorf("sync newemptytree error: %v", err) + log.Error(err, "failure creating new synctree") return } // process Noifications err = s.processNotifications(resp.GetNotification()) if err != nil { - log.Errorf("sync process notifications error: %v", err) + log.Error(err, "failed processing notifications") return } result, err := s.syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) if err != nil { - log.Errorf("sync tree export error: %v", err) + log.Error(err, "failure exporting synctree") return } err = s.runningStore.ApplyToRunning(s.ctx, s.paths, proto.NewProtoTreeImporter(result)) if err != nil { - log.Errorf("sync import to running error: %v", err) + log.Error(err, "failure importing synctree export into running") return } } @@ -150,6 +158,7 @@ type GetTarget interface { } func (s *GetSync) processNotifications(n []*sdcpb.Notification) error { + log := logger.FromContext(s.ctx) ts := time.Now().Unix() uif := treetypes.NewUpdateInsertFlags() @@ -157,15 +166,15 @@ func (s *GetSync) processNotifications(n []*sdcpb.Notification) error { // updates upds, err := treetypes.ExpandAndConvertIntent(s.ctx, s.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, noti.Update, ts) if err != nil { - log.Errorf("sync expanding error: %v", err) + log.Error(err, "failure expanding and converting notification") continue } for idx2, upd := range upds { _ = idx2 - _, err = s.syncTree.AddUpdateRecursive(s.ctx, upd.Path(), upd, uif) + _, err = s.syncTree.AddUpdateRecursive(s.ctx, upd.GetPath(), upd.GetUpdate(), uif) if err != nil { - log.Errorf("sync process notifications error: %v, continuing", err) + log.Error(err, "failure adding update to synctree") } } } diff --git a/pkg/datastore/target/gnmi/gnmi.go b/pkg/datastore/target/gnmi/gnmi.go index bed5986f..861fd703 100644 --- a/pkg/datastore/target/gnmi/gnmi.go +++ b/pkg/datastore/target/gnmi/gnmi.go @@ -31,20 +31,16 @@ import ( "github.com/sdcio/data-server/pkg/pool" "github.com/sdcio/data-server/pkg/utils" dsutils "github.com/sdcio/data-server/pkg/utils" + logf "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" - log "github.com/sirupsen/logrus" + "google.golang.org/grpc" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/keepalive" - "google.golang.org/protobuf/encoding/prototext" "google.golang.org/protobuf/proto" ) -const ( - syncRetryWaitTime = 10 * time.Second -) - type gnmiTarget struct { target *gtarget.Target encodings map[gnmi.Encoding]struct{} @@ -116,6 +112,9 @@ func (t *gnmiTarget) Subscribe(ctx context.Context, req *gnmi.SubscribeRequest, } func (t *gnmiTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) { + log := logf.FromContext(ctx).WithName("Get") + ctx = logf.IntoContext(ctx, log) + var err error gnmiReq := &gnmi.GetRequest{ Path: make([]*gnmi.Path, 0, len(req.GetPath())), @@ -135,6 +134,9 @@ func (t *gnmiTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb if err != nil { return nil, err } + + log.V(logf.VDebug).Info("gnmi request", "raw-request", utils.FormatProtoJSON(gnmiReq)) + // execute the gnmi get gnmiRsp, err := t.target.Get(ctx, gnmiReq) if err != nil { @@ -145,13 +147,16 @@ func (t *gnmiTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb Notification: make([]*sdcpb.Notification, 0, len(gnmiRsp.GetNotification())), } for _, n := range gnmiRsp.GetNotification() { - sn := utils.ToSchemaNotification(n) + sn := utils.ToSchemaNotification(ctx, n) schemaRsp.Notification = append(schemaRsp.Notification, sn) } return schemaRsp, nil } func (t *gnmiTarget) Set(ctx context.Context, source targetTypes.TargetSource) (*sdcpb.SetDataResponse, error) { + log := logf.FromContext(ctx).WithName("Set") + ctx = logf.IntoContext(ctx, log) + var upds []*sdcpb.Update var deletes []*sdcpb.Path var err error @@ -213,7 +218,7 @@ func (t *gnmiTarget) Set(ctx context.Context, source targetTypes.TargetSource) ( setReq.Update = append(setReq.Update, gupd) } - log.Debugf("gnmi set request:\n%s", prototext.Format(setReq)) + log.V(logf.VDebug).Info("gnmi request", "raw-request", utils.FormatProtoJSON(setReq)) rsp, err := t.target.Set(ctx, setReq) if err != nil { @@ -276,66 +281,6 @@ func (t *gnmiTarget) AddSyncs(ctx context.Context, sps ...*config.SyncProtocol) return nil } -// func (t *gnmiTarget) Sync(octx context.Context, syncConfig *config.Sync) { -// if t != nil && t.target != nil && t.target.Config != nil { -// log.Infof("starting target %s sync", t.target.Config.Name) -// } -// var cancel context.CancelFunc -// var ctx context.Context -// var err error -// START: -// if cancel != nil { -// cancel() -// } -// ctx, cancel = context.WithCancel(octx) -// defer cancel() - -// // todo: do not run read subscriptions for GET -// for _, gnmiSync := range syncConfig.Config { -// switch gnmiSync.Mode { -// case "once": -// err = t.periodicSync(ctx, gnmiSync) -// case "get": -// err = t.getSync(ctx, gnmiSync) -// default: -// err = t.streamSync(ctx, gnmiSync) -// } -// if err != nil { -// log.Errorf("target=%s: failed to sync: %v", t.target.Config.Name, err) -// time.Sleep(syncRetryWaitTime) -// goto START -// } -// } - -// defer t.target.StopSubscriptions() - -// rspch, errCh := t.target.ReadSubscriptions() -// for { -// select { -// case <-ctx.Done(): -// if !errors.Is(ctx.Err(), context.Canceled) { -// log.Errorf("datastore %s sync stopped: %v", t.target.Config.Name, ctx.Err()) -// } -// return -// case rsp := <-rspch: -// switch r := rsp.Response.Response.(type) { -// case *gnmi.SubscribeResponse_Update: -// syncCh <- &SyncUpdate{ -// Store: rsp.SubscriptionName, -// Update: utils.ToSchemaNotification(r.Update), -// } -// } -// case err := <-errCh: -// if err.Err != nil { -// t.target.StopSubscriptions() -// log.Errorf("%s: sync subscription failed: %v", t.target.Config.Name, err) -// time.Sleep(time.Second) -// goto START -// } -// } -// } -// } - func (t *gnmiTarget) Close(ctx context.Context) error { if t == nil { return nil diff --git a/pkg/datastore/target/gnmi/once.go b/pkg/datastore/target/gnmi/once.go index 6293c193..7014f2f0 100644 --- a/pkg/datastore/target/gnmi/once.go +++ b/pkg/datastore/target/gnmi/once.go @@ -25,6 +25,10 @@ type OnceSync struct { func NewOnceSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore, vpoolFactory pool.VirtualPoolFactory) *OnceSync { ctx, cancel := context.WithCancel(ctx) + // add the sync name to the logger values + log := logger.FromContext(ctx).WithValues("sync", c.Name) + ctx = logger.IntoContext(ctx, log) + return &OnceSync{ config: c, target: target, @@ -67,6 +71,8 @@ func (s *OnceSync) syncConfig() (*gnmi.SubscribeRequest, error) { } func (s *OnceSync) Start() error { + log := logger.FromContext(s.ctx) + log.Info("Starting Sync") subReq, err := s.syncConfig() if err != nil { diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index f83e2c5b..8c75e253 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -19,7 +19,6 @@ import ( dsutils "github.com/sdcio/data-server/pkg/utils" "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" - log "github.com/sirupsen/logrus" ) type StreamSync struct { @@ -35,6 +34,10 @@ type StreamSync struct { func NewStreamSync(ctx context.Context, target SyncTarget, c *config.SyncProtocol, runningStore types.RunningStore, schemaClient dsutils.SchemaClientBound, vpoolFactory pool.VirtualPoolFactory) *StreamSync { ctx, cancel := context.WithCancel(ctx) + // add the sync name to the logger values + log := logger.FromContext(ctx).WithValues("sync", c.Name) + ctx = logger.IntoContext(ctx, log) + return &StreamSync{ config: c, target: target, @@ -86,7 +89,7 @@ func (s *StreamSync) syncConfig() (*gnmi.SubscribeRequest, error) { func (s *StreamSync) Stop() error { log := logger.FromContext(s.ctx) - log.Info("Stopping Sync", "sync", s.config.Name) + log.Info("Stopping Sync") s.cancel() return nil } @@ -96,6 +99,8 @@ func (s *StreamSync) Name() string { } func (s *StreamSync) Start() error { + log := logger.FromContext(s.ctx) + log.Info("Starting Sync") updChan := make(chan *NotificationData, 20) @@ -115,9 +120,10 @@ func (s *StreamSync) Start() error { } func (s *StreamSync) buildTreeSyncWithDatastore(cUS <-chan *NotificationData, syncResponse <-chan struct{}) { + log := logger.FromContext(s.ctx) syncTree, err := s.runningStore.NewEmptyTree(s.ctx) if err != nil { - log.Errorf("error creating new sync tree: %v", err) + log.Error(err, "failure creating new sync tree") return } syncTreeMutex := &sync.Mutex{} @@ -132,7 +138,7 @@ func (s *StreamSync) buildTreeSyncWithDatastore(cUS <-chan *NotificationData, sy for { select { case <-s.ctx.Done(): - log.Debugf(">>>>>>>>>>>>>>>>>> Sync %s context done", s.config.Name) + log.V(logger.VDebug).Info("stopping sync due to context done") return case noti, ok := <-cUS: if !ok { @@ -140,15 +146,14 @@ func (s *StreamSync) buildTreeSyncWithDatastore(cUS <-chan *NotificationData, sy } err := syncTree.AddUpdatesRecursive(s.ctx, noti.updates, uif) if err != nil { - log.Errorf("error adding to sync tree: %v", err) + log.Error(err, "failed adding update to synctree") } syncTree.AddExplicitDeletes(tree.RunningIntentName, tree.RunningValuesPrio, noti.deletes) case <-syncResponse: syncTree, err = s.syncToRunning(syncTree, syncTreeMutex, true) tickerActive = true if err != nil { - // TODO - log.Errorf("syncToRunning Error %v", err) + log.Error(err, "failed committing synctree to running") } case <-ticker.C: if !tickerActive { @@ -158,8 +163,7 @@ func (s *StreamSync) buildTreeSyncWithDatastore(cUS <-chan *NotificationData, sy log.Info("SyncRunning due to ticker") syncTree, err = s.syncToRunning(syncTree, syncTreeMutex, true) if err != nil { - // TODO - log.Errorf("syncToRunning Error %v", err) + log.Error(err, "failed committing synctree to running") } } } @@ -167,7 +171,8 @@ func (s *StreamSync) buildTreeSyncWithDatastore(cUS <-chan *NotificationData, sy func (s *StreamSync) gnmiSubscribe(subReq *gnmi.SubscribeRequest, updChan chan<- *NotificationData, syncResponse chan<- struct{}) { var err error - log.Infof("sync %q: subRequest: %v", s.config.Name, subReq) + log := logger.FromContext(s.ctx) + log.V(logger.VTrace).Info("starting gnmi subscription", "subscripton", subReq) respChan, errChan := s.target.Subscribe(s.ctx, subReq, s.config.Name) @@ -185,7 +190,7 @@ func (s *StreamSync) gnmiSubscribe(subReq *gnmi.SubscribeRequest, updChan chan<- return } if err != nil { - log.Errorf("Error stream sync: %s", err) + log.Error(err, "failed stream sync") return } case resp, ok := <-respChan: @@ -196,22 +201,22 @@ func (s *StreamSync) gnmiSubscribe(subReq *gnmi.SubscribeRequest, updChan chan<- case *gnmi.SubscribeResponse_Update: err = taskPool.Submit(newNotificationProcessorTask(resp.GetUpdate(), taskParams)) if err != nil { - log.Errorf("error processing Notifications: %s", err) + log.Error(err, "failure processing notifications") continue } case *gnmi.SubscribeResponse_SyncResponse: - log.Info("SyncResponse flag received") - log.Infof("Duration since sync Start: %s", time.Since(syncStartTime)) + log.Info("SyncResponse flag received", "initial sync duration", time.Since(syncStartTime)) syncResponse <- struct{}{} case *gnmi.SubscribeResponse_Error: - log.Error(r.Error.Message) + log.Error(nil, "gnmi subscription error", "error", r.Error.Message) } } } } func (s *StreamSync) syncToRunning(syncTree *tree.RootEntry, m *sync.Mutex, logCount bool) (*tree.RootEntry, error) { + log := logger.FromContext(s.ctx) m.Lock() defer m.Unlock() @@ -223,7 +228,7 @@ func (s *StreamSync) syncToRunning(syncTree *tree.RootEntry, m *sync.Mutex, logC // all good no data present return syncTree, nil } - log.Errorf("sync tree export error: %v", err) + log.Error(err, "sync tree export error: %v") return s.runningStore.NewEmptyTree(s.ctx) } // extract the explicit deletes @@ -231,18 +236,18 @@ func (s *StreamSync) syncToRunning(syncTree *tree.RootEntry, m *sync.Mutex, logC // set them to nil result.ExplicitDeletes = nil if logCount { - log.Infof("Syncing: %d elements, %d deletes ", result.GetRoot().CountTerminals(), len(result.GetExplicitDeletes())) + log.V(logger.VDebug).Info("syncing to running", "elements", result.GetRoot().CountTerminals(), "deletes", len(result.GetExplicitDeletes())) } - log.Infof("TreeExport to proto took: %s", time.Since(startTime)) + log.V(logger.VTrace).Info("synctree export done", "duration", time.Since(startTime)) startTime = time.Now() err = s.runningStore.ApplyToRunning(s.ctx, deletes, proto.NewProtoTreeImporter(result)) if err != nil { - log.Errorf("sync import to running error: %v", err) + log.Error(err, "failed importing sync to running") return s.runningStore.NewEmptyTree(s.ctx) } - log.Infof("Import to SyncTree took: %s", time.Since(startTime)) + log.V(logger.VTrace).Info("import to running tree done", "duration", time.Since(startTime)) return s.runningStore.NewEmptyTree(s.ctx) } @@ -251,7 +256,7 @@ type SyncTarget interface { } type NotificationData struct { - updates treetypes.UpdateSlice + updates []*treetypes.PathAndUpdate deletes *sdcpb.PathSet } @@ -280,11 +285,12 @@ func newNotificationProcessorTask(item *gnmi.Notification, params *NotificationP } func (t *notificationProcessorTask) Run(ctx context.Context, _ func(pool.Task) error) error { - sn := dsutils.ToSchemaNotification(t.item) + log := logger.FromContext(ctx) + sn := dsutils.ToSchemaNotification(ctx, t.item) // updates upds, err := treetypes.ExpandAndConvertIntent(ctx, t.params.schemaClientBound, tree.RunningIntentName, tree.RunningValuesPrio, sn.GetUpdate(), t.item.GetTimestamp()) if err != nil { - log.Errorf("sync expanding error: %v", err) + log.Error(err, "expansion and conversion failed") } deletes := sdcpb.NewPathSet() diff --git a/pkg/datastore/target/netconf/nc.go b/pkg/datastore/target/netconf/nc.go index 210ed796..1f77aa0f 100644 --- a/pkg/datastore/target/netconf/nc.go +++ b/pkg/datastore/target/netconf/nc.go @@ -96,6 +96,7 @@ func (t *ncTarget) GetImportAdapter(ctx context.Context, req *sdcpb.GetDataReque func (t *ncTarget) internalGet(ctx context.Context, req *sdcpb.GetDataRequest) (*nctypes.NetconfResponse, error) { log := logf.FromContext(ctx).WithName("Get") ctx = logf.IntoContext(ctx, log) + if !t.Status().IsConnected() { return nil, fmt.Errorf("%s", types.TargetStatusNotConnected) } @@ -122,7 +123,8 @@ func (t *ncTarget) internalGet(ctx context.Context, req *sdcpb.GetDataRequest) ( if err != nil { return nil, err } - log.V(logf.VDebug).Info("using netconf filter", "filter", filterDoc) + + log.V(logf.VDebug).Info("netconf get", "filter", filterDoc, "source", source) // execute the GetConfig rpc ncResponse, err := t.driver.GetConfig(source, filterDoc) @@ -138,7 +140,6 @@ func (t *ncTarget) internalGet(ctx context.Context, req *sdcpb.GetDataRequest) ( } func (t *ncTarget) Get(ctx context.Context, req *sdcpb.GetDataRequest) (*sdcpb.GetDataResponse, error) { - ncResponse, err := t.internalGet(ctx, req) if err != nil { return nil, err diff --git a/pkg/datastore/transaction_rpc.go b/pkg/datastore/transaction_rpc.go index c4454569..90c6a47d 100644 --- a/pkg/datastore/transaction_rpc.go +++ b/pkg/datastore/transaction_rpc.go @@ -13,6 +13,7 @@ import ( "github.com/sdcio/data-server/pkg/tree" treeproto "github.com/sdcio/data-server/pkg/tree/importer/proto" treetypes "github.com/sdcio/data-server/pkg/tree/types" + "github.com/sdcio/data-server/pkg/utils" logf "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "github.com/sdcio/sdc-protos/tree_persist" @@ -174,10 +175,11 @@ func (d *Datastore) LoadAllButRunningIntents(ctx context.Context, root *tree.Roo if excludeDeviations && intent.Deviation { continue } - intentNames = append(intentNames, intent.GetIntentName()) log.V(logf.VDebug).Info("adding intent to tree", "intent", intent.GetIntentName()) + log.V(logf.VTrace).Info("adding intent to tree", "intent", intent.GetIntentName(), "content", utils.FormatProtoJSON(intent)) + + intentNames = append(intentNames, intent.GetIntentName()) protoLoader := treeproto.NewProtoTreeImporter(intent) - log.V(logf.VTrace).Info("adding intent to tree", "intent", intent.String()) err := root.ImportConfig(ctx, nil, protoLoader, intent.GetIntentName(), intent.GetPriority(), treetypes.NewUpdateInsertFlags()) if err != nil { return nil, err diff --git a/pkg/server/server.go b/pkg/server/server.go index ffb4e8c0..2e48bf48 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -209,8 +209,7 @@ func New(ctx context.Context, c *config.Config) (*Server, error) { } func (s *Server) Serve(ctx context.Context) error { - log := logf.FromContext(ctx).WithValues("grpc-server-address", s.config.GRPCServer.Address) - ctx = logf.IntoContext(ctx, log) + log := logf.FromContext(ctx) l, err := net.Listen("tcp", s.config.GRPCServer.Address) if err != nil { return err @@ -222,7 +221,7 @@ func (s *Server) Serve(ctx context.Context) error { go s.startDataServer(ctx) - log.Info("starting server") + log.Info("starting server", "address", s.config.GRPCServer.Address) err = s.srv.Serve(l) if err != nil { return err diff --git a/pkg/tree/entry_test.go b/pkg/tree/entry_test.go index 661e954b..70dc0185 100644 --- a/pkg/tree/entry_test.go +++ b/pkg/tree/entry_test.go @@ -1926,7 +1926,8 @@ func Test_Validation_MultiKey_Pattern(t *testing.T) { t.Error(err) } - validationResult, _ := root.Validate(context.TODO(), validationConfig) + sharedPool := pool.NewSharedTaskPool(ctx, runtime.NumCPU()) + validationResult, _ := root.Validate(context.TODO(), validationConfig, sharedPool) // Should have no errors - all keys match their respective patterns if validationResult.HasErrors() { diff --git a/pkg/tree/parallelImporter.go b/pkg/tree/parallelImporter.go index 096f5332..83f7ee2d 100644 --- a/pkg/tree/parallelImporter.go +++ b/pkg/tree/parallelImporter.go @@ -89,7 +89,7 @@ func importHandler(ctx context.Context, task importTask, submit func(importTask) schem := task.entry.GetSchema().GetContainer() if schem != nil && schem.IsPresence { tv := &sdcpb.TypedValue{Value: &sdcpb.TypedValue_EmptyVal{EmptyVal: &emptypb.Empty{}}} - upd := types.NewUpdate(task.entry.SdcpbPath(), tv, task.intentPrio, task.intentName, 0) + upd := types.NewUpdate(task.entry, tv, task.intentPrio, task.intentName, 0) task.entry.GetLeafVariantEntries().Add(NewLeafEntry(upd, task.insertFlags, task.entry)) } return nil @@ -112,11 +112,11 @@ func importHandler(ctx context.Context, task importTask, submit func(importTask) return nil case *sdcpb.SchemaElem_Field: - tv, err := task.importerElement.GetTVValue(x.Field.GetType()) + tv, err := task.importerElement.GetTVValue(ctx, x.Field.GetType()) if err != nil { return err } - upd := types.NewUpdate(task.entry.SdcpbPath(), tv, task.intentPrio, task.intentName, 0) + upd := types.NewUpdate(task.entry, tv, task.intentPrio, task.intentName, 0) task.entry.GetLeafVariantEntries().Add(NewLeafEntry(upd, task.insertFlags, task.entry)) return nil @@ -154,7 +154,7 @@ func importHandler(ctx context.Context, task importTask, submit func(importTask) scalarArr = &sdcpb.ScalarArray{Element: []*sdcpb.TypedValue{}} } - tv, err := task.importerElement.GetTVValue(x.Leaflist.GetType()) + tv, err := task.importerElement.GetTVValue(ctx, x.Leaflist.GetType()) if err != nil { return err } @@ -162,7 +162,7 @@ func importHandler(ctx context.Context, task importTask, submit func(importTask) scalarArr.Element = append(scalarArr.Element, tv) tv = &sdcpb.TypedValue{Value: &sdcpb.TypedValue_LeaflistVal{LeaflistVal: scalarArr}} } - le.Update = types.NewUpdate(task.entry.SdcpbPath(), tv, task.intentPrio, task.intentName, 0) + le.Update = types.NewUpdate(task.entry, tv, task.intentPrio, task.intentName, 0) if mustAdd { task.entry.GetLeafVariantEntries().Add(le) } diff --git a/pkg/utils/converter.go b/pkg/utils/converter.go index 208fd412..79883197 100644 --- a/pkg/utils/converter.go +++ b/pkg/utils/converter.go @@ -10,6 +10,7 @@ import ( "strconv" "strings" + "github.com/sdcio/logger" logf "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "google.golang.org/protobuf/proto" @@ -48,6 +49,7 @@ func (c *Converter) ExpandUpdates(ctx context.Context, updates []*sdcpb.Update) // expandUpdate Expands the value, in case of json to single typed value updates func (c *Converter) ExpandUpdate(ctx context.Context, upd *sdcpb.Update) ([]*sdcpb.Update, error) { + log := logf.FromContext(ctx) upds := make([]*sdcpb.Update, 0) rsp, err := c.schemaClientBound.GetSchemaSdcpbPath(ctx, upd.GetPath()) if err != nil { @@ -126,7 +128,7 @@ func (c *Converter) ExpandUpdate(ctx context.Context, upd *sdcpb.Update) ([]*sdc var jsonValue []byte if upd.GetValue() == nil { - log.Errorf("Error - Path: %s - TypedValue == nil", upd.Path.ToXPath(false)) + log.V(logger.VError).Info("Value is nil", "Path", upd.Path.ToXPath(false)) return nil, nil } switch upd.GetValue().Value.(type) { From c88c1103a9be418c32748eb5bea8cd9676e35f64 Mon Sep 17 00:00:00 2001 From: steiler Date: Thu, 11 Dec 2025 12:48:32 +0100 Subject: [PATCH 28/44] adjust duration output --- pkg/datastore/target/gnmi/stream.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index 8c75e253..e3dff865 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -205,7 +205,7 @@ func (s *StreamSync) gnmiSubscribe(subReq *gnmi.SubscribeRequest, updChan chan<- continue } case *gnmi.SubscribeResponse_SyncResponse: - log.Info("SyncResponse flag received", "initial sync duration", time.Since(syncStartTime)) + log.Info("SyncResponse flag received", "initial sync duration", time.Since(syncStartTime).String()) syncResponse <- struct{}{} case *gnmi.SubscribeResponse_Error: @@ -239,7 +239,7 @@ func (s *StreamSync) syncToRunning(syncTree *tree.RootEntry, m *sync.Mutex, logC log.V(logger.VDebug).Info("syncing to running", "elements", result.GetRoot().CountTerminals(), "deletes", len(result.GetExplicitDeletes())) } - log.V(logger.VTrace).Info("synctree export done", "duration", time.Since(startTime)) + log.V(logger.VTrace).Info("synctree export done", "duration", time.Since(startTime).String()) startTime = time.Now() err = s.runningStore.ApplyToRunning(s.ctx, deletes, proto.NewProtoTreeImporter(result)) @@ -247,7 +247,7 @@ func (s *StreamSync) syncToRunning(syncTree *tree.RootEntry, m *sync.Mutex, logC log.Error(err, "failed importing sync to running") return s.runningStore.NewEmptyTree(s.ctx) } - log.V(logger.VTrace).Info("import to running tree done", "duration", time.Since(startTime)) + log.V(logger.VTrace).Info("import to running tree done", "duration", time.Since(startTime).String()) return s.runningStore.NewEmptyTree(s.ctx) } From 3a59051c76e9c50eeca7689f3b44064abfb01fcb Mon Sep 17 00:00:00 2001 From: steiler Date: Thu, 11 Dec 2025 12:48:54 +0100 Subject: [PATCH 29/44] fix issue with unlocking non locked mutex --- pkg/pool/pool.go | 45 +++++++++++++++++++++++++-------------------- 1 file changed, 25 insertions(+), 20 deletions(-) diff --git a/pkg/pool/pool.go b/pkg/pool/pool.go index 627e17f7..83abfca9 100644 --- a/pkg/pool/pool.go +++ b/pkg/pool/pool.go @@ -151,30 +151,35 @@ func (p *Pool[T]) Start(handler func(ctx context.Context, item T, submit func(T) // then close the queue so workers exit. Also handle ctx cancellation (force-close). go func() { // We'll wait on the condition variable instead of busy looping. - p.inflightMu.Lock() - defer p.inflightMu.Unlock() - for { - // If CloseForSubmit was called, wait for inflight==0 and queue empty then close queue. - if p.closedForSubmit.Load() { - for atomic.LoadInt64(&p.inflight) != 0 || p.tasks.Len() != 0 { - p.inflightC.Wait() + shouldForceClose := false + func() { + p.inflightMu.Lock() + defer p.inflightMu.Unlock() + for { + // If CloseForSubmit was called, wait for inflight==0 and queue empty then close queue. + if p.closedForSubmit.Load() { + for atomic.LoadInt64(&p.inflight) != 0 || p.tasks.Len() != 0 { + p.inflightC.Wait() + } + // Now safe to close queue: there is no inflight and no queued items + p.closeOnce.Do(func() { p.tasks.Close() }) + return } - // Now safe to close queue: there is no inflight and no queued items - p.closeOnce.Do(func() { p.tasks.Close() }) - return - } - // If ctx canceled -> force-close path. - if p.ctx.Err() != nil { - // we hold inflightMu; unlock before calling forceClose (which may broadcast/use locks). - p.inflightMu.Unlock() - p.forceClose() - return + // If ctx canceled -> force-close path. + if p.ctx.Err() != nil { + shouldForceClose = true + return + } + + // Wait to be signalled when either inflight changes or CloseForSubmit is called. + p.inflightC.Wait() + // loop and recheck conditions } + }() - // Wait to be signalled when either inflight changes or CloseForSubmit is called. - p.inflightC.Wait() - // loop and recheck conditions + if shouldForceClose { + p.forceClose() } }() } From db7dd42aa442f2c72e2f71c13dc5a1176677a89c Mon Sep 17 00:00:00 2001 From: steiler Date: Thu, 11 Dec 2025 12:49:43 +0100 Subject: [PATCH 30/44] update sdc-protos --- go.mod | 4 +--- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 4100ec84..ad62bc50 100644 --- a/go.mod +++ b/go.mod @@ -4,8 +4,6 @@ go 1.24.0 replace github.com/openconfig/goyang v1.6.0 => github.com/sdcio/goyang v1.6.2-2 -replace github.com/sdcio/sdc-protos => /home/mava/projects/sdc-protos - require ( github.com/AlekSi/pointer v1.2.0 github.com/beevik/etree v1.6.0 @@ -26,7 +24,7 @@ require ( github.com/sdcio/cache v0.0.38 github.com/sdcio/logger v0.0.2 github.com/sdcio/schema-server v0.0.33 - github.com/sdcio/sdc-protos v0.0.46 + github.com/sdcio/sdc-protos v0.0.47 github.com/sdcio/yang-parser v0.0.12 github.com/spf13/cobra v1.10.1 github.com/spf13/pflag v1.0.10 diff --git a/go.sum b/go.sum index 514536cf..580c513e 100644 --- a/go.sum +++ b/go.sum @@ -189,8 +189,8 @@ github.com/sdcio/logger v0.0.2 h1:yNRGFL5ixMhpVBlU/c+hs55JRrYWjFVmYxlWl1da5PU= github.com/sdcio/logger v0.0.2/go.mod h1:yWaOxK/G6vszjg8tKZiMqiEjlZouHsjFME4zSk+SAEA= github.com/sdcio/schema-server v0.0.33 h1:RTeQMIchynAPSQaf61CZBgYHCCpsyDLEAWJn+ZKImIo= github.com/sdcio/schema-server v0.0.33/go.mod h1:q8leN1KhRNTnnqf6yxvkDk5tFl6DAsHcl81usVgYpoI= -github.com/sdcio/sdc-protos v0.0.46 h1:NsDdQrcPXPktLI7ZHBR6mA82AEYaFjdBYiLzJcuF55k= -github.com/sdcio/sdc-protos v0.0.46/go.mod h1:Rm51ch3DgeZ9rImQ/A3lzsNKK5DPGgd1ONULPpgTw8A= +github.com/sdcio/sdc-protos v0.0.47 h1:1SD8Ifx7MRc6fqjuJIM1wQ2NGEwjYj+QWgHNgENJiT4= +github.com/sdcio/sdc-protos v0.0.47/go.mod h1:cVCYbMq+tu7EKXyJKFq3p2bsCwjvXbxwsyOoMGiQddw= github.com/sdcio/yang-parser v0.0.12 h1:RSSeqfAOIsJx5Lno5u4/ezyOmQYUduQ22rBfU/mtpJ4= github.com/sdcio/yang-parser v0.0.12/go.mod h1:CBqn3Miq85qmFVGHxHXHLluXkaIOsTzV06IM4DW6+D4= github.com/sirikothe/gotextfsm v1.0.1-0.20200816110946-6aa2cfd355e4 h1:FHUL2HofYJuslFOQdy/JjjP36zxqIpd/dcoiwLMIs7k= From a53d580d3d1e9ebbb643d7d3fc1c4b2d8eb2fd9c Mon Sep 17 00:00:00 2001 From: steiler Date: Fri, 12 Dec 2025 16:24:45 +0100 Subject: [PATCH 31/44] update --- pkg/datastore/target/gnmi/gnmi.go | 12 ++++++++++-- pkg/datastore/target/gnmi/stream.go | 2 ++ pkg/server/datastore.go | 16 +++++++++++++--- pkg/utils/converter.go | 13 ++++++++++--- 4 files changed, 35 insertions(+), 8 deletions(-) diff --git a/pkg/datastore/target/gnmi/gnmi.go b/pkg/datastore/target/gnmi/gnmi.go index 861fd703..9dccc65a 100644 --- a/pkg/datastore/target/gnmi/gnmi.go +++ b/pkg/datastore/target/gnmi/gnmi.go @@ -182,7 +182,9 @@ func (t *gnmiTarget) Set(ctx context.Context, source targetTypes.TargetSource) ( if err != nil { return nil, err } - upds = []*sdcpb.Update{{Path: &sdcpb.Path{}, Value: &sdcpb.TypedValue{Value: &sdcpb.TypedValue_JsonVal{JsonVal: jsonBytes}}}} + if len(jsonBytes) > 0 { + upds = []*sdcpb.Update{{Path: &sdcpb.Path{}, Value: &sdcpb.TypedValue{Value: &sdcpb.TypedValue_JsonVal{JsonVal: jsonBytes}}}} + } } case "json_ietf": @@ -195,7 +197,9 @@ func (t *gnmiTarget) Set(ctx context.Context, source targetTypes.TargetSource) ( if err != nil { return nil, err } - upds = []*sdcpb.Update{{Path: &sdcpb.Path{}, Value: &sdcpb.TypedValue{Value: &sdcpb.TypedValue_JsonIetfVal{JsonIetfVal: jsonBytes}}}} + if len(jsonBytes) > 0 { + upds = []*sdcpb.Update{{Path: &sdcpb.Path{}, Value: &sdcpb.TypedValue{Value: &sdcpb.TypedValue_JsonIetfVal{JsonIetfVal: jsonBytes}}}} + } } case "proto": @@ -205,6 +209,10 @@ func (t *gnmiTarget) Set(ctx context.Context, source targetTypes.TargetSource) ( } } + if len(deletes) == 0 && len(upds) == 0 { + return &sdcpb.SetDataResponse{}, nil + } + setReq := &gnmi.SetRequest{ Delete: make([]*gnmi.Path, 0, len(deletes)), Update: make([]*gnmi.Update, 0, len(upds)), diff --git a/pkg/datastore/target/gnmi/stream.go b/pkg/datastore/target/gnmi/stream.go index e3dff865..03da073a 100644 --- a/pkg/datastore/target/gnmi/stream.go +++ b/pkg/datastore/target/gnmi/stream.go @@ -222,6 +222,8 @@ func (s *StreamSync) syncToRunning(syncTree *tree.RootEntry, m *sync.Mutex, logC startTime := time.Now() result, err := syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) + log.V(logger.VTrace).Info("exported tree", "tree", result.String()) + if err != nil { if errors.Is(err, tree.ErrorIntentNotPresent) { log.Info("sync no config changes") diff --git a/pkg/server/datastore.go b/pkg/server/datastore.go index 7b5353ad..c1f3a72f 100644 --- a/pkg/server/datastore.go +++ b/pkg/server/datastore.go @@ -88,13 +88,16 @@ func (s *Server) CreateDataStore(ctx context.Context, req *sdcpb.CreateDataStore name := req.GetDatastoreName() lName := len(name) if lName == 0 { + log.V(logf.VError).Info("missing datastore name attribute") return nil, status.Error(codes.InvalidArgument, "missing datastore name attribute") } if lName > math.MaxUint16 { - return nil, status.Error(codes.InvalidArgument, "missing datastore name attribute") + log.V(logf.VError).Info("datastore name attribute too long") + return nil, status.Error(codes.InvalidArgument, "datastore name attribute too long") } if _, err := s.datastores.GetDataStore(name); err == nil { + log.V(logf.VError).Info("datastore already exists") return nil, status.Errorf(codes.InvalidArgument, "datastore %s already exists", name) } @@ -112,6 +115,7 @@ func (s *Server) CreateDataStore(ctx context.Context, req *sdcpb.CreateDataStore case sdcpb.CommitCandidate_COMMIT_RUNNING: commitDatastore = "running" default: + log.V(logf.VError).Info("unknown commitDatastore", "datastore", req.GetTarget().GetNetconfOpts().GetCommitCandidate()) return nil, fmt.Errorf("unknown commitDatastore: %v", req.GetTarget().GetNetconfOpts().GetCommitCandidate()) } sbi.NetconfOptions = &config.SBINetconfOptions{ @@ -126,7 +130,8 @@ func (s *Server) CreateDataStore(ctx context.Context, req *sdcpb.CreateDataStore Encoding: req.GetTarget().GetGnmiOpts().GetEncoding(), } default: - return nil, fmt.Errorf("unknowm protocol type %s", req.GetTarget().GetType()) + log.V(logf.VError).Info("unknowm targetconnection protocol type", "type", req.GetTarget().GetType()) + return nil, fmt.Errorf("unknowm targetconnection protocol type %s", req.GetTarget().GetType()) } if req.GetTarget().GetTls() != nil { @@ -185,13 +190,15 @@ func (s *Server) CreateDataStore(ctx context.Context, req *sdcpb.CreateDataStore gnSyncConfig.Encoding = pSync.GetTarget().GetGnmiOpts().GetEncoding() case "netconf": default: - return nil, status.Errorf(codes.InvalidArgument, "unknown sync protocol: %q", pSync.GetTarget().GetType()) + log.V(logf.VError).Info("unknown targetsync protocol type", "protocol", pSync.GetTarget().GetType()) + return nil, status.Errorf(codes.InvalidArgument, "unknown targetsync protocol: %q", pSync.GetTarget().GetType()) } dsConfig.Sync.Config = append(dsConfig.Sync.Config, gnSyncConfig) } } err := dsConfig.ValidateSetDefaults() if err != nil { + log.Error(err, "invalid datastore config") return nil, status.Errorf(codes.InvalidArgument, "invalid datastore config: %v", err) } ds, err := datastore.New( @@ -201,12 +208,15 @@ func (s *Server) CreateDataStore(ctx context.Context, req *sdcpb.CreateDataStore s.cacheClient, s.gnmiOpts...) if err != nil { + log.Error(err, "failed creating new datastore") return nil, err } err = s.datastores.AddDatastore(ds) if err != nil { + log.Error(err, "failed adding new datastore") return nil, err } + log.Info("datastore created successfully") return &sdcpb.CreateDataStoreResponse{}, nil } diff --git a/pkg/utils/converter.go b/pkg/utils/converter.go index 79883197..5170ff2b 100644 --- a/pkg/utils/converter.go +++ b/pkg/utils/converter.go @@ -56,6 +56,9 @@ func (c *Converter) ExpandUpdate(ctx context.Context, upd *sdcpb.Update) ([]*sdc return nil, err } + p := upd.GetPath() + _ = p + // skip state if rsp.GetSchema().IsState() { return nil, nil @@ -144,9 +147,13 @@ func (c *Converter) ExpandUpdate(ctx context.Context, upd *sdcpb.Update) ([]*sdc if err != nil { return nil, err } - switch v := v.(type) { - case string: - upd.Value = &sdcpb.TypedValue{Value: &sdcpb.TypedValue_StringVal{StringVal: v}} + upd.Value = &sdcpb.TypedValue{Value: &sdcpb.TypedValue_StringVal{StringVal: string(jsonValue)}} + } + + if upd.Value.GetStringVal() != "" && rsp.Field.GetType().GetTypeName() != "string" { + upd.Value, err = Convert(ctx, upd.GetValue().GetStringVal(), rsp.Field.GetType()) + if err != nil { + return nil, err } } From 80a338076b0a30e64b7fdbaea7413f3f6a9986a6 Mon Sep 17 00:00:00 2001 From: steiler Date: Mon, 15 Dec 2025 10:55:23 +0100 Subject: [PATCH 32/44] fix netconf sync removed config from device was not removed from running config, due to the path attribute of the sync not being maintained --- pkg/datastore/target/gnmi/get.go | 6 +++--- pkg/datastore/target/netconf/sync.go | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index 28ac8d1f..d5116f80 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -32,7 +32,7 @@ func NewGetSync(ctx context.Context, target GetTarget, c *config.SyncProtocol, r ctx, cancel := context.WithCancel(ctx) // add the sync name to the logger values - log := logger.FromContext(ctx).WithValues("sync", c.Name) + log := logger.FromContext(ctx).WithValues("sync", c.Name).WithValues("type", "GET") ctx = logger.IntoContext(ctx, log) paths := make([]*sdcpb.Path, 0, len(c.Paths)) @@ -116,6 +116,7 @@ func (s *GetSync) Start() error { func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { log := logger.FromContext(s.ctx) + log.V(logger.VDebug).Info("syncing") s.syncTreeMutex.Lock() defer s.syncTreeMutex.Unlock() @@ -159,12 +160,11 @@ type GetTarget interface { func (s *GetSync) processNotifications(n []*sdcpb.Notification) error { log := logger.FromContext(s.ctx) - ts := time.Now().Unix() uif := treetypes.NewUpdateInsertFlags() for _, noti := range n { // updates - upds, err := treetypes.ExpandAndConvertIntent(s.ctx, s.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, noti.Update, ts) + upds, err := treetypes.ExpandAndConvertIntent(s.ctx, s.schemaClient, tree.RunningIntentName, tree.RunningValuesPrio, noti.Update, noti.GetTimestamp()) if err != nil { log.Error(err, "failure expanding and converting notification") continue diff --git a/pkg/datastore/target/netconf/sync.go b/pkg/datastore/target/netconf/sync.go index 98b0ddea..9bcb1526 100644 --- a/pkg/datastore/target/netconf/sync.go +++ b/pkg/datastore/target/netconf/sync.go @@ -52,6 +52,7 @@ func NewNetconfSyncImpl(ctx context.Context, targetName string, target GetXMLImp targetName: targetName, runningStore: runningStore, target: target, + paths: paths, }, nil } From d6b6b0a48b000a66283128169748823683184e02 Mon Sep 17 00:00:00 2001 From: steiler Date: Tue, 16 Dec 2025 12:43:56 +0100 Subject: [PATCH 33/44] update --- pkg/datastore/deviations.go | 2 ++ pkg/datastore/target/gnmi/get.go | 6 ++++++ 2 files changed, 8 insertions(+) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 47152273..831ef738 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -51,7 +51,9 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) case <-ctx.Done(): return case <-ticker.C: + log.Info("starting deviation calc run - pre-lock") d.m.RLock() + log.Info("starting deviation calc run - post-lock") deviationClientNames := make([]string, 0, len(d.deviationClients)) deviationClients := map[string]sdcpb.DataServer_WatchDeviationsServer{} for clientIdentifier, devStream := range d.deviationClients { diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index d5116f80..43d1fe9e 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -14,6 +14,7 @@ import ( dsutils "github.com/sdcio/data-server/pkg/utils" "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" + "google.golang.org/protobuf/encoding/protojson" ) type GetSync struct { @@ -147,6 +148,11 @@ func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { return } + if log.V(logger.VTrace).Enabled() { + data, _ := protojson.Marshal(result) + log.V(logger.VTrace).Info("sync content", "data", string(data)) + } + err = s.runningStore.ApplyToRunning(s.ctx, s.paths, proto.NewProtoTreeImporter(result)) if err != nil { log.Error(err, "failure importing synctree export into running") From 016a106efc884ab44211f0877c1cc83adb2125bf Mon Sep 17 00:00:00 2001 From: steiler Date: Thu, 18 Dec 2025 13:06:58 +0100 Subject: [PATCH 34/44] fix locking issue the synctree lock was not properly returned in some error cases --- pkg/datastore/datastore_rpc.go | 35 +++++++++++++------------------- pkg/datastore/deviations.go | 33 +++++++++++++++++++++--------- pkg/datastore/transaction_rpc.go | 31 ++++++++++++++++------------ pkg/server/datastore.go | 15 +++++++------- 4 files changed, 63 insertions(+), 51 deletions(-) diff --git a/pkg/datastore/datastore_rpc.go b/pkg/datastore/datastore_rpc.go index 6804c855..b04e03be 100644 --- a/pkg/datastore/datastore_rpc.go +++ b/pkg/datastore/datastore_rpc.go @@ -50,6 +50,7 @@ type Datastore struct { // schemaClient sdcpb.SchemaServerClient schemaClient schemaClient.SchemaClientBound + ctx context.Context // stop cancel func cfn context.CancelFunc @@ -57,10 +58,6 @@ type Datastore struct { m *sync.RWMutex deviationClients map[string]sdcpb.DataServer_WatchDeviationsServer - // per path intent deviations (no unhandled) - md *sync.RWMutex - currentIntentsDeviations map[string][]*sdcpb.WatchDeviationResponse - // datastore mutex locks the whole datasore for further set operations dmutex *sync.Mutex @@ -71,9 +68,6 @@ type Datastore struct { syncTree *tree.RootEntry syncTreeMutex *sync.RWMutex - // owned by sync - syncTreeCandidate *tree.RootEntry - taskPool *pool.SharedTaskPool } @@ -105,24 +99,23 @@ func New(ctx context.Context, c *config.DatastoreConfig, sc schema.Client, cc ca ccb := cache.NewCacheClientBound(c.Name, cc) + ctx, cancel := context.WithCancel(ctx) + ds := &Datastore{ - config: c, - schemaClient: scb, - cacheClient: ccb, - m: &sync.RWMutex{}, - md: &sync.RWMutex{}, - dmutex: &sync.Mutex{}, - deviationClients: make(map[string]sdcpb.DataServer_WatchDeviationsServer), - currentIntentsDeviations: make(map[string][]*sdcpb.WatchDeviationResponse), - syncTree: syncTreeRoot, - syncTreeMutex: &sync.RWMutex{}, - taskPool: pool.NewSharedTaskPool(ctx, runtime.NumCPU()), + config: c, + schemaClient: scb, + ctx: ctx, + cfn: cancel, + cacheClient: ccb, + m: &sync.RWMutex{}, + dmutex: &sync.Mutex{}, + deviationClients: make(map[string]sdcpb.DataServer_WatchDeviationsServer), + syncTree: syncTreeRoot, + syncTreeMutex: &sync.RWMutex{}, + taskPool: pool.NewSharedTaskPool(ctx, runtime.NumCPU()), } ds.transactionManager = types.NewTransactionManager(NewDatastoreRollbackAdapter(ds)) - ctx, cancel := context.WithCancel(ctx) - ds.cfn = cancel - // create cache instance if needed // this is a blocking call ds.initCache(ctx) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 831ef738..c0bd756a 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -27,14 +27,18 @@ func (d *Datastore) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdc pName := p.Addr.String() d.deviationClients[pName] = stream + + log := logf.FromContext(d.ctx) + log.Info("new deviation client", "client", pName) return nil } func (d *Datastore) StopDeviationsWatch(peer string) { + log := logf.FromContext(d.ctx) + log.Info("deviation client removed", "peer", peer) d.m.Lock() defer d.m.Unlock() delete(d.deviationClients, peer) - logf.DefaultLogger.V(logf.VDebug).Info("deviation watcher removed", "peer", peer) } func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) { @@ -45,19 +49,25 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) ticker := time.NewTicker(c.Interval) defer func() { ticker.Stop() + log.Info("deviation manager stopped") }() for { select { case <-ctx.Done(): + log.Info("datastore context done, stopping deviation manager") return case <-ticker.C: - log.Info("starting deviation calc run - pre-lock") + log.V(logf.VDebug).Info("deviation calc run - start") d.m.RLock() - log.Info("starting deviation calc run - post-lock") deviationClientNames := make([]string, 0, len(d.deviationClients)) deviationClients := map[string]sdcpb.DataServer_WatchDeviationsServer{} for clientIdentifier, devStream := range d.deviationClients { deviationClients[clientIdentifier] = devStream + if devStream.Context().Err() != nil { + log.V(logf.VWarn).Error(devStream.Context().Err(), "removing deviation client", "client", clientIdentifier) + delete(deviationClients, clientIdentifier) + continue + } deviationClientNames = append(deviationClientNames, clientIdentifier) } d.m.RUnlock() @@ -75,11 +85,13 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) log.Error(err, "error sending deviation", "client-identifier", clientIdentifier) } } - deviationChan, err := d.calculateDeviations(ctx) + start := time.Now() + deviationChan, err := d.calculateDeviations() if err != nil { log.Error(err, "failed to calculate deviations") continue } + log.V(logf.VDebug).Info("calculated deviations", "duration", time.Since(start)) d.SendDeviations(ctx, deviationChan, deviationClients) for clientIdentifier, dc := range deviationClients { err := dc.Send(&sdcpb.WatchDeviationResponse{ @@ -90,6 +102,7 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) log.Error(err, "error sending deviation", "client-identifier", clientIdentifier) } } + log.V(logf.VDebug).Info("deviation calc run - finished") } } } @@ -139,17 +152,17 @@ type DeviationEntry interface { ExpectedValue() *sdcpb.TypedValue } -func (d *Datastore) calculateDeviations(ctx context.Context) (<-chan *treetypes.DeviationEntry, error) { +func (d *Datastore) calculateDeviations() (<-chan *treetypes.DeviationEntry, error) { deviationChan := make(chan *treetypes.DeviationEntry, 10) d.syncTreeMutex.RLock() - deviationTree, err := d.syncTree.DeepCopy(ctx) + deviationTree, err := d.syncTree.DeepCopy(d.ctx) + d.syncTreeMutex.RUnlock() if err != nil { return nil, err } - d.syncTreeMutex.RUnlock() - addedIntentNames, err := d.LoadAllButRunningIntents(ctx, deviationTree, true) + addedIntentNames, err := d.LoadAllButRunningIntents(d.ctx, deviationTree, true) if err != nil { return nil, err } @@ -159,13 +172,13 @@ func (d *Datastore) calculateDeviations(ctx context.Context) (<-chan *treetypes. deviationChan <- treetypes.NewDeviationEntry(n, treetypes.DeviationReasonIntentExists, nil) } - err = deviationTree.FinishInsertionPhase(ctx) + err = deviationTree.FinishInsertionPhase(d.ctx) if err != nil { return nil, err } go func() { - deviationTree.GetDeviations(ctx, deviationChan) + deviationTree.GetDeviations(d.ctx, deviationChan) close(deviationChan) }() diff --git a/pkg/datastore/transaction_rpc.go b/pkg/datastore/transaction_rpc.go index 90c6a47d..4b4ee87c 100644 --- a/pkg/datastore/transaction_rpc.go +++ b/pkg/datastore/transaction_rpc.go @@ -397,24 +397,29 @@ func (d *Datastore) lowlevelTransactionSet(ctx context.Context, transaction *typ func (d *Datastore) writeBackSyncTree(ctx context.Context, updates tree.LeafVariantSlice, deletes treetypes.DeleteEntriesList) error { runningUpdates := updates.ToUpdateSlice().CopyWithNewOwnerAndPrio(tree.RunningIntentName, tree.RunningValuesPrio) - // lock the syncTree - d.syncTreeMutex.Lock() - - // perform deletes - err := d.syncTree.DeleteBranchPaths(ctx, deletes, tree.RunningIntentName) - if err != nil { - return err - } + // wrap the lock in an anonymous function to be able to utilize defer for the unlock + err := func() error { + // lock the syncTree + d.syncTreeMutex.Lock() + defer d.syncTreeMutex.Unlock() + + // perform deletes + err := d.syncTree.DeleteBranchPaths(ctx, deletes, tree.RunningIntentName) + if err != nil { + return err + } - // add the calculated updates to the tree, as running with adjusted prio and owner - err = d.syncTree.AddUpdatesRecursive(ctx, runningUpdates, treetypes.NewUpdateInsertFlags()) + // add the calculated updates to the tree, as running with adjusted prio and owner + err = d.syncTree.AddUpdatesRecursive(ctx, runningUpdates, treetypes.NewUpdateInsertFlags()) + if err != nil { + return err + } + return nil + }() if err != nil { return err } - // release the syncTree lock - d.syncTreeMutex.Unlock() - // export the synctree newRunningIntent, err := d.syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) if err != nil && err != tree.ErrorIntentNotPresent { diff --git a/pkg/server/datastore.go b/pkg/server/datastore.go index c1f3a72f..97d8e45f 100644 --- a/pkg/server/datastore.go +++ b/pkg/server/datastore.go @@ -253,20 +253,20 @@ func (s *Server) DeleteDataStore(ctx context.Context, req *sdcpb.DeleteDataStore func (s *Server) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdcpb.DataServer_WatchDeviationsServer) error { ctx := stream.Context() p, ok := peer.FromContext(ctx) - log := logf.FromContext(ctx).WithName("WatchDeviations").WithValues("peer", p.String()) + var peerName string + if ok { + peerName = p.Addr.String() + } + log := logf.FromContext(ctx).WithName("WatchDeviations").WithValues("peer", peerName) ctx = logf.IntoContext(ctx, log) log.V(logf.VDebug).Info("received request", "raw-request", utils.FormatProtoJSON(req)) - peerInfo, ok := peer.FromContext(ctx) if !ok { return status.Errorf(codes.InvalidArgument, "missing peer info") } - if req.GetName() == nil { - return status.Errorf(codes.InvalidArgument, "missing datastore name") - } - if len(req.GetName()) == 0 { + if req.GetName() == nil || len(req.GetName()) == 0 { return status.Errorf(codes.InvalidArgument, "missing datastore name") } @@ -279,9 +279,10 @@ func (s *Server) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdcpb. err = ds.WatchDeviations(req, stream) if err != nil { log.Error(err, "failed to watch deviations") + return err } <-stream.Context().Done() - ds.StopDeviationsWatch(peerInfo.Addr.String()) + ds.StopDeviationsWatch(peerName) return nil } From fd5671835ac1d5e8fe1c2beeff5be0cfafdb597d Mon Sep 17 00:00:00 2001 From: steiler Date: Thu, 18 Dec 2025 22:29:22 +0100 Subject: [PATCH 35/44] update --- pkg/datastore/deviations.go | 12 ++++---- pkg/pool/pool.go | 48 ++++++++++-------------------- pkg/pool/virtual_pool.go | 55 ++++++++++++++++++----------------- pkg/pool/virtual_pool_test.go | 44 ++++++++++++++++++++++++++++ 4 files changed, 94 insertions(+), 65 deletions(-) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index c0bd756a..21ab71d3 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -94,6 +94,9 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) log.V(logf.VDebug).Info("calculated deviations", "duration", time.Since(start)) d.SendDeviations(ctx, deviationChan, deviationClients) for clientIdentifier, dc := range deviationClients { + if dc.Context().Err() != nil { + continue + } err := dc.Send(&sdcpb.WatchDeviationResponse{ Name: d.config.Name, Event: sdcpb.DeviationEvent_END, @@ -109,11 +112,8 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) func (d *Datastore) SendDeviations(ctx context.Context, ch <-chan *treetypes.DeviationEntry, deviationClients map[string]sdcpb.DataServer_WatchDeviationsServer) { log := logf.FromContext(ctx) - wg := &sync.WaitGroup{} vPool := d.taskPool.NewVirtualPool(pool.VirtualTolerant, 1) - for de := range ch { - wg.Add(1) vPool.SubmitFunc(func(ctx context.Context, _ func(pool.Task) error) error { for clientIdentifier, dc := range deviationClients { if dc.Context().Err() != nil { @@ -137,11 +137,13 @@ func (d *Datastore) SendDeviations(ctx context.Context, ch <-chan *treetypes.Dev log.Error(err, "error sending deviation", "client-identifier", clientIdentifier) } } - wg.Done() return nil }) } - wg.Wait() + vPool.CloseForSubmit() + log.Info("waiting for tasks in pool to finish") + vPool.Wait() + log.Info("pool finished") } type DeviationEntry interface { diff --git a/pkg/pool/pool.go b/pkg/pool/pool.go index 83abfca9..a3fc67d0 100644 --- a/pkg/pool/pool.go +++ b/pkg/pool/pool.go @@ -115,14 +115,19 @@ func (p *Pool[T]) Start(handler func(ctx context.Context, item T, submit func(T) return } - // If ctx canceled, we must still decrement inflight for this item and skip handler. - if p.ctx.Err() != nil { - p.addInflight(-1) - continue - } - // run handler (handler may call p.Submit) - if err := handler(workerCtx, item, func(it T) error { return p.Submit(it) }); err != nil { + var err error + func() { + defer func() { + if r := recover(); r != nil { + p.addInflight(-1) + panic(r) + } + }() + err = handler(workerCtx, item, func(it T) error { return p.Submit(it) }) + }() + + if err != nil { // store first error safely (allocate on heap) ep := new(error) *ep = err @@ -207,35 +212,12 @@ func (p *Pool[T]) Wait() error { return nil } -// forceClose performs a one-time forced shutdown: cancel context, close queue and -// subtract any queued-but-unprocessed items from inflight so waiters don't block forever. +// forceClose performs a one-time forced shutdown: cancel context, close queue. +// We do NOT subtract queued items from inflight; we rely on workers draining the queue +// (even if closed) and processing items with a canceled context to ensure proper cleanup. func (p *Pool[T]) forceClose() { p.cancel() p.closeOnce.Do(func() { - // first capture queued items - queued := p.tasks.Len() - if queued > 0 { - // reduce inflight by queued. Use atomic and then broadcast condition. - // Ensure we don't go negative. - for { - cur := atomic.LoadInt64(&p.inflight) - // clamp - var toSub int64 = int64(queued) - if toSub > cur { - toSub = cur - } - if toSub == 0 { - break - } - if atomic.CompareAndSwapInt64(&p.inflight, cur, cur-toSub) { - p.inflightMu.Lock() - p.inflightC.Broadcast() - p.inflightMu.Unlock() - break - } - // retry on CAS failure - } - } // now close the queue to wake Get() waiters p.tasks.Close() }) diff --git a/pkg/pool/virtual_pool.go b/pkg/pool/virtual_pool.go index 822f5286..e209ae08 100644 --- a/pkg/pool/virtual_pool.go +++ b/pkg/pool/virtual_pool.go @@ -194,16 +194,7 @@ func (vt *virtualTask) Run(ctx context.Context, submit func(Task) error) error { // If virtual is closed due to fail-fast, skip executing the task. if vt.vp.isFailed() { // decrement inflight for skipped task and possibly close collector/done - if remaining := atomic.AddInt64(&vt.vp.inflight, -1); remaining == 0 && vt.vp.closed.Load() { - vt.vp.collectorOnce.Do(func() { - if vt.vp.ec != nil { - vt.vp.ec.close() - } - }) - vt.vp.waitOnce.Do(func() { - close(vt.vp.done) - }) - } + vt.vp.decrementInflight() return nil } @@ -214,6 +205,11 @@ func (vt *virtualTask) Run(ctx context.Context, submit func(Task) error) error { return vt.vp.submitInternal(t) } + // Ensure we decrement inflight even if panic occurs + defer func() { + vt.vp.decrementInflight() + }() + // Execute the actual task. err := vt.task.Run(ctx, submitWrapper) @@ -232,19 +228,6 @@ func (vt *virtualTask) Run(ctx context.Context, submit func(Task) error) error { } // return nil to shared pool so shared pool doesn't abort - // decrement inflight and possibly close collector if virtual is closed and no more inflight - if remaining := atomic.AddInt64(&vt.vp.inflight, -1); remaining == 0 && vt.vp.closed.Load() { - // close collector once - vt.vp.collectorOnce.Do(func() { - if vt.vp.ec != nil { - vt.vp.ec.close() - } - }) - // signal Wait() callers that virtual is drained - vt.vp.waitOnce.Do(func() { - close(vt.vp.done) - }) - } return nil } @@ -253,25 +236,43 @@ func (vt *virtualTask) Run(ctx context.Context, submit func(Task) error) error { // Submit enqueues a Task into this virtual pool. // It wraps the Task into a virtualTask that remembers the virtual identity. func (v *VirtualPool) Submit(t Task) error { + // Increment inflight BEFORE checking closed to avoid race where CloseForSubmit + // sees inflight=0 and closes the pool while we are in the middle of submitting. + atomic.AddInt64(&v.inflight, 1) + // fast-fail if virtual pool closed for submit if v.closed.Load() { + v.decrementInflight() return ErrVirtualPoolClosed } // If already failed (fail-fast), disallow further submissions. if v.isFailed() { + v.decrementInflight() return ErrVirtualPoolClosed } - // increment per-virtual inflight (will be decremented by worker after run) - atomic.AddInt64(&v.inflight, 1) + vt := &virtualTask{vp: v, task: t} if err := v.parent.submitWrapped(vt); err != nil { // submission failed: revert inflight - atomic.AddInt64(&v.inflight, -1) + v.decrementInflight() return err } return nil } +func (v *VirtualPool) decrementInflight() { + if remaining := atomic.AddInt64(&v.inflight, -1); remaining == 0 && v.closed.Load() { + v.collectorOnce.Do(func() { + if v.ec != nil { + v.ec.close() + } + }) + v.waitOnce.Do(func() { + close(v.done) + }) + } +} + // SubmitFunc convenience to submit a TaskFunc. func (v *VirtualPool) SubmitFunc(f TaskFunc) error { return v.Submit(f) } @@ -290,7 +291,7 @@ func (v *VirtualPool) submitInternal(t Task) error { vt := &virtualTask{vp: v, task: t} if err := v.parent.submitWrapped(vt); err != nil { // submission failed: revert inflight - atomic.AddInt64(&v.inflight, -1) + v.decrementInflight() return err } return nil diff --git a/pkg/pool/virtual_pool_test.go b/pkg/pool/virtual_pool_test.go index de0081d6..086abc2c 100644 --- a/pkg/pool/virtual_pool_test.go +++ b/pkg/pool/virtual_pool_test.go @@ -16,6 +16,7 @@ package pool import ( "context" + "sync" "sync/atomic" "testing" "time" @@ -202,3 +203,46 @@ func TestVirtualPool_Wait_FailFast(t *testing.T) { t.Fatalf("expected at least one task executed, got %d", got) } } + +func TestVirtualPool_CancellationHang(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + sp := NewSharedTaskPool(ctx, 1) + vp := sp.NewVirtualPool(VirtualFailFast, 0) + + var wg sync.WaitGroup + wg.Add(2) + + // Task 1: blocks until cancelled + vp.SubmitFunc(func(ctx context.Context, submit func(Task) error) error { + defer wg.Done() + <-ctx.Done() + return nil + }) + + // Task 2: should run even if cancelled (to decrement inflight) + vp.SubmitFunc(func(ctx context.Context, submit func(Task) error) error { + defer wg.Done() + return nil + }) + + // Give time for Task 1 to start and Task 2 to be queued + time.Sleep(100 * time.Millisecond) + + // Cancel the pool + cancel() + + vp.CloseForSubmit() + + done := make(chan struct{}) + go func() { + vp.Wait() + close(done) + }() + + select { + case <-done: + t.Log("Wait returned successfully") + case <-time.After(1 * time.Second): + t.Fatal("Wait timed out - likely hung due to dropped task or negative inflight") + } +} From f12b2a95240cc7fa9ba421d89c0ebbf2c636ff68 Mon Sep 17 00:00:00 2001 From: steiler Date: Thu, 18 Dec 2025 22:31:48 +0100 Subject: [PATCH 36/44] update --- pkg/datastore/deviations.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 21ab71d3..4ef48e95 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -2,7 +2,6 @@ package datastore import ( "context" - "sync" "time" "github.com/sdcio/data-server/pkg/config" From 754ddf91ca186f40b4865594367e15c44e84006c Mon Sep 17 00:00:00 2001 From: steiler Date: Fri, 19 Dec 2025 11:04:31 +0100 Subject: [PATCH 37/44] fix error --- pkg/tree/sharedEntryAttributes.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/pkg/tree/sharedEntryAttributes.go b/pkg/tree/sharedEntryAttributes.go index 81b355ce..c38423d8 100644 --- a/pkg/tree/sharedEntryAttributes.go +++ b/pkg/tree/sharedEntryAttributes.go @@ -1115,8 +1115,14 @@ func (s *sharedEntryAttributes) validateMinMaxElements(resultChan chan<- *types. ownersSet := map[string]struct{}{} for _, child := range childs { childAttributes := child.GetChilds(DescendMethodActiveChilds) - owner := childAttributes[contSchema.GetKeys()[0].GetName()].GetHighestPrecedence(nil, false, false, false)[0].Update.Owner() - ownersSet[owner] = struct{}{} + keyName := contSchema.GetKeys()[0].GetName() + if keyAttr, ok := childAttributes[keyName]; ok { + highestPrec := keyAttr.GetHighestPrecedence(nil, false, false, false) + if len(highestPrec) > 0 { + owner := highestPrec[0].Update.Owner() + ownersSet[owner] = struct{}{} + } + } } // dedup the owners owners := []string{} From b8de7f4f11e3180168552c5a44540db02902ad63 Mon Sep 17 00:00:00 2001 From: steiler Date: Fri, 19 Dec 2025 11:05:02 +0100 Subject: [PATCH 38/44] update --- pkg/datastore/deviations.go | 54 ++++++++++++++++++++++++++----------- 1 file changed, 39 insertions(+), 15 deletions(-) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 4ef48e95..269e6704 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -2,6 +2,7 @@ package datastore import ( "context" + "sync/atomic" "time" "github.com/sdcio/data-server/pkg/config" @@ -90,8 +91,9 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) log.Error(err, "failed to calculate deviations") continue } - log.V(logf.VDebug).Info("calculated deviations", "duration", time.Since(start)) + log.V(logf.VDebug).Info("calculate deviations", "duration", time.Since(start)) d.SendDeviations(ctx, deviationChan, deviationClients) + log.Info("Before sending DeviationEvent_END") for clientIdentifier, dc := range deviationClients { if dc.Context().Err() != nil { continue @@ -111,21 +113,28 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) func (d *Datastore) SendDeviations(ctx context.Context, ch <-chan *treetypes.DeviationEntry, deviationClients map[string]sdcpb.DataServer_WatchDeviationsServer) { log := logf.FromContext(ctx) - vPool := d.taskPool.NewVirtualPool(pool.VirtualTolerant, 1) + vPool := d.taskPool.NewVirtualPool(pool.VirtualTolerant, 100) + taskCount := 0 + + var started, done atomic.Int32 + for de := range ch { - vPool.SubmitFunc(func(ctx context.Context, _ func(pool.Task) error) error { + deviation := de // make sure each iteration uses its own var. + taskCount++ + err := vPool.SubmitFunc(func(ctx context.Context, _ func(pool.Task) error) error { + started.Add(1) for clientIdentifier, dc := range deviationClients { if dc.Context().Err() != nil { continue } err := dc.Send(&sdcpb.WatchDeviationResponse{ Name: d.config.Name, - Intent: de.IntentName(), + Intent: deviation.IntentName(), Event: sdcpb.DeviationEvent_UPDATE, - Reason: sdcpb.DeviationReason(de.Reason()), - Path: de.Path(), - ExpectedValue: de.ExpectedValue(), - CurrentValue: de.CurrentValue(), + Reason: sdcpb.DeviationReason(deviation.Reason()), + Path: deviation.Path(), + ExpectedValue: deviation.ExpectedValue(), + CurrentValue: deviation.CurrentValue(), }) if err != nil { // ignore client-side cancellation (context closed) as it's expected when a client disconnects @@ -136,11 +145,26 @@ func (d *Datastore) SendDeviations(ctx context.Context, ch <-chan *treetypes.Dev log.Error(err, "error sending deviation", "client-identifier", clientIdentifier) } } + done.Add(1) return nil }) + if err != nil { + log.Error(err, "failed to submit deviation task to pool") + } } + + time.Sleep(1 * time.Second) + log.Info("stats", "started", started.Load(), "done", done.Load()) + for started.Load() != done.Load() { + time.Sleep(2 * time.Second) + log.Info("stats", "started", started.Load(), "done", done.Load()) + } + log.V(logf.VDebug).Info("deviation channel drained, closing pool for submission", "task-count", taskCount) vPool.CloseForSubmit() log.Info("waiting for tasks in pool to finish") + for e := range vPool.ErrorChan() { + log.Error(e, "error sending deviation to client") + } vPool.Wait() log.Info("pool finished") } @@ -154,7 +178,6 @@ type DeviationEntry interface { } func (d *Datastore) calculateDeviations() (<-chan *treetypes.DeviationEntry, error) { - deviationChan := make(chan *treetypes.DeviationEntry, 10) d.syncTreeMutex.RLock() deviationTree, err := d.syncTree.DeepCopy(d.ctx) @@ -168,19 +191,20 @@ func (d *Datastore) calculateDeviations() (<-chan *treetypes.DeviationEntry, err return nil, err } - // Send IntentExists - for _, n := range addedIntentNames { - deviationChan <- treetypes.NewDeviationEntry(n, treetypes.DeviationReasonIntentExists, nil) - } - err = deviationTree.FinishInsertionPhase(d.ctx) if err != nil { return nil, err } + deviationChan := make(chan *treetypes.DeviationEntry, 10) go func() { + defer close(deviationChan) + // Send IntentExists + for _, n := range addedIntentNames { + deviationChan <- treetypes.NewDeviationEntry(n, treetypes.DeviationReasonIntentExists, nil) + } + deviationTree.GetDeviations(d.ctx, deviationChan) - close(deviationChan) }() return deviationChan, nil From b283c876a83096d203ee7f090ab161563696b8e9 Mon Sep 17 00:00:00 2001 From: steiler Date: Fri, 19 Dec 2025 12:58:47 +0100 Subject: [PATCH 39/44] revert to goroutine explosion --- pkg/datastore/deviations.go | 70 ++++++++++--------------------------- 1 file changed, 19 insertions(+), 51 deletions(-) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 269e6704..663232b1 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -2,11 +2,9 @@ package datastore import ( "context" - "sync/atomic" "time" "github.com/sdcio/data-server/pkg/config" - "github.com/sdcio/data-server/pkg/pool" treetypes "github.com/sdcio/data-server/pkg/tree/types" logf "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" @@ -113,60 +111,30 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) func (d *Datastore) SendDeviations(ctx context.Context, ch <-chan *treetypes.DeviationEntry, deviationClients map[string]sdcpb.DataServer_WatchDeviationsServer) { log := logf.FromContext(ctx) - vPool := d.taskPool.NewVirtualPool(pool.VirtualTolerant, 100) - taskCount := 0 - - var started, done atomic.Int32 - - for de := range ch { - deviation := de // make sure each iteration uses its own var. - taskCount++ - err := vPool.SubmitFunc(func(ctx context.Context, _ func(pool.Task) error) error { - started.Add(1) - for clientIdentifier, dc := range deviationClients { - if dc.Context().Err() != nil { + for deviation := range ch { + for clientIdentifier, dc := range deviationClients { + if dc.Context().Err() != nil { + continue + } + err := dc.Send(&sdcpb.WatchDeviationResponse{ + Name: d.config.Name, + Intent: deviation.IntentName(), + Event: sdcpb.DeviationEvent_UPDATE, + Reason: sdcpb.DeviationReason(deviation.Reason()), + Path: deviation.Path(), + ExpectedValue: deviation.ExpectedValue(), + CurrentValue: deviation.CurrentValue(), + }) + if err != nil { + // ignore client-side cancellation (context closed) as it's expected when a client disconnects + if dc.Context().Err() != nil || status.Code(err) == codes.Canceled { + log.V(logf.VDebug).Info("client context closed, skipping send", "client-identifier", clientIdentifier, "err", err) continue } - err := dc.Send(&sdcpb.WatchDeviationResponse{ - Name: d.config.Name, - Intent: deviation.IntentName(), - Event: sdcpb.DeviationEvent_UPDATE, - Reason: sdcpb.DeviationReason(deviation.Reason()), - Path: deviation.Path(), - ExpectedValue: deviation.ExpectedValue(), - CurrentValue: deviation.CurrentValue(), - }) - if err != nil { - // ignore client-side cancellation (context closed) as it's expected when a client disconnects - if dc.Context().Err() != nil || status.Code(err) == codes.Canceled { - log.V(logf.VDebug).Info("client context closed, skipping send", "client-identifier", clientIdentifier, "err", err) - continue - } - log.Error(err, "error sending deviation", "client-identifier", clientIdentifier) - } + log.Error(err, "error sending deviation", "client-identifier", clientIdentifier) } - done.Add(1) - return nil - }) - if err != nil { - log.Error(err, "failed to submit deviation task to pool") } } - - time.Sleep(1 * time.Second) - log.Info("stats", "started", started.Load(), "done", done.Load()) - for started.Load() != done.Load() { - time.Sleep(2 * time.Second) - log.Info("stats", "started", started.Load(), "done", done.Load()) - } - log.V(logf.VDebug).Info("deviation channel drained, closing pool for submission", "task-count", taskCount) - vPool.CloseForSubmit() - log.Info("waiting for tasks in pool to finish") - for e := range vPool.ErrorChan() { - log.Error(e, "error sending deviation to client") - } - vPool.Wait() - log.Info("pool finished") } type DeviationEntry interface { From b29e01519580c96caf8f6c93529bd874f94f8d0b Mon Sep 17 00:00:00 2001 From: steiler Date: Tue, 23 Dec 2025 11:58:53 +0100 Subject: [PATCH 40/44] add error message on mutiple DS mentioned in deviationwatchrequest --- pkg/server/datastore.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pkg/server/datastore.go b/pkg/server/datastore.go index 97d8e45f..3f65e796 100644 --- a/pkg/server/datastore.go +++ b/pkg/server/datastore.go @@ -269,12 +269,17 @@ func (s *Server) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdcpb. if req.GetName() == nil || len(req.GetName()) == 0 { return status.Errorf(codes.InvalidArgument, "missing datastore name") } + if len(req.GetName()) > 1 { + // although we have a slice in the req we support just a single datastore per request atm. + return status.Errorf(codes.InvalidArgument, "only single datastore name allowed per request") + } ds, err := s.datastores.GetDataStore(req.GetName()[0]) if err != nil { log.Error(err, "failed to get datastore") return status.Errorf(codes.NotFound, "unknown datastore") } + log.WithValues("datastore-name", req.GetName()[0]) err = ds.WatchDeviations(req, stream) if err != nil { @@ -282,6 +287,7 @@ func (s *Server) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdcpb. return err } <-stream.Context().Done() + log.Info("Stream context done", "errVal", stream.Context().Err()) ds.StopDeviationsWatch(peerName) return nil } From 9b5528e984c422e21dadabdbbefaff08f42c97d0 Mon Sep 17 00:00:00 2001 From: steiler Date: Mon, 29 Dec 2025 15:37:29 +0100 Subject: [PATCH 41/44] update --- pkg/datastore/deviations.go | 34 +++++++++++++++++++--------------- pkg/pool/pool.go | 2 +- pkg/pool/virtual_pool.go | 2 +- pkg/server/datastore.go | 2 +- 4 files changed, 22 insertions(+), 18 deletions(-) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 663232b1..7faf7b1a 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -14,8 +14,7 @@ import ( ) func (d *Datastore) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdcpb.DataServer_WatchDeviationsServer) error { - d.m.Lock() - defer d.m.Unlock() + log := logf.FromContext(d.ctx) ctx := stream.Context() p, ok := peer.FromContext(ctx) @@ -24,19 +23,20 @@ func (d *Datastore) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdc } pName := p.Addr.String() + d.m.Lock() + defer d.m.Unlock() d.deviationClients[pName] = stream - log := logf.FromContext(d.ctx) log.Info("new deviation client", "client", pName) return nil } func (d *Datastore) StopDeviationsWatch(peer string) { log := logf.FromContext(d.ctx) - log.Info("deviation client removed", "peer", peer) d.m.Lock() defer d.m.Unlock() delete(d.deviationClients, peer) + log.Info("deviation client removed", "peer", peer) } func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) { @@ -56,19 +56,23 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) return case <-ticker.C: log.V(logf.VDebug).Info("deviation calc run - start") - d.m.RLock() - deviationClientNames := make([]string, 0, len(d.deviationClients)) + deviationClients := map[string]sdcpb.DataServer_WatchDeviationsServer{} - for clientIdentifier, devStream := range d.deviationClients { - deviationClients[clientIdentifier] = devStream - if devStream.Context().Err() != nil { - log.V(logf.VWarn).Error(devStream.Context().Err(), "removing deviation client", "client", clientIdentifier) - delete(deviationClients, clientIdentifier) - continue + deviationClientNames := make([]string, 0, len(d.deviationClients)) + + // encap in func to use defer for the lock + func() { + d.m.RLock() + defer d.m.RUnlock() + for peerIdentifier, devStream := range d.deviationClients { + deviationClients[peerIdentifier] = devStream + if devStream.Context().Err() != nil { + log.V(logf.VWarn).Error(devStream.Context().Err(), "deviation client context error", "client", peerIdentifier, "error", devStream.Context().Err()) + continue + } + deviationClientNames = append(deviationClientNames, peerIdentifier) } - deviationClientNames = append(deviationClientNames, clientIdentifier) - } - d.m.RUnlock() + }() if len(deviationClients) == 0 { log.V(logf.VDebug).Info("no deviation clients present") continue diff --git a/pkg/pool/pool.go b/pkg/pool/pool.go index a3fc67d0..18f224eb 100644 --- a/pkg/pool/pool.go +++ b/pkg/pool/pool.go @@ -196,8 +196,8 @@ func (p *Pool[T]) CloseForSubmit() { p.closedForSubmit.Store(true) // kick the monitor by signaling condition in case inflight==0 already p.inflightMu.Lock() + defer p.inflightMu.Unlock() p.inflightC.Broadcast() - p.inflightMu.Unlock() } // Wait blocks until all workers have exited and returns the first error (if any). diff --git a/pkg/pool/virtual_pool.go b/pkg/pool/virtual_pool.go index e209ae08..850f4124 100644 --- a/pkg/pool/virtual_pool.go +++ b/pkg/pool/virtual_pool.go @@ -63,8 +63,8 @@ func (ec *ErrorCollector) add(err error) { return } ec.mu.Lock() + defer ec.mu.Unlock() ec.errs = append(ec.errs, err) - ec.mu.Unlock() select { case ec.Ch <- err: diff --git a/pkg/server/datastore.go b/pkg/server/datastore.go index 3f65e796..5ef9c0a3 100644 --- a/pkg/server/datastore.go +++ b/pkg/server/datastore.go @@ -287,7 +287,7 @@ func (s *Server) WatchDeviations(req *sdcpb.WatchDeviationRequest, stream sdcpb. return err } <-stream.Context().Done() - log.Info("Stream context done", "errVal", stream.Context().Err()) + log.Info("stream context done", "errVal", stream.Context().Err()) ds.StopDeviationsWatch(peerName) return nil } From 0daae2c3112b857f340f670f4c44f09d3b9bcb73 Mon Sep 17 00:00:00 2001 From: steiler Date: Fri, 2 Jan 2026 14:49:11 +0100 Subject: [PATCH 42/44] update --- pkg/datastore/datastore_rpc.go | 4 +- pkg/datastore/deviations.go | 13 +- pkg/datastore/sync.go | 14 ++ pkg/datastore/target/gnmi/get.go | 1 + pkg/datastore/transaction_rpc.go | 22 ++- pkg/tree/leaf_variants.go | 15 +- pkg/tree/leaf_variants_test.go | 286 ++++++++++++++++++++++++++++++ pkg/tree/sharedEntryAttributes.go | 2 +- 8 files changed, 342 insertions(+), 15 deletions(-) create mode 100644 pkg/tree/leaf_variants_test.go diff --git a/pkg/datastore/datastore_rpc.go b/pkg/datastore/datastore_rpc.go index b04e03be..0bdde54f 100644 --- a/pkg/datastore/datastore_rpc.go +++ b/pkg/datastore/datastore_rpc.go @@ -231,9 +231,9 @@ func (d *Datastore) BlameConfig(ctx context.Context, includeDefaults bool) (*sdc ctx = logger.IntoContext(ctx, log) // create a new TreeRoot by copying the syncTree - d.syncTreeMutex.Lock() + d.syncTreeMutex.RLock() root, err := d.syncTree.DeepCopy(ctx) - d.syncTreeMutex.Unlock() + d.syncTreeMutex.RUnlock() if err != nil { return nil, err } diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 7faf7b1a..0bb95828 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -6,6 +6,7 @@ import ( "github.com/sdcio/data-server/pkg/config" treetypes "github.com/sdcio/data-server/pkg/tree/types" + "github.com/sdcio/logger" logf "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "google.golang.org/grpc/codes" @@ -88,7 +89,7 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) } } start := time.Now() - deviationChan, err := d.calculateDeviations() + deviationChan, err := d.calculateDeviations(ctx) if err != nil { log.Error(err, "failed to calculate deviations") continue @@ -149,21 +150,21 @@ type DeviationEntry interface { ExpectedValue() *sdcpb.TypedValue } -func (d *Datastore) calculateDeviations() (<-chan *treetypes.DeviationEntry, error) { +func (d *Datastore) calculateDeviations(ctx context.Context) (<-chan *treetypes.DeviationEntry, error) { d.syncTreeMutex.RLock() - deviationTree, err := d.syncTree.DeepCopy(d.ctx) + deviationTree, err := d.syncTree.DeepCopy(ctx) d.syncTreeMutex.RUnlock() if err != nil { return nil, err } - addedIntentNames, err := d.LoadAllButRunningIntents(d.ctx, deviationTree, true) + addedIntentNames, err := d.LoadAllButRunningIntents(ctx, deviationTree, true) if err != nil { return nil, err } - err = deviationTree.FinishInsertionPhase(d.ctx) + err = deviationTree.FinishInsertionPhase(ctx) if err != nil { return nil, err } @@ -176,7 +177,7 @@ func (d *Datastore) calculateDeviations() (<-chan *treetypes.DeviationEntry, err deviationChan <- treetypes.NewDeviationEntry(n, treetypes.DeviationReasonIntentExists, nil) } - deviationTree.GetDeviations(d.ctx, deviationChan) + deviationTree.GetDeviations(ctx, deviationChan) }() return deviationChan, nil diff --git a/pkg/datastore/sync.go b/pkg/datastore/sync.go index e9cb3626..fc4b3aa4 100644 --- a/pkg/datastore/sync.go +++ b/pkg/datastore/sync.go @@ -8,6 +8,7 @@ import ( treetypes "github.com/sdcio/data-server/pkg/tree/types" "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" + "google.golang.org/protobuf/encoding/protojson" ) func (d *Datastore) ApplyToRunning(ctx context.Context, deletes []*sdcpb.Path, importer importer.ImportConfigAdapter) error { @@ -30,6 +31,19 @@ func (d *Datastore) ApplyToRunning(ctx context.Context, deletes []*sdcpb.Path, i return err } } + + // conditional trace logging + if log.GetV() <= logger.VTrace { + treeExport, err := d.syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) + if err == nil { + json, err := protojson.MarshalOptions{Multiline: false}.Marshal(treeExport) + if err == nil { + log.V(logger.VTrace).Info("synctree after sync apply", "content", string(json)) + } + } + + } + return nil } diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index 43d1fe9e..d64c38c6 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -158,6 +158,7 @@ func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { log.Error(err, "failure importing synctree export into running") return } + log.V(logger.VDebug).Info("syncing done") } type GetTarget interface { diff --git a/pkg/datastore/transaction_rpc.go b/pkg/datastore/transaction_rpc.go index 4b4ee87c..ceea0b19 100644 --- a/pkg/datastore/transaction_rpc.go +++ b/pkg/datastore/transaction_rpc.go @@ -14,9 +14,11 @@ import ( treeproto "github.com/sdcio/data-server/pkg/tree/importer/proto" treetypes "github.com/sdcio/data-server/pkg/tree/types" "github.com/sdcio/data-server/pkg/utils" + "github.com/sdcio/logger" logf "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "github.com/sdcio/sdc-protos/tree_persist" + "google.golang.org/protobuf/encoding/protojson" ) var ( @@ -195,9 +197,9 @@ func (d *Datastore) LoadAllButRunningIntents(ctx context.Context, root *tree.Roo func (d *Datastore) lowlevelTransactionSet(ctx context.Context, transaction *types.Transaction, dryRun bool) (*sdcpb.TransactionSetResponse, error) { log := logf.FromContext(ctx) // create a new TreeRoot - d.syncTreeMutex.Lock() + d.syncTreeMutex.RLock() root, err := d.syncTree.DeepCopy(ctx) - d.syncTreeMutex.Unlock() + d.syncTreeMutex.RUnlock() if err != nil { return nil, err } @@ -346,16 +348,17 @@ func (d *Datastore) lowlevelTransactionSet(ctx context.Context, transaction *typ log.V(logf.VTrace).Info("generated deletes", "deletes", strings.Join(deletes.SdcpbPaths().ToXPathSlice(), "\n")) for _, intent := range transaction.GetNewIntents() { + log := log.WithValues("intent", intent.GetName()) // retrieve the data that is meant to be send towards the cache updatesOwner := root.GetUpdatesForOwner(intent.GetName()) deletesOwner := root.GetDeletesForOwner(intent.GetName()) // logging strSl := treetypes.Map(updatesOwner, func(u *treetypes.Update) string { return u.String() }) - log.V(logf.VTrace).Info("updates owner", "updates-owner", strSl, "\n") + log.V(logf.VTrace).Info("updates owner", "updates-owner", strSl) delSl := deletesOwner.ToXPathSlice() - log.V(logf.VTrace).Info("deletes owner", "deletes-owner", delSl, "\n") + log.V(logf.VTrace).Info("deletes owner", "deletes-owner", delSl) protoIntent, err := root.TreeExport(intent.GetName(), intent.GetPriority(), intent.Deviation()) switch { @@ -364,6 +367,7 @@ func (d *Datastore) lowlevelTransactionSet(ctx context.Context, transaction *typ if err != nil { log.Error(err, "failed deleting intent from store") } + log.V(logf.VDebug).Info("delete intent from cache") continue case err != nil: return nil, err @@ -395,6 +399,7 @@ func (d *Datastore) lowlevelTransactionSet(ctx context.Context, transaction *typ // writeBackSyncTree applies the provided changes to the syncTree and applies to the running cache intent func (d *Datastore) writeBackSyncTree(ctx context.Context, updates tree.LeafVariantSlice, deletes treetypes.DeleteEntriesList) error { + log := logger.FromContext(ctx) runningUpdates := updates.ToUpdateSlice().CopyWithNewOwnerAndPrio(tree.RunningIntentName, tree.RunningValuesPrio) // wrap the lock in an anonymous function to be able to utilize defer for the unlock @@ -425,6 +430,15 @@ func (d *Datastore) writeBackSyncTree(ctx context.Context, updates tree.LeafVari if err != nil && err != tree.ErrorIntentNotPresent { return err } + + // conditional trace logging + if log.GetV() <= logger.VTrace { + json, err := protojson.MarshalOptions{Multiline: false}.Marshal(newRunningIntent) + if err != nil { + log.V(logger.VTrace).Info("writeback synctree", "content", string(json)) + } + } + // write the synctree to disk if newRunningIntent != nil { err = d.cacheClient.IntentModify(ctx, newRunningIntent) diff --git a/pkg/tree/leaf_variants.go b/pkg/tree/leaf_variants.go index 3a0e5018..e7d0d7ef 100644 --- a/pkg/tree/leaf_variants.go +++ b/pkg/tree/leaf_variants.go @@ -174,19 +174,30 @@ func (lv *LeafVariants) remainsToExist() bool { highest := lv.GetHighestPrecedence(false, false, true) - if highest != nil && highest.IsExplicitDelete { + if highest == nil || highest.IsExplicitDelete { return false } + defaultOrRunningExists := false + deleteExists := false // go through all variants for _, l := range lv.les { + if l.Owner() == RunningIntentName || l.Owner() == DefaultsIntentName { + defaultOrRunningExists = true + continue + } // if an entry exists that does not have the delete flag set, // then a remaining LeafVariant exists. if !l.GetDeleteFlag() { return true } + deleteExists = true + } + + if deleteExists { + return false } - return false + return defaultOrRunningExists } func (lv *LeafVariants) GetHighestPrecedenceValue(filter HighestPrecedenceFilter) int32 { diff --git a/pkg/tree/leaf_variants_test.go b/pkg/tree/leaf_variants_test.go new file mode 100644 index 00000000..6f380fe1 --- /dev/null +++ b/pkg/tree/leaf_variants_test.go @@ -0,0 +1,286 @@ +package tree + +import ( + "testing" + + "github.com/sdcio/data-server/pkg/tree/types" + sdcpb "github.com/sdcio/sdc-protos/sdcpb" +) + +type mockUpdateParent struct{} + +func (m *mockUpdateParent) SdcpbPath() *sdcpb.Path { + return &sdcpb.Path{} +} + +func TestLeafVariants_remainsToExist(t *testing.T) { + tests := []struct { + name string + setup func() *LeafVariants + expected bool + }{ + { + name: "Empty LeafVariants", + setup: func() *LeafVariants { + return &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + }, + expected: false, + }, + { + name: "Single entry, not deleted", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + le := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 10, "owner1", 0), + types.NewUpdateInsertFlags(), + nil, + ) + lv.Add(le) + return lv + }, + expected: true, + }, + { + name: "Single entry, deleted", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + le := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 10, "owner1", 0), + types.NewUpdateInsertFlags().SetDeleteFlag(), + nil, + ) + lv.Add(le) + return lv + }, + expected: false, + }, + { + name: "Multiple entries, all deleted", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + le1 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 10, "owner1", 0), + types.NewUpdateInsertFlags().SetDeleteFlag(), + nil, + ) + le2 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 20, "owner2", 0), + types.NewUpdateInsertFlags().SetDeleteFlag(), + nil, + ) + lv.Add(le1) + lv.Add(le2) + return lv + }, + expected: false, + }, + { + name: "Multiple entries, one remaining", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + le1 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 10, "owner1", 0), + types.NewUpdateInsertFlags().SetDeleteFlag(), + nil, + ) + le2 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 20, "owner2", 0), + types.NewUpdateInsertFlags(), + nil, + ) + lv.Add(le1) + lv.Add(le2) + return lv + }, + expected: true, + }, + { + name: "Explicit delete highest priority", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + // Explicit delete with priority 5 (lower is higher priority) + le1 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 5, "owner1", 0), + types.NewUpdateInsertFlags().SetExplicitDeleteFlag(), + nil, + ) + // Normal entry with priority 10 + le2 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 10, "owner2", 0), + types.NewUpdateInsertFlags(), + nil, + ) + lv.Add(le1) + lv.Add(le2) + return lv + }, + expected: false, + }, + { + name: "Explicit delete lower priority", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + // Explicit delete with priority 20 + le1 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 20, "owner1", 0), + types.NewUpdateInsertFlags().SetExplicitDeleteFlag(), + nil, + ) + // Normal entry with priority 10 (higher priority) + le2 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 10, "owner2", 0), + types.NewUpdateInsertFlags(), + nil, + ) + lv.Add(le1) + lv.Add(le2) + return lv + }, + expected: true, + }, + + { + name: "Delete all, no running", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + // Explicit delete with priority 20 + le1 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 20, "owner1", 0), + types.NewUpdateInsertFlags().SetDeleteFlag(), + nil, + ) + // Normal entry with priority 10 (higher priority) + le2 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 10, "owner2", 0), + types.NewUpdateInsertFlags().SetDeleteFlag(), + nil, + ) + lv.Add(le1) + lv.Add(le2) + return lv + }, + expected: false, + }, + { + name: "Delete all, with running", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + // Explicit delete with priority 20 + le1 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 20, "owner1", 0), + types.NewUpdateInsertFlags().SetDeleteFlag(), + nil, + ) + // Normal entry with priority 10 (higher priority) + le2 := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 10, "owner2", 0), + types.NewUpdateInsertFlags().SetDeleteFlag(), + nil, + ) + lerun := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, RunningValuesPrio, RunningIntentName, 0), + types.NewUpdateInsertFlags(), + nil, + ) + lv.Add(le1) + lv.Add(le2) + lv.Add(lerun) + return lv + }, + expected: false, + }, + { + name: "Only Running", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + lerun := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, RunningValuesPrio, RunningIntentName, 0), + types.NewUpdateInsertFlags(), + nil, + ) + lv.Add(lerun) + return lv + }, + expected: true, + }, + { + name: "Only Running + default", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + lerun := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, RunningValuesPrio, RunningIntentName, 0), + types.NewUpdateInsertFlags(), + nil, + ) + ledef := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, DefaultValuesPrio, DefaultsIntentName, 0), + types.NewUpdateInsertFlags(), + nil, + ) + lv.Add(lerun) + lv.Add(ledef) + return lv + }, + expected: true, + }, + { + name: "Running, default and delete", + setup: func() *LeafVariants { + lv := &LeafVariants{ + les: make(LeafVariantSlice, 0), + } + lerun := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, RunningValuesPrio, RunningIntentName, 0), + types.NewUpdateInsertFlags(), + nil, + ) + ledef := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, DefaultValuesPrio, DefaultsIntentName, 0), + types.NewUpdateInsertFlags(), + nil, + ) + ledel := NewLeafEntry( + types.NewUpdate(&mockUpdateParent{}, &sdcpb.TypedValue{}, 10, "owner1", 0), + types.NewUpdateInsertFlags().SetDeleteFlag(), + nil, + ) + lv.Add(lerun) + lv.Add(ledef) + lv.Add(ledel) + return lv + }, + expected: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + lv := tt.setup() + if got := lv.remainsToExist(); got != tt.expected { + t.Errorf("LeafVariants.remainsToExist() = %v, want %v", got, tt.expected) + } + }) + } +} diff --git a/pkg/tree/sharedEntryAttributes.go b/pkg/tree/sharedEntryAttributes.go index c38423d8..d8025333 100644 --- a/pkg/tree/sharedEntryAttributes.go +++ b/pkg/tree/sharedEntryAttributes.go @@ -1229,7 +1229,7 @@ func (s *sharedEntryAttributes) validatePattern(resultChan chan<- *types.Validat // defined by the schema are present either in the tree or in the index. func (s *sharedEntryAttributes) validateMandatory(ctx context.Context, resultChan chan<- *types.ValidationResultEntry, stats *types.ValidationStats) { log := logf.FromContext(ctx) - if s.shouldDelete() { + if !s.remainsToExist() { return } if s.schema != nil { From 4d5b7bce207997ce4044b4bac1f3de4c30b81e00 Mon Sep 17 00:00:00 2001 From: steiler Date: Fri, 2 Jan 2026 14:54:19 +0100 Subject: [PATCH 43/44] update --- pkg/datastore/deviations.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 0bb95828..083f5bef 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -121,6 +121,11 @@ func (d *Datastore) SendDeviations(ctx context.Context, ch <-chan *treetypes.Dev if dc.Context().Err() != nil { continue } + + if log.GetV() <= logger.VTrace && deviation.Reason() == treetypes.DeviationReasonNotApplied { + log.V(logger.VTrace).Info("NOT APPLIED", "path", deviation.Path().ToXPath(false), "actual value", deviation.CurrentValue().ToString(), "expected value", deviation.ExpectedValue().ToString(), "intent", deviation.IntentName()) + } + err := dc.Send(&sdcpb.WatchDeviationResponse{ Name: d.config.Name, Intent: deviation.IntentName(), From d8a4cd723b07fb8c200e8eea9694492e99549c8b Mon Sep 17 00:00:00 2001 From: steiler Date: Fri, 9 Jan 2026 12:02:58 +0100 Subject: [PATCH 44/44] finally fix logging trave level --- go.mod | 32 +++++----- go.sum | 80 ++++++++++++------------- main.go | 5 +- pkg/datastore/datastore_rpc.go | 5 +- pkg/datastore/deviations.go | 14 ++++- pkg/datastore/sync.go | 7 +-- pkg/datastore/target/gnmi/get.go | 4 +- pkg/datastore/target/target.go | 6 +- pkg/datastore/transaction_rpc.go | 12 +--- pkg/server/datastore.go | 12 ++-- pkg/server/transaction.go | 12 ++-- pkg/tree/processor_mark_owner_delete.go | 3 +- pkg/tree/root_entry.go | 2 +- pkg/utils/converter.go | 9 ++- 14 files changed, 104 insertions(+), 99 deletions(-) diff --git a/go.mod b/go.mod index ad62bc50..8a88acb5 100644 --- a/go.mod +++ b/go.mod @@ -22,21 +22,21 @@ require ( github.com/prometheus/client_golang v1.23.2 github.com/scrapli/scrapligo v1.3.3 github.com/sdcio/cache v0.0.38 - github.com/sdcio/logger v0.0.2 + github.com/sdcio/logger v0.0.3 github.com/sdcio/schema-server v0.0.33 github.com/sdcio/sdc-protos v0.0.47 github.com/sdcio/yang-parser v0.0.12 github.com/spf13/cobra v1.10.1 github.com/spf13/pflag v1.0.10 go.uber.org/mock v0.6.0 - google.golang.org/grpc v1.75.1 - google.golang.org/protobuf v1.36.9 + google.golang.org/grpc v1.77.0 + google.golang.org/protobuf v1.36.10 gopkg.in/yaml.v2 v2.4.0 sigs.k8s.io/controller-runtime v0.20.4 ) require ( - cloud.google.com/go/compute/metadata v0.7.0 // indirect + cloud.google.com/go/compute/metadata v0.9.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bufbuild/protocompile v0.14.1 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect @@ -84,21 +84,21 @@ require ( github.com/sirikothe/gotextfsm v1.0.1-0.20200816110946-6aa2cfd355e4 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/x448/float16 v0.8.4 // indirect - go.opentelemetry.io/auto/sdk v1.1.0 // indirect - go.opentelemetry.io/otel v1.37.0 // indirect - go.opentelemetry.io/otel/metric v1.37.0 // indirect - go.opentelemetry.io/otel/trace v1.37.0 // indirect + go.opentelemetry.io/auto/sdk v1.2.1 // indirect + go.opentelemetry.io/otel v1.38.0 // indirect + go.opentelemetry.io/otel/metric v1.38.0 // indirect + go.opentelemetry.io/otel/trace v1.38.0 // indirect go.yaml.in/yaml/v2 v2.4.2 // indirect - golang.org/x/crypto v0.41.0 // indirect + golang.org/x/crypto v0.43.0 // indirect golang.org/x/exp v0.0.0-20250218142911-aa4b98e5adaa // indirect - golang.org/x/net v0.43.0 // indirect - golang.org/x/oauth2 v0.30.0 // indirect - golang.org/x/sync v0.16.0 // indirect - golang.org/x/sys v0.35.0 // indirect - golang.org/x/term v0.34.0 // indirect - golang.org/x/text v0.28.0 // indirect + golang.org/x/net v0.46.1-0.20251013234738-63d1a5100f82 // indirect + golang.org/x/oauth2 v0.32.0 // indirect + golang.org/x/sync v0.17.0 // indirect + golang.org/x/sys v0.37.0 // indirect + golang.org/x/term v0.36.0 // indirect + golang.org/x/text v0.30.0 // indirect golang.org/x/time v0.8.0 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20250707201910-8d1bb00bc6a7 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20251022142026-3a174f9686a8 // indirect gopkg.in/evanphx/json-patch.v4 v4.12.0 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 580c513e..fabbbc23 100644 --- a/go.sum +++ b/go.sum @@ -1,6 +1,6 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go/compute/metadata v0.7.0 h1:PBWF+iiAerVNe8UCHxdOt6eHLVc3ydFeOCw78U8ytSU= -cloud.google.com/go/compute/metadata v0.7.0/go.mod h1:j5MvL9PprKL39t166CoB1uVHfQMs4tFQZZcKwksXUjo= +cloud.google.com/go/compute/metadata v0.9.0 h1:pDUj4QMoPejqq20dK0Pg2N4yG9zIkYGdBtwLoEkH9Zs= +cloud.google.com/go/compute/metadata v0.9.0/go.mod h1:E0bWwX5wTnLPedCKqk3pJmVgCBSM6qQI1yTBdEb3C10= github.com/AlekSi/pointer v1.2.0 h1:glcy/gc4h8HnG2Z3ZECSzZ1IX1x2JxRVuDzaJwQE0+w= github.com/AlekSi/pointer v1.2.0/go.mod h1:gZGfd3dpW4vEc/UlyfKKi1roIqcCgwOIvb0tSNSBle0= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= @@ -176,8 +176,8 @@ github.com/prometheus/procfs v0.16.1/go.mod h1:teAbpZRB1iIAJYREa1LsoWUXykVXA1KlT github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= github.com/rivo/uniseg v0.4.7/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= -github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= -github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= +github.com/rogpeppe/go-internal v1.14.1 h1:UQB4HGPB6osV0SQTLymcB4TgvyWu6ZyliaW0tI/otEQ= +github.com/rogpeppe/go-internal v1.14.1/go.mod h1:MaRKkUm5W0goXpeCfT7UZI6fk/L7L7so1lCWt35ZSgc= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/scrapli/scrapligo v1.3.3 h1:D9zj1QrOYNYAQ30YT7wfQBINvPGxvs5L5Lz+2LnL7V4= github.com/scrapli/scrapligo v1.3.3/go.mod h1:pOWxVyPsQRrWTrkoSSDg05tjOqtWfLffAZtAsCc0w3M= @@ -185,8 +185,8 @@ github.com/sdcio/cache v0.0.38 h1:8meykZC/DAgzKI+R3GtZvTs0/OXWRCeNN5mxVKG0Ba8= github.com/sdcio/cache v0.0.38/go.mod h1:PHJd4pTmfFZBLdPohtXJAVBNmAKhhKdZHTi3MejGrk0= github.com/sdcio/goyang v1.6.2-2 h1:qfeUKBmoKpiKAruuEc3+V8wgHKP/n1jRDEnTy23knV8= github.com/sdcio/goyang v1.6.2-2/go.mod h1:5WolITjek1NF8yrNERyVZ7jqjOClJTpO8p/+OwmETM4= -github.com/sdcio/logger v0.0.2 h1:yNRGFL5ixMhpVBlU/c+hs55JRrYWjFVmYxlWl1da5PU= -github.com/sdcio/logger v0.0.2/go.mod h1:yWaOxK/G6vszjg8tKZiMqiEjlZouHsjFME4zSk+SAEA= +github.com/sdcio/logger v0.0.3 h1:IFUbObObGry+S8lHGwOQKKRxJSuOphgRU/hxVhOdMOM= +github.com/sdcio/logger v0.0.3/go.mod h1:yWaOxK/G6vszjg8tKZiMqiEjlZouHsjFME4zSk+SAEA= github.com/sdcio/schema-server v0.0.33 h1:RTeQMIchynAPSQaf61CZBgYHCCpsyDLEAWJn+ZKImIo= github.com/sdcio/schema-server v0.0.33/go.mod h1:q8leN1KhRNTnnqf6yxvkDk5tFl6DAsHcl81usVgYpoI= github.com/sdcio/sdc-protos v0.0.47 h1:1SD8Ifx7MRc6fqjuJIM1wQ2NGEwjYj+QWgHNgENJiT4= @@ -215,18 +215,18 @@ github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM= github.com/x448/float16 v0.8.4/go.mod h1:14CWIYCyZA/cWjXOioeEpHeN/83MdbZDRQHoFcYsOfg= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA= -go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A= -go.opentelemetry.io/otel v1.37.0 h1:9zhNfelUvx0KBfu/gb+ZgeAfAgtWrfHJZcAqFC228wQ= -go.opentelemetry.io/otel v1.37.0/go.mod h1:ehE/umFRLnuLa/vSccNq9oS1ErUlkkK71gMcN34UG8I= -go.opentelemetry.io/otel/metric v1.37.0 h1:mvwbQS5m0tbmqML4NqK+e3aDiO02vsf/WgbsdpcPoZE= -go.opentelemetry.io/otel/metric v1.37.0/go.mod h1:04wGrZurHYKOc+RKeye86GwKiTb9FKm1WHtO+4EVr2E= -go.opentelemetry.io/otel/sdk v1.37.0 h1:ItB0QUqnjesGRvNcmAcU0LyvkVyGJ2xftD29bWdDvKI= -go.opentelemetry.io/otel/sdk v1.37.0/go.mod h1:VredYzxUvuo2q3WRcDnKDjbdvmO0sCzOvVAiY+yUkAg= -go.opentelemetry.io/otel/sdk/metric v1.37.0 h1:90lI228XrB9jCMuSdA0673aubgRobVZFhbjxHHspCPc= -go.opentelemetry.io/otel/sdk/metric v1.37.0/go.mod h1:cNen4ZWfiD37l5NhS+Keb5RXVWZWpRE+9WyVCpbo5ps= -go.opentelemetry.io/otel/trace v1.37.0 h1:HLdcFNbRQBE2imdSEgm/kwqmQj1Or1l/7bW6mxVK7z4= -go.opentelemetry.io/otel/trace v1.37.0/go.mod h1:TlgrlQ+PtQO5XFerSPUYG0JSgGyryXewPGyayAWSBS0= +go.opentelemetry.io/auto/sdk v1.2.1 h1:jXsnJ4Lmnqd11kwkBV2LgLoFMZKizbCi5fNZ/ipaZ64= +go.opentelemetry.io/auto/sdk v1.2.1/go.mod h1:KRTj+aOaElaLi+wW1kO/DZRXwkF4C5xPbEe3ZiIhN7Y= +go.opentelemetry.io/otel v1.38.0 h1:RkfdswUDRimDg0m2Az18RKOsnI8UDzppJAtj01/Ymk8= +go.opentelemetry.io/otel v1.38.0/go.mod h1:zcmtmQ1+YmQM9wrNsTGV/q/uyusom3P8RxwExxkZhjM= +go.opentelemetry.io/otel/metric v1.38.0 h1:Kl6lzIYGAh5M159u9NgiRkmoMKjvbsKtYRwgfrA6WpA= +go.opentelemetry.io/otel/metric v1.38.0/go.mod h1:kB5n/QoRM8YwmUahxvI3bO34eVtQf2i4utNVLr9gEmI= +go.opentelemetry.io/otel/sdk v1.38.0 h1:l48sr5YbNf2hpCUj/FoGhW9yDkl+Ma+LrVl8qaM5b+E= +go.opentelemetry.io/otel/sdk v1.38.0/go.mod h1:ghmNdGlVemJI3+ZB5iDEuk4bWA3GkTpW+DOoZMYBVVg= +go.opentelemetry.io/otel/sdk/metric v1.38.0 h1:aSH66iL0aZqo//xXzQLYozmWrXxyFkBJ6qT5wthqPoM= +go.opentelemetry.io/otel/sdk/metric v1.38.0/go.mod h1:dg9PBnW9XdQ1Hd6ZnRz689CbtrUp0wMMs9iPcgT9EZA= +go.opentelemetry.io/otel/trace v1.38.0 h1:Fxk5bKrDZJUH+AMyyIXGcFAPah0oRcT+LuNtJrmcNLE= +go.opentelemetry.io/otel/trace v1.38.0/go.mod h1:j1P9ivuFsTceSWe1oY+EeW3sc+Pp42sO++GHkg4wwhs= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= @@ -244,8 +244,8 @@ go.yaml.in/yaml/v2 v2.4.2/go.mod h1:081UH+NErpNdqlCXm3TtEran0rJZGxAYx9hb/ELlsPU= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.41.0 h1:WKYxWedPGCTVVl5+WHSSrOBT0O8lx32+zxmHxijgXp4= -golang.org/x/crypto v0.41.0/go.mod h1:pO5AFd7FA68rFak7rOAGVuygIISepHftHnr8dr6+sUc= +golang.org/x/crypto v0.43.0 h1:dduJYIi3A3KOfdGOHX8AVZ/jGiyPa3IbBozJ5kNuE04= +golang.org/x/crypto v0.43.0/go.mod h1:BFbav4mRNlXJL4wNeejLpWxB7wMbc79PdRGhWKncxR0= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20250218142911-aa4b98e5adaa h1:t2QcU6V556bFjYgu4L6C+6VrCPyJZ+eyRsABUPs1mz4= golang.org/x/exp v0.0.0-20250218142911-aa4b98e5adaa/go.mod h1:BHOTPb3L19zxehTsLoJXVaTktb06DFgmdW6Wb9s8jqk= @@ -263,18 +263,18 @@ golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.43.0 h1:lat02VYK2j4aLzMzecihNvTlJNQUq316m2Mr9rnM6YE= -golang.org/x/net v0.43.0/go.mod h1:vhO1fvI4dGsIjh73sWfUVjj3N7CA9WkKJNQm2svM6Jg= +golang.org/x/net v0.46.1-0.20251013234738-63d1a5100f82 h1:6/3JGEh1C88g7m+qzzTbl3A0FtsLguXieqofVLU/JAo= +golang.org/x/net v0.46.1-0.20251013234738-63d1a5100f82/go.mod h1:Q9BGdFy1y4nkUwiLvT5qtyhAnEHgnQ/zd8PfU6nc210= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.30.0 h1:dnDm7JmhM45NNpd8FDDeLhK6FwqbOf4MLCM9zb1BOHI= -golang.org/x/oauth2 v0.30.0/go.mod h1:B++QgG3ZKulg6sRPGD/mqlHQs5rB3Ml9erfeDY7xKlU= +golang.org/x/oauth2 v0.32.0 h1:jsCblLleRMDrxMN29H3z/k1KliIvpLgCkE6R8FXXNgY= +golang.org/x/oauth2 v0.32.0/go.mod h1:lzm5WQJQwKZ3nwavOZ3IS5Aulzxi68dUSgRHujetwEA= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.16.0 h1:ycBJEhp9p4vXvUZNszeOq0kGTPghopOL8q0fq3vstxw= -golang.org/x/sync v0.16.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= +golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= +golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -284,14 +284,14 @@ golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.35.0 h1:vz1N37gP5bs89s7He8XuIYXpyY0+QlsKmzipCbUtyxI= -golang.org/x/sys v0.35.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= -golang.org/x/term v0.34.0 h1:O/2T7POpk0ZZ7MAzMeWFSg6S5IpWd/RXDlM9hgM3DR4= -golang.org/x/term v0.34.0/go.mod h1:5jC53AEywhIVebHgPVeg0mj8OD3VO9OzclacVrqpaAw= +golang.org/x/sys v0.37.0 h1:fdNQudmxPjkdUTPnLn5mdQv7Zwvbvpaxqs831goi9kQ= +golang.org/x/sys v0.37.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= +golang.org/x/term v0.36.0 h1:zMPR+aF8gfksFprF/Nc/rd1wRS1EI6nDBGyWAvDzx2Q= +golang.org/x/term v0.36.0/go.mod h1:Qu394IJq6V6dCBRgwqshf3mPF85AqzYEzofzRdZkWss= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.28.0 h1:rhazDwis8INMIwQ4tpjLDzUhx6RlXqZNPEM0huQojng= -golang.org/x/text v0.28.0/go.mod h1:U8nCwOR8jO/marOQ0QbDiOngZVEBB7MAiitBuMjXiNU= +golang.org/x/text v0.30.0 h1:yznKA/E9zq54KzlzBEAWn1NXSQ8DIp/NYMy88xJjl4k= +golang.org/x/text v0.30.0/go.mod h1:yDdHFIX9t+tORqspjENWgzaCVXgk0yYnYuSZ8UzzBVM= golang.org/x/time v0.8.0 h1:9i3RxcPv3PZnitoVGMPDKZSq1xW1gK1Xy3ArNOGZfEg= golang.org/x/time v0.8.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -303,8 +303,8 @@ golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.36.0 h1:kWS0uv/zsvHEle1LbV5LE8QujrxB3wfQyxHfhOk0Qkg= -golang.org/x/tools v0.36.0/go.mod h1:WBDiHKJK8YgLHlcQPYQzNCkUxUypCaa5ZegCVutKm+s= +golang.org/x/tools v0.37.0 h1:DVSRzp7FwePZW356yEAChSdNcQo6Nsp+fex1SUW09lE= +golang.org/x/tools v0.37.0/go.mod h1:MBN5QPQtLMHVdvsbtarmTNukZDdgwdwlO5qGacAzF0w= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -318,17 +318,17 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250707201910-8d1bb00bc6a7 h1:pFyd6EwwL2TqFf8emdthzeX+gZE1ElRq3iM8pui4KBY= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250707201910-8d1bb00bc6a7/go.mod h1:qQ0YXyHHx3XkvlzUtpXDkS29lDSafHMZBAZDc03LQ3A= +google.golang.org/genproto/googleapis/rpc v0.0.0-20251022142026-3a174f9686a8 h1:M1rk8KBnUsBDg1oPGHNCxG4vc1f49epmTO7xscSajMk= +google.golang.org/genproto/googleapis/rpc v0.0.0-20251022142026-3a174f9686a8/go.mod h1:7i2o+ce6H/6BluujYR+kqX3GKH+dChPTQU19wjRPiGk= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.75.1 h1:/ODCNEuf9VghjgO3rqLcfg8fiOP0nSluljWFlDxELLI= -google.golang.org/grpc v1.75.1/go.mod h1:JtPAzKiq4v1xcAB2hydNlWI2RnF85XXcV0mhKXr2ecQ= -google.golang.org/protobuf v1.36.9 h1:w2gp2mA27hUeUzj9Ex9FBjsBm40zfaDtEWow293U7Iw= -google.golang.org/protobuf v1.36.9/go.mod h1:fuxRtAxBytpl4zzqUh6/eyUujkJdNiuEkXntxiD/uRU= +google.golang.org/grpc v1.77.0 h1:wVVY6/8cGA6vvffn+wWK5ToddbgdU3d8MNENr4evgXM= +google.golang.org/grpc v1.77.0/go.mod h1:z0BY1iVj0q8E1uSQCjL9cppRj+gnZjzDnzV0dHhrNig= +google.golang.org/protobuf v1.36.10 h1:AYd7cD/uASjIL6Q9LiTjz8JLcrh/88q5UObnmY3aOOE= +google.golang.org/protobuf v1.36.10/go.mod h1:HTf+CrKn2C3g5S8VImy6tdcUvCska2kB7j23XfzDpco= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/main.go b/main.go index deccf3c5..724bd8fe 100644 --- a/main.go +++ b/main.go @@ -30,6 +30,7 @@ import ( "github.com/go-logr/logr" "github.com/sdcio/data-server/pkg/config" "github.com/sdcio/data-server/pkg/server" + "github.com/sdcio/logger" logf "github.com/sdcio/logger" "github.com/spf13/pflag" ) @@ -60,10 +61,10 @@ func main() { ReplaceAttr: logf.ReplaceTimeAttr, } if debug { - slogOpts.Level = slog.Level(logf.VDebug) + slogOpts.Level = slog.Level(-logger.VDebug) } if trace { - slogOpts.Level = slog.Level(logf.VTrace) + slogOpts.Level = slog.Level(-logger.VTrace) } log := logr.FromSlogHandler(slog.NewJSONHandler(os.Stdout, slogOpts)) diff --git a/pkg/datastore/datastore_rpc.go b/pkg/datastore/datastore_rpc.go index 0bdde54f..b488217a 100644 --- a/pkg/datastore/datastore_rpc.go +++ b/pkg/datastore/datastore_rpc.go @@ -75,6 +75,8 @@ type Datastore struct { // func New(c *config.DatastoreConfig, schemaServer *config.RemoteSchemaServer) *Datastore { func New(ctx context.Context, c *config.DatastoreConfig, sc schema.Client, cc cache.Client, opts ...grpc.DialOption) (*Datastore, error) { + ctx, cancel := context.WithCancel(ctx) + log := logf.FromContext(ctx) log = log.WithName("datastore").WithValues( "datastore-name", c.Name, @@ -94,13 +96,12 @@ func New(ctx context.Context, c *config.DatastoreConfig, sc schema.Client, cc ca tc := tree.NewTreeContext(scb, tree.RunningIntentName) syncTreeRoot, err := tree.NewTreeRoot(ctx, tc) if err != nil { + cancel() return nil, err } ccb := cache.NewCacheClientBound(c.Name, cc) - ctx, cancel := context.WithCancel(ctx) - ds := &Datastore{ config: c, schemaClient: scb, diff --git a/pkg/datastore/deviations.go b/pkg/datastore/deviations.go index 083f5bef..e41f5e7f 100644 --- a/pkg/datastore/deviations.go +++ b/pkg/datastore/deviations.go @@ -68,7 +68,7 @@ func (d *Datastore) DeviationMgr(ctx context.Context, c *config.DeviationConfig) for peerIdentifier, devStream := range d.deviationClients { deviationClients[peerIdentifier] = devStream if devStream.Context().Err() != nil { - log.V(logf.VWarn).Error(devStream.Context().Err(), "deviation client context error", "client", peerIdentifier, "error", devStream.Context().Err()) + log.Error(devStream.Context().Err(), "deviation client context error", "severity", "WARN", "client", peerIdentifier, "error", devStream.Context().Err()) continue } deviationClientNames = append(deviationClientNames, peerIdentifier) @@ -122,8 +122,10 @@ func (d *Datastore) SendDeviations(ctx context.Context, ch <-chan *treetypes.Dev continue } - if log.GetV() <= logger.VTrace && deviation.Reason() == treetypes.DeviationReasonNotApplied { - log.V(logger.VTrace).Info("NOT APPLIED", "path", deviation.Path().ToXPath(false), "actual value", deviation.CurrentValue().ToString(), "expected value", deviation.ExpectedValue().ToString(), "intent", deviation.IntentName()) + if log := log.V(logger.VTrace); log.Enabled() { + if deviation.Reason() == treetypes.DeviationReasonNotApplied { // TODO add check for trace level Trace + log.Info("NOT APPLIED", "path", deviation.Path().ToXPath(false), "actual value", deviation.CurrentValue().ToString(), "expected value", deviation.ExpectedValue().ToString(), "intent", deviation.IntentName()) + } } err := dc.Send(&sdcpb.WatchDeviationResponse{ @@ -157,6 +159,8 @@ type DeviationEntry interface { func (d *Datastore) calculateDeviations(ctx context.Context) (<-chan *treetypes.DeviationEntry, error) { + log := logger.FromContext(ctx) + d.syncTreeMutex.RLock() deviationTree, err := d.syncTree.DeepCopy(ctx) d.syncTreeMutex.RUnlock() @@ -174,6 +178,10 @@ func (d *Datastore) calculateDeviations(ctx context.Context) (<-chan *treetypes. return nil, err } + if log := log.V(logger.VTrace); log.Enabled() { + log.Info("deviation tree", "content", deviationTree.String()) + } + deviationChan := make(chan *treetypes.DeviationEntry, 10) go func() { defer close(deviationChan) diff --git a/pkg/datastore/sync.go b/pkg/datastore/sync.go index fc4b3aa4..bb43a051 100644 --- a/pkg/datastore/sync.go +++ b/pkg/datastore/sync.go @@ -20,7 +20,7 @@ func (d *Datastore) ApplyToRunning(ctx context.Context, deletes []*sdcpb.Path, i for _, delete := range deletes { err := d.syncTree.DeleteBranch(ctx, delete, tree.RunningIntentName) if err != nil { - log.V(logger.VWarn).Error(err, "failed deleting path from datastore sync tree", "path", delete.ToXPath(false)) + log.Error(err, "failed deleting path from datastore sync tree", "severity", "WARN", "path", delete.ToXPath(false)) continue } } @@ -33,15 +33,14 @@ func (d *Datastore) ApplyToRunning(ctx context.Context, deletes []*sdcpb.Path, i } // conditional trace logging - if log.GetV() <= logger.VTrace { + if log := log.V(logger.VTrace); log.Enabled() { treeExport, err := d.syncTree.TreeExport(tree.RunningIntentName, tree.RunningValuesPrio, false) if err == nil { json, err := protojson.MarshalOptions{Multiline: false}.Marshal(treeExport) if err == nil { - log.V(logger.VTrace).Info("synctree after sync apply", "content", string(json)) + log.Info("synctree after sync apply", "content", string(json)) } } - } return nil diff --git a/pkg/datastore/target/gnmi/get.go b/pkg/datastore/target/gnmi/get.go index d64c38c6..d76bc0ad 100644 --- a/pkg/datastore/target/gnmi/get.go +++ b/pkg/datastore/target/gnmi/get.go @@ -148,9 +148,9 @@ func (s *GetSync) internalGetSync(req *sdcpb.GetDataRequest) { return } - if log.V(logger.VTrace).Enabled() { + if log := log.V(logger.VTrace); log.Enabled() { data, _ := protojson.Marshal(result) - log.V(logger.VTrace).Info("sync content", "data", string(data)) + log.Info("sync content", "data", string(data)) } err = s.runningStore.ApplyToRunning(s.ctx, s.paths, proto.NewProtoTreeImporter(result)) diff --git a/pkg/datastore/target/target.go b/pkg/datastore/target/target.go index aced9696..993069ae 100644 --- a/pkg/datastore/target/target.go +++ b/pkg/datastore/target/target.go @@ -18,6 +18,7 @@ import ( "context" "fmt" + "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "google.golang.org/grpc" @@ -68,7 +69,10 @@ func New(ctx context.Context, name string, cfg *config.SBI, schemaClient schemaC return nil, fmt.Errorf("unknown DS target type %q", cfg.Type) } - err = t.AddSyncs(ctx, syncConfigs...) + syncLog := logger.FromContext(ctx).WithName("sync") + syncCtx := logger.IntoContext(ctx, syncLog) + + err = t.AddSyncs(syncCtx, syncConfigs...) if err != nil { return nil, err } diff --git a/pkg/datastore/transaction_rpc.go b/pkg/datastore/transaction_rpc.go index ceea0b19..4cb07e20 100644 --- a/pkg/datastore/transaction_rpc.go +++ b/pkg/datastore/transaction_rpc.go @@ -230,12 +230,6 @@ func (d *Datastore) lowlevelTransactionSet(ctx context.Context, transaction *typ return nil, err } - // marksOwnerDeleteVisitor := tree.NewMarkOwnerDeleteVisitor(intent.GetName(), intent.GetOnlyIntended()) - // err := root.Walk(ctx, marksOwnerDeleteVisitor) - // if err != nil { - // return nil, err - // } - // clear the owners existing explicit delete entries, retrieving the old entries for storing in the transaction for possible rollback oldExplicitDeletes := root.RemoveExplicitDeletes(intent.GetName()) @@ -432,10 +426,10 @@ func (d *Datastore) writeBackSyncTree(ctx context.Context, updates tree.LeafVari } // conditional trace logging - if log.GetV() <= logger.VTrace { + if log := log.V(logger.VTrace); log.Enabled() { json, err := protojson.MarshalOptions{Multiline: false}.Marshal(newRunningIntent) - if err != nil { - log.V(logger.VTrace).Info("writeback synctree", "content", string(json)) + if err == nil { + log.Info("writeback synctree", "content", string(json)) } } diff --git a/pkg/server/datastore.go b/pkg/server/datastore.go index 5ef9c0a3..b4de173a 100644 --- a/pkg/server/datastore.go +++ b/pkg/server/datastore.go @@ -88,16 +88,16 @@ func (s *Server) CreateDataStore(ctx context.Context, req *sdcpb.CreateDataStore name := req.GetDatastoreName() lName := len(name) if lName == 0 { - log.V(logf.VError).Info("missing datastore name attribute") + log.Error(nil, "missing datastore name attribute") return nil, status.Error(codes.InvalidArgument, "missing datastore name attribute") } if lName > math.MaxUint16 { - log.V(logf.VError).Info("datastore name attribute too long") + log.Error(nil, "datastore name attribute too long") return nil, status.Error(codes.InvalidArgument, "datastore name attribute too long") } if _, err := s.datastores.GetDataStore(name); err == nil { - log.V(logf.VError).Info("datastore already exists") + log.Error(nil, "datastore already exists") return nil, status.Errorf(codes.InvalidArgument, "datastore %s already exists", name) } @@ -115,7 +115,7 @@ func (s *Server) CreateDataStore(ctx context.Context, req *sdcpb.CreateDataStore case sdcpb.CommitCandidate_COMMIT_RUNNING: commitDatastore = "running" default: - log.V(logf.VError).Info("unknown commitDatastore", "datastore", req.GetTarget().GetNetconfOpts().GetCommitCandidate()) + log.Error(nil, "unknown commitDatastore", "datastore", req.GetTarget().GetNetconfOpts().GetCommitCandidate()) return nil, fmt.Errorf("unknown commitDatastore: %v", req.GetTarget().GetNetconfOpts().GetCommitCandidate()) } sbi.NetconfOptions = &config.SBINetconfOptions{ @@ -130,7 +130,7 @@ func (s *Server) CreateDataStore(ctx context.Context, req *sdcpb.CreateDataStore Encoding: req.GetTarget().GetGnmiOpts().GetEncoding(), } default: - log.V(logf.VError).Info("unknowm targetconnection protocol type", "type", req.GetTarget().GetType()) + log.Error(nil, "unknowm targetconnection protocol type", "type", req.GetTarget().GetType()) return nil, fmt.Errorf("unknowm targetconnection protocol type %s", req.GetTarget().GetType()) } @@ -190,7 +190,7 @@ func (s *Server) CreateDataStore(ctx context.Context, req *sdcpb.CreateDataStore gnSyncConfig.Encoding = pSync.GetTarget().GetGnmiOpts().GetEncoding() case "netconf": default: - log.V(logf.VError).Info("unknown targetsync protocol type", "protocol", pSync.GetTarget().GetType()) + log.Error(nil, "unknown targetsync protocol type", "protocol", pSync.GetTarget().GetType()) return nil, status.Errorf(codes.InvalidArgument, "unknown targetsync protocol: %q", pSync.GetTarget().GetType()) } dsConfig.Sync.Config = append(dsConfig.Sync.Config, gnSyncConfig) diff --git a/pkg/server/transaction.go b/pkg/server/transaction.go index e58d9c41..78c1d6e4 100644 --- a/pkg/server/transaction.go +++ b/pkg/server/transaction.go @@ -22,14 +22,14 @@ func (s *Server) TransactionSet(ctx context.Context, req *sdcpb.TransactionSetRe log := logf.FromContext(ctx).WithName("TransactionSet") log = log.WithName("transaction").WithValues( "transaction-id", req.GetTransactionId(), + "datastore-name", req.GetDatastoreName(), ) ctx = logf.IntoContext(ctx, log) log.Info("TransactionSet", - "transaction-datastore-name", req.GetDatastoreName(), "transaction-dry-run", req.GetDryRun(), "transaction-timeout", req.GetTimeout(), - "transaction-peer", pr.String(), + "transaction-peer", pr.Addr.String(), ) log.V(logf.VDebug).Info("received request", "raw-request", utils.FormatProtoJSON(req)) @@ -89,12 +89,12 @@ func (s *Server) TransactionConfirm(ctx context.Context, req *sdcpb.TransactionC log := logf.FromContext(ctx).WithName("TransactionConfirm") log = log.WithName("transaction").WithValues( "transaction-id", req.GetTransactionId(), + "datastore-name", req.GetDatastoreName(), ) ctx = logf.IntoContext(ctx, log) log.Info("TransactionConfirm", - "transaction-datastore-name", req.GetDatastoreName(), - "transaction-peer", pr.String(), + "transaction-peer", pr.Addr.String(), ) log.V(logf.VDebug).Info("received request", "raw-request", utils.FormatProtoJSON(req)) @@ -118,12 +118,12 @@ func (s *Server) TransactionCancel(ctx context.Context, req *sdcpb.TransactionCa log := logf.FromContext(ctx).WithName("TransactionCancel") log = log.WithName("transaction").WithValues( "transaction-id", req.GetTransactionId(), + "datastore-name", req.GetDatastoreName(), ) ctx = logf.IntoContext(ctx, log) log.Info("TransactionCancel", - "transaction-datastore-name", req.GetDatastoreName(), - "transaction-peer", pr.String(), + "transaction-peer", pr.Addr.String(), ) log.V(logf.VDebug).Info("received request", "raw-request", utils.FormatProtoJSON(req)) diff --git a/pkg/tree/processor_mark_owner_delete.go b/pkg/tree/processor_mark_owner_delete.go index 1453d4e7..2dcf5c49 100644 --- a/pkg/tree/processor_mark_owner_delete.go +++ b/pkg/tree/processor_mark_owner_delete.go @@ -30,8 +30,7 @@ func (o *MarkOwnerDeleteProcessor) Run(e Entry, pool pool.VirtualPoolI) error { // wait for the pool to run dry pool.Wait() - return nil - + return pool.FirstError() } type OwnerDeleteMarkerTaskConfig struct { diff --git a/pkg/tree/root_entry.go b/pkg/tree/root_entry.go index 093ebb0d..8eea8a8f 100644 --- a/pkg/tree/root_entry.go +++ b/pkg/tree/root_entry.go @@ -254,7 +254,7 @@ func (r *RootEntry) FinishInsertionPhase(ctx context.Context) error { // navigate to the stated path entry, err := r.NavigateSdcpbPath(ctx, path) if err != nil { - log.V(logf.VWarn).Info("Applying explicit delete - path not found, skipping", "path", path.ToXPath(false)) + log.Error(nil, "Applying explicit delete - path not found, skipping", "severity", "WARN", "path", path.ToXPath(false)) } // walk the whole branch adding the explicit delete leafvariant diff --git a/pkg/utils/converter.go b/pkg/utils/converter.go index 5170ff2b..8586e586 100644 --- a/pkg/utils/converter.go +++ b/pkg/utils/converter.go @@ -11,7 +11,6 @@ import ( "strings" "github.com/sdcio/logger" - logf "github.com/sdcio/logger" sdcpb "github.com/sdcio/sdc-protos/sdcpb" "google.golang.org/protobuf/proto" ) @@ -49,7 +48,7 @@ func (c *Converter) ExpandUpdates(ctx context.Context, updates []*sdcpb.Update) // expandUpdate Expands the value, in case of json to single typed value updates func (c *Converter) ExpandUpdate(ctx context.Context, upd *sdcpb.Update) ([]*sdcpb.Update, error) { - log := logf.FromContext(ctx) + log := logger.FromContext(ctx) upds := make([]*sdcpb.Update, 0) rsp, err := c.schemaClientBound.GetSchemaSdcpbPath(ctx, upd.GetPath()) if err != nil { @@ -131,7 +130,7 @@ func (c *Converter) ExpandUpdate(ctx context.Context, upd *sdcpb.Update) ([]*sdc var jsonValue []byte if upd.GetValue() == nil { - log.V(logger.VError).Info("Value is nil", "Path", upd.Path.ToXPath(false)) + log.Error(nil, "value is nil", "path", upd.Path.ToXPath(false)) return nil, nil } switch upd.GetValue().Value.(type) { @@ -218,7 +217,7 @@ func (c *Converter) ExpandUpdateKeysAsLeaf(ctx context.Context, upd *sdcpb.Updat } func (c *Converter) ExpandContainerValue(ctx context.Context, p *sdcpb.Path, jv any, cs *sdcpb.SchemaElem_Container) ([]*sdcpb.Update, error) { - log := logf.FromContext(ctx) + log := logger.FromContext(ctx) // log.Debugf("expanding jsonVal %T | %v | %v", jv, jv, p) switch jv := jv.(type) { case string: @@ -611,7 +610,7 @@ func getLeafList(s string, cs *sdcpb.SchemaElem_Container) (*sdcpb.LeafListSchem } func getChild(ctx context.Context, name string, cs *sdcpb.SchemaElem_Container, scb SchemaClientBound) (any, bool) { - log := logf.FromContext(ctx) + log := logger.FromContext(ctx) searchNames := []string{name} if i := strings.Index(name, ":"); i >= 0 {