From 6a9d1e9618b76d30cfba26dfdff59c5f18cc0a8d Mon Sep 17 00:00:00 2001 From: hannahhoward Date: Tue, 21 Sep 2021 12:40:56 -0700 Subject: [PATCH 1/8] refactor(graphsync): cleanup status code utilities --- message/message.go | 15 +++++------ requestmanager/requestmanager.go | 25 +++--------------- responsecode.go | 44 ++++++++++++++++++++++++++++++++ 3 files changed, 54 insertions(+), 30 deletions(-) diff --git a/message/message.go b/message/message.go index 8d269ef1..564be890 100644 --- a/message/message.go +++ b/message/message.go @@ -21,26 +21,23 @@ import ( // IsTerminalSuccessCode returns true if the response code indicates the // request terminated successfully. +// DEPRECATED: use status.IsSuccess() func IsTerminalSuccessCode(status graphsync.ResponseStatusCode) bool { - return status == graphsync.RequestCompletedFull || - status == graphsync.RequestCompletedPartial + return status.IsSuccess() } // IsTerminalFailureCode returns true if the response code indicates the // request terminated in failure. +// DEPRECATED: use status.IsFailure() func IsTerminalFailureCode(status graphsync.ResponseStatusCode) bool { - return status == graphsync.RequestFailedBusy || - status == graphsync.RequestFailedContentNotFound || - status == graphsync.RequestFailedLegal || - status == graphsync.RequestFailedUnknown || - status == graphsync.RequestCancelled || - status == graphsync.RequestRejected + return status.IsFailure() } // IsTerminalResponseCode returns true if the response code signals // the end of the request +// DEPRECATED: use status.IsTerminal() func IsTerminalResponseCode(status graphsync.ResponseStatusCode) bool { - return IsTerminalSuccessCode(status) || IsTerminalFailureCode(status) + return status.IsTerminal() } // Exportable is an interface that can serialize to a protobuf diff --git a/requestmanager/requestmanager.go b/requestmanager/requestmanager.go index 8153e98e..222b762c 100644 --- a/requestmanager/requestmanager.go +++ b/requestmanager/requestmanager.go @@ -527,7 +527,7 @@ func (rm *RequestManager) processExtensionsForResponse(p peer.ID, response gsmsg if !ok { return false } - responseError := rm.generateResponseErrorFromStatus(graphsync.RequestFailedUnknown) + responseError := graphsync.RequestFailedUnknown.AsError() select { case requestStatus.terminalError <- responseError: default: @@ -541,10 +541,10 @@ func (rm *RequestManager) processExtensionsForResponse(p peer.ID, response gsmsg func (rm *RequestManager) processTerminations(responses []gsmsg.GraphSyncResponse) { for _, response := range responses { - if gsmsg.IsTerminalResponseCode(response.Status()) { - if gsmsg.IsTerminalFailureCode(response.Status()) { + if response.Status().IsTerminal() { + if response.Status().IsFailure() { requestStatus := rm.inProgressRequestStatuses[response.RequestID()] - responseError := rm.generateResponseErrorFromStatus(response.Status()) + responseError := response.Status().AsError() select { case requestStatus.terminalError <- responseError: default: @@ -556,23 +556,6 @@ func (rm *RequestManager) processTerminations(responses []gsmsg.GraphSyncRespons } } -func (rm *RequestManager) generateResponseErrorFromStatus(status graphsync.ResponseStatusCode) error { - switch status { - case graphsync.RequestFailedBusy: - return graphsync.RequestFailedBusyErr{} - case graphsync.RequestFailedContentNotFound: - return graphsync.RequestFailedContentNotFoundErr{} - case graphsync.RequestFailedLegal: - return graphsync.RequestFailedLegalErr{} - case graphsync.RequestFailedUnknown: - return graphsync.RequestFailedUnknownErr{} - case graphsync.RequestCancelled: - return graphsync.RequestCancelledErr{} - default: - return fmt.Errorf("Unknown") - } -} - func (rm *RequestManager) processBlockHooks(p peer.ID, response graphsync.ResponseData, block graphsync.BlockData) error { result := rm.blockHooks.ProcessBlockHooks(p, response, block) if len(result.Extensions) > 0 { diff --git a/responsecode.go b/responsecode.go index 3e05ec32..c96c2127 100644 --- a/responsecode.go +++ b/responsecode.go @@ -77,3 +77,47 @@ var ResponseCodeToName = map[ResponseStatusCode]string{ RequestFailedContentNotFound: "RequestFailedContentNotFound", RequestCancelled: "RequestCancelled", } + +// AsError generates an error from the status code for a failing status +func (c ResponseStatusCode) AsError() error { + if c.IsSuccess() { + return nil + } + switch c { + case RequestFailedBusy: + return RequestFailedBusyErr{} + case RequestFailedContentNotFound: + return RequestFailedContentNotFoundErr{} + case RequestFailedLegal: + return RequestFailedLegalErr{} + case RequestFailedUnknown: + return RequestFailedUnknownErr{} + case RequestCancelled: + return RequestCancelledErr{} + default: + return fmt.Errorf("Unknown") + } +} + +// IsSuccess returns true if the response code indicates the +// request terminated successfully. +func (c ResponseStatusCode) IsSuccess() bool { + return c == RequestCompletedFull || c == RequestCompletedPartial +} + +// IsFailure returns true if the response code indicates the +// request terminated in failure. +func (c ResponseStatusCode) IsFailure() bool { + return c == RequestFailedBusy || + c == RequestFailedContentNotFound || + c == RequestFailedLegal || + c == RequestFailedUnknown || + c == RequestCancelled || + c == RequestRejected +} + +// IsTerminal returns true if the response code signals +// the end of the request +func (c ResponseStatusCode) IsTerminal() bool { + return c.IsSuccess() || c.IsFailure() +} From 808c7fa4c15f2874e58986d4bebbec26074c470d Mon Sep 17 00:00:00 2001 From: hannahhoward Date: Tue, 21 Sep 2021 13:32:42 -0700 Subject: [PATCH 2/8] refactor(responsemanager): reorganize for clarity Clarify actor pattern and responsibilities across threads --- impl/graphsync.go | 2 +- responsemanager/client.go | 268 +++++++++++ responsemanager/messages.go | 109 +++++ responsemanager/queryexecutor.go | 50 +- responsemanager/querypreparer.go | 2 +- responsemanager/responsemanager.go | 578 ------------------------ responsemanager/responsemanager_test.go | 80 ++-- responsemanager/server.go | 260 +++++++++++ 8 files changed, 703 insertions(+), 646 deletions(-) create mode 100644 responsemanager/client.go create mode 100644 responsemanager/messages.go delete mode 100644 responsemanager/responsemanager.go create mode 100644 responsemanager/server.go diff --git a/impl/graphsync.go b/impl/graphsync.go index cb9dae69..6bfd2953 100644 --- a/impl/graphsync.go +++ b/impl/graphsync.go @@ -335,7 +335,7 @@ func (gsr *graphSyncReceiver) ReceiveMessage( ctx context.Context, sender peer.ID, incoming gsmsg.GraphSyncMessage) { - gsr.graphSync().responseManager.ProcessRequests(ctx, sender, incoming.Requests()) + gsr.graphSync().responseManager.ProcessRequests(sender, incoming.Requests()) totalMemoryAllocated := uint64(0) for _, blk := range incoming.Blocks() { totalMemoryAllocated += uint64(len(blk.RawData())) diff --git a/responsemanager/client.go b/responsemanager/client.go new file mode 100644 index 00000000..6611e453 --- /dev/null +++ b/responsemanager/client.go @@ -0,0 +1,268 @@ +package responsemanager + +import ( + "context" + "errors" + "time" + + logging "github.com/ipfs/go-log/v2" + "github.com/ipfs/go-peertaskqueue/peertask" + ipld "github.com/ipld/go-ipld-prime" + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/ipfs/go-graphsync" + "github.com/ipfs/go-graphsync/ipldutil" + gsmsg "github.com/ipfs/go-graphsync/message" + "github.com/ipfs/go-graphsync/notifications" + "github.com/ipfs/go-graphsync/responsemanager/hooks" + "github.com/ipfs/go-graphsync/responsemanager/responseassembler" +) + +var log = logging.Logger("graphsync") + +const ( + thawSpeed = time.Millisecond * 100 +) + +type inProgressResponseStatus struct { + ctx context.Context + cancelFn func() + request gsmsg.GraphSyncRequest + loader ipld.BlockReadOpener + traverser ipldutil.Traverser + signals ResponseSignals + updates []gsmsg.GraphSyncRequest + isPaused bool + subscriber *notifications.TopicDataSubscriber +} + +type responseKey struct { + p peer.ID + requestID graphsync.RequestID +} + +// ResponseSignals are message channels to communicate between the manager and the query +type ResponseSignals struct { + PauseSignal chan struct{} + UpdateSignal chan struct{} + ErrSignal chan error +} + +// ResponseTaskData returns all information needed to execute a given response +type ResponseTaskData struct { + Empty bool + Subscriber *notifications.TopicDataSubscriber + Ctx context.Context + Request gsmsg.GraphSyncRequest + Loader ipld.BlockReadOpener + Traverser ipldutil.Traverser + Signals ResponseSignals +} + +// QueryQueue is an interface that can receive new selector query tasks +// and prioritize them as needed, and pop them off later +type QueryQueue interface { + PushTasks(to peer.ID, tasks ...peertask.Task) + PopTasks(targetMinWork int) (peer.ID, []*peertask.Task, int) + Remove(topic peertask.Topic, p peer.ID) + TasksDone(to peer.ID, tasks ...*peertask.Task) + ThawRound() +} + +// RequestHooks is an interface for processing request hooks +type RequestHooks interface { + ProcessRequestHooks(p peer.ID, request graphsync.RequestData) hooks.RequestResult +} + +// RequestQueuedHooks is an interface for processing request queued hooks +type RequestQueuedHooks interface { + ProcessRequestQueuedHooks(p peer.ID, request graphsync.RequestData) +} + +// BlockHooks is an interface for processing block hooks +type BlockHooks interface { + ProcessBlockHooks(p peer.ID, request graphsync.RequestData, blockData graphsync.BlockData) hooks.BlockResult +} + +// UpdateHooks is an interface for processing update hooks +type UpdateHooks interface { + ProcessUpdateHooks(p peer.ID, request graphsync.RequestData, update graphsync.RequestData) hooks.UpdateResult +} + +// CompletedListeners is an interface for notifying listeners that responses are complete +type CompletedListeners interface { + NotifyCompletedListeners(p peer.ID, request graphsync.RequestData, status graphsync.ResponseStatusCode) +} + +// CancelledListeners is an interface for notifying listeners that requestor cancelled +type CancelledListeners interface { + NotifyCancelledListeners(p peer.ID, request graphsync.RequestData) +} + +// BlockSentListeners is an interface for notifying listeners that of a block send occuring over the wire +type BlockSentListeners interface { + NotifyBlockSentListeners(p peer.ID, request graphsync.RequestData, block graphsync.BlockData) +} + +// NetworkErrorListeners is an interface for notifying listeners that an error occurred sending a data on the wire +type NetworkErrorListeners interface { + NotifyNetworkErrorListeners(p peer.ID, request graphsync.RequestData, err error) +} + +// ResponseAssembler is an interface that returns sender interfaces for peer responses. +type ResponseAssembler interface { + DedupKey(p peer.ID, requestID graphsync.RequestID, key string) + IgnoreBlocks(p peer.ID, requestID graphsync.RequestID, links []ipld.Link) + Transaction(p peer.ID, requestID graphsync.RequestID, transaction responseassembler.Transaction) error +} + +type responseManagerMessage interface { + handle(rm *ResponseManager) +} + +// ResponseManager handles incoming requests from the network, initiates selector +// traversals, and transmits responses +type ResponseManager struct { + ctx context.Context + cancelFn context.CancelFunc + responseAssembler ResponseAssembler + queryQueue QueryQueue + requestHooks RequestHooks + linkSystem ipld.LinkSystem + requestQueuedHooks RequestQueuedHooks + updateHooks UpdateHooks + cancelledListeners CancelledListeners + completedListeners CompletedListeners + blockSentListeners BlockSentListeners + networkErrorListeners NetworkErrorListeners + messages chan responseManagerMessage + workSignal chan struct{} + qe *queryExecutor + inProgressResponses map[responseKey]*inProgressResponseStatus + maxInProcessRequests uint64 +} + +// New creates a new response manager for responding to requests +func New(ctx context.Context, + linkSystem ipld.LinkSystem, + responseAssembler ResponseAssembler, + queryQueue QueryQueue, + requestQueuedHooks RequestQueuedHooks, + requestHooks RequestHooks, + blockHooks BlockHooks, + updateHooks UpdateHooks, + completedListeners CompletedListeners, + cancelledListeners CancelledListeners, + blockSentListeners BlockSentListeners, + networkErrorListeners NetworkErrorListeners, + maxInProcessRequests uint64, +) *ResponseManager { + ctx, cancelFn := context.WithCancel(ctx) + messages := make(chan responseManagerMessage, 16) + workSignal := make(chan struct{}, 1) + rm := &ResponseManager{ + ctx: ctx, + cancelFn: cancelFn, + requestHooks: requestHooks, + linkSystem: linkSystem, + responseAssembler: responseAssembler, + queryQueue: queryQueue, + requestQueuedHooks: requestQueuedHooks, + updateHooks: updateHooks, + cancelledListeners: cancelledListeners, + completedListeners: completedListeners, + blockSentListeners: blockSentListeners, + networkErrorListeners: networkErrorListeners, + messages: messages, + workSignal: workSignal, + inProgressResponses: make(map[responseKey]*inProgressResponseStatus), + maxInProcessRequests: maxInProcessRequests, + } + rm.qe = &queryExecutor{ + blockHooks: blockHooks, + updateHooks: updateHooks, + cancelledListeners: cancelledListeners, + responseAssembler: responseAssembler, + queryQueue: queryQueue, + manager: rm, + ctx: ctx, + workSignal: workSignal, + ticker: time.NewTicker(thawSpeed), + } + return rm +} + +// ProcessRequests processes incoming requests for the given peer +func (rm *ResponseManager) ProcessRequests(p peer.ID, requests []gsmsg.GraphSyncRequest) { + rm.cast(&processRequestMessage{p, requests}) +} + +// UnpauseResponse unpauses a response that was previously paused +func (rm *ResponseManager) UnpauseResponse(p peer.ID, requestID graphsync.RequestID, extensions ...graphsync.ExtensionData) error { + response := make(chan error, 1) + return rm.call(&unpauseRequestMessage{p, requestID, response, extensions}, response) +} + +// PauseResponse pauses an in progress response (may take 1 or more blocks to process) +func (rm *ResponseManager) PauseResponse(p peer.ID, requestID graphsync.RequestID) error { + response := make(chan error, 1) + return rm.call(&pauseRequestMessage{p, requestID, response}, response) +} + +// CancelResponse cancels an in progress response +func (rm *ResponseManager) CancelResponse(p peer.ID, requestID graphsync.RequestID) error { + response := make(chan error, 1) + return rm.call(&errorRequestMessage{p, requestID, errCancelledByCommand, response}, response) +} + +// this is a test utility method to force all messages to get processed +func (rm *ResponseManager) synchronize() { + sync := make(chan error) + _ = rm.call(&synchronizeMessage{sync}, sync) +} + +// StartTask starts the given task from the peer task queue +func (rm *ResponseManager) StartTask(task *peertask.Task, responseTaskDataChan chan<- ResponseTaskData) { + rm.cast(&startTaskRequest{task, responseTaskDataChan}) +} + +// GetUpdates is called to read pending updates for a task and clear them +func (rm *ResponseManager) GetUpdates(p peer.ID, requestID graphsync.RequestID, updatesChan chan<- []gsmsg.GraphSyncRequest) { + rm.cast(&responseUpdateRequest{responseKey{p, requestID}, updatesChan}) +} + +// FinishTask marks a task from the task queue as done +func (rm *ResponseManager) FinishTask(task *peertask.Task, err error) { + rm.cast(&finishTaskRequest{task, err}) +} + +func (rm *ResponseManager) call(message responseManagerMessage, response chan error) error { + select { + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case rm.messages <- message: + } + select { + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case err := <-response: + return err + } +} + +func (rm *ResponseManager) cast(message responseManagerMessage) { + select { + case <-rm.ctx.Done(): + case rm.messages <- message: + } +} + +// Startup starts processing for the WantManager. +func (rm *ResponseManager) Startup() { + go rm.run() +} + +// Shutdown ends processing for the want manager. +func (rm *ResponseManager) Shutdown() { + rm.cancelFn() +} diff --git a/responsemanager/messages.go b/responsemanager/messages.go new file mode 100644 index 00000000..6cf27e3b --- /dev/null +++ b/responsemanager/messages.go @@ -0,0 +1,109 @@ +package responsemanager + +import ( + "github.com/ipfs/go-graphsync" + gsmsg "github.com/ipfs/go-graphsync/message" + "github.com/ipfs/go-peertaskqueue/peertask" + "github.com/libp2p/go-libp2p-core/peer" +) + +type processRequestMessage struct { + p peer.ID + requests []gsmsg.GraphSyncRequest +} + +type pauseRequestMessage struct { + p peer.ID + requestID graphsync.RequestID + response chan error +} + +func (prm *pauseRequestMessage) handle(rm *ResponseManager) { + err := rm.pauseRequest(prm.p, prm.requestID) + select { + case <-rm.ctx.Done(): + case prm.response <- err: + } +} + +type errorRequestMessage struct { + p peer.ID + requestID graphsync.RequestID + err error + response chan error +} + +func (erm *errorRequestMessage) handle(rm *ResponseManager) { + err := rm.abortRequest(erm.p, erm.requestID, erm.err) + select { + case <-rm.ctx.Done(): + case erm.response <- err: + } +} + +type synchronizeMessage struct { + sync chan error +} + +func (sm *synchronizeMessage) handle(rm *ResponseManager) { + select { + case <-rm.ctx.Done(): + case sm.sync <- nil: + } +} + +type unpauseRequestMessage struct { + p peer.ID + requestID graphsync.RequestID + response chan error + extensions []graphsync.ExtensionData +} + +func (urm *unpauseRequestMessage) handle(rm *ResponseManager) { + err := rm.unpauseRequest(urm.p, urm.requestID, urm.extensions...) + select { + case <-rm.ctx.Done(): + case urm.response <- err: + } +} + +type responseUpdateRequest struct { + key responseKey + updateChan chan<- []gsmsg.GraphSyncRequest +} + +func (rur *responseUpdateRequest) handle(rm *ResponseManager) { + updates := rm.getUpdates(rur.key) + select { + case <-rm.ctx.Done(): + case rur.updateChan <- updates: + } +} + +type finishTaskRequest struct { + task *peertask.Task + err error +} + +func (ftr *finishTaskRequest) handle(rm *ResponseManager) { + rm.finishTask(ftr.task, ftr.err) +} + +type startTaskRequest struct { + task *peertask.Task + taskDataChan chan<- ResponseTaskData +} + +func (str *startTaskRequest) handle(rm *ResponseManager) { + + taskData := rm.startTask(str.task) + + select { + case <-rm.ctx.Done(): + case str.taskDataChan <- taskData: + } +} + +func (prm *processRequestMessage) handle(rm *ResponseManager) { + rm.processRequests(prm.p, prm.requests) +} diff --git a/responsemanager/queryexecutor.go b/responsemanager/queryexecutor.go index 44d1a718..14c2d9ea 100644 --- a/responsemanager/queryexecutor.go +++ b/responsemanager/queryexecutor.go @@ -16,18 +16,26 @@ import ( "github.com/ipfs/go-graphsync/responsemanager/hooks" "github.com/ipfs/go-graphsync/responsemanager/responseassembler" "github.com/ipfs/go-graphsync/responsemanager/runtraversal" + "github.com/ipfs/go-peertaskqueue/peertask" ) var errCancelledByCommand = errors.New("response cancelled by responder") +// Manager providers an interface to the response manager +type Manager interface { + StartTask(task *peertask.Task, responseTaskDataChan chan<- ResponseTaskData) + GetUpdates(p peer.ID, requestID graphsync.RequestID, updatesChan chan<- []gsmsg.GraphSyncRequest) + FinishTask(task *peertask.Task, err error) +} + // TODO: Move this into a seperate module and fully seperate from the ResponseManager type queryExecutor struct { + manager Manager blockHooks BlockHooks updateHooks UpdateHooks cancelledListeners CancelledListeners responseAssembler ResponseAssembler queryQueue QueryQueue - messages chan responseManagerMessage ctx context.Context workSignal chan struct{} ticker *time.Ticker @@ -35,8 +43,8 @@ type queryExecutor struct { func (qe *queryExecutor) processQueriesWorker() { const targetWork = 1 - taskDataChan := make(chan responseTaskData) - var taskData responseTaskData + taskDataChan := make(chan ResponseTaskData) + var taskData ResponseTaskData for { pid, tasks, _ := qe.queryQueue.PopTasks(targetWork) for len(tasks) == 0 { @@ -51,31 +59,24 @@ func (qe *queryExecutor) processQueriesWorker() { } } for _, task := range tasks { - select { - case qe.messages <- &startTaskRequest{task, taskDataChan}: - case <-qe.ctx.Done(): - return - } + qe.manager.StartTask(task, taskDataChan) select { case taskData = <-taskDataChan: case <-qe.ctx.Done(): return } - if taskData.empty { + if taskData.Empty { log.Info("Empty task on peer request stack") continue } - log.Debugw("beginning response execution", "id", taskData.request.ID(), "peer", pid.String(), "root_cid", taskData.request.Root().String()) - status, err := qe.executeQuery(pid, taskData.request, taskData.loader, taskData.traverser, taskData.signals, taskData.subscriber) + log.Debugw("beginning response execution", "id", taskData.Request.ID(), "peer", pid.String(), "root_cid", taskData.Request.Root().String()) + _, err := qe.executeQuery(pid, taskData.Request, taskData.Loader, taskData.Traverser, taskData.Signals, taskData.Subscriber) isCancelled := err != nil && isContextErr(err) if isCancelled { - qe.cancelledListeners.NotifyCancelledListeners(pid, taskData.request) + qe.cancelledListeners.NotifyCancelledListeners(pid, taskData.Request) } - select { - case qe.messages <- &finishTaskRequest{task, status, err}: - case <-qe.ctx.Done(): - } - log.Debugw("finishing response execution", "id", taskData.request.ID(), "peer", pid.String(), "root_cid", taskData.request.Root().String()) + qe.manager.FinishTask(task, err) + log.Debugw("finishing response execution", "id", taskData.Request.ID(), "peer", pid.String(), "root_cid", taskData.Request.Root().String()) } } } @@ -85,7 +86,7 @@ func (qe *queryExecutor) executeQuery( request gsmsg.GraphSyncRequest, loader ipld.BlockReadOpener, traverser ipldutil.Traverser, - signals signals, + signals ResponseSignals, sub *notifications.TopicDataSubscriber) (graphsync.ResponseStatusCode, error) { updateChan := make(chan []gsmsg.GraphSyncRequest) err := runtraversal.RunTraversal(loader, traverser, func(link ipld.Link, data []byte) error { @@ -151,21 +152,18 @@ func (qe *queryExecutor) executeQuery( func (qe *queryExecutor) checkForUpdates( p peer.ID, request gsmsg.GraphSyncRequest, - signals signals, + signals ResponseSignals, updateChan chan []gsmsg.GraphSyncRequest, rb responseassembler.ResponseBuilder) error { for { select { - case <-signals.pauseSignal: + case <-signals.PauseSignal: rb.PauseRequest() return hooks.ErrPaused{} - case err := <-signals.errSignal: + case err := <-signals.ErrSignal: return err - case <-signals.updateSignal: - select { - case qe.messages <- &responseUpdateRequest{responseKey{p, request.ID()}, updateChan}: - case <-qe.ctx.Done(): - } + case <-signals.UpdateSignal: + qe.manager.GetUpdates(p, request.ID(), updateChan) select { case updates := <-updateChan: for _, update := range updates { diff --git a/responsemanager/querypreparer.go b/responsemanager/querypreparer.go index 067786e9..811e285f 100644 --- a/responsemanager/querypreparer.go +++ b/responsemanager/querypreparer.go @@ -26,7 +26,7 @@ type queryPreparer struct { func (qe *queryPreparer) prepareQuery(ctx context.Context, p peer.ID, - request gsmsg.GraphSyncRequest, signals signals, sub *notifications.TopicDataSubscriber) (ipld.BlockReadOpener, ipldutil.Traverser, bool, error) { + request gsmsg.GraphSyncRequest, signals ResponseSignals, sub *notifications.TopicDataSubscriber) (ipld.BlockReadOpener, ipldutil.Traverser, bool, error) { result := qe.requestHooks.ProcessRequestHooks(p, request) var transactionError error var isPaused bool diff --git a/responsemanager/responsemanager.go b/responsemanager/responsemanager.go deleted file mode 100644 index 41faab18..00000000 --- a/responsemanager/responsemanager.go +++ /dev/null @@ -1,578 +0,0 @@ -package responsemanager - -import ( - "context" - "errors" - "math" - "time" - - logging "github.com/ipfs/go-log/v2" - "github.com/ipfs/go-peertaskqueue/peertask" - ipld "github.com/ipld/go-ipld-prime" - "github.com/libp2p/go-libp2p-core/peer" - - "github.com/ipfs/go-graphsync" - "github.com/ipfs/go-graphsync/ipldutil" - gsmsg "github.com/ipfs/go-graphsync/message" - "github.com/ipfs/go-graphsync/notifications" - "github.com/ipfs/go-graphsync/responsemanager/hooks" - "github.com/ipfs/go-graphsync/responsemanager/responseassembler" -) - -var log = logging.Logger("graphsync") - -const ( - thawSpeed = time.Millisecond * 100 -) - -type inProgressResponseStatus struct { - ctx context.Context - cancelFn func() - request gsmsg.GraphSyncRequest - loader ipld.BlockReadOpener - traverser ipldutil.Traverser - signals signals - updates []gsmsg.GraphSyncRequest - isPaused bool - subscriber *notifications.TopicDataSubscriber -} - -type responseKey struct { - p peer.ID - requestID graphsync.RequestID -} - -type signals struct { - pauseSignal chan struct{} - updateSignal chan struct{} - errSignal chan error -} - -type responseTaskData struct { - empty bool - subscriber *notifications.TopicDataSubscriber - ctx context.Context - request gsmsg.GraphSyncRequest - loader ipld.BlockReadOpener - traverser ipldutil.Traverser - signals signals -} - -// QueryQueue is an interface that can receive new selector query tasks -// and prioritize them as needed, and pop them off later -type QueryQueue interface { - PushTasks(to peer.ID, tasks ...peertask.Task) - PopTasks(targetMinWork int) (peer.ID, []*peertask.Task, int) - Remove(topic peertask.Topic, p peer.ID) - TasksDone(to peer.ID, tasks ...*peertask.Task) - ThawRound() -} - -// RequestHooks is an interface for processing request hooks -type RequestHooks interface { - ProcessRequestHooks(p peer.ID, request graphsync.RequestData) hooks.RequestResult -} - -// RequestQueuedHooks is an interface for processing request queued hooks -type RequestQueuedHooks interface { - ProcessRequestQueuedHooks(p peer.ID, request graphsync.RequestData) -} - -// BlockHooks is an interface for processing block hooks -type BlockHooks interface { - ProcessBlockHooks(p peer.ID, request graphsync.RequestData, blockData graphsync.BlockData) hooks.BlockResult -} - -// UpdateHooks is an interface for processing update hooks -type UpdateHooks interface { - ProcessUpdateHooks(p peer.ID, request graphsync.RequestData, update graphsync.RequestData) hooks.UpdateResult -} - -// CompletedListeners is an interface for notifying listeners that responses are complete -type CompletedListeners interface { - NotifyCompletedListeners(p peer.ID, request graphsync.RequestData, status graphsync.ResponseStatusCode) -} - -// CancelledListeners is an interface for notifying listeners that requestor cancelled -type CancelledListeners interface { - NotifyCancelledListeners(p peer.ID, request graphsync.RequestData) -} - -// BlockSentListeners is an interface for notifying listeners that of a block send occuring over the wire -type BlockSentListeners interface { - NotifyBlockSentListeners(p peer.ID, request graphsync.RequestData, block graphsync.BlockData) -} - -// NetworkErrorListeners is an interface for notifying listeners that an error occurred sending a data on the wire -type NetworkErrorListeners interface { - NotifyNetworkErrorListeners(p peer.ID, request graphsync.RequestData, err error) -} - -// ResponseAssembler is an interface that returns sender interfaces for peer responses. -type ResponseAssembler interface { - DedupKey(p peer.ID, requestID graphsync.RequestID, key string) - IgnoreBlocks(p peer.ID, requestID graphsync.RequestID, links []ipld.Link) - Transaction(p peer.ID, requestID graphsync.RequestID, transaction responseassembler.Transaction) error -} - -type responseManagerMessage interface { - handle(rm *ResponseManager) -} - -// ResponseManager handles incoming requests from the network, initiates selector -// traversals, and transmits responses -type ResponseManager struct { - ctx context.Context - cancelFn context.CancelFunc - responseAssembler ResponseAssembler - queryQueue QueryQueue - requestHooks RequestHooks - linkSystem ipld.LinkSystem - requestQueuedHooks RequestQueuedHooks - updateHooks UpdateHooks - cancelledListeners CancelledListeners - completedListeners CompletedListeners - blockSentListeners BlockSentListeners - networkErrorListeners NetworkErrorListeners - messages chan responseManagerMessage - workSignal chan struct{} - qe *queryExecutor - inProgressResponses map[responseKey]*inProgressResponseStatus - maxInProcessRequests uint64 -} - -// New creates a new response manager for responding to requests -func New(ctx context.Context, - linkSystem ipld.LinkSystem, - responseAssembler ResponseAssembler, - queryQueue QueryQueue, - requestQueuedHooks RequestQueuedHooks, - requestHooks RequestHooks, - blockHooks BlockHooks, - updateHooks UpdateHooks, - completedListeners CompletedListeners, - cancelledListeners CancelledListeners, - blockSentListeners BlockSentListeners, - networkErrorListeners NetworkErrorListeners, - maxInProcessRequests uint64, -) *ResponseManager { - ctx, cancelFn := context.WithCancel(ctx) - messages := make(chan responseManagerMessage, 16) - workSignal := make(chan struct{}, 1) - qe := &queryExecutor{ - blockHooks: blockHooks, - updateHooks: updateHooks, - cancelledListeners: cancelledListeners, - responseAssembler: responseAssembler, - queryQueue: queryQueue, - messages: messages, - ctx: ctx, - workSignal: workSignal, - ticker: time.NewTicker(thawSpeed), - } - return &ResponseManager{ - ctx: ctx, - cancelFn: cancelFn, - requestHooks: requestHooks, - linkSystem: linkSystem, - responseAssembler: responseAssembler, - queryQueue: queryQueue, - requestQueuedHooks: requestQueuedHooks, - updateHooks: updateHooks, - cancelledListeners: cancelledListeners, - completedListeners: completedListeners, - blockSentListeners: blockSentListeners, - networkErrorListeners: networkErrorListeners, - messages: messages, - workSignal: workSignal, - qe: qe, - inProgressResponses: make(map[responseKey]*inProgressResponseStatus), - maxInProcessRequests: maxInProcessRequests, - } -} - -type processRequestMessage struct { - p peer.ID - requests []gsmsg.GraphSyncRequest -} - -// ProcessRequests processes incoming requests for the given peer -func (rm *ResponseManager) ProcessRequests(ctx context.Context, p peer.ID, requests []gsmsg.GraphSyncRequest) { - select { - case rm.messages <- &processRequestMessage{p, requests}: - case <-rm.ctx.Done(): - case <-ctx.Done(): - } -} - -type unpauseRequestMessage struct { - p peer.ID - requestID graphsync.RequestID - response chan error - extensions []graphsync.ExtensionData -} - -// UnpauseResponse unpauses a response that was previously paused -func (rm *ResponseManager) UnpauseResponse(p peer.ID, requestID graphsync.RequestID, extensions ...graphsync.ExtensionData) error { - response := make(chan error, 1) - return rm.sendSyncMessage(&unpauseRequestMessage{p, requestID, response, extensions}, response) -} - -type pauseRequestMessage struct { - p peer.ID - requestID graphsync.RequestID - response chan error -} - -// PauseResponse pauses an in progress response (may take 1 or more blocks to process) -func (rm *ResponseManager) PauseResponse(p peer.ID, requestID graphsync.RequestID) error { - response := make(chan error, 1) - return rm.sendSyncMessage(&pauseRequestMessage{p, requestID, response}, response) -} - -type errorRequestMessage struct { - p peer.ID - requestID graphsync.RequestID - err error - response chan error -} - -// CancelResponse cancels an in progress response -func (rm *ResponseManager) CancelResponse(p peer.ID, requestID graphsync.RequestID) error { - response := make(chan error, 1) - return rm.sendSyncMessage(&errorRequestMessage{p, requestID, errCancelledByCommand, response}, response) -} - -func (rm *ResponseManager) sendSyncMessage(message responseManagerMessage, response chan error) error { - select { - case <-rm.ctx.Done(): - return errors.New("context cancelled") - case rm.messages <- message: - } - select { - case <-rm.ctx.Done(): - return errors.New("context cancelled") - case err := <-response: - return err - } -} - -type synchronizeMessage struct { - sync chan error -} - -// this is a test utility method to force all messages to get processed -func (rm *ResponseManager) synchronize() { - sync := make(chan error) - _ = rm.sendSyncMessage(&synchronizeMessage{sync}, sync) -} - -type startTaskRequest struct { - task *peertask.Task - taskDataChan chan responseTaskData -} - -type finishTaskRequest struct { - task *peertask.Task - status graphsync.ResponseStatusCode - err error -} - -type responseUpdateRequest struct { - key responseKey - updateChan chan []gsmsg.GraphSyncRequest -} - -// Startup starts processing for the WantManager. -func (rm *ResponseManager) Startup() { - go rm.run() -} - -// Shutdown ends processing for the want manager. -func (rm *ResponseManager) Shutdown() { - rm.cancelFn() -} - -func (rm *ResponseManager) cleanupInProcessResponses() { - for _, response := range rm.inProgressResponses { - response.cancelFn() - } -} - -func (rm *ResponseManager) run() { - defer rm.cleanupInProcessResponses() - for i := uint64(0); i < rm.maxInProcessRequests; i++ { - go rm.qe.processQueriesWorker() - } - - for { - select { - case <-rm.ctx.Done(): - return - case message := <-rm.messages: - message.handle(rm) - } - } -} - -func (rm *ResponseManager) processUpdate(key responseKey, update gsmsg.GraphSyncRequest) { - response, ok := rm.inProgressResponses[key] - if !ok { - log.Warnf("received update for non existent request, peer %s, request ID %d", key.p.Pretty(), key.requestID) - return - } - if !response.isPaused { - response.updates = append(response.updates, update) - select { - case response.signals.updateSignal <- struct{}{}: - default: - } - return - } - result := rm.updateHooks.ProcessUpdateHooks(key.p, response.request, update) - err := rm.responseAssembler.Transaction(key.p, key.requestID, func(rb responseassembler.ResponseBuilder) error { - for _, extension := range result.Extensions { - rb.SendExtensionData(extension) - } - if result.Err != nil { - rb.FinishWithError(graphsync.RequestFailedUnknown) - rb.AddNotifee(notifications.Notifee{Data: graphsync.RequestFailedUnknown, Subscriber: response.subscriber}) - } - return nil - }) - if err != nil { - log.Errorf("Error processing update: %s", err) - } - if result.Err != nil { - delete(rm.inProgressResponses, key) - response.cancelFn() - return - } - if result.Unpause { - err := rm.unpauseRequest(key.p, key.requestID) - if err != nil { - log.Warnf("error unpausing request: %s", err.Error()) - } - } - -} - -func (rm *ResponseManager) unpauseRequest(p peer.ID, requestID graphsync.RequestID, extensions ...graphsync.ExtensionData) error { - key := responseKey{p, requestID} - inProgressResponse, ok := rm.inProgressResponses[key] - if !ok { - return errors.New("could not find request") - } - if !inProgressResponse.isPaused { - return errors.New("request is not paused") - } - inProgressResponse.isPaused = false - if len(extensions) > 0 { - _ = rm.responseAssembler.Transaction(p, requestID, func(rb responseassembler.ResponseBuilder) error { - for _, extension := range extensions { - rb.SendExtensionData(extension) - } - return nil - }) - } - rm.queryQueue.PushTasks(p, peertask.Task{Topic: key, Priority: math.MaxInt32, Work: 1}) - select { - case rm.workSignal <- struct{}{}: - default: - } - return nil -} - -func (rm *ResponseManager) abortRequest(p peer.ID, requestID graphsync.RequestID, err error) error { - key := responseKey{p, requestID} - rm.queryQueue.Remove(key, key.p) - response, ok := rm.inProgressResponses[key] - if !ok { - return errors.New("could not find request") - } - - if response.isPaused { - _ = rm.responseAssembler.Transaction(p, requestID, func(rb responseassembler.ResponseBuilder) error { - if isContextErr(err) { - - rm.cancelledListeners.NotifyCancelledListeners(p, response.request) - rb.ClearRequest() - } else if err == errNetworkError { - rb.ClearRequest() - } else { - rb.FinishWithError(graphsync.RequestCancelled) - rb.AddNotifee(notifications.Notifee{Data: graphsync.RequestCancelled, Subscriber: response.subscriber}) - } - return nil - }) - delete(rm.inProgressResponses, key) - response.cancelFn() - return nil - } - select { - case response.signals.errSignal <- err: - default: - } - return nil -} - -func (prm *processRequestMessage) handle(rm *ResponseManager) { - for _, request := range prm.requests { - key := responseKey{p: prm.p, requestID: request.ID()} - if request.IsCancel() { - _ = rm.abortRequest(prm.p, request.ID(), ipldutil.ContextCancelError{}) - continue - } - if request.IsUpdate() { - rm.processUpdate(key, request) - continue - } - rm.requestQueuedHooks.ProcessRequestQueuedHooks(prm.p, request) - ctx, cancelFn := context.WithCancel(rm.ctx) - sub := notifications.NewTopicDataSubscriber(&subscriber{ - p: key.p, - request: request, - ctx: rm.ctx, - messages: rm.messages, - blockSentListeners: rm.blockSentListeners, - completedListeners: rm.completedListeners, - networkErrorListeners: rm.networkErrorListeners, - }) - - rm.inProgressResponses[key] = - &inProgressResponseStatus{ - ctx: ctx, - cancelFn: cancelFn, - subscriber: sub, - request: request, - signals: signals{ - pauseSignal: make(chan struct{}, 1), - updateSignal: make(chan struct{}, 1), - errSignal: make(chan error, 1), - }, - } - // TODO: Use a better work estimation metric. - - rm.queryQueue.PushTasks(prm.p, peertask.Task{Topic: key, Priority: int(request.Priority()), Work: 1}) - - select { - case rm.workSignal <- struct{}{}: - default: - } - } -} - -func (str *startTaskRequest) handle(rm *ResponseManager) { - key := str.task.Topic.(responseKey) - taskData := str.responseTaskData(rm, key) - if taskData.empty { - rm.queryQueue.TasksDone(key.p, str.task) - } - select { - case <-rm.ctx.Done(): - case str.taskDataChan <- taskData: - } -} - -func (str *startTaskRequest) responseTaskData(rm *ResponseManager, key responseKey) responseTaskData { - response, hasResponse := rm.inProgressResponses[key] - if !hasResponse { - return responseTaskData{empty: true} - } - - if response.loader == nil || response.traverser == nil { - loader, traverser, isPaused, err := (&queryPreparer{rm.requestHooks, rm.responseAssembler, rm.linkSystem}).prepareQuery(response.ctx, key.p, response.request, response.signals, response.subscriber) - if err != nil { - response.cancelFn() - delete(rm.inProgressResponses, key) - return responseTaskData{empty: true} - } - response.loader = loader - response.traverser = traverser - if isPaused { - response.isPaused = true - return responseTaskData{empty: true} - } - } - return responseTaskData{false, response.subscriber, response.ctx, response.request, response.loader, response.traverser, response.signals} -} - -func (ftr *finishTaskRequest) handle(rm *ResponseManager) { - key := ftr.task.Topic.(responseKey) - rm.queryQueue.TasksDone(key.p, ftr.task) - response, ok := rm.inProgressResponses[key] - if !ok { - return - } - if _, ok := ftr.err.(hooks.ErrPaused); ok { - response.isPaused = true - return - } - if ftr.err != nil { - log.Infof("response failed: %w", ftr.err) - } - delete(rm.inProgressResponses, key) - response.cancelFn() -} - -func (rur *responseUpdateRequest) handle(rm *ResponseManager) { - response, ok := rm.inProgressResponses[rur.key] - var updates []gsmsg.GraphSyncRequest - if ok { - updates = response.updates - response.updates = nil - } else { - updates = nil - } - select { - case <-rm.ctx.Done(): - case rur.updateChan <- updates: - } -} - -func (sm *synchronizeMessage) handle(rm *ResponseManager) { - select { - case <-rm.ctx.Done(): - case sm.sync <- nil: - } -} - -func (urm *unpauseRequestMessage) handle(rm *ResponseManager) { - err := rm.unpauseRequest(urm.p, urm.requestID, urm.extensions...) - select { - case <-rm.ctx.Done(): - case urm.response <- err: - } -} - -func (prm *pauseRequestMessage) pauseRequest(rm *ResponseManager) error { - key := responseKey{prm.p, prm.requestID} - inProgressResponse, ok := rm.inProgressResponses[key] - if !ok { - return errors.New("could not find request") - } - if inProgressResponse.isPaused { - return errors.New("request is already paused") - } - select { - case inProgressResponse.signals.pauseSignal <- struct{}{}: - default: - } - return nil -} - -func (prm *pauseRequestMessage) handle(rm *ResponseManager) { - err := prm.pauseRequest(rm) - select { - case <-rm.ctx.Done(): - case prm.response <- err: - } -} - -func (crm *errorRequestMessage) handle(rm *ResponseManager) { - err := rm.abortRequest(crm.p, crm.requestID, crm.err) - select { - case <-rm.ctx.Done(): - case crm.response <- err: - } -} diff --git a/responsemanager/responsemanager_test.go b/responsemanager/responsemanager_test.go index 2b527144..b921d4e8 100644 --- a/responsemanager/responsemanager_test.go +++ b/responsemanager/responsemanager_test.go @@ -53,7 +53,7 @@ func TestIncomingQuery(t *testing.T) { td.requestHooks.Register(selectorvalidator.SelectorValidator(100)) responseManager.Startup() - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) testutil.AssertDoesReceive(td.ctx, t, td.completedRequestChan, "Should have completed request but didn't") for i := 0; i < len(blks); i++ { td.assertSendBlock() @@ -75,7 +75,7 @@ func TestCancellationQueryInProgress(t *testing.T) { cancelledListenerCalled <- struct{}{} }) responseManager.Startup() - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) // read one block td.assertSendBlock() @@ -84,7 +84,7 @@ func TestCancellationQueryInProgress(t *testing.T) { cancelRequests := []gsmsg.GraphSyncRequest{ gsmsg.CancelRequest(td.requestID), } - responseManager.ProcessRequests(td.ctx, td.p, cancelRequests) + responseManager.ProcessRequests(td.p, cancelRequests) responseManager.synchronize() testutil.AssertDoesReceive(td.ctx, t, cancelledListenerCalled, "should call cancelled listener") @@ -98,7 +98,7 @@ func TestCancellationViaCommand(t *testing.T) { responseManager := td.newResponseManager() td.requestHooks.Register(selectorvalidator.SelectorValidator(100)) responseManager.Startup() - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) // read one block td.assertSendBlock() @@ -117,13 +117,13 @@ func TestEarlyCancellation(t *testing.T) { responseManager := td.newResponseManager() td.requestHooks.Register(selectorvalidator.SelectorValidator(100)) responseManager.Startup() - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) // send a cancellation cancelRequests := []gsmsg.GraphSyncRequest{ gsmsg.CancelRequest(td.requestID), } - responseManager.ProcessRequests(td.ctx, td.p, cancelRequests) + responseManager.ProcessRequests(td.p, cancelRequests) responseManager.synchronize() @@ -143,7 +143,7 @@ func TestMissingContent(t *testing.T) { }) // delete the root block delete(td.blockStore, cidlink.Link{Cid: td.requests[0].Root()}) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestFailedContentNotFound) }) t.Run("missing other block", func(t *testing.T) { @@ -161,7 +161,7 @@ func TestMissingContent(t *testing.T) { break } } - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedPartial) }) } @@ -172,7 +172,7 @@ func TestValidationAndExtensions(t *testing.T) { defer td.cancel() responseManager := td.newResponseManager() responseManager.Startup() - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestRejected) }) @@ -184,7 +184,7 @@ func TestValidationAndExtensions(t *testing.T) { td.requestHooks.Register(func(p peer.ID, requestData graphsync.RequestData, hookActions graphsync.IncomingRequestHookActions) { hookActions.SendExtensionData(td.extensionResponse) }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestRejected) td.assertReceiveExtensionResponse() }) @@ -198,7 +198,7 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.ValidateRequest() hookActions.SendExtensionData(td.extensionResponse) }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertReceiveExtensionResponse() }) @@ -215,7 +215,7 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.SendExtensionData(td.extensionResponse) hookActions.TerminateWithError(errors.New("everything went to crap")) }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestFailedUnknown) td.assertReceiveExtensionResponse() }) @@ -231,7 +231,7 @@ func TestValidationAndExtensions(t *testing.T) { }) // hook validates request - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertReceiveExtensionResponse() @@ -239,7 +239,7 @@ func TestValidationAndExtensions(t *testing.T) { unregister() // now same request should fail - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestRejected) }) @@ -254,7 +254,7 @@ func TestValidationAndExtensions(t *testing.T) { }) // request fails with base loader reading from block store that's missing data - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestFailedContentNotFound) err := td.peristenceOptions.Register("chainstore", td.persistence) @@ -267,7 +267,7 @@ func TestValidationAndExtensions(t *testing.T) { } }) // hook uses different loader that should make request succeed - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertReceiveExtensionResponse() }) @@ -290,7 +290,7 @@ func TestValidationAndExtensions(t *testing.T) { }) // with default chooser, customer chooser not called - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) require.Equal(t, 0, customChooserCallCount) @@ -303,7 +303,7 @@ func TestValidationAndExtensions(t *testing.T) { }) // verify now that request succeeds and uses custom chooser - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertReceiveExtensionResponse() require.Equal(t, 5, customChooserCallCount) @@ -331,7 +331,7 @@ func TestValidationAndExtensions(t *testing.T) { Data: data, }), } - responseManager.ProcessRequests(td.ctx, td.p, requests) + responseManager.ProcessRequests(td.p, requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertIgnoredCids(set) }) @@ -352,7 +352,7 @@ func TestValidationAndExtensions(t *testing.T) { Data: data, }), } - responseManager.ProcessRequests(td.ctx, td.p, requests) + responseManager.ProcessRequests(td.p, requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertDedupKey("applesauce") }) @@ -365,7 +365,7 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.ValidateRequest() hookActions.PauseResponse() }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertPausedRequest() td.assertRequestDoesNotCompleteWhilePaused() testutil.AssertChannelEmpty(t, td.sentResponses, "should not send more blocks") @@ -385,7 +385,7 @@ func TestValidationAndExtensions(t *testing.T) { td.blockHooks.Register(func(p peer.ID, requestData graphsync.RequestData, blockData graphsync.BlockData, hookActions graphsync.OutgoingBlockHookActions) { hookActions.SendExtensionData(td.extensionResponse) }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) for i := 0; i < td.blockChainLength; i++ { td.assertReceiveExtensionResponse() @@ -403,7 +403,7 @@ func TestValidationAndExtensions(t *testing.T) { td.blockHooks.Register(func(p peer.ID, requestData graphsync.RequestData, blockData graphsync.BlockData, hookActions graphsync.OutgoingBlockHookActions) { hookActions.TerminateWithError(errors.New("failed")) }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestFailedUnknown) }) @@ -423,7 +423,7 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.PauseResponse() } }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertRequestDoesNotCompleteWhilePaused() td.verifyNResponses(blockCount) td.assertPausedRequest() @@ -450,7 +450,7 @@ func TestValidationAndExtensions(t *testing.T) { require.NoError(t, err) } }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertRequestDoesNotCompleteWhilePaused() td.verifyNResponses(blockCount + 1) td.assertPausedRequest() @@ -476,7 +476,7 @@ func TestValidationAndExtensions(t *testing.T) { err := responseManager.UnpauseResponse(td.p, td.requestID) require.NoError(t, err) }() - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertPausedRequest() td.verifyNResponses(td.blockChainLength) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) @@ -506,12 +506,12 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.UnpauseResponse() } }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertRequestDoesNotCompleteWhilePaused() td.verifyNResponses(blockCount) td.assertPausedRequest() - responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) + responseManager.ProcessRequests(td.p, td.updateRequests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) }) @@ -540,9 +540,9 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.SendExtensionData(td.extensionResponse) } }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) testutil.AssertDoesReceive(td.ctx, t, sent, "sends blocks") - responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) + responseManager.ProcessRequests(td.p, td.updateRequests) responseManager.synchronize() close(wait) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) @@ -570,12 +570,12 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.SendExtensionData(td.extensionResponse) } }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.verifyNResponses(blockCount) td.assertPausedRequest() // send update - responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) + responseManager.ProcessRequests(td.p, td.updateRequests) // receive data td.assertReceiveExtensionResponse() @@ -610,9 +610,9 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.TerminateWithError(errors.New("something went wrong")) } }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) testutil.AssertDoesReceive(td.ctx, t, sent, "sends blocks") - responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) + responseManager.ProcessRequests(td.p, td.updateRequests) responseManager.synchronize() close(wait) td.assertCompleteRequestWith(graphsync.RequestFailedUnknown) @@ -639,12 +639,12 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.TerminateWithError(errors.New("something went wrong")) } }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.verifyNResponses(blockCount) td.assertPausedRequest() // send update - responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) + responseManager.ProcessRequests(td.p, td.updateRequests) td.assertCompleteRequestWith(graphsync.RequestFailedUnknown) // cannot unpause @@ -665,7 +665,7 @@ func TestNetworkErrors(t *testing.T) { td.requestHooks.Register(func(p peer.ID, requestData graphsync.RequestData, hookActions graphsync.IncomingRequestHookActions) { hookActions.ValidateRequest() }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.verifyNResponses(td.blockChainLength) td.assertOnlyCompleteProcessingWith(graphsync.RequestCompletedFull) err := errors.New("something went wrong") @@ -678,7 +678,7 @@ func TestNetworkErrors(t *testing.T) { defer td.cancel() responseManager := td.newResponseManager() responseManager.Startup() - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertOnlyCompleteProcessingWith(graphsync.RequestRejected) err := errors.New("something went wrong") td.notifyStatusMessagesNetworkError(err) @@ -693,7 +693,7 @@ func TestNetworkErrors(t *testing.T) { td.requestHooks.Register(func(p peer.ID, requestData graphsync.RequestData, hookActions graphsync.IncomingRequestHookActions) { hookActions.ValidateRequest() }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertSendBlock() err := errors.New("something went wrong") td.notifyBlockSendsNetworkError(err) @@ -717,7 +717,7 @@ func TestNetworkErrors(t *testing.T) { hookActions.PauseResponse() } }) - responseManager.ProcessRequests(td.ctx, td.p, td.requests) + responseManager.ProcessRequests(td.p, td.requests) td.assertRequestDoesNotCompleteWhilePaused() td.verifyNResponsesOnlyProcessing(blockCount) td.assertPausedRequest() diff --git a/responsemanager/server.go b/responsemanager/server.go new file mode 100644 index 00000000..9dbfa79d --- /dev/null +++ b/responsemanager/server.go @@ -0,0 +1,260 @@ +package responsemanager + +import ( + "context" + "errors" + "math" + + "github.com/ipfs/go-graphsync" + "github.com/ipfs/go-graphsync/ipldutil" + gsmsg "github.com/ipfs/go-graphsync/message" + "github.com/ipfs/go-graphsync/notifications" + "github.com/ipfs/go-graphsync/responsemanager/hooks" + "github.com/ipfs/go-graphsync/responsemanager/responseassembler" + "github.com/ipfs/go-peertaskqueue/peertask" + "github.com/libp2p/go-libp2p-core/peer" +) + +func (rm *ResponseManager) cleanupInProcessResponses() { + for _, response := range rm.inProgressResponses { + response.cancelFn() + } +} + +func (rm *ResponseManager) run() { + defer rm.cleanupInProcessResponses() + for i := uint64(0); i < rm.maxInProcessRequests; i++ { + go rm.qe.processQueriesWorker() + } + + for { + select { + case <-rm.ctx.Done(): + return + case message := <-rm.messages: + message.handle(rm) + } + } +} + +func (rm *ResponseManager) processUpdate(key responseKey, update gsmsg.GraphSyncRequest) { + response, ok := rm.inProgressResponses[key] + if !ok { + log.Warnf("received update for non existent request, peer %s, request ID %d", key.p.Pretty(), key.requestID) + return + } + if !response.isPaused { + response.updates = append(response.updates, update) + select { + case response.signals.UpdateSignal <- struct{}{}: + default: + } + return + } + result := rm.updateHooks.ProcessUpdateHooks(key.p, response.request, update) + err := rm.responseAssembler.Transaction(key.p, key.requestID, func(rb responseassembler.ResponseBuilder) error { + for _, extension := range result.Extensions { + rb.SendExtensionData(extension) + } + if result.Err != nil { + rb.FinishWithError(graphsync.RequestFailedUnknown) + rb.AddNotifee(notifications.Notifee{Data: graphsync.RequestFailedUnknown, Subscriber: response.subscriber}) + } + return nil + }) + if err != nil { + log.Errorf("Error processing update: %s", err) + } + if result.Err != nil { + delete(rm.inProgressResponses, key) + response.cancelFn() + return + } + if result.Unpause { + err := rm.unpauseRequest(key.p, key.requestID) + if err != nil { + log.Warnf("error unpausing request: %s", err.Error()) + } + } + +} + +func (rm *ResponseManager) unpauseRequest(p peer.ID, requestID graphsync.RequestID, extensions ...graphsync.ExtensionData) error { + key := responseKey{p, requestID} + inProgressResponse, ok := rm.inProgressResponses[key] + if !ok { + return errors.New("could not find request") + } + if !inProgressResponse.isPaused { + return errors.New("request is not paused") + } + inProgressResponse.isPaused = false + if len(extensions) > 0 { + _ = rm.responseAssembler.Transaction(p, requestID, func(rb responseassembler.ResponseBuilder) error { + for _, extension := range extensions { + rb.SendExtensionData(extension) + } + return nil + }) + } + rm.queryQueue.PushTasks(p, peertask.Task{Topic: key, Priority: math.MaxInt32, Work: 1}) + select { + case rm.workSignal <- struct{}{}: + default: + } + return nil +} + +func (rm *ResponseManager) abortRequest(p peer.ID, requestID graphsync.RequestID, err error) error { + key := responseKey{p, requestID} + rm.queryQueue.Remove(key, key.p) + response, ok := rm.inProgressResponses[key] + if !ok { + return errors.New("could not find request") + } + + if response.isPaused { + _ = rm.responseAssembler.Transaction(p, requestID, func(rb responseassembler.ResponseBuilder) error { + if isContextErr(err) { + + rm.cancelledListeners.NotifyCancelledListeners(p, response.request) + rb.ClearRequest() + } else if err == errNetworkError { + rb.ClearRequest() + } else { + rb.FinishWithError(graphsync.RequestCancelled) + rb.AddNotifee(notifications.Notifee{Data: graphsync.RequestCancelled, Subscriber: response.subscriber}) + } + return nil + }) + delete(rm.inProgressResponses, key) + response.cancelFn() + return nil + } + select { + case response.signals.ErrSignal <- err: + default: + } + return nil +} + +func (rm *ResponseManager) processRequests(p peer.ID, requests []gsmsg.GraphSyncRequest) { + for _, request := range requests { + key := responseKey{p: p, requestID: request.ID()} + if request.IsCancel() { + _ = rm.abortRequest(p, request.ID(), ipldutil.ContextCancelError{}) + continue + } + if request.IsUpdate() { + rm.processUpdate(key, request) + continue + } + rm.requestQueuedHooks.ProcessRequestQueuedHooks(p, request) + ctx, cancelFn := context.WithCancel(rm.ctx) + sub := notifications.NewTopicDataSubscriber(&subscriber{ + p: key.p, + request: request, + ctx: rm.ctx, + messages: rm.messages, + blockSentListeners: rm.blockSentListeners, + completedListeners: rm.completedListeners, + networkErrorListeners: rm.networkErrorListeners, + }) + + rm.inProgressResponses[key] = + &inProgressResponseStatus{ + ctx: ctx, + cancelFn: cancelFn, + subscriber: sub, + request: request, + signals: ResponseSignals{ + PauseSignal: make(chan struct{}, 1), + UpdateSignal: make(chan struct{}, 1), + ErrSignal: make(chan error, 1), + }, + } + // TODO: Use a better work estimation metric. + + rm.queryQueue.PushTasks(p, peertask.Task{Topic: key, Priority: int(request.Priority()), Work: 1}) + + select { + case rm.workSignal <- struct{}{}: + default: + } + } +} + +func (rm *ResponseManager) taskDataForKey(key responseKey) ResponseTaskData { + response, hasResponse := rm.inProgressResponses[key] + if !hasResponse { + return ResponseTaskData{Empty: true} + } + if response.loader == nil || response.traverser == nil { + loader, traverser, isPaused, err := (&queryPreparer{rm.requestHooks, rm.responseAssembler, rm.linkSystem}).prepareQuery(response.ctx, key.p, response.request, response.signals, response.subscriber) + if err != nil { + response.cancelFn() + delete(rm.inProgressResponses, key) + return ResponseTaskData{Empty: true} + } + response.loader = loader + response.traverser = traverser + if isPaused { + response.isPaused = true + return ResponseTaskData{Empty: true} + } + } + return ResponseTaskData{false, response.subscriber, response.ctx, response.request, response.loader, response.traverser, response.signals} +} + +func (rm *ResponseManager) startTask(task *peertask.Task) ResponseTaskData { + key := task.Topic.(responseKey) + taskData := rm.taskDataForKey(key) + if taskData.Empty { + rm.queryQueue.TasksDone(key.p, task) + } + return taskData +} + +func (rm *ResponseManager) finishTask(task *peertask.Task, err error) { + key := task.Topic.(responseKey) + rm.queryQueue.TasksDone(key.p, task) + response, ok := rm.inProgressResponses[key] + if !ok { + return + } + if _, ok := err.(hooks.ErrPaused); ok { + response.isPaused = true + return + } + if err != nil { + log.Infof("response failed: %w", err) + } + delete(rm.inProgressResponses, key) + response.cancelFn() +} + +func (rm *ResponseManager) getUpdates(key responseKey) []gsmsg.GraphSyncRequest { + response, ok := rm.inProgressResponses[key] + if !ok { + return nil + } + updates := response.updates + response.updates = nil + return updates +} + +func (rm *ResponseManager) pauseRequest(p peer.ID, requestID graphsync.RequestID) error { + key := responseKey{p, requestID} + inProgressResponse, ok := rm.inProgressResponses[key] + if !ok { + return errors.New("could not find request") + } + if inProgressResponse.isPaused { + return errors.New("request is already paused") + } + select { + case inProgressResponse.signals.PauseSignal <- struct{}{}: + default: + } + return nil +} From 55781d4c7d1699e92fd36f4ad8d20eabd6267df9 Mon Sep 17 00:00:00 2001 From: hannahhoward Date: Tue, 21 Sep 2021 14:17:56 -0700 Subject: [PATCH 3/8] refactor(requestmanager): reorganize for clarity refactor requestmanager to clearly designate responsibilities in actor pattern --- impl/graphsync.go | 4 +- requestmanager/client.go | 359 ++++++++++++ requestmanager/messages.go | 85 +++ requestmanager/requestmanager.go | 694 ------------------------ requestmanager/requestmanager_test.go | 34 +- requestmanager/server.go | 296 ++++++++++ responsemanager/client.go | 61 ++- responsemanager/responsemanager_test.go | 80 +-- responsemanager/server.go | 3 + 9 files changed, 839 insertions(+), 777 deletions(-) create mode 100644 requestmanager/client.go create mode 100644 requestmanager/messages.go delete mode 100644 requestmanager/requestmanager.go create mode 100644 requestmanager/server.go diff --git a/impl/graphsync.go b/impl/graphsync.go index 6bfd2953..3531b8aa 100644 --- a/impl/graphsync.go +++ b/impl/graphsync.go @@ -206,7 +206,7 @@ func New(parent context.Context, network gsnet.GraphSyncNetwork, // Request initiates a new GraphSync request to the given peer using the given selector spec. func (gs *GraphSync) Request(ctx context.Context, p peer.ID, root ipld.Link, selector ipld.Node, extensions ...graphsync.ExtensionData) (<-chan graphsync.ResponseProgress, <-chan error) { - return gs.requestManager.SendRequest(ctx, p, root, selector, extensions...) + return gs.requestManager.NewRequest(ctx, p, root, selector, extensions...) } // RegisterIncomingRequestHook adds a hook that runs when a request is received @@ -335,7 +335,7 @@ func (gsr *graphSyncReceiver) ReceiveMessage( ctx context.Context, sender peer.ID, incoming gsmsg.GraphSyncMessage) { - gsr.graphSync().responseManager.ProcessRequests(sender, incoming.Requests()) + gsr.graphSync().responseManager.ProcessRequests(ctx, sender, incoming.Requests()) totalMemoryAllocated := uint64(0) for _, blk := range incoming.Blocks() { totalMemoryAllocated += uint64(len(blk.RawData())) diff --git a/requestmanager/client.go b/requestmanager/client.go new file mode 100644 index 00000000..43181f36 --- /dev/null +++ b/requestmanager/client.go @@ -0,0 +1,359 @@ +package requestmanager + +import ( + "context" + "errors" + "fmt" + "sync/atomic" + "time" + + "github.com/hannahhoward/go-pubsub" + blocks "github.com/ipfs/go-block-format" + logging "github.com/ipfs/go-log/v2" + "github.com/ipld/go-ipld-prime" + "github.com/ipld/go-ipld-prime/traversal/selector" + "github.com/libp2p/go-libp2p-core/peer" + + "github.com/ipfs/go-graphsync" + "github.com/ipfs/go-graphsync/listeners" + gsmsg "github.com/ipfs/go-graphsync/message" + "github.com/ipfs/go-graphsync/messagequeue" + "github.com/ipfs/go-graphsync/metadata" + "github.com/ipfs/go-graphsync/notifications" + "github.com/ipfs/go-graphsync/requestmanager/hooks" + "github.com/ipfs/go-graphsync/requestmanager/types" +) + +// The code in this file implements the public interface of the request manager. +// Functions in this file operate outside the internal thread and should +// NOT modify the internal state of the RequestManager. + +var log = logging.Logger("graphsync") + +const ( + // defaultPriority is the default priority for requests sent by graphsync + defaultPriority = graphsync.Priority(0) +) + +type inProgressRequestStatus struct { + ctx context.Context + startTime time.Time + cancelFn func() + p peer.ID + terminalError chan error + resumeMessages chan []graphsync.ExtensionData + pauseMessages chan struct{} + paused bool + lastResponse atomic.Value + onTerminated []chan<- error +} + +// PeerHandler is an interface that can send requests to peers +type PeerHandler interface { + AllocateAndBuildMessage(p peer.ID, blkSize uint64, buildMessageFn func(*gsmsg.Builder), notifees []notifications.Notifee) +} + +// AsyncLoader is an interface for loading links asynchronously, returning +// results as new responses are processed +type AsyncLoader interface { + StartRequest(graphsync.RequestID, string) error + ProcessResponse(responses map[graphsync.RequestID]metadata.Metadata, + blks []blocks.Block) + AsyncLoad(p peer.ID, requestID graphsync.RequestID, link ipld.Link, linkContext ipld.LinkContext) <-chan types.AsyncLoadResult + CompleteResponsesFor(requestID graphsync.RequestID) + CleanupRequest(requestID graphsync.RequestID) +} + +// RequestManager tracks outgoing requests and processes incoming reponses +// to them. +type RequestManager struct { + ctx context.Context + cancel func() + messages chan requestManagerMessage + peerHandler PeerHandler + rc *responseCollector + asyncLoader AsyncLoader + disconnectNotif *pubsub.PubSub + linkSystem ipld.LinkSystem + + // dont touch out side of run loop + nextRequestID graphsync.RequestID + inProgressRequestStatuses map[graphsync.RequestID]*inProgressRequestStatus + requestHooks RequestHooks + responseHooks ResponseHooks + blockHooks BlockHooks + networkErrorListeners *listeners.NetworkErrorListeners +} + +type requestManagerMessage interface { + handle(rm *RequestManager) +} + +// RequestHooks run for new requests +type RequestHooks interface { + ProcessRequestHooks(p peer.ID, request graphsync.RequestData) hooks.RequestResult +} + +// ResponseHooks run for new responses +type ResponseHooks interface { + ProcessResponseHooks(p peer.ID, response graphsync.ResponseData) hooks.UpdateResult +} + +// BlockHooks run for each block loaded +type BlockHooks interface { + ProcessBlockHooks(p peer.ID, response graphsync.ResponseData, block graphsync.BlockData) hooks.UpdateResult +} + +// New generates a new request manager from a context, network, and selectorQuerier +func New(ctx context.Context, + asyncLoader AsyncLoader, + linkSystem ipld.LinkSystem, + requestHooks RequestHooks, + responseHooks ResponseHooks, + blockHooks BlockHooks, + networkErrorListeners *listeners.NetworkErrorListeners, +) *RequestManager { + ctx, cancel := context.WithCancel(ctx) + return &RequestManager{ + ctx: ctx, + cancel: cancel, + asyncLoader: asyncLoader, + disconnectNotif: pubsub.New(disconnectDispatcher), + linkSystem: linkSystem, + rc: newResponseCollector(ctx), + messages: make(chan requestManagerMessage, 16), + inProgressRequestStatuses: make(map[graphsync.RequestID]*inProgressRequestStatus), + requestHooks: requestHooks, + responseHooks: responseHooks, + blockHooks: blockHooks, + networkErrorListeners: networkErrorListeners, + } +} + +// SetDelegate specifies who will send messages out to the internet. +func (rm *RequestManager) SetDelegate(peerHandler PeerHandler) { + rm.peerHandler = peerHandler +} + +type inProgressRequest struct { + requestID graphsync.RequestID + request gsmsg.GraphSyncRequest + incoming chan graphsync.ResponseProgress + incomingError chan error +} + +// NewRequest initiates a new GraphSync request to the given peer. +func (rm *RequestManager) NewRequest(ctx context.Context, + p peer.ID, + root ipld.Link, + selectorNode ipld.Node, + extensions ...graphsync.ExtensionData) (<-chan graphsync.ResponseProgress, <-chan error) { + if _, err := selector.ParseSelector(selectorNode); err != nil { + return rm.singleErrorResponse(fmt.Errorf("invalid selector spec")) + } + + inProgressRequestChan := make(chan inProgressRequest) + + rm.send(&newRequestMessage{p, root, selectorNode, extensions, inProgressRequestChan}, ctx.Done()) + var receivedInProgressRequest inProgressRequest + select { + case <-rm.ctx.Done(): + return rm.emptyResponse() + case receivedInProgressRequest = <-inProgressRequestChan: + } + + // If the connection to the peer is disconnected, fire an error + unsub := rm.listenForDisconnect(p, func(neterr error) { + rm.networkErrorListeners.NotifyNetworkErrorListeners(p, receivedInProgressRequest.request, neterr) + }) + + return rm.rc.collectResponses(ctx, + receivedInProgressRequest.incoming, + receivedInProgressRequest.incomingError, + func() { + rm.cancelRequestAndClose(receivedInProgressRequest.requestID, + receivedInProgressRequest.incoming, + receivedInProgressRequest.incomingError) + }, + // Once the request has completed, stop listening for disconnect events + unsub, + ) +} + +// Dispatch the Disconnect event to subscribers +func disconnectDispatcher(p pubsub.Event, subscriberFn pubsub.SubscriberFn) error { + listener := subscriberFn.(func(peer.ID)) + listener(p.(peer.ID)) + return nil +} + +// Listen for the Disconnect event for the given peer +func (rm *RequestManager) listenForDisconnect(p peer.ID, onDisconnect func(neterr error)) func() { + // Subscribe to Disconnect notifications + return rm.disconnectNotif.Subscribe(func(evtPeer peer.ID) { + // If the peer is the one we're interested in, call the listener + if evtPeer == p { + onDisconnect(fmt.Errorf("disconnected from peer %s", p)) + } + }) +} + +// Disconnected is called when a peer disconnects +func (rm *RequestManager) Disconnected(p peer.ID) { + // Notify any listeners that a peer has disconnected + _ = rm.disconnectNotif.Publish(p) +} + +func (rm *RequestManager) emptyResponse() (chan graphsync.ResponseProgress, chan error) { + ch := make(chan graphsync.ResponseProgress) + close(ch) + errCh := make(chan error) + close(errCh) + return ch, errCh +} + +func (rm *RequestManager) singleErrorResponse(err error) (chan graphsync.ResponseProgress, chan error) { + ch := make(chan graphsync.ResponseProgress) + close(ch) + errCh := make(chan error, 1) + errCh <- err + close(errCh) + return ch, errCh +} + +func (rm *RequestManager) cancelRequestAndClose(requestID graphsync.RequestID, + incomingResponses chan graphsync.ResponseProgress, + incomingErrors chan error) { + cancelMessageChannel := rm.messages + for cancelMessageChannel != nil || incomingResponses != nil || incomingErrors != nil { + select { + case cancelMessageChannel <- &cancelRequestMessage{requestID, false, nil, nil}: + cancelMessageChannel = nil + // clear out any remaining responses, in case and "incoming reponse" + // messages get processed before our cancel message + case _, ok := <-incomingResponses: + if !ok { + incomingResponses = nil + } + case _, ok := <-incomingErrors: + if !ok { + incomingErrors = nil + } + case <-rm.ctx.Done(): + return + } + } +} + +// CancelRequest cancels the given request ID and waits for the request to terminate +func (rm *RequestManager) CancelRequest(ctx context.Context, requestID graphsync.RequestID) error { + terminated := make(chan error, 1) + rm.send(&cancelRequestMessage{requestID, false, terminated, graphsync.RequestClientCancelledErr{}}, ctx.Done()) + select { + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case err := <-terminated: + return err + } +} + +// ProcessResponses ingests the given responses from the network and +// and updates the in progress requests based on those responses. +func (rm *RequestManager) ProcessResponses(p peer.ID, responses []gsmsg.GraphSyncResponse, + blks []blocks.Block) { + rm.send(&processResponseMessage{p, responses, blks}, nil) +} + +// UnpauseRequest unpauses a request that was paused in a block hook based request ID +// Can also send extensions with unpause +func (rm *RequestManager) UnpauseRequest(requestID graphsync.RequestID, extensions ...graphsync.ExtensionData) error { + response := make(chan error, 1) + rm.send(&unpauseRequestMessage{requestID, extensions, response}, nil) + select { + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case err := <-response: + return err + } +} + +// PauseRequest pauses an in progress request (may take 1 or more blocks to process) +func (rm *RequestManager) PauseRequest(requestID graphsync.RequestID) error { + response := make(chan error, 1) + rm.send(&pauseRequestMessage{requestID, response}, nil) + select { + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case err := <-response: + return err + } +} + +// ProcessBlockHooks processes block hooks for the given response & block and cancels +// the request as needed +func (rm *RequestManager) ProcessBlockHooks(p peer.ID, response graphsync.ResponseData, block graphsync.BlockData) error { + result := rm.blockHooks.ProcessBlockHooks(p, response, block) + if len(result.Extensions) > 0 { + updateRequest := gsmsg.UpdateRequest(response.RequestID(), result.Extensions...) + rm.SendRequest(p, updateRequest) + } + if result.Err != nil { + _, isPause := result.Err.(hooks.ErrPaused) + rm.send(&cancelRequestMessage{response.RequestID(), isPause, nil, nil}, nil) + } + return result.Err +} + +// TerminateRequest marks a request done +func (rm *RequestManager) TerminateRequest(requestID graphsync.RequestID) { + rm.send(&terminateRequestMessage{requestID}, nil) +} + +// SendRequest sends a request to the message queue +func (rm *RequestManager) SendRequest(p peer.ID, request gsmsg.GraphSyncRequest) { + sub := notifications.NewTopicDataSubscriber(&reqSubscriber{p, request, rm.networkErrorListeners}) + failNotifee := notifications.Notifee{Data: requestNetworkError, Subscriber: sub} + rm.peerHandler.AllocateAndBuildMessage(p, 0, func(builder *gsmsg.Builder) { + builder.AddRequest(request) + }, []notifications.Notifee{failNotifee}) +} + +// Startup starts processing for the WantManager. +func (rm *RequestManager) Startup() { + go rm.run() +} + +// Shutdown ends processing for the want manager. +func (rm *RequestManager) Shutdown() { + rm.cancel() +} + +func (rm *RequestManager) send(message requestManagerMessage, done <-chan struct{}) { + select { + case <-rm.ctx.Done(): + case <-done: + case rm.messages <- message: + } +} + +type reqSubscriber struct { + p peer.ID + request gsmsg.GraphSyncRequest + networkErrorListeners *listeners.NetworkErrorListeners +} + +func (r *reqSubscriber) OnNext(topic notifications.Topic, event notifications.Event) { + mqEvt, isMQEvt := event.(messagequeue.Event) + if !isMQEvt || mqEvt.Name != messagequeue.Error { + return + } + + r.networkErrorListeners.NotifyNetworkErrorListeners(r.p, r.request, mqEvt.Err) + //r.re.networkError <- mqEvt.Err + //r.re.terminateRequest() +} + +func (r reqSubscriber) OnClose(topic notifications.Topic) { +} + +const requestNetworkError = "request_network_error" diff --git a/requestmanager/messages.go b/requestmanager/messages.go new file mode 100644 index 00000000..b18efe24 --- /dev/null +++ b/requestmanager/messages.go @@ -0,0 +1,85 @@ +package requestmanager + +import ( + blocks "github.com/ipfs/go-block-format" + "github.com/ipfs/go-graphsync" + gsmsg "github.com/ipfs/go-graphsync/message" + "github.com/ipld/go-ipld-prime" + "github.com/libp2p/go-libp2p-core/peer" +) + +type pauseRequestMessage struct { + id graphsync.RequestID + response chan<- error +} + +func (prm *pauseRequestMessage) handle(rm *RequestManager) { + err := rm.pause(prm.id) + select { + case <-rm.ctx.Done(): + case prm.response <- err: + } +} + +type unpauseRequestMessage struct { + id graphsync.RequestID + extensions []graphsync.ExtensionData + response chan<- error +} + +func (urm *unpauseRequestMessage) handle(rm *RequestManager) { + err := rm.unpause(urm.id, urm.extensions) + select { + case <-rm.ctx.Done(): + case urm.response <- err: + } +} + +type processResponseMessage struct { + p peer.ID + responses []gsmsg.GraphSyncResponse + blks []blocks.Block +} + +func (prm *processResponseMessage) handle(rm *RequestManager) { + rm.processResponseMessage(prm.p, prm.responses, prm.blks) +} + +type cancelRequestMessage struct { + requestID graphsync.RequestID + isPause bool + onTerminated chan error + terminalError error +} + +func (crm *cancelRequestMessage) handle(rm *RequestManager) { + rm.cancelRequest(crm.requestID, crm.isPause, crm.onTerminated, crm.terminalError) +} + +type terminateRequestMessage struct { + requestID graphsync.RequestID +} + +func (trm *terminateRequestMessage) handle(rm *RequestManager) { + rm.terminateRequest(trm.requestID) +} + +type newRequestMessage struct { + p peer.ID + root ipld.Link + selector ipld.Node + extensions []graphsync.ExtensionData + inProgressRequestChan chan<- inProgressRequest +} + +func (nrm *newRequestMessage) handle(rm *RequestManager) { + var ipr inProgressRequest + + ipr.request, ipr.incoming, ipr.incomingError = rm.setupRequest(nrm.p, nrm.root, nrm.selector, nrm.extensions) + ipr.requestID = ipr.request.ID() + + select { + case nrm.inProgressRequestChan <- ipr: + case <-rm.ctx.Done(): + } +} diff --git a/requestmanager/requestmanager.go b/requestmanager/requestmanager.go deleted file mode 100644 index 222b762c..00000000 --- a/requestmanager/requestmanager.go +++ /dev/null @@ -1,694 +0,0 @@ -package requestmanager - -import ( - "context" - "errors" - "fmt" - "sync/atomic" - "time" - - "github.com/hannahhoward/go-pubsub" - 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" - "github.com/ipld/go-ipld-prime/traversal/selector" - "github.com/libp2p/go-libp2p-core/peer" - - "github.com/ipfs/go-graphsync" - "github.com/ipfs/go-graphsync/cidset" - "github.com/ipfs/go-graphsync/dedupkey" - ipldutil "github.com/ipfs/go-graphsync/ipldutil" - "github.com/ipfs/go-graphsync/listeners" - gsmsg "github.com/ipfs/go-graphsync/message" - "github.com/ipfs/go-graphsync/messagequeue" - "github.com/ipfs/go-graphsync/metadata" - "github.com/ipfs/go-graphsync/notifications" - "github.com/ipfs/go-graphsync/requestmanager/executor" - "github.com/ipfs/go-graphsync/requestmanager/hooks" - "github.com/ipfs/go-graphsync/requestmanager/types" -) - -var log = logging.Logger("graphsync") - -const ( - // defaultPriority is the default priority for requests sent by graphsync - defaultPriority = graphsync.Priority(0) -) - -type inProgressRequestStatus struct { - ctx context.Context - startTime time.Time - cancelFn func() - p peer.ID - terminalError chan error - resumeMessages chan []graphsync.ExtensionData - pauseMessages chan struct{} - paused bool - lastResponse atomic.Value - onTerminated []chan error -} - -// PeerHandler is an interface that can send requests to peers -type PeerHandler interface { - AllocateAndBuildMessage(p peer.ID, blkSize uint64, buildMessageFn func(*gsmsg.Builder), notifees []notifications.Notifee) -} - -// AsyncLoader is an interface for loading links asynchronously, returning -// results as new responses are processed -type AsyncLoader interface { - StartRequest(graphsync.RequestID, string) error - ProcessResponse(responses map[graphsync.RequestID]metadata.Metadata, - blks []blocks.Block) - AsyncLoad(p peer.ID, requestID graphsync.RequestID, link ipld.Link, linkContext ipld.LinkContext) <-chan types.AsyncLoadResult - CompleteResponsesFor(requestID graphsync.RequestID) - CleanupRequest(requestID graphsync.RequestID) -} - -// RequestManager tracks outgoing requests and processes incoming reponses -// to them. -type RequestManager struct { - ctx context.Context - cancel func() - messages chan requestManagerMessage - peerHandler PeerHandler - rc *responseCollector - asyncLoader AsyncLoader - disconnectNotif *pubsub.PubSub - linkSystem ipld.LinkSystem - - // dont touch out side of run loop - nextRequestID graphsync.RequestID - inProgressRequestStatuses map[graphsync.RequestID]*inProgressRequestStatus - requestHooks RequestHooks - responseHooks ResponseHooks - blockHooks BlockHooks - networkErrorListeners *listeners.NetworkErrorListeners -} - -type requestManagerMessage interface { - handle(rm *RequestManager) -} - -// RequestHooks run for new requests -type RequestHooks interface { - ProcessRequestHooks(p peer.ID, request graphsync.RequestData) hooks.RequestResult -} - -// ResponseHooks run for new responses -type ResponseHooks interface { - ProcessResponseHooks(p peer.ID, response graphsync.ResponseData) hooks.UpdateResult -} - -// BlockHooks run for each block loaded -type BlockHooks interface { - ProcessBlockHooks(p peer.ID, response graphsync.ResponseData, block graphsync.BlockData) hooks.UpdateResult -} - -// New generates a new request manager from a context, network, and selectorQuerier -func New(ctx context.Context, - asyncLoader AsyncLoader, - linkSystem ipld.LinkSystem, - requestHooks RequestHooks, - responseHooks ResponseHooks, - blockHooks BlockHooks, - networkErrorListeners *listeners.NetworkErrorListeners, -) *RequestManager { - ctx, cancel := context.WithCancel(ctx) - return &RequestManager{ - ctx: ctx, - cancel: cancel, - asyncLoader: asyncLoader, - disconnectNotif: pubsub.New(disconnectDispatcher), - linkSystem: linkSystem, - rc: newResponseCollector(ctx), - messages: make(chan requestManagerMessage, 16), - inProgressRequestStatuses: make(map[graphsync.RequestID]*inProgressRequestStatus), - requestHooks: requestHooks, - responseHooks: responseHooks, - blockHooks: blockHooks, - networkErrorListeners: networkErrorListeners, - } -} - -// SetDelegate specifies who will send messages out to the internet. -func (rm *RequestManager) SetDelegate(peerHandler PeerHandler) { - rm.peerHandler = peerHandler -} - -type inProgressRequest struct { - requestID graphsync.RequestID - request gsmsg.GraphSyncRequest - incoming chan graphsync.ResponseProgress - incomingError chan error -} - -type newRequestMessage struct { - p peer.ID - root ipld.Link - selector ipld.Node - extensions []graphsync.ExtensionData - inProgressRequestChan chan<- inProgressRequest -} - -// SendRequest initiates a new GraphSync request to the given peer. -func (rm *RequestManager) SendRequest(ctx context.Context, - p peer.ID, - root ipld.Link, - selectorNode ipld.Node, - extensions ...graphsync.ExtensionData) (<-chan graphsync.ResponseProgress, <-chan error) { - if _, err := selector.ParseSelector(selectorNode); err != nil { - return rm.singleErrorResponse(fmt.Errorf("invalid selector spec")) - } - - inProgressRequestChan := make(chan inProgressRequest) - - select { - case rm.messages <- &newRequestMessage{p, root, selectorNode, extensions, inProgressRequestChan}: - case <-rm.ctx.Done(): - return rm.emptyResponse() - case <-ctx.Done(): - return rm.emptyResponse() - } - var receivedInProgressRequest inProgressRequest - select { - case <-rm.ctx.Done(): - return rm.emptyResponse() - case receivedInProgressRequest = <-inProgressRequestChan: - } - - // If the connection to the peer is disconnected, fire an error - unsub := rm.listenForDisconnect(p, func(neterr error) { - rm.networkErrorListeners.NotifyNetworkErrorListeners(p, receivedInProgressRequest.request, neterr) - }) - - return rm.rc.collectResponses(ctx, - receivedInProgressRequest.incoming, - receivedInProgressRequest.incomingError, - func() { - rm.cancelRequest(receivedInProgressRequest.requestID, - receivedInProgressRequest.incoming, - receivedInProgressRequest.incomingError) - }, - // Once the request has completed, stop listening for disconnect events - unsub, - ) -} - -// Dispatch the Disconnect event to subscribers -func disconnectDispatcher(p pubsub.Event, subscriberFn pubsub.SubscriberFn) error { - listener := subscriberFn.(func(peer.ID)) - listener(p.(peer.ID)) - return nil -} - -// Listen for the Disconnect event for the given peer -func (rm *RequestManager) listenForDisconnect(p peer.ID, onDisconnect func(neterr error)) func() { - // Subscribe to Disconnect notifications - return rm.disconnectNotif.Subscribe(func(evtPeer peer.ID) { - // If the peer is the one we're interested in, call the listener - if evtPeer == p { - onDisconnect(fmt.Errorf("disconnected from peer %s", p)) - } - }) -} - -// Disconnected is called when a peer disconnects -func (rm *RequestManager) Disconnected(p peer.ID) { - // Notify any listeners that a peer has disconnected - _ = rm.disconnectNotif.Publish(p) -} - -func (rm *RequestManager) emptyResponse() (chan graphsync.ResponseProgress, chan error) { - ch := make(chan graphsync.ResponseProgress) - close(ch) - errCh := make(chan error) - close(errCh) - return ch, errCh -} - -func (rm *RequestManager) singleErrorResponse(err error) (chan graphsync.ResponseProgress, chan error) { - ch := make(chan graphsync.ResponseProgress) - close(ch) - errCh := make(chan error, 1) - errCh <- err - close(errCh) - return ch, errCh -} - -type cancelRequestMessage struct { - requestID graphsync.RequestID - isPause bool - onTerminated chan error - terminalError error -} - -func (rm *RequestManager) cancelRequest(requestID graphsync.RequestID, - incomingResponses chan graphsync.ResponseProgress, - incomingErrors chan error) { - cancelMessageChannel := rm.messages - for cancelMessageChannel != nil || incomingResponses != nil || incomingErrors != nil { - select { - case cancelMessageChannel <- &cancelRequestMessage{requestID, false, nil, nil}: - cancelMessageChannel = nil - // clear out any remaining responses, in case and "incoming reponse" - // messages get processed before our cancel message - case _, ok := <-incomingResponses: - if !ok { - incomingResponses = nil - } - case _, ok := <-incomingErrors: - if !ok { - incomingErrors = nil - } - case <-rm.ctx.Done(): - return - } - } -} - -// CancelRequest cancels the given request ID and waits for the request to terminate -func (rm *RequestManager) CancelRequest(ctx context.Context, requestID graphsync.RequestID) error { - terminated := make(chan error, 1) - return rm.sendSyncMessage(&cancelRequestMessage{requestID, false, terminated, graphsync.RequestClientCancelledErr{}}, terminated, ctx.Done()) -} - -type processResponseMessage struct { - p peer.ID - responses []gsmsg.GraphSyncResponse - blks []blocks.Block -} - -// ProcessResponses ingests the given responses from the network and -// and updates the in progress requests based on those responses. -func (rm *RequestManager) ProcessResponses(p peer.ID, responses []gsmsg.GraphSyncResponse, - blks []blocks.Block) { - select { - case rm.messages <- &processResponseMessage{p, responses, blks}: - case <-rm.ctx.Done(): - } -} - -type unpauseRequestMessage struct { - id graphsync.RequestID - extensions []graphsync.ExtensionData - response chan error -} - -// UnpauseRequest unpauses a request that was paused in a block hook based request ID -// Can also send extensions with unpause -func (rm *RequestManager) UnpauseRequest(requestID graphsync.RequestID, extensions ...graphsync.ExtensionData) error { - response := make(chan error, 1) - return rm.sendSyncMessage(&unpauseRequestMessage{requestID, extensions, response}, response, nil) -} - -type pauseRequestMessage struct { - id graphsync.RequestID - response chan error -} - -// PauseRequest pauses an in progress request (may take 1 or more blocks to process) -func (rm *RequestManager) PauseRequest(requestID graphsync.RequestID) error { - response := make(chan error, 1) - return rm.sendSyncMessage(&pauseRequestMessage{requestID, response}, response, nil) -} - -func (rm *RequestManager) sendSyncMessage(message requestManagerMessage, response chan error, done <-chan struct{}) error { - select { - case <-rm.ctx.Done(): - return errors.New("context cancelled") - case <-done: - return errors.New("context cancelled") - case rm.messages <- message: - } - select { - case <-rm.ctx.Done(): - return errors.New("context cancelled") - case <-done: - return errors.New("context cancelled") - case err := <-response: - return err - } -} - -// Startup starts processing for the WantManager. -func (rm *RequestManager) Startup() { - go rm.run() -} - -// Shutdown ends processing for the want manager. -func (rm *RequestManager) Shutdown() { - rm.cancel() -} - -func (rm *RequestManager) run() { - // NOTE: Do not open any streams or connections from anywhere in this - // event loop. Really, just don't do anything likely to block. - defer rm.cleanupInProcessRequests() - - for { - select { - case message := <-rm.messages: - message.handle(rm) - case <-rm.ctx.Done(): - return - } - } -} - -func (rm *RequestManager) cleanupInProcessRequests() { - for _, requestStatus := range rm.inProgressRequestStatuses { - requestStatus.cancelFn() - } -} - -type terminateRequestMessage struct { - requestID graphsync.RequestID -} - -func (nrm *newRequestMessage) setupRequest(requestID graphsync.RequestID, rm *RequestManager) (gsmsg.GraphSyncRequest, chan graphsync.ResponseProgress, chan error) { - log.Infow("graphsync request initiated", "request id", requestID, "peer", nrm.p, "root", nrm.root) - - request, hooksResult, err := rm.validateRequest(requestID, nrm.p, nrm.root, nrm.selector, nrm.extensions) - if err != nil { - rp, err := rm.singleErrorResponse(err) - return request, rp, err - } - doNotSendCidsData, has := request.Extension(graphsync.ExtensionDoNotSendCIDs) - var doNotSendCids *cid.Set - if has { - doNotSendCids, err = cidset.DecodeCidSet(doNotSendCidsData) - if err != nil { - rp, err := rm.singleErrorResponse(err) - return request, rp, err - } - } else { - doNotSendCids = cid.NewSet() - } - ctx, cancel := context.WithCancel(rm.ctx) - p := nrm.p - resumeMessages := make(chan []graphsync.ExtensionData, 1) - pauseMessages := make(chan struct{}, 1) - terminalError := make(chan error, 1) - requestStatus := &inProgressRequestStatus{ - ctx: ctx, startTime: time.Now(), cancelFn: cancel, p: p, resumeMessages: resumeMessages, pauseMessages: pauseMessages, terminalError: terminalError, - } - lastResponse := &requestStatus.lastResponse - lastResponse.Store(gsmsg.NewResponse(request.ID(), graphsync.RequestAcknowledged)) - rm.inProgressRequestStatuses[request.ID()] = requestStatus - incoming, incomingError := executor.ExecutionEnv{ - Ctx: rm.ctx, - SendRequest: rm.sendRequest, - TerminateRequest: rm.terminateRequest, - RunBlockHooks: rm.processBlockHooks, - Loader: rm.asyncLoader.AsyncLoad, - LinkSystem: rm.linkSystem, - }.Start( - executor.RequestExecution{ - Ctx: ctx, - P: p, - Request: request, - TerminalError: terminalError, - LastResponse: lastResponse, - DoNotSendCids: doNotSendCids, - NodePrototypeChooser: hooksResult.CustomChooser, - ResumeMessages: resumeMessages, - PauseMessages: pauseMessages, - }) - return request, incoming, incomingError -} - -func (nrm *newRequestMessage) handle(rm *RequestManager) { - var ipr inProgressRequest - ipr.requestID = rm.nextRequestID - rm.nextRequestID++ - ipr.request, ipr.incoming, ipr.incomingError = nrm.setupRequest(ipr.requestID, rm) - - select { - case nrm.inProgressRequestChan <- ipr: - case <-rm.ctx.Done(): - } -} - -func (trm *terminateRequestMessage) handle(rm *RequestManager) { - ipr, ok := rm.inProgressRequestStatuses[trm.requestID] - if ok { - log.Infow("graphsync request complete", "request id", trm.requestID, "peer", ipr.p, "total time", time.Since(ipr.startTime)) - } - delete(rm.inProgressRequestStatuses, trm.requestID) - rm.asyncLoader.CleanupRequest(trm.requestID) - if ok { - for _, onTerminated := range ipr.onTerminated { - select { - case <-rm.ctx.Done(): - case onTerminated <- nil: - } - } - } -} - -func (crm *cancelRequestMessage) handle(rm *RequestManager) { - inProgressRequestStatus, ok := rm.inProgressRequestStatuses[crm.requestID] - if !ok { - if crm.onTerminated != nil { - select { - case crm.onTerminated <- graphsync.RequestNotFoundErr{}: - case <-rm.ctx.Done(): - } - } - return - } - - if crm.onTerminated != nil { - inProgressRequestStatus.onTerminated = append(inProgressRequestStatus.onTerminated, crm.onTerminated) - } - if crm.terminalError != nil { - select { - case inProgressRequestStatus.terminalError <- crm.terminalError: - default: - } - } - - rm.sendRequest(inProgressRequestStatus.p, gsmsg.CancelRequest(crm.requestID)) - if crm.isPause { - inProgressRequestStatus.paused = true - } else { - inProgressRequestStatus.cancelFn() - } -} - -func (prm *processResponseMessage) handle(rm *RequestManager) { - filteredResponses := rm.processExtensions(prm.responses, prm.p) - filteredResponses = rm.filterResponsesForPeer(filteredResponses, prm.p) - rm.updateLastResponses(filteredResponses) - responseMetadata := metadataForResponses(filteredResponses) - rm.asyncLoader.ProcessResponse(responseMetadata, prm.blks) - rm.processTerminations(filteredResponses) -} - -func (rm *RequestManager) filterResponsesForPeer(responses []gsmsg.GraphSyncResponse, p peer.ID) []gsmsg.GraphSyncResponse { - responsesForPeer := make([]gsmsg.GraphSyncResponse, 0, len(responses)) - for _, response := range responses { - requestStatus, ok := rm.inProgressRequestStatuses[response.RequestID()] - if !ok || requestStatus.p != p { - continue - } - responsesForPeer = append(responsesForPeer, response) - } - return responsesForPeer -} - -func (rm *RequestManager) processExtensions(responses []gsmsg.GraphSyncResponse, p peer.ID) []gsmsg.GraphSyncResponse { - remainingResponses := make([]gsmsg.GraphSyncResponse, 0, len(responses)) - for _, response := range responses { - success := rm.processExtensionsForResponse(p, response) - if success { - remainingResponses = append(remainingResponses, response) - } - } - return remainingResponses -} - -func (rm *RequestManager) updateLastResponses(responses []gsmsg.GraphSyncResponse) { - for _, response := range responses { - rm.inProgressRequestStatuses[response.RequestID()].lastResponse.Store(response) - } -} - -func (rm *RequestManager) processExtensionsForResponse(p peer.ID, response gsmsg.GraphSyncResponse) bool { - result := rm.responseHooks.ProcessResponseHooks(p, response) - if len(result.Extensions) > 0 { - updateRequest := gsmsg.UpdateRequest(response.RequestID(), result.Extensions...) - rm.sendRequest(p, updateRequest) - } - if result.Err != nil { - requestStatus, ok := rm.inProgressRequestStatuses[response.RequestID()] - if !ok { - return false - } - responseError := graphsync.RequestFailedUnknown.AsError() - select { - case requestStatus.terminalError <- responseError: - default: - } - rm.sendRequest(p, gsmsg.CancelRequest(response.RequestID())) - requestStatus.cancelFn() - return false - } - return true -} - -func (rm *RequestManager) processTerminations(responses []gsmsg.GraphSyncResponse) { - for _, response := range responses { - if response.Status().IsTerminal() { - if response.Status().IsFailure() { - requestStatus := rm.inProgressRequestStatuses[response.RequestID()] - responseError := response.Status().AsError() - select { - case requestStatus.terminalError <- responseError: - default: - } - requestStatus.cancelFn() - } - rm.asyncLoader.CompleteResponsesFor(response.RequestID()) - } - } -} - -func (rm *RequestManager) processBlockHooks(p peer.ID, response graphsync.ResponseData, block graphsync.BlockData) error { - result := rm.blockHooks.ProcessBlockHooks(p, response, block) - if len(result.Extensions) > 0 { - updateRequest := gsmsg.UpdateRequest(response.RequestID(), result.Extensions...) - rm.sendRequest(p, updateRequest) - } - if result.Err != nil { - _, isPause := result.Err.(hooks.ErrPaused) - select { - case <-rm.ctx.Done(): - case rm.messages <- &cancelRequestMessage{response.RequestID(), isPause, nil, nil}: - } - } - return result.Err -} - -func (rm *RequestManager) terminateRequest(requestID graphsync.RequestID) { - select { - case <-rm.ctx.Done(): - case rm.messages <- &terminateRequestMessage{requestID}: - } -} - -func (rm *RequestManager) validateRequest(requestID graphsync.RequestID, p peer.ID, root ipld.Link, selectorSpec ipld.Node, extensions []graphsync.ExtensionData) (gsmsg.GraphSyncRequest, hooks.RequestResult, error) { - _, err := ipldutil.EncodeNode(selectorSpec) - if err != nil { - return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err - } - _, err = selector.ParseSelector(selectorSpec) - if err != nil { - return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err - } - asCidLink, ok := root.(cidlink.Link) - if !ok { - return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, 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 - } - request = request.ReplaceExtensions([]graphsync.ExtensionData{ - { - Name: graphsync.ExtensionDeDupByKey, - Data: dedupData, - }, - }) - } - err = rm.asyncLoader.StartRequest(requestID, hooksResult.PersistenceOption) - if err != nil { - return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err - } - return request, hooksResult, nil -} - -type reqSubscriber struct { - p peer.ID - request gsmsg.GraphSyncRequest - networkErrorListeners *listeners.NetworkErrorListeners -} - -func (r *reqSubscriber) OnNext(topic notifications.Topic, event notifications.Event) { - mqEvt, isMQEvt := event.(messagequeue.Event) - if !isMQEvt || mqEvt.Name != messagequeue.Error { - return - } - - r.networkErrorListeners.NotifyNetworkErrorListeners(r.p, r.request, mqEvt.Err) - //r.re.networkError <- mqEvt.Err - //r.re.terminateRequest() -} - -func (r reqSubscriber) OnClose(topic notifications.Topic) { -} - -const requestNetworkError = "request_network_error" - -func (rm *RequestManager) sendRequest(p peer.ID, request gsmsg.GraphSyncRequest) { - sub := notifications.NewTopicDataSubscriber(&reqSubscriber{p, request, rm.networkErrorListeners}) - failNotifee := notifications.Notifee{Data: requestNetworkError, Subscriber: sub} - rm.peerHandler.AllocateAndBuildMessage(p, 0, func(builder *gsmsg.Builder) { - builder.AddRequest(request) - }, []notifications.Notifee{failNotifee}) -} - -func (urm *unpauseRequestMessage) unpause(rm *RequestManager) error { - inProgressRequestStatus, ok := rm.inProgressRequestStatuses[urm.id] - if !ok { - return graphsync.RequestNotFoundErr{} - } - if !inProgressRequestStatus.paused { - return errors.New("request is not paused") - } - inProgressRequestStatus.paused = false - select { - case <-inProgressRequestStatus.pauseMessages: - rm.sendRequest(inProgressRequestStatus.p, gsmsg.UpdateRequest(urm.id, urm.extensions...)) - return nil - case <-rm.ctx.Done(): - return errors.New("context cancelled") - case inProgressRequestStatus.resumeMessages <- urm.extensions: - return nil - } -} -func (urm *unpauseRequestMessage) handle(rm *RequestManager) { - err := urm.unpause(rm) - select { - case <-rm.ctx.Done(): - case urm.response <- err: - } -} -func (prm *pauseRequestMessage) pause(rm *RequestManager) error { - inProgressRequestStatus, ok := rm.inProgressRequestStatuses[prm.id] - if !ok { - return graphsync.RequestNotFoundErr{} - } - if inProgressRequestStatus.paused { - return errors.New("request is already paused") - } - inProgressRequestStatus.paused = true - select { - case <-rm.ctx.Done(): - return errors.New("context cancelled") - case inProgressRequestStatus.pauseMessages <- struct{}{}: - return nil - } -} -func (prm *pauseRequestMessage) handle(rm *RequestManager) { - err := prm.pause(rm) - select { - case <-rm.ctx.Done(): - case prm.response <- err: - } -} diff --git a/requestmanager/requestmanager_test.go b/requestmanager/requestmanager_test.go index 7d383b6c..caaadad6 100644 --- a/requestmanager/requestmanager_test.go +++ b/requestmanager/requestmanager_test.go @@ -93,8 +93,8 @@ func TestNormalSimultaneousFetch(t *testing.T) { blockChain2 := testutil.SetupBlockChain(ctx, t, td.persistence, 100, 5) - returnedResponseChan1, returnedErrorChan1 := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) - returnedResponseChan2, returnedErrorChan2 := td.requestManager.SendRequest(requestCtx, peers[0], blockChain2.TipLink, blockChain2.Selector()) + returnedResponseChan1, returnedErrorChan1 := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan2, returnedErrorChan2 := td.requestManager.NewRequest(requestCtx, peers[0], blockChain2.TipLink, blockChain2.Selector()) requestRecords := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 2) @@ -172,8 +172,8 @@ func TestCancelRequestInProgress(t *testing.T) { defer cancel2() peers := testutil.GeneratePeers(1) - returnedResponseChan1, returnedErrorChan1 := td.requestManager.SendRequest(requestCtx1, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) - returnedResponseChan2, returnedErrorChan2 := td.requestManager.SendRequest(requestCtx2, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan1, returnedErrorChan1 := td.requestManager.NewRequest(requestCtx1, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan2, returnedErrorChan2 := td.requestManager.NewRequest(requestCtx2, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) requestRecords := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 2) @@ -232,7 +232,7 @@ func TestCancelRequestImperativeNoMoreBlocks(t *testing.T) { } }) - _, returnedErrorChan1 := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + _, returnedErrorChan1 := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) requestRecords := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1) @@ -276,7 +276,7 @@ func TestCancelManagerExitsGracefully(t *testing.T) { defer cancel() peers := testutil.GeneratePeers(1) - returnedResponseChan, returnedErrorChan := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] @@ -308,7 +308,7 @@ func TestFailedRequest(t *testing.T) { defer cancel() peers := testutil.GeneratePeers(1) - returnedResponseChan, returnedErrorChan := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] failedResponses := []gsmsg.GraphSyncResponse{ @@ -328,7 +328,7 @@ func TestLocallyFulfilledFirstRequestFailsLater(t *testing.T) { defer cancel() peers := testutil.GeneratePeers(1) - returnedResponseChan, returnedErrorChan := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] @@ -359,7 +359,7 @@ func TestLocallyFulfilledFirstRequestSucceedsLater(t *testing.T) { td.responseHooks.Register(func(p peer.ID, response graphsync.ResponseData, hookActions graphsync.IncomingResponseHookActions) { close(called) }) - returnedResponseChan, returnedErrorChan := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] @@ -387,7 +387,7 @@ func TestRequestReturnsMissingBlocks(t *testing.T) { defer cancel() peers := testutil.GeneratePeers(1) - returnedResponseChan, returnedErrorChan := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] @@ -419,7 +419,7 @@ func TestDisconnectNotification(t *testing.T) { // Send a request to the target peer targetPeer := peers[0] - td.requestManager.SendRequest(requestCtx, targetPeer, td.blockChain.TipLink, td.blockChain.Selector()) + td.requestManager.NewRequest(requestCtx, targetPeer, td.blockChain.TipLink, td.blockChain.Selector()) // Disconnect a random peer, should not fire any events randomPeer := peers[1] @@ -465,7 +465,7 @@ func TestEncodingExtensions(t *testing.T) { } } td.responseHooks.Register(hook) - returnedResponseChan, returnedErrorChan := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector(), td.extension1, td.extension2) + returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector(), td.extension1, td.extension2) rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] @@ -579,7 +579,7 @@ func TestBlockHooks(t *testing.T) { } } td.blockHooks.Register(hook) - returnedResponseChan, returnedErrorChan := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector(), td.extension1, td.extension2) + returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector(), td.extension1, td.extension2) rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] @@ -743,8 +743,8 @@ func TestOutgoingRequestHooks(t *testing.T) { } td.requestHooks.Register(hook) - returnedResponseChan1, returnedErrorChan1 := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector(), td.extension1) - returnedResponseChan2, returnedErrorChan2 := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan1, returnedErrorChan1 := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector(), td.extension1) + returnedResponseChan2, returnedErrorChan2 := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) requestRecords := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 2) @@ -809,7 +809,7 @@ func TestPauseResume(t *testing.T) { td.blockHooks.Register(hook) // Start request - returnedResponseChan, returnedErrorChan := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] @@ -895,7 +895,7 @@ func TestPauseResumeExternal(t *testing.T) { td.blockHooks.Register(hook) // Start request - returnedResponseChan, returnedErrorChan := td.requestManager.SendRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) + returnedResponseChan, returnedErrorChan := td.requestManager.NewRequest(requestCtx, peers[0], td.blockChain.TipLink, td.blockChain.Selector()) rr := readNNetworkRequests(requestCtx, t, td.requestRecordChan, 1)[0] diff --git a/requestmanager/server.go b/requestmanager/server.go new file mode 100644 index 00000000..afe372a0 --- /dev/null +++ b/requestmanager/server.go @@ -0,0 +1,296 @@ +package requestmanager + +import ( + "context" + "errors" + "fmt" + "time" + + blocks "github.com/ipfs/go-block-format" + "github.com/ipfs/go-cid" + "github.com/ipfs/go-graphsync" + "github.com/ipfs/go-graphsync/cidset" + "github.com/ipfs/go-graphsync/dedupkey" + "github.com/ipfs/go-graphsync/ipldutil" + gsmsg "github.com/ipfs/go-graphsync/message" + "github.com/ipfs/go-graphsync/requestmanager/executor" + "github.com/ipfs/go-graphsync/requestmanager/hooks" + "github.com/ipld/go-ipld-prime" + cidlink "github.com/ipld/go-ipld-prime/linking/cid" + "github.com/ipld/go-ipld-prime/traversal/selector" + "github.com/libp2p/go-libp2p-core/peer" +) + +// The code in this file implements the internal thread for the request manager. +// These functions can modify the internal state of the RequestManager + +func (rm *RequestManager) run() { + // NOTE: Do not open any streams or connections from anywhere in this + // event loop. Really, just don't do anything likely to block. + defer rm.cleanupInProcessRequests() + + for { + select { + case message := <-rm.messages: + message.handle(rm) + case <-rm.ctx.Done(): + return + } + } +} + +func (rm *RequestManager) cleanupInProcessRequests() { + for _, requestStatus := range rm.inProgressRequestStatuses { + requestStatus.cancelFn() + } +} + +func (rm *RequestManager) setupRequest(p peer.ID, root ipld.Link, selector ipld.Node, extensions []graphsync.ExtensionData) (gsmsg.GraphSyncRequest, chan graphsync.ResponseProgress, chan error) { + requestID := rm.nextRequestID + rm.nextRequestID++ + + log.Infow("graphsync request initiated", "request id", requestID, "peer", p, "root", root) + + request, hooksResult, err := rm.validateRequest(requestID, p, root, selector, extensions) + if err != nil { + rp, err := rm.singleErrorResponse(err) + return request, rp, err + } + doNotSendCidsData, has := request.Extension(graphsync.ExtensionDoNotSendCIDs) + var doNotSendCids *cid.Set + if has { + doNotSendCids, err = cidset.DecodeCidSet(doNotSendCidsData) + if err != nil { + rp, err := rm.singleErrorResponse(err) + return request, rp, err + } + } else { + doNotSendCids = cid.NewSet() + } + ctx, cancel := context.WithCancel(rm.ctx) + resumeMessages := make(chan []graphsync.ExtensionData, 1) + pauseMessages := make(chan struct{}, 1) + terminalError := make(chan error, 1) + requestStatus := &inProgressRequestStatus{ + ctx: ctx, startTime: time.Now(), cancelFn: cancel, p: p, resumeMessages: resumeMessages, pauseMessages: pauseMessages, terminalError: terminalError, + } + lastResponse := &requestStatus.lastResponse + lastResponse.Store(gsmsg.NewResponse(request.ID(), graphsync.RequestAcknowledged)) + rm.inProgressRequestStatuses[request.ID()] = requestStatus + incoming, incomingError := executor.ExecutionEnv{ + Ctx: rm.ctx, + SendRequest: rm.SendRequest, + TerminateRequest: rm.TerminateRequest, + RunBlockHooks: rm.ProcessBlockHooks, + Loader: rm.asyncLoader.AsyncLoad, + LinkSystem: rm.linkSystem, + }.Start( + executor.RequestExecution{ + Ctx: ctx, + P: p, + Request: request, + TerminalError: terminalError, + LastResponse: lastResponse, + DoNotSendCids: doNotSendCids, + NodePrototypeChooser: hooksResult.CustomChooser, + ResumeMessages: resumeMessages, + PauseMessages: pauseMessages, + }) + return request, incoming, incomingError +} + +func (rm *RequestManager) terminateRequest(requestID graphsync.RequestID) { + ipr, ok := rm.inProgressRequestStatuses[requestID] + if ok { + log.Infow("graphsync request complete", "request id", requestID, "peer", ipr.p, "total time", time.Since(ipr.startTime)) + } + delete(rm.inProgressRequestStatuses, requestID) + rm.asyncLoader.CleanupRequest(requestID) + if ok { + for _, onTerminated := range ipr.onTerminated { + select { + case <-rm.ctx.Done(): + case onTerminated <- nil: + } + } + } +} + +func (rm *RequestManager) cancelRequest(requestID graphsync.RequestID, isPause bool, onTerminated chan<- error, terminalError error) { + inProgressRequestStatus, ok := rm.inProgressRequestStatuses[requestID] + if !ok { + if onTerminated != nil { + select { + case onTerminated <- graphsync.RequestNotFoundErr{}: + case <-rm.ctx.Done(): + } + } + return + } + + if onTerminated != nil { + inProgressRequestStatus.onTerminated = append(inProgressRequestStatus.onTerminated, onTerminated) + } + if terminalError != nil { + select { + case inProgressRequestStatus.terminalError <- terminalError: + default: + } + } + + rm.SendRequest(inProgressRequestStatus.p, gsmsg.CancelRequest(requestID)) + if isPause { + inProgressRequestStatus.paused = true + } else { + inProgressRequestStatus.cancelFn() + } +} + +func (rm *RequestManager) processResponseMessage(p peer.ID, responses []gsmsg.GraphSyncResponse, blks []blocks.Block) { + filteredResponses := rm.processExtensions(responses, p) + filteredResponses = rm.filterResponsesForPeer(filteredResponses, p) + rm.updateLastResponses(filteredResponses) + responseMetadata := metadataForResponses(filteredResponses) + rm.asyncLoader.ProcessResponse(responseMetadata, blks) + rm.processTerminations(filteredResponses) +} + +func (rm *RequestManager) filterResponsesForPeer(responses []gsmsg.GraphSyncResponse, p peer.ID) []gsmsg.GraphSyncResponse { + responsesForPeer := make([]gsmsg.GraphSyncResponse, 0, len(responses)) + for _, response := range responses { + requestStatus, ok := rm.inProgressRequestStatuses[response.RequestID()] + if !ok || requestStatus.p != p { + continue + } + responsesForPeer = append(responsesForPeer, response) + } + return responsesForPeer +} + +func (rm *RequestManager) processExtensions(responses []gsmsg.GraphSyncResponse, p peer.ID) []gsmsg.GraphSyncResponse { + remainingResponses := make([]gsmsg.GraphSyncResponse, 0, len(responses)) + for _, response := range responses { + success := rm.processExtensionsForResponse(p, response) + if success { + remainingResponses = append(remainingResponses, response) + } + } + return remainingResponses +} + +func (rm *RequestManager) updateLastResponses(responses []gsmsg.GraphSyncResponse) { + for _, response := range responses { + rm.inProgressRequestStatuses[response.RequestID()].lastResponse.Store(response) + } +} + +func (rm *RequestManager) processExtensionsForResponse(p peer.ID, response gsmsg.GraphSyncResponse) bool { + result := rm.responseHooks.ProcessResponseHooks(p, response) + if len(result.Extensions) > 0 { + updateRequest := gsmsg.UpdateRequest(response.RequestID(), result.Extensions...) + rm.SendRequest(p, updateRequest) + } + if result.Err != nil { + requestStatus, ok := rm.inProgressRequestStatuses[response.RequestID()] + if !ok { + return false + } + responseError := graphsync.RequestFailedUnknown.AsError() + select { + case requestStatus.terminalError <- responseError: + default: + } + rm.SendRequest(p, gsmsg.CancelRequest(response.RequestID())) + requestStatus.cancelFn() + return false + } + return true +} + +func (rm *RequestManager) processTerminations(responses []gsmsg.GraphSyncResponse) { + for _, response := range responses { + if response.Status().IsTerminal() { + if response.Status().IsFailure() { + requestStatus := rm.inProgressRequestStatuses[response.RequestID()] + responseError := response.Status().AsError() + select { + case requestStatus.terminalError <- responseError: + default: + } + requestStatus.cancelFn() + } + rm.asyncLoader.CompleteResponsesFor(response.RequestID()) + } + } +} + +func (rm *RequestManager) validateRequest(requestID graphsync.RequestID, p peer.ID, root ipld.Link, selectorSpec ipld.Node, extensions []graphsync.ExtensionData) (gsmsg.GraphSyncRequest, hooks.RequestResult, error) { + _, err := ipldutil.EncodeNode(selectorSpec) + if err != nil { + return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err + } + _, err = selector.ParseSelector(selectorSpec) + if err != nil { + return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err + } + asCidLink, ok := root.(cidlink.Link) + if !ok { + return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, 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 + } + request = request.ReplaceExtensions([]graphsync.ExtensionData{ + { + Name: graphsync.ExtensionDeDupByKey, + Data: dedupData, + }, + }) + } + err = rm.asyncLoader.StartRequest(requestID, hooksResult.PersistenceOption) + if err != nil { + return gsmsg.GraphSyncRequest{}, hooks.RequestResult{}, err + } + return request, hooksResult, nil +} + +func (rm *RequestManager) unpause(id graphsync.RequestID, extensions []graphsync.ExtensionData) error { + inProgressRequestStatus, ok := rm.inProgressRequestStatuses[id] + if !ok { + return graphsync.RequestNotFoundErr{} + } + if !inProgressRequestStatus.paused { + return errors.New("request is not paused") + } + inProgressRequestStatus.paused = false + select { + case <-inProgressRequestStatus.pauseMessages: + rm.SendRequest(inProgressRequestStatus.p, gsmsg.UpdateRequest(id, extensions...)) + return nil + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case inProgressRequestStatus.resumeMessages <- extensions: + return nil + } +} + +func (rm *RequestManager) pause(id graphsync.RequestID) error { + inProgressRequestStatus, ok := rm.inProgressRequestStatuses[id] + if !ok { + return graphsync.RequestNotFoundErr{} + } + if inProgressRequestStatus.paused { + return errors.New("request is already paused") + } + inProgressRequestStatus.paused = true + select { + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case inProgressRequestStatus.pauseMessages <- struct{}{}: + return nil + } +} diff --git a/responsemanager/client.go b/responsemanager/client.go index 6611e453..f93834bf 100644 --- a/responsemanager/client.go +++ b/responsemanager/client.go @@ -18,6 +18,10 @@ import ( "github.com/ipfs/go-graphsync/responsemanager/responseassembler" ) +// The code in this file implements the public interface of the response manager. +// Functions in this file operate outside the internal thread and should +// NOT modify the internal state of the ResponseManager. + var log = logging.Logger("graphsync") const ( @@ -193,66 +197,75 @@ func New(ctx context.Context, } // ProcessRequests processes incoming requests for the given peer -func (rm *ResponseManager) ProcessRequests(p peer.ID, requests []gsmsg.GraphSyncRequest) { - rm.cast(&processRequestMessage{p, requests}) +func (rm *ResponseManager) ProcessRequests(ctx context.Context, p peer.ID, requests []gsmsg.GraphSyncRequest) { + rm.send(&processRequestMessage{p, requests}, ctx.Done()) } // UnpauseResponse unpauses a response that was previously paused func (rm *ResponseManager) UnpauseResponse(p peer.ID, requestID graphsync.RequestID, extensions ...graphsync.ExtensionData) error { response := make(chan error, 1) - return rm.call(&unpauseRequestMessage{p, requestID, response, extensions}, response) + rm.send(&unpauseRequestMessage{p, requestID, response, extensions}, nil) + select { + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case err := <-response: + return err + } } // PauseResponse pauses an in progress response (may take 1 or more blocks to process) func (rm *ResponseManager) PauseResponse(p peer.ID, requestID graphsync.RequestID) error { response := make(chan error, 1) - return rm.call(&pauseRequestMessage{p, requestID, response}, response) + rm.send(&pauseRequestMessage{p, requestID, response}, nil) + select { + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case err := <-response: + return err + } } // CancelResponse cancels an in progress response func (rm *ResponseManager) CancelResponse(p peer.ID, requestID graphsync.RequestID) error { response := make(chan error, 1) - return rm.call(&errorRequestMessage{p, requestID, errCancelledByCommand, response}, response) + rm.send(&errorRequestMessage{p, requestID, errCancelledByCommand, response}, nil) + select { + case <-rm.ctx.Done(): + return errors.New("context cancelled") + case err := <-response: + return err + } } // this is a test utility method to force all messages to get processed func (rm *ResponseManager) synchronize() { sync := make(chan error) - _ = rm.call(&synchronizeMessage{sync}, sync) + rm.send(&synchronizeMessage{sync}, nil) + select { + case <-rm.ctx.Done(): + case <-sync: + } } // StartTask starts the given task from the peer task queue func (rm *ResponseManager) StartTask(task *peertask.Task, responseTaskDataChan chan<- ResponseTaskData) { - rm.cast(&startTaskRequest{task, responseTaskDataChan}) + rm.send(&startTaskRequest{task, responseTaskDataChan}, nil) } // GetUpdates is called to read pending updates for a task and clear them func (rm *ResponseManager) GetUpdates(p peer.ID, requestID graphsync.RequestID, updatesChan chan<- []gsmsg.GraphSyncRequest) { - rm.cast(&responseUpdateRequest{responseKey{p, requestID}, updatesChan}) + rm.send(&responseUpdateRequest{responseKey{p, requestID}, updatesChan}, nil) } // FinishTask marks a task from the task queue as done func (rm *ResponseManager) FinishTask(task *peertask.Task, err error) { - rm.cast(&finishTaskRequest{task, err}) -} - -func (rm *ResponseManager) call(message responseManagerMessage, response chan error) error { - select { - case <-rm.ctx.Done(): - return errors.New("context cancelled") - case rm.messages <- message: - } - select { - case <-rm.ctx.Done(): - return errors.New("context cancelled") - case err := <-response: - return err - } + rm.send(&finishTaskRequest{task, err}, nil) } -func (rm *ResponseManager) cast(message responseManagerMessage) { +func (rm *ResponseManager) send(message responseManagerMessage, done <-chan struct{}) { select { case <-rm.ctx.Done(): + case <-done: case rm.messages <- message: } } diff --git a/responsemanager/responsemanager_test.go b/responsemanager/responsemanager_test.go index b921d4e8..2b527144 100644 --- a/responsemanager/responsemanager_test.go +++ b/responsemanager/responsemanager_test.go @@ -53,7 +53,7 @@ func TestIncomingQuery(t *testing.T) { td.requestHooks.Register(selectorvalidator.SelectorValidator(100)) responseManager.Startup() - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) testutil.AssertDoesReceive(td.ctx, t, td.completedRequestChan, "Should have completed request but didn't") for i := 0; i < len(blks); i++ { td.assertSendBlock() @@ -75,7 +75,7 @@ func TestCancellationQueryInProgress(t *testing.T) { cancelledListenerCalled <- struct{}{} }) responseManager.Startup() - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) // read one block td.assertSendBlock() @@ -84,7 +84,7 @@ func TestCancellationQueryInProgress(t *testing.T) { cancelRequests := []gsmsg.GraphSyncRequest{ gsmsg.CancelRequest(td.requestID), } - responseManager.ProcessRequests(td.p, cancelRequests) + responseManager.ProcessRequests(td.ctx, td.p, cancelRequests) responseManager.synchronize() testutil.AssertDoesReceive(td.ctx, t, cancelledListenerCalled, "should call cancelled listener") @@ -98,7 +98,7 @@ func TestCancellationViaCommand(t *testing.T) { responseManager := td.newResponseManager() td.requestHooks.Register(selectorvalidator.SelectorValidator(100)) responseManager.Startup() - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) // read one block td.assertSendBlock() @@ -117,13 +117,13 @@ func TestEarlyCancellation(t *testing.T) { responseManager := td.newResponseManager() td.requestHooks.Register(selectorvalidator.SelectorValidator(100)) responseManager.Startup() - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) // send a cancellation cancelRequests := []gsmsg.GraphSyncRequest{ gsmsg.CancelRequest(td.requestID), } - responseManager.ProcessRequests(td.p, cancelRequests) + responseManager.ProcessRequests(td.ctx, td.p, cancelRequests) responseManager.synchronize() @@ -143,7 +143,7 @@ func TestMissingContent(t *testing.T) { }) // delete the root block delete(td.blockStore, cidlink.Link{Cid: td.requests[0].Root()}) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestFailedContentNotFound) }) t.Run("missing other block", func(t *testing.T) { @@ -161,7 +161,7 @@ func TestMissingContent(t *testing.T) { break } } - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedPartial) }) } @@ -172,7 +172,7 @@ func TestValidationAndExtensions(t *testing.T) { defer td.cancel() responseManager := td.newResponseManager() responseManager.Startup() - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestRejected) }) @@ -184,7 +184,7 @@ func TestValidationAndExtensions(t *testing.T) { td.requestHooks.Register(func(p peer.ID, requestData graphsync.RequestData, hookActions graphsync.IncomingRequestHookActions) { hookActions.SendExtensionData(td.extensionResponse) }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestRejected) td.assertReceiveExtensionResponse() }) @@ -198,7 +198,7 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.ValidateRequest() hookActions.SendExtensionData(td.extensionResponse) }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertReceiveExtensionResponse() }) @@ -215,7 +215,7 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.SendExtensionData(td.extensionResponse) hookActions.TerminateWithError(errors.New("everything went to crap")) }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestFailedUnknown) td.assertReceiveExtensionResponse() }) @@ -231,7 +231,7 @@ func TestValidationAndExtensions(t *testing.T) { }) // hook validates request - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertReceiveExtensionResponse() @@ -239,7 +239,7 @@ func TestValidationAndExtensions(t *testing.T) { unregister() // now same request should fail - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestRejected) }) @@ -254,7 +254,7 @@ func TestValidationAndExtensions(t *testing.T) { }) // request fails with base loader reading from block store that's missing data - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestFailedContentNotFound) err := td.peristenceOptions.Register("chainstore", td.persistence) @@ -267,7 +267,7 @@ func TestValidationAndExtensions(t *testing.T) { } }) // hook uses different loader that should make request succeed - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertReceiveExtensionResponse() }) @@ -290,7 +290,7 @@ func TestValidationAndExtensions(t *testing.T) { }) // with default chooser, customer chooser not called - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) require.Equal(t, 0, customChooserCallCount) @@ -303,7 +303,7 @@ func TestValidationAndExtensions(t *testing.T) { }) // verify now that request succeeds and uses custom chooser - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertReceiveExtensionResponse() require.Equal(t, 5, customChooserCallCount) @@ -331,7 +331,7 @@ func TestValidationAndExtensions(t *testing.T) { Data: data, }), } - responseManager.ProcessRequests(td.p, requests) + responseManager.ProcessRequests(td.ctx, td.p, requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertIgnoredCids(set) }) @@ -352,7 +352,7 @@ func TestValidationAndExtensions(t *testing.T) { Data: data, }), } - responseManager.ProcessRequests(td.p, requests) + responseManager.ProcessRequests(td.ctx, td.p, requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) td.assertDedupKey("applesauce") }) @@ -365,7 +365,7 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.ValidateRequest() hookActions.PauseResponse() }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertPausedRequest() td.assertRequestDoesNotCompleteWhilePaused() testutil.AssertChannelEmpty(t, td.sentResponses, "should not send more blocks") @@ -385,7 +385,7 @@ func TestValidationAndExtensions(t *testing.T) { td.blockHooks.Register(func(p peer.ID, requestData graphsync.RequestData, blockData graphsync.BlockData, hookActions graphsync.OutgoingBlockHookActions) { hookActions.SendExtensionData(td.extensionResponse) }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) for i := 0; i < td.blockChainLength; i++ { td.assertReceiveExtensionResponse() @@ -403,7 +403,7 @@ func TestValidationAndExtensions(t *testing.T) { td.blockHooks.Register(func(p peer.ID, requestData graphsync.RequestData, blockData graphsync.BlockData, hookActions graphsync.OutgoingBlockHookActions) { hookActions.TerminateWithError(errors.New("failed")) }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertCompleteRequestWith(graphsync.RequestFailedUnknown) }) @@ -423,7 +423,7 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.PauseResponse() } }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertRequestDoesNotCompleteWhilePaused() td.verifyNResponses(blockCount) td.assertPausedRequest() @@ -450,7 +450,7 @@ func TestValidationAndExtensions(t *testing.T) { require.NoError(t, err) } }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertRequestDoesNotCompleteWhilePaused() td.verifyNResponses(blockCount + 1) td.assertPausedRequest() @@ -476,7 +476,7 @@ func TestValidationAndExtensions(t *testing.T) { err := responseManager.UnpauseResponse(td.p, td.requestID) require.NoError(t, err) }() - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertPausedRequest() td.verifyNResponses(td.blockChainLength) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) @@ -506,12 +506,12 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.UnpauseResponse() } }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertRequestDoesNotCompleteWhilePaused() td.verifyNResponses(blockCount) td.assertPausedRequest() - responseManager.ProcessRequests(td.p, td.updateRequests) + responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) }) @@ -540,9 +540,9 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.SendExtensionData(td.extensionResponse) } }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) testutil.AssertDoesReceive(td.ctx, t, sent, "sends blocks") - responseManager.ProcessRequests(td.p, td.updateRequests) + responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) responseManager.synchronize() close(wait) td.assertCompleteRequestWith(graphsync.RequestCompletedFull) @@ -570,12 +570,12 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.SendExtensionData(td.extensionResponse) } }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.verifyNResponses(blockCount) td.assertPausedRequest() // send update - responseManager.ProcessRequests(td.p, td.updateRequests) + responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) // receive data td.assertReceiveExtensionResponse() @@ -610,9 +610,9 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.TerminateWithError(errors.New("something went wrong")) } }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) testutil.AssertDoesReceive(td.ctx, t, sent, "sends blocks") - responseManager.ProcessRequests(td.p, td.updateRequests) + responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) responseManager.synchronize() close(wait) td.assertCompleteRequestWith(graphsync.RequestFailedUnknown) @@ -639,12 +639,12 @@ func TestValidationAndExtensions(t *testing.T) { hookActions.TerminateWithError(errors.New("something went wrong")) } }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.verifyNResponses(blockCount) td.assertPausedRequest() // send update - responseManager.ProcessRequests(td.p, td.updateRequests) + responseManager.ProcessRequests(td.ctx, td.p, td.updateRequests) td.assertCompleteRequestWith(graphsync.RequestFailedUnknown) // cannot unpause @@ -665,7 +665,7 @@ func TestNetworkErrors(t *testing.T) { td.requestHooks.Register(func(p peer.ID, requestData graphsync.RequestData, hookActions graphsync.IncomingRequestHookActions) { hookActions.ValidateRequest() }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.verifyNResponses(td.blockChainLength) td.assertOnlyCompleteProcessingWith(graphsync.RequestCompletedFull) err := errors.New("something went wrong") @@ -678,7 +678,7 @@ func TestNetworkErrors(t *testing.T) { defer td.cancel() responseManager := td.newResponseManager() responseManager.Startup() - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertOnlyCompleteProcessingWith(graphsync.RequestRejected) err := errors.New("something went wrong") td.notifyStatusMessagesNetworkError(err) @@ -693,7 +693,7 @@ func TestNetworkErrors(t *testing.T) { td.requestHooks.Register(func(p peer.ID, requestData graphsync.RequestData, hookActions graphsync.IncomingRequestHookActions) { hookActions.ValidateRequest() }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertSendBlock() err := errors.New("something went wrong") td.notifyBlockSendsNetworkError(err) @@ -717,7 +717,7 @@ func TestNetworkErrors(t *testing.T) { hookActions.PauseResponse() } }) - responseManager.ProcessRequests(td.p, td.requests) + responseManager.ProcessRequests(td.ctx, td.p, td.requests) td.assertRequestDoesNotCompleteWhilePaused() td.verifyNResponsesOnlyProcessing(blockCount) td.assertPausedRequest() diff --git a/responsemanager/server.go b/responsemanager/server.go index 9dbfa79d..c06f3f3e 100644 --- a/responsemanager/server.go +++ b/responsemanager/server.go @@ -15,6 +15,9 @@ import ( "github.com/libp2p/go-libp2p-core/peer" ) +// The code in this file implements the internal thread for the response manager. +// These functions can modify the internal state of the ResponseManager + func (rm *ResponseManager) cleanupInProcessResponses() { for _, response := range rm.inProgressResponses { response.cancelFn() From 7d2d2a7ba04e9ef3dd60e21769c5f7d8087b4f32 Mon Sep 17 00:00:00 2001 From: hannahhoward Date: Tue, 21 Sep 2021 14:59:30 -0700 Subject: [PATCH 4/8] refactor(asyncloader): remove go routines remove the actor pattern from asyncloader, as its not needed. it works well as simply a locked data structure and is about 10x simpler --- impl/graphsync.go | 1 - requestmanager/asyncloader/asyncloader.go | 306 ++++-------------- .../asyncloader/asyncloader_test.go | 1 - 3 files changed, 69 insertions(+), 239 deletions(-) diff --git a/impl/graphsync.go b/impl/graphsync.go index 3531b8aa..43cdb12b 100644 --- a/impl/graphsync.go +++ b/impl/graphsync.go @@ -196,7 +196,6 @@ func New(parent context.Context, network gsnet.GraphSyncNetwork, requestAllocator: requestAllocator, } - asyncLoader.Startup() requestManager.SetDelegate(peerManager) requestManager.Startup() responseManager.Startup() diff --git a/requestmanager/asyncloader/asyncloader.go b/requestmanager/asyncloader/asyncloader.go index bcbccba9..0bb458b1 100644 --- a/requestmanager/asyncloader/asyncloader.go +++ b/requestmanager/asyncloader/asyncloader.go @@ -4,6 +4,7 @@ import ( "context" "errors" "io/ioutil" + "sync" blocks "github.com/ipfs/go-block-format" logging "github.com/ipfs/go-log/v2" @@ -20,10 +21,6 @@ import ( var log = logging.Logger("gs-asyncloader") -type loaderMessage interface { - handle(al *AsyncLoader) -} - type alternateQueue struct { responseCache *responsecache.ResponseCache loadAttemptQueue *loadattemptqueue.LoadAttemptQueue @@ -37,11 +34,10 @@ type Allocator interface { // AsyncLoader manages loading links asynchronously in as new responses // come in from the network type AsyncLoader struct { - ctx context.Context - cancel context.CancelFunc - incomingMessages chan loaderMessage - outgoingMessages chan loaderMessage + ctx context.Context + cancel context.CancelFunc + stateLk sync.Mutex defaultLinkSystem ipld.LinkSystem activeRequests map[graphsync.RequestID]struct{} requestQueues map[graphsync.RequestID]string @@ -59,8 +55,6 @@ func New(ctx context.Context, linkSystem ipld.LinkSystem, allocator Allocator) * return &AsyncLoader{ ctx: ctx, cancel: cancel, - incomingMessages: make(chan loaderMessage), - outgoingMessages: make(chan loaderMessage), defaultLinkSystem: linkSystem, activeRequests: make(map[graphsync.RequestID]struct{}), requestQueues: make(map[graphsync.RequestID]string), @@ -71,52 +65,72 @@ func New(ctx context.Context, linkSystem ipld.LinkSystem, allocator Allocator) * } } -// Startup starts processing of messages -func (al *AsyncLoader) Startup() { - go al.messageQueueWorker() - go al.run() -} - -// Shutdown finishes processing of messages -func (al *AsyncLoader) Shutdown() { - al.cancel() -} - // RegisterPersistenceOption registers a new loader/storer option for processing requests func (al *AsyncLoader) RegisterPersistenceOption(name string, lsys ipld.LinkSystem) error { - if name == "" { - return errors.New("persistence option must have a name") + al.stateLk.Lock() + defer al.stateLk.Unlock() + _, existing := al.alternateQueues[name] + if existing { + return errors.New("already registerd a persistence option with this name") } - response := make(chan error, 1) - err := al.sendSyncMessage(®isterPersistenceOptionMessage{name, lsys, response}, response) - return err + responseCache, loadAttemptQueue := setupAttemptQueue(lsys, al.allocator) + 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 { - if name == "" { - return errors.New("persistence option must have a name") + al.stateLk.Lock() + defer al.stateLk.Unlock() + _, ok := al.alternateQueues[name] + if !ok { + return errors.New("unknown persistence option") + } + for _, requestQueue := range al.requestQueues { + if name == requestQueue { + return errors.New("cannot unregister while requests are in progress") + } } - response := make(chan error, 1) - err := al.sendSyncMessage(&unregisterPersistenceOptionMessage{name, response}, response) - return err + 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 { - response := make(chan error, 1) - err := al.sendSyncMessage(&startRequestMessage{requestID, persistenceOption, response}, response) - return err + 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(responses map[graphsync.RequestID]metadata.Metadata, blks []blocks.Block) { - select { - case <-al.ctx.Done(): - case al.incomingMessages <- &newResponsesAvailableMessage{responses, blks}: + 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]metadata.Metadata, len(requestIDs)) + for _, requestID := range requestIDs { + queueResponses[requestID] = responses[requestID] + } + responseCache.ProcessResponse(queueResponses, blks) + loadAttemptQueue.RetryLoads() } } @@ -124,9 +138,12 @@ func (al *AsyncLoader) ProcessResponse(responses map[graphsync.RequestID]metadat // 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) - response := make(chan error, 1) lr := loadattemptqueue.NewLoadRequest(p, requestID, link, linkContext, resultChan) - _ = al.sendSyncMessage(&loadRequestMessage{response, requestID, lr}, response) + al.stateLk.Lock() + defer al.stateLk.Unlock() + _, retry := al.activeRequests[requestID] + loadAttemptQueue := al.getLoadAttemptQueue(al.requestQueues[requestID]) + loadAttemptQueue.AttemptLoad(lr, retry) return resultChan } @@ -134,107 +151,26 @@ func (al *AsyncLoader) AsyncLoad(p peer.ID, requestID graphsync.RequestID, link // 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) { - select { - case <-al.ctx.Done(): - case al.incomingMessages <- &finishRequestMessage{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(requestID graphsync.RequestID) { - select { - case <-al.ctx.Done(): - case al.incomingMessages <- &cleanupRequestMessage{requestID}: - } -} - -func (al *AsyncLoader) sendSyncMessage(message loaderMessage, response chan error) error { - select { - case <-al.ctx.Done(): - return errors.New("context closed") - case al.incomingMessages <- message: - } - select { - case <-al.ctx.Done(): - return errors.New("context closed") - case err := <-response: - return err - } -} - -type loadRequestMessage struct { - response chan error - requestID graphsync.RequestID - loadRequest loadattemptqueue.LoadRequest -} - -type newResponsesAvailableMessage struct { - responses map[graphsync.RequestID]metadata.Metadata - blks []blocks.Block -} - -type registerPersistenceOptionMessage struct { - name string - linkSystem ipld.LinkSystem - response chan error -} - -type unregisterPersistenceOptionMessage struct { - name string - response chan error -} - -type startRequestMessage struct { - requestID graphsync.RequestID - persistenceOption string - response chan error -} - -type finishRequestMessage struct { - requestID graphsync.RequestID -} - -type cleanupRequestMessage struct { - requestID graphsync.RequestID -} - -func (al *AsyncLoader) run() { - for { - select { - case <-al.ctx.Done(): - return - case message := <-al.outgoingMessages: - message.handle(al) - } - } -} - -func (al *AsyncLoader) messageQueueWorker() { - var messageBuffer []loaderMessage - nextMessage := func() loaderMessage { - if len(messageBuffer) == 0 { - return nil - } - return messageBuffer[0] - } - outgoingMessages := func() chan<- loaderMessage { - if len(messageBuffer) == 0 { - return nil - } - return al.outgoingMessages - } - for { - select { - case incomingMessage := <-al.incomingMessages: - messageBuffer = append(messageBuffer, incomingMessage) - case outgoingMessages() <- nextMessage(): - messageBuffer = messageBuffer[1:] - case <-al.ctx.Done(): - return - } + al.stateLk.Lock() + defer al.stateLk.Unlock() + aq, ok := al.requestQueues[requestID] + if ok { + al.alternateQueues[aq].responseCache.FinishRequest(requestID) + delete(al.requestQueues, requestID) + return } + al.responseCache.FinishRequest(requestID) } func (al *AsyncLoader) getLoadAttemptQueue(queue string) *loadattemptqueue.LoadAttemptQueue { @@ -251,110 +187,6 @@ func (al *AsyncLoader) getResponseCache(queue string) *responsecache.ResponseCac return al.alternateQueues[queue].responseCache } -func (lrm *loadRequestMessage) handle(al *AsyncLoader) { - _, retry := al.activeRequests[lrm.requestID] - loadAttemptQueue := al.getLoadAttemptQueue(al.requestQueues[lrm.requestID]) - loadAttemptQueue.AttemptLoad(lrm.loadRequest, retry) - select { - case <-al.ctx.Done(): - case lrm.response <- nil: - } -} - -func (rpom *registerPersistenceOptionMessage) register(al *AsyncLoader) error { - _, existing := al.alternateQueues[rpom.name] - if existing { - return errors.New("already registerd a persistence option with this name") - } - responseCache, loadAttemptQueue := setupAttemptQueue(rpom.linkSystem, al.allocator) - al.alternateQueues[rpom.name] = alternateQueue{responseCache, loadAttemptQueue} - return nil -} - -func (rpom *registerPersistenceOptionMessage) handle(al *AsyncLoader) { - err := rpom.register(al) - select { - case <-al.ctx.Done(): - case rpom.response <- err: - } -} - -func (upom *unregisterPersistenceOptionMessage) unregister(al *AsyncLoader) error { - _, ok := al.alternateQueues[upom.name] - if !ok { - return errors.New("unknown persistence option") - } - for _, requestQueue := range al.requestQueues { - if upom.name == requestQueue { - return errors.New("cannot unregister while requests are in progress") - } - } - delete(al.alternateQueues, upom.name) - return nil -} - -func (upom *unregisterPersistenceOptionMessage) handle(al *AsyncLoader) { - err := upom.unregister(al) - select { - case <-al.ctx.Done(): - case upom.response <- err: - } -} - -func (srm *startRequestMessage) startRequest(al *AsyncLoader) error { - if srm.persistenceOption != "" { - _, ok := al.alternateQueues[srm.persistenceOption] - if !ok { - return errors.New("unknown persistence option") - } - al.requestQueues[srm.requestID] = srm.persistenceOption - } - al.activeRequests[srm.requestID] = struct{}{} - return nil -} - -func (srm *startRequestMessage) handle(al *AsyncLoader) { - err := srm.startRequest(al) - select { - case <-al.ctx.Done(): - case srm.response <- err: - } -} - -func (frm *finishRequestMessage) handle(al *AsyncLoader) { - delete(al.activeRequests, frm.requestID) - loadAttemptQueue := al.getLoadAttemptQueue(al.requestQueues[frm.requestID]) - loadAttemptQueue.ClearRequest(frm.requestID) -} - -func (nram *newResponsesAvailableMessage) handle(al *AsyncLoader) { - byQueue := make(map[string][]graphsync.RequestID) - for requestID := range nram.responses { - queue := al.requestQueues[requestID] - byQueue[queue] = append(byQueue[queue], requestID) - } - for queue, requestIDs := range byQueue { - loadAttemptQueue := al.getLoadAttemptQueue(queue) - responseCache := al.getResponseCache(queue) - responses := make(map[graphsync.RequestID]metadata.Metadata, len(requestIDs)) - for _, requestID := range requestIDs { - responses[requestID] = nram.responses[requestID] - } - responseCache.ProcessResponse(responses, nram.blks) - loadAttemptQueue.RetryLoads() - } -} - -func (crm *cleanupRequestMessage) handle(al *AsyncLoader) { - aq, ok := al.requestQueues[crm.requestID] - if ok { - al.alternateQueues[aq].responseCache.FinishRequest(crm.requestID) - delete(al.requestQueues, crm.requestID) - return - } - al.responseCache.FinishRequest(crm.requestID) -} - func setupAttemptQueue(lsys ipld.LinkSystem, allocator Allocator) (*responsecache.ResponseCache, *loadattemptqueue.LoadAttemptQueue) { unverifiedBlockStore := unverifiedblockstore.New(lsys.StorageWriteOpener) diff --git a/requestmanager/asyncloader/asyncloader_test.go b/requestmanager/asyncloader/asyncloader_test.go index 54a76903..5349a805 100644 --- a/requestmanager/asyncloader/asyncloader_test.go +++ b/requestmanager/asyncloader/asyncloader_test.go @@ -387,7 +387,6 @@ func withLoader(st *store, exec func(ctx context.Context, asyncLoader *AsyncLoad defer cancel() allocator := allocator.NewAllocator(256*(1<<20), 16*(1<<20)) asyncLoader := New(ctx, st.lsys, allocator) - asyncLoader.Startup() exec(ctx, asyncLoader) } From 32997dfb2417b0ecbc097b32aaf7e549be6f96f2 Mon Sep 17 00:00:00 2001 From: hannahhoward Date: Tue, 21 Sep 2021 15:04:28 -0700 Subject: [PATCH 5/8] docs(README): cleanup and update for LinkSystem branch --- README.md | 110 ++++-------------------------------------------------- go.mod | 2 +- 2 files changed, 8 insertions(+), 104 deletions(-) diff --git a/README.md b/README.md index 8b215b49..eb7594ca 100644 --- a/README.md +++ b/README.md @@ -43,7 +43,7 @@ If your existing library (i.e. `go-ipfs` or `go-filecoin`) uses these other olde ## Install -`go-graphsync` requires Go >= 1.11 and can be installed using Go modules +`go-graphsync` requires Go >= 1.13 and can be installed using Go modules ## Usage @@ -58,11 +58,10 @@ import ( var ctx context.Context var host libp2p.Host -var loader ipld.Loader -var storer ipld.Storer +var lsys ipld.LinkSystem network := gsnet.NewFromLibp2pHost(host) -exchange := graphsync.New(ctx, network, loader, storer) +exchange := graphsync.New(ctx, network, lsys) ``` Parameter Notes: @@ -70,12 +69,11 @@ Parameter Notes: 1. `context` is just the parent context for all of GraphSync 2. `network` is a network abstraction provided to Graphsync on top of libp2p. This allows graphsync to be tested without the actual network -3. `loader` is used to load blocks from content ids from the local block store. It's used when RESPONDING to requests from other clients. It should conform to the IPLD loader interface: https://github.com/ipld/go-ipld-prime/blob/master/linking.go -4. `storer` is used to store incoming blocks to the local block store. It's used when REQUESTING a graphsync query, to store blocks locally once they are validated as part of the correct response. It should conform to the IPLD storer interface: https://github.com/ipld/go-ipld-prime/blob/master/linking.go +3. `lsys` is an go-ipld-prime LinkSystem, which provides mechanisms loading and constructing go-ipld-prime nodes from a link, and saving ipld prime nodes to serialized data ### Using GraphSync With An IPFS BlockStore -GraphSync provides two convenience functions in the `storeutil` package for +GraphSync provides a convenience function in the `storeutil` package for integrating with BlockStore's from IPFS. ```golang @@ -92,103 +90,9 @@ var host libp2p.Host var bs blockstore.Blockstore network := gsnet.NewFromLibp2pHost(host) -loader := storeutil.LoaderForBlockstore(bs) -storer := storeutil.StorerForBlockstore(bs) +lsys := storeutil.LinkSystemForBlockstore(bs) -exchange := graphsync.New(ctx, network, loader, storer) -``` - -### Write A Loader For An IPFS BlockStore - -If you are using a traditional go-ipfs-blockstore, your link loading function looks like this: - -```golang -type BlockStore interface { - Get(lnk cid.Cid) (blocks.Block, error) -} -``` - -or, more generally: - -```golang -type Cid2BlockFn func (lnk cid.Cid) (blocks.Block, error) -``` - -in `go-ipld-prime`, the signature for a link loader is as follows: - -```golang -type Loader func(lnk Link, lnkCtx LinkContext) (io.Reader, error) -``` - -`go-ipld-prime` intentionally keeps its interfaces as abstract as possible to limit dependencies on other ipfs/filecoin specific packages. An IPLD Link is an abstraction for a CID, and IPLD expects io.Reader's rather than an actual block. IPLD provides a `cidLink` package for working with Links that use CIDs as the underlying data, and it's safe to assume that's the type in use if your code deals only with CIDs. A conversion would look something like this: - -```golang -import ( - ipld "github.com/ipld/go-ipld-prime" - cidLink "github.com/ipld/go-ipld-prime/linking/cid" -) - -func LoaderFromCid2BlockFn(cid2BlockFn Cid2BlockFn) ipld.Loader { - return func(lnk ipld.Link, lnkCtx ipld.LinkContext) (io.Reader, error) { - asCidLink, ok := lnk.(cidlink.Link) - if !ok { - return nil, fmt.Errorf("Unsupported Link Type") - } - block, err := cid2BlockFn(asCidLink.Cid) - if err != nil { - return nil, err - } - return bytes.NewReader(block.RawData()), nil - } -} -``` - -### Write A Storer From An IPFS BlockStore - -If you are using a traditional go-ipfs-blockstore, your storage function looks like this: - -```golang -type BlockStore interface { - Put(blocks.Block) error -} -``` - -or, more generally: - -```golang -type BlockStoreFn func (blocks.Block) (error) -``` - -in `go-ipld-prime`, the signature for a link storer is a bit different: - -```golang -type StoreCommitter func(Link) error -type Storer func(lnkCtx LinkContext) (io.Writer, StoreCommitter, error) -``` - -`go-ipld-prime` stores in two parts to support streaming -- the storer is called and returns an IO.Writer and a function to commit changes when finished. Here's how you can write a storer from a traditional block storing signature. - -```golang -import ( - blocks "github.com/ipfs/go-block-format" - ipld "github.com/ipld/go-ipld-prime" - cidLink "github.com/ipld/go-ipld-prime/linking/cid" -) - -func StorerFromBlockStoreFn(blockStoreFn BlockStoreFn) ipld.Storer { - return func(lnkCtx ipld.LinkContext) (io.Writer, ipld.StoreCommitter, error) { - var buffer bytes.Buffer - committer := func(lnk ipld.Link) error { - asCidLink, ok := lnk.(cidlink.Link) - if !ok { - return fmt.Errorf("Unsupported Link Type") - } - block := blocks.NewBlockWithCid(buffer.Bytes(), asCidLink.Cid) - return blockStoreFn(block) - } - return &buffer, committer, nil - } -} +exchange := graphsync.New(ctx, network, lsys) ``` ### Calling Graphsync diff --git a/go.mod b/go.mod index 060031c5..c36a8900 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/ipfs/go-graphsync -go 1.12 +go 1.13 require ( github.com/gogo/protobuf v1.3.2 From 3112f849ac20edc4ddab1ee477b5ec8e12c2a0ec Mon Sep 17 00:00:00 2001 From: hannahhoward Date: Tue, 21 Sep 2021 15:24:24 -0700 Subject: [PATCH 6/8] docs(architecture): update to explain actor pattern --- docs/architecture.md | 24 ++++++++++++++++++++---- docs/processes.png | Bin 88473 -> 86948 bytes docs/processes.puml | 1 - 3 files changed, 20 insertions(+), 5 deletions(-) diff --git a/docs/architecture.md b/docs/architecture.md index c83d3188..7269da72 100644 --- a/docs/architecture.md +++ b/docs/architecture.md @@ -11,6 +11,7 @@ This document explains the basic architecture for the go implementation of the G - [Requestor Implementation](#requestor-implementation) - [Responder Implementation](#responder-implementation) - [Message Sending Layer](#message-sending-layer) +- [Miscellaneous](#miscellaneous) ## Overview @@ -71,9 +72,8 @@ Having outlined all the steps to execute a single roundtrip Graphsync request, t To do this, GraphSync maintains several independent threads of execution (i.e. goroutines). Specifically: - On the requestor side: 1. We maintain an independent thread to make and track requests (RequestManager) -2. We maintain an independent thread to feed incoming blocks to selector verifications (AsyncLoader) -3. Each outgoing request has an independent thread performing selector verification -4. Each outgoing request has an independent thread collecting and buffering final responses before they are returned to the caller. Graphsync returns responses to the caller through a channel. If the caller fails to immediately read the response channel, this should not block other requests from being processed. +2. Each outgoing request has an independent thread performing selector verification +3. Each outgoing request has an independent thread collecting and buffering final responses before they are returned to the caller. Graphsync returns responses to the caller through a channel. If the caller fails to immediately read the response channel, this should not block other requests from being processed. - On the responder side: 1. We maintain an independent thread to receive incoming requests and track outgoing responses. As each incoming request is received, it's put into a prioritized queue. 2. We maintain fixed number of threads that continuously pull the highest priority request from the queue and perform the selector query for that request. We marshal and deduplicate outgoing responses and blocks before they are sent back. This minimizes data sent on the wire and allows queries to proceed without getting blocked by the network. @@ -172,9 +172,25 @@ The message consists of a PeerManager which tracks peers, and a message queue fo The message queue system contains a mechanism for applying backpressure to a query execution to make sure that a slow network connection doesn't cause us to load all the blocks for the query into memory while we wait for messages to go over the network. Whenever you attempt to queue data into the message queue, you provide an estimated size for the data that will be held in memory till the message goes out. Internally, the message queue uses the Allocator to track memory usage, and the call to queue data will block if there is too much data buffered in memory. When messages are sent out, memory is released, which will unblock requests to queue data for the message queue. -## Hooks And Listeners +## Miscellaneous + +### Hooks And Listeners go-graphsync provides a variety of points in the request/response lifecycle where one can provide a hook to inspect the current state of the request/response and potentially take action. These hooks provide the core mechanisms for authenticating requests, processing graphsync extensions, pausing and resuming, and generally enabling a higher level consumer of the graphsync to precisely control the request/response lifecycle. Graphsync also provides listeners that enable a caller to be notified when various asynchronous events happen in the request response lifecycle. Currently graphsync contains an internal pubsub notification system (see [notifications](../notifications)) to escalate low level asynchonous events back to high level modules that pass them to external listeners. A future refactor might look for a way to remove this notification system as it adds additional complexity. +### Actor Pattern In RequestManager And ResponseManager + +To manage concurrency in a predictable way, the RequestManager and the ResponseManager are informally implemented using the [Actor model](https://en.wikipedia.org/wiki/Actor_model) employed in distributed systems languages like Erlang. + +Each has isolated, internal state and a semi-asynchronous message queue (just a go channel with a 16 message buffer). The internal thread takes messages off the queue and dispatches them to call methods that modify internal state. + +Each implementation is spread out across three files: +- client.go - the public interface whose methods dispatch messages to the internal thread +- server.go - the methods run inside the thread that actually process messages and modify internal state +- messages.go - the differnt messages that are sent through the main message box + +To achieve the kind of dynamic dispatch one expects from the actor pattern based on message type, we use the visitor pattern to simulate sum types. (https://making.pusher.com/alternatives-to-sum-types-in-go/) This does mean the implementation is a bit verbose to say the least. + +However, implementing actors provides a more predictable way to handle concurrency issues than traditional select statements and helps make the logic of complex classes like the RequestManager and ResponseManager easier to follow. \ No newline at end of file diff --git a/docs/processes.png b/docs/processes.png index 3bfaa2f6c70d10cb0c8814d5ed77b1fa55d90143..964dffd323c3b6fb19fb282921df6a5f25c10403 100644 GIT binary patch delta 67021 zcmZs@Wk8f|*DkE6fFKA;Nsn}kbSo$zAt4Hq(ltXP&837OogyG04bn97+@;1lO>WonAn6Hf?g3;tX^>byIhshuJjcW?HSSCfZjXa`Qo+9p=` z>zx^qK!kWi#%tCClfIhqBa`WMU5c(U$2E*ifh+-xBn%|(Ermb*w{MV4S?G$#$z?oZ zzbS{yt(W}q!KIDaI_*0RGKhXsBg2B@&%L@74^`J&pBG&?RVANslQ+d+(Er^0tJQS9 z@>@O*(rUgXSIkap*!J1)*BK&+4jMK7Xldctrfg|e%pEiAM}7XHMUM<3&3dkTaxNf~ zWcL21kYbS8v$sY%PHqA;izeV`Q~XAs_^5?)7QV^4+dY?N7+M~^S9}LOjS1NB2cSz0u!t1>qlOX z-Fq%IHyf@}r7}u$y!sRRqI#H5%}fmz~J?)dzwY4YjVw48TQP^Q#1GaeC&z(}qf^qUoZiV7 zuEDI!_1@EG%f+Mz`R|0~er8Jjs!VwKlR5L1j{;dvi_=o(y<(BN**A~u2Lt$|e_si| zplm$Pz>Am&?w32|c;^&Yf!RQ6yp&!nvhCNkr_?z3j<3V(dAIaed-XAoR6_tbnP*Z`*gwHoTHHaX62ImR_)B*MYqrqzA**N#|E2;F?+I8ImLu-|HMZbNhpc! zXJY`_b#l0`Oni@($Eee*IH^97y`;wN=6B&FvA}~**9b8)`^N?MFWv9tm&ArqHhwl| zE?-_=4hswW`SS?`UG*&DpZX`J)Gzb#Q6%>QMgv z!d^J$`1ljF&0rO2jw2cH5)RM3{t=h%zGPA#b2_}LfwY}pO+AD_vfW&eL6VPnCq=2>Wu7$i4R?fhxq@_m*h9wcV$GiARpb(yIT-_?P&2 zx7|M-IqC)C9;cLLuhWfpzd_FHJCyN($U%2*qCKdN)p#u&HUBZx8+CU$?Ph&RePkn z_A9Sv-e|@AezZYiV&YJyQl=C=IMyiFn%$i?8mwElo=Z!!5(Nwot2G%>(A~F>K(vyi z6SV+C!df;lahXt<_>}(8U24(e#tc!nLj@PD6Fn-1*PGnIZYx=GNvC>WLsOx0^C4+? zUn~#Sg-*R^ZD&Z!V4>s)zghoxff@IbLaf(UYVlHPLVB3ravKvZ5w*XrmdOz*y59Ul z#4SM%{!s|tTdWN)!Y>Scj$t@z2G z``J(2-QBysaDek;TOHNtX!5&j+WmEB!m{A4rsl64b?d1rN8xXf;gcuVs%Xf`SucCK zx+o+(MOky-9UvrLU2)6tX{bKd?9ZWp)_FJPVG0z7|2 zC*D;6I^U5I5fQPpv|Jar7-k;8b-;NqZBqSy<8XaEb)MbK-ajQxfl;c=tR^$l^yN$Z z-diCfWtOA!ZK2SbORZ**WJGV5p_CMY5yPo-@0QK2`HT^r+nf&PXXB0b6pdcx zA8;wM6q`wpR%-K1<~Wuq2gNpGuK1(PgwW~tQ zCKYKtL(Kq!AV4?Jy=ZGV`xS%O;`%zfJG1*Qe?q!vm%vr>^78bUQKcgpMe$eXTa?19 z4RxHHoO3h431?qQe=?1NGrWT+}w?BVugZB#&=X&D&-OuttMSWwt7Eo2co?w9)~FRu(pB4G zZIG(}=MJQp+u=L6mH^_;Tb>sOqu?IdP!6oUs|yz!`>UX>!AzY;(H2+zc}meAkODb4 zI6zv>*&P7+D|)n`FF}O#3Pj&sM40Z@*=9AgRd+aH2~JR}(vBhj-DIpVtB}p_YsmVO zMT)83B9optvxtO*DUq!jx*qR9k6Cg}F{~tht4R&_S6=4}T`}@}%mo|27|~|Qi|=%D zgu~f!oZZ~yWtwFo+D0#~Kb@221_an#EVG!`E zHX-*J6@F$qZBDw(`ugjZnL|U;=bazhcXoCf>On*_2Kywg)x{lcp!_ysc}!Q|hZwLl zu8!oWkcBAy3G}d?bi87C^hlYU=iQEE2S1|Bd@W;FhwA?Q)LlWlv%?Kr(>hAOteyY7 z9TmPxTT4a{Y}Hyi7++=g)3Lm@?6@>G3o)p;Y05=DNO6dqpLW$B+Nb;#EFni&j3^&< zm1b#fHlpCf)z$#zYdhjpu+85edwp$b>8KB4vWbqtV2g1-+bFkXS#yCsvnrc3iKqR zcaOGaAV0;lqY&+S&#BH%xd?>d7v4zx2F8D0Z4qFW^n&-zQRVUR@y7y8Ojt7K+f8@% zLZpVBi~n)y%IeO8@9_Inr15s#3pfV?si%(q9t_sbJ9E#gr~zh^&2HzHX7mkUC_NY` zysjd1p1M?Mb*>T+hN$=3&<8maQb2y3s&<-hM57opGcywtMyrAAH!#iYLqLHabDFHM z1=W9jB!?6U5-5GN&(Pb#cMVCaRS@EJnVtzb&?%m$prkE#BzjsxK`s_p3DN{dSuv|6J}~FgSij zxUWZQYU(V*jx~ro+ve9fZy1=E0!_6R>|8p8o_W0d!aonvpvQZ4q3BNj0o$VNuP-Yl z#n7Ja+{CCD2lZ2^0w^q}^Kam%{u}rnpB(2yRej*ze=Ca;@?y{3!=}Wfr}H94E}FIW z;t(~LN-gFpC?v$Pg)VtHOFun+2Fs<~Qm5d|004{*Yx@21_UxcYcoZ8U=|d~YS#Ql! zv+FWJ-`?bDvWS#XiM-EZf7Jg5y4~K}`-zA6U~FPS&blS~X9!69(U&?k&fIbV0y%Rb zrlkL{-HKrsPDb|@WrHgqc$D|e^5|R^QX|z&|_Ic#6*?L zLMQP~ZGv+Tw{*sIty_~rdr?Wr-tO+;n3JQh!!oVVvxSJvmkFw#$&V(XSKgR-^rPSV zItEQB#X?j1Hd!-OTNo!e`J)XwiC4{~7v|-r!`NET*|8N3dzWeCmzPto-$-#izahrb zB5`_Wt(7L-!5;qbZ^eixR)Ur4h#(3x2H6D!kx%I*x2KYG|u;A~CdebW{vMmM|eyePiq7MA|3cPl5hr=&$F&xPn{qBWL$w zDilpXP(DEliHO9|`Pifj0u<)ueu}#4%K3K-P4H~;|2)b_w z!zjrhHDq)@jveTPKxN+v*bzlZ|4*-lKkGvP?vX99mx(!a=uw&brcbL*V0eG|SP@A@C7{87rqFQ{^T4#2zT*O^A>2E!29uqpg zym@EZZN|0*(o56f@Y`u{kYq;rSK7Uw&Un{_pB!C9)Q;@b@$Ra3I?X)b#YLNxz;Nl! z;9J<(Gn}%Gh$Kfp6Bw}KSBCBF24Z13afz`XbpLj?L$q_zo(AV3StxaGcT7A!-i#ibmqPtQL_m7xj(oKtlnJH5g=cB4UWBq&2&`pWE5owG49iD1W??ZO0+*1T{f)5U6N#BUpyV;n= zytgaYT%l?0aR?~70GTOxQeF1;j+%}o+g z?RHDcF(?+EG*uPhn|D`V`(dtR)K8B)M3Qu?8ogZIKEy@{NUsLJxUB` zLUd`7RT&?3Q82pDb?;4fe$}}2$}2}u29E00j|Y}Lg^vgq%9#BywL`6l##lQV8K4s(yy>hk9!i z`x$`>jDep&|I?h$m*@R8ygfE8cy0n!vlMbXa5SQAqY<*7tu`OWzr3u2JoWYUZEbDI z?j9r9{(bJhS%N5}CMe#$NTVY?P<;B))?y2rmzP&cYU(Rh)#Bn}=aK(qU06k=9kqAo zNEjZap8TesEh+Gm{QGxary7q>VI0dzx`)Uyif+>^)feG;~EJcqQ&3 z{DJ{?^(|pWdOI&}Me@;mlyMyYsPKkxpKmn6z0i-@9WU z$-TqN!}F8vii+S{N{)I#xmu&MGgp`F+}ythQfKT zMda92>ZM@-5Qhp~fsSHfviJ;kW$lcU?d^W8)ep$ho- zV%}4uU`LDnT1t$+PTf?QKTGNT&^y;ZmEm80X}h1A(*%pZt$m58i}@$(*a(jD zed~+kv+9Z4;^y6)Ity$k($nHcoc@qa5H5RaqTDYT6;ASP`3r*F@<~8|=0`$8tc!)1 zSaM@6`1+@gNI%2egT45@X*X-wEAo;UcF1;XvJ&l$++Wy2$K&}g=#hGEd5L5tBl1?$ zqNE!t<~2xZ*3k}bkgdXt=a=|g&5dKQ`&ND{<52)7Wfd7Aq;qXrv)Jokb+i!S{U=Y) z+FZ_=QoA0#cRd&pcG_w+6cKjit9fN-7iQPtjWtTQH`gM&QvKR^oP->zBBFTv77GE- zg`s$V(GCJzagqC?qT4UNw|rFEk&`uMFYVGVB=2 zlu|L33sB(w9j$JWyfZ=m_b~^8jEDLx_Xj&Ucl55vADQ)^jrsd;w{B*>^p%xg*3quA z98&qu!$Q*7=%<|(<#RLz44ZT)wg1N1QRr~a(pZ5WsekF@mk zjXN`K^djJH7YC>JnefGIAC7fo^X;M^-e&F1IugvaME9Ti&^W6-y7foa`kEGSu6+#`NiS$9qrjl1d7H6z)bjHD#X=GfWT7@g!a*)y zj2L0PtECkWnj21yN0pjWwk{^t7T{rA+%k466)@1nd9Kb|u*f18nNh_}G)w$Qm%8ZBzqT;O)(B;sf>za~N< z#W#@)hzdu$e}fou3DzZljG5Hfk@^RVrsl>4<8ErII_Prl$G&;3 zJ#b|APaVTw!psN?sy*7Ided78UA2SPXYE76^wZq8e`ksFO;!|L%{I|5e5RIU*LvBWt6EyLp#G-x{TZk!T>zfZFf_5od|1LSmD74NVD%*2@tvze*W>r1*Tkj#CyKK=-bh3|Am z&umPTNwY7q$W<-R>pU{y(rrM;8=?Kd-ouT_%GK2ooeWNtoB{Uso@UR#m;AWAnA8lN zr)P(jwZ`6?nlz5CK1ED6|NR~sAp8R=nr&VxXc1Zrt;}vTQ5pmaheR>F^11x4jqA?{)Nn1>GFqKs!tcDi5n*$HjXG`6-e&i2UIBr?$XlI zo*W$-GERT<`!GE(coU7KWp4vQF9E?8XyUu|cIgROTFVEHyxo8_l_X@j8v}`Ob*>Ecv-Xk&e09hqa9cubd6J(`Nsf~QK?6_c z^f80R$3lXFe8lu$kA0abLHDKqgZyP}LzV{y1|G+D&&&5F9t9aPW{| z^=LmlvwgdrnF`8A!t>TlLVNO zt|ZKNSY!Jz(LUr5;sb>ZQVsxX85kHeHa0dp`Po?1$sI@E^JjhJbLQ54#6up{gTdDD zrjTqM^M73jbSA~Yz2O7~+H|fK78Y`Ha@yK}=PlLd^pj~)rms`pKRTMxmb>X*Pax&| z^9Yxp0A> zg_+34lq}#O9UV8LFZhYNcPsE-S1rZvP#;YCM(*|apN|Xg=HAq5Fes$R#iFqL`L{us z{^l<3cde6OSVq!69DfxI=LRbhqoRDcSvWbZAeJJO<&96?L=R&n^rZiD6Kof5G@>TBbXw<* z^Fp^U|91LhxI)=0!k@M$Ed|UTf(93jf&NBr-1!Pcbe(*%EjA=`xg~$Y{C|H68p`HE zyq%1HdwakzfKWjDF1m9~51pKq3Ys)m*%>5AtP~)mK&^i>r~L1$pME=Sknu0I$2~8! zya4d`RFq5Qyy+jAe@xw-_{h7O*Y*Bx)HgUzwmxXn69D_$y z3B=%_(iF^)&jgOTZxqMG=F1>nlJgKldb_rft=-@4KvFsF0u{IF;+ewrZ!Cu5dh(NX zBqa#ZSIyVHB_yE9x?y<#BYQBq)qOu0D>SWUGp#YK*ljCB?9&?89k^=cXkOSKEVrU4}+ zoK7ZRvy7FN_B(ykZ($N@1; zWN}}Z*U51IPp08aA2FqYR zCp~7;$*)R6N%bkOy6ZJD9eVvQfb zAr6D-HB8bM)9rX0hS3kkO*57Drsu~y9;f@^tS(ClPMo?m$1S9KQCxbS15nEIiYspF z`8T5;{xvnCV2G@St>}+qD|oO7s}KTnA}~s&z0s_2tC5I;A|6Z-^{O0h-vW|`a4-`| zNZ{BQ1jYg*T#xZ$W9Cm(j=WmwVynbIXrkg2WhSmhj#fJrQ!n&id_66wzifQ}?j0>J zvf|IXKj5kcq`{+~LZDY~B}d6Z%&e`qwziUDV}-0H1&gX%Ti5J8Se?Qy{fhIz*>_^V zd%p>Fs%M0>l+LBP>@DhZX_u2rjMNq&DaF*c$&vj>kxJr4g@uCe*RR-u!MMmlA582| zL*H`o241?YY~BK$UP(v@NKE&`5Hg|E&rQ_S)WEqG5<(9im>V1W9&50MhW>b+a|q4^ zN-XGW$P}=iCOXSVOQW?1ZOO}*FH_Ld2QP1LM;b{WzrZIU-b9rv8Om}5?*JDeW_#n{ zhW~H{_#o`CMJ0~cyx4K=O*Hij*e>wd|2B%#gutZ;@j}MHC3*B+Fd~N{G^Db2Ci-nz z<(J7()lOU?%#9-EPQk&WOQ&j^w8`*DBP}pf6ps^j{BfM})V=-#Zr{|@1&{!wbbsYG zY<6wO#3OeGKQ1mKKVIXy@mWvTfb(zk@BkyS0o%T+wXL3T1V{?Iy1w_Ho z{ys<;55(Oo5}*Vf0QNyJ<=3R{^5@(0QuH(xrJcg4c34)QM)oTbX&~XTaz{X{@Vd_Z zq|W2)$DjGbY}NFDF=|og?LQrnE&Nu9QpGB#ecnL1__B>#PP21!4;N3rdx9iLMFsqo zkrCxHFNhZhMt!!*}10FR>{R6HhdDg;z4&l%MgCM)f^SLxQ3$8SBoV}t_fNp}pM4ZO}m zl$lQ}d2x~!KD<$o>;C9bRu@0;vCUU+?~RU~9;~g67I2A=Fr(7Y-ld9EL7GP)Ahb$A z21JnoMNDjYxj_B{?OC)`!1#NQ%?=oYW6j-seAgEXkKCu$^JIZ^lJrVc&&%>ZVt`?| zzJB=EuUWQw;WvCgRo~8A1=Pc;`BW9(O%%`t*;dLGW3}|ZG)Dnx!|75^K^^&HBmaHk z+sErAee4_@1G^A#na@XyIZEG?AMKKI>E=d+PK0s&dV#bUvH1Dn`a=mGp8F@3zjDJB z(n8;=P2?iDh(0g|{YMf3KwToMtUH4 zhQ`O=GHi3?!mHhmtbB2ZZRoldb_J%Lp;Ryr6PC1a#v_t`=5^i{O0)Fm&$}bQr=0Jd z>tk|vF}-h~4aUHxB`8SE=7J;b+|*drB(k{8keN!Mu+PCuwb(2MVhqAdm@cVSkoHT0 zk#M_9&`PY@w#r7|57i8Rphe!?C>#5Mo1y&(jxH%JA8(x=GTfDt5@ z9c^E;sJ**;joGz)t@FTIa(;N03i2t*=0q7T3iJzO#+Tb`zKek4CgwVm-m8h}d5dmDdesQL4K zhH>P(1`KU$PXe*b zstoE?6B`&>Y$aQ59Xjiubl>P$LT)tZu`JcN*Y9R>biIBtM&`bZBVo-1=CME=h%1BD|m6E6}Co87CZbj}K<(0?5r>bP5CZS^Er{0)l@%$bK zqk5)tEhc@5%JrTI6_f4Xcggv4wNj-=7P>NLXNnB5Bp?d?9>BH*Hr8L&>9i55=k1ye zq;TUCk{p)U;YP2W{PuSGm7f6HwO!t-vmYucJMFG~evsh6PCIv0SexnXludT1YLvU1 zO~m48T2sMVszc#h3tmEP4$$V<{kBtSgF`6qezJFUWJ{hGwP&s zck!Etr!|B8+Z@W0cgFScB+xv;<@MA)0w^V5F}%#sj%;pzR`ljoIlS6#0VhE5 z>!YU(_0ZKNP=1i0CXBR_yB{trkKul=FRW+&K1gpS`xPIeX~1u}JiGZ9wue2&GI(48 z&lr@eT9EVfGypQKUwL~q9wJuf$1smmhX@kh!_5tc&B?=b`TC-wqTU|GcEFft)Ntwq zNcQGw6{Tpjh2-@n;z?a!CnKCqNT8y*SKQo?JQ2y%ahHZM`On82Yc@nG$IYMLQOUT^ z2TRvqK**M+c|Cw%)U49%#KJ7Djv&Rb&xgO6ZEYzI;2XHnS~k8RHJWm4IvPdM`GciK zTCT2DzP``vJuilxMmbQ;>tSed1~U^=TW9Ag0DA7Xi}6rExd3v(*JD}sPs7J{vg=mX z)_~IIU|E`L2?S~pgOAK+Q&nZ9Au2VH&Z~B$WbE_KfufjUG}KMG-l-VDOL0k_X95x{ zFw56_<|6okg?DsOX|E=xQ$N?CdVSVe^Kq20WHPtp*())bs$6PpkQ|Tgo|jo1hG~p_ z$^ZkJ?oMIm7wj({Z;TbGavuHA7?|imT#KmnSm^IhPtgEOtQx8(9?NMH(q7oST-l59 zAkFp5Rm)D%m}DNwp7+0f`!eOl!K%=Y(pNv03hLj#)%>EQa0lp{MvE6LJNeeE@~nC= zL0hg>UG<4e!_g-AYUihaAR(4btKM#zO3=YKyeY)J@)4IUuL8$GC+dtGUXF20Faw9t zi;9ZAOcKviX70b^VMMVE0Z0d|1&ToQw{Ld{eMA4)oydnFuGtY@p(SCGML=FPM#AN& zHoHCBL=f3;a{KmeP{yZAOz+|Vh$PChCo$Bh=Xt{5pkrpX+#Soq_9izAXhSY~Qog0y z7MS)W{v%CU1%f01xB$$JqSpNMGLkpmnG+BQ$`B~Sgpx6c#&3xW=~|T?MuAjRmi`;| zOUM{5J#h|1XfZ)&pOO!Ou>06ijGoDxd~Ft1i(5Vq2BX;qI$L}&OG-)-#N52V+=8lj z!&jym=FrZc=6O*SM6opYXLtg2rbRrSky<9qi;5N6dzbz*w0?JQFS5gko0}UnM&{;_ zd8b5^;xbKVF5KSEPW5ERso~=62tY(U7w<-5R<$sbZrA-~SnZfJt?nYMgWo?1C_1z> zG-!uLNK?4OKK@n~q08snY|$;|79AwZw>gf4s> zpwjJan-?W~=7Y`jycAouXcnO9v8re%2b0Wo7toMRFeyct_EOHTpCjpgzHlWSE)Q5I zC(E}4wCyc5v8fem)?doF2b=?1J*yoFBG-JnhERNWFW&n+UI4oLu*6Eok3)-uIgpJ? zW*CM_a}E1r$5nb$a`SLQ@IoucEm4u*Nbbp#4+RC*0Ka<`rlrw$@}FXL@_YPn1ri&e ztGaXi_7E+Zt+abSZ^^HVuZ4s^6=+#=FSWfmJ)n~b`B|gvDk;+Qm3tn@ib#?IMk9$* z-|}uo$H(JBPV^US=KO}9iNQx`v4xNR1MZtGYC2ntZ!332C@9v*`F_;x%H@rX;YART zKTeYY^Ch-7rMl(4ucGc?e^t+m^Vpc+g9RHEx3F&Q*V{2vsidkak9@e^TTriCZ#~o7 z{!Icx$DQrGb`-9qNXoH8TSzo$ir$!~soh1wpr+N~?3G3SC#Q&O9-29ZiFk29a;btG zkWfa(dQ8r(tOyx2`vX;zVxs7HLqoDkDi~3S4cgC0`Qdx)56I|LISfEC25t1yPaEC7 z0am23Fo;YRuwj#bVuhn9e3YeIbr)z}+_%>Qe0gt|$dBDK08HGPBhASn9Yuv=NX*%W zS42eD$bgPXCi%?&ls-EV$iZ^dve~J9*yJqJUHHxTv-s+lY{Ka;ZWzZuXyd3D^24L} z1+di-m?D9`Ze?Y~H*XgBTYCO@&^Z--O&eO4a;MAxa=E|0fU%);KqIxssSATcS_8 zNBZi?cm{?kD(L>JVfa6<<1p^i8N3J2kb7xSg?|<1wt=arR#hBqi>rSJJw8tEVr)&H z1T;L)-S$pNIdY!pxE|T#Apus87|B%V$v+%hCx0W^2Q)RcT%0|5DN`Ap{HzBup+vK7 zD<7O+?c7x_-)!JFMS<%XUPBXV;Hp=tlZ`R?cyX2Vzt4QZSUvR5R?$y*i=hHLknWD zTg9S*4fb+uH?^K5-H?J;y}L0N{QrI*M4fY0dwY9fVIjf)+0)1)a&fznkaxDo%05d{Bh4?Qft zlrgwnx-U>;MZ@WX=5NuztZ-+?AOd6jO)6HscP3CRwVo_!mA~F(?a8vvS$DT8lIZz+ zKH!cg)l<-`} zDTb0%lhoL~`;0P5gtS$)n_FAIWcM$<>^z~s3XL@mun@(*oCW-%$#}I^eF9r$%p#~! z^Goea677i)-zysc+M_XnMHvieB?AJ#Ed2PXSI%y#^eSvFh5crflaih=H=-rKT$;JQ z+`EJ5mC?oJ3&TazmLxFVF~-wAOy*C(ofcBdW<$FNgp7Es40LIp57V!+Tbmr}%^j9~ zMa*|FJZ~ExA78)G3Ap2X(mgoHRWRvcxtuTP<=CkrpNn^SV)A9hb$Bt2`_;}UyPFfD zdgdgWjksy5e4JQxwj{H5E!ewF_a(2?xO9-uwq~u(j&;GfH`*7pUOe!>ta8{0tv4B> z6NgaJ%is5z?5$HKlbLR!R^IN>OpBkdkJ~cx$cEC;gW(S24;4tT$uRN@iTt_TxU{aU zZfg%X+4O3)r))0GI%tQS>vku9C%=@etSpdWKfJR66uXbzoQ^i9K<~Bq z=Z`YF)DIo}rjri(;PeuVPP}_NJE6z%(>gJ2pC%lp4}Sd?$xLB(?B)`yim`E+EZ_DM zsR}unZ(b=aEloN1q&21(kUr&X35QXKADr2KE|!}xS#mFIOOxblVe^x$lupw;FQW(sF_6bms z_wKd=QmpP(=F9Bt>=w%^19Kh_I$SkR!D!=;oe-sioE(mU-rn*A7)R=li2GZKKib-o zQWG@!cBxUOZ88p1cfqt7OdFs93X~kp~=;TkClM1=B^V6}c zylESZ)iOgPTvE;D%R^=8t32+*N4SeUuRui1w@8hfh!<}FB8NaU$>T6qjaE#o7VKEW z^CKec62)Y+d{=3D;sx&d0aE_s^`0SHS}OP54ki*3lD3cwXn}x`@FTxMJip@AJPGt8 zwCZS31x>g!5d9$MFoRQkLJWhu5kc?w~!lN*;w&u#tjb>BV)Y1whjsP4u z*kLFD^&7xlfXb3YPx=gh8A7c=gqE}vyk@^=XJHHPn!38c5`?DUGqA}6h|TF7buKg3 z6%8Pd=g>le9j070-3b9v(HcNX3jQje@nnmZ$=Nmsq_zr^W(Wq@_Ib5307TZ`d4L!` zC*tatmNjVJpTofplrvZw z*-xiGY=o(5yed>)sP5XSd&aCvZT0v4?-?X+q_u_Cl;A{Dr?qWbNT`{N&u+*4Ov%*% znNyk`)!a2M6!VArBX>EOI52&axh8tM; zM6n-yxgkZ;6S;9NgS0?8spb6qtT^8Gs!X<&LeH zdI-oeT9EwXCK@!3>$hz5%fPN6k@5f>9GQ$~s&sJo{DLzG8_@Xu(XIYS@dxDBpl_Y; zge#41_tqtiu73ai9W2X6MG;*k{J}E*oZ7H_Trb9mHi+$`JdjN z=M~X0*xXBAla!J&1zR6@*`Tv;I?b}>%9{CnpGo@pbDcW(O2Ep8goFqQ30;XoB>A8( z5;OyaAmAwT4}j3d@h>K6W3h~?qyi=6@CMXJOyIP3FGk`6JZ}_FD~_+S)RiJ%%etjkEw` z+AnJan0Z!F(etFGJvc#_WXVds=qI4A_Ku+%KRg(v`|+mC-XZt;*6fx<-|vb)Jpy`q z72B2v?m)C!_yh1()$e;^IN~knRig6p6ovRQpqEtXeRaz!N#Y45@YlhundEL=i=P3;0PQ#wD`MTH5wCGrxpztK-tV~f z3ueGPcS8r=e|LFDIR%Le$} zeFiLrJS_pO+6k$H=YgS-G-9WC*}7tFUvRrxg$1+c`;o z7u#P%OKG{(`&u|8MO#1fCBpN4r#Vv7jQS@d}PA-PupQ`Z`11=Vo0>5v+^5x3{m_mP+uPtT# z>FAzqyEQ$_ZHW0g>z%JiEl@NbZfGS66RQRk_k2KSwY)nW{LEke-In>_apT5~|C>61 zRxtfr2M6H*Smc#YKQ`?7#|6lof#y=ugmyKKdp;0!{LdJ9J1T5B0WAr{(mFt$%tG&V ze!LhTPC!vO*x+&nh3NJNgh8$H7<&wZ8AIKCXo^ku;SB+fwN_V$87 zwI-h%_fWd7lpEFD1-s$#YEXuAgNM=xftX=HoC1x&fa*l0)PClgNX)6 zc~@zm_7(!pcAUciTbnhN@BP2}TtK&AtrP0t%Y&XowzNfhGwof>LU!P6E} zn$-COk@Al2NvDHVPPh~umDuXmBL8>2dQMeYUh~06MR-)j1}zRm$il*~Hw-Hp^q&o~ zprL`H)+A_?AeKBdB~{XKJfjFRs~=2B?ogfT1MVpMdN9SP7NEiqQ|jS}YA4c~b@k*i z_MTl2AWc4?ZfUi`#yxwhKrL6S7{8K+dA{4#t*)Exi#I%;F0b_!lmNr4)6oyvuVlEIs~ENKEv+1)vo-vY%QS#Rd4UV6=MP93wpW(6|nd}aIN(-`!QI1d}~A3yJPI0*;e!$NwVyUgS36$(oItokY{_fS3+(A)TqBl@1qDf+oJGj$Ntt*M!JdvbF6p3m1c0fD+uns%QI4fw+yo?2Hz=aSL?ocDO^ zI5I|R++{c#jbMlwwkOKBM#I7w#>aPn9_0J$oQ$dJdbg{~aMuG$LfZ2Dexd&eyUUydQ(MWcrUIsrhzCX} zFg_wM*QvZ0tl(p&EHgnJq4JqfZw*3A-fzmP{O31Xw+sr6{` zs<_Ab|D)?GpsL!sw!r`kQ4|nSK?!LD1w^_9rMnerIT8|*8$|>Okrt2^DFLa2v?z^q zc{rq$!~p^6`saar-|Kz9_a6?&a177hd#$DTB^0XpR)I=e0R0jqLHd+rOc>zO+ft?>vzuE5YpDW%|?u?njz-1VRi>;8}32F@o~0OXGXnHV?U9b*?E}o zT_JmO)f&GZk*!~NsEj^_u_8>l!o6$q?Q+!DUPdf!iH)kS0K9o3#7Z?@Q#0QG#hdDZ zcU!mQJ{=je8Wb!7GvFN!!A~+Cgti9{B!?!$YpWw)-+|B9#UKx+Oj3&Tgv{KKh~1{+ z+bT{iRS`e6#75++o%x6tvCMM%NNwPDhb~vjxO#n8r8*RAC1~Ho$7|`ucH|RE31-Wc zl%O7ny}M~fWkT(2ktaX;HedMhTAwlXF|`lo?jz#W0R6Y@`EU#R1NtVqd0M0t-~2kd zL!Lp(X9_s+gzf4)S)s{a$JImf0`Y(FX<&@$kz%Cx(J&Ezk#TPN*1c@u%Aw|WX1B_% znDOC*C7`dw^W>d5^ySV^zBRn*lY=?{0I3Rx8PiXufMMGLi|@becYYH8waM{|7ezdL z!^2I&S;B39XQ<;i3}3NyUGPYh!|MlRz7|Es-o2lBjq+6mv?qq~veVDFYnFKkbH*$8 zO%EYN`0aGlZ~-zLnWrPP#rr79ijILEdyHrD!|sKO+P-ROmkoodV1;F82NW|O-^kiR z78jTB_NLR*#_7TBf%K&nD`6-ypAAKb zNhhb0jT_%^hhGx8eJmHphHMTW7ZDMGR!@Qrwp`vC`-yL8D3!e9>#2BauZ^x6#=8+9 zuQ%76srW{AXb07du_+EC-zTiV+ftpF?m5>_sqJ@-C5k(|g+u?@(j~zeGkDoK#`5aE z?<$Q+HAFf1)C6Rc3yx-P5DMPYQ7fazW*?gSQp+n~cHRf@zsyUs3ulTYNYChBZ7|kq zTz`mEsp&3><*|zaaSa3m`wem%ySyh%s0l|LT$o=dj*+PUF36lT*N%;Qk@ffcWe3h4 z4yLfgF83c0v3eUfLdQCkc#nK%P!iv>nr`Db_aaF!`HQj6)Kv0V`G7E`DoRZ3Fp|ek zODoC|$$HigcrzdaCL;5~jas{sGE$3@3!iTI-Y47RRCZr=XkQG1iM;w>GCWRM&IJ5C z6wO}dIDb)|NW^vPc?2YcSp*wg&vl!NY85`>M{s6GOm>=b-@8(nql2zC8FL@ut>^W{ zvo}FEAtVU#-W|IE)oTlF(S>~w~v?W%& zGW=yi4=_?{CzTc-jVX%li8^P5#^|g`?UL11p<@RmwM$1pXaEgpWTm&|f~Xn8QwP0Dsj5Z_&xifjgK-Q%JG3OvbVT`{F>ax6!XWLDJ(mkb;q;UwVli< z#4+y!RjG@7)a;+xl|E+i%MH^OZbj$J)bE<}RuaJ9Nm{Ldzk3~C4eDX{z98)&4yBZ%!QRXH2A<#{@m8go5J?Jjf)#T;7kql5srpLy<#2i`q z$0>dpu&Om~Zniy2JqyoOH_S@Ot|$AEt{}ysH5A2lt1Gu8-<`X7 z5%mTSL}1a|Txx8&G-_LpbGFvfu_qW}Pp2CYz7+gxJ?vRdt)(NW!6%hJKX>!{HtPxc zhBFkMXTc*;YGh|-K1wM0U3xd`xaOgqk4*dJEKY<02|8^0`halkvE5T^Q)){;Us*=y ze`|eMI8-ZNv3s}|B`PYaq@<*!bxwgpA@_bXh(7K0UIWGXQv0uH%}t@*jDl37wQA_@ zHJinJp>hc3*m!a^FzWoW%TxDfnf9S1QDkNON*z{G+I#5=9rOU;Q2_*-f27?j9`QN* zpt&M@PbLw3>(-p6>(p|N}h^ZGT$v6U-Zq7G{>keEw)p3zD;M2W|VPo=0 zHBdzt7QRt!Bb3_l#5?BrjW%ipVEF)UyRqAAhT{yZVW^2ls6CzP*uLl*@E&SjHJju? z!Em-@c(*YW?4@jTtga83VOb*ct+bKsV>Ee57z@V-wN+JBSS)ZI(Qs(FoQA>+KAxu| z@-0#^{NrQS4KORM$PoS1#00PX&o;nrkzBahR#Ri*+kU-joo?;b(UxtWQ_-=ed~yJH z!6&p65@-z#hxR&}UR~zp;Q?_M1>0~szeEa^*`ISH?xg_JOk!vuy(t#Hrtd7G z`Z+DmpIxAB_D1Rb+c^*3icK}iVz~%L_st#Abg#NXIV$V%eG5)8`d5iT{ZlFbVnLw<$IA40lS=m zVRo_o|Gx}~-M#TML0}^n`j;26adOX#C^`f$S4E$GVUIu1@uC2su*w4y#TP@`x9STQ zBYV6b#B>VZ5NKAwNT2K2b9-QHAn>f;_|soc8czQ)fxl1xLDCxn$p82${veM${jYsQs4Z~(Z@xMG0K(CM zKNbC$c)Q*&0UgCs{@gFq|M|{p{Gs;L3QB!`BiEl?{a?d`mX{1d<|GB0e}4?&RUyOs zg`euy=|u68!qq(2abh`QJhMEi%Vq8u9gA01yd;A|h82g>e(N6tpv>ZWm;e;IWHAx0 zIBAWoqx~J6S8ED3FI0@;+hTCOw^un0uA^1&!lJ%^2qM5cyV3z~fl=U7&gDV1xrI#K z)q64d5UwE2#*p}~Q`|W=fTZp6+4+#x+~UYryx(0?hisvA)ZCa`T=9Fr`3(#NKa|s> zS78$Uedmt9a}|WL@tU?W-oJ|Sv~mqu{nalna=KEQXBs;67K<7DukUr!x$^&Asa{RF zczP-+$SEnt)p&dG$M&gSWIUI1{dos`fj{b>FF-Q<5$?5L?BA$}%ilj%rS=~+i8)%2 z7GR!CK2P%bqW>DtozpA7l^65P>Y5b9kJp_t^`V+FZv269RP5 zCUf%D<>v?7hRct0neh9CcR+99F(b^SQB_sd{JcE5+v0esYDXh~Pdb!VWJclk;j2>( zW>$1SZW4y7WyQkQd@b<1f4L{2tdf-Ld3c9ykhabJ+&g^3i!4Wv;`i)?<=@MD*ZZqi zWoY<3t+B42q0@PQQBW0H-8sj@+V#FOR#o|gfqG96&Rl@LKuknxYb50LDZG5Lk^cxt zt=>=Ma*yl_joEK|^e4#?D1b5m@ZgC{&jso1X=e6Y1h4MSt>=p;G8ycB+tlhm-#$_- zq1Zo)&2*aF^KO)z&}TX|p7+_iJlVYVRU=UK6n4h#Q{lERc)#!7z3b}g0vNRszZZW0 zKdSxD5T4(T`*>*eNym5Z8;3!?Ein`WZsuEcqv!H1?;MR+#8_gb&bQJ=n=7p)9}vjl zbrH3v4=U4nsEKb;@6Vmr;4s5LW1X-u2%=BgJ1YwZ_*-{(d>%TnA23MJmw?a-NsD1j z_42YLe0HI>7WG}Lc(D{nA?pS!?4T$D^$Jfp+L(|luA4ZDHG4p1Ly0cyQoY>K_!da9 zH!7Evy@!^k_nJAc)|~#E%fs(2u6T7NkR3nyFSe`Bq6f6f0V1DP&ITt%Zi-;pwurs+-ly(%%fXtijv)u*lob@uR$ITNdmr?ArU zU2yh;4LoOzcW@LqB8iEJ#B0{6gno%2t>tZQBH$hLNfWvG0{BFa$|R)kBl@0U0RlLg~ppC1ND70`HPd=*XSPP)VHG}xn}l6ja=S0r=!6rAkFd3g(l>L zeqL%J!XZw!HL8$#N-M@20$C_y(fkJEJZOXC2)RLpVHE22qV0ys+C7EL@e$)1GS06D zw8v#$M6mr$rvVKt^ar%*4;5ZFdxfdo)dXsV+1V$CLzs^ez`@`Lojufn>QYi!i0G@> zSR7~Nx9ZBi6>igQQP5*X<2^k#63ioM4;N4LvbamDItqi{{A`=y9F4wmH@ zxMmGb#;@Wl7HfRBTo9gDZ{A+`{UuVmztjmZA|q)_8_VZVu2IkTyq8hFee-b%1F-cM zURS<7dxEw8J-gE9=?_$i8{6Q076zU;f(D}suI4Xdg1gdak_+E)EwK`HLDs0Y1awd< zY2#s_Q~-h8F#VPQz-q;)MP?nQ!+F57`g;i7BVex~|xUuD3s1t*C3FFHGcIyVCJo(h$Bc-BZj{aA^c>0}O zyL`*fFYhc9ZJw44W=BQ(thAW2*lRJ1!)zR9?^u|exhUGl$UvWR`G$9x%F`b4g!@P7 zPMl1TeE#uV0a0~C_dK&1W9Sqj{q%->=VryA|BQ``i(Si7+-mduqmWQlyQ$fji~22U zMUJ>>g9xdJt+B^ue^U&L$9WL5tY?fo|z>k<`t5OX0Uv|FJ zbNPIUxot9GN3QI+JB(FSJ;snFMw-&MDZC-Bndy(HQKL2oVu;HeB*z*{sw!u%2kQ-B zy@%LdP`DS0l^0Bp1DK2@TrQ^<5_DKAgtt)v!z|GE0!`RAA*Ql>%0=pO3Hgq0QB1a} znMA%vB&XgOOVX^z*T!2_nbz}_d1Ah5y1MyG8I0~dTecr*;A=}6=vX1Uj<(;jo!bYO zkS)KhY0C-9K{mIA`+D!BlY?n9H8M$vyf9X1EG{fIPj!D{&0brqj?RmSe3iGY#zazd ztVR!RrB(Z-o_N?m{CB}$7t`~T8asNNE{xL@d5B>>Ru>85>%YlYq|cO`lbgNv)%KWM%-J@3wTfdEw8`oV~di^3X~Bht&wHNMUK?d5+7CbBb>V zcgr)oN_1io)$_l91z9Gepi<;n#Hkb9&m>9HwY0V|-n}(owCgdR`1KmjPF_VNWYe{F zOy%*d*gxL=HAm`1Tv8TB`)DCXQGRSAadxM!k5?j0B?l!?eSx>|)?IsZUGEDY2ObT* z=Xqc7^D1R$&*{;NrrRk?0vXuY->~M$$6sfJLyR^nGt@aF&&Jps&eDhp4!&QglOlKc z@Zmqcm4DODk=6RUuCprNoSztGn{2$6zC(j=W}dHG8p|SveXEe#Y}zzn#anibVSOUb zPBx|R6791(B0b2=a4B&%Y=E=r73?EVv_8#?_uj}Io$j%td*3D>*H2Gpm)L&aXRga& zt}7djGA1BXT%HS$Kp_Od}#t zI=M@dg>HFa_MtEWQ`|JcwDT5_Y)NB}U2t~p*|sBjl8W+J!F?&377}WB7_`>8;x4>q zQWrK5Ge#pa-fWq(*=hK$&rs+(=3x+%nPwtLz#}K@pkr$uW2kIqcZOQV^Ec}+RZTO# zB!>)cKUwcGZ!W;}txo*B@_;oX@C2vZ>3dya$J~6cu)82n9otm`Z^^#i$|##iaQh;| zLhF!r(>dWkvsdaa#6=1X&BY0FZa*FA9t+a0)lk%xZnYnq&F(+`BG2`Va$W4GQHPF* zs(eQI`{rT&GoxeYC9qqP*#7(vHAW5f^=&iDjWnYvJqwTM)y{>y8Shi^ZoT{K%3La= z84EJWYQ{dvLH_!dTAHAeW8)lRB7<$?O)KehT&Zu!qPyboR9~Zqq=jm&u>#ZX6B_9< zpEYXnGNL`SY`tfJE;9lbPh)n{RVd~DD(k8-4KrogDkpj4Tyvg3lLrQ6KT@|oO=L}V z3mvWI+SJxN<~IJJ=mg1bx8n)a5;5wri8d0`Y+9%T&>Gk}Rrg+t-$ASHi07!Jld+y~ zsAemE*L7O?rGaG2?Up+&3N6YlYAu>AIxYHmA(`(9skITWB1%$)I7!p&#Hr8_9N06P zxyj~kTuaFiM_hc*26J(r(<}dliOE+?rF=~l4~;(j%*|D=^zxsG6Ddaub<}mUIf`pl z&ad3IIA_N;5P)9yLNjakOxgG09MeBM=g-HmsYh6u-8j!MHnqGlc1(hjN_VaK@^z!o za^JVXZzJEvzD<6c{x;`r;oDL*(dpsCdjpH#9~gNdF75K2ZVJ;O3 zzlf5+CNx^;#yM09r$wd@TwfRQP>-!_R-d;T7C%A&EGT$~5Sk@Q{f4vHqF3Qpr2- z;NIYVk)lf7Mv@Yz&gjYnT&Jd0zjvCn@fKuVG5S10&NyZ0qpkQH z4pV(WL%-xGUaT=U*t3}C3Hw_@b|@tn@^)8`P+k$}p33(hxJi9)gr-M}%dsyIRv5+{ zfoz58-5@jbAM=t}cxW6I>CPkoLnnY`WMLw5TmDULkwvU`tJ3R;?ho8l=!}!@C2qOQ zZc}4ptt^ewfw^Exm_%=5#bdp1aQ1fN40gw@Eszu4{UT$3&EO390ufsG6q{_vz3>hJ z0v^Hhs~&I&xATf(+tA1S2+P!SX3--UO^q8!DIw>=vJ z`;2?Y@-ypqUFN!looH(jdFFsD5|>$jIi|AiRvE*emJY$*XQGM)7gQZ#ClJB`uP%Aw zrt27eQ7>yLE-)~djG>yn%N)7E7}{W2c+>#mnTYIUl3hx-a{8Rp(bY`o8L*HyResdKUPhVjb*?ICM(r{;h zR2&zq-gHa?HD*<)gSvL^n`!>QqtsdTlvyzKc}V=%hC9~3o@{;UYuu(kllrY}P9+)D zt&HU*Wh)zhI87sd+{@}SFvq?6k$nzSGZ~JfL=-vY!LXL_ehul2hdr|qCQYVcxmBnl z9_EUw?bTh|o)3KfA3hjXJ`&{u%Ak~KK8DFvo!Qk{PQN+ReKbWHn~{u0 zV!;z2yPq;;>7ZcImi#%mE?*ejFYpXeaJE^-)P&eCju8dX_Z_End@E9^uhSIK46!(m z$@-6)ixu;v8NZlF70km19-45bqSYrYhn}Rjqo>nQU+{haZ7CXe>hX8AxJB?d|577O z>@BAqRk_9sCii!?n9C6tp`J8lFT#V*kS*H>!IBRPEot0;3$ITNSC#7)mXtq9a^P8? z@}a3v9#j7DgOQln>1S__|CUq7yVIn?-&Z1ZAMG?55{*q-hw?y!^v3VbBPOEp-}RttkI>#t=pzCtaKL_N__bL*n=EDLh7d1m*+6wppnaV|Ytsf@F+G zf@S*3k&WrsS-R?E$OEH6mI_wcuktZO&9R#+JKNAJRjI_wv{2+Gj|Iz6m~~5&8unp! z{8*p%i~vK zWG|tHcH$ttBk#N{s<0lvi*y;56o9VlxGmHKK%8OHJl!2{g?%3~dXxlb;HG)_a50dm zddwAH4WemP>j_)Yt@wZfN2E_qL6Z!*onzj)52oOSpMsaWszOC=&hoHp+yr^^6X$2dRhY_l&b|eAo?Kkx?GVF ze3LQh*L2h=DpI^tMbDi(HaUq8a@5RFn=^*-R5#KhH;H0+*@Go+J!wn(VAY$w?0{U= zX-FV@6<*{jw0K)UGim#0GT&Tr>bO6r@b;JEiHnAz7KwsCUR{jN92ENTd#ZV~&Zj{* z=u$Hit|;RB_HQJ66DT^0NzQjs3%RsaxMwonUGu*X$J$=8Q2RJH7hy$` z!o6c2Wjr1hwUldot7~1Zz4HFU?p)~&lVd6sY%|}Y*x7Y?kCWgyxHe`nS=q0rD>CCf z8be}lCk=eEwFR_jrXGD8HCP8*i_sj;zXT z$9NT_j6ipio#yN4`zSE?CcfG7G$@oZ3#QLuLjFD=_{9x_#M2K+#`*gq%W_ zz5w!3Hng@gHHdliUwWSDhEnAvAV{NyMxqFa2Y6$kjqPC(Hdf&(4ECv^rEYNVF^^4@ zF4d5Y!^F3Uc%)^ezE`$*F2`n8>pkn22=w?=XH3&~(UWWa{7r>B8snDKCadr97AhGP zo`Zt+^J+PQw3~5=>l}X+0?yx?n7dUceGcwBEHQLuMfP(G%S}!3=I-kP(X2@qJJ`d? z7^GWPhM!Ag$~9I5XFM>iBV#GO?@-+tU4H0++hF&oyV20&1pPH(#z)8BiHT;d5C*PW zQHO!I24;q;RkM_J=yFLWmT-m99_x$9hGxN|PWV2@D;;`g1(4Lw5b;Z>o zHazsq(En51F0nn_XH_ zbAkeiV1IMM-B-t)S!alvgqf`t7&Q|Tr3@jz@CZ)!n7V>yqg?2z`?>7GRU`?Ykogv( z_5PJ_(pe>sYRm~Due()cvScW)j7(HrM{`G|V`%gd-7-|nuk5oje3Pm{4`I=DFbHNT zi{oQcTsj#Mx)Jl?Fq-6c zrlp)3qmc-DFd+J<(nXHE#Hd)GSdsGZvonJiOv}+4W;nY0I$6BJ!UicS;F$ww<|-eh z(h7_63dAj%JtR9O@h=xkijG%mYwRZG@TNs-lyBkPAm_W-S;tbfI?F=8G6Y4yZ=<;b%7p2w9 zI1dcziDEJ(F|C*QEzCFGP3?$*e=QZfyP1xryn42xi5%LD!frVJ=WK`;;~)}-`<`xBR)~h zO`CCG>EMGZvqGHFPO(F|p|ZhN`h+!h^AUpiqTak=K0`F~{#%G-W`9@n@($OO*uEgu zR&f3rAFM7Ukxpy3Sbtj@U6r1A6|J>)OV{Uq-*_}WEP%XC?ujNe2Hs`A;NusxysI~~uzPHDHa4;Ela!wzlT&lJNw1UI{@{_sye0*-MIbZ# zM#BS0k`3PP(JZoTTKBt@ik{Wg!tLZ(b~g#BSmIM{MpOx?&s&~oXT`s=4Sw1gm8@zm z{oga<`Dd454+s5|Y8b(19^Ny!;7+Gp)VtBk?a~p=>unb6cE=jmcC@6LK2q@0prC0k zqV-Prl3O~sw#Bn@>5uthRAm&jsk&4z@ntO&Eb#HKPAj4KRY&!=k9%j06AX++U1Iz9 zq*GClCpqaO@v6;=m)g`Vq~ApH{+p9e%5D{mNnd6rJ=tBstItI&o7LsE?*$x(vyrF61 zowv{p)_ceuMK7B{FZW3xXLH~ z1_}BFYDe>+t`K4p-b)jvxOw~VyOZRTR$&88CY*^~Eb8Ll5V!g8lEsWH<*VvS>G&*& zuZbs`aOq&*ggWYRYaxX{109`2PA@C+CQ{Yci7wIT^-ng+7#26vXHK>vJp8KshWrot zozgvLGMQQHF-)bAO$Nm&*&i%5&Hw#26zcJ~&c9>*QK#~2NeDjFC&GvN-|?aTpw#~i z_05uJ4LDpGB1-Zy^b^5xZsVnmAnYUhMvBk>#DqMIRY)Cp6nQY>o*8e<_%OpHRiN}y zHr4rDi*3Mg^mW9J$)EAz4^3@AenQBR%?H*lN}`@d6*$uXy zO35v8Mv7}^E%>wj@#z@04xuhv?1YPlW>RG}L zm;G>#svz&aVmdlogVH_O;Ni~A@1)>=>mQsOnm?&jSYdf)3vR#EGn|>SzvcX8f21FJ zuLx@}^iD-z_?~jD-S2kST*;~z=cngPr6R#X_ma9WU30r*$k3Oz<+i0A+ZoCN-1%Q$ z_zpfBQq|rYXsd@iJ4Asoq$`o(WWIMx+*Zw|vo2*f>J1pIU3p(~ZQ9s*w35=)+ zu8Aa|OvWS1%F5)`1Ahkzvm<3+2fr!u5t^*3-nFh%%d*tX()j1Q-sZZdrltVD6je|5 z5X~|>s8e4_D2ho(x3&SV7gYQbP@wq)_|r%50}9nwaGBd5Cpq_jmlXIdpn9? zSPb70&m9Hi9a$Qg&8ClUUdXmKmckMsfBj596@8e9i0lU8XVnuIZ>c4>pF2SmRd@C= zB;5ZIE_DYQ8ZOV=NPK(aNLsgW0#`iBt0PUnbVY{79U3a*!@}q%05MtaG;iSZ^(mV0 zR1*PyPnKRr0X#SoMolkwFfCYBb|lRM>$=(TP*xatx|e zeME3@5KIskG=J?g&DElRFtzW&g(%6K^%y|ky5grIO68kxQHbayx7Wwe6aFJDdi|(o zQ!YOEzzHQXyhk>=?~y$<$Z@KSqA#K+{9AvxQ2AKgVggydBE@LsW4B5MMl{f0#RAK9XvM_j)ciEC!8=o85|`!aVAP# zuLVrqUt(K=gXmgqA{mscNIaRT@#@WMcl=r{A?o10Mxwn%_vAW>|B=Z-ygtK|4=&*2 zyuW`;PWLyLm@PO)-&kl*N-Y`7S zyLeiPdAKB)`8MN~s8@S&%&Ia<0Gi+7BFydW)gi;OC%J-W7T|Wf$!}4rZd7f&MpU9w zLC}Zd)Aaa7jZttaiO*Lt?Kd=AqS!H)2q9`Du-4dfMx2t0^?{(XG#iXf`h5afyn(wE z)^UCiqVXVzL?xUR&F4?bN_oVJXaRn0cb$CMEcmuOKIsnxEPd}0Vg4rEhDJu|)% zGIcvn7mjgj3oNwRK$O}E0r1+|8pzN8OHFmP;Lh6UU_1Kx^XJg;-@7~n91WFAEdp^a z(ZLcW@DI560nc97c+{^Fhsh^K(M1rh$#`-sG_QW%QgSG49&L>iyh?(O;F>Nn&pwd;cCkqSuBU)NJc4enzeywk?!=e0)2gT zEVSPyupz&yi0z{9M6fZpibX8gjPs}4?GzM*_qyCRr_Mcbq%F*Qy^0i6k?Z8n4H6eE ztR%z3uC&V?>V369y8u|8qWMpwuYjx=5nQaNgS1O!26}vQGU%cJKBac_*s2ZSZlc5Ri6QV)DiTKQ!bzUk=8sF%FndM1YMMWBU7Nd#lK9SzdWwnai@JGAAbH z!i$+iWE01?ZV}2T|ELtl%YPA*P_5Wg{4ho*DZ*vy63DTdPl#g8X}aIYM062()c+<7pTYw%>` zaB};ndsWW@0+0Yau?So%iL8C40X}K8Bd+hqrd3y3@Xc@3)@5AwKb6+?`E|x+^3K?b zI46M38y{Q+J{=ATJToa=;;X4Fxlo=*s^TGaft!a%{iN)xP6vA6H>2zZme`*IrA4K* z;rj6%|G#M=7-5f0LW^7BP-@nNjh=^RSHz# zb+0b6`YgcVQ#|OX)hBk0#$zpUD-GC^Rn9oK{gAm*e-b+Kmw}WctWtQc&hSUF-pu9V z2l<#VUC$)cUCig}#Q2l_^SRWs!#;g015QH?i~gJyT0bIy$6M;rXgi1nh#?{qwSX!y zWzk#Qe^&z177w=z5R1-7dRVxO(PVw8RfIDL&1@Z_2h^@xerrK_6$M^uuj5`1sqiu> zV+~w!JnI$)1_tXhs*+egk;cMHA^u?1>**SUph>=FU~ErA@xFj3mjd%=(n5LrA5V!d z9XWAg3#9L7on97j9azoBD=T+4GOgCfLKPztwGfzjCXnL5)uX}s7@VW_Zkq%txwgoY zU-x>O@nX^$+m&SES(|z+BDsNJ&^8Wfp2;LO+wOXpt;rPCl&G5)BMDpCD@?{Nv zU*L@ZS`TYrkb$c?)jT%3R23nxWY_w54@PJtx9Csgm1tFS9`0rSF7@QeTAYw86f$Nt zv^je&S8@n6w6qxH8r4AGw26tDTG)phQFno`f;km^6dpC?zis=jbgtx1#P~>K8}EI% zB^;;{P;g(VhX>+eLCoy9x-pXjS5CM?f)N-Cfo?hK%^N%VBsaS>m3XjxhW%Dxk$cr> zR!2E;%Gpz|&sx~?W2?M~sTBTLd$RzqAxrGMrm$8w*pTrh=;r8;A9|JDK%EPQo&qtXAatEdiL6GAX;wSxD6Z5eOpZHkqO!v}&29Um8py@{r^m*|>V@^KJ?&*Y^d09mtl<)lS;^wl z$V+18SN6aT!z{A~`Uj2^))@R9=eWv~-P>%u4Yx7>2;Ml0ge}s9G-1{R_jKVYno9)m zKLi1`X$E~7-fx86H-KZ5l1=NH$JQF~4#E8j8*q@5ui8inV>OCxNXgz?4M!bBJpc_S zI5!FoHzA;GK{Z!EWx?E70Dugy&E|L126d(cT55%E|I22>tb~bIY40}@iDL1-6y80e zL{QYAQdXV7{c=0)7ML65VzU!1wP7TJj}uVeNFg^VE-wCr#mA4)W$X#nXueD!Y;jv- zvdlv$)l1|VSKQ(PwTL`nCTdcsgTD^o4xVp1XHC=Qt0H>1Dift-U;_>aL0|2bQek!p zunSS`-UrG|31E9ZqQq6!fn!FfMsw@|5tZ=l2q|f=`3>phb}L>(o`@w3-s3%$(9S#I z!{XfCzJCT2lz-${yk5*0L4X5aGpdRe>9 zH2)$&8i8Yms!71SoLtITC9(S7454rCn*6YWZEQ~cxzLtL7^hWIT&=~3uN^1tJsy~e zoIE#5Fn9=ckV$T5L6uwHzl=O8Z?#41N>f=M(IIo%4%=?F){;Z%Q#uYD8AiSE#$e?@CC47eh2@2 z*RP`*1O1zR#SWneQb-tkca=o9)AZN)amw@)rxKptu2Uxtu!Z#J=d0g&gjf7NZkL)Y zyNAXE6A+&Fwbyg_e2y3t(9QRNOkB3<)f$XO(K#q0OZF&H{5cp;(%|(qLvqw6_mcpA zE0)8A2X362`ZT@ov9D2h^D(9?r=e;Q4aLXT$^P{qXGjOFRWn z58McbsP|sL!9K@ul(E4{g0$!^(zInO(c|lph>B=;3izyt%&%O;?|R|_919jWn#f`Z z2v|=4>&YW`2?)adtL^~_<+Ye*;Cn5^;uKOt3FO~N?C-~I(u=q3X|EF!s8Il!o=Rpn z*{i+6#@K^Z#nRIWFYO_cga`#K{z2m*oDf?A2ZS+1)X~i3(kh%#3`hXyZYm0zVtq5Ex@0~A0SMNqX z-CjM~N%0I3YwgHazX(XDZp(2)F227j-*@Z(nP`?VM8gOJ7d-=mtjq0RoN6P=+#l6P zuAlLYJM7Y_l4+2+f50=2M*@YN#ud z4f)Sd{ek)&(nDN)FHbh?cyn-T5IHUed^b+OPgFKQI{Enn|C%rCh^+75V#)i6#c^{& z{2yFIvQvYPCO<+ni6QvS)|M~)@@^UTb|d{aN`g~Ud!`HW2$#jGz^ym>h_NEx@{gTX zqxxPNqC!d_bo}=R#VcoW?pAUiZBo;N|3A_V1G|o z;fKT^`j{NH8eYf}*v4UehK*_Gpi{>`Aw(;4MUxHUa$iqnv;Uu; z)%93$Y}F3#V=tfaTV#5^c2(MQ*`?hL{{g}0#Qu4I}=XHge*;)Tf zfD``jZX5}I8r#PmrmNA0h2wTv)GDniKkR|!>kBLhdm+TT?2J=#>#Ny%9R38; zp8uPz*~#UPXY{YikZ8Yml*o2lMBe?obYO;UFOY2dDZ7{h9|e%&V;On9PDZt356w5m z;_7P#gVuht3EgSk(AgXHUmNr)9M=FN5Dn}Q`^>4JIsjKTUTOw)JF~CGNCK+&?QVsOOH#n=fzVVyq=kRbNW@i3XN{}~H!PP< zdL-rH%}hRUx%~(cf=_q2D;@_-sZO@3UNHFSB5(A0*m}p$odCC zTHFGpqwX4m)~Te%^u6=%=Xo2gcr_&}QejP^Ial#I_P3Sdtm`PM|NJqg>rkGYMd@H* z*hY@Q$F9-!xVRKE3*o^%UD-FbSxA``)dnBbC;Pp!$%Y(LGCNIbJT<{=))?httQ1l^ z=`_#nO}gxl3Fta&50vHb&pIS&F`yl`4RmZbL!ll8_Hr*(EVP(+;{QHV4B(@80e(;= zYPcW%-BCv74rkf~2hy>JZ(2OzGRoQ1hM)NApA!#1F4OXeioC9u^Qw=!Q9c7_xWR2V z5pYu8+-1z{p0>I9=+|NrUI6od3ZV3bvMnH9Ig`8Bm#~%VrrkwD?;FIaV&vn68TmAE z6~o)si|hfO01)j)EWN7k@y6gBOJybmTp~a~?!|x1G^Xy#y<};UF=flHw1s*)9P|~b zF?<=~tQ-rtC{W;t`%J+e&1>zE$yT1@ViiUJ2-z5>eK?Y)k{-6Ihq2=2eD1pU zV%DPndNDvcC=s?JyCXiplT>&66cFE~%*%Wh^Kuz*(DaWL;Auo~E6Z4Py5J>38ke7g z-p9WRkrS$qC|`0TOTq8No`1>8nfm~T&hKot+esllx}}a!^9C)ug;&0BLokl(&1nh} zd8DS1(M=QxA4tQCi?CDhQ}d$~DO%9o-TuMkMh zXWn;%iN$g&#`+}bi4P1-DD0p!^hk9mT;+IVXJbQ*39^3imrRh{KZSCYZupD$2*sU_ z_}%_dATQln*38e_!Q+JXVL||T;$@s**A9)Zl)uv6Uu+E|jkg3gk86I;(42oCBoap} zc8w{m9pF-lR?fZp?DWdxV{D;3b`(=n@hsGLvR!7fCPznyBEta%!)zO0L(_+p%dUkE z6=HiSwGC^%C|-KjtRgG309CT`m@jvlTFzDRyC`sjnDo4Ef%!&JW%=uvOyS3-cc@aY zKc4u6!$xkJe!_1+7c|+QY70w&PupEe)6UmDc~K|%gP_|@{hkj7+YM+`hH`QoV2e0d zj~}y(xq(=0^*>g;Boa5Z`tIO+piQ^e)VK@=V{?Inuiz0DJnChnz zVz$XTW#4rF_b857f1)YD{#u3s0?)L=oht^dLuDfOhLG>7)nRW7bicKVgY%JzNG zw&q2hr#}uV;iPEs)n|3SCrctYaK1P-Me;L@2bZWXpA!^LvNCU|Z@jHeqo&PdyW2SL zv0I*AeMBp+bD`Nr5xk<`Rtziu7%K#|WPC_X_-bb*(hEZ0Y2xJ1v*KnC^BK&QP<7kv zOyvurp<#4h_}LzCh3jVy;%?fOV37AZ`j)20_DoJ__%f+T|D*cXSLOUYrB<|SC0KF8 zVh@_5c>qxL&mbQ5a=Z>FN;5V{&S3@oK_PZP=7qVaPv6K>{i+$deb_M?^AR z{<|YT&kb-jKv?hn!ZEVUVP#f>;Tq%@Gyu8x@*&)xkPZA&i%ySTinR)3uXE04>X&Nt z+t%S4vyGbobO;%_waNH=%`Ct0K>s011-xEq|B#@x;4C}k(kbt-v$JYZxsAqcA?r8{ z#*j_xlt(wQI)z++3>>3-D2o!iBPt6+eQ{=c6O3R>#i%t56)Ihj5Ux*l7U1 zh#ksJT=j2^#}(=zD7d~#E@l4|=8b8lqomq9y8k=`u{<{aV9GbX5`{$d_RTTRlTTDW z9d)=B3oSlJ(*DL&VR4s=>+ySQqUq1!W(Nw+?1Trpz8AU1v|E>R#LQqVW?SUT@g*dF zCHPvbskcUE_^+#{m7XO)o25<`pzKw9_XqNC$0^^v-ESLrt) z$=%ppvQYY6Xv;3H#LC5qK_P>flLvq5uABb3`VVU#6Ok~N)FXooCeO2FUm!@th{uh$X(*X!(W#X6Sin?7z}GR@aEb$rPt zo&5jUy6$+Y_y4bbn{Lt&k(m``%WBy(Gb4(FjB^N)`Dx4EJ0v8#4i2)C#4$tYI7YIP zy*IztA*%cR-GA=mUZ3;%y!ZR{dcK~|*XtUB_0rE&o~5!2c+2x9$n+!2l7K27TQsFg z6>E8Lpxmv*LRA!7&hZV)KdZRq58Juc=+ftI>pbiWn7d)9>0DH5N{XxPo`tv?wF@;{ zWU$h{A98N0*R4J|ZCIA+f7m*@yFOitsPNer~4mQGu6d6t&OS2(YOe+ z^pXN~YbiXU?PN5n`>F}o|9ZaLp-#fP_gr}3cf4=eZKu;@QW=Q|JmNOUtNdSo`aO@+ zs{@0*@42)XzV$>omDK&OBYOp;ecAjXYzy_kjbJ-?1$FTF|9IQayxk2{7``p~#seCc zpYoqh_Uj_7g2Mixi|#5I?Mwf0=?zfEMYK;2zcFUbP1}Gdjtt)4fcm1iCpCUBQGFHB z_v#NMK}RHia^r6r%0#H}1PQj>YG3c!XT>7^^8rTe0SVCw1B5k9HvG5ie8V`Z0qQ^Q znHQPaVJ=4f`SR*PG7=gfn8ic{)Usv zQeX_iBA8r}XrilSBek~ zTLP4yfq~(bLIe*mK_FJmP4Pb0gH~Ao)i5FclN!K*oVqv#@DTY<{szR5S`)@iU;G7{ zIS5L+deB4h;KnQ4FKS#oO3SQ^-y4u${L&4!J(&z3E{YT{} zi5bhLOA61wNMWoqId_7IZi+>A6H`p*g!Jt zmmfX^0p!PI@+fD^eK$e;AA`9J}qG&o$)lHINOc7bp|=$!eDYg&{$p zz_om3`N|R?)~Ox8@FlWWmSo%cNdE~CCcx3az&L%8R{{_pplC!|=t9vGru$uo#cT={ z2pd6y%*tWKO#|^Y7ofLcBm$Y5{W^hcnR@T%bpjc(!63_G+`t(hm6+*jYPNW}s>A9kC=LhC-}($IWvxPPp}@U@HA}hL zsw_O4m0QzoU(hjVH#sb`0q}mSZ|k=f>SgO9Vg2L#e-HEfk^o0t6RU`_asnj3xM$C_ zF)`z&>%#;r98yH`52DIC%nV~pVzD5=(K&dchlEX%xdf4-3*lHeQ)NBK?vvn@4d{Q9 z(k$5nN$tC~RuRb~z{z8CHc&6* zOyD~ZCq#A;y0qRVWShFUv?e+*!s-7xLd&=Pe$)Fip9Oy=?;j4-%PdeJwB1(%fKhuh zYVb#U#2*+>D@3FP?I_S|z*Pv&xJ5(YfQ&S|8yNoPp;>2=H-is;iqK=9V*mO(N_6?* zOEd_sc-$2ePycS|w)taKKC%gktMtKtzY(-nt??>Eou@Xu6ziK}JwWkdpEX$8fZjxu zrzlqsFJl!5AlS{ecoxAVe9uk85!@z`&mCZ+qLyCD&OywjJwPW^knL*8>c7lYE2aAaYd+hvY)f}*+OAzlSPh_!`N|2y~$*=*>z zK*5=MU2Vbn_WG!RBRW*vc~m;0`7e8{A&<=4>k#v38bl4ldr6Z9>;4W zot>ySQoKAlM%y(4-mQ&o^U-v)c~0BSbK$in1s!$&Z2#RFt?cw`@w;tgzr4QI6CjPW z)kg5{9eY=5zqW!r=O0q^8=J)uBRO>it{jWxFObxJ{SR6#8S;BFp#`WJ|9NHye7E&O z!Jq;6^xzQp^?z5($bqWkci0i?cO;hVYraC>QDA-Rp0OG6p!hp^B4=nq&>6vdvmcM5 z)t*QP^X*UWkYC|zhI_Ke45X{UDwf0re39@rE|MdH~ z4b=yNz!3guku@`tY;a38cth87^~qyVb`h(le&2fWsGn{3CcX~>CXofYqIfAW+llYK z6m!~j?)FyLfxJ9?Y3Zq6ExGox4QQ?z_#Y~O6U1D|Nxo6gl99_JzWBp6IoZ~0Ea7AP z*@>^H#Nn&#rz=`gnn%iW(ktV`uG7rn9IWIZGZIj1PM1&|+;0TfLwqRfDrQ(+jqku=iiA+yf%wPZ=k&LnmXY+ z<@fa57(YM6KoirJo$%2$bSloP@C37SjIeF{>`1kIds*qDPXN-cj#zQ`_NUaxcIl^T z);6a~TyxVpr2G60GOy%GhxTv|ogFV5S)3btXXGxGmI;_ic~0$0L;Z7ts^J~I5m1&n zbP|?3oyQu7j1mn{xub3UI;;~io^-YWM@U@P^jhecIVWAp=cJ7WpM`yO+!c%9 z5O?yV>XV0~&_s2Zjw2`AZfG8(>=iI;oX;PK5p_dEn>xb87f@I%r|wgj$NfVsuIwck zfivtw#2ky2u#y8R0#*RiK@ilUey4{z%dPTcLv!>J9 z*9-5xTXu3=6);{I?_Pnriz=ud=TWm{8OZ?G(6u=E>`0zj6P3}B)Fc51($Y@QQW?161B^j|sCTA{ zIOUoB3k`f}Gksg?u=4ybO(SQ*tGW<_wZ$P+tMfnZ2OMAWC9`7dCnfC6+BDl}KkE0_ zxe^o699o|$AC|pM#&lfh`+}pGD>Q4O&~_Y0UNp@7)HZ8-B}y8roE(eemAjo>EvewA z!lAFw8ULBxCponyy2KZkio*1rG(%x|G-8djK7SrHWHhbEXiR@gip9cKcqRfYu0mc-2_wT{ytZz+TbZNnsWHtpk1%eRT}L5KIMXDP|6HqcwvZ|BN*W7m_#d@S$EZ@khfWsJ6j>>w^Kx3WUl- z4e}hmsrY!HvrSJ_(0wg-UFa?wJSMMt2hA};BmYHU?mu0bSw9Dnj3_EMP|miV;=y4w za?sAEkdGF~MjeUOkFZiGS-Y3^BNyNLLVO}m^pcxSiQ5eV7IcvxyY#pnJKUDiu{`JA zw7_Aa^}KXNq~a<&q?8A=uzKP1XnJLC#L=!teCYxgSYHR+PHBqBx_K#&#uj$9q@+(+ zNYx&^=Wqft-w=u$fSh8@g&&?BRtDx;2Xtm>{h$dA%2J@YzIa(6oX_m>Qfb~XInh80 zKf+4f*6JlK-uZt)N6sU7N=e^|vFoz^^1XggK6bP|7tQZ(7nTnQOCZT4C%BIk*qIG9 z#8ZLTRLeD(VTp6#ia|x`05Y&PCscl`_l=lFD6c7QdD+&UP|~;hqmRwEqj)u?nZu1q z%k?%mTw8~>0X_K65mB?Xz8Rmb!nh_sUDnaCgmeyYPn~z7b=?2VZtIf17nR*oZwc8q zE+1KglJJMh?13J$o8yJiKlPk*-vEV0Ry+da! zR5~85h57qm?1cMk%Y^Ifha3hxVxX!z&Knof;?^0$e%xa=XApv+ThD})g>~o);C?zl zba8%f`WFWofzf37MzB}68mE59_jv~oa%F^O?hsf83$%eb7cAVBKAY$Kl^{sq)1iL{ z!Qoi99y^U7dqhM6STK9!<5uc(w=d6Ym{%9S(~bSi%wooOvO!>z=+J>N{q~2oprN2# z_%OerotPghh!A=lZxr3^NLv6k9r9Q)Td}tm@-(Dtx|fx{-uC;VPNkXeH~8cq7VIED zDJ<9q)|15=-*tf5Fdz1Ow**zy=$L=-n?Ab>RQwkcK&7>A7u`Er(1FR((ISsIofWc? zWn?m)ta(-vm)W&+I^RtY-Ek9ie4bg%dY3{$i;DDK2uMwQ@#6lq9ak^>@)I6t-MJ7A zSg7IKgmRaR0xSO-ZM@*!4j#MVlJEUAu+9wQaf{jUpLPll#hZCF|4j2s)4?Hbdb97f z61o!S?nX=hIL38HIBIAlz~)Ki-4F8UiwW_Rn+D-d@KIz1Pi0C`Ew)3WvST2j;j!X~W#YI>Rh3X6($Twcfl)fUo3I-41qQ zvw3Y@9e@l#oR2YNF6GE9d##>wwM6RFdHGi)%+8Siij(G|mXr3Pj+5RZa>L0WV_QB& zuhVxFyEJ^qQU|ozFGZs{9j&314tsntR)zHi$szmCz_Iw{n6%mw5)tJ>-(pcTt z+St=L*0^MIurlOy>yEnJTmIDpzpALRZtI080O?p`Ng!ROHh^@mhJj@!pw63sm?k2L zVK;CrMSgF#ZMY<`6;);^>%hw#x7c(tonA!Ji+QE|dV=J$tb30n=keFpupU`97t#C_ z>7Iv98LZ_g5$^otEN)9FpG40iq2M5*21Eb!ZhB4ej>?a}8^ z0~bh7Z8kS{$e2Fy)<1uGg)b%-Zb`nX`NA>e&~-fI^!|b~t}oLYWRHQ!?Gi1HBc8xWZU5by z`{dW|83ms7Mn!a%meq4)~S<5Amrhno~15H+7#OMo$b#z zBGmV}u~)&QRl!A&T0w5xA5F-#?*8zkrl#gj&!@Y?U(e^&jqn&d z9Qen)HKPM6LmC$?2UBeOeG;meK^?=FP%jg>Ve{i<*1El4X{+V4$}K`&qoH6 zVn@LVDC^Gq`3Z=>gTN#P>MH1sj7XY&(@U4O@$Wp{2$IPSRA4*RdBev-QS;Y@k=zpN z?uV9ZACD+Q3Yn-{7OZ%io`^};4n%fn9=No&NG3$wXy=R134Fx26Hf^i?i!7tg;0 zwnFCt}jkSxv}0fC#>vu6Nm{q|U2vvTR>eFkFl^t4@TBq&8( zR~H5lC{XA7 zOC?LAA_2K1e!5gC5X4yx`&Z8rKY%kl_HHO+?-7>UIXW2G?SHhjoCj*z1I-#b^y)d$ z0AvQv$lNHb?&d+elRi(m$zcC&tvEw`eLeEs9(6tRD7e zI-)rQ^FmC~NZmtw4*+cON<98rvBwf@9N5EQ{McgH53pcA587?n485(!XMj}v_se?h zEf}jA*(OA(PPrjxe+l^#j~zTh1=z#H8hz=(Ryo4OI|2(w7bF+%>YM+uGv8D9gTRo7 zhiBTUXfh@8hxSYANds**iNJ#>Ucw2}Kc7tPheZtBS9^OL9>sIwz)Mp+2B}1h6PINK z{A*E0&;K|($TA}V@yRBRP%wV10;R(b!fwId60E8l6LqBt_B`o^-88u__Eo|a`|tTI z;DNULuT{w{9R2`*X52)z$jH_&b`td##DQcRz$%I8xdY}K*_!ab|M1H_C8PU-^Y{0c zfp=Il0t4Q|ZXMii_3Yi3eGqu4^%m6dC(1a8S9sU~`Td&b&<@B6dhoqbTWClKe~-ig zI&l|}SnU8EKWX#aG^rtw2B^34NfG*>C~K01=*|H?nF$TyTJIvqF9mqE1w){~K4eWxlM5D}$!CU4w0_gzOI zGGw>%b1=8vz@8RTYw5I)Q@Z)&$DjQxxg`|F#oby&d*vv4=j#(| zgo5Mqp<^o(J&AWMI>vH^1uV;-`WPj??%`=AJUtj?+izK8;w?=4g1q4=``MUpiKA+J zbySw4(JS#~Y6`_;&sq~TB9A+(*PRb(iSuh+GacMXNrIBxW=l@u#pHhTa>Wb%Q}}oF zLjin)=k*0GaJ3Utu-#(*@K16AN;s! z@v1Bv+Bh~`laZll8JYEsyeqK!zP4tls)Y97b%)kId(`OmeDr)%ru*Z^xA{(jQgLoS z`h(h<=j%3Gc}0o7E1U}*U?dY^&@Q=k{+nJndvq%7a7F0j-YMBY2Q`dbmYI|?`TqU; zp=c{Pxu1u}`KVf8cm|HNPxuK`vlc?%CY-+J2X0MPL7=)ZA@kD42ED z?MS1}6E?xfjc=%2NXWtHV)&@Z{Ly3}m96$M*|FZzJWM`YSF&Q(HZsC)GBUll?ZJv- zHjz5%uLxxKEn$UZXkc^~l><$Py-N5FaG2$i)RghhwYNIyn=RIOF0T+^ndO#G3^=&?Xu9RnU9EjrHabk2HB!gn-=v09 z<|7}W@_@Pw$*7Bsja}CE9Qb$sJ0BjGc;wajQ>;e|>_@(@Z722G_&%pz`+e2M(sUe3 zcTcX)fn9Rtpw3z-A-jm%b}<5vD|TI~litM`r2kW`#O?!iRVrFxj~rC+qa*Bw(Ny__ z^~sTc6795*MOpu#3%x^PkuHdmVX$;DxxAdDP8hQgU0huusrho9$QY_b2Ut|Yp#O^ZlAsBZ>+9=FTyBw@ z`PpoINmE@qLo>Y%5|<_Fxq&$YnW{9@r9-zt&iws^-NDH66atY5ry_kf708qU$vobW z1|~Sd63QmJ59dSEvJB9m-@`URaiS2qYGCsM6}17x9Z=UC(yrHppEs(H5i1WsLkPY) zAZPCK?&Sb!T6*IBKx7pZkxh`?k%i^cI>7RD7mWQ2Z{sK-yC4480dQ?*kfGN4Xtbae z-nB-gEn;AGJEipzNd0ssAIzSDyb-jrh%>IE;qWLkarCy!=yXE>`kQntN*CmsSFc_L zX_~n?d)aoMz$XWsXLxWU@o~=6bMZ}1$1QDst0BhPN=uE5& zPeo>OuLBX3nyo1s^5z3Gtp;>rPTB4DAQ-KZ01CFO>PSwjKFp}NxHzHyDu$W-Cr+C6 z+uL>l;v`M1L#LxO*Bba#Rd<>2Ft=Lc@g!v}kTVhqHc0xLowS7ASRczBaGJ4}5pee= z3<=0QGT!a&?G57&bZ^r-OL=+uAc1f&&uvyf-o3D}z!sMHFVlF2UVh*T*JS(PG&pgU zr(_%OeUSX4GCO^ppgpZ4Qwg$m&!qum&o(fr=Hx$H~K--*6NHw6wHP zGt5R?P8pMTyMZC!Wp&xn3<^DdLfEvbYijIbWJfKlr~cU`8u|gH8`EP8e!c1K8{i>8N+KAxU|_XE7|m=Hmj z+7}D6+A?@-FVnz47^hyI%4)Kt*D+|QjoYeQj0n80?8Apw2(VR18h0)HWda)q$c;e3 zKY$lP;?wSn)OLek1y#X7`=6qo94HmfHjg9hP^haFrUdm_NHu8ouZ{BJqtTAEarY}! z!!4_AU9Li&KDv9~wmEv+pzxArxXa@2hI=*PFbUPou_R#aB%VI7+_ApI?ibVxbR;UoqOPA;+lVP(J5SenA~>WA&Fz9EOD!QZm|E zq`Oa1^Z66RPU$!|q8}U)5+#!#vM42tLMm0VQ}8UT%C_2fqQ z@TADAFV*|G3%`wyt`e+~ZIatj=DAuhihn(V7qb%jN#wmJw<1_CR?As7u4Pc+eW>zP`^6bY(Mjx|Wr zg+xO+7LMh3$Z38?={f665**~%gQT-}E*M?1Jn1!kKP;>ja=FBa5m&p?$=L$Prsn1( zoTgvnQL6eO@8AFnW>YgXGD2XepiUb4O9h?9wsk zZarMqXf*ux{sFR3ZGc5y7{@q7#@n0J$GG-Z9xmv}C}ht;;R7F9MH$$6l&p+c0saBy zSO&ps4$h_QyoNiveBA~WMI%8Um6w;d{7E|EpS~5JI%RLpnt0n%ll8ReL|PF`7w+14IRVo z-@6xYUXON2Zp)55lNMp~5MZOicQV z{A^?V5^pZ})W5qs1eZv#NYTD~b{SJtOepF~f%U8LHtwi9bd0oRfanh^$>QsqxR0Lz zl&Jdg^;m_`HT|NR6*;4AdOvw(Dh1#$2kU`#ig1jUR@|f}98Lxa$Ynwr7E;0pWR&z1 zVbHRJ_B+?A%k6k=KSN2q6r3-lgYtbe>Et&aX@ozK1&de}&4f-?+hl;d_^UOPo?y06 zSMh5txe8zpqNqtn(&&LqT!=FlaqFS>(zf!uZn+1K?%Q0#zkXgR5N3a6+aV;Fx?h39PbmO(3B-1|-c_=}vf2zN5eL?|Os3DZd*=9u&A z)0G0j;;v;6pc3a-lqoLSN8t=^ z$?41UEsHxA&QWZ=;WE$gasCjxFcUT0GxK7s`>k#3Cmx(FTp#Cubgfz9*v00&pO@c< z`Gi3xLkPQDU%H(0nA{fVa$@hsH2b5iB?I`e;ulrB?)^(A1vUn0>qEI^<}$a5mwog3 zAA98lA3T3FYtX9gbOobTx~KEK=Tp3wt0s;$+2lEX zXK*>^)X`7 z6yM&FoKEeRd5Fg+8kcC&62k*TiKx3MYSJ&=47Sr$K(!On$;Qa>#1eDr7+88au(?*8|8$GSu*wUrsX#)qX8WUU0nU;(% zPV-l1p5GJ>Hq6lh)pDQI<~m|4h;ie`&8WS5=elNFqf#1GtBpCCJF4qmgb|1`9#P6i z0@gRlX`)J1v2NZac4l-1%fs4ywy{v}8?zC3)XVnGIbl^4Y)oz2uCA6t>5jcg*_4N+tB= zk2sFWhm3mrMzBXRmceZ|H{^x0`SpOG(&E`3! zH{V?61O{u;!ObPfToHq~Jg)cWyeX1%H{M(M_WMRiP*J3B#S;M;C6%c+7oJ>Q{zpoD zV3r8~t_1mX9sS8*YlqBsNLwM;dBVYuo=JLY+*-nUmSZ3WqS@6GxoF*yhXfYJpb&J{ zv1E1jKG#6dIQ;%>MhKeU!YM<$2l@F0nSK9$aI*bt)|q+z%QC(!d$fz3T4L#z0vtg= z_LM6I&AwL7*@s_0{$O7WTIv?e-pIG^S11$}eE6_4u=$#XtHRYLft<2fSdl@qfB!av6K>^?}(xyLZ*jwpl;}?sF=lYvU{yHFG&GcuD z_({w{HYl&j_Hz-JjR%bU9F=24&AY@yvYW8RR_v`cwx&>HUR1RG_}NIO@oy}Bx3_Hz z>JyS!q2xeZyPw1%VN1r7c=q<>maL1CGp#KL}b z`R3YcM{U2ga5vawT-uXT^!1*<>?v4GmZroVeNpjN6yz-s^O-mS593JeRH$aqK;7^ z`hrw`dzyBcMrtR1DpGSO%w~I_X_W+ooushoe+>dIYh(DvbLWWwfuHG#h_*IbkT+@bwsduG<4!LfBAqzd&NLA!?*9_@eb;IWm6Z%CT?o0?o z=Vwx#e5a!He&MMsGjX4z=H^|4J!mXK4DLePVs@8ROfm`E$+!}u7mRgnCZ+OiQZ*Gr zcB^5}3fPC}$;h?6D9n0$A57{AX~@NYG6H;R$M zTuHA_AV(Fy!_Z$9;wnc8mR*Tu!5zlF;-$#E0xczZF9(w-J;LU#mylR}<-vod#J&{GbcW-9w^)d1-aH9;>4k;uM~O+Kap?{H5-?>%Kh;ji zyR$O=*6Q;q#70s$#G7BTR(3iyg^W{tNsyjRJxaL&TK8M;O z=PSU|8XY1s3*CG0r7egbQi zx*z&n@hyH=$h>L@0kxtQN@0~pZI$xvYvtqJBU*l$M?nsMRe#IeZRz1>e=WjL~JQ;wNEFN*P34AqpgdU+5zy_b$JiNXPpAzGE)%SwB_0VB6 zVO;9**{kY9$yAHxGBW?jJ8m6EwGl!=pVA9z|CLJW`&vAUC`(BdEFbyh;K3=ZYBElH zYK*tPEzvb0(l#r~PKTVX*2sweq@uO;8&G@=+5v?Ej-)dO2nfmLh1;2VB-<_QQ~FXs zZoi5J;ubq!Ke709C7a)c|B?lqohV%he(GlmMkv6k3_ttztHhgQKeSabU1#+bvQ}2d z9;4CjW~sAZYF1U^8)V+d;$#a!yaQoD$RUc`G!azxWh5DNbdDYELFEDeoMZ*c@GX+3(;t%-fhLs=qBp6zpKcuSrpGT zlHbvU*jrxMxbDOK=^lp0`W`&{)>-teeWK&{8t4uFN(=u_I$@Ki0(|WR%eq`?U~W*+ z=$mg{H+;q@6eJtcPKi`tSy}d*c3pj&cuT?~!lp#BW!7tSvAtz*D&%I6lA2qrB36+L zBF>ffE8kRp8&?HO0EPr1Sw`v@U`(8p%CSNJNRz}SFCTX=S?Ycz{NQK4Jpb6)*4=5W zp(B!OX_<0S=Th^0V~zIf#cLPnXUYClI7Lg#mGi3b!>eWSC{Y)xospY2@$Tbi1tBWK zcM?>3lRtyfj2~s~Wuv8V4}=RvlAf7$sVC5#ja-u!!HQZt&hz}kx6vF@YjiMhQyad8 z+$1o0AhOu6Js_{j9>Sh|?{Td@1L2r}^PSs!1KkCFFJ-#-t42V-Vx@`{pe!W zrZw^HWgigsffACYm#DA65|n1hYyt^W63p!YZJhYLJI&wHE4+l^%(WO_5`6^ z1w4}I{xH1tuAyc+1JPiC{b}lZ+KWv*MG;H!bUc(>Z8{p=$ZX;|W^YGvW!uKkg3Au9 zvd+#<%*AU{cD6`Wdc-nb<;88sd-uSpMa0F#qSu@C`d@-Qq+kQz54&%tO8@-QQqa0- zdX#}kapZ{h@-|j4T7ZM%8*cMN6tB$cLTMjsp^bKfMXe{ZG&{nKhrIZ~QR)#Ln2jT~ z$`lt8B`E8ZpYLFuTAO+8JlkubAReeV51CpuOh_jM0!PKhW`<{1N_6RD8^P(t7T*sF zdHpHf8PMjm>IllP ztQl&1rz{_Ta$ok6WF%#d&Z#3uuB;zM>d!1;Era|psYy2fNhosZR#sT0A)n>!ca?K& zug?WB)V^vRO02pU#02a0;*(08Rmn;Vibpmdf0Fg(fkxOl<+ndBA3c0iRpk~7GYgAU zV2!1~Q~(pn)rFygEyrmAbdZm_m#MPFZWnP%4e z0qqHdba*XZhwZ|{%FL;AEZ)5=Uad6Y#Rmn?<-)*NoyjbDw})`XZLF$3QDW6QTMstu zi4_3$gX$9Q%WjzSl>*1&Z>ok<{@P8t^vf7AwfBb8;9-ew3#U}C*?LZE!q$2?LP#yf z>l$GViU3*1kSXkA?@!qH04gd~2K?LoD%yel*S%xf3$JiZeK-{KaGwaXZN4rvcRG9O_}Pkj z=2yX0xSq@-sPpfZ@JL9@-by2f(#*3as^P(&!kbUeO1O#K<23$>{o0}|Eb;RNgB7vO zS5%bo2O=SjaCSJF!qd>2UyYeP<2)IV>7*EV^g8NpS5bV@DczHHJceb2=txz9QpMBA zGj+id@E)siEHL&C%M;mKo8vN~>~6Z#Ow8$HHjRzp zNQy<+Y)7GExFR7#|En$EO=Kc8(iQh*TI7f+JAM!vP06xx=bt8G+iMDAmZyo}ud=DF z7-(Go@R%UkuiRrB@({(YxUi?T;euncCs8))qI}odY*o? zq2X{$(Ja~%#O3w{Me_^~{(Kf$YU-xCl&ksjK^p)5td%<^Va53Sl~Oxq$K*majLu7L z5BiX>HRhyoISflAIC(XVXMag_M>2jfGVOTh`3Qyb!+RC>!Lz@XF%fZXwop~5E?o|- zi^l9>sd@IgNzkdf<@>Xpe~%3^Fn6js@2)HIMUyR4fv zaKZS`VOiyXX{`KN?DBA-15Io@)FjFSiKHQ+@jCs=vp46|UtcIO)Ro9_n5GO3aabAU zMjtfLuzseLBoC+6QB*)74k?4~DF8SH4Znz6VG|n>rSo#&CVJ#>7?WMup_SZB%98W; z`mntS^zO~kre8fgdY;Io<6Eus6ZO$4G-Pg)%`qhIB%o{ZEr>fpl{dd`>-#?nBs88G`U}ba6H9&(fUMqMw5qm;oft%}tB-5Xh zW?$je5XmprH>#N#leo;!JEf9ka!}qCY6fNPAm$(zGiBsX`7V7`#S=a50=HELmKQ$m zQ~d`fQ+3;!&84_c$J@y{ceEisXz~v?y2|j2VZ-uF<|%L#$WMOhX*FykPatA`s_?906krHKn{`iK>N(#*{0N?SimlGBZB zdEHg)DgxSXfNj=cdB~t(w&RzV9pezE$sI5yEALhB76TqD6@N;J9>5Qj$G%bL@Q6-Xxg=(w7uEeHaDW~C9j5G?zg57J^dH}z-3r9ZSUQCxczy} zS~HM!U;P?}g1P{*^Ela*fsiVyD<)Ciboys=qU+L<@!eETsVi5GS2aAh+M#wV-m&wE zJ{^-tUpS#YOE>qBO(5yhBK02({^!#~^s#t6p1My-H^=;)d9z4e`Rmg0x5J`#GSLvr z!8fX8*e_-y=jUj(rK@)Jp>M*JrE^O8(fS0(Yjd9LtE!C>3@pDL<8WsZ_1xWksV1T) z+SzTP{IoMlIF<7}BV!nTSAp1Drqc`MDS$>H)1(O@4lUoms)#DyECm10hR+ z1wR`5;sCC~UIsM4Q?GK03*|hLV+tv><2g`bo(vi({n2RQ>hlHH~{%>?4P0Pf__;H2-9-QX1HEc6b9OSk`Fqtup{7O(m~x=f;bp-ag~$)f^V6KiR*Z;&~3*(7yOCzE9An9nK!)k`6&W0m$ zRPR5(*O1J6hWhJXkhUC3n)QwOe!;6@thmXH??I|oy{Few@8Zmr3|m9T^;>9s_$c<4 zQ3Dr41f0*Li2(nHtP-~-7x9i0`)({peo4=TFc3+^;mUBLi+Ug}|Kr(>M=z9&fn~e| z7lgKU=b(MJb@qYi{v1b1ebT}7`vP2>PiajE4}FxW|Fvv!(zk0)BPo-`QtfX5LiQG- z`qk5H!&JEM8HJwhWrXYAT_16uG49SyQ~qQ-*WewKp_yuY{^YS^U>O6Urq;v=`i48L_7TKN^8fp+StzmKQi|QXxt9xlrokzKX(F*Vo;|1^~=kq zQOGmu@smRhA1nd1_ZV9##68V4nMfHQA0HlD3Ot$sn$y32@zRty7_`iGXYDEuy61 z-@5B2Z}@?3$g4asI2d*A1h8Jy{876gxW^@bK1A-QjwtM2XH_NSqt`8n_lFgLXlA;nCnAqlHs zHU_Lh+Z#i^$PhPO*%Kmt)~_RfcgT4*yufC^5OBy(6Gts?ndj)JfmL=rn^TJ1T{9Thzl(yQmaIv=6LWtlu!G0e;M`pwqVbA60)xf&YN#x7R8bTeL0h-ZpX zF|nfW17Utgjrd=VoX*RrgHQyT7BW#+$*F4%flK0D7~vRAF#YSB)pK+y3@M#D9-VDC z2e$|HMd10C{%(>y(C)5=(AJgTPB5h-lzoB5RQ_(v8OM*hRdRf|nWk57ZNnYFUvt^? zOKSVR2KlifW2>I#e9uU4eZOflL~%3Rze*=|1k@)`k`Qy_(?RsIKc|$tfZH4XaLV90Su-MzM5*WHf`Fepg7ayzTn;_Qh0CT<|Fpn#g#^Mcc9 zzS|wr3*%9XvqURko|koRz%PJcBqkxH(B0=T1wun03V_8b#grOM?gp6B$4k6= z{rYvN9HPDbWu;?&K0*RU#&B^$*%qy&m^yoiTuRMtJ{=MtZ380@CR!c|<2Gn9&xv(g zG*eOe4B7?AxX(Nk=l8&QYGK+D4(wRv*lK`u(jgOGe8cPJpWnWH+YF?oX>zQOo=QxX zB0!4x#xqcWCG-;g#cL`H+@Y3;+5)#BtBMO#C%}X2emQA~fUJI-@2A}bwp^7LtdDE! z7EK0VTT|_$JYpsQh@{3RD4PSf5&Kn~E7bFVom-{Ya`EuRYn#C0!?=s*Jzyg<0o3Bc zwD%X<0TP?)lV0Tv6QN|3Pev{cF%xln5Z6+b7vYul91F?J5(Q9t5HK=Q3$Wkc!VxlEUkkw{@LXC;&))DeS<%PQ2big`xMe?UN z*aU#eWBb85-i{psOyt1f4G6*?jj$^I+LaM#?AnrX_wPTkDuQLe-0ar&Z!{yZcH!3~U9TzbmXC|}m9sG+`o#|Reh7_p`wbwny24FLVfO#+!ru>B?`^0>o_4b5 z#;oIqEnT=ADo5U_3yeFqK+z8yo6j7lOPsCa9>2x31@t{ZedU%?AMCEd$<&Mg73XJy zJtvn{j&p0{vp=+BD|)ZzCARQvQXKzU>e2jf!T|EkbNb+JIhQ0U$vbYi2iBOA}&b}i__f|J{3B(`u>OMxy`nsKY)u%F0;r6DZ90K&Q zAGZY|hkWy6+(OPAU7sHSwIzEXyiw|$%_OZ*ed*O(HQ%!b6^L&Khm{bY{g@=9E46^} z#%4G=K`dSL+^tVw{-Dmy9Na^5aQ^wdO{~8=Mn--3)CjlF;W6G%CjS)N8TnirP@xT4T=o>-b zS`!4yVF6c8bYgQsuyXd@#bvH^#b1Sb(jitX_AzAVF6dCo-&k1Cch;nt zg5XvHYwGt-fKraMIws(xxYefZwSTB#HgKFu0ZPx(CBnVfw}paO6iz<*lT4=mJN+XH zEJwB&0!Z0Nv8_Gr4eEh{)mYn~MH`s1wBlNqJ-ZH;1QP z5r;IV^!0UjyUh-3)rmm`>oCVI96HA4Mw->YGlMl_&x+}^OQrY*mb1sk}*fyBhz|; zEaBz${KpJ60)P;=te^nD1lABt)Fqtb{{jqhIBm|zlGU8}B-0s44ZOTKq||re)-5_n z-}_>iMi^`0M#4q9lGO?zX^80Bvj6n7gaMt1oeYSdob#pgBQveS>Lp^Mx^DCemGN6F z+YV1pIHEe<2ryRw_TF|O_m7HhmoAxRn;Wci-DYU5MUxysp2=gbp67C}LR zn#tb}pDI!%u$i@jXO|`HGSUWEP-@Dw>S#fw>r)#|h^mJveo!C~USAUzhuS6>sky!1;z^;SV4HU3J)I><)%P=@*;8hG4#;>dWF7y3*m{ z*eN;6>+&P6Mb!W+DaB=ru(W(hHVUqWt5+}Q=P6kpVfB0S3Q!1 zmTio|i_ zTyL)rPb(DP4!L^ABx)1zSki=`d_ZGRYP2}IrbwDyiHp@L+-3N@6~avP{R{D!Wc$W7 z%110-w(+5nOVaHwRI;0vBTNCjS7kuw`o;On>s1dAgUY%iXAGZnmmXyyPX6cnq#I3&-dJRx*LCj-`q4ShXn@p&Cn3^}5PgXwk*4WUT^e4?X3!e#v1tDrMI zH9$@QJjPr(l5g_ElZ13#t;&47VsF2`K0k|+OaIrm={AOw2e=h*eU6hPA{bm_B>$T^ z0G(s*$j0rp1%yZkZUG^eB^R<~lX;Tk*)>EHo&hO2c!1WO5fr<7`tVJ~E2Q!^+0&}( zU!<|fSf@YaPFY&C-wU)E2ns(9UE5pSaR@z!tI6joDK_)r$viBV76rRvq<#3;g4iHebcL z`Ln1Eo&~;m0FJX%t=e5hamlpUazbI zdvsS)y%BJU=BCp+;aE5>ocdj8p(phbs`tu6agz?^&4U>IgZj`{LAOY+;QGI48HEOK z51<|FCY@uP6^Sp9%BMbQOLzeUOb(FTAwps%#0~o2XSoz z%I3{czXx((U%?Y<%o0DDrDP70sNi~LC=vG_l!PooLf4_3w%t3>xu!N({c*3}P%b%1 zbb6eD>FQdGzNi`Uk;oVu^QsvN)BVlNNMhJDcoyE>7N*rM$c*c08Y~%Q$?(i1FcdkN z75Fx%2wL!++$(?=@PI;-H<5K{zy8YaQMVp3K{v-3*xy_HF8G#7Ghj`e7(3<@Y_k7g zn8|)}^7_?7vf9%kG>2jyZ(gI6A5F4GRuoQM8?ZXu?Bph9RocAYxiXz$A#IJWHQg7( zKVLs5)#((i_yiiTI3c<~-7!G%g5rWJj7&bw zkt2*(&hFqU1BLEpFWc|rQyqAk=2v@tO8dt{V+eQrm#9(ye`Q^HAXIxBrlPO3FUnG` zu3bpFN)nYNyDLkK$ZhO1wk+A^xOL;Ilr>>c*-}Z7VG2{%?J|S0FJpulWS4#Sy=SJn z_kQ2M<(xUc<(%L9yzlcoZ{X;YrlNf-1r5%B98NfqD2(YAtZAP~GBkA>nwap+`KfQO za^D;Sd)eWd%eNha2#w`*fXJ4eE1|h*=AYV1m-v(CysjPzZhy1*>Bx@~>!r%2Zb+Y# zSub0z|D9psY24>nXe|dWKc+G7FV_vv{%9iBUWU38#y69 zgBF`#=<(`>>GK+f8S$Eg{mgqI%#7C}Ea7Y698OB-o*Kw}EY;wY^@9N9i5G`{g;ycb z-A}O(vBlUrVxhzh-Iw3uI+Kx2PLS}pr11k}Jh0496Q9j2#n)>+{r>-6DX5$bI2sAq zbKKIXREt0$aBD>H1$!l7nGKs7_iv6nvhYZU{np35!mZcS+E~;cgFn#Ws<=mLFMH86 z7%xHgu(0L?DlU)`1qw<@=S>UBFlP1gL_tG#lR}|b=iUX8V4?w(pVodvKLv5zp!Whv z7Qf;njfnpwmulwA;bwn9!q#P;C3H}?t1Z++e=8t|uI*!U%h$uw9kv7LdMI?)qpcT3 z8T^U}&$(YsMb*bx;i=1?^$iW7Rnf8?s(gF6CF&;B=-}5jX+*Rgd~KVL531XTKFD%$ ztVrNwdM)V8gtx6Kpy7uJFIt#?3z) zcjnSC3ijYA2RWZJF0O7Pv;wa|0ygYj(RJt@x#F_YZd|DaN^`X`T6y;j`wv^EXp;0d zlclM7r+1Gp#<4)pSzgVM;aBs-l z!-)u}gVyMGQn~4rSJtu>{a`Dk$G00ic$GW)@_AQ>+ldM1cPD$cqGQn@AJShgOExDjK(jAVMmu>c`@x9Em}AH%rSBjz&RC_5KGvl9(}5kI7)$ST zWY|O)Z~?gXbUiu&QqKSt0x9rbyR|WDQDZ_mgKzt%&0Nsbyd!RZ9sI^0(~*3{uwpO#v-_PM^Yi@?-cW3@qy<9SXqdwEYneLGap~$@C;apI2L?KL)nvDp_5T~h;S26d`W~r&6;~-mOdz;4JIa7I4W;Yo(AVJ&Mjqg&0_%NdBZRNI z=PQ?ll&fnA3-V!AiFo8Y;QadheWJ|N5WmaN9)vJ|Ca+{(iLhJ;Eaz19vD-i-0B~Hxz3X( zodCT}M+W?8-O$lDmaNeA!Bd)%HEAa-D2NE|+!+mK3`t2eWF2S!1d~Fh41Z&pX#QzB zkh-=kN%q6%g@#hS8O*mcmUba8M#kF3xgQB1)Rmn^|Ji-4cdGKfTva@{A=U&9<&*)cN(HQvQL9uN7;fL6b)X61Y=WZ0`w}`thcmODl`MsR-eG;3A=KG{L+5XuJ0>d8M<@ zrx0i^i%BU5ohgQQ%Qc(Xs&cE6Yk~~U?)<|c*&013pp{IiI( z>TpZ0_($&^O#|Q@tj*+6L{97g$=@031BldAHwXdc@s{ zzESZZBu0(Ex_+!V*`w?S9z=XC>B0QX$)1aMds%c+Tbp@|Hq`o$>5z2kh8(`VWSU6;K5({;8-RRxBaK+hY#6Xq)ATDP+fo+_pW^m{K0{`AygoC(Ui zyS*iIowC}!*5o|cx?-UY_nH?ow-oMqP8^u)8cs>bvteW-9XU46f5Yk3L5Cu%|5P!vJS8NwiH=W7J0RIYO^CO?^$}m;UN6Jj5ZQih@&K9%LO3WuNru%vkpn zG*ndt;Q<|r&*zDEu+3G%7vn9p`^%nhNx)_h?E!6fifv+l@Sud$OwRYDe*VAVN^1@pQYrLEXRd$AAh{vKDutJqsGcc*}}2 z;}S+dgqw*gE4$>4W!9>D#L5P*_-fwZ_08DQxATC?EnS2Cm^ty%??;*?J z7a$wRw~a_0WkNvhxJ^NnFl1 zG&0W5_Uc{KBMEE~sRFmopt^vVd~kv6dZ1_w&he)5uwM^=|56hqgplg$b46CwDHEGX#8 zPs=#9fxIf@aVNHFb#*PS3esFrRYhveBRnYAM?joPL30F$n5?tF9lCHKy$EO3`S482 zU}G9!b80%Mm+j}qwu_u99t5IFfeI0~o}HxwQh=OV2<5lP@bDne#Kg;(4*?-aENar& ztz|wa&?4{W^!A<)jYBQDnaeTp+PkF^GSh|cvi>DnV8ziM|jmwltg54lE^cf5}ow$-I$jLQD)c#+6y* z6w>i~eX)gn)xkbjo;26H9zz6rTD^M0BdVm!h!4wjJ5T0rhd@mDJrvb=0M6&DUNPDa_XwY zX^Q3OWCxCPph&G8P@AIzbE%4|%-+uqRLBEkr4b0B6n)P_?r~{40NRhQ!uzl^H*Pyr zkyFY1(tPGlSDqisu%t=Mkcv>w&bgdBel+FKjz~a&lq|k{fQUS>)AT=jlRX^BkfK z3=Mt3oqv>@JsaZ6$@p=nq2Es;vDZ`uCsqN=|Bj>`6~uw0MXTsq&;Mh21Ai3D2XWRk zqWlvyipavN`m9K}bcBd?`l+c5wl%zq-8l@=R$TU|k3l6$RHz8MpS{r+gf%&ZA-#}p zcFx#>tG60y>*HGwq~t;)xR#JJyC zgel{De{C;Qa2b#{x5fTTX~ClHsnVP><;0^$?LcKAI(+VMOECYN%T~Zf0BPN)aYC_+ z(Zj$o)0&{F0$Zx|TjuRmtD8#p77W5370AjY6%PNDoPp4rK$c?aMw7+S#tW!swBH(V zoZV$kPDH&tpEM4}S>n!~JBP|Fs>IHSGtE|SpQWXYTot<@v0JiHGFIw>^lrTZPT*|$ zKhS;W&YkoGFEbf)XDd)wL|`#(BOf01-8G0bh%$&ZxNne%7!VE83?3U~8c5Y}f)1N_ zDEmVDRhW?c9+2e#YTd;J25_e9V7om&-1uE#(Gi^!y(z{eh8iOs8xcDgs~eXSw<(js z+4;49i!0Wlgarr2?8BKDJo84`c`1LaEA~1z32=L(4)XHrY0pOORrzr*ovEbqPxU$Q zkXv)uPsHWWiHamiC_7;f2jf|Oc{^jk&G3Uf-gm6|zj_KbX47!_ThvVGjG75@hYrl_ z?IaZfqJzvj1ow(c$EoHT$NfJB^~E;_-;>)G%~x@{?V<0u=)W-NnJ(wLT<>9V#8qZV z(`+U(qBtq_8J_V`{Dx-9)`z2>^ZX$3Z!3${<_czp^~0Bp8E$bUF%s# zC7sLTIr?pWq7xGZL`2k~fw=p}H#1M#FXrbaV=j4kj7RL2q!AV@zs;H}qU_Px0lEbF zOK-r=;)sgMLW9-PD{)bTNCE4V_h2TtFh5^Oq519yA4>?!G<5ZijYZT6B}GO4Ha0fg zmbbwt9asZ!oXfU4)o~tNycJg-zFp+V<mbxA1Z zG&s?Kp$nFB{a7cs?41aj1zRyH(2L-^hrbY7agP?hD}CqvP!SWNLBa_Jg?2 z67n~x9lWFfWxrkmt``i-ulTVJGJHzOHlA{X!qpnY#7`%jzNuWI)Qk&?zHaY zjqS+k6NKgBsYO2i!=GF8z>R}57I`wI$S~J}GTopbK=6PCLVSFDV4jDXTwtCdWJM2@ zhs9V&N7%c{^73@@z+aIQM?=^oi6i*NNdI zjc&EVmW4nQSZ!sado~&Q%Ecji7QM^PTHl=kO@ljqo`irEqL~)a+RYV7R>H4M* znuf7gk0`0ad+>=74IJTR$Q^JP6f-Hmya9OHvMZSy4UIN^hICK6)cTpOmt$< zWIeN_SYqU2YcqHdMv96d5BbFM#(mVjU+KGLks7Nn_SID-FqET_Fl#;59{8@)8S}$e<-MwIYsL*9Pf1x7^Mp>S z6_f|DCKCD`rntHSS3egM4@XSZc+0 z+BT9y)wOFL9e|^`h1oVVabWxp2gVbgXbPzv?tJefnLs zM&(eo#*1jU9i?LflkjneUIt{yo0V0|6{YGB^B;|2n{XZ`Dvdi!$Lt|+;ULj(WRh)2J$kj}jPI?YXpUCNW~dpG+8dG8w|VT} zG$qS8O8EY2OnyP+DX(6nEw4D^>e#z!=6RZXb@YkqoueJ(qDc@=JWXEEeSG%n*7wze zn}V2g6c=unK~lObq+w6)#KO{_KeJh7GK6k04Nr2(Ax70o^KMxwK|(*$1x9&Ur0(3U z&M7A5eq#@3mlkVjo(cPqGobsul}sUUj4tS*aRg?up^?-x-B0o=c#MJXv2oG844Eoufou)X_y(D--!@}CICI`Xe7qm!F-`Chw)>KDm}ndU*Ln8r#-Hb7;l6*K zx1+1b$;tWr`3VF;1_rb6FApXI($dn= zaSxh!l;us$qBhrQlPF(IbH6;D(Gj}6NsoYqHwqlyR8unF?X$YIIr)#GXDPdxI)yF*Kx zeXEm4TTeW(k1nh1qgbtp&LsvTMb=Grne7Cm@49Vur@-#s*h()tXZUA>v3Z5l^}hyo{2~jEb4h3^KyF>6&9J$huB!ho%xo~ z(9m100--PDl344{?U#F#zu;v7EBkQxxW>b?6_ekL6()cD`0-`bD+Fu;pIIBvh`7pS zYbIAM`{9P93?w77r}a`t=V*1v{Os`e7cNN>5|T*-+=|0MQ)Fc%PYZm|68oMhF$FbQ z5uc!7*P^{I6FogWx*z3iNmx{nw{oFw079ZJh1wic3o&pEx1+Ls$@0E+Qf_>v^Qt(B*n{ zaimjW4JR^)Pw`BT?rCZ=ZVi3R6$myw@?nIQKgQ#S*x1-OI5_0wsqyjgq7$v{&9Nql zX;Soei^2XqPfKj5=|D92pf1dUov%kQuo~2<&uCt1$tcG0f7ziUm0DH1v^rYYfKQg; zK*+tN9tn3|>!}?3*J>BNl;yhp$f z{9^MTcl*G)+S}PUIh#oO<6Sq#q2dhbFzr$^g<7|Rsp;ur^YzgJ&(r-}_1x&%+S=Vf z9DYN`Pdsq#G;u#1Drup&+s|CX{rvnI#5D@E4-#$aqQV6{Ps>V6Id_e(-rv3dEgNht zIdoneOTofYXxbGIwhwpFis@HJ3~i}d&zq_5#eRNL$pTJ1u<_4Le5WVSN!m$6L&N;+ z#>PevE^Ydc&N#;E%1UAAQg@@Tqod(*o_ek-m>w5yfV@?1zyneNr#1Cr&$C0gx=mX+ zX_dU1&C<~uzQWWDvi(wBm|C9gqjbt$C)~eakh+y({M<=ED1iMEu&uAo>sn!i-JQ zA*QtIuoH(sh8?Ae5>x7lFUSI7`B-~LpBLjE()7*kCOYEcw-5x(+L_FpUjo9y6biJ< z*|1t7+QP_1y*gv*)6&v_KeQEV73s&>$FmxcAy$z#Q2oVr6INUsu1HB~X;c33dr`ggX>Q$773zUEHP z$S8j-lf#6HiuxkEErL8G0(h)X<=o<;-z&Jab)DCRTQM~C-HuHohDHG$17l;nn5r|O zTQg4+DnP*`dWx`_sR@c0EzrrX<~lGV6LijpBM~+=J1B=6GqpNN10Ynkw3rTkQOIFV z(ljzkN7#rZib*P#DSn91uXRhaK1P#%146+gM$OoK|4mkTj=;%J8OyI6NeXxJ)VFZM zd=!Ke?US8cG7uD*x4L;TDZzQpl)R2)(1r26X?S<4#4yg)itV~IUY5s%an3hzjsl}P zcs{=HhZ`6ekhbD*(o$1PU1ZEcSEOx5%5wZxV}%#XsZcP4EVrU%mS~e9%y2*4Xh4Xk z{ZP42z=B6^7;Q{xYHBLmk|GM}6NLb`EJq+NciPCE@az{_F<&6edOodMa6o4FY2E-o zi7HjilCvluXAMv0fq%+uOG--e#q{eT;*RrFS;CfPuiUXn7k|Dq zH#gteN)~isi;0>mpM75?CMh2i6VtNzjNMTgva14iUD-3ew~(H}f0`t9G&kAmgK)ab z;G;mDsJ=YroPpS5h>4GfXj0x~i)Dh0fmaWlEIRw8xDN5&GPYB*Xw`HTjubulFbqBM z>a6$Tk3|r`^9|iczG`$n_PRV-o@)xoGZ?`(F*iRuIZ;Rvsx+wg;ycVv^&0%(Fy9=6 zkB@(}HLG>9(h@?@xjwesml^@B59j&_JTE%>-Jd40&>99A)Ol389lQyB-Ti{pk3_%` z4QT*flv^*nEYRjw?-(6DtUa2E+4Ko9HYC!ny_L%IX85@-eZyrOqm~>sUD5AR8f01A zmx7O-r2;fGD35DuYBD+JsXFG@y>4i=Jdxv9N)-v?V7F+lzdBzQ1+gXXDk1BSLHe3B zT781Epdp^|vd5Aa)hdfM5624itj51lh=?=pLmyJEQ#HCgf}Nv7jVZ0*`s4}BlM7E* z&B@R!a?kkoMCexrsi3oSOgNwHn!>Ku4X#vs*{rAd92^|NIF~0%%@q|DK>|tTflrbcq z>SeVvGe^^Q(8c`AB$p;==jbTG1I0~9sYF4?a`IEIP+mqxh6b68^=zH6nb{7ZWSBvG z_|l34zA)C9A+{3+4;)41%6ixWo7yU{zRVv)lE{ZqAmQ%f$y-<%Uy`JUA44)q71qZp z!I5xxVO|b73TYZ78xz~N{QUga)CcV0b8~Ziqit==TURqbH1yH)llG=vW?Pg;9iA91 z=cr^bF)?jURqh?-Jwju{1xHq@7sRzymOCRLzz7jj@qUevAYpvfG(4>SiL8@H{CQA2 zk2H;IFm+iuJM=(B<~$=?hxeRLsvR*ae#HJ~p+9D7-63{caFj_&NIdqI6kZI7`Jh;G zhRe#z@;Nv9-m7-m+MKTDMP$GDNV(`c@T$yW0OS^SmrY%a=2xX=t7L&7pwTLYhlcuE zN?RTt9uhWCB56}SD6MbGccD0jnvJub3wj)HgZhqC(An3|4}AM)qu>3}(NPdoyL)?E z>me2vmT#V$o15l+DPa*2359H)+YP9|yY;IaSC@JcTZE!akjFa+pf81%ArKc`IR*z3P+qEYzNPXtP1NL)_$pNnVwb8*k9YZn(sG%Rt-QMA$`FTKh| zzJukOUWSB)^;B=wF^a4WWCZy5`0UPsY=-c-+l|_t^HiH{lI_D}g-wL;Hw2Y*Xyi;# zrfwi%;0NDElyZxKTmFRZ+U0MuV8hP8C$ma^Jkz0g!pVu-3aQ7c)p@wC7JPrm_YHW^ zV7+`ilbGU;nMQXGyK(Dh{_)$y6ncAnZu=HJpSX)ljaq^edEl!0PlG% z_TeEuxh0c=4rh^WC1oKpXpE$Z^I&xdRWCpofk33Bq!1ws6+ybD_Q^@w>;A~u$%^Oi z`c3Z@2qq(`cD}hzMuksBQ6_tdp=sTQ- ze!Tj7bb3#f>G0-@RfIb_I~yq&1bJcO@7R-}?17V&0V?&S^0={f;7creRlfC|f+~0n zPxOT|FCAh)OzKjzBub?3RPkaQ25rUeH2OuyP(oHQ|J@CpX}Y7J)`l;|>#+m%#3zod zp!9FPVG8n^uyN2Bf}PyX&;DF(MMW$~hMwWNI%Sue(0u8T|1(ftZqO7<;+YL-qpGv5%rG! z+}L2cAPKmq9B2@~)0pjkT_b;CM!pc_U^fiV&kWR2wwFgWczpmX5x#V*97y)JzGK~5464vRNsXk4mb zGH#>u+qNdAkiK0&aoyy&AknrE;4GssAcHOL@>BSN6*RtWu}y(f{ZFfP-5-^N^)KQ6 zPlt3J)f1KIkZ2T{c)N7WkY2$3zAEPsJxMgv|MZ4((V__HFIz!lD~-ih6{eRPl;Yq8 zxCq5{C#V{2GW#8~0#l}NzOiU?JOHlmO8s9$P)F3f!Fp>YWs>x((PmSrSvs%)S1=k1oYD!^huInJ;65$-W2;{jMgS zDa#|;ym=coHJVb!DT3efCB7+)>DhSsUTlzni3YCXn9b6* zSIOI+6c+94XpNsIkGKym;ew!u#_sq4V!PMte&x<`98#9OXbXvaPy{Py*XU&)e`o=s z=@UF0qfJ;WAb=Cp%~qRRkk7d z*P_jQXi!G|_Uv5wSxdx+mIv3S|G-x4IsJ6PdBrDa<9c0a6s_KG+3!WyXe-r)`eH-C zFq0?R>wR89;t3MK={tiiGZ}7rshW*GINqlkqS^yZV`PalN>Z36OaYoN)f5-d9 z4YtCQb+NIaslz|gnb*$$z9@%9mG>~JaMP?8HP|iZx^9i5cR3+X!gIZyy+e2ZOoaXc$GjrxiV({Y zJZv<)&0j&J-f?y0vjQ(x2L^>+#8f<&Gtni{bI-!xQ}%`jUyYl-py;?rN01UPUD7cp zO9TRcn{=#g>@MOf>&&njKhGt5#c^0*O3-d$Rj9Mr5^PI>UUK3BC9? zkz6Jjm3302DLZ&D}*weNmgFYbx)iaM7N>@I7AJj@y zHZe0>`t{4!0tf7wpc28eKTd9oU3Y#yNbOjAw`Qp-`}enJiM{ubO`xFW;NVz6+=GDE zCuiS(^2f8$k{tIkSy}V<`Gn1~JtID^35T~eH=y>zuYKn~U;G@H%B7IP;j@YOHrOLk zGL?FloSRBSB*i-M--I8CUpoyw9jZV?SPn9=j~~xk?s!?t`%!YEg!ABkK1T2icmqUn zciy@2)vJ#Wa%c4SOZ*ZyQ!r%OHb3wQGB7ZJKAV}DnUgbSA?&|x;Y#D$S@POSrtWqX zovP%5*G(J2E|Zd#m6h07JU{VN#O}!fmpmm43kw*G z>HKC9gn0O0YkO(~9IA&PZ_loE}T&%5hyn2X^M6ae6 z=GY%7UTjSKG;;PT(Bajo9@YDbkw3pS!r4;amy(>LS6xMM1~WL4ItdKD28Oaad)Gg% z+yY0Q!go_QAraa?ynI01(Q)(J^J}@ou;O~xJ%7{(=S+ww3ZW@^oqJwh_dgGP!=?T?%UVwY=BrZcw)jP1eYVb% zFiuP*S)lyk$D@r`W9d>b(Npg^;?Vx@UkdX(`AkPGPa;dRdSDQvQTKY zE?iZyU6eN4MdfV_&B16w5%8Jph}%izx}V}SCR#%3@y?i;x|ds{nrJ4=_J zQN=Z^2hWs6mv1q_05N0^PN1sWMom>5KFvVum#GL*ue&=a_eG&g0!niqlydj|x8(8o z?rgBnC0vsxVWw}&YX58HP4tcU6G*L#g$O;P<&ix1Z>Vw`$94u>Uo~rYHyp=1uz@8kZrbj#oPw2vcJP7$}ByjNJx>d&7b`+`W1zrv=R^I{VX_C_ijXB%;o+NEUbi? z#%yRfm+GgLr^iz5)urG#{mDj_{Ir0J^$>8yV|&xoT2rVC%c$CJL7>q*Z+Qv<7g@_t zb%n>Nv;6h5LF4PWn#Q1Yy{dIHg~7&uroq%@!Z&W`<@M0y#peG0kAGdBnvBetD%(@V>?e6b>Keqvq9yp z>D&4Bu?#=h-tMXNgn3&FyrDQJ)b>ZCdDFVF1k*RmvBJs+sRityr);3(q?V##*ls1} zRtEFT?4to7ToHOir31Wlyho8?Jl2LBJZl zK9U!NVI&o{Jz<-^*!fDYhA{l$SB*Shbc?--N&EC}9{3ETUGj^O&z>=_WW4{S>?uqf zpof{mldfT!L^O8WKdp~8N%>vvQ89Aq(D?xr$7em&U+rQHIxz3SGux3i+Pb6@XWBKpAtF`*~2^86(mX(1E?J|p}9SCl#sjh?yL@_;7 zVKFcaj`sNl!5gNVGgZ#N*G#*#8zJhw)^)$X8Gfb18Q1k^n=<#u zuUEx^lG@l#wH!cRhk8MCC<93q6uc@Wa@N+?_BdI5Zg-$CO=HERmhm6w-}CrF%zG&M z)>hrMuPLByb0$o~6@hs#Wp`&`;WXBEeYAmADLVGZ#GJfMQFxt-XK?7K{%urJc9QC4 z46VzLL%I5wpNpf<2gpDfAy<&BRXjMqiOW<|mFBNk>m@7z(bDdeo2yiupM3ilF^t|% z$yeUUw&%7`!?_sG35Ic=?{8K!S2@F%H>PSmT-P_J1-C0S?=3u%rYYMyed>vi8Lvq2 z4c@C-b>nZ|pWwA_%1tt;$_&)~>3v(&;5)XNq{!J+rSeBu2S`jH70}}E;XTcAb6-4# zHmLJfNy9+2x_%4k-MH>gB86g{ ze#h6i^$Blz#9oZyR3XJ>&CRYN-IJY88{fUtP5r$LEz>4+Ei5cNLciNa{AY^);`VM+ z&qa{fBFQ1b&TD^J%x&t{7kOgMWr2T+^&($XlXe_#Ewb?fxS zXHesUp(*DGwn!=Bs{AxJosDOmPp3UtLr?moMm5R1&Jky4XN;^7-lFZ@Q1`FrRR7HQ zXQ?78ba-v1ID+f@{#~+!!l1IlBMY>C3E30`?jmRHj-k!cTKjR4qR0-Dfq?-~p{>j% zC6R=40r#?4R+zZE|MA&h>qEYqwEA-;B=l$~lB_BqChDYqHiek=+6r>`UNtG`Vq#^>( zgt~8}hW!orbMh{ ze{5uxY$c%fzEa`nXc2a~7rfs6Fz=mqw5uEH@yYG)Z|qz(gu36&NRSU2{wb7gcfsY&f2hbH;R7MiQuyzhyDvS=;FnpV_ryu!Or zb{hHMZz-)-@}IV$X?Fz_R>0iF#l}lZ?|C7Su(XZB_4%lgo%Faq z+e9*uc;LYI!?Lon5NgR113ukj*{kgkl-q7TUZ1XO?Gsg_|7`cKgDHU|41!Co7aY~^ zuY0pZr9WjXA$r8%pwsv5e5;CascQ%T{Pd^wd3s`OWpbMGHjIVj4ANQZW1$3le(mJf zT`0BeVHnaD31QnXu$-@jdyX$n|9K4*#(#RD-S)gA-(^k7R#}$wrAtYe@3=s*!*$;a ze*JS$Ivkb+ZT_owAJzMIbEmcUy664R7=5Jy5x1EIuG`0EgwCc?kv2c26c5?%?u3K$ zgX?ihps>H~F4_aWP`|zKRqjVn6li3Zx}N{nSb<{vTe#xig79?f!B%{5%g+iaz^{_pO~6MWbI9LM#wxK-}~?X^Zn_qaae@%ogvZdE*w}4*yOq= zs7XQIGoMon;gsSzMPoT{|Ge7bp zSX-ufywskOg9)gBt<0JDY`6rnQptX%!LqALCD?pnQ{n~u1)_V$Rv$(dO5vG5n-W2K zFrHuRO86jS-kZd}rlwpP_cr)rw0j>juOA&pS$?$|RUw~`(Z1kN7;1lXqbV6+6(pS#)`bmW|z zr79WHpb}{9>noyj%K#i5V0KuVZz&}U0Jep3M;CK4T=BWQJZHB6z$0UL9eaCwK@iY< zAfle84E)WgzXl-f;2ZTRBQq$?i(!^ zR~dkG7rUSY7h7BGBi`m>My*2FWSJ%4MxD22gq=5DHh;M|JAwf0Zl&aHar=tLX68Jv zsO}9CQQ&p}km-QZE)$N8duV9S%#tjc{I8fg&Ni#J*2anqvg97U#9xgBq|d^Z;8qP) z_@D~>;4}v>XD4$QMl_U0KWnLGy>s|}Z7j&dln*GfaY9~e5)5}hTuO(N zh%Lkf9?goMnxKd8LSGHjd#Fgy}zY%N}p7Qm>Uqg?)GgqRnyCVZVt zk9=;)=4h(}uw<}Rodn7te6TA|_`5ou?pqe`jOy4V^Vu~{PuBwW>v$iq{kA!Pi*(=q zd6QDz;v20}%99(@-|ZFvE=r@AG+tq2kQ;e4eX!Rj%FD}p1!zC^tGNjF5vNp2 zK|$o`DE?h>-1KKjinWj%nsslg=+*I#ar+Is>|2Jx<>d#Uf@B3aFa}UBbb`O`?$`%l z&_2M*BY+sfq{&<0Yk?~H!L1P`xt=dnr04TGPK#fPuLfdP)`4Q_X~xEmz~0E@JTMaG%;O+A3p$#=YDx!+0F@0 zLNQqhpqT+l1DFKvH^VT#HP0YX#$svxGT5<^#4%OoJ|vqJ+46D%XP?^4H)o zaPi+8ljZWw)q*aYl7y$*a}xBTCLe%^V*K)V#q3WXK)%IA_`zbx82agu>Wbc<)7%c$ zC?V1J7{I)IcL1={w_U==v-SwYo)*QN&Xc|6rgf?fW&S&27$(SK<2HRAc&^CW-8Vw? z+&U?nm15&LNA)GU-p0xaaH2N<&7u;z*74RbVi2zxMMmh5nVsvLBJ@Z)Bv&T_FgZUn zaA_3-;TW8(tgNArL$?ps$3leq%^*S{a1P(|d3Q@JMTnfD$z1FYDSf#1j`voUmaFsg zC6+@pu`n~U_zxd)c3u{vqGOicD+hs%pwn`?iu(?7z9ocRmwLhnmXlouH#bMU*lQs$ zz#!&%Q6+gx zVm^(912si!#jIDz4pnbpvNEeO!8{@{lHCZ3yw9x)2)N=xu_ySl!3jIhe*JuQtt=P- z`R56e&8VC-L^#rXUc^CHH!0^vEAwTG3MLU3aG@vnk2WTJ;C|xX68J2-fxq|n9T(LA zAT`z5DW`|%FySCczg)w^dw+57-aP>mos=r6uchG`Pxd~m7%Dk>B!XD}xBdaqqA3~< z^0JDT-zlJ5vyk^c6a&`I+1VK!B3T86mJ~p3K6x#*9sG`phv#tX%JTwzFM&mG*fcR* z?FY&JZ)63a`9BORB=U^B6t;}0uNRFq0bwMqO_?tpu8v{n08}T&Y0|HGZatjSn=YlcbCm1?LiA2Xd!^@DI#784 zA=lM%s=55cQg5<)g|$MIIRs#3Vy+xLKn(*xaADSN!N1Op*swf`>MXDuO;MgH$R6ImVs^Zm_#wP}B7 zeUgVq4|ho-d^FiAtuOPsp>2FbxDD?N_b7k-xK59BI&Mr@M1u%==_rfy81*djU|Hdl zoJ2f)kWx12HtK^wa<$2AgY%1vgD5zt*v8{o)V&XXBLQjqjnG5~NRVFUPi8F8y*o_5l0u1N5R*t>-^Mvj>xWVSP628))wO3YP-1%2 zRb?x5p^m>$t3M{0%T}vm5?{WJfsCb?ajM9J<-RRl#0C(S<>lx5Zfq>f$B$*nM~#Dm zF0>s~K6wty;!!tI{R7-h&06o?WiHUK-Q8QJO%ak6@0AGP`N>4Nm#gM2(0e?pg9)9O zR#%_Z19m=FS9a0<$;jnS1kHHR=}#sIfY=B5zpdhQ0GCI?OdZEkI4P!PsawJcM_vz9py3E5e-7m2x}=lv-4R#`=T7*g06RJd7Gi^Rt%{%UV3_?!tQq4r8!+r)$< z{%QQvkIuifnG@XO8O;~h1kgHq3&m}>Iih|y2i?dmLsnpcWZl~Hdq|7*tXB`q(pXKA z$vU=k*sj(04rHK`1;9^@HTt-{p z;IHXD%7os&Ii~PfXc6K;FHUl3a1S%m*4yr^&5x-&qf^MT>)`o0KYxp2ZgbNMl+MS; zZ`CEpCs073)~d8${HxKpMTt({vAAm#3BO?v;ankXX34~hfs|n!pVdiy4wx7_rr!)i zO;8X|SJ;qq@Pf*Fspo16pG?$C$WhPMb{iy+E^u7V@VoB*0_7Ia z9vp2>BWGsx-7FppC=9~pE3aX>32D&aDIjnhoWrxN7inUciz|v45Phl8tfjSeq4I_r zzBoUB5Vp8Xg!RR0_QWHGaY^aI_iPeX_SACxiT$q;h((}nC+I{Z0VNbD;n5nSqoV=RMNWs@r(XjCR*>>K+xkjG-){c| zl`xS(sfrK`v_?I6MSBCSpq`2#74+Zq8^oQo9L;|PT^uAFtJkDtVpmHosDLtsO=hq@ z+Kltp@&sSowN?Y8>$z88D)eQ2`e~k4Ul{`NRl9tU*k(8|XyWO;G{Yu0E(_&qUH&{w zVj)RI(fYe_jL4DvSD1s(X>`(e6|Wez(usIHZcAYj{n}qjbUHg&4^0LjdvR$Vo8kLX zdo#E>Uh3v#d6o6_Cx*D#&ITS6py(UX)kBr7$Na1WcbYQyF^lOifnwohSg~*#@`|@_ z--eR#+k;(F%T@%z%&o~E6ZpW_@rh!uTi;V4{bD zPKXDKC=`whtzL`dvbaZeWI}wOl09`7kB0DqJ?A^P!s-iYtV@KwJcKwo6*rn&Gh}wE z`f`+_J1Zebw9u#peO~PQ`(e$hK9nAm@})g=mgr2+&oIup?{_&pkGJPQ4D$k-Y+A*n z51;`Udk7BOtNBkoKW4=m|Hnyo@8!P)AY^y!E4NGk#LhJt3Hh!BC9CPGS2wNlJW>@b zcvdxuPtk?rqoP1Dc?Otoj5}0rsP`%$I|HQ~H{RWll@wE7=DP-s4Ig*VDnD`6u+}P+ z6n2d^`B1`4weanGcEZ-AYUzN8GGaCGg|4pdwq=F&^b&T{kDeZdn#V0GT*QB0r%C*E zlP=POj#o`#7v=M=3M35d@A3N(`Fm1GhT9VvoztkYA;67Uc|@xyLdto~&(Wn4k{$-b+*+xH#7>XkVN6iCZ(n5EnY z-n`+NM7`dn%)s_Q!GksFs@wmpaohDkpOPjuXM$zFp>AQA{-WO z>`C{T=NZ2Su0nnyEu0$iPakD_(p@?(C+ta?M*#`boWsV#mPrvy&o9M2@1gN)P&HEM z9UCPOezg1h#2KKxHnP-;Nu!+>+Dff$E=nY1CH#^7rD$@m_c?CL*Q}>f!=#a1&W6%c z^nf>?{P7J7v6*z%OOaTpi*^k_claC#^}Fg89+W}OLOig}lPtzU6nLHMbIsvZw_gb0 z!P^yty-x7#Ex(cVaLhBg;~?XvDe|nWy*%1hsnK$sdl}1jw4EE(!-#iVPO?(8d=U6e z`}9%jNr!32L*>aL1e!(RL)ou zdy;sBDium!kG^JOfyg3cIr&csB$k8sPYCpm@ILPmZD-!m``3bk<`n+L9ZDmr(J8`; zRb_^HCQUa34PvsH7-Ns)O2TU5H;5}Z=N02NROd1z%xbhLPPQHzl%2C35Cz=1MlRVc zu*sV-PnIyR0Jdw4X{&IJcLpglNXA_EKE@X9RI&bB*0r zSABdJaP13r3JZ-PQ76PZ>7bG@3o1{g^J2qI9H%pfpb|OsVkdzq<4$rNs#J5)IS|P* zPN-01rXp9_G?o)s=#%Xp#;$FS_ImRq<01mGva(D~Oq45enxJ3$oqqkDsUL=uvp+7pYOVXNi^i<_wL&XB4_Qca zTmM4Sq^Q(^N4Z7?F>sD^dmd=4EJtCK!io|13I>F8+#(j(?LPFWTQ}bj+sB@;Rkm2P zKI)TE8UFxzRDHKL-5e6RsDCae*IA(gKxqb^U*l{pp?MdiGIZ(Z`a;4DL0_1sg&(J^ zoF3%S%Aic0EDE7wnG*Mq)4Kw3SOWvK(nm33&%H>dqqWJL#{fdS9*Cb8!u4MX^F;FN z$jb(kAK7>;M0I)n9ij8n@Ah|=geuuzpgso7h#~^b*WZTfWRy=>JVM`>ge^tH3jGsL z*>Gd8j<2wqIxEgm27AYAYc&ak6;P49N-Ur_G*- zq0LzbvqdNN-ju`Cb|wKbz2&|(I#5VlJ`b`JoMQ@o`?Z4LcS#xDEz%NWXnq6SX#+4e z;R?8D7LIF#raO%j*6?ip+lRu#?}(bGIctU2JyR9Vs?&~!%HSPe-@SVWi~!VQAI!q>RZC3ts#_+s-wds0$?yC= zdt2BOaP1Y)vdGAxg5>l}jh4Z7Rb6{P(CGHKhU=|@Gv$U1)C2>(v z(Mnx1a`GZ&IzH{^9nm;0!CR3`<*RbrOS&0c=9VJY7QG1>mfJ(AVQRTQEnTq2NV(lR z5vbej=$$;DL{;+v-xaESgQ3M_V-*N~oPPf3^P@%rhO#9(v#g9O9N!49l$()pZmO~$ zw1?Cp5g#0Bu_Ek(g1lN@1EZpCjSv{g>6hI}UR_`8`8(0TxlV+Btw0>Bjpvn@uFAL= zBv_4ZHC9DXlLP<=hn;!W`%;LLD%MV!Sc&KNdPjkt>s4-aL9M3-z(WmY@A`>Euqn-D zgTj2W+zM#>xo!ugzfS1PVTVsz2s0}I9oAsI4&uE3OOH;A8fXu{{&f65bEzsbHs0ip zt?W7`1W0iYIe^&Xp?Czb_Y?mWCD9%hbU}+hx;Bu%cSUs~$fn;I8hXu%6C$pTjmOHZ z-0ypm1-YJrE=@?NRjejim{_;_rrP>w&v4O}kh}9dD8?h#hKo}LU82XmF3){a;!(|d z`fR+PB5i|fG(qbvPG4~rj&sXEV>wQe~G zllPG02b?(ka;q^C)8H6}E}jDhdaWDWSm@fp$;W6{01+iXMv%1tk<3bG99`J52D?Z! z47aDoOZXG03_ceT91Q|}oYlxz|1^D7-XBi7!=O1?880RW-B8d+{lX-TvqNSeL}Cz| zX#MAb6rDVhPM}*6AAUqqg8^mWrOWH?h#$}YM^jg9zVt$iPz~o6%ggP%?IL+puyp8xawH*s4fq zjq{tL9FD9}B1et&iEQuunvZ9lehZg|kM`-Y_$sr`VfilzLpjy&e)jopmorcz%awA| z(&BL3RvF8Gl`85Qy+8Ttvk;?GH`Sx4Qs)A}FTa0y>*ZZIwPqW*ege}8y21KDhR)uV zywBb5Hwz^Yp67ObKx7H%K3$<^K{rRymlR=Iw!z?{f;bR4gXGJAJlxcyO?e}s8~6-e zB_t3B=%lLbmu2!=><1&ilnTeYo|03#7L2PC`3L?nKug>=<2v3p)D;kFT+3%~HWg%QM$;K&&JArEIw2d69ZdMF4 zA5i&R1si(#zRFJwwCue=Bvj6AAca`vp;8SoGUD;`FU01>0yiED)-%!1_lGukXL7tT zV<)oYbsXt8X6i4wEr4uxvNzAjV|%N{HD=(7jBMfi^Jv2NFx;cTp`oJ%tAqWd!UgH7 z6vkJ>Ht(!-IWp`G^cHGYI3e@*%8ZNmNdjdVH1pZadwqqbo5n&!P zf`B{_AI~gES;I&9dQVzfY2wR3s6`AM=usKfzg^>ACW#0T=01ba+yn(2x$X5-jeBj? zR>*{|gstpNl;Q$UVnO!Pu0XyiMHdYypoPFF!OXyc((0O-nZk~e_R6BA^bonM9|ztV z=w0Y@Es^%RZ_VH=+M_(!So^hKczLwF7aN0f>rcTgqE=_rQV%fhi+QP%vTUGy_CevF zuEfQCSVAH;0;Ne1zea7DF4d`c&^T}WR9e-{f-l7x&k67@gyfA z5-Sv9a z76$DL75sBHt{E+@dIEsr&eyc(>S}|;k*!D?mF<3fy8M6K?P`Hsq1lhulmZp*y!-1x z4lAMN9lFjwvnTzQUGs3EO3W3Ya+PA z?Z6FvQZzUtWPP&8jGT|_Ku+!CBBH1u1YDQ$U)Z@j~46i;T?Vqb$Xfj8|yk(%EJF0Y(;WepbSpk}#@Lb4GR z;0Cy)XSfF4;o`5SYwb6Fquwu!meH1pN~tq=7=T4ih8r=MOZzK8^rw4zU~xtt){+rN ziXg#P7R&vly8=s5BC;1us@@D(U1|Ox-j8M9ak}qA_(1TWC{CJFHBx%@mG`X_Eowc? zi6pDoC1x=D78)eij6?5`9R8yT=!;!PXzSS;(Uz3br+xOzM_;jUORdJ7`@M`NGE@Hh z95DeP!T*52-ci=DmXMdmET*!uvU6QJl;v7Sddbigt-;mVY};PtoM&rpWk&qY!mSC#G$fycriXqc!%HRY36UxH+haiJ2K>fp?`^ zUrpSB&LU-VZke9;`VfP@nU9kAfdCK8$gl-}xz}$KSurzCl?mjE&xmDGdvrj65~C=Z z5jNacH%?Lmr^^G>ts`1>Lk$fJrrd6S+Y+JL{^xyCj3+NU(v3SpPd%OB^o$VW+^-H& zu+z6u9KAc&W@;rQo4@|eLH`D3iq2-bk8t)led6sM6-!BU6mzj)OioTtWNM^L@|cag z#Bi2CNy#=k{_FmRovQLlZwZ!YU+Q(oa>`W9)4IQ2L;)%ud7FIZ6vtNx`dP;KYxMWe ztsBD6>W~dTEU@0?49R%L2`_H6DvGAu5;lV>I z33+-^mX4!fQ&2ZQ7vt}Dg@KMpX_G9AzgKVJvajeCqd>-o2Yb>*(uKVo3nvEJi%^>$Wv1u|D@n-~Ywd zcgJJh_V1HIGLl4Qk&#Gb&oZMjl9lWk*JWj7^-+|SWG9)0vgaj)k}V_Imz7y&_U8A# z=)S+*&+GU6@jUl)c8K<5$Lcg22Yd(OV<5l4qC#b@K`&ouT0zd%!P`r}si@wL;!c6uX1+m^CZ_sDj7{BYQD3#-_|NQxL6(uFe9*QEO zqVQ3BkpeeMHBlnYVh>A})7N9hmQX~c&o@IXmKH7~;y$>yu((J|ny&HP(jjnszR&hS z&{=+)x=)(YOMhb3Dj=VwrlicC7NxR~oC!7nHdo4zR}7>Jyq7B}3LUOzp=CweUy^T>ozuGf(j-T=H#8Xb4>d`RU9Pwa3|Cxpd5&Zo8kkXTs~ir$=i-JO1t+r?rp?cY$D+^zFGBc8@n9M^!u9 z!>pHkzI8g6ZLvrvpr2U52KVRh-u;kYRwwtY*lV6bTW(`vAgmOr6a0~2IBr0f(IsiW zV4c+aPt5&6=hFG}N7->WU!z&VJ>*FI!=Z~fymJJIA5pt6ZaL901ni}K4@|c26tF7+ z(+TEH&dB%P%;Y4^@#9d`?z9pYr01*XE$wx!8`0by+R-5`C|cLZ>+>5?-LnZL>Bdch zc{KqgUz5<2j`6mYpS7F)7|Q0e5wqI$F+VYk_u z{|OO4*F3(YaDa8ry$LACHw-Q$LdQaVb8_Y}EO>eBH{K2v2Vtm>Ssm$nd@4!bit&#Q zQ=dw+f`}ejpb3V8O@{I`9LMAZM^#VHkK2Bc`D=&GUjSw?4)({NI4aA4fA|J9OF&y&TO!LD`Sha~4u3V^+IlEDp})Wu z-QS&ZxU_}4E8JR{AMkIzt9ZXgXqeyZ$7~8zqbg zPfu~`V7N+;DDNnL#tk$4uVKoT?CcRx=Gbx?zd&hLrO`ezsb*MnHvDOd$LV_LpT-$Z z!U@&qJ`}c7Wt^|N>1W7&stqAg7Od*E<~~{A#Yq`hk6Sv%#%B8{9r_L#D=7siRzaTn zqAjD!(}vv(>@oURiIbo-##(bfpzg|leu?-C|3M9aPh>%B4oV5M zk9;pouyqj@9e)*~zF+RCLUNxSR3nK0DPQ6KFU;cSB~BjwI)(#oHdt|}#KNbI1Z1Ib z@ee&)r0ySjHWarHzW?Sl-1InxcCZ`SLMngVK0|vDjYDl;;%4=2le+W@ifnAE53(v= z_6639AordR^(XiT1GBkT6j%3oY7Tl@L<*N9&~L{Y{KX@;?xV*UTK^EVj{|4_f$1a; zFHIXoDPzS<+YCh$pEKLRyEzehll_%laRklcjk6w=o~y4VW&hi4(#J#EC0ja7Sx?PFU^e^kFsc{!>y zdaB?8vALeQ^YK~dhE?fZewcj1KiW%_1IqqPs4YkGi4VxU_Wd4eImVK^@o{lC zdo6ydtINn#0nF;#HxR_C=`96X8dQNd0e2FCCT;Rxc$yAcDZb7sKs0y+@KU4VHYkZw zr=b9$2xSE}Z5t{tPQzLg`=>GRL@5zC)fJ6pHzWL7~qQ8p8A8c`R0u&q|bi1=qGi= zKsl;gMbpJi_ro0G%4J5XR5;>ZM* zSP&hVe6K@60hnbA|5*eS)RMjeXk$92a`?6A{o2}E!cD+5#Hrb=yZnn%wmoUmO|1+# zW#Vrfx5@fdc=PAZSMM5xay%phu+Ey zsM&%i6M^0+oH+g&Kzw11cgo$1A|oSp&c?*WNji*F0|ihk`)+S@^X$eFviETSfE#%x zWM&5nU@4M4zn7OY)!z`ReEaCO1b{$*v(Ud%7So2JH}twSPMl*r`Z?LLV>aJY$y!#L zg@>+fs9*5B1E}+efLE+v7RQ@3?>J4iF#h}sLTm45kjXe#@aj1+e~Wi@7=s$f$dG3t zqF$#Cz<%iGskO~BhA_jpg7q7(OBZ{EK!wMYl1w?TGIA!AT^(Eg6W&;BQCMKpA3hg7efARrK z3^vr&R)6N&ns@-|A;zQC0aVOh%akPBztR#85fE5@efek>By$1?&U%3IS^8^3l&rAK zp65ah4Im<9cSq|&f%mDg_lhs-s`shZr)t#jm zJufsRf`hbvOVPui0{s2e=8=vBuy&0IF3pvjPj;r((b*Y*l=DQ)o^2`DX$}*Dl8>{a zTj|F(kanmPH zpURY=FF)9>(0}%MVr@1V?K{viD0C^Y*Nk6RPVO*`(y7O&pn!nSEr3gxjUr`|(3z32 zAHEW!EuC2Sqw#9^t8uOK+c0>d;!5rJZ^>^+Vj$U3Q|YVi3xaC0;>8w1t~D1@Qy z?DhZ$2>+DCE>oZ+{Nyxv8#q^t^e)ES!lDX&_2b$0QmfzRww{>g#w}A+UhA2iK@$)N z9gaG@dRR-o9R>EccmiDL9^UdjS-hjd7GG&OSv+rJP`Xwed6j@ZCnqOw6pdU%{9PY$ zHeukf8&jMMuib9M%3zw~#I?7N^^Q(Vbmf^U*1rX|*ZuoTzz48F(M0IM8F;%EoQxSU zKld0Q{Q@>!q+4evSy)(df6UJ6zYwyP_#a5!43Gla`>tv1bQ*zk1=1=~FW;g@jc~@q zu4^4{4e{!R(Aajt`W#?1=h z`?N&@b3LU6oLg$t1d>4n0dNXa;kIZZIL<>rNZM28YA5x4Wo?aF%JTtiQqT(A0ZB=q zH%-b4oI*GZ%UsAvG^~WeJ>!G_obU)V!qx(iiSKLGHVmkqJuE*$e(s>d{>=$NLC3iI zal%P&t?cjLr)uqv>6tI!#u&pZJsq4~bZ^K5Q zb3QjTyeB>Ht{h-tsx8+aZQ*JV;o5{dO1EkrgGP_7<90VWGWkEus&3N6Sw!l;|9MQkV{&3%&_wp zFhr5}zmi!tIS~sW(g0T}slkf)_ROYMe>;1*tyPALb=GFynl9(zJ^&327eLVaUPlWmSm8banTt0{N%zdZK69)r0?501>Au$m>+ynbh{acb;4y& zN&Vi)S~h+a4@?vSue*+-Y+$)(pjy{yWiGX4fz=|CP*xGwNfP{Bi(_EDad*#CS)8kF z#EWyaseJ})!~AK&&phc}`W~CpxDul}^o+rPtuDvgJ1$!515?*UH5EJz3jCWN~wo|(_ zds5x?N_)?f(++9ME}wF-K~&f3#?CiE_LGaPygZ|!E!;Mq)^z`B)#2+jq&(o2=JPNE zY9jKIy9!7f8QIi*I6K)Y{79)D{EY8ZN!W3>dc;&HgWVvLp~5O zbN%eiyPH&H%iCh>rv|7j)ZD1&zR(gkTe%mi=N#xAO9_3|-$-%V|ESfazdoAN-=Dyl zGf0&;&?~wW-!HkjINs16et?W}Hv96Bb`~%x)RW`aT|69Iwv>vgU(``Ld!)I0NP4|c zOLwP#a||)%uEyB-XN51{65B8N3A~)j@joEM_C|4JH<>0ftbSbjR(?4&g)2%^#eXzB zZOov#zteeUKCOl8WFVA5U%!5xOn+`>)nwYms_Mhpy@x3jmJvd3nXj4dK6`FG&b_bC zRIFACC3DzQ%qs1`Dbc*{5>a;qRReIoftxD- zw!de#rgzpsHg#O8w$kZn8C|;I=0J!~qpL~}lh*HSoE2iN@l)FUcQ?5&N^B_sQ!pvA zY(Sd1#$PM!c*%VA{GL3d7AlI8)of>0M`l*TZIKJ_dLzD%F4MX-Gkq3xX9x+nV5F@9 z>B>-(n&pRo=+wZe1vC>u#R&Zt<}yq|<&7473{A;Q+WPwXep4pXtDsi)C@K!}v_@-O zX^?gMX}$68^pP2B+@#*hWSEg)(pam0%LagC0WwMoZYV~4sYr(zip!)ELy6lPH4EHtY9`$1ssIA7_y*Pvm zBj?}Z*>^c9cwOD{M9oLf7@5`l&qXHxoh$aA67a}>h zWRjjCWG)_g&Res-1)Myc*dWG?c$@&$h9U47@a>7#1eQW@v$BL z5i|9~EBNX_0E;qKKY3g2_Hnrgmh0+Bl3LVb@1Ml&VG%Dbo)w6UP}8+$Ys>XLP0Mg4 zfY_Li^Hg=z%1(m&o*n!5V43giH>DFM*Op7l%S)O>EAKDv2sPYHDo#@+o%+b``Bp1K zFy@9>WLr_C5@NRT8=Jk;Iwx znS$0_bdS()d~B`WN+5IWRynY*Ts2XzRnkDK`2}g|pT<8;6(u&*SaQD(VD3X+S2~ysi@8)g6k0SgI^D|+#Nr2RIdJVyX8M^) z*PYOgTJ-c3>1*6Mr>z?A)s$3=iR{7v(K%fu4iTePKi2g-qrE@p6mveyjPF3~Q4*5C z=g+m?-)?^#H~+K_V~kC#>J?%uNyY}MJ3CJgmRNUwH4*x-HP?4U*y`IzlIZ=xmSTQR z&Riqu7!ijDPmUM0FGRS8_M(KU4AiR(!ql`Et)iu1()W0Fjxvinnt`$?NL>GdI3@Zb zqA%t;*AMs`5L&h!Y+lMLYn*YbaE`XU(sQ&-d-3b#R!H7h3B2m*(V3KE{*I=ytTW@MR!{U{8X0EW z{phf(m2+aoe9bKBBcr48Mm3s;%?6QeQ4fncd}EFogOHcPKA*kOUoej-uo!Yirkg4T ziQL?}{1BV10&(T%caE9KgyJ!cLahu88f&rGjZXj^0ejV}^3x29-2hO#Fr;o`!v5q* z(OhT7n{6?Nk>61~v4d`tjIGi`92}NunZJ6M0#v2rl2M-P_KjaCgiplrAc{a?kXKVn zlxTF5+HgtHYPIg1GVWDqQSMD_Q*?_(!w~Lax$YO{p$%Zvr-B;H+J8BX^lND!?$DU1 zpHWv0xI^X}m+K)s(!1u?IK_w2@?nB~ER$dCV(T#*pq#Cg( zdShNytX9mMVDu6T)&-75K`(d9%??cy`7>u7lfK4R#~h=2l#5Wg%-0%Sy0%xO!`&Zt z7)uygje)|klCG%ZSiz4UXnkQ3uT2{$5ly1$Nxb%S76M(SI<%M2?CynbOGF^ZPd463 zP%?(>Tz8%AiwB)9uZ{7qrgiOR&6_JhE_1gVHh>0$YYvqi?2InYGu4EZl-<|%y+?j74=Lv?=-Qv@tW=1jB9qGp; zjIfDeJwlUp9&%Y-@2N&gA1Pc`Jejp)>J!mE+1HO72{Y6EbTlSjnBX_Eep+NDVG;}wj9&ne-}lMA>#@` zdu)tbLcVaW^fPdf$0D9H@Nr&{m8BobPo^g08qskp(~cENQjr(8>C&T+(%DOi-d#jS zg1pAH-#ElDIIP(=lg%?Z>~0r>K6j*7yofQ^lsa|QpAMZn=+I9B_;~ptB~7E`TgL`b zSRi&mwnrG!7oafa5yiXr$=RpBKNFCTA#|O4!=If;>!f+pCT|zol?y)jQRR9)+sgdM zDmH^r>UG%bkHgtW+1!ir^V7)sSFe78IPnk}JIp897oM%1%3* z@{I{&hhaRhznI3m8O3h{%{F4`y;fHG_WR*%xvH)V?KJC7Z%A_7Opq#l`QJ0!gNYq= z#Wt8Mf36v|d@#o`=8o6Dx&l%x=D7|5mQUPA$#W^RjYhprh&=2 z_2f<-I_0P!Cd;uG^IF5wyw?A9a~j2Sc&w%A&y^5O%16r$ zLbhhPC$F$^f_YWGHT-x@4fp*6i|aBjTI`#Y{ra0!zNf=^*Bz`PKemn7qvVO#&|B58Lto4i4r7L2 zoPL+H<|I?BtxN^ia!LVO=2lVe%G$C@I_1U9D;|1&&P5&%SK7R%wX+wiRsWqK{xE!w z?_b?=_UJpW^*lD6T&FlEa`c##a~aN%pIRMSMG}s;IUr+-@QHQf&V1yfn(C`yxO4^> zhNRXiZp1}wZB883t*jF+qHoaW1M5?V!H0dV)y4mbR*bX!%ZsK-*F+P2zb~2JwSFa{ zFW=7Sgq(T$ZLT<#=+IE(f?i^e8j`K&KSGrr8CNn2oG|Ei7^i63+=}md>eQXy9f9+H z8m+J`%$w?=xo#CmrLx}`o&J5s$p>Bk?oPd!MsKQ3bl8Yjcq)t8-u>{kH@_N+%4`U} z6!J9*(gpk9DY}!@J>6_ku4n)0Y)n(vB)&fDwUo*H14iNB7I1Eq^$DIHVoC(t#nUU?iKd_9=_a{&RC=| zVO?}5w1CB0zKF6 z`>Sk!LVbRNS^k|Y7o<#5SU5UYyE?Z?#l@PeVa2^6&xMqE^3pVyUg5ZV4O3bNBF5WS zZU}4lfvsczQV>mRiHFmhHoJJ5p4rQQ{kw3h3Rf{}W2f5>R|c>G1`c1I!_;1tFGPbv zgHSBEl#!(hZm!q)1%8?{ZBBf4`<5n@C0Z`8CpC)ro}C-wJErnB`y#X;lJA*k-cgrI~05 zMY$|)dQ!rG06B$`(ZT3GVh^{c1A-ldJDWwL7VO20VLAmiYymDZ_TAMkUls>_N)<*2 z!C9BMhh>P_s+f~dKJoe_EykX9QoTLBrU_hX9$9Ki>b=kXf*NlGzfat= z1^Vgq^misO*GmV)jEUC73@wZGaHfXO>%PS z7>kRoz9(P?uOH0d<31s6eVaU;8|>jKX&SqHywS&}W$yN#Y1d_sc}QQf1I$yoP!wNj=L?CWvV!z|3w&X2i6XU$}g)wP#JfU9*pV=Cw46 zAH(o2tNpA^Xrpa0l>MUbrLlt(vIaK}Y_yspw2Hn>L2ss(je^3{Z&n@CD~`J@Gzz!5 zI;ZllN}4~vA+!?lu5&!-Zs^_UyK#5l+CaBlSd9!XkjRr#_T za9u2UqnkW!lW)B7SxVlxc z;~UqDiVCNRv%Ewr_xqPqX?6bDvyR4tit!c~Z*Uuq1&)=DO^wlwE04R6r;qnF{TTB1 z9bh#^)^7hZ_NUr6tA>r8NwdEV7Ws@!F81uC`V%*~MA1uBTc7h9?!XeU` zRbxW=8EkG|395&-;1)Z!_9b;}6HN(A$;A?}w5Z68i>hA~dL&P*E>aYwCF`f+!mbST zeY3gi#6Le+&TReE?sjlTg0!ZT_o}wcmvzl>9}(RD$U0nz%ENsq%op=7OwNHfFy#y9 z{cugU-Vv;tTgZ)8{kPM1(b(=dLqUO`nlaKpG&<_(M9(H^6-SIYn;W0S`L3OC1LmTK zJutxy3c&>5U|aQI;s){zEm}jzgF>^ABXv0*OS`Q3E?)(TkyduFWKs++D0&uxTr^wV zip@0=6--cF>!na)arj$f>|)P95%reaKM^&UX<#xJ-I;YM(=*sO3;V)v-vHrbvc?#K zZ#q9DQ>tiSpRSB$Zk1!1k)wy}zKOMHp#V!1d=<$(wbqh1Wt`q4vVVb{)oA9)***?? z`pf>F#=~kkI;zA)9%U~T8Zr~dxP)^ro`PExTLv1Ab}^=_9XBS;#N*=GjwxN$e<_dn4hZXL zZ#8&+lPWPT<32~^x8?HY3#Me&w^f&-;hAXEhx`a@yUpcqy1G-I7oe)Tx6+NQ1&hyQ z#Y^aAX-%)V7xbcvu}8)CuK*Oi5w`yc>wUv`y?28fMz>#cGu)v~xXpU^RJHu2AmTE+ z-};5heD{qZEShoO$$G%2T|5_;q1DW7;ZruBt#9vhiihdd<;hvk z3qFqz?T}n@7_pzH$>)kxcAI?JY{5FS@80CXJ$j$@{7SqM@z0k!<<|#=DYfTWnl{T4 zDjtq=75L|Fkp^grd#xDly&bkaG3(34=qy-_DNJ9jSo=k60`VA+Q9zGgtD7#pF}SB*lGbLlQ( zzrRJ_I18r8Y&qtPgV596n-3?vMqM&!JV(2DdxYlm(TrjeuU>z=qN|f{=D0RD3-yO@ zfcx!fm9}f(Fw4?{$1j@4atH{Qt=fj;AIo%!j^%*bmR7}^>gpRL$7}{Nv=uj;N2-7D z=n0iCwTQcqy~L`H)%;+b%Y2vr^GvO*shdZ2 zdzG$SiR>+?1M*`co<{x-s5}1z)GYtMfO?w%wiva*zI`yl+CKVf$stNT zi%F~FLx{`fjjh%*PfT^xw#N9WWRV7wV<(ibpwn~Ce7rui^{HR6^T6=(ubiEooj6w; z&SkV|&9UD18iviqNKAd(L^JdD?=Qj;G-PAxdn>#Hz_)>TXKg(CatPz~aKY(r-o6qI zw#o0scF~W^8k?Kb^~<`3KZ7z~WOnBHDfPh_W-v54SN@z=mt!LQYiwFgWlFLpMVD}X znAeH%+H?F}Ljz5gxT!S2PHf)}@$V3Gmboj8v{_B~zt9{Uaki1^T49xiS7?6rmpB(# zP(V#npE93cu0thO$a-m_lIX}03row|+J~GDz;;REud2F3Q<~Ayp=x&1-o9-zK{ebk z2%`EeA~2&D)c)D1bem=IVGXIretiaKx-2KzGwf2zL&JmsICh76Fwus{LkzMvmraiv zD3xiMs;BhH=NBkk;5nun8lM~x>FQQ=k80{D$Qo3ex1(riz7pqpI2 z75r1V{bw#Zz7RpfVFSgo*m95VL1a&|DH<6ry=rT#6-?Ku7(9ji9xh!NO^CkiMABP4 zNcYkHM;RMUe!l(IY$rLhijs25G&-M(GG7nlsh-k^mfp4E{$iV6iBLW0nuxj z)rzgThjS%!*;^+WB_P=#?`5dv;$o4ZMbRYKsAAx?B1)MEyXw~NIS!=loqD-V_bO*4 zWR$F7U0LUC_%wu4rp}5tDTnt=ICt zVA0Ie-oObtsyXnK>^dB@jN3A+`#0E!m*)R5x4%E2kAcdh)gkXPS^=vU3U9tmue8^l zoQ#T#3r1hi#zrlh0{Uh9D}J3PO|Ku(0g`9c*+3FnSH)!$sv6R`R0FuQ2$ zRh~vwT7H6tX3}7PuoV9c76S^%y{YhKBNS_Gza`5HhDe=M@EG)voYeqfm#8bz?4?sQ_K|Z}^*h zSHCV&ntOWQ;|_hheulxDueak=I9d^**x=m#dX>yupcsUUfQsN01i#hrLhlH$YG zoUv2q)qM{IUM+kCMFm>Yk4o*~``P?WG1b$0%dHz5^vlzT)Z|k8`bQe(S_NwXI?8|aLHwQgw(ZfHI4MYeMf~)!U~zsjt3)OpWvEK`(8450}0a$dr*rx zn)bcmgx!2$-pZ#Tr0vo%Ur>u)2=82xHwNQ|OU7Uc4Q#P9 zE@ce28+E^8ExfC69;mga=~ogN{F*aQn@#cs%enyl=WguQU{HsF3_(Kk~Zo zLkMRdQpYV)bg%grPv7Ec4pIllYO41cO|$;|&$$h`wXLl>#8s=h=EE=D!*FWL*N6>W zvdGi9J5+!DU;)nl!ow1Fh_r#J9rvmN3?sRptjs;3ShCF(o4V#3w&LfIJd5s+6y>)c zhaT@2%p1t{ZMa`(Blp2NLUrB$MWngU=QDhv^&cuiqYt)=Y&#@l-Z>MjECX3cyoZVE(FqS@6zfbi7OLW8eBG=7I`xV-1a=;Z47fZzwF^nq7>f zbN+Z{Q`4>a>V|I!B6np(&1K>dG@n2U3#&xbBa~SCNWl+%0ZZj#tve}P&bjw2TBojb z*mP*)r~5rOL!)G^Ce5az>xa*gn_DyCeEf1)wHHp_XaZ{nE2ka|zqGM@Qy)qwqw^JH zgaqg!c|9o6Y1@7VDYrIyT%{VR8=M|-zZTLS@~meeC&zKw6Kcu-e7yNG#|)Qs)A9vz$;G%3 zd9O!_br4pRvpfas`W8~0YNJ=cN#A%-nd%Cmf6$M;C@oMY!`wEOUH*n&mk>)O$>KszjMIlaU_($ zLbvnEwM!v0?iu z+)5t&-jO{zz&##Svrblg7EYHS?8uPht#^fvazDvBpR5^(lBA_ZIF1c9>&K}Tg>%1F zX&nhj*#C1}H;?R*44-Vx0_KC*_ogs(0c`dW=R|e4*#}U`?!BDl7H`x|z zIZSl+JB^z0>-$Crmo1!V#8s$vnE^^fFJeD@AFsI4CN3eA4#~RDEQ4Qs17v#?-tf_q zS`;9?76ryw&wh+Kxb6C1Xg7s7V%8X|vkZo_Y4@$cesY$nH_IT0=5rz4aFm*A469&v z{0Vu^;i%A4=Yp^Gg?3sOVP|`Un2XWiHOqzOHeMr{dNnOlS0J??6yfGG@Uux?eLNYV zEXyaJ(Bma)))EdAGSHJ}Na2}#a_!IGHiWePfbz`J;2LGYW0)3Jo@;cWg{)-*0H6l%(hHl~pMJG=v&_Rmn-sS7L7H*E z{yHR<#2PFL7zsbHt-jNETf0`9cAVq`V*d7s5^I>txp*F7VH)hj7R)f|PtI~y@Dl&Z zAqrIycZl3h7xG7ETzV_LkiYzVz)tS+w}S}}BgZPxKB6N+#m-~l(3KV?8++P49Hh^* zL`tb~=TrG@p3y%C6c0P`)b`hpFFvQ`eZDSNgz|NIj2=eGb`uJ(yc@xiuzyF*n5GR9<7Py;*P@l1ERHEl}-B z17&M!_jJmUC3dAhhU^n2a>|#2LBvQXwx%SYRQWI`&L+3#Q(0a-cT&xflo0V*HVmq% z;!*lzKOx_cPKCkFo?y)ARD+1plrzM#g5xrFoI$%2Fl#my!qMPt3}=~xUtqzA|&?W?bW&HZ7T8? z=EFX>sDDy)Jf1eiZ+tZIu;^`V>{ciAQvqiWyt%>fwK01ZoPx70w6sSHPdEcsFM)uQ z-e7wBS;x9f+}3dtPLP^rAt_%Gz>wkTLZvBDdR>L5o&aXupUKwfSpQ8r&tF~NdxY@J zPJ^Z29Y(~9jPi8G@(6t%(f1s34JR}#{qNt}D_RZPwhxCF$Jb);&ose(x=b|m{>U6L z8Lq9xzuLOOgQqsCw(tpq-K1RPL;P3>_st62Wxu<~7ZKmvv{4 ziqM`099g9%Tbz0HF1wP>a&X&UF)xN(0y=gB0U2b}@TW(u_z{o0#iUtzwu)N3h>v^- z0trL_VqBvA3ihwrV$_XT$q%j#d_i)xU#Qp%ivo$=)ACGNgda4a#D1+wOErMy22+eV6F7Z~`e=ZAeWi+lTFF{31`c%$aj}08W!Wk%SA3gHn*gfU;S8uNds5px- zNKl;p=jPxlLWfV{K|mlBD@@RjaM7D;DK`z$f-Unddy#rBONlOGz_zueW#X@Hy+_A> z`Egv+te}^ipr6a5<9vJmzWK7JX^(g-sN{2^H7Yi?Y3VpvBFJ@FGbG-^3gO)m!fRvB zMdRUaJv>-~D+nI$hQx!#|Gyt0Mp$K2EWF}EceJ7gq`CndI?j>w*SxO_!Ca1>ri3XX zNUZ%FjgHMH^Hv48NL1m>#6!WON7jWtW*j>BY0|A8XsNXMu;JBwB^Bz4U{5`_qF^=e zoD)n;IKRLci}%jq9XBR8&2z{@CNjbIaTG!p`F8Nvk7d{=u=Y^E(f{~hApssM2D9}5 zps8W68$qxdwAj_`Pv@cwQQrC4YsO!m1~_4KxRc{?8NMWG7~VUF-(K)y_D}_!jZJvm zg*k?pnE1gu074#c0bJP(*bc`oMn8p@nM;9OGDo7Nqtk;%PSDaKfLfggbcYcZW~f4a z9;9y}dz~B)7#*l>asTO;lr3n94CsL_KNvYEw=s#k+~Dp9pbliH78di+4h|0R5Zj}r zZeWU3?(J^Pkwm?|1($Cn=&Hc8?e$0Co$N|P8WS8vHcWr$&YN$vq=b++gkl|XYzH4p zcqP~;1d9QXZ$pmo$LM8Hhp}tK=Lhnul zQeRMZ469+tK1ZWjRe7pt(L(hoR-uXTumH@-rD@!pm}4jGML_GX#)-OUA?14XpvQdq z)_i3pP%r?%1#gIbN6656Dk?26`Ap|NF15UP@d9L>MWoMjl)VDodLKXuq$o#1jUNH_ z1!y7=laohD0*)J=!T6Ei3{)Y52S1LOdzicJY_6D;o7p+iwaDVc{ujHU96sc=HsB}< zIk$w&`$cGQle5LnpMSp|zFjB1%V$6Q3B+^j6A*X?eB7PgWCN%kTmXi4f|u_m?PwEB zS}NP#^mm*$hIR4wR=Dz=`K2@L7Vok(snImz=(6IqaHfk0bAt3_)gd`9kL0t2V0sBX#H&ReDMz&td(R6N`9K3qp`!J{VO zt%RI}nX+;qpc&GZZajYcxY}AJhCc-W=Glbr*Vj)v?>=D~PgVJ|Fo$04e#(Snl$75i zl%;i{i7OIGZ{q=}SDsyCFjS2>uwjCF!r-wD8dEH=W7sGFQI(uu0fPGQ5_QYUk$I=E z`I})ItLL9{^6=b@f{gp?Nf5&|-vGu~wxu&17=ZOqn`7a>JG1r0dv~9U&%FrJ?4ofF ztAdBb;j#8^n3vKy#g@0AFe<&XjD#=Rz=Ogt?`mP|C*5(qwNO3J4DTa+!hFle$0u7R zvS3sj?GRZ{MZTkVawEFC zApC#b1REi&bWsm;gNbEVQP;3TiL0I}Oad@iYS`6Udb@fYWi zlhlrK8yY@@WFKM~c<7I}T|tbx@C8*kKM|p0djb@h+|HdjGmLvI`a{Yhe@lHRYosCI z;az(d->1cM)tUg^p?zqm68HYYhKqSI*4zs?ON&{|myEoZ%u`PE3F$%(t;~<~bn>>6 zTY}{Rl;u{Y^-G=J-zl1km-Gk}RpM~;r!jbtq#Vfwo96meY!{e5IJ5n?JxTg7p16r> zoa`OrhzFZPY?TNG!J96v_Omb=ySKX$_KNmt*<9&$_XPyb5CxOOIvkBcn{Jhj>C3GW zz$(gj`qd81lpu{TqN$b55uPr%8mjey9zHWvIfN2wvW;9B5gYrF4+)uPu&1&OD-E<1 zxoa#qNbV&+chpFG`%)pzQrvNj zxaca*&b5J2xDu=}I$VI6MfSFhG~rc}$X3OGCscM|tZ2tof!PKGy5h_T=_5o$b=B1( z%$@LtI2L$)v&Rgd?iowN~=le{8l9_S5t*t`WS$0y~d%iLM08n&+W$Po$$uossENN2&eb(&1ct z_PGFOae}>5|J+_)*i%eQb41is`~AN1?;Svbt<6AF(@4^R1N@qBkRE#@#=A@9c4y{cv(W z^%?)~o_su{-)wsnpOGrVH8d(~#bYFcouJa1e;DfYa)ig*yx*FF;Q_aq;5P~P8;k1& zDx^Qx)6gECHX*_#r;naV#8;}z1fLK*_hI=lv6g#w9x@9kD2phQlSd08zDNGk1j~(( z@^?L$o3je9{8ZS^czY9`V=N$l=K!vww`)@WOF4x`FR%Y?Hu#+8{3}cB)Y`bkx&Qoy z@Z!y(fcAXXO3Is@c=%-3#Bg^BUIhy4=;&w`Yl}zy0tpQn^TPjBuP8=E@-7h!%rz>7 zE3Duv@4KzGxi}|d8;^_q>O;-Vjn41{-$Ps%ftRNE_wU@ias@B{DBdTH|M^?8SMVYKOOGS5hMtZCS0`F6K&De_^ z|LgZyBqb%!ovSX^RR5DcKg!8}%&ZIh4$QIdQ7RKtQ%$bOpQeAx;Npkq4jf{9Y*YgH zo9Kbkm2hxFf4&Jxy%EC?LiIRW_Deq2c5MXbYLj>j);oj_>%GwB*ZZbhWF4uM(P1JruK|OiJkEZ~7-(hA_e!g0~vuVr&X8l*0M!V!eVOF_V%jT$|ZL zYIdqVQ=27NyDvw;xB?IFpZU;YU^xkpR!%~IwDuJDyg^h#s|M*49BJ#hU+Ct@(d85x z-m7X41hvioTP7hAdfZPB3ltFlIh~v++yE%szOq;_;ucZ}rN_5_kc|x`9zOjgZ}I3W zOJb-K_*%GqKcV!zHZ}gnWtuxZ`%uXnys&nsU`7cn7?J&eNHy&KROZP7;?JH>b_OJh zxR+j$#vI`f@>XcaKiK@Sle(^o6z?rPtWyheDHLk<@r(C^U{d5^Cwd>8>E`_-Jtg|y zebPtWAwSn<8I`q1m2WR58C3GWU84SJ?E*NEmxxYz&`C_!d_c}C((UvH9GsB9_+YeQ z6lI-XV6g3+%<22jjmchI2aL>@Tr6GENp2OD6r4-qBZOBl2sa zV?XsjL!dEspOKe8WNR(l4ym<*!2Ql6|JVC6HCx9stm73G{rQjoXJGV7>4So*hq~sQ zcrN^>bGH?+Q80C^_Y+QrbU$Av&rvo;mS)b`Ni`f(eC~nWW-Ty_`syxGKx~D}z~a&Y zdhbPIEC!{1ddd1&MigI0_>`r?e9CNRHzvT|W{5^hh|Jb=$Cy7*h2{PCoa&AFK8)pa zdJz$rbd`6hO_-u3NNT?*a3_b6hLt=Dv+vCytL`|0I zp}PD{zgzbA)1NIboV8c6(IkHB>=ygYa?giEcd~BO{o=W^BPmJcQ)iop#(ADK8n|U|oG6hq3DNGB2Fx$1MiGROS|Jf%2 zA0BK!yIlid#h;fbo_l@!7Ivg_WLzxnA}gzG{MGAw`J^#y1*G$>-2Z26wO9b(_m)cq zeaDAuO@ch%{Kyj*8BkqnE{iw~3! z0f2X0Qd=lCi=kQ)#%0!3@b?5l!#+MfQ5S>Pf29@v`mPvsyuVVqVT;XZZ+C!RFum9^ zL}8=TcBt@vTI1}AT+Ix>Xr3VZ=)3eRnR7x#Nc`o5f3N+ zv4Oou1PziNQtBC6!3dy0l9Ox8cUE(~y}9!H3-5g?uq;01EXv0h4hWB?haWZi!*$_6 zNWT2*JT%9huSY-hgtcL2|a3W|z~%b(ye z*d9!hoxaRp;Y5--)oBL`$sdV7zj*NqvNuRnzWE!46*+Zti|jB#N9ud`61eDtvfU@+ zXiQ9{BO>*RvpJm-j!?{s$!#~7&aU3v=%Tsb=6ND_co)DG-*SJHQ%Lt;OVi^_>M7T| zKjxnVn1%U@EfcM%&8jD}HFsNPdQDwt?~KNc#P2V|=f4ffT!c3Q51lwBmO9T%xG!HY zAI~$rmu~@_198`Mh3B4-(7KqfE%TfxtDu1f4@0Q+i3MBr#yxbZY*@$#O zzTn-n+}y6lO+eqst<}o{d*x0=@#Th>fK`L_H2V?6n_8A$2ydtWw@YlO7Sc$D=Zrpk zopQFCZ2NVpk)(4 zl~mVx6*Bt4Ev#KkK1$$Xxa|C3*7*-Ru*}ctg>Ns90Q|<2|L1sg!t79G8TZ?_+);je zO9=sun{`ab7)5?JB%d>?@H%zh%A7&en8f)T=M}ZsW&p0uhMu!YH^M&}4chV{c-rF;$H5fbPM{6+R2K=7Y z)j4yhHaP|2Haa?k(|YtaB@ejCAB`DkfZ}8Td{aV}Weuzs8ij)7gOt`FzDi5dbPDgY zEzsKBF+qRs7|fA$lN$P3Vlt?!Di#PU8$y<8WNt1^BgFz3J;r{6=f24S-{uM93rUa6 zsY9BREkL3p;AH$~U+oE+*c18ar+`B zA58W0i#Ss{oD%T2IGwmSS6|jXx-=IPkt@%dtXm}R2=qPN)g(WP{fS&hM8w3Vyuy** zN1Ho?@aN88`vd}(cRfH@$X;HN2FVtb2u&G&0g^w1Jfn)aLkci5YRv;~BfwQtjy%)< z!=z(^b75AiF?Q+@VXAmFzRD^4SJ27KuPek+tnNu{@Knq&_d@iPU+=I+zuX6Znw-sZ z5gGpLV3hw1I-NLfidx^(`PSbG3F9Rx4|_j+S_pbXV&%9Wfs+~R8RU;0f)9F?Hr4m+ zdvB$37ZgVWMU0@m{@FWKc4?O5V{~)<+;fuNa8W3j6#jL)$u0cgzoW)5hvYiC2&q=H z^j4k>`~0%DjWG=>IOWc*AKG5wrX(wM%kJkYYknhlTew=Z@wv>G;yX05%SV}M^YloQ zT{SWJ*XZ?wZ@4HbQ8g{0`SjJ7aVbpgQnrWL+U^?dY~Ky*B6@ReARFezd`p=@`$SmJ z!EAraa3^b*Nw)s3S?v@c25H2r0+||I4s6YjFuALw+Y`Hedy+2C-Jq?AN3ABEjw8Z{ zu116s=Vd-W6ZWy*Va^~9PiH>^4p_r*D zp(h!(^D%|5)-o|(>}JgX93MuQcS!%UB?4(bU$_1a%+@>BUt!WRndP1BbcEV@EGF@7 zroali`o{$AYVoSh3I0WXNy$#A6FzzHMO4)2m&Em6h^_Ym*|cjq1Re7|mIhwL1&lH8 zo|EykA8N4oT1gSR0XED_!yC#VWExc+k7NrpBRHecj%P`ty3V zdNDGB1|ra^MR01F85noWyNic}7P|`ufj6dN{mpe$`1X^ddJfDESx8It_g@Z46nx%i zi_a(@MqK7z`szLp2^bT%8+XBZ(S5(2Ed!mASf}=)cn|Qq8{_ z63q_uv5=U_F4z8yvqy~fI}o@?tE?iq#pKTgoN^%ZCNF!iTtWla+MTdf#^FnmEy$@h z^#eN8XT@ssZTzBlt6k#0A37l)z}9H|Ze;x?=#2Q|6idnE{%Vq;mC_9B@%6X25bMnq zTly^_XpOhnW&wR@dd?`1`dhf{9{cj$s@8tO&dP?DK z)kVU(VN%HVhjou7K73j7{jSAD-%{o_HWrpU87=8&P2#C$N!oxyhW#%dD(@Ur_dL=% z88Wkey&%?Af82B$UbfJ`6SMTBC~`W4_>Rw7v6@9wPp?1w*P~Z&?^^fopg$Sw_zd>( ze1{|M`6m*T)$l(((_rA?OMqM?Gu?Q_X{t9uAKY`&clRNih*dP}K%*9y$>IMN4lGS1 zRN$&D1-2g9dO}7sDZeoT6nt8(T}Wy$iS6>^fBKjM3+9--MmyDO;p-l-br19Zy9dnL z@{h_vWF&~H@x}q2X#@EO#hy(@A7*$oBXs`9r}aP9hMW9YnJRJ(7vDcPH~*vdb*}NA z|LNWaItA}qE2CZs*G+P>z3ZIXd;IAC90p%mH>2mwJB0AAA2(fKWPRUmIq(}UaD3;v z|H}nFP`wnXD8KT5xyOXrX`*Ute%BKd`9OeSNZ#%lz}xq|4Z!oEP>yjb=W>@WaE;8l z9Xz(51dIs%f{+xcc;*gd`+U#;hywU}YRcg^nU#dO2z+^V;t&6;;}*8N`<5Ux*}a5+ z4j#PnCk*!g2kwF@i|c5YP1y;jw@ns@%aE(DuSa4B?>sKK|J%u1p#?}E((GLjxhNgy zWe^Hfgn5uXnE`XR3558e@@$K(B=V+4e#f)lzv>@=C-~PSx<*CcA z55p1lMGW4B#L!0bH}`UOx&B3}o_Qf+MBz z^6?AO7pM=j@xcj!y>^fC)Q_55XGm0~wShPydqnWwZb!)pwPrioNIa%o$av z&%gLwNRD~yi1X-0C3A3A$XsoJ#ygfAVg#jm(ln>(g`AQyw=f2 zpAwLS8{oB53_DS~)OVx4B!`3*o;BG+db^;$d2#*2g`H&ExQ5M>Yj=f@zdD9S5pUn~ z`2ez=%n^|70AMLv?;O7(e>6-Z5{vy~DoHenf03`#`)+OGw=aPOu@wZ^U_~_ij=*h( z?YF_}ztQc>Y}puy5C)P-%1l48smBfY9hc$!pcy#kw>3Wyc0cri!+cCrLEF);_yq*a z7!H4>_d`H@4+kH%=1T_42HNl-jR27(e9f}dRfw!Yo;LslXZKC)*B`If)SeyIi)Nzg zCMw2A>ZP^#y29W%=0};zY>t9K1}60;FO7N8-Um#FKX8Byj+0Mpo#|ENH;ozUtS^z4 z5Ias0ePVO6-3bU@jtF8>lMI)0_f$wnW#$NAi2neqpgud^O@T6-I0^zUak0%woX278 zgpW^#08~P;Ykv1{Cd33L&RStEjPW#y#})p&LHP9`z)LAW+}A;DGKd)rQf@9E_z$v4 zg&>8IND#WRXLc0t8)FiT_%Dg&Lm~~5(E9BI)VOMK{72bQX0i(aiNF>Ix{GrVNT%bJ zd|j$qaFoak-l~GOy<{ZpYJ+T1_d2ObPc#b-P6DoYO_zhb;^X(-`^ZR(_~jRb6sCwC zNXQzhPYnJq@Ay-n;71-uY0Xz+0vC78dQAVKt>47w52r!66mRTns$f3R0;pjQ zsC)uAKv2WSJG?i_)WvMx+F3E4ayTY-7WWUFan+&AE zHb!H?r!`!@yqRT*HwJhxPTrwO;Pi2k+K2ltAnT);0EEiD)DHmsTn))wz!ochZE=xz zDLeH#=%AjgfAv6TMuN$tJj?I*7K8v)NKxy3qg~sgS5nN&Urd;P5xj+jY%`uZ;lR8$ z0gG45$?BSXBvBafosZE^LuI(j`(nOYzh>SXdKl7fJtQ)NQ68kJ_B=M06d7Y}{T(f7 z9lrVXuc#^gr%jLFvg->d9uSVViI19W(4)J$rt< zs!Yh^hc$gr5MW=qnqb8!Or~Qw9?(ih8d0e_s2HMf@D#xS`UW1sNt34E8 zzr9*Ht!I@MS-H-P0Lo~ne5|yf`%T(hW1C~i)2pvb4c`;cup1jGPHix)U}IG7zcbif zJ)mC}amMGQm_B_m>aDM%-8(UvGcYduUr>u~vxWnyR)mLtw#JKVneVAQ0q z#Ae^ady^cy^9U($$Df4(-Onj;G1qYiXXYcd}^TiMyO_roqxHJ0HX5w`UUQ5Ge5I|d!b*gBP&}2Id&8Xxa+T74aEh{X;#oj z*lxgekDUL8>(~H{pnYvr2eKp(vnj>83EO_YmZl%&)a7xON5KP7UhW%KI>nxWlQ$6& z%&&)goKz>!*T5H2 zP;RA%!P>=sJ;&u-yKwqAaqqqL`BdhFI4gBMeqm{;Dj8Nz?Q5B0RSib5Fm}t&tt0J; z^dgglB9Hv?N=T9d9H~PH<8ou7+EA*e`L|bcl{hpVW2XiU%4%7i(F&EO(frs8<+-{b zS^4Eo8O3!CpK!=BJV!p3(f7F4l=w#3BkQ?DxpVh@AZUd4zd~c}I37h~V%>oCWV!$4 zlqZvPp9;pcz8lvEP#~t8|nLeYy;_&6l1zPA|EBngqr}&xBoj}tf4a{UNexS)S<}l zq+}XKILt<@MGLZe<_6C_dP~h z?@sJt61{ZG@_PA+i0aS4Kp{8_81gR9st{vHQB!@if7E5+%O>LYw33AQNq+xdh+pm8 zzZA#kjjq38?vhmS9DIe=x$gGw91d2MGBGSf(17Onvy}Vyo8--i2Rfz@$dqH$AIP$I zAjwpkfa)KV-cu$5h&VUHpgs9U)2`!dT)u;;K(cY(+FaVvihBD2Ut>bN2Dq`N{t!pi z@02y4{d5a*F2CPbyo}=$ak=lDz^7C6FY$H3aaPG#NNO`s>yhC!-UbXj8A2#p3!w6E z$m@N}imm{7uA+y^ zQQC6SAz$&-c?0e3yxv3rBPkhmg*=z1aM_nI*s1j%^l<_e%w_qP4D5fqk()0{j~Ku& z9Q+=&p;|U3eskUtQ1548fB<>7v-&P9!CEn+kqsdpB+daPo|q$=%knd?chp<$^A~i> zriWffNnTu~r=sH2nH~J5zl@z?OZyWR^fA4=QhW9+wL+NUY5N#9@yy}!IgH7kxl@-n zU_aD(F)_QL`hP?Q0EqYZp+`7)LMem#iEo#A&znl}+FN*fWsz_7;q^u_)}!NtVgIIm+}Ldwfd3EtoO2-Am3!QF zZq{RY|Ld=Avh2IafE?dm**f_HBbhLlBw<=u&g3)=rhi?S)&zlQ&38$X z8|gzLcIm4W6b2hf#g*3B#zd{_jU}cqB@;!j#I`Dzd z9BcNZ%6L=8-4^)uSfDO_pk zTrsA>5vQhh)NS5nB-3Qv){4fr)JOodU;S*fE@6HSOtlGz{kcgVxmaq@_ZNm0IemIK ze!B}M|5fi3XZXWJugZlKfLj_g5Fm(``uwM<{}^PNNgnhwAtLoSK|2!kqdas$S`}$VTDE+2 z;ROn_#uVj>0Sos`|JV)TFN7YKe4n-kxo4a^s9jBfe|)D}-kbm=d~~XjtO;9jka2=e z@W`Fk-^TN`U%ZhuCSoHEBi9pTt4L&(oFJx%(s!F?_`2T+t~(hJx!1|}U;x~m`jb}n z+Yh+MAwrLJ6{yiQs|uUxs`2Nzf{<<44eR_ZR>rZ14${(pD9E;b2C{|Gy=?n7r%$-y* zUpc8MuA_QuU?YpvKjeRY>B*tBDWOt4-Q|5)5}WMLww?}Q2rie+MY1XXc20$Y)cJVp z@J-`YHL@GDp$+PqAKGJ*&zt}l8#~pX@1aT6%?*lMJ+kJxE>eqv6XxKKO-BTQ)pE@X3RmbxKA(SYce5#gj7%d#?#e~= zBUT8o+8HHW#TWX5fe>br;Y}zZ`F`+=bxh)AGO06WW@be#ge#YM^$TvY7ps4iIqAXL z)&odfD#Md9x9`HSWa zZ{2#?g|Jx0F@Mg@wK`5uFKS##1=(2M-$bK8O^RaL^ZQW0Cq^fRIF3LO)RAQez&?Rk z%Q;Ahti=FBICVVCeY|iE}hL&em=DNm#k!$d+ zV7AQ?RBE{^VSJ?OhY6~Q_M&=_l8BR1|1KN@>)KrslTmE{KmXia*^93T2Kyt%l*b3S z*4DL=BDJIZ21UF)aO-XPnQRLOcC7Egt&x-0-R0@IXTbeeIcj0{6kh3fY|ESbj{$D? z%Gbx(6{*ylWE{Q$2f(_l7OEfCtfhGnAA#eVXFej^Ez$ zJprb!C{}3+P-TdfcF9eCjlGOsB$WwQrf0JzI{GrhaRsf$_+NpP>_6Bf<^mLf@2;*1B5 zu}HR}ynG(a_j&q5AS2tpxyg*nVaP9EzC6{DV<31%QBkokhQ!ha=DO`l7zTLXtP=9i%FD_kU(EzTPWQ*iRfa`IMkYkN z%14s&QCQ`?&k_5D_Lp2VgF03kNz^4@S}?o**aX#tVr0!=ZLwu-A!f~HB~s^NUmB&v z^tA-#D5^SkDZMTxu1Xd$G3>3KzkjrP(rs>Mi+!P#r!KO~+PT>B1))@R_$^0O!JXKM z@M@eu)l{8b-px1WtK!}+9~CT|nfQ&3xzoS5e-Jq)8tHX!u#u}aQ}DeuURtH;=Ec?5fwI9!XvoZOU)4OX>;n zBZB_68;g!K;*=Pav6#4J>WJKniT6PK6OBTaFVb&Mj?1--+;Hqd4WTlBR?TXC<&lxy zk{h^K^9J2j9m&$==jK7vzp>EPKih&1`8~CvalQ!xp3B^#JS*Wx(0Co5Z1g~GOcVP; zxD3ViCst0H+t7&A`em*3-&1%;utMIoM^-$3r6bNa5W@2P_O9(N?MY8ktlXDY+J1hz zwdcvsn8`eXN&fBJij?C^H>n#e-r6=uJ^C{Fe8>8Rg1+TlipLOAlA6^hoYh|M2t4mP z>dEwVr6OC$691{PCc>T5r8T6SzK|tprg35R$-5@vVQlrTy~_w`Bao39E2&-yvo8xb zH^vN56TkVFI7mu_3>2}xMK#Yt(#)K-(e4g@mEwLAJAR=%YuUw&bZ@Q2PvKna*4lln z{mZpIo}K<@DvZhSj$PL?DAb&iXI{Ulu4ny%18iwBvXDJ<#TegrO+pIrw?ewdx~!r15(c;UF2g*l zZJF?6@yxxa52+-xH{YHPYvUQ;v-yr_jmUFEH2!(95}!$Kt`LH4)NXaC^kwaLZO>2& z92F1G6HtBjSZ>Gq2ijjXHmduU;)X)Ggm|-;Yks)CQ%Yr8ld;~l(dM0TSw3o3f(&E< zaOS*pm$mFjZ#P4ckQ$u7W|Lq^G+eIn)^D-l(#8>gB`(&ZZXUA>&K1I}+UGV%C! z*swvE_aBp{T|JD9;+8E1BDTJq5Tau-O;Lf5GDubdX`;V>I09nBt;SF#WRHfT#gaRM z77ZZ5PL#>6^{zVmz;IQdmcpVUthQccYD^@2rt;dk-dR$kNE1c+FEoMEX8->E7vF~H zyN#ZQ^xO;C$Nib+v-SsJe7`8(61{f%$kC$9xYnj~o~)xTs_0iSLa3ZO-byG(+)$L5>`CS&ItC2TngcZhoEmy|{0TgbQxKIn@V zKkY^7Xh_qPR)GiUV+TaQNs?V;i9ljmik);FWTGq<;|_fXUO=TmN@KE?8jTT~L(O@M z#^(aYKDLXX+xrQUI61wHgU;cn_276OIk^TZrZdM5PWpZ_$$$hFUVC7=KUm^Yz)Zsp zM7xdIND&#V3z)&Ue&X9WF?W+B3#~ z=`ihCY>jOIsNQoPRdr6JcVvvdN&W7K8-4R-z-0UcT+uUf_(zW)%Yf@!R+jMQjQ}q~ zTC1HQ7;slYqVC_nU&$C;p$@noVZ0LLk;&Ozo;up8z#0IbDbKdw2e*c`E}}bYcm)`Y z{W@SVUOy}b24Rfn;t=Q)g)Ez^3y-gYhEu51`8CM2m9VJL6;Lfg!8o~ZLO*RMx-W{L zMO1*S+iKj8C(#Lz!9lZJriC%7wFQ+HX)t#}SIq%K8{MB?E!RGKcT*3mAotz8c5d3t~C>+c|S#x z0D~Q?p9b{`;*Ng+d0XfTvPA51cX#hw6tZf6TX4CMC+tA}btkAC73wQKRejE7RWUvt z^kusnQ*<;f2Ee-Z`2L8t4+ll8ZWPV@z*@_{Bjqg=jY@@o8IxfS4-#`Iu|zm~irUW* z4;+u7l$^hyyap21N*C4&u(P3SNgzyy#2RsgNhCRaHzQ#$&TnH!^q|3EyFVi3JaG#$ z^nm!fc)o{Rlavolfk81^TuTH2^A~jAp$)449C4=Q*S9bB-szO*4&p)E%c}Oaq97hl zb@}|{a3zuro`EXRwb>kea;`hAtaChm|C1N7z)SvQhGm@T1{Dqp3PT0SExs=f27z?V>rnBz!*brza{ zYj8Lsn>xoIA!HrxP8tm;6zNpB=6PQ%1BYw0n1K>>dnp-tyFz(GqHwlB4LlmjcTYSW z=CAb0`6iyKmk;?!z|-%5JZIn)zTzKdyk(0{1*W!JcH}D5Yn(hOhpeJ`Cg^=~_)9DB zJIiG3hnq)1Bp$?DT5Atw zTbdoULsY2d8o3s)PC=J%0%jkt<5Psb{Vfah)gpYuxjMCY=CP_n87b+%4uhKDjO zu1D-VwLM*8;o!4XR98ZAoXVAysqJ>9pmhqI-%pC>fr>icR0j&ZKqL!<#Hy_h@`VM9 zKa-%YDAanhyf1dg(6A4&fSVa6ou5~I^|(T91Ed59WRc;fg&Q}ucwhc^zaMkm4=71l z9R{mtX!O8U3HXUUbNj!^l!gSArNzZgbv(6=fz@-&Bfq*eeFTcec7ZddNgx$`>*>m0 zt^v-kw=FUsh}qbicXbG}Q*}}bl#tbLX|4-Foa*pvg{$mrM{dFCBlGO|`#;{7+wO$!vBfpcR_fg>GLNS%o=!9buu9@QwM;st4MDVyLq$I!@ zyt1l~L*(F238#V+EsVStE0p$x!GXh)-&?Vt?vvPf!PIDW*J`i76s(E0)s_C$Jed}# zks87ewFsIDTt=+>b^KqzeVH5+eGH*T*YB!q*4O!$nh|>38XLFHQ$XRL%9WTz(9) zDzY|k5o$8%KT}F*=$N>w*C!9heS&q-!kZ72(We)23vym^-~0Od+E;~Xoeg~8yUF^6 zd60r&p&{vs+knXU#Y=c_J>e?95MiQY{#?ZF2dFxs1ggmGPsl^K1ZB~>MeYQJGQ8lJ z@8pQ_swus1_x0buy4PZ=bLGlsz_cJy<}%za#VfN!fZ;mD@bNYV%8|k(mJ*;SotH9P z%DXbQmzo{PHqpwThE$XPP}c$`5+W5gRc*pTgpR{S{qiPkNF?dyrK!wf)2)B({xN~p zc{K38=IUmXKPV;Zi4Ry4uO$X-F5wFse<9f*q#I}BW%tmRAN|UgNO^+JcjkV;w3D!x zWNY9K^$#U{ZOs%`xKChT^5=VY>&f=xE|v07jS5lf;Zwm!wWlj7PoWvgQKYv9U^o%4 zH!5K2g$dZF+r7Ut*DxX6~yACn+HQj#mmy6eIz50u$S`4+yrMl!QM%{XgC)UIJL38 zFW^6Ba9S${Ep@Vbt`^Yk-+t4T>*tqb@pLZ^cGt*m55apQF{Cak<3ydTEEi;{zjKF` z=qNcQWo&0xJSJ6H=JH_uW2cDSp{pK!k1wl*<*&9T8(M#9zMNuR?Zkf4uydhFW_>OL zFcUjzRg6dIPQ*=(9=?&`G&%80hL%r^j?UHO@mMtlh7O%@rzFK%F$}5t@k1GNK&;3x zx4++;0U4eS^(4*?TGH+365mU*8)0SKLq8Z0&G3kctx&su-tEMGZReqEJ@96eS#@ep zHlNf;dx{ae(!*!GY`BVhhF)JfU>ZBWYVi8}_C;YYBcsjOU3C9`)ZFOgg2Ymp$EkjN z?l?o&NztCW!U9lPI3vRay>W}2LWtK^YYESW8FoK!qpTCn&#*;dEydnjL^LP3+1H)L z@8W_G){wob)z+MaWbkN(8xkdxtiO(T$tZ;u5LS7!xjRsYTv`AtXp6E16a+5T3;5Vb z480Gj_514-@mh3hzbCrzw1E^%=&i-bDF%}r#AfPf#+8Ze6rzi0M`J||;#IlQf&p9m z{oF({I!PxRFV5=jK4$jq)i1Ykzr31%gDj_JX5a|3Y>BQ2mZ_St?kxCpCqs!k=U%+{ z)~=7nVjOjT?)IK+ZO!7DcDy8;s+4L}>XW$UlKmxe7i6+#LblJ35{xnN z-Hnr#-kF)Bldb>kk6=!~phoZbOY{wB+I4?mG_4!ER;$%rJJzwrOba!;eMi(&Q#_XN z60Y{DMGUlaT8}B_fOT}E0#I!kr=N|f*IzHD6$o(hSf&lrYotXljAX3PqmrQ#YC4Z5 z1(jp%u6=@f`G{ix)_HY-qo1aD+4kyLe;*RK1#1&F@^HqeztpVf#7E1?*7Td7j0af! zGf#>+evX#lHcx37xgC zu;<18qxxa(2gtEY^SzW+%9^ACR*6pIr%#Bsvxp2tV^S>x?y2MW)SoCKf^{eyc1sK9T%5X$;04V zj^A15(kC0Dn@|(xW>6c`GEQv@a+O;8~-EH zd&YQqiK+TZxx=YHBqg2V608v?V{Bv!6st;1Jj)<-+Ppny^|N;xRT$<`lv-+rLioN1 z4>;tjZlvieDO3G5C&HvU?<)%f4O1iS88VP;h@pZnk)?%ri6$olG@@hoSzF$SE100t z>dk1l#X`|l)rw>73p-rq9R}~#fl4#{t?#o{);i?8EI#Gt*{pQCvCb1T|9(x;OgkML zYZuq7k$RB#ubJR9on)#An#K%UkKU=NZ^(|3Dozp>v;HX>deq#U2o20lw9Bi7gxPyt zjzLwt@N5_LtyAE$YV_6-HEHup-OH%roAgJ<`Rgs&>4?K_w=U=**^Z*sD%HXZD((0hn;y9f|tf3LnscR^!t(%(X&6iS-H}2eX7^{4JSD6;! zQ|M9Au*vOn8XI-F%O4{)__Kc(_3L)|yHyJ%^f&UeXDeF3;=Z}{W;p+?l&xCdq}11# ze`_#z!5>Wut0*7*@h||h{iXy<5zT{2d)1$}$b~gJE-XhsciU@8l!A1a;Ce&e3H>yH zd2n!YnQWhMmR&VEZYBOZ-lB;`*>B39nZrSAZeo`BqPOm zRK7XB6<_z0VR~uB;MZ;}x~q1DGn=rtAhF4Ut(BF?+6g5^YE0}zqr`7*e`DVOy6p8& ziTCln6UiB3Ja*}D>~~{7XyqdH&Bk#Wg_7U_txyNyp4C-X_kFvxeoup$v>;sIsAl9B zjTJj67L_lE!-U?G`Zf*At{c&TNPpQc`c5J~B{MUFh(a zw|iCO3kYxOil&LF@-zBtU5|MyJo4ALJM2q1%vvl%OzaB9>(>)jMq6xOIpW5qreaaG z)=p#RDUK)145cP@A`Q!pU6lMcods-bB2Kj72+9m4tTD=EUg0%0QEi!H^F9GlEM+o_ z{d;xBmaf)nGe^BymcW8}kD~Vj7JI7jVi z*@j&1Iy=0J_HD_hP}A0sV$%kTEcRe;FhfcAGy(_hR&H3YW@DLDURSaQtg#{Rmz8mKY~&J}YluLweTmr8R}H5zxlH{ssw7a+d`D5d=SF z!{%hvwgRKUB!i8OEkcnUQW%qKc}GCb!$f*WRID$`O2I~KY@m4<&Xg-=izM`=R-6dQ z=W@~*!?oM|uNOr`L_p4DjZ{^BQg7cMIR|=?cURL|WWp%)zN%K>AsWF_70`YnSIxGi zkD{_8M2yEtYb>YTgKqbW4X^@iMHPRcgodG?5JKX<5gZaiWgtolPN0eOlQ$4|x}ba( zC;71#NzHwykD@nm0T>nEqqSDq31QEu9G)5xKR$WhD8TCUSbghu+EE%57{E+$M4f#U z3t!mi`1+0k`waqRkPbwJhnaXfy7Etxdyz{J#aF9(3tzE%$Cu?8Gim*UFHf?CaP>%_ z`jQPsF&Zs4Ym*cCgRtr7!yi{)VWs#UIf{x_gkq%JW5bU!49}kv-m-XOmqmgp<>H3k z&CHLzcChXF+NY9v<7D_zfWYtE-LiEb$UHF@nRm^d2|?ls(+)AM?p;$Ywy9^pa^5sv zKC+M^3tB6#8G2oKN6%28MQrtpk(KCj8mCREiI-lO36=Q6KtEMc@`kg9hUl2n-#>Kl zpRK()!CTvRxpJ8GHI}09x4r#jfbRrSzsO=xOQOq z-MqL?eEf3NwU>gtx;X*shwfV{0z`DHdnoS-L5wUV$h2vvlx?g&d46&XD$P1!Y zzNM4k>;lRUS+&-i_R`Qq$JNyp5@~6z?U)dS0^_Vl2A9-uJd z2rH}6ubxuCmBMGfCg}H9d7sO=k>?>|^CjVy>a}ki#sHo`>WJ6h9(s8}D)t8-AzSK* zqu53-0yC6Y+1M)TIhdI*g9w-M~pJ;28bTE9U;nTe3Zmw;{_ zo>8^>t}iRWHy0e$1e*|fAa^l7g325P+w{g4UsedAlEIjI0C)x)u2QM_LZH4ZZKgWS z>%ffgtN*P({Q4WGoJp3pi^37`#_q`nVqk}cokwB8mFy(P9MWR({g%RHP`dHv1rN63 ztUf2&lVqeI1q{5BhB&zPVUmvd5>4dsr{CT&I_r9dnVnkk0C!DX`8dx)3R`i0pU_P> zb~k$YF{6fP%o0hrM{Pv3`W~dbz|Y-YSfmg=&>}WTzAiM zlic0z9UtOe+A1Gdf6)cJi%;^jfzgh|-nQrBzTfVgd|YK7-FjQ~nmGp`-rz!(ro^M{!u&YfL? zZj8o3PhnddVNJbZ(=EDYCM@&tT=(eUKy}S6XGQW?UMV`I=?YFqSei$*cy21T2w0jb) zbr6%scwKzM`|r&{>{Z464rI8fn_1|t=W<_~cNr@xjv5GKWO;jsOg3MItWVWyZ{(;m zns^)67j*t@!>eM~g%4Sl`v@UQS4(cIT8~SreleXRS2J$XN7>cHo#9g48nBfz^UJS_ zJ_axDU8&DbF&)zS95g-gl#r{!KD}6$7~@$B6~$|7PZxS`eQxt>)w3SpjAPp<#gG^a zlTw$G(WL;k+~IFKtU=h3(4ZwB_~Gx*z({(rtCUJC>oyajcm;iPx$0P1PwTTJeW`!4 zoBa_C)D1Pl#O8w~I zD`{RHYYYiCIQY`_mwcgqOh^bTPdO2>dJNs%UR_HKB19W4g{Y^xCAi6E{5i2qfqUQ8 z1}p{XA#qs-AySzG-ku&>zqz_X&k_ZWs2cIH@o(jp=`ra_5P~Jb)T6FnJXRi<)<)fNv)ad zV(6PL^>u@68tpkYD+Zo&oJ*x|x$j=wf0{W0*8SFo!|P~0g*YEx#$UNI+{PvboDdZ_ z*+Wa;*1Yi;dH;BM*@EJDYwrl*M5RmPPk&J-&kK%{Ys=Tqnp&-&)CC>vK zZkAgNp*;T8ezfhP)%99P#SzY59C^K4>A2)^E|!1k76x>#EH8Cc0F+O0fSxCz-wd#J z1><5}9mQ(gr)L@S1JfTa!IBE8L~Wfv8jXhPG%mKjkf)u;plI=DH7 z`wVRsjg-Z7@ao8Ei!{5H%mm5%oxx?wKWRGNsRyo3z)|r2uE-#-M$lhhrY1iiF9^bl zChVix3G=Ux-yXDAG_QaCGJrBt&YcBLbunExK(TO?<0(>UINetvEOJDvZnPtb{+vl` z0Fy`GGJ&uJ_`+Ph!Y*Ck(q*^yF0R-+Wo`r2E;7o*QJuavBkH#sE_SKeeIce{ZGy{K zyYgI;>yL<{)m1%mT|*jYHX#RPN?y|DOhgB7cnkp;X}+sRkG3d2pTH&4mwfsmxr)q} z;MBC9dU@3bvY5A|`r%-(rE_J`Ba?teRPHIBH@te=4YK4k0<7L6C2+M2c}Dt6wq2*dePA=WU8c2c=X#5YwEB#(TgP$!M?*mom#B6# zEQ*g*+c@jk`3T8SBx)ap*#Y-8Q7!sPf^rOMpo?W#RwTCI;7FiM8;|cd7MWLlz~4%- z{UJ7v=Jx%z)4B(S%fKK8fSLT+<>d^y*>YPAEyJtQd|^kGO)u5FXfpQJYPF$Ri;UL# z2ey-H;`WJcm42+y^?ZM7{)?)gN-hgwxMPKPk=ssCB+l#LkX_G1uw5i7>BnGKLb!xE zsp5o~M%99G#B-!B&mqII@on@AdX^;yp;RfG9-c}}g_8Zyoe2OYaSQp)_?hQ2cC)kQ}czchm8g;H%b5BZ~gK8u3f#xXWzj9}Fgf&eL$GLMr>)Y3-Z@+;9I2G=; zoEwAWPgW3w+c6?nx`4DZ(C505ffavhe z^jnu6>~&ya{z2==Rw|W{K)t?tKW(mF?mKuumcz3eE6UGjujMKZ4H01l_ZM-nu&maz zS$=cV9nTL4!>{~oq7sQ+<{^H3z|bd|*J)Sq0eemIv3xC8QeW)lrSi7xG`Oq1w#M(5 z%1*lZ+C`R1nGz5x(wEL`?cdo1l2ya!V1NCth$qe6=oHHq3K&QM!z=e<43xCAIvb6P zR~&m=-?KobnlQq*=$l!Jd(VhFyZp~HYK;XynvrkdTxLh}y2~S>#o~cx@^gCHIYKDU zKJkGM2ij--BB+L1#1QmoUXb1u0k~7QHwhw7S1kO_B&}*-&UqeJcP_hTj`Ew7fB-LV zvJq|I`VqZ#Ys7dG?5^m<6TNR#-PGzFnI(QaMkhKdc21$-Uy`Y?z5CaChdZt5btns_ ziKm;xg;w&?7FX;Kzc|8={C#__-DN*nuQ3!Ru)3=`ytJ!!6x+J|StnV>Hv=wL&4RV0Jnceza!rtsXn9o{s)xnF9Se+%C}q;^$`ik} zB;4@vj7ySW_ZvyLc6U(D4@Z5%PbHeDB@ARqb~qdzsFW8ZVHOSskN~l|+UT#3;!MGV z0B)Yw06S9YD!5pipq#Az#Vu;UqtzH=~8Kh+j^PG~)?G^gXCJ4+`-R~Y1>@t3{8VBoD=8+@#Jq)l-7_x2RAySKo|_%DO_EnfVmoveRy@ZsmERv6HV0zB{z- zhNa-of0!(<^}FoqPqR02HwuyGr~GSey=s09ly)=<>Sy*N5mn4wjY>DQ0;SV=uim&n zRZ{7{bRyg);9J1iz`(#?f!Dd4#Q(Iy5aIzne<(!)go2O|BY_CW*peJzwCBw-Vow4% zGyvRyMIeE>=)4fP#51QomjCqL(+5wzo_a&JYX7II<^wza+`k$Vu7c;zHGIEY9wZy4 z3tm^43~uK+dOsXh!|+U?u{3R_;2hsc6MjElDO;ppzsds%KOFF9>mK~oI--wu5Y#4V zc?Vcqr7ABM7gtd-dfO=rKbc!>o7P{H-~3X5-I%cVubo-Lr#R~9W}zi_y$rmqTUQh< zpaVKZH~ungT~ZNXYH2-V6{$H2cIF>m)IM)3(QD07&svRI z45tU42qO@Vd?uAq7x%LMw`=#W>11p>W$e4oslorwh}ak{a*v+q=>Hrsy7b%wTtvq$ zW4k8fijdnP_+;VWlYM)u?IF}P*ZzO_WJOD6QMDX%F1w6h!(*8%(U+$bRxL2ZG?Foku z`tAGM`I@?drJt?yp!i<Z<%)a{PI(mWj9LpL|ZGJ}o!YRlKHY;32&Z4XTHst12phsqH~MKA>4TK;Y)<@8TxS z0(<$Y?gjW0rI0 zwm}e+Y$Dj(>oAGO;&kBu1(VR6)|JhT6>LoHX?QO$0i?_LSjc?*@#9CRCi|`$#PMj< zsZJu50(3XgF(H!J-QodVK+;%*AX3veFf-2d*wo*)b$BnS7%`+vrgAZZc` z3Wx_}lYs)#kkP6A)BT{A(fDgPB?-)q%lB28YfR1@i>q`FWCbkkLVP*i8Yy0BHheFe zTS3oxgmZu1%lIG+fFlA_oNNG2231>RO@jCU{C@ylAk{FhB^rhOtRsHy$QI~GO?qo_ zT3>e9@>ma$GB+Af;ZrtbUs+!UpyN(9Cg~F_1jjSXUa=Rr0=qOPbrd|EDy-AtWCc_J zA(FlP&y?X>4|qqc#9H<@XQ(*C4LmmC1i&Js8eh$lY$2abD5=03sW?L!aeAl|yBLeT)g2e*xWgOnr$Q#7U>aB9BPUfIs4ySIW6&H?BF5xp$N zRJ89TGR)$3jKrl}hfcdM2JKZ+eZlvp|5jC11Kq#d|E>j10+z2z4jnz^A5s4Y9Bqy1 z$XEu(iQ>m^Y%-|;09ZgvDrIGVec&^nSPH_3Jy!ZvU6Oydhsyut23TptAg`buX^a4_ z;Fq#5zjOA%C=K4vVUd0MNNBQxID`#1DHL=GhrGd^mbX2oLz!71J~sHO8-ho5 zBVS#+9Ys;?i(iZuTFz09e|~2So_3r2T0pzh2TOr&*{>EoTfns!k@@=jrAHUcpN@7I zI6iOGJn1#rV#y`R>mnqCvlqx06-QiTKSTIhbpU{XDCRaT*Wc3w6`Wlk?f8gv5* z4RXH&_Po%g$|*Dwa>Lv@+>>bhbGjf^!&;$QLu*iNJ77-b3v3QzesQ%WOO=UOjK#C* z;goTSFNW{0Z9md}Q#(=`V2hGC_P3R25+VPo%XH=WknIS9kmJe}Vee#7pz*S*oZd_{ z!(KlqnF2R96n{4{K_;(lY+E21-1^2BjV};?7Q)@jce@qK@PhJdYRh^;A;@at{Aqh6 zcj}6V#;!vZw?yAP&Fd!b~dgTUl> zJDNW*3Q9zFYdK89atU-6&bX`~Lyh-X{`EPncoioo=^-XWd{e8L)U@{j)SVMPPQe(k zHP-F&_@)#cVoS?Fbwrw;8@mc4-&B#y)B#E^9+6MnR;%-OtoQ3LT&C>EzWd#+atKmp zrsxRe8)YbM(*sUbAe&1pClsBeR;FU&0ulYna1oJx+IV!&bppFg4w@jvZ_t^BW^?Nw zBba1eH8T1fehZ(tlQw;fV~eY>8@C<3f9kuGeEG54JB^fgM%e#vKmz$2kyxEtZoR+j z1(bU!=c^<@9gRt-+Z27iqb4MwPRvhoF zt@^hlY39oX?E43Lm{c4cA7DbzA&Z8aZqy5`wxwF7CX^MTN5oSB`)A^1Z;fzW@x}C^*CZC&Ucv7()eHA*pH=qxl){SH@MVF3ZmL zs`YI<=gLt~8`xy(U0+?Rn>&{7a9dx0w=Av zpWf-)EVRo#zI2a9wKC0A0saxS&yf+mRsoghTx>C^w2IvQYOPA`6VgGcpM!w=sw1bB zqA#JA`g0&gDRqj|O*dQEqOrkNmcHgvu9FJ|3xe-Qwp+5%kP%X{i>PV`07+P{>k>#T z1~Guh`idI?@Qa)7Lno+Ur^W+)f{#{`v!8T{D-6QGN~_2sptbv}ojM8Xl?hL|qMeZFohP;QVsl5a8=`~Z_DDXtNO!;kD zYhjLUsS+}0Anl?8dA%Wxmvd8%&DSNZyk~0GHtQ8UaYUNMdMhwh$nuXE@lQ`bZdNG@ z-8x!5H#}9l7!Uec6BU%8s{-c@_o{C_?=$d2bwgY>$)GbHAc4LN76+(=Zx5=U$XAp3 zsgSUC*CqT+o!q)Jc{#?!(L@`Dg&tQ+^4!Ji0S6Dos6tj!SWV~%(lhmAv_320QHDm; zVx>D$f>_PEFH^i3VxIbAKyHO4HVHM8|aMo+J4M1_hz>UBJ*A zkcGxeCt${_mGu0Kb#B+G76@Sciv;bKTCvg-{~p}be+<<3e!E|TUBm{npY{JSGet*y z%Ew7f;kBM5%W2_G)!57|Y^}e-2_uPRvNtNfxD*8PixWMsWZbtkD$@8=9AgA9LET?# zw251Iyq9OJJ{VeKxi+Q2)`QO0V~yBNG8cMRmb;)1I0Y5k`@YH*5?z|=%9a0}Noj!; zkuc6I>&gVuc;;49Q&>nKxnhM7fjLM3svosZ)9X1U)5dGsTuWa_n;S&joj35Se#C=S z@|vjBj%-udw%MvO(o2I?je=F5Qqk4VnO(%21vWDTt;sJkbWT?@MqjtME$5ak_P;Kv zerxhi1zb)x*=^pAiLmosh2r<`2~Yq~r-pY9*XUJ7gqQ9Kt>d?=-NVOfoII0QI(JwR z!Py!nd#e{<*FBP?y_cGtc!Yy)+U^3!|9nalNG zcD}CMv`8I6%Pi(4vOz_nY z9B#?Dd&5|(K&3=;^g7kqzH4^3KFBcw28vhZ7?=!1d?;8Qzuqd0?)jI#*O8Pw$~n1v zK$ji-`df-oAHmXk7@ZBW+#mlr&h?pQKjj_n;oePD8TX2d>pT~i((%PQe$j?U4CewA zNs21i!~Bkq9qSys{GR+;wBTixVo0&`L_X=-&1ogdfnNeo58MdnL^NzPFcydGqr5*x z_s}EBK>;%IN)fLOnBg}`B^wIc3oJ9N<-D?Qhlk&{G)=WLT}n0FXdards`xlc93+xg z1RvElg=*S+o7+w**$E%jGa(r6RMvQ7T7K^V$3u<`j%9FVR-BZpSU`D&e zRCgQHjTz~kqoSB=vX`6z18lY7AIWui=15DZ8@F9>4$Kf%1GU%3WM|6+U?MtY6;jc@nYApYsCzX6?Fl zEBL;}elVw8-h*~m-|GN^L5=P{jL&ovdQ&Si<${(va4esG|6 zEjmBj)IsiMvH|ejwT{Q2`c9C+&6tF)r5y|ICCOptIucks84zqxn?(6G*!!ZyV*j_J ziN9J0Ah(c(iWpn}$-Gt;bvtRh2Av1w20V{AGA;mgd;MV4qevvm4Vc^dlfN$`lL8Rb z%8}WwRekS4tKvW_vz8~oD7T;l<8h+3THFipr-V=0$Zf{y#m-ma*OnO0h*$~!`q&h* zI?*oN?tTPDPHDNj@yilaY6vj%q|>E?N7-Ar@L$P9q%CwkE1Pq39Sc>IKp;C(o<^|F z=zPDy4)@PzZC4;shtM<*uORD6u4&637V@p`@o%o}R;|!BAKw`;J^=zXUcS|@mAR+m zh+jUHw|ECpam$+_hGb!b1Ah@x=$+{9>SoEiv1mRWx;<1NMznA%l(ByLqZPY!WMyTlC6RgTBx|lu1uyeyqcP?uCxviY= zKP{k^MjV`pnx@&d>a}5$xop%#Axq7vrKsSbnV?5FWkHy4QS;MwR?qR3ruh-w*E*e2 z-RV3pD)YVvoB1(kzMyr_SMSQUkxa10fbQj+)fRf3=BIch=N~LYh@~{S)=?KG@s*pQ zp<&kQPxhJ4ClGx^8eB!CzaZY~9fBjBR{a5gH(W?x{XyC@<-0^Zr7&H?KC=M^KJh9} z!!D4@naHHuA*Y&hcU$M`^k%3RgC1H1H)J}6 z(C_(Z1i7Zwq~Nipbg(#J348>%ZCX7ZpoA$JwWASAsZen%^BNsia4|!s-$%j@WbwXf zhh0Bj>aC!Z1%87ViIcy*nVEI&bb>5rydhBm+g%hcIy;qjb{|^yxN{kGj*B%MPI3_#4LqBG2_2pv-%5_rfXx{J5_%z5yLX58 z(%Te~@6&B{_z00ag*{Nlil^op32Bgam;MkvLnPVlxUyuW+GN#4gfe-8+;~DhtbjrA zoRM6E%X&UemAtv!?bv(smCX}*g`vhpCqoUU0S>nz<1J|@>^i45PFNYf5 z++;H!?k&_n7ZxkYZ^7$sbgDH+nrFOkQ4WH~Q$|m1FjEw#c?CBM_csp^JH5djz8ILJ zAX;o@q+%*XjlpfJyHPlxskLyp?m8xRyh0@NTt04m$ZFCNmbfa5^W*VQ)t#GjGPMii z=RSC1n`Xf~40SI*P0KhWIH*jO+8&P`Ej`TovMAm=4gPWLDwUb_sNGSvl@L{gg@{9+J4q;HRtF zpgbqP-MICOUKz-$0Z}9^FSPbUlxAB~WqVNUKM(O8{KYOEVbD=*pHE=^h(<%vtf=HF zqk~!nYO-U8X1_;m|6m5>@{=84Kc}>NP?|<6R_lz>KsDa_{M8@;P@{;cD0zPk#{`&h zpSa6YD;LniIXOpPw=`3>ThEi!Qay1Dd@JQ|_4wG1?%8%a(qn1i+x%>yk5KtJB;Qh@ zXMQG>5>NNP*y7*pX0xyCbbeCd`l7GJ)Uu|N`Y*^=gjFwkq|`uIgv&s1j|}mCr1ca5E|$eB&(LS1GiN=lVos+R$`s+G+bKKiIb*h_G%G}P5kckjC zIouN*>Z?Yc9Iy}F%J)HArFQf`P`H1c;-eV19mk8y8TtqX>bd#;FvC=}ig3*o^+m7Q zA4#e{$%0?dI`Sz)xSDz9^S&QhDUxFCl5j;}n%WD+tx=XVupUGdOpvAu(08cB5gB-pvQ@8SornhDvFry!070wrMF?; z^Shk!kR$;_z^wg(%Althz^Z9!ChNou z`H`U^!RLeN73_ME+S^V)YVPn=!ehB#18Ey)Zf4VQ_6;j27fiRQja}^-ekF#)Vtehw z(7h$^OiwdV$-oBOw||7^OiY}5o`c}x1M@3|!qkccQAbu_Y#BES8(3GU@d~P)GRcHm zKJFh-D+qv6WEkIypy6P)q5i&w$#zQ=#r0=zR-`#gtMKmoH@^^T@6E)||3$XGKUF*1 zC!r$5``t$cbMcM5N)R-++~e_Dr6vj8CF|-W+!l(MIDr44d?>l{Df)H9sB!Xly(8<_RCopW0;wWy{Q7J@mxBO;CjYOIA7Ia(4cO-f0W#PKfRv^~7;{8-tz^Gt`N4Os zg%@E=?b`p~-NV1EY4@H!>prg{BNUgISQX%7ar9^z&>*)!i4qt+CEa>W!M)F-K(D~` zg08c8=ccG~WKW_4e$wRi+CuSGI#JS2l1giySM_qC`|C%;=w=|@rBV2pzL=Y5ykVN# zo0!@CF~8hh>4IWmbQ?ii#p-=n5`8<}d+eO}2y-y3kVpA%lT@JYFWZTn77yFKX}?l$ zdOfDFR=tP$rR^o5#ft*DG+Ybrg_`jl5T3&7luc^$&sQf>pU?%SXMOcyqR67cf!5%W zt#gR^(g#!Z({GsVZ6JWyD|0Sw*MH_=8g+6SmmYSC-06*J=wD?m>TXsI8uquR_q`qV zZ+r`oQV7QN%)qdRTjd~EYWFi)CF-y&$QxO!kQjKaJ4hm+h>-Ns0dVfeOpwE?a5jzF zq%~`L$&3C{Q{I6m!rypVAWb9mpK*v;M{JXQ)x&tB* zb@t;1nSHAlnFgy?M33ne?MACvTq^tyg@n&X)Nwa`<%5lEMSdgR%*pGw zmI@_}OTk?P3SDt}Su9iyq`;iPtd(=$;0ib4TI|4+#NePu}( zfm|hiI>r34DpJfe+7QsT{J2qT?&#{h)$^0#pdYFe`rq;Gs8?e%oc@)+^MR2bP8>7JQwH zgb-V<3vQT`R#{dP=$UK zBXH(-q;3^OYPnE1U7*#wb=h&y7N+4O-A*U^B-)Aa@~(CIo>hn!!EzxL(^3pIZPX; z?4=Gpdw(gG6ek&fdpK8-)$lg9m&IZvXASq82Xm=AT>)+aYK&YW!BxTjMHgT1VLaVe z-a6zvK4W=)3@;s-_@#GtPwTEKlS-Ee@9K6WQ4F-(E{1-gXc{0CySU5K5vaONHj<`NHch3kYymzvJs6?9v)i_(0|6~ z$F;O7QeT7Na+1iXi2WAwhflt6iR{Vf0gWeyY zP;Uq*$=~g|mr4u`+-W=5)ZVn7bhzBbF#FzlWc78=jj|U0CBIk;IC`iV>pQl>gx{bA zt26oD>1Q~faG-*|mV=8m-XE$huE@USsSeA<4P3uIcFbDdlPmTuUYz<>Dh5R=SYg}I zxkJDqgbarpdLxcj-SNG+3-&_D#!ajq0XVFfaI6m42+gyE1YbXSSeK^a{sH+f$2Qj4 z>O^_@w}J#W{Aq`_iy^&D(?@%@DqQ~uB1PI z=JEcE$?yl4U5@OsVt4lBOMKzHwELN}GorKOrP-s49`nUdPw!1^c%1(HPb~w>aQVAe z4s03J5_d3P$1{BF%}(LCn-hV4p-vB6r|k#3hMAV(^Z<2R9w!?9kdskWQPI1$wzl{0 zuS}NpYZAQL9A4IU+%M+cwkuZPL(n(amK&dbWm%Fo9_Pj+@5NvqXEmJa+#t3_(} z%gJd3{WzRx5L#mQI%F+O-*uO*Y;DKK#*FrI!!`!}d{!0~%HXaEFy+gtyBpEt^|})C z&J?Cop5>G0vhii)XfegyEYc5!jJ zdbQ*TpSO^*2biQl%2P@SeSYl?FJB(Bva({3@AAkMAhxwN?)^2P$QEfO_rB!B#9X1v zSS%F#0Gvu_-pL+_u34o=Q9oEpIkzTn(Zh|jmY4EnaxwD`b@fftk# zCvh>cVLE+kF0k0bvz~Q(s<{~%_Rh|^{@*{p&u}#w!-OJ!K`3kZm)2U?`u`=q?T#-AybV&rr%F;IQ^ zP~o4fuBWlsf3&`KNrab|_tN($`4WcQEw_NF;TFjKfd}8Ny9H8cgL%I(>c!>-8xc-Q zJb!}r_pSrfRBXWH5DA%0!-AQ`1MG@R1pC5Dmd?UI>`> z46iZ*0S9hPz(BRD(`QO&$NR$)O>>M+C&?^4UA*P6;{**yX;_awu(#5??#=}+L6fz# zw16Fnh_JAYrR9^V+p3YDXfzspQdU+5kD56%V`OAxo`ju7UXD*qP1yx}7Znf?KxD;O z56Gtcoa_$v@^hEHy=VHW1Hj!$-GBOXDkeaxe+l{gE*8`dEv{9|JV_>o zbSuKH(hAGE@dRj1liVFAPQgy$e7i2Vfp1F3a>U2SvZ2@YNs zv#_u*J?%*=uxvs~Iu%~pS#adMJj%PJK8f|nDlIRE9NV4NDkm!o=8^F4zSFqNt@>9s+nG!3 zc*H*_zdzLU2J85&X`c1!W+NonnywKLf;oka?YraB7F#^Q++sV=X813g?g^vg1;@_c F_&-~aSn>b> diff --git a/docs/processes.puml b/docs/processes.puml index 19f01fec..953a80cd 100644 --- a/docs/processes.puml +++ b/docs/processes.puml @@ -13,7 +13,6 @@ if (operation type) then (outgoing request or incoming response) partition "Graphsync Requestor Implementation" { :RequestManager; if (operation type) then (incoming response) -:AsyncLoader; partition "Verifying Queries" { fork :ipld.Traverse; From 86c0646df23d0dcdae6f2d8432db9f27dcda50dc Mon Sep 17 00:00:00 2001 From: hannahhoward Date: Tue, 7 Sep 2021 12:20:53 +0200 Subject: [PATCH 7/8] docs(architecture.md): update for link system update references in architecture doc to reflect IPLD linksystem --- docs/architecture.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/architecture.md b/docs/architecture.md index 7269da72..9bd07dd8 100644 --- a/docs/architecture.md +++ b/docs/architecture.md @@ -29,7 +29,7 @@ go-graphsync also depends on the following external dependencies: 1. A network implementation, which provides basic functions for sending and receiving messages on a network. -2. A local blockstore implementation, expressed by a `loader` function and a `storer` function. +2. A local blockstore implementation, expressed by an IPLD `LinkSystem`. ## Request Lifecycle @@ -48,13 +48,13 @@ This order of these requirements corresponds roughly with the sequence they're e However, if you reverse the order of these requirements, it becomes clear that a GraphSync request is really an IPLD Selector Query performed locally that happens to be backed by another remote peer performing the same query on its machine and feeding the results to the requestor. -Selector queries, as implemented in the `go-ipld-prime` library, rely on a loader function to load data any time a link boundary is crossed during a query. The loader can be configured each time a selector query is performed. We use this to support network communication on both sides of a GraphSync query. +Selector queries, as implemented in the `go-ipld-prime` library, rely on a function to load data any time a link boundary is crossed during a query. The loader can be configured each time a selector query is performed. We use this to support network communication on both sides of a GraphSync query. -On the requestor side, instead of supplying the local storage loader, we supply it with a different loader that waits for responses from the network -- and also simultaneously stores them in local storage as they are loaded. Blocks that come back on the network that are never loaded as part of the local Selector traversal are simply dropped. Moreover, we can take advantage of the fact that blocks get stored locally as they are traversed to limit network traffic -- there's no need to send back a block twice because we can safely assume in a single query, once a block is traversed once, it's in the requestors local storage. +On the requestor side, instead of supplying a function to read from local storage, we supply a function that waits for responses from the network -- and also simultaneously stores them in local storage as they are loaded. Blocks that come back on the network that are never loaded as part of the local Selector traversal are simply dropped. Moreover, we can take advantage of the fact that blocks get stored locally as they are traversed to limit network traffic -- there's no need to send back a block twice because we can safely assume in a single query, once a block is traversed once, it's in the requestors local storage. -On the responder side, we employ a similar method -- while an IPLD Selector query operates at the finer grain of traversing IPLD Nodes, what we really care about is when it crosses a link boundary. At this point, IPLD asks the Loader to load the link, and here, we provide IPLD with a loader that wraps the local storage loader but also transmits every block loaded across the network. +On the responder side, we employ a similar method -- while an IPLD Selector query operates at the finer grain of traversing IPLD Nodes, what we really care about is when it crosses a link boundary. At this point, IPLD calls out to a function to load the link, and here, we provide IPLD with a function that loads from local storage but also transmits every block loaded across the network. -So, effectively what we are doing is using intercepted loaders on both sides to handle the transmitting and receiving of data across the network. +So, effectively what we are doing is using intercepted block loaders on both sides to handle the transmitting and receiving of data across the network. While the actual code operates in a way that is slightly more complicated, the basic sequence of a single GraphSync request is as follows: @@ -93,7 +93,7 @@ The network implementation needs to provide basic lower level utilities for send ### Local Blockstore Implementation -Interacting with a local blockstore is expressed by a `loader` function and a `storer` function. The `loader` function takes an IPLD Link and returns an `io.Reader` for corresponding block data, while the `storer` takes a Link and returns a `io.Writer` to write corresponding block data, plus a commit function to call when the data is ready to transfer to permanent storage. +Interacting with a local blockstore is expressed via an IPLD `LinkSystem`. The block loading function in an IPLD `LinkSystem` takes an IPLD Link and returns an `io.Reader` for corresponding block data, while the block storing function takes a Link and returns a `io.Writer` to write corresponding block data, plus a commit function to call when the data is ready to transfer to permanent storage. ## Requestor Implementation From 19b11e4a94b06ebfa3a70880930b6a61da232aa4 Mon Sep 17 00:00:00 2001 From: hannahhoward Date: Thu, 23 Sep 2021 19:42:41 -0700 Subject: [PATCH 8/8] refactor(style): cleanup errors, remove unused field --- requestmanager/asyncloader/asyncloader.go | 42 +++++++++++------------ responsecode.go | 2 +- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/requestmanager/asyncloader/asyncloader.go b/requestmanager/asyncloader/asyncloader.go index 0bb458b1..0bf02a8e 100644 --- a/requestmanager/asyncloader/asyncloader.go +++ b/requestmanager/asyncloader/asyncloader.go @@ -3,6 +3,7 @@ package asyncloader import ( "context" "errors" + "fmt" "io/ioutil" "sync" @@ -34,17 +35,17 @@ type Allocator interface { // AsyncLoader manages loading links asynchronously in as new responses // come in from the network type AsyncLoader struct { - ctx context.Context - cancel context.CancelFunc - - stateLk sync.Mutex - defaultLinkSystem ipld.LinkSystem - activeRequests map[graphsync.RequestID]struct{} - requestQueues map[graphsync.RequestID]string - alternateQueues map[string]alternateQueue - responseCache *responsecache.ResponseCache - loadAttemptQueue *loadattemptqueue.LoadAttemptQueue - allocator Allocator + ctx context.Context + cancel context.CancelFunc + allocator Allocator + + // 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 @@ -53,15 +54,14 @@ func New(ctx context.Context, linkSystem ipld.LinkSystem, allocator Allocator) * responseCache, loadAttemptQueue := setupAttemptQueue(linkSystem, allocator) ctx, cancel := context.WithCancel(ctx) return &AsyncLoader{ - ctx: ctx, - cancel: cancel, - defaultLinkSystem: linkSystem, - activeRequests: make(map[graphsync.RequestID]struct{}), - requestQueues: make(map[graphsync.RequestID]string), - alternateQueues: make(map[string]alternateQueue), - responseCache: responseCache, - loadAttemptQueue: loadAttemptQueue, - allocator: allocator, + 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, + allocator: allocator, } } @@ -84,7 +84,7 @@ func (al *AsyncLoader) UnregisterPersistenceOption(name string) error { defer al.stateLk.Unlock() _, ok := al.alternateQueues[name] if !ok { - return errors.New("unknown persistence option") + return fmt.Errorf("unknown persistence option: %s", name) } for _, requestQueue := range al.requestQueues { if name == requestQueue { diff --git a/responsecode.go b/responsecode.go index c96c2127..c4a1edfc 100644 --- a/responsecode.go +++ b/responsecode.go @@ -95,7 +95,7 @@ func (c ResponseStatusCode) AsError() error { case RequestCancelled: return RequestCancelledErr{} default: - return fmt.Errorf("Unknown") + return fmt.Errorf("unknown response status code: %d", c) } }