From 9d43ef98c1e80918ddf28112eba50879fc626b8b Mon Sep 17 00:00:00 2001 From: Jeremy Lewi Date: Sat, 7 Sep 2024 03:35:54 -0700 Subject: [PATCH] LogAnalysis should not rely on cell execution being recorded by LogEvents (#222) Now that we no longer need to process RunMe logs we can simplify log processing * For the watermark we can just keep track of a single file and its offset * Only the latest file will be active. Therefore we don't need to watch the filesystem for modifications we can just periodically scan the logs. ## Changes to BlockLog Proto * We can remove ExecTraceIds * Since the AI doesn't handle executions we don't have traces for execution * We should rely on the Frontend sending details of execution (e.g. execute code) which it currently isn't * Since LogEvents are processed in order we know that the most recent cell execution will be the final one reported by a LogEvent * Add a field to record suggestion status * Right now we only get log events for cells being accepted we don't record cells being rejected but we can reasonably infer that unaccepted cells have been rejected ## Config Changes * We can deprecate the logDirs field in learner since users should no longer need to configure it to monitor RunMe Logs * Simplifying the setup process of the learner is one of the main motivations of this PR. ## Analyzer Changes *. We no longer need CombineTraces functions for RunMe and Execute traces * We just accumulate LogEvents on the BlockLog object as we sequentially process the logs. ## Other changes * Add an RPC method to check the logs status. * This will report the watermark * Add ZapProto function to handle logging protos that include RunMe objects * We can't rely on the custom go plugin to generate the MarshalObject function because the RunMe protos aren't using that plugin Related to #211 --- app/pkg/agent/agent.go | 3 +- app/pkg/analyze/analyzer.go | 456 +++++---------- app/pkg/analyze/analyzer_test.go | 351 +----------- app/pkg/analyze/crud.go | 12 +- app/pkg/analyze/crud_test.go | 18 +- app/pkg/analyze/reader.go | 7 +- .../logs/foyle.logs.2024-04-16T19:06:47.json | 4 +- app/pkg/application/app.go | 9 +- app/pkg/config/config.go | 4 +- app/pkg/dbutil/locking_test.go | 11 +- app/pkg/logs/logs.go | 40 +- app/pkg/runme/converters/cells_to_blocks.go | 19 +- app/pkg/runme/converters/const.go | 5 + app/pkg/server/server.go | 4 +- protos/README.md | 4 +- protos/foyle/logs/blocks.proto | 16 +- protos/foyle/logs/logs.proto | 20 + protos/foyle/logs/traces.proto | 18 +- protos/foyle/v1alpha1/agent.proto | 3 + protos/go/foyle/logs/blocks.pb.go | 164 ++++-- protos/go/foyle/logs/blocks.zap.go | 12 +- protos/go/foyle/logs/logs.pb.go | 282 +++++++++ protos/go/foyle/logs/logs.zap.go | 63 +++ .../logs/logspbconnect/traces.connect.go | 29 + protos/go/foyle/logs/traces.pb.go | 533 ++++++------------ protos/go/foyle/logs/traces.zap.go | 81 +-- protos/go/foyle/logs/traces_grpc.pb.go | 36 ++ protos/go/foyle/v1alpha1/agent.pb.go | 99 ++-- protos/go/foyle/v1alpha1/agent.zap.go | 5 +- 29 files changed, 1087 insertions(+), 1221 deletions(-) create mode 100644 protos/foyle/logs/logs.proto create mode 100644 protos/go/foyle/logs/logs.pb.go create mode 100644 protos/go/foyle/logs/logs.zap.go diff --git a/app/pkg/agent/agent.go b/app/pkg/agent/agent.go index b48b296c..06a749b6 100644 --- a/app/pkg/agent/agent.go +++ b/app/pkg/agent/agent.go @@ -529,7 +529,8 @@ func (a *Agent) LogEvents(ctx context.Context, req *connect.Request[v1alpha1.Log func() { _, span := tp.Start(ctx, "LogEvent", trace.WithAttributes(attribute.String("eventType", event.Type.String()), attribute.String("contextId", event.ContextId), attribute.String("selectedCellId", event.SelectedId))) defer span.End() - log.Info("LogEvent", "eventType", event.Type, "contextId", event.ContextId, "selectedCellId", event.SelectedId, "event", zap.Object("event", event)) + // N.B we can't use zap.Object to log the event because it contains runme protos which don't have the zap marshaler bindings. + log.Info("LogEvent", "eventId", event.GetEventId(), "eventType", event.Type, "contextId", event.ContextId, "selectedCellId", event.SelectedId, logs.ZapProto("event", event)) }() } return connect.NewResponse(&v1alpha1.LogEventsResponse{}), nil diff --git a/app/pkg/analyze/analyzer.go b/app/pkg/analyze/analyzer.go index aeed86f7..a635bdcb 100644 --- a/app/pkg/analyze/analyzer.go +++ b/app/pkg/analyze/analyzer.go @@ -11,13 +11,14 @@ import ( "sync" "time" + "github.com/jlewi/foyle/app/pkg/runme/converters" + parserv1 "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/parser/v1" + "google.golang.org/protobuf/proto" + "github.com/fsnotify/fsnotify" "github.com/go-logr/zapr" "go.uber.org/zap" - "github.com/jlewi/foyle/app/pkg/docs" - runnerv1 "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/runner/v1" - "github.com/cockroachdb/pebble" "github.com/jlewi/foyle/app/api" "github.com/jlewi/foyle/app/pkg/dbutil" @@ -32,9 +33,6 @@ import ( ) const ( - // unsetExitCode is a random negative exit code so we can tell when it hasn't been set. - unsetExitCode = -2377 - // traceField is the field that contains the traceId in a log entry. We use this to identify processing related // to a particular trace. We don't use the field "traceId" because these log entries aren't actually part of the // trace. @@ -60,6 +58,7 @@ type Analyzer struct { rawLogsDB *dbutil.LockingDB[*logspb.LogEntries] // queue for log file processing queue workqueue.RateLimitingInterface + // Queue for block log processing // TODO(jeremy): We should really use a durable queue backed by files blockQueue workqueue.DelayingInterface @@ -70,7 +69,7 @@ type Analyzer struct { handleLogFileIsDone sync.WaitGroup handleBlocksIsDone sync.WaitGroup - logFileOffsets map[string]int64 + logFileOffsets *logspb.LogsWaterMark mu sync.Mutex logOffsetsFile string @@ -107,24 +106,28 @@ func NewAnalyzer(logOffsetsFile string, rawLogsDB *dbutil.LockingDB[*logspb.LogE }, nil } -func initOffsets(logOffsetsFile string) (map[string]int64, error) { +func initOffsets(logOffsetsFile string) (*logspb.LogsWaterMark, error) { log := zapr.NewLogger(zap.L()) raw, err := os.ReadFile(logOffsetsFile) if err != nil { if os.IsNotExist(err) { - return map[string]int64{}, nil + return &logspb.LogsWaterMark{}, nil } return nil, errors.Wrapf(err, "Failed to read watermarks file %s", logOffsetsFile) } - watermarks := map[string]int64{} - if err := json.Unmarshal(raw, &watermarks); err != nil { + watermark := &logspb.LogsWaterMark{} + + if err := protojson.Unmarshal(raw, watermark); err != nil { log.Error(err, "Failed to unmarshal watermarks file %s; watermarks will be reinitialized", logOffsetsFile) } - return watermarks, nil + return watermark, nil } type fileItem struct { path string + + // Whether the file is still active. If the file is no longer active we should stop processing it. + active bool } type blockItem struct { @@ -146,17 +149,18 @@ func (a *Analyzer) Run(ctx context.Context, logDirs []string, blockNotifier Post } // Enqueue an item to process each file - for _, f := range jsonFiles { - a.queue.Add(fileItem{path: f}) - } - - if err := a.registerDirWatchers(ctx, a.queue, logDirs); err != nil { - return err + for i, f := range jsonFiles { + // Only the last file should be active. + active := i == len(jsonFiles)-1 + a.queue.Add(fileItem{path: f, active: active}) } a.handleLogFileIsDone.Add(1) a.handleBlocksIsDone.Add(1) + // Important we should only process LogFileEvents in a single go func because the semantics of the watermark + // are that all log entries up to that mark have been processed. If we process log entries in parallel its not + // clear how we handle updating the waterMark. go a.handleLogFileEvents(ctx) go a.handleBlockEvents(ctx) @@ -164,6 +168,9 @@ func (a *Analyzer) Run(ctx context.Context, logDirs []string, blockNotifier Post } // TODO(jeremy): How do we make the Analyzer thread safe? I believe the DB classes are thread safe +// +// TODO(jeremy): Should we instrument this with OTEL to get metrics on how long it takes to process a log file? +// What we'd like is counters for how often a log file is processed. But maybe we should use logs for that? func (a *Analyzer) handleLogFileEvents(ctx context.Context) { q := a.queue log := logs.FromContext(ctx) @@ -191,6 +198,13 @@ func (a *Analyzer) handleLogFileEvents(ctx context.Context) { if a.signalFileDone != nil { a.signalFileDone <- fileItem.path } + // If the file is still active re-enqueue it. + if fileItem.active { + q.AddRateLimited(fileItem) + } else { + log.Info("Finished processing log file", "path", fileItem.path) + } + }() } } @@ -200,7 +214,13 @@ func (a *Analyzer) processLogFile(ctx context.Context, path string) error { log.V(logs.Debug).Info("Processing log file", "path", path) offset := a.getLogFileOffset(path) + if offset <= -1 { + // Offset of -1 means we are done processing the file because it is before the watermark + log.V(logs.Debug).Info("Logfile already processed", "path", path) + return nil + } lines, offset, err := readLinesFromOffset(ctx, path, offset) + if err != nil { return err } @@ -218,12 +238,17 @@ func (a *Analyzer) processLogFile(ctx context.Context, path string) error { continue } + if strings.HasSuffix(entry.Function(), "agent.(*Agent).LogEvents") { + a.processLogEvent(ctx, entry) + continue + } + // Ignore log entries without traces if entry.TraceID() == "" { continue } - // Drop all log entries that come from the Analyzer package itself. This should hadn't be neccessary + // Drop all log entries that come from the Analyzer package itself. This shouldn't be neccessary // but its a precaution to guard against someone accidentally adding a log message with the the field "traceId" // to log a message about processing that trace. If we include such messages as part of the trace // we could trigger an infinite loop @@ -258,24 +283,48 @@ func (a *Analyzer) processLogFile(ctx context.Context, path string) error { return nil } +func (a *Analyzer) GetWatermark() *logspb.LogsWaterMark { + a.mu.Lock() + defer a.mu.Unlock() + w := proto.Clone(a.logFileOffsets).(*logspb.LogsWaterMark) + return w +} + +// getLogOffSet returns the offset for the log file to start reading from. +// A value < 0 means the watermark is already past the end of the file and no more processing is needed. func (a *Analyzer) getLogFileOffset(path string) int64 { a.mu.Lock() defer a.mu.Unlock() - offset, ok := a.logFileOffsets[path] - if !ok { + // N.B. This code takes into account the full file path when deciding the ordering of the logfiles. + if path < a.logFileOffsets.File { + return -1 + } + if path > a.logFileOffsets.File { return 0 } - return offset + return a.logFileOffsets.Offset } func (a *Analyzer) setLogFileOffset(path string, offset int64) { a.mu.Lock() defer a.mu.Unlock() - a.logFileOffsets[path] = offset + oldWatermark := a.logFileOffsets + a.logFileOffsets = &logspb.LogsWaterMark{ + File: path, + Offset: offset, + } + + log := logs.NewLogger() + if path < oldWatermark.File { + log.Error(errors.New("Watermark is moving backwards"), "Watermark is moving backwards", zap.Object("oldWatermark", oldWatermark), zap.Object("newWatermark", a.logFileOffsets)) + } + + if oldWatermark.File != a.logFileOffsets.File { + log.Info("Logs watermark moving to new file", zap.Object("oldWatermark", oldWatermark), zap.Object("newWatermark", a.logFileOffsets)) + } - log := zapr.NewLogger(zap.L()) // Persist the watermarks - raw, err := json.Marshal(a.logFileOffsets) + raw, err := protojson.Marshal(a.logFileOffsets) if err != nil { log.Error(err, "Failed to marshal watermarks") return @@ -293,52 +342,6 @@ func (a *Analyzer) setLogFileOffset(path string, offset int64) { log.V(logs.Debug).Info("Wrote watermarks", "logOffsetsFile", a.logOffsetsFile) } -// registerDirWatchers sets up notifications for changes in the log directories. -// Any time a file is modified it will enqueue the file for processing. -func (a *Analyzer) registerDirWatchers(ctx context.Context, q workqueue.RateLimitingInterface, logDirs []string) error { - log := logs.FromContext(ctx) - watcher, err := fsnotify.NewWatcher() - a.watcher = watcher - if err != nil { - return err - } - for _, dir := range logDirs { - fullPath, err := filepath.Abs(dir) - if err != nil { - return errors.Wrapf(err, "Failed to get absolute path for %s", dir) - } - - log.Info("Watching logs directory", "dir", fullPath) - if err := watcher.Add(fullPath); err != nil { - return err - } - } - - go handleFsnotifications(ctx, watcher, q) - return nil -} - -// handleFsnotifications processes file system notifications by enqueuing the file for processing. -func handleFsnotifications(ctx context.Context, watcher *fsnotify.Watcher, q workqueue.RateLimitingInterface) { - log := logs.FromContext(ctx) - for { - select { - case event, ok := <-watcher.Events: - if !ok { - return - } - if event.Op&fsnotify.Write == fsnotify.Write { - q.AddRateLimited(fileItem{path: event.Name}) - } - case err, ok := <-watcher.Errors: - if !ok { - return - } - log.Error(err, "Error from watcher") - } - } -} - func (a *Analyzer) Shutdown(ctx context.Context) error { log := logs.FromContext(ctx) @@ -359,6 +362,85 @@ func (a *Analyzer) Shutdown(ctx context.Context) error { return nil } +// processLogEvent processes a log event. +func (a *Analyzer) processLogEvent(ctx context.Context, entry *api.LogEntry) { + log := logs.FromContext(ctx) + + event := &v1alpha1.LogEvent{} + + if !entry.GetProto("event", event) { + log.Error(errors.New("Failed to decode event"), "Failed to decode LogEvent", "entry", entry) + return + } + log = log.WithValues("eventId", event.GetEventId()) + switch event.Type { + case v1alpha1.LogEventType_EXECUTE: + bid := event.SelectedId + if bid == "" { + log.Error(errors.New("No selectedId"), "Execute event is missing selected id", "event", event) + return + } + + var cell *parserv1.Cell + for _, c := range event.GetCells() { + if converters.GetCellID(c) == bid { + cell = c + break + } + } + + if cell == nil { + log.Error(errors.New("Failed to find cell"), "Execution log event is missing the actual cell", "bid", bid, "event", event) + return + } + executedBlock, err := converters.CellToBlock(cell) + if err != nil { + jb, err := protojson.Marshal(cell) + if err != nil { + log.Error(err, "Failed to convert executed cell to block", "cellId", bid, "cell", string(jb)) + } else { + log.Error(err, "Failed to convert executed cell to block", "cellId", bid) + } + } + + if err := a.blocksDB.ReadModifyWrite(bid, func(block *logspb.BlockLog) error { + block.Id = bid + block.ExecutedBlock = executedBlock + return nil + }); err != nil { + log.Error(err, "Failed to update block with execution", "blockId", bid) + } + case v1alpha1.LogEventType_ACCEPTED: + fallthrough + case v1alpha1.LogEventType_REJECTED: + status := logspb.SuggestionStatus_SuggestionStatusUnknown + switch event.Type { + case v1alpha1.LogEventType_ACCEPTED: + status = logspb.SuggestionStatus_ACCEPTED + case v1alpha1.LogEventType_REJECTED: + status = logspb.SuggestionStatus_REJECTED + } + + for _, c := range event.GetCells() { + bid := converters.GetCellID(c) + if bid == "" { + log.Error(errors.New("No cell id"), "Cell is missing id", zap.Object("event", event)) + continue + } + + if err := a.blocksDB.ReadModifyWrite(bid, func(block *logspb.BlockLog) error { + block.Id = bid + block.SuggestionStatus = status + return nil + }); err != nil { + log.Error(err, "Failed to update block with execution", "blockId", bid) + } + } + default: + // Do Nothing with the event + } +} + // buildTrace creates the trace and initializes the blocks. func (a *Analyzer) buildTrace(ctx context.Context, tid string) error { log := logs.FromContext(ctx) @@ -419,38 +501,6 @@ func (a *Analyzer) buildTrace(ctx context.Context, tid string) error { return bids, errors.Wrapf(err, "Failed to set generate trace on block %s", bid) } } - case *logspb.Trace_Execute: - bid := t.Execute.Request.GetBlock().GetId() - if bid == "" { - return bids, nil - } - bids = append(bids, bid) - if err := a.blocksDB.ReadModifyWrite(bid, func(block *logspb.BlockLog) error { - block.Id = bid - if block.ExecTraceIds == nil { - block.ExecTraceIds = make([]string, 0, 10) - } - block.ExecTraceIds = append(block.ExecTraceIds, tid) - return nil - }); err != nil { - return bids, errors.Wrapf(err, "Failed to set execute trace on block %s", bid) - } - case *logspb.Trace_RunMe: - bid := t.RunMe.Request.GetKnownId() - if bid == "" { - return bids, nil - } - bids = append(bids, bid) - if err := a.blocksDB.ReadModifyWrite(bid, func(block *logspb.BlockLog) error { - block.Id = bid - if block.ExecTraceIds == nil { - block.ExecTraceIds = make([]string, 0, 10) - } - block.ExecTraceIds = append(block.ExecTraceIds, tid) - return nil - }); err != nil { - return bids, errors.Wrapf(err, "Failed to set RunMe trace on block %s", bid) - } default: log.Error(fmt.Errorf("Unknown trace type"), "Unknown trace type", "trace", t) } @@ -564,6 +614,8 @@ func (a *Analyzer) handleBlockEvents(ctx context.Context) { } } +// buildBlockLog updates blocklogs given a generate trace. +// Since a single generate trace can generate multiple blocks, its a one to many operation. func buildBlockLog(ctx context.Context, block *logspb.BlockLog, tracesDB *pebble.DB) error { log := logs.FromContext(ctx) log = log.WithValues("blockId", block.Id) @@ -605,96 +657,6 @@ func buildBlockLog(ctx context.Context, block *logspb.BlockLog, tracesDB *pebble }() } - // Dedupe the execution traces just in case - uEids := make(map[string]bool) - for _, eid := range block.GetExecTraceIds() { - uEids[eid] = true - } - block.ExecTraceIds = make([]string, 0, len(uEids)) - for eid := range uEids { - block.ExecTraceIds = append(block.ExecTraceIds, eid) - } - - eidToTime := make(map[string]time.Time) - - var lastTrace *logspb.Trace - // Get the last execution trace - for _, tid := range block.GetExecTraceIds() { - func() { - trace := &logspb.Trace{} - if err := dbutil.GetProto(tracesDB, tid, trace); err != nil { - log.Error(err, "Error getting execute trace", "execTraceId", tid) - return - } - - if trace.GetExecute() == nil && trace.GetRunMe() == nil { - log.Error(errors.New("Invalid execution trace for traceId"), "Error getting execute trace", "execTraceId", tid) - return - } - - eidToTime[tid] = trace.StartTime.AsTime() - - if lastTrace == nil { - lastTrace = trace - return - } - - if lastTrace.StartTime.AsTime().Before(trace.StartTime.AsTime()) { - lastTrace = trace - } - }() - } - - // Sort execTrace ids based on their time. This is so the ordering is stable for the unittest. - // It should also be convenient for manual analysis since we usually care about the last exec trace. - sort.Slice(block.ExecTraceIds, func(i, j int) bool { - left := block.ExecTraceIds[i] - right := block.ExecTraceIds[j] - leftTime := eidToTime[left] - rightTime := eidToTime[right] - return leftTime.Before(rightTime) - }) - - if lastTrace != nil { - if err := updateBlockForExecution(block, lastTrace); err != nil { - return err - } - } - - return nil -} - -// updateBlockForExecution updates fields in the block log based the last execution trace of that block -func updateBlockForExecution(block *logspb.BlockLog, lastTrace *logspb.Trace) error { - // If the block was executed as part of evaluation mode then consider it to be in evaluation mode. - if lastTrace.EvalMode { - block.EvalMode = true - } - block.ExecutedBlock = nil - block.ExitCode = unsetExitCode - - switch eTrace := lastTrace.Data.(type) { - case *logspb.Trace_Execute: - block.ExecutedBlock = eTrace.Execute.Request.GetBlock() - - for _, o := range eTrace.Execute.Response.GetOutputs() { - exitCode, ok := docs.GetExitCode(o) - if ok { - block.ExitCode = int32(exitCode) - break - } - } - case *logspb.Trace_RunMe: - // TODO(jeremy): Is this the right way to turn the command into a string? - block.ExecutedBlock = &v1alpha1.Block{ - Kind: v1alpha1.BlockKind_CODE, - Contents: strings.Join(eTrace.RunMe.Request.GetCommands(), " "), - Outputs: nil, - } - - default: - return errors.WithStack(errors.Errorf("Can't update BlockLog with execution information. The last trace, id %s is not an execution trace", lastTrace.Id)) - } return nil } @@ -711,14 +673,6 @@ func combineEntriesForTrace(ctx context.Context, entries []*api.LogEntry) (*logs return combineGenerateTrace(ctx, entries) } - if strings.HasSuffix(function, "executor.(*Executor).Execute") { - return combineExecuteTrace(ctx, entries) - } - - if strings.HasSuffix(function, "runner.(*runnerService).Execute") { - return combineRunMeTrace(ctx, entries) - } - if strings.HasSuffix(function, "agent.(*Agent).StreamGenerate") { // For now we do nothing with StreamGenerate traces. return nil, nil @@ -785,103 +739,3 @@ func combineGenerateTrace(ctx context.Context, entries []*api.LogEntry) (*logspb combineSpans(trace) return trace, nil } - -func combineExecuteTrace(ctx context.Context, entries []*api.LogEntry) (*logspb.Trace, error) { - eTrace := &logspb.ExecuteTrace{} - trace := &logspb.Trace{ - Data: &logspb.Trace_Execute{ - Execute: eTrace, - }, - } - evalMode := false - for _, e := range entries { - if trace.Id == "" { - trace.Id = e.TraceID() - } - if mode, present := e.EvalMode(); present { - // If any of the entries are marked as true then we will consider the trace to be in eval mode. - // We don't want to assume that the evalMode will be set on all log entries in the trace. - // So the logic is to assume its not eval mode by default and then set it to eval mode if we find - // One entry that is marked as eval mode. - if mode { - evalMode = mode - } - } - - if eTrace.Request == nil { - raw := e.Request() - if raw != nil { - request := &v1alpha1.ExecuteRequest{} - if err := protojson.Unmarshal([]byte(raw), request); err != nil { - return nil, err - } - - eTrace.Request = request - trace.StartTime = timestamppb.New(e.Time()) - } - } - if eTrace.Response == nil { - raw := e.Response() - if raw != nil { - v := &v1alpha1.ExecuteResponse{} - if err := protojson.Unmarshal([]byte(raw), v); err != nil { - return nil, err - } - eTrace.Response = v - trace.EndTime = timestamppb.New(e.Time()) - } - } - } - trace.EvalMode = evalMode - return trace, nil -} - -func combineRunMeTrace(ctx context.Context, entries []*api.LogEntry) (*logspb.Trace, error) { - rTrace := &logspb.RunMeTrace{} - trace := &logspb.Trace{ - Data: &logspb.Trace_RunMe{ - RunMe: rTrace, - }, - } - evalMode := false - for _, e := range entries { - if trace.Id == "" { - trace.Id = e.TraceID() - } - if mode, present := e.EvalMode(); present { - // If any of the entries are marked as true then we will consider the trace to be in eval mode. - // We don't want to assume that the evalMode will be set on all log entries in the trace. - // So the logic is to assume its not eval mode by default and then set it to eval mode if we find - // One entry that is marked as eval mode. - if mode { - evalMode = mode - } - } - - if rTrace.Request == nil { - raw := e.Request() - if raw != nil { - request := &runnerv1.ExecuteRequest{} - if err := protojson.Unmarshal([]byte(raw), request); err != nil { - return nil, err - } - - rTrace.Request = request - trace.StartTime = timestamppb.New(e.Time()) - } - } - if rTrace.Response == nil { - raw := e.Response() - if raw != nil { - v := &runnerv1.ExecuteResponse{} - if err := protojson.Unmarshal([]byte(raw), v); err != nil { - return nil, err - } - rTrace.Response = v - trace.EndTime = timestamppb.New(e.Time()) - } - } - } - trace.EvalMode = evalMode - return trace, nil -} diff --git a/app/pkg/analyze/analyzer_test.go b/app/pkg/analyze/analyzer_test.go index 6377abd9..b34448f7 100644 --- a/app/pkg/analyze/analyzer_test.go +++ b/app/pkg/analyze/analyzer_test.go @@ -4,14 +4,11 @@ import ( "context" "encoding/json" "io" - "math/rand" "os" "path/filepath" "testing" "time" - runnerv1 "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/runner/v1" - "github.com/cockroachdb/pebble" "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" @@ -34,11 +31,6 @@ func timeMustParse(layoutString, value string) *timestamppb.Timestamp { return timestamppb.New(t) } -func shuffle(in []string) []string { - rand.Shuffle(len(in), func(i, j int) { in[i], in[j] = in[j], in[i] }) - return in -} - func Test_BuildBlockLog(t *testing.T) { type testCase struct { name string @@ -77,60 +69,6 @@ func Test_BuildBlockLog(t *testing.T) { }, } - execTrace1 := &logspb.Trace{ - Id: "e456", - StartTime: timeMustParse(time.RFC3339, "2021-01-02T00:00:00Z"), - EndTime: timeMustParse(time.RFC3339, "2021-01-02T00:01:00Z"), - Data: &logspb.Trace_Execute{ - Execute: &logspb.ExecuteTrace{ - Request: &v1alpha1.ExecuteRequest{ - Block: &v1alpha1.Block{ - Contents: "echo hello", - Id: bid1, - }, - }, - Response: &v1alpha1.ExecuteResponse{ - Outputs: []*v1alpha1.BlockOutput{ - { - Items: []*v1alpha1.BlockOutputItem{ - { - TextData: "exitCode: 4", - }, - }, - }, - }, - }, - }, - }, - } - - execTrace2 := &logspb.Trace{ - Id: "e789", - StartTime: timeMustParse(time.RFC3339, "2021-01-03T00:00:00Z"), - EndTime: timeMustParse(time.RFC3339, "2021-01-03T00:01:00Z"), - Data: &logspb.Trace_Execute{ - Execute: &logspb.ExecuteTrace{ - Request: &v1alpha1.ExecuteRequest{ - Block: &v1alpha1.Block{ - Contents: "echo hello", - Id: bid1, - }, - }, - Response: &v1alpha1.ExecuteResponse{ - Outputs: []*v1alpha1.BlockOutput{ - { - Items: []*v1alpha1.BlockOutputItem{ - { - TextData: "exitCode: 7", - }, - }, - }, - }, - }, - }, - }, - } - // Create a block in evaluation mode const bid2 = "g456output1" genTrace2 := &logspb.Trace{ @@ -161,59 +99,21 @@ func Test_BuildBlockLog(t *testing.T) { EvalMode: true, } - execTrace3 := &logspb.Trace{ - Id: "e912", - StartTime: timeMustParse(time.RFC3339, "2021-01-03T00:00:00Z"), - EndTime: timeMustParse(time.RFC3339, "2021-01-03T00:01:00Z"), - Data: &logspb.Trace_Execute{ - Execute: &logspb.ExecuteTrace{ - Request: &v1alpha1.ExecuteRequest{ - Block: &v1alpha1.Block{ - Contents: "echo hello", - Id: bid2, - }, - }, - Response: &v1alpha1.ExecuteResponse{ - Outputs: []*v1alpha1.BlockOutput{ - { - Items: []*v1alpha1.BlockOutputItem{ - { - TextData: "exitCode: 7", - }, - }, - }, - }, - }, - }, - }, - } - traces[genTrace.Id] = genTrace traces[genTrace2.Id] = genTrace2 - traces[execTrace1.Id] = execTrace1 - traces[execTrace2.Id] = execTrace2 - traces[execTrace3.Id] = execTrace3 - - // We shuffle ExecTraceIds to make sure we properly set block log based on the later trace - execTraceIds := shuffle([]string{execTrace1.GetId(), execTrace2.GetId()}) cases := []testCase{ { name: "basic", block: &logspb.BlockLog{ - Id: bid1, - GenTraceId: genTrace.Id, - ExecTraceIds: execTraceIds, - }, - expected: &logspb.BlockLog{ Id: bid1, GenTraceId: genTrace.Id, - // ExecTraceIds should be sorted by the timestamp - ExecTraceIds: []string{execTrace1.GetId(), execTrace2.GetId()}, + }, + expected: &logspb.BlockLog{ + Id: bid1, + GenTraceId: genTrace.Id, Doc: genTrace.GetGenerate().Request.Doc, GeneratedBlock: genTrace.GetGenerate().Response.Blocks[0], - ExecutedBlock: execTrace2.GetExecute().Request.Block, - ExitCode: 7, EvalMode: false, }, traces: traces, @@ -223,17 +123,12 @@ func Test_BuildBlockLog(t *testing.T) { block: &logspb.BlockLog{ Id: bid2, GenTraceId: genTrace2.Id, - - ExecTraceIds: []string{execTrace3.Id}, }, expected: &logspb.BlockLog{ Id: bid2, GenTraceId: genTrace2.Id, - ExecTraceIds: []string{execTrace3.Id}, Doc: genTrace2.GetGenerate().Request.Doc, GeneratedBlock: genTrace2.GetGenerate().Response.Blocks[0], - ExecutedBlock: execTrace3.GetExecute().Request.Block, - ExitCode: 7, EvalMode: true, }, traces: traces, @@ -356,7 +251,24 @@ func Test_Analyzer(t *testing.T) { t.Logf("File processed: %s", fileDone) t.Logf("Output written to: %s", oDir) - waitForBlock(t, "23706965-8e3b-440d-ba1a-1e1cc035fbd4", 2, blockProccessed) + // Wait for the logs to be fully processed + done := false + timeend := time.Now().Add(1 * time.Minute) + var w *logspb.LogsWaterMark + for !done && time.Now().Before(timeend) { + w = a.GetWatermark() + if w.Offset < 23457 { + time.Sleep(5 * time.Second) + } else { + done = true + } + } + if !done { + t.Fatalf("Timed out waiting for logs to be processed; final offset %d", w.Offset) + } + + // Signal should be triggered once for the blocklog. + waitForBlock(t, "23706965-8e3b-440d-ba1a-1e1cc035fbd4", 1, blockProccessed) // This is a block that was generated via the AI and then executed so run some additional checks block := &logspb.BlockLog{} @@ -366,9 +278,7 @@ func Test_Analyzer(t *testing.T) { if block.GenTraceId == "" { t.Errorf("Expected GenTraceID to be set") } - if len(block.ExecTraceIds) == 0 { - t.Errorf("Expected ExecTraceIDs to be set") - } + if block.Doc == nil { t.Errorf("Expected Doc to be set") } @@ -527,218 +437,3 @@ func Test_CombineGenerateEntries(t *testing.T) { }) } } - -func Test_CombineExecuteEntries(t *testing.T) { - type testCase struct { - name string - linesFile string - expectedEvalMode bool - } - - cases := []testCase{ - { - name: "basic", - linesFile: "execute_traces_lines.jsonl", - expectedEvalMode: false, - }, - { - name: "eval_mode_true", - linesFile: "execute_traces_lines_eval_mode.jsonl", - expectedEvalMode: true, - }, - { - name: "eval_mode_false", - linesFile: "execute_traces_lines_eval_mode_false.jsonl", - expectedEvalMode: false, - }, - } - - cwd, err := os.Getwd() - if err != nil { - t.Fatalf("Failed to get current working directory: %v", err) - } - for _, c := range cases { - t.Run(c.name, func(t *testing.T) { - entries := make([]*api.LogEntry, 0, 10) - testFile, err := os.Open(filepath.Join(cwd, "test_data", c.linesFile)) - if err != nil { - t.Fatalf("Failed to open test file: %v", err) - } - d := json.NewDecoder(testFile) - for { - e := &api.LogEntry{} - err := d.Decode(e) - if err != nil { - if err == io.EOF { - break - } - t.Fatalf("Failed to unmarshal log entry: %v", err) - } - entries = append(entries, e) - } - trace, err := combineExecuteTrace(context.Background(), entries) - if err != nil { - t.Fatalf("combineExecuteTrace failed: %+v", err) - } - if trace == nil { - t.Fatalf("combineExecuteTrace should have returned non nil response") - } - - execTrace := trace.GetExecute() - if execTrace == nil { - t.Fatalf("Expected trace to have an execute trace") - } - // Assert the trace has a request and a response - if execTrace.Request == nil { - t.Errorf("Expected trace to have a request") - } - if execTrace.Response == nil { - t.Errorf("Expected trace to have a response") - } - if trace.EvalMode != c.expectedEvalMode { - t.Errorf("Expected EvalMode to be %v but got %v", c.expectedEvalMode, trace.EvalMode) - } - }) - } -} - -func Test_CombineRunmeEntries(t *testing.T) { - type testCase struct { - name string - linesFile string - expectedEvalMode bool - } - - cases := []testCase{ - { - name: "basic", - linesFile: "runme_traces_lines.jsonl", - expectedEvalMode: false, - }, - } - - cwd, err := os.Getwd() - if err != nil { - t.Fatalf("Failed to get current working directory: %v", err) - } - for _, c := range cases { - t.Run(c.name, func(t *testing.T) { - entries := make([]*api.LogEntry, 0, 10) - testFile, err := os.Open(filepath.Join(cwd, "test_data", c.linesFile)) - if err != nil { - t.Fatalf("Failed to open test file: %v", err) - } - d := json.NewDecoder(testFile) - for { - e := &api.LogEntry{} - err := d.Decode(e) - if err != nil { - if err == io.EOF { - break - } - t.Fatalf("Failed to unmarshal log entry: %v", err) - } - entries = append(entries, e) - } - trace, err := combineRunMeTrace(context.Background(), entries) - if err != nil { - t.Fatalf("combineRunMeTrace failed: %+v", err) - } - if trace == nil { - t.Fatalf("combineRunMeTrace should have returned non nil response") - } - - rTrace := trace.GetRunMe() - if rTrace == nil { - t.Fatalf("Expected trace to have a runme trace") - } - // Assert the trace has a request and no response - if rTrace.Request == nil { - t.Errorf("Expected trace to have a request") - } - // TODO(jeremy): We don't currently log the response with RunMe - // https://github.com/stateful/runme/blob/6e56cfae38c5a72193a86677356927e14ce87b27/internal/runner/service.go#L461 - if rTrace.Response != nil { - t.Errorf("Expected trace not to have a response") - } - if trace.EvalMode != c.expectedEvalMode { - t.Errorf("Expected EvalMode to be %v but got %v", c.expectedEvalMode, trace.EvalMode) - } - }) - } -} - -func Test_updateBlockForExecution(t *testing.T) { - type testCase struct { - name string - block *logspb.BlockLog - trace *logspb.Trace - expected *logspb.BlockLog - } - - cases := []testCase{ - { - name: "ExecuteTrace", - block: &logspb.BlockLog{}, - trace: &logspb.Trace{ - Data: &logspb.Trace_Execute{ - Execute: &logspb.ExecuteTrace{ - Request: &v1alpha1.ExecuteRequest{ - Block: &v1alpha1.Block{ - Contents: "echo hello", - }, - }, - Response: &v1alpha1.ExecuteResponse{ - Outputs: []*v1alpha1.BlockOutput{ - { - Items: []*v1alpha1.BlockOutputItem{ - { - TextData: "exitCode: 4", - }, - }, - }, - }, - }, - }, - }, - }, - expected: &logspb.BlockLog{ - ExecutedBlock: &v1alpha1.Block{ - Contents: "echo hello", - }, - ExitCode: 4, - }, - }, - { - name: "RunMeTrace", - block: &logspb.BlockLog{}, - trace: &logspb.Trace{ - Data: &logspb.Trace_RunMe{ - RunMe: &logspb.RunMeTrace{ - Request: &runnerv1.ExecuteRequest{ - Commands: []string{"prog1", "arg1"}, - }, - }, - }, - }, - expected: &logspb.BlockLog{ - ExecutedBlock: &v1alpha1.Block{ - Contents: "prog1 arg1", - Kind: v1alpha1.BlockKind_CODE, - }, - ExitCode: -2377, - }, - }, - } - - for _, c := range cases { - t.Run(c.name, func(t *testing.T) { - if err := updateBlockForExecution(c.block, c.trace); err != nil { - t.Fatalf("updateBlockForExecution failed: %v", err) - } - if d := cmp.Diff(c.expected, c.block, cmpopts.IgnoreUnexported(logspb.BlockLog{}), testutil.BlockComparer); d != "" { - t.Errorf("Unexpected diff:\n%s", d) - } - }) - } -} diff --git a/app/pkg/analyze/crud.go b/app/pkg/analyze/crud.go index fe899e1a..1feeaf35 100644 --- a/app/pkg/analyze/crud.go +++ b/app/pkg/analyze/crud.go @@ -20,13 +20,15 @@ type CrudHandler struct { cfg config.Config blocksDB *pebble.DB tracesDB *pebble.DB + analyzer *Analyzer } -func NewCrudHandler(cfg config.Config, blocksDB *pebble.DB, tracesDB *pebble.DB) (*CrudHandler, error) { +func NewCrudHandler(cfg config.Config, blocksDB *pebble.DB, tracesDB *pebble.DB, analyzer *Analyzer) (*CrudHandler, error) { return &CrudHandler{ cfg: cfg, blocksDB: blocksDB, tracesDB: tracesDB, + analyzer: analyzer, }, nil } @@ -94,3 +96,11 @@ func (h *CrudHandler) GetBlockLog(ctx context.Context, request *connect.Request[ return connect.NewResponse(&logspb.GetBlockLogResponse{BlockLog: bLog}), nil } + +func (h *CrudHandler) Status(ctx context.Context, request *connect.Request[logspb.GetLogsStatusRequest]) (*connect.Response[logspb.GetLogsStatusResponse], error) { + response := &logspb.GetLogsStatusResponse{ + Watermark: h.analyzer.GetWatermark(), + } + + return connect.NewResponse(response), nil +} diff --git a/app/pkg/analyze/crud_test.go b/app/pkg/analyze/crud_test.go index 340bf840..564be015 100644 --- a/app/pkg/analyze/crud_test.go +++ b/app/pkg/analyze/crud_test.go @@ -27,12 +27,16 @@ func populateDB(db *pebble.DB) error { func populateTraceDB(db *pebble.DB) error { trace := &logspb.Trace{ Id: "test-trace", - Data: &logspb.Trace_Execute{ - Execute: &logspb.ExecuteTrace{ - Request: &v1alpha1.ExecuteRequest{ - Block: &v1alpha1.Block{ - Id: "test-block", - Contents: "echo hello", + Data: &logspb.Trace_Generate{ + Generate: &logspb.GenerateTrace{ + Request: &v1alpha1.GenerateRequest{ + Doc: &v1alpha1.Doc{ + Blocks: []*v1alpha1.Block{ + { + Id: "test-block", + Contents: "echo hello", + }, + }, }, }, }, @@ -71,7 +75,7 @@ func createCrudHandler() (*CrudHandler, error) { } // Create a new CrudHandler with a mock configuration - return NewCrudHandler(cfg, db, tracesDB) + return NewCrudHandler(cfg, db, tracesDB, nil) } func tearDown(handler *CrudHandler) { diff --git a/app/pkg/analyze/reader.go b/app/pkg/analyze/reader.go index 05b3a919..6afe634e 100644 --- a/app/pkg/analyze/reader.go +++ b/app/pkg/analyze/reader.go @@ -3,6 +3,7 @@ package analyze import ( "bufio" "context" + "io" "os" "github.com/pkg/errors" @@ -10,16 +11,16 @@ import ( // readLinesFromOffset reads lines from a file starting at the given offset. // It will read until the end of the file. -func readLinesFromOffset(ctx context.Context, path string, offset int64) ([]string, int64, error) { +func readLinesFromOffset(ctx context.Context, path string, startOffset int64) ([]string, int64, error) { f, err := os.Open(path) if err != nil { return nil, 0, errors.Wrapf(err, "failed to open file %s", path) } defer f.Close() - offset, err = f.Seek(offset, 0) + offset, err := f.Seek(startOffset, io.SeekStart) if err != nil { - return nil, 0, errors.Wrapf(err, "failed to seek to offset %d in file %s", offset, path) + return nil, 0, errors.Wrapf(err, "failed to seek to offset %d in file %s", startOffset, path) } var lines []string diff --git a/app/pkg/analyze/test_data/logs/foyle.logs.2024-04-16T19:06:47.json b/app/pkg/analyze/test_data/logs/foyle.logs.2024-04-16T19:06:47.json index 94d8e4e5..8da2c325 100644 --- a/app/pkg/analyze/test_data/logs/foyle.logs.2024-04-16T19:06:47.json +++ b/app/pkg/analyze/test_data/logs/foyle.logs.2024-04-16T19:06:47.json @@ -74,5 +74,5 @@ {"severity":"info","time":1713319616.653316,"caller":"agent/agent.go:132","function":"github.com/jlewi/foyle/app/pkg/agent.(*Agent).completeWithRetries","message":"OpenAI:CreateChatCompletion response","traceId":"8038339848652bc1b97789cd9aa98341","resp":{"id":"chatcmpl-9EozHwRgILheEH7sxBtTMRdNlQzuI","object":"chat.completion","created":1713319615,"model":"gpt-3.5-turbo-0125","choices":[{"index":0,"message":{"role":"assistant","content":"To find the merge point with `origin/main` in your Git repository, you can use the following command:\n\n```bash\ngit merge-base HEAD origin/main\n```\n\nThis command will output the commit hash of the common ancestor between your current branch (`HEAD`) and `origin/main`, which represents the merge point.\n\nAfter running this command, you can see the commit hash that represents the merge point between your current branch and `origin/main`. Let's execute the command to get the specific commit hash."},"finish_reason":"stop"}],"usage":{"prompt_tokens":323,"completion_tokens":100,"total_tokens":423},"system_fingerprint":"fp_c2295e73ad"}} {"severity":"info","time":1713319616.6541498,"caller":"agent/agent.go:76","function":"github.com/jlewi/foyle/app/pkg/agent.(*Agent).Generate","message":"Agent.Generate returning blocks","traceId":"8038339848652bc1b97789cd9aa98341","blockIds":["9557680b-e08c-4d1d-b098-6dcd03e0e108","23706965-8e3b-440d-ba1a-1e1cc035fbd4","48d530be-254a-493f-8cf4-20627078f830"]} {"severity":"info","time":1713319616.654191,"caller":"agent/agent.go:83","function":"github.com/jlewi/foyle/app/pkg/agent.(*Agent).Generate","message":"Agent.Generate returning response","traceId":"8038339848652bc1b97789cd9aa98341","response":{"blocks":[{"kind":"MARKUP","language":"","contents":"To find the merge point with `origin/main` in your Git repository, you can use the following command:","outputs":[],"trace_ids":[],"id":"9557680b-e08c-4d1d-b098-6dcd03e0e108"},{"kind":"CODE","language":"bash","contents":"git merge-base HEAD origin/main\n","outputs":[],"trace_ids":[],"id":"23706965-8e3b-440d-ba1a-1e1cc035fbd4"},{"kind":"MARKUP","language":"","contents":"\n\nThis command will output the commit hash of the common ancestor between your current branch (`HEAD`) and `origin/main`, which represents the merge point.\n\nAfter running this command, you can see the commit hash that represents the merge point between your current branch and `origin/main`. Let's execute the command to get the specific commit hash.","outputs":[],"trace_ids":[],"id":"48d530be-254a-493f-8cf4-20627078f830"}]}} -{"severity":"info","time":1713319625.160002,"caller":"executor/executor.go:43","function":"github.com/jlewi/foyle/app/pkg/executor.(*Executor).Execute","message":"Executor.Execute","traceId":"071bd5002563938d3a5dfb6c191d9ed0","blockId":"23706965-8e3b-440d-ba1a-1e1cc035fbd4","request":{"block":{"kind":"CODE","language":"bash","contents":"git merge-base HEAD origin/main\n","outputs":[],"trace_ids":[],"id":"23706965-8e3b-440d-ba1a-1e1cc035fbd4"}}} -{"severity":"info","time":1713319625.19382,"caller":"executor/executor.go:61","function":"github.com/jlewi/foyle/app/pkg/executor.(*Executor).Execute","message":"Executed instructions","traceId":"071bd5002563938d3a5dfb6c191d9ed0","instructionsError":"json: unsupported type: chan string","response":{"outputs":[{"items":[{"mime":"text/plain","text_data":"exitCode: 0"}]},{"items":[{"mime":"text/plain","text_data":"stdout:\n93250b7b2eb08489a85f2422adfe828ed9dd6363"}]}]}} +{"severity":"info","time":1724993751.6962261,"caller":"agent/agent.go:530","function":"github.com/jlewi/foyle/app/pkg/agent.(*Agent).LogEvents","message":"LogEvent","eventId":"01J6GT7JMDA2Q4SWZQEQ2YPFYQ","eventType":"ACCEPTED","contextId":"01J6GT7BDHT5VN4QDADB1665D3","selectedCellId":"23706965-8e3b-440d-ba1a-1e1cc035fbd4","event":{"contextId":"01J6GT7BDHT5VN4QDADB1665D3","eventId":"01J6GT7JMDA2Q4SWZQEQ2YPFYQ","selectedId":"23706965-8e3b-440d-ba1a-1e1cc035fbd4","selectedIndex":10,"type":"ACCEPTED"}} +{"severity":"info","time":1724993755.738091,"caller":"agent/agent.go:530","function":"github.com/jlewi/foyle/app/pkg/agent.(*Agent).LogEvents","message":"LogEvent","eventId":"01J6GT7PJMZ0H6TXTEVZHXGXCH","eventType":"EXECUTE","contextId":"01J6GT7JM815EFXHDNBXDK91YW","selectedCellId":"23706965-8e3b-440d-ba1a-1e1cc035fbd4","event":{"cells":[{"kind":"CELL_KIND_MARKUP","value":"Fetch the latest logs using curl"},{"kind":"CELL_KIND_CODE","metadata":{"_ghostCell":"false","id":"23706965-8e3b-440d-ba1a-1e1cc035fbd4","runme.dev/id":"23706965-8e3b-440d-ba1a-1e1cc035fbd4"},"value":"curl"}],"contextId":"01J6GT7JM815EFXHDNBXDK91YW","eventId":"01J6GT7PJMZ0H6TXTEVZHXGXCH","selectedId":"23706965-8e3b-440d-ba1a-1e1cc035fbd4","selectedIndex":10,"type":"EXECUTE"}} diff --git a/app/pkg/application/app.go b/app/pkg/application/app.go index 96735e63..f53b59f1 100644 --- a/app/pkg/application/app.go +++ b/app/pkg/application/app.go @@ -479,12 +479,7 @@ func (a *App) Serve() error { } // Start any asynchronous workers in the components - logDirs := make([]string, 0, 2) - logDirs = append(logDirs, a.Config.GetRawLogDir()) - - if a.Config.Learner != nil { - logDirs = append(logDirs, a.Config.Learner.LogDirs...) - } + logDirs := []string{a.Config.GetRawLogDir()} if err := a.analyzer.Run(context.Background(), logDirs, a.learner.Enqueue); err != nil { return err @@ -506,7 +501,7 @@ func (a *App) Serve() error { return err } - s, err := server.NewServer(*a.Config, a.blocksDB, agent, a.TracesDB) + s, err := server.NewServer(*a.Config, a.blocksDB, agent, a.TracesDB, a.analyzer) if err != nil { return err diff --git a/app/pkg/config/config.go b/app/pkg/config/config.go index f140a3cd..1cf4a2c7 100644 --- a/app/pkg/config/config.go +++ b/app/pkg/config/config.go @@ -69,6 +69,8 @@ type Config struct { type LearnerConfig struct { // LogDirs is an additional list of directories to search for logs. + // Deprecated: We should remove this in v1alpha2. This is no longer needed now that we no longer rely on processing + // RunMe's logs but rather the UI sends the logs directly to the server. LogDirs []string `json:"logDirs" yaml:"logDirs"` // ExampleDirs is the list of directories to read/write examples. @@ -216,7 +218,7 @@ func (c *Config) GetLogDir() string { } func (c *Config) GetLogOffsetsFile() string { - return filepath.Join(c.GetLogDir(), "offsets.json") + return filepath.Join(c.GetLogDir(), "offsets.v1.json") } func (c *Config) GetRawLogDir() string { diff --git a/app/pkg/dbutil/locking_test.go b/app/pkg/dbutil/locking_test.go index f1bfadaa..03fe8373 100644 --- a/app/pkg/dbutil/locking_test.go +++ b/app/pkg/dbutil/locking_test.go @@ -5,8 +5,9 @@ import ( "sync" "testing" + "github.com/jlewi/foyle/protos/go/foyle/v1alpha1" + "github.com/cockroachdb/pebble" - "github.com/google/go-cmp/cmp" logspb "github.com/jlewi/foyle/protos/go/foyle/logs" "github.com/jlewi/monogo/helpers" ) @@ -82,7 +83,9 @@ func Test_Locking(t *testing.T) { execId := "exec" setExec := createModifyFunc(execId, wg, controlChan, func(block *logspb.BlockLog) error { - block.ExecTraceIds = []string{"exec1", "exec2"} + block.ExecutedBlock = &v1alpha1.Block{ + Id: execId, + } return nil }) @@ -132,7 +135,7 @@ func Test_Locking(t *testing.T) { t.Errorf("Expected GenTraceId to be generateID but got %s", block.GenTraceId) } - if d := cmp.Diff([]string{"exec1", "exec2"}, block.ExecTraceIds); d != "" { - t.Errorf("ExecTraceIds not as expected: diff:\n%v", d) + if block.ExecutedBlock.Id != execId { + t.Errorf("Expected ExecutedBlock.Id %s got %s", execId, block.ExecutedBlock.Id) } } diff --git a/app/pkg/logs/logs.go b/app/pkg/logs/logs.go index 6adf9103..085dc4a5 100644 --- a/app/pkg/logs/logs.go +++ b/app/pkg/logs/logs.go @@ -2,6 +2,10 @@ package logs import ( "context" + "encoding/json" + + "google.golang.org/protobuf/encoding/protojson" + "google.golang.org/protobuf/proto" "github.com/go-logr/logr" "github.com/go-logr/zapr" @@ -21,10 +25,38 @@ const ( func FromContext(ctx context.Context) logr.Logger { l, err := logr.FromContext(ctx) if err != nil { - // We need to AllowZapFields to ensure the protobuf message is logged correctly as a json object. - // For that to work we need to do logr.Info("message", zap.Object("key", protoMessage)) - // Which means we are passing zap.Field to the logr interface. - return zapr.NewLoggerWithOptions(zap.L(), zapr.AllowZapFields(true)) + return NewLogger() } return l } + +func NewLogger() logr.Logger { + // We need to AllowZapFields to ensure the protobuf message is logged correctly as a json object. + // For that to work we need to do logr.Info("message", zap.Object("key", protoMessage)) + // Which means we are passing zap.Field to the logr interface. + return zapr.NewLoggerWithOptions(zap.L(), zapr.AllowZapFields(true)) +} + +// ZapProto is a helper function to be able to log protos as JSON objects. +// We want protos to be logged using the proto json format so we can deserialize them from the logs. +// If you just log a proto with zap it will use the json serialization of the GoLang struct which will not match +// the proto json format. So we serialize the request to JSON and then deserialize it to a map so we can log it as a +// JSON object. A more efficient solution would be to use https://github.com/kazegusuri/go-proto-zap-marshaler +// to generate a custom zapcore.ObjectMarshaler implementation for each proto message. +func ZapProto(key string, pb proto.Message) zap.Field { + log := NewLogger() + reqObj := map[string]interface{}{} + reqJSON, err := protojson.Marshal(pb) + if err != nil { + log.Error(err, "failed to marshal request") + reqObj["error"] = err.Error() + return zap.Any(key, reqObj) + } + + if err := json.Unmarshal(reqJSON, &reqObj); err != nil { + log.Error(err, "failed to unmarshal request") + reqObj["error"] = err.Error() + } + + return zap.Any(key, reqObj) +} diff --git a/app/pkg/runme/converters/cells_to_blocks.go b/app/pkg/runme/converters/cells_to_blocks.go index a7715dc3..87f684d3 100644 --- a/app/pkg/runme/converters/cells_to_blocks.go +++ b/app/pkg/runme/converters/cells_to_blocks.go @@ -48,7 +48,8 @@ func CellToBlock(cell *parserv1.Cell) (*v1alpha1.Block, error) { id := "" if cell.Metadata != nil { - if newId, ok := cell.Metadata[IdField]; ok { + newId := GetCellID(cell) + if newId != "" { id = newId } } @@ -62,6 +63,22 @@ func CellToBlock(cell *parserv1.Cell) (*v1alpha1.Block, error) { }, nil } +// GetCellID returns the ID of a cell if it exists or none if it doesn't +func GetCellID(cell *parserv1.Cell) string { + if cell.Metadata != nil { + // See this thread + // See this thread https://discord.com/channels/1102639988832735374/1218835142962053193/1278863895813165128 + // RunMe uses two different fields for the ID field. We check both because the field we get could depend + // On how the cell was generated e.g. whether it went through the serializer or not. + if id, ok := cell.Metadata[RunmeIdField]; ok { + return id + } + if id, ok := cell.Metadata[IdField]; ok { + return id + } + } + return "" +} func CellKindToBlockKind(kind parserv1.CellKind) v1alpha1.BlockKind { switch kind { case parserv1.CellKind_CELL_KIND_CODE: diff --git a/app/pkg/runme/converters/const.go b/app/pkg/runme/converters/const.go index 147222f0..c57f5278 100644 --- a/app/pkg/runme/converters/const.go +++ b/app/pkg/runme/converters/const.go @@ -1,5 +1,10 @@ package converters const ( + // See this thread https://discord.com/channels/1102639988832735374/1218835142962053193/1278863895813165128 + + // RunmeIdField is the field name for the ID that Runme uses in memory + RunmeIdField = "runme.dev/id" + // IdField is the field name for the ID field when it is serialized IdField = "id" ) diff --git a/app/pkg/server/server.go b/app/pkg/server/server.go index f314d0b2..c1b231b9 100644 --- a/app/pkg/server/server.go +++ b/app/pkg/server/server.go @@ -62,7 +62,7 @@ type Server struct { } // NewServer creates a new server -func NewServer(config config.Config, blocksDB *pebble.DB, agent *agent.Agent, tracesDB *pebble.DB) (*Server, error) { +func NewServer(config config.Config, blocksDB *pebble.DB, agent *agent.Agent, tracesDB *pebble.DB, analyzer *analyze.Analyzer) (*Server, error) { e, err := executor.NewExecutor(config) if err != nil { return nil, err @@ -72,7 +72,7 @@ func NewServer(config config.Config, blocksDB *pebble.DB, agent *agent.Agent, tr return nil, errors.New("Agent is required") } - logsCrud, err := analyze.NewCrudHandler(config, blocksDB, tracesDB) + logsCrud, err := analyze.NewCrudHandler(config, blocksDB, tracesDB, analyzer) if err != nil { return nil, err } diff --git a/protos/README.md b/protos/README.md index 4c7d8dc8..8b322044 100644 --- a/protos/README.md +++ b/protos/README.md @@ -135,7 +135,7 @@ There are two ways I think we can iterate on the vscode extension when making ch In vscode-runme I'd gotten -``` +```ts {"id":"01J6FX93E4T2E0TPG5KX7GBT1Q"} ERROR in ./src/extension/ai/events.ts:66:33 TS2345: Argument of type 'LogEventsRequest' is not assignable to parameter of type 'PartialMessage'. Types of property 'events' are incompatible. @@ -153,7 +153,7 @@ TS2345: Argument of type 'LogEventsRequest' is not assignable to parameter of ty ``` -* I had updated the bufbuild/es package but not the connect/es package +* I had updated the bufbuild/es package but not the connect/es package * When Up updated the connect/es package the error went away ```bash {"id":"01J6CQT3CYQ76FRD1GX258JAW2"} diff --git a/protos/foyle/logs/blocks.proto b/protos/foyle/logs/blocks.proto index 00fc5e1f..53c53a72 100644 --- a/protos/foyle/logs/blocks.proto +++ b/protos/foyle/logs/blocks.proto @@ -15,9 +15,7 @@ message BlockLog { string id = 1; // gen_trace_id is the trace ID of the generation request string gen_trace_id = 2; - // exec_trace_ids are the trace IDs of the execution requests - // Doc is the doc that triggered the generated block - repeated string exec_trace_ids = 3; + // doc is the doc that triggered the generated block Doc doc = 4; // generatedBlock is the block generated by the AI @@ -33,4 +31,16 @@ message BlockLog { // ResourceVersion is an opaque string that can be used for optimistic concurrency control string resource_version = 9; + + SuggestionStatus suggestion_status = 10; + + reserved 3; +} + +// SuggestionStatus is the status of a suggestion +enum SuggestionStatus { + // SuggestionStatusUnknown is the default status + SuggestionStatusUnknown = 0; + ACCEPTED = 1; + REJECTED = 2; } \ No newline at end of file diff --git a/protos/foyle/logs/logs.proto b/protos/foyle/logs/logs.proto new file mode 100644 index 00000000..ca823725 --- /dev/null +++ b/protos/foyle/logs/logs.proto @@ -0,0 +1,20 @@ +syntax = "proto3"; + +package foyle.logs; + +option go_package = "github.com/jlewi/foyle/protos/go/foyle/logs;logspb"; + +// LogsWaterMark is the watermark keep track of the logs that have been processed +message LogsWaterMark { + // The file is the name of the file that is associated with the watermark + string file = 1; + // The offset is the offset in the file that is associated with the watermark + int64 offset = 2; +} + +message GetLogsStatusRequest { +} + +message GetLogsStatusResponse { + LogsWaterMark watermark = 1; +} \ No newline at end of file diff --git a/protos/foyle/logs/traces.proto b/protos/foyle/logs/traces.proto index 81ffe4b5..6b3b88fe 100644 --- a/protos/foyle/logs/traces.proto +++ b/protos/foyle/logs/traces.proto @@ -1,5 +1,6 @@ syntax = "proto3"; +import "foyle/logs/logs.proto"; import "foyle/v1alpha1/agent.proto"; import "foyle/v1alpha1/trainer.proto"; import "foyle/logs/blocks.proto"; @@ -7,6 +8,7 @@ import "runme/runner/v1/runner.proto"; import "google/protobuf/struct.proto"; import "google/protobuf/timestamp.proto"; +import "runme/parser/v1/parser.proto"; package foyle.logs; @@ -20,14 +22,14 @@ message Trace { // TODO(jeremy): Should these really be spans? oneof data { GenerateTrace generate = 4; - ExecuteTrace execute = 5; - RunMeTrace run_me = 7; } // Eval mode is true if the trace was generated in eval mode. bool eval_mode = 6; repeated Span spans = 8; + + reserved 5,7; } message Span { @@ -48,16 +50,6 @@ message GenerateTrace { GenerateResponse response = 2; } -message ExecuteTrace { - ExecuteRequest request = 1; - ExecuteResponse response = 2; -} - -message RunMeTrace { - runme.runner.v1.ExecuteRequest request = 1; - runme.runner.v1.ExecuteResponse response = 2; -} - // LogEntries is used to store log lines keyed by a trace id. message LogEntries { repeated string lines = 1; @@ -74,6 +66,8 @@ service LogsService { // These will include the rendered prompt and response. Unlike GetTraceRequest this has the // actual prompt and response of the LLM. rpc GetLLMLogs(GetLLMLogsRequest) returns (GetLLMLogsResponse) {} + + rpc Status(GetLogsStatusRequest) returns (GetLogsStatusResponse) {} } message GetTraceRequest { diff --git a/protos/foyle/v1alpha1/agent.proto b/protos/foyle/v1alpha1/agent.proto index 9c68614d..2e9049ff 100644 --- a/protos/foyle/v1alpha1/agent.proto +++ b/protos/foyle/v1alpha1/agent.proto @@ -158,6 +158,9 @@ message LogEvent { // The position of the cell with selected_id. This will usually be the position in the full context as specified // by selected_id. int32 selected_index = 5; + + // A unique ID for the event + string event_id = 6; } enum LogEventType { diff --git a/protos/go/foyle/logs/blocks.pb.go b/protos/go/foyle/logs/blocks.pb.go index 2a66eafb..1626e1e1 100644 --- a/protos/go/foyle/logs/blocks.pb.go +++ b/protos/go/foyle/logs/blocks.pb.go @@ -23,6 +23,57 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +// SuggestionStatus is the status of a suggestion +type SuggestionStatus int32 + +const ( + // SuggestionStatusUnknown is the default status + SuggestionStatus_SuggestionStatusUnknown SuggestionStatus = 0 + SuggestionStatus_ACCEPTED SuggestionStatus = 1 + SuggestionStatus_REJECTED SuggestionStatus = 2 +) + +// Enum value maps for SuggestionStatus. +var ( + SuggestionStatus_name = map[int32]string{ + 0: "SuggestionStatusUnknown", + 1: "ACCEPTED", + 2: "REJECTED", + } + SuggestionStatus_value = map[string]int32{ + "SuggestionStatusUnknown": 0, + "ACCEPTED": 1, + "REJECTED": 2, + } +) + +func (x SuggestionStatus) Enum() *SuggestionStatus { + p := new(SuggestionStatus) + *p = x + return p +} + +func (x SuggestionStatus) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SuggestionStatus) Descriptor() protoreflect.EnumDescriptor { + return file_foyle_logs_blocks_proto_enumTypes[0].Descriptor() +} + +func (SuggestionStatus) Type() protoreflect.EnumType { + return &file_foyle_logs_blocks_proto_enumTypes[0] +} + +func (x SuggestionStatus) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SuggestionStatus.Descriptor instead. +func (SuggestionStatus) EnumDescriptor() ([]byte, []int) { + return file_foyle_logs_blocks_proto_rawDescGZIP(), []int{0} +} + // BlockLog is the log of what happened to a block. It includes information about how a block was generated (if it // was generated by the AI) and how it was executed if it was. type BlockLog struct { @@ -33,9 +84,6 @@ type BlockLog struct { Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"` // gen_trace_id is the trace ID of the generation request GenTraceId string `protobuf:"bytes,2,opt,name=gen_trace_id,json=genTraceId,proto3" json:"gen_trace_id,omitempty"` - // exec_trace_ids are the trace IDs of the execution requests - // Doc is the doc that triggered the generated block - ExecTraceIds []string `protobuf:"bytes,3,rep,name=exec_trace_ids,json=execTraceIds,proto3" json:"exec_trace_ids,omitempty"` // doc is the doc that triggered the generated block Doc *v1alpha1.Doc `protobuf:"bytes,4,opt,name=doc,proto3" json:"doc,omitempty"` // generatedBlock is the block generated by the AI @@ -48,7 +96,8 @@ type BlockLog struct { // eval_mode is true if the block was generated as part of an evaluation and shouldn't be used for learning EvalMode bool `protobuf:"varint,8,opt,name=eval_mode,json=evalMode,proto3" json:"eval_mode,omitempty"` // ResourceVersion is an opaque string that can be used for optimistic concurrency control - ResourceVersion string `protobuf:"bytes,9,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + ResourceVersion string `protobuf:"bytes,9,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + SuggestionStatus SuggestionStatus `protobuf:"varint,10,opt,name=suggestion_status,json=suggestionStatus,proto3,enum=foyle.logs.SuggestionStatus" json:"suggestion_status,omitempty"` } func (x *BlockLog) Reset() { @@ -97,13 +146,6 @@ func (x *BlockLog) GetGenTraceId() string { return "" } -func (x *BlockLog) GetExecTraceIds() []string { - if x != nil { - return x.ExecTraceIds - } - return nil -} - func (x *BlockLog) GetDoc() *v1alpha1.Doc { if x != nil { return x.Doc @@ -146,6 +188,13 @@ func (x *BlockLog) GetResourceVersion() string { return "" } +func (x *BlockLog) GetSuggestionStatus() SuggestionStatus { + if x != nil { + return x.SuggestionStatus + } + return SuggestionStatus_SuggestionStatusUnknown +} + var File_foyle_logs_blocks_proto protoreflect.FileDescriptor var file_foyle_logs_blocks_proto_rawDesc = []byte{ @@ -158,38 +207,45 @@ var file_foyle_logs_blocks_proto_rawDesc = []byte{ 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xbf, 0x02, 0x0a, 0x08, 0x42, + 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xea, 0x02, 0x0a, 0x08, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x20, 0x0a, 0x0c, 0x67, 0x65, 0x6e, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x67, - 0x65, 0x6e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0e, 0x65, 0x78, 0x65, - 0x63, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x0c, 0x65, 0x78, 0x65, 0x63, 0x54, 0x72, 0x61, 0x63, 0x65, 0x49, 0x64, 0x73, 0x12, - 0x16, 0x0a, 0x03, 0x64, 0x6f, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x04, 0x2e, 0x44, - 0x6f, 0x63, 0x52, 0x03, 0x64, 0x6f, 0x63, 0x12, 0x2f, 0x0a, 0x0f, 0x67, 0x65, 0x6e, 0x65, 0x72, - 0x61, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x06, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x0e, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, - 0x74, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x2d, 0x0a, 0x0e, 0x65, 0x78, 0x65, 0x63, - 0x75, 0x74, 0x65, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x06, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x0d, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x78, 0x69, 0x74, 0x5f, - 0x63, 0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x65, 0x78, 0x69, 0x74, - 0x43, 0x6f, 0x64, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x6d, 0x6f, 0x64, - 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x65, 0x76, 0x61, 0x6c, 0x4d, 0x6f, 0x64, - 0x65, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x9a, 0x01, 0x0a, - 0x0e, 0x63, 0x6f, 0x6d, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x42, - 0x0b, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x32, - 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6a, 0x6c, 0x65, 0x77, 0x69, - 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x2f, 0x67, 0x6f, - 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x3b, 0x6c, 0x6f, 0x67, 0x73, - 0x70, 0x62, 0xa2, 0x02, 0x03, 0x46, 0x4c, 0x58, 0xaa, 0x02, 0x0a, 0x46, 0x6f, 0x79, 0x6c, 0x65, - 0x2e, 0x4c, 0x6f, 0x67, 0x73, 0xca, 0x02, 0x0a, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x5c, 0x4c, 0x6f, - 0x67, 0x73, 0xe2, 0x02, 0x16, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x5c, 0x4c, 0x6f, 0x67, 0x73, 0x5c, - 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x46, 0x6f, - 0x79, 0x6c, 0x65, 0x3a, 0x3a, 0x4c, 0x6f, 0x67, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x65, 0x6e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x03, 0x64, 0x6f, 0x63, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x04, 0x2e, 0x44, 0x6f, 0x63, 0x52, 0x03, 0x64, 0x6f, + 0x63, 0x12, 0x2f, 0x0a, 0x0f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x06, 0x2e, 0x42, 0x6c, 0x6f, + 0x63, 0x6b, 0x52, 0x0e, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x42, 0x6c, 0x6f, + 0x63, 0x6b, 0x12, 0x2d, 0x0a, 0x0e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x64, 0x5f, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x06, 0x2e, 0x42, 0x6c, 0x6f, + 0x63, 0x6b, 0x52, 0x0d, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, + 0x6b, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x65, 0x78, 0x69, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x1b, + 0x0a, 0x09, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x08, 0x65, 0x76, 0x61, 0x6c, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x49, 0x0a, 0x11, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x1c, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x53, + 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x10, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x4a, 0x04, 0x08, 0x03, 0x10, 0x04, 0x2a, 0x4b, 0x0a, 0x10, 0x53, 0x75, 0x67, 0x67, 0x65, + 0x73, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1b, 0x0a, 0x17, 0x53, + 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x55, + 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x41, 0x43, 0x43, 0x45, + 0x50, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x45, 0x4a, 0x45, 0x43, 0x54, + 0x45, 0x44, 0x10, 0x02, 0x42, 0x9a, 0x01, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e, 0x66, 0x6f, 0x79, + 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x42, 0x0b, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x50, + 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x6a, 0x6c, 0x65, 0x77, 0x69, 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x6c, + 0x6f, 0x67, 0x73, 0x3b, 0x6c, 0x6f, 0x67, 0x73, 0x70, 0x62, 0xa2, 0x02, 0x03, 0x46, 0x4c, 0x58, + 0xaa, 0x02, 0x0a, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x4c, 0x6f, 0x67, 0x73, 0xca, 0x02, 0x0a, + 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x5c, 0x4c, 0x6f, 0x67, 0x73, 0xe2, 0x02, 0x16, 0x46, 0x6f, 0x79, + 0x6c, 0x65, 0x5c, 0x4c, 0x6f, 0x67, 0x73, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x3a, 0x3a, 0x4c, 0x6f, 0x67, + 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -204,21 +260,24 @@ func file_foyle_logs_blocks_proto_rawDescGZIP() []byte { return file_foyle_logs_blocks_proto_rawDescData } +var file_foyle_logs_blocks_proto_enumTypes = make([]protoimpl.EnumInfo, 1) var file_foyle_logs_blocks_proto_msgTypes = make([]protoimpl.MessageInfo, 1) var file_foyle_logs_blocks_proto_goTypes = []interface{}{ - (*BlockLog)(nil), // 0: foyle.logs.BlockLog - (*v1alpha1.Doc)(nil), // 1: Doc - (*v1alpha1.Block)(nil), // 2: Block + (SuggestionStatus)(0), // 0: foyle.logs.SuggestionStatus + (*BlockLog)(nil), // 1: foyle.logs.BlockLog + (*v1alpha1.Doc)(nil), // 2: Doc + (*v1alpha1.Block)(nil), // 3: Block } var file_foyle_logs_blocks_proto_depIdxs = []int32{ - 1, // 0: foyle.logs.BlockLog.doc:type_name -> Doc - 2, // 1: foyle.logs.BlockLog.generated_block:type_name -> Block - 2, // 2: foyle.logs.BlockLog.executed_block:type_name -> Block - 3, // [3:3] is the sub-list for method output_type - 3, // [3:3] is the sub-list for method input_type - 3, // [3:3] is the sub-list for extension type_name - 3, // [3:3] is the sub-list for extension extendee - 0, // [0:3] is the sub-list for field type_name + 2, // 0: foyle.logs.BlockLog.doc:type_name -> Doc + 3, // 1: foyle.logs.BlockLog.generated_block:type_name -> Block + 3, // 2: foyle.logs.BlockLog.executed_block:type_name -> Block + 0, // 3: foyle.logs.BlockLog.suggestion_status:type_name -> foyle.logs.SuggestionStatus + 4, // [4:4] is the sub-list for method output_type + 4, // [4:4] is the sub-list for method input_type + 4, // [4:4] is the sub-list for extension type_name + 4, // [4:4] is the sub-list for extension extendee + 0, // [0:4] is the sub-list for field type_name } func init() { file_foyle_logs_blocks_proto_init() } @@ -245,13 +304,14 @@ func file_foyle_logs_blocks_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_foyle_logs_blocks_proto_rawDesc, - NumEnums: 0, + NumEnums: 1, NumMessages: 1, NumExtensions: 0, NumServices: 0, }, GoTypes: file_foyle_logs_blocks_proto_goTypes, DependencyIndexes: file_foyle_logs_blocks_proto_depIdxs, + EnumInfos: file_foyle_logs_blocks_proto_enumTypes, MessageInfos: file_foyle_logs_blocks_proto_msgTypes, }.Build() File_foyle_logs_blocks_proto = out.File diff --git a/protos/go/foyle/logs/blocks.zap.go b/protos/go/foyle/logs/blocks.zap.go index ba136558..0911e359 100644 --- a/protos/go/foyle/logs/blocks.zap.go +++ b/protos/go/foyle/logs/blocks.zap.go @@ -32,15 +32,6 @@ func (m *BlockLog) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) e keyName = "gen_trace_id" // field gen_trace_id = 2 enc.AddString(keyName, m.GenTraceId) - keyName = "exec_trace_ids" // field exec_trace_ids = 3 - enc.AddArray(keyName, go_uber_org_zap_zapcore.ArrayMarshalerFunc(func(aenc go_uber_org_zap_zapcore.ArrayEncoder) error { - for _, rv := range m.ExecTraceIds { - _ = rv - aenc.AppendString(rv) - } - return nil - })) - keyName = "doc" // field doc = 4 if m.Doc != nil { var vv interface{} = m.Doc @@ -74,5 +65,8 @@ func (m *BlockLog) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) e keyName = "resource_version" // field resource_version = 9 enc.AddString(keyName, m.ResourceVersion) + keyName = "suggestion_status" // field suggestion_status = 10 + enc.AddString(keyName, m.SuggestionStatus.String()) + return nil } diff --git a/protos/go/foyle/logs/logs.pb.go b/protos/go/foyle/logs/logs.pb.go new file mode 100644 index 00000000..8dd78f9e --- /dev/null +++ b/protos/go/foyle/logs/logs.pb.go @@ -0,0 +1,282 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.28.1 +// protoc (unknown) +// source: foyle/logs/logs.proto + +package logspb + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// LogsWaterMark is the watermark keep track of the logs that have been processed +type LogsWaterMark struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The file is the name of the file that is associated with the watermark + File string `protobuf:"bytes,1,opt,name=file,proto3" json:"file,omitempty"` + // The offset is the offset in the file that is associated with the watermark + Offset int64 `protobuf:"varint,2,opt,name=offset,proto3" json:"offset,omitempty"` +} + +func (x *LogsWaterMark) Reset() { + *x = LogsWaterMark{} + if protoimpl.UnsafeEnabled { + mi := &file_foyle_logs_logs_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LogsWaterMark) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LogsWaterMark) ProtoMessage() {} + +func (x *LogsWaterMark) ProtoReflect() protoreflect.Message { + mi := &file_foyle_logs_logs_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LogsWaterMark.ProtoReflect.Descriptor instead. +func (*LogsWaterMark) Descriptor() ([]byte, []int) { + return file_foyle_logs_logs_proto_rawDescGZIP(), []int{0} +} + +func (x *LogsWaterMark) GetFile() string { + if x != nil { + return x.File + } + return "" +} + +func (x *LogsWaterMark) GetOffset() int64 { + if x != nil { + return x.Offset + } + return 0 +} + +type GetLogsStatusRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *GetLogsStatusRequest) Reset() { + *x = GetLogsStatusRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_foyle_logs_logs_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetLogsStatusRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetLogsStatusRequest) ProtoMessage() {} + +func (x *GetLogsStatusRequest) ProtoReflect() protoreflect.Message { + mi := &file_foyle_logs_logs_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetLogsStatusRequest.ProtoReflect.Descriptor instead. +func (*GetLogsStatusRequest) Descriptor() ([]byte, []int) { + return file_foyle_logs_logs_proto_rawDescGZIP(), []int{1} +} + +type GetLogsStatusResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Watermark *LogsWaterMark `protobuf:"bytes,1,opt,name=watermark,proto3" json:"watermark,omitempty"` +} + +func (x *GetLogsStatusResponse) Reset() { + *x = GetLogsStatusResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_foyle_logs_logs_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetLogsStatusResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetLogsStatusResponse) ProtoMessage() {} + +func (x *GetLogsStatusResponse) ProtoReflect() protoreflect.Message { + mi := &file_foyle_logs_logs_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GetLogsStatusResponse.ProtoReflect.Descriptor instead. +func (*GetLogsStatusResponse) Descriptor() ([]byte, []int) { + return file_foyle_logs_logs_proto_rawDescGZIP(), []int{2} +} + +func (x *GetLogsStatusResponse) GetWatermark() *LogsWaterMark { + if x != nil { + return x.Watermark + } + return nil +} + +var File_foyle_logs_logs_proto protoreflect.FileDescriptor + +var file_foyle_logs_logs_proto_rawDesc = []byte{ + 0x0a, 0x15, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x2f, 0x6c, 0x6f, 0x67, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, + 0x6f, 0x67, 0x73, 0x22, 0x3b, 0x0a, 0x0d, 0x4c, 0x6f, 0x67, 0x73, 0x57, 0x61, 0x74, 0x65, 0x72, + 0x4d, 0x61, 0x72, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x66, 0x69, 0x6c, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, + 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, + 0x22, 0x16, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x50, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x4c, + 0x6f, 0x67, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x37, 0x0a, 0x09, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, + 0x73, 0x2e, 0x4c, 0x6f, 0x67, 0x73, 0x57, 0x61, 0x74, 0x65, 0x72, 0x4d, 0x61, 0x72, 0x6b, 0x52, + 0x09, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x42, 0x98, 0x01, 0x0a, 0x0e, 0x63, + 0x6f, 0x6d, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x42, 0x09, 0x4c, + 0x6f, 0x67, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x68, + 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6a, 0x6c, 0x65, 0x77, 0x69, 0x2f, 0x66, 0x6f, 0x79, + 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x66, 0x6f, 0x79, + 0x6c, 0x65, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x3b, 0x6c, 0x6f, 0x67, 0x73, 0x70, 0x62, 0xa2, 0x02, + 0x03, 0x46, 0x4c, 0x58, 0xaa, 0x02, 0x0a, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x4c, 0x6f, 0x67, + 0x73, 0xca, 0x02, 0x0a, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x5c, 0x4c, 0x6f, 0x67, 0x73, 0xe2, 0x02, + 0x16, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x5c, 0x4c, 0x6f, 0x67, 0x73, 0x5c, 0x47, 0x50, 0x42, 0x4d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x3a, + 0x3a, 0x4c, 0x6f, 0x67, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_foyle_logs_logs_proto_rawDescOnce sync.Once + file_foyle_logs_logs_proto_rawDescData = file_foyle_logs_logs_proto_rawDesc +) + +func file_foyle_logs_logs_proto_rawDescGZIP() []byte { + file_foyle_logs_logs_proto_rawDescOnce.Do(func() { + file_foyle_logs_logs_proto_rawDescData = protoimpl.X.CompressGZIP(file_foyle_logs_logs_proto_rawDescData) + }) + return file_foyle_logs_logs_proto_rawDescData +} + +var file_foyle_logs_logs_proto_msgTypes = make([]protoimpl.MessageInfo, 3) +var file_foyle_logs_logs_proto_goTypes = []interface{}{ + (*LogsWaterMark)(nil), // 0: foyle.logs.LogsWaterMark + (*GetLogsStatusRequest)(nil), // 1: foyle.logs.GetLogsStatusRequest + (*GetLogsStatusResponse)(nil), // 2: foyle.logs.GetLogsStatusResponse +} +var file_foyle_logs_logs_proto_depIdxs = []int32{ + 0, // 0: foyle.logs.GetLogsStatusResponse.watermark:type_name -> foyle.logs.LogsWaterMark + 1, // [1:1] is the sub-list for method output_type + 1, // [1:1] is the sub-list for method input_type + 1, // [1:1] is the sub-list for extension type_name + 1, // [1:1] is the sub-list for extension extendee + 0, // [0:1] is the sub-list for field type_name +} + +func init() { file_foyle_logs_logs_proto_init() } +func file_foyle_logs_logs_proto_init() { + if File_foyle_logs_logs_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_foyle_logs_logs_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LogsWaterMark); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_foyle_logs_logs_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetLogsStatusRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_foyle_logs_logs_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetLogsStatusResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_foyle_logs_logs_proto_rawDesc, + NumEnums: 0, + NumMessages: 3, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_foyle_logs_logs_proto_goTypes, + DependencyIndexes: file_foyle_logs_logs_proto_depIdxs, + MessageInfos: file_foyle_logs_logs_proto_msgTypes, + }.Build() + File_foyle_logs_logs_proto = out.File + file_foyle_logs_logs_proto_rawDesc = nil + file_foyle_logs_logs_proto_goTypes = nil + file_foyle_logs_logs_proto_depIdxs = nil +} diff --git a/protos/go/foyle/logs/logs.zap.go b/protos/go/foyle/logs/logs.zap.go new file mode 100644 index 00000000..ccd5086f --- /dev/null +++ b/protos/go/foyle/logs/logs.zap.go @@ -0,0 +1,63 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: foyle/logs/logs.proto + +package logspb + +import ( + fmt "fmt" + math "math" + proto "github.com/golang/protobuf/proto" + go_uber_org_zap_zapcore "go.uber.org/zap/zapcore" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +func (m *LogsWaterMark) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) error { + var keyName string + _ = keyName + + if m == nil { + return nil + } + + keyName = "file" // field file = 1 + enc.AddString(keyName, m.File) + + keyName = "offset" // field offset = 2 + enc.AddInt64(keyName, m.Offset) + + return nil +} + +func (m *GetLogsStatusRequest) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) error { + var keyName string + _ = keyName + + if m == nil { + return nil + } + + return nil +} + +func (m *GetLogsStatusResponse) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) error { + var keyName string + _ = keyName + + if m == nil { + return nil + } + + keyName = "watermark" // field watermark = 1 + if m.Watermark != nil { + var vv interface{} = m.Watermark + if marshaler, ok := vv.(go_uber_org_zap_zapcore.ObjectMarshaler); ok { + enc.AddObject(keyName, marshaler) + } + } + + return nil +} diff --git a/protos/go/foyle/logs/logspbconnect/traces.connect.go b/protos/go/foyle/logs/logspbconnect/traces.connect.go index 322fcca1..5743cecd 100644 --- a/protos/go/foyle/logs/logspbconnect/traces.connect.go +++ b/protos/go/foyle/logs/logspbconnect/traces.connect.go @@ -39,6 +39,8 @@ const ( LogsServiceGetBlockLogProcedure = "/foyle.logs.LogsService/GetBlockLog" // LogsServiceGetLLMLogsProcedure is the fully-qualified name of the LogsService's GetLLMLogs RPC. LogsServiceGetLLMLogsProcedure = "/foyle.logs.LogsService/GetLLMLogs" + // LogsServiceStatusProcedure is the fully-qualified name of the LogsService's Status RPC. + LogsServiceStatusProcedure = "/foyle.logs.LogsService/Status" ) // These variables are the protoreflect.Descriptor objects for the RPCs defined in this package. @@ -47,6 +49,7 @@ var ( logsServiceGetTraceMethodDescriptor = logsServiceServiceDescriptor.Methods().ByName("GetTrace") logsServiceGetBlockLogMethodDescriptor = logsServiceServiceDescriptor.Methods().ByName("GetBlockLog") logsServiceGetLLMLogsMethodDescriptor = logsServiceServiceDescriptor.Methods().ByName("GetLLMLogs") + logsServiceStatusMethodDescriptor = logsServiceServiceDescriptor.Methods().ByName("Status") ) // LogsServiceClient is a client for the foyle.logs.LogsService service. @@ -57,6 +60,7 @@ type LogsServiceClient interface { // These will include the rendered prompt and response. Unlike GetTraceRequest this has the // actual prompt and response of the LLM. GetLLMLogs(context.Context, *connect.Request[logs.GetLLMLogsRequest]) (*connect.Response[logs.GetLLMLogsResponse], error) + Status(context.Context, *connect.Request[logs.GetLogsStatusRequest]) (*connect.Response[logs.GetLogsStatusResponse], error) } // NewLogsServiceClient constructs a client for the foyle.logs.LogsService service. By default, it @@ -87,6 +91,12 @@ func NewLogsServiceClient(httpClient connect.HTTPClient, baseURL string, opts .. connect.WithSchema(logsServiceGetLLMLogsMethodDescriptor), connect.WithClientOptions(opts...), ), + status: connect.NewClient[logs.GetLogsStatusRequest, logs.GetLogsStatusResponse]( + httpClient, + baseURL+LogsServiceStatusProcedure, + connect.WithSchema(logsServiceStatusMethodDescriptor), + connect.WithClientOptions(opts...), + ), } } @@ -95,6 +105,7 @@ type logsServiceClient struct { getTrace *connect.Client[logs.GetTraceRequest, logs.GetTraceResponse] getBlockLog *connect.Client[logs.GetBlockLogRequest, logs.GetBlockLogResponse] getLLMLogs *connect.Client[logs.GetLLMLogsRequest, logs.GetLLMLogsResponse] + status *connect.Client[logs.GetLogsStatusRequest, logs.GetLogsStatusResponse] } // GetTrace calls foyle.logs.LogsService.GetTrace. @@ -112,6 +123,11 @@ func (c *logsServiceClient) GetLLMLogs(ctx context.Context, req *connect.Request return c.getLLMLogs.CallUnary(ctx, req) } +// Status calls foyle.logs.LogsService.Status. +func (c *logsServiceClient) Status(ctx context.Context, req *connect.Request[logs.GetLogsStatusRequest]) (*connect.Response[logs.GetLogsStatusResponse], error) { + return c.status.CallUnary(ctx, req) +} + // LogsServiceHandler is an implementation of the foyle.logs.LogsService service. type LogsServiceHandler interface { GetTrace(context.Context, *connect.Request[logs.GetTraceRequest]) (*connect.Response[logs.GetTraceResponse], error) @@ -120,6 +136,7 @@ type LogsServiceHandler interface { // These will include the rendered prompt and response. Unlike GetTraceRequest this has the // actual prompt and response of the LLM. GetLLMLogs(context.Context, *connect.Request[logs.GetLLMLogsRequest]) (*connect.Response[logs.GetLLMLogsResponse], error) + Status(context.Context, *connect.Request[logs.GetLogsStatusRequest]) (*connect.Response[logs.GetLogsStatusResponse], error) } // NewLogsServiceHandler builds an HTTP handler from the service implementation. It returns the path @@ -146,6 +163,12 @@ func NewLogsServiceHandler(svc LogsServiceHandler, opts ...connect.HandlerOption connect.WithSchema(logsServiceGetLLMLogsMethodDescriptor), connect.WithHandlerOptions(opts...), ) + logsServiceStatusHandler := connect.NewUnaryHandler( + LogsServiceStatusProcedure, + svc.Status, + connect.WithSchema(logsServiceStatusMethodDescriptor), + connect.WithHandlerOptions(opts...), + ) return "/foyle.logs.LogsService/", http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { switch r.URL.Path { case LogsServiceGetTraceProcedure: @@ -154,6 +177,8 @@ func NewLogsServiceHandler(svc LogsServiceHandler, opts ...connect.HandlerOption logsServiceGetBlockLogHandler.ServeHTTP(w, r) case LogsServiceGetLLMLogsProcedure: logsServiceGetLLMLogsHandler.ServeHTTP(w, r) + case LogsServiceStatusProcedure: + logsServiceStatusHandler.ServeHTTP(w, r) default: http.NotFound(w, r) } @@ -174,3 +199,7 @@ func (UnimplementedLogsServiceHandler) GetBlockLog(context.Context, *connect.Req func (UnimplementedLogsServiceHandler) GetLLMLogs(context.Context, *connect.Request[logs.GetLLMLogsRequest]) (*connect.Response[logs.GetLLMLogsResponse], error) { return nil, connect.NewError(connect.CodeUnimplemented, errors.New("foyle.logs.LogsService.GetLLMLogs is not implemented")) } + +func (UnimplementedLogsServiceHandler) Status(context.Context, *connect.Request[logs.GetLogsStatusRequest]) (*connect.Response[logs.GetLogsStatusResponse], error) { + return nil, connect.NewError(connect.CodeUnimplemented, errors.New("foyle.logs.LogsService.Status is not implemented")) +} diff --git a/protos/go/foyle/logs/traces.pb.go b/protos/go/foyle/logs/traces.pb.go index 2b95f332..c7ca76da 100644 --- a/protos/go/foyle/logs/traces.pb.go +++ b/protos/go/foyle/logs/traces.pb.go @@ -8,7 +8,8 @@ package logspb import ( v1alpha1 "github.com/jlewi/foyle/protos/go/foyle/v1alpha1" - v1 "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/runner/v1" + _ "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/parser/v1" + _ "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/runner/v1" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" _ "google.golang.org/protobuf/types/known/structpb" @@ -36,8 +37,6 @@ type Trace struct { // // Types that are assignable to Data: // *Trace_Generate - // *Trace_Execute - // *Trace_RunMe Data isTrace_Data `protobuf_oneof:"data"` // Eval mode is true if the trace was generated in eval mode. EvalMode bool `protobuf:"varint,6,opt,name=eval_mode,json=evalMode,proto3" json:"eval_mode,omitempty"` @@ -111,20 +110,6 @@ func (x *Trace) GetGenerate() *GenerateTrace { return nil } -func (x *Trace) GetExecute() *ExecuteTrace { - if x, ok := x.GetData().(*Trace_Execute); ok { - return x.Execute - } - return nil -} - -func (x *Trace) GetRunMe() *RunMeTrace { - if x, ok := x.GetData().(*Trace_RunMe); ok { - return x.RunMe - } - return nil -} - func (x *Trace) GetEvalMode() bool { if x != nil { return x.EvalMode @@ -147,20 +132,8 @@ type Trace_Generate struct { Generate *GenerateTrace `protobuf:"bytes,4,opt,name=generate,proto3,oneof"` } -type Trace_Execute struct { - Execute *ExecuteTrace `protobuf:"bytes,5,opt,name=execute,proto3,oneof"` -} - -type Trace_RunMe struct { - RunMe *RunMeTrace `protobuf:"bytes,7,opt,name=run_me,json=runMe,proto3,oneof"` -} - func (*Trace_Generate) isTrace_Data() {} -func (*Trace_Execute) isTrace_Data() {} - -func (*Trace_RunMe) isTrace_Data() {} - type Span struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -345,116 +318,6 @@ func (x *GenerateTrace) GetResponse() *v1alpha1.GenerateResponse { return nil } -type ExecuteTrace struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Request *v1alpha1.ExecuteRequest `protobuf:"bytes,1,opt,name=request,proto3" json:"request,omitempty"` - Response *v1alpha1.ExecuteResponse `protobuf:"bytes,2,opt,name=response,proto3" json:"response,omitempty"` -} - -func (x *ExecuteTrace) Reset() { - *x = ExecuteTrace{} - if protoimpl.UnsafeEnabled { - mi := &file_foyle_logs_traces_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ExecuteTrace) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ExecuteTrace) ProtoMessage() {} - -func (x *ExecuteTrace) ProtoReflect() protoreflect.Message { - mi := &file_foyle_logs_traces_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use ExecuteTrace.ProtoReflect.Descriptor instead. -func (*ExecuteTrace) Descriptor() ([]byte, []int) { - return file_foyle_logs_traces_proto_rawDescGZIP(), []int{4} -} - -func (x *ExecuteTrace) GetRequest() *v1alpha1.ExecuteRequest { - if x != nil { - return x.Request - } - return nil -} - -func (x *ExecuteTrace) GetResponse() *v1alpha1.ExecuteResponse { - if x != nil { - return x.Response - } - return nil -} - -type RunMeTrace struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - Request *v1.ExecuteRequest `protobuf:"bytes,1,opt,name=request,proto3" json:"request,omitempty"` - Response *v1.ExecuteResponse `protobuf:"bytes,2,opt,name=response,proto3" json:"response,omitempty"` -} - -func (x *RunMeTrace) Reset() { - *x = RunMeTrace{} - if protoimpl.UnsafeEnabled { - mi := &file_foyle_logs_traces_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *RunMeTrace) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*RunMeTrace) ProtoMessage() {} - -func (x *RunMeTrace) ProtoReflect() protoreflect.Message { - mi := &file_foyle_logs_traces_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use RunMeTrace.ProtoReflect.Descriptor instead. -func (*RunMeTrace) Descriptor() ([]byte, []int) { - return file_foyle_logs_traces_proto_rawDescGZIP(), []int{5} -} - -func (x *RunMeTrace) GetRequest() *v1.ExecuteRequest { - if x != nil { - return x.Request - } - return nil -} - -func (x *RunMeTrace) GetResponse() *v1.ExecuteResponse { - if x != nil { - return x.Response - } - return nil -} - // LogEntries is used to store log lines keyed by a trace id. type LogEntries struct { state protoimpl.MessageState @@ -468,7 +331,7 @@ type LogEntries struct { func (x *LogEntries) Reset() { *x = LogEntries{} if protoimpl.UnsafeEnabled { - mi := &file_foyle_logs_traces_proto_msgTypes[6] + mi := &file_foyle_logs_traces_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -481,7 +344,7 @@ func (x *LogEntries) String() string { func (*LogEntries) ProtoMessage() {} func (x *LogEntries) ProtoReflect() protoreflect.Message { - mi := &file_foyle_logs_traces_proto_msgTypes[6] + mi := &file_foyle_logs_traces_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -494,7 +357,7 @@ func (x *LogEntries) ProtoReflect() protoreflect.Message { // Deprecated: Use LogEntries.ProtoReflect.Descriptor instead. func (*LogEntries) Descriptor() ([]byte, []int) { - return file_foyle_logs_traces_proto_rawDescGZIP(), []int{6} + return file_foyle_logs_traces_proto_rawDescGZIP(), []int{4} } func (x *LogEntries) GetLines() []string { @@ -522,7 +385,7 @@ type GetTraceRequest struct { func (x *GetTraceRequest) Reset() { *x = GetTraceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_foyle_logs_traces_proto_msgTypes[7] + mi := &file_foyle_logs_traces_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -535,7 +398,7 @@ func (x *GetTraceRequest) String() string { func (*GetTraceRequest) ProtoMessage() {} func (x *GetTraceRequest) ProtoReflect() protoreflect.Message { - mi := &file_foyle_logs_traces_proto_msgTypes[7] + mi := &file_foyle_logs_traces_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -548,7 +411,7 @@ func (x *GetTraceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTraceRequest.ProtoReflect.Descriptor instead. func (*GetTraceRequest) Descriptor() ([]byte, []int) { - return file_foyle_logs_traces_proto_rawDescGZIP(), []int{7} + return file_foyle_logs_traces_proto_rawDescGZIP(), []int{5} } func (x *GetTraceRequest) GetId() string { @@ -569,7 +432,7 @@ type GetTraceResponse struct { func (x *GetTraceResponse) Reset() { *x = GetTraceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_foyle_logs_traces_proto_msgTypes[8] + mi := &file_foyle_logs_traces_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -582,7 +445,7 @@ func (x *GetTraceResponse) String() string { func (*GetTraceResponse) ProtoMessage() {} func (x *GetTraceResponse) ProtoReflect() protoreflect.Message { - mi := &file_foyle_logs_traces_proto_msgTypes[8] + mi := &file_foyle_logs_traces_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -595,7 +458,7 @@ func (x *GetTraceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTraceResponse.ProtoReflect.Descriptor instead. func (*GetTraceResponse) Descriptor() ([]byte, []int) { - return file_foyle_logs_traces_proto_rawDescGZIP(), []int{8} + return file_foyle_logs_traces_proto_rawDescGZIP(), []int{6} } func (x *GetTraceResponse) GetTrace() *Trace { @@ -616,7 +479,7 @@ type GetBlockLogRequest struct { func (x *GetBlockLogRequest) Reset() { *x = GetBlockLogRequest{} if protoimpl.UnsafeEnabled { - mi := &file_foyle_logs_traces_proto_msgTypes[9] + mi := &file_foyle_logs_traces_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -629,7 +492,7 @@ func (x *GetBlockLogRequest) String() string { func (*GetBlockLogRequest) ProtoMessage() {} func (x *GetBlockLogRequest) ProtoReflect() protoreflect.Message { - mi := &file_foyle_logs_traces_proto_msgTypes[9] + mi := &file_foyle_logs_traces_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -642,7 +505,7 @@ func (x *GetBlockLogRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetBlockLogRequest.ProtoReflect.Descriptor instead. func (*GetBlockLogRequest) Descriptor() ([]byte, []int) { - return file_foyle_logs_traces_proto_rawDescGZIP(), []int{9} + return file_foyle_logs_traces_proto_rawDescGZIP(), []int{7} } func (x *GetBlockLogRequest) GetId() string { @@ -663,7 +526,7 @@ type GetBlockLogResponse struct { func (x *GetBlockLogResponse) Reset() { *x = GetBlockLogResponse{} if protoimpl.UnsafeEnabled { - mi := &file_foyle_logs_traces_proto_msgTypes[10] + mi := &file_foyle_logs_traces_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -676,7 +539,7 @@ func (x *GetBlockLogResponse) String() string { func (*GetBlockLogResponse) ProtoMessage() {} func (x *GetBlockLogResponse) ProtoReflect() protoreflect.Message { - mi := &file_foyle_logs_traces_proto_msgTypes[10] + mi := &file_foyle_logs_traces_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -689,7 +552,7 @@ func (x *GetBlockLogResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetBlockLogResponse.ProtoReflect.Descriptor instead. func (*GetBlockLogResponse) Descriptor() ([]byte, []int) { - return file_foyle_logs_traces_proto_rawDescGZIP(), []int{10} + return file_foyle_logs_traces_proto_rawDescGZIP(), []int{8} } func (x *GetBlockLogResponse) GetBlockLog() *BlockLog { @@ -714,7 +577,7 @@ type GetLLMLogsRequest struct { func (x *GetLLMLogsRequest) Reset() { *x = GetLLMLogsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_foyle_logs_traces_proto_msgTypes[11] + mi := &file_foyle_logs_traces_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -727,7 +590,7 @@ func (x *GetLLMLogsRequest) String() string { func (*GetLLMLogsRequest) ProtoMessage() {} func (x *GetLLMLogsRequest) ProtoReflect() protoreflect.Message { - mi := &file_foyle_logs_traces_proto_msgTypes[11] + mi := &file_foyle_logs_traces_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -740,7 +603,7 @@ func (x *GetLLMLogsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetLLMLogsRequest.ProtoReflect.Descriptor instead. func (*GetLLMLogsRequest) Descriptor() ([]byte, []int) { - return file_foyle_logs_traces_proto_rawDescGZIP(), []int{11} + return file_foyle_logs_traces_proto_rawDescGZIP(), []int{9} } func (x *GetLLMLogsRequest) GetTraceId() string { @@ -771,7 +634,7 @@ type GetLLMLogsResponse struct { func (x *GetLLMLogsResponse) Reset() { *x = GetLLMLogsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_foyle_logs_traces_proto_msgTypes[12] + mi := &file_foyle_logs_traces_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -784,7 +647,7 @@ func (x *GetLLMLogsResponse) String() string { func (*GetLLMLogsResponse) ProtoMessage() {} func (x *GetLLMLogsResponse) ProtoReflect() protoreflect.Message { - mi := &file_foyle_logs_traces_proto_msgTypes[12] + mi := &file_foyle_logs_traces_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -797,7 +660,7 @@ func (x *GetLLMLogsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetLLMLogsResponse.ProtoReflect.Descriptor instead. func (*GetLLMLogsResponse) Descriptor() ([]byte, []int) { - return file_foyle_logs_traces_proto_rawDescGZIP(), []int{12} + return file_foyle_logs_traces_proto_rawDescGZIP(), []int{10} } func (x *GetLLMLogsResponse) GetRequestHtml() string { @@ -819,127 +682,114 @@ var File_foyle_logs_traces_proto protoreflect.FileDescriptor var file_foyle_logs_traces_proto_rawDesc = []byte{ 0x0a, 0x17, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x2f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a, 0x66, 0x6f, 0x79, 0x6c, 0x65, - 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x1a, 0x1a, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x61, - 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x1a, 0x1c, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, - 0x31, 0x2f, 0x74, 0x72, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, - 0x17, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x2f, 0x62, 0x6c, 0x6f, 0x63, - 0x6b, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x72, 0x75, 0x6e, 0x6d, 0x65, 0x2f, - 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x2f, 0x76, 0x31, 0x2f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xf6, 0x02, 0x0a, 0x05, 0x54, 0x72, 0x61, 0x63, 0x65, 0x12, - 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, - 0x39, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, - 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x35, 0x0a, 0x08, 0x65, 0x6e, - 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, - 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x07, 0x65, 0x6e, 0x64, 0x54, 0x69, 0x6d, - 0x65, 0x12, 0x37, 0x0a, 0x08, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, - 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, 0x00, - 0x52, 0x08, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x12, 0x34, 0x0a, 0x07, 0x65, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x66, 0x6f, - 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, - 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, 0x00, 0x52, 0x07, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, - 0x12, 0x2f, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x52, 0x75, - 0x6e, 0x4d, 0x65, 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, 0x00, 0x52, 0x05, 0x72, 0x75, 0x6e, 0x4d, - 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x65, 0x76, 0x61, 0x6c, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x26, - 0x0a, 0x05, 0x73, 0x70, 0x61, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, - 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x53, 0x70, 0x61, 0x6e, 0x52, - 0x05, 0x73, 0x70, 0x61, 0x6e, 0x73, 0x42, 0x06, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x47, - 0x0a, 0x04, 0x53, 0x70, 0x61, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x27, 0x0a, 0x03, 0x72, 0x61, 0x67, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, - 0x2e, 0x52, 0x41, 0x47, 0x53, 0x70, 0x61, 0x6e, 0x48, 0x00, 0x52, 0x03, 0x72, 0x61, 0x67, 0x42, - 0x06, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x45, 0x0a, 0x07, 0x52, 0x41, 0x47, 0x53, 0x70, - 0x61, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x24, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x52, 0x41, 0x47, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x6a, - 0x0a, 0x0d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x72, 0x61, 0x63, 0x65, 0x12, - 0x2a, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x10, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x52, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x08, 0x72, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, - 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x52, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x67, 0x0a, 0x0c, 0x45, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x65, 0x54, 0x72, 0x61, 0x63, 0x65, 0x12, 0x29, 0x0a, 0x07, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x45, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x07, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x85, 0x01, 0x0a, 0x0a, 0x52, 0x75, 0x6e, 0x4d, 0x65, 0x54, 0x72, 0x61, - 0x63, 0x65, 0x12, 0x39, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x72, 0x75, 0x6e, 0x6d, 0x65, 0x2e, 0x72, 0x75, 0x6e, 0x6e, - 0x65, 0x72, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x52, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3c, 0x0a, - 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x20, 0x2e, 0x72, 0x75, 0x6e, 0x6d, 0x65, 0x2e, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x2e, 0x76, - 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x52, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4d, 0x0a, 0x0a, 0x4c, - 0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6e, - 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x12, - 0x29, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x21, 0x0a, 0x0f, 0x47, 0x65, - 0x74, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, - 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x3b, 0x0a, - 0x10, 0x47, 0x65, 0x74, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x27, 0x0a, 0x05, 0x74, 0x72, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x11, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x54, 0x72, - 0x61, 0x63, 0x65, 0x52, 0x05, 0x74, 0x72, 0x61, 0x63, 0x65, 0x22, 0x24, 0x0a, 0x12, 0x47, 0x65, - 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x1a, 0x15, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x6c, 0x6f, 0x67, + 0x73, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1a, 0x66, 0x6f, + 0x79, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x61, 0x67, 0x65, + 0x6e, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, + 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x74, 0x72, 0x61, 0x69, 0x6e, 0x65, 0x72, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x17, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x6c, 0x6f, + 0x67, 0x73, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, + 0x1c, 0x72, 0x75, 0x6e, 0x6d, 0x65, 0x2f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x2f, 0x76, 0x31, + 0x2f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, + 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x72, 0x75, + 0x6e, 0x6d, 0x65, 0x2f, 0x70, 0x61, 0x72, 0x73, 0x65, 0x72, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x61, + 0x72, 0x73, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x9b, 0x02, 0x0a, 0x05, 0x54, + 0x72, 0x61, 0x63, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x02, 0x69, 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, + 0x35, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x07, 0x65, + 0x6e, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x08, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, + 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, + 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x72, + 0x61, 0x63, 0x65, 0x48, 0x00, 0x52, 0x08, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x12, + 0x1b, 0x0a, 0x09, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x08, 0x65, 0x76, 0x61, 0x6c, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x26, 0x0a, 0x05, + 0x73, 0x70, 0x61, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x66, 0x6f, + 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x53, 0x70, 0x61, 0x6e, 0x52, 0x05, 0x73, + 0x70, 0x61, 0x6e, 0x73, 0x42, 0x06, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x4a, 0x04, 0x08, 0x05, + 0x10, 0x06, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, 0x22, 0x47, 0x0a, 0x04, 0x53, 0x70, 0x61, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, - 0x22, 0x48, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, - 0x5f, 0x6c, 0x6f, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6f, 0x79, - 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, - 0x52, 0x08, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, 0x22, 0x49, 0x0a, 0x11, 0x47, 0x65, - 0x74, 0x4c, 0x4c, 0x4d, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x19, 0x0a, 0x08, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x74, 0x72, 0x61, 0x63, 0x65, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x6c, 0x6f, - 0x67, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6c, 0x6f, - 0x67, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x5c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4c, 0x4c, 0x4d, 0x4c, - 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x68, 0x74, 0x6d, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x74, 0x6d, 0x6c, 0x12, 0x23, - 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x68, 0x74, 0x6d, 0x6c, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, - 0x74, 0x6d, 0x6c, 0x32, 0xf7, 0x01, 0x0a, 0x0b, 0x4c, 0x6f, 0x67, 0x73, 0x53, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x12, 0x47, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x54, 0x72, 0x61, 0x63, 0x65, 0x12, - 0x1b, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x47, 0x65, 0x74, - 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x66, - 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x72, 0x61, - 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x0b, - 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, 0x12, 0x1e, 0x2e, 0x66, 0x6f, - 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, - 0x6b, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x66, 0x6f, - 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, - 0x6b, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4d, - 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4c, 0x4c, 0x4d, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x1d, 0x2e, 0x66, - 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x4c, 0x4d, - 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x66, 0x6f, - 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x4c, 0x4d, 0x4c, - 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x9a, 0x01, - 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, - 0x42, 0x0b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, - 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6a, 0x6c, 0x65, 0x77, - 0x69, 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x2f, 0x67, - 0x6f, 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x3b, 0x6c, 0x6f, 0x67, - 0x73, 0x70, 0x62, 0xa2, 0x02, 0x03, 0x46, 0x4c, 0x58, 0xaa, 0x02, 0x0a, 0x46, 0x6f, 0x79, 0x6c, - 0x65, 0x2e, 0x4c, 0x6f, 0x67, 0x73, 0xca, 0x02, 0x0a, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x5c, 0x4c, - 0x6f, 0x67, 0x73, 0xe2, 0x02, 0x16, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x5c, 0x4c, 0x6f, 0x67, 0x73, - 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x46, - 0x6f, 0x79, 0x6c, 0x65, 0x3a, 0x3a, 0x4c, 0x6f, 0x67, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x12, 0x27, 0x0a, 0x03, 0x72, 0x61, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, + 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x52, 0x41, 0x47, 0x53, 0x70, + 0x61, 0x6e, 0x48, 0x00, 0x52, 0x03, 0x72, 0x61, 0x67, 0x42, 0x06, 0x0a, 0x04, 0x64, 0x61, 0x74, + 0x61, 0x22, 0x45, 0x0a, 0x07, 0x52, 0x41, 0x47, 0x53, 0x70, 0x61, 0x6e, 0x12, 0x14, 0x0a, 0x05, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x12, 0x24, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x52, 0x41, 0x47, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, + 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x6a, 0x0a, 0x0d, 0x47, 0x65, 0x6e, 0x65, + 0x72, 0x61, 0x74, 0x65, 0x54, 0x72, 0x61, 0x63, 0x65, 0x12, 0x2a, 0x0a, 0x07, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x47, 0x65, 0x6e, + 0x65, 0x72, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x07, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x08, 0x72, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x4d, 0x0a, 0x0a, 0x4c, 0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x69, + 0x65, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x05, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x22, 0x21, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x3b, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x54, 0x72, 0x61, + 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x27, 0x0a, 0x05, 0x74, 0x72, + 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x66, 0x6f, 0x79, 0x6c, + 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x05, 0x74, 0x72, + 0x61, 0x63, 0x65, 0x22, 0x24, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, + 0x6f, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0x48, 0x0a, 0x13, 0x47, 0x65, 0x74, + 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x31, 0x0a, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6c, 0x6f, 0x67, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, + 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, 0x52, 0x08, 0x62, 0x6c, 0x6f, 0x63, 0x6b, + 0x4c, 0x6f, 0x67, 0x22, 0x49, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x4c, 0x4c, 0x4d, 0x4c, 0x6f, 0x67, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x74, 0x72, 0x61, 0x63, + 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x72, 0x61, 0x63, + 0x65, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x6c, 0x6f, 0x67, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6c, 0x6f, 0x67, 0x46, 0x69, 0x6c, 0x65, 0x22, 0x5c, + 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4c, 0x4c, 0x4d, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x68, 0x74, 0x6d, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x74, 0x6d, 0x6c, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x5f, 0x68, 0x74, 0x6d, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, + 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x74, 0x6d, 0x6c, 0x32, 0xc8, 0x02, 0x0a, + 0x0b, 0x4c, 0x6f, 0x67, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x47, 0x0a, 0x08, + 0x47, 0x65, 0x74, 0x54, 0x72, 0x61, 0x63, 0x65, 0x12, 0x1b, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, + 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, + 0x67, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, + 0x6b, 0x4c, 0x6f, 0x67, 0x12, 0x1e, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, + 0x73, 0x2e, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, + 0x73, 0x2e, 0x47, 0x65, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x4c, 0x6f, 0x67, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4d, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4c, 0x4c, + 0x4d, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x1d, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, + 0x67, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x4c, 0x4d, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, + 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x4c, 0x4d, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x12, 0x20, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, 0x47, 0x65, + 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x2e, + 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x9a, 0x01, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e, + 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x6c, 0x6f, 0x67, 0x73, 0x42, 0x0b, 0x54, 0x72, 0x61, 0x63, + 0x65, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, + 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6a, 0x6c, 0x65, 0x77, 0x69, 0x2f, 0x66, 0x6f, 0x79, 0x6c, + 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x66, 0x6f, 0x79, 0x6c, + 0x65, 0x2f, 0x6c, 0x6f, 0x67, 0x73, 0x3b, 0x6c, 0x6f, 0x67, 0x73, 0x70, 0x62, 0xa2, 0x02, 0x03, + 0x46, 0x4c, 0x58, 0xaa, 0x02, 0x0a, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x2e, 0x4c, 0x6f, 0x67, 0x73, + 0xca, 0x02, 0x0a, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x5c, 0x4c, 0x6f, 0x67, 0x73, 0xe2, 0x02, 0x16, + 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x5c, 0x4c, 0x6f, 0x67, 0x73, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x46, 0x6f, 0x79, 0x6c, 0x65, 0x3a, 0x3a, + 0x4c, 0x6f, 0x67, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -954,59 +804,51 @@ func file_foyle_logs_traces_proto_rawDescGZIP() []byte { return file_foyle_logs_traces_proto_rawDescData } -var file_foyle_logs_traces_proto_msgTypes = make([]protoimpl.MessageInfo, 13) +var file_foyle_logs_traces_proto_msgTypes = make([]protoimpl.MessageInfo, 11) var file_foyle_logs_traces_proto_goTypes = []interface{}{ (*Trace)(nil), // 0: foyle.logs.Trace (*Span)(nil), // 1: foyle.logs.Span (*RAGSpan)(nil), // 2: foyle.logs.RAGSpan (*GenerateTrace)(nil), // 3: foyle.logs.GenerateTrace - (*ExecuteTrace)(nil), // 4: foyle.logs.ExecuteTrace - (*RunMeTrace)(nil), // 5: foyle.logs.RunMeTrace - (*LogEntries)(nil), // 6: foyle.logs.LogEntries - (*GetTraceRequest)(nil), // 7: foyle.logs.GetTraceRequest - (*GetTraceResponse)(nil), // 8: foyle.logs.GetTraceResponse - (*GetBlockLogRequest)(nil), // 9: foyle.logs.GetBlockLogRequest - (*GetBlockLogResponse)(nil), // 10: foyle.logs.GetBlockLogResponse - (*GetLLMLogsRequest)(nil), // 11: foyle.logs.GetLLMLogsRequest - (*GetLLMLogsResponse)(nil), // 12: foyle.logs.GetLLMLogsResponse - (*timestamppb.Timestamp)(nil), // 13: google.protobuf.Timestamp - (*v1alpha1.RAGResult)(nil), // 14: RAGResult - (*v1alpha1.GenerateRequest)(nil), // 15: GenerateRequest - (*v1alpha1.GenerateResponse)(nil), // 16: GenerateResponse - (*v1alpha1.ExecuteRequest)(nil), // 17: ExecuteRequest - (*v1alpha1.ExecuteResponse)(nil), // 18: ExecuteResponse - (*v1.ExecuteRequest)(nil), // 19: runme.runner.v1.ExecuteRequest - (*v1.ExecuteResponse)(nil), // 20: runme.runner.v1.ExecuteResponse - (*BlockLog)(nil), // 21: foyle.logs.BlockLog + (*LogEntries)(nil), // 4: foyle.logs.LogEntries + (*GetTraceRequest)(nil), // 5: foyle.logs.GetTraceRequest + (*GetTraceResponse)(nil), // 6: foyle.logs.GetTraceResponse + (*GetBlockLogRequest)(nil), // 7: foyle.logs.GetBlockLogRequest + (*GetBlockLogResponse)(nil), // 8: foyle.logs.GetBlockLogResponse + (*GetLLMLogsRequest)(nil), // 9: foyle.logs.GetLLMLogsRequest + (*GetLLMLogsResponse)(nil), // 10: foyle.logs.GetLLMLogsResponse + (*timestamppb.Timestamp)(nil), // 11: google.protobuf.Timestamp + (*v1alpha1.RAGResult)(nil), // 12: RAGResult + (*v1alpha1.GenerateRequest)(nil), // 13: GenerateRequest + (*v1alpha1.GenerateResponse)(nil), // 14: GenerateResponse + (*BlockLog)(nil), // 15: foyle.logs.BlockLog + (*GetLogsStatusRequest)(nil), // 16: foyle.logs.GetLogsStatusRequest + (*GetLogsStatusResponse)(nil), // 17: foyle.logs.GetLogsStatusResponse } var file_foyle_logs_traces_proto_depIdxs = []int32{ - 13, // 0: foyle.logs.Trace.start_time:type_name -> google.protobuf.Timestamp - 13, // 1: foyle.logs.Trace.end_time:type_name -> google.protobuf.Timestamp + 11, // 0: foyle.logs.Trace.start_time:type_name -> google.protobuf.Timestamp + 11, // 1: foyle.logs.Trace.end_time:type_name -> google.protobuf.Timestamp 3, // 2: foyle.logs.Trace.generate:type_name -> foyle.logs.GenerateTrace - 4, // 3: foyle.logs.Trace.execute:type_name -> foyle.logs.ExecuteTrace - 5, // 4: foyle.logs.Trace.run_me:type_name -> foyle.logs.RunMeTrace - 1, // 5: foyle.logs.Trace.spans:type_name -> foyle.logs.Span - 2, // 6: foyle.logs.Span.rag:type_name -> foyle.logs.RAGSpan - 14, // 7: foyle.logs.RAGSpan.results:type_name -> RAGResult - 15, // 8: foyle.logs.GenerateTrace.request:type_name -> GenerateRequest - 16, // 9: foyle.logs.GenerateTrace.response:type_name -> GenerateResponse - 17, // 10: foyle.logs.ExecuteTrace.request:type_name -> ExecuteRequest - 18, // 11: foyle.logs.ExecuteTrace.response:type_name -> ExecuteResponse - 19, // 12: foyle.logs.RunMeTrace.request:type_name -> runme.runner.v1.ExecuteRequest - 20, // 13: foyle.logs.RunMeTrace.response:type_name -> runme.runner.v1.ExecuteResponse - 0, // 14: foyle.logs.GetTraceResponse.trace:type_name -> foyle.logs.Trace - 21, // 15: foyle.logs.GetBlockLogResponse.block_log:type_name -> foyle.logs.BlockLog - 7, // 16: foyle.logs.LogsService.GetTrace:input_type -> foyle.logs.GetTraceRequest - 9, // 17: foyle.logs.LogsService.GetBlockLog:input_type -> foyle.logs.GetBlockLogRequest - 11, // 18: foyle.logs.LogsService.GetLLMLogs:input_type -> foyle.logs.GetLLMLogsRequest - 8, // 19: foyle.logs.LogsService.GetTrace:output_type -> foyle.logs.GetTraceResponse - 10, // 20: foyle.logs.LogsService.GetBlockLog:output_type -> foyle.logs.GetBlockLogResponse - 12, // 21: foyle.logs.LogsService.GetLLMLogs:output_type -> foyle.logs.GetLLMLogsResponse - 19, // [19:22] is the sub-list for method output_type - 16, // [16:19] is the sub-list for method input_type - 16, // [16:16] is the sub-list for extension type_name - 16, // [16:16] is the sub-list for extension extendee - 0, // [0:16] is the sub-list for field type_name + 1, // 3: foyle.logs.Trace.spans:type_name -> foyle.logs.Span + 2, // 4: foyle.logs.Span.rag:type_name -> foyle.logs.RAGSpan + 12, // 5: foyle.logs.RAGSpan.results:type_name -> RAGResult + 13, // 6: foyle.logs.GenerateTrace.request:type_name -> GenerateRequest + 14, // 7: foyle.logs.GenerateTrace.response:type_name -> GenerateResponse + 0, // 8: foyle.logs.GetTraceResponse.trace:type_name -> foyle.logs.Trace + 15, // 9: foyle.logs.GetBlockLogResponse.block_log:type_name -> foyle.logs.BlockLog + 5, // 10: foyle.logs.LogsService.GetTrace:input_type -> foyle.logs.GetTraceRequest + 7, // 11: foyle.logs.LogsService.GetBlockLog:input_type -> foyle.logs.GetBlockLogRequest + 9, // 12: foyle.logs.LogsService.GetLLMLogs:input_type -> foyle.logs.GetLLMLogsRequest + 16, // 13: foyle.logs.LogsService.Status:input_type -> foyle.logs.GetLogsStatusRequest + 6, // 14: foyle.logs.LogsService.GetTrace:output_type -> foyle.logs.GetTraceResponse + 8, // 15: foyle.logs.LogsService.GetBlockLog:output_type -> foyle.logs.GetBlockLogResponse + 10, // 16: foyle.logs.LogsService.GetLLMLogs:output_type -> foyle.logs.GetLLMLogsResponse + 17, // 17: foyle.logs.LogsService.Status:output_type -> foyle.logs.GetLogsStatusResponse + 14, // [14:18] is the sub-list for method output_type + 10, // [10:14] is the sub-list for method input_type + 10, // [10:10] is the sub-list for extension type_name + 10, // [10:10] is the sub-list for extension extendee + 0, // [0:10] is the sub-list for field type_name } func init() { file_foyle_logs_traces_proto_init() } @@ -1014,6 +856,7 @@ func file_foyle_logs_traces_proto_init() { if File_foyle_logs_traces_proto != nil { return } + file_foyle_logs_logs_proto_init() file_foyle_logs_blocks_proto_init() if !protoimpl.UnsafeEnabled { file_foyle_logs_traces_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { @@ -1065,30 +908,6 @@ func file_foyle_logs_traces_proto_init() { } } file_foyle_logs_traces_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecuteTrace); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_foyle_logs_traces_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RunMeTrace); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_foyle_logs_traces_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*LogEntries); i { case 0: return &v.state @@ -1100,7 +919,7 @@ func file_foyle_logs_traces_proto_init() { return nil } } - file_foyle_logs_traces_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + file_foyle_logs_traces_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetTraceRequest); i { case 0: return &v.state @@ -1112,7 +931,7 @@ func file_foyle_logs_traces_proto_init() { return nil } } - file_foyle_logs_traces_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + file_foyle_logs_traces_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetTraceResponse); i { case 0: return &v.state @@ -1124,7 +943,7 @@ func file_foyle_logs_traces_proto_init() { return nil } } - file_foyle_logs_traces_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + file_foyle_logs_traces_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetBlockLogRequest); i { case 0: return &v.state @@ -1136,7 +955,7 @@ func file_foyle_logs_traces_proto_init() { return nil } } - file_foyle_logs_traces_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + file_foyle_logs_traces_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetBlockLogResponse); i { case 0: return &v.state @@ -1148,7 +967,7 @@ func file_foyle_logs_traces_proto_init() { return nil } } - file_foyle_logs_traces_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + file_foyle_logs_traces_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetLLMLogsRequest); i { case 0: return &v.state @@ -1160,7 +979,7 @@ func file_foyle_logs_traces_proto_init() { return nil } } - file_foyle_logs_traces_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + file_foyle_logs_traces_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetLLMLogsResponse); i { case 0: return &v.state @@ -1175,8 +994,6 @@ func file_foyle_logs_traces_proto_init() { } file_foyle_logs_traces_proto_msgTypes[0].OneofWrappers = []interface{}{ (*Trace_Generate)(nil), - (*Trace_Execute)(nil), - (*Trace_RunMe)(nil), } file_foyle_logs_traces_proto_msgTypes[1].OneofWrappers = []interface{}{ (*Span_Rag)(nil), @@ -1187,7 +1004,7 @@ func file_foyle_logs_traces_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_foyle_logs_traces_proto_rawDesc, NumEnums: 0, - NumMessages: 13, + NumMessages: 11, NumExtensions: 0, NumServices: 1, }, diff --git a/protos/go/foyle/logs/traces.zap.go b/protos/go/foyle/logs/traces.zap.go index 75c53a02..3931b04d 100644 --- a/protos/go/foyle/logs/traces.zap.go +++ b/protos/go/foyle/logs/traces.zap.go @@ -7,10 +7,11 @@ import ( fmt "fmt" math "math" proto "github.com/golang/protobuf/proto" - _ "github.com/jlewi/foyle/protos/go/foyle/v1alpha1" - _ "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/runner/v1" _ "google.golang.org/protobuf/types/known/structpb" _ "google.golang.org/protobuf/types/known/timestamppb" + _ "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/parser/v1" + _ "github.com/jlewi/foyle/protos/go/foyle/v1alpha1" + _ "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/runner/v1" go_uber_org_zap_zapcore "go.uber.org/zap/zapcore" github_com_golang_protobuf_ptypes "github.com/golang/protobuf/ptypes" ) @@ -52,28 +53,6 @@ func (m *Trace) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) erro } } - keyName = "execute" // field execute = 5 - if ov, ok := m.GetData().(*Trace_Execute); ok { - _ = ov - if ov.Execute != nil { - var vv interface{} = ov.Execute - if marshaler, ok := vv.(go_uber_org_zap_zapcore.ObjectMarshaler); ok { - enc.AddObject(keyName, marshaler) - } - } - } - - keyName = "run_me" // field run_me = 7 - if ov, ok := m.GetData().(*Trace_RunMe); ok { - _ = ov - if ov.RunMe != nil { - var vv interface{} = ov.RunMe - if marshaler, ok := vv.(go_uber_org_zap_zapcore.ObjectMarshaler); ok { - enc.AddObject(keyName, marshaler) - } - } - } - keyName = "eval_mode" // field eval_mode = 6 enc.AddBool(keyName, m.EvalMode) @@ -174,60 +153,6 @@ func (m *GenerateTrace) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncod return nil } -func (m *ExecuteTrace) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) error { - var keyName string - _ = keyName - - if m == nil { - return nil - } - - keyName = "request" // field request = 1 - if m.Request != nil { - var vv interface{} = m.Request - if marshaler, ok := vv.(go_uber_org_zap_zapcore.ObjectMarshaler); ok { - enc.AddObject(keyName, marshaler) - } - } - - keyName = "response" // field response = 2 - if m.Response != nil { - var vv interface{} = m.Response - if marshaler, ok := vv.(go_uber_org_zap_zapcore.ObjectMarshaler); ok { - enc.AddObject(keyName, marshaler) - } - } - - return nil -} - -func (m *RunMeTrace) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) error { - var keyName string - _ = keyName - - if m == nil { - return nil - } - - keyName = "request" // field request = 1 - if m.Request != nil { - var vv interface{} = m.Request - if marshaler, ok := vv.(go_uber_org_zap_zapcore.ObjectMarshaler); ok { - enc.AddObject(keyName, marshaler) - } - } - - keyName = "response" // field response = 2 - if m.Response != nil { - var vv interface{} = m.Response - if marshaler, ok := vv.(go_uber_org_zap_zapcore.ObjectMarshaler); ok { - enc.AddObject(keyName, marshaler) - } - } - - return nil -} - func (m *LogEntries) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) error { var keyName string _ = keyName diff --git a/protos/go/foyle/logs/traces_grpc.pb.go b/protos/go/foyle/logs/traces_grpc.pb.go index c62d9072..24f64cab 100644 --- a/protos/go/foyle/logs/traces_grpc.pb.go +++ b/protos/go/foyle/logs/traces_grpc.pb.go @@ -28,6 +28,7 @@ type LogsServiceClient interface { // These will include the rendered prompt and response. Unlike GetTraceRequest this has the // actual prompt and response of the LLM. GetLLMLogs(ctx context.Context, in *GetLLMLogsRequest, opts ...grpc.CallOption) (*GetLLMLogsResponse, error) + Status(ctx context.Context, in *GetLogsStatusRequest, opts ...grpc.CallOption) (*GetLogsStatusResponse, error) } type logsServiceClient struct { @@ -65,6 +66,15 @@ func (c *logsServiceClient) GetLLMLogs(ctx context.Context, in *GetLLMLogsReques return out, nil } +func (c *logsServiceClient) Status(ctx context.Context, in *GetLogsStatusRequest, opts ...grpc.CallOption) (*GetLogsStatusResponse, error) { + out := new(GetLogsStatusResponse) + err := c.cc.Invoke(ctx, "/foyle.logs.LogsService/Status", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // LogsServiceServer is the server API for LogsService service. // All implementations must embed UnimplementedLogsServiceServer // for forward compatibility @@ -75,6 +85,7 @@ type LogsServiceServer interface { // These will include the rendered prompt and response. Unlike GetTraceRequest this has the // actual prompt and response of the LLM. GetLLMLogs(context.Context, *GetLLMLogsRequest) (*GetLLMLogsResponse, error) + Status(context.Context, *GetLogsStatusRequest) (*GetLogsStatusResponse, error) mustEmbedUnimplementedLogsServiceServer() } @@ -91,6 +102,9 @@ func (UnimplementedLogsServiceServer) GetBlockLog(context.Context, *GetBlockLogR func (UnimplementedLogsServiceServer) GetLLMLogs(context.Context, *GetLLMLogsRequest) (*GetLLMLogsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetLLMLogs not implemented") } +func (UnimplementedLogsServiceServer) Status(context.Context, *GetLogsStatusRequest) (*GetLogsStatusResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method Status not implemented") +} func (UnimplementedLogsServiceServer) mustEmbedUnimplementedLogsServiceServer() {} // UnsafeLogsServiceServer may be embedded to opt out of forward compatibility for this service. @@ -158,6 +172,24 @@ func _LogsService_GetLLMLogs_Handler(srv interface{}, ctx context.Context, dec f return interceptor(ctx, in, info, handler) } +func _LogsService_Status_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetLogsStatusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(LogsServiceServer).Status(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/foyle.logs.LogsService/Status", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(LogsServiceServer).Status(ctx, req.(*GetLogsStatusRequest)) + } + return interceptor(ctx, in, info, handler) +} + // LogsService_ServiceDesc is the grpc.ServiceDesc for LogsService service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -177,6 +209,10 @@ var LogsService_ServiceDesc = grpc.ServiceDesc{ MethodName: "GetLLMLogs", Handler: _LogsService_GetLLMLogs_Handler, }, + { + MethodName: "Status", + Handler: _LogsService_Status_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "foyle/logs/traces.proto", diff --git a/protos/go/foyle/v1alpha1/agent.pb.go b/protos/go/foyle/v1alpha1/agent.pb.go index dcf7dc63..e298b828 100644 --- a/protos/go/foyle/v1alpha1/agent.pb.go +++ b/protos/go/foyle/v1alpha1/agent.pb.go @@ -999,6 +999,8 @@ type LogEvent struct { // The position of the cell with selected_id. This will usually be the position in the full context as specified // by selected_id. SelectedIndex int32 `protobuf:"varint,5,opt,name=selected_index,json=selectedIndex,proto3" json:"selected_index,omitempty"` + // A unique ID for the event + EventId string `protobuf:"bytes,6,opt,name=event_id,json=eventId,proto3" json:"event_id,omitempty"` } func (x *LogEvent) Reset() { @@ -1068,6 +1070,13 @@ func (x *LogEvent) GetSelectedIndex() int32 { return 0 } +func (x *LogEvent) GetEventId() string { + if x != nil { + return x.EventId + } + return "" +} + type LogEventsResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1190,7 +1199,7 @@ var file_foyle_v1alpha1_agent_proto_rawDesc = []byte{ 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x22, 0x35, 0x0a, 0x10, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x09, 0x2e, 0x4c, 0x6f, 0x67, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xc1, 0x01, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xdc, 0x01, 0x0a, 0x08, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0d, 0x2e, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x2b, 0x0a, @@ -1203,50 +1212,52 @@ var file_foyle_v1alpha1_agent_proto_rawDesc = []byte{ 0x09, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x65, 0x64, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x22, 0x13, 0x0a, 0x11, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x32, 0x0a, 0x0f, 0x41, 0x49, 0x53, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, - 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x4f, 0x4b, 0x10, 0x01, 0x12, 0x0a, - 0x0a, 0x06, 0x4e, 0x4f, 0x54, 0x5f, 0x4f, 0x4b, 0x10, 0x02, 0x2a, 0x6e, 0x0a, 0x0c, 0x4c, 0x6f, - 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x55, 0x4e, - 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x10, 0x00, 0x12, 0x0b, 0x0a, - 0x07, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x45, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x41, 0x43, - 0x43, 0x45, 0x50, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x45, 0x4a, 0x45, - 0x43, 0x54, 0x45, 0x44, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x53, 0x45, 0x53, 0x53, 0x49, 0x4f, - 0x4e, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x04, 0x12, 0x0f, 0x0a, 0x0b, 0x53, 0x45, 0x53, - 0x53, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x4e, 0x44, 0x10, 0x05, 0x32, 0x44, 0x0a, 0x0f, 0x47, 0x65, - 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x31, 0x0a, - 0x08, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x12, 0x10, 0x2e, 0x47, 0x65, 0x6e, 0x65, - 0x72, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x47, 0x65, + 0x78, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x13, 0x0a, 0x11, + 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x2a, 0x32, 0x0a, 0x0f, 0x41, 0x49, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, + 0x00, 0x12, 0x06, 0x0a, 0x02, 0x4f, 0x4b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x4f, 0x54, + 0x5f, 0x4f, 0x4b, 0x10, 0x02, 0x2a, 0x6e, 0x0a, 0x0c, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, + 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x45, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x41, 0x43, 0x43, 0x45, 0x50, 0x54, 0x45, + 0x44, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x45, 0x4a, 0x45, 0x43, 0x54, 0x45, 0x44, 0x10, + 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x53, 0x45, 0x53, 0x53, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, + 0x52, 0x54, 0x10, 0x04, 0x12, 0x0f, 0x0a, 0x0b, 0x53, 0x45, 0x53, 0x53, 0x49, 0x4f, 0x4e, 0x5f, + 0x45, 0x4e, 0x44, 0x10, 0x05, 0x32, 0x44, 0x0a, 0x0f, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, + 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x31, 0x0a, 0x08, 0x47, 0x65, 0x6e, 0x65, + 0x72, 0x61, 0x74, 0x65, 0x12, 0x10, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0x40, 0x0a, 0x0e, 0x45, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x2e, 0x0a, + 0x07, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x12, 0x0f, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xb2, 0x02, + 0x0a, 0x09, 0x41, 0x49, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x47, 0x0a, 0x0e, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x12, 0x16, 0x2e, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x32, 0x40, 0x0a, 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x12, 0x2e, 0x0a, 0x07, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x12, 0x0f, 0x2e, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, - 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x32, 0xb2, 0x02, 0x0a, 0x09, 0x41, 0x49, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x12, 0x47, 0x0a, 0x0e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, - 0x74, 0x65, 0x12, 0x16, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x47, 0x65, 0x6e, 0x65, 0x72, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x40, 0x0a, 0x0d, 0x47, 0x65, 0x6e, - 0x65, 0x72, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x15, 0x2e, 0x47, 0x65, 0x6e, - 0x65, 0x72, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x16, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x37, 0x0a, 0x0a, 0x47, - 0x65, 0x74, 0x45, 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x12, 0x2e, 0x47, 0x65, 0x74, 0x45, - 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, - 0x47, 0x65, 0x74, 0x45, 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x34, 0x0a, 0x09, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, - 0x73, 0x12, 0x11, 0x2e, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x12, 0x2e, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x2b, 0x0a, 0x06, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x12, 0x0e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0f, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x3f, 0x42, 0x0a, 0x41, 0x67, 0x65, 0x6e, 0x74, - 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, - 0x63, 0x6f, 0x6d, 0x2f, 0x6a, 0x6c, 0x65, 0x77, 0x69, 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, - 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x28, 0x01, 0x30, 0x01, 0x12, 0x40, 0x0a, 0x0d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, + 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x15, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, + 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x47, + 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x37, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x45, 0x78, 0x61, + 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x12, 0x2e, 0x47, 0x65, 0x74, 0x45, 0x78, 0x61, 0x6d, 0x70, 0x6c, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x47, 0x65, 0x74, 0x45, 0x78, + 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x34, 0x0a, 0x09, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x11, 0x2e, 0x4c, + 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x12, 0x2e, 0x4c, 0x6f, 0x67, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x2b, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, + 0x0e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x0f, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x42, 0x3f, 0x42, 0x0a, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x50, 0x01, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6a, + 0x6c, 0x65, 0x77, 0x69, 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x66, 0x6f, 0x79, 0x6c, 0x65, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, + 0x68, 0x61, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/protos/go/foyle/v1alpha1/agent.zap.go b/protos/go/foyle/v1alpha1/agent.zap.go index 6f1093da..d3145807 100644 --- a/protos/go/foyle/v1alpha1/agent.zap.go +++ b/protos/go/foyle/v1alpha1/agent.zap.go @@ -7,8 +7,8 @@ import ( fmt "fmt" math "math" proto "github.com/golang/protobuf/proto" - _ "google.golang.org/protobuf/types/known/structpb" _ "github.com/stateful/runme/v3/pkg/api/gen/proto/go/runme/parser/v1" + _ "google.golang.org/protobuf/types/known/structpb" go_uber_org_zap_zapcore "go.uber.org/zap/zapcore" ) @@ -397,6 +397,9 @@ func (m *LogEvent) MarshalLogObject(enc go_uber_org_zap_zapcore.ObjectEncoder) e keyName = "selected_index" // field selected_index = 5 enc.AddInt32(keyName, m.SelectedIndex) + keyName = "event_id" // field event_id = 6 + enc.AddString(keyName, m.EventId) + return nil }