From 553f5ca0f1d305236b1f19d51bfe676df37e56b0 Mon Sep 17 00:00:00 2001 From: Vladimir Lavor Date: Thu, 22 Feb 2024 16:32:32 +0100 Subject: [PATCH] Trace refactored (#116) * govpp trace refactored Signed-off-by: Vladimir Lavor * fix test Signed-off-by: Vladimir Lavor * simplify Signed-off-by: Vladimir Lavor * some fixes Signed-off-by: Vladimir Lavor * init records list out of the lock Signed-off-by: Vladimir Lavor * Added succeed/fail flag to traced message Signed-off-by: Vladimir Lavor * decrease the size of the trace prealocation Signed-off-by: Vladimir Lavor * make it safe to init trace during api calls Signed-off-by: Vladimir Lavor * Do not use trace lock if trace is disabled Signed-off-by: Vladimir Lavor * fix data race on trace nil check Signed-off-by: Vladimir Lavor --------- Signed-off-by: Vladimir Lavor Co-authored-by: Nathan Skrzypczak --- api/trace.go | 21 ++-- core/channel.go | 1 - core/connection.go | 30 +----- core/request_handler.go | 57 ++++++++-- core/trace.go | 109 +++++++++++++++----- core/trace_test.go | 177 ++++++++++++++++++++------------ examples/api-trace/README.md | 37 ++++--- examples/api-trace/api-trace.go | 66 ++++++------ test/integration/trace_test.go | 5 +- 9 files changed, 315 insertions(+), 188 deletions(-) diff --git a/api/trace.go b/api/trace.go index 4ff46e82..ae28f88a 100644 --- a/api/trace.go +++ b/api/trace.go @@ -18,29 +18,30 @@ import ( "time" ) -// Trace gives access to the API trace tool, capturing outcoming and incoming messages -// to and from GoVPP. +// Trace is the GoVPP utility tool capturing processed API messages. The trace is not operational +// by default. +// Enable trace for a given connection by calling `NewTrace(connection, size)` type Trace interface { - // Enable allows to enable or disable API trace for a connection. - Enable(enable bool) - - // GetRecords retrieves all messages collected (from all channels if they are used) - // since the point the trace was enabled or cleared. + // GetRecords returns all API messages from all channels captured since the trace + // was initialized or cleared up to the point of the call. GetRecords() []*Record - // GetRecordsForChannel retrieves messages collected by the given channel since - // the point the trace was enabled or cleared. + // GetRecordsForChannel returns all API messages recorded by the given channel. GetRecordsForChannel(chId uint16) []*Record // Clear erases messages captured so far. Clear() + + // Close the tracer and release associated resources + Close() } -// Record contains essential information about traced message, its timestamp and whether +// Record contains essential information about the traced message, its timestamp and whether // the message was received or sent type Record struct { Message Message Timestamp time.Time IsReceived bool ChannelID uint16 + Succeeded bool } diff --git a/core/channel.go b/core/channel.go index 40f882a1..7f279dad 100644 --- a/core/channel.go +++ b/core/channel.go @@ -249,7 +249,6 @@ func (req *requestCtx) ReceiveReply(msg api.Message) error { } else if lastReplyReceived { return errors.New("multipart reply recieved while a single reply expected") } - return nil } diff --git a/core/connection.go b/core/connection.go index 6c24c029..f04efd05 100644 --- a/core/connection.go +++ b/core/connection.go @@ -135,7 +135,8 @@ type Connection struct { msgControlPing api.Message msgControlPingReply api.Message - apiTrace *trace // API tracer (disabled by default) + traceLock sync.Mutex + trace *Trace // API tracer (disabled by default) } type backgroundLoopStatus int @@ -168,11 +169,7 @@ func newConnection(binapi adapter.VppAPI, attempts int, interval time.Duration, subscriptions: make(map[uint16][]*subscriptionCtx), msgControlPing: msgControlPing, msgControlPingReply: msgControlPingReply, - apiTrace: &trace{ - list: make([]*api.Record, 0), - mux: &sync.Mutex{}, - }, - channelIdPool: newIDPool(0x7fff), + channelIdPool: newIDPool(0x7fff), } c.channelPool = genericpool.New[*Channel](func() *Channel { if isDebugOn(debugOptChannels) { @@ -583,24 +580,3 @@ func (c *Connection) sendConnEvent(event ConnectionEvent) { log.Warn("Connection state channel is full, discarding value.") } } - -// Trace gives access to the API trace interface -func (c *Connection) Trace() api.Trace { - return c.apiTrace -} - -// trace records api message -func (c *Connection) trace(msg api.Message, chId uint16, t time.Time, isReceived bool) { - if atomic.LoadInt32(&c.apiTrace.isEnabled) == 0 { - return - } - entry := &api.Record{ - Message: msg, - Timestamp: t, - IsReceived: isReceived, - ChannelID: chId, - } - c.apiTrace.mux.Lock() - c.apiTrace.list = append(c.apiTrace.list, entry) - c.apiTrace.mux.Unlock() -} diff --git a/core/request_handler.go b/core/request_handler.go index 8c9f0a3c..9dc5bdb4 100644 --- a/core/request_handler.go +++ b/core/request_handler.go @@ -110,9 +110,21 @@ func (c *Connection) processRequest(ch *Channel, req *vppRequest) error { } // send the request to VPP - t := time.Now() - err = c.vppClient.SendMsg(context, data) - if err != nil { + if err = func() (err error) { + timestamp, enabled := c.trace.registerNew() + err = c.vppClient.SendMsg(context, data) + if enabled { + c.traceLock.Lock() + defer c.traceLock.Unlock() + c.trace.send(&api.Record{ + Message: req.msg, + Timestamp: timestamp, + ChannelID: ch.id, + Succeeded: err == nil, + }) + } + return + }(); err != nil { log.WithFields(logger.Fields{ "channel": ch.id, "msg_id": msgID, @@ -126,7 +138,6 @@ func (c *Connection) processRequest(ch *Channel, req *vppRequest) error { }).Warnf("Unable to send message") return err } - c.trace(req.msg, ch.id, t, false) if req.multi { // send a control ping to determine end of the multipart response @@ -143,16 +154,28 @@ func (c *Connection) processRequest(ch *Channel, req *vppRequest) error { "data_len": len(pingData), }).Debugf(" -> SEND MSG: %T", c.msgControlPing) } - - t = time.Now() - if err := c.vppClient.SendMsg(context, pingData); err != nil { + // send the control ping request to VPP + if err = func() (err error) { + timestamp, enabled := c.trace.registerNew() + err = c.vppClient.SendMsg(context, pingData) + if enabled { + c.traceLock.Lock() + defer c.traceLock.Unlock() + c.trace.send(&api.Record{ + Message: c.msgControlPing, + Timestamp: timestamp, + ChannelID: ch.id, + Succeeded: err == nil, + }) + } + return + }(); err != nil { log.WithFields(logger.Fields{ "context": context, "seq_num": req.seqNum, "error": err, }).Warnf("unable to send control ping") } - c.trace(c.msgControlPing, ch.id, t, false) } return nil @@ -188,11 +211,25 @@ func (c *Connection) msgCallback(msgID uint16, data []byte) { // decode and trace the message msg = reflect.New(reflect.TypeOf(msg).Elem()).Interface().(api.Message) - if err = c.codec.DecodeMsg(data, msg); err != nil { + if err = func() (err error) { + timestamp, enabled := c.trace.registerNew() + err = c.codec.DecodeMsg(data, msg) + if enabled { + c.traceLock.Lock() + defer c.traceLock.Unlock() + c.trace.send(&api.Record{ + Message: msg, + Timestamp: timestamp, + IsReceived: true, + ChannelID: chanID, + Succeeded: err == nil, + }) + } + return + }(); err != nil { log.WithField("msg", msg).Warnf("Unable to decode message: %v", err) return } - c.trace(msg, chanID, time.Now(), true) if log.Level == logger.DebugLevel { // for performance reasons - logrus does some processing even if debugs are disabled log.WithFields(logger.Fields{ diff --git a/core/trace.go b/core/trace.go index 5c5bdddc..cd8a5518 100644 --- a/core/trace.go +++ b/core/trace.go @@ -18,51 +18,104 @@ import ( "go.fd.io/govpp/api" "sort" "sync" - "sync/atomic" + "time" ) -// trace is the API tracer object synchronizing and keeping recoded messages. -type trace struct { - list []*api.Record - mux *sync.Mutex +// default buffer size +const bufferSize = 100 - isEnabled int32 +// Trace is the default trace API implementation. +type Trace struct { + *sync.RWMutex + wg sync.WaitGroup + + records []*api.Record + buffer chan *api.Record + index int + + closeFunc func() } -func (c *trace) Enable(enable bool) { - if enable && atomic.CompareAndSwapInt32(&c.isEnabled, 0, 1) { - log.Debugf("API trace enabled") - } else if atomic.CompareAndSwapInt32(&c.isEnabled, 1, 0) { - log.Debugf("API trace disabled") +// NewTrace initializes the trace object, always bound to a GoVPP connection. +// The size limits the number of records stored. +// Initializing a new trace for the same connection replaces the old one and +// discards all values already collected. +func NewTrace(c *Connection, size int) (t *Trace) { + t = &Trace{ + RWMutex: &sync.RWMutex{}, + records: make([]*api.Record, size), + buffer: make(chan *api.Record, bufferSize), } + c.traceLock.Lock() + c.trace = t + c.traceLock.Unlock() + t.closeFunc = func() { + c.trace = nil // no more records + close(t.buffer) + } + go func() { + for { + record, ok := <-t.buffer + if !ok { + return + } + if t.index < len(t.records) { + t.Lock() + t.records[t.index] = record + t.index++ + t.Unlock() + } + t.wg.Done() + } + }() + return } -func (c *trace) GetRecords() (list []*api.Record) { - c.mux.Lock() - list = append(list, c.list...) - c.mux.Unlock() +func (t *Trace) GetRecords() (list []*api.Record) { + // it is supposed to wait until all API messages sent to the + // buffer are processed before returning the list + t.wg.Wait() + list = make([]*api.Record, t.index) + t.RLock() + copy(list, t.records[:t.index]) + t.RUnlock() sort.Slice(list, func(i, j int) bool { return list[i].Timestamp.Before(list[j].Timestamp) }) return list } -func (c *trace) GetRecordsForChannel(chId uint16) (list []*api.Record) { - c.mux.Lock() - for _, entry := range c.list { - if entry.ChannelID == chId { - list = append(list, entry) +func (t *Trace) GetRecordsForChannel(chId uint16) (list []*api.Record) { + records := t.GetRecords() + for _, record := range records { + if record.ChannelID == chId { + list = append(list, record) } } - c.mux.Unlock() - sort.Slice(list, func(i, j int) bool { - return list[i].Timestamp.Before(list[j].Timestamp) - }) return list } -func (c *trace) Clear() { - c.mux.Lock() - c.list = make([]*api.Record, 0) - c.mux.Unlock() +func (t *Trace) Clear() { + t.Lock() + t.records = make([]*api.Record, len(t.records)) + t.index = 0 + t.Unlock() +} + +func (t *Trace) Close() { + t.closeFunc() +} + +func (t *Trace) registerNew() (now time.Time, enabled bool) { + if t != nil { + t.wg.Add(1) + enabled = true + } + return time.Now(), enabled +} + +func (t *Trace) send(record *api.Record) { + if t != nil { + t.buffer <- record + } } diff --git a/core/trace_test.go b/core/trace_test.go index da0bcc2a..4932c7f6 100644 --- a/core/trace_test.go +++ b/core/trace_test.go @@ -1,11 +1,7 @@ package core_test import ( - "strings" - "testing" - . "github.com/onsi/gomega" - "go.fd.io/govpp/api" interfaces "go.fd.io/govpp/binapi/interface" "go.fd.io/govpp/binapi/ip" @@ -13,27 +9,30 @@ import ( "go.fd.io/govpp/binapi/memclnt" "go.fd.io/govpp/binapi/memif" "go.fd.io/govpp/core" + "strings" + "testing" ) -func TestTraceEnabled(t *testing.T) { - t.Skipf("these randomly fail, see integration tests") +const traceSize = 10 +func TestTraceEnabled(t *testing.T) { ctx := setupTest(t, false) defer ctx.teardownTest() - Expect(ctx.conn.Trace()).ToNot(BeNil()) - ctx.conn.Trace().Enable(true) + trace := core.NewTrace(ctx.conn, traceSize) + Expect(trace).ToNot(BeNil()) + defer trace.Close() request := []api.Message{ &interfaces.CreateLoopback{}, &memif.MemifCreate{}, - &l2.BridgeDomainAddDel{}, + &l2.BridgeDomainAddDelV2{}, &ip.IPTableAddDel{}, } reply := []api.Message{ &interfaces.CreateLoopbackReply{}, &memif.MemifCreateReply{}, - &l2.BridgeDomainAddDelReply{}, + &l2.BridgeDomainAddDelV2Reply{}, &ip.IPTableAddDelReply{}, } @@ -42,13 +41,13 @@ func TestTraceEnabled(t *testing.T) { err := ctx.ch.SendRequest(request[i]).ReceiveReply(reply[i]) Expect(err).To(BeNil()) } - - traced := ctx.conn.Trace().GetRecords() - Expect(traced).ToNot(BeNil()) - Expect(traced).To(HaveLen(8)) - for i, entry := range traced { + records := trace.GetRecords() + Expect(records).ToNot(BeNil()) + Expect(records).To(HaveLen(len(request) + len(reply))) + for i, entry := range records { Expect(entry.Timestamp).ToNot(BeNil()) Expect(entry.Message.GetMessageName()).ToNot(Equal("")) + Expect(entry.Succeeded).To(BeTrue()) if strings.HasSuffix(entry.Message.GetMessageName(), "_reply") || strings.HasSuffix(entry.Message.GetMessageName(), "_details") { Expect(entry.IsReceived).To(BeTrue()) @@ -64,12 +63,12 @@ func TestTraceEnabled(t *testing.T) { } func TestMultiRequestTraceEnabled(t *testing.T) { - t.Skipf("these randomly fail, see integration tests") - ctx := setupTest(t, false) defer ctx.teardownTest() - ctx.conn.Trace().Enable(true) + trace := core.NewTrace(ctx.conn, traceSize) + Expect(trace).ToNot(BeNil()) + defer trace.Close() request := []api.Message{ &interfaces.SwInterfaceDump{}, @@ -101,48 +100,49 @@ func TestMultiRequestTraceEnabled(t *testing.T) { i++ } - traced := ctx.conn.Trace().GetRecords() - Expect(traced).ToNot(BeNil()) - Expect(traced).To(HaveLen(6)) - for _, entry := range traced { + records := trace.GetRecords() + Expect(records).ToNot(BeNil()) + Expect(records).To(HaveLen(6)) + for eIdx, entry := range records { Expect(entry.Timestamp).ToNot(BeNil()) Expect(entry.Message.GetMessageName()).ToNot(Equal("")) + Expect(entry.Succeeded).To(BeTrue()) if strings.HasSuffix(entry.Message.GetMessageName(), "_reply") || strings.HasSuffix(entry.Message.GetMessageName(), "_details") { Expect(entry.IsReceived).To(BeTrue()) } else { Expect(entry.IsReceived).To(BeFalse()) } - // FIXME: the way mock adapter works now prevents having the exact same order for each execution - /*if i == 0 { - Expect(request[0].GetMessageName()).To(Equal(entry.Message.GetMessageName())) - } else if i == len(traced)-1 { - msg := memclnt.ControlPing{} - Expect(msg.GetMessageName()).To(Equal(entry.Message.GetMessageName())) - } else { - Expect(reply[i-1].GetMessageName()).To(Equal(entry.Message.GetMessageName())) - }*/ + if eIdx == 0 { + Expect(request[0].GetMessageName()).To(Equal(entry.Message.GetMessageName())) + } else if eIdx == len(records)-2 { + msg := memclnt.ControlPing{} + Expect(msg.GetMessageName()).To(Equal(entry.Message.GetMessageName())) + } else if eIdx == len(records)-1 { + msg := memclnt.ControlPingReply{} + Expect(msg.GetMessageName()).To(Equal(entry.Message.GetMessageName())) + } else { + Expect(reply[i-1].GetMessageName()).To(Equal(entry.Message.GetMessageName())) + } } } func TestTraceDisabled(t *testing.T) { - t.Skipf("these randomly fail, see integration tests") - ctx := setupTest(t, false) defer ctx.teardownTest() - ctx.conn.Trace().Enable(false) + // do not enable trace request := []api.Message{ &interfaces.CreateLoopback{}, &memif.MemifCreate{}, - &l2.BridgeDomainAddDel{}, + &l2.BridgeDomainAddDelV2{}, &ip.IPTableAddDel{}, } reply := []api.Message{ &interfaces.CreateLoopbackReply{}, &memif.MemifCreateReply{}, - &l2.BridgeDomainAddDelReply{}, + &l2.BridgeDomainAddDelV2Reply{}, &ip.IPTableAddDelReply{}, } @@ -152,17 +152,22 @@ func TestTraceDisabled(t *testing.T) { Expect(err).To(BeNil()) } - traced := ctx.conn.Trace().GetRecords() - Expect(traced).To(BeNil()) + trace := core.NewTrace(ctx.conn, traceSize) + Expect(trace).ToNot(BeNil()) + defer trace.Close() + + records := trace.GetRecords() + Expect(records).ToNot(BeNil()) + Expect(records).To(BeEmpty()) } func TestTracePerChannel(t *testing.T) { - t.Skipf("these randomly fail, see integration tests") - ctx := setupTest(t, false) defer ctx.teardownTest() - ctx.conn.Trace().Enable(true) + trace := core.NewTrace(ctx.conn, traceSize) + Expect(trace).ToNot(BeNil()) + defer trace.Close() ch1 := ctx.ch ch2, err := ctx.conn.NewAPIChannel() @@ -171,12 +176,12 @@ func TestTracePerChannel(t *testing.T) { requestCh1 := []api.Message{ &interfaces.CreateLoopback{}, &memif.MemifCreate{}, - &l2.BridgeDomainAddDel{}, + &l2.BridgeDomainAddDelV2{}, } replyCh1 := []api.Message{ &interfaces.CreateLoopbackReply{}, &memif.MemifCreateReply{}, - &l2.BridgeDomainAddDelReply{}, + &l2.BridgeDomainAddDelV2Reply{}, } requestCh2 := []api.Message{ &ip.IPTableAddDel{}, @@ -187,18 +192,18 @@ func TestTracePerChannel(t *testing.T) { for i := 0; i < len(requestCh1); i++ { ctx.mockVpp.MockReply(replyCh1[i]) - err := ch1.SendRequest(requestCh1[i]).ReceiveReply(replyCh1[i]) + err = ch1.SendRequest(requestCh1[i]).ReceiveReply(replyCh1[i]) Expect(err).To(BeNil()) } for i := 0; i < len(requestCh2); i++ { ctx.mockVpp.MockReply(replyCh2[i]) - err := ch2.SendRequest(requestCh2[i]).ReceiveReply(replyCh2[i]) + err = ch2.SendRequest(requestCh2[i]).ReceiveReply(replyCh2[i]) Expect(err).To(BeNil()) } - trace := ctx.conn.Trace().GetRecords() - Expect(trace).ToNot(BeNil()) - Expect(trace).To(HaveLen(8)) + records := trace.GetRecords() + Expect(records).ToNot(BeNil()) + Expect(records).To(HaveLen(8)) // per channel channel1, ok := ch1.(*core.Channel) @@ -206,12 +211,13 @@ func TestTracePerChannel(t *testing.T) { channel2, ok := ch2.(*core.Channel) Expect(ok).To(BeTrue()) - tracedCh1 := ctx.conn.Trace().GetRecordsForChannel(channel1.GetID()) - Expect(tracedCh1).ToNot(BeNil()) - Expect(tracedCh1).To(HaveLen(6)) - for i, entry := range tracedCh1 { + recordsCh1 := trace.GetRecordsForChannel(channel1.GetID()) + Expect(recordsCh1).ToNot(BeNil()) + Expect(recordsCh1).To(HaveLen(6)) + for i, entry := range recordsCh1 { Expect(entry.Timestamp).ToNot(BeNil()) Expect(entry.Message.GetMessageName()).ToNot(Equal("")) + Expect(entry.Succeeded).To(BeTrue()) if strings.HasSuffix(entry.Message.GetMessageName(), "_reply") || strings.HasSuffix(entry.Message.GetMessageName(), "_details") { Expect(entry.IsReceived).To(BeTrue()) @@ -225,10 +231,10 @@ func TestTracePerChannel(t *testing.T) { } } - tracedCh2 := ctx.conn.Trace().GetRecordsForChannel(channel2.GetID()) - Expect(tracedCh2).ToNot(BeNil()) - Expect(tracedCh2).To(HaveLen(2)) - for i, entry := range tracedCh2 { + recordsCh2 := trace.GetRecordsForChannel(channel2.GetID()) + Expect(recordsCh2).ToNot(BeNil()) + Expect(recordsCh2).To(HaveLen(2)) + for i, entry := range recordsCh2 { Expect(entry.Timestamp).ToNot(BeNil()) Expect(entry.Message.GetMessageName()).ToNot(Equal("")) if strings.HasSuffix(entry.Message.GetMessageName(), "_reply") || @@ -246,12 +252,12 @@ func TestTracePerChannel(t *testing.T) { } func TestTraceClear(t *testing.T) { - t.Skipf("these randomly fail, see integration tests") - ctx := setupTest(t, false) defer ctx.teardownTest() - ctx.conn.Trace().Enable(true) + trace := core.NewTrace(ctx.conn, traceSize) + Expect(trace).ToNot(BeNil()) + defer trace.Close() request := []api.Message{ &interfaces.CreateLoopback{}, @@ -268,12 +274,51 @@ func TestTraceClear(t *testing.T) { Expect(err).To(BeNil()) } - traced := ctx.conn.Trace().GetRecords() - Expect(traced).ToNot(BeNil()) - Expect(traced).To(HaveLen(4)) + records := trace.GetRecords() + Expect(records).ToNot(BeNil()) + Expect(records).To(HaveLen(4)) + + trace.Clear() + + for i := 0; i < len(request); i++ { + ctx.mockVpp.MockReply(reply[i]) + err := ctx.ch.SendRequest(request[i]).ReceiveReply(reply[i]) + Expect(err).To(BeNil()) + } + records = trace.GetRecords() + Expect(records).ToNot(BeNil()) + Expect(records).To(HaveLen(4)) + + trace.Clear() + + records = trace.GetRecords() + Expect(records).To(BeEmpty()) +} + +func TestTraceUseIfClosed(t *testing.T) { + ctx := setupTest(t, false) + defer ctx.teardownTest() + + trace := core.NewTrace(ctx.conn, traceSize) + Expect(trace).ToNot(BeNil()) + trace.Close() + + request := []api.Message{ + &interfaces.CreateLoopback{}, + &memif.MemifCreate{}, + } + reply := []api.Message{ + &interfaces.CreateLoopbackReply{}, + &memif.MemifCreateReply{}, + } + + for i := 0; i < len(request); i++ { + ctx.mockVpp.MockReply(reply[i]) + err := ctx.ch.SendRequest(request[i]).ReceiveReply(reply[i]) + Expect(err).To(BeNil()) + } - ctx.conn.Trace().Clear() - traced = ctx.conn.Trace().GetRecords() - Expect(traced).To(BeNil()) - Expect(traced).To(BeEmpty()) + records := trace.GetRecords() + Expect(records).ToNot(BeNil()) + Expect(records).To(BeEmpty()) } diff --git a/examples/api-trace/README.md b/examples/api-trace/README.md index 6cb6ce88..58a2975d 100644 --- a/examples/api-trace/README.md +++ b/examples/api-trace/README.md @@ -1,17 +1,30 @@ # API trace example -The example demonstrates how to use GoVPP API trace functionality. Connection object `core.Connection` contains -API tracer able to record API messages sent to and from VPP. +The example demonstrates how to use the GoVPP API trace functionality. The trace is bound to a connection. +The `core.Connection` includes the API trace object capable of recording API messages processed during the VPP API +message exchange. Traced messages are called `records`. -Access to the tracer is done via `Trace()`. It allows accessing several methods to manage collected entries: -* `Enable()` either enables or disables the trace. Note that the trace is disabled by default and messages are not recorded while so. -* `GetRecords() []*api.Record` provide messages collected since the plugin was enabled or cleared. -* `GetRecordsForChannel() []*api.Record` provide messages collected on the given channel since the plugin was enabled or cleared. -* `Clear()` removes recorded messages. +Each record contains information about the API message, its direction, time, and the channel ID. -A record is represented by `Record` type. It contains information about the message, its direction, time and channel ID. Following fields are available: -* `Message api.Message` returns recorded entry as GoVPP Message. -* `Timestamp time.Time` is the message timestamp. -* `IsReceived bool` is true if the message is a reply or details message, false otherwise. -* `ChannelID uint16` is the ID of channel processing the traced message. +The trace is disabled by default. In order to enable it, call `NewTrace`: +```go +c, err := govpp.Connect(*sockAddr) +if err != nil { +// handler error +} +size := 10 +trace := core.NewTrace(c, size) +defer trace.Close() +``` + +The code above initializes the new tracer which records a certain number of messages defined by the `size`. + +The following methods are available to call on the trace: + +* `GetRecords() []*Record` returns all records beginning with the initialization of the trace till the point of the + method call. The size also restricts the maximum number of records. All records received after the tracer is full are + discarded. +* `GetRecordsForChannel(chId uint16) []*Record` works the same as the method above, but filters messages per channel. +* `Clear()` resets the tracer and allows to reuse it with (the size remains the same). +* `Close()` closes the tracer. \ No newline at end of file diff --git a/examples/api-trace/api-trace.go b/examples/api-trace/api-trace.go index bf430310..8ff13814 100644 --- a/examples/api-trace/api-trace.go +++ b/examples/api-trace/api-trace.go @@ -49,15 +49,19 @@ func main() { } defer conn.Disconnect() - singleChannel(conn) - multiChannel(conn) - stream(conn) + fmt.Printf("=> Enabling API trace...\n") + trace := core.NewTrace(conn, 50) + singleChannel(conn, trace) + multiChannel(conn, trace) + stream(conn, trace) + + trace.Close() fmt.Printf("Api-trace tool example finished\n\n") } -func singleChannel(conn *core.Connection) { - // create new channel and perform simple compatibility check +func singleChannel(conn *core.Connection, trace api.Trace) { + // create the new channel and perform simple compatibility check ch, err := conn.NewAPIChannel() if err != nil { log.Fatalln("ERROR: creating channel failed:", err) @@ -65,12 +69,10 @@ func singleChannel(conn *core.Connection) { defer ch.Close() fmt.Printf("=> Example 1\n\nEnabling API trace...\n") - conn.Trace().Enable(true) - - if err := ch.CheckCompatiblity(vpe.AllMessages()...); err != nil { + if err = ch.CheckCompatiblity(vpe.AllMessages()...); err != nil { log.Fatalf("compatibility check failed: %v", err) } - if err := ch.CheckCompatiblity(interfaces.AllMessages()...); err != nil { + if err = ch.CheckCompatiblity(interfaces.AllMessages()...); err != nil { log.Printf("compatibility check failed: %v", err) } @@ -83,18 +85,18 @@ func singleChannel(conn *core.Connection) { deleteLoopback(ch, idx) fmt.Println() - fmt.Printf("API trace (api calls: %d):\n", len(conn.Trace().GetRecords())) + fmt.Printf("API trace (api calls: %d):\n", len(trace.GetRecords())) fmt.Printf("--------------------\n") - for _, item := range conn.Trace().GetRecords() { + for _, item := range trace.GetRecords() { printTrace(item) } fmt.Printf("--------------------\n") fmt.Printf("Clearing API trace...\n\n") - conn.Trace().Clear() + trace.Clear() } -func multiChannel(conn *core.Connection) { +func multiChannel(conn *core.Connection, trace api.Trace) { ch1, err := conn.NewAPIChannel() if err != nil { log.Fatalln("ERROR: creating channel failed:", err) @@ -106,7 +108,7 @@ func multiChannel(conn *core.Connection) { } defer ch2.Close() - //do API calls again + // do API call again fmt.Printf("=> Example 2\n\nCalling VPP API (multi-channel)...\n") retrieveVersion(ch1) idx1 := createLoopback(ch1) @@ -127,31 +129,31 @@ func multiChannel(conn *core.Connection) { log.Fatalln("ERROR: incorrect type of channel 2:", err) } - fmt.Printf("API trace for channel 1 (api calls: %d):\n", len(conn.Trace().GetRecordsForChannel(chan1.GetID()))) + fmt.Printf("API trace for channel 1 (api calls: %d):\n", len(trace.GetRecordsForChannel(chan1.GetID()))) fmt.Printf("--------------------\n") - for _, item := range conn.Trace().GetRecordsForChannel(chan1.GetID()) { + for _, item := range trace.GetRecordsForChannel(chan1.GetID()) { printTrace(item) } fmt.Printf("--------------------\n") - fmt.Printf("API trace for channel 2 (api calls: %d):\n", len(conn.Trace().GetRecordsForChannel(chan2.GetID()))) + fmt.Printf("API trace for channel 2 (api calls: %d):\n", len(trace.GetRecordsForChannel(chan2.GetID()))) fmt.Printf("--------------------\n") - for _, item := range conn.Trace().GetRecordsForChannel(chan2.GetID()) { + for _, item := range trace.GetRecordsForChannel(chan2.GetID()) { printTrace(item) } fmt.Printf("--------------------\n") - fmt.Printf("cumulative API trace (api calls: %d):\n", len(conn.Trace().GetRecords())) + fmt.Printf("cumulative API trace (api calls: %d):\n", len(trace.GetRecords())) fmt.Printf("--------------------\n") - for _, item := range conn.Trace().GetRecords() { + for _, item := range trace.GetRecords() { printTrace(item) } fmt.Printf("--------------------\n") fmt.Printf("Clearing API trace...\n\n") - conn.Trace().Clear() + trace.Clear() } -func stream(conn *core.Connection) { - // create new channel and perform simple compatibility check +func stream(conn *core.Connection, trace api.Trace) { + // create the new channel and perform simple compatibility check s, err := conn.NewStream(context.Background()) if err != nil { log.Fatalln("ERROR: creating channel failed:", err) @@ -171,15 +173,15 @@ func stream(conn *core.Connection) { invokeDeleteLoopback(conn, idx) fmt.Println() - fmt.Printf("stream API trace (api calls: %d):\n", len(conn.Trace().GetRecords())) + fmt.Printf("stream API trace (api calls: %d):\n", len(trace.GetRecords())) fmt.Printf("--------------------\n") - for _, item := range conn.Trace().GetRecords() { + for _, item := range trace.GetRecords() { printTrace(item) } fmt.Printf("--------------------\n") fmt.Printf("Clearing API trace...\n\n") - conn.Trace().GetRecords() + trace.GetRecords() } func retrieveVersion(ch api.Channel) { @@ -267,7 +269,7 @@ func addIPAddress(addr string, ch api.Channel, index interface_types.InterfaceIn } reply := &interfaces.SwInterfaceAddDelAddressReply{} - if err := ch.SendRequest(req).ReceiveReply(reply); err != nil { + if err = ch.SendRequest(req).ReceiveReply(reply); err != nil { fmt.Printf("ERROR: %v\n", err) return } @@ -288,7 +290,7 @@ func invokeAddIPAddress(addr string, c api.Connection, index interface_types.Int } reply := &interfaces.SwInterfaceAddDelAddressReply{} - if err := c.Invoke(context.Background(), req, reply); err != nil { + if err = c.Invoke(context.Background(), req, reply); err != nil { fmt.Printf("ERROR: %v\n", err) return } @@ -319,11 +321,11 @@ func invokeInterfaceDump(c api.Connection) { fmt.Printf("ERROR: %v\n", err) return } - if err := s.SendMsg(&interfaces.SwInterfaceDump{}); err != nil { + if err = s.SendMsg(&interfaces.SwInterfaceDump{}); err != nil { fmt.Printf("ERROR: %v\n", err) return } - if err := s.SendMsg(&memclnt.ControlPing{}); err != nil { + if err = s.SendMsg(&memclnt.ControlPing{}); err != nil { fmt.Printf("ERROR: %v\n", err) return } @@ -348,6 +350,6 @@ func printTrace(item *api.Record) { if item.IsReceived { reply = "(reply)" } - fmt.Printf("%dh:%dm:%ds:%dns %s %s\n", h, m, s, - item.Timestamp.Nanosecond(), item.Message.GetMessageName(), reply) + fmt.Printf("%dh:%dm:%ds:%dns %s sucess: %t %s\n", h, m, s, + item.Timestamp.Nanosecond(), item.Message.GetMessageName(), item.Succeeded, reply) } diff --git a/test/integration/trace_test.go b/test/integration/trace_test.go index 2267c29d..fbb76c56 100644 --- a/test/integration/trace_test.go +++ b/test/integration/trace_test.go @@ -17,6 +17,7 @@ package integration import ( "context" "fmt" + "go.fd.io/govpp/core" "testing" "go.fd.io/govpp/api" @@ -27,11 +28,11 @@ import ( func TestTrace(t *testing.T) { test := vpptesting.SetupVPP(t) - test.Conn.Trace().Enable(true) + trace := core.NewTrace(test.Conn, 50) runTraceRequests(t, test) - records := test.Conn.Trace().GetRecords() + records := trace.GetRecords() if len(records) != 2 { t.Fatalf("expected 2 records, got %d", len(records))