From bead065cf3d47b95fc4b52323bce27ac3c9f870c Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Mon, 24 Jul 2023 13:59:19 -0500 Subject: [PATCH 01/50] benchmark test --- conduit/pipeline/pipeline_bench_test.go | 296 ++++++++++++++++++++++++ 1 file changed, 296 insertions(+) create mode 100644 conduit/pipeline/pipeline_bench_test.go diff --git a/conduit/pipeline/pipeline_bench_test.go b/conduit/pipeline/pipeline_bench_test.go new file mode 100644 index 00000000..93ebee36 --- /dev/null +++ b/conduit/pipeline/pipeline_bench_test.go @@ -0,0 +1,296 @@ +package pipeline + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/algorand/conduit/conduit/data" + "github.com/algorand/conduit/conduit/plugins" + "github.com/algorand/conduit/conduit/plugins/processors" + sdk "github.com/algorand/go-algorand-sdk/v2/types" + "github.com/prometheus/client_golang/prometheus" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +const ( + logLevel = log.ErrorLevel //log.InfoLevel // log.DebugLevel // + retryCount = 3 // math.MaxUint64 +) + +type sleepingImporter struct { + cfg plugins.PluginConfig + genesis sdk.Genesis + finalRound sdk.Round + getBlockSleep time.Duration // when non-0, sleep when GetBlock() even in the case of an error + returnError bool + onCompleteError bool + subsystem string + rndOverride uint64 + rndReqErr error +} + +func (m *sleepingImporter) Init(_ context.Context, _ data.InitProvider, cfg plugins.PluginConfig, _ *log.Logger) error { + m.cfg = cfg + return nil +} + +func (m *sleepingImporter) GetGenesis() (*sdk.Genesis, error) { + return &m.genesis, nil +} + +func (m *sleepingImporter) Close() error { + return nil +} + +func (m *sleepingImporter) Metadata() plugins.Metadata { + return plugins.Metadata{Name: "sleepingImporter"} +} + +func (m *sleepingImporter) GetBlock(rnd uint64) (data.BlockData, error) { + if m.getBlockSleep > 0 { + time.Sleep(m.getBlockSleep) + } + var err error + if m.returnError { + err = fmt.Errorf("importer") + } + return data.BlockData{BlockHeader: sdk.BlockHeader{Round: sdk.Round(rnd)}}, err +} + +func (m *sleepingImporter) OnComplete(input data.BlockData) error { + var err error + if m.onCompleteError { + err = fmt.Errorf("on complete") + } + m.finalRound = sdk.Round(input.BlockHeader.Round) + return err +} + +func (m *sleepingImporter) ProvideMetrics(subsystem string) []prometheus.Collector { + m.subsystem = subsystem + return nil +} + +func (m *sleepingImporter) RoundRequest(_ plugins.PluginConfig) (uint64, error) { + return m.rndOverride, m.rndReqErr +} + +type sleepingProcessor struct { + cfg plugins.PluginConfig + finalRound sdk.Round + processSleep time.Duration // when non-0, sleep when Process() even in the case of an error + returnError bool + onCompleteError bool + rndOverride uint64 + rndReqErr error +} + +func (m *sleepingProcessor) Init(_ context.Context, _ data.InitProvider, cfg plugins.PluginConfig, _ *log.Logger) error { + m.cfg = cfg + return nil +} + +func (m *sleepingProcessor) Close() error { + return nil +} + +func (m *sleepingProcessor) RoundRequest(_ plugins.PluginConfig) (uint64, error) { + return m.rndOverride, m.rndReqErr +} + +func (m *sleepingProcessor) Metadata() plugins.Metadata { + return plugins.Metadata{ + Name: "sleepingProcessor", + } +} + +func (m *sleepingProcessor) Process(input data.BlockData) (data.BlockData, error) { + if m.processSleep > 0 { + time.Sleep(m.processSleep) + } + var err error + if m.returnError { + err = fmt.Errorf("process") + } + return input, err +} + +func (m *sleepingProcessor) OnComplete(input data.BlockData) error { + var err error + if m.onCompleteError { + err = fmt.Errorf("on complete") + } + m.finalRound = sdk.Round(input.BlockHeader.Round) + return err +} + +type sleepingExporter struct { + cfg plugins.PluginConfig + finalRound sdk.Round + receiveSleep time.Duration // when non-0, sleep when Receive() even in the case of an error + returnError bool + onCompleteError bool + rndOverride uint64 + rndReqErr error +} + +func (m *sleepingExporter) Metadata() plugins.Metadata { + return plugins.Metadata{ + Name: "sleepingExporter", + } +} + +func (m *sleepingExporter) RoundRequest(_ plugins.PluginConfig) (uint64, error) { + return m.rndOverride, m.rndReqErr +} + +func (m *sleepingExporter) Init(_ context.Context, _ data.InitProvider, cfg plugins.PluginConfig, _ *log.Logger) error { + m.cfg = cfg + return nil +} + +func (m *sleepingExporter) Close() error { + return nil +} + +func (m *sleepingExporter) Receive(exportData data.BlockData) error { + if m.receiveSleep > 0 { + time.Sleep(m.receiveSleep) + } + var err error + if m.returnError { + err = fmt.Errorf("receive") + } + return err +} + +func (m *sleepingExporter) OnComplete(input data.BlockData) error { + var err error + if m.onCompleteError { + err = fmt.Errorf("on complete") + } + m.finalRound = sdk.Round(input.BlockHeader.Round) + return err +} + +type benchmarkCase struct { + name string + channelBuffSize int + importerSleep time.Duration + processorsSleep []time.Duration + exporterSleep time.Duration +} + +func pipeline5sec(b *testing.B, testCase benchmarkCase) int { + benchmarkDuration := 5 * time.Second + + importer := &sleepingImporter{getBlockSleep: testCase.importerSleep} + processors := make([]processors.Processor, len(testCase.processorsSleep)) + for i, procSleep := range testCase.processorsSleep { + processors[i] = &sleepingProcessor{processSleep: procSleep} + } + exporter := &sleepingExporter{receiveSleep: testCase.exporterSleep} + + ctx, cf := context.WithCancel(context.Background()) + + logger := log.New() + logger.SetLevel(logLevel) + pImpl := pipelineImpl{ + ctx: ctx, + cf: cf, + logger: logger, + initProvider: nil, + importer: importer, + processors: processors, + exporter: exporter, + pipelineMetadata: state{ + NextRound: 0, + GenesisHash: "", + }, + cfg: &data.Config{ + RetryDelay: 0 * time.Second, + RetryCount: retryCount, + ConduitArgs: &data.Args{ + ConduitDataDir: b.TempDir(), + }, + }, + } + + pImpl.registerLifecycleCallbacks() + + // cancel the pipeline after 5 seconds + go func() { + time.Sleep(benchmarkDuration) + cf() + }() + + b.StartTimer() + pImpl.Start() + pImpl.Wait() + assert.NoError(b, pImpl.Error()) + + fRound, err := finalRound(&pImpl) + require.NoError(b, err) + return int(fRound) +} + +func finalRound(pi *pipelineImpl) (sdk.Round, error) { + if mExp, ok := pi.exporter.(*sleepingExporter); ok { + return mExp.finalRound, nil + } + return 0, fmt.Errorf("not a sleepingExporter: %t", pi.exporter) +} + +func BenchmarkPipeline(b *testing.B) { + benchCases := []benchmarkCase{ + { + name: "vanilla 2 procs without sleep", + importerSleep: 0, + processorsSleep: []time.Duration{0, 0}, + exporterSleep: 0, + }, + { + name: "uniform sleep of 10ms", + importerSleep: 10 * time.Millisecond, + processorsSleep: []time.Duration{10 * time.Millisecond, 10 * time.Millisecond}, + exporterSleep: 10 * time.Millisecond, + }, + { + name: "exporter 10ms while others 1ms", + importerSleep: time.Millisecond, + processorsSleep: []time.Duration{time.Millisecond, time.Millisecond}, + exporterSleep: 10 * time.Millisecond, + }, + { + name: "importer 10ms while others 1ms", + importerSleep: 10 * time.Millisecond, + processorsSleep: []time.Duration{time.Millisecond, time.Millisecond}, + exporterSleep: time.Millisecond, + }, + { + name: "first processor 10ms while others 1ms", + importerSleep: time.Millisecond, + processorsSleep: []time.Duration{10 * time.Millisecond, time.Millisecond}, + exporterSleep: time.Millisecond, + }, + } + for _, buffSize := range []int{1} { + for _, bc := range benchCases { + bc.channelBuffSize = buffSize + b.Run(fmt.Sprintf("%s-size-%d", bc.name, bc.channelBuffSize), func(b *testing.B) { + rounds := 0 + for i := 0; i < b.N; i++ { + rounds += pipeline5sec(b, bc) + } + secs := b.Elapsed().Seconds() + rps := float64(rounds) / secs + // fmt.Printf("benchmark warmup results. N: %d, elapsed: %f, rounds/sec: %f\n", b.N, secs, rps) + b.ReportMetric(rps, "rounds/sec") + }) + } + } +} From cf130e20abb0431e504875c7ec0bb7720ac42ae8 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Tue, 25 Jul 2023 19:03:46 -0500 Subject: [PATCH 02/50] basic logic in place --- conduit/pipeline/common.go | 68 ++++++++++++++++++++++++- conduit/pipeline/pipeline_bench_test.go | 57 +++++++++++---------- conduit/pipeline/pipeline_test.go | 35 ++++++------- 3 files changed, 114 insertions(+), 46 deletions(-) diff --git a/conduit/pipeline/common.go b/conduit/pipeline/common.go index 49090b62..62a2818d 100644 --- a/conduit/pipeline/common.go +++ b/conduit/pipeline/common.go @@ -1,6 +1,13 @@ package pipeline -import log "github.com/sirupsen/logrus" +import ( + "errors" + "fmt" + "time" + + "github.com/algorand/conduit/conduit/data" + log "github.com/sirupsen/logrus" +) // HandlePanic function to log panics in a common way func HandlePanic(logger *log.Logger) { @@ -8,3 +15,62 @@ func HandlePanic(logger *log.Logger) { logger.Panicf("conduit pipeline experienced a panic: %v", r) } } + +type empty struct{} + +// Probly don't need this: +// func EmptyOutputter[X any](f func(a X) error) func(X) (empty, error) { +// return func(a X) (empty, error) { +// return empty{}, f(a) +// } +// } + +type pluginInput interface { + uint64 | data.BlockData | string | empty +} + +func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg string) (*Y, time.Duration, error) { + var err error + start := time.Now() + + for i := uint64(0); p.cfg.RetryCount == 0 || i <= p.cfg.RetryCount; i++ { + // the first time through, we don't sleep or mind the done signal + if i > 0 { + select { + case <-p.ctx.Done(): + return nil, time.Since(start), fmt.Errorf("%s: retry number %d/%d with err: %w. Done signal received: %w", msg, i, p.cfg.RetryCount, err, p.ctx.Err()) + default: + time.Sleep(p.cfg.RetryDelay) + } + } + opStart := time.Now() + y, err2 := f(x) + if err2 == nil { + return &y, time.Since(opStart), nil + } + + p.logger.Infof("%s: retry number %d/%d with err: %v", msg, i, p.cfg.RetryCount, err2) + if p.cfg.RetryCount > 0 { + err = errors.Join(err, err2) + } else { + // in the case of infinite retries, only keep the last error + err = err2 + } + } + + return nil, time.Since(start), fmt.Errorf("%s: giving up after %d retries: %w", msg, p.cfg.RetryCount, err) +} + +func RetriesNoOutput[X pluginInput](f func(a X) error, a X, p *pipelineImpl, msg string) (time.Duration, error) { + _, d, err := Retries(func(a X) (empty, error) { + return empty{}, f(a) + }, a, p, msg) + return d, err +} + +// func RetriesNoOutput[X pluginInput](times int, f func(a X) error, a X) error { +// _, err := Retries(times, func(a X) (empty, error) { +// return empty{}, f(a) +// }, a) +// return err +// } diff --git a/conduit/pipeline/pipeline_bench_test.go b/conduit/pipeline/pipeline_bench_test.go index 93ebee36..833ae8f5 100644 --- a/conduit/pipeline/pipeline_bench_test.go +++ b/conduit/pipeline/pipeline_bench_test.go @@ -2,6 +2,7 @@ package pipeline import ( "context" + "errors" "fmt" "testing" "time" @@ -17,7 +18,7 @@ import ( ) const ( - logLevel = log.ErrorLevel //log.InfoLevel // log.DebugLevel // + logLevel = log.ErrorLevel // log.DebugLevel // log.InfoLevel // log.TraceLevel // retryCount = 3 // math.MaxUint64 ) @@ -195,13 +196,13 @@ func pipeline5sec(b *testing.B, testCase benchmarkCase) int { } exporter := &sleepingExporter{receiveSleep: testCase.exporterSleep} - ctx, cf := context.WithCancel(context.Background()) + ctx, ccf := context.WithCancelCause(context.Background()) logger := log.New() logger.SetLevel(logLevel) pImpl := pipelineImpl{ ctx: ctx, - cf: cf, + ccf: ccf, logger: logger, initProvider: nil, importer: importer, @@ -225,7 +226,7 @@ func pipeline5sec(b *testing.B, testCase benchmarkCase) int { // cancel the pipeline after 5 seconds go func() { time.Sleep(benchmarkDuration) - cf() + ccf(errors.New("benchmark timeout")) }() b.StartTimer() @@ -247,36 +248,36 @@ func finalRound(pi *pipelineImpl) (sdk.Round, error) { func BenchmarkPipeline(b *testing.B) { benchCases := []benchmarkCase{ - { - name: "vanilla 2 procs without sleep", - importerSleep: 0, - processorsSleep: []time.Duration{0, 0}, - exporterSleep: 0, - }, - { - name: "uniform sleep of 10ms", - importerSleep: 10 * time.Millisecond, - processorsSleep: []time.Duration{10 * time.Millisecond, 10 * time.Millisecond}, - exporterSleep: 10 * time.Millisecond, - }, + // { + // name: "vanilla 2 procs without sleep", + // importerSleep: 0, + // processorsSleep: []time.Duration{0, 0}, + // exporterSleep: 0, + // }, + // { + // name: "uniform sleep of 10ms", + // importerSleep: 10 * time.Millisecond, + // processorsSleep: []time.Duration{10 * time.Millisecond, 10 * time.Millisecond}, + // exporterSleep: 10 * time.Millisecond, + // }, { name: "exporter 10ms while others 1ms", importerSleep: time.Millisecond, processorsSleep: []time.Duration{time.Millisecond, time.Millisecond}, exporterSleep: 10 * time.Millisecond, }, - { - name: "importer 10ms while others 1ms", - importerSleep: 10 * time.Millisecond, - processorsSleep: []time.Duration{time.Millisecond, time.Millisecond}, - exporterSleep: time.Millisecond, - }, - { - name: "first processor 10ms while others 1ms", - importerSleep: time.Millisecond, - processorsSleep: []time.Duration{10 * time.Millisecond, time.Millisecond}, - exporterSleep: time.Millisecond, - }, + // { + // name: "importer 10ms while others 1ms", + // importerSleep: 10 * time.Millisecond, + // processorsSleep: []time.Duration{time.Millisecond, time.Millisecond}, + // exporterSleep: time.Millisecond, + // }, + // { + // name: "first processor 10ms while others 1ms", + // importerSleep: time.Millisecond, + // processorsSleep: []time.Duration{10 * time.Millisecond, time.Millisecond}, + // exporterSleep: time.Millisecond, + // }, } for _, buffSize := range []int{1} { for _, bc := range benchCases { diff --git a/conduit/pipeline/pipeline_test.go b/conduit/pipeline/pipeline_test.go index 6392a74e..9feda9ab 100644 --- a/conduit/pipeline/pipeline_test.go +++ b/conduit/pipeline/pipeline_test.go @@ -264,12 +264,12 @@ func TestPipelineRun(t *testing.T) { var pExporter exporters.Exporter = &mExporter var cbComplete conduit.Completed = &mProcessor - ctx, cf := context.WithCancel(context.Background()) + ctx, ccf := context.WithCancelCause(context.Background()) l, _ := test.NewNullLogger() pImpl := pipelineImpl{ ctx: ctx, - cf: cf, + ccf: ccf, logger: l, initProvider: nil, importer: pImporter, @@ -291,7 +291,7 @@ func TestPipelineRun(t *testing.T) { go func() { time.Sleep(1 * time.Second) - cf() + ccf(errors.New("testing timeout")) }() pImpl.Start() @@ -358,11 +358,11 @@ func TestPipelineErrors(t *testing.T) { var pExporter exporters.Exporter = &mExporter var cbComplete conduit.Completed = &mProcessor - ctx, cf := context.WithCancel(context.Background()) + ctx, ccf := context.WithCancelCause(context.Background()) l, _ := test.NewNullLogger() pImpl := pipelineImpl{ ctx: ctx, - cf: cf, + ccf: ccf, cfg: &data.Config{ RetryDelay: 0 * time.Second, RetryCount: math.MaxUint64, @@ -383,37 +383,38 @@ func TestPipelineErrors(t *testing.T) { go pImpl.Start() time.Sleep(time.Millisecond) - pImpl.cf() + cancel := errors.New("testing timeout") + pImpl.ccf(cancel) pImpl.Wait() assert.Error(t, pImpl.Error(), fmt.Errorf("importer")) mImporter.returnError = false mProcessor.returnError = true - pImpl.ctx, pImpl.cf = context.WithCancel(context.Background()) - pImpl.setError(nil) + pImpl.ctx, pImpl.ccf = context.WithCancelCause(context.Background()) + pImpl.joinError(nil) go pImpl.Start() time.Sleep(time.Millisecond) - pImpl.cf() + pImpl.ccf(cancel) pImpl.Wait() assert.Error(t, pImpl.Error(), fmt.Errorf("process")) mProcessor.returnError = false mProcessor.onCompleteError = true - pImpl.ctx, pImpl.cf = context.WithCancel(context.Background()) - pImpl.setError(nil) + pImpl.ctx, pImpl.ccf = context.WithCancelCause(context.Background()) + pImpl.joinError(nil) go pImpl.Start() time.Sleep(time.Millisecond) - pImpl.cf() + pImpl.ccf(cancel) pImpl.Wait() assert.Error(t, pImpl.Error(), fmt.Errorf("on complete")) mProcessor.onCompleteError = false mExporter.returnError = true - pImpl.ctx, pImpl.cf = context.WithCancel(context.Background()) - pImpl.setError(nil) + pImpl.ctx, pImpl.ccf = context.WithCancelCause(context.Background()) + pImpl.joinError(nil) go pImpl.Start() time.Sleep(time.Millisecond) - pImpl.cf() + pImpl.ccf(cancel) pImpl.Wait() assert.Error(t, pImpl.Error(), fmt.Errorf("exporter")) } @@ -433,11 +434,11 @@ func Test_pipelineImpl_registerLifecycleCallbacks(t *testing.T) { var pProcessor processors.Processor = &mProcessor var pExporter exporters.Exporter = &mExporter - ctx, cf := context.WithCancel(context.Background()) + ctx, ccf := context.WithCancelCause(context.Background()) l, _ := test.NewNullLogger() pImpl := pipelineImpl{ ctx: ctx, - cf: cf, + ccf: ccf, cfg: &data.Config{}, logger: l, initProvider: nil, From 66f128841f9e5c43111a11c502b424935c4b8a23 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Tue, 25 Jul 2023 19:04:06 -0500 Subject: [PATCH 03/50] basic logic in place --- conduit/pipeline/pipeline.go | 262 ++++++++++++++++++++++++++++++++--- 1 file changed, 246 insertions(+), 16 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index b77529d4..f0266f6f 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -3,6 +3,7 @@ package pipeline import ( "context" "encoding/base64" + "errors" "fmt" "net/http" "os" @@ -41,13 +42,13 @@ type Pipeline interface { type pipelineImpl struct { ctx context.Context - cf context.CancelFunc + ccf context.CancelCauseFunc wg sync.WaitGroup cfg *data.Config logger *log.Logger profFile *os.File err error - mu sync.RWMutex + errMu sync.RWMutex initProvider *data.InitProvider @@ -59,16 +60,28 @@ type pipelineImpl struct { pipelineMetadata state } +var ( + errStopCause = errors.New("pipeline stopped") + errImporterCause = errors.New("importer cancelled") + errProcessorCause = errors.New("processor cancelled") + errExporterCause = errors.New("exporter cancelled") +) + func (p *pipelineImpl) Error() error { - p.mu.RLock() - defer p.mu.RUnlock() + p.errMu.RLock() + defer p.errMu.RUnlock() return p.err } -func (p *pipelineImpl) setError(err error) { - p.mu.Lock() - defer p.mu.Unlock() - p.err = err +func (p *pipelineImpl) joinError(err error) { + p.errMu.Lock() + defer p.errMu.Unlock() + p.err = errors.Join(p.err, err) +} + +func (p *pipelineImpl) cancel(err error) { + p.ccf(err) + p.joinError(err) } func (p *pipelineImpl) registerLifecycleCallbacks() { @@ -372,7 +385,7 @@ func (p *pipelineImpl) Init() error { } func (p *pipelineImpl) Stop() { - p.cf() + p.ccf(errStopCause) p.wg.Wait() if p.profFile != nil { @@ -431,8 +444,225 @@ func addMetrics(block data.BlockData, importTime time.Duration) { metrics.ImportedTxnsPerBlock.Observe(float64(len(block.Payset)) + float64(innerTxn)) } +func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-chan uint64, blkOutChan chan<- data.BlockData) { + p.wg.Add(1) + go func() { + defer p.wg.Done() + lastRnd := uint64(0) + totalSelectWait := time.Duration(0) + totalFeedWait := time.Duration(0) + defer func() { + p.logger.Debugf("importer handler exiting. lastRnd=%d totalSelectWait=%dms; totalFeedWait=%dms", lastRnd, totalSelectWait.Milliseconds(), totalFeedWait.Milliseconds()) + }() + + for { + selectStart := time.Now() + select { + case <-p.ctx.Done(): + p.logger.Infof("importer handler exiting. lastRnd=%d", lastRnd) + return + case rnd := <-roundChan: + lastRnd = rnd + waitTime := time.Since(selectStart) + totalSelectWait += waitTime + p.logger.Tracef("importer handler @ round %d received %s", rnd, waitTime) + + blkDataPtr, importTime, lastError := Retries(importer.GetBlock, rnd, p, importer.Metadata().Name) + if lastError != nil { + p.cancel(fmt.Errorf("importer %s handler (%w): failed to import round %d after %dms: %w", importer.Metadata().Name, errImporterCause, rnd, importTime.Milliseconds(), lastError)) + return + } + if blkDataPtr == nil { + p.cancel(fmt.Errorf("importer %s handler (%w): receieved nil blkDataPtr. this should never happen! failed to import round", importer.Metadata().Name, errImporterCause)) + return + } + metrics.ImporterTimeSeconds.Observe(importTime.Seconds()) + + feedStart := time.Now() + select { + case <-p.ctx.Done(): + return + case blkOutChan <- *blkDataPtr: + waitTime := time.Since(feedStart) + totalFeedWait += waitTime + p.logger.Tracef("imported round %d into blkOutChan after waiting %dms on channel", rnd, waitTime.Milliseconds()) + } + + } + } + }() +} + +func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkInChan <-chan data.BlockData, blkOutChan chan<- data.BlockData) { + p.wg.Add(1) + go func() { + defer p.wg.Done() + var lastRnd uint64 + totalSelectWait := time.Duration(0) + totalFeedWait := time.Duration(0) + defer func() { + p.logger.Debugf("processor[%d] %s handler exiting. lastRnd=%d totalSelectWait=%dms, totalFeedWait=%dms", idx, proc.Metadata().Name, lastRnd, totalSelectWait.Milliseconds(), totalFeedWait.Milliseconds()) + }() + + for { + selectStart := time.Now() + select { + case <-p.ctx.Done(): + p.logger.Infof("processor[%d] %s handler exiting lastRnd=%d", idx, proc.Metadata().Name, lastRnd) + return + case blkData := <-blkInChan: + waitTime := time.Since(selectStart) + totalSelectWait += waitTime + p.logger.Tracef("processor[%d] %s handler received block data for round %d after wait of %s", idx, proc.Metadata().Name, lastRnd, waitTime) + lastRnd = blkData.Round() + + blkDataPtr, procTime, lastError := Retries(proc.Process, blkData, p, proc.Metadata().Name) + if lastError != nil { + p.cancel(fmt.Errorf("processor[%d] %s handler (%w): failed to process round %d after %dms: %w", idx, proc.Metadata().Name, errProcessorCause, lastRnd, procTime.Milliseconds(), lastError)) + return + } + if blkDataPtr == nil { + p.cancel(fmt.Errorf("processor[%d] %s handler (%w): receieved nil blkDataPtr. this should never happen! failed to process round %d", idx, proc.Metadata().Name, errProcessorCause, lastRnd)) + return + } + metrics.ProcessorTimeSeconds.WithLabelValues(proc.Metadata().Name).Observe(procTime.Seconds()) + + selectStart := time.Now() + select { + case <-p.ctx.Done(): + return + case blkOutChan <- blkData: + waitTime := time.Since(selectStart) + totalFeedWait += waitTime + p.logger.Tracef("processor[%d] %s for round %d into blkOutChan after waiting %dms", idx, proc.Metadata().Name, lastRnd, totalFeedWait.Milliseconds()) + } + } + } + }() +} + +// ExporterHandler handles the exporter's Receive method, updating the metadata's NextRound, +// saving the metadata, and invoking the activated plugin callbacks. If the context's cancellation is received +// before the exporter's Receive method has succeeded, cancel immediately. +// However, after the exporter's Receive() method has succeeded we try every component at least once +// but will abort if a failure occurs at when a cancellation is received. +func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-chan data.BlockData) { + p.wg.Add(1) + go func() { + defer p.wg.Done() + var lastError error + lastRnd := uint64(0) + totalSelectWait := time.Duration(0) + totalExportWork := time.Duration(0) + + defer func() { + p.logger.Debugf("exporter %s handler exiting. lastRnd=%d totalSelectWait=%dms, totalExportWork=%dms", exporter.Metadata().Name, lastRnd, totalSelectWait.Milliseconds(), totalExportWork.Milliseconds()) + }() + + defer func() { + if lastError != nil { + err := fmt.Errorf("exporter %s handler (%w) after round %d: %w", exporter.Metadata().Name, errExporterCause, lastRnd, lastError) + p.cancel(err) + p.logger.Error(err) + } + }() + + for { + p.logger.Tracef("exporter handler waiting for block data. round=%d", lastRnd) + selectStart := time.Now() + select { + case <-p.ctx.Done(): + p.logger.Infof("exporter handler exiting lastRnd=%d", lastRnd) + return + case blkData := <-blkChan: + waitTime := time.Since(selectStart) + totalSelectWait += waitTime + p.logger.Tracef("exporter handler received block data for round %d after wait of %s", lastRnd, waitTime) + lastRnd = blkData.Round() + + // TODO: the next level of concurrency will require an atomic NextRound counter + if p.pipelineMetadata.NextRound != lastRnd { + lastError = fmt.Errorf("aborting after out of order block data. %d != %d", p.pipelineMetadata.NextRound, lastRnd) + return + } + + var exportTime time.Duration + exportTime, lastError = RetriesNoOutput(exporter.Receive, blkData, p, exporter.Metadata().Name) + if lastError != nil { + lastError = fmt.Errorf("aborting after failing to export round %d: %w", lastRnd, lastError) + return + } + metrics.ExporterTimeSeconds.Observe(exportTime.Seconds()) + + // Increment Round, update metadata + p.pipelineMetadata.NextRound = lastRnd + 1 + _, lastError = RetriesNoOutput(p.pipelineMetadata.encodeToFile, p.cfg.ConduitArgs.ConduitDataDir, p, "pipelineMetadata save") + if lastError != nil { + lastError = fmt.Errorf("aborting after updating NextRound=%d and failing to save metadata: %w", lastRnd+1, lastError) + return + } + p.logger.Tracef("exporter %s @ round %d saved pipeline metadata", exporter.Metadata().Name, p.pipelineMetadata.NextRound) + + for i, cb := range p.completeCallback { + p.logger.Tracef("exporter %s @ round=%d NextRound=%d executing callback %d", exporter.Metadata().Name, lastRnd, p.pipelineMetadata.NextRound, i) + _, lastError = RetriesNoOutput(cb, blkData, p, fmt.Sprintf("callback %d", i)) + if lastError != nil { + lastError = fmt.Errorf("aborting due to failed callback %d: %w", i, lastError) + return + } + } + lastError = nil + } + } + }() +} + // Start pushes block data through the pipeline func (p *pipelineImpl) Start() { + p.logger.Trace("Pipeline.Start()") + + // Setup channels + roundChan := make(chan uint64) + processorBlkInChan := make(chan data.BlockData) + p.ImportHandler(p.importer, roundChan, processorBlkInChan) + + var processorBlkOutChan chan data.BlockData + for i, proc := range p.processors { + processorBlkOutChan = make(chan data.BlockData) + p.ProcessorHandler(i, proc, processorBlkInChan, processorBlkOutChan) + processorBlkInChan = processorBlkOutChan + } + + p.ExporterHandler(p.exporter, processorBlkOutChan) + + p.wg.Add(1) + // Main loop + go func(startRound uint64) { + defer p.wg.Done() + rnd := startRound + totalFeedWait := time.Duration(0) + defer p.logger.Debugf("round channel feed exiting. lastRnd=%d totalFeedWait=%dms", rnd, totalFeedWait.Milliseconds()) + for { + selectStart := time.Now() + p.logger.Tracef("pushing round %d into roundChan", rnd) + select { + case <-p.ctx.Done(): + p.logger.Infof("round channel feed exiting. lastRnd=%d", rnd) + return + case roundChan <- rnd: + waitTime := time.Since(selectStart) + totalFeedWait += waitTime + rnd++ + } + } + }(p.pipelineMetadata.NextRound) + + <-p.ctx.Done() + // TODO: send a prometheus observation based on context.Cause(ctx) +} + +// Start pushes block data through the pipeline +func (p *pipelineImpl) OStart() { p.wg.Add(1) retry := uint64(0) go func() { @@ -463,7 +693,7 @@ func (p *pipelineImpl) Start() { blkData, err := p.importer.GetBlock(p.pipelineMetadata.NextRound) if err != nil { p.logger.Errorf("%v", err) - p.setError(err) + p.joinError(err) retry++ goto pipelineRun } @@ -480,7 +710,7 @@ func (p *pipelineImpl) Start() { blkData, err = proc.Process(blkData) if err != nil { p.logger.Errorf("%v", err) - p.setError(err) + p.joinError(err) retry++ goto pipelineRun } @@ -491,7 +721,7 @@ func (p *pipelineImpl) Start() { err = p.exporter.Receive(blkData) if err != nil { p.logger.Errorf("%v", err) - p.setError(err) + p.joinError(err) retry++ goto pipelineRun } @@ -509,7 +739,7 @@ func (p *pipelineImpl) Start() { err = cb(blkData) if err != nil { p.logger.Errorf("%v", err) - p.setError(err) + p.joinError(err) retry++ goto pipelineRun } @@ -519,7 +749,7 @@ func (p *pipelineImpl) Start() { if p.pipelineMetadata.NextRound > 1 { addMetrics(blkData, time.Since(start)) } - p.setError(nil) + p.joinError(nil) retry = 0 } } @@ -554,11 +784,11 @@ func MakePipeline(ctx context.Context, cfg *data.Config, logger *log.Logger) (Pi return nil, fmt.Errorf("MakePipeline(): logger was empty") } - cancelContext, cancelFunc := context.WithCancel(ctx) + cancelContext, cancelCauseFunc := context.WithCancelCause(ctx) pipeline := &pipelineImpl{ ctx: cancelContext, - cf: cancelFunc, + ccf: cancelCauseFunc, cfg: cfg, logger: logger, initProvider: nil, From 42fa4fd225bb97888b3d10e195fc7adbc5056753 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Tue, 25 Jul 2023 23:19:08 -0500 Subject: [PATCH 04/50] common.go unit test --- conduit/pipeline/common.go | 32 ++--- conduit/pipeline/common_test.go | 210 ++++++++++++++++++++++++++++++++ conduit/pipeline/pipeline.go | 14 +-- 3 files changed, 224 insertions(+), 32 deletions(-) create mode 100644 conduit/pipeline/common_test.go diff --git a/conduit/pipeline/common.go b/conduit/pipeline/common.go index 62a2818d..ebf93264 100644 --- a/conduit/pipeline/common.go +++ b/conduit/pipeline/common.go @@ -1,6 +1,7 @@ package pipeline import ( + "context" "errors" "fmt" "time" @@ -18,35 +19,29 @@ func HandlePanic(logger *log.Logger) { type empty struct{} -// Probly don't need this: -// func EmptyOutputter[X any](f func(a X) error) func(X) (empty, error) { -// return func(a X) (empty, error) { -// return empty{}, f(a) -// } -// } - type pluginInput interface { uint64 | data.BlockData | string | empty } -func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg string) (*Y, time.Duration, error) { - var err error +func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg string) (y Y, dur time.Duration, err error) { start := time.Now() for i := uint64(0); p.cfg.RetryCount == 0 || i <= p.cfg.RetryCount; i++ { - // the first time through, we don't sleep or mind the done signal + // the first time through, we don't sleep or mind ctx's done signal if i > 0 { select { case <-p.ctx.Done(): - return nil, time.Since(start), fmt.Errorf("%s: retry number %d/%d with err: %w. Done signal received: %w", msg, i, p.cfg.RetryCount, err, p.ctx.Err()) + dur = time.Since(start) + err = fmt.Errorf("%s: retry number %d/%d with err: %w. Done signal received: %w", msg, i, p.cfg.RetryCount, err, context.Cause(p.ctx)) + return default: time.Sleep(p.cfg.RetryDelay) } } opStart := time.Now() - y, err2 := f(x) + y2, err2 := f(x) if err2 == nil { - return &y, time.Since(opStart), nil + return y2, time.Since(opStart), nil } p.logger.Infof("%s: retry number %d/%d with err: %v", msg, i, p.cfg.RetryCount, err2) @@ -58,7 +53,9 @@ func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg } } - return nil, time.Since(start), fmt.Errorf("%s: giving up after %d retries: %w", msg, p.cfg.RetryCount, err) + dur = time.Since(start) + err = fmt.Errorf("%s: giving up after %d retries: %w", msg, p.cfg.RetryCount, err) + return } func RetriesNoOutput[X pluginInput](f func(a X) error, a X, p *pipelineImpl, msg string) (time.Duration, error) { @@ -67,10 +64,3 @@ func RetriesNoOutput[X pluginInput](f func(a X) error, a X, p *pipelineImpl, msg }, a, p, msg) return d, err } - -// func RetriesNoOutput[X pluginInput](times int, f func(a X) error, a X) error { -// _, err := Retries(times, func(a X) (empty, error) { -// return empty{}, f(a) -// }, a) -// return err -// } diff --git a/conduit/pipeline/common_test.go b/conduit/pipeline/common_test.go new file mode 100644 index 00000000..a283956b --- /dev/null +++ b/conduit/pipeline/common_test.go @@ -0,0 +1,210 @@ +package pipeline + +import ( + "context" + "errors" + "fmt" + "testing" + "time" + + "github.com/algorand/conduit/conduit/data" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/require" +) + +// TestRetries tests the retry logic +func TestRetries(t *testing.T) { + errSucceedAfter := errors.New("succeed after has failed") + + succeedAfterFactory := func(succeedAfter uint64, never bool) func(uint64) (uint64, error) { + tries := uint64(0) + + return func(x uint64) (uint64, error) { + if tries >= succeedAfter && !never { + // ensure not to return the zero value on success + return tries + 1, nil + } + tries++ + return 0, fmt.Errorf("%w: tries=%d", errSucceedAfter, tries-1) + } + } + + succeedAfterFactoryNoOutput := func(succeedAfter uint64, never bool) func(uint64) error { + tries := uint64(0) + + return func(x uint64) error { + if tries >= succeedAfter && !never { + return nil + } + tries++ + return fmt.Errorf("%w: tries=%d", errSucceedAfter, tries-1) + } + } + + cases := []struct { + name string + retryCount uint64 + succeedAfter uint64 + neverSucceed bool // neverSucceed trumps succeedAfter + }{ + { + name: "retry forever succeeds after 0", + retryCount: 0, + succeedAfter: 0, + neverSucceed: false, + }, + { + name: "retry forever succeeds after 1", + retryCount: 0, + succeedAfter: 1, + neverSucceed: false, + }, + { + name: "retry forever succeeds after 7", + retryCount: 0, + succeedAfter: 7, + neverSucceed: false, + }, + { + name: "retry 5 succeeds after 0", + retryCount: 5, + succeedAfter: 0, + neverSucceed: false, + }, + { + name: "retry 5 succeeds after 1", + retryCount: 5, + succeedAfter: 1, + neverSucceed: false, + }, + { + name: "retry 5 succeeds after 5", + retryCount: 5, + succeedAfter: 5, + neverSucceed: false, + }, + { + name: "retry 5 succeeds after 7", + retryCount: 5, + succeedAfter: 7, + neverSucceed: false, + }, + { + name: "retry 5 never succeeds", + retryCount: 5, + succeedAfter: 0, + neverSucceed: true, + }, + { + name: "retry foerever never succeeds", + retryCount: 0, + succeedAfter: 0, + neverSucceed: true, + }, + } + + for _, tc := range cases { + tc := tc + + // run cases for Retries() + t.Run("Retries() "+tc.name, func(t *testing.T) { + t.Parallel() + ctx, ccf := context.WithCancelCause(context.Background()) + p := &pipelineImpl{ + ctx: ctx, + ccf: ccf, + logger: log.New(), + cfg: &data.Config{ + RetryCount: tc.retryCount, + RetryDelay: 1 * time.Millisecond, + }, + } + succeedAfter := succeedAfterFactory(tc.succeedAfter, tc.neverSucceed) + + if tc.retryCount == 0 && tc.neverSucceed { + // avoid infinite loop by cancelling the context + + yChan := make(chan uint64) + errChan := make(chan error) + go func() { + y, _, err := Retries(succeedAfter, 0, p, "test") + yChan <- y + errChan <- err + }() + time.Sleep(5 * time.Millisecond) + errTestCancelled := errors.New("test cancelled") + go func() { + ccf(errTestCancelled) + }() + y := <-yChan + err := <-errChan + require.ErrorIs(t, err, errTestCancelled, tc.name) + require.ErrorIs(t, err, errSucceedAfter, tc.name) + require.Zero(t, y, tc.name) + return + } + + y, _, err := Retries(succeedAfter, 0, p, "test") + if tc.retryCount == 0 { // WLOG tc.neverSucceed == false + require.NoError(t, err, tc.name) + require.Equal(t, tc.succeedAfter, y-1, tc.name) + } else { // retryCount > 0 so doesn't retry forever + if tc.neverSucceed || tc.succeedAfter > tc.retryCount { + require.ErrorContains(t, err, fmt.Sprintf("%d", tc.retryCount), tc.name) + require.ErrorIs(t, err, errSucceedAfter, tc.name) + require.Zero(t, y, tc.name) + } else { // !tc.neverSucceed && succeedAfter <= retryCount + require.NoError(t, err, tc.name) + require.Equal(t, tc.succeedAfter, y-1, tc.name) + } + } + }) + + // run cases for RetriesNoOutput() + t.Run("RetriesNoOutput() "+tc.name, func(t *testing.T) { + t.Parallel() + ctx, ccf := context.WithCancelCause(context.Background()) + p := &pipelineImpl{ + ctx: ctx, + ccf: ccf, + logger: log.New(), + cfg: &data.Config{ + RetryCount: tc.retryCount, + RetryDelay: 1 * time.Millisecond, + }, + } + succeedAfterNoOutput := succeedAfterFactoryNoOutput(tc.succeedAfter, tc.neverSucceed) + + if tc.retryCount == 0 && tc.neverSucceed { + // avoid infinite loop by cancelling the context + + errChan := make(chan error) + go func() { + _, err := RetriesNoOutput(succeedAfterNoOutput, 0, p, "test") + errChan <- err + }() + time.Sleep(5 * time.Millisecond) + errTestCancelled := errors.New("test cancelled") + go func() { + ccf(errTestCancelled) + }() + err := <-errChan + require.ErrorIs(t, err, errTestCancelled, tc.name) + require.ErrorIs(t, err, errSucceedAfter, tc.name) + return + } + + _, err := RetriesNoOutput(succeedAfterNoOutput, 0, p, "test") + if tc.retryCount == 0 { // WLOG tc.neverSucceed == false + require.NoError(t, err, tc.name) + } else { // retryCount > 0 so doesn't retry forever + if tc.neverSucceed || tc.succeedAfter > tc.retryCount { + require.ErrorContains(t, err, fmt.Sprintf("%d", tc.retryCount), tc.name) + require.ErrorIs(t, err, errSucceedAfter, tc.name) + } else { // !tc.neverSucceed && succeedAfter <= retryCount + require.NoError(t, err, tc.name) + } + } + }) + } +} diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index f0266f6f..8dab7624 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -467,22 +467,18 @@ func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-ch totalSelectWait += waitTime p.logger.Tracef("importer handler @ round %d received %s", rnd, waitTime) - blkDataPtr, importTime, lastError := Retries(importer.GetBlock, rnd, p, importer.Metadata().Name) + blkData, importTime, lastError := Retries(importer.GetBlock, rnd, p, importer.Metadata().Name) if lastError != nil { p.cancel(fmt.Errorf("importer %s handler (%w): failed to import round %d after %dms: %w", importer.Metadata().Name, errImporterCause, rnd, importTime.Milliseconds(), lastError)) return } - if blkDataPtr == nil { - p.cancel(fmt.Errorf("importer %s handler (%w): receieved nil blkDataPtr. this should never happen! failed to import round", importer.Metadata().Name, errImporterCause)) - return - } metrics.ImporterTimeSeconds.Observe(importTime.Seconds()) feedStart := time.Now() select { case <-p.ctx.Done(): return - case blkOutChan <- *blkDataPtr: + case blkOutChan <- blkData: waitTime := time.Since(feedStart) totalFeedWait += waitTime p.logger.Tracef("imported round %d into blkOutChan after waiting %dms on channel", rnd, waitTime.Milliseconds()) @@ -516,15 +512,11 @@ func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkI p.logger.Tracef("processor[%d] %s handler received block data for round %d after wait of %s", idx, proc.Metadata().Name, lastRnd, waitTime) lastRnd = blkData.Round() - blkDataPtr, procTime, lastError := Retries(proc.Process, blkData, p, proc.Metadata().Name) + blkData, procTime, lastError := Retries(proc.Process, blkData, p, proc.Metadata().Name) if lastError != nil { p.cancel(fmt.Errorf("processor[%d] %s handler (%w): failed to process round %d after %dms: %w", idx, proc.Metadata().Name, errProcessorCause, lastRnd, procTime.Milliseconds(), lastError)) return } - if blkDataPtr == nil { - p.cancel(fmt.Errorf("processor[%d] %s handler (%w): receieved nil blkDataPtr. this should never happen! failed to process round %d", idx, proc.Metadata().Name, errProcessorCause, lastRnd)) - return - } metrics.ProcessorTimeSeconds.WithLabelValues(proc.Metadata().Name).Observe(procTime.Seconds()) selectStart := time.Now() From b8a5ee1fae362e8199c25a5489947afc8547345f Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Tue, 25 Jul 2023 23:32:45 -0500 Subject: [PATCH 05/50] commentariat --- conduit/pipeline/common.go | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/conduit/pipeline/common.go b/conduit/pipeline/common.go index ebf93264..1bb3b185 100644 --- a/conduit/pipeline/common.go +++ b/conduit/pipeline/common.go @@ -23,6 +23,17 @@ type pluginInput interface { uint64 | data.BlockData | string | empty } +// Retries is a helper function to retry a function call with a delay and a max retry count. +// The pipeline's configuration determines the retry behavior: +// - When p.cfg.retryCount == 0, the function will retry forever or until the context is canceled. +// - When p.cfg.retryCount > 0, the function will retry p.cfg.retryCount times before giving up. +// - Upon success, a nil error is returned. +// - Upon failure, a non-nil error is returned. In the case of p.cfg.retryCount > 0, the error +// will be a joined error of all the errors encountered during the retries. In the case of +// p.cfg.retryCount == 0, the error will be the last error encountered. +// - In the case of failure or cancellation, the duration returned is the total time spent in the function, including retries. +// - In the case of success, the duration measures the time spent in the call to f() that succeeded. +// - In the case of failure or cancellation, the value returned will be the zero value for Y. func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg string) (y Y, dur time.Duration, err error) { start := time.Now() @@ -46,6 +57,7 @@ func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg p.logger.Infof("%s: retry number %d/%d with err: %v", msg, i, p.cfg.RetryCount, err2) if p.cfg.RetryCount > 0 { + // TODO: this feels like a code smell. Probly better to always keep only the last error. err = errors.Join(err, err2) } else { // in the case of infinite retries, only keep the last error @@ -58,9 +70,10 @@ func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg return } -func RetriesNoOutput[X pluginInput](f func(a X) error, a X, p *pipelineImpl, msg string) (time.Duration, error) { - _, d, err := Retries(func(a X) (empty, error) { - return empty{}, f(a) +// RetriesNoOutput applies the same logic as Retries, but for functions that return no output. +func RetriesNoOutput[X pluginInput](f func(x X) error, a X, p *pipelineImpl, msg string) (time.Duration, error) { + _, d, err := Retries(func(x X) (empty, error) { + return empty{}, f(x) }, a, p, msg) return d, err } From 5504a23cc198ce43cf2ff40c67cc5bb45c8b8302 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 26 Jul 2023 09:37:26 -0500 Subject: [PATCH 06/50] better function comment --- conduit/pipeline/common.go | 29 +++++++++++++++---------- conduit/pipeline/common_test.go | 14 ++++++------ conduit/pipeline/pipeline_bench_test.go | 1 - 3 files changed, 25 insertions(+), 19 deletions(-) diff --git a/conduit/pipeline/common.go b/conduit/pipeline/common.go index 1bb3b185..a15c2ee4 100644 --- a/conduit/pipeline/common.go +++ b/conduit/pipeline/common.go @@ -23,17 +23,24 @@ type pluginInput interface { uint64 | data.BlockData | string | empty } -// Retries is a helper function to retry a function call with a delay and a max retry count. -// The pipeline's configuration determines the retry behavior: -// - When p.cfg.retryCount == 0, the function will retry forever or until the context is canceled. -// - When p.cfg.retryCount > 0, the function will retry p.cfg.retryCount times before giving up. -// - Upon success, a nil error is returned. -// - Upon failure, a non-nil error is returned. In the case of p.cfg.retryCount > 0, the error -// will be a joined error of all the errors encountered during the retries. In the case of -// p.cfg.retryCount == 0, the error will be the last error encountered. -// - In the case of failure or cancellation, the duration returned is the total time spent in the function, including retries. -// - In the case of success, the duration measures the time spent in the call to f() that succeeded. -// - In the case of failure or cancellation, the value returned will be the zero value for Y. +// Retries is a wrapper for retrying a function call f() with a cancellation context, +// a delay and a max retry count. It attempts to call the wrapped function at least once +// and only after the first attempt will pay attention to a context cancellation. +// This can allow the pipeline to receive a cancellation and guarantee attempting to finish +// the round with at least one attempt for each pipeline component. +// - Retry behavior is configured by p.cfg.retryCount. +// - when 0, the function will retry forever or until the context is canceled +// - when > 0, the function will retry p.cfg.retryCount times before giving up +// +// - Upon success: +// - a nil error is returned even if there were many intermediate failures +// - the returned duration dur measures the time spent in the call to f() that succeeded +// +// - Upon failure: +// - the return value y is the zero value of type Y and a non-nil error is returned +// - when p.cfg.retryCount > 0, the error will be a join of all the errors encountered during the retries +// - when p.cfg.retryCount == 0, the error will be the last error encountered +// - the returned duration dur is the total time spent in the function, including retries func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg string) (y Y, dur time.Duration, err error) { start := time.Now() diff --git a/conduit/pipeline/common_test.go b/conduit/pipeline/common_test.go index a283956b..f55b69e6 100644 --- a/conduit/pipeline/common_test.go +++ b/conduit/pipeline/common_test.go @@ -14,7 +14,7 @@ import ( // TestRetries tests the retry logic func TestRetries(t *testing.T) { - errSucceedAfter := errors.New("succeed after has failed") + errSentinelCause := errors.New("succeed after has failed") succeedAfterFactory := func(succeedAfter uint64, never bool) func(uint64) (uint64, error) { tries := uint64(0) @@ -25,7 +25,7 @@ func TestRetries(t *testing.T) { return tries + 1, nil } tries++ - return 0, fmt.Errorf("%w: tries=%d", errSucceedAfter, tries-1) + return 0, fmt.Errorf("%w: tries=%d", errSentinelCause, tries-1) } } @@ -37,7 +37,7 @@ func TestRetries(t *testing.T) { return nil } tries++ - return fmt.Errorf("%w: tries=%d", errSucceedAfter, tries-1) + return fmt.Errorf("%w: tries=%d", errSentinelCause, tries-1) } } @@ -139,7 +139,7 @@ func TestRetries(t *testing.T) { y := <-yChan err := <-errChan require.ErrorIs(t, err, errTestCancelled, tc.name) - require.ErrorIs(t, err, errSucceedAfter, tc.name) + require.ErrorIs(t, err, errSentinelCause, tc.name) require.Zero(t, y, tc.name) return } @@ -151,7 +151,7 @@ func TestRetries(t *testing.T) { } else { // retryCount > 0 so doesn't retry forever if tc.neverSucceed || tc.succeedAfter > tc.retryCount { require.ErrorContains(t, err, fmt.Sprintf("%d", tc.retryCount), tc.name) - require.ErrorIs(t, err, errSucceedAfter, tc.name) + require.ErrorIs(t, err, errSentinelCause, tc.name) require.Zero(t, y, tc.name) } else { // !tc.neverSucceed && succeedAfter <= retryCount require.NoError(t, err, tc.name) @@ -190,7 +190,7 @@ func TestRetries(t *testing.T) { }() err := <-errChan require.ErrorIs(t, err, errTestCancelled, tc.name) - require.ErrorIs(t, err, errSucceedAfter, tc.name) + require.ErrorIs(t, err, errSentinelCause, tc.name) return } @@ -200,7 +200,7 @@ func TestRetries(t *testing.T) { } else { // retryCount > 0 so doesn't retry forever if tc.neverSucceed || tc.succeedAfter > tc.retryCount { require.ErrorContains(t, err, fmt.Sprintf("%d", tc.retryCount), tc.name) - require.ErrorIs(t, err, errSucceedAfter, tc.name) + require.ErrorIs(t, err, errSentinelCause, tc.name) } else { // !tc.neverSucceed && succeedAfter <= retryCount require.NoError(t, err, tc.name) } diff --git a/conduit/pipeline/pipeline_bench_test.go b/conduit/pipeline/pipeline_bench_test.go index 833ae8f5..882d8592 100644 --- a/conduit/pipeline/pipeline_bench_test.go +++ b/conduit/pipeline/pipeline_bench_test.go @@ -289,7 +289,6 @@ func BenchmarkPipeline(b *testing.B) { } secs := b.Elapsed().Seconds() rps := float64(rounds) / secs - // fmt.Printf("benchmark warmup results. N: %d, elapsed: %f, rounds/sec: %f\n", b.N, secs, rps) b.ReportMetric(rps, "rounds/sec") }) } From 62da700481cbf19455a0cc59b301446a552840dc Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 26 Jul 2023 09:43:36 -0500 Subject: [PATCH 07/50] lint --- conduit/pipeline/common.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/conduit/pipeline/common.go b/conduit/pipeline/common.go index a15c2ee4..67b4abdc 100644 --- a/conduit/pipeline/common.go +++ b/conduit/pipeline/common.go @@ -6,8 +6,9 @@ import ( "fmt" "time" - "github.com/algorand/conduit/conduit/data" log "github.com/sirupsen/logrus" + + "github.com/algorand/conduit/conduit/data" ) // HandlePanic function to log panics in a common way From 0aeb4835f86956a1b282791a90514c00cea4d9bb Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 26 Jul 2023 14:37:16 -0500 Subject: [PATCH 08/50] wip --- conduit/pipeline/pipeline.go | 3 +- conduit/pipeline/pipeline_bench_test.go | 48 ++++++++++++------------- 2 files changed, 25 insertions(+), 26 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 8dab7624..53c7890b 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -537,7 +537,7 @@ func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkI // saving the metadata, and invoking the activated plugin callbacks. If the context's cancellation is received // before the exporter's Receive method has succeeded, cancel immediately. // However, after the exporter's Receive() method has succeeded we try every component at least once -// but will abort if a failure occurs at when a cancellation is received. +// but will abort if a failure occurs after a cancellation is received. func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-chan data.BlockData) { p.wg.Add(1) go func() { @@ -572,7 +572,6 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch p.logger.Tracef("exporter handler received block data for round %d after wait of %s", lastRnd, waitTime) lastRnd = blkData.Round() - // TODO: the next level of concurrency will require an atomic NextRound counter if p.pipelineMetadata.NextRound != lastRnd { lastError = fmt.Errorf("aborting after out of order block data. %d != %d", p.pipelineMetadata.NextRound, lastRnd) return diff --git a/conduit/pipeline/pipeline_bench_test.go b/conduit/pipeline/pipeline_bench_test.go index 882d8592..5b48bc7f 100644 --- a/conduit/pipeline/pipeline_bench_test.go +++ b/conduit/pipeline/pipeline_bench_test.go @@ -248,36 +248,36 @@ func finalRound(pi *pipelineImpl) (sdk.Round, error) { func BenchmarkPipeline(b *testing.B) { benchCases := []benchmarkCase{ - // { - // name: "vanilla 2 procs without sleep", - // importerSleep: 0, - // processorsSleep: []time.Duration{0, 0}, - // exporterSleep: 0, - // }, - // { - // name: "uniform sleep of 10ms", - // importerSleep: 10 * time.Millisecond, - // processorsSleep: []time.Duration{10 * time.Millisecond, 10 * time.Millisecond}, - // exporterSleep: 10 * time.Millisecond, - // }, + { + name: "vanilla 2 procs without sleep", + importerSleep: 0, + processorsSleep: []time.Duration{0, 0}, + exporterSleep: 0, + }, + { + name: "uniform sleep of 10ms", + importerSleep: 10 * time.Millisecond, + processorsSleep: []time.Duration{10 * time.Millisecond, 10 * time.Millisecond}, + exporterSleep: 10 * time.Millisecond, + }, { name: "exporter 10ms while others 1ms", importerSleep: time.Millisecond, processorsSleep: []time.Duration{time.Millisecond, time.Millisecond}, exporterSleep: 10 * time.Millisecond, }, - // { - // name: "importer 10ms while others 1ms", - // importerSleep: 10 * time.Millisecond, - // processorsSleep: []time.Duration{time.Millisecond, time.Millisecond}, - // exporterSleep: time.Millisecond, - // }, - // { - // name: "first processor 10ms while others 1ms", - // importerSleep: time.Millisecond, - // processorsSleep: []time.Duration{10 * time.Millisecond, time.Millisecond}, - // exporterSleep: time.Millisecond, - // }, + { + name: "importer 10ms while others 1ms", + importerSleep: 10 * time.Millisecond, + processorsSleep: []time.Duration{time.Millisecond, time.Millisecond}, + exporterSleep: time.Millisecond, + }, + { + name: "first processor 10ms while others 1ms", + importerSleep: time.Millisecond, + processorsSleep: []time.Duration{10 * time.Millisecond, time.Millisecond}, + exporterSleep: time.Millisecond, + }, } for _, buffSize := range []int{1} { for _, bc := range benchCases { From 3ce28b8f8407271d5706a509ad13869c41074083 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 26 Jul 2023 20:29:47 -0500 Subject: [PATCH 09/50] fix mocking tests --- conduit/pipeline/pipeline_test.go | 81 +++++++++++++++++++++++-------- 1 file changed, 61 insertions(+), 20 deletions(-) diff --git a/conduit/pipeline/pipeline_test.go b/conduit/pipeline/pipeline_test.go index 9feda9ab..01fe119a 100644 --- a/conduit/pipeline/pipeline_test.go +++ b/conduit/pipeline/pipeline_test.go @@ -34,10 +34,14 @@ import ( "github.com/algorand/conduit/conduit/telemetry" ) +const ( + initialRound = 1337 +) + // a unique block data to validate with tests var uniqueBlockData = data.BlockData{ BlockHeader: sdk.BlockHeader{ - Round: 1337, + Round: initialRound, }, } @@ -52,6 +56,8 @@ type mockImporter struct { subsystem string rndOverride uint64 rndReqErr error + getBlockCalls int + onCompleteCalls int } func (m *mockImporter) Init(_ context.Context, _ data.InitProvider, cfg plugins.PluginConfig, _ *log.Logger) error { @@ -77,8 +83,12 @@ func (m *mockImporter) GetBlock(rnd uint64) (data.BlockData, error) { err = fmt.Errorf("importer") } m.Called(rnd) - // Return an error to make sure we - return uniqueBlockData, err + m.getBlockCalls++ + return data.BlockData{ + BlockHeader: sdk.BlockHeader{ + Round: sdk.Round(rnd), + }, + }, err } func (m *mockImporter) OnComplete(input data.BlockData) error { @@ -88,6 +98,7 @@ func (m *mockImporter) OnComplete(input data.BlockData) error { } m.finalRound = sdk.Round(input.BlockHeader.Round) m.Called(input) + m.onCompleteCalls++ return err } @@ -109,6 +120,8 @@ type mockProcessor struct { onCompleteError bool rndOverride uint64 rndReqErr error + processCalls int + onCompleteCalls int } func (m *mockProcessor) Init(_ context.Context, _ data.InitProvider, cfg plugins.PluginConfig, _ *log.Logger) error { @@ -135,8 +148,8 @@ func (m *mockProcessor) Process(input data.BlockData) (data.BlockData, error) { if m.returnError { err = fmt.Errorf("process") } + m.processCalls++ m.Called(input) - input.BlockHeader.Round++ return input, err } @@ -146,6 +159,7 @@ func (m *mockProcessor) OnComplete(input data.BlockData) error { err = fmt.Errorf("on complete") } m.finalRound = sdk.Round(input.BlockHeader.Round) + m.onCompleteCalls++ m.Called(input) return err } @@ -159,6 +173,8 @@ type mockExporter struct { onCompleteError bool rndOverride uint64 rndReqErr error + receiveCalls int + onCompleteCalls int } func (m *mockExporter) Metadata() plugins.Metadata { @@ -185,6 +201,7 @@ func (m *mockExporter) Receive(exportData data.BlockData) error { if m.returnError { err = fmt.Errorf("receive") } + m.receiveCalls++ m.Called(exportData) return err } @@ -195,6 +212,7 @@ func (m *mockExporter) OnComplete(input data.BlockData) error { err = fmt.Errorf("on complete") } m.finalRound = sdk.Round(input.BlockHeader.Round) + m.onCompleteCalls++ m.Called(input) return err } @@ -249,33 +267,33 @@ func mockPipeline(t *testing.T, dataDir string) (*pipelineImpl, *test.Hook, *moc // TestPipelineRun tests that running the pipeline calls the correct functions with mocking func TestPipelineRun(t *testing.T) { - mImporter := mockImporter{} - mImporter.On("GetBlock", mock.Anything).Return(uniqueBlockData, nil) + mImporter := mockImporter{rndOverride: initialRound + 1} + mImporter.On("GetBlock", mock.Anything).Return(mock.Anything, nil) + mImporter.On("OnComplete", mock.Anything).Return(nil) mProcessor := mockProcessor{} processorData := uniqueBlockData processorData.BlockHeader.Round++ - mProcessor.On("Process", mock.Anything).Return(processorData) + mProcessor.On("Process", mock.Anything).Return(mock.Anything) mProcessor.On("OnComplete", mock.Anything).Return(nil) mExporter := mockExporter{} mExporter.On("Receive", mock.Anything).Return(nil) + mExporter.On("OnComplete", mock.Anything).Return(nil) var pImporter importers.Importer = &mImporter var pProcessor processors.Processor = &mProcessor var pExporter exporters.Exporter = &mExporter - var cbComplete conduit.Completed = &mProcessor ctx, ccf := context.WithCancelCause(context.Background()) l, _ := test.NewNullLogger() pImpl := pipelineImpl{ - ctx: ctx, - ccf: ccf, - logger: l, - initProvider: nil, - importer: pImporter, - processors: []processors.Processor{pProcessor}, - exporter: pExporter, - completeCallback: []conduit.OnCompleteFunc{cbComplete.OnComplete}, + ctx: ctx, + ccf: ccf, + logger: l, + initProvider: nil, + importer: pImporter, + processors: []processors.Processor{pProcessor}, + exporter: pExporter, pipelineMetadata: state{ NextRound: 0, GenesisHash: "", @@ -283,6 +301,10 @@ func TestPipelineRun(t *testing.T) { cfg: &data.Config{ RetryDelay: 0 * time.Second, RetryCount: math.MaxUint64, + Processors: []data.NameConfigPair{{ + Name: "mockProcessor", + Config: map[string]interface{}{}, + }}, ConduitArgs: &data.Args{ ConduitDataDir: t.TempDir(), }, @@ -294,14 +316,32 @@ func TestPipelineRun(t *testing.T) { ccf(errors.New("testing timeout")) }() + err := pImpl.Init() + assert.NoError(t, err) pImpl.Start() pImpl.Wait() assert.NoError(t, pImpl.Error()) - assert.Equal(t, mProcessor.finalRound, uniqueBlockData.BlockHeader.Round+1) + finishedRounds := mExporter.receiveCalls + assert.Equal(t, finishedRounds, mExporter.onCompleteCalls) + assert.Len(t, mExporter.Calls, 2*finishedRounds) - mock.AssertExpectationsForObjects(t, &mImporter, &mProcessor, &mExporter) + assert.Equal(t, finishedRounds, mProcessor.onCompleteCalls) + assert.LessOrEqual(t, mProcessor.processCalls, finishedRounds+1) // 1 additional Process() in the pipeline @ cancellation + assert.GreaterOrEqual(t, mProcessor.processCalls, finishedRounds) // 1 additional Process() in the pipeline @ cancellation + assert.Len(t, mProcessor.Calls, mProcessor.processCalls+mProcessor.onCompleteCalls) + assert.Equal(t, finishedRounds, mImporter.onCompleteCalls) + assert.LessOrEqual(t, mImporter.getBlockCalls, finishedRounds+2) // 2 additional GetBlock() in the pipeline @ cancellation + assert.GreaterOrEqual(t, mImporter.getBlockCalls, finishedRounds) // 2 additional GetBlock() in the pipeline @ cancellation + assert.Len(t, mImporter.Calls, mImporter.getBlockCalls+mImporter.onCompleteCalls) + + finalRound := sdk.Round(initialRound + finishedRounds) + assert.Equal(t, finalRound, mExporter.finalRound) + assert.Equal(t, finalRound, mProcessor.finalRound) + assert.Equal(t, finalRound, mImporter.finalRound) + + mock.AssertExpectationsForObjects(t, &mImporter, &mProcessor, &mExporter) } // TestPipelineCpuPidFiles tests that cpu and pid files are created when specified @@ -772,9 +812,10 @@ func TestPipelineRetryVariables(t *testing.T) { var pProcessor processors.Processor = &mockProcessor{} var pExporter exporters.Exporter = &mockExporter{} l, hook := test.NewNullLogger() - ctx, cf := context.WithCancel(context.Background()) + ctx, ccf := context.WithCancelCause(context.Background()) pImpl := pipelineImpl{ ctx: ctx, + ccf: ccf, cfg: &data.Config{ RetryCount: testCase.retryCount, RetryDelay: testCase.retryDelay, @@ -818,7 +859,7 @@ func TestPipelineRetryVariables(t *testing.T) { go func() { time.Sleep(maxDuration) if !done { - cf() + ccf(errors.New("test over timeout")) assert.Equal(t, testCase.totalDuration, maxDuration) } }() From 06586e884a427b3db1f47465f0f5317345d163d0 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 26 Jul 2023 21:00:06 -0500 Subject: [PATCH 10/50] WhyStopped --- conduit/pipeline/pipeline.go | 14 +++++++++++--- conduit/pipeline/pipeline_test.go | 5 ++++- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 53c7890b..6a647e73 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -36,6 +36,7 @@ type Pipeline interface { Init() error Start() Stop() + WhyStopped() error Error() error Wait() } @@ -418,6 +419,16 @@ func (p *pipelineImpl) Stop() { } } +// WhyStopped returns nil if a context was never provided or the pipeline +// is yet to have stopped. Otherwise, it returns the cause of the pipeline's +// context cancellation. +func (p *pipelineImpl) WhyStopped() error { + if p.ctx == nil { + return nil + } + return context.Cause(p.ctx) +} + func numInnerTxn(txn sdk.SignedTxnWithAD) int { result := 0 for _, itxn := range txn.ApplyData.EvalDelta.InnerTxns { @@ -549,9 +560,6 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch defer func() { p.logger.Debugf("exporter %s handler exiting. lastRnd=%d totalSelectWait=%dms, totalExportWork=%dms", exporter.Metadata().Name, lastRnd, totalSelectWait.Milliseconds(), totalExportWork.Milliseconds()) - }() - - defer func() { if lastError != nil { err := fmt.Errorf("exporter %s handler (%w) after round %d: %w", exporter.Metadata().Name, errExporterCause, lastRnd, lastError) p.cancel(err) diff --git a/conduit/pipeline/pipeline_test.go b/conduit/pipeline/pipeline_test.go index 01fe119a..5849b559 100644 --- a/conduit/pipeline/pipeline_test.go +++ b/conduit/pipeline/pipeline_test.go @@ -311,9 +311,10 @@ func TestPipelineRun(t *testing.T) { }, } + errTestCancellation := errors.New("test cancellation") go func() { time.Sleep(1 * time.Second) - ccf(errors.New("testing timeout")) + ccf(errTestCancellation) }() err := pImpl.Init() @@ -342,6 +343,8 @@ func TestPipelineRun(t *testing.T) { assert.Equal(t, finalRound, mImporter.finalRound) mock.AssertExpectationsForObjects(t, &mImporter, &mProcessor, &mExporter) + + assert.ErrorIs(t, pImpl.WhyStopped(), errTestCancellation) } // TestPipelineCpuPidFiles tests that cpu and pid files are created when specified From 6173e381882ebf63bda8303c780d1666cbd787f0 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 26 Jul 2023 21:07:46 -0500 Subject: [PATCH 11/50] cancelWithProblem --- conduit/pipeline/pipeline.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 6a647e73..4dad52d1 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -80,7 +80,7 @@ func (p *pipelineImpl) joinError(err error) { p.err = errors.Join(p.err, err) } -func (p *pipelineImpl) cancel(err error) { +func (p *pipelineImpl) cancelWithProblem(err error) { p.ccf(err) p.joinError(err) } @@ -480,7 +480,7 @@ func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-ch blkData, importTime, lastError := Retries(importer.GetBlock, rnd, p, importer.Metadata().Name) if lastError != nil { - p.cancel(fmt.Errorf("importer %s handler (%w): failed to import round %d after %dms: %w", importer.Metadata().Name, errImporterCause, rnd, importTime.Milliseconds(), lastError)) + p.cancelWithProblem(fmt.Errorf("importer %s handler (%w): failed to import round %d after %dms: %w", importer.Metadata().Name, errImporterCause, rnd, importTime.Milliseconds(), lastError)) return } metrics.ImporterTimeSeconds.Observe(importTime.Seconds()) @@ -525,7 +525,7 @@ func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkI blkData, procTime, lastError := Retries(proc.Process, blkData, p, proc.Metadata().Name) if lastError != nil { - p.cancel(fmt.Errorf("processor[%d] %s handler (%w): failed to process round %d after %dms: %w", idx, proc.Metadata().Name, errProcessorCause, lastRnd, procTime.Milliseconds(), lastError)) + p.cancelWithProblem(fmt.Errorf("processor[%d] %s handler (%w): failed to process round %d after %dms: %w", idx, proc.Metadata().Name, errProcessorCause, lastRnd, procTime.Milliseconds(), lastError)) return } metrics.ProcessorTimeSeconds.WithLabelValues(proc.Metadata().Name).Observe(procTime.Seconds()) @@ -562,7 +562,7 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch p.logger.Debugf("exporter %s handler exiting. lastRnd=%d totalSelectWait=%dms, totalExportWork=%dms", exporter.Metadata().Name, lastRnd, totalSelectWait.Milliseconds(), totalExportWork.Milliseconds()) if lastError != nil { err := fmt.Errorf("exporter %s handler (%w) after round %d: %w", exporter.Metadata().Name, errExporterCause, lastRnd, lastError) - p.cancel(err) + p.cancelWithProblem(err) p.logger.Error(err) } }() From 77e267555b97e25b10f841c573a50a1a350cc41c Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 26 Jul 2023 21:50:21 -0500 Subject: [PATCH 12/50] should still work even when there's no processors --- conduit/pipeline/pipeline.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 4dad52d1..dedf272a 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -476,7 +476,7 @@ func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-ch lastRnd = rnd waitTime := time.Since(selectStart) totalSelectWait += waitTime - p.logger.Tracef("importer handler @ round %d received %s", rnd, waitTime) + p.logger.Tracef("importer handler waited %s to receive round %d", waitTime, rnd) blkData, importTime, lastError := Retries(importer.GetBlock, rnd, p, importer.Metadata().Name) if lastError != nil { @@ -632,7 +632,7 @@ func (p *pipelineImpl) Start() { processorBlkInChan = processorBlkOutChan } - p.ExporterHandler(p.exporter, processorBlkOutChan) + p.ExporterHandler(p.exporter, processorBlkInChan) p.wg.Add(1) // Main loop @@ -657,7 +657,7 @@ func (p *pipelineImpl) Start() { }(p.pipelineMetadata.NextRound) <-p.ctx.Done() - // TODO: send a prometheus observation based on context.Cause(ctx) + // TODO: send a prometheus observation based on WhyStopped() } // Start pushes block data through the pipeline From 5940c9f7713f5a70e15717b9c075b0dd557179f1 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 26 Jul 2023 22:46:01 -0500 Subject: [PATCH 13/50] addMetrics --- conduit/pipeline/pipeline.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index dedf272a..b2d0d6e5 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -592,6 +592,13 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch return } metrics.ExporterTimeSeconds.Observe(exportTime.Seconds()) + // Ignore round 0 (which is empty). + if lastRnd > 0 { + // Previously we reported time starting after block fetching is complete + // through the end of the export operation. Now that each plugin is running + // in its own goroutine, report only the time of the export. + addMetrics(blkData, exportTime) + } // Increment Round, update metadata p.pipelineMetadata.NextRound = lastRnd + 1 From 35e635c9eaba4eed5e381becdf64b9fbccf504c6 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Thu, 27 Jul 2023 10:57:39 -0500 Subject: [PATCH 14/50] more originator details in algod_importer errors --- conduit/plugins/importers/algod/algod_importer.go | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/conduit/plugins/importers/algod/algod_importer.go b/conduit/plugins/importers/algod/algod_importer.go index f705428a..ba195d4f 100644 --- a/conduit/plugins/importers/algod/algod_importer.go +++ b/conduit/plugins/importers/algod/algod_importer.go @@ -470,11 +470,10 @@ func (algodImp *algodImporter) getBlockInner(rnd uint64) (data.BlockData, error) nodeRound, err := waitForRoundWithTimeout(algodImp.ctx, algodImp.logger, algodImp.aclient, rnd, waitForRoundTimeout) if err != nil { - // If context has expired. if algodImp.ctx.Err() != nil { - return blk, fmt.Errorf("GetBlock ctx error: %w", err) + return blk, fmt.Errorf("importer algod.GetBlock() ctx cancelled: %w", err) } - algodImp.logger.Errorf(err.Error()) + algodImp.logger.Errorf("importer algod.GetBlock() called waitForRoundWithTimeout: %v", err) return data.BlockData{}, err } start := time.Now() @@ -483,7 +482,7 @@ func (algodImp *algodImporter) getBlockInner(rnd uint64) (data.BlockData, error) dt := time.Since(start) getAlgodRawBlockTimeSeconds.Observe(dt.Seconds()) if err != nil { - algodImp.logger.Errorf("error getting block for round %d: %s", rnd, err.Error()) + algodImp.logger.Errorf("importer algod.GetBlock() error getting block for round %d: %s", rnd, err.Error()) return data.BlockData{}, err } tmpBlk := new(models.BlockResponse) @@ -503,9 +502,9 @@ func (algodImp *algodImporter) getBlockInner(rnd uint64) (data.BlockData, error) delta, err = algodImp.getDelta(rnd) if err != nil { if nodeRound < rnd { - err = fmt.Errorf("ledger state delta not found: node round (%d) is behind required round (%d), ensure follower node has its sync round set to the required round: %w", nodeRound, rnd, err) + err = fmt.Errorf("importer algod.GetBlock() ledger state delta not found: node round (%d) is behind required round (%d), ensure follower node has its sync round set to the required round: %w", nodeRound, rnd, err) } else { - err = fmt.Errorf("ledger state delta not found: node round (%d), required round (%d): verify follower node configuration and ensure follower node has its sync round set to the required round, re-deploying the follower node may be necessary: %w", nodeRound, rnd, err) + err = fmt.Errorf("importer algod.GetBlock() ledger state delta not found: node round (%d), required round (%d): verify follower node configuration and ensure follower node has its sync round set to the required round, re-deploying the follower node may be necessary: %w", nodeRound, rnd, err) } algodImp.logger.Error(err.Error()) return data.BlockData{}, err @@ -523,10 +522,10 @@ func (algodImp *algodImporter) GetBlock(rnd uint64) (data.BlockData, error) { if err != nil { target := &SyncError{} if errors.As(err, &target) { - algodImp.logger.Warnf("Sync error detected, attempting to set the sync round to recover the node: %s", err.Error()) + algodImp.logger.Warnf("importer algod.GetBlock() sync error detected, attempting to set the sync round to recover the node: %s", err.Error()) _, _ = algodImp.aclient.SetSyncRound(rnd).Do(algodImp.ctx) } else { - err = fmt.Errorf("error getting block for round %d, check node configuration: %s", rnd, err) + err = fmt.Errorf("importer algod.GetBlock() error getting block for round %d, check node configuration: %s", rnd, err) algodImp.logger.Errorf(err.Error()) } return data.BlockData{}, err From 15a62dfea0b23f7b0d868f1be39889a0abf0bf13 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Thu, 27 Jul 2023 12:48:50 -0500 Subject: [PATCH 15/50] pass unit test --- .../importers/algod/algod_importer_test.go | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/conduit/plugins/importers/algod/algod_importer_test.go b/conduit/plugins/importers/algod/algod_importer_test.go index 49dc3fd7..95315900 100644 --- a/conduit/plugins/importers/algod/algod_importer_test.go +++ b/conduit/plugins/importers/algod/algod_importer_test.go @@ -3,17 +3,18 @@ package algodimporter import ( "context" "fmt" - "github.com/sirupsen/logrus" - "github.com/sirupsen/logrus/hooks/test" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "gopkg.in/yaml.v3" "net/http" "net/http/httptest" "strings" "testing" "time" + "github.com/sirupsen/logrus" + "github.com/sirupsen/logrus/hooks/test" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "gopkg.in/yaml.v3" + "github.com/algorand/go-algorand-sdk/v2/client/v2/algod" "github.com/algorand/go-algorand-sdk/v2/client/v2/common/models" sdk "github.com/algorand/go-algorand-sdk/v2/types" @@ -700,7 +701,7 @@ func TestGetBlockErrors(t *testing.T) { blockResponder: BlockResponder, deltaResponder: MakeMsgpStatusResponder("get", "/v2/deltas/", http.StatusNotFound, ""), err: fmt.Sprintf("wrong round returned from status for round: 50 != 200"), - logs: []string{"wrong round returned from status for round: 50 != 200", "Sync error detected, attempting to set the sync round to recover the node"}, + logs: []string{"wrong round returned from status for round: 50 != 200", "sync error detected, attempting to set the sync round to recover the node"}, }, { name: "Cannot get delta (caught up)", @@ -755,7 +756,8 @@ func TestGetBlockErrors(t *testing.T) { // Make sure each of the expected log messages are present for _, log := range tc.logs { found := false - for _, entry := range hook.AllEntries() { + hookEntries := hook.AllEntries() + for _, entry := range hookEntries { fmt.Println(strings.Contains(entry.Message, log)) found = found || strings.Contains(entry.Message, log) } From 4956c9fdccc3988e23e85f15d3f8c0a9d2aa26ef Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Thu, 27 Jul 2023 13:40:45 -0500 Subject: [PATCH 16/50] NewSyncError --- .../plugins/importers/algod/algod_importer.go | 25 ++++++++++++------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/conduit/plugins/importers/algod/algod_importer.go b/conduit/plugins/importers/algod/algod_importer.go index ba195d4f..6cc1ba7f 100644 --- a/conduit/plugins/importers/algod/algod_importer.go +++ b/conduit/plugins/importers/algod/algod_importer.go @@ -421,10 +421,23 @@ func (algodImp *algodImporter) getDelta(rnd uint64) (sdk.LedgerStateDelta, error type SyncError struct { rnd uint64 expected uint64 + err error +} + +func NewSyncError(rnd, expected uint64, err error) *SyncError { + return &SyncError{ + rnd: rnd, + expected: expected, + err: err, + } } func (e *SyncError) Error() string { - return fmt.Sprintf("wrong round returned from status for round: %d != %d", e.rnd, e.expected) + return fmt.Sprintf("wrong round returned from status for round: %d != %d: %v", e.rnd, e.expected, e.err) +} + +func (e *SyncError) Unwrap() error { + return e.err } func waitForRoundWithTimeout(ctx context.Context, l *logrus.Logger, c *algod.Client, rnd uint64, to time.Duration) (uint64, error) { @@ -440,10 +453,7 @@ func waitForRoundWithTimeout(ctx context.Context, l *logrus.Logger, c *algod.Cli if rnd <= status.LastRound { return status.LastRound, nil } - return 0, &SyncError{ - rnd: status.LastRound, - expected: rnd, - } + return 0, NewSyncError(status.LastRound, rnd, fmt.Errorf("this check should never be required: %w", err)) } // If there was a different error and the node is responsive, call status before returning a SyncError. @@ -454,10 +464,7 @@ func waitForRoundWithTimeout(ctx context.Context, l *logrus.Logger, c *algod.Cli return 0, fmt.Errorf("unable to get status after block and status: %w", errors.Join(err, err2)) } if status2.LastRound < rnd { - return 0, &SyncError{ - rnd: status.LastRound, - expected: rnd, - } + return 0, NewSyncError(status2.LastRound, rnd, fmt.Errorf("status2.LastRound mismatch: %w", err)) } // This is probably a connection error, not a SyncError. From 579f2e4ee83ec1c4d9213f8d8d020e7e304e0713 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Thu, 27 Jul 2023 13:43:03 -0500 Subject: [PATCH 17/50] lint --- conduit/plugins/importers/algod/algod_importer.go | 1 + 1 file changed, 1 insertion(+) diff --git a/conduit/plugins/importers/algod/algod_importer.go b/conduit/plugins/importers/algod/algod_importer.go index 6cc1ba7f..fae0dc1b 100644 --- a/conduit/plugins/importers/algod/algod_importer.go +++ b/conduit/plugins/importers/algod/algod_importer.go @@ -424,6 +424,7 @@ type SyncError struct { err error } +// NewSyncError creates a new SyncError. func NewSyncError(rnd, expected uint64, err error) *SyncError { return &SyncError{ rnd: rnd, From 94b7bd80ce57a7e8970e92a2b139bce37df59e20 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Fri, 28 Jul 2023 11:34:22 -0500 Subject: [PATCH 18/50] bring back E2E finish signaller --- conduit/pipeline/pipeline.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index b2d0d6e5..1c96a5e7 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -618,6 +618,9 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch } } lastError = nil + // WARNING: removing the following will BREAK the E2E test. + // Modify with CAUTION. (Search for "Pipeline round:" in subslurp.py) + p.logger.Infof("FINISHED Pipeline round: %v", p.pipelineMetadata.NextRound) } } }() @@ -650,6 +653,7 @@ func (p *pipelineImpl) Start() { defer p.logger.Debugf("round channel feed exiting. lastRnd=%d totalFeedWait=%dms", rnd, totalFeedWait.Milliseconds()) for { selectStart := time.Now() + p.logger.Infof("Pipeline round kickoff: %v", rnd) p.logger.Tracef("pushing round %d into roundChan", rnd) select { case <-p.ctx.Done(): From cd7e53c976944aeeb46bce9f3bb331a84235e3d7 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Sat, 29 Jul 2023 00:00:25 -0500 Subject: [PATCH 19/50] does e2e pass in CI? --- e2e_tests/src/e2e_conduit/runner.py | 3 +- e2e_tests/src/e2e_conduit/subslurp.py | 52 +++++++++++++++++++-------- pkg/cli/cli.go | 2 +- 3 files changed, 40 insertions(+), 17 deletions(-) diff --git a/e2e_tests/src/e2e_conduit/runner.py b/e2e_tests/src/e2e_conduit/runner.py index 5692edc5..a9dbde0c 100644 --- a/e2e_tests/src/e2e_conduit/runner.py +++ b/e2e_tests/src/e2e_conduit/runner.py @@ -47,6 +47,7 @@ def setup_scenario(self, scenario): yaml.dump( { "log-level": "info", + "retry-count": 1, "importer": { "name": scenario.importer.name, "config": scenario.importer.config_input, @@ -91,7 +92,7 @@ def run_scenario(self, scenario): return 1 if indexerout.round < scenario.importer.lastblock: - logger.error("conduit did not reach round={scenario.importer.lastblock}") + logger.error(f"conduit did not reach round={scenario.importer.lastblock}") sys.stderr.write(indexerout.dump()) return 1 diff --git a/e2e_tests/src/e2e_conduit/subslurp.py b/e2e_tests/src/e2e_conduit/subslurp.py index e8f50bdf..8a521035 100644 --- a/e2e_tests/src/e2e_conduit/subslurp.py +++ b/e2e_tests/src/e2e_conduit/subslurp.py @@ -6,29 +6,46 @@ logger = logging.getLogger(__name__) +# Matches conduit log output: +# "FINISHED Pipeline round: 110. UPDATED Pipeline round: 111" +FINISH_ROUND: re.Pattern = re.compile( + b"FINISHED Pipeline round: (\d+). UPDATED Pipeline round" +) + +# Matches error from attempting to sync past network's final round: +# "waitForRoundWithTimeout: wrong round returned from status for round: 102 != 103: status2.LastRound mismatch: context deadline exceeded" +# END_OF_TEST_SYNC_MISMATCH: re.Pattern = re.compile( +# b"wrong round returned from status for round: (\d+) != (\d+).*context deadline exceeded" +# ) + class subslurp: """accumulate stdout or stderr from a subprocess and hold it for debugging if something goes wrong""" + def __init__(self, f): self.f = f self.buf = io.BytesIO() self.gz = gzip.open(self.buf, "wb") self.timeout = timedelta(seconds=120) - # Matches conduit log output: "Pipeline round: 110" - self.round_re = re.compile(b'.*"Pipeline round: ([0-9]+)"') self.round = 0 self.error_log = None - def logIsError(self, log_line): - if b"error" in log_line: - self.error_log = log_line - return True - return False + # def is_log_error(self, log_line): + # if b"error" in log_line: + # # match = re.search(END_OF_TEST_SYNC_MISMATCH, log_line) + # # if match: + # # x, y = list(map(int, match.groups())) + # # if x + 1 == y and x == lastround: + # # return False + + # self.error_log = log_line + # return True + # return False def tryParseRound(self, log_line): - m = self.round_re.match(log_line) - if m is not None and m.group(1) is not None: - self.round = int(m.group(1)) + match = FINISH_ROUND.search(log_line) + if match and (r := int(match.group(1))) is not None: + self.round = r def run(self, lastround): if len(self.f.peek().strip()) == 0: @@ -41,16 +58,21 @@ def run(self, lastround): datetime.now() - start < self.timeout and datetime.now() - lastlog < timedelta(seconds=15) ): - for line in self.f: + for i, line in enumerate(self.f): lastlog = datetime.now() if self.gz is not None: self.gz.write(line) self.tryParseRound(line) if self.round >= lastround: - logger.info(f"Conduit reached desired lastround: {lastround}") + logger.info( + f"Conduit reached desired lastround after parsing line {i+1}: {lastround}" + ) return - if self.logIsError(line): - raise RuntimeError(f"E2E tests logged an error: {self.error_log}") + if b"error" in line: + # NOTE this quite very strict criterion!!! + raise RuntimeError( + f"E2E tests logged an error at line {i+1}: {self.error_log}" + ) def dump(self) -> str: if self.gz is not None: @@ -58,4 +80,4 @@ def dump(self) -> str: self.gz = None self.buf.seek(0) r = gzip.open(self.buf, "rt") - return r.read() # type: ignore + return r.read() # type: ignore diff --git a/pkg/cli/cli.go b/pkg/cli/cli.go index 840af6f5..934d3b9e 100644 --- a/pkg/cli/cli.go +++ b/pkg/cli/cli.go @@ -134,7 +134,7 @@ Detailed documentation is online: https://github.com/algorand/conduit`, Run: func(cmd *cobra.Command, args []string) { err := runConduitCmdWithConfig(cfg) if err != nil { - fmt.Fprintf(os.Stderr, "\nExiting with error:\n%s.\n", err) + fmt.Fprintf(os.Stderr, "\nExiting with error:\t%s.\n", err) os.Exit(1) } }, From 4c1756f401b78b3627e63045a5599076ffa45597 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Sun, 30 Jul 2023 21:20:22 -0500 Subject: [PATCH 20/50] finer logging granularity --- conduit/data/config.go | 11 +++++++++++ conduit/loggers/loggers.go | 2 ++ conduit/pipeline/logging.go | 3 +++ conduit/pipeline/pipeline.go | 31 ++++++++++++++++--------------- 4 files changed, 32 insertions(+), 15 deletions(-) diff --git a/conduit/data/config.go b/conduit/data/config.go index 62668003..7376ce69 100644 --- a/conduit/data/config.go +++ b/conduit/data/config.go @@ -10,6 +10,17 @@ import ( yaml "gopkg.in/yaml.v3" ) +const ( + // ConduitTimeFormat is the time format used by conduit's logger. + // time.RFC3339Nano = "2006-01-02T15:04:05.999999999Z07:00" + // Compare with logrus' default time format: + // time.RFC1123Z = "Mon, 02 Jan 2006 15:04:05 -0700" + ConduitTimeFormat = time.RFC3339Nano + + // Unfortunately, variations on this theme didn't work: + // ConduitTimeFormat = "Mon, 02 Jan 2006 15:04:05.123456789 -0700" +) + // DefaultConfigBaseName is the default conduit configuration filename without the extension. var DefaultConfigBaseName = "conduit" diff --git a/conduit/loggers/loggers.go b/conduit/loggers/loggers.go index 77c8fd02..31b1979d 100644 --- a/conduit/loggers/loggers.go +++ b/conduit/loggers/loggers.go @@ -8,6 +8,7 @@ import ( log "github.com/sirupsen/logrus" + "github.com/algorand/conduit/conduit/data" "github.com/algorand/conduit/conduit/pipeline" ) @@ -16,6 +17,7 @@ func MakeThreadSafeLoggerWithWriter(level log.Level, writer io.Writer) *log.Logg formatter := pipeline.PluginLogFormatter{ Formatter: &log.JSONFormatter{ DisableHTMLEscape: true, + TimestampFormat: data.ConduitTimeFormat, }, Type: "Conduit", Name: "main", diff --git a/conduit/pipeline/logging.go b/conduit/pipeline/logging.go index 40a628c9..0a8e31d6 100644 --- a/conduit/pipeline/logging.go +++ b/conduit/pipeline/logging.go @@ -2,6 +2,8 @@ package pipeline import ( log "github.com/sirupsen/logrus" + + "github.com/algorand/conduit/conduit/data" ) // PluginLogFormatter formats the log message with special conduit tags @@ -23,6 +25,7 @@ func makePluginLogFormatter(pluginType string, pluginName string) PluginLogForma return PluginLogFormatter{ Formatter: &log.JSONFormatter{ DisableHTMLEscape: true, + TimestampFormat: data.ConduitTimeFormat, }, Type: pluginType, Name: pluginName, diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 1c96a5e7..188c557d 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -554,46 +554,46 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch go func() { defer p.wg.Done() var lastError error - lastRnd := uint64(0) + lastRound := uint64(0) totalSelectWait := time.Duration(0) totalExportWork := time.Duration(0) defer func() { - p.logger.Debugf("exporter %s handler exiting. lastRnd=%d totalSelectWait=%dms, totalExportWork=%dms", exporter.Metadata().Name, lastRnd, totalSelectWait.Milliseconds(), totalExportWork.Milliseconds()) + p.logger.Debugf("exporter %s handler exiting. lastRnd=%d totalSelectWait=%dms, totalExportWork=%dms", exporter.Metadata().Name, lastRound, totalSelectWait.Milliseconds(), totalExportWork.Milliseconds()) if lastError != nil { - err := fmt.Errorf("exporter %s handler (%w) after round %d: %w", exporter.Metadata().Name, errExporterCause, lastRnd, lastError) + err := fmt.Errorf("exporter %s handler (%w) after round %d: %w", exporter.Metadata().Name, errExporterCause, lastRound, lastError) p.cancelWithProblem(err) p.logger.Error(err) } }() for { - p.logger.Tracef("exporter handler waiting for block data. round=%d", lastRnd) + p.logger.Tracef("exporter handler waiting for block data. round=%d", lastRound) selectStart := time.Now() select { case <-p.ctx.Done(): - p.logger.Infof("exporter handler exiting lastRnd=%d", lastRnd) + p.logger.Infof("exporter handler exiting lastRnd=%d", lastRound) return case blkData := <-blkChan: waitTime := time.Since(selectStart) totalSelectWait += waitTime - p.logger.Tracef("exporter handler received block data for round %d after wait of %s", lastRnd, waitTime) - lastRnd = blkData.Round() + p.logger.Tracef("exporter handler received block data for round %d after wait of %s", lastRound, waitTime) + lastRound = blkData.Round() - if p.pipelineMetadata.NextRound != lastRnd { - lastError = fmt.Errorf("aborting after out of order block data. %d != %d", p.pipelineMetadata.NextRound, lastRnd) + if p.pipelineMetadata.NextRound != lastRound { + lastError = fmt.Errorf("aborting after out of order block data. %d != %d", p.pipelineMetadata.NextRound, lastRound) return } var exportTime time.Duration exportTime, lastError = RetriesNoOutput(exporter.Receive, blkData, p, exporter.Metadata().Name) if lastError != nil { - lastError = fmt.Errorf("aborting after failing to export round %d: %w", lastRnd, lastError) + lastError = fmt.Errorf("aborting after failing to export round %d: %w", lastRound, lastError) return } metrics.ExporterTimeSeconds.Observe(exportTime.Seconds()) // Ignore round 0 (which is empty). - if lastRnd > 0 { + if lastRound > 0 { // Previously we reported time starting after block fetching is complete // through the end of the export operation. Now that each plugin is running // in its own goroutine, report only the time of the export. @@ -601,16 +601,17 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch } // Increment Round, update metadata - p.pipelineMetadata.NextRound = lastRnd + 1 + nextRound := lastRound + 1 + p.pipelineMetadata.NextRound = nextRound _, lastError = RetriesNoOutput(p.pipelineMetadata.encodeToFile, p.cfg.ConduitArgs.ConduitDataDir, p, "pipelineMetadata save") if lastError != nil { - lastError = fmt.Errorf("aborting after updating NextRound=%d and failing to save metadata: %w", lastRnd+1, lastError) + lastError = fmt.Errorf("aborting after updating NextRound=%d and failing to save metadata: %w", nextRound, lastError) return } p.logger.Tracef("exporter %s @ round %d saved pipeline metadata", exporter.Metadata().Name, p.pipelineMetadata.NextRound) for i, cb := range p.completeCallback { - p.logger.Tracef("exporter %s @ round=%d NextRound=%d executing callback %d", exporter.Metadata().Name, lastRnd, p.pipelineMetadata.NextRound, i) + p.logger.Tracef("exporter %s @ round=%d NextRound=%d executing callback %d", exporter.Metadata().Name, lastRound, p.pipelineMetadata.NextRound, i) _, lastError = RetriesNoOutput(cb, blkData, p, fmt.Sprintf("callback %d", i)) if lastError != nil { lastError = fmt.Errorf("aborting due to failed callback %d: %w", i, lastError) @@ -620,7 +621,7 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch lastError = nil // WARNING: removing the following will BREAK the E2E test. // Modify with CAUTION. (Search for "Pipeline round:" in subslurp.py) - p.logger.Infof("FINISHED Pipeline round: %v", p.pipelineMetadata.NextRound) + p.logger.Infof("FINISHED Pipeline round: %d. UPDATED Pipeline round: %d", lastRound, nextRound) } } }() From c332cfc22ae1351539d8901aed84d23866941aca Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Mon, 31 Jul 2023 11:16:48 -0500 Subject: [PATCH 21/50] comments cleanup --- conduit/pipeline/common.go | 2 +- e2e_tests/src/e2e_conduit/subslurp.py | 18 ------------------ 2 files changed, 1 insertion(+), 19 deletions(-) diff --git a/conduit/pipeline/common.go b/conduit/pipeline/common.go index 67b4abdc..4943f953 100644 --- a/conduit/pipeline/common.go +++ b/conduit/pipeline/common.go @@ -34,7 +34,7 @@ type pluginInput interface { // - when > 0, the function will retry p.cfg.retryCount times before giving up // // - Upon success: -// - a nil error is returned even if there were many intermediate failures +// - a nil error is returned even if there were intermediate failures // - the returned duration dur measures the time spent in the call to f() that succeeded // // - Upon failure: diff --git a/e2e_tests/src/e2e_conduit/subslurp.py b/e2e_tests/src/e2e_conduit/subslurp.py index 8a521035..7dc06254 100644 --- a/e2e_tests/src/e2e_conduit/subslurp.py +++ b/e2e_tests/src/e2e_conduit/subslurp.py @@ -12,12 +12,6 @@ b"FINISHED Pipeline round: (\d+). UPDATED Pipeline round" ) -# Matches error from attempting to sync past network's final round: -# "waitForRoundWithTimeout: wrong round returned from status for round: 102 != 103: status2.LastRound mismatch: context deadline exceeded" -# END_OF_TEST_SYNC_MISMATCH: re.Pattern = re.compile( -# b"wrong round returned from status for round: (\d+) != (\d+).*context deadline exceeded" -# ) - class subslurp: """accumulate stdout or stderr from a subprocess and hold it for debugging if something goes wrong""" @@ -30,18 +24,6 @@ def __init__(self, f): self.round = 0 self.error_log = None - # def is_log_error(self, log_line): - # if b"error" in log_line: - # # match = re.search(END_OF_TEST_SYNC_MISMATCH, log_line) - # # if match: - # # x, y = list(map(int, match.groups())) - # # if x + 1 == y and x == lastround: - # # return False - - # self.error_log = log_line - # return True - # return False - def tryParseRound(self, log_line): match = FINISH_ROUND.search(log_line) if match and (r := int(match.group(1))) is not None: From 2aae293d50de00f97542f41c60200a3bea68e5e4 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Mon, 31 Jul 2023 11:22:59 -0500 Subject: [PATCH 22/50] commentary --- e2e_tests/src/e2e_conduit/subslurp.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_tests/src/e2e_conduit/subslurp.py b/e2e_tests/src/e2e_conduit/subslurp.py index 7dc06254..ed5e134f 100644 --- a/e2e_tests/src/e2e_conduit/subslurp.py +++ b/e2e_tests/src/e2e_conduit/subslurp.py @@ -51,7 +51,7 @@ def run(self, lastround): ) return if b"error" in line: - # NOTE this quite very strict criterion!!! + # NOTE this quite strict criterion!!! raise RuntimeError( f"E2E tests logged an error at line {i+1}: {self.error_log}" ) From 4f5576352b6cad9eee190883f5af0b4dbe1e2cc4 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 2 Aug 2023 14:27:14 -0500 Subject: [PATCH 23/50] conform logs with internal-tools logstats.go --- conduit/pipeline/pipeline.go | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 188c557d..e26b4250 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -476,7 +476,7 @@ func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-ch lastRnd = rnd waitTime := time.Since(selectStart) totalSelectWait += waitTime - p.logger.Tracef("importer handler waited %s to receive round %d", waitTime, rnd) + p.logger.Tracef("importer handler waited %dms to receive round %d", waitTime.Milliseconds(), rnd) blkData, importTime, lastError := Retries(importer.GetBlock, rnd, p, importer.Metadata().Name) if lastError != nil { @@ -549,7 +549,7 @@ func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkI // before the exporter's Receive method has succeeded, cancel immediately. // However, after the exporter's Receive() method has succeeded we try every component at least once // but will abort if a failure occurs after a cancellation is received. -func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-chan data.BlockData) { +func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-chan data.BlockData, startStart time.Time) { p.wg.Add(1) go func() { defer p.wg.Done() @@ -619,9 +619,11 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch } } lastError = nil - // WARNING: removing the following will BREAK the E2E test. - // Modify with CAUTION. (Search for "Pipeline round:" in subslurp.py) - p.logger.Infof("FINISHED Pipeline round: %d. UPDATED Pipeline round: %d", lastRound, nextRound) + // WARNING: removing the following will BREAK: + // - the E2E test (Search for "Pipeline round:" in subslurp.py) + // - the internal tools logstats collector (See func ConduitCollector in logstats.go of internal-tools repo) + // Modify with CAUTION!!!! + p.logger.Infof("UPDATED Pipeline NextRound=%d. FINISHED Pipeline round r=%d (%d txn) exported in %s", nextRound, lastRound, len(blkData.Payset), time.Since(startStart)) } } }() @@ -629,6 +631,7 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch // Start pushes block data through the pipeline func (p *pipelineImpl) Start() { + startStart := time.Now() p.logger.Trace("Pipeline.Start()") // Setup channels @@ -643,7 +646,7 @@ func (p *pipelineImpl) Start() { processorBlkInChan = processorBlkOutChan } - p.ExporterHandler(p.exporter, processorBlkInChan) + p.ExporterHandler(p.exporter, processorBlkInChan, startStart) p.wg.Add(1) // Main loop From 2c2e38702b4b647b0bcac64ae112126cd0f23202 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 2 Aug 2023 17:12:58 -0500 Subject: [PATCH 24/50] changes after CR discussion --- conduit/pipeline/common.go | 26 +++++++++++--------------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/conduit/pipeline/common.go b/conduit/pipeline/common.go index 4943f953..4c31a3ff 100644 --- a/conduit/pipeline/common.go +++ b/conduit/pipeline/common.go @@ -2,7 +2,6 @@ package pipeline import ( "context" - "errors" "fmt" "time" @@ -21,7 +20,11 @@ func HandlePanic(logger *log.Logger) { type empty struct{} type pluginInput interface { - uint64 | data.BlockData | string | empty + uint64 | data.BlockData | string +} + +type pluginOutput interface { + pluginInput | empty } // Retries is a wrapper for retrying a function call f() with a cancellation context, @@ -42,7 +45,7 @@ type pluginInput interface { // - when p.cfg.retryCount > 0, the error will be a join of all the errors encountered during the retries // - when p.cfg.retryCount == 0, the error will be the last error encountered // - the returned duration dur is the total time spent in the function, including retries -func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg string) (y Y, dur time.Duration, err error) { +func Retries[X pluginInput, Y pluginOutput](f func(x X) (Y, error), x X, p *pipelineImpl, msg string) (y Y, dur time.Duration, err error) { start := time.Now() for i := uint64(0); p.cfg.RetryCount == 0 || i <= p.cfg.RetryCount; i++ { @@ -58,19 +61,12 @@ func Retries[X, Y pluginInput](f func(x X) (Y, error), x X, p *pipelineImpl, msg } } opStart := time.Now() - y2, err2 := f(x) - if err2 == nil { - return y2, time.Since(opStart), nil - } - - p.logger.Infof("%s: retry number %d/%d with err: %v", msg, i, p.cfg.RetryCount, err2) - if p.cfg.RetryCount > 0 { - // TODO: this feels like a code smell. Probly better to always keep only the last error. - err = errors.Join(err, err2) - } else { - // in the case of infinite retries, only keep the last error - err = err2 + y, err = f(x) + if err == nil { + dur = time.Since(opStart) + return } + p.logger.Infof("%s: retry number %d/%d with err: %v", msg, i, p.cfg.RetryCount, err) } dur = time.Since(start) From d2e0d3da5b7403c830d87fca3cf5307cb206cedb Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 2 Aug 2023 17:20:54 -0500 Subject: [PATCH 25/50] per CR --- conduit/pipeline/pipeline_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/conduit/pipeline/pipeline_test.go b/conduit/pipeline/pipeline_test.go index 5849b559..31842cea 100644 --- a/conduit/pipeline/pipeline_test.go +++ b/conduit/pipeline/pipeline_test.go @@ -271,8 +271,6 @@ func TestPipelineRun(t *testing.T) { mImporter.On("GetBlock", mock.Anything).Return(mock.Anything, nil) mImporter.On("OnComplete", mock.Anything).Return(nil) mProcessor := mockProcessor{} - processorData := uniqueBlockData - processorData.BlockHeader.Round++ mProcessor.On("Process", mock.Anything).Return(mock.Anything) mProcessor.On("OnComplete", mock.Anything).Return(nil) mExporter := mockExporter{} From b2e67de6c5147b57a37b5e9334e6214489ba90ab Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 2 Aug 2023 17:23:35 -0500 Subject: [PATCH 26/50] explanatory comment per CR discussion --- conduit/pipeline/common_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/conduit/pipeline/common_test.go b/conduit/pipeline/common_test.go index f55b69e6..cd9dcd4a 100644 --- a/conduit/pipeline/common_test.go +++ b/conduit/pipeline/common_test.go @@ -147,6 +147,9 @@ func TestRetries(t *testing.T) { y, _, err := Retries(succeedAfter, 0, p, "test") if tc.retryCount == 0 { // WLOG tc.neverSucceed == false require.NoError(t, err, tc.name) + + // note we subtract 1 from y below because succeedAfter has added 1 to its output + // to disambiguate with the zero value which occurs on failure require.Equal(t, tc.succeedAfter, y-1, tc.name) } else { // retryCount > 0 so doesn't retry forever if tc.neverSucceed || tc.succeedAfter > tc.retryCount { From deec57e61df93b3293e03eae9871ae8464785418 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 2 Aug 2023 17:33:21 -0500 Subject: [PATCH 27/50] rename the sentinel cause and ignore lint warnings --- conduit/pipeline/pipeline.go | 4 ++-- e2e_tests/src/e2e_conduit/subslurp.py | 6 ++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index e26b4250..c294956f 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -62,7 +62,7 @@ type pipelineImpl struct { } var ( - errStopCause = errors.New("pipeline stopped") + stopCause = errors.New("pipeline stopped") //nolint:revive // this is a sentinel error errImporterCause = errors.New("importer cancelled") errProcessorCause = errors.New("processor cancelled") errExporterCause = errors.New("exporter cancelled") @@ -386,7 +386,7 @@ func (p *pipelineImpl) Init() error { } func (p *pipelineImpl) Stop() { - p.ccf(errStopCause) + p.ccf(stopCause) p.wg.Wait() if p.profFile != nil { diff --git a/e2e_tests/src/e2e_conduit/subslurp.py b/e2e_tests/src/e2e_conduit/subslurp.py index ed5e134f..23fc12b5 100644 --- a/e2e_tests/src/e2e_conduit/subslurp.py +++ b/e2e_tests/src/e2e_conduit/subslurp.py @@ -7,10 +7,8 @@ logger = logging.getLogger(__name__) # Matches conduit log output: -# "FINISHED Pipeline round: 110. UPDATED Pipeline round: 111" -FINISH_ROUND: re.Pattern = re.compile( - b"FINISHED Pipeline round: (\d+). UPDATED Pipeline round" -) +# "UPDATED Pipeline NextRound=111. FINISHED Pipeline round r=110 (23 txn) exported in 1.23435sec" +FINISH_ROUND: re.Pattern = re.compile(b"FINISHED Pipeline round r=(\d+)") class subslurp: From 4103de1c50b8ef0c89af9ec3197437b9808b067f Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 2 Aug 2023 18:21:07 -0500 Subject: [PATCH 28/50] adding PipelineData to data.BlockData --- conduit/data/block_export_data.go | 13 ++++++++++- conduit/pipeline/pipeline.go | 33 ++++++++++++++++++++------- e2e_tests/src/e2e_conduit/subslurp.py | 2 +- 3 files changed, 38 insertions(+), 10 deletions(-) diff --git a/conduit/data/block_export_data.go b/conduit/data/block_export_data.go index a8cb9805..27124950 100644 --- a/conduit/data/block_export_data.go +++ b/conduit/data/block_export_data.go @@ -1,6 +1,8 @@ package data import ( + "time" + sdk "github.com/algorand/go-algorand-sdk/v2/types" "github.com/algorand/conduit/conduit/telemetry" @@ -21,9 +23,15 @@ type InitProvider interface { GetTelemetryClient() telemetry.Client } +// PipelineData is used to keep track of pipeline performance +type PipelineData struct { + //StartRoundTime is the time the pipeline started processing the block + StartRoundTime time.Time `json:"startRoundTime,omitempty"` + FinishImportTime time.Time `json:"finishImportTime,omitempty"` +} + // BlockData is provided to the Exporter on each round. type BlockData struct { - // BlockHeader is the immutable header from the block BlockHeader sdk.BlockHeader `json:"block,omitempty"` @@ -35,6 +43,9 @@ type BlockData struct { // Certificate contains voting data that certifies the block. The certificate is non deterministic, a node stops collecting votes once the voting threshold is reached. Certificate *map[string]interface{} `json:"cert,omitempty"` + + // PipelineData is used to keep track of pipeline performance + PipelineData } // Round returns the round to which the BlockData corresponds diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index c294956f..df09e1f4 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -467,14 +467,14 @@ func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-ch }() for { - selectStart := time.Now() + startRound := time.Now() select { case <-p.ctx.Done(): p.logger.Infof("importer handler exiting. lastRnd=%d", lastRnd) return case rnd := <-roundChan: lastRnd = rnd - waitTime := time.Since(selectStart) + waitTime := time.Since(startRound) totalSelectWait += waitTime p.logger.Tracef("importer handler waited %dms to receive round %d", waitTime.Milliseconds(), rnd) @@ -485,12 +485,23 @@ func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-ch } metrics.ImporterTimeSeconds.Observe(importTime.Seconds()) - feedStart := time.Now() + // TODO: Verify that the block was built with a known protocol version. + + // Start time currently measures operations after block fetching is complete. + // This is for backwards compatibility w/ Indexer's metrics + // run through processors + + importFinish := time.Now() + blkData.PipelineData = data.PipelineData{ + StartRoundTime: startRound, + FinishImportTime: importFinish, + } + select { case <-p.ctx.Done(): return case blkOutChan <- blkData: - waitTime := time.Since(feedStart) + waitTime := time.Since(importFinish) totalFeedWait += waitTime p.logger.Tracef("imported round %d into blkOutChan after waiting %dms on channel", rnd, waitTime.Milliseconds()) } @@ -549,7 +560,7 @@ func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkI // before the exporter's Receive method has succeeded, cancel immediately. // However, after the exporter's Receive() method has succeeded we try every component at least once // but will abort if a failure occurs after a cancellation is received. -func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-chan data.BlockData, startStart time.Time) { +func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-chan data.BlockData) { p.wg.Add(1) go func() { defer p.wg.Done() @@ -623,7 +634,14 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch // - the E2E test (Search for "Pipeline round:" in subslurp.py) // - the internal tools logstats collector (See func ConduitCollector in logstats.go of internal-tools repo) // Modify with CAUTION!!!! - p.logger.Infof("UPDATED Pipeline NextRound=%d. FINISHED Pipeline round r=%d (%d txn) exported in %s", nextRound, lastRound, len(blkData.Payset), time.Since(startStart)) + p.logger.Infof( + "UPDATED Pipeline NextRound=%d after [%s] from round kickoff. FINISHED Pipeline round r=%d (%d txn) exported in %s", + nextRound, + time.Since(blkData.StartRoundTime), + lastRound, + len(blkData.Payset), + time.Since(blkData.FinishImportTime), + ) } } }() @@ -631,7 +649,6 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch // Start pushes block data through the pipeline func (p *pipelineImpl) Start() { - startStart := time.Now() p.logger.Trace("Pipeline.Start()") // Setup channels @@ -646,7 +663,7 @@ func (p *pipelineImpl) Start() { processorBlkInChan = processorBlkOutChan } - p.ExporterHandler(p.exporter, processorBlkInChan, startStart) + p.ExporterHandler(p.exporter, processorBlkInChan) p.wg.Add(1) // Main loop diff --git a/e2e_tests/src/e2e_conduit/subslurp.py b/e2e_tests/src/e2e_conduit/subslurp.py index 23fc12b5..9d4d5dfe 100644 --- a/e2e_tests/src/e2e_conduit/subslurp.py +++ b/e2e_tests/src/e2e_conduit/subslurp.py @@ -7,7 +7,7 @@ logger = logging.getLogger(__name__) # Matches conduit log output: -# "UPDATED Pipeline NextRound=111. FINISHED Pipeline round r=110 (23 txn) exported in 1.23435sec" +# "UPDATED Pipeline NextRound=111 after [123.45ms] from round kickoff. FINISHED Pipeline round r=110 (23 txn) exported in 78.567ms", FINISH_ROUND: re.Pattern = re.compile(b"FINISHED Pipeline round r=(\d+)") From a0edb98908ce76095f619d10f09707c0835f2fcf Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 2 Aug 2023 18:22:58 -0500 Subject: [PATCH 29/50] lint --- conduit/data/block_export_data.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/conduit/data/block_export_data.go b/conduit/data/block_export_data.go index 27124950..8ba4f122 100644 --- a/conduit/data/block_export_data.go +++ b/conduit/data/block_export_data.go @@ -26,7 +26,9 @@ type InitProvider interface { // PipelineData is used to keep track of pipeline performance type PipelineData struct { //StartRoundTime is the time the pipeline started processing the block - StartRoundTime time.Time `json:"startRoundTime,omitempty"` + StartRoundTime time.Time `json:"startRoundTime,omitempty"` + + //FinishImportTime is the time the pipeline received the block from the importer FinishImportTime time.Time `json:"finishImportTime,omitempty"` } From fc2fd4154cad2a26e74a667d762dd570209f6431 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 2 Aug 2023 18:23:38 -0500 Subject: [PATCH 30/50] lint --- conduit/data/block_export_data.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/conduit/data/block_export_data.go b/conduit/data/block_export_data.go index 8ba4f122..e7821825 100644 --- a/conduit/data/block_export_data.go +++ b/conduit/data/block_export_data.go @@ -25,10 +25,10 @@ type InitProvider interface { // PipelineData is used to keep track of pipeline performance type PipelineData struct { - //StartRoundTime is the time the pipeline started processing the block + // StartRoundTime is the time the pipeline started processing the block StartRoundTime time.Time `json:"startRoundTime,omitempty"` - //FinishImportTime is the time the pipeline received the block from the importer + // FinishImportTime is the time the pipeline received the block from the importer FinishImportTime time.Time `json:"finishImportTime,omitempty"` } From 5ae8a21baaabd327d143c18f68efa79338ef3338 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Thu, 3 Aug 2023 11:20:33 -0500 Subject: [PATCH 31/50] don't pollute data.BlockData with pipeline statistics --- conduit/data/block_export_data.go | 12 ------ conduit/pipeline/pipeline.go | 69 +++++++++++++++++++------------ 2 files changed, 43 insertions(+), 38 deletions(-) diff --git a/conduit/data/block_export_data.go b/conduit/data/block_export_data.go index e7821825..b005f757 100644 --- a/conduit/data/block_export_data.go +++ b/conduit/data/block_export_data.go @@ -1,8 +1,6 @@ package data import ( - "time" - sdk "github.com/algorand/go-algorand-sdk/v2/types" "github.com/algorand/conduit/conduit/telemetry" @@ -24,13 +22,6 @@ type InitProvider interface { } // PipelineData is used to keep track of pipeline performance -type PipelineData struct { - // StartRoundTime is the time the pipeline started processing the block - StartRoundTime time.Time `json:"startRoundTime,omitempty"` - - // FinishImportTime is the time the pipeline received the block from the importer - FinishImportTime time.Time `json:"finishImportTime,omitempty"` -} // BlockData is provided to the Exporter on each round. type BlockData struct { @@ -45,9 +36,6 @@ type BlockData struct { // Certificate contains voting data that certifies the block. The certificate is non deterministic, a node stops collecting votes once the voting threshold is reached. Certificate *map[string]interface{} `json:"cert,omitempty"` - - // PipelineData is used to keep track of pipeline performance - PipelineData } // Round returns the round to which the BlockData corresponds diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index df09e1f4..488dae16 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -61,6 +61,18 @@ type pipelineImpl struct { pipelineMetadata state } +type pipelineData struct { + startRoundTime time.Time + finishImportTime time.Time +} + +type pipelineBlock struct { + data.BlockData + pipelineData +} + +type pluginChannel chan pipelineBlock + var ( stopCause = errors.New("pipeline stopped") //nolint:revive // this is a sentinel error errImporterCause = errors.New("importer cancelled") @@ -455,7 +467,7 @@ func addMetrics(block data.BlockData, importTime time.Duration) { metrics.ImportedTxnsPerBlock.Observe(float64(len(block.Payset)) + float64(innerTxn)) } -func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-chan uint64, blkOutChan chan<- data.BlockData) { +func (p *pipelineImpl) importerHandler(importer importers.Importer, roundChan <-chan uint64, blkOutChan pluginChannel) { p.wg.Add(1) go func() { defer p.wg.Done() @@ -492,15 +504,18 @@ func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-ch // run through processors importFinish := time.Now() - blkData.PipelineData = data.PipelineData{ - StartRoundTime: startRound, - FinishImportTime: importFinish, + pipelineBlk := pipelineBlock{ + BlockData: blkData, + pipelineData: pipelineData{ + startRoundTime: startRound, + finishImportTime: importFinish, + }, } select { case <-p.ctx.Done(): return - case blkOutChan <- blkData: + case blkOutChan <- pipelineBlk: waitTime := time.Since(importFinish) totalFeedWait += waitTime p.logger.Tracef("imported round %d into blkOutChan after waiting %dms on channel", rnd, waitTime.Milliseconds()) @@ -511,7 +526,7 @@ func (p *pipelineImpl) ImportHandler(importer importers.Importer, roundChan <-ch }() } -func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkInChan <-chan data.BlockData, blkOutChan chan<- data.BlockData) { +func (p *pipelineImpl) processorHandler(idx int, proc processors.Processor, blkInChan pluginChannel, blkOutChan pluginChannel) { p.wg.Add(1) go func() { defer p.wg.Done() @@ -528,13 +543,15 @@ func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkI case <-p.ctx.Done(): p.logger.Infof("processor[%d] %s handler exiting lastRnd=%d", idx, proc.Metadata().Name, lastRnd) return - case blkData := <-blkInChan: + case pluginBlk := <-blkInChan: waitTime := time.Since(selectStart) totalSelectWait += waitTime p.logger.Tracef("processor[%d] %s handler received block data for round %d after wait of %s", idx, proc.Metadata().Name, lastRnd, waitTime) - lastRnd = blkData.Round() + lastRnd = pluginBlk.Round() - blkData, procTime, lastError := Retries(proc.Process, blkData, p, proc.Metadata().Name) + var procTime time.Duration + var lastError error + pluginBlk.BlockData, procTime, lastError = Retries(proc.Process, pluginBlk.BlockData, p, proc.Metadata().Name) if lastError != nil { p.cancelWithProblem(fmt.Errorf("processor[%d] %s handler (%w): failed to process round %d after %dms: %w", idx, proc.Metadata().Name, errProcessorCause, lastRnd, procTime.Milliseconds(), lastError)) return @@ -545,7 +562,7 @@ func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkI select { case <-p.ctx.Done(): return - case blkOutChan <- blkData: + case blkOutChan <- pluginBlk: waitTime := time.Since(selectStart) totalFeedWait += waitTime p.logger.Tracef("processor[%d] %s for round %d into blkOutChan after waiting %dms", idx, proc.Metadata().Name, lastRnd, totalFeedWait.Milliseconds()) @@ -555,12 +572,12 @@ func (p *pipelineImpl) ProcessorHandler(idx int, proc processors.Processor, blkI }() } -// ExporterHandler handles the exporter's Receive method, updating the metadata's NextRound, +// exporterHandler handles the exporter's Receive method, updating the metadata's NextRound, // saving the metadata, and invoking the activated plugin callbacks. If the context's cancellation is received // before the exporter's Receive method has succeeded, cancel immediately. // However, after the exporter's Receive() method has succeeded we try every component at least once // but will abort if a failure occurs after a cancellation is received. -func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-chan data.BlockData) { +func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan pluginChannel) { p.wg.Add(1) go func() { defer p.wg.Done() @@ -585,11 +602,11 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch case <-p.ctx.Done(): p.logger.Infof("exporter handler exiting lastRnd=%d", lastRound) return - case blkData := <-blkChan: + case pluginBlk := <-blkChan: waitTime := time.Since(selectStart) totalSelectWait += waitTime p.logger.Tracef("exporter handler received block data for round %d after wait of %s", lastRound, waitTime) - lastRound = blkData.Round() + lastRound = pluginBlk.Round() if p.pipelineMetadata.NextRound != lastRound { lastError = fmt.Errorf("aborting after out of order block data. %d != %d", p.pipelineMetadata.NextRound, lastRound) @@ -597,7 +614,7 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch } var exportTime time.Duration - exportTime, lastError = RetriesNoOutput(exporter.Receive, blkData, p, exporter.Metadata().Name) + exportTime, lastError = RetriesNoOutput(exporter.Receive, pluginBlk.BlockData, p, exporter.Metadata().Name) if lastError != nil { lastError = fmt.Errorf("aborting after failing to export round %d: %w", lastRound, lastError) return @@ -608,7 +625,7 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch // Previously we reported time starting after block fetching is complete // through the end of the export operation. Now that each plugin is running // in its own goroutine, report only the time of the export. - addMetrics(blkData, exportTime) + addMetrics(pluginBlk.BlockData, exportTime) } // Increment Round, update metadata @@ -623,7 +640,7 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch for i, cb := range p.completeCallback { p.logger.Tracef("exporter %s @ round=%d NextRound=%d executing callback %d", exporter.Metadata().Name, lastRound, p.pipelineMetadata.NextRound, i) - _, lastError = RetriesNoOutput(cb, blkData, p, fmt.Sprintf("callback %d", i)) + _, lastError = RetriesNoOutput(cb, pluginBlk.BlockData, p, fmt.Sprintf("callback %d", i)) if lastError != nil { lastError = fmt.Errorf("aborting due to failed callback %d: %w", i, lastError) return @@ -637,10 +654,10 @@ func (p *pipelineImpl) ExporterHandler(exporter exporters.Exporter, blkChan <-ch p.logger.Infof( "UPDATED Pipeline NextRound=%d after [%s] from round kickoff. FINISHED Pipeline round r=%d (%d txn) exported in %s", nextRound, - time.Since(blkData.StartRoundTime), + time.Since(pluginBlk.startRoundTime), lastRound, - len(blkData.Payset), - time.Since(blkData.FinishImportTime), + len(pluginBlk.Payset), + time.Since(pluginBlk.finishImportTime), ) } } @@ -653,17 +670,17 @@ func (p *pipelineImpl) Start() { // Setup channels roundChan := make(chan uint64) - processorBlkInChan := make(chan data.BlockData) - p.ImportHandler(p.importer, roundChan, processorBlkInChan) + processorBlkInChan := make(pluginChannel) + p.importerHandler(p.importer, roundChan, processorBlkInChan) - var processorBlkOutChan chan data.BlockData + var processorBlkOutChan pluginChannel for i, proc := range p.processors { - processorBlkOutChan = make(chan data.BlockData) - p.ProcessorHandler(i, proc, processorBlkInChan, processorBlkOutChan) + processorBlkOutChan = make(pluginChannel) + p.processorHandler(i, proc, processorBlkInChan, processorBlkOutChan) processorBlkInChan = processorBlkOutChan } - p.ExporterHandler(p.exporter, processorBlkInChan) + p.exporterHandler(p.exporter, processorBlkInChan) p.wg.Add(1) // Main loop From 3d06838cc155145bde4a2f8af28c091c67955ef4 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Thu, 3 Aug 2023 11:21:54 -0500 Subject: [PATCH 32/50] Update conduit/data/block_export_data.go --- conduit/data/block_export_data.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/conduit/data/block_export_data.go b/conduit/data/block_export_data.go index b005f757..7fa7ed65 100644 --- a/conduit/data/block_export_data.go +++ b/conduit/data/block_export_data.go @@ -21,8 +21,6 @@ type InitProvider interface { GetTelemetryClient() telemetry.Client } -// PipelineData is used to keep track of pipeline performance - // BlockData is provided to the Exporter on each round. type BlockData struct { // BlockHeader is the immutable header from the block From 30c6a4d563c35e247fabbe74d863f098a02208c7 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Thu, 3 Aug 2023 11:22:50 -0500 Subject: [PATCH 33/50] revert --- conduit/data/block_export_data.go | 1 + 1 file changed, 1 insertion(+) diff --git a/conduit/data/block_export_data.go b/conduit/data/block_export_data.go index 7fa7ed65..a8cb9805 100644 --- a/conduit/data/block_export_data.go +++ b/conduit/data/block_export_data.go @@ -23,6 +23,7 @@ type InitProvider interface { // BlockData is provided to the Exporter on each round. type BlockData struct { + // BlockHeader is the immutable header from the block BlockHeader sdk.BlockHeader `json:"block,omitempty"` From a8337b3a4f9f32877fad1296973908da8adff695 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Tue, 8 Aug 2023 23:21:38 -0500 Subject: [PATCH 34/50] time out after 15 secs instead of 5 --- .../plugins/importers/algod/algod_importer.go | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/conduit/plugins/importers/algod/algod_importer.go b/conduit/plugins/importers/algod/algod_importer.go index fae0dc1b..43e62560 100644 --- a/conduit/plugins/importers/algod/algod_importer.go +++ b/conduit/plugins/importers/algod/algod_importer.go @@ -42,7 +42,7 @@ const ( ) var ( - waitForRoundTimeout = 5 * time.Second + waitForRoundTimeout = 15 * time.Second ) const catchpointsURL = "https://algorand-catchpoints.s3.us-east-2.amazonaws.com/consolidated/%s_catchpoints.txt" @@ -418,23 +418,25 @@ func (algodImp *algodImporter) getDelta(rnd uint64) (sdk.LedgerStateDelta, error } // SyncError is used to indicate algod and conduit are not synchronized. +// The retrievedRound is the round returned from an algod status call. +// The expectedRound is the round conduit expected to have gotten back. type SyncError struct { - rnd uint64 - expected uint64 - err error + retrievedRound uint64 + expectedRound uint64 + err error } // NewSyncError creates a new SyncError. -func NewSyncError(rnd, expected uint64, err error) *SyncError { +func NewSyncError(retrievedRound, expectedRound uint64, err error) *SyncError { return &SyncError{ - rnd: rnd, - expected: expected, - err: err, + retrievedRound: retrievedRound, + expectedRound: expectedRound, + err: err, } } func (e *SyncError) Error() string { - return fmt.Sprintf("wrong round returned from status for round: %d != %d: %v", e.rnd, e.expected, e.err) + return fmt.Sprintf("wrong round returned from status for round: retrieved(%d) != expected(%d): %v", e.retrievedRound, e.expectedRound, e.err) } func (e *SyncError) Unwrap() error { From 05580c76f86f998f1fef7ff5f18c6f2fed785420 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Tue, 8 Aug 2023 23:22:22 -0500 Subject: [PATCH 35/50] wip --- conduit/pipeline/pipeline.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 488dae16..c5c5aa46 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -490,12 +490,16 @@ func (p *pipelineImpl) importerHandler(importer importers.Importer, roundChan <- totalSelectWait += waitTime p.logger.Tracef("importer handler waited %dms to receive round %d", waitTime.Milliseconds(), rnd) + // TODO: DO NOT MERGE THE FOLLOWING!!!! + // time.Sleep(5 * time.Second) + blkData, importTime, lastError := Retries(importer.GetBlock, rnd, p, importer.Metadata().Name) if lastError != nil { p.cancelWithProblem(fmt.Errorf("importer %s handler (%w): failed to import round %d after %dms: %w", importer.Metadata().Name, errImporterCause, rnd, importTime.Milliseconds(), lastError)) return } metrics.ImporterTimeSeconds.Observe(importTime.Seconds()) + p.logger.Tracef("importer handler imported round %d in %dms", rnd, importTime.Milliseconds()) // TODO: Verify that the block was built with a known protocol version. @@ -636,7 +640,7 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug lastError = fmt.Errorf("aborting after updating NextRound=%d and failing to save metadata: %w", nextRound, lastError) return } - p.logger.Tracef("exporter %s @ round %d saved pipeline metadata", exporter.Metadata().Name, p.pipelineMetadata.NextRound) + p.logger.Tracef("exporter %s incremented pipeline metadata NextRound to %d", exporter.Metadata().Name, p.pipelineMetadata.NextRound) for i, cb := range p.completeCallback { p.logger.Tracef("exporter %s @ round=%d NextRound=%d executing callback %d", exporter.Metadata().Name, lastRound, p.pipelineMetadata.NextRound, i) From cdc85e04019615e505c115679596c63d9e09a094 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Tue, 8 Aug 2023 23:36:39 -0500 Subject: [PATCH 36/50] fix test --- .../importers/algod/algod_importer_test.go | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/conduit/plugins/importers/algod/algod_importer_test.go b/conduit/plugins/importers/algod/algod_importer_test.go index 95315900..49dc3fd7 100644 --- a/conduit/plugins/importers/algod/algod_importer_test.go +++ b/conduit/plugins/importers/algod/algod_importer_test.go @@ -3,17 +3,16 @@ package algodimporter import ( "context" "fmt" - "net/http" - "net/http/httptest" - "strings" - "testing" - "time" - "github.com/sirupsen/logrus" "github.com/sirupsen/logrus/hooks/test" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "gopkg.in/yaml.v3" + "net/http" + "net/http/httptest" + "strings" + "testing" + "time" "github.com/algorand/go-algorand-sdk/v2/client/v2/algod" "github.com/algorand/go-algorand-sdk/v2/client/v2/common/models" @@ -701,7 +700,7 @@ func TestGetBlockErrors(t *testing.T) { blockResponder: BlockResponder, deltaResponder: MakeMsgpStatusResponder("get", "/v2/deltas/", http.StatusNotFound, ""), err: fmt.Sprintf("wrong round returned from status for round: 50 != 200"), - logs: []string{"wrong round returned from status for round: 50 != 200", "sync error detected, attempting to set the sync round to recover the node"}, + logs: []string{"wrong round returned from status for round: 50 != 200", "Sync error detected, attempting to set the sync round to recover the node"}, }, { name: "Cannot get delta (caught up)", @@ -756,8 +755,7 @@ func TestGetBlockErrors(t *testing.T) { // Make sure each of the expected log messages are present for _, log := range tc.logs { found := false - hookEntries := hook.AllEntries() - for _, entry := range hookEntries { + for _, entry := range hook.AllEntries() { fmt.Println(strings.Contains(entry.Message, log)) found = found || strings.Contains(entry.Message, log) } From 689d7bd6aaffbf576c64f10892fa32e6bf8a6c0c Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 9 Aug 2023 09:31:43 -0500 Subject: [PATCH 37/50] fix test... really! --- .../importers/algod/algod_importer_test.go | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/conduit/plugins/importers/algod/algod_importer_test.go b/conduit/plugins/importers/algod/algod_importer_test.go index 49dc3fd7..c7cfc185 100644 --- a/conduit/plugins/importers/algod/algod_importer_test.go +++ b/conduit/plugins/importers/algod/algod_importer_test.go @@ -3,17 +3,18 @@ package algodimporter import ( "context" "fmt" - "github.com/sirupsen/logrus" - "github.com/sirupsen/logrus/hooks/test" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "gopkg.in/yaml.v3" "net/http" "net/http/httptest" "strings" "testing" "time" + "github.com/sirupsen/logrus" + "github.com/sirupsen/logrus/hooks/test" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "gopkg.in/yaml.v3" + "github.com/algorand/go-algorand-sdk/v2/client/v2/algod" "github.com/algorand/go-algorand-sdk/v2/client/v2/common/models" sdk "github.com/algorand/go-algorand-sdk/v2/types" @@ -699,8 +700,8 @@ func TestGetBlockErrors(t *testing.T) { blockAfterResponder: MakeBlockAfterResponder(models.NodeStatus{LastRound: 50}), blockResponder: BlockResponder, deltaResponder: MakeMsgpStatusResponder("get", "/v2/deltas/", http.StatusNotFound, ""), - err: fmt.Sprintf("wrong round returned from status for round: 50 != 200"), - logs: []string{"wrong round returned from status for round: 50 != 200", "Sync error detected, attempting to set the sync round to recover the node"}, + err: fmt.Sprintf("wrong round returned from status for round: retrieved(50) != expected(200)"), + logs: []string{"wrong round returned from status for round: retrieved(50) != expected(200)", "sync error detected, attempting to set the sync round to recover the node"}, }, { name: "Cannot get delta (caught up)", @@ -755,7 +756,8 @@ func TestGetBlockErrors(t *testing.T) { // Make sure each of the expected log messages are present for _, log := range tc.logs { found := false - for _, entry := range hook.AllEntries() { + hookEntries := hook.AllEntries() + for _, entry := range hookEntries { fmt.Println(strings.Contains(entry.Message, log)) found = found || strings.Contains(entry.Message, log) } From 674c949c0acf1738f8a8e299cfd69b10f5f54d05 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 9 Aug 2023 21:54:06 -0500 Subject: [PATCH 38/50] remove OStart() --- conduit/pipeline/pipeline.go | 97 ------------------------------------ 1 file changed, 97 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index c5c5aa46..5c8a6c10 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -713,103 +713,6 @@ func (p *pipelineImpl) Start() { // TODO: send a prometheus observation based on WhyStopped() } -// Start pushes block data through the pipeline -func (p *pipelineImpl) OStart() { - p.wg.Add(1) - retry := uint64(0) - go func() { - defer p.wg.Done() - // We need to add a separate recover function here since it launches its own go-routine - defer HandlePanic(p.logger) - for { - pipelineRun: - metrics.PipelineRetryCount.Observe(float64(retry)) - if retry > p.cfg.RetryCount && p.cfg.RetryCount != 0 { - p.logger.Errorf("Pipeline has exceeded maximum retry count (%d) - stopping...", p.cfg.RetryCount) - return - } - - if retry > 0 { - p.logger.Infof("Retry number %d resuming after a %s retry delay.", retry, p.cfg.RetryDelay) - time.Sleep(p.cfg.RetryDelay) - } - - select { - case <-p.ctx.Done(): - return - default: - { - p.logger.Infof("Pipeline round: %v", p.pipelineMetadata.NextRound) - // fetch block - importStart := time.Now() - blkData, err := p.importer.GetBlock(p.pipelineMetadata.NextRound) - if err != nil { - p.logger.Errorf("%v", err) - p.joinError(err) - retry++ - goto pipelineRun - } - metrics.ImporterTimeSeconds.Observe(time.Since(importStart).Seconds()) - - // TODO: Verify that the block was built with a known protocol version. - - // Start time currently measures operations after block fetching is complete. - // This is for backwards compatibility w/ Indexer's metrics - // run through processors - start := time.Now() - for _, proc := range p.processors { - processorStart := time.Now() - blkData, err = proc.Process(blkData) - if err != nil { - p.logger.Errorf("%v", err) - p.joinError(err) - retry++ - goto pipelineRun - } - metrics.ProcessorTimeSeconds.WithLabelValues(proc.Metadata().Name).Observe(time.Since(processorStart).Seconds()) - } - // run through exporter - exporterStart := time.Now() - err = p.exporter.Receive(blkData) - if err != nil { - p.logger.Errorf("%v", err) - p.joinError(err) - retry++ - goto pipelineRun - } - p.logger.Infof("round r=%d (%d txn) exported in %s", p.pipelineMetadata.NextRound, len(blkData.Payset), time.Since(start)) - - // Increment Round, update metadata - p.pipelineMetadata.NextRound++ - err = p.pipelineMetadata.encodeToFile(p.cfg.ConduitArgs.ConduitDataDir) - if err != nil { - p.logger.Errorf("%v", err) - } - - // Callback Processors - for _, cb := range p.completeCallback { - err = cb(blkData) - if err != nil { - p.logger.Errorf("%v", err) - p.joinError(err) - retry++ - goto pipelineRun - } - } - metrics.ExporterTimeSeconds.Observe(time.Since(exporterStart).Seconds()) - // Ignore round 0 (which is empty). - if p.pipelineMetadata.NextRound > 1 { - addMetrics(blkData, time.Since(start)) - } - p.joinError(nil) - retry = 0 - } - } - - } - }() -} - func (p *pipelineImpl) Wait() { p.wg.Wait() } From 7e88db3d2c9fea7afc600b0cee46f1f7c4e4fffd Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 9 Aug 2023 22:22:34 -0500 Subject: [PATCH 39/50] don't shodow the pipeline package --- pkg/cli/cli.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/cli/cli.go b/pkg/cli/cli.go index 934d3b9e..23cf4776 100644 --- a/pkg/cli/cli.go +++ b/pkg/cli/cli.go @@ -80,7 +80,7 @@ func runConduitCmdWithConfig(args *data.Args) error { } ctx := context.Background() - pipeline, err := pipeline.MakePipeline(ctx, pCfg, logger) + pline, err := pipeline.MakePipeline(ctx, pCfg, logger) if err != nil { err = fmt.Errorf("pipeline creation error: %w", err) @@ -91,7 +91,7 @@ func runConduitCmdWithConfig(args *data.Args) error { return err } - err = pipeline.Init() + err = pline.Init() if err != nil { // Suppress log, it is about to be printed to stderr. if pCfg.LogFile != "" { @@ -99,10 +99,10 @@ func runConduitCmdWithConfig(args *data.Args) error { } return fmt.Errorf("pipeline init error: %w", err) } - pipeline.Start() - defer pipeline.Stop() - pipeline.Wait() - return pipeline.Error() + pline.Start() + defer pline.Stop() + pline.Wait() + return pline.Error() } // MakeConduitCmdWithUtilities creates the main cobra command with all utilities From a160e282db7e80cc4c8b9e9d13d561138d52a5dd Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 9 Aug 2023 22:29:29 -0500 Subject: [PATCH 40/50] enable `if !errors.Is(pl.WhyStopped(), pipeline.BecauseStopMethod)` --- conduit/pipeline/pipeline.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 5c8a6c10..00417757 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -74,7 +74,9 @@ type pipelineBlock struct { type pluginChannel chan pipelineBlock var ( - stopCause = errors.New("pipeline stopped") //nolint:revive // this is a sentinel error + // BecauseStopMethod is the sentinel error that signals the pipeline was stopped via Stop(). + BecauseStopMethod = errors.New("pipeline stopped") //nolint:revive // this is a sentinel error + errImporterCause = errors.New("importer cancelled") errProcessorCause = errors.New("processor cancelled") errExporterCause = errors.New("exporter cancelled") @@ -398,7 +400,7 @@ func (p *pipelineImpl) Init() error { } func (p *pipelineImpl) Stop() { - p.ccf(stopCause) + p.ccf(BecauseStopMethod) p.wg.Wait() if p.profFile != nil { From 7b7cf2f6adc2a9fcd487ce4423f5270779f851b5 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 9 Aug 2023 22:36:22 -0500 Subject: [PATCH 41/50] Update conduit/pipeline/pipeline.go --- conduit/pipeline/pipeline.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 00417757..7e0cb200 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -492,9 +492,6 @@ func (p *pipelineImpl) importerHandler(importer importers.Importer, roundChan <- totalSelectWait += waitTime p.logger.Tracef("importer handler waited %dms to receive round %d", waitTime.Milliseconds(), rnd) - // TODO: DO NOT MERGE THE FOLLOWING!!!! - // time.Sleep(5 * time.Second) - blkData, importTime, lastError := Retries(importer.GetBlock, rnd, p, importer.Metadata().Name) if lastError != nil { p.cancelWithProblem(fmt.Errorf("importer %s handler (%w): failed to import round %d after %dms: %w", importer.Metadata().Name, errImporterCause, rnd, importTime.Milliseconds(), lastError)) From cb86036bd9a7825c95d594e3d610f7a4171c9d16 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Wed, 9 Aug 2023 23:10:54 -0500 Subject: [PATCH 42/50] fail fast if can't save metadata and carry on without retrying the callbacks --- conduit/pipeline/pipeline.go | 26 +++++++++++++++----------- docs/PluginDevelopment.md | 10 ++++++++-- 2 files changed, 23 insertions(+), 13 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 7e0cb200..38a61b9c 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -588,11 +588,12 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug lastRound := uint64(0) totalSelectWait := time.Duration(0) totalExportWork := time.Duration(0) + eName := exporter.Metadata().Name defer func() { - p.logger.Debugf("exporter %s handler exiting. lastRnd=%d totalSelectWait=%dms, totalExportWork=%dms", exporter.Metadata().Name, lastRound, totalSelectWait.Milliseconds(), totalExportWork.Milliseconds()) + p.logger.Debugf("exporter %s handler exiting. lastRnd=%d totalSelectWait=%dms, totalExportWork=%dms", eName, lastRound, totalSelectWait.Milliseconds(), totalExportWork.Milliseconds()) if lastError != nil { - err := fmt.Errorf("exporter %s handler (%w) after round %d: %w", exporter.Metadata().Name, errExporterCause, lastRound, lastError) + err := fmt.Errorf("exporter %s handler (%w) after round %d: %w", eName, errExporterCause, lastRound, lastError) p.cancelWithProblem(err) p.logger.Error(err) } @@ -617,7 +618,7 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug } var exportTime time.Duration - exportTime, lastError = RetriesNoOutput(exporter.Receive, pluginBlk.BlockData, p, exporter.Metadata().Name) + exportTime, lastError = RetriesNoOutput(exporter.Receive, pluginBlk.BlockData, p, eName) if lastError != nil { lastError = fmt.Errorf("aborting after failing to export round %d: %w", lastRound, lastError) return @@ -634,21 +635,24 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug // Increment Round, update metadata nextRound := lastRound + 1 p.pipelineMetadata.NextRound = nextRound - _, lastError = RetriesNoOutput(p.pipelineMetadata.encodeToFile, p.cfg.ConduitArgs.ConduitDataDir, p, "pipelineMetadata save") + lastError = p.pipelineMetadata.encodeToFile(p.cfg.ConduitArgs.ConduitDataDir) if lastError != nil { - lastError = fmt.Errorf("aborting after updating NextRound=%d and failing to save metadata: %w", nextRound, lastError) + lastError = fmt.Errorf("aborting after updating NextRound=%d BUT failing to save metadata: %w", nextRound, lastError) return } - p.logger.Tracef("exporter %s incremented pipeline metadata NextRound to %d", exporter.Metadata().Name, p.pipelineMetadata.NextRound) + p.logger.Tracef("exporter %s incremented pipeline metadata NextRound to %d", eName, nextRound) for i, cb := range p.completeCallback { - p.logger.Tracef("exporter %s @ round=%d NextRound=%d executing callback %d", exporter.Metadata().Name, lastRound, p.pipelineMetadata.NextRound, i) - _, lastError = RetriesNoOutput(cb, pluginBlk.BlockData, p, fmt.Sprintf("callback %d", i)) - if lastError != nil { - lastError = fmt.Errorf("aborting due to failed callback %d: %w", i, lastError) - return + p.logger.Tracef("exporter %s @ round=%d NextRound=%d executing callback %d", eName, lastRound, nextRound, i) + callbackErr := cb(pluginBlk.BlockData) + if callbackErr != nil { + p.logger.Errorf( + "exporter %s # round %d failed callback #%d but CONTINUING to NextRound=%d: %v", + eName, lastRound, i, nextRound, lastError, + ) } } + lastError = nil // WARNING: removing the following will BREAK: // - the E2E test (Search for "Pipeline round:" in subslurp.py) diff --git a/docs/PluginDevelopment.md b/docs/PluginDevelopment.md index c09e6c39..189cfc34 100644 --- a/docs/PluginDevelopment.md +++ b/docs/PluginDevelopment.md @@ -62,11 +62,15 @@ The context provided to this function should be saved, and used to terminate any ### Per-round function -Each plugin type has a function which is called once per round: +Each plugin type has a function termed the _Plugin Method_ which is called once per round: * Importer: `GetBlock` is called when a particular round is required. * Processor: `Process` is called to process a round. -* Exporter: `Receive` for consuming a round. +* Exporter: `Receive` for consuming and typically persisting a round. + +> NOTE: When a _Plugin Method_ returns an error, the pipeline will retry according to the configured `retry-count` employing a delay of `retry-delay` +> between tries. In most cases, it is recommended that you use Conduit's +> built-in retry mechanism rather than implementing your own. ### Close @@ -104,6 +108,8 @@ type RoundRequestor interface { When all processing has completed for a round, the `OnComplete` function is called on any plugin that implements it. +> NOTE: The error that `OnComplete()` returns is non-fatal and Conduit's pipeline will continue on to the next round. + ```go // Completed is called by the conduit pipeline after every exporter has // finished. It can be used for things like finalizing state. From 1158f757ad8429bca46dc2ce96d7eee4fa77f639 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Fri, 11 Aug 2023 16:05:43 -0500 Subject: [PATCH 43/50] simplify cancellation cause and punt to issue Graceful Pipeline Exit #100 --- conduit/pipeline/pipeline.go | 21 +-------------------- conduit/pipeline/pipeline_test.go | 2 +- 2 files changed, 2 insertions(+), 21 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 38a61b9c..49e88be2 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -36,7 +36,6 @@ type Pipeline interface { Init() error Start() Stop() - WhyStopped() error Error() error Wait() } @@ -74,9 +73,6 @@ type pipelineBlock struct { type pluginChannel chan pipelineBlock var ( - // BecauseStopMethod is the sentinel error that signals the pipeline was stopped via Stop(). - BecauseStopMethod = errors.New("pipeline stopped") //nolint:revive // this is a sentinel error - errImporterCause = errors.New("importer cancelled") errProcessorCause = errors.New("processor cancelled") errExporterCause = errors.New("exporter cancelled") @@ -400,7 +396,7 @@ func (p *pipelineImpl) Init() error { } func (p *pipelineImpl) Stop() { - p.ccf(BecauseStopMethod) + p.ccf(nil) p.wg.Wait() if p.profFile != nil { @@ -433,16 +429,6 @@ func (p *pipelineImpl) Stop() { } } -// WhyStopped returns nil if a context was never provided or the pipeline -// is yet to have stopped. Otherwise, it returns the cause of the pipeline's -// context cancellation. -func (p *pipelineImpl) WhyStopped() error { - if p.ctx == nil { - return nil - } - return context.Cause(p.ctx) -} - func numInnerTxn(txn sdk.SignedTxnWithAD) int { result := 0 for _, itxn := range txn.ApplyData.EvalDelta.InnerTxns { @@ -502,10 +488,6 @@ func (p *pipelineImpl) importerHandler(importer importers.Importer, roundChan <- // TODO: Verify that the block was built with a known protocol version. - // Start time currently measures operations after block fetching is complete. - // This is for backwards compatibility w/ Indexer's metrics - // run through processors - importFinish := time.Now() pipelineBlk := pipelineBlock{ BlockData: blkData, @@ -713,7 +695,6 @@ func (p *pipelineImpl) Start() { }(p.pipelineMetadata.NextRound) <-p.ctx.Done() - // TODO: send a prometheus observation based on WhyStopped() } func (p *pipelineImpl) Wait() { diff --git a/conduit/pipeline/pipeline_test.go b/conduit/pipeline/pipeline_test.go index 31842cea..43e7ce54 100644 --- a/conduit/pipeline/pipeline_test.go +++ b/conduit/pipeline/pipeline_test.go @@ -342,7 +342,7 @@ func TestPipelineRun(t *testing.T) { mock.AssertExpectationsForObjects(t, &mImporter, &mProcessor, &mExporter) - assert.ErrorIs(t, pImpl.WhyStopped(), errTestCancellation) + assert.ErrorIs(t, context.Cause(pImpl.ctx), errTestCancellation) } // TestPipelineCpuPidFiles tests that cpu and pid files are created when specified From 8b88301151e6648d783272a4d7c5be8466cf8259 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Fri, 11 Aug 2023 16:42:22 -0500 Subject: [PATCH 44/50] logstatsE2Elog --- conduit/pipeline/pipeline.go | 24 +++++++++++++----------- conduit/pipeline/pipeline_test.go | 19 +++++++++++++++++++ 2 files changed, 32 insertions(+), 11 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index 49e88be2..bca9a4ce 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -636,23 +636,25 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug } lastError = nil - // WARNING: removing the following will BREAK: - // - the E2E test (Search for "Pipeline round:" in subslurp.py) + // WARNING: removing/re-log-levelling the following will BREAK: + // - the E2E test (Search for "Pipeline round" in subslurp.py) // - the internal tools logstats collector (See func ConduitCollector in logstats.go of internal-tools repo) - // Modify with CAUTION!!!! - p.logger.Infof( - "UPDATED Pipeline NextRound=%d after [%s] from round kickoff. FINISHED Pipeline round r=%d (%d txn) exported in %s", - nextRound, - time.Since(pluginBlk.startRoundTime), - lastRound, - len(pluginBlk.Payset), - time.Since(pluginBlk.finishImportTime), - ) + p.logger.Infof(logstatsE2Elog(nextRound, lastRound, len(pluginBlk.Payset), exportTime)) } } }() } +func logstatsE2Elog(nextRound, lastRound uint64, topLevelTxnCount int, exportTime time.Duration) string { + return fmt.Sprintf( + "UPDATED Pipeline NextRound=%d. FINISHED Pipeline round r=%d (%d txn) exported in %s", + nextRound, + lastRound, + topLevelTxnCount, + exportTime, + ) +} + // Start pushes block data through the pipeline func (p *pipelineImpl) Start() { p.logger.Trace("Pipeline.Start()") diff --git a/conduit/pipeline/pipeline_test.go b/conduit/pipeline/pipeline_test.go index 43e7ce54..9dd014cb 100644 --- a/conduit/pipeline/pipeline_test.go +++ b/conduit/pipeline/pipeline_test.go @@ -988,3 +988,22 @@ func TestMetrics(t *testing.T) { } assert.Equal(t, 4, found) } + +func TestLogStatsE2Elog(t *testing.T) { + nextRound := uint64(1337) + round := uint64(42) + numTxns := 13 + duration := 12345600 * time.Microsecond + + expectedLog := "UPDATED Pipeline NextRound=1337. FINISHED Pipeline round r=42 (13 txn) exported in 12.3456s" + log := logstatsE2Elog(nextRound, round, numTxns, duration) + require.Equal(t, expectedLog, log) + + logstatsRex, err := regexp.Compile(`round r=(\d+) \((\d+) txn\) exported in (.*)`) + require.NoError(t, err) + matches := logstatsRex.FindStringSubmatch(log) + require.Len(t, matches, 4) + require.Equal(t, "42", matches[1]) + require.Equal(t, "13", matches[2]) + require.Equal(t, "12.3456s", matches[3]) +} From 867973f82e586f7d736e53e44eaf6d1f08871275 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Mon, 14 Aug 2023 20:41:18 -0500 Subject: [PATCH 45/50] don't send metrics through the plugin channels --- conduit/pipeline/pipeline.go | 41 ++++++++------------------- conduit/pipeline/pipeline_test.go | 6 ++++ e2e_tests/src/e2e_conduit/subslurp.py | 2 +- 3 files changed, 19 insertions(+), 30 deletions(-) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index bca9a4ce..aa5c3c5d 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -60,17 +60,7 @@ type pipelineImpl struct { pipelineMetadata state } -type pipelineData struct { - startRoundTime time.Time - finishImportTime time.Time -} - -type pipelineBlock struct { - data.BlockData - pipelineData -} - -type pluginChannel chan pipelineBlock +type pluginChannel chan data.BlockData var ( errImporterCause = errors.New("importer cancelled") @@ -489,18 +479,11 @@ func (p *pipelineImpl) importerHandler(importer importers.Importer, roundChan <- // TODO: Verify that the block was built with a known protocol version. importFinish := time.Now() - pipelineBlk := pipelineBlock{ - BlockData: blkData, - pipelineData: pipelineData{ - startRoundTime: startRound, - finishImportTime: importFinish, - }, - } select { case <-p.ctx.Done(): return - case blkOutChan <- pipelineBlk: + case blkOutChan <- blkData: waitTime := time.Since(importFinish) totalFeedWait += waitTime p.logger.Tracef("imported round %d into blkOutChan after waiting %dms on channel", rnd, waitTime.Milliseconds()) @@ -528,15 +511,15 @@ func (p *pipelineImpl) processorHandler(idx int, proc processors.Processor, blkI case <-p.ctx.Done(): p.logger.Infof("processor[%d] %s handler exiting lastRnd=%d", idx, proc.Metadata().Name, lastRnd) return - case pluginBlk := <-blkInChan: + case blk := <-blkInChan: waitTime := time.Since(selectStart) totalSelectWait += waitTime p.logger.Tracef("processor[%d] %s handler received block data for round %d after wait of %s", idx, proc.Metadata().Name, lastRnd, waitTime) - lastRnd = pluginBlk.Round() + lastRnd = blk.Round() var procTime time.Duration var lastError error - pluginBlk.BlockData, procTime, lastError = Retries(proc.Process, pluginBlk.BlockData, p, proc.Metadata().Name) + blk, procTime, lastError = Retries(proc.Process, blk, p, proc.Metadata().Name) if lastError != nil { p.cancelWithProblem(fmt.Errorf("processor[%d] %s handler (%w): failed to process round %d after %dms: %w", idx, proc.Metadata().Name, errProcessorCause, lastRnd, procTime.Milliseconds(), lastError)) return @@ -547,7 +530,7 @@ func (p *pipelineImpl) processorHandler(idx int, proc processors.Processor, blkI select { case <-p.ctx.Done(): return - case blkOutChan <- pluginBlk: + case blkOutChan <- blk: waitTime := time.Since(selectStart) totalFeedWait += waitTime p.logger.Tracef("processor[%d] %s for round %d into blkOutChan after waiting %dms", idx, proc.Metadata().Name, lastRnd, totalFeedWait.Milliseconds()) @@ -588,11 +571,11 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug case <-p.ctx.Done(): p.logger.Infof("exporter handler exiting lastRnd=%d", lastRound) return - case pluginBlk := <-blkChan: + case blk := <-blkChan: waitTime := time.Since(selectStart) totalSelectWait += waitTime p.logger.Tracef("exporter handler received block data for round %d after wait of %s", lastRound, waitTime) - lastRound = pluginBlk.Round() + lastRound = blk.Round() if p.pipelineMetadata.NextRound != lastRound { lastError = fmt.Errorf("aborting after out of order block data. %d != %d", p.pipelineMetadata.NextRound, lastRound) @@ -600,7 +583,7 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug } var exportTime time.Duration - exportTime, lastError = RetriesNoOutput(exporter.Receive, pluginBlk.BlockData, p, eName) + exportTime, lastError = RetriesNoOutput(exporter.Receive, blk, p, eName) if lastError != nil { lastError = fmt.Errorf("aborting after failing to export round %d: %w", lastRound, lastError) return @@ -611,7 +594,7 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug // Previously we reported time starting after block fetching is complete // through the end of the export operation. Now that each plugin is running // in its own goroutine, report only the time of the export. - addMetrics(pluginBlk.BlockData, exportTime) + addMetrics(blk, exportTime) } // Increment Round, update metadata @@ -626,7 +609,7 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug for i, cb := range p.completeCallback { p.logger.Tracef("exporter %s @ round=%d NextRound=%d executing callback %d", eName, lastRound, nextRound, i) - callbackErr := cb(pluginBlk.BlockData) + callbackErr := cb(blk) if callbackErr != nil { p.logger.Errorf( "exporter %s # round %d failed callback #%d but CONTINUING to NextRound=%d: %v", @@ -639,7 +622,7 @@ func (p *pipelineImpl) exporterHandler(exporter exporters.Exporter, blkChan plug // WARNING: removing/re-log-levelling the following will BREAK: // - the E2E test (Search for "Pipeline round" in subslurp.py) // - the internal tools logstats collector (See func ConduitCollector in logstats.go of internal-tools repo) - p.logger.Infof(logstatsE2Elog(nextRound, lastRound, len(pluginBlk.Payset), exportTime)) + p.logger.Infof(logstatsE2Elog(nextRound, lastRound, len(blk.Payset), exportTime)) } } }() diff --git a/conduit/pipeline/pipeline_test.go b/conduit/pipeline/pipeline_test.go index 9dd014cb..ea8db42c 100644 --- a/conduit/pipeline/pipeline_test.go +++ b/conduit/pipeline/pipeline_test.go @@ -1006,4 +1006,10 @@ func TestLogStatsE2Elog(t *testing.T) { require.Equal(t, "42", matches[1]) require.Equal(t, "13", matches[2]) require.Equal(t, "12.3456s", matches[3]) + + e2eRex, err := regexp.Compile(`FINISHED Pipeline round r=(\d+)`) + require.NoError(t, err) + matches = e2eRex.FindStringSubmatch(log) + require.Len(t, matches, 2) + require.Equal(t, "42", matches[1]) } diff --git a/e2e_tests/src/e2e_conduit/subslurp.py b/e2e_tests/src/e2e_conduit/subslurp.py index 9d4d5dfe..796982ce 100644 --- a/e2e_tests/src/e2e_conduit/subslurp.py +++ b/e2e_tests/src/e2e_conduit/subslurp.py @@ -7,7 +7,7 @@ logger = logging.getLogger(__name__) # Matches conduit log output: -# "UPDATED Pipeline NextRound=111 after [123.45ms] from round kickoff. FINISHED Pipeline round r=110 (23 txn) exported in 78.567ms", +# "UPDATED Pipeline NextRound=1337. FINISHED Pipeline round r=42 (13 txn) exported in 12.3456s" FINISH_ROUND: re.Pattern = re.compile(b"FINISHED Pipeline round r=(\d+)") From b36eb1500fa5f1136fb057a2b20bcba35ba3cdf0 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Fri, 18 Aug 2023 10:48:24 -0500 Subject: [PATCH 46/50] Apply suggestions from code review Co-authored-by: Will Winder --- conduit/pipeline/pipeline.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/conduit/pipeline/pipeline.go b/conduit/pipeline/pipeline.go index aa5c3c5d..c492ae0c 100644 --- a/conduit/pipeline/pipeline.go +++ b/conduit/pipeline/pipeline.go @@ -480,6 +480,7 @@ func (p *pipelineImpl) importerHandler(importer importers.Importer, roundChan <- importFinish := time.Now() + // check context in case out channel is full. select { case <-p.ctx.Done(): return @@ -527,6 +528,7 @@ func (p *pipelineImpl) processorHandler(idx int, proc processors.Processor, blkI metrics.ProcessorTimeSeconds.WithLabelValues(proc.Metadata().Name).Observe(procTime.Seconds()) selectStart := time.Now() + // check context in case out channel is full. select { case <-p.ctx.Done(): return From 07e2e03f12797c3203ff4aaccabf91992135339f Mon Sep 17 00:00:00 2001 From: Will Winder Date: Tue, 15 Aug 2023 17:33:06 -0400 Subject: [PATCH 47/50] docs: Remove quotes in file_write examples. (#137) --- README.md | 2 +- conduit/plugins/exporters/filewriter/README.md | 2 +- conduit/plugins/exporters/filewriter/sample.yaml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 2dc1ea2a..ff83f71f 100644 --- a/README.md +++ b/README.md @@ -72,7 +72,7 @@ importer: processors: exporter: - name: "file_writer" + name: file_writer config: # the default config writes block data to the data directory. ``` diff --git a/conduit/plugins/exporters/filewriter/README.md b/conduit/plugins/exporters/filewriter/README.md index dddbea50..21098847 100644 --- a/conduit/plugins/exporters/filewriter/README.md +++ b/conduit/plugins/exporters/filewriter/README.md @@ -4,7 +4,7 @@ Write block data to files. This plugin works with the file rerader plugin to cre ## Configuration ```yml @sample.yaml -name: "file_writer" +name: file_writer config: # BlocksDir is the path to a directory where block data should be stored. # The directory is created if it doesn't exist. If no directory is provided diff --git a/conduit/plugins/exporters/filewriter/sample.yaml b/conduit/plugins/exporters/filewriter/sample.yaml index 04ed355c..59895661 100644 --- a/conduit/plugins/exporters/filewriter/sample.yaml +++ b/conduit/plugins/exporters/filewriter/sample.yaml @@ -1,4 +1,4 @@ -name: "file_writer" +name: file_writer config: # BlocksDir is the path to a directory where block data should be stored. # The directory is created if it doesn't exist. If no directory is provided From 136996bb4ff7b660ed60d566d7a893e0c9b27c53 Mon Sep 17 00:00:00 2001 From: Will Winder Date: Wed, 16 Aug 2023 10:11:55 -0400 Subject: [PATCH 48/50] build: add version to release filename. (#138) --- .goreleaser.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.goreleaser.yaml b/.goreleaser.yaml index 55323293..a7612a90 100644 --- a/.goreleaser.yaml +++ b/.goreleaser.yaml @@ -81,7 +81,8 @@ archives: {{- title .Os }}_ {{- if eq .Arch "amd64" }}x86_64 {{- else if eq .Arch "386" }}i386 - {{- else }}{{ .Arch }}{{ end }} + {{- else }}{{ .Arch }}{{ end }}_ + {{- .Version }} checksum: name_template: 'checksums.txt' snapshot: From 300677f3434bd4cfc2c53b1e2e3c4d301a5b0369 Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Mon, 21 Aug 2023 09:24:44 -0500 Subject: [PATCH 49/50] temp commit --- go.sum | 2 -- 1 file changed, 2 deletions(-) diff --git a/go.sum b/go.sum index 14295a01..addf5085 100644 --- a/go.sum +++ b/go.sum @@ -49,8 +49,6 @@ github.com/algorand/go-algorand-sdk/v2 v2.2.0 h1:zWwK+k/WArtZJUSkDXTDj4a0GUik2iO github.com/algorand/go-algorand-sdk/v2 v2.2.0/go.mod h1:+3+4EZmMUcQk6bgmtC5Ic5kKZE/g6SmfiW098tYLkPE= github.com/algorand/go-codec/codec v1.1.10 h1:zmWYU1cp64jQVTOG8Tw8wa+k0VfwgXIPbnDfiVa+5QA= github.com/algorand/go-codec/codec v1.1.10/go.mod h1:YkEx5nmr/zuCeaDYOIhlDg92Lxju8tj2d2NrYqP7g7k= -github.com/algorand/indexer/v3 v3.0.0 h1:FxQVt1KdwvJrKUAhJPeo+YAOygnJzgjKT8MUEawH+zc= -github.com/algorand/indexer/v3 v3.0.0/go.mod h1:P+RpgLu0lR/6RT8ZwspLHBNKVeAMzwRfCSMVsfiwf40= github.com/algorand/oapi-codegen v1.12.0-algorand.0 h1:W9PvED+wAJc+9EeXPONnA+0zE9UhynEqoDs4OgAxKhk= github.com/algorand/oapi-codegen v1.12.0-algorand.0/go.mod h1:tIWJ9K/qrLDVDt5A1p82UmxZIEGxv2X+uoujdhEAL48= github.com/apapsch/go-jsonmerge/v2 v2.0.0 h1:axGnT1gRIfimI7gJifB699GoE/oq+F2MU7Dml6nw9rQ= From ded0284424db3621b3d400e8c91b48021713d91f Mon Sep 17 00:00:00 2001 From: Zeph Grunschlag Date: Mon, 21 Aug 2023 09:25:09 -0500 Subject: [PATCH 50/50] revert --- go.sum | 2 ++ 1 file changed, 2 insertions(+) diff --git a/go.sum b/go.sum index addf5085..14295a01 100644 --- a/go.sum +++ b/go.sum @@ -49,6 +49,8 @@ github.com/algorand/go-algorand-sdk/v2 v2.2.0 h1:zWwK+k/WArtZJUSkDXTDj4a0GUik2iO github.com/algorand/go-algorand-sdk/v2 v2.2.0/go.mod h1:+3+4EZmMUcQk6bgmtC5Ic5kKZE/g6SmfiW098tYLkPE= github.com/algorand/go-codec/codec v1.1.10 h1:zmWYU1cp64jQVTOG8Tw8wa+k0VfwgXIPbnDfiVa+5QA= github.com/algorand/go-codec/codec v1.1.10/go.mod h1:YkEx5nmr/zuCeaDYOIhlDg92Lxju8tj2d2NrYqP7g7k= +github.com/algorand/indexer/v3 v3.0.0 h1:FxQVt1KdwvJrKUAhJPeo+YAOygnJzgjKT8MUEawH+zc= +github.com/algorand/indexer/v3 v3.0.0/go.mod h1:P+RpgLu0lR/6RT8ZwspLHBNKVeAMzwRfCSMVsfiwf40= github.com/algorand/oapi-codegen v1.12.0-algorand.0 h1:W9PvED+wAJc+9EeXPONnA+0zE9UhynEqoDs4OgAxKhk= github.com/algorand/oapi-codegen v1.12.0-algorand.0/go.mod h1:tIWJ9K/qrLDVDt5A1p82UmxZIEGxv2X+uoujdhEAL48= github.com/apapsch/go-jsonmerge/v2 v2.0.0 h1:axGnT1gRIfimI7gJifB699GoE/oq+F2MU7Dml6nw9rQ=