diff --git a/caboose.go b/caboose.go index 869fc11..ffdb096 100644 --- a/caboose.go +++ b/caboose.go @@ -2,65 +2,70 @@ package caboose import ( "context" - "errors" - "fmt" + "encoding/json" "io" "net/http" "net/url" "os" - "strings" "time" - "github.com/filecoin-saturn/caboose/tieredhashing" + requestcontext "github.com/willscott/go-requestcontext" ipfsblockstore "github.com/ipfs/boxo/blockstore" ipath "github.com/ipfs/boxo/coreiface/path" gateway "github.com/ipfs/boxo/gateway" blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" + + "go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" + + "github.com/filecoin-saturn/caboose/internal/state" ) const ( - SaturnEnvKey = "STRN_ENV_TAG" + BackendOverrideKey = "CABOOSE_BACKEND_OVERRIDE" ) type Config struct { - // OrchestratorEndpoint is the URL of the Saturn orchestrator. + // OrchestratorEndpoint is the URL for fetching upstream nodes. OrchestratorEndpoint *url.URL - // OrchestratorClient is the HTTP client to use when communicating with the Saturn orchestrator. + // OrchestratorClient is the HTTP client to use when communicating with the orchestrator. OrchestratorClient *http.Client // OrchestratorOverride replaces calls to the orchestrator with a fixed response. - OrchestratorOverride []string + OrchestratorOverride []state.NodeInfo - // LoggingEndpoint is the URL of the logging endpoint where we submit logs pertaining to our Saturn retrieval requests. + // LoggingEndpoint is the URL of the logging endpoint where we submit logs pertaining to retrieval requests. LoggingEndpoint url.URL // LoggingClient is the HTTP client to use when communicating with the logging endpoint. LoggingClient *http.Client // LoggingInterval is the interval at which we submit logs to the logging endpoint. LoggingInterval time.Duration - // SaturnClient is the HTTP client to use when retrieving content from the Saturn network. - SaturnClient *http.Client + // Client is the HTTP client to use when retrieving content from upstream nodes. + Client *http.Client ExtraHeaders *http.Header - // DoValidation is used to determine if we should validate the blocks recieved from the Saturn network. + // DoValidation is used to determine if we should validate the blocks recieved from the upstream. DoValidation bool // If set, AffinityKey is used instead of the block CID as the key on the - // Saturn node pool to determine which Saturn node to retrieve the block from. + // pool to determine which upstream to retrieve the request from. // NOTE: If gateway.ContentPathKey is present in request context, // it will be used as AffinityKey automatically. AffinityKey string - // PoolRefresh is the interval at which we refresh the pool of Saturn nodes. + // PoolRefresh is the interval at which we refresh the pool of upstreams from the orchestrator. PoolRefresh time.Duration + // PoolTargetSize is a baseline size for the pool - the pool will accept decrements in performance to reach maintain at least this size. + PoolTargetSize int + // MirrorFraction is what fraction of requests will be mirrored to another random node in order to track metrics / determine the current best nodes. MirrorFraction float64 - // MaxRetrievalAttempts determines the number of times we will attempt to retrieve a block from the Saturn network before failing. + // MaxRetrievalAttempts determines the number of times we will attempt to retrieve a block from upstreams before failing. MaxRetrievalAttempts int // MaxFetchFailuresBeforeCoolDown is the maximum number of retrieval failures across the pool for a url before we auto-reject subsequent @@ -71,17 +76,21 @@ type Config struct { // before we start making retrieval attempts for it. FetchKeyCoolDownDuration time.Duration - // SaturnNodeCoolOff is the cool off duration for a saturn node once we determine that we shouldn't be sending requests to it for a while. - SaturnNodeCoolOff time.Duration + // CoolOff is the cool off duration for a node once we determine that we shouldn't be sending requests to it for a while. + CoolOff time.Duration - TieredHashingOpts []tieredhashing.Option + // Harness is an internal test harness that is set during testing. + Harness *state.State + + // ComplianceCidPeriod controls how many requests caboose makes on average before requesting a compliance cid + ComplianceCidPeriod int64 } const DefaultLoggingInterval = 5 * time.Second -const DefaultSaturnOrchestratorRequestTimeout = 30 * time.Second +const DefaultOrchestratorRequestTimeout = 30 * time.Second -const DefaultSaturnBlockRequestTimeout = 19 * time.Second -const DefaultSaturnCarRequestTimeout = 30 * time.Minute +const DefaultBlockRequestTimeout = 19 * time.Second +const DefaultCarRequestTimeout = 30 * time.Minute // default retries before failure unless overridden by MaxRetrievalAttempts const defaultMaxRetries = 3 @@ -89,85 +98,22 @@ const defaultMaxRetries = 3 // default percentage of requests to mirror for tracking how nodes perform unless overridden by MirrorFraction const defaultMirrorFraction = 0.01 -const maxBlockSize = 4194305 // 4 Mib + 1 byte -const DefaultOrchestratorEndpoint = "https://orchestrator.strn.pl/nodes/nearby?count=200" +const DefaultOrchestratorEndpoint = "https://orchestrator.strn.pl/nodes?maxNodes=200" const DefaultPoolRefreshInterval = 5 * time.Minute +const DefaultPoolTargetSize = 30 + +const DefaultComplianceCidPeriod = int64(100) -// we cool off sending requests to Saturn for a cid for a certain duration +// we cool off sending requests for a cid for a certain duration // if we've seen a certain number of failures for it already in a given duration. // NOTE: before getting creative here, make sure you dont break end user flow // described in https://github.com/ipni/storetheindex/pull/1344 const defaultMaxFetchFailures = 3 * defaultMaxRetries // this has to fail more than DefaultMaxRetries done for a single gateway request const defaultFetchKeyCoolDownDuration = 1 * time.Minute // how long will a sane person wait and stare at blank screen with "retry later" error before hitting F5? -// we cool off sending requests to a Saturn node if it returns transient errors rather than immediately downvoting it; +// we cool off sending requests to a node if it returns transient errors rather than immediately downvoting it; // however, only upto a certain max number of cool-offs. -const defaultSaturnNodeCoolOff = 5 * time.Minute - -var ErrNotImplemented error = errors.New("not implemented") -var ErrNoBackend error = errors.New("no available saturn backend") -var ErrContentProviderNotFound error = errors.New("saturn failed to find content providers") -var ErrSaturnTimeout error = errors.New("saturn backend timed out") - -type ErrSaturnTooManyRequests struct { - Node string - retryAfter time.Duration -} - -func (e *ErrSaturnTooManyRequests) Error() string { - return fmt.Sprintf("saturn node %s returned Too Many Requests error, please retry after %s", e.Node, humanRetry(e.retryAfter)) -} - -func (e *ErrSaturnTooManyRequests) RetryAfter() time.Duration { - return e.retryAfter -} - -type ErrCoolDown struct { - Cid cid.Cid - Path string - retryAfter time.Duration -} - -func (e *ErrCoolDown) Error() string { - switch true { - case e.Cid != cid.Undef && e.Path != "": - return fmt.Sprintf("multiple saturn retrieval failures seen for CID %q and Path %q, please retry after %s", e.Cid, e.Path, humanRetry(e.retryAfter)) - case e.Path != "": - return fmt.Sprintf("multiple saturn retrieval failures seen for Path %q, please retry after %s", e.Path, humanRetry(e.retryAfter)) - case e.Cid != cid.Undef: - return fmt.Sprintf("multiple saturn retrieval failures seen for CID %q, please retry after %s", e.Cid, humanRetry(e.retryAfter)) - default: - return fmt.Sprintf("multiple saturn retrieval failures for unknown CID/Path (BUG), please retry after %s", humanRetry(e.retryAfter)) - } -} - -func (e *ErrCoolDown) RetryAfter() time.Duration { - return e.retryAfter -} - -func humanRetry(d time.Duration) string { - return d.Truncate(time.Second).String() -} - -// ErrPartialResponse can be returned from a DataCallback to indicate that some of the requested resource -// was successfully fetched, and that instead of retrying the full resource, that there are -// one or more more specific resources that should be fetched (via StillNeed) to complete the request. -type ErrPartialResponse struct { - error - StillNeed []string -} - -func (epr ErrPartialResponse) Error() string { - if epr.error != nil { - return fmt.Sprintf("partial response: %s", epr.error.Error()) - } - return "caboose received a partial response" -} - -// ErrInvalidResponse can be returned from a DataCallback to indicate that the data provided for the -// requested resource was explicitly 'incorrect' - that blocks not in the requested dag, or non-car-conforming -// data was returned. -type ErrInvalidResponse error +const defaultNodeCoolOff = 5 * time.Minute type Caboose struct { config *Config @@ -189,28 +135,40 @@ func NewCaboose(config *Config) (*Caboose, error) { config.MaxFetchFailuresBeforeCoolDown = defaultMaxFetchFailures } - if config.SaturnNodeCoolOff == 0 { - config.SaturnNodeCoolOff = defaultSaturnNodeCoolOff + if config.CoolOff == 0 { + config.CoolOff = defaultNodeCoolOff } if config.MirrorFraction == 0 { config.MirrorFraction = defaultMirrorFraction } if override := os.Getenv(BackendOverrideKey); len(override) > 0 { - config.OrchestratorOverride = strings.Split(override, ",") + var overrideNodes []state.NodeInfo + err := json.Unmarshal([]byte(override), &overrideNodes) + if err != nil { + goLogger.Warnf("Error parsing BackendOverrideKey:", "err", err) + return nil, err + } + config.OrchestratorOverride = overrideNodes + } + if config.PoolTargetSize == 0 { + config.PoolTargetSize = DefaultPoolTargetSize } + logger := newLogger(config) c := Caboose{ config: config, - pool: newPool(config), - logger: newLogger(config), + pool: newPool(config, logger), + logger: logger, } - c.pool.logger = c.logger - if c.config.SaturnClient == nil { - c.config.SaturnClient = &http.Client{ - Timeout: DefaultSaturnCarRequestTimeout, + if c.config.Client == nil { + c.config.Client = &http.Client{ + Timeout: DefaultCarRequestTimeout, } } + + c.config.Client.Transport = otelhttp.NewTransport(c.config.Client.Transport) + if c.config.OrchestratorEndpoint == nil { var err error c.config.OrchestratorEndpoint, err = url.Parse(DefaultOrchestratorEndpoint) @@ -219,6 +177,10 @@ func NewCaboose(config *Config) (*Caboose, error) { } } + if c.config.ComplianceCidPeriod == 0 { + c.config.ComplianceCidPeriod = DefaultComplianceCidPeriod + } + if c.config.PoolRefresh == 0 { c.config.PoolRefresh = DefaultPoolRefreshInterval } @@ -227,6 +189,13 @@ func NewCaboose(config *Config) (*Caboose, error) { c.config.MaxRetrievalAttempts = defaultMaxRetries } + // Set during testing to leak internal state to the harness. + if c.config.Harness != nil { + c.config.Harness.ActiveNodes = c.pool.ActiveNodes + c.config.Harness.AllNodes = c.pool.AllNodes + c.config.Harness.PoolController = c.pool + } + // start the pool c.pool.Start() @@ -238,38 +207,47 @@ var _ ipfsblockstore.Blockstore = (*Caboose)(nil) func (c *Caboose) Close() { c.pool.Close() - c.logger.Close() + if c.logger != nil { + c.logger.Close() + } } // Fetch allows fetching car archives by a path of the form `/ipfs/[/path/to/file]` func (c *Caboose) Fetch(ctx context.Context, path string, cb DataCallback) error { + traceID := requestcontext.IDFromContext(ctx) + tid, err := trace.TraceIDFromHex(traceID) + ctx, span := spanTrace(ctx, "Fetch", trace.WithAttributes(attribute.String("path", path))) defer span.End() - return c.pool.fetchResourceWith(ctx, path, cb, c.getAffinity(ctx)) + if err == nil { + sc := trace.NewSpanContext(trace.SpanContextConfig{ + TraceID: tid, + SpanID: span.SpanContext().SpanID(), + Remote: true, + }) + ctx = trace.ContextWithRemoteSpanContext(ctx, sc) + } + + return c.pool.fetchResourceWith(ctx, path, cb, c.GetAffinity(ctx)) } func (c *Caboose) Has(ctx context.Context, it cid.Cid) (bool, error) { ctx, span := spanTrace(ctx, "Has", trace.WithAttributes(attribute.Stringer("cid", it))) defer span.End() - blk, err := c.pool.fetchBlockWith(ctx, it, c.getAffinity(ctx)) + blk, err := c.pool.fetchBlockWith(ctx, it, c.GetAffinity(ctx)) if err != nil { return false, err } return blk != nil, nil } -// for testing only -func (c *Caboose) GetPoolPerf() map[string]*tieredhashing.NodePerf { - return c.pool.th.GetPerf() -} - func (c *Caboose) Get(ctx context.Context, it cid.Cid) (blocks.Block, error) { ctx, span := spanTrace(ctx, "Get", trace.WithAttributes(attribute.Stringer("cid", it))) defer span.End() - blk, err := c.pool.fetchBlockWith(ctx, it, c.getAffinity(ctx)) + blk, err := c.pool.fetchBlockWith(ctx, it, c.GetAffinity(ctx)) if err != nil { return nil, err } @@ -281,14 +259,14 @@ func (c *Caboose) GetSize(ctx context.Context, it cid.Cid) (int, error) { ctx, span := spanTrace(ctx, "GetSize", trace.WithAttributes(attribute.Stringer("cid", it))) defer span.End() - blk, err := c.pool.fetchBlockWith(ctx, it, c.getAffinity(ctx)) + blk, err := c.pool.fetchBlockWith(ctx, it, c.GetAffinity(ctx)) if err != nil { return 0, err } return len(blk.RawData()), nil } -func (c *Caboose) getAffinity(ctx context.Context) string { +func (c *Caboose) GetAffinity(ctx context.Context) string { // https://github.com/ipfs/bifrost-gateway/issues/53#issuecomment-1442732865 if affG := ctx.Value(gateway.ContentPathKey); affG != nil { contentPath := affG.(ipath.Path).String() diff --git a/caboose_test.go b/caboose_test.go index 3fe47b7..d609c8b 100644 --- a/caboose_test.go +++ b/caboose_test.go @@ -3,11 +3,14 @@ package caboose_test import ( "bytes" "context" - "crypto/tls" - "encoding/json" "fmt" + "io" + "net/http" + "testing" + "time" + "github.com/filecoin-saturn/caboose" - "github.com/filecoin-saturn/caboose/tieredhashing" + "github.com/filecoin-saturn/caboose/internal/util" "github.com/ipfs/go-cid" "github.com/ipld/go-car/v2" "github.com/ipld/go-ipld-prime/linking" @@ -17,118 +20,71 @@ import ( selectorparse "github.com/ipld/go-ipld-prime/traversal/selector/parse" "github.com/multiformats/go-multicodec" "github.com/stretchr/testify/require" - "io" - "net/http" - "net/http/httptest" - "net/url" - "strings" - "testing" - "time" ) +var testBlock = []byte("hello World") + func TestCidCoolDown(t *testing.T) { ctx := context.Background() - ch := BuildCabooseHarness(t, 3, 3, WithMaxFailuresBeforeCoolDown(2), WithCidCoolDownDuration(1*time.Second)) + ch := util.BuildCabooseHarness(t, 3, 3, util.WithMaxFailuresBeforeCoolDown(2), util.WithCidCoolDownDuration(1*time.Second)) testCid, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum(testBlock) // Invalidate all servers so we cool down cids - ch.failNodesWithCode(t, func(e *ep) bool { + ch.FailNodesWithCode(t, func(e *util.Endpoint) bool { return true }, 503) // Fetch should fail with fetch error - ch.fetchAndAssertFailure(t, ctx, testCid, "503") + ch.FetchAndAssertFailure(t, ctx, testCid, "503") // second fetch should fail with fetch error - ch.fetchAndAssertFailure(t, ctx, testCid, "503") + ch.FetchAndAssertFailure(t, ctx, testCid, "503") // next fetch should fail with cool down error - ch.fetchAndAssertCoolDownError(t, ctx, testCid) + ch.FetchAndAssertCoolDownError(t, ctx, testCid) // one more fetch should fail with cool down error - ch.fetchAndAssertCoolDownError(t, ctx, testCid) + ch.FetchAndAssertCoolDownError(t, ctx, testCid) - ch.recoverNodes(t, func(e *ep) bool { + ch.RecoverNodes(t, func(e *util.Endpoint) bool { return true }) // fetch should eventually succeed once cid is removed from the cool down cache require.Eventually(t, func() bool { - _, err := ch.c.Get(ctx, testCid) + _, err := ch.Caboose.Get(ctx, testCid) return err == nil }, 10*time.Second, 500*time.Millisecond) } func TestFetchBlock(t *testing.T) { ctx := context.Background() - h := BuildCabooseHarness(t, 3, 3, WithTieredHashingOpts( - []tieredhashing.Option{tieredhashing.WithMaxMainTierSize(1), tieredhashing.WithCorrectnessWindowSize(2), - tieredhashing.WithLatencyWindowSize(2)})) + h := util.BuildCabooseHarness(t, 3, 3) testCid, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum(testBlock) - h.fetchAndAssertSuccess(t, ctx, testCid) + h.FetchAndAssertSuccess(t, ctx, testCid) // ensure we have a success recording - h.assertPoolSize(t, 0, 3, 3) - h.assertCorrectnessCount(t, 1) - h.assertLatencyCount(t, 1) + h.AssertPoolSize(t, 3, 3) + h.AssertCorrectnessCount(t, 1) + h.AssertLatencyCount(t, 1) - h.fetchAndAssertSuccess(t, ctx, testCid) - h.assertCorrectnessCount(t, 2) - h.assertLatencyCount(t, 2) + h.FetchAndAssertSuccess(t, ctx, testCid) + h.AssertCorrectnessCount(t, 2) + h.AssertLatencyCount(t, 2) // all nodes fail - h.failNodesWithCode(t, func(e *ep) bool { + h.FailNodesWithCode(t, func(e *util.Endpoint) bool { return true }, http.StatusNotAcceptable) - h.fetchAndAssertFailure(t, ctx, testCid, "406") -} - -func (h *CabooseHarness) assertLatencyCount(t *testing.T, expected int) { - nds := h.c.GetPoolPerf() - count := 0 - - for _, perf := range nds { - count += int(perf.NLatencyDigest) - } - require.EqualValues(t, expected, count) -} - -func (h *CabooseHarness) assertCorrectnessCount(t *testing.T, expected int) { - nds := h.c.GetPoolPerf() - count := 0 - - for _, perf := range nds { - count += int(perf.NCorrectnessDigest) - } - require.EqualValues(t, expected, count) -} - -func (h *CabooseHarness) assertPoolSize(t *testing.T, mainS, unknownS, totalS int) { - nds := h.c.GetPoolPerf() - require.Equal(t, totalS, len(nds)) - - var eMain int - var eUnknown int - - for _, perf := range nds { - if perf.Tier == "main" { - eMain++ - } - if perf.Tier == "unknown" { - eUnknown++ - } - } - - require.EqualValues(t, eMain, mainS) - require.EqualValues(t, eUnknown, unknownS) + h.FetchAndAssertFailure(t, ctx, testCid, "406") } func TestResource(t *testing.T) { - h := BuildCabooseHarness(t, 1, 3) + h := util.BuildCabooseHarness(t, 1, 3) // some setup. buf := bytes.NewBuffer(nil) ls := cidlink.DefaultLinkSystem() @@ -141,10 +97,11 @@ func TestResource(t *testing.T) { // make our carv1 car.TraverseV1(context.Background(), &ls, rt, selectorparse.CommonSelector_MatchPoint, buf) - h.pool[0].resp = buf.Bytes() + h.Endpoints[0].Resp = buf.Bytes() + h.Endpoints[0].CarWrap = false // ask for it. - if err := h.c.Fetch(context.Background(), "/path/to/car", func(resource string, reader io.Reader) error { + if err := h.Caboose.Fetch(context.Background(), "/path/to/car", func(resource string, reader io.Reader) error { if resource != "/path/to/car" { t.Fatal("incorrect path in resource callback") } @@ -161,7 +118,7 @@ func TestResource(t *testing.T) { } // confirm that errors propogate. - if err := h.c.Fetch(context.Background(), "/path/to/car", func(resource string, reader io.Reader) error { + if err := h.Caboose.Fetch(context.Background(), "/path/to/car", func(resource string, reader io.Reader) error { return fmt.Errorf("test error") }); err.Error() != "test error" { t.Fatalf("expected error. got %v", err) @@ -169,12 +126,11 @@ func TestResource(t *testing.T) { // confirm partial failures work as expected. second := false - if err := h.c.Fetch(context.Background(), "/path/to/car1", func(resource string, reader io.Reader) error { + if err := h.Caboose.Fetch(context.Background(), "/path/to/car1", func(resource string, reader io.Reader) error { if resource == "/path/to/car1" { return caboose.ErrPartialResponse{StillNeed: []string{"/path/to/car2"}} } if resource == "/path/to/car2" { - fmt.Printf("doing second...\n") second = true return nil } @@ -187,79 +143,3 @@ func TestResource(t *testing.T) { t.Fatal("expected fall-over progress") } } - -type HarnessOption func(config *caboose.Config) - -func WithTieredHashingOpts(opts []tieredhashing.Option) HarnessOption { - return func(config *caboose.Config) { - config.TieredHashingOpts = opts - } -} - -func WithMaxFailuresBeforeCoolDown(max int) func(config *caboose.Config) { - return func(config *caboose.Config) { - config.MaxFetchFailuresBeforeCoolDown = max - } -} - -func WithCidCoolDownDuration(duration time.Duration) func(config *caboose.Config) { - return func(config *caboose.Config) { - config.FetchKeyCoolDownDuration = duration - } -} - -func BuildCabooseHarness(t *testing.T, n int, maxRetries int, opts ...HarnessOption) *CabooseHarness { - ch := &CabooseHarness{} - - ch.pool = make([]*ep, n) - purls := make([]string, n) - for i := 0; i < len(ch.pool); i++ { - ch.pool[i] = &ep{} - ch.pool[i].Setup() - purls[i] = strings.TrimPrefix(ch.pool[i].server.URL, "https://") - } - ch.goodOrch = true - orch := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - ch.gol.Lock() - defer ch.gol.Unlock() - if ch.goodOrch { - json.NewEncoder(w).Encode(purls) - } else { - json.NewEncoder(w).Encode([]string{}) - } - })) - - saturnClient := &http.Client{ - Transport: &http.Transport{ - TLSClientConfig: &tls.Config{ - InsecureSkipVerify: true, - ServerName: "example.com", - }, - }, - } - - ourl, _ := url.Parse(orch.URL) - - conf := &caboose.Config{ - OrchestratorEndpoint: ourl, - OrchestratorClient: http.DefaultClient, - LoggingEndpoint: *ourl, - LoggingClient: http.DefaultClient, - LoggingInterval: time.Hour, - - SaturnClient: saturnClient, - DoValidation: false, - PoolRefresh: time.Millisecond * 50, - MaxRetrievalAttempts: maxRetries, - } - - for _, opt := range opts { - opt(conf) - } - - bs, err := caboose.NewCaboose(conf) - require.NoError(t, err) - - ch.c = bs - return ch -} diff --git a/cmd/caboose/main.go b/cmd/caboose/main.go index 1aef900..22caecc 100644 --- a/cmd/caboose/main.go +++ b/cmd/caboose/main.go @@ -11,9 +11,9 @@ import ( "time" "github.com/filecoin-saturn/caboose" - carv2 "github.com/ipfs/boxo/ipld/car/v2" - "github.com/ipfs/boxo/ipld/car/v2/blockstore" "github.com/ipfs/go-cid" + carv2 "github.com/ipld/go-car/v2" + "github.com/ipld/go-car/v2/blockstore" cidlink "github.com/ipld/go-ipld-prime/linking/cid" "github.com/ipld/go-ipld-prime/storage/bsadapter" selectorparse "github.com/ipld/go-ipld-prime/traversal/selector/parse" @@ -49,7 +49,7 @@ func main1() int { cb, err := caboose.NewCaboose(&caboose.Config{ OrchestratorClient: &http.Client{ - Timeout: caboose.DefaultSaturnOrchestratorRequestTimeout, + Timeout: caboose.DefaultOrchestratorRequestTimeout, }, LoggingEndpoint: *le, @@ -58,7 +58,7 @@ func main1() int { DoValidation: true, PoolRefresh: caboose.DefaultPoolRefreshInterval, - SaturnClient: &saturnClient, + Client: &saturnClient, }) if err != nil { return err diff --git a/errors.go b/errors.go new file mode 100644 index 0000000..10934a8 --- /dev/null +++ b/errors.go @@ -0,0 +1,89 @@ +package caboose + +import ( + "errors" + "fmt" + "time" + + "github.com/ipfs/go-cid" +) + +// ErrNotImplemented is used when using caboose as a blockstore and attempting to mutate data. +var ErrNotImplemented error = errors.New("not implemented") + +// ErrNoBackend is returned when no connection can be made to learn of available backends. +var ErrNoBackend error = errors.New("no available backend") + +// ErrContentProviderNotFound is used to indicate that data could not be found upstream. +var ErrContentProviderNotFound error = errors.New("failed to find data") + +// ErrTimeout is used to indicate that attempts to find data timed out. +var ErrTimeout error = errors.New("upstream timed out") + +// ErrTooManyRequests indicates that the client has been rate limited by upstreams. +type ErrTooManyRequests struct { + Node string + retryAfter time.Duration +} + +func (e *ErrTooManyRequests) Error() string { + return fmt.Sprintf("upstream %s returned Too Many Requests error, please retry after %s", e.Node, humanRetry(e.retryAfter)) +} + +func (e *ErrTooManyRequests) RetryAfter() time.Duration { + return e.retryAfter +} + +// ErrCoolDown indicates that the requested CID has been requested too many times recently. +type ErrCoolDown struct { + Cid cid.Cid + Path string + retryAfter time.Duration +} + +func (e *ErrCoolDown) Error() string { + switch true { + case e.Cid != cid.Undef && e.Path != "": + return fmt.Sprintf("multiple retrieval failures seen for CID %q and Path %q, please retry after %s", e.Cid, e.Path, humanRetry(e.retryAfter)) + case e.Path != "": + return fmt.Sprintf("multiple retrieval failures seen for Path %q, please retry after %s", e.Path, humanRetry(e.retryAfter)) + case e.Cid != cid.Undef: + return fmt.Sprintf("multiple retrieval failures seen for CID %q, please retry after %s", e.Cid, humanRetry(e.retryAfter)) + default: + return fmt.Sprintf("multiple retrieval failures for unknown CID/Path (BUG), please retry after %s", humanRetry(e.retryAfter)) + } +} + +func (e *ErrCoolDown) RetryAfter() time.Duration { + return e.retryAfter +} + +func humanRetry(d time.Duration) string { + return d.Truncate(time.Second).String() +} + +// ErrPartialResponse can be returned from a DataCallback to indicate that some of the requested resource +// was successfully fetched, and that instead of retrying the full resource, that there are +// one or more more specific resources that should be fetched (via StillNeed) to complete the request. +type ErrPartialResponse struct { + error + StillNeed []string +} + +func (epr ErrPartialResponse) Error() string { + if epr.error != nil { + return fmt.Sprintf("partial response: %s", epr.error.Error()) + } + return "caboose received a partial response" +} + +// ErrInvalidResponse can be returned from a DataCallback to indicate that the data provided for the +// requested resource was explicitly 'incorrect' - that blocks not in the requested dag, or non-car-conforming +// data was returned. +type ErrInvalidResponse struct { + Message string +} + +func (e ErrInvalidResponse) Error() string { + return e.Message +} diff --git a/failure_test.go b/failure_test.go index 5543f3e..7b0399b 100644 --- a/failure_test.go +++ b/failure_test.go @@ -3,32 +3,32 @@ package caboose_test import ( "context" "errors" + "net/http" + "testing" + "time" + "github.com/filecoin-saturn/caboose" + "github.com/filecoin-saturn/caboose/internal/util" "github.com/ipfs/go-cid" "github.com/multiformats/go-multicodec" "github.com/stretchr/testify/require" - "net/http" - "net/http/httptest" - "sync" - "testing" - "time" ) var expRetryAfter = 1 * time.Second func TestHttp429(t *testing.T) { ctx := context.Background() - ch := BuildCabooseHarness(t, 3, 3) + ch := util.BuildCabooseHarness(t, 3, 3) testCid, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum(testBlock) - ch.failNodesWithCode(t, func(e *ep) bool { + ch.FailNodesWithCode(t, func(e *util.Endpoint) bool { return true }, http.StatusTooManyRequests) - _, err := ch.c.Get(ctx, testCid) + _, err := ch.Caboose.Get(ctx, testCid) require.Error(t, err) - var ferr *caboose.ErrSaturnTooManyRequests + var ferr *caboose.ErrTooManyRequests ok := errors.As(err, &ferr) require.True(t, ok) require.EqualValues(t, expRetryAfter, ferr.RetryAfter()) @@ -36,40 +36,40 @@ func TestHttp429(t *testing.T) { func TestCabooseFailures(t *testing.T) { ctx := context.Background() - ch := BuildCabooseHarness(t, 3, 3) + ch := util.BuildCabooseHarness(t, 3, 3) testCid, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum(testBlock) - ch.fetchAndAssertSuccess(t, ctx, testCid) + ch.FetchAndAssertSuccess(t, ctx, testCid) // fail primary - ch.failNodesAndAssertFetch(t, func(e *ep) bool { - return e.cnt > 0 && e.valid + ch.FailNodesAndAssertFetch(t, func(e *util.Endpoint) bool { + return e.Count() > 0 && e.Valid }, 2, testCid) // fail primary and secondary. - ch.failNodesAndAssertFetch(t, func(e *ep) bool { - return e.cnt > 0 && e.valid + ch.FailNodesAndAssertFetch(t, func(e *util.Endpoint) bool { + return e.Count() > 0 && e.Valid }, 1, testCid) // force pool down to the 1 remaining good node. - ch.stopOrchestrator() - ch.runFetchesForRandCids(50) - ch.fetchAndAssertSuccess(t, ctx, testCid) + ch.StopOrchestrator() + ch.RunFetchesForRandCids(50) + ch.FetchAndAssertSuccess(t, ctx, testCid) // invalidate ALL nodes - ch.failNodes(t, func(ep *ep) bool { + ch.FailNodes(t, func(ep *util.Endpoint) bool { return true }) - ch.runFetchesForRandCids(50) - require.EqualValues(t, 0, ch.nNodesAlive()) + ch.RunFetchesForRandCids(50) + require.EqualValues(t, 0, ch.NNodesAlive()) - _, err := ch.c.Get(context.Background(), testCid) + _, err := ch.Caboose.Get(context.Background(), testCid) require.Error(t, err) // more nodes should populate - ch.startOrchestrator() + ch.StartOrchestrator() cnt := 0 - ch.recoverNodes(t, func(ep *ep) bool { + ch.RecoverNodes(t, func(ep *util.Endpoint) bool { if cnt == 0 { cnt++ return true @@ -79,128 +79,7 @@ func TestCabooseFailures(t *testing.T) { time.Sleep(time.Millisecond * 100) //steady state-ify - ch.runFetchesForRandCids(50) - _, err = ch.c.Get(context.Background(), testCid) - require.NoError(t, err) -} - -type CabooseHarness struct { - c *caboose.Caboose - pool []*ep - - gol sync.Mutex - goodOrch bool -} - -func (ch *CabooseHarness) runFetchesForRandCids(n int) { - for i := 0; i < n; i++ { - randCid, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum([]byte{uint8(i)}) - _, _ = ch.c.Get(context.Background(), randCid) - } -} - -func (ch *CabooseHarness) fetchAndAssertCoolDownError(t *testing.T, ctx context.Context, cid cid.Cid) { - _, err := ch.c.Get(ctx, cid) - require.Error(t, err) - - var coolDownErr *caboose.ErrCoolDown - ok := errors.As(err, &coolDownErr) - require.True(t, ok) - require.EqualValues(t, cid, coolDownErr.Cid) - require.NotZero(t, coolDownErr.RetryAfter()) -} - -func (ch *CabooseHarness) fetchAndAssertFailure(t *testing.T, ctx context.Context, testCid cid.Cid, contains string) { - _, err := ch.c.Get(ctx, testCid) - require.Error(t, err) - require.Contains(t, err.Error(), contains) -} - -func (ch *CabooseHarness) fetchAndAssertSuccess(t *testing.T, ctx context.Context, c cid.Cid) { - blk, err := ch.c.Get(ctx, c) + ch.RunFetchesForRandCids(50) + _, err = ch.Caboose.Get(context.Background(), testCid) require.NoError(t, err) - require.NotEmpty(t, blk) -} -func (ch *CabooseHarness) failNodesWithCode(t *testing.T, selectorF func(ep *ep) bool, code int) { - for _, n := range ch.pool { - if selectorF(n) { - n.valid = false - n.httpCode = code - } - } -} - -func (ch *CabooseHarness) recoverNodes(t *testing.T, selectorF func(ep *ep) bool) { - for _, n := range ch.pool { - if selectorF(n) { - n.valid = true - } - } -} - -func (ch *CabooseHarness) failNodesAndAssertFetch(t *testing.T, selectorF func(ep *ep) bool, nAlive int, cid cid.Cid) { - ch.failNodes(t, selectorF) - require.EqualValues(t, nAlive, ch.nNodesAlive()) - ch.fetchAndAssertSuccess(t, context.Background(), cid) -} - -func (ch *CabooseHarness) failNodes(t *testing.T, selectorF func(ep *ep) bool) { - for _, n := range ch.pool { - if selectorF(n) { - n.valid = false - } - } -} - -func (ch *CabooseHarness) nNodesAlive() int { - cnt := 0 - for _, n := range ch.pool { - if n.valid { - cnt++ - } - } - return cnt -} - -func (ch *CabooseHarness) stopOrchestrator() { - ch.gol.Lock() - ch.goodOrch = false - ch.gol.Unlock() -} - -func (ch *CabooseHarness) startOrchestrator() { - ch.gol.Lock() - ch.goodOrch = true - ch.gol.Unlock() -} - -type ep struct { - server *httptest.Server - valid bool - cnt int - httpCode int - resp []byte -} - -var testBlock = []byte("hello World") - -func (e *ep) Setup() { - e.valid = true - e.resp = testBlock - e.server = httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - time.Sleep(time.Millisecond * 20) - e.cnt++ - if e.valid { - w.Write(e.resp) - } else { - if e.httpCode == http.StatusTooManyRequests { - w.Header().Set("Retry-After", "1") - } - if e.httpCode == 0 { - e.httpCode = 500 - } - w.WriteHeader(e.httpCode) - w.Write([]byte("error")) - } - })) } diff --git a/fetcher.go b/fetcher.go index 28ce84d..4a4b371 100644 --- a/fetcher.go +++ b/fetcher.go @@ -7,24 +7,24 @@ import ( "hash/crc32" "io" "net/http" + "net/http/httptrace" "os" "strconv" + "strings" "time" - "github.com/filecoin-saturn/caboose/tieredhashing" + "go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" "github.com/google/uuid" - blocks "github.com/ipfs/go-block-format" - "github.com/ipfs/go-cid" servertiming "github.com/mitchellh/go-server-timing" "github.com/tcnksm/go-httpstat" ) -var saturnReqTmpl = "/ipfs/%s?format=raw" - const ( + UserAgentTag = "STRN_ENV_TAG" + saturnNodeIdKey = "Saturn-Node-Id" saturnTransferIdKey = "Saturn-Transfer-Id" saturnCacheHitKey = "Saturn-Cache-Status" @@ -44,60 +44,22 @@ var ( "4.0 Mib", "8.0 Mib", "16.0 Mib", "32.0 Mib", "64.0 Mib", "128.0 Mib", "256.0 Mib", "512.0 Mib", "1.0 Gib", "2.0 Gib", "4.0 Gib", "8.0 Gib"} ) -// doFetch attempts to fetch a block from a given Saturn endpoint. It sends the retrieval logs to the logging endpoint upon a successful or failed attempt. -func (p *pool) doFetch(ctx context.Context, from string, c cid.Cid, attempt int) (b blocks.Block, rm tieredhashing.ResponseMetrics, e error) { - reqUrl := fmt.Sprintf(saturnReqTmpl, c) - - rm, e = p.fetchResource(ctx, from, reqUrl, "application/vnd.ipld.raw", attempt, func(rsrc string, r io.Reader) error { - block, err := io.ReadAll(io.LimitReader(r, maxBlockSize)) - if err != nil { - switch { - case err == io.EOF && len(block) >= maxBlockSize: - // we don't expect to see this error any time soon, but if IPFS - // ecosystem ever starts allowing bigger blocks, this message will save - // multiple people collective man-months in debugging ;-) - return fmt.Errorf("strn responded with a block bigger than maxBlockSize=%d", maxBlockSize-1) - case err == io.EOF: - // This is fine :-) - // Zero-length block may be valid (example: bafkreihdwdcefgh4dqkjv67uzcmw7ojee6xedzdetojuzjevtenxquvyku) - // We accept this as non-error and let it go over CID validation later. - default: - return fmt.Errorf("unable to read strn response body: %w", err) - } - } - - if p.config.DoValidation { - nc, err := c.Prefix().Sum(block) - if err != nil { - return blocks.ErrWrongHash - } - if !nc.Equals(c) { - return blocks.ErrWrongHash - } - } - b, e = blocks.NewBlockWithCid(block, c) - if e != nil { - return e - } - return nil - }) - return -} - // TODO Refactor to use a metrics collector that separates the collection of metrics from the actual fetching -func (p *pool) fetchResource(ctx context.Context, from string, resource string, mime string, attempt int, cb DataCallback) (rm tieredhashing.ResponseMetrics, err error) { - rm = tieredhashing.ResponseMetrics{} +func (p *pool) fetchResource(ctx context.Context, from *Node, resource string, mime string, attempt int, cb DataCallback) (err error) { resourceType := resourceTypeCar if mime == "application/vnd.ipld.raw" { resourceType = resourceTypeBlock } // if the context is already cancelled, there's nothing we can do here. if ce := ctx.Err(); ce != nil { - fetchRequestContextErrorTotalMetric.WithLabelValues(resourceType, fmt.Sprintf("%t", errors.Is(ce, context.Canceled)), "fetchResource-init").Add(1) - return rm, ce + return ce } - ctx, span := spanTrace(ctx, "Pool.FetchResource", trace.WithAttributes(attribute.String("from", from), attribute.String("of", resource), attribute.String("mime", mime))) + p.ActiveNodes.lk.RLock() + isCore := p.ActiveNodes.IsCore(from) + p.ActiveNodes.lk.RUnlock() + + ctx, span := spanTrace(ctx, "Pool.FetchResource", trace.WithAttributes(attribute.String("from", from.URL), attribute.String("of", resource), attribute.String("mime", mime), attribute.Bool("core", isCore))) defer span.End() requestId := uuid.NewString() @@ -111,12 +73,21 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, proto := "unknown" respReq := &http.Request{} received := 0 - reqUrl := fmt.Sprintf("https://%s%s", from, resource) + + reqUrl := "" + if strings.Contains(from.URL, "://") { + reqUrl = fmt.Sprintf("%s%s", from.URL, resource) + } else { + reqUrl = fmt.Sprintf("https://%s%s", from.URL, resource) + } + var respHeader http.Header saturnNodeId := "" saturnTransferId := "" isCacheHit := false networkError := "" + verificationError := "" + otherError := "" isBlockRequest := false if mime == "application/vnd.ipld.raw" { @@ -145,10 +116,6 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, if cacheHit == saturnCacheHit { isCacheHit = true } - - for k, v := range respHeader { - received = received + len(k) + len(v) - } } durationSecs := time.Since(start).Seconds() @@ -161,19 +128,14 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, ttfbMs = fb.Sub(start).Milliseconds() cacheStatus := getCacheStatus(isCacheHit) - if isBlockRequest { - fetchSizeBlockMetric.Observe(float64(received)) - } else { + if !isBlockRequest { fetchSizeCarMetric.WithLabelValues("success").Observe(float64(received)) } durationMs := response_success_end.Sub(start).Milliseconds() fetchSpeedPerPeerSuccessMetric.WithLabelValues(resourceType, cacheStatus).Observe(float64(received) / float64(durationMs)) fetchCacheCountSuccessTotalMetric.WithLabelValues(resourceType, cacheStatus).Add(1) // track individual block metrics separately - if isBlockRequest { - fetchTTFBPerBlockPerPeerSuccessMetric.WithLabelValues(cacheStatus).Observe(float64(ttfbMs)) - fetchDurationPerBlockPerPeerSuccessMetric.WithLabelValues(cacheStatus).Observe(float64(response_success_end.Sub(start).Milliseconds())) - } else { + if !isBlockRequest { ci := 0 for index, value := range carSizes { if float64(received) < value { @@ -186,20 +148,11 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, fetchTTFBPerCARPerPeerSuccessMetric.WithLabelValues(cacheStatus, carSizeStr).Observe(float64(ttfbMs)) fetchDurationPerCarPerPeerSuccessMetric.WithLabelValues(cacheStatus).Observe(float64(response_success_end.Sub(start).Milliseconds())) } - - // update L1 server timings - updateSuccessServerTimingMetrics(respHeader.Values(servertiming.HeaderKey), resourceType, isCacheHit, durationMs, ttfbMs, received) } else { - if isBlockRequest { - fetchDurationPerBlockPerPeerFailureMetric.Observe(float64(time.Since(start).Milliseconds())) - } else { + if !isBlockRequest { fetchDurationPerCarPerPeerFailureMetric.Observe(float64(time.Since(start).Milliseconds())) fetchSizeCarMetric.WithLabelValues("failure").Observe(float64(received)) } - - if code == http.StatusBadGateway || code == http.StatusGatewayTimeout { - updateLassie5xxTime(respHeader.Values(servertiming.HeaderKey), resourceType) - } } if err == nil || !errors.Is(err, context.Canceled) { @@ -215,12 +168,14 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, HTTPProtocol: proto, TTFBMS: int(ttfbMs), // my address - Range: "", - Referrer: respReq.Referer(), - UserAgent: respReq.UserAgent(), - NodeId: saturnNodeId, - NodeIpAddress: from, - IfNetworkError: networkError, + Range: "", + Referrer: respReq.Referer(), + UserAgent: respReq.UserAgent(), + NodeId: saturnNodeId, + NodeIpAddress: from.URL, + IfNetworkError: networkError, + VerificationError: verificationError, + OtherError: otherError, } } } @@ -230,19 +185,21 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, // which is the amount of time without any NEW data from the server, but // that can be added later. We need both because a slow trickle of data // could take a large amount of time. - requestTimeout := DefaultSaturnCarRequestTimeout + requestTimeout := DefaultCarRequestTimeout if isBlockRequest { - requestTimeout = DefaultSaturnBlockRequestTimeout + requestTimeout = DefaultBlockRequestTimeout } reqCtx, cancel := context.WithTimeout(ctx, requestTimeout) defer cancel() - req, err := http.NewRequestWithContext(reqCtx, http.MethodGet, reqUrl, nil) + clientTrace := otelhttptrace.NewClientTrace(reqCtx) + subReqCtx := httptrace.WithClientTrace(reqCtx, clientTrace) + req, err := http.NewRequestWithContext(subReqCtx, http.MethodGet, reqUrl, nil) if err != nil { - if recordIfContextErr(resourceType, reqCtx, "build-http-request") { - return rm, reqCtx.Err() + if isCtxError(reqCtx) { + return reqCtx.Err() } - return rm, err + return err } req.Header.Add("Accept", mime) @@ -256,20 +213,19 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, } agent := req.Header.Get("User-Agent") - req.Header.Set("User-Agent", os.Getenv(SaturnEnvKey)+"/"+agent) + req.Header.Set("User-Agent", os.Getenv(UserAgentTag)+"/"+agent) //trace req = req.WithContext(httpstat.WithHTTPStat(req.Context(), &result)) var resp *http.Response saturnCallsTotalMetric.WithLabelValues(resourceType).Add(1) - startReq := time.Now() - resp, err = p.config.SaturnClient.Do(req) + resp, err = p.config.Client.Do(req) if err != nil { - if recordIfContextErr(resourceType, reqCtx, "send-http-request") { + if isCtxError(reqCtx) { if errors.Is(err, context.Canceled) { - return rm, reqCtx.Err() + return reqCtx.Err() } } @@ -279,17 +235,14 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, saturnConnectionFailureTotalMetric.WithLabelValues(resourceType, "non-timeout").Add(1) } networkError = err.Error() - rm.ConnFailure = true - return rm, fmt.Errorf("http request failed: %w", err) + return fmt.Errorf("http request failed: %w", err) } respHeader = resp.Header - headerTTFBPerPeerMetric.WithLabelValues(resourceType, getCacheStatus(respHeader.Get(saturnCacheHitKey) == saturnCacheHit)).Observe(float64(time.Since(startReq).Milliseconds())) defer resp.Body.Close() code = resp.StatusCode - rm.ResponseCode = code proto = resp.Proto respReq = resp.Request @@ -299,7 +252,7 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, if subReqTiming, err := servertiming.ParseHeader(th); err == nil { for _, m := range subReqTiming.Metrics { m.Extra["attempt"] = fmt.Sprintf("%d", attempt) - m.Extra["subreq"] = subReqID(from, resource) + m.Extra["subreq"] = subReqID(from.URL, resource) timing.Add(m) } } @@ -320,23 +273,23 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, } if retryAfter == 0 { - retryAfter = p.config.SaturnNodeCoolOff + retryAfter = p.config.CoolOff } - return rm, fmt.Errorf("http error from strn: %d, err=%w", resp.StatusCode, &ErrSaturnTooManyRequests{retryAfter: retryAfter, Node: from}) + return fmt.Errorf("http error from strn: %d, err=%w", resp.StatusCode, &ErrTooManyRequests{retryAfter: retryAfter, Node: from.URL}) } // empty body so it can be re-used. if resp.StatusCode == http.StatusGatewayTimeout { - return rm, fmt.Errorf("http error from strn: %d, err=%w", resp.StatusCode, ErrSaturnTimeout) + return fmt.Errorf("http error from strn: %d, err=%w", resp.StatusCode, ErrTimeout) } // This should only be 502, but L1s were not translating 404 from Lassie, so we have to support both for now. if resp.StatusCode == http.StatusNotFound || resp.StatusCode == http.StatusBadGateway { - return rm, fmt.Errorf("http error from strn: %d, err=%w", resp.StatusCode, ErrContentProviderNotFound) + return fmt.Errorf("http error from strn: %d, err=%w", resp.StatusCode, ErrContentProviderNotFound) } - return rm, fmt.Errorf("http error from strn: %d", resp.StatusCode) + return fmt.Errorf("http error from strn: %d", resp.StatusCode) } if respHeader.Get(saturnCacheHitKey) == saturnCacheHit { isCacheHit = true @@ -345,13 +298,11 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, wrapped := TrackingReader{resp.Body, time.Time{}, 0} err = cb(resource, &wrapped) received = wrapped.len - // drain body so it can be re-used. - _, _ = io.Copy(io.Discard, resp.Body) if err != nil { - if recordIfContextErr(resourceType, reqCtx, "read-http-response") { + if isCtxError(reqCtx) { if errors.Is(err, context.Canceled) { - return rm, reqCtx.Err() + return reqCtx.Err() } } if errors.Is(err, context.DeadlineExceeded) { @@ -361,9 +312,16 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, saturnCallsFailureTotalMetric.WithLabelValues(resourceType, fmt.Sprintf("failed-response-read-%s", getCacheStatus(isCacheHit)), fmt.Sprintf("%d", code)).Add(1) } - networkError = err.Error() - rm.NetworkError = true - return rm, err + var target = ErrInvalidResponse{} + if errors.As(err, &target) { + verificationError = err.Error() + goLogger.Errorw("failed to read response; verification error", "err", err.Error()) + } else { + otherError = err.Error() + goLogger.Errorw("failed to read response; no verification error", "err", err.Error()) + } + + return err } fb = wrapped.firstByte @@ -371,6 +329,7 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, // trace-metrics // request life-cycle metrics + saturnCallsSuccessTotalMetric.WithLabelValues(resourceType, getCacheStatus(isCacheHit)).Add(1) fetchRequestSuccessTimeTraceMetric.WithLabelValues(resourceType, getCacheStatus(isCacheHit), "tcp_connection").Observe(float64(result.TCPConnection.Milliseconds())) fetchRequestSuccessTimeTraceMetric.WithLabelValues(resourceType, getCacheStatus(isCacheHit), "tls_handshake").Observe(float64(result.TLSHandshake.Milliseconds())) @@ -385,79 +344,18 @@ func (p *pool) fetchResource(ctx context.Context, from string, resource string, fetchRequestSuccessTimeTraceMetric.WithLabelValues(resourceType, getCacheStatus(isCacheHit), "start_transfer"). Observe(float64(result.StartTransfer.Milliseconds())) - rm.TTFBMs = float64(wrapped.firstByte.Sub(start).Milliseconds()) - rm.Success = true - rm.SpeedPerMs = float64(received) / float64(response_success_end.Sub(start).Milliseconds()) - saturnCallsSuccessTotalMetric.WithLabelValues(resourceType, getCacheStatus(isCacheHit)).Add(1) + from.RecordSuccess(start, float64(wrapped.firstByte.Sub(start).Milliseconds()), float64(received)/float64(response_success_end.Sub(start).Milliseconds())) - return rm, nil + return nil } -func recordIfContextErr(resourceType string, ctx context.Context, requestState string) bool { +func isCtxError(ctx context.Context) bool { if ce := ctx.Err(); ce != nil { - fetchRequestContextErrorTotalMetric.WithLabelValues(resourceType, fmt.Sprintf("%t", errors.Is(ce, context.Canceled)), requestState).Add(1) return true } return false } -func updateLassie5xxTime(timingHeaders []string, resourceType string) { - if len(timingHeaders) == 0 { - goLogger.Debug("no timing headers in request response.") - return - } - - for _, th := range timingHeaders { - if subReqTiming, err := servertiming.ParseHeader(th); err == nil { - for _, m := range subReqTiming.Metrics { - switch m.Name { - case "shim_lassie_headers": - if m.Duration.Milliseconds() != 0 { - lassie5XXTimeMetric.WithLabelValues(resourceType).Observe(float64(m.Duration.Milliseconds())) - } - return - default: - } - } - } - } -} - -// todo: refactor for dryness -func updateSuccessServerTimingMetrics(timingHeaders []string, resourceType string, isCacheHit bool, totalTimeMs, ttfbMs int64, recieved int) { - if len(timingHeaders) == 0 { - goLogger.Debug("no timing headers in request response.") - return - } - - for _, th := range timingHeaders { - if subReqTiming, err := servertiming.ParseHeader(th); err == nil { - for _, m := range subReqTiming.Metrics { - switch m.Name { - case "shim_lassie_headers": - if m.Duration.Milliseconds() != 0 && !isCacheHit { - fetchDurationPerPeerSuccessCacheMissTotalLassieMetric.WithLabelValues(resourceType).Observe(float64(m.Duration.Milliseconds())) - } - - case "nginx": - // sanity checks - if totalTimeMs != 0 && ttfbMs != 0 && m.Duration.Milliseconds() != 0 { - fetchDurationPerPeerSuccessTotalL1NodeMetric.WithLabelValues(resourceType, getCacheStatus(isCacheHit)).Observe(float64(m.Duration.Milliseconds())) - networkTimeMs := totalTimeMs - m.Duration.Milliseconds() - if networkTimeMs > 0 { - s := float64(recieved) / float64(networkTimeMs) - fetchNetworkSpeedPerPeerSuccessMetric.WithLabelValues(resourceType).Observe(s) - } - networkLatencyMs := ttfbMs - m.Duration.Milliseconds() - fetchNetworkLatencyPeerSuccessMetric.WithLabelValues(resourceType).Observe(float64(networkLatencyMs)) - } - default: - } - } - } - } -} - func getCacheStatus(isCacheHit bool) string { if isCacheHit { return "Cache-hit" diff --git a/go.mod b/go.mod index 99229d5..8bca168 100644 --- a/go.mod +++ b/go.mod @@ -3,41 +3,43 @@ module github.com/filecoin-saturn/caboose go 1.19 require ( - github.com/asecurityteam/rolling v0.0.0-20230418204413-b4052899307d github.com/google/uuid v1.3.0 - github.com/ipfs/boxo v0.8.0-rc2.0.20230329082438-360b031ed895 + github.com/ipfs/boxo v0.11.0 github.com/ipfs/go-block-format v0.1.2 - github.com/ipfs/go-cid v0.4.0 + github.com/ipfs/go-cid v0.4.1 github.com/ipfs/go-log/v2 v2.5.1 - github.com/ipld/go-car v0.6.0 - github.com/ipld/go-car/v2 v2.9.1-0.20230325062757-fff0e4397a3d + github.com/ipld/go-car v0.6.2 + github.com/ipld/go-car/v2 v2.10.2-0.20230622090957-499d0c909d33 github.com/ipld/go-ipld-prime v0.20.0 github.com/ipld/go-ipld-prime/storage/bsadapter v0.0.0-20230102063945-1a409dc236dd github.com/mitchellh/go-server-timing v1.0.1 - github.com/multiformats/go-multicodec v0.8.1 + github.com/multiformats/go-multicodec v0.9.0 github.com/patrickmn/go-cache v2.1.0+incompatible github.com/prometheus/client_golang v1.14.0 - github.com/serialx/hashring v0.0.0-20200727003509-22c0c7ab6b1b - github.com/stretchr/testify v1.8.2 + github.com/stretchr/testify v1.8.4 github.com/tcnksm/go-httpstat v0.2.0 github.com/urfave/cli/v2 v2.24.2 - go.opentelemetry.io/otel v1.14.0 - go.opentelemetry.io/otel/trace v1.14.0 + github.com/willscott/go-requestcontext v0.0.1 + github.com/willscott/hashring v0.0.0-20230731155239-15f93a2dfb44 + github.com/zyedidia/generic v1.2.2-0.20230625215236-3404399b19f1 + go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.43.0 + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.43.0 + go.opentelemetry.io/otel v1.17.0 + go.opentelemetry.io/otel/trace v1.17.0 ) require ( github.com/alecthomas/units v0.0.0-20210927113745-59d0afb8317a // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/cespare/xxhash v1.1.0 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect github.com/crackcomm/go-gitignore v0.0.0-20170627025303-887ab5e44cc3 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.1.0 // indirect github.com/dustin/go-humanize v1.0.0 // indirect - github.com/felixge/httpsnoop v1.0.0 // indirect + github.com/felixge/httpsnoop v1.0.3 // indirect github.com/gabriel-vasile/mimetype v1.4.1 // indirect - github.com/go-logr/logr v1.2.3 // indirect + github.com/go-logr/logr v1.2.4 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/gddo v0.0.0-20180823221919-9d8ff1c67be5 // indirect @@ -46,6 +48,7 @@ require ( github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/hashicorp/golang-lru v0.5.4 // indirect + github.com/hashicorp/golang-lru/v2 v2.0.1 // indirect github.com/ipfs/bbloom v0.0.4 // indirect github.com/ipfs/go-bitfield v1.1.0 // indirect github.com/ipfs/go-blockservice v0.5.0 // indirect @@ -56,14 +59,12 @@ require ( github.com/ipfs/go-ipfs-redirects-file v0.1.1 // indirect github.com/ipfs/go-ipfs-util v0.0.2 // indirect github.com/ipfs/go-ipld-cbor v0.0.6 // indirect - github.com/ipfs/go-ipld-format v0.4.0 // indirect - github.com/ipfs/go-ipld-legacy v0.1.1 // indirect - github.com/ipfs/go-ipns v0.3.0 // indirect - github.com/ipfs/go-libipfs v0.6.0 // indirect + github.com/ipfs/go-ipld-format v0.5.0 // indirect + github.com/ipfs/go-ipld-legacy v0.2.1 // indirect github.com/ipfs/go-log v1.0.5 // indirect - github.com/ipfs/go-merkledag v0.10.0 // indirect + github.com/ipfs/go-merkledag v0.11.0 // indirect github.com/ipfs/go-metrics-interface v0.0.1 // indirect - github.com/ipfs/go-unixfsnode v1.6.0 // indirect + github.com/ipfs/go-unixfsnode v1.7.1 // indirect github.com/ipfs/go-verifcid v0.0.2 // indirect github.com/ipld/go-codec-dagpb v1.6.0 // indirect github.com/jbenet/goprocess v0.1.4 // indirect @@ -73,10 +74,10 @@ require ( github.com/libp2p/go-doh-resolver v0.4.0 // indirect github.com/libp2p/go-libp2p v0.26.3 // indirect github.com/libp2p/go-libp2p-asn-util v0.2.0 // indirect - github.com/libp2p/go-libp2p-kad-dht v0.21.1 // indirect + github.com/libp2p/go-libp2p-kad-dht v0.23.0 // indirect github.com/libp2p/go-libp2p-kbucket v0.5.0 // indirect github.com/libp2p/go-libp2p-record v0.2.0 // indirect - github.com/libp2p/go-libp2p-routing-helpers v0.4.0 // indirect + github.com/libp2p/go-libp2p-routing-helpers v0.7.0 // indirect github.com/libp2p/go-msgio v0.3.0 // indirect github.com/libp2p/go-netroute v0.2.1 // indirect github.com/mattn/go-isatty v0.0.17 // indirect @@ -88,8 +89,8 @@ require ( github.com/multiformats/go-base36 v0.2.0 // indirect github.com/multiformats/go-multiaddr v0.8.0 // indirect github.com/multiformats/go-multiaddr-dns v0.3.1 // indirect - github.com/multiformats/go-multibase v0.1.1 // indirect - github.com/multiformats/go-multihash v0.2.1 // indirect + github.com/multiformats/go-multibase v0.2.0 // indirect + github.com/multiformats/go-multihash v0.2.3 // indirect github.com/multiformats/go-multistream v0.4.1 // indirect github.com/multiformats/go-varint v0.0.7 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect @@ -109,6 +110,7 @@ require ( github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 // indirect github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 // indirect go.opencensus.io v0.24.0 // indirect + go.opentelemetry.io/otel/metric v1.17.0 // indirect go.uber.org/atomic v1.10.0 // indirect go.uber.org/multierr v1.9.0 // indirect go.uber.org/zap v1.24.0 // indirect @@ -120,7 +122,8 @@ require ( golang.org/x/sys v0.6.0 // indirect golang.org/x/tools v0.3.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect - google.golang.org/protobuf v1.28.1 // indirect + gonum.org/v1/gonum v0.11.0 // indirect + google.golang.org/protobuf v1.30.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect lukechampine.com/blake3 v1.1.7 // indirect ) diff --git a/go.sum b/go.sum index fd223d3..362284f 100644 --- a/go.sum +++ b/go.sum @@ -33,8 +33,6 @@ cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9 dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= -github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= @@ -42,8 +40,6 @@ github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRF github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/alecthomas/units v0.0.0-20210927113745-59d0afb8317a h1:E/8AP5dFtMhl5KPJz66Kt9G0n+7Sn41Fy1wv9/jHOrc= github.com/alecthomas/units v0.0.0-20210927113745-59d0afb8317a/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE= -github.com/asecurityteam/rolling v0.0.0-20230418204413-b4052899307d h1:OD9AM8JZUHPrkEa9/SgQW2cCX6Om8d6n0akPS5leWJA= -github.com/asecurityteam/rolling v0.0.0-20230418204413-b4052899307d/go.mod h1:tWDU1S7csNXWrzNpkbCk/dXpZkVcL4PfKn6Akwrffok= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -51,8 +47,6 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= -github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= @@ -85,11 +79,11 @@ github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymF github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/felixge/httpsnoop v1.0.0 h1:gh8fMGz0rlOv/1WmRZm7OgncIOTsAj21iNJot48omJQ= github.com/felixge/httpsnoop v1.0.0/go.mod h1:3+D9sFq0ahK/JeJPhCBUV1xlf4/eIYrUQaxulT0VzX8= +github.com/felixge/httpsnoop v1.0.3 h1:s/nj+GCswXYzN5v2DpNMuMQYe+0DDwt5WVCU6CWBdXk= +github.com/felixge/httpsnoop v1.0.3/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/flynn/noise v1.0.0 h1:DlTHqmzmvcEiKj+4RYo/imoswx/4r6iBlCMfVtrMXpQ= github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJnXKk= -github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/frankban/quicktest v1.14.4 h1:g2rn0vABPOOXmZUj+vbmUp0lPoXEMuhTpIluN0XL9UY= github.com/gabriel-vasile/mimetype v1.4.1 h1:TRWk7se+TOjCYgRth7+1/OYLNiRNIotknkFtf/dnN7Q= github.com/gabriel-vasile/mimetype v1.4.1/go.mod h1:05Vi0w3Y9c/lNvJOdmIwvrrAhX3rYhfQQCaf9VJcv7M= @@ -105,8 +99,8 @@ github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= -github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.2.4 h1:g01GSCwiDw2xSZfjJ2/T9M+S6pFdcNtFYsp+Y43HYDQ= +github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= @@ -194,30 +188,28 @@ github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc= github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= +github.com/hashicorp/golang-lru/v2 v2.0.1 h1:5pv5N1lT1fjLg2VQ5KWc7kmucp2x/kvFOnxuVTqZ6x4= +github.com/hashicorp/golang-lru/v2 v2.0.1/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM= github.com/huin/goupnp v1.0.3 h1:N8No57ls+MnjlB+JPiCVSOyy/ot7MJTqlo7rn+NYSqQ= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ipfs/bbloom v0.0.4 h1:Gi+8EGJ2y5qiD5FbsbpX/TMNcJw8gSqr7eyjHa4Fhvs= github.com/ipfs/bbloom v0.0.4/go.mod h1:cS9YprKXpoZ9lT0n/Mw/a6/aFV6DTjTLYHeA+gyqMG0= -github.com/ipfs/boxo v0.8.0-rc2.0.20230329082438-360b031ed895 h1:bh+8xMBQSOnieUSg7qToTFhPpf4Oc8QkvPSRmQSrnpc= -github.com/ipfs/boxo v0.8.0-rc2.0.20230329082438-360b031ed895/go.mod h1:RIsi4CnTyQ7AUsNn5gXljJYZlQrHBMnJp94p73liFiA= +github.com/ipfs/boxo v0.11.0 h1:urMxhZ3xoF4HssJVD3+0ssGT9pptEfHfbL8DYdoWFlg= +github.com/ipfs/boxo v0.11.0/go.mod h1:8IfDmp+FzFGcF4zjAgHMVPpwYw4AjN9ePEzDfkaYJ1w= github.com/ipfs/go-bitfield v1.1.0 h1:fh7FIo8bSwaJEh6DdTWbCeZ1eqOaOkKFI74SCnsWbGA= github.com/ipfs/go-bitfield v1.1.0/go.mod h1:paqf1wjq/D2BBmzfTVFlJQ9IlFOZpg422HL0HqsGWHU= github.com/ipfs/go-bitswap v0.11.0 h1:j1WVvhDX1yhG32NTC9xfxnqycqYIlhzEzLXG/cU1HyQ= github.com/ipfs/go-block-format v0.0.2/go.mod h1:AWR46JfpcObNfg3ok2JHDUfdiHRgWhJgCQF+KIgOPJY= -github.com/ipfs/go-block-format v0.0.3/go.mod h1:4LmD4ZUw0mhO+JSKdpWwrzATiEfM7WWgQ8H5l6P8MVk= github.com/ipfs/go-block-format v0.1.2 h1:GAjkfhVx1f4YTODS6Esrj1wt2HhrtwTnhEr+DyPUaJo= github.com/ipfs/go-block-format v0.1.2/go.mod h1:mACVcrxarQKstUU3Yf/RdwbC4DzPV6++rO2a3d+a/KE= github.com/ipfs/go-blockservice v0.5.0 h1:B2mwhhhVQl2ntW2EIpaWPwSCxSuqr5fFA93Ms4bYLEY= github.com/ipfs/go-blockservice v0.5.0/go.mod h1:W6brZ5k20AehbmERplmERn8o2Ni3ZZubvAxaIUeaT6w= github.com/ipfs/go-cid v0.0.1/go.mod h1:GHWU/WuQdMPmIosc4Yn1bcCT7dSeX4lBafM7iqUPQvM= -github.com/ipfs/go-cid v0.0.2/go.mod h1:GHWU/WuQdMPmIosc4Yn1bcCT7dSeX4lBafM7iqUPQvM= github.com/ipfs/go-cid v0.0.3/go.mod h1:GHWU/WuQdMPmIosc4Yn1bcCT7dSeX4lBafM7iqUPQvM= -github.com/ipfs/go-cid v0.0.4/go.mod h1:4LLaPOQwmk5z9LBgQnpkivrx8BJjUyGwTXCd5Xfj6+M= github.com/ipfs/go-cid v0.0.5/go.mod h1:plgt+Y5MnOey4vO4UlUazGqdbEXuFYitED67FexhXog= github.com/ipfs/go-cid v0.0.6/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqgC/I= -github.com/ipfs/go-cid v0.0.7/go.mod h1:6Ux9z5e+HpkQdckYoX1PG/6xqKspzlEIR5SDmgqgC/I= -github.com/ipfs/go-cid v0.4.0 h1:a4pdZq0sx6ZSxbCizebnKiMCx/xI/aBBFlB73IgH4rA= -github.com/ipfs/go-cid v0.4.0/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk= +github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s= +github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk= github.com/ipfs/go-datastore v0.5.0/go.mod h1:9zhEApYMTl17C8YDp7JmU7sQZi2/wqiYh73hakZ90Bk= github.com/ipfs/go-datastore v0.6.0 h1:JKyz+Gvz1QEZw0LsX1IBn+JFCJQH4SJVFtM4uWU0Myk= github.com/ipfs/go-datastore v0.6.0/go.mod h1:rt5M3nNbSO/8q1t4LNkLyUwRs8HupMeN/8O4Vn9YAT8= @@ -244,37 +236,31 @@ github.com/ipfs/go-ipfs-util v0.0.2/go.mod h1:CbPtkWJzjLdEcezDns2XYaehFVNXG9zrdr github.com/ipfs/go-ipld-cbor v0.0.6 h1:pYuWHyvSpIsOOLw4Jy7NbBkCyzLDcl64Bf/LZW7eBQ0= github.com/ipfs/go-ipld-cbor v0.0.6/go.mod h1:ssdxxaLJPXH7OjF5V4NSjBbcfh+evoR4ukuru0oPXMA= github.com/ipfs/go-ipld-format v0.0.1/go.mod h1:kyJtbkDALmFHv3QR6et67i35QzO3S0dCDnkOJhcZkms= -github.com/ipfs/go-ipld-format v0.2.0/go.mod h1:3l3C1uKoadTPbeNfrDi+xMInYKlx2Cvg1BuydPSdzQs= -github.com/ipfs/go-ipld-format v0.4.0 h1:yqJSaJftjmjc9jEOFYlpkwOLVKv68OD27jFLlSghBlQ= -github.com/ipfs/go-ipld-format v0.4.0/go.mod h1:co/SdBE8h99968X0hViiw1MNlh6fvxxnHpvVLnH7jSM= -github.com/ipfs/go-ipld-legacy v0.1.1 h1:BvD8PEuqwBHLTKqlGFTHSwrwFOMkVESEvwIYwR2cdcc= -github.com/ipfs/go-ipld-legacy v0.1.1/go.mod h1:8AyKFCjgRPsQFf15ZQgDB8Din4DML/fOmKZkkFkrIEg= -github.com/ipfs/go-ipns v0.3.0 h1:ai791nTgVo+zTuq2bLvEGmWP1M0A6kGTXUsgv/Yq67A= -github.com/ipfs/go-ipns v0.3.0/go.mod h1:3cLT2rbvgPZGkHJoPO1YMJeh6LtkxopCkKFcio/wE24= -github.com/ipfs/go-libipfs v0.6.0 h1:3FuckAJEm+zdHbHbf6lAyk0QUzc45LsFcGw102oBCZM= -github.com/ipfs/go-libipfs v0.6.0/go.mod h1:UjjDIuehp2GzlNP0HEr5I9GfFT7zWgst+YfpUEIThtw= +github.com/ipfs/go-ipld-format v0.5.0 h1:WyEle9K96MSrvr47zZHKKcDxJ/vlpET6PSiQsAFO+Ds= +github.com/ipfs/go-ipld-format v0.5.0/go.mod h1:ImdZqJQaEouMjCvqCe0ORUS+uoBmf7Hf+EO/jh+nk3M= +github.com/ipfs/go-ipld-legacy v0.2.1 h1:mDFtrBpmU7b//LzLSypVrXsD8QxkEWxu5qVxN99/+tk= +github.com/ipfs/go-ipld-legacy v0.2.1/go.mod h1:782MOUghNzMO2DER0FlBR94mllfdCJCkTtDtPM51otM= github.com/ipfs/go-log v1.0.5 h1:2dOuUCB1Z7uoczMWgAyDck5JLb72zHzrMnGnCNNbvY8= github.com/ipfs/go-log v1.0.5/go.mod h1:j0b8ZoR+7+R99LD9jZ6+AJsrzkPbSXbZfGakb5JPtIo= github.com/ipfs/go-log/v2 v2.1.3/go.mod h1:/8d0SH3Su5Ooc31QlL1WysJhvyOTDCjcCZ9Axpmri6g= github.com/ipfs/go-log/v2 v2.5.1 h1:1XdUzF7048prq4aBjDQQ4SL5RxftpRGdXhNRwKSAlcY= github.com/ipfs/go-log/v2 v2.5.1/go.mod h1:prSpmC1Gpllc9UYWxDiZDreBYw7zp4Iqp1kOLU9U5UI= -github.com/ipfs/go-merkledag v0.10.0 h1:IUQhj/kzTZfam4e+LnaEpoiZ9vZF6ldimVlby+6OXL4= -github.com/ipfs/go-merkledag v0.10.0/go.mod h1:zkVav8KiYlmbzUzNM6kENzkdP5+qR7+2mCwxkQ6GIj8= +github.com/ipfs/go-merkledag v0.11.0 h1:DgzwK5hprESOzS4O1t/wi6JDpyVQdvm9Bs59N/jqfBY= +github.com/ipfs/go-merkledag v0.11.0/go.mod h1:Q4f/1ezvBiJV0YCIXvt51W/9/kqJGH4I1LsA7+djsM4= github.com/ipfs/go-metrics-interface v0.0.1 h1:j+cpbjYvu4R8zbleSs36gvB7jR+wsL2fGD6n0jO4kdg= github.com/ipfs/go-metrics-interface v0.0.1/go.mod h1:6s6euYU4zowdslK0GKHmqaIZ3j/b/tL7HTWtJ4VPgWY= github.com/ipfs/go-peertaskqueue v0.8.1 h1:YhxAs1+wxb5jk7RvS0LHdyiILpNmRIRnZVztekOF0pg= github.com/ipfs/go-unixfs v0.4.5 h1:wj8JhxvV1G6CD7swACwSKYa+NgtdWC1RUit+gFnymDU= -github.com/ipfs/go-unixfsnode v1.6.0 h1:JOSA02yaLylRNi2rlB4ldPr5VcZhcnaIVj5zNLcOjDo= -github.com/ipfs/go-unixfsnode v1.6.0/go.mod h1:PVfoyZkX1B34qzT3vJO4nsLUpRCyhnMuHBznRcXirlk= +github.com/ipfs/go-unixfsnode v1.7.1 h1:RRxO2b6CSr5UQ/kxnGzaChTjp5LWTdf3Y4n8ANZgB/s= +github.com/ipfs/go-unixfsnode v1.7.1/go.mod h1:PVfoyZkX1B34qzT3vJO4nsLUpRCyhnMuHBznRcXirlk= github.com/ipfs/go-verifcid v0.0.2 h1:XPnUv0XmdH+ZIhLGKg6U2vaPaRDXb9urMyNVCE7uvTs= github.com/ipfs/go-verifcid v0.0.2/go.mod h1:40cD9x1y4OWnFXbLNJYRe7MpNvWlMn3LZAG5Wb4xnPU= -github.com/ipld/go-car v0.6.0 h1:d5QrGLnHAxiNLHor+DKGrLdqnM0dQJh2whfSXRDq6J0= -github.com/ipld/go-car v0.6.0/go.mod h1:tBrW1XZ3L2XipLxA69RnTVGW3rve6VX4TbaTYkq8aEA= -github.com/ipld/go-car/v2 v2.9.1-0.20230325062757-fff0e4397a3d h1:22g+x1tgWSXK34i25qjs+afr7basaneEkHaglBshd2g= -github.com/ipld/go-car/v2 v2.9.1-0.20230325062757-fff0e4397a3d/go.mod h1:SH2pi/NgfGBsV/CGBAQPxMfghIgwzbh5lQ2N+6dNRI8= +github.com/ipld/go-car v0.6.2 h1:Hlnl3Awgnq8icK+ze3iRghk805lu8YNq3wlREDTF2qc= +github.com/ipld/go-car v0.6.2/go.mod h1:oEGXdwp6bmxJCZ+rARSkDliTeYnVzv3++eXajZ+Bmr8= +github.com/ipld/go-car/v2 v2.10.2-0.20230622090957-499d0c909d33 h1:0OZwzSYWIuiKEOXd/2vm5cMcEmmGLFn+1h6lHELCm3s= +github.com/ipld/go-car/v2 v2.10.2-0.20230622090957-499d0c909d33/go.mod h1:sQEkXVM3csejlb1kCCb+vQ/pWBKX9QtvsrysMQjOgOg= github.com/ipld/go-codec-dagpb v1.6.0 h1:9nYazfyu9B1p3NAgfVdpRco3Fs2nFC72DqVsMj6rOcc= github.com/ipld/go-codec-dagpb v1.6.0/go.mod h1:ANzFhfP2uMJxRBr8CE+WQWs5UsNa0pYtmKZ+agnUw9s= -github.com/ipld/go-ipld-prime v0.9.1-0.20210324083106-dc342a9917db/go.mod h1:KvBLMr4PX1gWptgkzRjVZCrLmSGcZCb/jioOQwCqZN8= github.com/ipld/go-ipld-prime v0.20.0 h1:Ud3VwE9ClxpO2LkCYP7vWPc0Fo+dYdYzgxUJZ3uRG4g= github.com/ipld/go-ipld-prime v0.20.0/go.mod h1:PzqZ/ZR981eKbgdr3y2DJYeD/8bgMawdGVlJDE8kK+M= github.com/ipld/go-ipld-prime/storage/bsadapter v0.0.0-20230102063945-1a409dc236dd h1:gMlw/MhNr2Wtp5RwGdsW23cs+yCuj9k2ON7i9MiJlRo= @@ -309,12 +295,10 @@ github.com/koron/go-ssdp v0.0.3 h1:JivLMY45N76b4p/vsWGOKewBQu6uf39y8l+AQ7sDKx8= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= -github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= -github.com/libp2p/go-buffer-pool v0.0.2/go.mod h1:MvaB6xw5vOrDl8rYZGLFdKAuk/hRoRZd1Vi32+RXyFM= github.com/libp2p/go-buffer-pool v0.1.0 h1:oK4mSFcQz7cTQIfqbe4MIj9gLW+mnanjyFtc6cdF0Y8= github.com/libp2p/go-buffer-pool v0.1.0/go.mod h1:N+vh8gMqimBzdKkSMVuydVDq+UV5QTWy5HSiZacSbPg= github.com/libp2p/go-cidranger v1.1.0 h1:ewPN8EZ0dd1LSnrtuwd4709PXVcITVeuwbag38yPW7c= @@ -326,14 +310,14 @@ github.com/libp2p/go-libp2p v0.26.3 h1:6g/psubqwdaBqNNoidbRKSTBEYgaOuKBhHl8Q5tO+ github.com/libp2p/go-libp2p v0.26.3/go.mod h1:x75BN32YbwuY0Awm2Uix4d4KOz+/4piInkp4Wr3yOo8= github.com/libp2p/go-libp2p-asn-util v0.2.0 h1:rg3+Os8jbnO5DxkC7K/Utdi+DkY3q/d1/1q+8WeNAsw= github.com/libp2p/go-libp2p-asn-util v0.2.0/go.mod h1:WoaWxbHKBymSN41hWSq/lGKJEca7TNm58+gGJi2WsLI= -github.com/libp2p/go-libp2p-kad-dht v0.21.1 h1:xpfp8/t9+X2ip1l8Umap1/UGNnJ3RHJgKGAEsnRAlTo= -github.com/libp2p/go-libp2p-kad-dht v0.21.1/go.mod h1:Oy8wvbdjpB70eS5AaFaI68tOtrdo3KylTvXDjikxqFo= +github.com/libp2p/go-libp2p-kad-dht v0.23.0 h1:sxE6LxLopp79eLeV695n7+c77V/Vn4AMF28AdM/XFqM= +github.com/libp2p/go-libp2p-kad-dht v0.23.0/go.mod h1:oO5N308VT2msnQI6qi5M61wzPmJYg7Tr9e16m5n7uDU= github.com/libp2p/go-libp2p-kbucket v0.5.0 h1:g/7tVm8ACHDxH29BGrpsQlnNeu+6OF1A9bno/4/U1oA= github.com/libp2p/go-libp2p-kbucket v0.5.0/go.mod h1:zGzGCpQd78b5BNTDGHNDLaTt9aDK/A02xeZp9QeFC4U= github.com/libp2p/go-libp2p-record v0.2.0 h1:oiNUOCWno2BFuxt3my4i1frNrt7PerzB3queqa1NkQ0= github.com/libp2p/go-libp2p-record v0.2.0/go.mod h1:I+3zMkvvg5m2OcSdoL0KPljyJyvNDFGKX7QdlpYUcwk= -github.com/libp2p/go-libp2p-routing-helpers v0.4.0 h1:b7y4aixQ7AwbqYfcOQ6wTw8DQvuRZeTAA0Od3YYN5yc= -github.com/libp2p/go-libp2p-routing-helpers v0.4.0/go.mod h1:dYEAgkVhqho3/YKxfOEGdFMIcWfAFNlZX8iAIihYA2E= +github.com/libp2p/go-libp2p-routing-helpers v0.7.0 h1:sirOYVD0wGWjkDwHZvinunIpaqPLBXkcnXApVHwZFGA= +github.com/libp2p/go-libp2p-routing-helpers v0.7.0/go.mod h1:R289GUxUMzRXIbWGSuUUTPrlVJZ3Y/pPz495+qgXJX8= github.com/libp2p/go-libp2p-testing v0.12.0 h1:EPvBb4kKMWO29qP4mZGyhVzUyR25dvfUIK5WDu6iPUA= github.com/libp2p/go-msgio v0.3.0 h1:mf3Z8B1xcFN314sWX+2vOTShIE0Mmn2TXn3YCUQGNj0= github.com/libp2p/go-msgio v0.3.0/go.mod h1:nyRM819GmVaF9LX3l03RMh10QdOroF++NBbxAb0mmDM= @@ -386,23 +370,20 @@ github.com/multiformats/go-multiaddr-dns v0.3.1/go.mod h1:G/245BRQ6FJGmryJCrOuTd github.com/multiformats/go-multiaddr-fmt v0.1.0 h1:WLEFClPycPkp4fnIzoFoV9FVd49/eQsuaL3/CWe167E= github.com/multiformats/go-multibase v0.0.1/go.mod h1:bja2MqRZ3ggyXtZSEDKpl0uO/gviWFaSteVbWT51qgs= github.com/multiformats/go-multibase v0.0.3/go.mod h1:5+1R4eQrT3PkYZ24C3W2Ue2tPwIdYQD509ZjSb5y9Oc= -github.com/multiformats/go-multibase v0.1.1 h1:3ASCDsuLX8+j4kx58qnJ4YFq/JWTJpCyDW27ztsVTOI= -github.com/multiformats/go-multibase v0.1.1/go.mod h1:ZEjHE+IsUrgp5mhlEAYjMtZwK1k4haNkcaPg9aoe1a8= -github.com/multiformats/go-multicodec v0.8.1 h1:ycepHwavHafh3grIbR1jIXnKCsFm0fqsfEOsJ8NtKE8= -github.com/multiformats/go-multicodec v0.8.1/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI16i14xuaojr/H7Ai54k= +github.com/multiformats/go-multibase v0.2.0 h1:isdYCVLvksgWlMW9OZRYJEa9pZETFivncJHmHnnd87g= +github.com/multiformats/go-multibase v0.2.0/go.mod h1:bFBZX4lKCA/2lyOFSAoKH5SS6oPyjtnzK/XTFDPkNuk= +github.com/multiformats/go-multicodec v0.9.0 h1:pb/dlPnzee/Sxv/j4PmkDRxCOi3hXTz3IbPKOXWJkmg= +github.com/multiformats/go-multicodec v0.9.0/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI16i14xuaojr/H7Ai54k= github.com/multiformats/go-multihash v0.0.1/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.10/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew= github.com/multiformats/go-multihash v0.0.13/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= -github.com/multiformats/go-multihash v0.0.14/go.mod h1:VdAWLKTwram9oKAatUcLxBNUjdtcVwxObEQBtRfuyjc= -github.com/multiformats/go-multihash v0.0.15/go.mod h1:D6aZrWNLFTV/ynMpKsNtB40mJzmCl4jb1alC0OvHiHg= -github.com/multiformats/go-multihash v0.2.1 h1:aem8ZT0VA2nCHHk7bPJ1BjUbHNciqZC/d16Vve9l108= -github.com/multiformats/go-multihash v0.2.1/go.mod h1:WxoMcYG85AZVQUyRyo9s4wULvW5qrI9vb2Lt6evduFc= +github.com/multiformats/go-multihash v0.2.3 h1:7Lyc8XfX/IY2jWb/gI7JP+o7JEq9hOa7BFvVU9RSh+U= +github.com/multiformats/go-multihash v0.2.3/go.mod h1:dXgKXCXjBzdscBLk9JkjINiEsCKRVch90MdaGiKsvSM= github.com/multiformats/go-multistream v0.4.1 h1:rFy0Iiyn3YT0asivDUIR05leAdwZq3de4741sbiSdfo= github.com/multiformats/go-multistream v0.4.1/go.mod h1:Mz5eykRVAjJWckE2U78c6xqdtyNUEhKSM0Lwar2p77Q= github.com/multiformats/go-varint v0.0.1/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/multiformats/go-varint v0.0.5/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= -github.com/multiformats/go-varint v0.0.6/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE= github.com/multiformats/go-varint v0.0.7 h1:sWSGR+f/eu5ABZA2ZpYKBILXTTs9JWpdEM/nEGOHFS8= github.com/multiformats/go-varint v0.0.7/go.mod h1:r8PUYw/fD/SjBCiKOoDlGF6QawOELpZAu9eioSos/OU= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= @@ -424,7 +405,6 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/polydawn/refmt v0.0.0-20190221155625-df39d6c2d992/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o= -github.com/polydawn/refmt v0.0.0-20190807091052-3d65705ee9f1/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o= github.com/polydawn/refmt v0.89.0 h1:ADJTApkvkeBZsN0tBTx8QjpD9JkmxbKp0cxfr9qszm4= github.com/polydawn/refmt v0.89.0/go.mod h1:/zvteZs/GwLtCgZ4BL6CBsk9IKIlexP43ObX9AxTqTw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= @@ -464,8 +444,6 @@ github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZV github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/serialx/hashring v0.0.0-20200727003509-22c0c7ab6b1b h1:h+3JX2VoWTFuyQEo87pStk/a99dzIO1mM9KxIyLPGTU= -github.com/serialx/hashring v0.0.0-20200727003509-22c0c7ab6b1b/go.mod h1:/yeG0My1xr/u+HZrFQ1tOQQQQrOawfyMUH13ai5brBc= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= @@ -474,10 +452,8 @@ github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1 github.com/smartystreets/assertions v1.2.0 h1:42S6lae5dvLc7BrLu/0ugRtcFVjoJNMC/N3yZFZkDFs= github.com/smartystreets/assertions v1.2.0/go.mod h1:tcbTF8ujkAEcZ8TElKY+i30BzYlVhC/LOxJk7iOWnoo= github.com/smartystreets/goconvey v0.0.0-20190222223459-a17d461953aa/go.mod h1:2RVY1rIf+2J2o/IM9+vPq9RzmHDSseB7FoXiSNIUsoU= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/smartystreets/goconvey v1.7.2 h1:9RBaZCeXEQ3UselpuwUQHltGVXvdwm6cv1hgR6gDIPg= github.com/smartystreets/goconvey v1.7.2/go.mod h1:Vw0tHAZW6lzCRk3xgdin6fKYcG+G3Pg9vgXWeJpQFMM= -github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -491,8 +467,8 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= -github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tcnksm/go-httpstat v0.2.0 h1:rP7T5e5U2HfmOBmZzGgGZjBQ5/GluWUylujl0tJ04I0= github.com/tcnksm/go-httpstat v0.2.0/go.mod h1:s3JVJFtQxtBEBC9dwcdTTXS9xFnM3SXAZwPG41aurT8= github.com/tj/assert v0.0.3 h1:Df/BlaZ20mq6kuai7f5z2TvPFiwC3xaWJSDQNiIS3Rk= @@ -503,7 +479,6 @@ github.com/urfave/cli/v2 v2.24.2 h1:q1VA+ofZ8SWfEKB9xXHUD4QZaeI9e+ItEqSbfH2JBXk= github.com/urfave/cli/v2 v2.24.2/go.mod h1:GHupkWPMM0M/sj1a2b4wUrWBPzazNrIjouW6fmdJLxc= github.com/warpfork/go-testmark v0.11.0 h1:J6LnV8KpceDvo7spaNU4+DauH2n1x+6RaO2rJrmpQ9U= github.com/warpfork/go-wish v0.0.0-20180510122957-5ad1f5abf436/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= -github.com/warpfork/go-wish v0.0.0-20200122115046-b9ea61034e4a/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0 h1:GDDkbFiaK8jsSDJfjId/PEGEShv6ugrt4kYsC5UIDaQ= github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= github.com/whyrusleeping/base32 v0.0.0-20170828182744-c30ac30633cc h1:BCPnHtcboadS0DvysUuJXZ4lWVv5Bh5i7+tbIyi+ck4= @@ -516,6 +491,10 @@ github.com/whyrusleeping/cbor-gen v0.0.0-20230126041949-52956bd4c9aa/go.mod h1:f github.com/whyrusleeping/chunker v0.0.0-20181014151217-fe64bd25879f h1:jQa4QT2UP9WYv2nzyawpKMOCl+Z/jW7djv2/J50lj9E= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 h1:EKhdznlJHPMoKr0XTrX+IlJs1LH3lyx2nfr1dOlZ79k= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h1:8UvriyWtv5Q5EOgjHaSseUEdkQfvwFv1I/In/O2M9gc= +github.com/willscott/go-requestcontext v0.0.1 h1:qHL7y9r4MOO/4MRdTP/JB0f0uEle+qlueTZJQVvT1YU= +github.com/willscott/go-requestcontext v0.0.1/go.mod h1:23J4EoOLguNM3JeGv2AUDtcWnzK6AFieymcLTDqXQfg= +github.com/willscott/hashring v0.0.0-20230731155239-15f93a2dfb44 h1:zs4g7LTzNzjl8WC0XPJqJx2lga4/6RSH5QaZ3nXOHCg= +github.com/willscott/hashring v0.0.0-20230731155239-15f93a2dfb44/go.mod h1:8ORHm5iheceXLsLvS8Ch8nFWBSjxwajLoKA3a05cjL4= github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 h1:bAn7/zixMGCfxrRTfdpNzjtPYqr8smhKouy9mxVdGPU= github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673/go.mod h1:N3UwUGtsrSj3ccvlPHLoLsHnpR27oXr4ZE984MbSER8= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -523,6 +502,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/zyedidia/generic v1.2.2-0.20230625215236-3404399b19f1 h1:lhqUv/UHe/luHH0hBunV87yeMkvXmQUTcnbgX73qUjg= +github.com/zyedidia/generic v1.2.2-0.20230625215236-3404399b19f1/go.mod h1:ly2RBz4mnz1yeuVbQA/VFwGjK3mnHGRj1JuoG336Bis= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= @@ -530,10 +511,16 @@ go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/otel v1.14.0 h1:/79Huy8wbf5DnIPhemGB+zEPVwnN6fuQybr/SRXa6hM= -go.opentelemetry.io/otel v1.14.0/go.mod h1:o4buv+dJzx8rohcUeRmWUZhqupFvzWis188WlggnNeU= -go.opentelemetry.io/otel/trace v1.14.0 h1:wp2Mmvj41tDsyAJXiWDWpfNsOiIyd38fy85pyKcFq/M= -go.opentelemetry.io/otel/trace v1.14.0/go.mod h1:8avnQLK+CG77yNLUae4ea2JDQ6iT+gozhnZjy/rw9G8= +go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.43.0 h1:lp9h55W1raxWOkKkasHTnqse5R1YKVNJ5/NPcWXYjRM= +go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.43.0/go.mod h1:haEjy8B8Upz9+p1zuhvsKm2uPiKeYFHaNB6BddllMBE= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.43.0 h1:HKORGpiOY0R0nAPtKx/ub8/7XoHhRooP8yNRkuPfelI= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.43.0/go.mod h1:e+y1M74SYXo/FcIx3UATwth2+5dDkM8dBi7eXg1tbw8= +go.opentelemetry.io/otel v1.17.0 h1:MW+phZ6WZ5/uk2nd93ANk/6yJ+dVrvNWUjGhnnFU5jM= +go.opentelemetry.io/otel v1.17.0/go.mod h1:I2vmBGtFaODIVMBSTPVDlJSzBDNf93k60E6Ft0nyjo0= +go.opentelemetry.io/otel/metric v1.17.0 h1:iG6LGVz5Gh+IuO0jmgvpTB6YVrCGngi8QGm+pMd8Pdc= +go.opentelemetry.io/otel/metric v1.17.0/go.mod h1:h4skoxdZI17AxwITdmdZjjYJQH5nzijUUjm+wtPph5o= +go.opentelemetry.io/otel/trace v1.17.0 h1:/SWhSRHmDPOImIAetP1QAeMnZYiQXrTy4fMMYOdSKWQ= +go.opentelemetry.io/otel/trace v1.17.0/go.mod h1:I/4vKTgFclIsXRVucpH25X0mpFSczM7aHeaz0ZBLWjY= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.10.0 h1:9qC72Qh0+3MqyJbAn8YU5xVq1frD8bn3JtD2oXtafVQ= @@ -559,7 +546,6 @@ golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20210220033148-5ea612d1eb83/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.6.0 h1:qfktjS5LUO+fFKeJXZ+ikTRijMmljikvG68fpMMruSc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -670,7 +656,6 @@ golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -693,7 +678,6 @@ golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -707,7 +691,6 @@ golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -776,6 +759,8 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 h1:H2TDz8ibqkAF6YGhCdN3jS9O0/s90v0rJh3X/OLHEUk= golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= +gonum.org/v1/gonum v0.11.0 h1:f1IJhK4Km5tBJmaiJXtk/PkL4cdVX6J+tGiM187uT5E= +gonum.org/v1/gonum v0.11.0/go.mod h1:fSG4YDCxxUZQJ7rKsQrj0gMOg00Il0Z96/qMA4bVQhA= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= @@ -852,8 +837,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w= -google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng= +google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/internal/state/state.go b/internal/state/state.go new file mode 100644 index 0000000..96b959a --- /dev/null +++ b/internal/state/state.go @@ -0,0 +1,20 @@ +package state + +type State struct { + ActiveNodes any + AllNodes any + PoolController +} + +type PoolController interface { + DoRefresh() +} + +type NodeInfo struct { + ID string `json:"id"` + IP string `json:"ip"` + Distance float32 `json:"distance"` + Weight int `json:"weight"` + ComplianceCid string `json:"complianceCid"` + Core bool `json:"core"` +} diff --git a/internal/util/harness.go b/internal/util/harness.go new file mode 100644 index 0000000..fe304de --- /dev/null +++ b/internal/util/harness.go @@ -0,0 +1,338 @@ +package util + +import ( + "context" + "crypto/tls" + "encoding/json" + "errors" + "math/rand" + "net/http" + "net/http/httptest" + "net/url" + "strings" + "sync" + "testing" + "time" + + "github.com/filecoin-saturn/caboose" + "github.com/filecoin-saturn/caboose/internal/state" + "github.com/ipfs/go-cid" + "github.com/ipld/go-car" + "github.com/ipld/go-car/util" + "github.com/multiformats/go-multicodec" + "github.com/stretchr/testify/require" +) + +func BuildCabooseHarness(t *testing.T, n int, maxRetries int, opts ...HarnessOption) *CabooseHarness { + ch := &CabooseHarness{} + + ch.Endpoints = make([]*Endpoint, n) + purls := make([]state.NodeInfo, n) + for i := 0; i < len(ch.Endpoints); i++ { + ch.Endpoints[i] = &Endpoint{} + ch.Endpoints[i].Setup() + ip := strings.TrimPrefix(ch.Endpoints[i].Server.URL, "https://") + + cid, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum([]byte(testBlock)) + + purls[i] = state.NodeInfo{ + IP: ip, + ID: "node-id", + Weight: rand.Intn(100), + Distance: rand.Float32(), + ComplianceCid: cid.String(), + } + } + ch.goodOrch = true + orch := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + ch.gol.Lock() + defer ch.gol.Unlock() + if ch.goodOrch { + json.NewEncoder(w).Encode(purls) + } else { + json.NewEncoder(w).Encode([]string{}) + } + })) + + saturnClient := &http.Client{ + Transport: &http.Transport{ + TLSClientConfig: &tls.Config{ + InsecureSkipVerify: true, + ServerName: "example.com", + }, + }, + } + + ourl, _ := url.Parse(orch.URL) + + conf := &caboose.Config{ + OrchestratorEndpoint: ourl, + OrchestratorClient: http.DefaultClient, + LoggingEndpoint: *ourl, + LoggingClient: http.DefaultClient, + LoggingInterval: time.Hour, + + Client: saturnClient, + DoValidation: false, + PoolRefresh: time.Second * 50, + MaxRetrievalAttempts: maxRetries, + Harness: &state.State{}, + + MirrorFraction: 1.0, + } + + for _, opt := range opts { + opt(conf) + } + + bs, err := caboose.NewCaboose(conf) + require.NoError(t, err) + + ch.Caboose = bs + ch.CabooseActiveNodes = conf.Harness.ActiveNodes.(*caboose.NodeRing) + ch.CabooseAllNodes = conf.Harness.AllNodes.(*caboose.NodeHeap) + ch.CaboosePool = conf.Harness.PoolController + ch.Config = conf + return ch +} + +type CabooseHarness struct { + Caboose *caboose.Caboose + Endpoints []*Endpoint + + CabooseActiveNodes *caboose.NodeRing + CabooseAllNodes *caboose.NodeHeap + CaboosePool state.PoolController + Config *caboose.Config + + gol sync.Mutex + goodOrch bool +} + +type NodeStats struct { + Start time.Time + Latency float64 + Size float64 +} + +func (ch *CabooseHarness) RunFetchesForRandCids(n int) { + for i := 0; i < n; i++ { + randCid, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum([]byte{uint8(i)}) + _, _ = ch.Caboose.Get(context.Background(), randCid) + } +} + +func (ch *CabooseHarness) FetchAndAssertCoolDownError(t *testing.T, ctx context.Context, cid cid.Cid) { + _, err := ch.Caboose.Get(ctx, cid) + require.Error(t, err) + + var coolDownErr *caboose.ErrCoolDown + ok := errors.As(err, &coolDownErr) + require.True(t, ok) + require.Contains(t, coolDownErr.Path, cid.String()) + require.NotZero(t, coolDownErr.RetryAfter()) +} + +func (ch *CabooseHarness) FetchAndAssertFailure(t *testing.T, ctx context.Context, testCid cid.Cid, contains string) { + _, err := ch.Caboose.Get(ctx, testCid) + require.Error(t, err) + require.Contains(t, err.Error(), contains) +} + +func (ch *CabooseHarness) FetchAndAssertSuccess(t *testing.T, ctx context.Context, c cid.Cid) { + blk, err := ch.Caboose.Get(ctx, c) + require.NoError(t, err) + require.NotEmpty(t, blk) +} + +func (ch *CabooseHarness) RecordSuccesses(t *testing.T, nodes []*caboose.Node, s NodeStats, n int) { + for _, node := range nodes { + s.Start = time.Now().Add(-time.Second * 5) + for i := 0; i < n; i++ { + node.RecordSuccess(s.Start, s.Latency, s.Size) + } + } +} + +func (ch *CabooseHarness) RecordFailures(t *testing.T, nodes []*caboose.Node, n int) { + for _, node := range nodes { + for i := 0; i < n; i++ { + node.RecordFailure() + } + } +} + +func (ch *CabooseHarness) FailNodesWithCode(t *testing.T, selectorF func(ep *Endpoint) bool, code int) { + for _, n := range ch.Endpoints { + if selectorF(n) { + n.Valid = false + n.httpCode = code + } + } +} + +func (ch *CabooseHarness) RecoverNodes(t *testing.T, selectorF func(ep *Endpoint) bool) { + for _, n := range ch.Endpoints { + if selectorF(n) { + n.Valid = true + } + } +} + +func (ch *CabooseHarness) FailNodesAndAssertFetch(t *testing.T, selectorF func(ep *Endpoint) bool, nAlive int, cid cid.Cid) { + ch.FailNodes(t, selectorF) + require.EqualValues(t, nAlive, ch.NNodesAlive()) + ch.FetchAndAssertSuccess(t, context.Background(), cid) +} + +func (ch *CabooseHarness) FailNodes(t *testing.T, selectorF func(ep *Endpoint) bool) { + for _, n := range ch.Endpoints { + if selectorF(n) { + n.Valid = false + } + } +} + +func (ch *CabooseHarness) NNodesAlive() int { + cnt := 0 + for _, n := range ch.Endpoints { + if n.Valid { + cnt++ + } + } + return cnt +} + +func (ch *CabooseHarness) StopOrchestrator() { + ch.gol.Lock() + ch.goodOrch = false + ch.gol.Unlock() +} + +func (ch *CabooseHarness) StartOrchestrator() { + ch.gol.Lock() + ch.goodOrch = true + ch.gol.Unlock() +} + +func (h *CabooseHarness) AssertLatencyCount(t *testing.T, expected int) { + nds := h.CabooseActiveNodes + count := 0 + + ndl := nds.Len() + nodes, err := nds.GetNodes("", ndl) + require.NoError(t, err, "Getting nodes should succeed") + for _, perf := range nodes { + perf.Samples.Each(func(_ caboose.NodeSample) { + count += 1 + }) + } + require.EqualValues(t, expected, count) +} + +func (h *CabooseHarness) AssertCorrectnessCount(t *testing.T, expected int) { + nds := h.CabooseActiveNodes + count := 0 + + ndl := nds.Len() + nodes, err := nds.GetNodes("", ndl) + require.NoError(t, err, "Getting nodes should succeed") + for _, perf := range nodes { + perf.Samples.Each(func(_ caboose.NodeSample) { + count += 1 + }) + } + require.EqualValues(t, expected, count) +} + +func (h *CabooseHarness) AssertPoolSize(t *testing.T, activeS, totalS int) { + active := h.CabooseActiveNodes.Len() + all := h.CabooseAllNodes.Len() + + require.Equal(t, totalS, all) + require.Equal(t, activeS, active) +} + +type HarnessOption func(config *caboose.Config) + +func WithMaxFailuresBeforeCoolDown(max int) func(config *caboose.Config) { + return func(config *caboose.Config) { + config.MaxFetchFailuresBeforeCoolDown = max + } +} + +func WithComplianceCidPeriod(n int64) func(config *caboose.Config) { + return func(config *caboose.Config) { + config.ComplianceCidPeriod = n + } +} + +func WithMirrorFraction(n float64) func(config *caboose.Config) { + return func(config *caboose.Config) { + config.MirrorFraction = n + } +} + +func WithCidCoolDownDuration(duration time.Duration) func(config *caboose.Config) { + return func(config *caboose.Config) { + config.FetchKeyCoolDownDuration = duration + } +} + +type Endpoint struct { + Server *httptest.Server + Valid bool + count int + httpCode int + Resp []byte + CarWrap bool + lk sync.Mutex +} + +var testBlock = []byte("hello World") + +func (e *Endpoint) Setup() { + e.Valid = true + e.CarWrap = true + e.Resp = testBlock + e.Server = httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + e.lk.Lock() + defer e.lk.Unlock() + e.count++ + if e.Valid { + if e.CarWrap { + c, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum(e.Resp) + + car.WriteHeader(&car.CarHeader{ + Roots: []cid.Cid{c}, + Version: 1, + }, w) + util.LdWrite(w, c.Bytes(), e.Resp) + } else { + w.Write(e.Resp) + } + } else { + if e.httpCode == http.StatusTooManyRequests { + w.Header().Set("Retry-After", "1") + } + if e.httpCode == 0 { + e.httpCode = 500 + } + w.WriteHeader(e.httpCode) + w.Write([]byte("error")) + } + })) +} + +func (ep *Endpoint) SetResp(resp []byte, wrap bool) { + ep.lk.Lock() + defer ep.lk.Unlock() + ep.Resp = resp + ep.CarWrap = wrap +} + +func (ep *Endpoint) Count() int { + ep.lk.Lock() + defer ep.lk.Unlock() + return ep.count +} diff --git a/log.go b/log.go index f43561f..7248558 100644 --- a/log.go +++ b/log.go @@ -104,4 +104,6 @@ type log struct { NodeId string `json:"nodeId"` IfNetworkError string `json:"ifNetworkError"` NodeIpAddress string `json:"nodeIpAddress"` + VerificationError string `json:"verificationError"` + OtherError string `json:"otherError"` } diff --git a/metrics.go b/metrics.go index 07404a6..4350a8d 100644 --- a/metrics.go +++ b/metrics.go @@ -1,15 +1,10 @@ package caboose import ( - "sync" - "github.com/prometheus/client_golang/prometheus" ) var ( - // needed to sync over these global vars in tests - distLk sync.Mutex - peerLatencyDistribution prometheus.Collector // guarded by pool.lock ) @@ -36,10 +31,6 @@ var ( ) var ( - // size buckets from 256 KiB (default chunk in Kubo) to 4MiB (maxBlockSize), 256 KiB wide each - // histogram buckets will be [256KiB, 512KiB, 768KiB, 1MiB, ... 4MiB] -> total 16 buckets +1 prometheus Inf bucket - blockSizeHistogram = prometheus.LinearBuckets(262144, 262144, 16) - // TODO: Speed max bucket could use some further refinement, // for now we don't expect speed being bigger than transfering 4MiB (max block) in 500ms // histogram buckets will be [1byte/milliseconds, ... 8387 bytes/milliseconds] -> total 20 buckets +1 prometheus Inf bucket @@ -47,24 +38,15 @@ var ( // ----- Histogram buckets to record fetch duration metrics ----- // The upper bound on the fetch duration buckets are informed by the timeouts per block and per peer request/retry. - - // buckets to record duration in milliseconds to fetch a block, - // histogram buckets will be [50ms,.., 60 seconds] -> total 20 buckets +1 prometheus Inf bucket - durationMsPerBlockHistogram = prometheus.ExponentialBucketsRange(50, 60000, 20) - // buckets to record duration in milliseconds to fetch a CAR, // histogram buckets will be [50ms,.., 30 minutes] -> total 40 buckets +1 prometheus Inf bucket durationMsPerCarHistogram = prometheus.ExponentialBucketsRange(50, 1800000, 40) - - // buckets to measure latency between a caboose peer a Saturn L1, - // [50ms, 100ms, 150ms, ..., 2000 ms] - latencyDistMsHistogram = prometheus.LinearBuckets(50, 50, 40) ) var ( fetchResponseCodeMetric = prometheus.NewCounterVec(prometheus.CounterOpts{ Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_response_code"), - Help: "Response codes observed during caboose fetches for a block", + Help: "Response codes observed during caboose fetches", }, []string{"resourceType", "code"}) // success cases @@ -80,46 +62,6 @@ var ( }, []string{"resourceType", "cache_status"}) ) -// block metrics -var ( - fetchDurationPerBlockPerPeerSuccessMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_duration_block_peer_success"), - Help: "Latency observed during successful caboose fetches from a single peer in milliseconds", - Buckets: durationMsPerBlockHistogram, - }, []string{"cache_status"}) - - fetchDurationBlockSuccessMetric = prometheus.NewHistogram(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_duration_block_success"), - Help: "Latency observed during successful caboose fetches for a block across multiple peers and retries in milliseconds", - Buckets: durationMsPerBlockHistogram, - }) - - fetchTTFBPerBlockPerPeerSuccessMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_ttfb_block_peer_success"), - Help: "TTFB observed during a successful caboose fetch from a single peer in milliseconds", - Buckets: durationMsPerBlockHistogram, - }, []string{"cache_status"}) - - // failures - fetchDurationPerBlockPerPeerFailureMetric = prometheus.NewHistogram(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_duration_block_peer_failure"), - Help: "Latency observed during failed caboose fetches from a single peer in milliseconds", - Buckets: durationMsPerBlockHistogram, - }) - - fetchDurationBlockFailureMetric = prometheus.NewHistogram(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_duration_block_failure"), - Help: "Latency observed during failed caboose fetches for a block across multiple peers and retries in milliseconds", - Buckets: durationMsPerBlockHistogram, - }) - - fetchSizeBlockMetric = prometheus.NewHistogram(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_size_block"), - Help: "Size in bytes of caboose block fetches", - Buckets: blockSizeHistogram, - }) -) - // CAR metrics var ( fetchDurationPerCarPerPeerSuccessMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ @@ -140,11 +82,6 @@ var ( Buckets: durationMsPerCarHistogram, }, []string{"cache_status", "car_size"}) - headerTTFBPerPeerMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "header_ttfb_peer"), - Buckets: durationMsPerCarHistogram, - }, []string{"resourceType", "cache_status"}) - // failure fetchDurationPerCarPerPeerFailureMetric = prometheus.NewHistogram(prometheus.HistogramOpts{ Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_duration_car_peer_failure"), @@ -165,53 +102,11 @@ var ( }, []string{"error_status"}) ) -// Saturn Server-timings -var ( - // ---------------------- For successful fetches ONLY for now---------------------- - // L1 server timings - // nginx + l1 compute + lassie - fetchDurationPerPeerSuccessTotalL1NodeMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_duration_peer_total_saturn_l1"), - Help: "Total time spent on an L1 node for a successful fetch per peer in milliseconds", - Buckets: durationMsPerCarHistogram, - }, []string{"resourceType", "cache_status"}) - - // total only on lassie - fetchDurationPerPeerSuccessCacheMissTotalLassieMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_duration_peer_cache_miss_total_lassie"), - Help: "Time spent in Lassie for a Saturn L1 Nginx cache miss for a successful fetch per peer in milliseconds", - Buckets: durationMsPerCarHistogram, - }, []string{"resourceType"}) - - lassie5XXTimeMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_duration_5xx_total_lassie"), - Help: "Time spent in Lassie for a Saturn L1 Nginx cache miss for a 5xx in milliseconds", - Buckets: durationMsPerCarHistogram, - }, []string{"resourceType"}) - - // network timing - fetchNetworkSpeedPerPeerSuccessMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_network_speed_peer_success"), - Help: "Network speed observed during successful caboose fetches from a single peer in bytes per milliseconds", - Buckets: speedBytesPerMsHistogram, - }, []string{"resourceType"}) - - fetchNetworkLatencyPeerSuccessMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_network_latency_peer_success"), - Help: "Network latency observed during successful caboose fetches from a single peer in milliseconds", - Buckets: durationMsPerCarHistogram, - }, []string{"resourceType"}) -) - var ( fetchCalledTotalMetric = prometheus.NewCounterVec(prometheus.CounterOpts{ Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_called_total"), }, []string{"resourceType"}) - fetchRequestContextErrorTotalMetric = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_request_context_error_total"), - }, []string{"resourceType", "errorType", "requestStage"}) - fetchRequestSuccessTimeTraceMetric = prometheus.NewHistogramVec(prometheus.HistogramOpts{ Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_request_success_time_trace"), Buckets: durationMsPerCarHistogram, @@ -238,6 +133,10 @@ var ( mirroredTrafficTotalMetric = prometheus.NewCounterVec(prometheus.CounterOpts{ Name: prometheus.BuildFQName("ipfs", "caboose", "mirrored_traffic_total"), }, []string{"error_status"}) + + complianceCidCallsTotalMetric = prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: prometheus.BuildFQName("ipfs", "caboose", "compliance_cids_total"), + }, []string{"error_status"}) ) var CabooseMetrics = prometheus.NewRegistry() @@ -247,53 +146,32 @@ func init() { CabooseMetrics.MustRegister(poolRefreshErrorMetric) CabooseMetrics.MustRegister(poolSizeMetric) CabooseMetrics.MustRegister(poolNewMembersMetric) - CabooseMetrics.MustRegister(poolRemovedFailureTotalMetric) - CabooseMetrics.MustRegister(poolRemovedConnFailureTotalMetric) - CabooseMetrics.MustRegister(poolRemovedReadFailureTotalMetric) - CabooseMetrics.MustRegister(poolRemovedNon2xxTotalMetric) - CabooseMetrics.MustRegister(poolMembersNotAddedBecauseRemovedMetric) - CabooseMetrics.MustRegister(poolMembersRemovedAndAddedBackMetric) - CabooseMetrics.MustRegister(poolEnoughObservationsForMainSetDurationMetric) CabooseMetrics.MustRegister(poolTierChangeMetric) CabooseMetrics.MustRegister(fetchResponseCodeMetric) CabooseMetrics.MustRegister(fetchSpeedPerPeerSuccessMetric) - CabooseMetrics.MustRegister(fetchDurationPerBlockPerPeerSuccessMetric) CabooseMetrics.MustRegister(fetchDurationPerCarPerPeerSuccessMetric) - CabooseMetrics.MustRegister(fetchDurationPerBlockPerPeerFailureMetric) CabooseMetrics.MustRegister(fetchDurationPerCarPerPeerFailureMetric) - CabooseMetrics.MustRegister(fetchDurationBlockSuccessMetric) CabooseMetrics.MustRegister(fetchDurationCarSuccessMetric) - CabooseMetrics.MustRegister(fetchDurationBlockFailureMetric) CabooseMetrics.MustRegister(fetchDurationCarFailureMetric) - CabooseMetrics.MustRegister(fetchTTFBPerBlockPerPeerSuccessMetric) CabooseMetrics.MustRegister(fetchTTFBPerCARPerPeerSuccessMetric) - CabooseMetrics.MustRegister(headerTTFBPerPeerMetric) CabooseMetrics.MustRegister(fetchCacheCountSuccessTotalMetric) - CabooseMetrics.MustRegister(fetchDurationPerPeerSuccessTotalL1NodeMetric) - CabooseMetrics.MustRegister(fetchDurationPerPeerSuccessCacheMissTotalLassieMetric) - CabooseMetrics.MustRegister(lassie5XXTimeMetric) - - CabooseMetrics.MustRegister(fetchNetworkSpeedPerPeerSuccessMetric) - CabooseMetrics.MustRegister(fetchNetworkLatencyPeerSuccessMetric) - CabooseMetrics.MustRegister(m_collector{&peerLatencyDistribution}) CabooseMetrics.MustRegister(fetchSizeCarMetric) - CabooseMetrics.MustRegister(fetchSizeBlockMetric) - - CabooseMetrics.MustRegister(fetchRequestContextErrorTotalMetric) CabooseMetrics.MustRegister(fetchCalledTotalMetric) - CabooseMetrics.MustRegister(fetchRequestSuccessTimeTraceMetric) CabooseMetrics.MustRegister(saturnCallsTotalMetric) CabooseMetrics.MustRegister(saturnCallsFailureTotalMetric) CabooseMetrics.MustRegister(saturnConnectionFailureTotalMetric) + CabooseMetrics.MustRegister(complianceCidCallsTotalMetric) CabooseMetrics.MustRegister(saturnCallsSuccessTotalMetric) CabooseMetrics.MustRegister(mirroredTrafficTotalMetric) + + CabooseMetrics.MustRegister(fetchRequestSuccessTimeTraceMetric) } diff --git a/node.go b/node.go new file mode 100644 index 0000000..f21f1b2 --- /dev/null +++ b/node.go @@ -0,0 +1,142 @@ +package caboose + +import ( + "math" + "sync" + "time" + + "github.com/filecoin-saturn/caboose/internal/state" + "github.com/zyedidia/generic/queue" +) + +const ( + sampleWindow = 100.0 + defaultLatencyMS = 300.0 +) + +type Node struct { + URL string + ComplianceCid string + Core bool + + PredictedLatency float64 + PredictedThroughput float64 + PredictedReliability float64 + + Samples *queue.Queue[NodeSample] + successes int + lk sync.RWMutex +} + +func NewNode(info state.NodeInfo) *Node { + return &Node{ + URL: info.IP, + ComplianceCid: info.ComplianceCid, + Core: info.Core, + Samples: queue.New[NodeSample](), + } +} + +type NodeSample struct { + Start, End time.Time + Success bool + // recorded in milliseconds + Latency float64 + Size float64 +} + +func (n *Node) RecordFailure() { + n.lk.Lock() + defer n.lk.Unlock() + n.Samples.Enqueue(NodeSample{Success: false}) + n.update() +} + +func (n *Node) RecordSuccess(start time.Time, latency, size float64) { + n.lk.Lock() + defer n.lk.Unlock() + n.Samples.Enqueue(NodeSample{Start: start, End: time.Now(), Success: true, Latency: latency, Size: size}) + n.successes += 1 + n.update() +} + +func (n *Node) update() { + s := n.Samples.Peek() + successN := 0 + if s.Success { + successN = 1 + } + decayFactor := (sampleWindow - 1) / sampleWindow + n.PredictedReliability = n.PredictedReliability*decayFactor + float64(successN)/sampleWindow + + to := time.Time{} + totalBytesPerMS := 0.0 + latency := defaultLatencyMS + + decay := 1 * math.Pow(decayFactor, float64(n.successes)) + // We want to treat this as an expontential decay of the samples, e.g. + // latest * 1/n + (2nd latest * 1/n + ...) * (n-1/n) + // This basic decay-weighting is enough for latency measurements + // with the caveat that we have 'missing' (non-successful) elements that shouldn't affect the denominator. + // For throughput, when there are subsequent retrievals that happen in the same time period, we want to + // count that as a term like `(latest + 2nd latest) * 2/n` - the observed throughput at that point is higher + // than the individual measurements. + n.Samples.Each(func(t NodeSample) { + if !t.Success { + return + } + decay /= decayFactor + + totalBytesPerMS += t.Size / float64(t.End.Sub(t.Start).Milliseconds()) + + // fix up in case + if t.Start.Before(to) { + // overlap + fractionOverlap := float64(to.Sub(t.Start)) / float64(t.End.Sub(t.Start)) + totalBytesPerMS *= decay / fractionOverlap + } else { + totalBytesPerMS *= decay + } + to = t.End + }) + n.PredictedLatency = latency + n.PredictedThroughput = totalBytesPerMS + + if n.Samples.Len() > sampleWindow { + old := n.Samples.Dequeue() + if old.Success { + n.successes -= 1 + } + } +} + +func (n *Node) Equals(other *Node) bool { + return n.URL == other.URL +} + +func (n *Node) Priority() float64 { + n.lk.RLock() + defer n.lk.RUnlock() + latency := n.PredictedLatency + if latency == 0 { + latency = defaultLatencyMS + } + + return n.PredictedReliability / latency * n.PredictedThroughput +} + +func (n *Node) Rate() float64 { + n.lk.RLock() + defer n.lk.RUnlock() + + len := n.Samples.Len() + if len == 0 { + return 0 + } + last := n.Samples.Peek() + return float64(len) / float64(time.Since(last.Start)) +} + +func (n *Node) String() string { + return n.URL +} diff --git a/node_heap.go b/node_heap.go new file mode 100644 index 0000000..c71771f --- /dev/null +++ b/node_heap.go @@ -0,0 +1,105 @@ +package caboose + +import ( + "container/heap" + "math/rand" + "sync" +) + +// NodeHeap is a collection of nodes organized by performance +type NodeHeap struct { + Nodes []*Node + lk sync.RWMutex +} + +func NewNodeHeap() *NodeHeap { + return &NodeHeap{Nodes: make([]*Node, 0)} +} + +func (nh *NodeHeap) Add(n *Node) { + nh.lk.Lock() + defer nh.lk.Unlock() + heap.Push(nh, n) +} + +func (nh *NodeHeap) AddIfNotPresent(n *Node) bool { + nh.lk.Lock() + defer nh.lk.Unlock() + + for _, e := range nh.Nodes { + if e.Equals(n) { + return false + } + } + heap.Push(nh, n) + return true +} + +func (nh *NodeHeap) Best() *Node { + nh.lk.Lock() + defer nh.lk.Unlock() + heap.Init(nh) + item := heap.Pop(nh) + return item.(*Node) +} + +func (nh *NodeHeap) PeekRandom() *Node { + nh.lk.RLock() + defer nh.lk.RUnlock() + + if len(nh.Nodes) == 0 { + return nil + } + + randIdx := rand.Intn(len(nh.Nodes)) + return nh.Nodes[randIdx] +} + +func (nh *NodeHeap) TopN(n int) []*Node { + m := make([]*Node, 0, n) + temp := make([]*Node, 0, n) + nh.lk.Lock() + defer nh.lk.Unlock() + + heap.Init(nh) + for i := 0; i < n && nh.Len() > 0; i++ { + item := heap.Pop(nh) + node := item.(*Node) + m = append(m, node) + temp = append(temp, node) + } + + for _, node := range temp { + heap.Push(nh, node) + } + + return m +} + +/* below functions implement the heap interface */ +var _ heap.Interface = (*NodeHeap)(nil) + +func (nh *NodeHeap) Len() int { return len(nh.Nodes) } + +func (nh *NodeHeap) Less(i, j int) bool { + // We want Pop to give us the highest, not lowest, priority so we use greater than here. + return nh.Nodes[i].Priority() > nh.Nodes[j].Priority() +} + +func (nh *NodeHeap) Swap(i, j int) { + nh.Nodes[i], nh.Nodes[j] = nh.Nodes[j], nh.Nodes[i] +} + +func (nh *NodeHeap) Push(a any) { + if n, ok := a.(*Node); ok { + nh.Nodes = append(nh.Nodes, n) + } +} + +func (nh *NodeHeap) Pop() any { + n := len(nh.Nodes) + item := nh.Nodes[n-1] + nh.Nodes[n-1] = nil + nh.Nodes = nh.Nodes[0 : n-1] + return item +} diff --git a/node_ring.go b/node_ring.go new file mode 100644 index 0000000..eafbbf3 --- /dev/null +++ b/node_ring.go @@ -0,0 +1,187 @@ +package caboose + +import ( + "fmt" + "strings" + "sync" + + "github.com/willscott/hashring" +) + +// NodeRing represents a set of nodes organized for stable hashing. +type NodeRing struct { + Nodes map[string]*Node + ring hashring.HashRing + targetSize int + + lk sync.RWMutex +} + +func NewNodeRing(targetSize int) *NodeRing { + return &NodeRing{ + Nodes: map[string]*Node{}, + ring: *hashring.New([]string{}), + targetSize: targetSize, + } +} + +func (nr *NodeRing) updateRing() error { + // this method expects that the lk is held when called. + rs := make(map[string]int) + for _, n := range nr.Nodes { + // TODO: weight multiples + rs[n.URL] = 1 + } + nr.ring.UpdateWithWeights(rs) + return nil +} + +// A score of '0' ==> overall experience is the same as the current state +// A positive score ==> overall experience is better than the current state +// A negative score ==> overall experience is worse than the current state +func (nr *NodeRing) getScoreForUpdate(candidate string, priority float64, weight int) float64 { + changes := nr.ring.ConsiderUpdateWeightedNode(candidate, weight) + delta := float64(0) + var neighbor *Node + + for n, v := range changes { + neighbor = nr.Nodes[n] + neighborVolume := neighbor.Rate() + if neighborVolume < 1 { + neighborVolume = 1 + } + + amntChanged := v + // for now, add some bounds + if amntChanged < -1 { + amntChanged = -1 + } else if amntChanged > 1 { + amntChanged = 1 + } + // a negative amntChanged means that we're replacing the neighbor with the candidate. + amntChanged *= -1 + + // how much worse is candidate? + diff := priority - neighbor.Priority() + cs := diff * neighborVolume * float64(amntChanged) + delta += cs + // fmt.Printf("+%f (n %s: diff %f=(n %f - candidate %f) * volume %f * v = %f)", cs, neighbor.URL, diff, neighbor.Priority(), priority, neighborVolume, amntChanged) + } + return delta +} + +func (nr *NodeRing) MaybeSubstituteOrAdd(candidate *Node, activationThreshold int64) (bool, error) { + nr.lk.Lock() + defer nr.lk.Unlock() + + _, ok := nr.ring.GetNode(candidate.URL) + if !ok { + // ring is empty. in this case we always want to add. + nr.Nodes[candidate.URL] = candidate + return true, nr.updateRing() + } + + // how much space is being claimed? + delta := nr.getScoreForUpdate(candidate.URL, candidate.Priority(), 1) + + if delta >= float64(activationThreshold) { + nr.Nodes[candidate.URL] = candidate + return true, nr.updateRing() + } + + // not a clear benefit to add, but maybe acceptable for substitution: + worst := candidate.Priority() + worstN := "" + for _, n := range nr.Nodes { + if n.Priority() < worst { + worst = n.Priority() + worstN = n.URL + } + } + + // todo: the '+1' is an arbitrary threshold to prevent thrashing. it should be configurable. + if worstN != "" && candidate.Priority()-worst > float64(activationThreshold)+1 { + nr.Nodes[candidate.URL] = candidate + delete(nr.Nodes, worstN) + return true, nr.updateRing() + + } + + // fmt.Printf("did not add - delta %f activation %d, node priority %f\n", delta, activationThreshold, candidate.Priority()) + return false, nil +} + +func (nr *NodeRing) Add(n *Node) error { + nr.lk.Lock() + defer nr.lk.Unlock() + nr.Nodes[n.URL] = n + return nr.updateRing() +} + +func (nr *NodeRing) Remove(n *Node) error { + nr.lk.Lock() + defer nr.lk.Unlock() + + if _, ok := nr.Nodes[n.URL]; ok { + delete(nr.Nodes, n.URL) + return nr.updateRing() + } + return ErrNoBackend +} + +func (nr *NodeRing) Contains(n *Node) bool { + nr.lk.RLock() + defer nr.lk.RUnlock() + + _, ok := nr.Nodes[n.URL] + return ok +} + +func (nr *NodeRing) IsCore(n *Node) bool { + nr.lk.RLock() + defer nr.lk.RUnlock() + + nd, ok := nr.Nodes[n.URL] + if !ok { + return false + } + return nd.Core +} + +func (nr *NodeRing) GetNodes(key string, number int) ([]*Node, error) { + nr.lk.RLock() + defer nr.lk.RUnlock() + + if number > nr.ring.Size() { + number = nr.ring.Size() + } + keys, ok := nr.ring.GetNodes(key, number) + if !ok { + return nil, ErrNoBackend + } + nodes := make([]*Node, 0, len(keys)) + for _, k := range keys { + if n, ok := nr.Nodes[k]; ok { + nodes = append(nodes, n) + } + } + return nodes, nil +} + +func (nr *NodeRing) Len() int { + nr.lk.RLock() + defer nr.lk.RUnlock() + return nr.ring.Size() +} + +func (nr *NodeRing) String() string { + nr.lk.RLock() + defer nr.lk.RUnlock() + + ns := make([]string, 0, len(nr.Nodes)) + for _, n := range nr.Nodes { + ns = append(ns, n.String()) + } + + return fmt.Sprintf("NodeRing[len %d]{%s}", nr.ring.Size(), strings.Join(ns, ",")) +} diff --git a/node_ring_test.go b/node_ring_test.go new file mode 100644 index 0000000..c24e14d --- /dev/null +++ b/node_ring_test.go @@ -0,0 +1,23 @@ +package caboose_test + +import ( + "fmt" + "testing" + + "github.com/filecoin-saturn/caboose" +) + +func TestNodeRing(t *testing.T) { + nr := caboose.NewNodeRing(30) + nodes := make([]*caboose.Node, 0) + for i := 0; i < 100; i++ { + nodes = append(nodes, &caboose.Node{URL: fmt.Sprintf("node%d", i)}) + } + + for _, n := range nodes { + err := nr.Add(n) + if err != nil { + t.Fatalf("adding should always work: %v", err) + } + } +} diff --git a/pool.go b/pool.go index b20e156..64dca91 100644 --- a/pool.go +++ b/pool.go @@ -2,21 +2,20 @@ package caboose import ( "context" + cryptoRand "crypto/rand" "encoding/json" "errors" "fmt" "io" + "math/big" "math/rand" "net/url" - "os" "sync" "time" - "github.com/asecurityteam/rolling" - "github.com/patrickmn/go-cache" - "github.com/prometheus/client_golang/prometheus" + "github.com/filecoin-saturn/caboose/internal/state" - "github.com/filecoin-saturn/caboose/tieredhashing" + "github.com/patrickmn/go-cache" "github.com/ipfs/boxo/path" blocks "github.com/ipfs/go-block-format" @@ -25,14 +24,15 @@ import ( ) const ( - tierMainToUnknown = "main-to-unknown" - tierUnknownToMain = "unknown-to-main" - BackendOverrideKey = "CABOOSE_BACKEND_OVERRIDE" + blockPathPattern = "/ipfs/%s?format=car&dag-scope=block" + defaultMirroredConcurrency = 5 ) -// loadPool refreshes the set of Saturn endpoints in the pool by fetching an updated list of responsive Saturn nodes from the -// Saturn Orchestrator. -func (p *pool) loadPool() ([]string, error) { +var complianceCidReqTemplate = "/ipfs/%s?format=raw" + +// loadPool refreshes the set of endpoints in the pool by fetching an updated list of nodes from the +// Orchestrator. +func (p *pool) loadPool() ([]state.NodeInfo, error) { if p.config.OrchestratorOverride != nil { return p.config.OrchestratorOverride, nil } @@ -44,17 +44,20 @@ func (p *pool) loadPool() ([]string, error) { } defer resp.Body.Close() - responses := make([]string, 0) + responses := make([]state.NodeInfo, 0) + if err := json.NewDecoder(resp.Body).Decode(&responses); err != nil { goLogger.Warnw("failed to decode backends from orchestrator", "err", err, "endpoint", p.config.OrchestratorEndpoint.String()) return nil, err } + goLogger.Infow("got backends from orchestrators", "cnt", len(responses), "endpoint", p.config.OrchestratorEndpoint.String()) + return responses, nil } type mirroredPoolRequest struct { - node string + node *Node path string // the key for node affinity for the request key string @@ -73,22 +76,14 @@ type pool struct { fetchKeyFailureCache *cache.Cache // guarded by fetchKeyLk fetchKeyCoolDownCache *cache.Cache // guarded by fetchKeyLk - lk sync.RWMutex - th *tieredhashing.TieredHashing - - poolInitDone sync.Once + ActiveNodes *NodeRing + AllNodes *NodeHeap } -func newPool(c *Config) *pool { - noRemove := false - if len(os.Getenv(BackendOverrideKey)) > 0 { - noRemove = true - } - - topts := append(c.TieredHashingOpts, tieredhashing.WithNoRemove(noRemove)) - +func newPool(c *Config, logger *logger) *pool { p := pool{ config: c, + logger: logger, started: make(chan struct{}), refresh: make(chan struct{}, 1), done: make(chan struct{}, 1), @@ -96,7 +91,9 @@ func newPool(c *Config) *pool { fetchKeyCoolDownCache: cache.New(c.FetchKeyCoolDownDuration, 1*time.Minute), fetchKeyFailureCache: cache.New(c.FetchKeyCoolDownDuration, 1*time.Minute), - th: tieredhashing.New(topts...), + + ActiveNodes: NewNodeRing(c.PoolTargetSize), + AllNodes: NewNodeHeap(), } return &p @@ -107,73 +104,36 @@ func (p *pool) Start() { go p.checkPool() } -func (p *pool) doRefresh() { +func (p *pool) DoRefresh() { newEP, err := p.loadPool() if err == nil { - p.refreshWithNodes(newEP) + for _, n := range newEP { + node := NewNode(n) + p.AllNodes.AddIfNotPresent(node) + } } else { poolRefreshErrorMetric.Add(1) } -} - -func (p *pool) refreshWithNodes(newEP []string) { - p.lk.Lock() - defer p.lk.Unlock() - - // for tests to pass the -race check when accessing global vars - distLk.Lock() - defer distLk.Unlock() - - added, alreadyRemoved, back := p.th.AddOrchestratorNodes(newEP) - poolNewMembersMetric.Set(float64(added)) - poolMembersNotAddedBecauseRemovedMetric.Set(float64(alreadyRemoved)) - poolMembersRemovedAndAddedBackMetric.Set(float64(back)) - - // update the tier set - mu, um := p.th.UpdateMainTierWithTopN() - poolTierChangeMetric.WithLabelValues(tierMainToUnknown).Set(float64(mu)) - poolTierChangeMetric.WithLabelValues(tierUnknownToMain).Set(float64(um)) - - mt := p.th.GetPoolMetrics() - poolSizeMetric.WithLabelValues(string(tieredhashing.TierUnknown)).Set(float64(mt.Unknown)) - poolSizeMetric.WithLabelValues(string(tieredhashing.TierMain)).Set(float64(mt.Main)) - - // Update aggregate latency & speed distribution for peers - latencyHist := prometheus.NewHistogramVec(prometheus.HistogramOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "fetch_peer_latency_dist"), - Help: "Fetch latency distribution for peers in millis", - Buckets: latencyDistMsHistogram, - }, []string{"tier", "percentile"}) - - percentiles := []float64{25, 50, 75, 90, 95} - - for _, perf := range p.th.GetPerf() { - perf := perf - if perf.NLatencyDigest <= 0 { - continue - } - - for _, pt := range percentiles { - latencyHist.WithLabelValues(string(perf.Tier), fmt.Sprintf("P%f", pt)).Observe(perf.LatencyDigest.Reduce(rolling.Percentile(pt))) - } + if err := updateActiveNodes(p.ActiveNodes, p.AllNodes); err != nil { + goLogger.Warnw("failed to update active nodes", "error", err) } - peerLatencyDistribution = latencyHist } +// refreshPool is a background thread triggering `DoRefresh` every `config.PoolRefresh` interval. func (p *pool) refreshPool() { t := time.NewTimer(0) started := sync.Once{} for { select { case <-t.C: - p.doRefresh() + p.DoRefresh() started.Do(func() { close(p.started) }) t.Reset(p.config.PoolRefresh) case <-p.refresh: - p.doRefresh() + p.DoRefresh() started.Do(func() { close(p.started) }) @@ -188,29 +148,67 @@ func (p *pool) refreshPool() { } } +func (p *pool) fetchComplianceCid(node *Node) error { + sc := node.ComplianceCid + if len(node.ComplianceCid) == 0 { + goLogger.Warnw("failed to find compliance cid ", "for node", node) + return fmt.Errorf("compliance cid doesn't exist for node: %s ", node) + } + trialTimeout, cancel := context.WithTimeout(context.Background(), 30*time.Second) + reqUrl := fmt.Sprintf(complianceCidReqTemplate, sc) + goLogger.Debugw("fetching compliance cid", "cid", reqUrl, "from", node) + err := p.fetchResourceAndUpdate(trialTimeout, node, reqUrl, 0, p.mirrorValidator) + cancel() + return err +} + func (p *pool) checkPool() { + sem := make(chan struct{}, defaultMirroredConcurrency) + for { select { case msg := <-p.mirrorSamples: - // see if it is to a main-tier node - if so find appropriate test node to test against. - p.lk.RLock() - if p.th.NodeTier(msg.node) != tieredhashing.TierMain { - p.lk.RUnlock() - continue - } - testNodes := p.th.GetNodes(tieredhashing.TierUnknown, msg.key, 1) - p.lk.RUnlock() - if len(testNodes) == 0 { - continue - } - trialTimeout, cancel := context.WithTimeout(context.Background(), 30*time.Second) - err := p.fetchResourceAndUpdate(trialTimeout, testNodes[0], msg.path, 0, p.mirrorValidator) - cancel() - if err != nil { - mirroredTrafficTotalMetric.WithLabelValues("error").Inc() - } else { - mirroredTrafficTotalMetric.WithLabelValues("no-error").Inc() - } + sem <- struct{}{} + go func(msg mirroredPoolRequest) { + defer func() { <-sem }() + + // see if it is to a main-tier node - if so find appropriate test node to test against. + if !p.ActiveNodes.Contains(msg.node) { + return + } + testNode := p.AllNodes.PeekRandom() + if testNode == nil { + return + } + if p.ActiveNodes.Contains(testNode) { + rand := big.NewInt(1) + if p.config.ComplianceCidPeriod > 0 { + rand, _ = cryptoRand.Int(cryptoRand.Reader, big.NewInt(p.config.ComplianceCidPeriod)) + } + + if rand.Cmp(big.NewInt(0)) == 0 { + err := p.fetchComplianceCid(testNode) + if err != nil { + goLogger.Warnw("failed to fetch compliance cid ", "err", err) + complianceCidCallsTotalMetric.WithLabelValues("error").Add(1) + } else { + complianceCidCallsTotalMetric.WithLabelValues("success").Add(1) + } + } + return + } + + trialTimeout, cancel := context.WithTimeout(context.Background(), 30*time.Second) + err := p.fetchResourceAndUpdate(trialTimeout, testNode, msg.path, 0, p.mirrorValidator) + + cancel() + if err != nil { + mirroredTrafficTotalMetric.WithLabelValues("error").Inc() + } else { + mirroredTrafficTotalMetric.WithLabelValues("no-error").Inc() + } + }(msg) + case <-p.done: return } @@ -240,7 +238,7 @@ func (p *pool) mirrorValidator(resource string, reader io.Reader) error { return err } - br, err := car.NewCarReader(reader) + br, err := car.NewCarReaderWithOptions(reader, car.WithErrorOnEmptyRoots(false)) if err != nil { return err } @@ -272,80 +270,27 @@ func (p *pool) Close() { } } -func cidToKey(c cid.Cid) string { - return c.Hash().B58String() -} - func (p *pool) fetchBlockWith(ctx context.Context, c cid.Cid, with string) (blk blocks.Block, err error) { fetchCalledTotalMetric.WithLabelValues(resourceTypeBlock).Add(1) - if recordIfContextErr(resourceTypeBlock, ctx, "fetchBlockWith") { - return nil, ctx.Err() - } // wait for pool to be initialised <-p.started - // if the cid is in the cool down cache, we fail the request. - p.fetchKeyLk.RLock() - if at, ok := p.fetchKeyCoolDownCache.Get(cidToKey(c)); ok { - p.fetchKeyLk.RUnlock() - - expireAt := at.(time.Time) - return nil, &ErrCoolDown{ - Cid: c, - retryAfter: time.Until(expireAt), - } - } - p.fetchKeyLk.RUnlock() - - aff := with - if aff == "" { - aff = cidToKey(c) - } - - p.lk.RLock() - nodes := p.th.GetNodes(tieredhashing.TierMain, aff, p.config.MaxRetrievalAttempts) - if len(nodes) < p.config.MaxRetrievalAttempts { - nodes = append(nodes, - p.th.GetNodes(tieredhashing.TierUnknown, aff, p.config.MaxRetrievalAttempts-len(nodes))..., - ) - } - p.lk.RUnlock() - if len(nodes) == 0 { - return nil, ErrNoBackend - } - - blockFetchStart := time.Now() - for i := 0; i < len(nodes); i++ { - if recordIfContextErr(resourceTypeBlock, ctx, "fetchBlockWithLoop") { - return nil, ctx.Err() + cb := func(resource string, reader io.Reader) error { + br, err := car.NewCarReader(reader) + if err != nil { + return err } - - blk, err = p.fetchBlockAndUpdate(ctx, nodes[i], c, i) - if err != nil && errors.Is(err, context.Canceled) { - return nil, err + b, err := br.Next() + if err != nil { + return err } - - if err == nil { - durationMs := time.Since(blockFetchStart).Milliseconds() - fetchDurationBlockSuccessMetric.Observe(float64(durationMs)) - - // mirror successful request - if p.config.MirrorFraction > rand.Float64() { - select { - case p.mirrorSamples <- mirroredPoolRequest{node: nodes[i], path: fmt.Sprintf("/ipfs/%s?format=car&car-scope=block", c), key: aff}: - default: - } - } - - return + if b.Cid().Equals(c) { + blk = b + return nil } + return blocks.ErrWrongHash } - - fetchDurationBlockFailureMetric.Observe(float64(time.Since(blockFetchStart).Milliseconds())) - - p.updateFetchKeyCoolDown(cidToKey(c)) - - // Saturn fetch failed after exhausting all retrieval attempts, we can return the error. + err = p.fetchResourceWith(ctx, fmt.Sprintf(blockPathPattern, c), cb, with) return } @@ -379,7 +324,7 @@ func (p *pool) updateFetchKeyCoolDown(key string) { func (p *pool) fetchResourceWith(ctx context.Context, path string, cb DataCallback, with string) (err error) { fetchCalledTotalMetric.WithLabelValues(resourceTypeCar).Add(1) - if recordIfContextErr(resourceTypeCar, ctx, "fetchResourceWith") { + if isCtxError(ctx) { return ctx.Err() } @@ -404,16 +349,10 @@ func (p *pool) fetchResourceWith(ctx context.Context, path string, cb DataCallba aff = path } - p.lk.RLock() - nodes := p.th.GetNodes(tieredhashing.TierMain, aff, p.config.MaxRetrievalAttempts) - if len(nodes) < p.config.MaxRetrievalAttempts { - nodes = append(nodes, - p.th.GetNodes(tieredhashing.TierUnknown, aff, p.config.MaxRetrievalAttempts-len(nodes))..., - ) - } else { - goLogger.Infow("using all main set nodes for CAR", "path", path, "aff", aff, "numNodes", len(nodes)) + nodes, err := p.ActiveNodes.GetNodes(aff, p.config.MaxRetrievalAttempts) + if err != nil { + return err } - p.lk.RUnlock() if len(nodes) == 0 { return ErrNoBackend } @@ -422,7 +361,7 @@ func (p *pool) fetchResourceWith(ctx context.Context, path string, cb DataCallba pq := []string{path} for i := 0; i < len(nodes); i++ { - if recordIfContextErr(resourceTypeCar, ctx, "fetchResourceWithLoop") { + if isCtxError(ctx) { return ctx.Err() } @@ -433,6 +372,7 @@ func (p *pool) fetchResourceWith(ctx context.Context, path string, cb DataCallba default: } } + old := pq[0] err = p.fetchResourceAndUpdate(ctx, nodes[i], pq[0], i, cb) if err != nil && errors.Is(err, context.Canceled) { return err @@ -447,6 +387,8 @@ func (p *pool) fetchResourceWith(ctx context.Context, path string, cb DataCallba //fetchSpeedPerBlockMetric.Observe(float64(float64(len(blk.RawData())) / float64(durationMs))) fetchDurationCarSuccessMetric.Observe(float64(durationMs)) return + } else if pq[0] == old { + continue } else { // TODO: potentially worth doing something smarter here based on what the current state // of permanent vs temporary errors is. @@ -462,11 +404,16 @@ func (p *pool) fetchResourceWith(ctx context.Context, path string, cb DataCallba } pq = pq[1:] pq = append(pq, epr.StillNeed...) - // TODO: potentially worth doing something smarter here based on what the current state - // of permanent vs temporary errors is. - // for now: reset i on partials so we also give them a chance to retry. - i = -1 + if pq[0] == old { + continue + } else { + // TODO: potentially worth doing something smarter here based on what the current state + // of permanent vs temporary errors is. + + // for now: reset i on partials so we also give them a chance to retry. + i = -1 + } } } @@ -477,70 +424,16 @@ func (p *pool) fetchResourceWith(ctx context.Context, path string, cb DataCallba return } -func (p *pool) fetchBlockAndUpdate(ctx context.Context, node string, c cid.Cid, attempt int) (blk blocks.Block, err error) { - blk, rm, err := p.doFetch(ctx, node, c, attempt) - if err != nil && errors.Is(err, context.Canceled) { - return nil, err - } - if err != nil { - goLogger.Debugw("fetch attempt failed", "from", node, "attempt", attempt, "of", c, "error", err) - } - - err = p.commonUpdate(node, rm, err) - return -} - -func (p *pool) fetchResourceAndUpdate(ctx context.Context, node string, path string, attempt int, cb DataCallback) (err error) { - rm, err := p.fetchResource(ctx, node, path, "application/vnd.ipld.car", attempt, cb) +func (p *pool) fetchResourceAndUpdate(ctx context.Context, node *Node, path string, attempt int, cb DataCallback) (err error) { + err = p.fetchResource(ctx, node, path, "application/vnd.ipld.car", attempt, cb) if err != nil && errors.Is(err, context.Canceled) { return err } + if err != nil { goLogger.Debugw("fetch attempt failed", "from", node, "attempt", attempt, "of", path, "error", err) } - p.commonUpdate(node, rm, err) - return -} - -func (p *pool) commonUpdate(node string, rm tieredhashing.ResponseMetrics, err error) (ferr error) { - p.lk.Lock() - defer p.lk.Unlock() - - ferr = err - if err == nil && rm.Success { - p.th.RecordSuccess(node, rm) - - if p.th.IsInitDone() { - p.poolInitDone.Do(func() { - poolEnoughObservationsForMainSetDurationMetric.Set(float64(time.Since(p.th.StartAt).Milliseconds())) - }) - } - - // Saturn fetch worked, we return the block. - return - } - - fr := p.th.RecordFailure(node, rm) - if fr != nil { - poolRemovedFailureTotalMetric.WithLabelValues(string(fr.Tier), fr.Reason).Inc() - poolRemovedConnFailureTotalMetric.WithLabelValues(string(fr.Tier)).Add(float64(fr.ConnErrors)) - poolRemovedReadFailureTotalMetric.WithLabelValues(string(fr.Tier)).Add(float64(fr.NetworkErrors)) - poolRemovedNon2xxTotalMetric.WithLabelValues(string(fr.Tier)).Add(float64(fr.ResponseCodes)) - - if fr.MainToUnknownChange != 0 || fr.UnknownToMainChange != 0 { - poolTierChangeMetric.WithLabelValues(tierMainToUnknown).Set(float64(fr.MainToUnknownChange)) - poolTierChangeMetric.WithLabelValues(tierUnknownToMain).Set(float64(fr.UnknownToMainChange)) - } - } - - if p.th.DoRefresh() { - select { - case p.refresh <- struct{}{}: - default: - } - } - return } diff --git a/pool_dynamics_test.go b/pool_dynamics_test.go new file mode 100644 index 0000000..dc2d4a9 --- /dev/null +++ b/pool_dynamics_test.go @@ -0,0 +1,335 @@ +package caboose_test + +import ( + "context" + cryptoRand "crypto/rand" + "fmt" + "math/rand" + "net/url" + "testing" + "time" + + "github.com/filecoin-saturn/caboose" + "github.com/filecoin-saturn/caboose/internal/util" + "github.com/ipfs/go-cid" + "github.com/multiformats/go-multicodec" + "github.com/stretchr/testify/assert" +) + +const ( + nodesSize = 6 +) +const blockPathPattern = "/ipfs/%s?format=car&dag-scope=block" + +/* +This function tests if the caboose pool converges to a set of nodes that are expected +based on given controled scenarios. The function continuously injects stats into +certain nodes and simulates the caboose pool refreshing over time and updating its +active set of nodes based on the stats injected. + +The tests are designed such that there is two groups of nodes: "bad", and "good". Those +are picked randomly in the beginning of each test. At the end of each test, the pool should +always be converging to the "good" nodes. +*/ +func TestPoolDynamics(t *testing.T) { + baseStatSize := 100000 + baseStatLatency := 100 + poolRefreshNo := 10 + ctx := context.Background() + testCid, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum(testBlock) + + // This test ensures that when the pool is intialized, it should converge to a set + // of nodes that have stats vs a set of nodes that don't have any stats. + t.Run("pool converges to good nodes vs nodes with no stats", func(t *testing.T) { + ch, controlGroup := getHarnessAndControlGroup(t, nodesSize, nodesSize/2) + ch.FetchAndAssertSuccess(t, ctx, testCid) + + goodNodes := make([]*caboose.Node, 0) + for _, n := range ch.CabooseAllNodes.Nodes { + _, ok := controlGroup[n.URL] + if ok { + goodNodes = append(goodNodes, n) + } + } + + for i := 0; i < 1; i++ { + goodStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) / float64(10), + Size: float64(baseStatSize) * float64(10), + } + + ch.RecordSuccesses(t, goodNodes, goodStats, 1000) + ch.CaboosePool.DoRefresh() + } + + for n := range controlGroup { + assert.Contains(t, ch.CabooseActiveNodes.Nodes, n) + } + }) + + t.Run("pool converges to good nodes vs nodes with worse stats", func(t *testing.T) { + ch, controlGroup := getHarnessAndControlGroup(t, nodesSize, nodesSize/2) + ch.FetchAndAssertSuccess(t, ctx, testCid) + + goodNodes := make([]*caboose.Node, 0) + badNodes := make([]*caboose.Node, 0) + for _, n := range ch.CabooseAllNodes.Nodes { + _, ok := controlGroup[n.URL] + if ok { + goodNodes = append(goodNodes, n) + } else { + badNodes = append(badNodes, n) + } + } + + for i := 0; i < poolRefreshNo; i++ { + + goodStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) / float64(10), + Size: float64(baseStatSize) * float64(10), + } + badStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) * float64(10), + Size: float64(baseStatSize) / float64(10), + } + + ch.RecordSuccesses(t, goodNodes, goodStats, 1000) + ch.RecordSuccesses(t, badNodes, badStats, 1000) + ch.CaboosePool.DoRefresh() + } + + for n := range controlGroup { + assert.Contains(t, ch.CabooseActiveNodes.Nodes, n) + } + }) + + // When new nodes join, if they start consistently performing better than the nodes in the current pool, + // then those nodes should replace the nodes in the current pool. + t.Run("pool converges to new nodes that are better than the current pool", func(t *testing.T) { + ch, controlGroup := getHarnessAndControlGroup(t, nodesSize, nodesSize/2) + ch.FetchAndAssertSuccess(t, ctx, testCid) + + goodNodes := make([]*caboose.Node, 0) + badNodes := make([]*caboose.Node, 0) + + for _, n := range ch.CabooseAllNodes.Nodes { + _, ok := controlGroup[n.URL] + if ok { + goodNodes = append(goodNodes, n) + } else { + badNodes = append(badNodes, n) + } + } + + // Give the bad nodes some stats, those nodes then become the main active tier. + // The good nodes have 0 stats after this should not be picked at this point. + for i := 0; i < poolRefreshNo; i++ { + badStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) * float64(10), + Size: float64(baseStatSize) / float64(10), + } + ch.RecordSuccesses(t, badNodes, badStats, 1000) + ch.CaboosePool.DoRefresh() + } + + // Add some new "good" nodes that have better stats over a longer period of time. + for i := 0; i < poolRefreshNo*2; i++ { + goodStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) / float64(10), + Size: float64(baseStatSize) * float64(10), + } + ch.RecordSuccesses(t, goodNodes, goodStats, 2000) + ch.CaboosePool.DoRefresh() + } + + ch.CaboosePool.DoRefresh() + for n := range controlGroup { + assert.Contains(t, ch.CabooseActiveNodes.Nodes, n) + } + + }) + + // If the current active main pool starts failing, the pool should converge to + // to nodes that are not failing. + t.Run("pool converges to other nodes if the current ones start failing", func(t *testing.T) { + ch, controlGroup := getHarnessAndControlGroup(t, nodesSize, nodesSize/2) + ch.FetchAndAssertSuccess(t, ctx, testCid) + + goodNodes := make([]*caboose.Node, 0) + badNodes := make([]*caboose.Node, 0) + + for _, n := range ch.CabooseAllNodes.Nodes { + _, ok := controlGroup[n.URL] + if ok { + goodNodes = append(goodNodes, n) + } else { + badNodes = append(badNodes, n) + } + } + + // Start with the bad nodes having better stats than the good nodes + for i := 0; i < poolRefreshNo; i++ { + goodStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) / float64(10), + Size: float64(baseStatSize) * float64(10), + } + badStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) * float64(10), + Size: float64(baseStatSize) / float64(10), + } + + ch.RecordSuccesses(t, goodNodes, badStats, 1000) + ch.RecordSuccesses(t, badNodes, goodStats, 1000) + ch.CaboosePool.DoRefresh() + } + + // Start failing the bad nodes and keep giving the same stats to the good nodes. + for i := 0; i < poolRefreshNo*2; i++ { + badStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) * float64(10), + Size: float64(baseStatSize) / float64(10), + } + + ch.RecordSuccesses(t, goodNodes, badStats, 1000) + ch.RecordFailures(t, badNodes, 1000) + ch.CaboosePool.DoRefresh() + } + + ch.CaboosePool.DoRefresh() + for n := range controlGroup { + assert.Contains(t, ch.CabooseActiveNodes.Nodes, n) + } + + }) + +} + +func TestPoolAffinity(t *testing.T) { + baseStatSize := 100000 + baseStatLatency := 100 + // statVarianceFactor := 0.1 + poolRefreshNo := 10 + simReqCount := 10000 + ctx := context.Background() + cidList := generateRandomCIDs(20) + + t.Run("selected nodes remain consistent for same cid reqs", func(t *testing.T) { + // 80 nodes will be in the good pool. 20 will be added later with the same stats. + // So, 20% of the nodes in the pool will eventually be "new nodes" that have been added later. + ch, controlGroup := getHarnessAndControlGroup(t, 100, 80) + _, _ = ch.Caboose.Get(ctx, cidList[0]) + + existingNodes := make([]*caboose.Node, 0) + newNodes := make([]*caboose.Node, 0) + + for _, n := range ch.CabooseAllNodes.Nodes { + _, ok := controlGroup[n.URL] + if ok { + existingNodes = append(existingNodes, n) + } else { + newNodes = append(newNodes, n) + } + } + + // Send requests to control group nodes to bump their selection into the pool. + for i := 0; i < poolRefreshNo; i++ { + baseStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) / float64(10), + Size: float64(baseStatSize) * float64(10), + } + + ch.RecordSuccesses(t, existingNodes, baseStats, 1000) + ch.CaboosePool.DoRefresh() + } + + // Make a bunch of requests to similar cids to establish a stable hashring + for i := 0; i < simReqCount; i++ { + rand.New(rand.NewSource(time.Now().Unix())) + idx := rand.Intn(len(cidList)) + _, _ = ch.Caboose.Get(ctx, cidList[idx]) + } + ch.CaboosePool.DoRefresh() + + // Introduce new nodes by sendng same stats to those nodes. + for i := 0; i < poolRefreshNo/2; i++ { + baseStats := util.NodeStats{ + Start: time.Now().Add(-time.Second * 2), + Latency: float64(baseStatLatency) / float64(10), + Size: float64(baseStatSize) * float64(10), + } + + ch.RecordSuccesses(t, existingNodes, baseStats, 100) + ch.RecordSuccesses(t, newNodes, baseStats, 10) + + ch.CaboosePool.DoRefresh() + } + + rerouteCount := 0 + + // Get the candidate nodes for each cid in the cid list to see if it's been rerouted to a new node. + for _, c := range cidList { + aff := ch.Caboose.GetAffinity(ctx) + if aff == "" { + aff = fmt.Sprintf(blockPathPattern, c) + } + nodes, _ := ch.CabooseActiveNodes.GetNodes(aff, ch.Config.MaxRetrievalAttempts) + + for _, n := range newNodes { + n := n + if n.URL == nodes[0].URL { + rerouteCount++ + } + } + } + + // no more than 5 cids from the cid list of 20 should get re-routed (25%) + assert.LessOrEqual(t, rerouteCount, 5) + }) +} + +func getHarnessAndControlGroup(t *testing.T, nodesSize int, poolSize int) (*util.CabooseHarness, map[string]string) { + ch := util.BuildCabooseHarness(t, nodesSize, 3, func(config *caboose.Config) { + config.PoolTargetSize = nodesSize / 2 + }) + + ch.StartOrchestrator() + + rand.New(rand.NewSource(0)) + eps := ch.Endpoints + controlGroup := make(map[string]string) + + rand.Shuffle(len(eps), func(i, j int) { + eps[i], eps[j] = eps[j], eps[i] + }) + + for _, ep := range eps[:poolSize] { + url, _ := url.Parse(ep.Server.URL) + controlGroup[url.Host] = ep.Server.URL + } + + return ch, controlGroup +} + +func generateRandomCIDs(count int) []cid.Cid { + var cids []cid.Cid + for i := 0; i < count; i++ { + block := make([]byte, 32) + cryptoRand.Read(block) + c, _ := cid.V1Builder{ + Codec: uint64(multicodec.Raw), + MhType: uint64(multicodec.Sha2_256), + }.Sum(block) + + cids = append(cids, c) + } + return cids +} diff --git a/pool_metrics.go b/pool_metrics.go index 6e6ca69..f61da20 100644 --- a/pool_metrics.go +++ b/pool_metrics.go @@ -4,31 +4,6 @@ import "github.com/prometheus/client_golang/prometheus" // pool metrics var ( - poolRemovedFailureTotalMetric = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "pool_removed_failure_total"), - }, []string{"tier", "reason"}) - - poolRemovedConnFailureTotalMetric = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "pool_removed_conn_failure_total"), - }, []string{"tier"}) - - poolRemovedReadFailureTotalMetric = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "pool_removed_read_failure_total"), - }, []string{"tier"}) - - poolRemovedNon2xxTotalMetric = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "pool_removed_non2xx_total"), - }, []string{"tier"}) - - // The below metrics are only updated periodically on every Caboose pool refresh - poolMembersNotAddedBecauseRemovedMetric = prometheus.NewGauge(prometheus.GaugeOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "pool_members_not_added"), - }) - - poolMembersRemovedAndAddedBackMetric = prometheus.NewGauge(prometheus.GaugeOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "pool_removed_and_added_back"), - }) - poolRefreshErrorMetric = prometheus.NewCounter(prometheus.CounterOpts{ Name: prometheus.BuildFQName("ipfs", "caboose", "pool_refresh_errors"), Help: "Number of errors refreshing the caboose pool", @@ -44,10 +19,6 @@ var ( Help: "New members added to the Caboose pool", }) - poolEnoughObservationsForMainSetDurationMetric = prometheus.NewGauge(prometheus.GaugeOpts{ - Name: prometheus.BuildFQName("ipfs", "caboose", "pool_observations_for_main_set_duration"), - }) - poolTierChangeMetric = prometheus.NewGaugeVec(prometheus.GaugeOpts{ Name: prometheus.BuildFQName("ipfs", "caboose", "pool_tier_change"), }, []string{"change"}) diff --git a/pool_refresh_test.go b/pool_refresh_test.go index 1012585..5ae4ad4 100644 --- a/pool_refresh_test.go +++ b/pool_refresh_test.go @@ -1,70 +1,57 @@ package caboose import ( - "github.com/filecoin-saturn/caboose/tieredhashing" - "github.com/stretchr/testify/require" + "math/rand" "testing" + + "github.com/filecoin-saturn/caboose/internal/state" + "github.com/ipfs/go-cid" + "github.com/multiformats/go-multicodec" + "github.com/stretchr/testify/require" ) func TestPoolRefresh(t *testing.T) { - opts := []tieredhashing.Option{tieredhashing.WithCorrectnessWindowSize(1), tieredhashing.WithMaxPoolSize(5)} - - p := newPool(&Config{TieredHashingOpts: opts}) + p := newPool(&Config{}, nil) // Add 3 nodes nodes := []string{"node1", "node2", "node3"} - andAndAssertPool(t, p, nodes, 0, 3, 3, 3) + addAndAssertPool(t, p, nodes, 3) // add no node - andAndAssertPool(t, p, nil, 0, 3, 3, 0) + addAndAssertPool(t, p, nil, 3) // add a new node - andAndAssertPool(t, p, []string{"node4"}, 0, 4, 4, 1) + addAndAssertPool(t, p, []string{"node4"}, 4) // add a new node with already added nodes - andAndAssertPool(t, p, []string{"node1", "node2", "node3", "node4", "node5"}, 0, 5, 5, 1) + addAndAssertPool(t, p, []string{"node1", "node2", "node3", "node4", "node5"}, 5) // record failure so that node is removed and then assert - rm := p.th.RecordFailure("node4", tieredhashing.ResponseMetrics{ConnFailure: true}) - require.NotNil(t, rm) - require.EqualValues(t, "node4", rm.Node) - + // TODO // removed node is NOT added back as pool is full without it - andAndAssertPool(t, p, []string{"node1", "node2", "node3", "node4", "node5", "node6"}, 0, 5, 5, 0) - nds := p.th.GetPerf() - for node := range nds { - require.NotEqual(t, "node4", node) - } - + // TODO } -func TestPoolRefreshWithLatencyDistribution(t *testing.T) { - t.Skip("ENABLE if we go back to tiered hashing") - opts := []tieredhashing.Option{tieredhashing.WithLatencyWindowSize(2), tieredhashing.WithMaxMainTierSize(2)} - - p := newPool(&Config{TieredHashingOpts: opts}) - nodes := []string{"node1", "node2", "node3"} - andAndAssertPool(t, p, nodes, 0, 3, 3, 3) - - // record success so a node becomes a main node - p.th.RecordSuccess("node1", tieredhashing.ResponseMetrics{TTFBMs: 10}) - andAndAssertPool(t, p, nodes, 0, 3, 3, 0) - - p.th.RecordSuccess("node1", tieredhashing.ResponseMetrics{TTFBMs: 20}) - andAndAssertPool(t, p, nodes, 0, 3, 3, 0) - - p.th.RecordSuccess("node2", tieredhashing.ResponseMetrics{TTFBMs: 30}) - p.th.RecordSuccess("node2", tieredhashing.ResponseMetrics{TTFBMs: 40}) - andAndAssertPool(t, p, nodes, 2, 1, 3, 0) +func addAndAssertPool(t *testing.T, p *pool, nodes []string, expectedTotal int) { + nodeStructs := genNodeStructs(nodes) + for _, n := range nodeStructs { + p.AllNodes.AddIfNotPresent(NewNode(n)) + } + require.Equal(t, expectedTotal, p.AllNodes.Len()) } -func andAndAssertPool(t *testing.T, p *pool, nodes []string, expectedMain, expectedUnknown, expectedTotal, expectedNew int) { - p.refreshWithNodes(nodes) - nds := p.th.GetPerf() - require.Equal(t, expectedTotal, len(nds)) - mts := p.th.GetPoolMetrics() - - require.EqualValues(t, expectedMain, mts.Main) - require.EqualValues(t, expectedUnknown, mts.Unknown) - require.EqualValues(t, expectedTotal, mts.Total) +func genNodeStructs(nodes []string) []state.NodeInfo { + var nodeStructs []state.NodeInfo + + for _, node := range nodes { + cid, _ := cid.V1Builder{Codec: uint64(multicodec.Raw), MhType: uint64(multicodec.Sha2_256)}.Sum([]byte(node)) + nodeStructs = append(nodeStructs, state.NodeInfo{ + IP: node, + ID: node, + Weight: rand.Intn(100), + Distance: rand.Float32(), + ComplianceCid: cid.String(), + }) + } + return nodeStructs } diff --git a/pool_test.go b/pool_test.go index fa10a90..543078b 100644 --- a/pool_test.go +++ b/pool_test.go @@ -1,19 +1,13 @@ -package caboose +package caboose_test import ( "bytes" "context" - "crypto/tls" - "net/http" - "net/http/httptest" - "net/url" - "strings" - "sync" "testing" "time" "unsafe" - "github.com/filecoin-saturn/caboose/tieredhashing" + "github.com/filecoin-saturn/caboose/internal/util" "github.com/ipfs/go-cid" "github.com/ipld/go-car/v2" "github.com/ipld/go-ipld-prime" @@ -24,127 +18,80 @@ import ( "github.com/multiformats/go-multicodec" ) -type ep struct { - server *httptest.Server - valid bool - cnt int - httpCode int - resp []byte - lk sync.Mutex -} - -var testBlock = []byte("hello World") - -func (e *ep) Setup() { - e.valid = true - e.resp = testBlock - e.server = httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - e.lk.Lock() - defer e.lk.Unlock() - e.cnt++ - if e.valid { - w.Write(e.resp) - } else { - if e.httpCode == http.StatusTooManyRequests { - w.Header().Set("Retry-After", "1") - } - if e.httpCode == 0 { - e.httpCode = 500 - } - w.WriteHeader(e.httpCode) - w.Write([]byte("error")) - } - })) -} - -func TestPoolMiroring(t *testing.T) { +func TestPoolMirroring(t *testing.T) { if unsafe.Sizeof(unsafe.Pointer(nil)) <= 4 { t.Skip("skipping for 32bit architectures because too slow") } - opts := []tieredhashing.Option{ - tieredhashing.WithCorrectnessWindowSize(2), - tieredhashing.WithLatencyWindowSize(2), - tieredhashing.WithMaxMainTierSize(1), - } - - saturnClient := &http.Client{ - Transport: &http.Transport{ - TLSClientConfig: &tls.Config{ - InsecureSkipVerify: true, - }, - }, - } - data := []byte("hello world") + data := []byte("hello World") ls := cidlink.DefaultLinkSystem() lsm := memstore.Store{} ls.SetReadStorage(&lsm) ls.SetWriteStorage(&lsm) finalCL := ls.MustStore(ipld.LinkContext{}, cidlink.LinkPrototype{Prefix: cid.NewPrefixV1(uint64(multicodec.Raw), uint64(multicodec.Sha2_256))}, basicnode.NewBytes(data)) finalC := finalCL.(cidlink.Link).Cid - cw, err := car.NewSelectiveWriter(context.TODO(), &ls, finalC, selectorparse.CommonSelector_MatchAllRecursively) + + carBytes := bytes.NewBuffer(nil) + _, err := car.TraverseV1(context.TODO(), &ls, finalC, selectorparse.CommonSelector_MatchAllRecursively, carBytes) if err != nil { t.Fatal(err) } - carBytes := bytes.NewBuffer(nil) - cw.WriteTo(carBytes) - - e := ep{} - e.Setup() - e.lk.Lock() - e.resp = carBytes.Bytes() - eURL := strings.TrimPrefix(e.server.URL, "https://") - e.lk.Unlock() - - e2 := ep{} - e2.Setup() - e2.lk.Lock() - e2.resp = carBytes.Bytes() - e2URL := strings.TrimPrefix(e2.server.URL, "https://") - e2.lk.Unlock() - - conf := Config{ - OrchestratorEndpoint: &url.URL{}, - OrchestratorClient: http.DefaultClient, - OrchestratorOverride: []string{eURL, e2URL}, - LoggingEndpoint: url.URL{}, - LoggingClient: http.DefaultClient, - LoggingInterval: time.Hour, - - SaturnClient: saturnClient, - DoValidation: false, - PoolRefresh: time.Minute, - MaxRetrievalAttempts: 1, - TieredHashingOpts: opts, - MirrorFraction: 1.0, + + ch := util.BuildCabooseHarness(t, 2, 3) + + if err != nil { + t.Fatal(err) + } + + // we don't know if any individual request is going to deterministically trigger a mirror request. + // Make 10 requests, and expect some fraction trigger a mirror. + + for i := 0; i < 10; i++ { + _, err = ch.Caboose.Get(context.Background(), finalC) + if err != nil { + t.Fatal(err) + } + + } + + time.Sleep(100 * time.Millisecond) + ch.Caboose.Close() + + ec := ch.Endpoints[0].Count() + + e2c := ch.Endpoints[1].Count() + if ec+e2c < 10 { + t.Fatalf("expected at least 10 fetches, got %d", ec+e2c) } +} - p := newPool(&conf) - p.doRefresh() - p.config.OrchestratorOverride = nil - p.Start() +func TestFetchComplianceCid(t *testing.T) { + if unsafe.Sizeof(unsafe.Pointer(nil)) <= 4 { + t.Skip("skipping for 32bit architectures because too slow") + } - // promote one node to main pool. other will remain in uknown pool. - p.th.RecordSuccess(eURL, tieredhashing.ResponseMetrics{Success: true, TTFBMs: 30, SpeedPerMs: 30}) - p.th.RecordSuccess(eURL, tieredhashing.ResponseMetrics{Success: true, TTFBMs: 30, SpeedPerMs: 30}) - p.th.UpdateMainTierWithTopN() + ch := util.BuildCabooseHarness(t, 1, 1, util.WithComplianceCidPeriod(1), util.WithMirrorFraction(1.0)) - _, err = p.fetchBlockWith(context.Background(), finalC, "") + ch.CaboosePool.DoRefresh() + + ls := cidlink.DefaultLinkSystem() + lsm := memstore.Store{} + ls.SetReadStorage(&lsm) + ls.SetWriteStorage(&lsm) + finalCL := ls.MustStore(ipld.LinkContext{}, cidlink.LinkPrototype{Prefix: cid.NewPrefixV1(uint64(multicodec.Raw), uint64(multicodec.Sha2_256))}, basicnode.NewBytes(testBlock)) + finalC := finalCL.(cidlink.Link).Cid + + _, err := ch.Caboose.Get(context.Background(), finalC) if err != nil { t.Fatal(err) } time.Sleep(100 * time.Millisecond) - p.Close() + ch.Caboose.Close() - e.lk.Lock() - defer e.lk.Unlock() - if e.cnt != 1 { - t.Fatalf("expected 1 primary fetch, got %d", e.cnt) - } - e2.lk.Lock() - defer e2.lk.Unlock() - if e2.cnt != 1 { - t.Fatalf("expected 1 mirrored fetch, got %d", e2.cnt) + e := ch.Endpoints[0] + + if e.Count() != 2 { + t.Fatalf("expected 2 primary fetch, got %d", e.Count()) } } diff --git a/pool_tier_promotion.go b/pool_tier_promotion.go new file mode 100644 index 0000000..36cf98a --- /dev/null +++ b/pool_tier_promotion.go @@ -0,0 +1,29 @@ +package caboose + +var ( + activationThreshold = 0 +) + +func updateActiveNodes(active *NodeRing, all *NodeHeap) error { + candidates := all.TopN(active.targetSize) + added := 0 + for _, c := range candidates { + if active.Contains(c) { + continue + } + activeSize := active.Len() + discount := active.targetSize - activeSize + if discount < 0 { + discount = 0 + } + thisThreshold := int64(activationThreshold - discount) + add, err := active.MaybeSubstituteOrAdd(c, thisThreshold) + if err != nil { + return err + } + if add { + added += 1 + } + } + return nil +} diff --git a/tieredhashing/config.go b/tieredhashing/config.go deleted file mode 100644 index fb8e08a..0000000 --- a/tieredhashing/config.go +++ /dev/null @@ -1,64 +0,0 @@ -package tieredhashing - -import "time" - -type TieredHashingConfig struct { - MaxPoolSize int - AlwaysMainFirst bool - FailureDebounce time.Duration - LatencyWindowSize int - CorrectnessWindowSize int - CorrectnessPct float64 - MaxMainTierSize int - NoRemove bool -} - -type Option func(config *TieredHashingConfig) - -func WithNoRemove(noRemove bool) Option { - return func(c *TieredHashingConfig) { - c.NoRemove = noRemove - } -} - -func WithAlwaysMainFirst() Option { - return func(c *TieredHashingConfig) { - c.AlwaysMainFirst = true - } -} - -func WithMaxMainTierSize(n int) Option { - return func(c *TieredHashingConfig) { - c.MaxMainTierSize = n - } -} - -func WithCorrectnessPct(pct float64) Option { - return func(c *TieredHashingConfig) { - c.CorrectnessPct = pct - } -} - -func WithCorrectnessWindowSize(n int) Option { - return func(c *TieredHashingConfig) { - c.CorrectnessWindowSize = n - } -} - -func WithLatencyWindowSize(n int) Option { - return func(c *TieredHashingConfig) { - c.LatencyWindowSize = n - } -} - -func WithMaxPoolSize(n int) Option { - return func(c *TieredHashingConfig) { - c.MaxPoolSize = n - } -} - -func WithFailureDebounce(n time.Duration) Option { - return func(c *TieredHashingConfig) { - c.FailureDebounce = n - } -} diff --git a/tieredhashing/sorting.go b/tieredhashing/sorting.go deleted file mode 100644 index 7f9487b..0000000 --- a/tieredhashing/sorting.go +++ /dev/null @@ -1,42 +0,0 @@ -package tieredhashing - -import ( - "sort" - - "github.com/asecurityteam/rolling" -) - -func (t *TieredHashing) nodesSortedLatency() []nodeWithLatency { - var nodes []nodeWithLatency - - for n, perf := range t.nodes { - pc := perf - - if t.isLatencyWindowFull(pc) { - nodes = append(nodes, nodeWithLatency{ - node: n, - latency: pc.LatencyDigest.Reduce(rolling.Percentile(PLatency)), - }) - } - } - - sort.Sort(sortedNodes(nodes)) - return nodes -} - -type nodeWithLatency struct { - node string - latency float64 -} - -type sortedNodes []nodeWithLatency - -func (n sortedNodes) Len() int { return len(n) } -func (n sortedNodes) Less(i, j int) bool { - return n[i].latency <= n[j].latency -} -func (n sortedNodes) Swap(i, j int) { n[i], n[j] = n[j], n[i] } - -func (t *TieredHashing) isLatencyWindowFull(perf *NodePerf) bool { - return perf.NLatencyDigest >= float64(t.cfg.LatencyWindowSize) -} diff --git a/tieredhashing/sorting_test.go b/tieredhashing/sorting_test.go deleted file mode 100644 index 4b10525..0000000 --- a/tieredhashing/sorting_test.go +++ /dev/null @@ -1,144 +0,0 @@ -package tieredhashing - -import ( - "testing" - - "github.com/asecurityteam/rolling" - - "github.com/stretchr/testify/require" -) - -func TestNodesSortedLatency(t *testing.T) { - window := 3 - - tcs := map[string]struct { - expected []nodeWithLatency - initFn func(*TieredHashing) - }{ - "no nodes": { - expected: nil, - }, - "one node with not enough observations": { - expected: nil, - initFn: func(h *TieredHashing) { - h.nodes["node1"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - - for i := 0; i < window-1; i++ { - h.nodes["node1"].LatencyDigest.Append(1) - h.nodes["node1"].NLatencyDigest++ - } - }, - }, - "two nodes with not enough observations": { - expected: nil, - initFn: func(h *TieredHashing) { - h.nodes["node1"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - - h.nodes["node2"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - - }, - }, - "one node with enough observations": { - initFn: func(h *TieredHashing) { - h.nodes["node1"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - - for i := 0; i < window; i++ { - h.nodes["node1"].LatencyDigest.Append(1) - h.nodes["node1"].NLatencyDigest++ - } - }, - expected: []nodeWithLatency{{node: "node1", latency: 1}}, - }, - "two nodes with enough observations": { - initFn: func(h *TieredHashing) { - h.nodes["node1"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - h.nodes["node2"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - - for i := 0; i < window; i++ { - h.nodes["node1"].LatencyDigest.Append(2) - h.nodes["node1"].NLatencyDigest++ - - h.nodes["node2"].LatencyDigest.Append(1) - h.nodes["node2"].NLatencyDigest++ - } - }, - expected: []nodeWithLatency{{node: "node2", latency: 1}, {node: "node1", latency: 2}}, - }, - "3 nodes; 2 have enough observations": { - initFn: func(h *TieredHashing) { - h.nodes["node1"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - h.nodes["node2"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - - h.nodes["node3"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - - for i := 0; i < window; i++ { - h.nodes["node2"].LatencyDigest.Append(20) - h.nodes["node2"].NLatencyDigest++ - - h.nodes["node3"].LatencyDigest.Append(10) - h.nodes["node3"].NLatencyDigest++ - - if i != window-1 { - h.nodes["node1"].LatencyDigest.Append(3) - h.nodes["node1"].NLatencyDigest++ - } - } - }, - expected: []nodeWithLatency{{node: "node3", latency: 10}, {node: "node2", latency: 20}}, - }, - "rolling window": { - initFn: func(h *TieredHashing) { - h.nodes["node1"] = &NodePerf{ - Tier: TierUnknown, - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(window)), - } - - for i := 0; i < window+10; i++ { - h.nodes["node1"].LatencyDigest.Append(float64(i)) - h.nodes["node1"].NLatencyDigest++ - } - }, - expected: []nodeWithLatency{{node: "node1", latency: 12}}, - }, - } - - for name, tc := range tcs { - t.Run(name, func(t *testing.T) { - th := NewTieredHashingHarness(WithLatencyWindowSize(window)) - - if tc.initFn != nil { - tc.initFn(th.h) - } - - nds := th.h.nodesSortedLatency() - require.EqualValues(t, tc.expected, nds) - }) - } -} diff --git a/tieredhashing/tiered_hashing.go b/tieredhashing/tiered_hashing.go deleted file mode 100644 index d3b7f40..0000000 --- a/tieredhashing/tiered_hashing.go +++ /dev/null @@ -1,427 +0,0 @@ -package tieredhashing - -import ( - "math" - "net/http" - "time" - - "github.com/asecurityteam/rolling" - - "github.com/patrickmn/go-cache" - - "github.com/serialx/hashring" -) - -// TODO Make env vars for tuning -const ( - maxPoolSize = 50 - maxMainTierSize = 25 - PLatency = 90 - PMaxLatencyWithoutWindowing = 100 - - // main tier has the top `maxMainTierSize` nodes - TierMain = Tier("main") - TierUnknown = Tier("unknown") - - reasonCorrectness = "correctness" - - // use rolling windows for latency and correctness calculations - latencyWindowSize = 100 - correctnessWindowSize = 1000 - - // ------------------ CORRECTNESS ------------------- - // minimum correctness pct expected from a node over a rolling window over a certain number of observations - minAcceptableCorrectnessPct = float64(70) - - // helps shield nodes against bursty failures - failureDebounce = 2 * time.Second - removalDuration = 3 * time.Hour -) - -type Tier string - -type NodePerf struct { - LatencyDigest *rolling.PointPolicy - NLatencyDigest float64 - - CorrectnessDigest *rolling.PointPolicy - NCorrectnessDigest float64 - - Tier - - lastFailureAt time.Time - - // accumulated errors - connFailures int - networkErrors int - responseCodes int -} - -// locking is left to the caller -type TieredHashing struct { - nodes map[string]*NodePerf - - mainSet *hashring.HashRing - unknownSet *hashring.HashRing - - removedNodesTimeCache *cache.Cache - - // config - cfg TieredHashingConfig - - StartAt time.Time - initDone bool -} - -func New(opts ...Option) *TieredHashing { - cfg := &TieredHashingConfig{ - MaxPoolSize: maxPoolSize, - FailureDebounce: failureDebounce, - LatencyWindowSize: latencyWindowSize, - CorrectnessWindowSize: correctnessWindowSize, - CorrectnessPct: minAcceptableCorrectnessPct, - MaxMainTierSize: maxMainTierSize, - NoRemove: false, - } - for _, opt := range opts { - opt(cfg) - } - - return &TieredHashing{ - nodes: make(map[string]*NodePerf), - mainSet: hashring.New(nil), - unknownSet: hashring.New(nil), - removedNodesTimeCache: cache.New(removalDuration, 1*time.Minute), - cfg: *cfg, - - StartAt: time.Now(), - } -} - -func (t *TieredHashing) IsInitDone() bool { - return t.initDone -} - -func (t *TieredHashing) RecordSuccess(node string, rm ResponseMetrics) { - if _, ok := t.nodes[node]; !ok { - return - } - perf := t.nodes[node] - t.recordCorrectness(perf, true) - // record the latency and update the last bad latency record time if needed - perf.LatencyDigest.Append(rm.TTFBMs) - perf.NLatencyDigest++ -} - -type RemovedNode struct { - Node string - Tier - Reason string - ConnErrors int - NetworkErrors int - ResponseCodes int - MainToUnknownChange int - UnknownToMainChange int - ResponseCodesMap map[int]int -} - -func (t *TieredHashing) DoRefresh() bool { - return t.GetPoolMetrics().Total <= (t.cfg.MaxPoolSize / 10) -} - -func (t *TieredHashing) RecordFailure(node string, rm ResponseMetrics) *RemovedNode { - if _, ok := t.nodes[node]; !ok { - return nil - } - - perf := t.nodes[node] - if time.Since(perf.lastFailureAt) < t.cfg.FailureDebounce { - return nil - } - - recordFailureFnc := func() { - t.recordCorrectness(perf, false) - perf.lastFailureAt = time.Now() - } - - if rm.ConnFailure { - recordFailureFnc() - perf.connFailures++ - } else if rm.NetworkError { - recordFailureFnc() - perf.networkErrors++ - } else if rm.ResponseCode != http.StatusBadGateway && rm.ResponseCode != http.StatusGatewayTimeout && - rm.ResponseCode != http.StatusTooManyRequests && rm.ResponseCode != http.StatusForbidden { - // TODO Improve this in the next iteration but keep it for now as we are seeing a very high percentage of 502s - recordFailureFnc() - perf.responseCodes++ - } - - if !t.cfg.NoRemove { - if _, ok := t.isCorrectnessPolicyEligible(perf); !ok { - mc, uc := t.removeFailedNode(node) - return &RemovedNode{ - Node: node, - Tier: perf.Tier, - Reason: reasonCorrectness, - ConnErrors: perf.connFailures, - NetworkErrors: perf.networkErrors, - ResponseCodes: perf.responseCodes, - MainToUnknownChange: mc, - UnknownToMainChange: uc, - } - } - } - - return nil -} - -type PoolMetrics struct { - Unknown int - Main int - Total int -} - -func (t *TieredHashing) GetPoolMetrics() PoolMetrics { - unknown := t.unknownSet.Size() - mainS := t.mainSet.Size() - - return PoolMetrics{ - Unknown: unknown, - Main: mainS, - Total: unknown + mainS, - } -} - -func (t *TieredHashing) GetNodes(from Tier, key string, n int) []string { - var nodes []string - var ok bool - - if from == TierUnknown { - nodes, ok = t.unknownSet.GetNodes(key, t.unknownPossible(n)) - if !ok { - return nil - } - } else if from == TierMain { - nodes, ok = t.mainSet.GetNodes(key, t.mainPossible(n)) - if !ok { - return nil - } - } - - return nodes -} - -func (t *TieredHashing) NodeTier(node string) Tier { - for k, n := range t.nodes { - if k == node { - return n.Tier - } - } - return TierUnknown -} - -func (t *TieredHashing) unknownPossible(n int) int { - if n > t.unknownSet.Size() { - return t.unknownSet.Size() - } else { - return n - } -} - -func (t *TieredHashing) mainPossible(n int) int { - if n > t.mainSet.Size() { - return t.mainSet.Size() - } else { - return n - } -} - -func (t *TieredHashing) GetPerf() map[string]*NodePerf { - return t.nodes -} - -func (t *TieredHashing) AddOrchestratorNodes(nodes []string) (added, alreadyRemoved, removedAndAddedBack int) { - for _, node := range nodes { - // TODO Add nodes that are closer than the ones we have even if the pool is full - if len(t.nodes) >= t.cfg.MaxPoolSize { - return - } - - // do we already have this node ? - if _, ok := t.nodes[node]; ok { - continue - } - - // have we kicked this node out for bad correctness or latency ? - if _, ok := t.removedNodesTimeCache.Get(node); ok { - alreadyRemoved++ - continue - } - - added++ - t.nodes[node] = &NodePerf{ - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(int(t.cfg.LatencyWindowSize))), - CorrectnessDigest: rolling.NewPointPolicy(rolling.NewWindow(int(t.cfg.CorrectnessWindowSize))), - Tier: TierUnknown, - } - t.unknownSet = t.unknownSet.AddNode(node) - } - - // Avoid Pool starvation -> if we still don't have enough nodes, add the ones we have already removed - // we ensure we iterate in descending order of node closeness - for _, node := range nodes { - if len(t.nodes) >= t.cfg.MaxPoolSize { - return - } - - // do we already have this node ? - if _, ok := t.nodes[node]; ok { - continue - } - - if _, ok := t.removedNodesTimeCache.Get(node); !ok { - continue - } - - added++ - removedAndAddedBack++ - t.nodes[node] = &NodePerf{ - LatencyDigest: rolling.NewPointPolicy(rolling.NewWindow(int(t.cfg.LatencyWindowSize))), - CorrectnessDigest: rolling.NewPointPolicy(rolling.NewWindow(int(t.cfg.CorrectnessWindowSize))), - Tier: TierUnknown, - } - t.unknownSet = t.unknownSet.AddNode(node) - t.removedNodesTimeCache.Delete(node) - } - - return -} - -func (t *TieredHashing) MoveBestUnknownToMain() int { - min := math.MaxFloat64 - var node string - - for n, perf := range t.nodes { - pc := perf - if pc.Tier == TierUnknown { - latency := pc.LatencyDigest.Reduce(rolling.Percentile(PMaxLatencyWithoutWindowing)) - if latency != 0 && latency < min { - min = latency - node = n - } - } - } - - if len(node) == 0 { - return 0 - } - - t.unknownSet = t.unknownSet.RemoveNode(node) - t.mainSet = t.mainSet.AddNode(node) - t.nodes[node].Tier = TierMain - return 1 -} - -func (t *TieredHashing) UpdateMainTierWithTopN() (mainToUnknown, unknownToMain int) { - // sort all nodes by P95 and pick the top N as main tier nodes - nodes := t.nodesSortedLatency() - if len(nodes) == 0 { - return - } - - // bulk update initially so we don't end up dosing the nodes - if !t.initDone { - if len(nodes) < t.cfg.MaxMainTierSize { - return - } - t.initDone = true - } - - // Main Tier should have MIN(number of eligible nodes, max main tier size) nodes - n := t.cfg.MaxMainTierSize - if len(nodes) < t.cfg.MaxMainTierSize { - n = len(nodes) - } - - mainTier := nodes[:n] - unknownTier := nodes[n:] - - for _, nodeL := range mainTier { - if t.nodes[nodeL.node].Tier == TierUnknown { - unknownToMain++ - n := nodeL.node - t.mainSet = t.mainSet.AddNode(n) - t.unknownSet = t.unknownSet.RemoveNode(n) - t.nodes[n].Tier = TierMain - } - } - - for _, nodeL := range unknownTier { - if t.nodes[nodeL.node].Tier == TierMain { - mainToUnknown++ - n := nodeL.node - t.unknownSet = t.unknownSet.AddNode(n) - t.mainSet = t.mainSet.RemoveNode(n) - t.nodes[n].Tier = TierUnknown - } - } - - return -} - -func (t *TieredHashing) isCorrectnessPolicyEligible(perf *NodePerf) (float64, bool) { - // we don't have enough observations yet - if perf.NCorrectnessDigest < float64(t.cfg.CorrectnessWindowSize) { - return 0, true - } else { - perf.NCorrectnessDigest = float64(t.cfg.CorrectnessWindowSize) - } - - totalSuccess := perf.CorrectnessDigest.Reduce(func(w rolling.Window) float64 { - var result float64 - for _, bucket := range w { - for _, p := range bucket { - if p == 1 { - result++ - } - } - } - return result - }) - - // should satisfy a certain minimum percentage - pct := totalSuccess / perf.NCorrectnessDigest * 100 - - return pct, pct >= t.cfg.CorrectnessPct -} - -func (t *TieredHashing) removeFailedNode(node string) (mc, uc int) { - perf := t.nodes[node] - t.mainSet = t.mainSet.RemoveNode(node) - t.unknownSet = t.unknownSet.RemoveNode(node) - delete(t.nodes, node) - t.removedNodesTimeCache.Set(node, struct{}{}, cache.DefaultExpiration) - - if perf.Tier == TierMain { - // if we've removed a main set node we should replace it - mc, uc = t.UpdateMainTierWithTopN() - // if we weren't able to fill the main set, pick the best from the unknown set - if t.mainSet.Size() < t.cfg.MaxMainTierSize { - uc = uc + t.MoveBestUnknownToMain() - } - } - return -} - -func (t *TieredHashing) recordCorrectness(perf *NodePerf, success bool) { - if success { - perf.CorrectnessDigest.Append(1) - } else { - perf.CorrectnessDigest.Append(0) - } - perf.NCorrectnessDigest++ - if perf.NCorrectnessDigest > float64(t.cfg.CorrectnessWindowSize) { - perf.NCorrectnessDigest = float64(t.cfg.CorrectnessWindowSize) - } -} diff --git a/tieredhashing/tiered_hashing_test.go b/tieredhashing/tiered_hashing_test.go deleted file mode 100644 index 0d21bc3..0000000 --- a/tieredhashing/tiered_hashing_test.go +++ /dev/null @@ -1,560 +0,0 @@ -package tieredhashing - -import ( - "fmt" - "net/http" - "sort" - "testing" - - "github.com/asecurityteam/rolling" - - "github.com/stretchr/testify/require" -) - -func TestRecordSuccess(t *testing.T) { - window := 3 - - th := NewTieredHashingHarness(WithLatencyWindowSize(window), WithFailureDebounce(0)) - // main node - unknownNode := th.genAndAddAll(t, 1)[0] - th.assertSize(t, 0, 1) - - // cache hits - th.recordCacheHitAndAssertSet(t, unknownNode, 200, 0, 1, string(TierUnknown)) - require.EqualValues(t, 1, th.h.nodes[unknownNode].NLatencyDigest) - require.EqualValues(t, 200, th.h.nodes[unknownNode].LatencyDigest.Reduce(rolling.Sum)) - - th.h.RecordSuccess(unknownNode, ResponseMetrics{TTFBMs: 150}) - require.EqualValues(t, 2, th.h.nodes[unknownNode].NLatencyDigest) - require.EqualValues(t, 350, th.h.nodes[unknownNode].LatencyDigest.Reduce(rolling.Sum)) - - th.h.RecordSuccess(unknownNode, ResponseMetrics{TTFBMs: 50}) - require.EqualValues(t, 3, th.h.nodes[unknownNode].NLatencyDigest) - require.EqualValues(t, 400, th.h.nodes[unknownNode].LatencyDigest.Reduce(rolling.Sum)) - - // windowing - th.h.RecordSuccess(unknownNode, ResponseMetrics{TTFBMs: 60}) - require.EqualValues(t, 4, th.h.nodes[unknownNode].NLatencyDigest) - require.EqualValues(t, 260, th.h.nodes[unknownNode].LatencyDigest.Reduce(rolling.Sum)) - - // node gets removed for unacceptable latency - th.h.RecordSuccess(unknownNode, ResponseMetrics{TTFBMs: 600}) - th.assertSize(t, 0, 1) - - // node does not get removed for unacceptable latency if not enough observations - unknownNode = th.genAndAddAll(t, 1)[0] - th.assertSize(t, 0, 2) - th.h.RecordSuccess(unknownNode, ResponseMetrics{TTFBMs: 6000}) - - th.assertSize(t, 0, 2) - - th.h.RecordSuccess(unknownNode, ResponseMetrics{TTFBMs: 6000}) -} - -func (th *TieredHashingHarness) recordCacheHitAndAssertSet(t *testing.T, node string, ttfbMS float64, mc, uc int, tier string) { - prevSuccess := th.nSuccess(node) - th.h.RecordSuccess(node, ResponseMetrics{TTFBMs: ttfbMS}) - require.EqualValues(t, prevSuccess+1, th.nSuccess(node)) - th.assertSize(t, mc, uc) - require.EqualValues(t, th.h.nodes[node].Tier, tier) -} - -func TestRecordFailure(t *testing.T) { - window := 3 - - th := NewTieredHashingHarness(WithCorrectnessWindowSize(window), WithFailureDebounce(0), WithMaxMainTierSize(1), WithLatencyWindowSize(2)) - // unknown node - unknownNode := th.genAndAddAll(t, 1)[0] - th.assertSize(t, 0, 1) - - // 502 status code no change - require.Nil(t, th.h.RecordFailure(unknownNode, ResponseMetrics{ResponseCode: http.StatusBadGateway})) - require.Nil(t, th.h.RecordFailure(unknownNode, ResponseMetrics{ResponseCode: http.StatusGatewayTimeout})) - require.Nil(t, th.h.RecordFailure(unknownNode, ResponseMetrics{ConnFailure: true})) - require.EqualValues(t, 1, th.h.nodes[unknownNode].connFailures) - - require.Nil(t, th.h.RecordFailure(unknownNode, ResponseMetrics{NetworkError: true})) - require.EqualValues(t, 1, th.h.nodes[unknownNode].networkErrors) - - // node is evicted as we have enough observations - rm := th.h.RecordFailure(unknownNode, ResponseMetrics{NetworkError: true}) - require.NotNil(t, rm) - require.EqualValues(t, TierUnknown, rm.Tier) - require.EqualValues(t, unknownNode, rm.Node) - - // when main node is removed, it is replaced - nodes := th.genAndAddAll(t, 2) - mn := nodes[0] - node2 := nodes[1] - th.assertSize(t, 0, 2) - th.h.RecordSuccess(mn, ResponseMetrics{TTFBMs: 100}) - th.h.RecordSuccess(mn, ResponseMetrics{TTFBMs: 150}) - - th.h.RecordSuccess(node2, ResponseMetrics{TTFBMs: 100}) - th.h.RecordSuccess(node2, ResponseMetrics{TTFBMs: 150}) - th.h.RecordFailure(mn, ResponseMetrics{NetworkError: true}) - th.h.RecordFailure(mn, ResponseMetrics{NetworkError: true}) - th.h.RecordFailure(mn, ResponseMetrics{NetworkError: true}) - th.assertSize(t, 0, 1) -} - -func TestMoveBestUnknownToMain(t *testing.T) { - th := NewTieredHashingHarness() - require.Zero(t, th.h.MoveBestUnknownToMain()) - - nodes := th.genAndAddAll(t, 2) - - th.assertSize(t, 0, 2) - th.h.RecordSuccess(nodes[0], ResponseMetrics{TTFBMs: 100}) - th.h.RecordSuccess(nodes[1], ResponseMetrics{TTFBMs: 50}) - - require.EqualValues(t, 1, th.h.MoveBestUnknownToMain()) - th.assertSize(t, 1, 1) - - th.h.nodes[nodes[1]].Tier = TierMain - th.h.nodes[nodes[0]].Tier = TierUnknown -} - -func TestNodeNotRemovedWithVar(t *testing.T) { - window := 2 - th := NewTieredHashingHarness(WithCorrectnessWindowSize(window), WithFailureDebounce(0), WithNoRemove(true)) - // unknown node - unknownNode := th.genAndAddAll(t, 1)[0] - th.assertSize(t, 0, 1) - - for i := 0; i < 1000; i++ { - require.Nil(t, th.h.RecordFailure(unknownNode, ResponseMetrics{NetworkError: true})) - } - th.assertSize(t, 0, 1) -} - -func TestNodeEvictionWithWindowing(t *testing.T) { - window := 4 - - th := NewTieredHashingHarness(WithCorrectnessWindowSize(window), WithFailureDebounce(0), WithCorrectnessPct(80)) - // main node - unknownNode := th.genAndAddAll(t, 1)[0] - th.assertSize(t, 0, 1) - - th.h.mainSet.AddNode(unknownNode) - th.h.unknownSet.RemoveNode(unknownNode) - th.h.nodes[unknownNode].Tier = TierMain - - // record success - th.h.RecordSuccess(unknownNode, ResponseMetrics{}) - th.h.RecordSuccess(unknownNode, ResponseMetrics{}) - th.h.RecordSuccess(unknownNode, ResponseMetrics{}) - th.h.RecordSuccess(unknownNode, ResponseMetrics{}) - - // evicted as pct < 80 because of windowing - rm := th.h.RecordFailure(unknownNode, ResponseMetrics{NetworkError: true}) - require.NotNil(t, rm) - require.EqualValues(t, TierMain, rm.Tier) - require.EqualValues(t, unknownNode, rm.Node) -} - -func TestGetNodes(t *testing.T) { - th := NewTieredHashingHarness(WithAlwaysMainFirst()) - - assertCountF := func(t *testing.T, resp []string, mc, uc int) { - var countMain int - var countUnknown int - for _, n := range resp { - if th.h.nodes[n].Tier == TierMain { - countMain++ - } else { - countUnknown++ - } - } - require.EqualValues(t, mc, countMain) - require.EqualValues(t, uc, countUnknown) - } - - // empty - nds := th.h.GetNodes(TierMain, "test", 1) - require.Empty(t, nds) - - // has 3 unknown, 0 main - unknownNodes := th.genAndAddAll(t, 3) - th.assertSize(t, 0, 3) - resp := th.h.GetNodes(TierUnknown, "test", 100) - require.Len(t, resp, 3) - assertCountF(t, resp, 0, 3) - - // has 2 main, 3 unknown - mainNodes := th.genAndAddAll(t, 2) - for _, n := range mainNodes { - th.h.nodes[n].Tier = TierMain - th.h.mainSet = th.h.mainSet.AddNode(n) - th.h.unknownSet = th.h.unknownSet.RemoveNode(n) - } - - th.assertSize(t, 2, 3) - resp = th.h.GetNodes(TierMain, "test", 100) - require.Len(t, resp, 2) - assertCountF(t, resp, 2, 0) - - assertGetAndCountF := func(t *testing.T, mainS int, unknownS int, n int, mc, uc int) { - th.assertSize(t, mainS, unknownS) - resp = th.h.GetNodes(TierMain, "test", n) - require.Len(t, resp, mc) - assertCountF(t, resp, mc, 0) - resp = th.h.GetNodes(TierUnknown, "test", n) - require.Len(t, resp, uc) - assertCountF(t, resp, 0, uc) - } - - // has both main - assertGetAndCountF(t, 2, 3, 2, 2, 2) - - th.h.removeFailedNode(mainNodes[0]) - assertGetAndCountF(t, 1, 3, 10, 1, 3) - - // has 1 main, 1 unknown - assertGetAndCountF(t, 1, 3, 2, 1, 2) - - // has 1 main, 0 unknown - assertGetAndCountF(t, 1, 3, 1, 1, 1) - - // has 1 main, 2 unknown - th.h.removeFailedNode(unknownNodes[0]) - assertGetAndCountF(t, 1, 2, 10, 1, 2) - - // has 0 main, 1 unknown - th.h.removeFailedNode(mainNodes[1]) - assertGetAndCountF(t, 0, 2, 1, 0, 1) - - // has 0 main, 0 unknown - th.h.removeFailedNode(unknownNodes[1]) - th.h.removeFailedNode(unknownNodes[2]) - assertGetAndCountF(t, 0, 0, 1, 0, 0) -} - -func TestConsistentHashing(t *testing.T) { - th := NewTieredHashingHarness(WithAlwaysMainFirst()) - - th.genAndAddAll(t, 10) - th.assertSize(t, 0, 10) - resp1 := th.h.GetNodes(TierUnknown, "test", 3) - require.Len(t, resp1, 3) - - resp2 := th.h.GetNodes(TierMain, "test", 2) - require.Len(t, resp2, 0) -} - -func TestRecordCorrectness(t *testing.T) { - window := 3 - th := NewTieredHashingHarness(WithCorrectnessWindowSize(window)) - perf := &NodePerf{ - CorrectnessDigest: rolling.NewPointPolicy(rolling.NewWindow(int(window))), - } - th.h.recordCorrectness(perf, true) - require.EqualValues(t, 1, perf.NCorrectnessDigest) - require.EqualValues(t, 1, perf.CorrectnessDigest.Reduce(rolling.Sum)) - - th.h.recordCorrectness(perf, true) - require.EqualValues(t, 2, perf.CorrectnessDigest.Reduce(rolling.Sum)) - require.EqualValues(t, 2, perf.NCorrectnessDigest) - - th.h.recordCorrectness(perf, false) - require.EqualValues(t, 3, perf.NCorrectnessDigest) - require.EqualValues(t, 2, perf.CorrectnessDigest.Reduce(rolling.Sum)) - - th.h.recordCorrectness(perf, false) - require.EqualValues(t, 3, perf.NCorrectnessDigest) - require.EqualValues(t, 1, perf.CorrectnessDigest.Reduce(rolling.Sum)) -} - -func (th *TieredHashingHarness) updateTiersAndAsert(t *testing.T, mcs, ucs, mains, unknowns int, isInitDone bool, mainNodes []string) { - mc, uc := th.h.UpdateMainTierWithTopN() - require.EqualValues(t, mcs, mc) - require.EqualValues(t, ucs, uc) - th.assertSize(t, mains, unknowns) - require.EqualValues(t, isInitDone, th.h.IsInitDone()) - - var mnodes []string - for n, perf := range th.h.nodes { - perf := perf - if perf.Tier == TierMain { - mnodes = append(mnodes, n) - } - } - - sort.Slice(mainNodes, func(i, j int) bool { - return mainNodes[i] < mainNodes[j] - }) - - sort.Slice(mnodes, func(i, j int) bool { - return mnodes[i] < mnodes[j] - }) - - require.EqualValues(t, mainNodes, mnodes) -} - -func TestUpdateMainTierWithTopN(t *testing.T) { - t.Skip("we probably dont need this will we turn on tiered hashing") - windowSize := 2 - th := NewTieredHashingHarness(WithLatencyWindowSize(windowSize), WithMaxMainTierSize(2)) - - mc, uc := th.h.UpdateMainTierWithTopN() - require.Zero(t, mc) - require.Zero(t, uc) - - // main node - nodes := th.genAndAddAll(t, 5) - th.assertSize(t, 0, 5) - - th.updateTiersAndAsert(t, 0, 0, 0, 5, false, nil) - - // Record 1 observation for a node -> no change - th.h.RecordSuccess(nodes[0], ResponseMetrics{TTFBMs: 100}) - th.updateTiersAndAsert(t, 0, 0, 0, 5, false, nil) - - // record 1 more observation for the same node -> no change as not enough nodes for bulk update - th.h.RecordSuccess(nodes[0], ResponseMetrics{TTFBMs: 90}) - th.updateTiersAndAsert(t, 0, 0, 0, 5, false, nil) - - // record 2 observations for second node -> change as we now have enough - th.h.RecordSuccess(nodes[1], ResponseMetrics{TTFBMs: 500}) - th.updateTiersAndAsert(t, 0, 0, 0, 5, false, nil) - - th.h.RecordSuccess(nodes[1], ResponseMetrics{TTFBMs: 90}) - th.updateTiersAndAsert(t, 0, 2, 2, 3, true, []string{nodes[0], nodes[1]}) - - // main node gets replaced with unknown node - th.h.RecordSuccess(nodes[2], ResponseMetrics{TTFBMs: 3}) - th.h.RecordSuccess(nodes[2], ResponseMetrics{TTFBMs: 5}) - - th.updateTiersAndAsert(t, 1, 1, 2, 3, true, []string{nodes[0], nodes[2]}) - - // say have less than N eligible nodes - th.h.removeFailedNode(nodes[0]) - th.h.removeFailedNode(nodes[1]) - th.h.removeFailedNode(nodes[2]) - th.updateTiersAndAsert(t, 0, 0, 0, 2, true, nil) - - // update works even with 1 node - th.h.RecordSuccess(nodes[3], ResponseMetrics{TTFBMs: 3}) - th.h.RecordSuccess(nodes[3], ResponseMetrics{TTFBMs: 5}) - th.updateTiersAndAsert(t, 0, 1, 1, 1, true, []string{nodes[3]}) - - th.h.removeFailedNode(nodes[3]) - th.updateTiersAndAsert(t, 0, 0, 0, 1, true, nil) - th.h.RecordSuccess(nodes[4], ResponseMetrics{TTFBMs: 3}) - th.h.RecordSuccess(nodes[4], ResponseMetrics{TTFBMs: 5}) - th.updateTiersAndAsert(t, 0, 1, 1, 0, true, []string{nodes[4]}) - th.h.removeFailedNode(nodes[4]) - th.updateTiersAndAsert(t, 0, 0, 0, 0, true, nil) -} - -func TestIsCorrectnessPolicyEligible(t *testing.T) { - window := 10 - - tcs := map[string]struct { - perf *NodePerf - correct bool - pct float64 - initF func(perf *NodePerf) - }{ - "no observations": { - perf: &NodePerf{}, - correct: true, - }, - "no success but not enough observations for failure": { - initF: func(perf *NodePerf) { - for i := 0; i < window-1; i++ { - perf.CorrectnessDigest.Append(0) - perf.NCorrectnessDigest++ - } - }, - perf: &NodePerf{ - CorrectnessDigest: rolling.NewPointPolicy(rolling.NewWindow(int(window))), - }, - correct: true, - }, - "some success but fail as enough observations": { - initF: func(perf *NodePerf) { - perf.CorrectnessDigest.Append(1) - perf.NCorrectnessDigest++ - perf.CorrectnessDigest.Append(1) - perf.NCorrectnessDigest++ - - for i := 0; i < int(window)-2; i++ { - perf.CorrectnessDigest.Append(0) - perf.NCorrectnessDigest++ - } - - }, - perf: &NodePerf{ - CorrectnessDigest: rolling.NewPointPolicy(rolling.NewWindow(int(window))), - }, - correct: false, - pct: 20, - }, - "some success but not enough observations": { - initF: func(perf *NodePerf) { - for i := 0; i < int(window)-1; i++ { - perf.CorrectnessDigest.Append(1) - perf.NCorrectnessDigest++ - } - - }, - perf: &NodePerf{ - CorrectnessDigest: rolling.NewPointPolicy(rolling.NewWindow(int(window))), - }, - correct: true, - pct: 0, - }, - "rolling window": { - initF: func(perf *NodePerf) { - // add window success - for i := 0; i < int(window); i++ { - perf.CorrectnessDigest.Append(1) - perf.NCorrectnessDigest++ - } - - // add 2 failures - for i := 0; i < 2; i++ { - perf.CorrectnessDigest.Append(0) - perf.NCorrectnessDigest++ - } - - }, - perf: &NodePerf{ - CorrectnessDigest: rolling.NewPointPolicy(rolling.NewWindow(int(window))), - }, - correct: true, - pct: 80, - }, - } - - for name, tc := range tcs { - t.Run(name, func(t *testing.T) { - th := NewTieredHashingHarness(WithCorrectnessWindowSize(window)) - if tc.initF != nil { - tc.initF(tc.perf) - } - - perf := tc.perf - pct, ok := th.h.isCorrectnessPolicyEligible(perf) - require.EqualValues(t, tc.correct, ok) - require.EqualValues(t, tc.pct, pct) - }) - } -} - -func TestAddOrchestratorNodes(t *testing.T) { - th := NewTieredHashingHarness() - - nodes := th.genAndAddAll(t, 10) - th.assertSize(t, 0, 10) - - nodes2 := th.genNodes(t, 10) - th.addNewNodesAll(t, nodes2) - th.assertSize(t, 0, 20) - - th.addAndAssert(t, append(nodes[:3], nodes2[:3]...), 0, 0, 0, 0, 20) - - th.h.removeFailedNode(nodes[0]) - th.assertSize(t, 0, 19) - - // removed node gets added back as we are not full - th.addAndAssert(t, append(nodes[:3], nodes2[:3]...), 1, 1, 1, 0, 20) -} - -func TestAddOrchestratorNodesMax(t *testing.T) { - th := NewTieredHashingHarness(WithMaxPoolSize(10)) - - // empty -> 10 get added - nodes := th.genNodes(t, 30) - a, _, _ := th.h.AddOrchestratorNodes(nodes) - require.EqualValues(t, 10, a) - th.assertSize(t, 0, 10) - - // nothing gets added as we are full - nodes2 := th.genNodes(t, 30) - a, _, _ = th.h.AddOrchestratorNodes(append(nodes, nodes2...)) - require.EqualValues(t, 0, a) - th.assertSize(t, 0, 10) - - // remove 2 nodes -> - th.h.removeFailedNode(nodes[0]) - th.assertSize(t, 0, 9) - th.h.removeFailedNode(nodes[1]) - th.assertSize(t, 0, 8) - - // 2 get added now - a, _, _ = th.h.AddOrchestratorNodes(append(nodes, nodes2...)) - require.EqualValues(t, 2, a) - th.assertSize(t, 0, 10) - - th.h.removeFailedNode(nodes[2]) - th.assertSize(t, 0, 9) - - // removed node does not get added back as we are already full without it - a, ar, back := th.h.AddOrchestratorNodes(append(nodes, "newnode")) - require.EqualValues(t, 1, a) - require.EqualValues(t, 3, ar) - th.assertSize(t, 0, 10) - require.EqualValues(t, 0, back) -} - -type TieredHashingHarness struct { - count int - h *TieredHashing -} - -func NewTieredHashingHarness(opts ...Option) *TieredHashingHarness { - return &TieredHashingHarness{ - h: New(opts...), - } -} - -func (th *TieredHashingHarness) genAndAddAll(t *testing.T, n int) []string { - nodes := th.genNodes(t, n) - th.addNewNodesAll(t, nodes) - return nodes -} - -func (th *TieredHashingHarness) genNodes(t *testing.T, n int) []string { - var nodes []string - // generate n random strings - for i := 0; i < n; i++ { - nodes = append(nodes, fmt.Sprintf("%d-test", th.count+i)) - } - th.count = th.count + n - return nodes -} - -func (th *TieredHashingHarness) addNewNodesAll(t *testing.T, nodes []string) { - var old []string - - for key := range th.h.nodes { - old = append(old, key) - } - - added, already, _ := th.h.AddOrchestratorNodes(append(nodes, old...)) - require.Zero(t, already) - require.EqualValues(t, len(nodes), added) -} - -func (th *TieredHashingHarness) addAndAssert(t *testing.T, nodes []string, added, already, ab int, main, unknown int) { - a, ar, addedBack := th.h.AddOrchestratorNodes(nodes) - require.EqualValues(t, added, a) - - require.EqualValues(t, already, ar) - th.assertSize(t, main, unknown) - - require.EqualValues(t, ab, addedBack) -} - -func (th *TieredHashingHarness) assertSize(t *testing.T, main int, unknown int) { - mt := th.h.GetPoolMetrics() - require.EqualValues(t, unknown, mt.Unknown) - require.EqualValues(t, main, mt.Main) - require.EqualValues(t, main+unknown, mt.Total) -} - -func (th *TieredHashingHarness) nSuccess(node string) int { - return int(th.h.nodes[node].CorrectnessDigest.Reduce(rolling.Sum)) -} diff --git a/tieredhashing/types.go b/tieredhashing/types.go deleted file mode 100644 index 1e225cd..0000000 --- a/tieredhashing/types.go +++ /dev/null @@ -1,12 +0,0 @@ -package tieredhashing - -type ResponseMetrics struct { - ConnFailure bool - NetworkError bool - ResponseCode int - - Success bool - - TTFBMs float64 - SpeedPerMs float64 -}