diff --git a/docs/async-loading.png b/docs/async-loading.png deleted file mode 100644 index 4b856ce2..00000000 Binary files a/docs/async-loading.png and /dev/null differ diff --git a/docs/async-loading.puml b/docs/async-loading.puml deleted file mode 100644 index 5517eaf8..00000000 --- a/docs/async-loading.puml +++ /dev/null @@ -1,75 +0,0 @@ -@startuml async loading -participant IPLD -participant "Intercepted Loader" as ILoader -participant RequestManager -participant AsyncLoader -participant LoadAttemptQueue -participant ResponseCache -participant Loader -participant Storer -IPLD -> ILoader: Load Link -activate ILoader -ILoader -> AsyncLoader: Load Link Asynchronously -activate AsyncLoader -ILoader <-- AsyncLoader: Channel For future response -... transfer to internal process ... -AsyncLoader -> LoadAttemptQueue: Try Loading This Link -deactivate AsyncLoader - -LoadAttemptQueue -> ResponseCache: Try Loading This Link -alt response cache has block -ResponseCache -> Storer: Store the block for later -ResponseCache -> ResponseCache: Remove the block from cache -LoadAttemptQueue <-- ResponseCache: "Here's the block" -note over LoadAttemptQueue: Response = Block -else response cache told block is missing by remote peer -LoadAttemptQueue <-- ResponseCache: "We're missing this link" -note over LoadAttemptQueue: Response = Error Missing Link -else local store has block -LoadAttemptQueue <-- ResponseCache: "I Don't Have it!" -LoadAttemptQueue -> Loader: Try Loading This Link From Local Cache -LoadAttemptQueue <-- Loader: "Here's the block" -note over LoadAttemptQueue: Response = Block -else no block or known missing link yet -LoadAttemptQueue <-- ResponseCache: "I Don't Have it!" -LoadAttemptQueue -> Loader: Try Loading This Link From Local Cache -LoadAttemptQueue <-- Loader: "I Don't Have it!" -LoadAttemptQueue -> LoadAttemptQueue: Store load request to try later -end -loop 0 or more times till I have a response for the link -... -opt new responses comes in -RequestManager -> AsyncLoader: New Responses Present -activate AsyncLoader -AsyncLoader -> ResponseCache: New Responses Present -... transfer to internal process ... -AsyncLoader -> LoadAttemptQueue: Try Loading Again -deactivate AsyncLoader -LoadAttemptQueue -> ResponseCache: Try Loading Stored Links -alt response cache now has block -ResponseCache -> Storer: Store the block for later -ResponseCache -> ResponseCache: Remove the block from cache -LoadAttemptQueue <-- ResponseCache: "Here's the block" -note over LoadAttemptQueue: Response = Block -else response cache now knows link is missing by remote peer -LoadAttemptQueue <-- ResponseCache: "We're missing this link" -note over LoadAttemptQueue: Response = Error Missing Link -else still no response -LoadAttemptQueue <-- ResponseCache: "I don't have it" -LoadAttemptQueue -> LoadAttemptQueue: Store load request to try later -end -end -opt no more responses -RequestManager -> AsyncLoader: No more responses -activate AsyncLoader -... transfer to internal process ... -AsyncLoader -> LoadAttemptQueue: Cancel attempts to load -note over LoadAttemptQueue: Response = Error Request Finished -deactivate AsyncLoader -end -end -ILoader <-- LoadAttemptQueue: Response Sent Over Channel -IPLD <-- ILoader : "Here's the stream of block\n data or an error" -deactivate ILoader - -@enduml \ No newline at end of file diff --git a/docs/processes.png b/docs/processes.png index 964dffd3..ddd3289c 100644 Binary files a/docs/processes.png and b/docs/processes.png differ diff --git a/docs/processes.puml b/docs/processes.puml index 953a80cd..c44775c7 100644 --- a/docs/processes.puml +++ b/docs/processes.puml @@ -12,16 +12,17 @@ partition "Top Level Interface" { if (operation type) then (outgoing request or incoming response) partition "Graphsync Requestor Implementation" { :RequestManager; -if (operation type) then (incoming response) -partition "Verifying Queries" { +partition "Executing Requests" { +:TaskQueue; fork -:ipld.Traverse; +:Executor; fork again -:ipld.Traverse; +:Executor; fork again -:ipld.Traverse; +:Executor; end fork } +if (operation type) then (verified responses) partition "Collecting Responses" { fork :Response Collector; @@ -33,7 +34,7 @@ end fork } :Responses returned to client; stop -else (outgoing request) +else (request messages) :Send Request To Network; endif } @@ -41,13 +42,13 @@ else (incoming request) partition "Graphsync Responder Implementation" { :ResponseManager; partition "Performing Queries" { -:PeerTaskQueue; +:TaskQueue; fork -:ipld.Traverse; +:QueryExecutor; fork again -:ipld.Traverse; +:QueryExecutor; fork again -:ipld.Traverse; +:QueryExecutor; end fork } } diff --git a/docs/request-execution.png b/docs/request-execution.png new file mode 100644 index 00000000..8487c510 Binary files /dev/null and b/docs/request-execution.png differ diff --git a/docs/request-execution.puml b/docs/request-execution.puml new file mode 100644 index 00000000..b715436a --- /dev/null +++ b/docs/request-execution.puml @@ -0,0 +1,102 @@ +@startuml Request Execution +participant "GraphSync\nTop Level\nInterface" as TLI +participant RequestManager +participant TaskQueue +participant RequestExecutor as RE +participant ReconciledLoader +participant TraversalRecord +participant Verifier +participant LocalStorage +participant Traverser +participant Network + +== Initialization == + +TLI -> RequestManager ** : Setup +TLI -> RE ** : Setup +TLI -> TaskQueue ** : Setup + +== Executing A Request == + +par +note over TLI : Request Initiation +TLI -> RequestManager : New Request +RequestManager -> RequestManager : Create Request Context +RequestManager -> TaskQueue : Push Request +else +note over RE: Request Execution +TaskQueue -> RE : Next Request\nTo Process +RE -> RequestManager : Initiate request execution +RequestManager -> Traverser ** : Create to manage selector traversal +RequestManager -> ReconciledLoader ** : create to manage +RequestManager -> RE : Traverser + ReconciledLoader +note over RE: Local loading phase +loop until traversal complete, request context cancelled, or missing block locally +Traverser -> RE : Request to load blocks\nto perform traversal +RE -> ReconciledLoader : Load next block +ReconciledLoader -> LocalStorage : Load Block +LocalStorage --> ReconciledLoader : Block or missing +ReconciledLoader -> TraversalRecord : Record link traversal +TraversalRecord --> ReconciledLoader +ReconciledLoader --> RE : Block or missing +opt block is present +RE --> Traverser : Next block to load +end +end +RE -> Network : Send Graphsync Request +RE -> ReconciledLoader : remote online +ReconciledLoader -> Verifier ** : Create new from traversal record +ReconciledLoader -> RE +note over RE: Remote loading phase +loop until traversal complete, request context cancelled, or missing block locally +Traverser -> RE : Request to load blocks\nto perform traversal +RE -> ReconciledLoader : Load next block +alt on missing path for remote +ReconciledLoader -> LocalStorage : Load Block +LocalStorage --> ReconciledLoader : Block or missing +else +loop until block loaded, missing, or error +opt new remote responses + alt verification not done + ReconciledLoader -> Verifier : verify next response + alt success + Verifier --> ReconciledLoader : verified + ReconciledLoader -> ReconciledLoader : wait for more responses + else failure + Verifier --> ReconciledLoader : error + end + else verification done + alt next response matches current block load + + alt next response contains a block + ReconciledLoader -> LocalStorage : store remote block + LocalStorage --> ReconciledLoader + ReconciledLoader -> ReconciledLoader : block laoded from remote + else next response does not contain block + opt next response is missing + ReconciledLoader -> ReconciledLoader : record missing path + end + ReconciledLoader -> LocalStorage : load block + LocalStorage --> ReconciledLoader : block or missing + end + else next response doesn not match + ReconciledLoader -> ReconciledLoader : error + end + end +end +opt remote goes offline +ReconciledLoader -> LocalStorage : load block +LocalStorage --> ReconciledLoader : block or missing +end +end +ReconciledLoader -> TraversalRecord : Record link traversal +TraversalRecord --> ReconciledLoader +ReconciledLoader --> RE : Block, missing or error +RE -> Traverser : Next block to load +end +end +else +Network -> RequestManager : New Responses +RequestManager -> ReconciledLoader : Ingest Responses +end +@enduml \ No newline at end of file diff --git a/docs/responder-sequence.puml b/docs/responder-sequence.puml index e9885d4d..65ded652 100644 --- a/docs/responder-sequence.puml +++ b/docs/responder-sequence.puml @@ -1,24 +1,20 @@ @startuml Responding To A Request participant "GraphSync\nTop Level\nInterface" as TLI participant ResponseManager -participant "Query Executor" as QW -participant PeerTaskQueue +participant "QueryExecutor" as QW +participant TaskQueue participant PeerTracker participant Traverser participant ResponseAssembler participant LinkTracker -participant ResponseBuilder -participant "Intercepted Loader" as ILoader participant Loader participant "Message Sending\nLayer" as Message == Initialization == TLI -> ResponseManager ** : Setup -ResponseManager -> QW ** : Create -activate QW -TLI -> PeerTaskQueue ** : Setup -TLI -> PeerResponseManager ** : Setup +TLI -> QW ** : Setup +TLI -> TaskQueue ** : Setup == Responding To Request == @@ -27,10 +23,8 @@ loop until shutdown note over TLI : Request Queueing Loop TLI -> ResponseManager : Process requests alt new request -ResponseManager -> PeerTaskQueue : Push Request -PeerTaskQueue -> PeerTracker ** : Create for peer\n as neccesary -PeerTaskQueue -> PeerTracker : Push Request ResponseManager -> ResponseManager : Create Request Context +ResponseManager -> TaskQueue : Push Request else cancel request ResponseManager -> ResponseManager : Cancel Request Context end @@ -38,27 +32,23 @@ end else loop until shutdown note over QW: Request Processing Loop -QW -> PeerTaskQueue : Pop Request -PeerTaskQueue -> PeerTracker : Pop Request -PeerTracker -> PeerTaskQueue : Next Request\nTo Process -PeerTaskQueue -> QW : Next Request\nTo Process +TaskQueue -> QW : Next Request\nTo Process +activate QW QW -> QW : Process incoming request hooks -QW -> ILoader ** : Create w/ Request, Peer, and Loader QW -> Traverser ** : Create to manage selector traversal loop until traversal complete or request context cancelled note over Traverser: Selector Traversal Loop -Traverser -> ILoader : Request to load blocks\nto perform traversal -ILoader -> Loader : Load blocks\nfrom local storage -Loader -> ILoader : Blocks From\nlocal storage or error -ILoader -> Traverser : Blocks to continue\n traversal or error -ILoader -> QW : Block or error to Send Back +Traverser -> QW : Request to load blocks\nto perform traversal +QW -> Loader : Load blocks\nfrom local storage +Loader -> QW : Blocks From\nlocal storage or error +QW -> Traverser : Blocks to continue\n traversal or error QW -> QW: Processing outgoing block hooks QW -> ResponseAssembler: Add outgoing responses activate ResponseAssembler ResponseAssembler -> LinkTracker ** : Create for peer if not already present ResponseAssembler -> LinkTracker : Notify block or\n error, ask whether\n block is duplicate LinkTracker -> ResponseAssembler : Whether to\n send block -ResponseAssembler -> ResponseBuilder : Aggregate Response Metadata & Block +ResponseAssembler -> ResponseAssembler : Aggregate Response Metadata & Blocks ResponseAssembler -> Message : Send aggregate response deactivate ResponseAssembler end @@ -67,7 +57,7 @@ QW -> ResponseAssembler : Request Finished activate ResponseAssembler ResponseAssembler -> LinkTracker : Query If Errors\n Were Present LinkTracker -> ResponseAssembler : True/False\n if errors present -ResponseAssembler -> ResponseBuilder : Aggregate request finishing +ResponseAssembler -> ResponseAssembler : Aggregate request finishing ResponseAssembler -> Message : Send aggregate response deactivate ResponseAssembler end diff --git a/graphsync.go b/graphsync.go index bae45ed8..94848c86 100644 --- a/graphsync.go +++ b/graphsync.go @@ -127,14 +127,29 @@ func (e RequestNotFoundErr) Error() string { } // RemoteMissingBlockErr indicates that the remote peer was missing a block -// in the selector requested. It is a non-terminal error in the error stream +// in the selector requested, and we also don't have it locally. +// It is a -terminal error in the error stream // for a request and does NOT cause a request to fail completely type RemoteMissingBlockErr struct { Link ipld.Link + Path ipld.Path } func (e RemoteMissingBlockErr) Error() string { - return fmt.Sprintf("remote peer is missing block: %s", e.Link.String()) + return fmt.Sprintf("remote peer is missing block (%s) at path %s", e.Link.String(), e.Path) +} + +// RemoteIncorrectResponseError indicates that the remote peer sent a response +// to a traversal that did not correspond with the expected next link +// in the selector traversal based on verification of data up to this point +type RemoteIncorrectResponseError struct { + LocalLink ipld.Link + RemoteLink ipld.Link + Path ipld.Path +} + +func (e RemoteIncorrectResponseError) Error() string { + return fmt.Sprintf("expected link (%s) at path %s does not match link sent by remote (%s), possible malicious responder", e.LocalLink, e.Path, e.RemoteLink) } var ( @@ -223,6 +238,8 @@ type LinkMetadataIterator func(cid.Cid, LinkAction) // LinkMetadata is used to access link metadata through an Iterator type LinkMetadata interface { + // Length returns the number of metadata entries + Length() int64 // Iterate steps over individual metadata one by one using the provided // callback Iterate(LinkMetadataIterator) diff --git a/impl/graphsync.go b/impl/graphsync.go index 0f546b24..3c9381d4 100644 --- a/impl/graphsync.go +++ b/impl/graphsync.go @@ -20,13 +20,12 @@ import ( gsnet "github.com/ipfs/go-graphsync/network" "github.com/ipfs/go-graphsync/peermanager" "github.com/ipfs/go-graphsync/peerstate" + "github.com/ipfs/go-graphsync/persistenceoptions" "github.com/ipfs/go-graphsync/requestmanager" - "github.com/ipfs/go-graphsync/requestmanager/asyncloader" "github.com/ipfs/go-graphsync/requestmanager/executor" requestorhooks "github.com/ipfs/go-graphsync/requestmanager/hooks" "github.com/ipfs/go-graphsync/responsemanager" responderhooks "github.com/ipfs/go-graphsync/responsemanager/hooks" - "github.com/ipfs/go-graphsync/responsemanager/persistenceoptions" "github.com/ipfs/go-graphsync/responsemanager/queryexecutor" "github.com/ipfs/go-graphsync/responsemanager/responseassembler" "github.com/ipfs/go-graphsync/selectorvalidator" @@ -50,7 +49,6 @@ type GraphSync struct { requestManager *requestmanager.RequestManager responseManager *responsemanager.ResponseManager queryExecutor *queryexecutor.QueryExecutor - asyncLoader *asyncloader.AsyncLoader responseQueue taskqueue.TaskQueue requestQueue taskqueue.TaskQueue requestExecutor *executor.Executor @@ -230,10 +228,9 @@ func New(parent context.Context, network gsnet.GraphSyncNetwork, } peerManager := peermanager.NewMessageManager(ctx, createMessageQueue) - asyncLoader := asyncloader.New(ctx, linkSystem) requestQueue := taskqueue.NewTaskQueue(ctx) - requestManager := requestmanager.New(ctx, asyncLoader, linkSystem, outgoingRequestHooks, incomingResponseHooks, networkErrorListeners, outgoingRequestProcessingListeners, requestQueue, network.ConnectionManager(), gsConfig.maxLinksPerOutgoingRequest) - requestExecutor := executor.NewExecutor(requestManager, incomingBlockHooks, asyncLoader.AsyncLoad) + requestManager := requestmanager.New(ctx, persistenceOptions, linkSystem, outgoingRequestHooks, incomingResponseHooks, networkErrorListeners, outgoingRequestProcessingListeners, requestQueue, network.ConnectionManager(), gsConfig.maxLinksPerOutgoingRequest) + requestExecutor := executor.NewExecutor(requestManager, incomingBlockHooks) responseAssembler := responseassembler.New(ctx, peerManager) var ptqopts []peertaskqueue.Option if gsConfig.maxInProgressIncomingRequestsPerPeer > 0 { @@ -266,7 +263,6 @@ func New(parent context.Context, network gsnet.GraphSyncNetwork, requestManager: requestManager, responseManager: responseManager, queryExecutor: queryExecutor, - asyncLoader: asyncLoader, responseQueue: responseQueue, requestQueue: requestQueue, requestExecutor: requestExecutor, @@ -339,19 +335,11 @@ func (gs *GraphSync) RegisterOutgoingRequestHook(hook graphsync.OnOutgoingReques // RegisterPersistenceOption registers an alternate loader/storer combo that can be substituted for the default func (gs *GraphSync) RegisterPersistenceOption(name string, lsys ipld.LinkSystem) error { - err := gs.asyncLoader.RegisterPersistenceOption(name, lsys) - if err != nil { - return err - } return gs.persistenceOptions.Register(name, lsys) } // UnregisterPersistenceOption unregisters an alternate loader/storer combo func (gs *GraphSync) UnregisterPersistenceOption(name string) error { - err := gs.asyncLoader.UnregisterPersistenceOption(name) - if err != nil { - return err - } return gs.persistenceOptions.Unregister(name) } diff --git a/impl/graphsync_test.go b/impl/graphsync_test.go index 921f4215..6773d263 100644 --- a/impl/graphsync_test.go +++ b/impl/graphsync_test.go @@ -7,13 +7,11 @@ import ( "fmt" "io" "io/ioutil" - "math" "os" "path/filepath" "testing" "time" - blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-blockservice" "github.com/ipfs/go-cid" "github.com/ipfs/go-datastore" @@ -45,7 +43,6 @@ import ( "github.com/ipfs/go-graphsync/cidset" "github.com/ipfs/go-graphsync/donotsendfirstblocks" "github.com/ipfs/go-graphsync/ipldutil" - gsmsg "github.com/ipfs/go-graphsync/message" gsnet "github.com/ipfs/go-graphsync/network" "github.com/ipfs/go-graphsync/requestmanager/hooks" "github.com/ipfs/go-graphsync/storeutil" @@ -69,130 +66,6 @@ var protocolsForTest = map[string]struct { "(v1.0 -> v1.0)": {[]protocol.ID{gsnet.ProtocolGraphsync_1_0_0}, []protocol.ID{gsnet.ProtocolGraphsync_1_0_0}}, } -func TestMakeRequestToNetwork(t *testing.T) { - - // create network - ctx := context.Background() - ctx, collectTracing := testutil.SetupTracing(ctx) - ctx, cancel := context.WithTimeout(ctx, 10*time.Second) - defer cancel() - td := newGsTestData(ctx, t) - r := &receiver{ - messageReceived: make(chan receivedMessage), - } - td.gsnet2.SetDelegate(r) - graphSync := td.GraphSyncHost1() - - blockChainLength := 100 - blockChain := testutil.SetupBlockChain(ctx, t, td.persistence1, 100, blockChainLength) - - requestCtx, requestCancel := context.WithCancel(ctx) - defer requestCancel() - graphSync.Request(requestCtx, td.host2.ID(), blockChain.TipLink, blockChain.Selector(), td.extension) - - var message receivedMessage - testutil.AssertReceive(ctx, t, r.messageReceived, &message, "did not receive message sent") - - sender := message.sender - require.Equal(t, td.host1.ID(), sender, "received message from wrong node") - - received := message.message - receivedRequests := received.Requests() - require.Len(t, receivedRequests, 1, "Did not add request to received message") - receivedRequest := receivedRequests[0] - receivedSpec := receivedRequest.Selector() - require.Equal(t, blockChain.Selector(), receivedSpec, "did not transmit selector spec correctly") - _, err := selector.ParseSelector(receivedSpec) - require.NoError(t, err, "did not receive parsible selector on other side") - - returnedData, found := receivedRequest.Extension(td.extensionName) - require.True(t, found) - require.Equal(t, td.extensionData, returnedData, "Failed to encode extension") - - drain(graphSync) - - tracing := collectTracing(t) - require.ElementsMatch(t, []string{ - "request(0)->newRequest(0)", - "request(0)->executeTask(0)", - "request(0)->terminateRequest(0)", - "message(0)->sendMessage(0)", - }, tracing.TracesToStrings()) - - // make sure the attributes are what we expect - requestSpans := tracing.FindSpans("request") - require.Equal(t, td.host2.ID().Pretty(), testutil.AttributeValueInTraceSpan(t, requestSpans[0], "peerID").AsString()) - require.Equal(t, blockChain.TipLink.String(), testutil.AttributeValueInTraceSpan(t, requestSpans[0], "root").AsString()) - require.Equal(t, []string{string(td.extensionName)}, testutil.AttributeValueInTraceSpan(t, requestSpans[0], "extensions").AsStringSlice()) - require.Equal(t, int64(0), testutil.AttributeValueInTraceSpan(t, requestSpans[0], "requestID").AsInt64()) -} - -func TestSendResponseToIncomingRequest(t *testing.T) { - // create network - ctx := context.Background() - ctx, cancel := context.WithTimeout(ctx, 3*time.Second) - defer cancel() - td := newGsTestData(ctx, t) - r := &receiver{ - messageReceived: make(chan receivedMessage), - } - td.gsnet1.SetDelegate(r) - - var receivedRequestData datamodel.Node - // initialize graphsync on second node to response to requests - gsnet := td.GraphSyncHost2() - gsnet.RegisterIncomingRequestHook( - func(p peer.ID, requestData graphsync.RequestData, hookActions graphsync.IncomingRequestHookActions) { - var has bool - receivedRequestData, has = requestData.Extension(td.extensionName) - require.True(t, has, "did not have expected extension") - hookActions.SendExtensionData(td.extensionResponse) - }, - ) - - blockChainLength := 100 - blockChain := testutil.SetupBlockChain(ctx, t, td.persistence2, 100, blockChainLength) - - requestID := graphsync.NewRequestID() - - builder := gsmsg.NewBuilder() - builder.AddRequest(gsmsg.NewRequest(requestID, blockChain.TipLink.(cidlink.Link).Cid, blockChain.Selector(), graphsync.Priority(math.MaxInt32), td.extension)) - message, err := builder.Build() - require.NoError(t, err) - // send request across network - err = td.gsnet1.SendMessage(ctx, td.host2.ID(), message) - require.NoError(t, err) - // read the values sent back to requestor - var received gsmsg.GraphSyncMessage - var receivedBlocks []blocks.Block - var receivedExtensions []datamodel.Node - for { - var message receivedMessage - testutil.AssertReceive(ctx, t, r.messageReceived, &message, "did not receive complete response") - - sender := message.sender - require.Equal(t, td.host2.ID(), sender, "received message from wrong node") - - received = message.message - receivedBlocks = append(receivedBlocks, received.Blocks()...) - receivedResponses := received.Responses() - receivedExtension, found := receivedResponses[0].Extension(td.extensionName) - if found { - receivedExtensions = append(receivedExtensions, receivedExtension) - } - require.Len(t, receivedResponses, 1, "Did not receive response") - require.Equal(t, requestID, receivedResponses[0].RequestID(), "Sent response for incorrect request id") - if receivedResponses[0].Status() != graphsync.PartialResponse { - break - } - } - - require.Len(t, receivedBlocks, blockChainLength, "Send incorrect number of blocks or there were duplicate blocks") - require.Equal(t, td.extensionData, receivedRequestData, "did not receive correct request extension data") - require.Len(t, receivedExtensions, 1, "should have sent extension responses but didn't") - require.Equal(t, td.extensionResponseData, receivedExtensions[0], "did not return correct extension data") -} - func TestRejectRequestsByDefault(t *testing.T) { // create network @@ -224,7 +97,7 @@ func TestRejectRequestsByDefault(t *testing.T) { "request(0)->newRequest(0)", "request(0)->executeTask(0)", "request(0)->terminateRequest(0)", - "processResponses(0)->loaderProcess(0)->cacheProcess(0)", + "processResponses(0)", "processRequests(0)->transaction(0)->execute(0)->buildMessage(0)", "message(0)->sendMessage(0)", "message(1)->sendMessage(0)", @@ -278,8 +151,8 @@ func TestGraphsyncRoundTripRequestBudgetRequestor(t *testing.T) { require.Contains(t, traceStrings, "request(0)->newRequest(0)") require.Contains(t, traceStrings, "request(0)->executeTask(0)") require.Contains(t, traceStrings, "request(0)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block // has ErrBudgetExceeded exception recorded in the right place tracing.SingleExceptionEvent(t, "request(0)->executeTask(0)", "ErrBudgetExceeded", "traversal budget exceeded", true) @@ -327,8 +200,8 @@ func TestGraphsyncRoundTripRequestBudgetResponder(t *testing.T) { require.Contains(t, traceStrings, "request(0)->newRequest(0)") require.Contains(t, traceStrings, "request(0)->executeTask(0)") require.Contains(t, traceStrings, "request(0)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block // has ContextCancelError exception recorded in the right place // the requester gets a cancel, the responder gets a ErrBudgetExceeded @@ -411,8 +284,8 @@ func TestGraphsyncRoundTrip(t *testing.T) { require.Contains(t, traceStrings, "request(0)->newRequest(0)") require.Contains(t, traceStrings, "request(0)->executeTask(0)") require.Contains(t, traceStrings, "request(0)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block processUpdateSpan := tracing.FindSpanByTraceString("response(0)") require.Equal(t, int64(0), testutil.AttributeValueInTraceSpan(t, *processUpdateSpan, "priority").AsInt64()) @@ -420,18 +293,18 @@ func TestGraphsyncRoundTrip(t *testing.T) { // each verifyBlock span should link to a cacheProcess span that stored it - cacheProcessSpans := tracing.FindSpans("cacheProcess") - cacheProcessLinks := make(map[string]int64) + processResponsesSpans := tracing.FindSpans("processResponses") + processResponsesLinks := make(map[string]int64) verifyBlockSpans := tracing.FindSpans("verifyBlock") for _, verifyBlockSpan := range verifyBlockSpans { require.Len(t, verifyBlockSpan.Links, 1, "verifyBlock span should have one link") found := false - for _, cacheProcessSpan := range cacheProcessSpans { - sid := cacheProcessSpan.SpanContext.SpanID().String() + for _, prcessResponseSpan := range processResponsesSpans { + sid := prcessResponseSpan.SpanContext.SpanID().String() if verifyBlockSpan.Links[0].SpanContext.SpanID().String() == sid { found = true - cacheProcessLinks[sid] = cacheProcessLinks[sid] + 1 + processResponsesLinks[sid] = processResponsesLinks[sid] + 1 break } } @@ -440,9 +313,9 @@ func TestGraphsyncRoundTrip(t *testing.T) { // each cacheProcess span should be linked to one verifyBlock span per block it stored - for _, cacheProcessSpan := range cacheProcessSpans { - blockCount := testutil.AttributeValueInTraceSpan(t, cacheProcessSpan, "blockCount").AsInt64() - require.Equal(t, cacheProcessLinks[cacheProcessSpan.SpanContext.SpanID().String()], blockCount, "cacheProcess span should be linked to one verifyBlock span per block it processed") + for _, processResponseSpan := range processResponsesSpans { + blockCount := testutil.AttributeValueInTraceSpan(t, processResponseSpan, "blockCount").AsInt64() + require.Equal(t, processResponsesLinks[processResponseSpan.SpanContext.SpanID().String()], blockCount, "cacheProcess span should be linked to one verifyBlock span per block it processed") } }) } @@ -487,7 +360,7 @@ func TestGraphsyncRoundTripPartial(t *testing.T) { for err := range errChan { // verify the error is received for leaf beta node being missing - require.EqualError(t, err, fmt.Sprintf("remote peer is missing block: %s", tree.LeafBetaLnk.String())) + require.EqualError(t, err, fmt.Sprintf("remote peer is missing block (%s) at path linkedList/2", tree.LeafBetaLnk.String())) } require.Equal(t, tree.LeafAlphaBlock.RawData(), td.blockStore1[tree.LeafAlphaLnk]) require.Equal(t, tree.MiddleListBlock.RawData(), td.blockStore1[tree.MiddleListNodeLnk]) @@ -510,8 +383,8 @@ func TestGraphsyncRoundTripPartial(t *testing.T) { require.Contains(t, traceStrings, "request(0)->newRequest(0)") require.Contains(t, traceStrings, "request(0)->executeTask(0)") require.Contains(t, traceStrings, "request(0)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block } func TestGraphsyncRoundTripIgnoreCids(t *testing.T) { @@ -744,8 +617,8 @@ func TestPauseResume(t *testing.T) { require.Contains(t, traceStrings, "request(0)->newRequest(0)") require.Contains(t, traceStrings, "request(0)->executeTask(0)") require.Contains(t, traceStrings, "request(0)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block // pause recorded tracing.SingleExceptionEvent(t, "response(0)->executeTask(0)", "github.com/ipfs/go-graphsync/responsemanager/hooks.ErrPaused", hooks.ErrPaused{}.Error(), false) @@ -827,8 +700,8 @@ func TestPauseResumeRequest(t *testing.T) { require.Contains(t, traceStrings, "request(0)->executeTask(0)") require.Contains(t, traceStrings, "request(0)->executeTask(1)") require.Contains(t, traceStrings, "request(0)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block // has ErrPaused exception recorded in the right place tracing.SingleExceptionEvent(t, "request(0)->executeTask(0)", "ErrPaused", hooks.ErrPaused{}.Error(), false) @@ -1115,8 +988,8 @@ func TestNetworkDisconnect(t *testing.T) { require.Contains(t, traceStrings, "request(0)->newRequest(0)") require.Contains(t, traceStrings, "request(0)->executeTask(0)") require.Contains(t, traceStrings, "request(0)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block // has ContextCancelError exception recorded in the right place tracing.SingleExceptionEvent(t, "request(0)->executeTask(0)", "ContextCancelError", ipldutil.ContextCancelError{}.Error(), false) @@ -1256,8 +1129,8 @@ func TestGraphsyncRoundTripAlternatePersistenceAndNodes(t *testing.T) { require.Contains(t, traceStrings, "request(1)->newRequest(0)") require.Contains(t, traceStrings, "request(1)->executeTask(0)") require.Contains(t, traceStrings, "request(1)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(1)->verifyBlock(0)") // should have one of these per block (TODO: why request(1) and not (0)?) + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(1)->verifyBlock(0)") // should have one of these per block (TODO: why request(1) and not (0)?) // TODO(rvagg): this is randomly either a SkipMe or a ipldutil.ContextCancelError; confirm this is sane // tracing.SingleExceptionEvent(t, "request(0)->newRequest(0)","request(0)->executeTask(0)", "SkipMe", traversal.SkipMe{}.Error(), true) @@ -1345,8 +1218,8 @@ func TestGraphsyncRoundTripMultipleAlternatePersistence(t *testing.T) { require.Contains(t, traceStrings, "request(1)->newRequest(0)") require.Contains(t, traceStrings, "request(1)->executeTask(0)") require.Contains(t, traceStrings, "request(1)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block } // TestRoundTripLargeBlocksSlowNetwork test verifies graphsync continues to work @@ -1600,8 +1473,8 @@ func TestUnixFSFetch(t *testing.T) { require.Contains(t, traceStrings, "request(0)->newRequest(0)") require.Contains(t, traceStrings, "request(0)->executeTask(0)") require.Contains(t, traceStrings, "request(0)->terminateRequest(0)") - require.Contains(t, traceStrings, "processResponses(0)->loaderProcess(0)->cacheProcess(0)") // should have one of these per response - require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block + require.Contains(t, traceStrings, "processResponses(0)") // should have one of these per response + require.Contains(t, traceStrings, "request(0)->verifyBlock(0)") // should have one of these per block } func TestGraphsyncBlockListeners(t *testing.T) { @@ -1841,43 +1714,6 @@ func (td *gsTestData) GraphSyncHost2(options ...Option) graphsync.GraphExchange return New(td.ctx, td.gsnet2, td.persistence2, options...) } -type receivedMessage struct { - message gsmsg.GraphSyncMessage - sender peer.ID -} - -// Receiver is an interface for receiving messages from the GraphSyncNetwork. -type receiver struct { - messageReceived chan receivedMessage -} - -func (r *receiver) ReceiveMessage( - ctx context.Context, - sender peer.ID, - incoming gsmsg.GraphSyncMessage) { - - select { - case <-ctx.Done(): - case r.messageReceived <- receivedMessage{incoming, sender}: - } -} - -func (r *receiver) ReceiveError(_ peer.ID, err error) { - fmt.Println("got receive err") -} - -func (r *receiver) Connected(p peer.ID) { -} - -func (r *receiver) Disconnected(p peer.ID) { -} - func processResponsesTraces(t *testing.T, tracing *testutil.Collector, responseCount int) []string { - traces := testutil.RepeatTraceStrings("processResponses({})->loaderProcess(0)->cacheProcess(0)", responseCount-1) - finalStub := tracing.FindSpanByTraceString(fmt.Sprintf("processResponses(%d)->loaderProcess(0)", responseCount-1)) - require.NotNil(t, finalStub) - if len(testutil.AttributeValueInTraceSpan(t, *finalStub, "requestIDs").AsStringSlice()) == 0 { - return append(traces, fmt.Sprintf("processResponses(%d)->loaderProcess(0)", responseCount-1)) - } - return append(traces, fmt.Sprintf("processResponses(%d)->loaderProcess(0)->cacheProcess(0)", responseCount-1)) + return testutil.RepeatTraceStrings("processResponses({})", responseCount) } diff --git a/message/message.go b/message/message.go index 9fa471e9..354cc236 100644 --- a/message/message.go +++ b/message/message.go @@ -351,6 +351,11 @@ func (gslm GraphSyncLinkMetadata) Iterate(iter graphsync.LinkMetadataIterator) { } } +// Length returns the number of metadata entries +func (gslm GraphSyncLinkMetadata) Length() int64 { + return int64(len(gslm.linkMetadata)) +} + // RawMetadata accesses the raw GraphSyncLinkMetadatum contained in this object, // this is not exposed via the graphsync.LinkMetadata API and in general the // Iterate() method should be used instead for accessing the individual metadata diff --git a/responsemanager/persistenceoptions/persistenceoptions.go b/persistenceoptions/persistenceoptions.go similarity index 100% rename from responsemanager/persistenceoptions/persistenceoptions.go rename to persistenceoptions/persistenceoptions.go diff --git a/requestmanager/asyncloader/asyncloader.go b/requestmanager/asyncloader/asyncloader.go deleted file mode 100644 index b97c1576..00000000 --- a/requestmanager/asyncloader/asyncloader.go +++ /dev/null @@ -1,216 +0,0 @@ -package asyncloader - -import ( - "context" - "errors" - "fmt" - "io/ioutil" - "sync" - - blocks "github.com/ipfs/go-block-format" - "github.com/ipld/go-ipld-prime" - peer "github.com/libp2p/go-libp2p-core/peer" - "go.opentelemetry.io/otel" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/trace" - - "github.com/ipfs/go-graphsync" - "github.com/ipfs/go-graphsync/requestmanager/asyncloader/loadattemptqueue" - "github.com/ipfs/go-graphsync/requestmanager/asyncloader/responsecache" - "github.com/ipfs/go-graphsync/requestmanager/asyncloader/unverifiedblockstore" - "github.com/ipfs/go-graphsync/requestmanager/types" -) - -type alternateQueue struct { - responseCache *responsecache.ResponseCache - loadAttemptQueue *loadattemptqueue.LoadAttemptQueue -} - -// AsyncLoader manages loading links asynchronously in as new responses -// come in from the network -type AsyncLoader struct { - ctx context.Context - cancel context.CancelFunc - - // this mutex protects access to the state of the async loader, which covers all data fields below below - stateLk sync.Mutex - activeRequests map[graphsync.RequestID]struct{} - requestQueues map[graphsync.RequestID]string - alternateQueues map[string]alternateQueue - responseCache *responsecache.ResponseCache - loadAttemptQueue *loadattemptqueue.LoadAttemptQueue -} - -// New initializes a new link loading manager for asynchronous loads from the given context -// and local store loading and storing function -func New(ctx context.Context, linkSystem ipld.LinkSystem) *AsyncLoader { - responseCache, loadAttemptQueue := setupAttemptQueue(linkSystem) - ctx, cancel := context.WithCancel(ctx) - return &AsyncLoader{ - ctx: ctx, - cancel: cancel, - activeRequests: make(map[graphsync.RequestID]struct{}), - requestQueues: make(map[graphsync.RequestID]string), - alternateQueues: make(map[string]alternateQueue), - responseCache: responseCache, - loadAttemptQueue: loadAttemptQueue, - } -} - -// RegisterPersistenceOption registers a new loader/storer option for processing requests -func (al *AsyncLoader) RegisterPersistenceOption(name string, lsys ipld.LinkSystem) error { - al.stateLk.Lock() - defer al.stateLk.Unlock() - _, existing := al.alternateQueues[name] - if existing { - return errors.New("already registerd a persistence option with this name") - } - responseCache, loadAttemptQueue := setupAttemptQueue(lsys) - al.alternateQueues[name] = alternateQueue{responseCache, loadAttemptQueue} - return nil -} - -// UnregisterPersistenceOption unregisters an existing loader/storer option for processing requests -func (al *AsyncLoader) UnregisterPersistenceOption(name string) error { - al.stateLk.Lock() - defer al.stateLk.Unlock() - _, ok := al.alternateQueues[name] - if !ok { - return fmt.Errorf("unknown persistence option: %s", name) - } - for _, requestQueue := range al.requestQueues { - if name == requestQueue { - return errors.New("cannot unregister while requests are in progress") - } - } - delete(al.alternateQueues, name) - return nil -} - -// StartRequest indicates the given request has started and the manager should -// continually attempt to load links for this request as new responses come in -func (al *AsyncLoader) StartRequest(requestID graphsync.RequestID, persistenceOption string) error { - al.stateLk.Lock() - defer al.stateLk.Unlock() - if persistenceOption != "" { - _, ok := al.alternateQueues[persistenceOption] - if !ok { - return errors.New("unknown persistence option") - } - al.requestQueues[requestID] = persistenceOption - } - al.activeRequests[requestID] = struct{}{} - return nil -} - -// ProcessResponse injests new responses and completes asynchronous loads as -// neccesary -func (al *AsyncLoader) ProcessResponse( - ctx context.Context, - responses map[graphsync.RequestID]graphsync.LinkMetadata, - blks []blocks.Block) { - - requestIds := make([]string, 0, len(responses)) - for requestID := range responses { - requestIds = append(requestIds, requestID.String()) - } - ctx, span := otel.Tracer("graphsync").Start(ctx, "loaderProcess", trace.WithAttributes( - attribute.StringSlice("requestIDs", requestIds), - )) - defer span.End() - - al.stateLk.Lock() - defer al.stateLk.Unlock() - byQueue := make(map[string][]graphsync.RequestID) - for requestID := range responses { - queue := al.requestQueues[requestID] - byQueue[queue] = append(byQueue[queue], requestID) - } - for queue, requestIDs := range byQueue { - loadAttemptQueue := al.getLoadAttemptQueue(queue) - responseCache := al.getResponseCache(queue) - queueResponses := make(map[graphsync.RequestID]graphsync.LinkMetadata, len(requestIDs)) - for _, requestID := range requestIDs { - queueResponses[requestID] = responses[requestID] - } - responseCache.ProcessResponse(ctx, queueResponses, blks) - loadAttemptQueue.RetryLoads() - } -} - -// AsyncLoad asynchronously loads the given link for the given request ID. It returns a channel for data and a channel -// for errors -- only one message will be sent over either. -func (al *AsyncLoader) AsyncLoad(p peer.ID, requestID graphsync.RequestID, link ipld.Link, linkContext ipld.LinkContext) <-chan types.AsyncLoadResult { - resultChan := make(chan types.AsyncLoadResult, 1) - lr := loadattemptqueue.NewLoadRequest(p, requestID, link, linkContext, resultChan) - al.stateLk.Lock() - defer al.stateLk.Unlock() - _, retry := al.activeRequests[requestID] - loadAttemptQueue := al.getLoadAttemptQueue(al.requestQueues[requestID]) - loadAttemptQueue.AttemptLoad(lr, retry) - return resultChan -} - -// CompleteResponsesFor indicates no further responses will come in for the given -// requestID, so if no responses are in the cache or local store, a link load -// should not retry -func (al *AsyncLoader) CompleteResponsesFor(requestID graphsync.RequestID) { - al.stateLk.Lock() - defer al.stateLk.Unlock() - delete(al.activeRequests, requestID) - loadAttemptQueue := al.getLoadAttemptQueue(al.requestQueues[requestID]) - loadAttemptQueue.ClearRequest(requestID) -} - -// CleanupRequest indicates the given request is complete on the client side, -// and no further attempts will be made to load links for this request, -// so any cached response data is invalid can be cleaned -func (al *AsyncLoader) CleanupRequest(p peer.ID, requestID graphsync.RequestID) { - al.stateLk.Lock() - defer al.stateLk.Unlock() - responseCache := al.responseCache - aq, ok := al.requestQueues[requestID] - if ok { - responseCache = al.alternateQueues[aq].responseCache - delete(al.requestQueues, requestID) - } - responseCache.FinishRequest(requestID) -} - -func (al *AsyncLoader) getLoadAttemptQueue(queue string) *loadattemptqueue.LoadAttemptQueue { - if queue == "" { - return al.loadAttemptQueue - } - return al.alternateQueues[queue].loadAttemptQueue -} - -func (al *AsyncLoader) getResponseCache(queue string) *responsecache.ResponseCache { - if queue == "" { - return al.responseCache - } - return al.alternateQueues[queue].responseCache -} - -func setupAttemptQueue(lsys ipld.LinkSystem) (*responsecache.ResponseCache, *loadattemptqueue.LoadAttemptQueue) { - unverifiedBlockStore := unverifiedblockstore.New(lsys.StorageWriteOpener) - responseCache := responsecache.New(unverifiedBlockStore) - loadAttemptQueue := loadattemptqueue.New(func(p peer.ID, requestID graphsync.RequestID, link ipld.Link, linkContext ipld.LinkContext) types.AsyncLoadResult { - // load from response cache - data, err := responseCache.AttemptLoad(requestID, link, linkContext) - if err != nil { - return types.AsyncLoadResult{Err: err, Local: false} - } - if data != nil { - return types.AsyncLoadResult{Data: data, Local: false} - } - // fall back to local store - if stream, err := lsys.StorageReadOpener(linkContext, link); stream != nil && err == nil { - if localData, err := ioutil.ReadAll(stream); err == nil && localData != nil { - return types.AsyncLoadResult{Data: localData, Local: true} - } - } - return types.AsyncLoadResult{Local: false} - }) - - return responseCache, loadAttemptQueue -} diff --git a/requestmanager/asyncloader/asyncloader_test.go b/requestmanager/asyncloader/asyncloader_test.go deleted file mode 100644 index 5845e260..00000000 --- a/requestmanager/asyncloader/asyncloader_test.go +++ /dev/null @@ -1,396 +0,0 @@ -package asyncloader - -import ( - "context" - "io" - "testing" - "time" - - blocks "github.com/ipfs/go-block-format" - ipld "github.com/ipld/go-ipld-prime" - cidlink "github.com/ipld/go-ipld-prime/linking/cid" - "github.com/stretchr/testify/require" - - "github.com/ipfs/go-graphsync" - "github.com/ipfs/go-graphsync/message" - "github.com/ipfs/go-graphsync/requestmanager/types" - "github.com/ipfs/go-graphsync/testutil" -) - -func TestAsyncLoadInitialLoadSucceedsLocallyPresent(t *testing.T) { - block := testutil.GenerateBlocksOfSize(1, 100)[0] - st := newStore() - link := st.Store(t, block) - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - requestID := graphsync.NewRequestID() - p := testutil.GeneratePeers(1)[0] - resultChan := asyncLoader.AsyncLoad(p, requestID, link, ipld.LinkContext{}) - assertSuccessResponse(ctx, t, resultChan) - st.AssertLocalLoads(t, 1) - }) -} - -func TestAsyncLoadInitialLoadSucceedsResponsePresent(t *testing.T) { - blocks := testutil.GenerateBlocksOfSize(1, 100) - block := blocks[0] - link := cidlink.Link{Cid: block.Cid()} - - st := newStore() - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - requestID := graphsync.NewRequestID() - responses := map[graphsync.RequestID]graphsync.LinkMetadata{ - requestID: message.NewLinkMetadata( - []message.GraphSyncLinkMetadatum{{ - Link: link.Cid, - Action: graphsync.LinkActionPresent, - }}), - } - p := testutil.GeneratePeers(1)[0] - asyncLoader.ProcessResponse(context.Background(), responses, blocks) - resultChan := asyncLoader.AsyncLoad(p, requestID, link, ipld.LinkContext{}) - - assertSuccessResponse(ctx, t, resultChan) - st.AssertLocalLoads(t, 0) - st.AssertBlockStored(t, block) - }) -} - -func TestAsyncLoadInitialLoadFails(t *testing.T) { - st := newStore() - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - link := testutil.NewTestLink() - requestID := graphsync.NewRequestID() - - responses := map[graphsync.RequestID]graphsync.LinkMetadata{ - requestID: message.NewLinkMetadata( - []message.GraphSyncLinkMetadatum{{ - Link: link.(cidlink.Link).Cid, - Action: graphsync.LinkActionMissing, - }}), - } - p := testutil.GeneratePeers(1)[0] - asyncLoader.ProcessResponse(context.Background(), responses, nil) - - resultChan := asyncLoader.AsyncLoad(p, requestID, link, ipld.LinkContext{}) - assertFailResponse(ctx, t, resultChan) - st.AssertLocalLoads(t, 0) - }) -} - -func TestAsyncLoadInitialLoadIndeterminateWhenRequestNotInProgress(t *testing.T) { - st := newStore() - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - link := testutil.NewTestLink() - requestID := graphsync.NewRequestID() - p := testutil.GeneratePeers(1)[0] - resultChan := asyncLoader.AsyncLoad(p, requestID, link, ipld.LinkContext{}) - assertFailResponse(ctx, t, resultChan) - st.AssertLocalLoads(t, 1) - }) -} - -func TestAsyncLoadInitialLoadIndeterminateThenSucceeds(t *testing.T) { - blocks := testutil.GenerateBlocksOfSize(1, 100) - block := blocks[0] - link := cidlink.Link{Cid: block.Cid()} - - st := newStore() - - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - requestID := graphsync.NewRequestID() - err := asyncLoader.StartRequest(requestID, "") - require.NoError(t, err) - p := testutil.GeneratePeers(1)[0] - resultChan := asyncLoader.AsyncLoad(p, requestID, link, ipld.LinkContext{}) - - st.AssertAttemptLoadWithoutResult(ctx, t, resultChan) - - responses := map[graphsync.RequestID]graphsync.LinkMetadata{ - requestID: message.NewLinkMetadata( - []message.GraphSyncLinkMetadatum{{ - Link: link.Cid, - Action: graphsync.LinkActionPresent, - }}), - } - asyncLoader.ProcessResponse(context.Background(), responses, blocks) - assertSuccessResponse(ctx, t, resultChan) - st.AssertLocalLoads(t, 1) - st.AssertBlockStored(t, block) - }) -} - -func TestAsyncLoadInitialLoadIndeterminateThenFails(t *testing.T) { - st := newStore() - - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - link := testutil.NewTestLink() - requestID := graphsync.NewRequestID() - err := asyncLoader.StartRequest(requestID, "") - require.NoError(t, err) - p := testutil.GeneratePeers(1)[0] - resultChan := asyncLoader.AsyncLoad(p, requestID, link, ipld.LinkContext{}) - - st.AssertAttemptLoadWithoutResult(ctx, t, resultChan) - - responses := map[graphsync.RequestID]graphsync.LinkMetadata{ - requestID: message.NewLinkMetadata( - []message.GraphSyncLinkMetadatum{{ - Link: link.(cidlink.Link).Cid, - Action: graphsync.LinkActionMissing, - }}), - } - asyncLoader.ProcessResponse(context.Background(), responses, nil) - assertFailResponse(ctx, t, resultChan) - st.AssertLocalLoads(t, 1) - }) -} - -func TestAsyncLoadInitialLoadIndeterminateThenRequestFinishes(t *testing.T) { - st := newStore() - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - link := testutil.NewTestLink() - requestID := graphsync.NewRequestID() - err := asyncLoader.StartRequest(requestID, "") - require.NoError(t, err) - p := testutil.GeneratePeers(1)[0] - resultChan := asyncLoader.AsyncLoad(p, requestID, link, ipld.LinkContext{}) - st.AssertAttemptLoadWithoutResult(ctx, t, resultChan) - asyncLoader.CompleteResponsesFor(requestID) - assertFailResponse(ctx, t, resultChan) - st.AssertLocalLoads(t, 1) - }) -} - -func TestAsyncLoadTwiceLoadsLocallySecondTime(t *testing.T) { - blocks := testutil.GenerateBlocksOfSize(1, 100) - block := blocks[0] - link := cidlink.Link{Cid: block.Cid()} - st := newStore() - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - requestID := graphsync.NewRequestID() - responses := map[graphsync.RequestID]graphsync.LinkMetadata{ - requestID: message.NewLinkMetadata( - []message.GraphSyncLinkMetadatum{{ - Link: link.Cid, - Action: graphsync.LinkActionPresent, - }}), - } - p := testutil.GeneratePeers(1)[0] - asyncLoader.ProcessResponse(context.Background(), responses, blocks) - resultChan := asyncLoader.AsyncLoad(p, requestID, link, ipld.LinkContext{}) - - assertSuccessResponse(ctx, t, resultChan) - st.AssertLocalLoads(t, 0) - - resultChan = asyncLoader.AsyncLoad(p, requestID, link, ipld.LinkContext{}) - assertSuccessResponse(ctx, t, resultChan) - st.AssertLocalLoads(t, 1) - - st.AssertBlockStored(t, block) - }) -} - -func TestRegisterUnregister(t *testing.T) { - st := newStore() - otherSt := newStore() - blocks := testutil.GenerateBlocksOfSize(3, 100) - link1 := otherSt.Store(t, blocks[0]) - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - - requestID1 := graphsync.NewRequestID() - err := asyncLoader.StartRequest(requestID1, "other") - require.EqualError(t, err, "unknown persistence option") - - err = asyncLoader.RegisterPersistenceOption("other", otherSt.lsys) - require.NoError(t, err) - requestID2 := graphsync.NewRequestID() - err = asyncLoader.StartRequest(requestID2, "other") - require.NoError(t, err) - p := testutil.GeneratePeers(1)[0] - resultChan1 := asyncLoader.AsyncLoad(p, requestID2, link1, ipld.LinkContext{}) - assertSuccessResponse(ctx, t, resultChan1) - err = asyncLoader.UnregisterPersistenceOption("other") - require.EqualError(t, err, "cannot unregister while requests are in progress") - asyncLoader.CompleteResponsesFor(requestID2) - asyncLoader.CleanupRequest(p, requestID2) - err = asyncLoader.UnregisterPersistenceOption("other") - require.NoError(t, err) - - requestID3 := graphsync.NewRequestID() - err = asyncLoader.StartRequest(requestID3, "other") - require.EqualError(t, err, "unknown persistence option") - }) -} -func TestRequestSplittingLoadLocallyFromBlockstore(t *testing.T) { - st := newStore() - otherSt := newStore() - block := testutil.GenerateBlocksOfSize(1, 100)[0] - link := otherSt.Store(t, block) - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - err := asyncLoader.RegisterPersistenceOption("other", otherSt.lsys) - require.NoError(t, err) - requestID1 := graphsync.NewRequestID() - p := testutil.GeneratePeers(1)[0] - - resultChan1 := asyncLoader.AsyncLoad(p, requestID1, link, ipld.LinkContext{}) - requestID2 := graphsync.NewRequestID() - err = asyncLoader.StartRequest(requestID2, "other") - require.NoError(t, err) - resultChan2 := asyncLoader.AsyncLoad(p, requestID2, link, ipld.LinkContext{}) - - assertFailResponse(ctx, t, resultChan1) - assertSuccessResponse(ctx, t, resultChan2) - st.AssertLocalLoads(t, 1) - }) -} - -func TestRequestSplittingSameBlockTwoStores(t *testing.T) { - st := newStore() - otherSt := newStore() - blocks := testutil.GenerateBlocksOfSize(1, 100) - block := blocks[0] - link := cidlink.Link{Cid: block.Cid()} - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - err := asyncLoader.RegisterPersistenceOption("other", otherSt.lsys) - require.NoError(t, err) - requestID1 := graphsync.NewRequestID() - requestID2 := graphsync.NewRequestID() - err = asyncLoader.StartRequest(requestID1, "") - require.NoError(t, err) - err = asyncLoader.StartRequest(requestID2, "other") - require.NoError(t, err) - p := testutil.GeneratePeers(1)[0] - resultChan1 := asyncLoader.AsyncLoad(p, requestID1, link, ipld.LinkContext{}) - resultChan2 := asyncLoader.AsyncLoad(p, requestID2, link, ipld.LinkContext{}) - responses := map[graphsync.RequestID]graphsync.LinkMetadata{ - requestID1: message.NewLinkMetadata( - []message.GraphSyncLinkMetadatum{{ - Link: link.Cid, - Action: graphsync.LinkActionPresent, - }}), - requestID2: message.NewLinkMetadata( - []message.GraphSyncLinkMetadatum{{ - Link: link.Cid, - Action: graphsync.LinkActionPresent, - }}), - } - asyncLoader.ProcessResponse(context.Background(), responses, blocks) - - assertSuccessResponse(ctx, t, resultChan1) - assertSuccessResponse(ctx, t, resultChan2) - st.AssertBlockStored(t, block) - otherSt.AssertBlockStored(t, block) - }) -} - -func TestRequestSplittingSameBlockOnlyOneResponse(t *testing.T) { - st := newStore() - otherSt := newStore() - blocks := testutil.GenerateBlocksOfSize(1, 100) - block := blocks[0] - link := cidlink.Link{Cid: block.Cid()} - withLoader(st, func(ctx context.Context, asyncLoader *AsyncLoader) { - err := asyncLoader.RegisterPersistenceOption("other", otherSt.lsys) - require.NoError(t, err) - requestID1 := graphsync.NewRequestID() - requestID2 := graphsync.NewRequestID() - err = asyncLoader.StartRequest(requestID1, "") - require.NoError(t, err) - err = asyncLoader.StartRequest(requestID2, "other") - require.NoError(t, err) - p := testutil.GeneratePeers(1)[0] - resultChan1 := asyncLoader.AsyncLoad(p, requestID1, link, ipld.LinkContext{}) - resultChan2 := asyncLoader.AsyncLoad(p, requestID2, link, ipld.LinkContext{}) - responses := map[graphsync.RequestID]graphsync.LinkMetadata{ - requestID2: message.NewLinkMetadata( - []message.GraphSyncLinkMetadatum{{ - Link: link.Cid, - Action: graphsync.LinkActionPresent, - }}), - } - asyncLoader.ProcessResponse(context.Background(), responses, blocks) - asyncLoader.CompleteResponsesFor(requestID1) - - assertFailResponse(ctx, t, resultChan1) - assertSuccessResponse(ctx, t, resultChan2) - otherSt.AssertBlockStored(t, block) - }) -} - -type store struct { - internalLoader ipld.BlockReadOpener - lsys ipld.LinkSystem - blockstore map[ipld.Link][]byte - localLoads int - called chan struct{} -} - -func newStore() *store { - blockstore := make(map[ipld.Link][]byte) - st := &store{ - lsys: testutil.NewTestStore(blockstore), - blockstore: blockstore, - localLoads: 0, - called: make(chan struct{}), - } - st.internalLoader = st.lsys.StorageReadOpener - st.lsys.StorageReadOpener = st.loader - return st -} - -func (st *store) loader(lnkCtx ipld.LinkContext, lnk ipld.Link) (io.Reader, error) { - select { - case <-st.called: - default: - close(st.called) - } - st.localLoads++ - return st.internalLoader(lnkCtx, lnk) -} - -func (st *store) AssertLocalLoads(t *testing.T, localLoads int) { - require.Equalf(t, localLoads, st.localLoads, "should have loaded locally %d times", localLoads) -} - -func (st *store) AssertBlockStored(t *testing.T, blk blocks.Block) { - require.Equal(t, blk.RawData(), st.blockstore[cidlink.Link{Cid: blk.Cid()}], "should store block") -} - -func (st *store) AssertAttemptLoadWithoutResult(ctx context.Context, t *testing.T, resultChan <-chan types.AsyncLoadResult) { - testutil.AssertDoesReceiveFirst(t, st.called, "should attempt load with no result", resultChan, ctx.Done()) -} - -func (st *store) Store(t *testing.T, blk blocks.Block) ipld.Link { - writer, commit, err := st.lsys.StorageWriteOpener(ipld.LinkContext{}) - require.NoError(t, err) - _, err = writer.Write(blk.RawData()) - require.NoError(t, err, "seeds block store") - link := cidlink.Link{Cid: blk.Cid()} - err = commit(link) - require.NoError(t, err, "seeds block store") - return link -} - -func withLoader(st *store, exec func(ctx context.Context, asyncLoader *AsyncLoader)) { - ctx := context.Background() - ctx, cancel := context.WithTimeout(ctx, 10*time.Second) - defer cancel() - asyncLoader := New(ctx, st.lsys) - exec(ctx, asyncLoader) -} - -func assertSuccessResponse(ctx context.Context, t *testing.T, resultChan <-chan types.AsyncLoadResult) { - t.Helper() - var result types.AsyncLoadResult - testutil.AssertReceive(ctx, t, resultChan, &result, "should close response channel with response") - require.NotNil(t, result.Data, "should send response") - require.Nil(t, result.Err, "should not send error") -} - -func assertFailResponse(ctx context.Context, t *testing.T, resultChan <-chan types.AsyncLoadResult) { - t.Helper() - var result types.AsyncLoadResult - testutil.AssertReceive(ctx, t, resultChan, &result, "should close response channel with response") - require.Nil(t, result.Data, "should not send responses") - require.NotNil(t, result.Err, "should send an error") -} diff --git a/requestmanager/asyncloader/loadattemptqueue/loadattemptqueue.go b/requestmanager/asyncloader/loadattemptqueue/loadattemptqueue.go deleted file mode 100644 index 618c889d..00000000 --- a/requestmanager/asyncloader/loadattemptqueue/loadattemptqueue.go +++ /dev/null @@ -1,96 +0,0 @@ -package loadattemptqueue - -import ( - "errors" - - "github.com/ipld/go-ipld-prime" - "github.com/libp2p/go-libp2p-core/peer" - - "github.com/ipfs/go-graphsync" - "github.com/ipfs/go-graphsync/requestmanager/types" -) - -// LoadRequest is a request to load the given link for the given request id, -// with results returned to the given channel -type LoadRequest struct { - p peer.ID - requestID graphsync.RequestID - link ipld.Link - linkContext ipld.LinkContext - resultChan chan types.AsyncLoadResult -} - -// NewLoadRequest returns a new LoadRequest for the given request id, link, -// and results channel -func NewLoadRequest( - p peer.ID, - requestID graphsync.RequestID, - link ipld.Link, - linkContext ipld.LinkContext, - resultChan chan types.AsyncLoadResult) LoadRequest { - return LoadRequest{p, requestID, link, linkContext, resultChan} -} - -// LoadAttempter attempts to load a link to an array of bytes -// and returns an async load result -type LoadAttempter func(peer.ID, graphsync.RequestID, ipld.Link, ipld.LinkContext) types.AsyncLoadResult - -// LoadAttemptQueue attempts to load using the load attempter, and then can -// place requests on a retry queue -type LoadAttemptQueue struct { - loadAttempter LoadAttempter - pausedRequests []LoadRequest -} - -// New initializes a new AsyncLoader from loadAttempter function -func New(loadAttempter LoadAttempter) *LoadAttemptQueue { - return &LoadAttemptQueue{ - loadAttempter: loadAttempter, - } -} - -// AttemptLoad attempts to loads the given load request, and if retry is true -// it saves the loadrequest for retrying later -func (laq *LoadAttemptQueue) AttemptLoad(lr LoadRequest, retry bool) { - response := laq.loadAttempter(lr.p, lr.requestID, lr.link, lr.linkContext) - if response.Err != nil || response.Data != nil { - lr.resultChan <- response - close(lr.resultChan) - return - } - if !retry { - laq.terminateWithError("No active request", lr.resultChan) - return - } - laq.pausedRequests = append(laq.pausedRequests, lr) -} - -// ClearRequest purges the given request from the queue of load requests -// to retry -func (laq *LoadAttemptQueue) ClearRequest(requestID graphsync.RequestID) { - pausedRequests := laq.pausedRequests - laq.pausedRequests = nil - for _, lr := range pausedRequests { - if lr.requestID == requestID { - laq.terminateWithError("No active request", lr.resultChan) - } else { - laq.pausedRequests = append(laq.pausedRequests, lr) - } - } -} - -// RetryLoads attempts loads on all saved load requests that were loaded with -// retry = true -func (laq *LoadAttemptQueue) RetryLoads() { - // drain buffered - pausedRequests := laq.pausedRequests - laq.pausedRequests = nil - for _, lr := range pausedRequests { - laq.AttemptLoad(lr, true) - } -} - -func (laq *LoadAttemptQueue) terminateWithError(errMsg string, resultChan chan<- types.AsyncLoadResult) { - resultChan <- types.AsyncLoadResult{Data: nil, Err: errors.New(errMsg)} - close(resultChan) -} diff --git a/requestmanager/asyncloader/loadattemptqueue/loadattemptqueue_test.go b/requestmanager/asyncloader/loadattemptqueue/loadattemptqueue_test.go deleted file mode 100644 index 9c83a426..00000000 --- a/requestmanager/asyncloader/loadattemptqueue/loadattemptqueue_test.go +++ /dev/null @@ -1,184 +0,0 @@ -package loadattemptqueue - -import ( - "context" - "fmt" - "testing" - "time" - - ipld "github.com/ipld/go-ipld-prime" - "github.com/libp2p/go-libp2p-core/peer" - "github.com/stretchr/testify/require" - - "github.com/ipfs/go-graphsync" - "github.com/ipfs/go-graphsync/requestmanager/types" - "github.com/ipfs/go-graphsync/testutil" -) - -func TestAsyncLoadInitialLoadSucceeds(t *testing.T) { - ctx := context.Background() - ctx, cancel := context.WithTimeout(ctx, 10*time.Second) - defer cancel() - callCount := 0 - loadAttempter := func(peer.ID, graphsync.RequestID, ipld.Link, ipld.LinkContext) types.AsyncLoadResult { - callCount++ - return types.AsyncLoadResult{ - Data: testutil.RandomBytes(100), - } - } - loadAttemptQueue := New(loadAttempter) - - link := testutil.NewTestLink() - linkContext := ipld.LinkContext{} - requestID := graphsync.NewRequestID() - p := testutil.GeneratePeers(1)[0] - - resultChan := make(chan types.AsyncLoadResult, 1) - lr := NewLoadRequest(p, requestID, link, linkContext, resultChan) - loadAttemptQueue.AttemptLoad(lr, false) - - var result types.AsyncLoadResult - testutil.AssertReceive(ctx, t, resultChan, &result, "should close response channel with response") - require.NotNil(t, result.Data, "should send response") - require.Nil(t, result.Err, "should not send error") - - require.NotZero(t, callCount, "should attempt to load link from local store") -} - -func TestAsyncLoadInitialLoadFails(t *testing.T) { - ctx := context.Background() - ctx, cancel := context.WithTimeout(ctx, 10*time.Second) - defer cancel() - callCount := 0 - loadAttempter := func(peer.ID, graphsync.RequestID, ipld.Link, ipld.LinkContext) types.AsyncLoadResult { - callCount++ - return types.AsyncLoadResult{ - Err: fmt.Errorf("something went wrong"), - } - } - loadAttemptQueue := New(loadAttempter) - - link := testutil.NewTestLink() - linkContext := ipld.LinkContext{} - requestID := graphsync.NewRequestID() - resultChan := make(chan types.AsyncLoadResult, 1) - p := testutil.GeneratePeers(1)[0] - - lr := NewLoadRequest(p, requestID, link, linkContext, resultChan) - loadAttemptQueue.AttemptLoad(lr, false) - - var result types.AsyncLoadResult - testutil.AssertReceive(ctx, t, resultChan, &result, "should close response channel with response") - require.Nil(t, result.Data, "should not send responses") - require.NotNil(t, result.Err, "should send an error") - require.NotZero(t, callCount, "should attempt to load link from local store") -} - -func TestAsyncLoadInitialLoadIndeterminateRetryFalse(t *testing.T) { - ctx := context.Background() - ctx, cancel := context.WithTimeout(ctx, 10*time.Second) - defer cancel() - callCount := 0 - loadAttempter := func(peer.ID, graphsync.RequestID, ipld.Link, ipld.LinkContext) types.AsyncLoadResult { - var result []byte - if callCount > 0 { - result = testutil.RandomBytes(100) - } - callCount++ - return types.AsyncLoadResult{ - Data: result, - } - } - - loadAttemptQueue := New(loadAttempter) - - link := testutil.NewTestLink() - linkContext := ipld.LinkContext{} - requestID := graphsync.NewRequestID() - p := testutil.GeneratePeers(1)[0] - - resultChan := make(chan types.AsyncLoadResult, 1) - lr := NewLoadRequest(p, requestID, link, linkContext, resultChan) - loadAttemptQueue.AttemptLoad(lr, false) - - var result types.AsyncLoadResult - testutil.AssertReceive(ctx, t, resultChan, &result, "should close response channel with response") - require.Nil(t, result.Data, "should not send responses") - require.NotNil(t, result.Err, "should send an error") - require.Equal(t, 1, callCount, "should attempt to load once and then not retry") -} - -func TestAsyncLoadInitialLoadIndeterminateRetryTrueThenRetriedSuccess(t *testing.T) { - ctx := context.Background() - ctx, cancel := context.WithTimeout(ctx, 10*time.Second) - defer cancel() - callCount := 0 - called := make(chan struct{}, 2) - loadAttempter := func(peer.ID, graphsync.RequestID, ipld.Link, ipld.LinkContext) types.AsyncLoadResult { - var result []byte - called <- struct{}{} - if callCount > 0 { - result = testutil.RandomBytes(100) - } - callCount++ - return types.AsyncLoadResult{ - Data: result, - } - } - loadAttemptQueue := New(loadAttempter) - - link := testutil.NewTestLink() - linkContext := ipld.LinkContext{} - requestID := graphsync.NewRequestID() - resultChan := make(chan types.AsyncLoadResult, 1) - p := testutil.GeneratePeers(1)[0] - lr := NewLoadRequest(p, requestID, link, linkContext, resultChan) - loadAttemptQueue.AttemptLoad(lr, true) - - testutil.AssertDoesReceiveFirst(t, called, "should attempt load with no result", resultChan, ctx.Done()) - loadAttemptQueue.RetryLoads() - - var result types.AsyncLoadResult - testutil.AssertReceive(ctx, t, resultChan, &result, "should close response channel with response") - require.NotNil(t, result.Data, "should send response") - require.Nil(t, result.Err, "should not send error") - require.Equal(t, 2, callCount, "should attempt to load multiple times till success") -} - -func TestAsyncLoadInitialLoadIndeterminateThenRequestFinishes(t *testing.T) { - ctx := context.Background() - ctx, cancel := context.WithTimeout(ctx, 10*time.Second) - defer cancel() - callCount := 0 - called := make(chan struct{}, 2) - loadAttempter := func(peer.ID, graphsync.RequestID, ipld.Link, ipld.LinkContext) types.AsyncLoadResult { - var result []byte - called <- struct{}{} - if callCount > 0 { - result = testutil.RandomBytes(100) - } - callCount++ - return types.AsyncLoadResult{ - Data: result, - } - } - loadAttemptQueue := New(loadAttempter) - - link := testutil.NewTestLink() - linkContext := ipld.LinkContext{} - requestID := graphsync.NewRequestID() - resultChan := make(chan types.AsyncLoadResult, 1) - p := testutil.GeneratePeers(1)[0] - lr := NewLoadRequest(p, requestID, link, linkContext, resultChan) - loadAttemptQueue.AttemptLoad(lr, true) - - testutil.AssertDoesReceiveFirst(t, called, "should attempt load with no result", resultChan, ctx.Done()) - loadAttemptQueue.ClearRequest(requestID) - loadAttemptQueue.RetryLoads() - - var result types.AsyncLoadResult - testutil.AssertReceive(ctx, t, resultChan, &result, "should close response channel with response") - require.Nil(t, result.Data, "should not send responses") - require.NotNil(t, result.Err, "should send an error") - require.Equal(t, 1, callCount, "should attempt to load only once because request is finised") -} diff --git a/requestmanager/asyncloader/responsecache/responsecache.go b/requestmanager/asyncloader/responsecache/responsecache.go deleted file mode 100644 index ffaab02e..00000000 --- a/requestmanager/asyncloader/responsecache/responsecache.go +++ /dev/null @@ -1,107 +0,0 @@ -package responsecache - -import ( - "context" - "sync" - - blocks "github.com/ipfs/go-block-format" - "github.com/ipfs/go-cid" - logging "github.com/ipfs/go-log/v2" - "github.com/ipld/go-ipld-prime" - cidlink "github.com/ipld/go-ipld-prime/linking/cid" - "go.opentelemetry.io/otel" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/trace" - - "github.com/ipfs/go-graphsync" - "github.com/ipfs/go-graphsync/linktracker" -) - -var log = logging.Logger("graphsync") - -// UnverifiedBlockStore is an interface for storing blocks -// as they come in and removing them as they are verified -type UnverifiedBlockStore interface { - PruneBlocks(func(ipld.Link, uint64) bool) - PruneBlock(ipld.Link) - VerifyBlock(ipld.Link, ipld.LinkContext) ([]byte, error) - AddUnverifiedBlock(trace.Link, ipld.Link, []byte) -} - -// ResponseCache maintains a store of unverified blocks and response -// data about links for loading, and prunes blocks as needed. -type ResponseCache struct { - responseCacheLk sync.RWMutex - - linkTracker *linktracker.LinkTracker - unverifiedBlockStore UnverifiedBlockStore -} - -// New initializes a new ResponseCache using the given unverified block store. -func New(unverifiedBlockStore UnverifiedBlockStore) *ResponseCache { - return &ResponseCache{ - linkTracker: linktracker.New(), - unverifiedBlockStore: unverifiedBlockStore, - } -} - -// FinishRequest indicate there is no more need to track blocks tied to this -// response. It returns the total number of bytes in blocks that were being -// tracked but are no longer in memory -func (rc *ResponseCache) FinishRequest(requestID graphsync.RequestID) { - rc.responseCacheLk.Lock() - rc.linkTracker.FinishRequest(requestID) - - rc.unverifiedBlockStore.PruneBlocks(func(link ipld.Link, amt uint64) bool { - return rc.linkTracker.BlockRefCount(link) == 0 - }) - rc.responseCacheLk.Unlock() -} - -// AttemptLoad attempts to laod the given block from the cache -func (rc *ResponseCache) AttemptLoad(requestID graphsync.RequestID, link ipld.Link, linkContext ipld.LinkContext) ([]byte, error) { - rc.responseCacheLk.Lock() - defer rc.responseCacheLk.Unlock() - if rc.linkTracker.IsKnownMissingLink(requestID, link) { - return nil, graphsync.RemoteMissingBlockErr{Link: link} - } - data, _ := rc.unverifiedBlockStore.VerifyBlock(link, linkContext) - return data, nil -} - -// ProcessResponse processes incoming response data, adding unverified blocks, -// and tracking link metadata from a remote peer -func (rc *ResponseCache) ProcessResponse( - ctx context.Context, - responses map[graphsync.RequestID]graphsync.LinkMetadata, - blks []blocks.Block) { - - ctx, span := otel.Tracer("graphsync").Start(ctx, "cacheProcess", trace.WithAttributes( - attribute.Int("blockCount", len(blks)), - )) - traceLink := trace.LinkFromContext(ctx) - defer span.End() - - rc.responseCacheLk.Lock() - - for _, block := range blks { - log.Debugf("Received block from network: %s", block.Cid().String()) - rc.unverifiedBlockStore.AddUnverifiedBlock(traceLink, cidlink.Link{Cid: block.Cid()}, block.RawData()) - } - - for requestID, md := range responses { - md.Iterate(func(c cid.Cid, la graphsync.LinkAction) { - log.Debugf("Traverse link %s on request ID %s", c.String(), requestID.String()) - rc.linkTracker.RecordLinkTraversal(requestID, cidlink.Link{Cid: c}, la == graphsync.LinkActionPresent) - }) - } - - // prune unused blocks right away - for _, block := range blks { - if rc.linkTracker.BlockRefCount(cidlink.Link{Cid: block.Cid()}) == 0 { - rc.unverifiedBlockStore.PruneBlock(cidlink.Link{Cid: block.Cid()}) - } - } - - rc.responseCacheLk.Unlock() -} diff --git a/requestmanager/asyncloader/responsecache/responsecache_test.go b/requestmanager/asyncloader/responsecache/responsecache_test.go deleted file mode 100644 index 7cc8497f..00000000 --- a/requestmanager/asyncloader/responsecache/responsecache_test.go +++ /dev/null @@ -1,147 +0,0 @@ -package responsecache - -import ( - "context" - "fmt" - "testing" - - blocks "github.com/ipfs/go-block-format" - ipld "github.com/ipld/go-ipld-prime" - cidlink "github.com/ipld/go-ipld-prime/linking/cid" - "github.com/stretchr/testify/require" - "go.opentelemetry.io/otel/trace" - - "github.com/ipfs/go-graphsync" - "github.com/ipfs/go-graphsync/message" - "github.com/ipfs/go-graphsync/testutil" -) - -type fakeUnverifiedBlockStore struct { - inMemoryBlocks map[ipld.Link][]byte -} - -func (ubs *fakeUnverifiedBlockStore) AddUnverifiedBlock(_ trace.Link, lnk ipld.Link, data []byte) { - ubs.inMemoryBlocks[lnk] = data -} - -func (ubs *fakeUnverifiedBlockStore) PruneBlocks(shouldPrune func(ipld.Link, uint64) bool) { - for link, data := range ubs.inMemoryBlocks { - if shouldPrune(link, uint64(len(data))) { - delete(ubs.inMemoryBlocks, link) - } - } -} - -func (ubs *fakeUnverifiedBlockStore) PruneBlock(link ipld.Link) { - delete(ubs.inMemoryBlocks, link) -} - -func (ubs *fakeUnverifiedBlockStore) VerifyBlock(lnk ipld.Link, linkCtx ipld.LinkContext) ([]byte, error) { - data, ok := ubs.inMemoryBlocks[lnk] - if !ok { - return nil, fmt.Errorf("Block not found") - } - delete(ubs.inMemoryBlocks, lnk) - return data, nil -} - -func (ubs *fakeUnverifiedBlockStore) blocks() []blocks.Block { - blks := make([]blocks.Block, 0, len(ubs.inMemoryBlocks)) - for link, data := range ubs.inMemoryBlocks { - blk, err := blocks.NewBlockWithCid(data, link.(cidlink.Link).Cid) - if err == nil { - blks = append(blks, blk) - } - } - return blks -} - -func TestResponseCacheManagingLinks(t *testing.T) { - blks := testutil.GenerateBlocksOfSize(5, 100) - requestID1 := graphsync.NewRequestID() - requestID2 := graphsync.NewRequestID() - - request1Metadata := []message.GraphSyncLinkMetadatum{ - { - Link: blks[0].Cid(), - Action: graphsync.LinkActionPresent, - }, - { - Link: blks[1].Cid(), - Action: graphsync.LinkActionMissing, - }, - { - Link: blks[3].Cid(), - Action: graphsync.LinkActionPresent, - }, - } - - request2Metadata := []message.GraphSyncLinkMetadatum{ - { - Link: blks[1].Cid(), - Action: graphsync.LinkActionPresent, - }, - { - Link: blks[3].Cid(), - Action: graphsync.LinkActionPresent, - }, - { - Link: blks[4].Cid(), - Action: graphsync.LinkActionPresent, - }, - } - - responses := map[graphsync.RequestID]graphsync.LinkMetadata{ - requestID1: message.NewLinkMetadata(request1Metadata), - requestID2: message.NewLinkMetadata(request2Metadata), - } - - fubs := &fakeUnverifiedBlockStore{ - inMemoryBlocks: make(map[ipld.Link][]byte), - } - responseCache := New(fubs) - - responseCache.ProcessResponse(context.Background(), responses, blks) - - require.Len(t, fubs.blocks(), len(blks)-1, "should prune block with no references") - testutil.RefuteContainsBlock(t, fubs.blocks(), blks[2]) - - lnkCtx := ipld.LinkContext{} - // should load block from unverified block store - data, err := responseCache.AttemptLoad(requestID2, cidlink.Link{Cid: blks[4].Cid()}, lnkCtx) - require.NoError(t, err) - require.Equal(t, blks[4].RawData(), data, "did not load correct block") - - // which will remove block - require.Len(t, fubs.blocks(), len(blks)-2, "should prune block once verified") - testutil.RefuteContainsBlock(t, fubs.blocks(), blks[4]) - - // fails as it is a known missing block - data, err = responseCache.AttemptLoad(requestID1, cidlink.Link{Cid: blks[1].Cid()}, lnkCtx) - require.Error(t, err) - require.Nil(t, data, "no data should be returned for missing block") - - // should succeed for request 2 where it's not a missing block - data, err = responseCache.AttemptLoad(requestID2, cidlink.Link{Cid: blks[1].Cid()}, lnkCtx) - require.NoError(t, err) - require.Equal(t, blks[1].RawData(), data) - - // which will remove block - require.Len(t, fubs.blocks(), len(blks)-3, "should prune block once verified") - testutil.RefuteContainsBlock(t, fubs.blocks(), blks[1]) - - // should be unknown result as block is not known missing or present in block store - data, err = responseCache.AttemptLoad(requestID1, cidlink.Link{Cid: blks[2].Cid()}, lnkCtx) - require.NoError(t, err) - require.Nil(t, data, "no data should be returned for unknown block") - - responseCache.FinishRequest(requestID1) - // should remove only block 0, since it now has no refering outstanding requests - require.Len(t, fubs.blocks(), len(blks)-4, "should prune block when it is orphaned") - testutil.RefuteContainsBlock(t, fubs.blocks(), blks[0]) - - responseCache.FinishRequest(requestID2) - // should remove last block since are no remaining references - require.Len(t, fubs.blocks(), 0, "should prune block when it is orphaned") - testutil.RefuteContainsBlock(t, fubs.blocks(), blks[3]) -} diff --git a/requestmanager/asyncloader/unverifiedblockstore/unverifiedblockstore.go b/requestmanager/asyncloader/unverifiedblockstore/unverifiedblockstore.go deleted file mode 100644 index 2478b0f5..00000000 --- a/requestmanager/asyncloader/unverifiedblockstore/unverifiedblockstore.go +++ /dev/null @@ -1,109 +0,0 @@ -package unverifiedblockstore - -import ( - "context" - "fmt" - - logging "github.com/ipfs/go-log/v2" - ipld "github.com/ipld/go-ipld-prime" - "go.opentelemetry.io/otel" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/trace" -) - -var log = logging.Logger("gs-unverifiedbs") - -type settableWriter interface { - SetBytes([]byte) error -} - -// UnverifiedBlockStore holds an in memory cache of receied blocks from the network -// that have not been verified to be part of a traversal -type UnverifiedBlockStore struct { - inMemoryBlocks map[ipld.Link]tracedBlock - storer ipld.BlockWriteOpener - dataSize uint64 -} - -type tracedBlock struct { - block []byte - traceLink trace.Link -} - -// New initializes a new unverified store with the given storer function for writing -// to permaneant storage if the block is verified -func New(storer ipld.BlockWriteOpener) *UnverifiedBlockStore { - return &UnverifiedBlockStore{ - inMemoryBlocks: make(map[ipld.Link]tracedBlock), - storer: storer, - } -} - -// AddUnverifiedBlock adds a new unverified block to the in memory cache as it -// comes in as part of a traversal. -func (ubs *UnverifiedBlockStore) AddUnverifiedBlock(traceLink trace.Link, lnk ipld.Link, data []byte) { - ubs.inMemoryBlocks[lnk] = tracedBlock{data, traceLink} - ubs.dataSize = ubs.dataSize + uint64(len(data)) - log.Debugw("added in-memory block", "total_queued_bytes", ubs.dataSize) -} - -// PruneBlocks removes blocks from the unverified store without committing them, -// if the passed in function returns true for the given link -func (ubs *UnverifiedBlockStore) PruneBlocks(shouldPrune func(ipld.Link, uint64) bool) { - for link, data := range ubs.inMemoryBlocks { - if shouldPrune(link, uint64(len(data.block))) { - delete(ubs.inMemoryBlocks, link) - ubs.dataSize = ubs.dataSize - uint64(len(data.block)) - } - } - log.Debugw("finished pruning in-memory blocks", "total_queued_bytes", ubs.dataSize) -} - -// PruneBlock deletes an individual block from the store -func (ubs *UnverifiedBlockStore) PruneBlock(link ipld.Link) { - delete(ubs.inMemoryBlocks, link) - ubs.dataSize = ubs.dataSize - uint64(len(ubs.inMemoryBlocks[link].block)) - log.Debugw("pruned in-memory block", "total_queued_bytes", ubs.dataSize) -} - -// VerifyBlock verifies the data for the given link as being part of a traversal, -// removes it from the unverified store, and writes it to permaneant storage. -func (ubs *UnverifiedBlockStore) VerifyBlock(lnk ipld.Link, linkContext ipld.LinkContext) ([]byte, error) { - data, ok := ubs.inMemoryBlocks[lnk] - if !ok { - return nil, fmt.Errorf("block not found") - } - - ctx := linkContext.Ctx - if ctx == nil { - ctx = context.Background() - } - _, span := otel.Tracer("graphsync").Start( - ctx, - "verifyBlock", - trace.WithLinks(data.traceLink), - trace.WithAttributes(attribute.String("cid", lnk.String()))) - defer span.End() - - delete(ubs.inMemoryBlocks, lnk) - ubs.dataSize = ubs.dataSize - uint64(len(data.block)) - log.Debugw("verified block", "total_queued_bytes", ubs.dataSize) - - buffer, committer, err := ubs.storer(linkContext) - if err != nil { - return nil, err - } - if settable, ok := buffer.(settableWriter); ok { - err = settable.SetBytes(data.block) - } else { - _, err = buffer.Write(data.block) - } - if err != nil { - return nil, err - } - err = committer(lnk) - if err != nil { - return nil, err - } - return data.block, nil -} diff --git a/requestmanager/asyncloader/unverifiedblockstore/unverifiedblockstore_test.go b/requestmanager/asyncloader/unverifiedblockstore/unverifiedblockstore_test.go deleted file mode 100644 index 92bd66e5..00000000 --- a/requestmanager/asyncloader/unverifiedblockstore/unverifiedblockstore_test.go +++ /dev/null @@ -1,47 +0,0 @@ -package unverifiedblockstore - -import ( - "bytes" - "io" - "testing" - - "github.com/ipld/go-ipld-prime" - cidlink "github.com/ipld/go-ipld-prime/linking/cid" - "github.com/stretchr/testify/require" - "go.opentelemetry.io/otel/trace" - - "github.com/ipfs/go-graphsync/testutil" -) - -func TestVerifyBlockPresent(t *testing.T) { - blocksWritten := make(map[ipld.Link][]byte) - lsys := testutil.NewTestStore(blocksWritten) - unverifiedBlockStore := New(lsys.StorageWriteOpener) - block := testutil.GenerateBlocksOfSize(1, 100)[0] - reader, err := lsys.StorageReadOpener(ipld.LinkContext{}, cidlink.Link{Cid: block.Cid()}) - require.Nil(t, reader) - require.Error(t, err, "block should not be loadable till it's verified and stored") - - data, err := unverifiedBlockStore.VerifyBlock(cidlink.Link{Cid: block.Cid()}, ipld.LinkContext{}) - require.Nil(t, data) - require.Error(t, err, "block should not be verifiable till it's added as an unverifiable block") - - unverifiedBlockStore.AddUnverifiedBlock(trace.Link{}, cidlink.Link{Cid: block.Cid()}, block.RawData()) - reader, err = lsys.StorageReadOpener(ipld.LinkContext{}, cidlink.Link{Cid: block.Cid()}) - require.Nil(t, reader) - require.Error(t, err, "block should not be loadable till it's verified") - - data, err = unverifiedBlockStore.VerifyBlock(cidlink.Link{Cid: block.Cid()}, ipld.LinkContext{}) - require.NoError(t, err) - require.Equal(t, block.RawData(), data, "block should be returned on verification if added") - - reader, err = lsys.StorageReadOpener(ipld.LinkContext{}, cidlink.Link{Cid: block.Cid()}) - require.NoError(t, err) - var buffer bytes.Buffer - _, err = io.Copy(&buffer, reader) - require.NoError(t, err) - require.Equal(t, block.RawData(), buffer.Bytes(), "block should be stored and loadable after verification") - data, err = unverifiedBlockStore.VerifyBlock(cidlink.Link{Cid: block.Cid()}, ipld.LinkContext{}) - require.Nil(t, data) - require.Error(t, err, "block cannot be verified twice") -} diff --git a/requestmanager/client.go b/requestmanager/client.go index 5002f516..9e282e32 100644 --- a/requestmanager/client.go +++ b/requestmanager/client.go @@ -28,7 +28,7 @@ import ( "github.com/ipfs/go-graphsync/peerstate" "github.com/ipfs/go-graphsync/requestmanager/executor" "github.com/ipfs/go-graphsync/requestmanager/hooks" - "github.com/ipfs/go-graphsync/requestmanager/types" + "github.com/ipfs/go-graphsync/requestmanager/reconciledloader" "github.com/ipfs/go-graphsync/taskqueue" ) @@ -61,6 +61,8 @@ type inProgressRequestStatus struct { inProgressErr chan error traverser ipldutil.Traverser traverserCancel context.CancelFunc + lsys *ipld.LinkSystem + reconciledLoader *reconciledloader.ReconciledLoader } // PeerHandler is an interface that can send requests to peers @@ -68,28 +70,23 @@ type PeerHandler interface { AllocateAndBuildMessage(p peer.ID, blkSize uint64, buildMessageFn func(*messagequeue.Builder)) } -// AsyncLoader is an interface for loading links asynchronously, returning -// results as new responses are processed -type AsyncLoader interface { - StartRequest(graphsync.RequestID, string) error - ProcessResponse(ctx context.Context, responses map[graphsync.RequestID]graphsync.LinkMetadata, blks []blocks.Block) - AsyncLoad(p peer.ID, requestID graphsync.RequestID, link ipld.Link, linkContext ipld.LinkContext) <-chan types.AsyncLoadResult - CompleteResponsesFor(requestID graphsync.RequestID) - CleanupRequest(p peer.ID, requestID graphsync.RequestID) +// PersistenceOptions is an interface for getting loaders by name +type PersistenceOptions interface { + GetLinkSystem(name string) (ipld.LinkSystem, bool) } // RequestManager tracks outgoing requests and processes incoming reponses // to them. type RequestManager struct { - ctx context.Context - cancel context.CancelFunc - messages chan requestManagerMessage - peerHandler PeerHandler - rc *responseCollector - asyncLoader AsyncLoader - disconnectNotif *pubsub.PubSub - linkSystem ipld.LinkSystem - connManager network.ConnManager + ctx context.Context + cancel context.CancelFunc + messages chan requestManagerMessage + peerHandler PeerHandler + rc *responseCollector + persistenceOptions PersistenceOptions + disconnectNotif *pubsub.PubSub + linkSystem ipld.LinkSystem + connManager network.ConnManager // maximum number of links to traverse per request. A value of zero = infinity, or no limit maxLinksPerRequest uint64 @@ -118,7 +115,7 @@ type ResponseHooks interface { // New generates a new request manager from a context, network, and selectorQuerier func New(ctx context.Context, - asyncLoader AsyncLoader, + persistenceOptions PersistenceOptions, linkSystem ipld.LinkSystem, requestHooks RequestHooks, responseHooks ResponseHooks, @@ -132,7 +129,7 @@ func New(ctx context.Context, return &RequestManager{ ctx: ctx, cancel: cancel, - asyncLoader: asyncLoader, + persistenceOptions: persistenceOptions, disconnectNotif: pubsub.New(disconnectDispatcher), linkSystem: linkSystem, rc: newResponseCollector(ctx), diff --git a/requestmanager/executor/executor.go b/requestmanager/executor/executor.go index 4659a70d..507cba02 100644 --- a/requestmanager/executor/executor.go +++ b/requestmanager/executor/executor.go @@ -7,7 +7,8 @@ import ( logging "github.com/ipfs/go-log/v2" "github.com/ipfs/go-peertaskqueue/peertask" - "github.com/ipld/go-ipld-prime" + "github.com/ipld/go-ipld-prime/datamodel" + "github.com/ipld/go-ipld-prime/linking" "github.com/ipld/go-ipld-prime/traversal" "github.com/libp2p/go-libp2p-core/peer" "go.opentelemetry.io/otel" @@ -36,9 +37,12 @@ type BlockHooks interface { ProcessBlockHooks(p peer.ID, response graphsync.ResponseData, block graphsync.BlockData) hooks.UpdateResult } -// AsyncLoadFn is a function which given a request id and an ipld.Link, returns -// a channel which will eventually return data for the link or an err -type AsyncLoadFn func(peer.ID, graphsync.RequestID, ipld.Link, ipld.LinkContext) <-chan types.AsyncLoadResult +// ReconciledLoader is an interface that can be used to load blocks from a local store or a remote request +type ReconciledLoader interface { + SetRemoteOnline(online bool) + RetryLastLoad() types.AsyncLoadResult + BlockReadOpener(lctx linking.LinkContext, link datamodel.Link) types.AsyncLoadResult +} // Executor handles actually executing graphsync requests and verifying them. // It has control of requests when they are in the "running" state, while @@ -46,18 +50,15 @@ type AsyncLoadFn func(peer.ID, graphsync.RequestID, ipld.Link, ipld.LinkContext) type Executor struct { manager Manager blockHooks BlockHooks - loader AsyncLoadFn } // NewExecutor returns a new executor func NewExecutor( manager Manager, - blockHooks BlockHooks, - loader AsyncLoadFn) *Executor { + blockHooks BlockHooks) *Executor { return &Executor{ manager: manager, blockHooks: blockHooks, - loader: loader, } } @@ -84,6 +85,7 @@ func (e *Executor) ExecuteTask(ctx context.Context, pid peer.ID, task *peertask. span.RecordError(err) if !ipldutil.IsContextCancelErr(err) { e.manager.SendRequest(requestTask.P, gsmsg.NewCancelRequest(requestTask.Request.ID())) + requestTask.ReconciledLoader.SetRemoteOnline(false) if !isPausedErr(err) { span.SetStatus(codes.Error, err.Error()) select { @@ -110,17 +112,12 @@ type RequestTask struct { P peer.ID InProgressErr chan error Empty bool - InitialRequest bool + ReconciledLoader ReconciledLoader } func (e *Executor) traverse(rt RequestTask) error { - onlyOnce := &onlyOnce{e, rt, false} + requestSent := false // for initial request, start remote right away - if rt.InitialRequest { - if err := onlyOnce.startRemoteRequest(); err != nil { - return err - } - } for { // check if traversal is complete isComplete, err := rt.Traverser.IsComplete() @@ -131,23 +128,20 @@ func (e *Executor) traverse(rt RequestTask) error { lnk, linkContext := rt.Traverser.CurrentRequest() // attempt to load log.Debugf("will load link=%s", lnk) - resultChan := e.loader(rt.P, rt.Request.ID(), lnk, linkContext) - var result types.AsyncLoadResult - // check for immediate result - select { - case result = <-resultChan: - default: - // if no immediate result - // initiate remote request if not already sent (we want to fill out the doNotSendCids on a resume) - if err := onlyOnce.startRemoteRequest(); err != nil { + result := rt.ReconciledLoader.BlockReadOpener(linkContext, lnk) + // if we've only loaded locally so far and hit a missing block + // initiate remote request and retry the load operation from remote + if _, ok := result.Err.(graphsync.RemoteMissingBlockErr); ok && !requestSent { + requestSent = true + + // tell the loader we're online now + rt.ReconciledLoader.SetRemoteOnline(true) + + if err := e.startRemoteRequest(rt); err != nil { return err } - // wait for block result - select { - case <-rt.Ctx.Done(): - return ipldutil.ContextCancelError{} - case result = <-resultChan: - } + // retry the load + result = rt.ReconciledLoader.RetryLastLoad() } log.Debugf("successfully loaded link=%s, nBlocksRead=%d", lnk, rt.Traverser.NBlocksTraversed()) // advance the traversal based on results @@ -161,7 +155,6 @@ func (e *Executor) traverse(rt RequestTask) error { if err != nil { return err } - } } @@ -191,14 +184,18 @@ func (e *Executor) advanceTraversal(rt RequestTask, result types.AsyncLoadResult case <-rt.Ctx.Done(): return ipldutil.ContextCancelError{} case rt.InProgressErr <- result.Err: - rt.Traverser.Error(traversal.SkipMe{}) + if _, ok := result.Err.(graphsync.RemoteMissingBlockErr); ok { + rt.Traverser.Error(traversal.SkipMe{}) + } else { + rt.Traverser.Error(result.Err) + } return nil } } return rt.Traverser.Advance(bytes.NewBuffer(result.Data)) } -func (e *Executor) processResult(rt RequestTask, link ipld.Link, result types.AsyncLoadResult) error { +func (e *Executor) processResult(rt RequestTask, link datamodel.Link, result types.AsyncLoadResult) error { var err error if result.Err == nil { err = e.onNewBlock(rt, &blockData{link, result.Local, uint64(len(result.Data)), int64(rt.Traverser.NBlocksTraversed())}) @@ -233,29 +230,15 @@ func isPausedErr(err error) bool { return isPaused } -type onlyOnce struct { - e *Executor - rt RequestTask - requestSent bool -} - -func (so *onlyOnce) startRemoteRequest() error { - if so.requestSent { - return nil - } - so.requestSent = true - return so.e.startRemoteRequest(so.rt) -} - type blockData struct { - link ipld.Link + link datamodel.Link local bool size uint64 index int64 } // Link is the link/cid for the block -func (bd *blockData) Link() ipld.Link { +func (bd *blockData) Link() datamodel.Link { return bd.link } diff --git a/requestmanager/executor/executor_test.go b/requestmanager/executor/executor_test.go index b25adb98..a3e8562b 100644 --- a/requestmanager/executor/executor_test.go +++ b/requestmanager/executor/executor_test.go @@ -4,12 +4,15 @@ import ( "context" "errors" "math/rand" + "sync" "sync/atomic" "testing" "time" + blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-peertaskqueue/peertask" - "github.com/ipld/go-ipld-prime" + "github.com/ipld/go-ipld-prime/datamodel" + "github.com/ipld/go-ipld-prime/linking" cidlink "github.com/ipld/go-ipld-prime/linking/cid" "github.com/ipld/go-ipld-prime/node/basicnode" "github.com/ipld/go-ipld-prime/traversal" @@ -22,7 +25,6 @@ import ( gsmsg "github.com/ipfs/go-graphsync/message" "github.com/ipfs/go-graphsync/requestmanager/executor" "github.com/ipfs/go-graphsync/requestmanager/hooks" - "github.com/ipfs/go-graphsync/requestmanager/testloader" "github.com/ipfs/go-graphsync/requestmanager/types" "github.com/ipfs/go-graphsync/testutil" ) @@ -45,15 +47,15 @@ func TestRequestExecutionBlockChain(t *testing.T) { configureRequestExecution: func(p peer.ID, requestID graphsync.RequestID, tbc *testutil.TestBlockChain, ree *requestExecutionEnv) { ree.customRemoteBehavior = func() { // pretend the remote sent five blocks before encountering a missing block - ree.fal.SuccessResponseOn(p, requestID, tbc.Blocks(0, 5)) + ree.reconciledLoader.successResponseOn(tbc.Blocks(0, 5)) missingCid := cidlink.Link{Cid: tbc.Blocks(5, 6)[0].Cid()} - ree.fal.ResponseOn(p, requestID, missingCid, types.AsyncLoadResult{Err: graphsync.RemoteMissingBlockErr{Link: missingCid}}) + ree.reconciledLoader.responseOn(missingCid, types.AsyncLoadResult{Err: graphsync.RemoteMissingBlockErr{Link: missingCid, Path: tbc.PathTipIndex(5)}}) } }, verifyResults: func(t *testing.T, tbc *testutil.TestBlockChain, ree *requestExecutionEnv, responses []graphsync.ResponseProgress, receivedErrors []error) { tbc.VerifyResponseRangeSync(responses, 0, 5) require.Len(t, receivedErrors, 1) - require.Equal(t, receivedErrors[0], graphsync.RemoteMissingBlockErr{Link: cidlink.Link{Cid: tbc.Blocks(5, 6)[0].Cid()}}) + require.Equal(t, receivedErrors[0], graphsync.RemoteMissingBlockErr{Link: cidlink.Link{Cid: tbc.Blocks(5, 6)[0].Cid()}, Path: tbc.PathTipIndex(5)}) require.Equal(t, []requestSent{{ree.p, ree.request}}, ree.requestsSent) // we should only call block hooks for blocks we actually received require.Len(t, ree.blookHooksCalled, 5) @@ -194,9 +196,11 @@ func TestRequestExecutionBlockChain(t *testing.T) { ctx := context.Background() ctx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() - persistence := testutil.NewTestStore(make(map[ipld.Link][]byte)) + persistence := testutil.NewTestStore(make(map[datamodel.Link][]byte)) tbc := testutil.SetupBlockChain(ctx, t, persistence, 100, 10) - fal := testloader.NewFakeAsyncLoader() + reconciledLoader := &fakeReconciledLoader{ + responses: make(map[datamodel.Link]chan types.AsyncLoadResult), + } requestID := graphsync.NewRequestID() p := testutil.GeneratePeers(1)[0] requestCtx, requestCancel := context.WithCancel(ctx) @@ -209,14 +213,13 @@ func TestRequestExecutionBlockChain(t *testing.T) { blockHookResults: make(map[blockHookKey]hooks.UpdateResult), doNotSendFirstBlocks: 0, request: gsmsg.NewRequest(requestID, tbc.TipLink.(cidlink.Link).Cid, tbc.Selector(), graphsync.Priority(rand.Int31())), - fal: fal, tbc: tbc, initialRequest: true, inProgressErr: make(chan error, 1), traverser: ipldutil.TraversalBuilder{ Root: tbc.TipLink, Selector: tbc.Selector(), - Visitor: func(tp traversal.Progress, node ipld.Node, tr traversal.VisitReason) error { + Visitor: func(tp traversal.Progress, node datamodel.Node, tr traversal.VisitReason) error { responsesReceived = append(responsesReceived, graphsync.ResponseProgress{ Node: node, Path: tp.Path, @@ -225,12 +228,14 @@ func TestRequestExecutionBlockChain(t *testing.T) { return nil }, }.Start(requestCtx), + reconciledLoader: reconciledLoader, } - fal.OnAsyncLoad(ree.checkPause) + reconciledLoader.onAsyncLoad(ree.checkPause) if data.configureRequestExecution != nil { data.configureRequestExecution(p, requestID, tbc, ree) } - ree.fal.SuccessResponseOn(p, requestID, tbc.Blocks(0, ree.loadLocallyUntil)) + reconciledLoader.successResponseOn(tbc.Blocks(0, ree.loadLocallyUntil)) + reconciledLoader.responseOn(tbc.LinkTipIndex(ree.loadLocallyUntil), types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{Link: tbc.LinkTipIndex(ree.loadLocallyUntil)}}) var errorsReceived []error errCollectionErr := make(chan error, 1) go func() { @@ -247,7 +252,7 @@ func TestRequestExecutionBlockChain(t *testing.T) { } } }() - executor.NewExecutor(ree, ree, fal.AsyncLoad).ExecuteTask(ctx, ree.p, &peertask.Task{}) + executor.NewExecutor(ree, ree).ExecuteTask(ctx, ree.p, &peertask.Task{}) require.NoError(t, <-errCollectionErr) ree.traverser.Shutdown(ctx) data.verifyResults(t, tbc, ree, responsesReceived, errorsReceived) @@ -263,12 +268,12 @@ type requestSent struct { type blockHookKey struct { p peer.ID requestID graphsync.RequestID - link ipld.Link + link datamodel.Link } type pauseKey struct { requestID graphsync.RequestID - link ipld.Link + link datamodel.Link } type requestExecutionEnv struct { @@ -282,6 +287,7 @@ type requestExecutionEnv struct { externalPause pauseKey loadLocallyUntil int traverser ipldutil.Traverser + reconciledLoader *fakeReconciledLoader inProgressErr chan error initialRequest bool customRemoteBehavior func() @@ -292,9 +298,62 @@ type requestExecutionEnv struct { // deps tbc *testutil.TestBlockChain - fal *testloader.FakeAsyncLoader } +type fakeReconciledLoader struct { + responsesLk sync.Mutex + responses map[datamodel.Link]chan types.AsyncLoadResult + lastLoad datamodel.Link + online bool + cb func(datamodel.Link) +} + +func (frl *fakeReconciledLoader) onAsyncLoad(cb func(datamodel.Link)) { + frl.cb = cb +} + +func (frl *fakeReconciledLoader) responseOn(link datamodel.Link, result types.AsyncLoadResult) { + response := frl.asyncLoad(link, true) + response <- result + close(response) +} + +func (frl *fakeReconciledLoader) successResponseOn(blks []blocks.Block) { + + for _, block := range blks { + frl.responseOn(cidlink.Link{Cid: block.Cid()}, types.AsyncLoadResult{Data: block.RawData(), Local: false, Err: nil}) + } +} + +func (frl *fakeReconciledLoader) asyncLoad(link datamodel.Link, force bool) chan types.AsyncLoadResult { + frl.responsesLk.Lock() + response, ok := frl.responses[link] + if !ok || force { + response = make(chan types.AsyncLoadResult, 1) + frl.responses[link] = response + } + frl.responsesLk.Unlock() + return response +} + +func (frl *fakeReconciledLoader) BlockReadOpener(_ linking.LinkContext, link datamodel.Link) types.AsyncLoadResult { + frl.lastLoad = link + if frl.cb != nil { + frl.cb(link) + } + return <-frl.asyncLoad(link, false) +} + +func (frl *fakeReconciledLoader) RetryLastLoad() types.AsyncLoadResult { + if frl.cb != nil { + frl.cb(frl.lastLoad) + } + return <-frl.asyncLoad(frl.lastLoad, false) +} + +func (frl *fakeReconciledLoader) SetRemoteOnline(online bool) { + frl.online = true +} func (ree *requestExecutionEnv) ReleaseRequestTask(_ peer.ID, _ *peertask.Task, err error) { ree.terminalError = err close(ree.inProgressErr) @@ -314,7 +373,7 @@ func (ree *requestExecutionEnv) GetRequestTask(_ peer.ID, _ *peertask.Task, requ P: ree.p, InProgressErr: ree.inProgressErr, Empty: false, - InitialRequest: ree.initialRequest, + ReconciledLoader: ree.reconciledLoader, } go func() { select { @@ -328,7 +387,7 @@ func (ree *requestExecutionEnv) SendRequest(p peer.ID, request gsmsg.GraphSyncRe ree.requestsSent = append(ree.requestsSent, requestSent{p, request}) if request.Type() == graphsync.RequestTypeNew { if ree.customRemoteBehavior == nil { - ree.fal.SuccessResponseOn(p, request.ID(), ree.tbc.Blocks(ree.loadLocallyUntil, len(ree.tbc.AllBlocks()))) + ree.reconciledLoader.successResponseOn(ree.tbc.Blocks(ree.loadLocallyUntil, len(ree.tbc.AllBlocks()))) } else { ree.customRemoteBehavior() } @@ -341,8 +400,8 @@ func (ree *requestExecutionEnv) ProcessBlockHooks(p peer.ID, response graphsync. return ree.blockHookResults[bhk] } -func (ree *requestExecutionEnv) checkPause(requestID graphsync.RequestID, link ipld.Link, result <-chan types.AsyncLoadResult) { - if ree.externalPause.link == link && ree.externalPause.requestID == requestID { +func (ree *requestExecutionEnv) checkPause(link datamodel.Link) { + if ree.externalPause.link == link { ree.externalPause = pauseKey{} ree.pauseMessages <- struct{}{} } diff --git a/requestmanager/reconciledloader/injest.go b/requestmanager/reconciledloader/injest.go new file mode 100644 index 00000000..2150a0e7 --- /dev/null +++ b/requestmanager/reconciledloader/injest.go @@ -0,0 +1,44 @@ +package reconciledloader + +import ( + "github.com/ipfs/go-cid" + "github.com/ipfs/go-graphsync" + "go.opentelemetry.io/otel/trace" +) + +// IngestResponse ingests new remote items into the reconciled loader +func (rl *ReconciledLoader) IngestResponse(md graphsync.LinkMetadata, traceLink trace.Link, blocks map[cid.Cid][]byte) { + if md.Length() == 0 { + return + } + duplicates := make(map[cid.Cid]struct{}, md.Length()) + items := make([]*remotedLinkedItem, 0, md.Length()) + md.Iterate(func(link cid.Cid, action graphsync.LinkAction) { + newItem := newRemote() + newItem.link = link + newItem.action = action + if action == graphsync.LinkActionPresent { + if _, isDuplicate := duplicates[link]; !isDuplicate { + duplicates[link] = struct{}{} + newItem.block = blocks[link] + } + } + newItem.traceLink = traceLink + items = append(items, newItem) + }) + rl.lock.Lock() + + // refuse to queue items when the request is ofline + if !rl.open { + // don't hold block memory if we're dropping these + freeList(items) + rl.lock.Unlock() + return + } + + buffered := rl.remoteQueue.queue(items) + rl.signal.Signal() + rl.lock.Unlock() + + log.Debugw("injested blocks for new response", "request_id", rl.requestID, "total_queued_bytes", buffered) +} diff --git a/requestmanager/reconciledloader/load.go b/requestmanager/reconciledloader/load.go new file mode 100644 index 00000000..2f92f960 --- /dev/null +++ b/requestmanager/reconciledloader/load.go @@ -0,0 +1,184 @@ +package reconciledloader + +import ( + "context" + "io/ioutil" + + "github.com/ipfs/go-graphsync" + "github.com/ipfs/go-graphsync/requestmanager/types" + "github.com/ipld/go-ipld-prime/datamodel" + "github.com/ipld/go-ipld-prime/linking" + cidlink "github.com/ipld/go-ipld-prime/linking/cid" + "go.opentelemetry.io/otel" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/trace" +) + +// BlockReadOpener synchronously loads the next block result +// as long as the request is online, it will wait for more remote items until it can load this link definitively +// once the request is offline +func (rl *ReconciledLoader) BlockReadOpener(lctx linking.LinkContext, link datamodel.Link) types.AsyncLoadResult { + if !rl.mostRecentLoadAttempt.empty() { + // since we aren't retrying the most recent load, it's time to record it in the traversal record + rl.traversalRecord.RecordNextStep( + rl.mostRecentLoadAttempt.linkContext.LinkPath.Segments(), + rl.mostRecentLoadAttempt.link.(cidlink.Link).Cid, + rl.mostRecentLoadAttempt.successful, + ) + rl.mostRecentLoadAttempt = loadAttempt{} + } + + // the private method does the actual loading, while this wrapper simply does the record keeping + usedRemote, result := rl.blockReadOpener(lctx, link) + + // now, we cache to allow a retry if we're offline + rl.mostRecentLoadAttempt.link = link + rl.mostRecentLoadAttempt.linkContext = lctx + rl.mostRecentLoadAttempt.successful = result.Err == nil + rl.mostRecentLoadAttempt.usedRemote = usedRemote + return result +} + +func (rl *ReconciledLoader) blockReadOpener(lctx linking.LinkContext, link datamodel.Link) (usedRemote bool, result types.AsyncLoadResult) { + + // catch up the remore or determine that we are offline + hasRemoteData, err := rl.waitRemote() + if err != nil { + return false, types.AsyncLoadResult{Err: err, Local: !hasRemoteData} + } + + // if we're offline just load local + if !hasRemoteData { + return false, rl.loadLocal(lctx, link) + } + + // only attempt remote load if after reconciliation we're not on a missing path + if !rl.pathTracker.stillOnMissingRemotePath(lctx.LinkPath) { + data, err := rl.loadRemote(lctx, link) + if data != nil { + return true, types.AsyncLoadResult{Data: data, Local: false} + } + if err != nil { + return true, types.AsyncLoadResult{Err: err, Local: false} + } + } + // remote had missing or duplicate block, attempt load local + return true, rl.loadLocal(lctx, link) +} + +func (rl *ReconciledLoader) loadLocal(lctx linking.LinkContext, link datamodel.Link) types.AsyncLoadResult { + stream, err := rl.lsys.StorageReadOpener(lctx, link) + if err != nil { + return types.AsyncLoadResult{Err: graphsync.RemoteMissingBlockErr{Link: link, Path: lctx.LinkPath}, Local: true} + } + // skip a stream copy if it's not needed + if br, ok := stream.(byteReader); ok { + return types.AsyncLoadResult{Data: br.Bytes(), Local: true} + } + localData, err := ioutil.ReadAll(stream) + if err != nil { + return types.AsyncLoadResult{Err: graphsync.RemoteMissingBlockErr{Link: link, Path: lctx.LinkPath}, Local: true} + } + return types.AsyncLoadResult{Data: localData, Local: true} +} + +func (rl *ReconciledLoader) loadRemote(lctx linking.LinkContext, link datamodel.Link) ([]byte, error) { + rl.lock.Lock() + head := rl.remoteQueue.first() + buffered := rl.remoteQueue.consume() + rl.lock.Unlock() + + // verify it matches the expected next load + if !head.link.Equals(link.(cidlink.Link).Cid) { + return nil, graphsync.RemoteIncorrectResponseError{ + LocalLink: link, + RemoteLink: cidlink.Link{Cid: head.link}, + Path: lctx.LinkPath, + } + } + + // update path tracking + rl.pathTracker.recordRemoteLoadAttempt(lctx.LinkPath, head.action) + + // if block == nil, + // it can mean: + // - metadata had a Missing Action (Block is missing) + // - metadata had a Present Action but no block data in message + // - block appeared twice in metadata for a single message. During + // InjestResponse we decided to hold on to block data only for the + // first metadata instance + // Regardless, when block == nil, we need to simply try to load form local + // datastore + if head.block == nil { + return nil, nil + } + + // get a context + ctx := lctx.Ctx + if ctx == nil { + ctx = context.Background() + } + + // start a span + _, span := otel.Tracer("graphsync").Start( + ctx, + "verifyBlock", + trace.WithLinks(head.traceLink), + trace.WithAttributes(attribute.String("cid", link.String()))) + defer span.End() + + log.Debugw("verified block", "request_id", rl.requestID, "total_queued_bytes", buffered) + + // save the block + buffer, committer, err := rl.lsys.StorageWriteOpener(lctx) + if err != nil { + return nil, err + } + if settable, ok := buffer.(settableWriter); ok { + err = settable.SetBytes(head.block) + } else { + _, err = buffer.Write(head.block) + } + if err != nil { + return nil, err + } + err = committer(link) + if err != nil { + return nil, err + } + + // return the block + return head.block, nil +} + +func (rl *ReconciledLoader) waitRemote() (bool, error) { + rl.lock.Lock() + defer rl.lock.Unlock() + for { + // Case 1 item is waiting + if !rl.remoteQueue.empty() { + if rl.verifier == nil || rl.verifier.Done() { + rl.verifier = nil + return true, nil + } + path := rl.verifier.CurrentPath() + head := rl.remoteQueue.first() + rl.remoteQueue.consume() + err := rl.verifier.VerifyNext(head.link, head.action != graphsync.LinkActionMissing) + if err != nil { + return true, err + } + rl.pathTracker.recordRemoteLoadAttempt(path, head.action) + continue + + } + + // Case 2 no available item and channel is closed + if !rl.open { + return false, nil + } + + // Case 3 nothing available, wait for more items + rl.signal.Wait() + } +} diff --git a/requestmanager/reconciledloader/pathtracker.go b/requestmanager/reconciledloader/pathtracker.go new file mode 100644 index 00000000..85037afd --- /dev/null +++ b/requestmanager/reconciledloader/pathtracker.go @@ -0,0 +1,41 @@ +package reconciledloader + +import ( + "github.com/ipfs/go-graphsync" + "github.com/ipld/go-ipld-prime/datamodel" +) + +// pathTracker is just a simple utility to track whether we're on a missing +// path for the remote +type pathTracker struct { + lastMissingRemotePath datamodel.Path +} + +// stillOnMissingRemotePath determines whether the next link load will be from +// a path missing from the remote +// if it won't be, based on the linear nature of selector traversals, it wipes +// the last missing state +func (pt *pathTracker) stillOnMissingRemotePath(newPath datamodel.Path) bool { + // is there a known missing path? + if pt.lastMissingRemotePath.Len() == 0 { + return false + } + // are we still on it? + if newPath.Len() <= pt.lastMissingRemotePath.Len() { + // if not, reset to no known missing remote path + pt.lastMissingRemotePath = datamodel.NewPath(nil) + return false + } + // otherwise we're on a missing path + return true +} + +// recordRemoteLoadAttempt records the results of attempting to load from the remote +// at the given path +func (pt *pathTracker) recordRemoteLoadAttempt(currentPath datamodel.Path, action graphsync.LinkAction) { + // if the last remote link was missing + if action == graphsync.LinkActionMissing { + // record the last known missing path + pt.lastMissingRemotePath = currentPath + } +} diff --git a/requestmanager/reconciledloader/reconciledloader.go b/requestmanager/reconciledloader/reconciledloader.go new file mode 100644 index 00000000..e74d3ee9 --- /dev/null +++ b/requestmanager/reconciledloader/reconciledloader.go @@ -0,0 +1,124 @@ +/* +Package reconciledloader implements a block loader that can load from two different sources: +- a local store +- a series of remote responses for a given graphsync selector query + +It verifies the sequence of remote responses matches the sequence +of loads called from a local selector traversal. + +The reconciled loader also tracks whether or not there is a remote request in progress. + +When there is no request in progress, it loads from the local store only. + +When there is a request in progress, waits for remote responses before loading, and only calls +upon the local store for duplicate blocks and when traversing paths the remote was missing. + +The reconciled loader assumes: +1. A single thread is calling AsyncLoad to load blocks +2. When a request is online, a seperate thread may call IngestResponse +3. Either thread may call SetRemoteState or Cleanup +4. The remote sends metadata for all blocks it traverses in the query (per GraphSync protocol spec) - whether or not +the actual block is sent. +*/ +package reconciledloader + +import ( + "context" + "errors" + "sync" + + "github.com/ipfs/go-graphsync" + "github.com/ipfs/go-graphsync/requestmanager/reconciledloader/traversalrecord" + "github.com/ipfs/go-graphsync/requestmanager/types" + logging "github.com/ipfs/go-log/v2" + "github.com/ipld/go-ipld-prime/datamodel" + "github.com/ipld/go-ipld-prime/linking" +) + +var log = logging.Logger("gs-reconciledlaoder") + +type settableWriter interface { + SetBytes([]byte) error +} + +type byteReader interface { + Bytes() []byte +} + +type loadAttempt struct { + link datamodel.Link + linkContext linking.LinkContext + successful bool + usedRemote bool +} + +func (lr loadAttempt) empty() bool { + return lr.link == nil +} + +// ReconciledLoader is an instance of the reconciled loader +type ReconciledLoader struct { + requestID graphsync.RequestID + lsys *linking.LinkSystem + mostRecentLoadAttempt loadAttempt + traversalRecord *traversalrecord.TraversalRecord + pathTracker pathTracker + + lock *sync.Mutex + signal *sync.Cond + open bool + verifier *traversalrecord.Verifier + remoteQueue remoteQueue +} + +// NewReconciledLoader returns a new reconciled loader for the given requestID & localStore +func NewReconciledLoader(requestID graphsync.RequestID, localStore *linking.LinkSystem) *ReconciledLoader { + lock := &sync.Mutex{} + traversalRecord := traversalrecord.NewTraversalRecord() + return &ReconciledLoader{ + requestID: requestID, + lsys: localStore, + lock: lock, + signal: sync.NewCond(lock), + traversalRecord: traversalRecord, + } +} + +// SetRemoteState records whether or not the request is online +func (rl *ReconciledLoader) SetRemoteOnline(online bool) { + rl.lock.Lock() + defer rl.lock.Unlock() + wasOpen := rl.open + rl.open = online + if !rl.open && wasOpen { + // if the queue is closing, trigger any expecting new items + rl.signal.Signal() + return + } + if rl.open && !wasOpen { + // if we're opening a remote request, we need to reverify against what we've loaded so far + rl.verifier = traversalrecord.NewVerifier(rl.traversalRecord) + } +} + +// Cleanup frees up some memory resources for this loader prior to throwing it away +func (rl *ReconciledLoader) Cleanup(ctx context.Context) { + rl.lock.Lock() + rl.remoteQueue.clear() + rl.lock.Unlock() +} + +// RetryLastLoad retries the last offline load, assuming one is present +func (rl *ReconciledLoader) RetryLastLoad() types.AsyncLoadResult { + if rl.mostRecentLoadAttempt.link == nil { + return types.AsyncLoadResult{Err: errors.New("cannot retry offline load when none is present")} + } + retryLoadAttempt := rl.mostRecentLoadAttempt + rl.mostRecentLoadAttempt = loadAttempt{} + if retryLoadAttempt.usedRemote { + rl.lock.Lock() + rl.remoteQueue.retryLast() + rl.lock.Unlock() + } + return rl.BlockReadOpener(retryLoadAttempt.linkContext, retryLoadAttempt.link) +} diff --git a/requestmanager/reconciledloader/reconciledloader_test.go b/requestmanager/reconciledloader/reconciledloader_test.go new file mode 100644 index 00000000..a27ec70f --- /dev/null +++ b/requestmanager/reconciledloader/reconciledloader_test.go @@ -0,0 +1,628 @@ +package reconciledloader_test + +import ( + "bytes" + "context" + "testing" + "time" + + blocks "github.com/ipfs/go-block-format" + "github.com/ipfs/go-cid" + "github.com/ipfs/go-graphsync" + "github.com/ipfs/go-graphsync/ipldutil" + "github.com/ipfs/go-graphsync/message" + "github.com/ipfs/go-graphsync/requestmanager/reconciledloader" + "github.com/ipfs/go-graphsync/requestmanager/types" + "github.com/ipfs/go-graphsync/testutil" + "github.com/ipld/go-ipld-prime" + "github.com/ipld/go-ipld-prime/datamodel" + cidlink "github.com/ipld/go-ipld-prime/linking/cid" + selectorparse "github.com/ipld/go-ipld-prime/traversal/selector/parse" + "github.com/stretchr/testify/require" + "go.opentelemetry.io/otel/trace" +) + +func TestReconciledLoader(t *testing.T) { + ctx := context.Background() + testBCStorage := make(map[datamodel.Link][]byte) + bcLinkSys := testutil.NewTestStore(testBCStorage) + testChain := testutil.SetupBlockChain(ctx, t, bcLinkSys, 100, 100) + testTree := testutil.NewTestIPLDTree() + testCases := map[string]struct { + root cid.Cid + baseStore map[datamodel.Link][]byte + presentRemoteBlocks []blocks.Block + presentLocalBlocks []blocks.Block + remoteSeq []message.GraphSyncLinkMetadatum + steps []step + }{ + "load entirely from local store": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentLocalBlocks: testChain.AllBlocks(), + steps: syncLoadRange(testChain, 0, 100, true), + }, + "load entirely from remote store": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentRemoteBlocks: testChain.AllBlocks(), + remoteSeq: metadataRange(testChain, 0, 100, false), + steps: append([]step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 100}, + }, syncLoadRange(testChain, 0, 100, false)...), + }, + "load from local store, then go online": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentLocalBlocks: testChain.Blocks(0, 50), + presentRemoteBlocks: testChain.Blocks(50, 100), + remoteSeq: metadataRange(testChain, 0, 100, false), + steps: append(append( + // load first 50 locally + syncLoadRange(testChain, 0, 50, true), + []step{ + // should fail next because it's not stored locally + syncLoad{ + loadSeq: 50, + expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{Link: testChain.LinkTipIndex(50), Path: testChain.PathTipIndex(50)}}, + }, + // go online + goOnline{}, + // retry now that we're online -- note this won't return until we injest responses + asyncRetry{}, + // injest responses from remote peer + injest{ + metadataStart: 0, + metadataEnd: 100, + }, + // verify the retry worked + verifyAsyncResult{ + expectedResult: types.AsyncLoadResult{Local: false, Data: testChain.Blocks(50, 51)[0].RawData()}, + }, + }...), + // verify we can load the remaining items from the remote + syncLoadRange(testChain, 51, 100, false)...), + }, + "retry while offline": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentLocalBlocks: testChain.Blocks(0, 50), + steps: append( + // load first 50 locally + syncLoadRange(testChain, 0, 50, true), + []step{ + // should fail next because it's not stored locally + syncLoad{ + loadSeq: 50, + expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{Link: testChain.LinkTipIndex(50), Path: testChain.PathTipIndex(50)}}, + }, + retry{ + expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{Link: testChain.LinkTipIndex(50), Path: testChain.PathTipIndex(50)}}, + }, + }...), + }, + "retry while online": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentRemoteBlocks: testChain.AllBlocks(), + remoteSeq: metadataRange(testChain, 0, 100, false), + steps: append(append([]step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 100}, + }, + syncLoadRange(testChain, 0, 50, false)...), + retry{ + expectedResult: types.AsyncLoadResult{Data: testChain.Blocks(49, 50)[0].RawData(), Local: true}, + }), + }, + "retry online load after going offline": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentRemoteBlocks: testChain.AllBlocks(), + remoteSeq: metadataRange(testChain, 0, 100, false), + steps: append(append([]step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 100}, + }, + syncLoadRange(testChain, 0, 50, false)...), + goOffline{}, + retry{ + expectedResult: types.AsyncLoadResult{Data: testChain.Blocks(49, 50)[0].RawData(), Local: true}, + }), + }, + "error reconciling local results": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentLocalBlocks: testChain.Blocks(0, 50), + presentRemoteBlocks: testChain.Blocks(50, 100), + remoteSeq: append(append(metadataRange(testChain, 0, 30, false), + message.GraphSyncLinkMetadatum{ + Link: testChain.LinkTipIndex(53).(cidlink.Link).Cid, + Action: graphsync.LinkActionPresent, + }), + metadataRange(testChain, 31, 100, false)...), + steps: append( + // load first 50 locally + syncLoadRange(testChain, 0, 50, true), + []step{ + // should fail next because it's not stored locally + syncLoad{ + loadSeq: 50, + expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{Link: testChain.LinkTipIndex(50), Path: testChain.PathTipIndex(50)}}, + }, + // go online + goOnline{}, + // retry now that we're online -- note this won't return until we injest responses + asyncRetry{}, + // injest responses from remote peer + injest{ + metadataStart: 0, + metadataEnd: 100, + }, + // we should get an error cause of issues reconciling against previous local log + verifyAsyncResult{ + expectedResult: types.AsyncLoadResult{Local: false, Err: graphsync.RemoteIncorrectResponseError{ + LocalLink: testChain.LinkTipIndex(30), + RemoteLink: testChain.LinkTipIndex(53), + Path: testChain.PathTipIndex(30), + }}, + }, + }...), + }, + "remote sends out of order block": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentRemoteBlocks: testChain.AllBlocks(), + remoteSeq: append(append(metadataRange(testChain, 0, 30, false), + message.GraphSyncLinkMetadatum{ + Link: testChain.LinkTipIndex(53).(cidlink.Link).Cid, + Action: graphsync.LinkActionPresent, + }), + metadataRange(testChain, 31, 100, false)...), + steps: append(append([]step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 100}, + }, syncLoadRange(testChain, 0, 30, false)...), + // we should get an error cause the remote sent and incorrect response + syncLoad{ + loadSeq: 30, + expectedResult: types.AsyncLoadResult{Local: false, Err: graphsync.RemoteIncorrectResponseError{ + LocalLink: testChain.LinkTipIndex(30), + RemoteLink: testChain.LinkTipIndex(53), + Path: testChain.PathTipIndex(30), + }}, + }, + ), + }, + "remote missing block": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentRemoteBlocks: testChain.AllBlocks(), + remoteSeq: append(metadataRange(testChain, 0, 30, false), + message.GraphSyncLinkMetadatum{ + Link: testChain.LinkTipIndex(30).(cidlink.Link).Cid, + Action: graphsync.LinkActionMissing, + }), + steps: append(append([]step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 31}, + }, syncLoadRange(testChain, 0, 30, false)...), + // we should get an error that we're missing a block for our response + syncLoad{ + loadSeq: 30, + expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{ + Link: testChain.LinkTipIndex(30), + Path: testChain.PathTipIndex(30), + }}, + }, + ), + }, + "remote missing chain that local has": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentRemoteBlocks: testChain.AllBlocks(), + presentLocalBlocks: testChain.Blocks(30, 100), + remoteSeq: append(metadataRange(testChain, 0, 30, false), + message.GraphSyncLinkMetadatum{ + Link: testChain.LinkTipIndex(30).(cidlink.Link).Cid, + Action: graphsync.LinkActionMissing, + }), + steps: append(append(append( + []step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 31}, + }, + // load the blocks the remote has + syncLoadRange(testChain, 0, 30, false)...), + []step{ + // load the block the remote missing says it's missing locally + syncLoadRange(testChain, 30, 31, true)[0], + asyncLoad{loadSeq: 31}, + // at this point we have no more remote responses, since it's a linear chain + verifyNoAsyncResult{}, + // we'd expect the remote would terminate here, since we've sent the last missing block + goOffline{}, + // this will cause us to start loading locally only again + verifyAsyncResult{ + expectedResult: types.AsyncLoadResult{Local: true, Data: testChain.Blocks(31, 32)[0].RawData()}, + }, + }...), + syncLoadRange(testChain, 30, 100, true)..., + ), + }, + "remote missing chain that local has partial": { + root: testChain.TipLink.(cidlink.Link).Cid, + baseStore: testBCStorage, + presentRemoteBlocks: testChain.AllBlocks(), + presentLocalBlocks: testChain.Blocks(30, 50), + remoteSeq: append(metadataRange(testChain, 0, 30, false), + message.GraphSyncLinkMetadatum{ + Link: testChain.LinkTipIndex(30).(cidlink.Link).Cid, + Action: graphsync.LinkActionMissing, + }), + steps: append(append(append(append([]step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 31}, + }, + // load the blocks the remote has + syncLoadRange(testChain, 0, 30, false)...), + []step{ + // load the block the remote missing says it's missing locally + syncLoadRange(testChain, 30, 31, true)[0], + asyncLoad{loadSeq: 31}, + // at this point we have no more remote responses, since it's a linear chain + verifyNoAsyncResult{}, + // we'd expect the remote would terminate here, since we've sent the last missing block + goOffline{}, + // this will cause us to start loading locally only + verifyAsyncResult{ + expectedResult: types.AsyncLoadResult{Local: true, Data: testChain.Blocks(31, 32)[0].RawData()}, + }, + }...), + // should follow up to the end of the local chain + syncLoadRange(testChain, 32, 50, true)...), + // but then it should return missing + syncLoad{ + loadSeq: 50, + expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{ + Link: testChain.LinkTipIndex(50), + Path: testChain.PathTipIndex(50), + }}, + }, + ), + }, + "remote duplicate blocks can load from local": { + root: testTree.RootBlock.Cid(), + baseStore: testTree.Storage, + presentRemoteBlocks: []blocks.Block{ + testTree.RootBlock, + testTree.MiddleListBlock, + testTree.MiddleMapBlock, + testTree.LeafAlphaBlock, + testTree.LeafBetaBlock, + }, + presentLocalBlocks: nil, + remoteSeq: []message.GraphSyncLinkMetadatum{ + {Link: testTree.RootBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.MiddleListBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.LeafAlphaBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.LeafAlphaBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.LeafBetaBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.LeafAlphaBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.MiddleMapBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.LeafAlphaBlock.Cid(), Action: graphsync.LinkActionPresent}, + }, + steps: []step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 8}, + syncLoad{loadSeq: 0, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.RootBlock.RawData()}}, + syncLoad{loadSeq: 1, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.MiddleListBlock.RawData()}}, + syncLoad{loadSeq: 2, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.LeafAlphaBlock.RawData()}}, + syncLoad{loadSeq: 3, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.LeafAlphaBlock.RawData()}}, + syncLoad{loadSeq: 4, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.LeafBetaBlock.RawData()}}, + syncLoad{loadSeq: 5, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.LeafAlphaBlock.RawData()}}, + syncLoad{loadSeq: 6, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.MiddleMapBlock.RawData()}}, + syncLoad{loadSeq: 7, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.LeafAlphaBlock.RawData()}}, + }, + }, + "remote missing branch finishes to end": { + root: testTree.RootBlock.Cid(), + baseStore: testTree.Storage, + presentRemoteBlocks: []blocks.Block{ + testTree.RootBlock, + testTree.MiddleMapBlock, + testTree.LeafAlphaBlock, + }, + presentLocalBlocks: nil, + remoteSeq: []message.GraphSyncLinkMetadatum{ + {Link: testTree.RootBlock.Cid(), Action: graphsync.LinkActionPresent}, + // missing the whole list tree + {Link: testTree.MiddleListBlock.Cid(), Action: graphsync.LinkActionMissing}, + {Link: testTree.MiddleMapBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.LeafAlphaBlock.Cid(), Action: graphsync.LinkActionPresent}, + }, + steps: []step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 4}, + syncLoad{loadSeq: 0, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.RootBlock.RawData()}}, + syncLoad{loadSeq: 1, expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{Link: testTree.MiddleListNodeLnk, Path: datamodel.ParsePath("linkedList")}}}, + syncLoad{loadSeq: 6, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.MiddleMapBlock.RawData()}}, + syncLoad{loadSeq: 7, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.LeafAlphaBlock.RawData()}}, + }, + }, + "remote missing branch with partial local": { + root: testTree.RootBlock.Cid(), + baseStore: testTree.Storage, + presentLocalBlocks: []blocks.Block{ + testTree.MiddleListBlock, + testTree.LeafAlphaBlock, + }, + presentRemoteBlocks: []blocks.Block{ + testTree.RootBlock, + testTree.MiddleMapBlock, + testTree.LeafAlphaBlock, + }, + remoteSeq: []message.GraphSyncLinkMetadatum{ + {Link: testTree.RootBlock.Cid(), Action: graphsync.LinkActionPresent}, + // missing the whole list tree + {Link: testTree.MiddleListBlock.Cid(), Action: graphsync.LinkActionMissing}, + {Link: testTree.MiddleMapBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.LeafAlphaBlock.Cid(), Action: graphsync.LinkActionPresent}, + }, + steps: []step{ + goOnline{}, + injest{metadataStart: 0, metadataEnd: 4}, + syncLoad{loadSeq: 0, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.RootBlock.RawData()}}, + syncLoad{loadSeq: 1, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.MiddleListBlock.RawData()}}, + syncLoad{loadSeq: 2, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.LeafAlphaBlock.RawData()}}, + syncLoad{loadSeq: 3, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.LeafAlphaBlock.RawData()}}, + syncLoad{ + loadSeq: 4, + expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{Link: testTree.LeafBetaLnk, Path: datamodel.NewPath([]datamodel.PathSegment{ + datamodel.PathSegmentOfString("linkedList"), + datamodel.PathSegmentOfInt(2), + })}}, + }, + syncLoad{loadSeq: 5, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.LeafAlphaBlock.RawData()}}, + syncLoad{loadSeq: 6, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.MiddleMapBlock.RawData()}}, + syncLoad{loadSeq: 7, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.LeafAlphaBlock.RawData()}}, + }, + }, + "remote missing branch during reconciliation": { + root: testTree.RootBlock.Cid(), + baseStore: testTree.Storage, + presentLocalBlocks: []blocks.Block{ + testTree.RootBlock, + testTree.MiddleListBlock, + testTree.LeafAlphaBlock, + }, + presentRemoteBlocks: []blocks.Block{ + testTree.RootBlock, + testTree.MiddleMapBlock, + testTree.LeafAlphaBlock, + }, + remoteSeq: []message.GraphSyncLinkMetadatum{ + {Link: testTree.RootBlock.Cid(), Action: graphsync.LinkActionPresent}, + // missing the whole list tree + {Link: testTree.MiddleListBlock.Cid(), Action: graphsync.LinkActionMissing}, + {Link: testTree.MiddleMapBlock.Cid(), Action: graphsync.LinkActionPresent}, + {Link: testTree.LeafAlphaBlock.Cid(), Action: graphsync.LinkActionPresent}, + }, + steps: []step{ + syncLoad{loadSeq: 0, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.RootBlock.RawData()}}, + syncLoad{loadSeq: 1, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.MiddleListBlock.RawData()}}, + syncLoad{loadSeq: 2, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.LeafAlphaBlock.RawData()}}, + syncLoad{loadSeq: 3, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.LeafAlphaBlock.RawData()}}, + // here we have an offline load that is missing the local beta block + syncLoad{ + loadSeq: 4, + expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{Link: testTree.LeafBetaLnk, Path: datamodel.NewPath([]datamodel.PathSegment{ + datamodel.PathSegmentOfString("linkedList"), + datamodel.PathSegmentOfInt(2), + })}}, + }, + goOnline{}, + injest{metadataStart: 0, metadataEnd: 4}, + // what we want to verify here is that when we retry loading, the reconciliation still works, + // even though the remote is missing a brnach that's farther up the tree + retry{ + expectedResult: types.AsyncLoadResult{Local: true, Err: graphsync.RemoteMissingBlockErr{Link: testTree.LeafBetaLnk, Path: datamodel.NewPath([]datamodel.PathSegment{ + datamodel.PathSegmentOfString("linkedList"), + datamodel.PathSegmentOfInt(2), + })}}, + }, + syncLoad{loadSeq: 5, expectedResult: types.AsyncLoadResult{Local: true, Data: testTree.LeafAlphaBlock.RawData()}}, + syncLoad{loadSeq: 6, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.MiddleMapBlock.RawData()}}, + syncLoad{loadSeq: 7, expectedResult: types.AsyncLoadResult{Local: false, Data: testTree.LeafAlphaBlock.RawData()}}, + }, + }, + } + + for testCase, data := range testCases { + t.Run(testCase, func(t *testing.T) { + ctx, cancel := context.WithTimeout(ctx, 5*time.Second) + defer cancel() + localStorage := make(map[datamodel.Link][]byte) + for _, lb := range data.presentLocalBlocks { + localStorage[cidlink.Link{Cid: lb.Cid()}] = lb.RawData() + } + localLsys := testutil.NewTestStore(localStorage) + requestID := graphsync.NewRequestID() + + remoteStorage := make(map[cid.Cid][]byte) + for _, rb := range data.presentRemoteBlocks { + remoteStorage[rb.Cid()] = rb.RawData() + } + + // collect sequence of an explore all + var loadSeq []loadRequest + traverser := ipldutil.TraversalBuilder{ + Root: cidlink.Link{Cid: data.root}, + Selector: selectorparse.CommonSelector_ExploreAllRecursively, + }.Start(ctx) + for { + isComplete, err := traverser.IsComplete() + require.NoError(t, err) + if isComplete { + break + } + lnk, linkCtx := traverser.CurrentRequest() + loadSeq = append(loadSeq, loadRequest{linkCtx: linkCtx, link: lnk}) + traverser.Advance(bytes.NewReader(data.baseStore[lnk])) + } + ts := &testState{ + ctx: ctx, + remoteBlocks: remoteStorage, + remoteSeq: data.remoteSeq, + loadSeq: loadSeq, + asyncLoad: nil, + } + + rl := reconciledloader.NewReconciledLoader(requestID, &localLsys) + for _, step := range data.steps { + step.execute(t, ts, rl) + } + }) + } +} + +type loadRequest struct { + linkCtx ipld.LinkContext + link ipld.Link +} + +type testState struct { + ctx context.Context + remoteBlocks map[cid.Cid][]byte + remoteSeq []message.GraphSyncLinkMetadatum + loadSeq []loadRequest + asyncLoad <-chan types.AsyncLoadResult +} + +type step interface { + execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) +} + +type goOffline struct{} + +func (goOffline) execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) { + rl.SetRemoteOnline(false) +} + +type goOnline struct{} + +func (goOnline) execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) { + rl.SetRemoteOnline(true) +} + +type syncLoad struct { + loadSeq int + expectedResult types.AsyncLoadResult +} + +func (s syncLoad) execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) { + require.Nil(t, ts.asyncLoad) + result := rl.BlockReadOpener(ts.loadSeq[s.loadSeq].linkCtx, ts.loadSeq[s.loadSeq].link) + require.Equal(t, s.expectedResult, result) +} + +type retry struct { + expectedResult types.AsyncLoadResult +} + +func (s retry) execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) { + require.Nil(t, ts.asyncLoad) + result := rl.RetryLastLoad() + require.Equal(t, s.expectedResult, result) +} + +type asyncLoad struct { + loadSeq int +} + +func (s asyncLoad) execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) { + require.Nil(t, ts.asyncLoad) + asyncLoad := make(chan types.AsyncLoadResult, 1) + ts.asyncLoad = asyncLoad + go func() { + result := rl.BlockReadOpener(ts.loadSeq[s.loadSeq].linkCtx, ts.loadSeq[s.loadSeq].link) + asyncLoad <- result + }() +} + +type asyncRetry struct { +} + +func (s asyncRetry) execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) { + require.Nil(t, ts.asyncLoad) + asyncLoad := make(chan types.AsyncLoadResult, 1) + ts.asyncLoad = asyncLoad + go func() { + result := rl.RetryLastLoad() + asyncLoad <- result + }() +} + +type verifyNoAsyncResult struct{} + +func (verifyNoAsyncResult) execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) { + require.NotNil(t, ts.asyncLoad) + time.Sleep(20 * time.Millisecond) + select { + case <-ts.asyncLoad: + require.FailNow(t, "should have no async load result but does") + default: + } +} + +type verifyAsyncResult struct { + expectedResult types.AsyncLoadResult +} + +func (v verifyAsyncResult) execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) { + require.NotNil(t, ts.asyncLoad) + select { + case <-ts.ctx.Done(): + require.FailNow(t, "expected async load but failed") + case result := <-ts.asyncLoad: + ts.asyncLoad = nil + require.Equal(t, v.expectedResult, result) + } +} + +type injest struct { + metadataStart int + metadataEnd int + traceLink trace.Link +} + +func (i injest) execute(t *testing.T, ts *testState, rl *reconciledloader.ReconciledLoader) { + linkMetadata := ts.remoteSeq[i.metadataStart:i.metadataEnd] + rl.IngestResponse(message.NewLinkMetadata(linkMetadata), i.traceLink, ts.remoteBlocks) + // simulate no dub blocks + for _, lmd := range linkMetadata { + delete(ts.remoteBlocks, lmd.Link) + } +} + +func syncLoadRange(tbc *testutil.TestBlockChain, from int, to int, local bool) []step { + blocks := tbc.Blocks(from, to) + steps := make([]step, 0, len(blocks)) + for i := from; i < to; i++ { + steps = append(steps, syncLoad{i, types.AsyncLoadResult{Data: blocks[i-from].RawData(), Local: local}}) + } + return steps +} + +func metadataRange(tbc *testutil.TestBlockChain, from int, to int, missing bool) []message.GraphSyncLinkMetadatum { + tm := make([]message.GraphSyncLinkMetadatum, 0, to-from) + for i := from; i < to; i++ { + action := graphsync.LinkActionPresent + if missing { + action = graphsync.LinkActionMissing + } + tm = append(tm, message.GraphSyncLinkMetadatum{Link: tbc.LinkTipIndex(i).(cidlink.Link).Cid, Action: action}) + } + return tm +} diff --git a/requestmanager/reconciledloader/remotequeue.go b/requestmanager/reconciledloader/remotequeue.go new file mode 100644 index 00000000..8cf31f8b --- /dev/null +++ b/requestmanager/reconciledloader/remotequeue.go @@ -0,0 +1,121 @@ +package reconciledloader + +import ( + "sync" + + "github.com/ipfs/go-cid" + "github.com/ipfs/go-graphsync" + "go.opentelemetry.io/otel/trace" +) + +var linkedRemoteItemPool = sync.Pool{ + New: func() interface{} { + return new(remotedLinkedItem) + }, +} + +type remoteItem struct { + link cid.Cid + action graphsync.LinkAction + block []byte + traceLink trace.Link +} + +type remotedLinkedItem struct { + remoteItem + next *remotedLinkedItem +} + +func newRemote() *remotedLinkedItem { + newItem := linkedRemoteItemPool.Get().(*remotedLinkedItem) + // need to reset next value to nil we're pulling out of a pool of potentially + // old objects + newItem.next = nil + return newItem +} + +func freeList(remoteItems []*remotedLinkedItem) { + for _, ri := range remoteItems { + ri.block = nil + linkedRemoteItemPool.Put(ri) + } +} + +type remoteQueue struct { + head *remotedLinkedItem + tail *remotedLinkedItem + dataSize uint64 + // we hold a reference to the last consumed item in order to + // allow us to retry while online + lastConsumed *remotedLinkedItem +} + +func (rq *remoteQueue) empty() bool { + return rq.head == nil +} + +func (rq *remoteQueue) first() remoteItem { + if rq.head == nil { + return remoteItem{} + } + + return rq.head.remoteItem +} + +// retry last will put the last consumed item back in the queue at the front +func (rq *remoteQueue) retryLast() { + if rq.lastConsumed != nil { + rq.head = rq.lastConsumed + } +} + +func (rq *remoteQueue) consume() uint64 { + // release and clear the previous last consumed item + if rq.lastConsumed != nil { + linkedRemoteItemPool.Put(rq.lastConsumed) + rq.lastConsumed = nil + } + // update our total data size buffered + rq.dataSize -= uint64(len(rq.head.block)) + // wipe the block reference -- if its been consumed, its saved + // to local store, and we don't need it - let the memory get freed + rq.head.block = nil + + // we hold the last consumed, minus the block, around so we can retry + rq.lastConsumed = rq.head + + // advance the queue + rq.head = rq.head.next + return rq.dataSize +} + +func (rq *remoteQueue) clear() { + for rq.head != nil { + rq.consume() + } + // clear any last consumed reference left over + if rq.lastConsumed != nil { + linkedRemoteItemPool.Put(rq.lastConsumed) + rq.lastConsumed = nil + } +} + +func (rq *remoteQueue) queue(newItems []*remotedLinkedItem) uint64 { + for _, newItem := range newItems { + // update total size buffered + + // TODO: this is a good place to hold off on accepting data + // to let the local traversal catch up + // a second enqueue/dequeue signal would allow us + // to make this call block until datasize dropped below a certain amount + rq.dataSize += uint64(len(newItem.block)) + if rq.head == nil { + rq.tail = newItem + rq.head = rq.tail + } else { + rq.tail.next = newItem + rq.tail = rq.tail.next + } + } + return rq.dataSize +} diff --git a/requestmanager/reconciledloader/traversalrecord/traversalrecord.go b/requestmanager/reconciledloader/traversalrecord/traversalrecord.go new file mode 100644 index 00000000..9caba96f --- /dev/null +++ b/requestmanager/reconciledloader/traversalrecord/traversalrecord.go @@ -0,0 +1,181 @@ +package traversalrecord + +import ( + "errors" + + "github.com/ipfs/go-cid" + "github.com/ipfs/go-graphsync" + "github.com/ipld/go-ipld-prime/datamodel" + cidlink "github.com/ipld/go-ipld-prime/linking/cid" +) + +// TraversalRecord records the links traversed by a selector and their paths in a space efficient manner +type TraversalRecord struct { + link *cid.Cid + successful bool + childSegments map[datamodel.PathSegment]int + children []*traversalLink +} + +type traversalLink struct { + segment datamodel.PathSegment + *TraversalRecord +} + +// NewTraversalRecord returns a new traversal record +func NewTraversalRecord() *TraversalRecord { + return &TraversalRecord{ + childSegments: make(map[datamodel.PathSegment]int), + } +} + +// RecordNextStep records the next step in the traversal into the tree +// based on its path, link, and whether the load was successful or not +func (tr *TraversalRecord) RecordNextStep(p []datamodel.PathSegment, link cid.Cid, successful bool) { + if len(p) == 0 { + tr.link = &link + tr.successful = successful + return + } + if _, ok := tr.childSegments[p[0]]; !ok { + child := traversalLink{ + TraversalRecord: NewTraversalRecord(), + segment: p[0], + } + tr.childSegments[p[0]] = len(tr.children) + tr.children = append(tr.children, &child) + } + tr.children[tr.childSegments[p[0]]].RecordNextStep(p[1:], link, successful) +} + +// AllLinks returns all links traversed for a given record +func (tr *TraversalRecord) AllLinks() []cid.Cid { + if len(tr.children) == 0 { + return []cid.Cid{*tr.link} + } + links := make([]cid.Cid, 0) + if tr.link != nil { + links = append(links, *tr.link) + } + for _, v := range tr.children { + links = append(links, v.AllLinks()...) + } + return links +} + +// GetLinks returns all links starting at the path in the tree rooted at 'root' +func (tr *TraversalRecord) GetLinks(root datamodel.Path) []cid.Cid { + segs := root.Segments() + switch len(segs) { + case 0: + if tr.link != nil { + return []cid.Cid{*tr.link} + } + return []cid.Cid{} + case 1: + // base case 1: get all paths below this child. + next := segs[0] + if childIndex, ok := tr.childSegments[next]; ok { + return tr.children[childIndex].AllLinks() + } + return []cid.Cid{} + default: + } + + next := segs[0] + if _, ok := tr.childSegments[next]; !ok { + // base case 2: not registered sub-path. + return []cid.Cid{} + } + return tr.children[tr.childSegments[next]].GetLinks(datamodel.NewPathNocopy(segs[1:])) +} + +// Verifier allows you to verify series of links loads matches a previous traversal +// order when those loads are successful +// At any point it can reconstruct the current path. +type Verifier struct { + stack []*traversalLink +} + +func NewVerifier(root *TraversalRecord) *Verifier { + v := &Verifier{ + stack: []*traversalLink{{TraversalRecord: root}}, + } + v.appendUntilLink() + return v +} + +func (v *Verifier) tip() *traversalLink { + if len(v.stack) == 0 { + return nil + } + return v.stack[len(v.stack)-1] +} + +func (v *Verifier) appendUntilLink() { + for v.tip().link == nil && len(v.tip().children) > 0 { + v.stack = append(v.stack, v.tip().children[0]) + } +} + +func (v *Verifier) nextLink(exploreChildren bool) { + last := v.tip() + if len(last.children) > 0 && exploreChildren { + v.stack = append(v.stack, last.children[0]) + v.appendUntilLink() + return + } + // pop the stack + v.stack = v.stack[:len(v.stack)-1] + if len(v.stack) == 0 { + return + } + parent := v.tip() + // find this segments index + childIndex := parent.childSegments[last.segment] + // if this is the last child, parents next sibling + if childIndex == len(parent.children)-1 { + v.nextLink(false) + return + } + // otherwise go to next sibling + v.stack = append(v.stack, parent.children[childIndex+1]) + v.appendUntilLink() +} + +func (v *Verifier) CurrentPath() datamodel.Path { + if v.Done() { + return datamodel.NewPathNocopy(nil) + } + segments := make([]datamodel.PathSegment, 0, len(v.stack)-1) + for i, seg := range v.stack { + if i == 0 { + continue + } + segments = append(segments, seg.segment) + } + return datamodel.NewPathNocopy(segments) +} + +func (v *Verifier) Done() bool { + return len(v.stack) == 0 || (len(v.stack) == 1 && v.stack[0].link == nil) +} + +func (v *Verifier) VerifyNext(link cid.Cid, successful bool) error { + if v.Done() { + return errors.New("nothing left to verify") + } + next := v.tip() + if !next.link.Equals(link) { + return graphsync.RemoteIncorrectResponseError{ + LocalLink: cidlink.Link{Cid: *next.link}, + RemoteLink: cidlink.Link{Cid: link}, + Path: v.CurrentPath(), + } + } + if !next.successful && successful { + return errors.New("verifying against tree with additional data not possible") + } + v.nextLink(successful) + return nil +} diff --git a/requestmanager/reconciledloader/traversalrecord/traversalrecord_test.go b/requestmanager/reconciledloader/traversalrecord/traversalrecord_test.go new file mode 100644 index 00000000..28453c44 --- /dev/null +++ b/requestmanager/reconciledloader/traversalrecord/traversalrecord_test.go @@ -0,0 +1,215 @@ +package traversalrecord_test + +import ( + "bytes" + "context" + "errors" + "testing" + + "github.com/ipfs/go-cid" + "github.com/ipfs/go-graphsync" + "github.com/ipfs/go-graphsync/ipldutil" + "github.com/ipfs/go-graphsync/requestmanager/reconciledloader/traversalrecord" + "github.com/ipfs/go-graphsync/testutil" + "github.com/ipld/go-ipld-prime" + "github.com/ipld/go-ipld-prime/datamodel" + cidlink "github.com/ipld/go-ipld-prime/linking/cid" + "github.com/ipld/go-ipld-prime/traversal" + selectorparse "github.com/ipld/go-ipld-prime/traversal/selector/parse" + "github.com/stretchr/testify/require" +) + +func TestTraversalRecord(t *testing.T) { + testTree := testutil.NewTestIPLDTree() + + traversalRecord := buildTraversalRecord(t, testTree.Storage, testTree.RootNodeLnk) + + expectedAllLinks := []cid.Cid{ + testTree.RootBlock.Cid(), + testTree.MiddleListBlock.Cid(), + testTree.LeafAlphaBlock.Cid(), + testTree.LeafAlphaBlock.Cid(), + testTree.LeafBetaBlock.Cid(), + testTree.LeafAlphaBlock.Cid(), + testTree.MiddleMapBlock.Cid(), + testTree.LeafAlphaBlock.Cid(), + testTree.LeafAlphaBlock.Cid(), + } + require.Equal(t, expectedAllLinks, traversalRecord.AllLinks()) + + expectedListLinks := []cid.Cid{ + testTree.MiddleListBlock.Cid(), + testTree.LeafAlphaBlock.Cid(), + testTree.LeafAlphaBlock.Cid(), + testTree.LeafBetaBlock.Cid(), + testTree.LeafAlphaBlock.Cid(), + } + + require.Equal(t, expectedListLinks, traversalRecord.GetLinks(datamodel.ParsePath("linkedList"))) + + expectedMapLinks := []cid.Cid{ + testTree.MiddleMapBlock.Cid(), + testTree.LeafAlphaBlock.Cid(), + } + + require.Equal(t, expectedMapLinks, traversalRecord.GetLinks(datamodel.ParsePath("linkedMap"))) + + require.Empty(t, traversalRecord.GetLinks(datamodel.ParsePath("apples"))) +} + +type linkStep struct { + link cid.Cid + successful bool +} + +func TestVerification(t *testing.T) { + testTree := testutil.NewTestIPLDTree() + // add a missing element to the original tree + delete(testTree.Storage, testTree.LeafBetaLnk) + traversalRecord := buildTraversalRecord(t, testTree.Storage, testTree.RootNodeLnk) + + testCases := map[string]struct { + linkSequence []linkStep + expectedError error + expectedPaths []string + }{ + "normal successful verification": { + linkSequence: []linkStep{ + {testTree.RootBlock.Cid(), true}, + {testTree.MiddleListBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.LeafBetaBlock.Cid(), false}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.MiddleMapBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + }, + expectedPaths: []string{ + "", + "linkedList", + "linkedList/0", + "linkedList/1", + "linkedList/2", + "linkedList/3", + "linkedMap", + "linkedMap/nested/alink", + "linkedString", + }, + }, + "successful verification with missing items": { + linkSequence: []linkStep{ + {testTree.RootBlock.Cid(), true}, + {testTree.MiddleListBlock.Cid(), false}, + {testTree.MiddleMapBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + }, + expectedPaths: []string{ + "", + "linkedList", + "linkedMap", + "linkedMap/nested/alink", + "linkedString", + }, + }, + "mismatched verification": { + linkSequence: []linkStep{ + {testTree.RootBlock.Cid(), true}, + {testTree.MiddleListBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.LeafBetaBlock.Cid(), false}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.MiddleMapBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + }, + expectedError: graphsync.RemoteIncorrectResponseError{ + LocalLink: testTree.LeafAlphaLnk, + RemoteLink: testTree.LeafBetaLnk, + Path: datamodel.NewPath([]datamodel.PathSegment{datamodel.PathSegmentOfString("linkedList"), datamodel.PathSegmentOfInt(1)}), + }, + expectedPaths: []string{ + "", + "linkedList", + "linkedList/0", + "linkedList/1", + }, + }, + "additional data on unsuccessful loads": { + linkSequence: []linkStep{ + {testTree.RootBlock.Cid(), true}, + {testTree.MiddleListBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.LeafAlphaBlock.Cid(), true}, + {testTree.LeafBetaBlock.Cid(), true}, + }, + expectedError: errors.New("verifying against tree with additional data not possible"), + expectedPaths: []string{ + "", + "linkedList", + "linkedList/0", + "linkedList/1", + "linkedList/2", + }, + }, + } + + for testCase, data := range testCases { + t.Run(testCase, func(t *testing.T) { + verifier := traversalrecord.NewVerifier(traversalRecord) + var actualErr error + var actualPaths []datamodel.Path + for _, step := range data.linkSequence { + require.False(t, verifier.Done()) + actualPaths = append(actualPaths, verifier.CurrentPath()) + actualErr = verifier.VerifyNext(step.link, step.successful) + if actualErr != nil { + break + } + } + if data.expectedError == nil { + require.NoError(t, actualErr) + require.True(t, verifier.Done()) + } else { + require.EqualError(t, actualErr, data.expectedError.Error()) + require.False(t, verifier.Done()) + } + require.Equal(t, data.expectedPaths, toPathStrings(actualPaths)) + }) + } +} + +func buildTraversalRecord(t *testing.T, storage map[datamodel.Link][]byte, root ipld.Link) *traversalrecord.TraversalRecord { + ctx := context.Background() + traversalRecord := traversalrecord.NewTraversalRecord() + traverser := ipldutil.TraversalBuilder{ + Root: root, + Selector: selectorparse.CommonSelector_ExploreAllRecursively, + }.Start(ctx) + for { + isComplete, err := traverser.IsComplete() + require.NoError(t, err) + if isComplete { + break + } + lnk, linkCtx := traverser.CurrentRequest() + data, successful := storage[lnk] + traversalRecord.RecordNextStep(linkCtx.LinkPath.Segments(), lnk.(cidlink.Link).Cid, successful) + if successful { + traverser.Advance(bytes.NewReader(data)) + } else { + traverser.Error(traversal.SkipMe{}) + } + } + return traversalRecord +} + +func toPathStrings(paths []datamodel.Path) []string { + pathStrings := make([]string, 0, len(paths)) + for _, path := range paths { + pathStrings = append(pathStrings, path.String()) + } + return pathStrings +} diff --git a/requestmanager/requestmanager_test.go b/requestmanager/requestmanager_test.go index 6e5fce3c..1f230914 100644 --- a/requestmanager/requestmanager_test.go +++ b/requestmanager/requestmanager_test.go @@ -18,14 +18,12 @@ import ( "github.com/ipfs/go-graphsync" "github.com/ipfs/go-graphsync/dedupkey" - "github.com/ipfs/go-graphsync/donotsendfirstblocks" "github.com/ipfs/go-graphsync/listeners" gsmsg "github.com/ipfs/go-graphsync/message" "github.com/ipfs/go-graphsync/messagequeue" + "github.com/ipfs/go-graphsync/persistenceoptions" "github.com/ipfs/go-graphsync/requestmanager/executor" "github.com/ipfs/go-graphsync/requestmanager/hooks" - "github.com/ipfs/go-graphsync/requestmanager/testloader" - "github.com/ipfs/go-graphsync/requestmanager/types" "github.com/ipfs/go-graphsync/taskqueue" "github.com/ipfs/go-graphsync/testutil" ) @@ -67,13 +65,6 @@ func TestNormalSimultaneousFetch(t *testing.T) { } td.requestManager.ProcessResponses(peers[0], firstResponses, firstBlocks) - td.fal.VerifyLastProcessedBlocks(ctx, t, firstBlocks) - td.fal.VerifyLastProcessedResponses(ctx, t, map[graphsync.RequestID][]gsmsg.GraphSyncLinkMetadatum{ - requestRecords[0].gsr.ID(): firstMetadata1, - requestRecords[1].gsr.ID(): firstMetadata2, - }) - td.fal.SuccessResponseOn(peers[0], requestRecords[0].gsr.ID(), td.blockChain.AllBlocks()) - td.fal.SuccessResponseOn(peers[0], requestRecords[1].gsr.ID(), blockChain2.Blocks(0, 3)) td.blockChain.VerifyWholeChain(requestCtx, returnedResponseChan1) blockChain2.VerifyResponseRange(requestCtx, returnedResponseChan2, 0, 3) @@ -89,13 +80,6 @@ func TestNormalSimultaneousFetch(t *testing.T) { } td.requestManager.ProcessResponses(peers[0], moreResponses, moreBlocks) - td.fal.VerifyLastProcessedBlocks(ctx, t, moreBlocks) - td.fal.VerifyLastProcessedResponses(ctx, t, map[graphsync.RequestID][]gsmsg.GraphSyncLinkMetadatum{ - requestRecords[1].gsr.ID(): moreMetadata, - }) - - td.fal.SuccessResponseOn(peers[0], requestRecords[1].gsr.ID(), moreBlocks) - blockChain2.VerifyRemainder(requestCtx, returnedResponseChan2, 3) testutil.VerifyEmptyErrors(requestCtx, t, returnedErrorChan1) testutil.VerifyEmptyErrors(requestCtx, t, returnedErrorChan2) @@ -129,9 +113,6 @@ func TestCancelRequestInProgress(t *testing.T) { } td.requestManager.ProcessResponses(peers[0], firstResponses, firstBlocks) - - td.fal.SuccessResponseOn(peers[0], requestRecords[0].gsr.ID(), firstBlocks) - td.fal.SuccessResponseOn(peers[0], requestRecords[1].gsr.ID(), firstBlocks) td.blockChain.VerifyResponseRange(requestCtx1, returnedResponseChan1, 0, 3) cancel1() rr := readNNetworkRequests(requestCtx, t, td, 1)[0] @@ -146,8 +127,6 @@ func TestCancelRequestInProgress(t *testing.T) { gsmsg.NewResponse(requestRecords[1].gsr.ID(), graphsync.RequestCompletedFull, moreMetadata), } td.requestManager.ProcessResponses(peers[0], moreResponses, moreBlocks) - td.fal.SuccessResponseOn(peers[0], requestRecords[0].gsr.ID(), moreBlocks) - td.fal.SuccessResponseOn(peers[0], requestRecords[1].gsr.ID(), moreBlocks) testutil.VerifyEmptyResponse(requestCtx, t, returnedResponseChan1) td.blockChain.VerifyWholeChain(requestCtx, returnedResponseChan2) @@ -168,16 +147,6 @@ func TestCancelRequestImperativeNoMoreBlocks(t *testing.T) { defer cancel() peers := testutil.GeneratePeers(1) - postCancel := make(chan struct{}, 1) - loadPostCancel := make(chan struct{}, 1) - td.fal.OnAsyncLoad(func(graphsync.RequestID, ipld.Link, <-chan types.AsyncLoadResult) { - select { - case <-postCancel: - loadPostCancel <- struct{}{} - default: - } - }) - _, returnedErrorChan1 := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) requestRecords := readNNetworkRequests(requestCtx, t, td, 1) @@ -192,14 +161,12 @@ func TestCancelRequestImperativeNoMoreBlocks(t *testing.T) { gsmsg.NewResponse(requestRecords[0].gsr.ID(), graphsync.PartialResponse, firstMetadata), } td.requestManager.ProcessResponses(peers[0], firstResponses, firstBlocks) - td.fal.SuccessResponseOn(peers[0], requestRecords[0].gsr.ID(), firstBlocks) }() timeoutCtx, timeoutCancel := context.WithTimeout(ctx, time.Second) defer timeoutCancel() err := td.requestManager.CancelRequest(timeoutCtx, requestRecords[0].gsr.ID()) require.NoError(t, err) - postCancel <- struct{}{} rr := readNNetworkRequests(requestCtx, t, td, 1)[0] @@ -212,11 +179,6 @@ func TestCancelRequestImperativeNoMoreBlocks(t *testing.T) { require.Len(t, errors, 1) _, ok := errors[0].(graphsync.RequestClientCancelledErr) require.True(t, ok) - select { - case <-loadPostCancel: - t.Fatalf("Loaded block after cancel") - case <-requestCtx.Done(): - } } func TestCancelManagerExitsGracefully(t *testing.T) { @@ -237,7 +199,6 @@ func TestCancelManagerExitsGracefully(t *testing.T) { gsmsg.NewResponse(rr.gsr.ID(), graphsync.PartialResponse, firstMetadata), } td.requestManager.ProcessResponses(peers[0], firstResponses, firstBlocks) - td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), firstBlocks) td.blockChain.VerifyResponseRange(ctx, returnedResponseChan, 0, 3) managerCancel() @@ -247,7 +208,6 @@ func TestCancelManagerExitsGracefully(t *testing.T) { gsmsg.NewResponse(rr.gsr.ID(), graphsync.RequestCompletedFull, moreMetadata), } td.requestManager.ProcessResponses(peers[0], moreResponses, moreBlocks) - td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), moreBlocks) testutil.VerifyEmptyResponse(requestCtx, t, returnedResponseChan) testutil.VerifyEmptyErrors(requestCtx, t, returnedErrorChan) } @@ -275,6 +235,13 @@ func TestFailedRequest(t *testing.T) { td.tcm.RefuteProtected(t, peers[0]) } +/* +TODO: Delete? These tests no longer seem relevant, or at minimum need a rearchitect +- the new architecture will simply never fire a graphsync request if all of the data is +preset + +Perhaps we should put this back in as a mode? Or make the "wait to fire" and exprimental feature? + func TestLocallyFulfilledFirstRequestFailsLater(t *testing.T) { ctx := context.Background() td := newTestData(ctx, t) @@ -285,7 +252,7 @@ func TestLocallyFulfilledFirstRequestFailsLater(t *testing.T) { returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) - rr := readNNetworkRequests(requestCtx, t, td, 1)[0] + rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] // async loaded response responds immediately td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), td.blockChain.AllBlocks()) @@ -294,7 +261,7 @@ func TestLocallyFulfilledFirstRequestFailsLater(t *testing.T) { // failure comes in later over network failedResponses := []gsmsg.GraphSyncResponse{ - gsmsg.NewResponse(rr.gsr.ID(), graphsync.RequestFailedContentNotFound, nil), + gsmsg.NewResponse(rr.gsr.ID(), graphsync.RequestFailedContentNotFound), } td.requestManager.ProcessResponses(peers[0], failedResponses, nil) @@ -316,14 +283,14 @@ func TestLocallyFulfilledFirstRequestSucceedsLater(t *testing.T) { }) returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) - rr := readNNetworkRequests(requestCtx, t, td, 1)[0] + rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] // async loaded response responds immediately td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), td.blockChain.AllBlocks()) td.blockChain.VerifyWholeChain(requestCtx, returnedResponseChan) - md := metadataForBlocks(td.blockChain.AllBlocks(), graphsync.LinkActionPresent) + md := encodedMetadataForBlocks(t, td.blockChain.AllBlocks(), true) firstResponses := []gsmsg.GraphSyncResponse{ gsmsg.NewResponse(rr.gsr.ID(), graphsync.RequestCompletedFull, md), } @@ -333,6 +300,7 @@ func TestLocallyFulfilledFirstRequestSucceedsLater(t *testing.T) { testutil.VerifyEmptyErrors(ctx, t, returnedErrorChan) testutil.AssertDoesReceive(requestCtx, t, called, "response hooks called for response") } +*/ func TestRequestReturnsMissingBlocks(t *testing.T) { ctx := context.Background() @@ -351,9 +319,6 @@ func TestRequestReturnsMissingBlocks(t *testing.T) { gsmsg.NewResponse(rr.gsr.ID(), graphsync.RequestCompletedPartial, md), } td.requestManager.ProcessResponses(peers[0], firstResponses, nil) - for _, block := range td.blockChain.AllBlocks() { - td.fal.ResponseOn(peers[0], rr.gsr.ID(), cidlink.Link{Cid: block.Cid()}, types.AsyncLoadResult{Data: nil, Err: fmt.Errorf("Terrible Thing")}) - } testutil.VerifyEmptyResponse(ctx, t, returnedResponseChan) errs := testutil.CollectErrors(ctx, t, returnedErrorChan) require.NotEqual(t, len(errs), 0, "did not send errors") @@ -574,11 +539,6 @@ func TestBlockHooks(t *testing.T) { } td.requestManager.ProcessResponses(peers[0], firstResponses, firstBlocks) - td.fal.VerifyLastProcessedBlocks(ctx, t, firstBlocks) - td.fal.VerifyLastProcessedResponses(ctx, t, map[graphsync.RequestID][]gsmsg.GraphSyncLinkMetadatum{ - rr.gsr.ID(): firstMetadata, - }) - td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), firstBlocks) ur := readNNetworkRequests(requestCtx, t, td, 1)[0] receivedUpdateData, has := ur.gsr.Extension(td.extensionName1) @@ -637,11 +597,6 @@ func TestBlockHooks(t *testing.T) { expectedUpdateChan <- update } td.requestManager.ProcessResponses(peers[0], secondResponses, nextBlocks) - td.fal.VerifyLastProcessedBlocks(ctx, t, nextBlocks) - td.fal.VerifyLastProcessedResponses(ctx, t, map[graphsync.RequestID][]gsmsg.GraphSyncLinkMetadatum{ - rr.gsr.ID(): nextMetadata, - }) - td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), nextBlocks) ur = readNNetworkRequests(requestCtx, t, td, 1)[0] receivedUpdateData, has = ur.gsr.Extension(td.extensionName1) @@ -683,6 +638,8 @@ func TestOutgoingRequestHooks(t *testing.T) { defer cancel() peers := testutil.GeneratePeers(1) + alternateStore := testutil.NewTestStore(make(map[datamodel.Link][]byte)) + td.persistenceOptions.Register("chainstore", alternateStore) hook := func(p peer.ID, r graphsync.RequestData, ha graphsync.OutgoingRequestHookActions) { _, has := r.Extension(td.extensionName1) if has { @@ -709,20 +666,11 @@ func TestOutgoingRequestHooks(t *testing.T) { gsmsg.NewResponse(requestRecords[1].gsr.ID(), graphsync.RequestCompletedFull, md), } td.requestManager.ProcessResponses(peers[0], responses, td.blockChain.AllBlocks()) - td.fal.VerifyLastProcessedBlocks(ctx, t, td.blockChain.AllBlocks()) - td.fal.VerifyLastProcessedResponses(ctx, t, map[graphsync.RequestID][]gsmsg.GraphSyncLinkMetadatum{ - requestRecords[0].gsr.ID(): md, - requestRecords[1].gsr.ID(): md, - }) - td.fal.SuccessResponseOn(peers[0], requestRecords[0].gsr.ID(), td.blockChain.AllBlocks()) - td.fal.SuccessResponseOn(peers[0], requestRecords[1].gsr.ID(), td.blockChain.AllBlocks()) td.blockChain.VerifyWholeChainWithTypes(requestCtx, returnedResponseChan1) td.blockChain.VerifyWholeChain(requestCtx, returnedResponseChan2) testutil.VerifyEmptyErrors(ctx, t, returnedErrorChan1) testutil.VerifyEmptyErrors(ctx, t, returnedErrorChan2) - td.fal.VerifyStoreUsed(t, requestRecords[0].gsr.ID(), "chainstore") - td.fal.VerifyStoreUsed(t, requestRecords[1].gsr.ID(), "") } type outgoingRequestProcessingEvent struct { @@ -765,11 +713,6 @@ func TestOutgoingRequestListeners(t *testing.T) { gsmsg.NewResponse(requestRecords[0].gsr.ID(), graphsync.RequestCompletedFull, md), } td.requestManager.ProcessResponses(peers[0], responses, td.blockChain.AllBlocks()) - td.fal.VerifyLastProcessedBlocks(ctx, t, td.blockChain.AllBlocks()) - td.fal.VerifyLastProcessedResponses(ctx, t, map[graphsync.RequestID][]gsmsg.GraphSyncLinkMetadatum{ - requestRecords[0].gsr.ID(): md, - }) - td.fal.SuccessResponseOn(peers[0], requestRecords[0].gsr.ID(), td.blockChain.AllBlocks()) td.blockChain.VerifyWholeChain(requestCtx, returnedResponseChan1) testutil.VerifyEmptyErrors(requestCtx, t, returnedErrorChan1) @@ -812,7 +755,6 @@ func TestPauseResume(t *testing.T) { gsmsg.NewResponse(rr.gsr.ID(), graphsync.RequestCompletedFull, md), } td.requestManager.ProcessResponses(peers[0], responses, td.blockChain.AllBlocks()) - td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), td.blockChain.AllBlocks()) // attempt to unpause while request is not paused (note: hook on second block will keep it from // reaching pause point) @@ -831,34 +773,37 @@ func TestPauseResume(t *testing.T) { // verify no further responses come through time.Sleep(100 * time.Millisecond) testutil.AssertChannelEmpty(t, returnedResponseChan, "no response should be sent request is paused") - td.fal.CleanupRequest(peers[0], rr.gsr.ID()) // unpause err = td.requestManager.UnpauseRequest(rr.gsr.ID(), td.extension1, td.extension2) require.NoError(t, err) - // verify the correct new request with Do-no-send-cids & other extensions - resumedRequest := readNNetworkRequests(requestCtx, t, td, 1)[0] - doNotSendFirstBlocksData, has := resumedRequest.gsr.Extension(graphsync.ExtensionsDoNotSendFirstBlocks) - doNotSendFirstBlocks, err := donotsendfirstblocks.DecodeDoNotSendFirstBlocks(doNotSendFirstBlocksData) - require.NoError(t, err) - require.Equal(t, pauseAt, int(doNotSendFirstBlocks)) - require.True(t, has) - ext1Data, has := resumedRequest.gsr.Extension(td.extensionName1) - require.True(t, has) - require.Equal(t, td.extensionData1, ext1Data) - ext2Data, has := resumedRequest.gsr.Extension(td.extensionName2) - require.True(t, has) - require.Equal(t, td.extensionData2, ext2Data) + /* + TODO: these are no longer used as the old responses are consumed upon restart, to minimize + network utilization -- does this make sense? Maybe we should throw out these responses while paused? + + // verify the correct new request with Do-no-send-cids & other extensions + resumedRequest := readNNetworkRequests(requestCtx, t, td, 1)[0] + doNotSendFirstBlocksData, has := resumedRequest.gsr.Extension(graphsync.ExtensionsDoNotSendFirstBlocks) + doNotSendFirstBlocks, err := donotsendfirstblocks.DecodeDoNotSendFirstBlocks(doNotSendFirstBlocksData) + require.NoError(t, err) + require.Equal(t, pauseAt, int(doNotSendFirstBlocks)) + require.True(t, has) + ext1Data, has := resumedRequest.gsr.Extension(td.extensionName1) + require.True(t, has) + require.Equal(t, td.extensionData1, ext1Data) + ext2Data, has := resumedRequest.gsr.Extension(td.extensionName2) + require.True(t, has) + require.Equal(t, td.extensionData2, ext2Data) + */ // process responses td.requestManager.ProcessResponses(peers[0], responses, td.blockChain.RemainderBlocks(pauseAt)) - td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), td.blockChain.AllBlocks()) - // verify the correct results are returned, picking up after where there request was paused td.blockChain.VerifyRemainder(ctx, returnedResponseChan, pauseAt) testutil.VerifyEmptyErrors(ctx, t, returnedErrorChan) } + func TestPauseResumeExternal(t *testing.T) { ctx := context.Background() td := newTestData(ctx, t) @@ -893,7 +838,6 @@ func TestPauseResumeExternal(t *testing.T) { gsmsg.NewResponse(rr.gsr.ID(), graphsync.RequestCompletedFull, md), } td.requestManager.ProcessResponses(peers[0], responses, td.blockChain.AllBlocks()) - td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), td.blockChain.AllBlocks()) // verify responses sent read ONLY for blocks BEFORE the pause td.blockChain.VerifyResponseRange(ctx, returnedResponseChan, 0, pauseAt) // wait for the pause to occur @@ -906,13 +850,15 @@ func TestPauseResumeExternal(t *testing.T) { // verify no further responses come through time.Sleep(100 * time.Millisecond) testutil.AssertChannelEmpty(t, returnedResponseChan, "no response should be sent request is paused") - td.fal.CleanupRequest(peers[0], rr.gsr.ID()) // unpause err := td.requestManager.UnpauseRequest(rr.gsr.ID(), td.extension1, td.extension2) require.NoError(t, err) // verify the correct new request with Do-no-send-cids & other extensions + /* TODO: these are no longer used as the old responses are consumed upon restart, to minimize + network utilization -- does this make sense? Maybe we should throw out these responses while paused? + resumedRequest := readNNetworkRequests(requestCtx, t, td, 1)[0] doNotSendFirstBlocksData, has := resumedRequest.gsr.Extension(graphsync.ExtensionsDoNotSendFirstBlocks) doNotSendFirstBlocks, err := donotsendfirstblocks.DecodeDoNotSendFirstBlocks(doNotSendFirstBlocksData) @@ -924,11 +870,10 @@ func TestPauseResumeExternal(t *testing.T) { require.Equal(t, td.extensionData1, ext1Data) ext2Data, has := resumedRequest.gsr.Extension(td.extensionName2) require.True(t, has) - require.Equal(t, td.extensionData2, ext2Data) + require.Equal(t, td.extensionData2, ext2Data)*/ // process responses td.requestManager.ProcessResponses(peers[0], responses, td.blockChain.RemainderBlocks(pauseAt)) - td.fal.SuccessResponseOn(peers[0], rr.gsr.ID(), td.blockChain.AllBlocks()) // verify the correct results are returned, picking up after where there request was paused td.blockChain.VerifyRemainder(ctx, returnedResponseChan, pauseAt) @@ -1016,7 +961,7 @@ func metadataForBlocks(blks []blocks.Block, action graphsync.LinkAction) []gsmsg type testData struct { requestRecordChan chan requestRecord fph *fakePeerHandler - fal *testloader.FakeAsyncLoader + persistenceOptions *persistenceoptions.PersistenceOptions tcm *testutil.TestConnManager requestHooks *hooks.OutgoingRequestHooks responseHooks *hooks.IncomingResponseHooks @@ -1024,6 +969,8 @@ type testData struct { requestManager *RequestManager blockStore map[ipld.Link][]byte persistence ipld.LinkSystem + localBlockStore map[ipld.Link][]byte + localPersistence ipld.LinkSystem blockChain *testutil.TestBlockChain extensionName1 graphsync.ExtensionName extensionData1 datamodel.Node @@ -1043,7 +990,7 @@ func newTestData(ctx context.Context, t *testing.T) *testData { td := &testData{} td.requestRecordChan = make(chan requestRecord, 3) td.fph = &fakePeerHandler{td.requestRecordChan} - td.fal = testloader.NewFakeAsyncLoader() + td.persistenceOptions = persistenceoptions.New() td.tcm = testutil.NewTestConnManager() td.requestHooks = hooks.NewRequestHooks() td.responseHooks = hooks.NewResponseHooks() @@ -1051,9 +998,10 @@ func newTestData(ctx context.Context, t *testing.T) *testData { td.networkErrorListeners = listeners.NewNetworkErrorListeners() td.outgoingRequestProcessingListeners = listeners.NewOutgoingRequestProcessingListeners() td.taskqueue = taskqueue.NewTaskQueue(ctx) - lsys := cidlink.DefaultLinkSystem() - td.requestManager = New(ctx, td.fal, lsys, td.requestHooks, td.responseHooks, td.networkErrorListeners, td.outgoingRequestProcessingListeners, td.taskqueue, td.tcm, 0) - td.executor = executor.NewExecutor(td.requestManager, td.blockHooks, td.fal.AsyncLoad) + td.localBlockStore = make(map[ipld.Link][]byte) + td.localPersistence = testutil.NewTestStore(td.localBlockStore) + td.requestManager = New(ctx, td.persistenceOptions, td.localPersistence, td.requestHooks, td.responseHooks, td.networkErrorListeners, td.outgoingRequestProcessingListeners, td.taskqueue, td.tcm, 0) + td.executor = executor.NewExecutor(td.requestManager, td.blockHooks) td.requestManager.SetDelegate(td.fph) td.requestManager.Startup() td.taskqueue.Startup(6, td.executor) diff --git a/requestmanager/server.go b/requestmanager/server.go index 796a3fbd..cab8aadf 100644 --- a/requestmanager/server.go +++ b/requestmanager/server.go @@ -8,9 +8,11 @@ import ( "time" blocks "github.com/ipfs/go-block-format" + "github.com/ipfs/go-cid" "github.com/ipfs/go-peertaskqueue/peertask" "github.com/ipfs/go-peertaskqueue/peertracker" "github.com/ipld/go-ipld-prime" + "github.com/ipld/go-ipld-prime/linking" cidlink "github.com/ipld/go-ipld-prime/linking/cid" "github.com/ipld/go-ipld-prime/traversal" "github.com/ipld/go-ipld-prime/traversal/selector" @@ -28,6 +30,7 @@ import ( "github.com/ipfs/go-graphsync/peerstate" "github.com/ipfs/go-graphsync/requestmanager/executor" "github.com/ipfs/go-graphsync/requestmanager/hooks" + "github.com/ipfs/go-graphsync/requestmanager/reconciledloader" ) // The code in this file implements the internal thread for the request manager. @@ -63,7 +66,7 @@ func (rm *RequestManager) newRequest(parentSpan trace.Span, p peer.ID, root ipld log.Infow("graphsync request initiated", "request id", requestID.String(), "peer", p, "root", root) - request, hooksResult, err := rm.validateRequest(requestID, p, root, selector, extensions) + request, hooksResult, lsys, err := rm.validateRequest(requestID, p, root, selector, extensions) if err != nil { span.RecordError(err) span.SetStatus(codes.Error, err.Error()) @@ -97,6 +100,7 @@ func (rm *RequestManager) newRequest(parentSpan trace.Span, p peer.ID, root ipld nodeStyleChooser: hooksResult.CustomChooser, inProgressChan: make(chan graphsync.ResponseProgress), inProgressErr: make(chan error), + lsys: lsys, } requestStatus.lastResponse.Store(gsmsg.NewResponse(request.ID(), graphsync.RequestAcknowledged, nil)) rm.inProgressRequestStatuses[request.ID()] = requestStatus @@ -113,9 +117,7 @@ func (rm *RequestManager) requestTask(requestID graphsync.RequestID) executor.Re } log.Infow("graphsync request processing begins", "request id", requestID.String(), "peer", ipr.p, "total time", time.Since(ipr.startTime)) - var initialRequest bool if ipr.traverser == nil { - initialRequest = true var budget *traversal.Budget if rm.maxLinksPerRequest > 0 { budget = &traversal.Budget{ @@ -147,6 +149,7 @@ func (rm *RequestManager) requestTask(requestID graphsync.RequestID) executor.Re Budget: budget, }.Start(ctx) + ipr.reconciledLoader = reconciledloader.NewReconciledLoader(ipr.request.ID(), ipr.lsys) inProgressCount := len(rm.inProgressRequestStatuses) rm.outgoingRequestProcessingListeners.NotifyOutgoingRequestProcessingListeners(ipr.p, ipr.request, inProgressCount) } @@ -162,7 +165,7 @@ func (rm *RequestManager) requestTask(requestID graphsync.RequestID) executor.Re Traverser: ipr.traverser, P: ipr.p, InProgressErr: ipr.inProgressErr, - InitialRequest: initialRequest, + ReconciledLoader: ipr.reconciledLoader, Empty: false, } } @@ -190,7 +193,9 @@ func (rm *RequestManager) terminateRequest(requestID graphsync.RequestID, ipr *i rm.connManager.Unprotect(ipr.p, requestID.Tag()) delete(rm.inProgressRequestStatuses, requestID) ipr.cancelFn() - rm.asyncLoader.CleanupRequest(ipr.p, requestID) + if ipr.reconciledLoader != nil { + ipr.reconciledLoader.Cleanup(rm.ctx) + } if ipr.traverser != nil { ipr.traverserCancel() ipr.traverser.Shutdown(rm.ctx) @@ -255,7 +260,7 @@ func (rm *RequestManager) cancelOnError(requestID graphsync.RequestID, ipr *inPr rm.terminateRequest(requestID, ipr) } else { ipr.cancelFn() - rm.asyncLoader.CompleteResponsesFor(requestID) + ipr.reconciledLoader.SetRemoteOnline(false) } } @@ -271,16 +276,22 @@ func (rm *RequestManager) processResponses(p peer.ID, ctx, span := otel.Tracer("graphsync").Start(rm.ctx, "processResponses", trace.WithAttributes( attribute.String("peerID", p.Pretty()), attribute.StringSlice("requestIDs", requestIds), + attribute.Int("blockCount", len(blks)), )) defer span.End() filteredResponses := rm.processExtensions(responses, p) filteredResponses = rm.filterResponsesForPeer(filteredResponses, p) - responseMetadata := make(map[graphsync.RequestID]graphsync.LinkMetadata, len(responses)) - for _, response := range responses { - responseMetadata[response.RequestID()] = response.Metadata() + blkMap := make(map[cid.Cid][]byte, len(blks)) + for _, blk := range blks { + blkMap[blk.Cid()] = blk.RawData() + } + for _, response := range filteredResponses { + reconciledLoader := rm.inProgressRequestStatuses[response.RequestID()].reconciledLoader + if reconciledLoader != nil { + reconciledLoader.IngestResponse(response.Metadata(), trace.LinkFromContext(ctx), blkMap) + } } rm.updateLastResponses(filteredResponses) - rm.asyncLoader.ProcessResponse(ctx, responseMetadata, blks) rm.processTerminations(filteredResponses) log.Debugf("end processing responses for peer %s", p) } @@ -338,30 +349,33 @@ func (rm *RequestManager) processTerminations(responses []gsmsg.GraphSyncRespons if response.Status().IsFailure() { rm.cancelOnError(response.RequestID(), rm.inProgressRequestStatuses[response.RequestID()], response.Status().AsError()) } - rm.asyncLoader.CompleteResponsesFor(response.RequestID()) + ipr, ok := rm.inProgressRequestStatuses[response.RequestID()] + if ok && ipr.reconciledLoader != nil { + ipr.reconciledLoader.SetRemoteOnline(false) + } } } } -func (rm *RequestManager) validateRequest(requestID graphsync.RequestID, p peer.ID, root ipld.Link, selectorSpec ipld.Node, extensions []graphsync.ExtensionData) (gsmsg.GraphSyncRequest, hooks.RequestResult, error) { +func (rm *RequestManager) validateRequest(requestID graphsync.RequestID, p peer.ID, root ipld.Link, selectorSpec ipld.Node, extensions []graphsync.ExtensionData) (gsmsg.GraphSyncRequest, hooks.RequestResult, *linking.LinkSystem, error) { _, err := ipldutil.EncodeNode(selectorSpec) if err != nil { - return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err + return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, nil, err } _, err = selector.ParseSelector(selectorSpec) if err != nil { - return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err + return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, nil, err } asCidLink, ok := root.(cidlink.Link) if !ok { - return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, fmt.Errorf("request failed: link has no cid") + return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, nil, fmt.Errorf("request failed: link has no cid") } request := gsmsg.NewRequest(requestID, asCidLink.Cid, selectorSpec, defaultPriority, extensions...) hooksResult := rm.requestHooks.ProcessRequestHooks(p, request) if hooksResult.PersistenceOption != "" { dedupData, err := dedupkey.EncodeDedupKey(hooksResult.PersistenceOption) if err != nil { - return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err + return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, nil, err } request = request.ReplaceExtensions([]graphsync.ExtensionData{ { @@ -370,11 +384,15 @@ func (rm *RequestManager) validateRequest(requestID graphsync.RequestID, p peer. }, }) } - err = rm.asyncLoader.StartRequest(requestID, hooksResult.PersistenceOption) - if err != nil { - return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err + lsys := rm.linkSystem + if hooksResult.PersistenceOption != "" { + var has bool + lsys, has = rm.persistenceOptions.GetLinkSystem(hooksResult.PersistenceOption) + if !has { + return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, nil, errors.New("unknown persistence option") + } } - return request, hooksResult, nil + return request, hooksResult, &lsys, nil } func (rm *RequestManager) unpause(id graphsync.RequestID, extensions []graphsync.ExtensionData) error { diff --git a/requestmanager/testloader/asyncloader.go b/requestmanager/testloader/asyncloader.go deleted file mode 100644 index 624cda25..00000000 --- a/requestmanager/testloader/asyncloader.go +++ /dev/null @@ -1,173 +0,0 @@ -package testloader - -import ( - "context" - "sync" - "testing" - - blocks "github.com/ipfs/go-block-format" - "github.com/ipfs/go-cid" - "github.com/ipld/go-ipld-prime" - cidlink "github.com/ipld/go-ipld-prime/linking/cid" - peer "github.com/libp2p/go-libp2p-core/peer" - "github.com/stretchr/testify/require" - - "github.com/ipfs/go-graphsync" - "github.com/ipfs/go-graphsync/message" - "github.com/ipfs/go-graphsync/requestmanager/types" - "github.com/ipfs/go-graphsync/testutil" -) - -type requestKey struct { - p peer.ID - requestID graphsync.RequestID - link ipld.Link -} - -type storeKey struct { - requestID graphsync.RequestID - storeName string -} - -// FakeAsyncLoader simultates the requestmanager.AsyncLoader interface -// with mocked responses and can also be used to simulate a -// executor.AsycLoadFn -- all responses are stubbed and no actual processing is -// done -type FakeAsyncLoader struct { - responseChannelsLk sync.RWMutex - responseChannels map[requestKey]chan types.AsyncLoadResult - responses chan map[graphsync.RequestID]graphsync.LinkMetadata - blks chan []blocks.Block - storesRequestedLk sync.RWMutex - storesRequested map[storeKey]struct{} - cb func(graphsync.RequestID, ipld.Link, <-chan types.AsyncLoadResult) -} - -// NewFakeAsyncLoader returns a new FakeAsyncLoader instance -func NewFakeAsyncLoader() *FakeAsyncLoader { - return &FakeAsyncLoader{ - responseChannels: make(map[requestKey]chan types.AsyncLoadResult), - responses: make(chan map[graphsync.RequestID]graphsync.LinkMetadata, 10), - blks: make(chan []blocks.Block, 10), - storesRequested: make(map[storeKey]struct{}), - } -} - -// StartRequest just requests what store was requested for a given requestID -func (fal *FakeAsyncLoader) StartRequest(requestID graphsync.RequestID, name string) error { - fal.storesRequestedLk.Lock() - fal.storesRequested[storeKey{requestID, name}] = struct{}{} - fal.storesRequestedLk.Unlock() - return nil -} - -// ProcessResponse just records values passed to verify expectations later -func (fal *FakeAsyncLoader) ProcessResponse(_ context.Context, responses map[graphsync.RequestID]graphsync.LinkMetadata, - blks []blocks.Block) { - fal.responses <- responses - fal.blks <- blks -} - -// VerifyLastProcessedBlocks verifies the blocks passed to the last call to ProcessResponse -// match the expected ones -func (fal *FakeAsyncLoader) VerifyLastProcessedBlocks(ctx context.Context, t *testing.T, expectedBlocks []blocks.Block) { - t.Helper() - var processedBlocks []blocks.Block - testutil.AssertReceive(ctx, t, fal.blks, &processedBlocks, "did not process blocks") - require.Equal(t, expectedBlocks, processedBlocks, "did not process correct blocks") -} - -// VerifyLastProcessedResponses verifies the responses passed to the last call to ProcessResponse -// match the expected ones -func (fal *FakeAsyncLoader) VerifyLastProcessedResponses(ctx context.Context, t *testing.T, - expectedResponses map[graphsync.RequestID][]message.GraphSyncLinkMetadatum) { - t.Helper() - var responses map[graphsync.RequestID]graphsync.LinkMetadata - testutil.AssertReceive(ctx, t, fal.responses, &responses, "did not process responses") - actualResponses := make(map[graphsync.RequestID][]message.GraphSyncLinkMetadatum) - for rid, lm := range responses { - actualResponses[rid] = make([]message.GraphSyncLinkMetadatum, 0) - lm.Iterate(func(c cid.Cid, la graphsync.LinkAction) { - actualResponses[rid] = append(actualResponses[rid], - message.GraphSyncLinkMetadatum{Link: c, Action: la}) - }) - } - require.Equal(t, expectedResponses, actualResponses, "did not process correct responses") -} - -// VerifyNoRemainingData verifies no outstanding response channels are open for the given -// RequestID (CleanupRequest was called last) -func (fal *FakeAsyncLoader) VerifyNoRemainingData(t *testing.T, requestID graphsync.RequestID) { - t.Helper() - fal.responseChannelsLk.RLock() - for key := range fal.responseChannels { - require.NotEqual(t, key.requestID, requestID, "did not clean up request properly") - } - fal.responseChannelsLk.RUnlock() -} - -// VerifyStoreUsed verifies the given store was used for the given request -func (fal *FakeAsyncLoader) VerifyStoreUsed(t *testing.T, requestID graphsync.RequestID, storeName string) { - t.Helper() - fal.storesRequestedLk.RLock() - _, ok := fal.storesRequested[storeKey{requestID, storeName}] - require.True(t, ok, "request should load from correct store") - fal.storesRequestedLk.RUnlock() -} - -func (fal *FakeAsyncLoader) asyncLoad(p peer.ID, requestID graphsync.RequestID, link ipld.Link, linkContext ipld.LinkContext) chan types.AsyncLoadResult { - fal.responseChannelsLk.Lock() - responseChannel, ok := fal.responseChannels[requestKey{p, requestID, link}] - if !ok { - responseChannel = make(chan types.AsyncLoadResult, 1) - fal.responseChannels[requestKey{p, requestID, link}] = responseChannel - } - fal.responseChannelsLk.Unlock() - return responseChannel -} - -// OnAsyncLoad allows you to listen for load requests to the loader and perform other actions or tests -func (fal *FakeAsyncLoader) OnAsyncLoad(cb func(graphsync.RequestID, ipld.Link, <-chan types.AsyncLoadResult)) { - fal.cb = cb -} - -// AsyncLoad simulates an asynchronous load with responses stubbed by ResponseOn & SuccessResponseOn -func (fal *FakeAsyncLoader) AsyncLoad(p peer.ID, requestID graphsync.RequestID, link ipld.Link, linkContext ipld.LinkContext) <-chan types.AsyncLoadResult { - res := fal.asyncLoad(p, requestID, link, linkContext) - if fal.cb != nil { - fal.cb(requestID, link, res) - } - return res -} - -// CompleteResponsesFor in the case of the test loader does nothing -func (fal *FakeAsyncLoader) CompleteResponsesFor(requestID graphsync.RequestID) {} - -// CleanupRequest simulates the effect of cleaning up the request by removing any response channels -// for the request -func (fal *FakeAsyncLoader) CleanupRequest(p peer.ID, requestID graphsync.RequestID) { - fal.responseChannelsLk.Lock() - for key := range fal.responseChannels { - if key.requestID == requestID { - delete(fal.responseChannels, key) - } - } - fal.responseChannelsLk.Unlock() -} - -// ResponseOn sets the value returned when the given link is loaded for the given request. Because it's an -// "asynchronous" load, this can be called AFTER the attempt to load this link -- and the client will only get -// the response at that point -func (fal *FakeAsyncLoader) ResponseOn(p peer.ID, requestID graphsync.RequestID, link ipld.Link, result types.AsyncLoadResult) { - responseChannel := fal.asyncLoad(p, requestID, link, ipld.LinkContext{}) - responseChannel <- result - close(responseChannel) -} - -// SuccessResponseOn is convenience function for setting several asynchronous responses at once as all successes -// and returning the given blocks -func (fal *FakeAsyncLoader) SuccessResponseOn(p peer.ID, requestID graphsync.RequestID, blks []blocks.Block) { - for _, block := range blks { - fal.ResponseOn(p, requestID, cidlink.Link{Cid: block.Cid()}, types.AsyncLoadResult{Data: block.RawData(), Local: false, Err: nil}) - } -} diff --git a/responsemanager/responsemanager_test.go b/responsemanager/responsemanager_test.go index 09ea3731..c7b88fb9 100644 --- a/responsemanager/responsemanager_test.go +++ b/responsemanager/responsemanager_test.go @@ -29,8 +29,8 @@ import ( gsmsg "github.com/ipfs/go-graphsync/message" "github.com/ipfs/go-graphsync/messagequeue" "github.com/ipfs/go-graphsync/notifications" + "github.com/ipfs/go-graphsync/persistenceoptions" "github.com/ipfs/go-graphsync/responsemanager/hooks" - "github.com/ipfs/go-graphsync/responsemanager/persistenceoptions" "github.com/ipfs/go-graphsync/responsemanager/queryexecutor" "github.com/ipfs/go-graphsync/responsemanager/responseassembler" "github.com/ipfs/go-graphsync/selectorvalidator" diff --git a/testutil/testchain.go b/testutil/testchain.go index f6e15deb..01142537 100644 --- a/testutil/testchain.go +++ b/testutil/testchain.go @@ -8,6 +8,7 @@ import ( blocks "github.com/ipfs/go-block-format" cid "github.com/ipfs/go-cid" "github.com/ipld/go-ipld-prime" + "github.com/ipld/go-ipld-prime/datamodel" cidlink "github.com/ipld/go-ipld-prime/linking/cid" "github.com/ipld/go-ipld-prime/node/basicnode" "github.com/ipld/go-ipld-prime/traversal/selector" @@ -149,16 +150,19 @@ func (tbc *TestBlockChain) NodeTipIndex(fromTip int) ipld.Node { return tbc.MiddleNodes[height-1] } } + +// PathTipIndex returns the path to the block at the given index from the tip +func (tbc *TestBlockChain) PathTipIndex(fromTip int) ipld.Path { + expectedPath := make([]datamodel.PathSegment, 0, 2*fromTip) + for i := 0; i < fromTip; i++ { + expectedPath = append(expectedPath, datamodel.PathSegmentOfString("Parents"), datamodel.PathSegmentOfInt(0)) + } + return datamodel.NewPath(expectedPath) +} + func (tbc *TestBlockChain) checkResponses(responses []graphsync.ResponseProgress, start int, end int, verifyTypes bool) { require.Len(tbc.t, responses, (end-start)*blockChainTraversedNodesPerBlock, "traverses all nodes") - expectedPath := "" - for i := 0; i < start; i++ { - if expectedPath == "" { - expectedPath = "Parents/0" - } else { - expectedPath = expectedPath + "/Parents/0" - } - } + expectedPath := tbc.PathTipIndex(start).String() for i, response := range responses { require.Equal(tbc.t, expectedPath, response.Path.String(), "response has correct path") if i%2 == 0 {