From b65e00ae50dde3973a2f3e20ec9e9d5be59b4d82 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Fri, 1 Nov 2019 17:56:39 +0100 Subject: [PATCH] docs: new tech note on TxnCoordSender --- docs/tech-notes/txn_coord_sender.md | 638 ++++++++++++++++++ .../txn_coord_sender/distsender.png | Bin 0 -> 31451 bytes .../txn_coord_sender/distsender.puml | 26 + .../tech-notes/txn_coord_sender/erecovery.png | Bin 0 -> 18067 bytes .../txn_coord_sender/erecovery.puml | 22 + .../txn_coord_sender/eunrecoverable.png | Bin 0 -> 14114 bytes .../txn_coord_sender/eunrecoverable.puml | 16 + .../txn_coord_sender/interceptors.png | Bin 0 -> 48127 bytes .../txn_coord_sender/interceptors.puml | 35 + docs/tech-notes/txn_coord_sender/leafbase.png | Bin 0 -> 60840 bytes .../tech-notes/txn_coord_sender/leafbase.puml | 53 ++ docs/tech-notes/txn_coord_sender/mismatch.png | Bin 0 -> 35488 bytes .../tech-notes/txn_coord_sender/mismatch.puml | 31 + docs/tech-notes/txn_coord_sender/txnbase.png | Bin 0 -> 23538 bytes docs/tech-notes/txn_coord_sender/txnbase.puml | 21 + .../txn_coord_sender/txncoordsender.png | Bin 0 -> 55505 bytes .../txn_coord_sender/txncoordsender.puml | 37 + docs/tech-notes/txn_coord_sender/zombie.png | Bin 0 -> 34467 bytes docs/tech-notes/txn_coord_sender/zombie.puml | 29 + 19 files changed, 908 insertions(+) create mode 100644 docs/tech-notes/txn_coord_sender.md create mode 100644 docs/tech-notes/txn_coord_sender/distsender.png create mode 100644 docs/tech-notes/txn_coord_sender/distsender.puml create mode 100644 docs/tech-notes/txn_coord_sender/erecovery.png create mode 100644 docs/tech-notes/txn_coord_sender/erecovery.puml create mode 100644 docs/tech-notes/txn_coord_sender/eunrecoverable.png create mode 100644 docs/tech-notes/txn_coord_sender/eunrecoverable.puml create mode 100644 docs/tech-notes/txn_coord_sender/interceptors.png create mode 100644 docs/tech-notes/txn_coord_sender/interceptors.puml create mode 100644 docs/tech-notes/txn_coord_sender/leafbase.png create mode 100644 docs/tech-notes/txn_coord_sender/leafbase.puml create mode 100644 docs/tech-notes/txn_coord_sender/mismatch.png create mode 100644 docs/tech-notes/txn_coord_sender/mismatch.puml create mode 100644 docs/tech-notes/txn_coord_sender/txnbase.png create mode 100644 docs/tech-notes/txn_coord_sender/txnbase.puml create mode 100644 docs/tech-notes/txn_coord_sender/txncoordsender.png create mode 100644 docs/tech-notes/txn_coord_sender/txncoordsender.puml create mode 100644 docs/tech-notes/txn_coord_sender/zombie.png create mode 100644 docs/tech-notes/txn_coord_sender/zombie.puml diff --git a/docs/tech-notes/txn_coord_sender.md b/docs/tech-notes/txn_coord_sender.md new file mode 100644 index 000000000000..19aa143fa1b1 --- /dev/null +++ b/docs/tech-notes/txn_coord_sender.md @@ -0,0 +1,638 @@ +# Transactional interface between SQL and KV (and TxnCoordSender) + +Original authors: knz, andrei + +This tech note explains how the SQL/KV interface currently works, up +to the level of detail necessary to understand the processing of +batches, error handling and SQL savepoints, to understand bug findings +in this area and to participate in design discussions. + +Table of contents: + +- [Introduction](#Introduction) +- [client.Txn and RootTxns](#clientTxn-and-RootTxns) +- [LeafTxns and txn state repatriation](#LeafTxns-and-txn-state-repatriation) +- [client.Txn, meta and TxnCoordSender](#clientTxn-meta-and-TxnCoordSender) +- [Interceptors: between TxnCoordSender and DistSender](#Interceptors-between-TxnCoordSender-and-DistSender) +- [TxnCoordSender state and TxnCoordMeta](#TxnCoordSender-state-and-TxnCoordMeta) +- [Summary of the all-is-well path](#Summary-of-the-all-is-well-path) +- [Error handling in TxnCoordSender](#Error-handling-in-TxnCoordSender) +- [Error handling with LeafTxns](#Error-handling-with-LeafTxns) +- [Concurrency between root and leaf](#Concurrency-between-roof-and-leaf) +- [KV sequence numbers](#KV-sequence-numbers) +- [Seqnum consistency across TxnCoordSenders](#Seqnum-consistency-across-TxnCoordSenders) + +## Introduction + +CockroachDB's at a high level is architected into the following layers: + +1. SQL (incl. pgwire, SQL state machine, planning and execution) +2. Transactional (YOU ARE HERE) +3. Distribution (incl. range leasing, rebalancing) +4. Replication (incl. Raft, replica lifecycle) +5. Storage (incl. engines: RocksDB, Pebble) + +This tech note pertains to level 2 in this list and especially the +boundary between levels 1 and 2. + +Conceptually, the "transactional layer" offers an API to the SQL layer +which enables it to consider the CockroachDB cluster as a +transactional KV store. The API offers ways to "define a KV +transaction" and "execute KV operations" (and get results and errors), +and maintains the lifecycle of KV transaction objects. + +In particular, it is responsible for a number of optimizations relating +to transient fault recovery (incl. implicit/automatic retries inside +that layer, invisible to SQL) and transaction conflict resolution +(incl. implicit/automatic txn reordering by updating timestamps). + +Its other boundary, between levels 2 and 3 above, is another API +offered by level 3 called `DistSender`. That level also allows the +levels above it to "execute KV operations" but it has very little +logic for error recovery and does not maintain KV transaction state +itself. (In fact, the levels 3 and below are mostly state-less with +regards to SQL client connections. The transactional layer is the last +level that manages internal state on behalf of a single SQL client.) + +The transactional layer's role is thus to *translate* requests coming +from above (SQL) to go below (DistSender), performing optimizations +and error recovery during that process. + +Since the interactions are relatively complex, the remainder of the +tech note introduces the concepts incrementally. The explanations at +the beginning are thus somewhat inaccurate, merely providing an upramp +to understanding for the reader. + +Interestingly, in the context of the transactional layer, the word +**"client"** designates the local SQL layer (e.g. a SQL executor or a +distsql processor), not the remote client app. Likewise, the word +**"server"** designates the local `DistSender` on each node, not the +CockroachDB node as a whole. This differs from the terminology in +each other layer. For example, in SQL: "client" = remote client app, +"server" = gateway or distsql execution server; in replication: +"client" = layer 2, "server" = leaseholder for some range. Depending +on the reader's own technical background, some caution will be needed +while reading the rest of this tech note. + +## client.Txn and RootTxns + +The first two actors in this explanation are: + +- the SQL executor, which organizes the state of the SQL transaction + and the sequencing of statements on the gateway. +- a transaction object called "RootTxn" (the name will be motivated + later), that exists on the SQL gateway, and which stores the "main" + state of the SQL/KV transaction - for example whether it's aborted, + waiting for a client-side retry, etc. + +A simplified view of the interaction between the two is as follows: + +![](txn_coord_sender/txnbase.png) + +- the SQL executor instantiates an object of Go type `*client.Txn` with + its type set to `RootTxn` (hence the name) +- during query execution, the SQL exec code (on the gateway, we ignore + distributed execution for now) uses the `Run()` method on that + object to run KV operations. +- "under the hood" the RootTxn translates the Run() calls into + BatchRequests into the cluster, and translates the + BatchResponses back into updates into the `client.Batch` object + provided by the SQL code. +- at the end of the SQL transaction (either commit or rollback, or + close on error), a call is made to the RootTxn to + finalize its state. + +## LeafTxns and txn state repatriation + +When a query becomes distributed, we want other nodes to be able to +run KV operations "on behalf" of the main SQL transaction running on +the gateway. This needs the same txn ID, timestamp, list of write +intents, etc, so we can't just create a fresh new RootTxn on each node +where a distsql processor runs. + +Instead, there is some new complexity, involving three new actors: + +- one or more distSQL servers running on other nodes than the + gateway, which receive requests from the gateway to execute + work on behalf of a SQL session running there. +- distSQL units of work, called "flows", which are specified + to run some processing code and, relevant here, operate + using... +- ... another transaction object called "LeafTxn", which contains + a copy of many fields of the original RootTxn and is + able to run KV **read** operations. + +This works as follows: + +![](txn_coord_sender/leafbase.png) + +- the SQL executor instantiates the RootTxn as usual. +- when a distributed query is about to start, the distsql + execution code pulls out a struct from the RootTxn + called "TxnCoordMeta", then "trims it down" + using `TrimRootToLeaf()` to turn it into the necessary + and sufficient input to create LeafTxn objects + on other nodes. This contains e.g. the txn ID, + timestamp and write intents as outlined above. +- the trimmed meta struct is sent along with the flow + request to a remote distsql server. +- on the other node, the distsql server instantiates the + `LeafTxn` object using the provided meta struct as input. +- the distsql processor(s) (e.g a table reader) then uses + the LeafTxn to run KV batches. +- when query execution completes, the distsql processor + extracts a similar state struct off the LeafTxn, + trims it down using `TrimLeafToRoot()` and the + result is repatriated on the gateway when the + flow is shut down. +- on the gateway, repatriated LeafTxn state structs + are merged into the RootTxn using `AugmentTxnCoordMeta()`. +- on the gateway, any error produced by a LeafTxn is also "ingested" + in the RootTxn to perform additional error recovery and clean-up, + using `UpdateStateOnRemoteRetryableErr()`. + +Why do we need to bring back state from a LeafTxn into a RootTxn? + +There are many uses for this data repatriation, not +all will be detailed further here. + +One use which is good to explain why the repatriation is _necessary_ +is that of refresh spans: as KV reads are issued by the LeafTxn, it +populates a list of refresh spans. If we did not repatriate these +spans, then a subsequent txn conflict check would not detect that +reads performed by the LeafTxn are stale and incorrectly decide to +refresh the txn (bump its commit ts into the future and retry +automatically, instead of pushing the error back to the client). + +Another use of repatriation that's not strictly necessary but is +nevertheless a useful optimization, is the case when the transaction +is aborted concurrently (e.g. if a deadlock was detected by another +txn). If the KV reads done on behalf of the LeafTxn detect the txn +record to become aborted, this new state will be repatriated and the +RootTxn will know that the entire KV txn has become aborted. This is +faster than letting the RootTxn discover this state later at the first +next KV operation launched on its behalf. + +Related issues: +https://github.com/cockroachdb/cockroach/issues/41222 +https://github.com/cockroachdb/cockroach/issues/41992 + +## client.Txn, meta and TxnCoordSender + +The two sections above used a simplified picture using +a single "transaction object". + +In truth, the [type +`*client.Txn`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/internal/client/txn.go#L32) +is merely a thin facade for the SQL client. It contains, between other things: + +- a type tag (RootTxn/LeafTxn) +- a reference of [interface type + `TxnSender`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/internal/client/sender.go#L57), + which abstracts the `Send()` operation to send batch requests to the + rest of the cluster. + +In particular it does not contain the "main" txn payload including +commit timestamp, intents, etc. + +Where is that payload then? Also, where are the refresh spans and +other in-flight txn properties stored? + +The object referenced by `*client.Txn` is an instance of a coordinator +component called the "TxnCoordSender" of [type +`kv.TxnCoordSender`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_coord_sender.go#L104). + +The TxnCoordSender (hereafter abbreviated TCS), as its name implies, +is in charge of maintaining the state of the txn at the top of the KV +layer, and is in charge of coordinating the distribution of KV batches +to layer underneath together with error handling, txn conflict +management, etc. + +The TCS is also, itself, a rather thin data structure. + +Its main payload is what the KV team actually calls the "txn object", +of [type +`roachpb.Transaction`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/roachpb/data.proto#L302), +which in turn also +[contains](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/roachpb/data.proto#L310) +a copy of the "txn meta" object, of [type +`enginepb.TxnMeta`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/storage/engine/enginepb/mvcc3.proto#L18). + +The separation of purpose between `roachpb.Transaction` and +`enginepb.TxnMeta` is not further relevant in this RFC, and we will just +call them collectively "the txn object". + +With this in place, the interaction goes roughly as follows: + +![](txn_coord_sender/txncoordsender.png) + +The txn object is sent along in the header of every `BatchRequest` +produced by TCS while it processes a `client.Batch` from +SQL or other KV clients. This is passed along the +transaction/replication/storage boundaries and the low-level MVCC code in +storage has access to (a sufficient part of) the txn object during +processing of each single KV operation. + +Additionally, the execution of low-level KV operations can _update_ +their copy of (parts of) the txn object. This will populate e.g. the +list of observed timestamps, used for later txn conflict resolution. +The resulting txn state then flows back to TCS in the +header of every `BatchResponse`. Upon receiving a BatchResponse, the +TCS *merges* the received txn object in the response with +the txn object it already has, using the `txn.Update()` method. + +## Interceptors: between TxnCoordSender and DistSender + +The explanation above suggested that TCS sends +BatchRequests to "the cluster". + +In truth, "the cluster" is the entry point of the distribution layer, +the overall architectural layer immediately under the transaction +layer in CockroachDb. Its entry point is an object called +[`DistSender`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/dist_sender.go), +of which there is one instance per node. + +The interface between TCS and DistSender is an interface +called `client.Sender` which defines a method `Send(BatchRequest) +(BatchResponse, error)`. + +So _conceptually_, we have something like this in the code: + +![](txn_coord_sender/distsender.png) + +However, there's a little more complexity hidden in there. If we had a +direct call from `TCS.Send()` into `DistSender.Send()`, +then a single blob of code in TCS itself would need to deal +with all the complexity of txn pipelining, parallel commits, etc. + +To facilitate reasoning about the code and to ease maintenance, the +txn management logic is split away from TCS itself, and +across multiple other components arranged in a _pipeline_ placed between +TCS and DistSender. Each stage of this pipeline is called +an "interceptor" and responsible for a single aspect of txn +coordination. Each also contains additional local state. + +Two example interceptors that happen to be relevant to this RFC are: + +- the + [`txnSpanRefresher`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_interceptor_span_refresher.go#L103), + which contains and manages the read and write refresh spans already + mentioned above. +- the + [`txnSeqNumAllocator`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_interceptor_seq_num_allocator.go#L58), + which assigns [sequence numbers](#KV-sequence-numbers) to individual KV + operations in batches. + +Thus, in reality, the call stack looks more like this: + +![](txn_coord_sender/interceptors.png) + +TCSs allocated for RootTxns use [the full pipeline of +interceptors (6 of them as of this +writing)](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_coord_sender.go#L529), +whereas LeafTxns, which only handle read requests, use [only a +subset](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_coord_sender.go#L556). + +## TxnCoordSender state and TxnCoordMeta + +The overall "current state" of a TCS is thus distributed +between various Go structs: + +- the txn object (`roachpb.Transaction`), +- the set of its interceptors (each interceptor contains a portion of the txncoordsender state + sufficient and necessary for its local processing), +- its "own" very few fields, including a summary of the lifecycle of + the txn object called `txnState` (relevant to this RFC, we'll come + back to this later). + +This overall state is a native Go struct and not a protobuf. However, +[as we've seen above](#LeafTxns-and-txn-state-repatriation) distributed execution needs to take the +"current state" of a RootTxn and carry it over to another node to +build a LeafTxn. + +For this purpose, a separate [protobuf message +TxnCoordMeta](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/roachpb/data.proto#L617) +is defined. The TCS's `GetMeta()` method initially +populates it by asking every interceptor in turn to write its portion +of the state into it. + +(side note: arguably, the name "Meta" here is ill-chosen. There's +nothing meta about it; this struct is really a mere serializable copy +of the txncoordsender's state, and would not be necessary if the +TCS state was natively stored in a protobuf-encodable +struct already.) + +Conversely, when the state of a LeafTxn is repatriated and to be +"merged" into the RootTxn, the `AugmentMeta()` method uses the +`Update()` method on the `roachpb.Transaction` sub-object (which +merges the state of the txn object itself) then asks every interceptor, +in turn, to collect bits of state it may be interested to merge in too. + +For example, that's where the RootTxn's txnSpanRefresher interceptor +picks up the spans accumulated in the LeafTxn. + +## Summary of the all-is-well path + +To summarize the previous sections, the SQL/KV interface +involves the following actors: + +- a `client.Txn` object, which doesn't know much, other than... +- a reference to a `TCS`, which stores: + - (a copy of) the current txn object incl `roachpb.Transaction` and `enginepb.TxnMeta`, + - a set of txn interceptors, each with its own local state, + - at the end of the interceptor pipeline, a reference to the local node's `DistSender` object, + - a little additional TCS local state including a "txn status" field called `txnState`. + +When a KV batch request arrives from SQL through `client.Txn`, it is +passed through TCS, the stack of interceptors, delivered to +DistSender, and the responses flow back up the same path. + +Now on the next question: *What of errors?* + +## Error handling in TxnCoordSender + +For simplicity in this section, we'll start with the simple +case of a RootTxn without any LeafTxn. + +When an error is encountered either in DistSender or "underneath" +(remote replicas etc), it flows back through the interceptors back +into the TCS's `Send()` method. + +Some interceptors peek into the error object and update their local +state. Some of them (like the `txnSpanRefresher`) fully absorb the +error to turn it into a non-error. + +Additionally, some interceptors can generate errors of their own +either "on the way in" (towards DistSender), which causes a shortcut +to the return path; or "on the way out" (alongside a BatchResponse). + +When `(TCS).Send()` receives an error from the chain +of interceptors, it then separates between 6 kinds of errors, currently +split into two groups: + +- sub-group 1: *recoverable* errors, which cause the TCS to + perform partial or full error recovery. + + This group contains 3 kinds of errors: + + 1. *recoverable errors with in-place recovery*, where the TCS + will handle the error internally, then retry the operation + in a way that's invisible to the higher levels. + In this case, the txn object remains "live" and + its "identity" (ID, epoch) is unchanged. + + For example, txn refreshes are processes automatically + in this way. + + 2. *recoverable errors with txn restart*, where the + TCS resets the txn object to a state where the + client (the SQL layer) can restart the operation, + or tell the client to attempt the operation again + (client-side retries). In this case, + the txn object remains "live" but its identity + (epoch) changes immediately. + + Example sequence diagram in the case of a recoverable error with txn + restart: + + ![](txn_coord_sender/erecovery.png) + + 3. *deferred retry errors*, where the TCS remember the error + has occurred but pretends the operation succeeded for the benefit + of the (SQL) client. The error is only reported at + the end of the SQL txn, where the client is requested to + perform a client-side retry. + + This is currently used only for `WriteTooOldError`. + +- sub-group 2: *unrecoverable* errors, which cause the TCS (and, + together with it, the `client.Txn`) to become "trashed". + + This group contains 3 kinds of errors: + + 1. *transaction aborts* (`TransactionAbortedError`), which occurs when + the transaction gets aborted preemptively (typically by another + txn, as a result of deadlock detection, prio conflict, etc). + + 2. *transient processing errors*, for which it is certain that + further processing is theoretically still possible after + the error occurs. For example, attempting to read data using + a historical timestamp that has already been garbage collected, + `CPut` condition failure, transient network error on the read path, etc. + + 3. *unhandled errors*, for which it is not certain that further + processing is safe or sound (or where we haven't yet proven that + it is). For example, "ambiguous result" errors, "store + unavailable" and internal assertion errors fall in this category. + + When an unrecoverable error occurs, the TCS changes its `txnState` to + `txnError`. After this happens, any further attempt to use the + TCS will be rejected without even attempting further + processing. At the SQL level, this is then recognized as a forced txn + abort after which only ROLLBACK is accepted (or where COMMIT will + produce a "txn is aborted" error). + + Example sequence diagram in the case of an unrecoverable error: + + ![](txn_coord_sender/eunrecoverable.png) + +Summary table: + +| Group | Error kind | Example | Current recovery | +|---------------|------------------------------------|--------------------------------------------|-----------------------------------------------------------| +| recoverable | recoverable with in-place recovery | `ReadWithinUncertaintyIntervalError` | internal auto retry, txn identity preserved | +| recoverable | recoverable with txn restart | commit deadline exceeded error | tell client to retry, reset txn object to new epoch | +| recoverable | deferred retry | transaction push, write too old (?) | store error state, reveal retry error only at commit time | +| unrecoverable | transaction aborts | transaction aborted by concurrent txn | hard fail, TCS becomes unusable | +| unrecoverable | transient processing errors | CPut condition failure | hard fail, TCS becomes unusable (see below) | +| unrecoverable | unhandled errors | store unavailable error, assertion failure | hard fail, TCS becomes unusable | + +The keen reader may wonder why transient processing errors cause the txn +object and the TCS to become unusable. Indeed, there is no good reason +for that. It is actually silly, as a SQL client may legitimately want +to continue using the txn object after detecting a logical error (eg +duplicate row) or transient error (eg network connection reset). **This +is to change with the introduction of savepoints.** + +Another important aspect of "recoverable errors with txn restart", +which will become more noteworthy below, is that the txn object stored +inside the TCS may become different "on the way out" (back to +client.Txn and the SQL layer) from what it was "on the way in". It is +currently the responsibility of the client (SQL layer), which may have +its own copy of the txn object, to pick up this change. + +## Concurrency between root and leaf + +Today, it is not valid (= KV/SQL protocol violation) to perform KV +operations using LeafTxns concurrently with a RootTxn, +or use multiple RootTxns for the same txn object side-by-side. + +Note that while the SQL code is architected to take this restriction +into account, *it is not currently enforced on the KV side*. We +sometimes see bugs (eg #41222 / #41992) occuring because we do not +have infrastructure in place to detect violations of this restriction. + +This restriction exists for 3 reasons, one of them actually invalid +(a flawed past understanding): + +- KV writes must be uniquely identified, for txn pipelining. Since the + identification is currently performed using a single counter in the txn + object, there cannot be more than one TCS using this counter at a time. + + Today only RootTxn can process KV writes, so this restricts + write concurrency to just 1 RootTxn. Even if LeafTxns had enough + complexity to process writes, concurrency would be limited by this + counter, until/unless we can guarantee that two separate TCSs + generate separate operation identifiers. + + (For example, by combining sequence bits with a TCS ID.) + +- RootTxns update the txn object during error processing (see previous section). + + If we let the RootTxn process operations and perform error recovery + while a LeafTxn is active, we'd need to answer difficult questions. + + Consider the following sequence: + + ![](txn_coord_sender/mismatch.png) + + In this situation, at t1 the RootTxn handles a retriable error + by preparing the next client retry attempt via a new txn object, + then at later instant t2 is augmented by a LeafTxn whose + state was part of its "past" using the original txn object. + How to combine the states from the two "generations" of the txn object? + + To avoid this situation altogether, any use of a LeafTxn + comes with a requirement to not use the RootTxn at all + while the LeafTxn is active. + +- (Mistaken) Expectation that distributed reads are able to observe + concurrent writes on other nodes. + + The KV read and write operations are mutually ordered using seqnums. + If we were to expect that a read is able to observe a write + performed on a separate node, it would be necessary to synchronize + seqnums across nodes for every KV write. This is neither practical + nor currently implemented. + + This restriction currently mandates that there be no LeafTxn active + while KV writes are being processed by a RootTxn. + + (The restriction is lifted by observing that the expectation is + invalid: PostgreSQL semantics require that all reads performed by a + mutation statement observe the state of the db prior to any + write. So there is no requirement of read-the-writes inside a single + SQL statement. The current crdb behavior actually is a bug, our + [current halloween + problem](https://github.com/cockroachdb/cockroach/issues/28842). + Since LeafTxns are re-generated across SQL statements, it's trivial + to get the right semantics without a restriction on LeafTxn/RootTxn + concurrency.) + +The astute reader may wonder how distSQL deals with the requirement +that no LeafTxn be active while a RootTxn is active, or no RootTxn be +active while LeafTxns are active. To make this happen there is code in +the distsql planner to select whether to use _either_ multiple +LeafTxns, one per node / distsql processor, _or_ a single RootTxn, +shared by all distsql processors (and forces them to run on the +gateway, serially using a single goroutine) (also, this code has bugs. +See eg issues #41222 / #41992). + +## KV sequence numbers + +At the SQL/KV interface, KV operations are associated with *sequence numbers* (seqnums): + +- write operations generate new seqnums, which are stored inside write + intents. +- read operations operate "at" a particular seqnum: a MVCC read that + encounters an intent ignores the values written at later seqnums + and returns the most recent value at that seqnum instead. + +Today the TCS (the component that receives KV request batches from +SQL) is responsible for generating seqnums. + +The seqnum counter's current value is split between three locations: + +- a local variable in one of the interceptors, called `txnSeqNumAllocator` inside the TCS; +- the `enginepb.TxnMeta` record, inside the `roachpb.Transaction` held inside the `TxnCoordMeta`. +- the `enginepb.TxnMeta` record, inside the `roachpb.Transaction` held inside the header of every executed KV batch. + +These three values are synchronized as follows: + +- The interceptor's counter is incremented for every KV write operation, + and the current counter value (with or without increment) is copied to + the `Sequence` field in the *request header* of every KV operation + flowing through the interceptor. This ensures that: + + - every write gets a new sequence number. + - every read gets a copy of the seqnum of the last write. +- The `Sequence` field in the request header of individual KV + operations is also copied to same-name field in `TxnMeta` of the + batch header in certain circumstnaces (most notably by another later + interceptor, the `txnPipeliner`) for use during txn conflict + resolution and write reordering. +- When a TCS is instantiated from a TxnCoordMeta (e.g. forking a + RootTxn into a LeafTxn), the counter value from the TxnMeta inside the TxnCoordMeta + is copied into the interceptor. +- When a TxnCoordMeta is constructed from a TCS, the value is copied + from the interceptor to the TxnCoordMeta. +- When a TCS is augmented by a TxnCoordMeta from a leaf, if the + sequence number from the incoming TxnCoordMeta is greater it is used + to bump the current TCS's counter. + + (Note that this last logic step is currently never executed—i.e. it seems + to be over-engineered—since LeafTxns cannot issue writes and will + thus never increase the counter.) + +Final note: the seqnum is scoped to a current txn epoch. When the +epoch field is incremented, the seqnum generator resets to 0. The +overall ordering of operation thus also needs to take the epoch into +account. + +## Seqnum consistency across TxnCoordSenders + +The current code was designed with the assumption that a single TCS +can issue writes and assign new seqnums to requests. + +Today the code is organized to use only a single RootTxn (and no LeafTxns) for +SQL statements that perform writes, so that anything that may +update the seqnum ends up running sequentially in a single goroutine. + +It's interesting to consider how this would with LeafTxns if +we were to relax the restriction and allow multiple readers +with one writer. + +The main mechanism that helps is that without writes, a TCS will +continue to assign the same seqnum to every read. A LeafTxn forked +from a RootTxn will thus continue to use the seqnum last generated by +the RootTxn before it was forked. + +So if we have a SQL sequence like this: + +1. UPDATE +2. SELECT + +and the SELECT is distributed with LeafTxns, all the read requests +performed on its behalf by other nodes will use the last (epoch, seqnum) +generated for UPDATE and thus be able to "observe the writes". + +The astute reader can then consider what happens for the UPDATE +itself. What if the UPDATE itself happens to be distributed, with some +LeafTxns on other nodes running the "read part" of the UPDATE, +and the RootTxn on the gateway issuing the KV operations? + +Here it would also work, as follows: + +- at the beginning of the UPDATE's execution, _before any writes have + been issued_, the UPDATE's LeafTxn are forked. This ensures that any + further distributed reads by the UPDATE will be using the last + (epoch, seqnum) generated by the statement _before_ the UPDATE. +- during the UPDATE's execution, the RootTxn increments its counter + to perform the mutation. This increase remains invisible + to the update's LeafTxns. + +By ensuring that the read path only sees the writes prior to the +seqnum at the start of execution, it will be unaffected by subsequent +writes. This solves crdb's [current halloween +problem](https://github.com/cockroachdb/cockroach/issues/28842). + diff --git a/docs/tech-notes/txn_coord_sender/distsender.png b/docs/tech-notes/txn_coord_sender/distsender.png new file mode 100644 index 0000000000000000000000000000000000000000..503cd446ad45832550be387c6fb68bb36800400c GIT binary patch literal 31451 zcmeFZXH=AFv?huXMM?~ySR_Sq5RnW5Dj-pEkc=QXO3q*^6cj|FBB97R=O`iqMb1e; z1e6StGj|tYx7~B*&RTcQT{E-h^pCdNg0H^tzWdov*xyT8X;B0o5)TIl2O%ydERTb8 z@``g*7EBs z2=t-EwJViT>}xJBk%ENG{v;a7Z-s}6n2MiN({`%8qI}{pB!)*HIzW}!J$SWXec_S^ z4|{%2zz1uVB>otNR6m4-(u=L=@t&P!YucOg;pDe8jczAiurWOQBX9Qo<&D(KJ3il- zo()Zpy%|akWOuylDroJo z{8{gK)U@L+-N^@M?5e0v=vhjoP?#WEnV*RxQ4{x0lvNU%lvb5m%3QsUnIIkB_O6Yp zLb)~$_lh$zNIl5D!!zyS(eL?K<5b#15t$H#;mFnD6`x%?riY32cnx1%B6|gtC5qlg zZ6d4v{l3a*@V|4I+e!}$$T~H3>?&iU`F2fh{ukY#Phw&P4+5sgZO(tb-AypJb(q@3 zKSy9^G(DG)N##^q{QQmWNm{ed`ooU5nWG-^izqDxXTI7#6h7q}ZmN+43&qh>T==%4 zW8ZYysr!m5?-LuVpWTTeJP~{AHj@3YasQbKoO8Dcjh=*rekLEW(|sE;N_SV()}Zh7 zy*Hm{@4g6OWABr*VWoP)RVp0Fcj73^8A7HruiibB77TwUZD?R^`+?_Y=;zecNlx(5*4UyvL0I5<5V`7N^1csLHM-=fgp?sE@RM%W+qM149# zkxEQ($*M&*v$i+KFw*JAC2j4YvsdrlT>JJ^K0PDD@FUq1K3zIocEVZxrR#?g$Jhyv zzbh^-&eE%Kc6Kh`Ug;0HrujBfer07PU*$YmSIiY5o+xz7TnsJ}&xj5mDI>U@`|7l1 z=`U3AkL7Od5RT@Xi?4>~{R0BdBaxG{v*hID?(^SnOXh7)8wv)z*3{5&b#;w!Tj{M2 zVNp+tEiKi^C*?2+ELK%f3E|Gl$+1yYeb+13V8&KlQo_c@Ml1b+Zc6CR9rvBZ$a>`R zQ){Dzu3G=A6z*%oU%zTpwa)kDnJ&zZ)&-1du75MlD5EiTW{^U>b=B!4PR%Aoc=#qL^-{2M(k{GDhKpVYZl`KOOl4kNu$YDfyE z*TIh;KVIk^sqvF^>d!ZqA1rq(3G^f5qZvDSo@R;rxY{MJAelO(>fGdHU&3vY7FlBQ z7c3HjmPSUh-D0X;*n5jQQmG)rzg$NfNf+~qJ0PyFD_zs&N5iEut=)R<9r=8Xa@V3| z3HPNK3UMAjKJ{L+xxupR&^K>%jHB55U%h&Tuj)Em^m8hf>M9vp%=&Uxl2y62S-o<6moMMKlsL=`tR z6L({@d}A_2URqq-KYJ@MJJmnCw10%yNS{<$V7Y;Vc2s8-0 zQgJ?oMpIjxMPD_hl*712vzG3GIcN9}U`RaGP9 zo3o7c^sKMYstAt$H7{~!u?wBO+Wu<-+cT;0@pqBmrluGxq>b|WU=fqA8YNYJR62yn zb3x4f8kChBc^cPh-5kL$FD)%yZas#9rBqg0`gLHSzrR0-9#!VDy24GViha6nTfb`f zIgh};Dd?4DCCQo86M1`@LV%QnWNKo9e#9sH9dFD1Br0BBH4H8RxtXbHz%NE7rc|!u z28D^OMzKsnVS#*nd~r<6be7_kvf0bdWy8306}#>+w^f7Y22G9Z?ep==F~=3+rR2!^ zFkCqJ7t;zdxfU;ZEOL3oq=wwF{G|WKsWb1Lx%?}I!E>8TbaZcmHdkf>B6rvPcDe1R z?{jf+ITh5Lt;2Ae_5Bzm=!0!P|1BvYVzRLeU3RJdiysAxW!Z9o&t?1g(y>mn=Dy*X~3vfC247C7=%#Suj^NC-qdif3Foy-&d9JdG&I!aVCU!8EaFleu)KR0wGcd>7vv%o z6ejVrDct4P_v*!;&9J35y#%+=y=9ENB{n~sWCYfQj}BzMEi5cNf_wUfJq3@ou(J30 zRdFHs&49OW-@kwV&@8AgRrtR2>ry2Jg%~D`v`C?-FR9ay)3dX)m6es*Z5!mw9K?kT zQD~#YxL=hRqa;@z`C1HSW2w2oGMVeu3KWN=-rUAAioMP8FdAw=pr4;#N=nL>>&8Sv zr#k*-EE>w!gZuY;+*SvE&dp``3YR4bFG;Ijz0+II+F_NIk|OE;C}dbZruBtaU8vF8 zz4hs7vqv0b&D-h)797cj@az{Pw~i3p{5CTa9vVtFlF?g6ann&&Lv=Ee-MB5oeVMUJ zV$fsPZ9UEdd&fone7(JSVVy*>FSYXx#mK8CI_3-`a8P23u=qRt7}v@*65=tD(bsQLBdhk>e@v9W)O=P>gezwhH<6Hb?+iLoez5#24zvwlnS z^JfSY?y`!;=jJM+H2saU54XtT+!8uuR-b3uZ9g~2Jey}dCQhKRW=KBS~f5qPW#E!SE3eiSK0b8uM2C3n2VZrh!%`S2K^ zp(hH^8VdDc9Bju2Ac;jtPn7ixj;DK9OS;Up0(YynRxQmm^H?7uQZ`00m4)`5h0pd# zjB2J`0?jAko~GlZi%14L@!nRgG>Q-3)GUtGG~l%bZRJ+n%nCJSb+(ckN-cTXs8(6Yk( z(X;LlCEUXF)odGHY2`QRqX}g{UZ!yb=IE42-+T5lGaSKfG_L(=^}*ey;VLhgh?}ZP zO3S}$D71@hUrO?Y2iQ^z&alX|v<&wjoCGdnTXJnBz9QdpsGLAxEh;uvn9}jrcae^O zCh_98kt-pzM&t7{2WRmVwL|S{xjoZ-3J(u9($%wsAs78c`6oWaGOinYI6Xm~9{Tev z3M_@XGCa0TUq%&f!P!hrPRh#g(qb=9J0c2QRiJSG0i~JYk<-YHka;5gA3Hy|D*R8( z4wk74mnSFVEyvoth3nfMJBj3V<>NR@^U0nQx(|)+qK1I(%VCcv=3!fu{kr>p+Idv0 zaqSnuHH}cmm1zNX+d|XC_iturEzZ-d@vo0Iz}>5!>2E9(q`ymoKXnrhJbg|}E{l)X zS;8+;4+ua#L2p4)>{n|3$6fIJB<8$4u)fS=i?i=kt zcV~tsQN^g!x4v%j6T$NyF>5c2BrEzT6DeRfhFPDP@!FZHXmZGMl6 z1Fv|v8Bf#E!KX#j?VE41_1D)6eP8TQI2nCWSsGLJtPgYRfloW@JaW?LY;d=Vwo?70 zvk&@3?rYr`uWR9|MKWY9E-nhVyVg|fZuUxscvF83W?!1_brW2xq*|xHMoD{_hX;CB zm6U#1S}KauJ3qoUc9$!fm{b=&f3K!@;3wIT*|n9pqpV({f_eRVPC_b9ddg+ESU@KS z?eNGxUg}*>%e-&g1_4D=0N&Ws-Cq=jV+nzAF)M4q)7P#Hd$kekXXUaB($!WJZ!q_| zD0NyMK4NE*Q?Xk1XnEN(Y+!jHE;qB=9gR1+#jeAz#b!xuC;eZap1*vWil+QgUCmQg#Kq3aI=QvJ+KDJ% zi?;p#8OJW{#}9&?whb1>FTTr#A%d8CaWbwZ6Hl*BS z%<`zH@8~5pW3%b2{N>aorsmA@L*70viZi(jZ)K7igR=_?R%!&L`^w#7j4->~6&E`j zXRpPjP1id=M?kNLy@|gY&7h9t++3~5aQ_+JxZsnzwX;z+_j+?}WObJMk86lUUllcZ#kuC? zai9L@SX@YiKk3?xWqoSLb=Sp*RvaCeVb_&jQ5ix0`kESrq8~|9%;AA0r}CB`f4Y#M z=gHA==<3#wFp>yIO6BeP8)s)%PwcLGL{_*KansW~uP^Yx)3Vkn>yY(7S#@81E?_ks zy|7#F1ORGn1r1abqAVFK=9>LaoH=LG*``($b|t{Cic5GRIYf(ny;c-uYYS?fmo* zU>5ilx!c<@B3?8lyUH!QG&e!4gRsjzVL7l-V6DMr=`QumoHw>sQh>vGDe-of?8#r1 zPt{#w&{k3&O0JCZzvg}ee$E6dZRI~yj9 z+H&s&+$LLNERj3t%6Y$q;K3Vx;XxJ3#AWy%^`#|&`Pmr)N@B$;87K_*B5Z}N?-w)f zpIl3bjF_KyjN%e%pT`7LcFHn(v6%#Jat(Dbj=g`+&mRGI+0OE~_YOZ9+@sS(!w!SJ zq~{lpEJwY_oIw#fX;!!|T(hv4HbkT)Z!{J@DyNm1-GZLlW!*Z(FlbhW1XJyap9)X#V3|+>RQ?#^uApwtk#WGkqtq3}z=a>zAgWpe0X@(fFCxmJ)8h z1(JuhOMw@)?zarOk3;d+!v|n2sBAaq3jT{q>Y$ z(hAbBE$eR+JT&z5#nHMBew9HOY%K$_r@SL&xg-`vv8D5tw<2xJf~qlK*=StWQKx+% zJRHEIslk<=on5R=DuKjwY0Y_^;~Le&Em+9$1ZHp(CT0hnr+&7H%#~Pv{`~R~N+vro z!(%gJhe6!E+uFe)jGcZyLC{T0&4q>DTwmKQ`HgLvC_-WpuMT-ceuyFC+exnpu)x5bdp22=SGI`k>+I{=s608g49)A{Qup9S zf8<>M8*0AHpWXd)k2DLOJT=(sjWr{Bq_Ms6?C;)KO9;W^x;9_#wuy&_7dtrA!)N`< zI@Y?>y2H-SE-`_clwq$;Z`yCuiEH1c%~7<;im?{BWkYM?`kA^C<9-|2|Jp!M&G=}jaRmM=)ahP1T{?$d0Xq{)sb^jy&*qeb;x7DBO-hGpFJOq=n_z)xRkF@hu`Sj@ctFW#1BuUB6t@A}fukBY>^e z)skasQ_FU_(vwt3Xkv#-5P#|ffg~_2pjbH5E+20T9m2t~*8>)XhviQ|py1!!!6@se zY`>5FmICGS%t_}w&dKAW#7K!%TI|xIe);(55o>(JSjU?!tWb1|;`lA0R@uvceipYG z0H2*^IO4RK77);W=B7C+O+rRSMoi4=FC}&7WcMzir@mrid8c8NC1bUaY`}!ED6|e8cv`<7het-G> z`}ZAlB?}7+UESzm=VKrWSX#~%+5F^WXaBl+NUiVgK)7h_}$9IUF+8~*g%+}u1lZ}i6xOfYB(-(}+B;xHRcO-=gs!Nz%gA3uI<4C5m44v%xr zH|vdVpZE|S{<`lfv$453M}NnTiPEiGx9sflsEGo~BuN6TMMOj-B)mJWJ1>ooj*fyR z>~3;BJjp#1q=RQz!L6o-mXWdEIR)RrVSPd0PCMAQsZEZUqogJ@EW)V$$l1}0IK5}X zN%yFC?og)fW*TTJ?(XGU87z#93JH~*=6&pL8%rRF1P2EnJ9VLS{?n=CDJp8}&aN)+ zuc@Fz^yM03HdM5ns5v?Fv$C?#=qn(Fbs`y;sc2|0EAcZg{rn=?45c4E%9QQl^B; zL8qu>M1bP69Ft>97|h-w&u<`;J7JcXtIRRtE`A+Hoj`YgUwCmD663@xAYv!TL~{0D50^IV8l;g5jgq!UDx2mj zp*GKP5X$7a+Xnl7dW%v^u9<$x*(qzBTvFm9NJH$IhRezxs35fPR7j*JoqcKT*NC{d zxLO(~9i7ooxf@2}5si?#HQb_f0ZM!4GA(+^N~^(Ed45njat6NvZ*8qih;&zT%a-@Z zlbFs=PR68qYZh8PSf1=;Wo5({S8e*A#?794!RPBKPa zO=P;E38mI5`#i_6S)xJTPkC>>HBE4Ga4ZwsJGW8kz|1 zM|U#1eZto^HW(P$Zrsq-)2ntDw)G|FPZk)geX9mLrMkMBWAL^>V)~puKgyKcY1F^8 zv=mlnQ^N5)WD zh2Mg5x(DXlo^kz_aC<*}`oxaB=jiD8-~l>2yQG8<>9De#5#2}6_eNQpuVN0_5I9j?=fT6OcE~=)!pch< z)xtir4KFcMFFYnIS=l=4WlgDPRQW0a1^A~cOO+~>scx%LTPRN|YC;r?bTP(PD1{vRu8y1>g zBMN6Azhub1E&k-t)4xFQSmqVDMM!PN>)|^`v8W38gw;7c58(pLVjYw}gvqBTxwwz~ zhS2o~g`%OjoQHm6?0<{jbV$6V2XIWpv44D4EQrPSs85Kd>n~0QM?^$`O++C&IXe2# z$7Ez=gv;Wq7$u_p_Nxp!|H!>1iX-HopU(;F34@Wq?UtRQU}9ukTv#Z!9>Y7yMmqfX z9v1%bMXYUX&YwT;l_AiVNUrdGWW-^*=k_Pi@A$w+KQ>Mwub=?G)Y3v;S6BCr%aZ)$ zQ>@5})oMLXoH&t~p00dxb#91)oIJhNh(`KDp?N>2zP_kvjZHKEPdKfa8RlcEis6=k zkp%opZ_EUZ+hU`mDIetlf^%^dfG?1peM3J7!0_KN89$py=>t1wy)(l-u6z^xGV6)9 zIE+D2QBm7<2M33wirm}|aOzBQ2xHyTlo@d&f`Z~KGA=+JY-?+S%`%pg{aDRLUteEG z=kmc>yR^$%SfMH~EFdsYQC*#JIb$Cxl_#@0esrd&)YQ~)4LQX$C;R_RU$Ws#&6Zs{T+x84u zlp&~*%U@IiYh|#{zJ(llFQPJv3h*cljoMgW*EcZe>h317IDh7h71)Sz8ft3E(s}FiDF4lyHwz2aajvspiyn=9QSFx1XX~A)XQ^Nkj7zo9%XiONzHgF~ zs*+%=e?$Qhe*M9dUMJ<=7${yv?4-D??h`S$a5!t+wOsmaL_nm66I9OdQT zDyJau@cIfYBa@T&pz8L8#^RVbkzXQJf+Zz|g-eO(PwzQQACAd*Y_yY4ah0^mhs8zh zJl5(8Gf5oAMI-_G-1nC>f+fDcM7INU_CbY14H$GDzsm{~@x){kW6oSIB$AX=LtWjp z>^U?@C?8sxh>i){W5ZGo&?ssU#mYmCs5;HO~vaO+TPPzpHbY}O^A{uLJ-}w0~<`ooB@H=-*FQNfa z6iva%3%lvG0JZt@SOL({VJ`}0@}6E=L|%tMm5b2G_wQ01ibW}T@$p~A2V0&!JG#uL z4&H-_Cw%#4>%kI}6WTB3`dP5lnV6(vTAz7&UA%CCfw5XbCH4NW8!mmKN6u^<%HF}j z(7*uo>{($+iOjnj`m3`8@r^yX#_gf3dVzj7mpc@&#`;EAv-?WU^HUdBl!Tb`RPmYx zHdDdXzIDs9zz|ur}q{oia;V01LV{667TW|dR z(d`poe953;Zbk~E;$h3|@7jlhAUgw<-7H+^?ab57?@T>zKv4F|^>qTX!^GAn@1g?q z_8T?P7uS(kFi0Rq#I2s)Uu(CX_zwfiz$9H+KZJdd0#c|WSc3;!i^sL07H7^K)eEIL zU^%{L9+3AO{WzC>;QheC0CDiSjp#h7i~E;;4!IUg1=JhS?XenY^^a7JynN18pepw(7LNh^J)W zjg!1MrzKD}&^llbg*_i`3ySj4UGx^|=Zl7qr;h%0$(6S^=YtkD+E21v9N^MastPz; z?>jEvS3h~;EcVyVJfNJ8LSmr)Tb=Cc99YZoxXF8M*>jO%_nOBvI)PO#bM7D2{}lAD zwbYZDGB;@OR`B2vK+)C_vo^F|8ttGC{p-#=^(@a%V!b+acUR;jmI@pvg-7ZMDF^`Y z@yOxhhJVn=)f0=bOwQrgLPxvbJxZM0N4CEo8~z?C#CDdyKLg?jkCaRi7RNqK#X?yX z|EHUoCvgsX2vP!C+Jq~E$xBO1u$^%6pV$Es0i?Y+Sb9-tPGhD2kQ@G9fnMz`U0q!v z+U`7bbYd~B`n*hPcM>!BV5x z=2dRy4_#fie98Fsjz}sdR8)Csn!a*Y1iRlMtD@?kdeuA`ErBj7;$dcHrliyrGd7ZzZYp(JoHoNYH?JP=)jfXp z_cx#37SH@Yj4b+xBXnl+^>&|86si{)o$NQmbLDeudF##TR zf$2UFMwZ0%0~oZeZjYy*(smjP>gnnE^5qK(g&H3>QZh~L2mlYeKP(@s4L~mn zt$w@%-HJ+zaCKjQe%QC#Xt1cI4~DKPj&So-}e-PULk10va6839TY zMy!3Bw5)uo36&1jaIzz*%MTdu{We+2*B_v{*bd#rBJ3Bk4 z$q^0@>*ilA$(`2jJ9Fuhs;sQe_Ba#S^SBM>PQ4Z3FcTwv{kYG#2WX1hwg+hUu4hH> z!!w-^WFRJ)G%Y(YR$rvQ%4rVOJEzV=#_1K3BN=wv`}f6mQ_7MQT&3CyCbf-?<`x#S zlanUK#&QMh#c@iXo2JbYi4fcEPCPEky4PHCvBc|>m2?3#0-#L5b6WlJV|273oUgm- zIBR+IXJRpIF~Pe|C@v*+^U?QbS{WDlel0I6DH}X|`1#8h@9h`kr2Ni`Zl(Uzc@mP6 zaFa|+pSsFoPYb{M=0fdNCnqNk4vz5f@C6%)gn$49M2odvHOJuHvPg_&hEFkRnL9aDeR~CKvgE&1O2k> z2UVzdY=)<6g8P>u*~Fz9cGzTETL4G%1be?~DSI980UT2Q$_)y19!uGykJJ)h8vLSn zHa<12dUcONw})et$gx-ci7`k9#cNwGm)QLHmzG}AHBZ za=$$a+T^jzl^>NgHs&8@si!8Rt5;oFxqv*g(EC^gOn&oa*S|+W9gRTW*N@ z*Q*Z-++YGZliCH+9(RS(DiKY$o>ZQ3Xrb&LKhorv)E-$ySF`4Q?xfV-D#W>dlPq^! zF>4b)W`uq2h~Gd* zG!QxJ^?(en%yUmzjoSqA7u$W`dUA+opQ&R7)+hpyEcd#ASE!w17FNf9!|+0tl^?dl z!XZ|RO-P`nq6&4Zud7RXpOTwQgV?Z zPpfXevjwOa3rhy0O8Og-JxO@!9aOi^pFfu)?+aQOj3r<0?^hFYX}4zV|6Efu*OT>> z=`gm3ifWGC141;~q2S&=+SO9xuy>%Wuu`&42G4~vXJj=s)2`Xu+h0@9mrCh8e*Cz_ zU}-NHiJPl)1Ox=#M(pQlrN8Ezhb;tq7yk-oR=4Ga+`wYO&~}WgvU2dW+@v-{C8s5e z7>Fn0vyJYjW$&!a^aGqjeFQ2Dr(F7I1f_~_8>Fu_x_D<_8BK+7nh1}_rL%@C1o^Jc7zJlyrbT;%{Y>vr}uNdzj@>w{`8F+VLAu} zc2@sO;{+Ah&CLx01=u*%MAHERJ+&Qs_WTlXZGBus@CStt6toAT}Ox{pYG0@Qo5U(vx zveO)jl8LEle_tPz94=5~?Oar^9*W&$`*;rAp)*I^h@+Z$K!-C(L_dD~7!)Nr3`k}1 z$jP5SL*Bhp)z-Eg4ZrIH221>$#O<^#KxfQN-8l|L#q69M1$vM5v5@mLQrA<^MbZkd z>>qpvT!UNR8!Q=)3J9O%)Qyu+)zu#l&mNzhm4*#O{mF>#%JGfsw<2wrE?BZsc9y>PEU(oy(rB!0zn2JIwU)!7vKzYw!+PO3Uz@Cblb?|2I zdoC+4fA0MGk>O#qYZt(hSv`Va$opZ+7|0bTFAqc>3I@;Y8S!`(j$2b!M@L7RdO@-e zh{2%2<}amEXEpprBMg5OzjN5ISt4Oauk7o)Hj6**>5uLX%(*A@B`BIY%7p@zGmLcW z8c_*c*ttoPJbQ;H{ZH*OPy8FGH84s zcP)NTgG^dges5y*FIw4FUoW{VIfYDC769Xs!?cS|S^MW1yB_LLwQ|!S5jF zvv1ho8tc?N;j%i5IDML&Rrgs%g@=Jbd>>UzFl2lWI48dP>7k9`z>pAi=RZIM#CfQG zI847Gq%q}_>`vsuId;a~1_kz|Od0EaEzw_ZuioJ2z~@~p)kt;ly?KiIMqSR5~jyxmHJM%8OZ<1*~UF1v>-^d!ji&MLDst z-|!MG)OcD@HHU!5?*H26WK{Jmn^b}z)}PyZpg4#;q5}}&+djE>Lx8w`^n40hqLtDf zS=~pImX<4r{&N+@TTdR7{k#uRe^_I^1)$MyWL+{aozg}GX)bN%Rmq>(XJcXq6#xh4 zD*}uxD7=sZ?m|p!xpp{HSE&fqV=$nfq-zx2xo`+h9jC=`UtHWY(b+D17nJhE%+eGw zbFV(l>^ucUg=Rgo^b`mH6+a}z3pF^vx$mc4_1S~aaW)Xmp*BqF8XEr0*00Z1 zGuP4i2Eq4xyAkCcyF2sqRO<4jZHGi0sC*v5jys(l$6kKc3LSZD@#QE6Y*&7^bQWDUn^h0 zFoLM~`t@rdoUN^`P?o2r!XhIxx~Je?b-cV4vwlZaN=hnF;?-c#6ZxXc&mkrY4oZ73 z1U5=xR71PMeH+@&3o2R_?q#Zo@bBzj0N#v@j2=7~24hqW(*LRVvy56PbC^STp;mmZ z4+FREnr5=aKI!W3>dN0|T}l1*$iK>JzWLr(=+XBDHT9B-5!_Vh@B*!kjgyl8&>5W) zT6Hxw!zTdJ7-5tk_l94O3xX(UdU{$BgLPq_ojl)QVwB!a7^6x2Xrhf8j4^<(gyDw? z#jo!t$p_!RCKM_$@z4Pl0ZnoO~l!zDz44Fis~K{V%qY6N?7( zi!B9>3aT&Ph(%MnqCD<8JbJ{Mg5C_Ld~JUvFfb6h5-?rpA_3Sq`mL*~3MhL`in6oi z5C2}Hzfp>V!wwvs$>?FT@rm{XW`0v~CL7593kV2cqOA#x{00rUNf?gVaVo~Nel2xU zxU}xES~8ThLf?thBBYUt*y=TmTwWuJD-~=;C6~zhGJT?MTt_H(-p=VyI=QKv303Ql)9oWsLb#gD@Y@ z2Ocv^tO~f#4%)W82tm($7rsOFX*9FUcrVCt-FDoluw_l{aqoo89suu$@Ue!@zPz`u zy15+tpG6U?7o?JVv5xDT!2tmZf}B-m=G!UgY#;YI+2AK;s#^OqCtB_!rV}WAdHg4v`T&8egxB9 z7dNdpe{Ze#D(Yp)7zC*DdYu=QIf zwOeH?4BSmsq?b&^FH^Vx=z35G^2}rEAj$Oc+W!TFQ zE5gheW}CwK96$=j1pE8@*Z7jgtOrTkByNQ5o%#KFeGYVQ*aU6;-9|9 zi@c`Mu$!qqWDGOUAU5IX5Fqh%hF+k^+nSQqij&Gxn{MxUAUGg^9zq?xIOsa(xv1Bc z;Olj*G;?u#P|6qVFsw9oYd6%W)vY~Foye4&#XgetE3IhjehdH=@bbdYuITygk$Rrw z9%lPB5U_}-!8kd7rDn(W8?lG5V%dQnC%7vM ze3#)t?odUgx6+nZBvKjewEnW0+_g|QR9J+t+)JwCcg3N&LrTDm50^hS-0r(SHLhqg zlZ)=X`hHQACCC_mJ2hU15F`bFX4i^YzF2M8+!x)`%J$=d!j6eHN#lWl73{WA!z5I{ zfkoWC1|2heXTb_Ih^o>6I`&X^$^|`Zr}N4do_@#U(JK-1LquAtl8pgYFQRg}&+VbbMb< z`alhV#v6trbRRy9y~m+Q@4DQna_Q0~uzs=56IP=0>XLol)ujwk!azm_2G|F=YBS@l zF)*hDDbcm0%ITzy%0KFNWNIo?ULRCHFqEOKm9ch?N=s`L6M7Dz!MlS>m;%co0FTpn zcneEQ8o(@o`v)$!p-%JRBVhNi$0GAW)%H(+!hC;J_0Q_<;i~Q7?XO57x~R(b3xX?%iW4 zTapLOD4T|zU9n*f6U6v;n9hdCIKWR)MWr5_ru)qb7?G=b36Fr7+(mA{XykCGiZ&z{ z1U0WOEG!r_N4RZoEN4{gIQM4j%O_+HHaE|;poI!72F@jDlsaaqy;SUR#HCNwEK%-F z>d9o{{0gBzXXsqxU#r3ofi@nv{D&)KHdyL}!Q2buyca1@79S51ihkvjV=Sffhs6(L zSCZ#F_Kq`fUF``nVHWoLBtlt;7SeT1EJ$TW<>bI2J8wlpe#9!Z_c}D=^iv{EFjE_} z09GHp9mANXrzaE-ckteZir#SZghQ$?Nka{QJ~2>V&#sJJT+^~A6dU#^Rzbem7sPrN z{uz+)rviD}e9!6p$ZFHjLeHX6nCZMc85#;RJ1}U6p^W$Dz{cx3Wnsv+Ov^G(?b2kY zxNO`!Oss-?FSxTd0`sAAT|Nm&WHR^<+2ut=PH>faHNN36i_*mFL4{G(|4|F z`7T2FA_PJs0s~_^O`bk^0zzg&Dxo-s$;kuuz-hV1&bW1aFVXh_dF`zG1j$h78_r@E z<6hbhI*=;A7gg`V5qZ&QJaFlYq!TuOgjvRd*7eZNpo6frD}Yu8^94LF`SC`x#2}$^ zh{8qiyTA~5VD-0e-}?I0ur(9E9Yi+hH1Q=7L?cpQp1gP=d!f2vKQa1Ft(sbiQ6+c=)N>A*NZw3 zWAETY94gEx(OWr%Xet>3M}vKTX5t`lr{sb)9)Uk~RDCF)Cguex)Y~PwHvVy4-Y?o{v3k=tCy$e&6_j56M^&!FwuJN4^{Ld|JZ$pu$`^2 zIeRy2CoxT;*(ZZH3$eBQ2of9)9I@-fTcoD^^R4fAGDug%%N1=+5;cjYwSI>Rad1wH zI)s(4c$7ZsVU#=G|J|SMc=c_l<5-$Q`l9OyjL-CFQ&_$Bf3Wgm=)6~1!14OqFgvxM z)dKy0OMj!CN8Co%#}?!watrrYS3^t>MVqgd)y-UC+2$RtX!XSs!SuBuH0w$DQb;7Arj;sZ z4S%0EKi9roPE}J}%<{v-(R%Xj_f0(?jTi!7ZxW63c2|1hrppM_GK-?;y~ zxP%Sqd7=aVhp2RFFUCiviH#G8UAJl=RdCBHDJjXyf{LXLuXxB!LNPqgDBoa!=l~53 z>K_>9uK*IQ2|F@QLr%^gc4DUqyg?!S!v|NeF=Y@2GMxnWc6OlluWxQ{uB|=P*H7vM zCqViGcPr#i-?kqCqIDO^;JVnzwU90kV;9ha+sn$nNl5q!7-9Lf5S<#rDJ(2pYw+N~ z16bT9)Y&nfo;b$+3uE=z5*u!BYkPC`t4$lerKKf(ZM3moY(7vH{q3FY&8n&@7^X-A z-7Sf?9E6HrBQ-9Lj!YVbvTD+AvG*TGM+iZ*4c(>lL|jNTq~EKPt%44DmhC4%Gd^NV5!2pf$?fzI?nz^%_VM>6CZ z*3U=sV*Pyg)=af5`M9lgg;?Gx<3bj>21sN{sj^_el}ZY%*jkWGQlQ5@dUX10(V`DD zAx+Oe*G*t^1Y>QG^S>3xM3OQUCqC(1^#x@7l=iYV&xVGEjSc6jV6SJ-Ku2dBgLf4` z1K~?`XiwyzK_&2%1Lum!vu_E}I|=>ysRijgzb`UWd%0 z(F|&7%Bd-^ik(d1T@#RvxO(^1{OW3RLxbXsd3*dl@Pp(oW@wcan{+b3`z&Ze!ovq> z5k&niORb_omYBJ(MQrTswm^#Z$&UgjxAkakN0LI2Wuc0a(mNnepH;T+mYx^`hp=?+ z^08%trC|2lsDFhDp{9)KK!)XE0=!M6x6Fm>PbqRWP;bC|W6`p_f4Oe#ik(v7Dd)v+ z@_UJ5Fz3PV{`Rv6@^RU;-Job4z)}l#q^GC#Ko{Gr&w_+p)sUrNw4{G!evjLiC~m0E z`y)Z1)m^?85*R{LstB1laLvgUKDZf~eb?bhWM&UE#oen1LZ|FDa*Tv?@iP~Ac`cg0 z@$SU}=x{v4ZCH1!qBj#5SCY!TXZ`pqmSRW-~3xWMhU&0R}gzya5L`a{hB*Y zi9v^;areCP7?5BgW2d~w4aH013`v2rLlk)-orvC)!#QZ2Z{tV|ixS|;_UL_;)#o#F z6p^KwM#8V~b08*;+bv9{@o(s2s@b{0GkMMqd)Ds~pPGUxzaMe}&5x9$W>(5K;?;fU z9$F_^DMAGGzJ-ra$4Jjt*$kQE^ckIS!JRWZi?&iTTS77LPA2ocL5chrh8cRpI#O9l?vxo zT1UAkDFsb%jfuE+T7LGCg>>ep&C~L-ySFUGh5k7zH5?h;F~E#Va_$@@U-%7u%q$#A(ES#=}2?$djG3v~ueCZ=MKjYB_c2oEV~iLBle{0&V~#`ymxPILBw-;#4t9e7j5aGhjt)Yh}CrE_5Ei#qlB9~X;+>#w^! z)R_GC{?C+LW(BDNt*DPi^l2AbMDR=ALR8!Kerjw#!_Z^sJMfN_|JT!MFSob2)qmg_ z73&+FZ{s}i8RjMpwOku464-dzp$jdtdn|gh0fO6GRDX`@?vL?;I%!1(^kkIuuL-O@ zCkf|UA-wko4+s1DH0%qSe&@BpN&<28V;1`#u35m=^{-bOj)Y-pt{Tt&gc9_PPhg@0 z`q;Ju5BMA;mTFn`AV&>)Gb1BotE{@LvNFV|;dMDg2c4BTRtM)ckXv|2g13q);mQYUk)2o;?dx0Am1xA*8b?1l>!$ylNmk3z@49zrPX%^|~l?7#!@lXAjA5i;0LlXK#R@*}ei;WM6Bj?~tv~yL`a1 zwk2F5TxNOT^7dI4)*-?Qj1v6v>fpy`yYKF7gZOX`&&d>gsxH5Q=`VNohnfyLJRF>P zGT|_YNh3JeVN&t4;xq$4zgtJ590)VOX)wB2X|NYdePkl9O{xDK%%%vLAt50Ns{V5% zDH(-ycSW_|&jOyaF&ZGVvrZ`K&K*K*7ovZ+Zn4tGdtbZ-Cf%$O-K`j-Pgf0jdMoC9ur+GVNtNiBd zy+dK3Nfoi3;ITn#$I8y0`#a}0Df;jth(KE~$c>*twxHS@F24(rfX+Ix-us%`pt`0P z*oJ;DJjjEe&9R*};8JF=b#S2leNwsxAQYjHCTr<&B4;^hk7#IUbRfV+F&HnLq$r0R zT##Q_XNzUUwz4K+0NbWNkZ~BBenrM-4+}0qI$WMZ(LP~T)vf}=jzyT*q5&DtDRal z2V+w30zl9rdV2256~j9|;bn2jey%tA2ImJ`K)cJK@Ti&?hSb4oA{yM!kh1aLv`TBc z7PP+Jwl?eEVxN+bzQZJc(jdENPNh+l^uB%$Qyx{8Os)Gxq3`G2 z07K~3HGhN_2EzD1?9E8_-2Zl%mb0XcyEmsXcBtG@@8cg_>U7ROxs;lqeGUmw%f&V4y%-<_aVd^lH{2xQ zGz;aiiBIdFhta8bWg=}Cnh_jrnA^gK>jf7d?nNWTL8`o5bNn$xb71Hhf=eKJZ?FH1 zbYTCDIQs`Pv#PDFZJDqIff!zbYKVx$fBsj6KQQHcqBMx6zePG76n@bE+`i~qX&=1l z?)nKKL2Rmezer;zZm=)G!~RVo^?xvC17e<6{hu>C*c!Yyl)g7&^+xBPGdsQ?{-%ri zFzM5bbIJ&7#E3pV`n0k!tDbs{$Y*C56uP)*g(Dv5@4tt3fz0W(4#ZnJZ*$eq2v9#Y zhZr_?kZK<}c4{dS1a~Lrd^p%nJ5iBZ`SNnpRL1`nDZjNOwnu`{NHoB;?YBEQK<$k4 z`!_#6Dhk~lrYN(S*V?;$ha-DQ0{b_5<~58m)u~J{1Kc@p`>9^kmVQRxPG}|hqe>ul z7Ekmbv@uN!!en!Lzw>9>bx)hXRKfM}>`D%~KL#CP@#*c$l1Miod71wrc~@+vn?}d$ zUc^1sk2a!P7yoOz>2$c>Mqm7wSFR-67pZ^GL^+52|H=6NOoS60gRCAm{a<5q*zO6k zp$C&2qMg{u4fH#me>rEX>7jp5J~1#d@;KN>LJYn8;#-PodhArz&hARnF(&UtjgQg9 zx2|43JWLtlN#o7>MXV~#B=c6s=P$9jG-_#bo8xC+D+kz6@&^*>>d*h z@k`-XbPi|E61~vSAY?ax`@^Y^%KTbmUWiT8MOwq`=L5svXFoOlKb2i~Jk^ccu19I2 zsBo+@vK?fkjHiUi%E&&-DqChqLMb7Yy(4l$$T}IvQ9a2>=5dT@*vBe+g!ld))boDc z&+mOdZ~v&%IedM`J+Aw@uDfdLL_`ze-MGT~&2{PZ^~v$^-A5GX-orc$N&~RNP*PSl z)(5W*Ma6g7`Z>VFQfJ)Il;2jg$3k*uG&M7e2ErPGslnhHro@jyZEJ4sPaJW5zFY$n zpM&)PW^!<(Lh+GJ1g%II^NE_qMh(*5hnhdVTo>1nBc$tjNq8Ds(Gd8tLa zB_9SwFtY($OMZKxhQuZ&f)A>#g^txUvYMW5pgu2fgt1Ly zUEM{&iwk@^ZT}|4Fy%{=m?EPS?f3j(;3-G^aBfC}qTW#S%_;tbpQ{M~!9i~VjLV6M z377zrW5DOP#`)mA6W)?~ii!kgFU{oXpFbbVsmXvW7cbs@xTOQQ{@DRL zta8sO{k&d!kcDL0gHOT_WJq?YrCF1JVLaixwloZtAv`xgwtqp93w5Uy00fT3@^w9L z!>9GyVMPIPhO2$y7~D}JBAVF%#LKx3oCkJoVZfoi`}WaL2gRj>8rirG8JGGhLT!-U zN0k5&e;*5qfq@HM!eUwI>L$EhA}a6LQULPf{nQH{9(h;BhAt+we6FhtOC99z&<7&M zFIxZ|0l$aAGjL@=te%YrSt3lQbI@q>DnwfVBRH`#D7)s=6hW!4aOLft9MY|$JQcg? z=)BzA5>H;Y02|`MGr&~`YZ5avGoZCX;Z0O)H?I#}%qHAtOfv;rmIcX-r?<{H=ZPH` zlqS)G4+i4e1G22*^9%|6MnE&~tnKY6bC`qpRwj%zQ zcPH1{;fNb`{zmRYPm#82>G zz0J<+GR3+n?(o0(d=kaxzcsot&>j~o{7a+Twh3YeZ%@4ptqnN5kUl)7MM>16h?-ED zKzYsk__tHWcA=QdaKA#q1V#~VE;Phjmf%VQ4opzq*YW)JquAEKZK$lQ4EORX`B^l1 zv>uU(gQNU!`8|*jrB5pQ^K^oRGc-dFgFPESx&pCEQKSpZ4YW4R4YY3uoY&UYhVE#2 zx;lUgnlKn{16JNu=9X1h`0}MA47q@%+t=5p^C&iZMhq(Z6H4S?@(9%%@a^_0g=CI!&^iMBqOj`KtLvaQfx!~7S>+ovgkHTKBq(qNt(Oey3zfT$i;2Ag_XRS{Ns#Oy zK^7woK|qDBg@myLeXE(IUYL=w$S!?oghuiM$Mr0fz`EljMYv~34d=y@W^Q2t7@GMn zUWoM_=i?heVs{z*+IT)bvL|98ArxJE;iB|hrhp?fnc#M(wgi_gARwR{mF;l+G)(I5 z{t~qIV2PYd%>7`RXpQ%Yza%x1od&Jg@%*BC_TJB*&ofCxl~EymTfQpXex{W^f=wig5u!FU$faa zZhxRZKx4!6hEH`rmDrneH1Iu*pdGD*e24-tA}7Zl3KE#K73$==gWS1~^eAvS*@I59 z_}McqD1BsQKZD5j`#&K7)R`qdPDp@-3wf1;0lU~c3#~(!vr^Z84MTB*Cu+Mn#=KRe zlN*5c>Cho=Ntp3^`Se4v0vUL8c$oJ{m2KN`Yim{2JrVgTTb6D+3tR%_<~uFDzWCv& zf(OAVY3kcV3OxsiK>vgU`tkA`1CLe6WHMkamX?;WLPlr*-V7|B*JuV2(Tjt!v2j9b z>KCw*V)q$2)_+ew0OR)O{8JcpxBF?q!C~9BqK&wJn?@^a&1{4sGNiHQ+dguy?a9%N zDo_DvVb8uWgxMw{7I%<~SdSr<>*Z=5CT2~<7!REAz#{_UCe6-r@C6*_+5)|}-qk_t z18J2~CcDM$hpiVIbDL&oh@{P4fnw^2PlXvcc5$6Uta@he{?5E%!GWK?ZaU|jUy(4) z_I6NdDNY?);g%d^!}Zy!VktOTRN9WO_6g>K!poDbmB++fIayIsM4!!khL}`TOUEkA z_)T40i)ehSI`h(p4Gsa06?CE@^e08s0-4R%^kTkuMRZpK{o}*t0mEbwf!EZz4gH2L zWc$rGoJEhwgC7Q^1OiLII89MSeKX<%;d?4PtjvB>SDF^{CtO@+uQV~gd{4Zy`%k3L zO}ae6HW!aRtg$jGD6NU!U4?Fvp=N+X*qUJV@WlCCmuQN&-^yVH&+L$}PT=WWH)m~J zP^vsn{&(-K&|os% zPOwPm0ZTv#PQZmSGGZR4-qyL2TMZ|68_mQs zAY=S5ElBBhGy1K#~y>M+K8y3dvmADOS zme6-y1kOgWBbWyQcA?XOD=87iR2ptD2EAYOZdAhCLKe6gHLsh%|CYp!GP2rLd21Xv2cPdpDDegcFb^V0Jt@+Xqx!$TF&t z$NACV1Ov}Age*V`VW&a`@HDLPTUn1!L?q?J)rZEa0EA6QW)#>oqO9?q1(mXX;UFvo zSkYd(#%gfL_^++|;Q46J1TJXpF!23gzV6-&mP+8(*OUyaFohrl$4Nae-pGza4dpd4&*x{qoW{>1ka&om(nLM?rpuD zCoG29s*AsVr#b2R@OL%XUXJpaGoK>ur=|T=TMNnX8eqH*4th9BjtV`QRF{*6LM=C{ zMCxY2-Dnk|L%VOEZhn}jQn3d>cZEQ<^dai*l`BJKx0Gr|+vD!IF(Jr1#pS4d8r3~Y z8ndtgro~P0#k95_gkG2^4C@^9dKfzL%_;r`?&Z?OCNzgZzghyy2mTC-gjKaUqwp`i{G7YHQ%1!?oCHJwEMxO z&O;v*sQw5i5BT4(y9Y!UK0f7k&QRlktrxigcKqaALL_nbmMFd!f#U&@dba)gUEeS+d0|OH1BY^T)LO&Nf%~?f8e^=sz`**;P1}PD? zt}+wP7SK(Bf!Zc;`^d?E-3BV3Y{tjr15O8^^&}593NOhB3=U$(bOjM3B^&=!E%V1M2xur2yMwLx5B~>8 zt42axW~Uht@;m}rQaYN`&CPI$(X6sV*DkEwJ@UkaYGZXryLpPCmg4A6`&7hoc+8<% z>}8rxY!sVgnM=W=GSmO`u9nv33#U)4>HRF^5{mV-63t)uG2ot*dlamiVQWW;WE7^L zPZinP0V_jFi zI@1xjX3(-2dV4Q{#1ep)xNaF4Rd_$LhogIH>-$Q~_WYPITekxH1c)^-E#(dbR~js6 zP{D+m468u2SLBdYARS$x2foEp)vK_g1RXmtdbtC?PjgQH=XWN3kdg-yAN`r@OMv-+ z5lcJJ+rb)o4pxR>S5r36)EPwrjXB1r2BsO@flx?tPQ#?-KYUa1{1c7HY7eN=9)OQE zr$ZC!)V1cPnmdO)o_n7*c8JH3%FwMu)RN4$T_R(Hs)TFUaIVj|Ufars>2L*Rm1ZVFV9U7vi|L(uO>ct_unB%`zls25)YD-n|i9w>>M)!uT zt*86!KTcmlx#wQKEi4@Ooea6|#le2ay{)@sW__8|cCNGVpuC^YdZo82ve(0_LHHPp zl+eJ1JiZ*2AO_SOnzt=)0Ft^pQvhEM+xBN zt?O?XfD0gAau!GEO}{RwS6bQs(8SEt38>09$uFY)S9-iX5=od94fT%`?CjD&D_mXt zQ1r#0kUpkVl5%IdP0OBFiydcgkWg+j)yE6eQ~8zgurj=8evi81O2=uF(%gj8(ilrd zBO?VmIu1E^Y|*d8BH55aB12|T4AA^H3Q97B=6QKxDQ;c zN>|>{RSgdg9{cJ2HQ+|ezHrU56m9$chdM@Rd+f2#lni6&6bJxEQ+ zPLG8Jd7s(%2>DaQcK7pd9W`MiRejm&{Hy8w*Z1B3*3tPm)W}bkI(YC(W@cCFO`Vy0 z3Q0){{vGe=n#N~{&0p5nUf;>hw#$#Qw>N(jr1G_g}TY_{RFP?i6YLpBXF*sVhE3f>BF2 z?DMd;?wS^xdqAQn_{P1 zUjzhH(d)iSni+Ph=-oF{E=-7;a-4cy-gk63yT)rrepPvIr&|B_$&q=j!is&2)9nfx ziJngH!(b2GV0DYWCw7D$e%9DwmFCZRS9t);yV- zOL>3n!;#Y8zn~aKet9HHJ@0@1THZ5HQf|8LJXorHhlmcW3E#K+HR!>u4sGTV4XN~; z;Cd!GtJ=#(J0Kn!p#hE%8^)SrqAARsU$rP~LVQcSP z&n;=H8zX5iF%68%TD3ioN*TZF-CKj|&gpAe)2SL8&FZ7e_?e|y5h*Mw(SD<@aTNsj z-$;?Is1>sQ-He>~lEu$$zGSH4+TUk!ouQ4CZ3rbwnJ-3D4 z$Koc0nNNFuhBPVUkCo~!vyz>5?QQTxj(Z!GPbG?R_#$t& z2Y~KbSn{&T6uw&wrS3-$Hm<&Co0O<}Ia18zUq6oABRK|#pX9|5wVAI@mhlc55PbBw zz8d5o@wxH2Gcv8NYC6MTQm@}7n9gfuAQFi0MrBz&$l3K3*ETFY?LgVzvAUA_Af(Jp zj*rs;e@rPx`ipGGy+jt_+_b^LiodA*BG`QA;bDbSuH$@ZB0G(TrztdLst5OTbwO}# zHcR~a4>izp#dbVe03Ky1N;Ft>QByTAH_~r}>SrE$v`t zlLK#qvX?LK5thlHdz}4OtB=bd%R2&tR=MejJ$d2&=~k0k$)3Dj*CzWWu-G2V&Fe)H zE1_tzh`Gp(#c%NnOLa+OsNbJ=HGiJT|_=DAO&c?XJ$_y1Ev=b$6~NUK}C3cr9U zn>DZ*3a+i)9jO>;HmtgChpqXho+9hCGC!Ak%h-4A=H%i}X|EoBzkMEB)SDOZIw$P4 z^dOnxY-NiAcbi|m!$6&Q zbnm-UTw80u@x-o3r|e0_c87r*Ww z)YT=W)#c4(eBtP{nsh;-X#2c}U$Zzd3HZ%^Jh9&Am^CZYF2~F#)XFAzb8p&v4&H&I z2j_phex-Y-k)QwIq{$s?KOLR0tl`})m&9(pXui?OY8a7^8-wR85LJH`_FquRYAq@Z zPAp`=N;pGg$60;lCkZY14+Ob*at;~vBCj+d@4d#C{CjT)rG{gGR_rl1=f}Wcq$DbW zHE+Bd6;ar~U!d&&*hyoJxtcKSu(`Iw;ushy&75kJ-3R4edYTYy{{Wj)$}8}~6W)!V zUEpUslF7ylTRVhYGoaQ?8u|~lrpg6HHpeTNsk)5=oloKI=+Hhgmi(s-o?tWBJe>_1?EKIJ_+yA3Mn6p+4xw@#>bdk%1O;YZ3t?%?j-Q&0Q zAwK||pS!u8)T3;~gFPQoC^fX<3gz6x!j1)$F$ROt);@dGH8tv+0)R?T?Afn)p1>d TxnCoordSender : sender.Send(BatchRequest) +TxnCoordSender -> TxnCoordSender : (some txn logic) +TxnCoordSender -> DistSender : sender.Send(BatchRequest) +DistSender -> leaseholder1 : (distribute batch) +DistSender -> leaseholder2 : (distribute batch) +DistSender -> leaseholder3 : (distribute batch) +leaseholder1 --> DistSender : BatchResponse +leaseholder2 --> DistSender : BatchResponse +leaseholder3 --> DistSender : BatchResponse +DistSender -> DistSender : (merge responses) +DistSender --> TxnCoordSender : BatchResponse +TxnCoordSender -> TxnCoordSender : (some txn logic) +TxnCoordSender --> Txn : BatchResponse +@enduml diff --git a/docs/tech-notes/txn_coord_sender/erecovery.png b/docs/tech-notes/txn_coord_sender/erecovery.png new file mode 100644 index 0000000000000000000000000000000000000000..e36729270d350046218381d4ce9745801cb4d74a GIT binary patch literal 18067 zcmd6PbyQVb`z|&in~;!35GfJq&P{hCpn!CDr=Ykg>FzBJBGL_t(%lUTY&w*bzH@`; z{J!)3zWc`=_m1DVXN)tpuD$kJbImp9`##U}KCeMa3X+)U#ONp}D45bxVk#&o*S>)N zM^LYWBe&DWSiuLqllU_yLt8s{D`OKU6iH(nV~3Yc#zvF|?v!RuPIkP^%yw2UZJeB~ zt)3X#TH`$BAq7`xF;{!$^w)8eYv3|&DGe$E3R6$t2IxBsttY~>Zpc=ottzhjOwPiuD||>#Ue6eQ=UR*ET{3>3DIIy32s(qEau5c3DZ@75TBA&M zT|upg*~-@T^6(EeP0fojJ@4OVbtE#m7$_*hwgX?V!3PCj`Po3UXYLcTsnPS_tNLjS zn<;o=7*rM!id;?5RSqrQd*#-3zQ@h_VQznaQZ7F@R)`HMuDlU{1|6KX$gyJCH+ve+ws@U;>aQHQxaC!MmaE;Q{w@_cQoKQus zt=R@XA)$1MV4olTwj^TwoE+~6#9}&gNFZm<34@v~IXO9lQUlDSol}K{h5vp7-Kxa5 z^h``lw6rjvDW%r1z_y&60slai38$}cJ33Boydhpx(t^xP>ay2`*Ky`TeiNNKo4MX zn3bLVQ{IXWw-Yw6Q-oV(Q!)z*MEy(P-cg>p3iH7sAd?7PO@ z_v>EN8Hn#F@$McrQS7c27ISky^F2SBKRCcWNb&b~v)wLv^av{Xjy09f)dKpJ{>V!v zQQ6XhcX?~8)KC)7^C6FW0}Q6)xa$trE^mW3kdVsUFzwkP%r}^-x$*kJ-tKzw z7{e}}x@1pAy^5x0yfvEmt<=>GtLVg*66Msl`Hw%BF zj<^#&_F$f--`uc`SJkawb`0uPT-a`+;5Y3} z_eh1!);hmwMZhSc%rB;E*oh4oRO<;gJD{E{+_sIYP6t*6N=GLLEy{j-&8@9e$I>TG z@94dP3w94Ls5#!cyX~1prOvG9Dr{y=VqmzAk4uf4Oik&e^F%l6IB!$QWPwMj!5lS# zF}UY52qlO@aG1Z0jt27uwT03A;)EVik_ycl?=B&*$WwCIR?`T9qrN?=AF5ulK za6iAamX^Bldt1{yYm4kj0?W+-GfkB{TheT7(eYYcL)g!B!ru?jO2)((F?amrB*Xf)T zf{g6KNOApzSkN$yE@jo1$qgkMW-$M%?O^`f<>tQy`F^*4yCcGFHEyZag7N`DAAf$d ztlG&P^YNy)x4yTAH`hw>W=!V7`Ej83Rbr^K1WcExu5lUSm5c@J^`zbfjzLQ4l8@5FdzgmA~y8JpO6Env3A2hl~=dZ=ZL z@~U;@#!$#TfCa|)Tf)#&rAe~0^?Yg`v$IF&;u9VQ1(U8VFPFG(`!cYt#E7aM4-Ez8 z%BI$NsqP?PSQqg{Mfgm-y1p~>?UBXb`{RjH5)x>r(&-1Z>4khTT-94%bYTe4sHV0b zYnv=A;WUI?-j$M+!RN1KA!34SuI-)-_ri9lO6I~!O1P^Y2?a|EykcG_x{{KT6}^kj z2LTwE$3gd|R=I>_WZ0Q{wq{P!J{tL62r;DGzjJ3jQ!=``>rMbez5T-XJmnm@HK(oT zNwKj!?@&o6*T1+QY)CQ}a|xZ=jm9$8SX;NaZk_JTJmNGu+zcrziB!7(X?x32NEO#Z zCWjaOHoVWBU%)p$jU3uzV}4Zf-rypI$2W;grhEDqyy{f8ebZku3 zgdnkaJCtm_yXUK_s!;CIfIWBl*z2KbFQ1v(TTh<2uy^4GC3d@R&}-Q_#SCuy9&K}? z3DD@x^rof{sK3;I`mMv9{qRJ>v^VPwOfko@yM0C7)HGR9=#0x^vr8;S>V*IUHbmVG zIpg(UBl?}$+*3d9;aaa2?_h^^P(C!ibK3(#ah_Cj6ex7T^QwdC=i$!pFab`nklPZ& zC_d>FkN{}O7ZB@7gA1=i#LGW}KYm2##sf*9`NFzc-7m0B{1(ih_JGu~mzHTwN{kD31dr)Jc;g^zB5XD03*DjJ6y zJ@>%Ok7pgT0w!oGM>~~^xp-Yt?%#K&r z)YL2pzWMrB?2T*tksdN<*}7Ggm5|9miU{d}S4ShgX3A}W#BXq@*k^cOJ4%!}J#xC% zb88oa{gLQ*rB_F{0h}H>A4}wF2;AIP|DAz*Tqmt(VsS<;9Q4Y zfBpdZF;9FWK@SCHaP!&{A?5$>3hJTyR#qG3=0gc*j?29~e0=LGE0jwfl6H1KbL5ll z0#LZsbVw(MhmCA(dS|=8jg4_JGjIEP3`#yCyDPC}PU11MR9ec4fe~Wbm-Ykwnq8g- zH@kp!{KU0oVGI$0T+ z&J-TU%bPKhTG-n9F0^8fQa^rP)Zg2i%9pK@|MBA!0e<(Lth&0o$jHdTLQ4~qoKuW| zJu$wRgW8%JA`%i$clQ^KUWq&(<^1sR@gF>R0C3HtM~})(`%JbQn{{$dyIQ(%c+yLE zjJx8bAL7c&%1*wFc7iuLInPP84GiFDqP~JwG{cQPE-jhNI)yUEB(j;7a?O^NmUi~^ zSeu*EGcp?9dlJbXrqlaDtE`}~FgYpdeP*V&msbq4-NtxHKtMoWntPxQj1}u?uJjW!>waoueqJq1pOBcCSXQeHoWY5`2+>jhXLOht7_##6>MKe|gi-2PWV{a} zr4nG>I@t}5d91fot|!FB9en@ty0^C%hDE{fSak`pVGnnB(@WR^4LFX9io(Ug@ngLC z8Ls_93Ab}^b;xtEA53gqT%3xEL?@VB1BmvsXV2?gid-S;Il>N>K0ZE{mg_yqTu@O_ zyg2$@hl6S*9?|7ut+Kb?LNU1G6_#TRPAkaes@QtDUUtbQr>CuKY;={C;q}nnot@Tq zB&J42i(6Z|&z`lDhK7WYuo%2)?nAhTGpc?EE3z$=$WUKj-_S5RAtCRRTw*-qW62dS z){{00U#khs%}?tdaF4F8u3u!HV3|fjgWi#FDWvk^i;U9tOC4;^d2o(R7j`ghSOJ)O zjWihNW`iGTk}2oWVDOvPQj2cpS5OxJj5f7LM#ifigAdcZ*1t|^&2{8vU+qM{>+ej~ z-Hlh>@umC^F|i_~Wm%d~AT6lZY1j+4#Z{%J1fyR~zx1)v4nDeNXVw%>$=Nei3UiBA zKa}XkiBrcWP1WH6H}-PbYz` z?{kgnCgfIv-YZl;=(Ud#B2M`EKYb=aY97jwAIeg=b>~jk#R3U39<3xFsk_k>4(bV5 zL({8@^7A{Ur|U~gRia6?H8dVwVBJTih&&IU*qWvg?G=fv^1Z!1A~useTL&g5p%*gZ zhZ_?&V9jt8$mZN&^JQ!pKEp&mic(ck8P{2C26<;CzJh;KiC_urP!O^Z2i*%J(qA4Pn5oJK!rjf{*K1+T1_QBY7E-HWL)o0^)M zb6-x~*sz++O)poJ6zg0TegD!{6S2L0F=3isf*ATWX=7%3cD&EJ-`dh5pTwSDP_PD8 zD2T%#l<_$7KYG;K*B4&?Oi!;b?|D77goMPj=W7gn#$j^!0R{*3$Ey|8epQ$X{`846 zl8&tq!UpksH=-rSWk2RZXaMn(MVl#5FI=JB42ka(U#!gE_{FICv3*Wqq@^XhzCNXB zW^^=e!qvecs&8A2*TBrI-8oOGM069!qDhGj4z|r+sdphZhmB5uD@EQpTacs>(FX(2EGVD!2Kz2?}DONx*zD2mw34o2=ul7c?1!-t#sHm=k*YHJ{ceuIP zA0KviIyWgPY1V7EXDC;3=={s;J8j_3Ae0>&wfoyG!z0B#~5P(z3G2g5JC*N_m$Lm0m?u z&<)L%PmW7WL<8$qpOTE2kdVvZ8|tiV>dEjO3a?LwZIFX}CY?$#2?>V$yV^f}PJgw< z&GbX+&W#1{g&#JWQR+J^pI1r8xUOTAkjaVR*oWlW%(DykBfJ0$6Gqw0&0$Kb4n81EuNB0Lz zurHw#S&=-tb^8_A`qyCH7SH{eqfbsiI7 z52~DNr07UsqO^JH!td>3IW}-0sfcZh#4arF7_;u|o@6Xe9yNWKVn(x}t9^6%<3RI_ z;Zv1)t;7iLewW*+^<%#`&+pOAV}b$rNxXOv&f10Gh0n0= z*Q&C8exhPBdo&?^fufQR#F;QNKY(4IWY1#c)iIyxjcIDvY)+^GqXo}ny;0ao>V0k)^hOsz8-8XB!a%6D?p!NF)ZSCNa>vnV~{z{EC zTUGryxMm(6yiAK=)jSmweg~$p0DjA}*%nS6;}<1Nk5*Jv)PjAD8xKV!I5=40lz>+gcF7vcYU-K$%p3p)1yw_6(& z<$teM z3Mt%v$oWAoO_1G(P}f#YdGX5gQaG)XItx?@{a?T4vYxz&oO%d;w6>~hbWM%Bu`vxP z15`^ZF(9A?z)9pU3ksnWLO%5tBMhddrqtBb`D(<(#2!oCiH)(^9v;V`L~PY_A#2al`J?Yl%ZnY+TU%Rdsh2?gk*y@^@uoQb7qE)e zHDQC=to-~bquV>%oh|<%>lKD;m6<3H$}|Rsg&q9*^)hS-p9Fws9UWOqM_*sxQM|=S zkpFd005&duUgH3in(oLF!!`l}0u@=Q53=!3QiXg4i<=A#3{Fo^+q?F5#6t*zs)D^Y zU*EwR{rH0KW%JEnvuOLG?Ql`VGc7H|(o$u4c}#3{kUpM=Y)>a>uW&DQ$J8KIgQZ%6jne6#z=#tVKU&m4{5^l5J_9gWOtAUgLXl zp2T5!{rdF;7)XKTiWguLl-cYSkiHGpX^NjlO z?(Xi1iKF&NYEejb1^?NaKL|Gk2M}Yoy_JDaFTW<7u^6^~(n@Bne1CUqvcj@ka5dp1 zQN_ZdpuekTTUiCo+}yl;Wou(2E1ShL2Qafk8Y7xnezkA#yH&PxaZ(S}GNr&1e(A}f z|6v@C5&KnmOVDWrTfK7%V5z9Hda=f+4_mmQ4MwV3jHts-@ofG<^TTu z`}1dET=beuplYA_L`4+6fB)X!-`}DRPFUI|9RjlYaxB@dR!w|7;cmq9==Ait$w`?_ z5P4trm4EuA-!M9I_Rd>KNQhT*=}ajALL&-=f{{HL?<-MOj ze{OAkTd?F~!v=7|!ON=@pRGb)6U49<2@R(Z67)HBiv-KlpH)#eYYzZbkO3;IeN~YinyMSD?Cx^O~rg-P7$<3}lx63~!S8o6g`u zV&NY?$ely9k+5U90}fuAZiPi`Sy>qfvuX^On3w{NOLB8EOacO_4Nc~R%TM0a*46^T zi-Uv1!NI}2EeUrVUjti^j&RF1_OLuMCU|S2><$v*)bg^^(XC{L3sRMpS5|6hYHm+f zbPWxa*?Qv;5X41BKJW!NaoO3~Io)d~2<+@8-k6D(TlJfpn*w>HNl{U(roE*1Q6RTK z1d$P<$47Xh@z{KQi|OLxLM<5<8XB5K!^3!8hG}McI`rMUhWh#^dX3$rww9JU^=>8k z`T3cd3!_CkT6*JCQ)P#BafhfQ3fB4u06K_@iss$CcQ1OP%#@3r-RbOLYogm6up}cW z2?!Bm+QD|sY=Xmwkny_dOM!_v~y+W%s(<$5&bsNnFsD5db6nlf(35aO4atV~PAQ+(XsO6C|G8RrxPlFc(WGv31m_BHJ z#fQrH?;JWI#?kH(bXw1xJ42<%xIbH`Zaq82ZJFwo09sX`B8CQ>W<(;kT69d_zb2X z>;?q|0TmNRuS->x-u~^?93ehVcYr!ck73grG2(f2DMdxa=|_t)GIa^wlRrKpiuIa$ z*CS63w}CeDh?4S^JJ5ARPmI)aB)q-7XNha@(N?8fy_t3EGdO=A9|MjRba_JrMy%tH z=3AX1J>A_Ysj1n{31{d81)Is;pFhI?xP;o*u-PEM+-_uk>Ac9Rn_ zm;&sUot+)i5d|&!@WG8wRB5!&YyReq&v0QTGLM6eiJw31!1C)XGaG0y>cEw`F)`w8 zVDKKSU4W-xFi8;+zv`lPZw$gG>>M1~EvuE*Q{xK{2^}4GgM-lzya;#`v%dOR`2PO+ zQfB?`-Mdy+R(Td%%be5_q52e+RaM2h4cZK`QOSAl$$9pE|5{jR3nyF!8;Ne0W^AYf zkmc3Y)oI06x2}$^F0dR)v&R9v0*Yfj89h*Ar~Z)!ekeVCYI)>iu+aKWHCI8sO?h)n z$oDh5JCh`*_)9ohd(0~5(+pri$jc?;rt0zZ`c4PgdDOH7`DhdA+{ zth|fQqP~M-Ig-N&lcRsD_N{d)hBL4CHxwT)zDwCF=9mJSl?3@^#HXWG66mwKk-j~5 zKDXc2TbO?px79IXG`1#c?^Oro9utG*X-B6-hAAF@(8#GQ2!cK`4_v=>rCBlC&Le3! zIBC-CGH9cK<#owHLGVY84w*wPFCl>q$ku;-3I1})sHz#tzrsZ?Mpfl6W@wRo@n3#- zHYbJL9e>D=MgiGr zN&iZWS_!<=qQ2h2B2P$87V`QX!IT~rO3V=)h8p{L=oMgLVEF!14t{>Hb<>T`e8`2<)QB{iUl=y5x2l z1?PZ_xwgJe$fW&^^kL!c+qXxZ7Or%tfiFnu>{kn+)lo91_3^si1%pbd+>O>SGM#77 zegohS#1m#9)SRClZJ4WRT62|``J8zWS);}FiFa_e^Dl1ZoSGwNrhNEP#HIk)C?-Ch z?ZkC^wxXiK_#ibU1!S7XyV{-TV4Zhv43&k2g_WynXh^IwAd2J;fZpF1!>k7GSNOfI_6P7~Xc|=j0si>)QrlZ~72A_Cf1wqq0=T0=UqG!>rrFJ0J&nXuRxT#f*@8 zcoo^yG^M0a$KaQe75q}NqNmqwWNov8ng>jgW>m(+5C@PACH*-?g9Z`}v5%K_9`xXoK z=S#ev)uG%MQ&q0-DCUe!MnCL!GOy-_Ddd*0+yqpa8}DZ~y(?el!f<}(p{V3xO&GbM z;$+(T`do0>mUh#5Jgj@}KwppX5E;KvaY;x{5~miO*GD%o(Q0g8enF3^tj)-P1Z5R| zxSRQAt{OR--_i;Sv--Rbeq8Lz%XRNp&8|IVH)5Y;HCKsOLw|>SJa7&@1B1-d`@a4+ z$bcE9=;CCKFD13k$+_JiXt}Sg{r>VHY zKpVRh){qMQpDJB9Gx8{kW;TCU2fB)h4F#1N2VrD#&EeMh&Og-hIc=hi%}v|s>Mw$LH8cXY zf6L`#5Rg`lY|n0jAe_NW#%nqHVQtM~P{hfp9He>2l*8F|AVmOM3gaet)Vm~{8vs3) zE22|c20rfqYj(SSSHZ=Fb8|>>GMAc4-4UlI(Q|u?nd08Ph2>>Q?ZsI%88JB!dx1i_ z0gcod+^v+%I}`0u;-+kHp^KIFMoN4Azy;KmFhQ&JDV}J%VGIKueb#eZVDe}_gc;~8 zE?d*mCqI7txC0@@Zi>pkDYK$cq%AIFu2*mIgo=3Mk7MG}%XgQOZOM6^R*I@--+S|lw&80B zsXXPacHcD#Q17CXTQS*csB7Y{5;i848Eo=7vTifPhP%!xxv@;;(b-tvh}5YK2QvimzE9+419%*98@s$8P8OWU!$X=0F0KWM??sl zobGfnBSNxc1=nLhF||`iF+}0H~nCH zrcUxC#;}u+?;$D`< zl$uG&$(m|vfrOWpkzLNr3sP-e-LUQPuRxbBd<8J7gU5e~;!jRU9x9#u`Xh>81LDcX z9YjH6@0}{>2rt;Uua4Al08%w>Y(y%r_eK5`-v5-s*4zT_R$@k92hi}pv~K$V%2}jv zZa@rAkWDr0)~Oj6g0VA|ZATP?~q|f9Vnz zMw@yt^QPUW=Z_*jyi|ZaR~3dQrH__W(LdyF zdT)~PMip(x+H%(Xs~nC_3VesjDJjpLj$ujD%|oh!y<|(8`WzgxSj1O$5T$1L`e>1? zloYr9{7uI<^0H+xH$>uX(Z6-?8!*=Tl@(C%gJoQl>EYgeN2QG%fu?vLX{V^Uxww~x z&{--jlB#F?fZY0{mR~~xFltml-;pzYg}snm{wgB>Md^`Z2KW-m#jm~uf4RH>z-s^L zI}puzv8nSiGhg3FN*c%)j6Cs2e0J7n67b*XV&XYhB@dtZ;Z15Z%FCG-C-Do&+isOz4l=$ zmB&>si7-%#^z-xMnE8Ej;(IV*TAzzFV%{;fXbPx=w zrba7=Akv@)(a!H=XIJg9&rs$I3a65ilCwj|*8n8`)l0ho*1OQqYDQO}$~^!>ssQ>T z&;gxXTzV7PXjxd48~nl#mh;S=!;vC*5qN=+?+GFSYi(=0bNjY#oy)t?1)u}7nRM%i0l?WW z0@plK@zfw7I|Xsm%@T(}V;T;!#mhM?OjOiobuP#dd!5@!iEj^RA?5d7?d@4OIS+pR zq@$zL4{K{_d3T@1|D>i~j>q$G%h|KqZ3{U#r4{BPn1Xh!#xWbAENa{Gnbz_ZDptpdSf@**~g?L^`hCjUKf=0G41VV0) zxt)6#97TbQ(7_z;DVp#hs!*?kg$Rm3>J~R0b@w|)Q$M5QCEvEX%-UT3@*@{#LruOF zw%@Y&hlH2Vr!80N=BV<(2E}>H53XdFr$YVveP-FCs!w+F%?%RUi;V{gKyK?tt~@2~ z@;2)Boa~8XUH6=5xbh{XGYgjlt^>tek7t?D-b!52q6ShC8Ck~pvg6h5n9H4n=}Onn z4+Co0BU)NQeEgRmz)_e#IbTdve@aTq;GjwDy_J3UW@{~JxUP~GO?ba>Yin1D} zn5MqXp*KDtfW4z{36BkOwRe!w-v63|8DR7d4i3ry{-Prx*GdU=tVkz5BEoL|8ya%5 zMr?07JNb~I%LD>fRZi*JZ~!1F^2waYqDTO=C2RCAfQJU_RMOBedHM3!#snMkjjgTq z0|r*CUC}u)F##wKL+snPZ_{`jb8NE#;1e`#y}M<9`^AeFva(%)fq|g#9e7(E%IQb#oH2x;jzSh>C&H)!rT<1OneK z5G)l1kAT;!N{_5_(>FMbmm%4!$gtP#tcze+*%Couc_W$Og(85u& zSp8R*#*&+k&eVKMh`hW!z>C1@)Op-IJ9`eSI-=z8$$ZRiAFv_-y@}N~FktmO*m(Q) zwhdKB<0lZOt;`G!sUxL;mFSyJOVJj9g;p;_krIwvSxpm~TNlrzD~wNl8EEB+i5*io z{yv_{;>hhYG6FA;yNiXDyI*%&=kT9G@0ua7ZN!vEF<%+E5*D0Vkj!{=bZ~NdZZ4hU zmU~uBP2!l~{{BAjO{8$!Q#l%4S-pmFao4YWUVZDu2?_T>v0;a`!E4~X_KO{uf-JPe*_i*clj*gDT&45J_QuXggUoL{#~8#AJeNjn1utH9uX za>_Bp?i93jbOgz>)wJwX2I~;IzkA5|m&}ul3q-T^4Z^rKuSfH3+o(c zD4QgrI*suXq(aMiNs(| z+$|S19%j)J9RhQu|K#`e_2l??+RGcpGOP6hRe_F~lZ_3~XhDa?zzC>0lVVdXGWr)Z zK#9l&JlL(qpPKk~C$LzJ=7bi}y{>cFBra3rUGm9^NlIeabJ+5XQ4`hHCT6PTD5EK-A`ZVH==oa0j=i{MKVP0NdMn=Y@ z%}T(A{op*XthpPvd1-7R5H0)<#}=2DK{ZIl6G-CYY)p?IKPz^2_BEUXE}h;;cOf-6 zG^oG7AB-#nDSd)#4-5=|HP0^N#OnDxFXfN(?U5{`qO^2=6IDb@XgBB{GE!GKtpK&+ z>({S9!ldteFMb$BBTI*Aq(zjf=@d?nQ()AQ^f{f@rK0Dj!v2T&DCS&tH*kiFDD2&L; zMHX0|8Qt;E6=~~gYjbjQ9fgP%m}r@q0fn=VF#v8$LIMIXm-Y3eb0v4|@+687Cw~la z>SWZ#dKafbtzzf40?C$JBZ~wsmF38A)`D6faASuyYq6idtb_2PHXCFsBz^;hrA-#~ zIXoj^jQilt^}~0j_c-jTegteJ2E;w&^YZuNZHcRG6PmoVFJ(b$R6H~GT%FAFO%4y* z!-}Bk%Mw+dj3hQlkvzq}jn+Lk<8RLKIbXZFL7+V~eZ0G7Oy%oJHmu2BZhf|eNkVg` zeIaVB%K^|4nDrJmN>0yqkuAA$I;o$P>~{9iU#3d6!do(y2c$P_ZRRI3d9J{r1w9ZO z9biXYv}wqI!s0J{t%u`~p=q)!*$s(%Km`lG=j!K|S^d8oI4CcX?%$-~%G^;0P2X+O z07jAtl5YhWyv`3m2g-Tx6^QMhllr$>(}wX@#w`G|^uLoG18G>jPy?8BNr;G~vj@Ri zV>9jDD;gdg8k#sscCp_Q%>yNOb8{jb9A#D2-5no1URY6YcXz16k8=%oMH1=!72r`t z7PayUp|Ab@1&>##S%FW^(%E?*X#D&F0w<${fG7gQyVI5~i$((Sqa*i zfOD3Ymlv3hV)p|Y8U#F>EZxJEHivJUG`)XM_3+^vnWro)i2|N3AR3O@Zm0F6j1CNl zaF&;pL`prBmX@9`n%6j7?6_C?1dc4^uh>k9B2xI>2WMtd3ml!DL0w}JY^|n-JgPs% zFm2hlFxb%Xi}M#Xf9s^Up*gTb$aidG; z=jR{j+o{1r)F0#*7wgqHAif3NOQbez3j>=!D=y+${F4{FzUN-LzAfLr{XRO1j9*0{ z-G{dLCzF6Ee|S;n()Zz;)>qb4Ag$r{Og**oEDA(Lll^gQfyR4Z5h=&M3J>7*gUz0y)A0rS?T z!?P8aQP+BN5Jr)tKG2+9r;pJz@bKFDsyp=~V4S`r(MlOona8^9+vNjmK67@yHXuC! zA+Zh8l7rAWz_u-2`^$8|jR#P_!StkaXxwh70I*ooMckx_P-r?3*Ooo&YX_tS5R>ei z9{PKAo_fh{oJPhV2Uu8KR*Z!AukW8CZqO*q=Qm!}htwp2=U^J{vlB#nH9WeEnk8L? zrlOv@N)nj^~`NR}@U( z9G>@u-~rW8&$dzy)@0?3koU`{lJzbM$j(`g_FSEAIKp?%Yg;Xj(+Z)(Sr`JXFjWC< zMRXMjHY~8gb-jAJk++*RFXL*t)jTg-dQ?KLT4fD&T+;Ylw*X6tjg1AxG%(MBZdMk= z;J2~zT%^bAPZ1x&`Ru-bKAiK_15HBIBmMoa{gq67 zRs+yEcLxS(j>ROM^aMWieV9hmtb~x9tTINqpRyg33aeQR7(9UfWHX zER>XYIDyygr@ntMsSL-!{MKi`H*Mh|KF9R54~?P#p+S@I&PjKoQFY9f`xO?X)tm+m zsfea+^4VFGv)=r5IER(SwyfCD3$>l$6yY-wHlO>j3H{<}w^Y>G@=A3vv zIo0z{y-()zggssw8seN~dDph$35t&#om@l`f92=*VDISN+E>kWQP~U#;Ip!NXjQhk zS=5&z{iCSX6^BLzJks=S%Q9$ey;|&5GbboEx1@eEY>RS3_Ev^bY?r$4?%csr7)o^V zyLrpKupRzzBPHcRY3Z#0vvA-1{EH1V*DX?K2jZSI;0@W*({Vq{0$jT51Hv5jLy_db zN;I{M3@iTVdv(jBGze(p!+R!*7-AJ-W)34M6%=gJ{EesXx~jr?GR(IfYD%5V>486G zHOa+AfkVK_BY19`wG$TRTbjH7$vQg*0q1 zn{Jr(X#EsHR_k-t`~bSgW%ca69Mj|yasa=zbp<6R%n00N1p9#vW`>oGtMS*{g+FbW zN|xX@s-0Ll5zue4D;3ay-mG_ngUn6O35bcMii1N=#Ka-{ySw9cheEMLPBL)%vsLqN zL)oAeg#KvfE9{~R^iUI!6*t+~FScvit#sX0kn6^c?29ogib-z@s4JNNhzLlF%2zP* zW-7Da4V)GK)BK6w^w+YWWD8<1s_#(}M6H?XNxma5_qL!N4we*n#bmrOlR1z6EXWkxx<2XY z>`alk3EABL&hL4U89$ax;lsV&89QBBInU?nb7XUmq!e_MPmD{^-5qy51STTT!KzAj z>vjJ!e_C3TtYSFaq^kV%nR1D5Z;DV)^5NFsjjR6{yA>zp5NLyrP*bq3_HByjm6JIh z{`ytkdCt1CXQtch;`7KzTAVi3YX$EdWw%WZY8jeGiU+*zJC*_WpaGykeSLkMo-L$% z%(XSld1>iKetx>@?g}@uOW52-u*qiTIw2unl5x;CIbgDndCbi#vwGNc>YDr}EvndN zxwmhn;OW!*^PnaD#59gSaHZHFML>ajd{QJ<42o*iFytB%T%3^t5X7{w*;*=Ihs+g z9NEd%sP|)zWgBeF%2c6Xb literal 0 HcmV?d00001 diff --git a/docs/tech-notes/txn_coord_sender/erecovery.puml b/docs/tech-notes/txn_coord_sender/erecovery.puml new file mode 100644 index 000000000000..6a744d62ea09 --- /dev/null +++ b/docs/tech-notes/txn_coord_sender/erecovery.puml @@ -0,0 +1,22 @@ +@startuml +participant "client.Txn" as Txn +participant TxnCoordSender +participant interceptors + +Txn -> TxnCoordSender : (first instantiation) +TxnCoordSender -> TxnCoordSender : init txn object (epoch = 1) +activate TxnCoordSender +... +Txn -> TxnCoordSender : Send(BatchRequest) +TxnCoordSender -> interceptors : Send(BatchRequest) +... +interceptors -[#red]-> TxnCoordSender : recoverable error! +deactivate TxnCoordSender +TxnCoordSender -> TxnCoordSender : new txn object (epoch++) +activate TxnCoordSender +TxnCoordSender -> Txn : retry error +... +Txn -> TxnCoordSender : Send(BatchRequest) +TxnCoordSender -> interceptors : Send(BatchRequest) +... +@enduml diff --git a/docs/tech-notes/txn_coord_sender/eunrecoverable.png b/docs/tech-notes/txn_coord_sender/eunrecoverable.png new file mode 100644 index 0000000000000000000000000000000000000000..87d831ed059b51e6d2fb150490a347442246b801 GIT binary patch literal 14114 zcmd6Oby$>NyDljuWzdbJgc2ekgOo@kAc!;)(jpB*mnb39Ez+%YhX~T$-5rC}kaO0+ z@B8-m{m#D5Is2b|w%6tLGV{h-?~41rpZj?xNLf)DAD03b1qB6P=DDN_3JPjA_^*zQ z2EHkIo_q-oZ1z&G?BCeDbFnlwu}6_Mwl;omU~ge6Y|Z}UfKWSdlXdg80W+c6$P8g2RG~RV4icua_>LuGg-`2tSR$#-!N)_ zJ!Hn!tjzi-2KQaMd}i>$Y?$H;zV+W`EH9Sa!oniCD0sXVG8KPCqianR*kaTbm#;nN zbS%WmM1_Q*CX%_)=A!BN&sUmI8?f5SLHL56W4sabaF_C?s{ZO}mJO|JX&|Pg*!mGj zmr0%L{#3}PO)mvqy`nF2XaHy51>T86*(x=PJL##Q)>RfJRsMy!D=f ziJ{4qP&|K_mtwGp6Hb`+{a>PX4>9JM&oIeQQ25SeB*oO6C$^JtUfnUR-qpN^)=HP~ zDAhQMF_zfJ0K?}+^rM>_l{cUBjs6O}ZFkSydWB*&pi(kO|BKx(F-7IHwBkx{>;-G_ zKB}R64=7B1inHg?7(cR-Mc5^Xk+IngyrIb-) z*m{=g_t)QVp8k|JoV$Oqe=3}HwBG!p+Th|TYRctf%skA_ue;EDHz3uB_6WSwcSB3N zl&o`uv|0LP`s4>mVJP*cCPF_9CadgLJxlaKno*u&{uH!&i5Yxbe74 zKF;D60kzPuFTzV-$#2 zQB5oVg9plN`fP7noZeildFTn@)q9ho@DhlNs<5GRQF*{>95DsS=Dyyo>(S|TEh`&(&jy|`^30udeI&&qvK!I?+N*^s?&Im4A zZKXe`JzkBkBsZSmK(pz0k5OW=%l z#fdTKSkBb`9DZqS$n)XyVk0jqUi;h5Te8R|atexK#?+T${L@oZj=Q?Q2`0NNVHan! zI1bapcbR5loPF8RX`Yhn#Vp9#;lXayJL9Q!4-SSvWn&z-UNzTs%R z-g&k$-^}~+CCe<7*K%iS)9W0S5)Lq7AgyI%~B}!^$n&jFDbc* z;}v$P*_ruxYhA8ZY{v_W3eFeI-OWjN2fo1hSZKnQ({n+RXZHc0p5S6@RgqppY2gmv zaQaJXij5*B7_l)gFG4k<7k&qK_UBI>OaBCcurLb#F9mj&=gFpN`Ryg;}%t#ZHk{Wf2QBk>_T!X4>oFDqu@!2jsb~GLo zbNbPoSsc>+eY}K)h50fFezw-HH*P+y^@71YNT7`x=7pXTu$NJV>4&CQi+Q(GDgFTW zs;3fx1gTu726=Ltn{{qke5kaXdT-t^3O$d^9B^STai9^sw5VvHoG&Ob8+^aP#JGrO z&7Z1!u+3Y~MJ`PRjgIDiSELfxWA7S7p~3EH#I32R`R=$xEkf9GA|@jpT5Z%$d2pnN zGj}|jb!a3Nbig!2=hTQ!0HtV+Ss@|9h3=84aVXW#Q3tHtXJKak=&D`$AW{9f!{O)z zIxW7A1nk8*c8I*lV>g0&ev5Bd_&mBbjz5^|)+piCMpSzRg{Kt88YOK^hyGk>kewfH z$JH^q@3-5*1m(%$6L7%HoMr6rura}Mo4%2vtRTdSdcN?9kb&Wjy@vSDz}IMc`d@o5 zFAy0){cM^Ul4*IJkqZ$uHT|%oG-t@gZ{sG6$U=k(W8?z0hg2>$fx0xZNS$vRaJojO-% z_`CbeGJ;NA!#zC#9B43y$;aK@qVSUt_tm7hmKM!|0)GLU%TqfN$99^>pXcW2(u^BD z47)GhORRm7bt)zEtA7VOyij=knhL)L08(+WuifgU_2v2OF>+#tGj3#{LZaXUQ)uGe z+%rBOm&cEFq72^FzyHv{7f|E0pJVd=&u<9iP3f^+bbkI}SH&m|O;71<;)g=rarqC7 zK9JX$DJ5^k#2mlO3QcWIO4<$y3E|sg*#CsDq$FV7Dwp6%L6Zbf$|^B4b62BCcWq}! zQa8Fi|D{Kx{4T7p@V)DxIzsSMAJgGZxz(B9x1Ya^`m$dzR*60ov@(4jb6N#mjJ9qOW(6cJn3?oJFiM1=cYt+ltki7R;Zq{ z!3VG}M9=ACT}Xe=ZY*GRJ%&|OpTbQ)pwKo3qR*kj0lX7pHDbX~p(wNjUW5o?ykI=o zJj&JgbIqvW`vhx_BVndq3%)5SaafVJxP z!^z+W+hjz5M^>+C<`-do`KjJcGNhM^<&kBlcEsn)z3S3fXY=PBzkB#}Zloj&U6Car z)Tl2fwKqxFX{|3+#O)}3EzmiorqE(Q;F>ZE!-TMAl|(Njzjw8_r=F>1aK$rrN3O_~ z6c!eaXu{T`uISixS~|%$>`(L0pW@XK*>P+R{ZwX|e0uQ?R#pW)K1xN2S8Tos*`R{@ z$y5hA%Tz5?Oboov+nYTKd>hKA8qeKboti3U+BC|o81CiCRQ}rDihS*>mOpez5T5-g z{EKxO_4}&w(taat7taqX&v`RW2J)k1GkScA^b#3R5wezLYZE+uVimY2ZBGTz-BOz1 z1kbB|Jm*I)!RR$L5H(`qiNm-tU`;+m=Xj{l?{pXlaTHoKbRSQ8FG6qBriOR-8v`4? z=JYD>&PnLgiPQZjM_(d(qA6UM*asqPYfQMS3AV3k=7+Gb6d{M)k!ng-Ts*w9t&(08 z2=-hA9h`!a(#q2E^pqGEofyi*#8h5h-re1eLW==28}>OoJbd`@p|!QOiHXVkswWH# z3te%%b#mWIOH0=Wv+R0#dHDFK9vc2i6tuIRt{%3_#u+xK0eGF6U*bvYoDO*|dwR4p zjRzA$Nm1t{qo6q5soA|n5GiYHY`k2kQv-E(cN1tc?M+TfPA>GQudA!8u0Cxh6is|! zAfDnA#x_;sWHX%m^78Uh2(uo{d{2@v>6|YLq~P`ETOzWj&fLVYBZWGfOG`#(X5@G8 zN(rk!a%BsmX7cRkEihNQz@f|ECk$3;f51xgxFyABa`}SVXjciHK;CZrR?MZw|E76EC?P z@89vg3cg~+!p+?~Ihjzs4}*=>xe4q_3ME69yoJ!9B z#=*(S&aU)CT-LhmM|0_Pw4Q^JWD`lJ3kwOs$Hvz0DblTbovN(%@@4-BKP&48d>h=$ z%uHroqp5oLv$QZsJfGPuyVk)$UQyAMXX1R?#YXKun0U^qw@FD!$;h5R$e5gU+ZEY+ z@#A=8c(_2j>is!e2kRZ`LzA$a?4)o;X=Wy-=~@?6 zXr7!z#+|#M?fT0fbjx}eUr+W|OB#gf9pB{`fB_3dL8D@NEFXtERj?CvU&GlKu`^wB zUp*h2fWSnT&0#=seVBNi(3!mJ64kme?}P$XLzjY#~l*^xN*SMBvb%boIZ zWE~gp8JU=*)*N?Mt%t|7Yb~w&?Cd05 z+GHdot=h8k@@lMiBvEnU0FI7W*u+1ia@xIp`*v8hKV-|raJ5!%I=HdA`n&et z)1{>))+kvDC1>ZyT7x4a(f%)f52i?9-MFDb9n}jL!lY$mU@+{A=41fVF7@>3Q%g%r zpjb46*h^OZC*mHyY4yh=Wi`DA>nBNYVz}2)6jua6V!&WBCI}EH$;r)ZY?R3Hw`EAV zb-@hRoG8f3zISpG5D+-t+iUOczAJqb^}dkML`TQRBkKeK8I(7|D$sSpfofQ`pSe^V?25ZDioZ0~g%A%A^C19P196K>V zX4H6gH)rScj11d`Z}V1xM?<>0x{+P3Zf>iqtNV)SrKNX7^#f2K?XTZWS39U?%9DS% zU!;Vf^RpaSPs!(>uC}{>A7LwZ|B9vBu+vc>_)>G7<;hUn@z#JZvT)KX8mmL0&4Dh~ zy9d-S{i@ZNt@^QRafm^5iHT$~B_D<(W3rQN-anmD{?yL&ub5E%JW&Dj%|&pX1z)s9 z*;9ST?xCP)WA=h^gJ$-$hyo8ngT*fWe)<+a0GFY|Kkvz?jyfg59V*W5xCo9!5VmJg zQ=FW2reu{*)ng&h0DU&H=xmUmP&>wzAx>mW84ayd;52`=KQP35)Je6}o9TS@d{n54 zx>Zw4eySKd8_hW$l6+H)hV3^MPh!r7RiFIzMv~a)cok?+?3yG=M^%oIxR+_5#XxNJvK3J&W-{6MbG7K;EW> zgebVU2sUPY`|{;W@)LDT@NQqNPb=!{AWI=U62Tj;z0 zgiS)CnV9hzfV9<-0xglicxCExVG9e34xzL71|8~qOzv*SJIK*~o_C3lkMDjw?Sz9R zn^9a`{C>1hr@p?vj&iBQv{z9{NkepYW=2;>=VCsPRx$QLs#4SMJF!_&r)OuIci@gY zQ-V%A+F;CmOE>gn4_Mb1>m>DG*EkkLL_}m{7^|t>*?u*?mT}LzEB4{a%8G}F$JXMc zmX;PN=W8QCIU5_*W`oR++LDrz#!dZlu|yAhAiVYY`5aH4B)U3H zyiiuoH-_`>;!3@IdDlwE*T-iJmN{o*ZoVMZx;f`WhWEl!gO43nRSC^BM zlaawm7|bz+^1Y630qYUVti1tDMWl#}i%VR5e060dhgNy80?PFCG`KkKy@EnWS{fr| zog`JJyyTn3Xr36k7_NuLvXx_WSWf8S-2ZE0z#MUUuQuB|2GH@}n6XPdB0eYb+! zhdBHN)KR>I2rWffm{B@ZURJi+Wj_{M&#q!*()SAApQ3$(%n67cs$bUslMb%Zy!Os4TC-w7s|JYb;g|xM_xcm#$ zkq8#0w>NJy%eVFPkX*g*1gZJz`N@8yH^AANsw!$>M|o^$>2Ss|C30F$2O9`!o{$OI zv#QdFxbh{?n)#4*L~-fWcSQ!JPH_vUsHpf^uX^k}<4n)Y97sOcTl`&WXJkP^LGj*( zJ`!dEWte+ozu1Z&C;v?3V;5kWsoE$NxuaTT;b81a=}@XvMHiNVy@SpWLAw&ifq_9+#;;$J{^DYHzIt(Ua2R=hVeliH;0X)}P#3v7 z-ImM}t&s32Bu2SQpa3l3E;%$wMa58+ob)}vz`zoIus$`(9G+YV+Ux7n4Zk=EE>eYs zGcYieTaGjE0KUb*!1&eL`smT4FHzY%Zo1h?9$Xfx5i7t!^4k`jw1W}lI`~vS|Z>5NSIAezvtxSWM^kbOPew^rEU9*==-;{w1WKn zx4Y#o`zA+Eh?`nlGxPK1FC+wV<W_JLetWm0BlW+osY7~((1^D}4;_`y9nF%iwJ z#}(6w!#eaLrqSuoQoHS9YY=kmBH!Hlno}GYDE<01#qoWL8OOIu+sn&R?fDfIP97eR zT}fFvxjl3fFTyYRo}&d?0>&MY-fycR7rsQrL8pHMelb`PMia|DG`&!(wH9MaN?CRg zjNHILY+%Bl$1FN-@g6Y!b{F0gwca|57eK{;6Oj>}-{*h0)yb3M{L96p-1vhl8a9mn z0UcZ;eyh_t{WP)diy3l8z=K+lWU)2AyBycz9?ravs#P1R95a| zNr8&9q@|^aBF^m`QBvHG#%U37>2T1ux%Krp5IumEucy~w)cyq%kL3Gsc2REDx1aI+ z7HZ6_Ll6y$P_bpEUB2NwWNe)NX^zq4;WE(cyu7?e=ZBl4Q9ct^NJs zY{1d_sBla3#ihfQg4WQ{(aBL(Q4tTKH~t`>ytcK~Rmwe7+<{VQ{`T!x=*NQAqnFy+ zT?Z{mUzN|a%^u%~im4r3qCUanu0;WB@3XbH(%Lz|zp;G`N%YX*;NaN(%*;$qPEK}r zTO}25U7lq?XYOZ5FAR2P>TK#yGMziw(OKEqaR~@A3s_aV0gM*3PLc`}ryf-eyt~QH z?Vp*W62j+t@b(HRn;Py!NA>}<-{V6~kzE^bd{w6jgdjdIWB|zRi;Ih#xz$yt`Jb43 zV7Q)!0K5cIhls~{?4$H4TitqS9Ghy^26>9|S$vmb+v6k?=q&;*FG7mvmmt*n5E23c z&iw4`Scto;MpH~|ESM7PhmyTO1e<#c(t>|vWX{_Y{5z&lApG@3&%g0m0amjV=B_&c z7=24;yVN)ewautFF_9`mMaJDfFwpKdFj_mHt;eUEMRuZZZ`TF zG<1A?m@=;qaxE6`s|90@wD| z)+3vluAQ?PPs9bWk`(5SuJdHMRcUzoP|+ZG_jdjfUsInnWiS6y}Jv}{u za3ytjH1ZaG`ouTlpPA0Tgc;L3nO(oqAldXw$mjh0 z&?aBuoA{o~)NabvF<%CLm~yjt&L|(u#?0*ZP!2USb8qA7?ks(bQ)FbL zWZQL9f7V|74#1}-AV73o>bl}=UG>0*!A}7tB*gQA6K@P@0e!BQ#Ew7-r6oMNTJoO6 zDw^ZrvDtrOD1mA553}JPuyc-w)Sm#_kizpXTSI7BZw29X`xJ3loeqdGHkJ%5DX?Lx zv}uTmZCASDxcDbW3Ej42&Tg1*9D2_q7J>To#;Wh@>vNiQuc)on&QSv+yE9SNIXbE# zJGZ&%jqJ$~;hd_BDc~okt+A#N5H1VvH?-o3yoo_t1i$Qe9MiLj$31cSxTxWu=(c2){r-l+S|AedU~@52L~3Tg=OXC>S}5nz<%c7kgppjn9rKc zRlDwv&T(VbiG%|(RN4?gFyFg(lk!n8PG`lehuh{z!Bz%7U2x$!KkSplk1$NH;YVA80Ct{kfteT|q1(0haor*qkL1+4lyWv)XnoGmDg}bx} z@(hXJ>1;zorZZx1{E|N6y|KYr@HxRvymi$L1Msx(bVD|m#;UdGBu$zU=sAi3KN?7~ z0}pS2?kdxss>KjK2ED{gt>4S>_U;qY*d@_EZ{1D0$tX(l48d!)PJdDgqSjZ)ppwqa zCoE@VX28`}Qzf)}^XAzfT^*FGy?0L&*7Eze&1`+$*RNhVz06+ZzUM%fG$LbG-+S2g z!+9DuQv~(NsO62&m4#iHS?QFE?%gcm**PNi$j%u_UswQ&*UdAKQU>KGjejw zm zlmy5djvY@AA<)5cC2C}`h5azR-6($>0sNBhSMR&-8-ukm;8k3Y1= zL1${0@6oRe)DGm!U>gAI7@Y>Xx3@Q0VZ)7NP2@WMqNw(S=1FWQm>8>8Um9{sQQ_d= zHTDcQFk7Tt+J9XX&suV3)U`BB)ldFS7epD`cEAJGXYf)|YQmCvRhlvlFZ#wCxk-G; zI-QrB>k|Y(<4Zg)E^hsaMnAbY&9bXx9mu!GVuS8r`+&aq(8>4^>RCV6f^j3wb!0UF z5e8WL7-C>?ixJ%u1o&xVhIi?Y^k00y(|qW_vZK4c{~i_u3g@)CvhTsgU>X0*{hyZ_ z0|^^4IN_wo0O7C3{(1iwDmZMuq7)N>gh6Cufx%C6*PrT8+ieNJ3_80|!x#ZkqCkS4 z`2$En4MV@)fmfSy9TmQ?xL9E`tE;SxqIJdF+;dt$54VO;Hk_LEOS zO}wgo$rc+!Ij=XZzp`?0fGh#zYJz;`EG#WcfBazM=Ej1nsi~+KT3Ad@P2EH;A$3lW zhNiuJdvRq2WC@sKIvE)ldJ+W(Ywm#%E&<{$bo_+z|IHq{^38Alm+c{75LbWz#M9F= z32hdn2v&QN{>nH!529gmuZ_W$2Sn!Z{(W(2Y4FF7->C0X)2p;G*y4Xyh@f4WRWEN^mh^2x~w->q-+Uzp@bI9@g4 zB~?~dHa0%>VVl03aoY_F`!D%KAPv;+^Z&;u0$%+avHy0T2n~KZ==0y`69EEwiT6L) zCjv0MtqJ>|?h}DjWC-a9JWxF~}yCuI`H)gc!^@K*@}WztJ0wB{OL z#-H_P(^xF6^)6#(t-vkPvYrL}33L%~`E#7Q7DEP$8a(x1wT*v>1)h<{A=|Z@N-k4;Xg~ea61ONaQT+WNZ@T%ZCu7Ad*CbAN2cx0tR5dEH5vA z!P1kFkx@!6L6!obq}X%BA~OOh`Dw?Ovx~athzK`4sQf^M)2{B-ozp(o9{=cV*!EPF z-Og0i)ud!&PEJmOG{Ar4aVb?E1y@*20xpSTG0LmGHk7>OnJKRvC8k{+Z9!q-62iD8 zi02pGXVTKUV?TTgs@%^US2U~bqyGE?xvtoRgtL>AJ&0lAWaudhfc2lv&A`C?5BOH= z?Cjh>I(o#*3-Y)6w66Dul0F9k-x#C{BtvSdtF0$WgYX~|{ev>dzS|G^;Nai@)p_^s z-Q(Ta6akx=feblKT##o-tFDeeUe0})cL%qnAPWeH)1?R{=R8}fNHbvJcZW?(OfV}Z z#)gNB8K1^<*azhB+O2eTbjTVifb1Da(`A{(cc5H*`w0mNo12?49Gb=YO=v5%|7P9C z<2bMwG58RWrvm(ch~{)}2(9}n8BSgHH(AT}?&8#eOSzI;M_Zdqr}|eU8?RvrWt6!# zu=#Pl!`9s48CL>KIa{jk((tPbDPKjeX=@mfFI>tb#XGtBR=*roY{v zU-fJ*)XzGrlo%<>sLSa51)Q4S8X5gknFDT+HtRdRTP>_WJ*$0y2?ZoTe0+V)K6P~U0P{)E6J%&*ck=(4p`8JvuNM5Z7?|~-#_?*v zK{7G3JBj(}eEpqvxSvqGh?}FVEbf`gvP9!2Qf?KY=%l2Jg-_hY&>;*c`*!wFs&~eJ z_)xj*8aUy=ln-w=TA0R(`#{kEJ>gLWI$!bA0$j$ zKHl9jg+{7?BzmzbjMxj547Aplf@nDhBp?1BEPG&N;lL}mvq$N(myzk&4xluEA624Z z+7)XH{4v=h$sgI@*??LM_Om1$q}&!VZTe3Yf15Mn;#m60 z-MeGdDcJB26ky8%x3sGh~RD1szk=s?u;r>GDsULI%Vhm1#!-_QW&pwx3| ze&255mckPi)+d&aDNato*tin@#;_M3oqnDbxIgnyO3|PW(!LV^?LJ`%3lD$iG#)<{ zK~8o0&HAn;RPumD&5ilbjaN{j$3kKWPSR2~ z(iLWS?9T}4FdO6LRi%$JHjf@!wEgN@MQa2=hz7q$08zI@i#6DavEnDc4l+*X7MZ=W zd`PJy{k`3F5*QFa~JBB$jF<^K`c{xg`pPOoCz_&=ms=|N3S znZ=xi8w}Qyvwz|I8MAEGOzVQXdMnq%spIK!5uf@eE^c%kySxu!@TT8gVzZ$&S-qP) z@Pk3NSDdeIxX&{_ghgGtTqLQU9n7;%fA{$6rQ~^CnB)Yxg-$z-SZc>~e{BBe82=7z zJ?y&7xaX?cH)-rU5?Mtr4pc!}&%wa~Z0W{8iBWaDXt5$V_MDqZW*0wBAOZF1(WDI? z9$vc&uz^5rmyQ-_xNd>bYd0=#KvcH^l;i+&wfIh*T%Q)k-yTpJ3y+MvgFL4kpslHi z1Do$FLzY0_hoHfeJJV4k$|4x z+L)M{Qsbf}CQfB#9Ycj3lECdTLfu7zJy23eI0iP=BUD;P?G}k#b=`;Wvzxox+D5CK zEQV12CZsvvwE< z_la5E!tx174c9wtvLAIFZCgO0t$h6M+(Vx(y^7pgQ~fMfy6D2W9@*&5Z?FqIB$q^Z z91hDWaJ_KiyV)@Z;-B&GzTB_zprWtFspw6hmB}Oc<;AXF_0Gn}Tf3mZHVT#SYjBYq zvS`&A9TctRuB@*=gJj4FJLb<%f3-JTj*M9ySOACxy>zjQpPO%{EW?0o1RX&F39Ryq zxZ#ttJk8A#evQ(?cj9?3~XOZcXp*bdFrjnOt;2fnD%C3mwC=TMdN=i@QCkDUxQjE+2+A5 z2huwSS`;_;|G@W*Sh7f!k|v(j4KhzOiM0|;j*nk<#U<71INAKCg|Af9+m+FxwhVDj zWbs9bwu@7vMS8C_oZ`&~atZJcM{mJD6vxFOa@tLyxc)W<^IYXWyAys(hfuVS2zq+- zmb@5Q($vyAFv^o-{$4?=IFi-K#Xd1@#_d`*yq#;i#+gJCt$bFB*VCgo$2IQd6(d|H zIjvJ0+oT)+*p{lMc>Qb5n1SketI62bc*O172+4EzgS8^dvYi4@SNV%_iNLwhTdDb9 z%0XEaT>YvSg#kO;9ds@JCV6_x=Ckk|Di#EP5tNQ!)$+1Ov*^JHB=(~Kt4WD~A|u87>D8we z+umUTh4gS%55H%bG>IZuv+?}>zBkB=bZTBj=pNuhb$CMI>Kg7L#d(@bUrxnIQE87s zJ=?S#ch%8f1+Z8oaK75~>l$f)YhR^oX6i^3kG&QwVq)Ct`}Fe?pnUg1pDE%JG4N|G zGGFCe|1d}$wNFpKbw2A)Lu{|Tm0rL2#63YHe8ki^dVBF#3n$6RdtXN;K|#vOGtwP( z4P+%Q!=zG_@7c<1-Cd>ujTnl^8*nvbRs6rdTC+%@jj0ZfNQ3f9$ z(-YywY^%|>HVZBB|6XTj%CW7zGj$UElN%JSCkeHJGJmiTt-sB%H5BTKJb_l;jYv-v zjxjeil^{Vy*50nH(4L>S*6;UpEiB}ZjQmUx@VpSjt1%*^dJe+-eja_%A1fYJ^!73t z!mn?N>pg~_UNUoXh6YRnK++sLC z2eSJ8mmWp)bj^txui(iW+LVVAXM%F-Bd#lx7KDVR6G0!yl2+sax%~>BDP>VQ>g+tr zRsF5J*wb^=+WIWsIsBw0AS;VfU@@ax6hEv7pwuHp)xqOD4Q_iUCkF0LqY-vd{Y)ok z4GqCMvphti)Y=zUr^#}6@fQY$Ja31x!>3YWLGF~7=RHqLY}~lMfIwiF=%o{dm6Ceo zb3Gk%Wr{h~Fi^GMNC~R;zl)ltDkA>G3{PdM1*Fb*(D$(4spqbJ@C)n<+DoFY+E0r% z4j?~I1=rGXKj)PMA;eWZFo5@esT>YOBw@B&RXH4sbA2V!BlSVrPe>Xd zR(o}007Y*-;LyGSLd7@ejRv`}#YM8#?8)<+&{M~o*nmg&uW4ohOApW~_?3-m=xxe! z<3gF|2xpzw@=y5hEx$31=kirjy8cZL7KCSQs1LY7!uD%@$s%ql2NJctI-~sS3D>2- zqo5RcH!H=1;nm90Qr|_63Tbz^iR^xcriO;bCI|eW!cG7y9q)$NE2K}_mP*-I2 zKgxhXV#Ks5OGB~AyoqPcf!2Wl^7G!bL~XkEo>tDMzbbzzDJbkopsveT#m~COFIMN$ zCM0F6Lmb;6kWlqD(6vmqdrtIqI6TF>v-inrWYcx{0`=H*M311z^~O% NWTX@&3&r(){uk$fPznG5 literal 0 HcmV?d00001 diff --git a/docs/tech-notes/txn_coord_sender/eunrecoverable.puml b/docs/tech-notes/txn_coord_sender/eunrecoverable.puml new file mode 100644 index 000000000000..1b39166a7535 --- /dev/null +++ b/docs/tech-notes/txn_coord_sender/eunrecoverable.puml @@ -0,0 +1,16 @@ +@startuml +participant "client.Txn" as Txn +participant TxnCoordSender +participant interceptors + +Txn -> TxnCoordSender : Send(BatchRequest) +TxnCoordSender -> interceptors : Send(BatchRequest) +... +interceptors -[#red]-> TxnCoordSender : unrecoverable error! +TxnCoordSender -> TxnCoordSender : "txnState = txnError" +activate TxnCoordSender #red +TxnCoordSender -> Txn : error +... +Txn -> TxnCoordSender : Send(BatchRequest) +TxnCoordSender -> Txn : error (txn trashed) +@enduml diff --git a/docs/tech-notes/txn_coord_sender/interceptors.png b/docs/tech-notes/txn_coord_sender/interceptors.png new file mode 100644 index 0000000000000000000000000000000000000000..e8acc29e2c10bb69d2c92911f494cf31819bd13b GIT binary patch literal 48127 zcmce;cRbZ^A3sh?%SZ~Dl~8sO%BoKygviP)%HHeP$0!-0aO_b@M94gjtwMJ8I<{kk zWADTHT?a|({(kTK_xs(qe>#uT`&{p9zQ*(Ud|f{GWF?4?Q6Iy@!y}fwBc_0dcZeAe zkKoFY1K=BMqIU}T=aS8BRU2J%3kQ=YdNz0xPt2ZJJ+gW7_^gh@Spypzi|brm7AB9( zY-~+UICafU$+)l4;^7^*VyLKU^YeQ=e9(-2bgH6^`5^a+BK1Mh7mmlCi(UBcJM!rV zMLt``cjX(1w7cQ&Lr>CR+`FJ9p+#q4Z;*mEizu z)j=s5_CC~U3xWj>a)U?jLVDTnncWM!e2(MCLAdhxr_(;w-Y#Lsk;3^H{m>6j$=_Vf z(r$S7Fd&_}uA5(r@X2HL7s7~X?(3v0qO6r3RKBI^^JsU=qgP)E$6K)z-F&a`ikDfF zBJrK}f$FbG{#2*GI*v;%5ax0C&@d_-C4@Agyl6`!yN6R{hB8@9N!AnVIMo&BRSwit zd-9=f-W1}`5Gzn+RtUPL8%W%B;&pk~gM*|_EdLf?wMON}g? z&8jl8V@48ToE{mXuRd^ZolMHQ-sM}>&ONr_zxFB+$&i0ub=hnqRz3LBaX$ueg-G>t z_f?(0uZ&*4`^;u~($KOEL9%wUfz*8C6qC`bPnV0W!X8`+O(I5B%w%Fvq@5u(Ewdp3 zZbAO1rrwGkQU3%*7jc>%d*m~Xhv$YTDRxuQzH74RxQ51}`*Mt%yxyr-Q9*>a!e4pF z2m41IyVUj6um34~&`E=_U_Me)9nQxPQ@#S3)ZwN(W5sNob5e7;W;Gm_&-t8_kAIM2 zz%QoWWWc^G`B3qsq}UNfFTX2mnC|()${{2b;SruM%j?$|M4$7F@|E;WLq{+|1QAP2 zV8`PA?&3AS-2Q~u-=&0ZNZUkJ)95F5Ej(yQ3^>kzmC4F@`jg3^7lB9cqxX_$o(vs8 z=p+p?isW}4BA(S;z3&-#frThj3sKH@II)Mn^?hKU(1iWj_wU8brszUtkQO&AuGI2g ziZ{_Xt3ssY{8d7w;Pts;HmK3etKp|Nmslb5P!R`|zvul&My@|3M_HF!UVS4d{F3QI zLvlpcD}wn+i2u+A@y;vj_r6jEQNE<@JX^0A&Z)v+lV&AH=Oa`q$R3q9;MV^*+)zQh zQ=qr+C7Tzk$YA`0#`hC_Fwv_kcdS02WnEUe665@%6W;4M(F5sgomdRx)ci3ug;ML6 z?0wYvmH2SQku!RcA@UWe9JgRc8HDFNyjIdMAIc<1RxwbN5m zPfOX_*;#SST-cr_-KRu6GSwI3t#}k8W*Ql}47{hSYBTb7>!Ly zPChABw2m$SuO8h0UU8IO(v^XD39WsdhMCu~ad4cLDw_Fx+`MQpwCrL3zxz`qGpNEh zMP+5XvF~r%wV|5n?biQRkE5#V^jXMZQA9lQh^o`KPBYBDs;)~UgmvezEoU1x*Xku9 z(Sn-NNn+1cL24ki}uxLxUvvA`3 zZ>$Pl3${<%L5!5qk`UwuD?GI>v9qF;bvZB22#RH}LnFB%X%&^DYY{x2p`1k}`D^X% zBl+CiGtUbPWd^?+8(e=*2G_8%e4?K!fW0%f(g)7snHOFn5t5aUi zl8lc~x_^H#vjH#9)I!I|EbZj0teiMwZ3N09S&{$gvpVOM0j37wF!L)hwiu46)P6~L zfudPEMwJ0u?LvI=nooOQqZIGb{nIM$=1?bBJ3AwnjaOx5qOLP#Y?T2UqXeYvd3-9P z44AXGk>0$#LR>~r^=|3Yr{%;X`lepNy8Sm3$5mx@X9r8t(>8P16wx2f=20c;WV>Q4 z7Zaq@eJvN_8H6XRVNS@y1p;x{#gb5T&-(n)MVn2GxybrtZpnJV9oFRwu6_+Y&y!bP za$M@XB=u~i^z?vsiR*1=4E%llJvwsNLXYUc^7iWn-gpF&kLP278k4SErG(FyI5O^dwNRXwquQJ##q%Qh=JW_ zatJAJvc+}R&&m@18^YeLBXxc!D7?d5*SqPIOKx3Oi>E{6X$&PI@87hz>@rtHgOBe@ z4#Dc%#mZ_JMbk{$iD!qej>Y#qd)~0NGGruC{b?>?a4@dNWtna;1dSLav5Zqu&Xu0j zz^s%K99L7<4&iqgjUmX$fU=yPhV%&=U@zKn!jzlOA5(R7^staI$Z>XvUf`@?BSo)- zIQbGOUG{N5YljZF)fy_0J7C?!DMRykdi6K&_=xnZ6! z$x%OAP8nx69N^LIWz5aXXS&wQ1zl~hGoxqDu&N3EK2Ua8_p5ML=P_mLCm(!RdeDx} zD7(Vv*D1Y@UNLBasYmzKM%?5ooUxH>_^{k7cp~-alWpQ$Vw_b8cC%)%4!gNrLN;$+q{Q4pww?_QsdB8I z`uaf%5}mt*#mqxD{4ZrZGM~OQt9_Y^5bMq5^QFApMLPP$!Vp{fsfLJofBs&3W!fTO z4YKr~e@o;r3o~^QuDu4GqOI8^fw2fR#>J3$HP4dgUNxGWKF4nIULh%%#VAn z1hr&eR^HNLqd19;6h_$%DhweX!g<~%M11}X-Ixt*7FS2y&M`9)VqIDV$9h`bY<8m_ zf`&mgawqW}f}v}P?3N6SdavjP;l~>gb$;?+5+o4R=4Xr6yMzlct#|I(- z4i?STQ?s{h?N{U&oM(K$wfdg+K?^8OdF!$?kaV$XonHPxlK)YNDWSzKH-EV|i6st+ zY+ZamBYh+&y3pV&&v5PlHIaXEa-vPX>bUW7V2l#_PLM6v>alfH!K~Ef@_QC1tEcQq zbME?iJ_(M7H@Au2dk{id{IH?4cR%wWa-DP|6u1Tp`aS$ z59JCyKA=~Zuav|nl2#wxDLN?6JnM&w7gl13o4747=}d$Vle)+%Wn(3>l{R%&Atf}^uO+JQD@d8@vuXFylm{xKGHtz(e0LYV zuk?GWi1{eFvV4ZI!7tvKTfRShdUCjW(tOjQb}CI@#jMgP2_;S9^~i&W+7?A&n7bzb zcvbnfT~Sfd*9+kyF8QjQBPd&w4H>#bXqZz&Y~@#ZRyM3ABGMCz7^TK?E+@&hx}uhr zOCGEyhf_+6J4;S#*FUU%Xqm)caz>@*ZG%dSy23#7z+NX#9x$*AVi)sE6WK{ut; z=u>8&ro4TLJ{^=1eM;nSfQfNd3%{{?hXM zkgj@Vb-13NN_(m_rTJ7-f(qR$U{ZYOso!Rotd2lszG^NUR5Huysx&zyh`7>?)HQ<$ z7g!1Q-r~@<;wB=0{5^>c{_UlwxsuIo)Ov=fVucn0GN}sfvF$CeLkU=gRHY*0uPd!< zv%?F`s#R14xgKaaPbr$AvY}Ws*bV`>8KQ0_VOhv(E(@a-eWV^=u}L4jIhmdF9!yOG z1x9pph($V-otL_=cnK;3VOP0*_;5iCSt`;RcvbmExv6kAw3x}K9_pEH_Y|DXg?2kuO`;w3>k(H=& z%BZ9j($kAf%%R$cpSiocREnR zRa_r<4&9LWJX|w2-#}R#OLzY6E3A;`Tti|fZoWTLt3!v*U0*}b&9QtnBqhSUNkFM8 zx7|un!G2N2pY~f+U~#g~cMooNQN6k_f+3Tuc z!z~{ig1r-7=#i=sVK@lF+E`75#N-=0^!COct`$9LH~cNQNn4gu#dJKqyeogfJIN=j za8?{vf~_MJp_=b9E}_00uQNI@kX2YzM2iTOEJ0j0*LsTS!5hyA5)jTnwz9&T6ODL_ zi(J<8laj9eGyn2sX@c}LrThKfZ$>$9+BHW_SGoE+9INIm8bk_YG^q7z^Uwi;W-6P~ zguo(3M3DMgq$u)>iPdg)Hn?7>w`2cxDnfCmA~gUQ*_YTHQp@kywe+cK&4?HfG6>F1 z&md$`c960SjM)`(q|pX_IuTn#vyZgDwP#PgcP_@N9bRl3R(Q`jXruq|rLwZke-gsO zoxT$HC9y7JR{Ksd`07pH&Su7~NeXhLyLO*vLV2_YBDP9b=vRU#K9YhubIZlj~K zP{-d#b;_m&_&P4EwcFD4#XQsSv2Mb!ROCq!k#@g3iSIcnPCnR3VaT<5@_s9>!-$^L zD|z09D6rTv$JrSRl7SJ-X;wUl00{a?1S7Pi)Yi-M@C!2d2_(mb7%?W{^;`NUr_GOq z7Z53lGr)=It#1zTMO9Bd0?F_Pq_f>zrzC1bBsR*GmH!s3x^7I@;2vVN<+as)@V|GR z?GzA*z(gzSc`;m8yOuYsfT-etMODk7^aiu6eYM-jqH)RPndG}G-tRQRl8}mKto*R3 z23clS{xf=St%A2Qct%*195L?u>bkG`EFd3cycGXpoyLomTtopu76Dp&Z^bhUIwB6> zhOIXZFa}v^z6G5)dVNo8y2QV7*quHESIc-c`pa)8-Hb4rS09k{_YhHWjtVBCr(G>T!`3v~7O8(mWRSK3GoJ$w1;}d4pY;c^$dn-0+8Y z$XQxBc^_ow7N${3b?(dE;49S?hpfuC8p_$z5U=T1$Yz_mISv?T5MI_8Z@E!;f7Z3U zfRBoZ#7j3>au|Ob*f_}Yh5PSV%$ zFu~=73v)!fNKkl)JRj!{xRRo|i&K1m{fZ5)m5xJ2)P0LxzcrIh%WwAtJvnK^ed4~< z`G)O@JS%X<>+fjNVPa~8m7fP^E}6_mD!^zq)H2xmi`!WUX6efXPfx29PW1L_f&N@N z>EPBCJQkDkU?h@4fS>k+vyoo-MAN&#OiOnFI>cO$59%g_AYLOpR?-;rQe2Gkh9sJv z<{Q@F`lzaIW0Mh4_VBwlTWqXly6cq_KRZfKOA9(0eE!wpBw?RkMC6k+LwD;h_+^QD zUIARLMlGA9z?1rWRqxBiX#CuX_}9C3*?EUXV$O$|Z^%i9E>=ZPyp()}@Mf zBtYT@5uP9(*9lr=rwl9qYd(sz0{vts2ueu`@eWpQe}m^S^z)m;to&8X4sM5zkk}vm zy8XK(q`}R(do;&S(9Rt3#(#KN3p~yYqXFOGmCgSAh5#1zI9RCkz`?^p2Xx%%+$fKb zD4J!T-EHX;0T*7GWpFidlK&yMeq7sI-?&%HWJ=OM#P=Ye$G==kQhNL#=z!YGpAF%4 zxKq0I=SiXnTPhCRumDYOe{%!H+9BV`RhP;^gM=c6x4yrv--24C^kmT3%c{bm-9dqT?tlSR!a#dD{q@<*>GR0rs zs>rG&Cl}Znc9*$1O|xL6HezUcIxZ%rr@fs8GBP(ehp-+^Zv7D+9-g0n-Ej5$_wN<{ zGz13@WUEcZ3OfxO(b3bp4tzfJ1jWh8*=E4&?(Tl=+BF?ELkDSTX-CKW4rD`v`RW`h zCMIT%*|A7#GI$t&XHQJx72dFTZB-CVaXp5UhbQa(`--`+gyYAKojiH6tE(%&pg<*6 zEyrPA55bd|kZ^{GQ2sLnjXXm`!^Y89?Bb-OGg$6N+1=eeG&ne1&7oG*XrQQUW_Z;- zyfZ`hdA4(Jp+jqH>mVa2Xf1^8RS8SRo(jCz&`y%Fms^Hy?RPkZDDC? zWMrg1PPDJGQX;<5q%-rj>loeX)2&D(d!2T%^EGz%yXaavI=ZI9u_miCZALl7#KcX` znwl{hi1OGvR(PSBi%U^!W&7Yqux==Z@)4t)$UnxJK*1a=d8)rPMWqHC88+6`e9GRz zp(`J+hxyy~iL2Da zf2^S{N19gAkBJGkItg)cY9d0m>F?qw&`E!P|E5BN>1xq9Wi_>A4Q;K|H*ao&C{wSx zx;j5U-|PAFf`S5=#HCA@9w4lk{%Dqk<86FA8!M}ty83f3FR&tQsp^T%*cCudG{;Yo_s=*S2&3rkJOI}R?&+m4-?PpTJDtwBLSxu)IB9Hd8&en?KH zrlmc_5*!#RlT{fzW*q~1r06N7RGC9Yae9kt*)#*eg1rOc-Y3)R+1~??)$fI z-zFtp%uy?{82CI?8Q7hUe9MupT_SS!?2Y)w$({lgElz5ud-v{1OZyeNT3TiZgsNJ9 z@gfFW5u_6==NTFGYr`))EtyzqILw39?(ctCQ-T)%{k+~oNG^pHK&G|wZ9Z6c`x4L3 z&ufZ_>#oC=7+b#(+|rPs!t608z`)34R-TbZ1QLqao7FTI$q=^a%@F)DZ#aWjMK8A? z#~>sitafLu3l7DUm8ozrIL7n2OKhEfH#&R#BLPv0W){V^Dvsr}3%=Lr-#TrM*k?WO zO_J#+``rpwwB;A+KK-QQ*vqx)J$-WZ4Zjh&{gx5jv0c}4Soxd9?7V}>G2yD$ahv> zI@WSkYW*Tn&&}1G9n0GSg3bTzK9zOfQouEvp7X02e7HVI9NaaZz@U0%G&%f0lJyB> zRd#KSitIPzUhyWxe^tPE49u!wCjCvW0X$< zHk0zysdw!YiT0@6l1IC?-hI^Q_^yM)^)?_`Q%fs0GV({Ul7q*Ru8(Y@phZ94UmlBj z^-5k|{w9k46*x;DKe8WaXr;hS-B0gAL#rexB!v0=?D$+@deKss$D!kjtgSJSUIY~D zYBLPe5`}#x$1q;&C8!}mHycyclwXxlOo64?>{3WY(Uc@x-TB~bnM(blz zlvBz#Cw}~xR5Vi`W!L+<3A_U@wG23dg~dYitD|wiHq^%otDCSXzg+&zzqlwP zBV%f63Uk#cw9m*^yLt2GKmYuruC5M&EGH|a@R|3TB(fd{Xn)qU^Qa&U2Gh~PfPKu= zbluRKtm6eNZ%Tw?^Ek(tJf#(dY?TrS!KYat{lS)fVpvc;tNJ>F6 zQQt^b>_y6!q@<*g5k14rIaJM~M~{}+;Mr>S_A9`$mP5-63j8fp3*hsBJ}P2aAptQ6f`83IVmvuDqA<|``1;~Q1g)r}YX zE*vKzp6R*Ic=5r?%g-2T)-cP`Bf{$Re_5alM)hcug#7%#?Oy7sC-a7PQjj(mKI9kOSl<` zT?vG55MO45PIr}6n_h|V_4NfVs>Ns#9L7KfQ8Nc>5cI*3R({!4q|lWw+Ss-BIcYiJ zMMXQiB?C4%SY)0+p*io85~#g>UdQ~4Gd#6?n8~aLdZ)<@FoO|1`e@+P)H;wzT3TB6 ztI)q3eWzt~k8~Jkj7MhSa<9`{d%?{2-&y`*nh0BRE8_!2x*8j2H`k|GE?mG2vxt@* zxI(%U8K0otGNHmErzj}Ayu1KUoNC;jrnE_5sCr}MSI}#wfWXhmz&k+;!eBhR{zGR4 zfWPE3BKSR?3pQzx?R_VwdV^%!d2N|0+=EqL%uekId2u`Vwbib*#W|bTR>3B${1kY< z#pl4xs%XXz@(B<lcAtn)=jzn`KM7=kn|CkA9*#w4I3T8r+KHrO~*hiMTZ9 z%j`;pj;b4_Ul&a$#qobjMS_Htg4%80XE1LhJ~zw5ez)}tKxXH$UNrn?ozE_Q-}shh zhN}W|NPAAOg?fKr?JqO_4$t`QwP5mcSy<9H?&K(Du^4|oVCB6xewE1-=5uiI3hdo< zML&u=%v$s|ff{3_IC4Bo!7+;Hj@V8NVR!bxc0xh}MnD-y3WpW0SSgApET7N7Jo}`m z#{VlIaqq{i4V7FDjI=1QXX!*L&L`ts@~>%1>}t9--LB*6%q}J|!925Xb%xUI;s)YF zKYPY$ZXCBF+m2ASla;?`S6gY?uF3gJnzrMscRcgYy|R_2?OXES(zJiEV{w~qcRaXF z7sTH-EVIt)LX_0+>f%~(} z9?7ZL*w|EcIPk4Wipk(SpJr2TX=!<3j^gCvl9Q3i(Lf7ak3pBZ6A-!XOEuB&?5wQj zx^j#c9XI;>Uza`XfSYUE*et9lqhGvu0nAK>B6>R-AQasT#uwF z$b=&z9^AjLyIRu(Y~azOM}e2_>FG&PPVFjm$U;UmH#HFvdR`n5grHye`@=%b`-+7x zU!Dc|+|<;RrKP16iXyewfR`0`$H*JClN}ki)I`jC3xE7DvZNLk7LMXGRm;-zFZ2<3 z*>Tj9;dgtKEBK+OyE`f>N)Wo~%)r3lFP`x5wM#9|OXY2gV0Ez(k=X4EiL}Z6lq^R}CU1#UQl9Cb| z0dsS62l67V^hSfJ_19nbbQIMYB}p?v9R&b$&}}n0Z6s zigEcBH8u5cB2wr5$B!RVQjCEcDJW2P2myNo#-?&*1^;)vq#zjB|Hjj&Pp77)-nw-Q zEXhP$YV){FlQi(BK0ZEg>1q5cy1LX>8P+nD5Z2b#!NI{O?WWK$zw<@FGo#k#i~xoJ zf`{Nx5Kk&9DG`&9_y$YL$*IsYF)=;CZf^W#Coci<0S^z4eqAJLXo#GiJ~lYGxwVz4 z=<%Qavc$?Sx6pNC&CLxDtQp9`91sSR;G}VIaLlchmzNK!>eLa_@Oh`cL$Sz4d%t)Q zeOb3+vNNmP5>*G5+tbsN1DhY^_t2!I$@QJ1{X_3k>u3180Xoh@%rU$76hOE%^IH(6 z9Ef;=@oj6M8l6MG%9QcG69qEr-YMp7QsgahH%+Np|*-GUY_5gd^9;5VMfOdNVlP!-_5PS&O zBXzi$zqj|j4~lQzykS%RnDpVpnZD}n1kVV8w6wY2<4$n+o6xSu*&0-?;DeXZ(S>$X zYDHS20s;oyR7+Fclyr2$#V|oe#`w2yb=YAA@7`TtWAocH?LXhj3QJB*to$L(9BkO0 zrfEG=JG;ExP+yOpmXmC+#dz@R2}%A zTKPTuaw~$cV>#~VNuFhfQH0!R$C zYzz@Z#yMSHUWA6lX=$=)NSy{fn7USKadGdDA!AErux@jKI@URAGJlM-=|FJRg<~^6 zT2oNaI%%76!()Yog~RojoSSJ$ZQD7PylVyqvpQeB2utxveYeV^U;nkTa(#VW`uY6Q zQYb%2si^(E{#e6|Ln`T7z|Fu$7#SIf2|Z!*rtwywlOV4#)XIjS;^ct#Q7Mfpz3uz= zv!W;po+nS9XliPH`SJyL((3q-kdTiWh@d~34LGkYp}y3^k0D1_A)8DpweLJoEn;^ zLmn@Erztyg>gVf0o{XbFbWMA79M9Q2SPUXof-~3r*^=-x_J@&UALjCJb4dW%Q6=s1 z#=WE1X{LUDHfox7389^ad~WV7{3C8=G;W#>z;l0jBf49w5yqpV4efsgusKEn&jY`R zlc>k>d=#as>vkwxKi0%P7ST3w3&KM?gba>;5jSOb+d}6CXegtE|MtI^ADNWAg^qke z>6#wYzlR5qvHuV8H?XfDW&AS${|%JKVGeufB;4YHZ@}tqC7*wHGg~wYE^h&D#;vNb zOLqN-V^mmJSWu7-0!e0bUDMT7gdYFa>EDT3;Zaq;WdXN$C^HBe?oM#o4<(9_eE zj1Q9z>}I$C8Mk05bNV;g8Saa2#*ACM)lZfRa#HBuq-Q)2A~GS2I!i8S&!xa21E1bNE7=11x2HxbUTNhit6P^`3s=s z15QUJEnP##gsCp>nl-bCNN-=C_Vs8u9A4<7mELZ^Yk!bPX^ZmwNpPC6uK?Z$Bw-1w z-@e`PEPIH$V)UPz`;&ogZ;ow|lslAUY0SaEfPl25Bm+}ZI%;YoV`C7D#Kh9fBiya2MZEbDU*zaMf)iEGUj*e!>2IUrNoEm>p6MCsVOhsBssw>wtDqHBng?Gux zMiv(LqVEFUZtfyTgx8m+!GUiV|7|Gf3oHNYpuXT&P>)-)jBVm>2ha}`{hhGsQG)_T z=Gj=isY5=zs`NEpn*R>Py2Ii=5^|VMYyg(`2xIcDy~k0@zmO_i>}nB&eUQ8QTBUYk zIo)N$T_V{dsFvOlM_unS16x!gcW#^Tl*r``&Jl@3`ptG-xbOwhg$iImbheq|ZKfJ> zI=bpmV)E;EeYf>hwv7H|oSoSB%2)t7k@T(5A*^s*(5&WMq73%(_t~R3!?ne#10S~? zBEPuRm$sm_(6_wVuhTj`dxR=^-ylD_tbc!Dew=Z*vJ1Bx_w|{?W&P8dAPDOJ95)qt z#S+_^K8ddgnr2sqm0J1jTXY3_C@-1*DxGcnb$3Nhj3CFmi`_~lB#F@7E{9k$kR(@c-&kFof z9VEVx06FOR_&5llU0q#k!nmdJN%VPECZDH`{VZ-12Oya)EwBEE(zJwmYy1 zzq;w-bu&xk4p##bZ89=45fKrc`Sgnap%-FRMYE6$N2uzOXYRU zd;z2%ukB6#@t+U9saq7lG{ zo;&vjU<(BWPOZsGQT*nif8aWd^xtGrvXYVrkb9_#a&a}ZxAR}U3OHn?HA{g0*wxpy zkw`cv>d=i3AJWmQ44_-`=N4T3$;CtqUcb&~($W8yq1=5h0hG!hjBmp1A!AfN<=;a; zrS=zhy@LGy{WbgBw+vGA;_>nc3J>VJe#C+tD>5o7VHGUs^JmW_=hae`C$%HfQ&MPV zZ`iReOE9=xNH;gOPT8Gh=Q$9s4;FRsf1Vtk&Kn}BdEOB~8}_CeM!uI|{F1XYRvW=H zJ39-Iw@tOJ;|<3vO+yuIyLQDI=O2)+pnYykKkuP8S+=$rEKN%JU|9Oc9?Tl>dyf;mu2&D{e9b$?8C znJuT}Dq2vubH_X8S%47A?%fGB^tCHjnjUhi3XkGa4G{gfekJgUG~%UBSPcjWXjC>6 zR1i=>pn@q`tK<--XD)-_9yRiIN!-M=|Ua=fH?yCh6jw4e|5D1x=p{#CHW?`3R8mAjG5WjgY}p-fT4_xlCy>Q`Bf&h zH*LqrNKetw6j+begO26q@?`{r4yBCpg;sOKIV&hM0TOVMh79l_DmFK@w0eV>q`zL* zE2;Tg&#qeziamZj40sj2hFDES#S8Epz)As;hjxMjhw7)-`T1S-|2UA#OEEYMlR=@_ znVFfySVyVJ=GlJ?Y#Trv5UsE`bxA-~i~1KsrhCa~%t0P@_wI|1D!@}c(9#m=sy1Cj zRwy9+{QRDY4PtKVhnD!7*LZt)c&M8pZg0+JHHZLCrxHj*Zm&{K$5hwt+b|*XV@1oo zI(P0D zQ?rfw;GX*3DdC#JnPZhCr`O+)PZy4LFC5A7pI_lNXfR_xz# zB@4lp(kuV5iAk&{JL$#xTR493d4}V{_%pw&HQ1>c;*7yoMt+XiykeO+{N8T`eaOD|i2wdU3CSpB)lS}RAX4s`pzT`QAFeI&zbHw1r`P84gU{}fZNJuz{ z-VrOZ8V4XogPwiS>JgHmY56@2&gGq6@4Aj%C~J#6`RU-d)9{}|+m-_EB<6c?y=~PO z;9U0ptc(no;qA-$wgLnEy?VM`U@=@F&bzsI?QZQpB#->eb{w?h=y8PLeD$s@14vte zWT339Y-3|%<=egTEd}7tUi0u2bZ7xy&~~ED+xrMi>NGWVzD~JsWMt%QDiVc4efaPJ z@In%a&2$15cnterW5E&_6dVkoxRmyZ!-o$O6MiXnSqJi|O1PE&%acShe{)hxHIki7CH#av6i-ZXihq<94z~?EW z)>l_Kw2KR~vhMearMLUJ4F0VTvWYddwI4rx$iO@q2d3-k)2F$)xo9+6DMi_|x9|bF zc3U+OG7}>Rsi>%ss)3vL#fFF51IN9zw3L@80935X=DgG}n7W$GSv|YSj&mGw{N`G= zwmAa+fDDs~;!AyyCEhDWduQ_C-zMKsX+3zDe`yThgurn%Id5ws=H&JNUnF^do16c#5^wg8u|Vd=7RH++A|kT%>gHx= zkCKu$R8@`t_z|RaCF1Vw+qVI*XO!bO@cCI~AOl<*aa&!h#MK3CN%8p30!VtH2Jh1A z*RMlEDI7Kck!Z=DF#-fbUzMBY$XR5?#Kr5wFY5xy%3$8yNL^hWIFO;x9^(Be41QoS z@;4|i(8rEDSKCljL0E|U>%Aw6I6`~ZB9t;x>5HL6sAu})_%NC9svG;0fte!}!OP1r zI3=FFi&gKksK81x>Xx}qY0|+h9qO({>UW+H0p!Z3b?+bUjQpH@PGd5EX{#llv~Q-p zjpMykdiWQSWdQW~;K!f*Ef9Qa|CRSi;^@ZUHj_iOKLGD16ObDsFWD#w%u83trE)W0 z`Be%9Mh7aiLy2vi|M}Zn;emugD@F@}0Z3Ztwib_^V1EkZk+C?1$*i8Bpv&Lvp3^n` zbBNr^*X3fIdK)kqe3u{oF^|6zKxV=hQvXNI-5xb0&h^OSRNB7;BY{~m_?NA$nN zOoc(03S4|YSHG2Q?}RE_eF*;-*xoPQH_#8@WE&9Qn{sjuK!XKDNtF?02wZy8IlTs=dDJ|lbwtz6Ch|k zPRV(Z4Fc45;_*j`h-#~<$kZik)Ya6Qnw!fava%*45Dy|toL8+>Rl|`HApLvi z(3pU_^0C#3hMM}_DiEXtpCcTlWaf~ww&s`;m|p(inU`0!sBc}X(bYevn?v{El?X9> zts=)9OuB;)r?8AWm||o^i5&=8P1$BZ_8 zD<|^q6aAM>|Ct@g1XNFeV~7*-pig-MDWATcUVY@%n#xKRAt50amc*14X-6O`0r~-8 z{Xew+cy^qUUep!oilQ)e*jQTx*a)@1-gIx#oK<6or-z4{n%eLpNQlZazDCmUnVbN7 zQZ8D6?97>Kx)m=vv-JIjs-yV%`9Z9tzG=(9-+~>`nLz>d8}6PGWUkH^492;gva`cA zVdKu?OzDaEpf}r^>0L&dyWYFnWwPzL5p=r z%z^iqd}y#$FRC_#f5U{5b}iZp+0VO4{1mh9cWI8qmbbBXZPCb#>!;xw;BKg)}TQloc>5 z#s&s2o&5!3Em z88!NoCl!>Gt_lcTlw}5(oCDAbglw!Y^K2W%JyBX-FkPZtTzzeA^i|`~z(6^BdwW?~ zSrwJA1(1odva$lLp+ZV42rz_PF?k!+99?q&+ul<5*Yy@Tp-^u#4j(#{lb!u-0_aHm zbbYS!n{=E7V!=28%kx*RSXI0@lcnE4Q3>NWH!~w4A<4x|m=SFst372FD^}YoW?M7$ zj7P4n#S?xeyArN9H&=EMjcsWbF33+iCMyC*jRQlkb;9*G1mY7x1^GtvVN%&Z3Xc+3 zY!@j>u@NxiTt`Ir#>Ka&Z9QeJYvD5+7+GI9OU`0@_n8*PZTegl=>jVzIiW ze`x^*C?p^@{yLe#_m|oERcJHoG4qq3-XmT9nafK2s($><{ryfd{fhMW#@&DO3Akd& zeM|656}Bzg{vCVU)^>A8#60R!Nj(PuaVDm6D+D7bfCRw+u55Ep5fGsCYuO;+NZz~m zQY#&jW(7!vLx&H8x;e(*U+PAO9=`qWhWh(9cq50E+s9EBC&CoK>8NtP2dx=?_lByBCGs*dwY>cB&cr$ zWitl6ksRu|-$;voXJM*h5b^Q1A}KZ1IKP1C8PqG{2$^6uct_R)RH#ZydOCbrcu&*uf1%hid;K2;`F|{8g_cHv%d5nxlGMU8%cSNaQ8f;0nM7Bgizqn*wqaGO4sZDddXpBj|aD#mKdGC?> z-`^Sjj~2J4=eR>(o@ei6z$-kBlRmTWW%C-Yi-FJ60%6s52`aVn*rz}V(6UmK=^h|= zBKr3syHj$v6}xuqf9BM|iOAIMMX)-L+KQw5DfC3Z?D{${^_7W@2nQUQVU_S+Sfi*K)* z3Z=BIpzBf_$8CT@ThWY@Nh5C$2VkGszgvLeDoHP`P9GLyPqmBc|M~pK{%!D{G55(~ z>BYQ{BMWPs9-fIn`A$MBa04}>@c9|dubPo(mB(UtN-m3+e(Gk^(G=ihT}K;UVR z6K@w?0wp~SzZubyBjAokr=`gZjO>bWdQtAfvI8(F+tC;Zy01jrQBquMwNpP16b8nK zxU7X~V|5(%;x(0-UGfPG91!C_L7)^{6$bRHVMltNiiDv5+Er+wb1B;SAN@=!7`c2OUBGwgL!920^!-qE_DS1LQ7eGk>U@E{p1VAalRf5S(eZN*t zsjZs&y*kEE*FV`RMUfTNo?BE{_)*ym^~Tin%3wF|dSja_t^AiC*R?mN{7^uy`?nxG0IrnzV<&3;tce6(b9ngiYI`aN+cH=< zOg*^Xh=|Dhz$&!EiRM!hTBso`?B%^yNw$!uZwtgOJ;p0gr|7S*WoNuIuaqLf?l6-x?}4&8D^+D_1Hn;o#I=8 z^B0etnj6_5Wm$zBN@n#qtL7{sC+r|gCmVC!S+vA@CQA2BTwGjc6Sy*gPqh-Jjutw1 z?i}Eq9smR^A}p*O{I#`(1_hmm$9?Ml{P`ctJ39Ub<={#NKCEfhQFGI2rln#TCkY*M z(dA&TI@5zyRaI>zGXC2$|Jtn-U!s)iHG#!|iMt;O!l3ajD4Ru%19?qKdNH`XHnn|% zlwJsMPM~N7l=A?-j-KA^N<;_@6dZws1=Ca#n~JHJ1H~j6*xK5=vs%?_pti-c?6qrri3(AJShJxy(muHP!?gIiZX~6L+`lX!W*b+@W92i=!(341R&~<0&~@yC>ma_5 zFliFUKOxY6go?lhkMjsEeYSa+qYxDHAxXPpBQXGq=|OgZih9HN?WM!JZh?OgP2^Ir z;Nz_TA00NGk|+au@jEn!w#W6|{rK6L%BcD*hA?3|o`GDxt>N6j-!mNdF>`EtDAqw| z&eW8ZKUk*A?ZB^|dN%*@wg6Bp@{;eETdC#27x=FUC~tskRrE3=g!VS-UQN7n`?I3i z&IeZhoewaYoev`ts1f+7-KNfgZFsYPCa-&#bN{1mKbr&Mh3y}^IowCAavXZ+f8eYI znAICc#_%*w2neD15H&k*-hK476~Xpz?jdmZse`|VVC`L{@eSkgj?jYbf8xM(Jj#P4 zfdw+?KORj&ZvO;tcO2nhNAxM&z^!K`88_k6?FAh&3oiKM(Z38CNNE0#LoRhrMq(N+ zZ0!NC;m#i1-UJY!xISxoYI|SYILzr*ngQ<5`S}!{P5*%-Kyh}LFZl4qvlgYGTU*m{ z4^ib&#rPlothT!p+{Xc`(yA=@?np^p2R9~cdxT%>t0O|#+ujZb)&fwESE7ljX=q%W zoxx=SNk|Q;JPU$jM`$g;-T<>Y&p89?5uZ{p@9;A;S{+tjCc`{J7lImv$s^l*YypIB5i^Wl@UN)7NKVV$!5zId|@+ zvY8WC+&x4pAfZ6<3WZ++AVYfZQl$F_24Zb!GhRkTrF`rIY_WZp2%s8_t*oqu zmw@Zm%X;7(1u$Pw{Xt?UlE8@I-!ya}yYk0c_Z~uzk6(LZrCner^MGQaycyK`tSTy3 zg2WbxBcH4StP0$V`HHK=W;_TPrIjAWUokN;@dTB-Ja-hfoa?Gy1pX-sH)R(E6;oPwBFi1?Md#+{Kxu#DMk$~{KGytriZ+`6 za4~^XIS+xTA_nS+kL!h)nr;d|l(e?UK+#9sEP$y3S0XhV@Pe8~fVzdIpEhUdQ=#<| zqK}%Ga*5)DY!8*P=5{N%KJL=MXX0NcrnaCIKx-B&GXo!#(ctEqtWJ|G7Io1aNLY7o z4c%n*thCs*^p77~jB@61R};;-OpkKHJ>jvR<}cAz70qhLw+X%D&ZI@DSyP>xg1=Xs0ccb5p@gdQ7{5*^L(gGuDBdrZ{mK@0#ebHD{-6`^?UjQIAS>PL?d~6OUa7w^ac)S|Yvb9r1f%X63cZ*{7)*Gl#k^KVw^$H*>& zYav{xo%*si+rQy14CG{bw`X$*$-38ccZsLxL5c>yO3j9UDF)Xga5CMc-xHq@xx>RE zWRu=X^$=GgiKIFy0*B}uqDWQRP{rp$!PO(7?4oNmTMgs<){dPPvT}z+ul7WkLzlz5 ztuw8#BD#G}`C(4fbz)z~K@Pvy%G2}C3wlloq%ZJoh0GZUDFPQ2D?#kG3(V5T8v- zSe}4O5^~qixshn#x$A;EA%41Cyd4)4VZu!MYkk3}cES(Yeq3CIi$5$(6cFyXNGkyB zfvrdjpk~2!!}!1a#6gmd{XxyXFav}J`@)P`HcD_A+RtC^j{YV}Z{-^3`CYzk|GpL7 zaW`{nh*W|T*WL~RHJYIQom4Av%9OEBHoox!3kxW_=LR70o+t*-M!*!MOaZKMx>nKu z)7+PUL*2IR_p~o0Er_J2O?IUaq7BKu?@P%VLxb#8Dn+R5*$Ua0EXCL*$sozjWFJEI zWo%>q*UTWQ_j|whf4txKy@%sDp65Aczkk2`zOL)M&hxx-Amao_8TDig`aM8+0HiZe zQx_AFt8&Bfb<3GnXOF*TC+r1Q|EuZ<1VTnu7UVQKZ}f_DYz74c1xM7s#RPC2ICzi- z+YCCE#CjT9+KO)gDzWPp0K&#VLG=DAw`go_Wowt*yXZVW%>feXiV8L;h*DE!)peDX zA^GOy-9brt&S5eR^VsePN!SiJmIH2B+W<@qIshyDaBYSZmg>fen=q*SBiBfavHCJf zWgtBhnrmv(z}MvacVmMppkaV3T~X1U@13AKZv|RFRX~9&VG&@lh6a_faY*b@(i$2s zn$2kZ&j3aYJ%{%kmge$8)8H4b+zY?%gBUFob{wE#=qX*- z*Y}SFH21o?pXKYn!O5*P2?Nbd8qd;VL~;b{4t;%nfbbr9c$}j5LOaO2CAVgS{wIAA zq`?qmfhKt31d-x}WFBWMy~ls2T>N@|TiZXpG=<3a`L4c1P`vsaIQzA}{tT3~-@b8V zfjiqmJw@TmGQ zS2E@v0Fz7t)>qiUxaMP7ndi4@98OhL6-<3jFhs}1gp0VgT~B+Y@}sYh+4gx*P@GeM zAmk=4hJtKtk}d&)P{#`jw*KS$5rvclhhEVVGjnt00_%agcnG~zR8&CGiHtnltOU0g z4X4(L2x!+0fq6!3?c)ZA(T-vkbF4fqge!o|?Rmt+bmqkteiw zk+%feWyeFxH!LOp)?C(er*hk!-UckpPH_> z5)lkF+$;LBb8?_gKwz@d^7HY@a!6!o9(Lyc;7m8j*h-LQ4Km7==Fg5 zd-z>+^bPQ5F@4*j#|@nj8JFC)Hl10W$kCD3EDOk>03`+^G5~Kp!~Q7No9bil=OM;zv6~wY6Tj!r>`4EGzot?ZNMFDk z1mGaHn<`0a*G>f!9q6dqA?gHhXqvs7g#I}QxS%)h5eUpEwE5SsUn>nW;*!X9${wB8 z`g;GcuzZIpBX{Vn?V_cH3WC^M1FuQoOS2AwZw|O_veQWF>vuyL*(1}{-7Uf>&fe48 z%Q`4JK`Yx@2Bw$L)A#OP+V%0pZ?~?!n}U$ocONS%px=fVHw51afw0epSeuYi6DP4~jz9}c_A`I~eGCkVLdo&*)D@BU z*(jUAjld8)Jvo`F#=i6RO}vsi9q4x~J1>HE1`JsnJn7B8ef);O_)Yy<{|G)CIYIN` zFDc>OgHzKx|KZh{^Pg{{713p+qq7518W1UUCmb;*{iN-M99EtrxH=*-o?R zDvA3>P6R*!jb$7uf8?Q~uFk?6AZTK2EEfQUAE_kL4|!u%CjR~pYb%3Qaq-M9){_hO zpwR%zJwHE@*$!(U6mS1B*Os4b~pu=mGu0%mz2$J;WnH zkZg!W`{O>k+K1SAK7G0i98;me>*2B8U_(?59Su#4LOg&7(#Ik50jnjTZ1d~Mk7F&4 z(_-zRU@GW-v}rc5a)|LsRamRwZW+oH(Y2w#-TdJQg_gs-+bhj{nbE4YY7vUh@sM)z zLd;l(p121E5DNoTG1S5rMa&{|5-8conXlXGcgWNd z15XDz;U=feMWOxKV~HY8GhMFiHa@%9d_?w}OuCc@&+*1?-o?NAGe_Upd}?iZji`Pc z+a(|4aJQf;{zAu{)rg=_ocEpqT1lvY&V{Say|eSn+27(at)o56a9w>t)$HT=CAHqV z4vmd#PF17Y8l!^gaw^Y?2y7TAJ?SUvsuoCFq;kMdw#eCXW2D zv%T=6|0!iz#G{usR=+-xC;n2~-uiTMh@xB3Ut|o8T z+jjVHfnn5^od-IItGZl9Z6gAkMNj&_4-RT(8p}&cRtTkHz^NESt&m)X(rbCHTMa1& z?Pmm96JSua3%=jol!wKW;{apIpZF~Jr>6qb)NU|2D-LsyuieS2Hb(kSPXekc$W+Js zq3sPUCcry@yanyJbVyk(L34WD3uwG*yJUxyhrvqamujcF4WXSB%s zJqht=T>AsKwdDhK8A>XMBmPssX9mSEFdjf6td@n)HZqFE6s(PI0!^rw;J3rg{gim# zF5OBbfO{QJ)TPr97F}ko!Bz`MA(8i?lXSz-kd;kT)7!_#f(Z<@&VbK0%o7AONlQzM z?0O1vUU10n*UQ>Dj0DXOSe~l02Y|}czE9nZYiD=utq&Iwh?I1fQjdLL9Hgh$0Y%|b zVTBO}@`o9(XKzGcAnpGB{d#@4k0K46U)NkaVoRo|^7|;9_9>)#$JXtbLp}&zpn}o$ z&$`E~I=mKqyY)tMi@zhBqre$fP0X~m$@ zy)=?Z8*uqnS^BuGTepIH%srb8S!(RQH&I$FG#?Zl{Wa?}1`AIPgOfA-Tvi;oltV}c z)wP`57qBP5PV-Vl0SXA}WKf9ND*^)}WIj;Ve0ZU4Mwh*DaOzJt2HZXTPBU+uST(-8 zVX#10;|;0uZ7=ObfZmglX&NXo1_nTk@E`d~R1*W><^DH~_o`~hd?YYO+D_maES;-! z3)rNeJ(ErI7x|DP?@w+Xfy?U7p#R0zlEO)lCoPwT>vw_G<%kZJlbwC9rpI+CB6YR3 zIS8{SLvp8}`q}5O9*|b>o`7|T+y~dU+*U$LN=djP?CH}(A_|Ax|ILrmA zAVLIb0YT6t@SY)_;!8&%82}f0y0gZMmP4ZsgGmwsQ%-TZ*^6LYq; z;7xCoH((ywdNXT|&X>q;=r+LVL2jS@b5Lg~##mcF5+_Fi+}~Uw*H6h8LFiZ;^|q9p zha`H z%0XXl%_+|vCGax_XN(PFu)THU?0Y4vtrTa}K0`@ZaQz`P`@>eC+NF|vRlHiyOl(c9 zgMbBqWjA5I0Z_ftKp8u+DxLO%GHrwm<=P^mSg58(xpp+Mhsn?3azKu-ZDrNy@cV4_ zq1NA4$`Z*q{_b}F9_p@m2(aQpp#@dHzu#S+LH^;fysw#r8B3lZDpe0*_5gnsA<2rE zAgR5?@3|)V3{?@2m~CMeVdvEKlN+1r*`sSOZ}M$Kn6b9j_&5{syRL@uIxg5eq4dcN z3>LoHq%$og^`KotgYRdYB=^R+O9XnPM-*%??f*jnR$mIhUvh{SZFL=XC%6%^fd(x* zv+pWg>bHKzb@HkGomkwh&PhlexV2y$v7Nj0eaX=CIz}kj%E-{<4%z(;@yvtSr3ptK>H|6;FBb(`)bYZaaiI101F90`9 za7ggO!ki4K8njVJgZH1I67n~=l?nV9bUO6J221}gUn#$gj;UT*3yS~m;EkP0J=?c! zT#c(2PqN|ixm*eaU<5Aos2~7?OCFS(NJ)VA1_K`zm`*^03!Q8^J8qDy!qz6w4)uP8 zV4E-XzWnrb1v7SAQj7);N_(|7VoMqCwLWj3`x(#Kj;`II|CfN{AAYqTTh@oxR?X9u z0B>U@V`aj?j%*t8R2&Xs@N?<6FJJ0`Dz^E-XiOVm{De zrvC*hzQT8lxb#?k8K<`V_W8MreJ(diZ7=o5OE}k3q#b>!&&?}@e4RN!+rBsqh^FMX za5kJow2hTY*IMR7We?q_kT!x!UWD|YA3edBFU_@B=G$^qgYUN=QR#CbVphYdGYzWl z52e3;3ldT4my`vl7x#*`5woGyeKhjYtO7#UF6W(-q?fJ9sz{^`;)4FeSAC$8q@|4p zYR4hs9q}{%z?3s$?{3v+UW;KwIe7zelbSjL`7sRdVak(fZ51mB6N{3NkwKW46f?%M zl1MMhxmK|W1RHJ`W;8WW56pQeB&tlI|AlDFL&0$mRt=b236Vs8&Psd?%|o(b3Y@_S zo2BCSGOX`hVr$*_1F%ep6au;fIBtSK9n}gP5Nt&t_&mE6%V}x~&JKYa0oL*H`$E9u zUrPo$*xMW2xG_03#WDz^%!wonL*VF(`yBKaq_%*8A7|*R0M`WhV`35$ID*3fB7O=G zB%~b*7Z6_BQwhHm=hFX~-7bFRzt(0XFaB+)V}R?!TkF=H6_>btLFu0%ko(qx5>YIw z2$T~vNWtF->M*Q{w6vER%`q*LPvB6&%a#=z8(VrEs?EVz^!ui!CMZsT*YO0T3OaRq zgS=2B#w3Y=i3aovK_$I?yMXgtCg!n|c-o~Lmf@We_+O*SthI}lH-2HyVGIlO3=%5cXx+L z74q`W%o4~9OFvmqql1S-RBiv|o9FCCJN#I2FtY-vto9ZQfmx5Uwyq*^$nkR1d%<@F zz(hz0bK_Kb*j`#%P!v4cKyyrY(GV#gzEW|PW0snbol2WX?6H*->=S2+?PgBsReVWq zE5nE6N4V03!g;l(>taXeonuPxf(s-}Mf1i_{3rv6QUP^Vxn@mHea)MZvVFhtEYs@| zE(vPgbFW;A&&~k>QE?wBJzwdc^nR6Bl`cuGwPkW;9hlieZ6GTi$aU57MFC}K!dj96 z_ZDwpI6`#sIMXFRjNk1H?*(`8#Z`%X=iLKu1A}kW22iDabjnVL2$;sc$oB3=^W(s? z$|&Ue)y<Z-vUUY>1Jj`jC|UP~W=CDwVUyE)sAE~zvKvd>1n8mfbqeh#)BA+vaEA%t8-t|>QJb(@sP|poeDAd_==U_yEWAVLBP_qpF_>pI1 z6QWUIX=kVAX9OELSw&Mzi``apnY?GtO$WxF%!S$(7O75L@-x5E2MwH|BqGj`VJCx; zNaijt(15K1BcUZCc>BjL_fD7*2jE%RQk%zj{SuXTsm8r4`yU#TmM`N{Ub1Tj^5iqn z3xp%jmv6AYAG9L8Y&cNC_BiVge)HIz!VcQMirnUgkz@3!amDDlwWy?g01z6rl$5NY zPqIxA%0$(G(i-UZu~s!i@SA8l-1PK5U^rB=(yg%k8;INP&t9RXGYFRzNdhN9w; z0MTu#^yxsrs6r0`&ofQ!z>kKx_;hYw-rJC@!_2yOxhNFNAXs_?1_Z>^ z7z2m~GAs_;!|n3yk2VB39c>s~X}fGEangQrmW1(W=Mm3{)SVO*Ldc#g;2Z%qvfvzn z0V+oQ+_A{imb{9gmq<3P0w?vvdJo&Z6=13}?v1-~SrmAIdl@RiPMF5-Vu_Gr`OD#P$dbXj{`A6P zDlTG~VbMH4>t1{o?pQMSnH;@}*f$G9)id>UtS%yn76Og~Dc~@G+cPl}Af3XQ@^|i` zDDo0xI&}*|9JQt{Hmm9gE#8Q(nLifW->QV@XxT~vn21L#Fz>ea9regfoLiv@6PIL~2??d`bYSVKpN~1R#402Zo zvr)P&ZMD|35x%wuadtW?!wHWHfs=@ojS7B5JWtpBEVs?Vyh2|W=BnFqvMe_EtU}Gj zVm{ulIlujWsm+`g4uLrgGNj-FyNVK>eaprMcH}oOb8~c@cWX3pZ2go41YzKwf#GbW(2CT8bUxp1K_|06@%9G$ z8XGX00|TxMP-Z;L~;eueaz|wuX zv_JpXa5$c7n{6c8>vHM5{eK6CyVi^1gucM46XbsehYRk@Syt#knNK7Q-RoCEV6z@E zP;AD=8@~d@CZsG7q@`qK12yMnW5@2AAgKe{e4;|Iy*3{MJ)Xmc2chx2jsy#l$Qf>1l(|h&^W{D+$gj-M z3sSDOcxAJ6#Qx0FE2G!j*XB-cE0HkYog5NO)I_an$mkv42Z$S?UzV`p4-cyBUB&aO z6av8$Iqv05$11H` zI3v0T24Glp?yc^R)6>&iQfsH*J+p|dF$VuqaE+5&Cc`fg35PQMHPz`D8G+e00OP>{ zRUztgcL19jlWS{n`SQkJ<#u8Jk?0!*af;SY^%*Rq#eXR6k2tk#S&d09zv1jp9vdG| zNK6dLQ-@LQGO-!zs6Y&W>*S zjmPPEKUGdm_AsYBo!y(4n_Eu3VQ>nW&wmuP!usw&yjz*K(a5PwlsHdc+ z7DnV^c6yt?|IEm=ZA10G&1;XnkpD~8-|t^1<0r4%_9DU0&e75HVRK7Mf>S@tX@-LC z;K75yl?4bqJ)UOoUUsc)IpLq=jpQjHZzv{Uz`h^Gxnn`~)poq?FwE&qR!P-(n-Cd! zvAM|j^uq@a;+2TZSujblRnXEpuxr<$ooaqgvmxc8_H6~uxg-47FDxFjTBd^D?c2VA zBvc{xtx3RGGb!xU6K*Sn)Y(t#GtZFu=fvsAE5{^MJwc_J?jsbL2Tz5o&jxlSGAnWM zi?_!B32;f#LlTM_ruUsP$G*GW%yZIz&IW=6`}_bssoEilXUUlebkwD$4@7fV+zh;_ zlkg>aVAMUG~;s=88r;4n6(Mb9 z=MIyq9SWhJl4c@+5mcV(AJSH+HsxBg{7Q0Ig5Sx_W`bMAFaPD0e`@5w@%u}BULmqW z|K}eya{fK9>Z|U7A*syF%z|aM16`-sHbnUOS$7ouwK+VIV|DHFkR=){xXG&A0DB!2 zW8s^$v>Gx8h&T#*<^BR5N2g0I42cHbpuJP#IQx*%DfrKkyosF~VDS0M-i4&G1Rs-s zt>j@=n}a7B|2l8BHv~q2Twl5g&lA<~#NVFXo(UDbhNdRAmdqC?`FcA{ewQ}@pq{8o z%G5|E^8^eE@#WyFUuEHN`yIM7o(1Qwk^#_xPU3#4qoqgW8^ z4u~w2hNLAM_LqA+5a@z-J@631qY@M8NczW^GeBi(^$-5w%@MyymV`AZEX$` zlsnmRao?dW-O&1d3R)lz2B>%MRN8**J2M3~=jimN$9Aly2~Mm#Ou@KFQPDTu35-OK zNe1wzV9#R#q-e_AksA?N`0 z)Pj!{OiBe&19(w`rD=r*B2CTD0Ge_wKgy@WJrHJi)iU6w7i1!L0Mh#b%|w_f)6H=W zVIy9wr>^B%Nf8ixb&X+khiBX%Zj!a~=u%j7`iaDnU!XAyrv zz^BZ31%)pX`3kX2t_x{0u{`u%y7MiKJp}3?1#I}V_Ar>OgxI3mohGV4(sgQ7%Y0=k z6Y)ljMT@xi0ecc#5Jm^X1aeJw8hI2b3HT2BQDv(c@Tzw@2&vfl|{H1{;s@+&|*nJ?1Cwg`mQBnD9<2ta+ zOa$G(pxx-p^mGz;P~2qkPuGIzy8|Z5_H-4@5l)9aaCkG8@v}f5{PhOeU?SKx-%fB5 z7cYa(F0dh7M%|cR-XXy!#E7#*w&S^U?b<7uSjXvRqe5K%h&o||$qBvQHCJfGA!4N+ zBf+p*aY}@GKA*FYX9-)Y4?gdDbCg=}0gKOpfPN~EICIT>|)ZDpePHXzEkZI!hL;7t#M zC30WKO`5e3oXK(C+IN-H2=Q#S`;s}5=`uHH<7(9ve`TybJvPcH)go%m$3V5_SRkb% zDc?U2;aw|y&!B6XfWFGe?@w!;Wv-rk_ilVT7oJwu3&tqoX62+8e!ZIW}Qo zVYnoOVg)UXtMwfJF*1@`Zwi*uSOnPjz!35jl|;(TFvohRCu?wM$baP6b^;9D2msG$ zEY|r+4FOwlqGQJJ<-BL40evtg56Fu^g(q|VySSa4olEc84J;u- zmW-V}%$664r~NV*eXa`4kzfdC2B57`QBnS9VA3yluoP$(BHb;3L=W;`Ud=gje1wFf zv;sT`u)JhD3#A4a8vE(fr){Z20|V&KpLO!*GW0o5oOtBv368FV!^7aDG4VM}(0^)` zRmLrlFCdv=j8ug85gJg?`A~zm*=& zQK*{ALtq&EcWZ$Ve@8G?A1_FoLWpiq4FIeR`^{_+x7$m#PS3!3ZS*}+0i<+JR@NLO z7JWYq7<6nEUB>V;{S9CeaHHp?=PO;Rlp~=F8)dexAls0>#s078lmGOi|3`FUHuojZ zlCt;ZuE+5IB~JOLyApJHW7E?#lo>EYZT4%tV(dLPw{u`lhsT4`?}j1=Ip#Wnvt*`G z^+B+e)EF#w+Y+mUst!Gy0aw_uyD(2WPjpqJevp2*#+YKq4#jvSkhs9Cl;LO??sDkR z`2%lGfK!dH=Xjp;{O$HzAMPzk_AU$e8mVPm)-!J;vdT&5G1M}vx_83E4MdG{AKa@{ ztGyO*C7-&`kEyCt=0kj2qQMGE@U9Ij-%r~LBIFp}&AwUCLe-+&6&Fl5_8Ov0SQHVI zTrpxH)}=Tt!&}%7?gYANTBzanu*teF2G?TRdj;12^iUx-g!+!}7;J4pcL$yfo-|K6 zvgDLR+Tt7h=PzXNFDl)LA2e;wBA8eCKI<*N-gwcEh8bU9oUx znCB#vyub!zya2{pE5&mhh9?-*$=U-5Zd0AeKRya^Ckn)MZzVDjzy1ox-i+@nP8q09 zfqP^hs-K|#D(&}LY9=WdUFcZFOIil$iTsS;Gn*9-$4b2X@8m=O`m2Ic{T;unj1(@8 zV-5&krlBH8yCd3BgpOXJ$g*=o?jytUD$`q8gTD5k@2KQ%e0}z8h2cgSWtn<=JC*UW zhvEzOrsEzE9&KEFv`}Y?UGdno*->K$d|7QGW`g-Jv!#1_vmoQv z(EYt9nT?t$7H1!#x|B+f9ApaHLEkBY7CAHX>@?*CZ`Le3d{fix>}WPZfHxm~&h;>I zz3qvR{Mx%k!PgF2Ap@#9A}oT~WR!)v#%fS6sTZRb6y2C4YIj}6F6v7WX66_Q@k17b zQFoEd%uMj2eH$C=e)BF5>QeKMu$`etD!GZ@d?tz`mX%6K0K3U+lnEO_zi{Qn!$Yh% zCVhc$&Cxc;5$Bmscc>8#$HQ>h+`PR3-pOtw_JE>4V(BcIhVEsnZ_7y!v(d7S+9t?rY4IZT^V^xtwYZ2X?s zjL*0%rOQ$2OS}B%eNOr>a1%y5OAg25;0t6G;**=rdNn$vhUIu2V>k3|<*C${`Fx6% z@6X?x6`v|XbCU5avS((jo5^M78ZdGi*Bc>8YtAnoL|Qp{;CR`0)Ua@^x?1w6c+A3& z9CV{XKDe2EoWDmKT1yT9WQ)f2KjN1V*<@$C8igsk4yLc>8WcaS@OlLduaWSW>{T$Urw^^Glc$mo~B!o@+|!r4!48#;U_Ywb>- z@%Ma|krm;ti^5taD9LiDcLlznpO}bXqGLQ~OK2azPZ`?pldCi=K2bJTsXFY={mjY^ zA7*-)$|bIgb#!6k+W!44HZ}%sZiO(!#m$$QF#Le_J=5p;!aXvkK2p}!Cl>DNtI@?P4@7C9 zj23RtmQB#QrcHMgC#l=e(@Zm4v7bSI)%XC-P{qKf>GGGAh(>T;yB%Z9Ek)eE*u`FZO+MYjck#SdSc#Qz&P2Q z5ql*jQm^^T`d23{+9p+Xopt1d?mtB)Qr`YlRdul{DYqz9cedq&BsQgwB}<6vZ7&7y zUpHPLFBSIm-2ayAnW8PY_*w9^TCrD*!@`W%W#Pq5^&gIwJt@~Ia7uBIZZF7ZWoy(f z)OjH4^zq4ij3FTMFH&|Q{syoSlg4Ln_0j=Ry zG3u9wJsd&fuPzOqcL8X$J&r%Z`8IMFzWB9Y>%^{RswJN6*B_sVxLFP#6i9!wnDdb_b{(GLVCY|**F2DJ zdTeECcx%^tciGBphwCN>8^M%85#9GKm$`FKl>ElH{q~;k{(c+9N*O7V?T2&}RI0at zfxZsF3i{ynij~{DC}P9=1O$7;DXhH2lT6ZbCSGzkUg&Ss5~EKydn0%n)_%+5`r*fx zr?8tZDBxn_Xtyq0#Z`(_`{lO9V8(^&aDLG|O&gz@peZe;Lp+>fx3_ye$)ly*O-+5b z`c{&T{-hLRMIlxv_x4G#<^t!})2GUrahanQDCv0C7;v^=d~0;;iR}9Di+xwa5;C*1 z>4t98zy7F9Ff$WMx1Xq(=#{Rnth^#AdG^-zn^vRyo*WyNOMLSUrTFzMb|0??{Vghi zOYyDGZA?FH7*azI*}KuwEW*yAZmDYGnAjknkr??vkM?q=gPii)ZH%n@o*czCoZD13 za_N@baZVi}O}r47g_gSdt7kj5$?=Q_%*{C;qI>)}Z98Rc&Q%_KaX#@5bm)_z%r-zE z8Y+W*P$!i(eWaerSf`SF^jN8z{FUp`L((@*i?w&qetzG4)+*+@DJA8W8&bvT#lv6L znTR-8yD=(fO8ez%=f2C?vC-vq?l&H6N)~bJPn|o~Pu~0`?nh&{r^JQ%@GHHLMM1>8 zSmS9}1NH#N`SR;jFWXkvE7Nv~?m$-M1l0{m?pEWzQOx zC=q69`qxn#2lTmaoXGNc@hofIndm;rK-S^5bIyYe5=W-kH@kCH*V9S1l{+UZ=jqb+OEWE;YG4hH_of@`XnE^dHFqN2`?jEa&(Qod9&|H3 zEiKPewD;gf2vNQ)S?bDxToG(`PA!+HC>08ufbU%MSXoq)D-XSEc8f(O(Dq5~bm7S(#65;A{ zf`bD)j=efnGBZOr#Aj&q*)J#(^PT&6j5C~}LpthCk-|Ndw)!Y)Q?a()jEt|>+fIJp zd?YJ#*PXT~Ot?CO|MXayBYu&hYL>Xeo{3b{mih>{Y{-ba!2CEpWlyno%k9O5gZHh@ zQ>yc3S{n1uY9gd8e>DA2)d}X3Qyx^kp3^yiY;5%tJVdomiT3$ty*=k+Sa6*s7SYkx zZMn9T0pD!fIh2`BcI8eU5Sr=B&JKd3k(c-21^fHaryM4~+!1a`RK03%pLRP!tmuQ= z=`Sd&&~ar;o2h`Fo{iXN?{-FGXq57825+b|+-c5WavjNMgtJ2{Zw4>X&O&a}VrMX?XP4 zJ>Ak`y#{w3MFO0jwU3ML1VvA2 zspmtAeg;yUP~qhdKt^#FSjVqFKyg}P{yF#5`Xhy3n~5MnX<`xp3!jVF#U2$K!BC^ojwAgVt~!LBkb|H_N&6-BP0Im6@I!1iUUC@ z!<2FbD)$^_FeZMj=?Q!m;cDqOWW>Z|oAM+BXSVHep>VIINzZq*g+2cjSH&k5>*TIk zxM-_JuaSkQkCn9@3ZY0y;5|2cgELf+b)W#!Cs9wAo#sE+eNJF0XL6f+M@5lytz@^k zs=%3^y8-r85mvA?ln3|J>KIuTP7126TK=J2)Xnc-Tn5}f-QY124hW*#!De7UdA``l z14%`1E%H1P!Ko)rT>0C_)L8KjlIG?ThK6D=lvkd0-@g2#mXfpUCRwq!9pfS=PU@!% z+6>$|b+hTsqNPb;s{d`5xb08$d#Ynj*oumt6gcUh?kspRX9sndFHfF++4uYRWfWtC z5(;+8Y01Aady{Ks$&OWthLI$e{Sv-n#g|BgDez$*h2Q z$QtD%$;rZp4_JPXr!vrnK2*3;3|%E>aK!sGOXD#G1o^L5hF1YG$ISpw_jq;nz~dt< zd>NaGAIQSTK>S4KpyLL9X$9Bf(y7H25cI*zZwa5bIhHEp8Inh-`Oy(+w9RAUBi=IT zDonW;TAIo52=@4C3gxaV2P9CzXRe?m;IBDIPy*FolF1PD#4qF(S_~dCxuJldk{c$_ zq&)L@< zjl^^|u*)HMX|IWY`mtln$F0sm97pA8KFmGzBL0vlE1J{v?d9rjEamIZFS6ji-Q-I- zkgTjB+Bx@i;T|7ta#PMI?)#?-^I8>qk)4WrQ2PZh8tU?Mi+wLK`}=YF%dO<~h^k%n zHn@$U_CR9>R2)!wUUzrBm>@1gZCzci{sWzchpF^PUj!CfYMX@LCeiPzq~9O*?q$u4 z%x8KjiRbG!#bAu8!#7EDv9XolHS;)wju?8(UT=VI=JTD@D#yn+-@ad2VEMARzr^q9 zWp%mxE~RqEyg$^K?4b7==}9W~JGxCy53LPtRz*cO!Qw{A{N;B5Q=xz19VqX4I~LUe zg_O!(n_Ikc??h=*$cq<884o8(^-z-5{=8eQJIf!JL1OpIB0mOIx2W!S(B>9M-O>9f z_ TxnCoordSender : Send(ba) +TxnCoordSender -> TxnCoordSender : (some txn logic) +TxnCoordSender -> interceptor1 : Send(ba) +interceptor1 -> interceptor2 : Send(ba) +interceptor2 -> txnSeqNumAllocator : Send(ba) +txnSeqNumAllocator -> txnSeqNumAllocator : assign seqnums in batch +txnSeqNumAllocator -> interceptor3 : Send(ba) +interceptor3 -> txnSpanRefresher : Send(ba) +txnSpanRefresher -> interceptor4 : Send(ba) +interceptor4 -> DistSender : Send(ba) +DistSender -> cluster : (distribute requests) +... +cluster --> DistSender : (merge responses) +DistSender --> interceptor4 : BatchResponse +interceptor4 --> txnSpanRefresher : BatchResponse +txnSpanRefresher --> txnSpanRefresher : (collect spans) +txnSpanRefresher --> interceptor3 : BatchResponse +interceptor3 --> txnSeqNumAllocator : BatchResponse +txnSeqNumAllocator --> interceptor2 : BatchResponse +interceptor2 --> interceptor1 : BatchResponse +interceptor1 --> TxnCoordSender : BatchResponse +TxnCoordSender -> TxnCoordSender : (some txn logic) +TxnCoordSender --> Txn : BatchResponse +@enduml diff --git a/docs/tech-notes/txn_coord_sender/leafbase.png b/docs/tech-notes/txn_coord_sender/leafbase.png new file mode 100644 index 0000000000000000000000000000000000000000..8a4d3b0a25db86ed327c5217ac534c8da6eeeeb1 GIT binary patch literal 60840 zcmdSBcT`hp)IO@ijEn_PQ4kTZ0U}L6ihzJ3Akw7QfJ&1l3WQ#CR2W5y^bXQ%P&xsE zic0S#^oaC?-b4NE0JizQ-(C05duOfDWs-BwTXuQ&v!D08xUDEdap2^EJ$v?0$lko6 zvS-hq(eS_g-+#a-rEifD@E4n%l$M>L^+P91V-veQGR6;#ZSL9`8=b!Abo#!X-9uq+ z?uV9lAJ`$SEV&G=t&Uu}a%#^Wvi)XiT6W(*-?JBP;}{jJuAtN-akwlyXzS9qpw!M! zw{Oq9iGN1*g5}xKxJq+FrGVp0s1hds^|gjC_D9Lj=CgM830n7;hCkwZHQbFBwcwe* zX!L=ksjX{x_8bj@ZlPNb<{ht)XztK5|QT~pE$Fk z5qpLuQ^Ksm#WlsY1U;Rnb;6TV`rBdND4pA_Xs)n}8+a>i+m>j5jrRD{GPQEXVw-_c z_Gc<|7rZbk!VE_-3Tze<5_g!WD+KPKPl-Jrovt?gF7jZthuJ%Yg@F zx&&W!2JP5FwSbZCis&!vNA3G(%spJp_uYLp|AGx+uEBllnQx|U>6Y6Z`RnZ@ToCmyz^jI(}{h89GmT2qzA8HG2AAF?<~ zC=dguHPgZl$X8FsRXOIUesR`&Rr5OY5?8KCPnpN~irUS)BSnRy?6!#4D!LSwOLvf2 z@4jA+TL=n~n`6RwtW7<1Dgz|rgDchVfxuPH| zxzZ7l6}5G^Q`3oSqB)@^XTOrZ-|De52%Z3}4*RMv!e2XMn$M-`eo~AV+3DBPO*do) z9tq-~H_N)*OmSJjCOs`{zVK?y-KwsB2RFeQO}*&cOCr~y^o-1%7APqMg067&!hP_JC7?~ z$^YjWbIq3M=b3do>V?k7wc&$&7xyv~*lXfL7X(Z^_9n0j8Qvw&xV2}`<2|xBuBkb8 zPxKwsQ$ud9cpN!&IJo4r;E|aaX7fjKhgB;q%$KrCjkQhE#$W1~_j8x4lp3+b|D$v3 zkAOtWC<1)C0TxbHa*$NJsKr`(_32&tufIICY}Rg?+HX9Dn`?Wx+q%{ zf4dA=}!sl8#4y8c_C_QRf{kojY~W{;Z?JL z?m>9%8%NXc(Lpy(EKuRA&|y%Ymk}W7xhCV@^)aDF%kg z$;stuoPn0s1q0*$>gqI_3J$cdmo)YbkKYwL2CqNRlyL+JERV)o`==Mw##(RhR_sGo zuPzddtKoSGR~~fcnW<*$mt|&Utu2fP1_dQrJHAO@!UP8g=QPJ$bt$mM^<%LPi}?1O z8#u+y@~!2q_4W0k5Vy4@pO-INPS$W3VDz{Ota>`Sx`@Ql*mCE^bj(I`p)xl&cX9pf zV5PBpPk|M0x!an3_|`;jqY=gei`rKZ91_ApYh`H}H2HFF&W5faB0M}=%uQ!BH*tP{ zT576RzIm{yyf)Wbfz9wO?f8CWdtDsg?F7?+M<*&@(%9DV%nTZo#O6DT{>!!;z?{?D z+pCA$8z{LKK@MAk(6lzXbiU%B`O(GSp)<$xIDeQ2re(} z#Llp5Wcqvanbi4b1Z}*Yjk0O~;zJ*Nd_?RtUj$YC%I>V}>(Z7g%ag?|7LkWlr)_C|}Qwetcn)`Tcc+Q=Z z95G(GcW104L58YWScm4z=g-yHXb}ft_=k>8O@8|W_Dh#;bFnZpA3uJ)ni2-q6P9vwJ|2v? zuJ&)Gk=mLXBmEA;I=(yWeAPm)45bc=jGWjSv>s~Iabx(DzR^j)<0pI2s2GK?OCQ1- z(QPhRCuisMEreB=4TkILRm3P!n2L%j99QCyeduXoTDtosXm8#VJMCbl3 zFE7^%%BB)uf6GA!=HpCH)+7Az>*G2}m0F9}gBf zuSc2p?^d1~^jP;<*Urd(s)a!N?wYY-es@(}U7p9Ct<4S1Y<+ci!RUkpLwkFBg-R_O znPPlzvF(>vh@eLQS9E+V+}zVGNpd_qJbqV~!W!By#%9xtxvo4vNNZGe^yJC(Vn$6( z&2VwYnLF36J(<%!wwRv2W_Tx|#@gVpMMq}w?6gvPDib=xUN>GTR*dTELQ{gDSa-4Q zD1?{@%8|3@&IMDor?Q8ZsiOjpynRB|equ|WmvT_6`J?ZZeeUNimUA@>$lsyAhnqcL z%=4K{NDJB3)djAJQH&DeVP~%moAshGJ3~%Re$Gqd9@rhM+D$J?hN!40PYQbbp{gTp zE=7+9r40-WhBtg_s-!oL+s3-iz2LCKqM{=sBTpKxE>*slh7o3!TCs_X8*u4kT(vvW zh{a+RJx4|CCvJySz0IU~T0A#52QzneanaS*Ca$d9G&SEmT8boeq&2=DsgJbS*cp4k z|84A9AFBgdjZdAcU{LMhq4VkL?yhl^W0Qo)A{4WS6if%iJrB{vuipyD%gcNGcn=xV z9gB-#YOwk_hTLTfM{(Sc87isw*fER=-}mo$gH_G&v$zD~^O)e@r{Q`6qYC z<8cQ>AQQX1>$ltcd|4$V{e{gj+!VN&+7v=lCqlD;6Car?bJFQz&&khs{=Tn|z@Gnk z>m~$tLfi>`UqWk5Y5djkk0zuJH^qyL!c*ezqF1kOfENVV7h2meN$SF_x=ZUzl zJ1QzR)JF=A3+D}?I4iQGNdOsj7VK)vH7Qxi(|9^PUmlNxcadTtsCpm@)AD^pE_g15t{ivUwiQ~6JlrhO3%J$!R267zj zY<7Lyq*{}G+^33u-q8GUWs;I1Z+xf!5PGC zfyG?I?#;?2+%wc%TPZ5EFl{a?!46c=w?CC2jV$waa!`=f#@cdTp6g9}>5lpNMuW7B z?p)L7US8he;o%7hSrNXx)um6LP72}@>75+54IoSrTm8u8>-&BU4XVY7aksIieOaM? z>DlM-0dsSWU;L;X>)M5nyr^ny`o8jrex_u_MwYCh{NvD;@7FfCqjdh zHom0P)pg1+jw`|_-x3sK)|+~ICMUYea2{L6(q9<4xiuG0-=3B^O2%H-;2XGBgI!y; z9s0}V)4y?xgwfQD15_Rp%jH{oBbBe{GBHcF3$vAkN}q##n^W6ne(5zT@C2dd^Z3f_ zGVBT$hrH`ZIIGVzdM}x?loaRsREe#_v`MX8qpH$hRR>x^rSxZb~ z88B{Z-{wcD*5_lL@aizzjm2!!{0H1}nOU6U!g0s#5%7cEH#fsLgFdF`5C)tSEc zE^HKbh)Q1po2cw1w=kruSsdZcQ3Px2WREG;DJ{s~wq*Mj@E!G61((lNB)p?!=x#+y zuulk+lk3(mOS>=kU8&#R&xNep&W*jLS@Iw2>g1=m=JDz7>$CBzk(xsxHMPaA!A`>$ z&rLVHIRqG(dR*ATq89c{+P<&Hy8OJ?23FriTL1QgKH;ov0)C}M0{cM?t!OLk>S&0= zU_5bGnQOPF99`zdN`}D3$sDaee(QocMu2{03)_{;YSnp3)1)Iic7B!6*s-v?{?Adi z58q}fV>bu}P7!hk4jk=wvEEFStr$|btPQyt?T|JoM}{C*oe-u=trmAScCU}XA~y@3 z9%t82B0SdUMDFYN>S`iA$@AF)pvQ2YY|LmQ(^yxluE|h_GrDo5WIHmvKZbr#<$tY) zeteN3kAcBzYc7-zKUZ4n7+pC}FR0krZGU@&lcSCwH(*dnil!{h$h~{1JQkW)Yig*s zRw|zpu&VlwXTE)=txPYE?!}d3Yq`Y{`kP%!;)sB3K2zW6j`|IcEkSP-na*A8#!?~E z*jgFe&VJ|R^-)q5tSx63yRA)tlho=z_YzOGR=?_O*O$Yrf=QP%x7%@>%TCy8cHa`0 z(Y7`QFgt^?2qCkUOIc3~N~PsvMCUrPT9>Dym6ZI(TPtPCZbjUlTA3`Qo2ywI@RB2^ zz$b7l&pr}&nyc{GV9(aq-?dcY@a7gVc3`gco|RP>xOPhtckOBUvZaRT2`TU7+Oc}D zhV|U3zLK8Ut*B%o(+Ob@We$i;QufjEF3YH(JPT~Y$Hqq0`RP-nm@;!9iF5ck_RE*< zeoX7kz!TGkPh&2Nh{6ZH6Irh9diU4+u52g0Au!J71mzm0Gya-~ofeyg@5{YXh!p7V z>gw+(ki2uJoa7Yx%pU|SwIos$1r!zSsaYw@9t}Q2e|*r_8?2dxF2CG_y!3hb?#Zyt zmC8UA%6HX$UT7y|_N8<)$VTiYOlte+7o^|bH$kwfn&T-U&h`{G@}Br(4E;qkU?*8a z2uEOV8uIUx+=IZXug%3!XCV^DJ z!9(C@yX61?gHttt2kaOSULBoCW8H)yoWVJz9OZqI67zU8%N32Vl}{wsdh9ucHd)3e zU`V5$Kv{fz@!@dOE41&6ZKwCaA?ufuo}eTLkqD=2%GcKS8?PFDPVTH~ck#&Oeb9W8 z=+$|%b<^*}p}omCq=HB0rP-YeDZY-1EMu;QX3HB`_C*#s*H+32x-Tqwj?96pzL`ZU zs@mAt=;`U%+2tKlSh{?+E6P$$fK$XjIQX*#>z#iPrUw96RBjJ&UvOZc?fS~>%E}5^ z{mY5B69p6N+j;b|3N|S@xu(1nYE8C^eCczMfj%|K#^g4(woa3O%9j!%@LG4j?ckm` zlwNT_OpH5Z@hdHimMlh^#$LvS#-rAKXesQQ*Jv(49?Z;An;W0uzn{a|1`~I_A+Tyc zhoi2q?>^qbY?e4!>BlNSos*GMk<*hizk?}mB0?i0*(+YRwY32zSc%XYopE$!s2VX! z$T91B^6}$!$n(_GXdNbYJc5BrY|(prnJMrcS-L7Rnkc>M>T20+@i~=%{yX1?INe_+ zM;fByzOznFPP}KUYiepzUo7`fRHc&bF57J=a$woMAv6hwQe_!~A5z@6EhFxlENvgg zP4|y+z_r7RGWxe_=wD5w@2sJp_~uR0eNkMM&iXTtJ6e$}i+hM%WOfXczya2Ixf zW#tF*?>6=2F3Bo)_=3D`YumlRZGt#&x9&^Pdky!zZcl!a~ziLAs zy}q^V5vtp=5c=xeuR{U^TEb;MAr}dRCR8VHB)y=rcpVrh`OAQsko9QSnA0P938V;>X55GznHejo< z5L4VYRuSdyePI*|FyhJXd|unlbtigydNa=O(ATqhX03Kh0hvp)l}xyIhF*2Gn&FxP ziPqLhy)V7Jkulkke5QbZ#nSbH_E7HMe`QPJ&YiENPV@-`X_|+%^`mK3%uvp_tA}N=nkerU^StakwSLbf_xP*90cV2(}jXhArRixGx^;-O8p@NGF9-{x@wUu-HqA3O=fn+6*WLRaXT;>YdNlfxH47=?(Ke4BwdoyY zDC@;y6A&T?i6^P&74v}sV@q+G-P}0kv8C^_lzBV!(-#!u<@omb5*0D)oG23A(?p+D6Ej?lLNHH-nycs)LaJ>FyKE8|s!Y5h-*MZSdO`y;w zJ^%RQ5BrJEH>CiLruxf7E?sJRACs7zj7Fnx1r!$*rPe1~Tg&*IOmeqWBA5W1(a_LP zBO-3$j8Pm7ydAnxI{S4w8Tc1Zlqwn}>9vIaJT%0~$jAjqrME1S)4*8&Wkz?#D^Y{x zHJk_6a!uW%kkEKg1L6lTu};ua>Y)YRIV-bgMcVXI!B!)6QW_M zKQ}(xPc2~C8K6wh_)}YBx3Tgp53~=z*xC-!PE1(Tx6d^~?Hr zy`5<#Cne>Nl*^$O3uI|&WyUWo>bR^aLZnhGyD*=V*qvd%Ktv--LZIMeK3|&RluLrR z-6Aa{hK7{rDx1=IICS#vSNWe|!35@bdDC(~@Kv%NCuYxyttZAOX6iR!H=vC6rWK#U zHd;dwL17pD^r=<8W_v0@BRj_G?p@4-zEfm^2?U{)9k$1gL5E`;o#0#1SeBQ*VyD4S zq8NUWT}eo|yT2?l3)3~%`7R5eAM3tqFkR8+5Ny{iz7=jNzYYEEfT9o=@2{^X&Nn)= zAWymO1RFGE7Vo6kzvB03PH&x55?%eMDMG1!)lD-26=()zL349$yHdTAQ?ZHB1aWhX zdpM)cVzd94n|G*+)=#S4D1BIWZNrf6+T}j<&n!s6dmH0Oj0XGAKM#?8AblwneL`&WND0wE9 zyxi4kD+Qmwn;1!{=1|rhymIKw?T7zpKX(2{k+ARg@48L#!j#Rz*JQBGKds?Fy(H|t zjfRZo)&G3KdEmF+_ZNin__ z<#Mun@#4j=_mnHJ9c=(WQM&QI#xu`-AurfR%1LOhXs9__=+QslA%?K0#{Fta2H`xb z9s$|7B)L#byT8T~-KK`&>&ml!HyJ}k(U2gwdCMKwIKbKUf|~WbyZuyevD5qrKZRLz z#h}GWZ&4bj8>E{L4-NhH>u{q+aM(nTqtw(?{D}a1h1#@+-c_9osATp-0Z;hLKqPXv z#$l^>i>x=tC3ZG}w5lY>O!yuMjWQ=Gv9%Q`pk2u46|1tRN_9Q#(J(%GrVb8;zPT&t1 zeJ67tEFOUBZJ_8$->A;Wd z_&;~}wCIo7yTq0L^Q&AJ(Me${!R9m^sh%AOJzPn>4Js(D4 zYJ<>B$lI-_woL<}>4r5nRa%_SwFbI&HdrNro=<&JI2=V6YJVtempLZ6LHab>|FB$YDs9-2J73CqjY!__!M>;8|5efy9y)q6H&EfwU%D8Zr zsunyaLeSbjBg4nr+ovvdbatpaEbMP8KVK6QtIK5Lz;pHW3S0KACXQSW?<)6@pYB%# zq;l}@hwK*|wY9k{BKd1^A3mh(3YJibolv%N-k7v5e^*d&dpg6m(vA!3pszR|KT(Zw zOK9_US@w0Ec#Ha|koOF{Q-u|ebv-s+1F zVbe>zuDMk1R?T_S(WUBY>imv#d`jYdWjQuW(IN*mvz*TgbxLYuEVUorc;=URBYB{r zgl-CLjS1&2^q6?Z<>^rOo|4`@5gLHdXKP$pNio*P)Rc1Yyq%}$LzdW2w)|&x%sYUA z?n0CatA=x5aO|qAR4h|49wtOMbiFeyiq!S$?HyOgp5rs{xT+am?&9#qxj{igUvK>- zFSl-UsD3)|x;G8z`gzDi^d{FXZ7rk}_xBgbh5T8g87^2l@6&{AZf+*0Vw9Ja1xDc_ z2Zzy)`R^K2Bq}#2r^ta@B|Ma_IJJLdGErvFJ0$dLotJ8Sg@Rjt;rAF3eBSYhN_p18)L2p*hBkkc6K&JF}kL<_IJZ= z93|vU*}!sRFiX=IkxwfqzB$)uVUNKt1QbfZW39*9{1HZxcq(oHRom!MU6GKGm=LK; zUR{+uiGE5>HRXJoi;IhynORUU^j9xISdnV&(~E=V0|SUq&VvHCMLbq2q8P89>d2g;G8-PpIe+U(FExg{ zu!yx~XE$IH&LgZ7{gmxUOv~)TAE)a7hKvcs1_7O=8@n(w)>&-37S6|1=HhThYMYBx z;(p)!Y7Z$m@MD+n@&cMbRJI-@Wpf#9{yM6*AqGEciW4d?enw({b{bUByZeGL%^64} zI#h47I^Uya`=%>`^JBKY&O24s+O=g`Hj9k({d}4Ih0sy4V^HGWF+MDB2?~EpeyHs% zg8VS!ZVHoMa0}zj_1FsQFN^Ljw5C1$Rpa(dPDe5Cu*vrTLg4l|pl1@KYW^jtQAwOr zJJ;kP7FDpcq`{5poNkCV@L)2Lt`ai!g+IOZa{1*0#NFnz{XDI~6&dh(6h=wuMhWtg}h14w+V{-^n?<*_0Be~GmZ1E^Cj2!L0I?h+)VsBMeDqKzRY65N6$_RAC^|BMk`4>LZuC-fZ$9wK( zvEKO4hWoT3}n-=P^P|6{dxF!Roqg1>J&H;Js; zCf{~GX8Ez<64>yuz*?h--(ydGVoC}t3k##D*t7E>e*bNH z|IxS?S*rCkw|;So#(8{vLIU*k0|Emf3H=7U=xl&pgoZe<9J1XEH_CLxox4#wuTz0( ziglGyQB}3IwWa$Nr_rhG(Dkw$kn z;3p>NXW2T6Vl-eVz$!4~!B#PId60ZmAX;_S`fZ5ut&BgHuzRHSF0@z$OYqfuS{`AQv zWOqu{(6^!LK%C3;)y3{zO1h>#K0Y3VjG%kl*w|2t5!FVjH;S!I5VFfr*5p7<(QxUF z4Gi41un+;h1b&;Yi=`9g?@M|#_@&y@Zc5V3Y#eyMsOV_sbt7r6llokdokfolGBPrN zHqzr@XODylMMp=6hGM((E%-~Oo>GZ5acFhL0%@Htcxmok>=j#Xegl3$M!%+Yc!COOBW*Dg6s_4@n!&r|)mZ(l>C zVB?HU*C*H|=+^~#TVsgHz4BZa1zRVC*#xbvrxO#;7oQ-}hdDROp1k%e10k9h6%z(@ zgV(No0BSAPW$KD?ad5Euqet&XIET;FUYEY~GyFb5mAuL2;?gf28y~;ovSdlTIFGmD z(kqIqlQSH9bpU+1-eBYnlD7tn73Abp=zCUDQt}$1*@_70yTHcQOuRWf1$a_r1okH_ zX0D}BPXNlTq<%z*MZbJ5VTb^-u^^=n&pkbBN=Fh>n+-LO(a->OdvQJ_Bt!?nN(1g^Wp-uOegBfQ2tyNFK7YBroef4{y z@c#4s#GS1rw5?lR&=CaTO-oBlL$ruB4uim4oF2*?H&l|9-GAT!fDQErEKkx+Orgpf z;e_^(@tJccNLEhjX_30O8@GrvVHf%N+uPgug@wC6f0nA>Z-{>I?bC}~)5Z?X`#@5I z;j}+MjILVb$qjz-(apQB>=vEfT3RwfL~~8Bp>K6)t(QD6tFtUXL9I2kFWXg;aYt#*fG%dUn(nW*Za#{K!KN0nh*Wj zw0USswIFH$0l3J9SDkFL(9G02a`b5YdU_p+ZZ0n}uq6m%4O$Vml$0Fo?Ll@&B+ie5 zOZuI<62B={cg1!jH8$bkfdc{Fo}m_2_kjG#Nsl5QT2e1j#3g{W*FJzQXGvb(0`S>K zjvT@FF)w*iGNc!e%*@Qdn)Ceg<_D~xwCl^>rGb;qf@n9+RY3j>^0HaW`EwMpW(9qCKO-(6XU2JPkkO|HJhtl@!2-t4NH)G(|Q>HjT4)F}ttH6>*fjU>F97Cy)1VriM_ z=jSJi1fiad3$nN^uB@zVl@XhlcX^ZR%5Pz{ZkGOz? zGJ+j?3W9%qupJ_2aEAM%>2U^eNfPGS!IopfamC4fN9HAn%dr!4YD@8)!b-TEM7|AX z(3F=J^F+)AFTX*&T|ahS>(6M{d&wQ+BER4K^ke{o7qiy$Dyom%f!dEpe*GRozwe6| zve8F5V*Sw~c%jVGV!=OC3ouHTkeZb4$)!}P$%Zk8zD}<;+uYhh}F;q-wT1eDKZTurCX}4#NUXy+g5FVF>C#478A#p7EAdH^}5&jIE!mc8(S%U z|Bpg~5$~hKl{+T*<^>;g@yWFU+gB>o$Gcl z9{ftS%^dDz(Ou3MkwLGMZ`SHbi~Lzn0hzc>5bXk{y_~58QqEh95_4NkONq&WT=ec8 zozTNOko%t6N7t4N($dhl z&sDQRV*^Nmjn&1TTvI>ic96sv@88Lb-;HJC-rP|2blGx@^K733E}GZ)rA=gN>IHFe zaSICz509-tR;7!ACj=k#etdoqi7Lp;v-|Rj4%$;k-2sYIaEB^b_7*imxB=}IS4eO$ zbU)kS+uB+S=z&HcmOqd)9Xcg+yU%~35h7YUy_%X@6_=BvV`q1_V#H;T;NK(+CEDZy zyL;otjjB~p*9zVH^lxdY=st>{XmT&-8^ZrRG0E#dVv102F@cs(YcWYq?D_0jl*j2y zmmc<)I(dfHvOxR}2@FiobDggj95%60E)F-r$`(7hxUe!aSASkvaSRV-##2BvpgDDl z!v*=X+dE1`1G~)t5@*P+fBp4WYl@P`RPpEu5V?@D&-PB|Jws8`a_a+_jDlkfAU9hS zpykrLJpwxStjx?HCtKV3fPkaX8Nns7!pK|UAZAL0R`$Sf&I_pc7pHRc=1n3HtOkOX zou`jn;DZh0%_d%eepPErOIzi)sVUU_U2~vS?2t&~CQ|*f3%HxEqZR8S1(j&H5NI`= zRh##|2CTM@PI@)#cJos^WyD&t86B>Hg(DEak~={Zo>O;XO>rhOQ>=GDK(}9GRPyGp zX{9EPltXq=U)*y$gt>WoMur7QYrMVXAmI$4pf%hZFXfFVxJHlk@i$`KW=Eypvzlmt}~ZF(a!9@$t~sudS`+Ho(+btCNE7uCv@fgU|%j21$vu`XF|P zT<4aJC^3m6w6wO801dZ+`{p|FZ5XSSYfO!EzrlOKJMC6K=(nzzz4yuJ2*?2T*j%T*%o)O zBo?tK#aKc@A0ICU;ox8iDn9;Rd%8L_+oAsrzUKYng>neDZh^eLN}2Xfcz^!}&DF|tn?q?UxLXD_(T!p={*B@ejtT^2FDQXRgDm^=0 zp`<7_dn-VpGX-(>rpPbF$K^uQn&9FB;vc6wXU$0X{YQ+uT!Ly9b_<=j9F<3+jDD`g zU94FYrdE6Q?*CXtZTszlf7fA1ZIR#WpPg zu`w|yJyEySyr)kq;MCBGeLuV%;ib`sWDvHyz{zQq-f{85g)oRUg@uJs2W4aspu0{@ zO-)GZ?Oucqd=QBs zAoBC^wYrw_=9<)>R8msvY66uFh}zOen%XlojU{VJ{~X$>ouZ?njHIs~Pqr<_%-iVD zE@r8kV%vx5&r{7$bmc}^1fE}-o4a@7#!sWtQNwcT7H|j&tpJdOP{b(iz7E>1;Nddo zMPb*K8Q7e}gk+`I&0gC^bt|fHP&0+v_;>#M>Qx>jZ2`-$%v;4L4j;bbNUcMcVC@|N zVNBYOF*+AU{rjt(BuwA{n8BiJU(^*_Mz;m!;cxKJVUtm& zzYiUXf+AvI_)AoFMzb4Xjv_#qlZ&fEFVfYb&B*xtuLHh`C9bZ5;(%=)a*>^bL$>oE zs086OPBckLm{BJuUiJAPEmtaSyL(gNelly3w%+&-n-}pAA9ZzfoSmI9tN0B3cu6`U z6soMn*_ygE)s!G2sUo=Ql;o700c~8U9BOR3UOKM$@$7cr0vJ8%&1xyG8Lc)Y_g=yU(>+6K2)q+%2DXHgbU7)^bH~40NB!Eo|m!J|8 zZ$Qg{jj5i#3IVQk*&`{jS7LN>ky9|4U~Y#)yFNXCdU)VIiW4{n8s0gw8Ns~S;SS_-Aej}(;ICB;7}>=t=jjHoe5Ls4Wi{L;tAry+m5hpv7e$(t~*r)_Uv zkefRX=3@b!0t-RBI|jvti~C;m=2sT z1*51I&a_9`_f)a_hI2SHu6PhhCnOgSau7cMHASK z;`GGCM5uO3ii$48X2Xni+}$w^`E+B#i_3Qv!+E7})A6Dpe1n8-E$SFM^h5Zfgx5pm z(+g{&rZK`>od^WxTEIU`OG~B=QOaCknX@Dge<$_%q9kj_(Ha?yTbug1Z|^t9pON&u zFwdLoID2oCf)tg%6Idi}>Q^pm8!?k)pgST8;1E-2zwN)qil! z5ir^|q(NXyWl*KP|NO#cZF7-7c%sVpkRDcM$-e_*p0}iD&kj<3$MXYlsDxA@kkaV( z!@gJhzE4O* zMN_Kb_A_=H`{>mx9T}`+N0|h}p`5jGXB zx`je3I!*G#e-e1^|3?$Jg@9y+fK_+f@YL(qufdNv7n9H(N)6xwP>B;(dOL1vX>|f8 z0@k~Wu*k;H5o{MP<`fn}+DReg9oi8d@;EGlGk}B=xJ43>-o<(MOHu6qYr>{q=+%kb z(pW}HZzt%?zsl9HtJUM6m;=$~xoJsw?Amm>5~QW?nBNgWd-CM!`g~(bN{ZYbBNG!4 z$mV-&Z9C^SHfmH_A4IrQT$dNOVQ=RVyfldKVJ>5NJhycZx%fi?h7f4pi&z}9c|B|(} zH3V0vP63)@$0AX--Y=o}CAyPV>gQwubo($~73Sf2hc}JisqS2XGO7*bhO@HnBX!}P zk|=ZR^JmY7L{Iab!G^VQy zaw$qCF%H}{S+JE9@#S7y^;q+50Ks`HT}oQIru9OjC%}gbBB}}2XbdOlS0Bj`eSax9 zVH=Or(A*kWfOY{8g{`ggAmvq8!u+tp36!y_XX=jZIIrDSg{#_QaZc5hv@Qr3`-jM9AUf#JhLzltOYjW9W#mQqV1ljmKd>+Z)0^7(VEDkl8(=bsZ`q|Gu+!A?YI)K9&CFEN}B z9+<|2=5sGu6*!-7QWt(%_6D0gNZ;Y;;qs}I#mBR*VhaVj^Z@Nc55vicI9wZwbYEz$ z0--`A%Gz6yalJo?$MDPVo4q)V^QaBs1Pl(|7?{mnI&)o1E6k?-<#BdUN++tE@9||2 zN#5k#3jibk?k8xF(Q;_rR#Q9g5ZZHJTI&A&)VbHWAD)tDFx}znMQ$r_muR*XxWl02 zee~#&HoOI)E?cj-rM-PItFT`M74ZIquwCj%`l~J!hJuYrAh0riI?#IVgi@4{G8i=i z3$KsZo)W(2a;7^}!yf7kD^2OwT{_vt<>5{ec)1<+_{1S$|MgQvqg(!< ze?zOok5#4NwfSodP1A+EwXKyEJ2feEKF#fZd#l5)$Z)&ujWBINEL(JIUp-TDM)i8h zop7wE+xJ5`b_h2{5WI$kowVjXD1;X(CDFFqd@?BFh{BKOk;#CBFAdBmQrQ}Cx++XL-Hu&fh$*5f@{~rhkqeNv7%_jc&>prU!N81 zjbI)7RS?PVhgut!M^v6sS8DpD_lUCChtPE1%!q1l7XHr{jHAEEge89Byj|@Zh!jZ~ znADT`!Hev`%I}ljWqY?j90LXFwjd?%;qdYAS3d6OLE7-Kuq4x50$E=YFv#1w$h>~k z+Ne10-o1O~9f9koS#|!z%Zcou!X1VIt}tmr2CwONgxY_<7wKAgdHMP{5cLsb==P61 zVeOc%f@-z4mYtLH$Gu^!hHzg+L`pJZ?qwc)y?=ix8& zaoL2tmIv8g?;$I{K5}el!AMtf+M{hv-vkCC&Bi&2@ zx{&_i+LBevHaOj$Hkw9c5+`mlp(XByO-C$a%el02&NnqS_IG!GtVkH-Yh0+Si;C~h zjQcwL@87YM42(JN%mnYuA#6Q1^vS8d&O;eK$q2kqgZY&<;il)O@GvnbL=UAv0|gjbP*R2!>=VqO4NNShok zHm6{eMH8EVecOEF@|&?X6yGJcmXX;yCEwTfBu;3Dtz-~06`m9!nfWOVF4 zJeJIeylt3==LjfA>g(LKpDqlQIz81xSn26mLZk1Z;98|P4R8t*!|k%D0MJ2hrKvV9 zPfwp+!)1gH$}st;cVwzOICot5(LE)nHaHVCSh>71AB)RGQ>jm@BkOfq{})lx^PGpf z(xbwz5{Ds+Y$BoL-4pT)KAgC7FB_`xJ1-8P+=k>k4Gu(Y)Ooq7enhH%&~=wN*=gL? zrcb0G?gZ~AIjb<(A=B8&;P5L+cvJNGyvwz>S&AuaOjEsY8;Q?ZwCQP#%TTiw?R3$>gDncBrCiTud%mNRGWu9vyI1`A4MEy`O;HxM7q z%cC87BU^&mQ6;I3irw#%8;#&k0aXi}^3IRHw^WF0h~-V1{^u{h>W_e=;5{d%uIsJ5 z+;Gw*WNdq41FSfai25p&h~3!XUuWeVote3r0Sv$dNW&B+IX?*T=eh~ z2q8K3R*IZg8+v=?yQOWTkKTkn1wV$+#z{ldpw@V;dakAh+a6gh&F>&z7M`75rg!k! zY4R@y5wXO&@|k~p8?g$}-PjeBYN6vI-hoN6w+sV(nEcMrJGf zKt-aezuAr3!lZ(kWn?e;{X3)VG5#-vI<+Gd) z`g0z+qyE&`N_7Scp9sS=Vtweb8yVRqy(P44i|t8rAuZC&_Ti~}x;INN>OSo1cz z3M#=iEOLA8>&*xqtj4j!Tn^J*b@kDx^{X#L9B9PW-`YB_8igy#h7V0o$GaLSkNQxE zzE@6gMQ-NMu=JI4&Q~27NBfDbUVwL-@bS&V$&251v=;al3(l$vgjWo4=1fK6_GzSPzx zmzf!NYI*agh6a_sPqTx8&I=FkV^LY5SZ=29*4?>hEoatrJ1^N;;xK*hpP=A~(JPK9 zfysfsl5=f6H}{pV+!J>b>{#Bz97!x0;>V+*%BwHD7ZMuE7x%|D2qI-Tl#~Ji>*aH5 zT$eioUJF9II6i2%@{nRvzrbwsi3oM<&|3ao$rFtb-lRY894n!mLb;Jgh8(Dtc)PjA>iN_G#YOS=Q+=b^ARjve6HbMpCm5i=<3 z#NGAcrywekUW#LlGm-e~$3)%|K&-|*37WV-kWtb13wu=#?!A0~_$V;G7RCl7>W#iE z-9k9lSo`?E&f>)pbncL9)REy~**;_V?{X>WocSKo#1w3%z=Bom;3*-gYj7Tx?)Hv! z1ycl9?`Jg(dEcM{vzeLd zZeF4idgm^qp`oF#uMhnTk~(j2{I6Z;7NLsWfb-ra=kq9T>K}n)tIAgYy-)GZIgMoV z*5A)*1hUyj&;EL<18sssT6O0&N8B}UViR#mkBxPc6zx&p;zaQMcEALCvZUeIfw#$P zcZAzd9{u(@=<=H*4OqJ0P6$}6qB0EMUG65LSGyZo4i!6Vo9>+2Q0dT6FLOZT|N3AO z2bMv&g}2Zg#s$2o=J-11dsY3n(NeS^nVGW{qdHqZe5mQ@uvuSW^w^w_teNvzH=Knd z5&HVkfQ7_X?mX+b!W8{F4&=FtSt4Sy&nn+Y4z`uxie{ZdxfnKAv45?qJlxvkDRU9Y zx5jauexDWh)!3N0gO%XLAI&>psl7OM-%<)*m_k%On!xkW#cj=XYcpOucis3ZB>QUH zjNE)Xu65AET67$m>Td7o@5h7CwXZn$t+YsTzD}3duX62u_9fE02Rbr?e?4Pj;g9;M zs1AyA9`xA@@25-6ah>dLdV2}f)&^?;H)|f^B&y61qzTOtC#|<+?#b&&X|J@%`C0a! zF!_26*Q(@@of*Z+lWX2i{ow;+jg9PwXV3l*b?+J0WV%HS+Zh{-V;exw5f$l3m#U&7 zAfN&wC7{xzs}Opzj4-Ic&^t))i8PUffR)}`AT*_u0Fe&T-u)z~=$Y^R@%?+xk8_O* zPraXg@4fa~Yfk{N`M|-F$)zbbyYaAx%MHLMlH=!?b864|1bG;eV_kmlFWt>O*)hBnVvB)%|7?@(4n*THsetV7|2GnTGoquP>)NP zrts7Qj*fO3gKlnP88NA+awyYTxn$<7vyX8nIop+#O$+f3MW6zChzWeBF*slXto6#G z6w{Qk4zJ~L2Q2G?wS3n1ODxnCmrA>0yaL>B;IsY8`A%TVz@5L|J7$G9QhWi#i1&C@ z6Xuz(uPo?Rl$4ajFkUi#8B;6>P zBs9onT&C~NPm~?6lj~8Kte*GlPdg}pikJYhJR4iq^3teUl;iawpNy`<90i&N6#t>? z3+NL<6@iwnM8(@?#rPa6-CUNju-fC)qP<6Vid*f#Ud3p2pj~E1{qvx2@9pO&hbC+2 z>$CrSU64-p_FnXus&VzsuqlbgGWH%1Opn0|3ol~-DxUUs_8fzRH+`<9pf+6?GP5d| z-#9onrNEnNW-0V=5q>^_|~orH@&acSexa1Sp&}9dfKNu=5220 zqe5d>1H2TC;zcx_7Azpo z&|sLLv@TCBNNT4pEJ)JTC#iG08E0?glaWm0x<)m-RI3^RTZ|@;`n>lltN1A6arVQ?GLR*;XbF#ZJqyB68j~K^E@#?%xjKoMegfVXmF#0s(8pYrzTn4ROPGScSCyjeQ?)7-7SqERa7=$!{Ww=v*5 z+M4gQ)!%DMwkF?~?`ZB|#oN#9`0ZthgN;r*j$gKTF1HSMN2DcE)K<|aom{l6UM{c)!p5svNADA++)it)V%^LLxb$nVf=oqe-584E4r|=0LYOc zs~ky7UR;ZUSPcu#6*Nm<886Rpe4~D zV*+HjkgrFgh{p2MOrpR!*!NDoYf_WS@F2e+U&fYnPm=N{E?h8_wEGexF>xAPfPOTO zP1@1Gas*A+$cTLUCu3ZavXrx9v;95jV`rL8l#J6slQrH*coX%OkY=$rf;cd)$!z{V znEz|=)Lt5spGJ*My$eaz^bk!MaL;iga?zDZ6b(g1MLbBLz-rC`9nk z_l3so4>m;ndDtcHp1If&xiO(TR?GpVt=J5fcOObiPaHc2?MlHP(#i3A_FX_44Gai? z(uLEkV5^|zE#ldXDy4Yq_nx~aS)g77n;VvOpEtWH5qcsSA z3Cx}IC_s6Dv20>;;^$ugs}EG;_|n1@xcw@>G=!w6sZUW3PTJE2oc7@} zc1~s@-||!hQo4f1tKx=xR%K0Fo5U;d>yHPHGGK&*TeJY2SQ|1uKqAG@;a(Se0=y-*x@Glxj-+j(iPHYVAotEX0eMd12`Rki3ET=;J7kK=Tz2R zh1O@B?;BzFL+PqS(0;ddbO5!FOB`4wWU$TwryTBX^gY{zP(EETprwmUIhg-841T%DA3&fKLawklA{8RTHrL$n#6Dt_t|W&(qtB2mE7!gYTjU_tNdL z>PAbgY1;fVM9l^arJJw_mL@BC3L>ia2Y|qTWe|EWz=nYqL~tZ@tBAGIO?ZATE(ix^ zK)y^-iv@-y7!#!=S~`Kx<_Q&WeM(triV(Fhs^}x*fDl*&wm2kZgqPQ&u>Nc^3-5KW zToH6%a&&NTKrDH}4q&$$C%=4o7n*ood#BbZkJ zB@hsouC(fvxDzsSfzdx;0uMge$++)hY-soHr|dG$<+=4PZ=fFt)*G6qNx&>k(B~G( z(_`-i^W%HAwz;5ui9}d95_ew!HSkbiL65txi&~;8kk{j6fmsT^MLvhydh3SgC;F~l zy_&NO4MT;C7t_5dBO^||AGSx^2fL+)Awo+Rc6Q8UB8?D${9gw64H_Mo(;eQZ|p$lv;$*2>qhoDBr)S*N_4Ophy}wkptKmdPY*XB zU=Yx^4j;H=o~{w%tRPRMOv}2N2|Mj@jxE-(gBEbO0(5zRpnlV}P#QS0e@}Wfd;s|f ze@2ChvFI3E;QgtI2}oPC7_p$%P%nN=!V^Je0m^3ZFc|^TE>wl?BT1n_13*rM41#1) zsUGsguYkg?<-M@bvN+wOQn_t++z(zua0;b}KxoOh?CgP83!j*qlLJPj&?;azK#NgB zoqcpAJoH|PWW7K(6ErYKtv1q3%Sz}OTiIL^2Dnwd*VIkKPcuX{pNEW6w+~GDs+T6>t zy4O!zMXg0aghq&Y$EMGWJt**RFEhu-c(*M1(i%_6vS|z+Fd~3d*$y7m>fH>%<24>% zbs7AZfHe$ya(njLZf3&DRasG6f}uJ&!j(jVjxFlOijK4xp1Qbl%8IQfC0Zja8Ez)m;-{biHI~Ky-Af# znl-ik{YGP=ux%NAj%`AFdsjt>?#$Gl5^#5)>2`M+6)1_)3^6D!!rJusGBW0rsLhPC zt#mn5ND_y4SuNE@H@|q{NolRQ8fw`&f4wTi2clzO%PiG^o#kt$Wz)xxw?4h$r&+6f z=dbMl`USdMVfP8DM~9lz)Gr=Y8A_fs5$r>8$}aTp^-hI${~w1>N=P^Yy9wF`NSo87 zCIak%I9!e@LbvU}-o1Tfa{4xJe{PHB)CS-M2N_6>g7r`Hiy(dPx~lqZ(6c{k&@97^T3<=5$v6YdP?7=KGS2#!2F882RZ0@fS33vV|PUz~e2+i{{7 zxDCtOQ4U}2z|Orj6#Iwa)~$=s|2@cmC}k;-S4~NW=hUpnYN(L=lJa}6Ik6nJ=qCoQ z6WL&fo~F)fEGvqBwy#gifbHsai_SLc{6v0%jR&{#*E#A^CC^nPIYD3tSQY0EcXO#H zUlNLzc50koqyQ~o?|}m+ERa*woUjc9DXu6-K56jVODdA2xN+?+FHb`ZPjtQffwW< zIh@C31>{*TD=FQ#w-2$3Ic<9}9Sw{T^3W|`u)+|IWO zOl&gqX1G4YTw$#&JTx>^rF@i|TZw@G5c73J2(eF6jF5|?96kC-Sm;%n`07I4`;8l8 zK5F5ert<6}Mo^rA=TQV_{%g0H%lNz$r$>KSl;I;NrI`U0ZN z^^EH(3+nfonVE&KD*;9mHM8Sgx=s$^Bum&+guBr>1LT$rp2KSQ$<18h#6?qzT}-I| zZY5rX^iY{^-eTUKV+=W0j5Oq2Fm9RJ(Xlg& z&3$2Dv`#K1!$F)t0n0W!$o>xS{%L9?ie$fQI!s1^^na%>P#mLsw=d+xU1bc&$OMhvZpnwE}x^ zN0cT$^V}@%;uyBa)Xz;Jy2`-MTxBo1dZHCPZSx-$qY;^@|lpkduV_eJQG%v-Lvr|*Yq-hHH=(tp<0$1a*# z<;NTGrJz%*bNXbv`|9ucp%n5d_QYLMB*Sa^m8PXw8Z1jou>d~w4P@e0YW^tKvU00I zdmD9bEPSQbdB%XQsb1>|S1|0CL2hO=XNkI1G8%dbNN)V&A0^4X76o$kFJou7u7Sn$ zp5eX8hfd{kwqi3`VO1mArCLZxS%ZM>*f?x~Gr!-m z9orWL-1ubUospxtG5uWhuad@I#4&0XP`GQ(e@l0f18SUJh&4&9(N4R}s4jMOki)j>?K{ll z!4Ns))dBDFrqhmJj(jT+XiO9@ZX?LpfPxZuGy;-u)`6SWeX2G=a_vq|J0{QfkANlm zw|y3Y>4Dw3|2`!5W^uthTKvE>azB0JQ@y$SLSS@g$`Q5mF{`{{uB|?E{7?o$ z4b{k*Ld^4N#V)NFr@J)`eRFFj8HVYh;c0f3ZAY-52MjGEz<1@yvpFyK~mCe<9#~bY@n9WK{ zinK-lW=3*D6n*3iVq}F}^X5Nkmq@!iI9Q)x?S9(d%kLlmfAdm4dHc11Z`J-O4~7!2 z?RT%oQ6Bu^L36J9Nrw*=FtS9Vz8lsdD`X?`$dOjS+@-VM5J>%5T5K8r9Cu5ohs&am zeXSlZ8b#i|9V!F*XOAlmaV~U^rR~@8T#u`Q`+>$)-1xKCamJT^=@{$>7pIH>dxnD; z&{|bgS6A27nIZ-`$U5YJnLfY@0RfFU>3Frbs;a8k@t<9UPHAQ$9Rc^&+v-0%H$^Kp zjWliEylE6l=HFgkQ%OF2Dw-NS@-tFjo5;mm%>hVNAYZJUt-4wf<+9rP%hO!JxbWt~ zo>n8}kj$;i<4d_e(11RXOpEbIAVWeIPH1-f={ZijXs-1K$8nR;;Q13PSCUmwkVPQO z{QH-Rb$3)IZ(S8@LyFpq$B*-~tZMWLEUwElAn6^RwuZSIFLtxyQpJf_s2k;ikt7;+&%|%*R#q2t9`FQ^)l698|X)+0O_o$uW+<5M~ zS|SlN?nh_XnYCZrTLpP3SSc$%Yh*teKHeQI={)`3>&sNGZlO@)d~VZ-q+y`(mze#O z>7VW>{SJ{&yD?6lf%@)ULyw#Iy*Ce1I8~fgwYw9&r?L&Poy9U>zQO6k*5Nu_pPa0R zU0fJ4v+8gjZ);GegX_l8F*1~c`+Y(VMTlcnD6Q-l?XlKfDZRbU%hOFkot>WrN;=!h z&#B{7seMpQN^)0Hs|3m_Kv#y1!%!L$iZidg+AM0SdZWx&*1+}DEV(6ntiy%b!#?S2 zUUger8{kO2zdr1T4}C-VeXb0ad^ImBaER`uEPZ)@jLDC5Vud}~@)IN*zWtFAkK#;ASndeEem$gea(eo&&!4+zXIm>L&0Ba? zJUqHa+i&IEo#_4Y$L7n_ch8?|0c1iQPTl_7OBe-nfrEBlo*TMaTC&R*l4IQ?Zr||$ zHt8sX68pJVdP{#z%lbFkqrin0IJ#ia2lF>T{kqN@Ln{f7IlI{_KEB~&=;A{A$5Q*{ z*$(eHs@m!}-rG|tZ9X6~lPE21URE||@8Ch69Pl?DmNX7XCY6-LM(Xl$>`aQb?)tjy zW=+KSK;m(^NPkX&F^IRWVqy8OoLsN)$x`^W!Q%)n zYCk&6vGwS~pb)XwE^`}<3y+n>$heJnhU;7rOjnCNvlkDo&5AndzNnyP0~mg=xOHn` zcd<-T9*UTtu$*gZuso8&bxjscku^5D14GbPLjDoI$T(2u9-Mt zs+Eobmm}ctSR_(fu?C016D=U%-48E4TzVt=vU3VZbCtTntGX*4uIs2SYMp|P^1An| zK-By=GDFnW$BPxG-ns0>*u9ri7V;%zr$u5Bl35AonvFF2va{!-^ejDhTco?s??5pK zZu+iU1sWVZ;L~>Q7Zh$I;>dDh;N*?f>5+#UH_j&a5b`KsW`ykaNY;#Gd6f>!F9rSP z=ev`8vj6KJCDG>Hg~BbzA{MSS*ZWGN}$5bfBtguIkJ!Em#oPwaX0r&W!px zs{bSGl9G)2CBwR-#dC^h1AyS?!J>lhV~Eq}AQqp#Z&dHO8`T_1Av&;&a+iNdP@+>E zkZlx-9Xph`#dV4=n;S--jYoZPRzKI&D=oM&*K`>^!fVE`aLu|%@;1Z@H(+jLUKSE| zux2%02sjQv-xq$~L(5$bdl_~)IyWW`J-5kK+&g4greEmhQaRV7#v=7K}VbfPI76w;Zpq@TvA+YpoxZUdq*&>pN(o&pZTq8Ofi|zXO?0wJzbZWjX zw5J8%A0s&$ue4@WkdyeL1Fljdqh&4=Y;L~jOH$?tTEG^wP{&YZ(^e~MG1ETUKU%yQ zosL7t1{4bwM_;otFpZPGcldC}NRxbjMs=k77W&Fq%N>I;l+3J*)#;{v#`=KjbSs_G z5OSvS6vm^jKtL=l2#=9HCf(W5-L0ctq9dxj7hn4!n&|Ywz}=}2f)Ob;-0Y5263Kny z2|UJ%cW~vqv!-5m2hd6ZuYW=nqRcwCnCi;<))t7Uc2TIu6B9! z;J$w6#o2xIYI;UI5ur&v$8bLyYu;#gN;F7hxEEA*5WS}(TzSaUKD2v1R0OuULoW{LU~7Snrbzb z|HbmHmDbR}=DEx496fwIpnseL5QTzoA{-#!_dAhPSb=y7lKl0-APn9-JT*w8*S=@A zwFVY_^&#cWT8&3vBHw?WR@TvcuaLrGGrf?YA0H|Dkc<;aBeUh-YiM(>j|B0*w!p45 zjFL28Eh5aKvd=X3NX9vY^W*&K%|_;Bn~NWzS5Qc)JRi>wge5?fKzn%n`1)>Va3KYd z`q-eLJV3A%@qDQ_57@Ki;VvH|gOl}n4(rw~tS7sxt7FVdm7&8=~$9SMN4Rd6}QT3lcYcbB;LG%VC0N55OFY_w`V#yJdqv%jwgF@RFg0 zaNIt+d!R8eXDhnU2v}$O-YCxSL^eYp*CaXH$=8iTw*_pTcJAH{&QT{$R2BurOr8We zR})^eZ&7c_^6oTFo7W1XMW2SqeGwO$;LU)DbhQb7>fKr<7Z zJAk3VOp2Ir<9D}(DJZbm`+k1$!4qF3B_s}=zJER(SdZIx?dn8EWq{4n%1BD=^%YEn z|A|x0X5c@j|N3rRb>5wZ+l`5eIRntDyLkS5P#gLWRix1vsunJETtec;xW&>kqITN8 zb7vsu(cOgzm@jbrPv?DOrdIvl!>+GVK9C__1p$(=2dKn%S=G$T9xP#IXD^2S_5VB( zD#-sS&1nfn7r|lbAoQSCn&2gaMV!_v#DX)|D6aaDtm_TjrG$npTee)v4Tj4x`36Yq z7v47pwHQdyfS9uL0I%Tm3y3bjwNdH# zYIS?Hrqs?Ue#E;I33`Ztc~A*w?I-^sZI8_my9%xA2M2lg7_+(h8W~Vy-p2#^%U7$; z(jn-z3>g9S7zMrZa(8%Vu;YlL+{N}T04dX8O!|d9a7v`X!S9SoeZTKn)c`d3n+_9% zFj*j5NiW~0#b+0)dFheuF?s}sD*~*m`b=pfAAZG_zz+wlC?r^4ja4}8$B_M% ze(K274&8z&Yn6_Kt9^A=q{R;gbgTwqF@m7jvE6iT-dd))^Qu6-=D(pbKZAuqyi((R zUHLJ@!ZY8nS7Z|i^I8%%RquJq<^FK}ZT_@%(01Q%uqlEYAzLyn6{UZgWfk9N8X?uh z8g_`IaW~fpF5ki~K#R0MPTvp!I9xlCn>R6ayEm>gxpXz z_xzrWupZt%n7Xo5P_pc^ZU@X&Y8k_U|DMF@(?DRUN1Kq)3kwUt4S=4qF!B)S^pOWn zv6@B_`dmnL8VQiF3AAHGS$w>_6d+j30%TFfDhTA1F%UdId3s#EwP2nJw0z*k2#&D} zurU3Mb4w4`HZ){IgLfd}t#MlgI5|-}3j^0(3MS(p)ZDwB0?+?Eb&k6iSOCyC?}EuL zckh1bvw94ZP$u3p$%&M~=Hf>#Pd?kyzap;Up8$-jrfBXS{oC&-h5cb=$ z4PiVn1ZueOEYuOtSGi!l0Ja|RB%=5FkCI{V&`eCibIU9Z7qEZR)zC=n+C`G$kU4#? z_dNJ0AY*U+BP-2e-x`4ysAWxS>nnh=!7u4b$m&-LHb4eKGqf@?t~2)>*XE~yo7ZVc zNu=L5TR(rF!-_*e(Kp!Nt;yX#$V#iF4Cl7^>gx^9;+>wpuWGGB=SdRoORtNr4}4}3 zHO~Vz2|?}rrG=`^2yL?_!$h+OJP! z_jjp@meSH*fo&8_P^$D5ulRr%d;ZJkN@SQWi~-j_2jZa;c43p}6R3cJO8)KPPL!;I z=K|sy=O=H#Y?q6Wk3FYGs;URuYXyfNcAvk6%qNN$T0sfgk%=A^u|(X80m9_X#NR5@IHbv>OBz(=u*o~ zDg^wosurIrhx1*p;|3%{p{$HkOzgV$ z{K1_(W4FMBI(Xs~E~pWd^+8VI4aGtxccdU8Eb)o;hc8upBm}HEQa%2CQ{JkSuLTZQ zuy2tl7S!jpg#Kh94|wu0wd0Zsm!E!?PJS`c1es zwfi)Vr@IDoaFihuM3761(qETVPrW^)vyc}J3=iLcc;qLy(gtuAq?iv8`qa)5%!B(nVS zvA*swQ|oxK*uKWVN@X4D-i1|mH|t8UXI^K|d$ZegKaga=*bGchZiHT)gb&aN=7-Iw z$t~yQJ)FR(4{;nl3QW6`9QQ{gVZMOz=ihe>>*L-- zW*9us_$L^{;XPK1M|GiV-QvQhla~sKM9!5V6${VXid{+N<(Fn^Ts%Wqckm4ebIYGQ z2jAY`IFgQWo95U6T(fSjiH|?^QCtGsfdgV5D@(8+l{=_k=by-L0CzWt2=Mu<;V^J= z=h%M`Pl3ra=*N98%}iXcho81JfthJR(v&WI!LIfej{qV6tFOILFsi93^(u#j1(N7j zozd2Q-PHRz<)w&aTYYJ1GGxDtb5!x}4cGQ1DENgmPftw+?l$*$=f&00pno4&93}8d zq1zsanw*(ogFp-u)32mw1;5t%;}0mcAW%2BIB-4PzW2my*`vZ_>H*NYTo>1 zax$Rn3UvlvTKOD^*yWF&XHWWYLkCpy(gN)~Ts$-H#dh-%3$aSq1Ao6J*V0s9j8tG? zm|@%v&ji3@v+klo+B1m`4CGkm;;E{#gwhvQ1u!By^32o+_Uggnossp2&~k7U-|v8R zbFosJEE$*i&Iy27%*LyZwVeAqrKm+s;OWRISX_|G&9QXfEqRi#*wM3c-L=pT6|da< zB#qirS%NS!Y10sp#*)RyFWwm8GSi5;+Cfyv+sRF;*5c$Ul)^KEVx*`2z3>P6;k0Q) zf#CH<{y5cng^g}=lIMr@e*T#a!bi*w&JCxd^z!TFuQ|-JK`>kQ{?tAh*THD%vXs9` zpYOGt`=d*8-Di83z$mzxI2ZdggFSs;*EmZS9RE_oz_MEv0b7!?{z^9FdgHd>eJ3Jb zrQp>Tz6Oen(c7WlIk8u=6Kc6{ECeeLY(h=^`fq2uB;5WgzAhELlh#8;eup#Y(#RwGv0C|wEXY2KEs3m?F@fAAe`jpq0+)lwA3>79jdIH zDK%_WYQA$H4YdlI4}{zj&O?F0@UgVP@JO#2xn>4I+9krDXiewRTTwi;zxU$q{D1RO z-t0Y`acXii_uwj{JJViTihO(}@uxMrMz261e~ATngtV-B9pd&q?WHvU=pr%%=Uap4 z=p_>{Gu}MM)xLYQksGOP=#b9OTmS%(t{r{0`VtIJ@85^HsvTg%rBtLGj#qV)P*5lX z#~bj_5d8W>od_BJ7X=uo#s%4uiVAp!Is#*&Dj%jHQcGaEdDLmJ_IzIagKi9`la9ft z>%G4FmPFDC2ZuRF4UHhM?}LpBrpUsK$$TsoMhBF^hGLLZ4_sT=`G6pqG}{4uU%=5U zzh$uMMRV3Y#aRt^f);?)W%c&aR2qvz+-tP?37`15D>nIz#2_@2DI?QfKD~R z_=zvb#-p46b=hvZ+l7h-#@rO_RUiZe<*|g*q>-@`AOmw?{Q`wHIHzmrJ6-6ybpHGY zEx+<=T^m9YZ)j0aB#|9JI>c9KA{t3ITs9%~7v0AG!P5>j$Hi{TJ_Rm&niERf^AB!0S6M;gLOz1rZB7&Xi}B%#6p2> z4jU_^SB1IM+b z**>LxBLscZM9MzIz9%xW18Bj;%mh~PJkB+i7ZOe!0|XYId6)T?Oq)EiHL~w4 z+dt-dEhHPeHNk)uwQ1``w3hqHX{V8$#4)B}zEvFFq|MNe7K#w&4;^bb<3t!etp8$> z!ivx87|-6_W6XTWfU`(C9$D)&o)4Y;^d-z-mb?`=uTxjna))oHgZ7gkgJzkuH3a21 zCKzrOWf}}vkOR?@&6IgbaA-vsbh*6kcFw)G!$Yw-gMHCjwVZ`ES)t4emC{qeCJ4J~ z%}3=^MbW6OoaDs4NZ8-yTQE}lMDAjBHO3qEVTc<6!1?xbZ(M7+w?2tgMFBu$PE4?H zSY(~%D#`mUumE5YP?w;-Dt&_MLDT(FpAKoWn;|zrW7*MLyeC&fmlLWS7-F3Yr5k^} z$sxk%lxk8+t+wXf6&UFA-HjW?@FfSyC1^4Z`dR*KKc}-0d>wCyQ zHUv6%Y;EsYYbk$YZQ9%e-IUtW9*9+aeSPY$WVndSt5Zyj2xaV>GF*Weu1-u7hIRGa z+ncLnWf--359C^KZ$I`Omnbh{uN)9l!KM^w2=%EcVt|Jk`qT}>u9Hr8Nc)3a|q zC;D}=+Y-5ip%Me}5g_$8z?&Ty#>wR{*|8z#{|nIyJ};$fhzPeSXn>EY{nUEZc*Yp??p+Hqh8F0GGzYhhqUctwcfb zoSqpH8@o!*z*d&hY*qVDwjWn{zYMOs#ugzRvBcPl6ivR z0|*4fr0&!oar!NKY*^-`!XjHzE+r&Qerm} z@Sz)O9Ruf);{>jg*?&0z9P@Xl8w?93OF~H@i6#g2m%-3h7^50D7x(`C`vKpa+8{1< z^Op4ed-mL$>{lE}#<5b}(ZN7rgZ-X9@P(&;fY1m6N?xY*V)y8u5w){Z@#= zX@_w=FmQu<)5O)n6NmW#m!E8@US196rRZ0N!@p5v5zBmB9gZY~X}_bV;~Jn50saT>R+OHLuN)W^r{xvJ_^ZY_Z?9^QrVeg;e7tf|7zRi_q4iP+C6rfzQorEVylWl$QGm(i`cCuB!G4ae0l*QOu@iCrMVphvrz&O z)Kd$JY zYHNd&M;FMS004vn7aAVHk>FO_f_NIc%|!t31*D?d+Ou-v;to*hf>Bv%!7Kw_wF^-r zwK!}`r?5)Cb7uN^;h{7~aPOjuo+s(A{fjFKu|3-}8;p{&nZm0C|MW?luc#_A=3I%8s2o z2MXsG7cE30PN+m=qVTQrtt&T)X|Y~hPRZ^8DY)V|j?B4#SxC0)f)P;n@L`U;X{*&+ zhRR1@`#G_#gwIPs9`Yj&b{&>1qVF(lmXkS4>Uuu)wyd*XjwVvRMGhaBKeIsi?q8j@ea|>5BcAwHLbiU#-~4#F1n`Byw)K{cD@G~ z;ZRIRmKm#-jFBITA$-*>XP0VZE_wRHN5r&8bTAxyd1@_LhpO`x25&dor1nk$HmofB zF8zX97nFXvOvJk?rf*#D=`?Ee?T_!~O^qfGIQ3kIL)M~`V04*dg0-X@X@d}lP{IJ?kW*pa;hDf-s7 zNN5s%>qH0!Viv!=XF{q2D=W6&e}4(REaO}5?kgGCV)B}S=Y93DBv8=d&pzIkgP=|@ zAq$j(qp*d6k@kxhF91&A&H(og)fnk^2$_IngL5!=&p5M=e%IrP+&@UbrvzF_o;Hp9 zSbh%Q#I<}@dz-?1mqCB`QdD7 zox2zruV0S@JP6gR4>w2t{5Zq3 ziTelpjd{DSK`JNb_x8u?*#HA1b9o95?_zmbmaPt{^OE4U_3WIRVkn2>!crqi;(Zn} z0;_qtO`fTnfBgabf!WPI`kltiOD~5lV|x1P{eSrx7=7UOP&ItE4SkHf?rVdC7w9&L zVRSCJM&AWJiT0C30X>Jc9GPXRT6mSSu8P=M(8^;lAm`pb$TMoIVJ%e2Kt~sbab@ zRREKuwdj+i>B0vZUdF9STmy1s#kZ~payEW)X4_E93EBh5(rM@dWO&b8z8?^Z_rsJUjAu3>>Pm)B!~KaF?(QLL;Lmk3kke<)Czi<#H1T>4gSrQ%SW89*M< ze1r%czLgATpq-3}9K5|NGC3Fgic+qp#{DSZV7CElN+B5Gan$-h|2`0_L}Jr>4QgE& zAR#O#7yGSBh3yK=DJ!CIsw()uPd9*!-JFC}YqTW*9}iqexC$}N?4X=QcEnGPnZZ^% zFF!2++jeFp5W-HO9KI=qVcV8Axe(6V7Zo_TD{UI$?d_uD@)qlRPkXBUIHYFG2vD(A zRU?*`{Dbo;@7_uG_PIpP+daqLgx?c)c%7SDGu8p)rwL{S-8mm2A!9!K&4vnwb}Itm z3x}q%U%6f3y1JYQtk6xHMnxmYz&l=?-7Sqae}E%B*(sk8zZBz4769$yKhLLS9IWqr zG>fB?g=exYD~q_bia9!7&Ab2sLKkq|dS2gqOq*Ro_v^$2TweYv+F4Yn>wTOfU(E7G z6I^TKr3)7l3M@yu(}VGoYG71tXa7n&ClRKB^cg?(9sksySW(4#^{meDsqqI+=&n*9 zl)ii6DO8I1ScruV0FwW%v$NC;s3G35411SZCny^zfh}WP6@uHJs^Q%19*IP#_ zw(r%!6}H?*XB`(N9|s)FXQ@Jv$}kGC~qH+Obr#x%)ZJmC}0;#tf+|$^szke(&$p=&+%ByayTB7 z>+UvmHQ^DE$>KPU?BVbEklu;Dm>hi|OLeSMfwca0ok;W5EjW=Oo>VVsKfTcVEL#if zYt_@OJQkIDvgDwFr6aldL9b1FKz$>>e0YDdi023@MLM3&TShzivCot&-H*Dj-gsd} zb5^PSl%j(1LWhl94<loC8vvk}R8)?St>y6W z!DZ|U%%53BG6aP0-0V=37CrTzDKIfJ#;x^2>)cF!q;z4h^8yA)$BGBXtdtg8Qf~oK zE6pzqAwYjPR`u*@X=+BrV{>X4E~2D_;M|q@!62r2Tr%y5v3^W3NshcDCI%fM;5WZC zbxJa_(FObFOqp`1`zoPnjWms1xx7P=(|2w(t3iAHo_}TXveVAKV4K`vp%YztI@kZD;t6V2q}M<3vpLJhhF^9;%6q$=eW zv@`LlsgRKVSbJ`9Lc)n7Z&p6NJ?PV;yh&~Gpe*vf=eh}Plno0rx{yY#aL-$yMgQE}50zLTKKTPaEIqG3u313S-7VJTGX<=iS({otG zPqbi`4;rroYfTrGSbutx0`26PJp%emv7q12V218+g~lrix)ebDW4vI{YC7zo)DpRy zMPlE6AF;g6lB^9eySh&CP((q8GciHv>WaOV(NSON>H7xxE}hf%(5?)46VR)KE+{yX zfKhrkM1l2Im4lL?XBi{Slz>-dYi6sze=!-~)of?hQt37|XbsG35H{)DkZ}I+kIB`m z?v-)_13q`}H=b8motqK_d)k(sKSh!#%?hIki@PtXmd&*KH_aY{gY4rnwk8%ce(41k zL&v4AljexbR2;aW#L6s^)#OzCk7el2arMpV;PMhiBb!IEbYTGMwhRyPb5z<=EpoD* z6t0utJU$b;@Zp2|*|V5R7na>J@FCVGJuB&Evbc54gUJ>|(3LiZCPqf!?GgR+O(`C` zv#R|FbY;GW7QRdgg4x*htgqCeTgi#LfF2*vGV^)pR^{`rkzVT@oFpn3 z=8FU$6`uoqowMU(7DgPBCCxfZzK^%b zD)tq|hSNt_xZ%KobJ60%61aa8m?aZzAvREqrbC z^5Lpvctxbw*-f)!N%OXCgEI=BtBnI}hT|`cNibf-K{13#U0`bEpWmGhK`=PI2r{C! znyl6t%3g4uq!)Bh)eYmgfs#)1NLqs(fqsn?vq-!{l2yaoH2yk*St3;~41W5~r~CdV zdPz>Ri$*qCq>Y1|08)GIF=2<1zi3 z{(H#D0dG*gbm<=$mjkoS8Zku#IjeEW7f+q83Ai|Q*7zY-_&^;3%nayL#y`Hh;FlEO zha593hWS$`&9u4^8ahkr?U?2&eRcZY|0AeYWyMo_+LUBrM2>U$19=wE=E?sK7 zx^h4wkF-(alp5Jsz>ntwLZn|=a`Hy_>pegD#%jKUG4!=SbPvKQ!!_}HP>FY9c76NB z)8O2s$%TV+U25(s@reDg-r@H$u6Z3yy(X~8DKtwX>~BU;_n)|gfM={JN!NJ2|J#zJ zrLJq7$F<*0KM!0OA-a>)Cu22O1R0hn&bq)fB^7Pfxi#CX@Bc7I!P*FF`YRT9O+?fS zCH|!2luUlwh}LpoqW7s7P92dozE&!3Mpz#yMB8qUY3$JHRc{< z|5x(QSB6s(ee%TVL)o0z90i@E^z*?7{%|*Lt2J~Qrsq|f?K<}cTmDC*3Vf%J;rJgm7)*h{jWAbrae7BaK92ZNpVzh&YfBaaS3eOb;JnV6c#r!w2(lw9HIR zUYs7;d$P*P$^s;rpD0)ZVqs}g<}4^u7N$C!T{^%_nTffv9}>&TNe%^l>(*+`gtTsY zLi3E8SPNqoe^PW6CYmTcU?y$cJ7-{;%b~E%s;&M4p^%=`1fj*O$gQdVfe@5F-JAm< zldL~RSPW1s?QB(>^{)ssX@{Z(LJB=&Ehkt%(U`QSa!Jd;XKogg z04>*6?PBTPsk*YT`Ne;Wi&t}5$>YGGrd^9b@p-Xc*ecwd{{a_CbhHK0J+=gGk#kX< zu*tp)Hv5^M1isvAL?9p{QmxD^4}rbm4f+^N?01FMLe@tRt`+(?WzEWTH{j#w zYXL#8l%zQ%`AqDt*&mjh+jpbqfXOc}b+dPcf)aAE598GS6azk)&+_78VPOmRikf5x*VGJh5qDOVM{&Q3TuquGVrK>B z=>j1VqPXv&(*fb(6+bkWlDVTecuRvd7PeD^{UCPk!ZX~NyKIz*om|1GM!U9%zl79X z2i2XO;D;;z(Dsv-6qB}d#D^kX9Y3yo>;$9-RUF(^F4L*N`pb}87P)|sx>BA} zQ)=x^*5@3ceyeBvljH$LLSIeE${Y@I$#f=ie~L3T;{vVj3!Gh=1ZDeNa(M{1CI zsDO6k+geQWETjk}+qOwoa=fCe$c&v_U8$zA=!QG#;ueF>qisCSa@6xKQr8}=Y0{j+ zK$E7L9H-WxA=lKnb`QU(Wk2ZQtU(cH(VFRx(JgUz1BhQqX~#O%Qgsfca*7_0^T!yebEubJ9e^5@9m zU0W|A>A{*b6%4=Y_{@?SQsgoN%8o2X%_NQnzLgsI1bipPTj z8ltIgM+CuHbp9)ah~}3WaodM3^mY)>)5>8@P>@^V7;$&E=h3>zQgyhN|}| zJ$>?{z>AG>k4`!UC~TR(lyEfa_dn0>`kk-ld1>SAh$s`@>}uk%(i_FH#$kkA2LGyX z?0EX`)sv3@Dr`ES=ENmYP$=9t)P0{_+jgiS8oYcw3QcFb=I+*h-oPNuP~20u5>-c{ zWD;6QRRj+gH`LIfar{i{J@IadqtdSD4Ey$8?D#qrOT_4%Mx`u{3&bd&TYfy6+)VNF zaII}Yzg{r(8I05kZ5m(c=Zs@utX6lJqsA^a#d=0a9xP(qfPGKd@DH~m+b1*_w;kXO zo2RZ8pKEuUX=%?j?dfbUT&fa?Md9p7>6$(2&$?{Y-T=!q&oW?sp{@km@AbE1aFF!m z4AH`Ubx~6d=4R|w>%3*xQ1N2_3~DEncuPiJ-pM4bSdW$IOzo8}=dR^BwIVyS!aJXn zReJoMPm7XTOMU#WrD&u!FTC4%)r7p%`^;2boug!q5LtcLg1e++23N9V+*KX!-rhc^ ztfPON@3cwnRxlW$e39G`zeAtFvFrC4oN{Q)H3v>L_72k5#DJA9>NV?Pr#zj}r3GUp zT$-8o%@+#9^p!7OFyA)evQzY{{Ry&$gLl;l0oJJmb4%BFUF_nRje5{@OGZY^1+m8o z)`bJB&1Tb=^Odc{yF{)nRW4;GPx37sN{+HQ^I#F2p!zi%2kZXlDxk~UgF3Y(D^EgF zGW8><6np|g{(e>^cOgD`&?`Gz+>g00KFHzzeZc`V>3XvRpLC}NHj$MC3=(umKWlSZ zj2Y2z;5onEyrM=BwIP8iOs-X<_zCoXK0i5G(V5NBsu8a5dYlQl`9JkHAg_*9LqDY} zTBJf9&()JaX_eVeoC^(Oh$^GWP#kuM4^5rky*qlFBXlKo19@gP6o%gOAVLSt&$D;M4t;q zvm}|h&b>!+zzYig*S|3JxaIct?FA=$u+RKTE-ZV7ovg_=e73SQl+VT{hswK?qZ1iQ z^hv)LKJhzkQTVWOwCF27>~@MitIH~OKDFn`GatffQPG@^vo2rWR#1zC%c|j#?~WMP zS%c?%WY@PQDD7Su8(Yk*Vs{m6f>}VBCv4O}4Sx9NKdr8HRddLz0L7+2gyik}z;4|CN?O^w{UmomuMIj<3`76~)~r1F7yz zQz5$2_Egp3Xd-joObw{WGk;^1Tfz5Wg8u%xpeX!!cA`w~_dX%p%rw`>$@uFlZH5J7 z?SZM|$tiU!pPA%gT^2E7Hy=s=gx=uTg`Rh@vwZdHGyFH(`<)K}@%uK@H70&@0!TpSidRq-HqK$JI)*AhiHGixy_SEJI)4uFJdyLxwn4Ekf`8qsqD?P=>sUF)xySay|8Cm>v*ajxY;5du z7V?Otxw;>O(DMT}o0*B=%Mm>a6!e35*_M7w!76YY<*padt@ zFI&4&dTgcr_=6wsZzFvPaLjK&mI)gi0$CBP_jj?1S-v6ZKuHfiashxjYZq9v_4(1) z&9^i9bvdG7COhyGr=1Box_Ww}BO~o3Ql%Um6*&i-UU2h!N~T;9=$c{dMk*Fd1)obh z$u9V8ZRfw-sv5f!cFMUy@f6H69(06Z%Y}u2O&gn-%w|CBkdT-MRR~!(ASB9K%I0%> zQc4O;r8Bby%|bWKAG5Yco)aut>}F(1YCAc?DuIro`%>Rwm=B*0oyz!l#P@Vh7WqlX zv@zFZEYS~YPH-UwsRr~wkA^ffG%SyyLG%sHIrScxE#cRHqlyCBt^(U3RjXP^1%Yi@ zSL6HBZSycu<(Xuro7L6AoeOevZ2^D*H3ujQq0p_VsoA#EF8OcMih$$TqRFSYMc~Q_ zz*u>%z?c~JLx)DmWEK_{9nd`Oy8)(vB1U;k@X`Tmdkur^|A{?jM070{$p%?cwCd2O zPe<^e`US5;1sQ2+&{M&fo6H$pLASHA#mmN6#JQF4=I26~010y$uaXf)57?VRLpFdq zG~@tNM>abI%ihiHL*6UwPiz~61&eAc-&4lM_)Kn-yPBJu%gf8*1N!>)E69cTbzfD# zo52D7q}y$-3nV(pi36IA0SQ51h$e7F`}t-V5Iu z`Evl4FL_4N2?TYKlYGdi5Jii+c!eQMeRFeS(@0PrT6X5l8dZ?s0@HIj{3N=e$l6G}dA0jlKqA=U z3#WwLH%zQ? z_LDxoz6UfkbT5=)ro`2ZZ+2uWWI_#KU0WLp)IfrRDz}-JM9=(gMX=C-@eKuedFU+u zZ6hC?X<~y{6b@}%zJWy$*s?Mwes!UwwvZl6vrt0`tQB!lr?jM_%e2<(0k20v91hj#fDwYP`r1*n z496$iFr&D??}^$pcY%+f`U&J96#ZAJE2p`7@x`pizOEt+svn`DoSSd1PC!-k;m^W3 zR&NddwgOXrdPD@*ASYXMJgY0S<|1}Dtt`dQ#X@lVnu}ZmEwltCH#Sz3%;*Lx=7FVK z{c-hht>bEZ$aGkjmvAW z)ba8CP!{qZPU}1R=Vhrcd=x)&?Zd4vd^p0J;kuI$$*G)adBw^mIEk z`!$02Dmm%>`{U{R`>R{6@{{)T4G#JXz{p$>vVg>lh?p4rF{Pu$pl}clob!~|KO#r> z?|%)$_vZZ(2t&2))VFxQkSqZ9f0~5JpX~|t4b+lTcyaZxv~(VUaN{IsR%j$>J_!X^ z&6XqziSVOG2L?qzxhch_TOMxlBJc$OygtlV2v^GrHETRI$yKW0-?#63Urp49(C~16 zlP{`&N5{lmzI<68H$WzXC(VKit|&4ZWJykgytJ8_nW3SfjmQ54g(Oa z|HQ4QqX?qrkk1qKIV*QjMn*v!y9aDOJi|aX46S38y220WLp=P&|13xm`2h%h%J;y= zQVdYVRK6pgmXJs+%cV3Zwnw!pRW@*?y!_Cs({@QxJ+p9}(6 z>7Eeb?JGw}!d>gsLQi2NjJ!89OJu{11AeN5_OWM`V9<5FW%G@QC`F zo-6z6J6cqV5MF5!nMXm%e44e)9d@VRtCE)-v+rY|zyGZg@lG0ubquiY{>Lw@KUu_Xe=2i#UX7GiUP+bZqs-LoBqMSjbEfnmt8Pz^JK{G!U)ucJ=R9a=AFv7kjXlbXywmH{ zgI;l&{pjD%!bDKgeI?Pe7h5l-X>;qs05w+lPBn!z_ThlilgZPz9G|>SRjGIm=Tusk z#NuvgO6Odt9Y0zu>$g|YWHsWT8O0a&pe?#Z=C~G)5z$6fqih2Px?t}EJGTm)SqUYn zp9wtJnRXnxZ((psFCsY1Us$rUMKI>#8@u1{E158?7gvsv;TV?g5n@lL{b_KipFd@G znw-qv?tE&B-oda-RpR&4mwsy1M5zU2EW!|C_K!HH1$9V&)KEPoZoY zDJjU5hC)7bqJ-V*x3y!hMHv@A#G%x8b8;?1dFwNZhjc(tP(G!_3?>c44%zDx<&VA7 zQ&0DQABSwl%t zadLE&buBC@JmLQVWYl}){0?VlWhsW865`>JN!t$()!cl&B04LpNs(EX<%4I&9k-M9 zO-*-mOOoBqWgIdX2>n92O193IJa|Crl)zRbja!+7KZs0WX<|~-^ZCs4qtAb{17Kb^ZQ2yz?|-Bpn!#sh zX72HSqlPV>j=E@cy6+xWZxXmEwAK{8+Lhp0&o>#D?mMZ5K&Ky|;pXbOmuT zM_p(vB*Jn+sjUAb#0m|?d{N>3<=z+!`Kl$F% zW0WSU7G47^Jf5;v4P)%r$8;+Br(#TA1}M@wsKd082}^7&45OvvcuqTFD72tekY+ddlx4SN|qkP!qG)jw~E_+Oz7D1pJkKnqhO^xf?0dq#X|dO$=3 z4P#MEh1B>>6G70o4_fPy*~Q)SXfG)WP+;LTEfx0RbJ$?OhYd>yZWGUl5rg2)>KYpZ z!@?wo|M68hqhO*k2yH za`d6|77XImT3-l4sV)(Dg@&e^5OlsmFRtJc8sx-&08s2!hK4xw@qx~p5u;pKy85# zCiRc28;gBvZT_a2if(_L$sd?$bNM2S@hsF|EZwx~CXkSZq(32k+g$*2tAalBTO|2} zu<^xP`SP5vw^4WG&$~1ZXI5wFd*M*hJx}5}P1GEi(;e9C{QFXn!@mFTk~vEiW(YUL z_DK8LUZ~&k*cIQHQj_A))v5E8FKc1#QIW7Ke9_STK&e?WYpp3md;g4VI?Xcj(#ml#nYVJlT>_9ll zwSY&6KqVmIHT*$K{|d;~trocBHLO7;e;EYfOzVQb&C)A;CM*p?LT;A+{cyN2Vp$O2 zN=O#04u+C9V5@-kd?%PRJ$6^eEDntF?c*B|yYq`Wl`@2JD=?PBd6JEgOxFJkD79bc zg7K@c5zKP}LeK8CPRpaYT7%3|3f?A>p@(K0FLEUHQ-QacGcBGtLyIz=vPQNvoS}U0 z>YY?wbx;v{9|M*RCMf}P0fy|&n*-dSxLXG)ULYT!@f<2~;*yd?4(|8O{zVkRFelLD zqfAYqF&Jfec~eVE&8N9AWF92R(bm>JJx~{SJMBDexNu)ay!_lSI%dxN9tWPO2kalu zm5@WZA*ZYjK$#M_6BxJ!3NW{2LEfnHWX;sZE6*j7z-V=G3mi$yJ{(EX#igadSUxj? zZr-ENrT-=J#O*TZfd-L7kk5h278XI2c^es`rG$h8|Nr0`6~9bVC;+1|Iy#|oRWSXH ze+-|z$y2iygA;BrRwy}F3ycp4v-k+z09BOG&`{4u>Td@hJ$@W&_3@QojP}O0)kKW7 zA4*G2Eh<_^TN|Q6{N5(2R|KBy(xr!Zb`W>Gdi6~-$QJf?bwx#6;rpQ^S=4#_3y)EK zq7F`2`$>F&9BqZ?lXz*J6HLAk-e{J7t@jQw{pjgrs`KZCjMEg?_&bA(Ab--6iwyf6 zNDlUFjN#zq3|N4kHc%1A+S`MA9V8H@s>twI)4*>zXWZgY4FC7x6;^+-h6C#s)r~k+ zfhZf+)zy&)24IR0#H*FdPoJDK(%08_aLA6g^wIyY&iAS_c(^fA7)Vcs{Rc54Awp-qA-s6HZnIEf<~C;1njvNL4cp19|CF+R61Y| zLTAwWBA<6i8UzJ0GDKBRSXo(d3qw~ugy2AE*Lv^jYw?DmUg~c(Ez|zrS1a5OaIhys zSr3bg>TuE{2&|8enjnPoA|$7#sxfWZy441aX56x+8Cv;$BYk|VL6`~;s4!f73Pexk zp+Hj=8ZG^yi%CgYd89?f)N}v>`v%>pfJqWrajMzZ`u$h%_c!nU`}DPT6emV_N;uk=sXC>a+c403VVoMM6&*D{ntSecs)jh7yg^E4)m6KD1 zTvX4+G2vNm{WPvMwA}EIqsXXOfv?>TJYiWD*29cG?Me0f(Z8kBXt84uwHfy?czvD- zIoR)N$<)zclEW8C+Hc|zU;w?ed%vd7?V&bncG3vrC0ZbeRzX`bGeMM z^wN^or6F~kL~JEuMAj2dTnh7mSc-;1G}y zW+lk~nax`BiOFJ+&tZR`(fIBIcJ4h-#Uw4e#soL;V_ImPmdkMsm!gcxb$P8;M1Y& zx!9=q;jCXDCy%&<1ZY!2)%78qZ^knXTwYin3daQ(QF3iWiCVQun$#WwLHZ}mJg_96 z{iRldH*G-y0ZkE`KN4XB9lBUEQ(dK;*19@9NOs*T=u`O|FSW)rs<7aYw9Hz7_W=v> zmkgJ)`sI@LcD+aEB~Do7PHA4d7Tn@J**lckuK71Br(bff3wU6dCm-($l3dVd4uOPx z-ogc7S7XTEga0v&nqoak4063vj!nLk!N9Wqbq4deGEB2Eo5|*83ipQ6LVre?%z~g87vbIpK1alO@Ir8!G zsUh)SfB~cxe0+R-_4M@e?{<8I3ibHM>_ZlrWVDo&v$e1ZsZZrU@@3w({?;nLvSmY` zJ(fLoajs962^+o{YJ(1YDEqAky0NJ#I4q0+iUHhu^@)(%gr`ktRF|UTav18$kKvYD zZb%oTmtP+HwYv&cro}rTuz-XGsAC0b$tB)4H8u55tC)XY6>(`rfd)L=7?L;*qCb{z%j$IVf7P&RV4DwEG9E6j(QS|j@W zmRjRv`#rVxu_8DaPZJUna!)+m7i7Ie2hcGIA{t)q#Hlg8I^PJsbWp_YL@VB1Ef}CB z3`_h>srA_vUMYtrcau@!L%xPfcT+Y`-S;>kOGl}-Aivqz(gWW_Wl-kmJUTEk@8y*E zX%UUtL;G`jCf+Iy;a;=u5s04Tj&*nhV5}4V}1TXC}2dFQQQ4rqy`l(HkPKUf)&FA*%1c2e7ZJ)JrRNhU6Ij z;j9)4WqEH_DQBy?nwsGk$8he$+gVxTEK*v>>5NU&?s_S?{%=k$=S2l1elD88SO=?) zVAiml`pAd~6u`Wi8dW_#++IW&QcKUdVf?kIhO!_b*$a88%FAt#pR?y=z5#J5;=}Zh zIEFU`C2R-R40^plz64H!N@(nr(3pjD$A#5wx%t%wpk|7_o`ab=O7=X=y-d(ZjJEN~ z&B{6z1(h)`UJS<*VFLjPAlG~8AE;1n;OZ08Kw)Dp^NsZPlcyXZKnDF62y-(a)2k5TAewNiJ`z-4)q3|Nm^GV@a+D>5k0N~VtjyH-euBDcVBCC(V z0$skCp_YfgFD;BBMkFI&a-A71pe>tD-&J(Y*Vp&h7#{L<@7^7|A2L1EiUEyNzQ5H# z@Ck5PVPT;G(?AF0jzNL$JtR;;BRNgPX=Zfq%d?=S1F#M%*rSd53jkPF1`#9TYs#?J zb8k4nOk7O@$sRz_4d=Yo{$P8D1Rq*bSzck0!*sn7FJH2;vL2z8h8dN-xSnTf&sW0T zJ(^|-5h#0IE-dQm)sOAm#)yp-a{dc&=>6AZwCG2$T=hAEGVW;T;vE8#XfV08win7u z7rDfw=dTY9e+su?S&LGUKz1&OYq3uTa2zD2!G0)9j%`fc@cuw28i=MyPls`*CEoQQ z7r%6XpZ`j1-E&R}3aBhLE`5}CORpdHsAI@pnjm!YMPeKJ#;G|rYJLnML)|KIEe;h` zY&&={=U%84mHH^N9&1P9?_bWk$-Slf`SWt?C%)_*IhFfwbZo|=*7ay3nY!VN}ORT#1RpLP3Hzv%> z>&1&9Bn7w*-9m0MEpqYQKrutq-}hEBXFyI;Gmd{$#dy&F88X5`1WB#r+b(=2(-au> zx)yDvfeV^5B2fkMw-q8HIB~=AZ3i`JsOtvdDdN7jE5m=@2STc1fyR7d_&Y|88YiC3 zRECHSepx@4iN=G62Exyk<5#klwj#gI3YAqoH&-q=D$e>5Ra&vFKy-x3MQjr_9KSao zP35R*8p$+n_J6@O;S_MXm76%Oh16vs6v@if_FjHRLp2ch_YeO6eoF=D$V8zD+V_+-_h&Kqt?ivja<|RM_57xxzhY3O{po^RrI*R9X(DU@DMRqiyDkfHQala&Tku@gacp4!&E6e^5v?qGN9)x`-r>C{ru6iq#l#~oiIfjml zr$-vm)$Ji9gEr9N!#yQtvYlOCXEJ{HGa^Za)eTFpZ;Rr2aT2DNLl=*_rlzQf2&mgZ znP$sn!tg^D6l86mP?CVu^!NAw`c-B1tP^;4Rmjx=J_Yw(DI#k5?MROAmlFpOWW4&V z+l`<>4E-7mgQ&>_Zj#G&!5WHotarN@->Pjiyn*#z!zOF0i>kG#k zt?%1jQ?viC(_k*+Z!n#V8-l6ZP?q8Z*}HRRDj8xN&5ey?{m0f36Qb?Rr;kI50w!oS zG-#A0#l?M&vqocMK)ZTbxLPtw>Q;AI0Pv@Ss(dXyh{|4FKL-Z~emG>dut$S z7#SJq$Y?&zcUfI8B=AEwi4XkW;U67}aHd@6cKxq9CBXr1DgP>Bef=@0=+bEguM3X> zYHY2@yH^{MgV%HGR`jk!9F8SPJH!0tY{X~xv9Yo4?(P$opyC1YG~`CIXGHwu6(t$0 z$KX@Pskq%1w&OV7E371uiIF$_kC5_@RU?Kn4mbAn^HT$Yq$YDk<>zhmGf>08d4Cxs z;PKjWnQ3XClRtam69r~Z&`*$pc>E|YuWBu%d0+;%P`jUHY2`=Zp%yV~*)M}o zr`(S7gMzPz8R((pm#!=pLPfMIbSz?g@Gg2dTRS_=D`$6pFl#$ffz{3|X4v{J>k%ah zh~)qn#{r4fg9kr-+-u5wSphn$S3y|q8^;$JMi!C9*P!#8S8XxvR<4r2T%sHh`aRg% zx(*R^3~W>p5HphFYx!%u`lczdBL7(iYv($j5sr=!LN-1KXDAcNbDzizdA8_MP7DWB z1CdjpcEI2u^uvuzIffsq_w|)kR>mbJlKc9g5K*ffCUmgs;v}$?!Ai0A1qkohhhkqA zs+%z}FsKvR(}M?|#9K38H(e!LND=+OQsubh4=>G+jHDA^Hm8r4o%?e);)u@ z^?bXEoRm+2B;NVOKy|42Djay(kXwZmO>Ni_3K?*yeW>Y6L3DjVMV(u!{_qcizg0Ns z?5Qo51clA7Pct-(mx45E zq%9l1eLL3u*4MH+Ejs!Z6eZP)LGjh?X+S`W9q2u%(}*)%Wr+a(VoN*>4;?xLnu0Br zl@{=c15YIHqNAe&7H@WH8o9`Jl9MD(hh1u;|)RY@{~o(Nd?C{lUfIg z>+1FHKsD4ZIVi-Md9Z~U0@~<677grNe=JmzWM)poL^=jjX2GpYOfaFUm-+zd`Hy)S zA*BT|z@rBb;_~wi5_n<|gKz|NACLM(K*Gjtd=vRoBl$*^nuQYuG&JH%1KO{MF^$GnYu*mpFYDW=H!(T(NVylG9VHp6>RO$82l)&3Be3K|=R`}&#= zzUYwQ_-K3iVnj&lN2KEFw_}*4cLTVS&@}G=3mS<0#%SJuFvAEbKF!K{AXhMKhyN< zF&{#+t1Bn3AVbIG+{*JJJGYz_oS`V|xmw30>xpG&%*Xdw<NjcW<%g%q$t)_Huhc z!CexL;+`Myt`B`5JGuGxS@>=HzrOyxYve=SQ27UVKDrjNz@YqcXNAv(HUUCl)ve8R zu+4OnDd1P4iKiz74sSqtExgX4ou9yGNPEaR_aPvE|L&iK4$yPG*W70%<##D-NM-QgX7T zsi}brjUwS48czD)#6CiklH>+fQ>f`lU)0dh0C&g9$r&W5m*+Tj6h(i(nS6ps2nHJo zLb)m``>MS_JGY0vpr~9MPA_wW_T#gmB4)SOJ?Lz~VM*)G2PhnjZBxE9lzb!mUt*y? z1;1FmCUZ8w(%FJL=)a+(*sgO+fy#tbhR7}ikRRZ*bA-+fj}04X|rG>=>=DNKUpGA&q@F;_L!(UZM+>t zpV5tWT{NrAu;5jTrE9Z~dL}9B>)UyhFSQ=CJQ@46H$rD%1jjC9kVQyFE%s^YX;Av? z3{Gqm6&d@7A{Z8KwR!9KVsrK0<(|+~X9jW$`^{hzx=iAADc@k+b{uUhgP>-1(_ggC z(z8Ew=b!V?*ma;avu}=xzvstc2UD;tf=YKRD(B1fOqYh*x3;^wn%U1-ls9IhE5M{?8egccMH84GmSip{oB=H+rOv1@By5;rhl5Q*2Rmi!!93W_v-}Cw$r{hm^ za&jHJ!AkBHfy^CycjL_HCox5-ocZxP1>}OmgyBG)rCA5BZ=UP9Kk zMQb#W5@r9FJ{rf&meU_&&7(vZYtVwehwN?A_Fmm63{h9DwoYrgW83ZVnLr(*s<6#b zk0-}!+=%X1>`7x?5~a^4?E7BN5cTvhl)=K~r4j8ZewHA1mu2jy_Pyh61$J6oc&%lM8LR4-_h zBkx&*P*pJ%L%Nl5G{PvG*lZ)Y;XR&s>Ebtm{iF)#n8C~b<=)2u*W^Gd9b)r_DpqQN2`uaj^?3KSyEcf$%DJ2)Q@t|C1 z6)LlBM;<@e2zudaMg%3rnAS|kA85h6ET@^fvHUuk!MsNBY2DX{dLt3vJAAuT>!m|0 z8A#*Za;+I}Wswb~EH$N~3`=^hJr|Z#)U5H|OY0>{CQtq%Mfi!Vkx?9Fw=&JUVf1*x zkPmUOiz~e)tfV9iB*aB|d6gmzKrdX#QI?B~(VnyE4benv|E)Z;Ng`8iZA1}S|E?lI zKzHu$;I-*QExv|rIz7GlF^flDQEY=_+I$@XC~E&2MQr49o@@l*t*#X$b@N|hDObJ) z24&o18@$LVm$tj?bN+-<0{3#BarNFX$_7GM+L)&Q9R1SdE{i?mXXAbn`W7~hU=$BID#F? zO)2+Mc6IL+n#lQ)*)FZRIP=(?ew$_9MAa8xD;X3IM9F7Z(!nX(-+doR8R58wcsrGE zS0l14EtBD!d*KynJvWI`yO4Rid~1Ia<-VzuIM_6)t%8>$pTs0*;_>ENq{Y4+KX3V5 ztnHc=eDwCzm(A;?-R-sgGxNm=X}X}y&mj3t0xMoZa{l4Twx7#s6h9ZZ89LHt`f4n@ zN*s=9pVXP<+8$LsoG1AYyqLlw667#yu@bbo!BsoZXYHZYas~UZ`~w2@R_Za2-^m^RXx4AD$T)iV`3bgv-6QRF zgAx}VN%ET0_(Cy9@*9&RsF0bt$DY0d4Lw@ zq^Vb#bcL&IYIm8c47!ajvjYSQDALLG6GcxlzY=B14X&e#oc5!;8`zuMw{FQEX%};o zwXKai$}nU`*80)NE|qa52R1@>`SsvlbTowtumLYGQ+Q zLFUyn4y0EcQs>#Wqvj+cXeNDtynxX|>Ky$bW1VgH?>!CbE$U_4Y{K2&Fb-{@rd!&f z#S!$pn`u2x@;sNXt91u;L RootTxn : client.NewTxn(RootTxn) +... +note left of SQL + forking RootTxn + into LeafTxn meta +end note +SQL -> RootTxn : txn.GetTxnCoordMeta() +RootTxn --> SQL : TxnCoordMeta +SQL -> SQL : leafmeta.StripRootToLeaf() +SQL -> dSQLServer : SetupFlow(proc spec, leafmeta) +note over dSQLServer,LeafTxn: (dSQL proc starts exec) +note left of SQL: actually instantiating LeafTxn +create LeafTxn +dSQLServer -> LeafTxn : client.NewTxnWithCoordMeta() +... +note left of SQL + LeafTxn issuing reads + on behalf of RootTxn +end note +dSQLServer -> LeafTxn : txn.Run(client.Batch) +LeafTxn -> cluster : txn.sender.Send(roachpb.BatchRequest) +... +cluster --> LeafTxn : BatchResponse +LeafTxn --> dSQLServer : Batch modified in-place +dSQLServer --> SQL : some results +... +note over dSQLServer,LeafTxn: (dSQL proc finishes exec) +note left of SQL + updates to LeafTxn + repatriated into RootTxn +end note +dSQLServer -> LeafTxn : GetTxnCoordMeta() +LeafTxn --> dSQLServer : TxnCoordMeta +dSQLServer -> dSQLServer : leafmeta.StripLeafToRoot() +dSQLServer --> SQL : final results + leafmeta +SQL -> RootTxn : txn.AugmentTxnCoordMeta(leafmeta) +... +SQL -> RootTxn : Commit/Rollback/CleanupOnError +@enduml diff --git a/docs/tech-notes/txn_coord_sender/mismatch.png b/docs/tech-notes/txn_coord_sender/mismatch.png new file mode 100644 index 0000000000000000000000000000000000000000..87ad41c63009133683b864c4589fa144f564f065 GIT binary patch literal 35488 zcmc$GbyQVvw>5%-D50RDq(P~KAR(=SC~@dc0jWbHohmAVA_CGK(j9`NilBgqw6t`0 z$)Ucr58}`F{oZlM9e3PseEy+$ID4OGKe5(abIx^q|Gf*}4&boLkPwl?x zSqwE7J4v!+c$e4KCDKU&6-lP!^#@lEED$;Tv|>20R7N4?aM#;jEi-^BQzDh`+I;=h zU)E2kj+9oAT%M=Qef9A{yd0~?UGuo-W+w-#H5XDJeHI__Os*0mPZ%WjkYrwRyCX5~ z`(4tA!R>Q_KqqIzk%Y^6;@wqsS=}YsJqhou^IdCEZXz|Yr>?egG09jM$yL2lOZXPr zNf~5CtnZgsFTZ@t_kqUc?euCgw>kZ3MtVyPCaOXoSlOE-w(+}?D#vKD(k;!@mAN;`n0`v}4C+f-k?MPT;QPUF zIWj@V@9W~(yfaVjN)`BsUVfn3h#9AIzET->svf zRoy0eL1m(Pm2z}sOVRlN1t*0@ojx;9xVQbsMXK=^wAPO!lHLpUE`2S|cWdFc-5emO ze(2SErt7@gifmM+_OZNGU%U&K8vR~LJ)pET&`i2oinyIe&KUD z!DvVS`-`xNTPF-oaGjMeF#S$cOMlF7Jt4p3u9(-;2MLYmAMJQ9j>JrwVz^Sjv{cUD zr;VB7Psx~CNlPE-X-}bg-UIs)hZX;*4zkU!qmci}k^j$s#A!RYe}}(*e6Xn)Gv$CxxdX-rO6?1MilrudA>bWqZCajY>iC@82VWodbN@)10iQ6eKFt`PDR49%AA|Ow`&vmq3WM^P^(~*=k^sPnD zmjS-}$=VeWHmW&?v9p2Oo|IRZyx2bC?@8o279PMPipJzMnL z0=euKBd4vFPJecKy4C3XQTDA$PtU5Qr5t41aN0&5rut-AY__zp`>DJ&X$Bt1ky)IkZYPY9q^=+Q%_;kP_~d@`%4k|1{n1 z50h6LqoIrLo0vF87!E3#afm!STZYcP5Hr>OVs~eL#E0CQR||ztw4nLURz{-kLw-IH z;ZUMDF(>Brix+-m1EV=Ij%wycBVTSvg?>};CnDUKeRrRZw>e!xQu0NRV(7DH$Mf=b zH`-MrE&I1*`WVn_%S{ACFGHy{d)D@rA*~KZ9~87}G$eY49Nm&Pd||xFQ(>NfKRzDs z>eT^Nx_U*IRTm|pr5|}ojN4CTY!)A)B168$Y`=IGoRZShsmm0h5=25XgB+a)jEdxN zWBUuXc|JdZU?^Px7duwaP1YifGYl_85(U3t}_z}?wty2H?4 zXv_kGM^kqjIfN0?C~9i{f#8U?Hn)`6`!i}SnpPE@rV6PQ?kQnms(!*XwaL+yC0tui zy9lxw85juO^hvrb4xJIV@`@JOZTkIG64uu5U()Qx5}I1u&A3grcyRBQ%?n?lOl*qk z>U(#efT1thuV_b``o@jzp`o#&!`x@CEi8We^d?1iK09-Zk=?4ZEiEzDbH{Ni)O&M% z-F5KE!1#DdUrE44&)t=j#h4gN)$A=iZa7vZoZJ$!y7OZl_DwO|hK6T$mqqG=7@trc z)F&Kh5~gxk92)P;@m#H;nBUlRO)C>K8}Sj@e(f>a9baUoOL}-gb*ZAfS9s%|1*$Q} zA>V4CjD#eboa__@1zm~T#`6l-0YbtJi{;e1cu~S2CjVFDZ;LGEhldw2)9lXzQeAU& z8l7f~#svknGPAOh3@cn+Po11??PN|I5Wb1bCZ#;;UeCMxZ056Rl60Z79YO!ncdou)zYFasxL&_sc~Km*5}wTUb6|9^yk} zYJKFptgdepSwF)G*xuAxi27s*r&`0XbHTCOdbx@8VY6 zsxbC|h~*El&x=&}N88AqHwr~uadB}E*bYN@#r3jJB*0+WM>glHD551$sNiuc%bQ93 zX-erK_*&&o0hR>tu9iouUWZp!GbB7bJdBKz1r^H~-w_5(B`Kt7P|86dEry3RG5vtK zaS(1LCMF(pLHs^^DEr(~W^bmd64KxIdkryp7RE*?q$cuR&co_qr3ZIFVvXU5>7hf1 zHXN&`-r_CIKv`MYP5@4qIB5qd*XsezxMC-3_Ev}IOp}-47FnT2grnHG z#s(22z{0B!BH%9^^rOH2*$ccl@Q(!E7gtx;>23dZ((#Zg7pm=3I5^xK_qprbT}Map zdb%`z{Z2X$A_PyM&+E;1WF?xjx|O}Iq=8jK7VT}w8%9t6u;X-eu~AXwRs)rZxBYcW ztRGqu;iMA6r{;1;Mr>JZ9c^xI&QQ&fk(NFoP}k6qNQH+CDUQ*+F`4mlth4EdM~9?B zSdI%w$jAu0tT6YTMs`M$^Q-mx!NEaEeijzRYBfGh2%LvD--#dvYdy?yoR(U95wlu{7 z=kaK9`5^I&=gyzkZwO(<@DUOcw%mc8t7nM|i6mYw=rTlvG2SXZ=cRkvd8t`he!jj_ zT?Hu>74G-$CCQ`&Nn2UXlQYRM6PQE#l(=VXygt?5kQyN!E9^d8>o-;X>({R{SB;oI z(=#$+HaB~^x(?yvOA^)}h>3|wk&W%Jy653hspJc@_Z0=pi!6*MDfP8u$BxNqYg?Q> z#lgW*?M*7;x@K-_I$3DiI`%dELiknVZ!exd4-(VU)BF6ACP%xBak9v)^IHTreDIw- zQEw8Iq-0>>R&b$00QFUmtzVGNIcNz*Zrr$0zxej;+tJZcjNfZK`c}a>D_l8+Zmhe1(qB3$ibBhQb}a|XqHp4hMCTxx`)}=7#$nCG$|FzwmQ))N@i+m z+9bIAf?g$4ZE$9$T(25mLN-QlBhg=&Go7*izL1d6__yfGo{$q1m6Vc_YHlfQH>z;zO;IKb2Cd< zM7Pn$U-!hZpSn6bMbsf7rvyyo2EIiL*v(E|vFHvP8p1=4B)a7*`aHFWN4evCzoDTa zIyooD3g))Y_z(dB69a?L#t3^&{Xkn=+pAZ{vh<%Ae17?^tSlud$!@-1yPbrU+vnLc zIV(N+J9qAcaT@x*OPBB_ahR$m6?tLfIRDmj*A>#ul4{F&;?+6y&-6OT{0#+_Tbw#A z-lSG*OSVDX!PQ&a6-i&sI!PqH6*N6XMe|c)4{(Go>Y&fTCXwCph>N76qN1=I>N+|x z;o)f>DULP1CoL^4%{npz3O^Awzbx)|k;+humg$5|=tV#(>9Ri8hlfuj=aD2Ph2pX5 z7j&3W+t!Bjs#$EweU)2YvE;<#@SZvux$&G%!!)y0R1mNq;%nU9rZc9tc>#g~! z-AYs>kI42)P!hXtCCentmg&jO-uq2ix>Y*vCFu6_FJJDEqt8>0jEp4hRKM=+)oK-a zr=sKZ&f3@oHR{)qz=*M%9%H@~H&;|VucTlr=gw?em*&{1-DgHd!Q0f*!g+C$b-Nmk zt~8V9O+&&cX--6;bO@^^kI~oD46Hsbo~b%*_d=z9{LcN2jwon_P6ZTFfJCM~+vZr^I;v^kC4sd0?!QfnVqd$REB!;=YQcl9>M?J49n z#wr*Ab2!+|XsLw6d!U0NN+tUu1DDw8CQN3@+Dn|5%bupz^;gWVOboh=d?Dy76Y$(x z96;bJ#2)7d)UFnOQIog*cjj7!o09ooa;b@#hNI8-BuT_icT~jp-xwMk42-GV=`Iq5 zoq{R;fSz4ivfUgBD#E9$KiwXetxZ#jK?m7VWoBfwhF>k)zj3ZZ;JwA9sQGU))xw*_ z;|kH@>Cc&2RZzm)cC!x-cwp8aIk}bpGvaU~J#@xx zWKt<5;B%;DCJQEfH{J^}7?QG2uS|AM>0luQMCl;L)j;~`_w*SL zr#O8+9Dw|T1co?9l$cFz9*w`CMXofGb8wK%)I#){Lg(A==F8Fs`c#Sg?vrCX(+EzI zBVQi(X!~?C()DP?$&&*tI)c>$0}TK)VOGL8L|a-rdHvXwR6NLOgf!FN78Jnw<1tk* z5w8_T&*iwXS!F{6lGVfD7iO=>8fL$ zjR4lu`O-!Ul^HjLyul+7e!2IIK76S@GL9uspsY@hXYeg9KF7t~YG|kmsRoQjKs*IF z&KuS%8AYY2<#wHqk>6$Q*Wx+GbhY9+Vk@h4sFK>Y+qtu}6{5MS`zj*eym`}2#?H>m z*Yf$Z0ibMjfrUj+XoyFea?u?Hs7hvgii5`4)Hw@E`K4HOr|qQM3`?(j;V1r{slyqJ z&i`2ynR`AvNa`i$!t-)7$Q>u z?x@dRSi)2(=M+h$A0^z#;u3fMRmWuJx;EqS^F98m>TJ7_t4k>j|LePjQ&7jij`^Z1 zpS`tJ7;u4rRr{~uZ2uXK$?y0PjKlI*_&8(6KI>?=wMuuW=SN}dF5O4`o}V9NlafA7 zMEdS{{%Y-JXr~bfNt&bNw|jtQt>48}iexo~vgG*j3;G7TibN8-ySPuEl6?K@rt9vs zv&B^5BBU`t8J5~$_Ygyj4NBx~Bee6j+!1H~iV#FQ12Gt}Rp#Djr2}jalSAKZUYtt>wRj4)LjOqi&^}Xj>)q*BwrJ zb^WSopZuIc#OkMd1|Q>8X$J1^V{1yy-Bq?8O)I43PrVW6>NxH%vg2Bj2lbCSqoo&( z;4h!HHf2n#|Ac2;Vo^gJKRI3gX>*JeyMl&RZ+V$AIk|nG^27bnJc@`Ct|&88B``cW zvM|^Pg{kqk86KL3%a4wf>IpzjL7cR{A=yawTkJV-}%nz*+5m<5L)%>=FOWP znmG|+sngj<#IVP*E$MirU~h4btn}9m^tWV`6lQff*o{0->n#-V6l`vFO*zJ!YqL;? z5PR6NvRTGJ&W_~iu&)x#EXuI7r{ZBbW?<6u7L}vpKjj&}{yqM2UC?bvLq&h8s5@ru z#y)z_gantKWkHE~`~vR!yl0LMH-~(u_Q_{^CzFtBR}|1coO3yH9_)!9Oh~};7<+K` z_mu4+?Em&lVZjk2*%}x1{Ii}|*@h#P5P~Dg->MYGBWKWuPtjvXVQ0#seoc3GhmM0> zXRGJ2NR6Pt)wwrSVISUF)|)S1JHBnvm|u14)-6`G+_U`r{KCS*7cO}1!EpT;dM2iZ zvIPdIkUKv#%}=oq?+)wV7h`*+#rRImw(cdZkYwi}B#m&$0%Z{z7RG!)w*MtN*U?kI z{MAfeemNA_Ku;mX2FQR`n*u@Rp9BNe2gs&I{G8m}w$KG<>j&%F$DV1OeJ5?3Hxj4~ z_&@t75@q#c?z{>r{z*2=$;r7Eekh}dd9Ikz1w`Ly>o$Qe;v4zRX+etfqCZ?3FEw}p$ zK~YiE#>V5mzDH^gA*IO!b$fO4_>~*CyJ}en#7upBd;pr8=?s)Mah@bvs{Z&9f^2OG z&fOTGD!5Gz(PYPu=NUGHbYw7UFfh^6OGonu+Z;f~c>VH3KKgW!n7)2ORaI4VwD#S* zcO4ww?i@JfVp5@~>VW!<`uq1#e;BiFY^7Cmj1dhSPcMMQ$vH_qPvjq z{uuKHzDbwGkkV-qIKoxGN<>iLP zMrE`4=0xwnx%LdzzV^f9O!@`}i8+iKHRz(-8QU6-sq2SKGnL6N4t)Fe(&hjH4Zhd6 zeX`pD+?Rj%18$wxJKHxM9O6>sS5q@Hm0Doxr#@(GOrgDZ5*`HyEBE12B`^*tq{u|K z=!$V@WU9Cpb`QKV;u^|g5*PV?+(KUq0VsrUn$KEI0qTz5CODK^!xYXQ%mJp*K$w!dJ0Jc;LE!QA75UcI_v zSbt)FYo_@W6ckYU6Mr9g(>jS+nY_R7^M=HHe}(J&2M6le(OmRoRg1M$Qx%lePl;WZ z8+qHkZu?Qb>FwdL9l5jZD)(B<+&oiIzFY1$(m!ww;>ERVzrKcF#qc`AFd8Fy{b>YL z+*6gY&uCS62K5e2MoSBIvdW(^NJvXdhq7tE5SQE5ZW3DWg*vNUJ3}SQ_r(kIyY$(a znQlO7mi65roAN$e&#NyeBt%0^OdO>YdQ`5I^Y-i7&Wo1Rx`Wx1~S$&d^8@ZwAd&D>~2tS@EZ@qaE=`j zM|&FWD4Q49`@)tK4E;5>>f0MU@3}Ie#%Jr@_=;ehVTBjQ^jPjdGMkM=N1f*jSJpIb zxj_4$4jJ!B%ZNPVbZ-70#N83Hj0XBT3xdnLos4xZa^y~g+r*Hu&M65dxfe26X`RIu z4DtRu4LaH%{C9;MEqsTc70ww1T0CX0UC{IXnIDC#BZPg^!(&~bZl9MgU;6tuefc71+ji?UK<{Xpm|B@VRxd&cb&5@^#AS8LeRFMc zJmWcYS?S9^zf?yl*&PxYtFiRFUWE_A8*;@>x(We=3PjL6l3Dku!N30hSlIV>8b(k& zvi>Xb@YV;WbpKp}WY1*xwu8vONI=>ash@JGR*rztEXrMB_*!=V_$(mcnh>iU*EQ4)n;F)u7|p;CMydOE$}oT z#7H#;K*3%?Op;Md%}Pd2{@ITzC6@fuDaaK(`?s=0+X4+mE96p+naW_2jbUMD7jc~H z^``SUed2^&F`Deyv4!T_bZLBPX=#D2YQGc9sqLu8j~}NgrnLMX@O{M~3LU4x)lS{c zZq5HY7yNqci#}afSBJSte)@EY!Do`y2lydDp`pAsLt+GgO``>5!>+`al&ICMudE2V zV+uX0fjO(FsF+MdAWUoJyELDSjEut%JNm=I!azleg$FMzwCgT3g<0Sz?LJ3(OBndK zO3c({XQ!X<4@gM8bWy`=nCvOGa&d9Fb*lzKuyIq&_Rw?oOB=rlq}#r|cNpDIAm^r= zmzUeLHQ8@xI|;wxG*GDT-o2B=w7iiLa#{JVK)$ir?>c|c@$n0w0Jbr!7;o-eYWMp3 z`rf|2;6gt?zlRZDLzn2T+H?aa(f=~M}2_Q~z1Uhf3#kv_k-(m-$9$i{eVtbk*7NW?D;=G2K3ZSEmZ zLNyPEvg`IkpgDP2rB{=$zrR0{*NW}pMOdectgOnhw15CiTN$@6kdu)mONZa2y9Ky4 zY^@Q6g4yn@aCID|;s;9d{J}EEd6nq1JUl#HTuK{DGC&?=lM;M@A%R!d7u zK~XUyJ^fo0-&I(!)ybjDz;U#*^9c&NO?PDdGg7f%gWhzo4GtZD4SQ^DqIsCq^A{re z1#1@-6#;XzxV#Lh;^^VScci2gmkjjvOYJ5Tc8O_P(Jk+7*XH`Br>8GBE-Wm(8g6$x zPRYc?wCa@tbcu&8@!l$Fa+{uH(R^$EHH<@2gKP{Vx;pwK^r&NUr&9m#*mmk+ssy@I z{+?}e7!dRK!QZSTT|gpyKYSw?adRGh@cZ`YFD zl9wCOeU$G?j_Z|``U&T;#OF^KtL)Va2O~a9IS4P=^s28-$->YY!%++&dV+NtU@E~_T%&C@t9Z=&N+cm65zmC+ ze&93@b58>047C4nIU7p_a<%kkYMnd!#s$nlZcxprIweM7jj)IEH3tc@uF?g zPSQh1r~i`=Ib>XZ$2md0kB^6j^heb%)mN94lzgW1?Ct1S%lk~*0KoCiorT54yu3U@ zBBHN#bsX|fZpdqJ??sumkwJx^|8V@}m!Qy!jET{am$zA+YVRsAuKG3f+&+|_o`#mz z1G8e9=LhinC5^yKD&9E&74_9Au^!!aC-Gr_|GD)!1#%IQD%fib?vpKVARe{}m1Jgq z?d$7HjoA7%>@QhbUam!Z>g35ZQVRwxAzBetiG_9iSQ@R%0V0coLa9M;jYZgR;`o`5^|v?9Sgt6{cG|ID|;AI%Vw- zOIN3S$dzNowYH?Rb}?tGk6Vs|%up(ygyl!!_^$-wz>q52(~~N%_n6p@EXLADbpGY#w#J`-((s5XaZA`! z%ZDhwUo~n+l625t5TOY#M%I#UrQy8qNr%17+2V#yNMK8#S0~xZZLpsCJsQ4tu^Z}Q zUF$u&1sT&O`>%$#i``OPFwmJ=B z)6TB&>m8^IIWK}Z((EUX9msXmj|db>MR>X8(V?RuEb8Zj6l`smniFqN)}~5KE0Qlx zL$KA=rFib@W=KLOJTp9eze%==;N!=SA?&)q{=w}rXtMPd-GqdMPcru`{ju;Wjw+Ec zOB!Xz9~~5==;5*Bo?iFroT!3=LPY-PKUxP=7U%Yg*|ObQyQ#J`C^l`fQC>aAurTGV(L4YE7zIL+`nKPLH_VqW z%Os{sB)&ejSK^8JjDICrolT4Hx z$&W(QpujKRzCoQ9+N>t*j#+`4Hl1{Mc-Z)BSUfevoQe5)D>yHwPct(x6F9lJ#0a~W zKJ?o#jHz#kyk?W&;0r4&Yc!8VcNE`!dN}goWMd5SYHCo_eKza#*zl}@vOJpGEJU6=K;N5$CgSyL zdee=|rf<&k8EySq1pQVYr^V3g(WBSHHMI2&4b!I#3ps8MRJymewqCw;DO;z4A9zp- zUAH_-DkO5U)n2f_b^Eqd0l-Gbv9D)F2U}%DEJ}}fDU)YWS`ECD7w-njLy#WbqQ*Hm zI_fYX_ede`6fD$|b(+sYH>hqqm}&g*e+5&o#56RzuQmx@@B!FzO+divNA8Ul2x~hN ziT*FreU`NTRM#1wPn1N}(PfI(q8PSbgy2qL=b?zLin@_UV?D&9#Ts)zzVPEhsKlUCIIhHQlGVlv3@< zheZwU{>tTtFGE8q(6|e)G7wkJ`t*2DrMmz-dohEQk&3GE#f^9G(#V-es+LWyy$Q*0 zdSEX0-F+jLVI4hZy}RQv-Pxpie`uKE(7|7!p`NVXZ{F09P`0Vo&vduzbr*8f)DR&2 z7ImB0HIFSP0P_v?^=;C+x&x0MJt}QFLP8=VDe1CQr|TPB1;FpzxpR#AyRaVsm?|nL zlse95qrF4u#Kgo>i*qN&&A4lOdR&*s8W|WExVgCjL{2TP$1!Rs13$o%RNLcd=@rds z2smfqN`{1ZZEdBy`$*0&F%J(Qj=e(IqTkE}Zn0N7$+`{I2YZ6k=Ie8InJe-yZ;XM! zS61e@xzyk!K_^_Uo;3dPNz-dY;#-^doCk#`aCxRcPch}CIK;K9^5i#MnwFbHa*XP$ z|ITUY1C!c$`cEPwdm^~a=vlX=3x~6c`&%xlq_nP1wvPY&;1vjz!aAa;>h64B!t=oZ z2+fMjX+KRtL1B{!fdfQk>8NXq9&KV7Ma8jrP`fZvT#XK1WLIq7ye{bjQZmmdKa=(bw%*?_~Jc>#6ZSmGZ6 zFjW74C;)n#)%b}BfI&}Os0zTA1&atE*~Q7}%YpSoSOgII?Iv4p2}x(v?m^z4F#z_o zO0HHUFNvF}l7l+X(V>!07!nfVD7;pr;BfN9%r+>Rd;%GC1e$%p5noPG@XFRpOS(wy z=0-%!YmA{TAz?HB+qj3?Tl20k4g(GCssNs4(N&yvuhhD1%S8W*x1}s8ajWS*eyB3w z?Rj0?z@F6mF5t#6WF{sUg#DSOtk)K1Tws;kv2nD|*_>Dg?NnfL3J4_>?W#pry;c>y7p5?a2c8|!o5 z=sdR^+aB+g%5fL3t#wmQR-0N5#tQNQ!z@$*=dL4Z6={ zA3xqw>1wa4(goU`zi%)zGYgzW{lUK#0q>m3?xbG)>8UBZv9FeY?(Q=~UpFK%Stl^_ z)T3k9|KiIFKo5i@Gm3kjo_+t2IN^f7puU;afa#}a@3%Jx6qi7wxceTb%3-$K#i=Ay zXFK$1hR+Y_flA>7?tiF$xO-JmIw8b#&5ZsQty+3@f;n440Zuc9T~Y5?Em4#j_W^+Z z5wE3UJtKSB?yLrq3gD;k`wLdu85lMG!@O4>*h6^`G=^Dg z-0pk=+c%XEDRcklN+NtJlu{<~Xay^^irtdSwpW}M$O8g~U(yYLTW*$A#4X>fvqx5_ zy~?!hFW)1xi{08x469}c3H7gQSPTT{Da8B*7I-82Y9J18A7mL$I&7xoWluA>We7_6X;Rb2~i!S_f z(YABFpUHHqz@xBn#)W3#fJ%+4n}C0a!O~Kj0q1$>Jwpp+QUTi$FVEU; zsZAOhfs9?flBAJ1l?^Wfk$^z5M5{~S7@X?+7g;gCuAt3Gs=F>tO6=3KA3t_ah?Y7p z3#{gT|NbyMd_IG;DRwtWHRtx?M&rtl>hN;Ro=x1cmx4clhP&C=B!7 zz70gZY7I!aQ`+CkAs;%4yGn(G)xCY$mc(J@rCn&^0{WP|e1SA|@>tl;^^|PSJ2N?s zp*QSplL)z+2i{t0ul+FSPrs^N{>vG!Nu%+%4))JoY})S4_dB|*SPQ{C&Gki#?EIQv ztzCK}WgK2Y?6B~YNkLI{(o;BL_uv@eHUx z;7Yhn4|4OlyEt}clhj}OGFTm{tsVU>s>^~uL{-U+{Pd4;Sx-vOtzX}$csXGnM;ofi zezptT0ly)Jh?1Gdkfo_eK;kCIVUxyHmcJBPUak$9&|_y~;9}X@tZV0o z`1?0M0$+p|%I2kNptiLma{*?16GDm4GmxKm-6z2jY}! z|DYom{ik=rwFu*2wvybNVC!6oC&hH+-I#^SM`la}_;Oejr=J^@(>Ey3G{^ZRA9W0xgN=j`|`6MJJ z0+_$b$+=%Pg)spc+0qmW6_K7*o_a3VmjCm{!^uoggkh#fVV=7C6n)#;xAqXt!y9K!>`K2%`)z#IFlSzNdJz%)9L{U@3uTW`+9DAOMTSdyG+_$x@Z))OY z;)WTsad5~^O~nrCfeH{rWRi4tuPu6)P_IM6!me5NmW-`tZL%nSI4c4UNpPtB{=h;` z#4z?F-%x5AYKK6Ckd%oAM239abyV9GMxRfg#5_Gc1q908zkd%>+BP1`%6FHP?3oBC z#%lej+A9_p7q1$9y&_*J4D3@*`I7JR=TZRc=+BY1U+T!#o`#xtV&WPwj8wdqr@1QM zzU8@eDIK8w+S*z`KmdQVOkI`p^4QkKvXG!)lJvz7D?W$!?29UN=#v@kv{`GN7UFoi zuWA*7K!3HevcgU>sDGS$=0n2wP{EQMIg;*rT~uGs-hTCayl7BhAU!Jte?G?g&f7zd zCMG6>G+-0}A75Mu`v%3_S)W_aPn~I_A}7D<_Gw7|!)BXG&Cp{&arC-#Y*zS>K)VBFZ&BVGGeO~!XT0C;#u3uIpCybKR_*g($2jS{5 z$eGSAE`+1n_;yw`VPIbIQ$t*36EoAbuBHkHC+8^gfi>s@wp!;gZ`TNWEjUFLqLey;F zvRC6(@-$|f(N)bxTq7WzlG>4BxovC^LV^dlY|(-|*F8gJQl`OQ33VLd4Z>LbCUnZQzkjhJ_Kokh+sS9^+~WYQa>ta)!z+ zadDn2WU4AEm0L>t3yTmq`Dgs5wG z?N?)-F+T&=1*Fo+y`7TS=yVLNZ)d~6%NrW2g$D$PhF?UX4jPRHain#khHZ~7cPwLR z4h(1=^qx8ZOYxTkiD!|a5ESyA7b{Jj?j94-cAWTrj*X238~@;T{~lhDI25-^GJC4X z<`zPCL6CgP;|J&yet3`njOKTM!bS<~<)26}_}D;?WP|S&$TbfZL}%ixu-O5kKq_J1 zbA)6RbQ>H8sqy~2r-J(^WuDjX5Z1`29eOYN>*?KCknGsLT@&BG^!l=NCs21au zw?Eeh2$(+=yvBacVK4LG+(nzfbz7fHIs>~8F{l$*79RoHZ-HQn;S&i2ZA626L1bYc zPXBK9?&xkey*f}a|6ENvsb*P6MR6FW0XFYlmfinSpDcx{8B!V7s+tN}v1P}JIcbq= z`fJ}vKA5s4=8^CJwn+^X%KsoyPC%gsq51okfx5O>SGzrz!%bD}$36(*46*-5nEahE z4Z`z?QVFj_AHXpyIMQ^_iFg?ybnV-qt9$fljB?oIRFp!^TS%R zsN}@;aAsYB}Q=S!Gb~Uj#TR9o;=y7x0dSgTn@>A~+i5 zVmL1hK~#x&2@a9=UEh%}t}NjVBNiQycjdWj_hu<;OeQkF4ZIF!T^0k{$q$j*gDLKKGU+ zuw&DpwN&p&OH=b&MtM{|dh^B*vL(oX3TkRjyWR2jFEpYP6Ad7CdGREbS=<8_5x5B0 zN6&a)7+yzt7`CO#d+sd9GTc>FRz7*+L^(9=;SrFS_g(Ok!(mxoLU4iH^%e?zuwHo<7VOec+hY>4papTfSLYvmnCy&5_wZ`_~2p#bL95 zb0YV$2o2n>IFi zJNe$2PauK%-h+t*St(Pytivv2C8Nw?HbNt`qsuh<$&G^5Lk%W;o9pxQ165T@ouGVX zX%bOLTaTrapKNt=;E-ElY14&KFq|$ zwes=FQRp2677mRQvY@4-W670Daq{H$YL2HnW-=u>s?vQkIVD9(>XWL`o)%5n{^*TV zC_Vwf^SLjrKsM`@Do@>)l9cRsTW(~Eb!%Bch8+@N5a&c%`MCJgqY9TOxJ~AN}+&#x^(}f*5wdUU$cf5 z9OG|MNb3jakh(6DZ*p_57^=dWn3$w~G9|1F4-cQBcu+)t7o2g4;(n#p1kMQ&>~A@s zSs{@m-p!2h$`v_h=5vDF+(qf>lHFLn{SH7nWeUrIjRVC$f8OP*{ize=(k>RpW!h3U zUuiZpQ~2#XGqcI!Pz~T*9R@zN#?DLQ>s7nkpz~b5eEAkiOIzD7=Gie?+U#EXhW&|J zI;!iw+CEYj)Ysb!!vjB*abtwCfikHqgjtI5J2aK00)Qsu8~gSXi`#=;NmB8z{A^{>$0-4 zye{X#jIlLB`8yt7%{A}J=P;-(tj%7y?s-o|hun)UoqeAu#ou2ILaw5S)3@5_i>6cO zjWU@VE<(V(jzS{Q^{W{^`bgYstKe1VAhZtukTBXAv9u1-Q;Ym;#hX&?Hd5o6@&du{F8y293v zCy5|$lpe`o{GN2em|9V|RbB<9FylQx#e*5~Z4AC8a&_-IFg!Y!-vcf4UQfynhl5OW zitgXP0ci4*e){^hA=y7;@DH7t;mVxjB>ZQ{5lbH%HarLP?85gy7>z4!jI)>b55T3h zoCs2YDflN$Cm3v9*`|U^@{({_J#&m#RNJvSo@~1S8 zLhn{XD0|8}E&_c3X>=r^6ZV_lr67J-^_8|y_#8g}MBzBiX#}v#ktY}E5*|GoDd@ZuFX}zjzj*9FRGVleT5!bQ zxbg7uV9j$1fQ>AzMl*vJglfp6(@9iAQGBeE~{>ch1J_h#`L=+GUli3n}{cC@oU(V1$ zNsTB`zs#k%UGO(Z@(T!P5?3qaKa7|#u(AU3ZL!oM|mcePSkm2;T?xH77zq7ssHOXW2sA_jJ( z>ISH^@yhwau9cvAN^~YLFi;-@LHT9|B#f8Td_W;M-M)PrT2i6wK}`w;&bFGvlYb7y znQ_n?p(0jlsjRG=q7)USf8W!y34vKsgXQ)v&kwMYnhI&}?vP&8`xogh0O$maY`|SB zyY^3yBbc`!WwEibDJ1$Ji?Cf3n~-pxTQ`IHkbDAIaqhekmmOJ)?zx)+Rxko62cT|v zJUOF^PeNS+K5?jVdrNGFXSxd5D^>s_hUH)0GlUHlJ|6k{ReJU4dkwNHo;zFeiQ?KN zUV`$C6g{T}95Us-78dyLwmfO62h25q7r}7^ zU+l{w~s=NX79gcU+>Yof})jwYP>$NBr%Eo~yZ zZKMF&<*(d*fK2RjV5j7n`AVQ~5^M_?-epQw=$9PzVT#EuDw-Xr@^o=2Z6>d4Y#e3k zgbd(bf0UhKPC6&89N4(lE-%Bbd_#lU7p8vfGkDtp>dvTN@h;aIq2EjkMNMD zW(wUU*-Fg?=$tipwr`yP58SpLbRE~$8j(PfYRgW9Fa7}9WWEro98r?ckQN<%Q|V%t z-lmeTfeEw&LW(N*okX6FEh2qv&;>{Ew2=j~GHpZ(ech1wkQ73i2F^87$$V;XuC5OF9&bMs2pJ5$V_F z-yiJNAF4`9S`{w#fMemCRuWvo+Uy5O0@Vsl0v-~Z1n0=&_1sZS`?+tLzWL>XwnAOzB$1n|tWUB73n?bFp;>P!HEB%2=U?RL4Ojl7WZ-C&6$E5UicE{ zuUuJxSd=WqWAflcK2XOH)FUDyT3QtBbLeGa1k;kaFlmshNv4R>{B}7_OZR? zbIL`N0lP{`d5LVVrGd|~_Zox9KTQqm5#>lf1nTsDKZGqUxP-XPI=(O=F4(HRGM(ee;k?9cW#!E7S*BgqLqNKeu>X7NPOi(-g||AXX1legTSh~Q zRxPNg%E8HnJlaFX`NYng0gv^*o^4E@gLP`Y$#R3f&y!oM(+pTV0k9lJ*FHd^zEWzc zXW#HKzvH7q09lu%y83pm_2BHS=N`_fTW*&CX=3X3Wrl)#(B8GA z72*ITXd$DhcHG_LR#i*&>)RORVhc?(!#8ZCQ%zAwY}--)1~<96 zi_6AHU9q`^m8M3&(d_<1o@zF<>fag-?ZB*6^c7up0D%mF%-5e!<{vs#Y+x|GL&@^{ zrbs^saxUW%$1?RTEsUKIDAyI`KYtb-hbScoYDU80kA9eI6k=!Q4|a8p+HGtRLWltd(0*va)pCXeH=1f?^WB zXhvik$eyWeU`n9g<4}N@{BNCSDwqVJ)iy==iUnvK1E;#jgT{Cp;|Ho9(l&4f*8nX+sK#p?xn@cfmIU{EpR}KnI+mpkVR=l?^<|l7&Hq zVk(5SjHhtG-1nDF54_Gr!emwOr3XjnkCBlf4PZrXbyd~`5-bWk=TdYk#CzCTv`e|J zT~jkpOijJRLEqYN@;|Faa`R96p^kWV{6d?z!jmXUWVVr$6q3qFMiX54=`Obx^cehi zzX~IsQIZhK8AxNnRe?O(MZ)ln6jS^tXd1+b>&9e^(TR9yi42HsKGCUgY1j6-+2;s zb9Afb1_#gifb0iY)Og|Nhrkf(x*Xmg>}t8!IaRZ@)UDd_jpWqT(~=lK`tvGULKP0g zy?>hOBenuCeUd%lG%HrovbuAXa=If3ILj~Gn6)0O)vK_0w<1qpM2L3e!0(GpuHbmG zR#0dOlHF0L+(MyH#Ds)tu3_Mf+o}GSuf6bFA@=#isDi=<|Ir6X@Sj~@Tudv|()+IA ze*XOVwlcKz39OVzWg#xA^Qvt7I?@5W6ydqv=ZL=nU&{hDtzOda$bfF6-$U3RE%x=M zZb;ZA@7`sp7l^lTiuhWWuXbqKKeu;qP!TQqp9oqUeLE8qCV3fq0#^Kh2Ht;AurTS+ zIQPQa`}WH`PY!T>B0Y8ci<2de*u`u98{c-u92iW$468jqS7Bi!O3O z1Ts*mVS>YxX|uB=5gi8!I$#oy1Ja?sDH~af{a!l+6P&YrxW-_K?XufTs|_uuA=VPN9@y?XBq$(4kDWS(3M*>G$EB^1hDc5g;X)K|SV#r@}Zv@OEjfxK+T zup%P)W1D0l& z(fZ)nqn_nURd-&_1Q)A=i2-=<(S7mddh5CQRHQHPz2~QoySHrbu}Miw_ZC~}LTfqu zabzq(r&1X2)_ww0$BrufZA8psv?@R@_^oHfO@8_E<;Qx?otM)0mT0BM0D^MroClf> zW)H+8q;CM{dWIair5E@Ady(ody(%f*S~s|>ngJimA@@Y?G-Moq4e9rf5~72ZmkPY4 zh@eu!h?_w#D#&7|k{C@&x~7+P1`hwPDqZ_^o~h}$!D7=@-+yNkjrT|t!~!@nc0FCkaaEMV*)0#qzd!Nrh!68w zg7T!q&$u;Y#|&SQOvNSBMDED&n06}~G}b9mqMG^1X==|DE$Lnwi@q++YD^UQA@ywI z86~!hVfRuG`r0arePa(TFp6%}Ia+SL>K?ngkhz`Ju%1;^Ik4`!-fdcf%+ zpe0I?0PUUK&mkqoCMqN(1gl&qtV?AH*nohjr@?*)YgKWva&dxI`e1KT1^NdJyl$=wBrgJv+pfxloX)pcE|IexjeEMR zq-4G~*QqsIMb&2AZ-Hq+ebCj_L?Qj>1wX>=Ep zLq~$eU0vYpMMOez2@pL0jp|C&WGqgap{YBvW6htgt54ANJ9HQDBo|lFk1xabkDAXq zOdh`c{nb z?h+TD^*4G<1d(khC@j2no`<3)RZjwbr;#WQlYME7 z_fHJ$f!6}wDPB9RW6cHqX{172ok1|IoiJK7X`8Q;axGRT66&T$NI zhtXbRV3i#mWV8VUR;dK65_s=Ce*mz;7hv(erBBx-8I~bWV|a8_2h^Je?_tF39R_o& zt$SS6S1aDd65EikM- zeSCJU+QmKAQ_lx))a3qz#_65Ab}?=hZK$iWdG}yD3?qh2=>MQrUv|Oo^t#8~3JTe( z);FLQ30u$Z;ftIS4>rMg9-A8^V3mh=BNt1;@pz4Xgl4{{-^@v0Z3!$oRoyjVeN|M~ z*~e`RPe!b$d5`0K-IzV1UMJ-=PZ%W$DiMzk<<6B(&H*OPH1*~E-|w{ww!7tzc%?C9;F_2SGC!I`wniAmL28n zLe~S9*ix=!LNF7%E`nXwW@})tkdz9oFGvc}j`Dt%ifajG3;Qq6$6f9%M_s&_?_BYi zxLa|LesoT9K}c-`bQ1IW+e|mib{$$tXlDp%rUd*edT9@`XJ`~N8gXkxtFYvt;QAX~ zgEPsY(O;yA82~)280*;Fm6qhOE}4Jfu+q%~OY~R?mT>>0rAy{6`3q)EzQB7GbjL!= z@Jm9ggNc0GfcWdbf7GG6zJy(PikqY0?OQ2m#Wbx=Zy?FS{CWONl8|y zO{jWD(SHFaGBDUh+&T3ZiGB3;PX%|N0T>$@fi`30+c$zBh6HZl|9Ouhgs*sLw>tu_ z-?~Nc9$wD@9wCxN?8nDzSFT*i47`b?fGo_#i4MFKMz;O;f{oJmw={wQEG68s^P&lJ z6>1_dQZ&MZ))(J;{vhR|sVQPzF4%A@-GcIA^lNqD>(}QHk^uoH)rBG>^99O``BWNI z?#eDG-;9Et>k!vjSZIVzpSpQ;a8UTtr|UWnzD?nb#ZE{Bg!8fYb!)53r+jx{4Nc8k zyKlnXw1X8B1;0fEh=7?-Y6gg_DC@86L9I#;4{CC&8IC1^R@;8;ri?W-dd+YEHZEH$ zxhi}Ifeu5Urk3;nvqdQ%hmvND?y9S%wuzbfDe0_i1w8hKGzTT6N_0V@)S5lv`tP6o ziK$n`{M`p748)+-u^ZrN--pKjvh*QhBe@h)V+s)nqHKoWA7f(Wz&(G7a&L73N)74} zMtoqdwUawW1_ESER5Xt+KCKMYp1@M@$YGd+ZYFoQO1_)24)h^3BL*{Qs3C?_9f}ch z6W}WkF;ul5bM0;5HxT3ImZ#(;8nKWoKYjdYG5n=+0<4<7o!!H-FJJ&w%>TMzYm%zc zG{>sl^}u>KTq9{N4g^Lc-LeA^beY+_2n4pvxnW~)3SHyzwU36o+0$EuFJE|beI>d?fVmJI)SjXtl!gE3;*&d60b(=5rZ2j5 za(p~$Y}01Wr9gKJJvh#E7FZ^Gyhcr;V{X+q#F2Ol0Sw;W-ddS9ad44nSHF34G}C>j zW0Ea}p+>R|;tDG(E86C{Js^2(P6A??J-z2Pb6;z==3cVnHfzDH*Mb0N{hWe3w{1HM z=@KzJEIZ_2jfK^bW$oIKeSdj)Fs)uq6XoGL%*lCd`j_XAe}&%*)63O$S>s8bjm>t$ zmC|Lx*9YVLU%is(2oL*)u*mLis<9$QcgC|jckV>5wia}G#e-2&$8TR3_9uBQi4rgj zXoGDF#s>-{9`(d96nf=ev9Z2AeMVqMZ!vKoUYwbX(nip6+fk&w4nEkoc(1ehUBRwm z2F2dU5NX1Av9|}afQU~?_0yV~nYjCTK{oU`;bCE$2s$lRJqg?e0Dc(ALw)7TAr&r1 z#!>wlPoG_W1f7;-#OXNanq)kj?h?@GylT&10cBIpIel54iYVzEaMhcM z+eQY4`_ZCe%s_b=nfGF)CN|>88$g<8YDx;bfW8cs8r`@IJzLxE`h_KF!&xNn64_UG zcBjfh{!w1kG7&QI*Bh%F?osv8=;&WrEMJmu>G+IsdL}!v=tO<+EYIMLs$-1{>+;=z zQM^F4|S(#?LWv20AI!TaV)#g_3 zfaVi68P^A%JF3M`D$SMHxG`7jUNwo zs{9nSW$vc*{MB-#v-LAazV+2{Ca1}29NUYCACwttB;SFZ@^ zHus4o_Jv|SpZjaR04e}BckGn--v9CQ_sh?#Cw^^8F#)rR#;Dx@We+6plfz#>fBd)~ z6e!}u3{t?TJbbu`5cvqEK_tyZqpkd^e`*d^`T+f&m}T5lmv~N zxK-!fK>=P~GrSFe&>bvDB@0H@lXA{nr|=PpUhp25Aw7(1ojTMN+*yCh$h?b5$k}@X zWtzkENPX>eg4&BuH#X|*Z%C3|ww!UZH^^^wb$e*A=GXL7+LZ2zndJgWptxDs-Z$s^ zN~OMT4{ywi&E32>1L^GK!w7ttsG`%wckh<=CnA#J@bwu-4QFUz;D52Fug{rSp>=%G zpuJPTjvoBpjY-%q?w7Tn`8<3}`-P-{S5{3#=Gj+9rl+NBEqDM_L^YQ1=OckB^@%gReA!7+cLqllS zp!`Y<01cNB(3o)=J<7rdYCRVaOM-6WA8+9_g_KK0$D`Q&D8dRM=@L%QIo9u&%Q#PZD4#2 zpW#YipyqtZU$dT_Jx+y&EWQqP8jR9ZjvxD5UC-YySECHCNy-WM7CY}rdg0_(OGp{L z-_YrZXEQ@HgV;2=ap;#e!UY5}xW7NguxxYqbCAL5U5|S%E%UE*`IH2b3@0jJ+a$j< zqAkakTZmAxrluyapmp`pMbRoTkRxCfEx4DgOXL6f&MfmP^D82^QFQl+{dm&<=C>2< zi=X|VZ|+nguRvnFv`0qr!E(06pObvg>)P*{u9S{pXYYYpH7TV)eo;z(^aa?YpL4~hn2?-Y&!?C2`H6i$puwpBOP&X48PA5H`y{E zkoRWqK;}Rqt-6+1+pG!3;+D$LN4sv$M_^qbp%U16cJ}O%VLriz*`nQzCB8n496HV6 z@<_rI40bCxv1#7k`sBv3%qLIGW~PKU`eoc_@9FJ*&az-5NWS_$J}OFHA$B!Wf|Ye# zNqMJa<-C32iFH^oeDOc*To`>9%maOgrk6P5JOpUDH^tA9%S&W_8mn&38LNn;Ok!f` z!qxX{Eu$XQjeurgNf^fy^AFtAs*pU>)V*+U*4K}v3a&tL_#Q&`{$~O6&9nPs^dqRi zBV}Vw(lFap&YYnK4Otqc=`{5OvEkLjgl8+v;5s2!!O(^v=44ghmwU>!gOK;SFs z=)`+>b#_WgOGDe*^b>Gtlsj&pj?gm1L0}ZE*|Zn$ax?6u5Q8ai55OhJJ_R98$QPO_itMK7y!rNw5 zmTL|uvs@7tk;Dfs3^>>-b8uOM(Cq|h|NXu!|1L@6b1hNxK{KjO9!GwZ_c&57?bVUE zo|BxCLXLm)iYJ*Ia8T1@f}5?z5wYlfVkvy6LKfkb@gF}P{`2B%e>c6FhH|d>z<~o` z)1UzIw#27NqUu?&g}&aaP*)%GJ#`>pQ;p;Fgms@wG9TzFga>%r`+NEso&mc+fb(Ip zHTe}436KWNtU~d~cIe_BF0ORUZelZc9QN8Pj$|Qhnc=;!F)NVqg%4=Un@*V0#O_-`MTl@ z+gug+%7PI#=b**0;TDj*1ltx&iGmdxx0*%GL5E>7r)7_u5`i{)1*@=95vz2QiY^}? zpDk+al9J40WY)u$`bgNZN);6mX+{)lVj}axqg;N!>z-ii^Tfo7=>b>zD!!ZUPV<=b zR~}PQEEO^F@n(N**gPMXvti-tckf4}rq)4c=;1mQI^{F28czbQ)(Q<^P)w2b)Sig% znHq46z(Iy3o&a$8g&!j{l!;yVwBc5#Z?7RK{9>RrVgM)Pd|PnbKs8Py+_$8(yEbOU znk|s+h@5?OB{}(EQU2d+RbylHN*?WuWW``H*xq31+{4FrMo@9F>koJl?-6BroC#?HR8J!$VR-dly&VWQ1F$14Gw*cE{) z-@y%m3036c{oBguU~HsyJ1HMff?7reJ$m& zH3;G+q{Tuw1-VaDAQ4FTqmXlD10EN&{GA;MrSjcmRTj8QN2K1M_(-M>!e+2%NRMeS5%{Nb z)HD5muZ*gKf|)t82O!TcjdpJ}OTpQJ zl&;HUGktT6`#ZOZB~cA`RVk(M=7Y)t*{Q;CUw$5;6hniSGd{?*j|KA-F(Un$+3qnHk?#ewuEK^VyUFK>^$GVT|g_|8JoSRQP925XH%%29oMup(pmM)6hSD#R{%f74 z?r8#w-PL2#{NbY+EDT;W8ZfE?!qhysuI@Ju(Yc3~w;c(Ii$mB|6oRW}h(2Z8u!CV3 z|LI>;T)c{bAvq}t_pf{O=+yu{z$-4(J-Q;W9@*PxgMtU-n&$lxGBn5!z77oVIr=V# zM`2R}$Ht9BjN(Z61vfM)K$?ii?XIhoJ$Ewb+fV;fCz$5h{Z0(7?{`E5;sO zQ1)g1E_Wa}Y5`y^8`*1Ybxr)Vu998+e*O`STIe59G=L)dg)i&dQ9wt4!7)Im1L;#Y zw_0snOO~}+DL&<-rEaryu5yu}Y(DPfl!YA>`H~1}iksfc_hJ1B^XA-I0g3pg9M{CP4X`fpY%)*uv+sE{n`e+}^Yl*|dF^)%P`a}$fil7`Td)`Yzio`7hH zS3m*=xV5FTtuIL0Wy6eC~24gNpNx;el6T7|BaBkIMKE8o9)o&rwgEL$v{JlfD(NB;zk4)I`fvZQ!B7hB4B{&^O#Vw(5 znOL9@TkM;@K&WhOZJl}A?b!vzTdtN&8a3MH`p^Qx(Kak$YdaS4*rB_SV|IjNHv}ih z^m#M4xv-Osk5dI;zhB@at`376m|3;Bphw)xd}!YP8`=2j5qcnX5IS+dJnJ@Y3>LBN z2Zf>}9<&bc3uDy5g?iXMDgZNXi-?VaKj^z`0i+ezf)n?TijQ}h8Slbu1JSwtKzH_v zY)jV0GjSRnFh{hv9|RB#^VOaX8075lS z*i;mKv900Yr}Bylzq>;7I}lgLwdIU*6bcRv%Ch_Z=|j2|zFu|llH#l^D<8RC;UEf% zw@d%K3TIB$xu?NkV=$`qHK-Y~ew^kbUjl`&w`6>;J|I(Mz*Wu6)2aSG3`rkar zt6tYt+c{mY;4cLDzW%{nPjADw&0*i}31weV)|8JumvMVra^(m~ zV5IftAPqRlG!^G6#gJxZWP&ca6N)0&R+Ld>THzdQ7qKg}dr3qGa~4-dyiMJbty5|IdS8~$JpYzcX^ zs?i3Xlfg5cY_3A4!%n~=)b1Yy#m3_~4YG}YuI-biDAcwOc9eJ->vy`X*xCTV94b0} z9l0);v9VkS29~S&&;ehb-(&!d=XreKKAOb+e3%3)!mtf(2P8I8*YkNv6-i3ET9%K zI7aIIy8pF*TTeHiWXayEAc^_Kk)cRXL~8cAW89RpzAI{pSF4k?&gqV)Y~b zWV3gtoM`dnlok~gg<`rtch*Hkg3U_EwBPBkw?+IefX+xAhzv%S(%!`pbT=RiL4hbJ ziMadHl2u;&;R|Ins3aPpim&M4&k@h-cJMR3r57IBBQ6(eXf6P#G32YoX$9XU@u z(Vv#)a88~HKum~oW_t+Cn#f!nS^d^)Z9xoVmN-U=m7U#rZbJr?9x%&5y;zE+0)qz& z{d#xrzZ4$dyD*bUdV_Siusj)&TIOr=TC+7ietzQLUM&lD9`!GE1waqgn2JkVDF3bM z)N&&&tw&g6ChSstp#YkwSbYE^=$FVMMOb$QVdoKboyjW25__VvgFP-$Mxe~9;z<4H zSKG}32cd>LhW+Ggd^V`;8m-_bj z*?sw#8c2hz=S|os-tNE`toXGSr11PpDA6mU)(HhO)X#0J4%>Vz#gtg)_I}<4>?I`O z*!kKpgWV9H%|H8zFSF`RS8XX!)?H^Ju0lki{zPJ?8Lxa>`X75XV&~r=5OG2n{pIyT z1n5Gf$&=@>o3aSf-z_aJxlnm{QI);yMThFSZ8B!Eg!+ERW#@|mbfHw0WpU%a*y zp56WThka_1-P!cjbaGa-wHKHuger-^G3FD5h$s5K{%hhA@CO8vHb<2#C5I#|^V@U_I_7tTDHS?%gL9KwjBzoO}Dhj9(NzpqgvSKCfYtAh{U6%!Q=B(_y9ZthU}{>rf{ zYKA*}o9_i|(t4sAO={j)h%SaGsLS5Hdmx3B$!hb5UmPX&9avk^Cq-f4J;|*?yO*$! zIOoLDGV!2>x17^oE_sr~d%3Hb2?B6((})V#S?ZyUDhj^_hM0|0J^%Y97bjZ+t3H8@ zyhzW{cG}Cw=kXt}!H{QUq5d-WAtgbxl5QT@=Iejk0OU;weLw%Br7vPn$jV;Yy7dUo zkbo3Gq$QE8)t{L_Qc=-;`?kD~PH!$Rmzo$3lRal_JP;$#r}d-R++`>j40&B0d&fYQ zeKntsLT2~@9QIUD(-wTW|3gt>;pYy&S$aoFFnwICXCPOM!3fz=$0g}L({_$yqIMSA znJdpB%UPtNrJKoCm-5(d$KZ>?w_8O<&lz<1jgMyvnrUtLWB*C@(>jmE?7roeJzVP( zn36u5 z+dB8LLnC(mVQr3zSYtE`}1?t1*!%q-rhgj@>Vo9)4RI6QFXmU z2pfYDzCGES4e3L!E~5u>AtAGO9P@U=iI@uEqPuq=7IsKRaeBQu-+g;mMMQw)rODs% z9k{su7;_ykI$w=bpDI>x>}_Yp4#F{f(>a@WH@8gM z_itYyW2@Xk{nTMSwEXO}@yAy=u9MA^q#O7S9Iq`Oe@cp@P{Q(}_{68aMN|ah?5Bv> zeUzEvth+*tywrAW+Np@qV$~`}eX%3X*DmWuX{*P%x|%n3h0Cu9Z~tD%`HV7oIFI#@ zlC(Gy9(l=GZr6g{J5%nmi#RnV9xmR&H9K5wo4=bE1K~ruN7F4PcA<;%rNot zUf0$8!FjNq|NHk`m_tU$FrVI$EOVKvtO*FuuS-2Zt!ff`KVHChv^Jbmiov%en(XuB zNig|k`hY%JuWu6G}@7c&v5SCQdU-uNIu2jnV05t zosly%$z9Qvc4XsY+wTTkh`@6+rO(ip*Vju>K>hESj(DAO?tO=T8K-nAxkvEwUw4>d zb!P)2({B6tovGlf=jYoQHLpVg8(vH(cZP?(KAtulThv{< z8r)^7spGLs5LLQq(TXo|W>YnFhxz8X*nI}fJ3r{N`=8l>99BGZlF+y2?W$-!`xaM4qq)3sPvpjXj(&!%{}j) zYVsIjaA<#g#I+YLtoI>hWAp)}xdxwP+2$K~l(O^CnFO7)Hq*S?7x}b>BTHRJ8)tJ> z$j01wG!K6uc|TY1WscWx$!Qh2GY(0O+;)TQ(xane{_LZS8Cq5`vJ$UeUB7PN)r`|r zOUqggn>JMhI}M5_S6I$=pZPpAs2s0V^dML92>EqMOxKOouE!`E%@QQuEW@&+BP*G4 zba_pUt-&|1T!92<232#H65^=^4W4SRFc8g>7w(zJ7^Z`A3PZHf3he zFRkO=Anb)07LP>6qJBuqyt`JsgVn#|UIIfUh9C}e%h*LW9(eIwV9Ao-m&r>0d3tV~ z?Ph)9IC{Ay+c7+A^>m?lYRN+!khaqqb$Y)40oNR-@gYTV=LpP0YtIY!F51mFMc(sOFlKm-DBei5dB@okfc`*ZKAUpLW#@Da`y=`-ux6)7|pN;%h6+g&h9Uj`Q6=h@Q+Dv@Q6KV{(h;UnytAWm> z=7__cu#5)}R)q5xmtbzyMpR5=Y^k``>X@ebol(r@pv%iUy1w70JI`q+Psw_d{Zm z^9>k!>bHO2T;Q_~nsxQgi8YK{k*hcX00VZZtei$qxp1nn1WreuWGgrOS|pT&L%aC^ zM>nMQG4@eOv*wB|@9l)FEWSL`j-I|%wfv5LS8c7XcKsBSSznNj`l)6j6}P88_mFM> z`D3>xO4t&x{hTZ|cu353*#~)^6#bmndHj(jcDI^vswgqA`fhGesIxs%DHuzlI}P2_ zs%dZ6Z*)0!U`MSiaWwo#9Ac7H9j*J)?tAQg_BbT7pMM^pq36W%a^~T#&&RLbQHZ9I z8E+ow1AQ~NbeU7lrO{G*_V(k1>pf7zn{TJksFoi!-w+$1V2sF#m8^R(o#((BOClr8(EyM)33_Wrezmt zzSU$eXXwrdXNP(l^DT`@W3Rg4@u%i$6iI!3HvRx0G>U<17mkrfbQOb9li%(WPEfo> zW?9K8(m(e1$4*sKul-1fof6pp%`M__fW`2IPd?nN5sI_JH;%Ect5FOBBd>glt`aO+ zEhZe1_AX4A_1wr-qt(A}didbELV&i)h=EyuxpB42Ogn8hWM<6Nz9-eJ#S0Gu$H(p! zS-H~w`RlgQC-It1+lgf;AE)?+ds&~_IdQXAvTlQ)XW6P%s|EuWZxYHR+$7(YHJjVn zHjX{jNOYMt5ufxA_GZh^Ran1XcdFmE;>YCJ*AMBmz(872#lfOX77h*%Jf3dXwv*;c z)7^FP&MH!@CO!8t_NrV~0{OK*_3ZA~#_HO-)TD%jF7AXWIgJ>zU7=ogg|ssoiklPW z*X)S@eV1$Q3dNlhI{#_bY{?6(l*_1ZEZ)Ljf%Oe)<~-KLrS%Dk!AN2Lfuk|cV_jHr z|M6q2Vvw;BFF{{(C1C#}exje4TYO(xyd|#4LgdgjvzJ@8I-ELsBg#k7(w|tzNraJO z_ss*(4jm-^g#w`lfBbW5yQ!=c@teIQU+TKM)Wd+9Qhk#=P;8M3ZcRJWCOjen@oDex zxy@a~IGDgp^!g$6W8Ta5WIg9%zAjp`ke*-WuPjznQi2159&B5CgT5wi(_&UHH!ttA zpD$T! RootTxn : NewTxn() +RootTxn -> RootTxn : init txn object (epoch = 1) +activate RootTxn +SQL -> RootTxn : GetMeta() +RootTxn --> SQL : leafmeta +create LeafTxn +SQL -> LeafTxn : NewTxn(leafmeta) +LeafTxn -> LeafTxn : init txn object (epoch = 1) +activate LeafTxn +... +SQL -> RootTxn : Send(BatchRequest) +RootTxn -> cluster : Send(BatchRequest) +cluster -[#red]-> RootTxn : recoverable error! +deactivate RootTxn +RootTxn -> RootTxn : new txn object (epoch = 2) +activate RootTxn +... +SQL -> LeafTxn : Send(BatchRequest) +LeafTxn -> cluster : Send(BatchRequest) +cluster -[#red]-> LeafTxn : recoverable error! +deactivate LeafTxn +LeafTxn --> SQL : leafmeta + error +SQL -> RootTxn : Augment(leafmeta) +RootTxn -> RootTxn : woops? (leafmeta.epoch = 1 vs txn epoch = 2) +@enduml diff --git a/docs/tech-notes/txn_coord_sender/txnbase.png b/docs/tech-notes/txn_coord_sender/txnbase.png new file mode 100644 index 0000000000000000000000000000000000000000..f578d3a4e57d175363ffd7cea66ed691845682f7 GIT binary patch literal 23538 zcmdqJbyQT{`#x+J0*VR(0#ec-NJy(RNC?s;odZ%sgO35y($W$`hrrArZP4Aw3?VHs zAl>lWXTVQO^!WF=Kk zojMZ%{=0bT4EQAdG21ur4~w&uma~cdOLto{bLUesW_D(dM$TrYca7ceS~@$w6yoA~ zX=`NX>;kjpG_i+Wzb|;_)Tz_^R?oGZ|9*bzG&qf0B1~1m33LBijZMRm5fjtHpWnJn z-xDW2y@b2^?!%dJO!#z2cb`FdN!{d_1);gDE@zD9?z$*^Lrd5yi7V()hB>i-)0wiz zO^6>!BB$c6+MhYC!Y$9HFX#AqU+}=y#wb`9$8t@U{Qk4MVkx=b2Z>;J&pd9R{m6mw zW3&?cm7k%*nnn1nTaoILSvZxRC8gf>hMvUtD2iq&t}l}ny-R8W*Ph)x$ck{!WAPWK z#+`l{)*@i6sWpra09paMZojAgSzBSj9sx<>`_5ozAYQ0I=u zcC1cNKAQxekE)T%Wd8X_Ry+rV+-DVy)MeidN-XiqX8G{=^O&iB`mrnD6wac6vm?u7 zR?YDl3%wFGl_wZdz$cO+qvYguZ36XcuDU(*E1k=Oh8F0&_BkTNCN; z280{k^88hb*giFzMj}g+##esol(v_gaUW69dFQqwkG2Yj&hf7{Zor^vv z#Vd=GBD2ZE=zLdrhN|ztk|1>GVlFWt>za@Bu4(i^VJ@tYbe1{A{(~xRxy%7Iw(#?MgBaKBgq)PaEY%Sr&$Bis3 zX6NQuHh%aM#|A1;7wESXW5GE;;ToS~wY9Yi3ky3sI&22=n_F8Wm2)b~%SZgnyZICR z(5JijF+N|`=ld1T-FIEmLz#6X3ii+mmZ529I#Udb3`8$p^#AnfQ+t8{B}ssMaVse) zX^Z^AK!I+KD&p0vSL6@OWrI@h9_@7cUbt{UzViYhMO|IpZ&{m`C>90_D_1g=Q#cb?a3olyRd0rrzRGugY^k*JA zPQAy_xmc~NulMn{uXT$1&iAoMSxtnIm|;C)llW|6cox>@dY?UerW)INzWPVtl&KvS z4J%zlK z(7D9fcO7_^%$3^py9l9Ez{meto;TwFvd($_@oA{1hy^hzZ$1C{x|Y1GPF#P1u0Wh#txt8Teuk^RY+Am7IF*=})_hM|TdrDG=u>%} zqNnmwp1n`UA}uLTb-~Nq(V`+E>n)v4YW>D3FdJpDmAV^3TgCkZ=c(#GVaP}$=B&da zBO-co)!8g8ELKu_3Ijh|g2`p|i_tb-`1w=IDmW}Gwlhp@x2eA$Wn3S~Q})y4ou6OJ z)J%IqXR;{U>A@nyu_*lvQ7e0UE*FRLJ}!eQW*73tDA5D9|m7(GH>DyFx@1Be&x< zpOcj_9cPjJZ11+V=94F{`#d)nb41)?IwP#Hot>R3=z^(UzYvf=)BRtM|%H|qMn<~WwB09PG+l9&5=Brn-5$j7f52*wbPLAUn%a_bKvAc zq#wW+wVKOuINZU(!8Fg4J;l|bQj0v#7vEl=->N>|Zp>=%rD2u^lHxes7Zdt2GBVtG zbUu%;rz|O9;+aUBK_z#r&GKk9EiJ7yHMKbF6#|0Ei3x_!-`-qY^G-dEWHuT_1|vu- zH96C~=Q(s3gBq{$aw)BJX+m9HY*vcTF4(AEhG0R$(qX5L&wTbc2A)0zJNpSIlLKqd zd60FZm&T|&QLI&)vWuTdk6FRzXu4r*&xr%M?}Hm`)WOmz<%oZG6mqy|0eggff3Afj zd@NmbO z%*^fybZ^wrXqx7@qi+TOplOQH0@9tza&mIY%AIX(A|C6O1_snj4$w%kANb(awLHSY zdwn@-6d^MpqJaf3gEiYdpHD47FD53&&CSihq1fgow8H*P@*X z4GM}ZqJ8inGV&qxh+%aUX*iV4;5{ZLh3^I3mLNlbNAn8Z%@1-8cns`nSPcygOYATT zjJLSlN#?@&9nDOw2X~MDhTKTt zt&~4oMYI#!2V0Z#Dr_2(d)sb0y1MB&5}Qgmj>4a~sCy3~w(<0Ur;q;e%_Z^-jF=`r# zAAT{+szu7a4n4Uhqk@uBvi+2F*>xDm2RnI1a8l1m5jr{sCW~qzmY{PN2A+E+izU@5 zuI}}rqw+Mjm>Ak91Xmf${UWCJes{Yxp!5;)jL@R`=7{1L((LZ748SBmbok8RDdE3K z>Y!I%()a<52rDQEx%mx)(V3Cg$GOjTSXk~Iq6+l+Z9fivdi-i7VXEivfb{k5C}iM# zkM`cegQi-Rs|6~QQ!ZJ|S`(0k_Ex@U@WH4G98B{e^?6`N>%FIUD57@T;(lz@M37P) zT!2>Ei*k(JVByY8J3;WJ{{H@z%H_=*9Fr2myy&>^;SP~ZBdh#KM~B89w@sdgVhl$@ zFj~P7OG~Q^@?6&EmyI}VBd5On0#;x(TC*aj_7v$Iy=By(hb1p|X|z4OlUBlGpiqA@ z*02QNBg}=?ez^HeAO)XDPj4(plE2&HOq&y2I_4){`y3=kI`0e`hMAhyoj<>mS1P$v z@{3s9bo7gZ+gy)ad%{H)+~FY~d`XYhy_~1qb=k3bZZJR2ePjM`xwdnADw6oej}!pX zKH+}&4_0j(U)(vobBA>xXHwXz$Hv&kWeKd2hA^6`g#|Mk8+zdip99R8Po?Hwpxm;; z5fu2|2-@hj#Bgx8*jqFQc>nNo7kYAQkm%0O`jHXsh`FvVKP;Lxz7T4Xt1i5}2-~`!K3N z(0kK)f(T(7oLuWM92{-07(^qkwy8iK&9~6S{|P2}#;kdH(C1+MQK2dnUfmnMhcUJ5 z3OLveCGF6(QG}t;C=iYV!otErLT&{VGE11cl&ET>&V7iEu9v&h#s4TSJSnN#dk@|m zPr`oXvex&QUtESY``gySqN4inI`9Z@I>nNg2x;f!>zsM$vCfQ-=4K5!YFP{xii(QS zk$-du9zThG#b|r^5XAJbA~Zl~y*)jquCf!9nto05ric>j%fIRZZY0*7hMkQfrw9Qd zROTjgvrWmnyQ%VDNzmtfH|F~rHL($Mfe#s{6~TDfFPG};Fo>Nxxo%Ty>m&n_CFQ5K z54P%4Ojb5216CNP~$j zT3zAP&d$!s<)vz58eD##R9z&nr}Vf7Jzb(V)0d&Z!!G-%mwKDEHSCD zoqdsUb?&WDL4}f%5~ak4;VCI$FYrSOjI-9|c2#}X74`dLTieq%kP=S8)Y^2gx#a@N zabe|P4nDyG%Wd}Tov=oB&>F2wBxPgETg`w$$Nnnr+6)s9jk{0Fd%tVw+qaI!EbHY_ z7h>wgq7G5-T~w3o5vzJ~wqE&@#!!@UvRAt^E-rzrl|6Y+_nBO5&mPPeI_I<=mTr<( zf?{^I)_rYe`}=nzn?blu_jlbmmz9ZxrKM<6cFUuUL3%`89n#gw>C4M;LL>~0X&44O zq@^6q*c|g#x9HUF20>t`9b$yX#X7IX`_0qt;Am271wFkChV_8~U)1|U&yC!L3FGD8 zTE!NfOtlA_G}dK&HWdR!%K!ns`*6)(ZYz3oq|(r?r%o-cK(o_jq{2=m zR7_1xe^nK$@3xVQ#0vARaszU{-17DH^?deQalF(4a25*n?px$TsxJOF_2rj0=Pj{X z#Y96TrsttqP#Hy1k7Oik>f4^`q^B5Q&Wuf z^0g$ydZ=ESo(RBFjEv`<&SEYwYhn)PvbZM-q(ej&aX9NiIE66!UAz|-qZ7oB6b|xa zeg3)J9!5q=G>BC*{*a-DyH}__KUfqN?W~qFn6C|6nRw4tbKB0+1@q?Ixrbzmp@det zXpos`=S#hOeVLB0Pd@ZONH@1y!a0T&?itj-Sz^|DAFSI~tC=v=aJ4WkmwnTx$I32C zcU~}H3!kOWd)w@7jU5TQFaBg3A4er7;i83qI0n4XD-;wgl3!YwpTBndca=#aLBqvx zg=B<46E%#w^n7le7F zk9OTnt4JF{r@953V(-m#2|*9Hj-y{&ez}RUXEX2?TivjOgF`?-0N}(iZz-Y6eRhMD zS&k=t=u%H7_gEO%e*N0WzN(vnr^nDcwxzwD_TEgbeMIG#pDE(@^9fU6@pmM9k9^pa zl+s17SGYS>?{WP$C;KCs2Ah(z9dm(z)6 zBl-neLG>MPEijVOh zK`|H##iuZtP-cdLuNoAxA$nI=Q$w4Sqq$!gO1R zI7T9gEDQi3fpU@ve9D5VHMs&=6|?_C@Fb6^_*^TCo)qtM;@zb`!7a@8n3^xlGkXUK zEo#S<5dTc0lwm=$hx==$?yPt!c%0^tFgZ-<9KRc&#WddQ-MbqLu8Thb!~mHSpB^o- zPf?;}tzYFL@3kiiI(4T>87~3ClzM`uF#Kw;PT%p6fURpz3LPAm1@Ne+!9C*!L8Tl2_hmRyP;GK(6=~8yg$b(a{Bl&P`1*uxxuT=m|%t2K2gpKlpZ$2Eqa0?%{JbmAYkiy;;g! zrF^6cvmHrfoO)W$*~7!b;q)>WE?$f^mL;CM(6GO=W>%Dj|4nN-nvWvnV>1?u9U0MA zQuVWoAiH@pm`Zq{v9YncyBMe@7M;oJ!m_y(w{QRc_3N$&w%jR8$vkIJ7p%Qp{mRXy zk;W!e@9s#4t5D$fk|U1 zOGj58t~rva%yww?>py3il#_wjH~WEBqK}A5$oZk$pHJH^U^?34`5rK0V>*HE=|@Ou zP~qGhb&K%!{TBoI+Ny$%Q?eiqNFh{X)#kxDHs)3QVBKl;5sV5MoGGfB8m7!T*VoFO zX4lQJK$Z-l6wtUgy_>0?ry;X+oq|Hhx{rAhS8tAG4Cz&=Ty-<3u5v?SG`KWaHN5EV zew|HnH8=p+f|&U^dBN{(smBn+*|*Up(ZFT2e?Xr-a|S%Lygu>0#Zn4B>ujpJEqVR6 ztoU>-7=eA&O2eP=ABx(~^z`&%IrTGmjJ}^EBZ~zXMowP`r~*d6XG+=G1lKFrw_P^n}QXjZtHl?pdgRekxO8?6fNrSySCgs{sM z&$V^~L^ezf10hS!G7PDS{+E}jTCgBmq?x&iNxJ5f%&OOg9TjdX#&0eVMnpse>W61U zjE;_~=IE9!0XPm8EUSyvv*rE;{dCjno`+UF3=-aba?i}o%`=rejf_%x`RzwV`1#Sl zYa&3-v=>+zLeDu+l#eBchtujBX@JxyFt)h4?%Zz|5Rycv?e^{pe3hI`0QFN_TGblY z47m53Hwy(?*9*}32BPWiYoCC&AP@E%ssB4ZN9X*B@o}CPk7fa5PKWj^mJWrRowblh zqcVc1y zaRuxb2+bf-kmlfk*cfn~_|C%@`$KSNXJ-(vC6&OARZvt^_tkItAPW`%a_r%Q32O3k zpflDNT7G|fnT~(@z)27hI6swd-DDK;PQf7g<40ylc9mBu+E*$~wKOp{#+2FHH|2MV zZViL*mO*ms7DDU?5dlG@9O6ytW}5K$=}F!+98kr>L)=FhmizOnh04rY8+6M!`_OKH zxd$eh?J7Dtz(Xjg{fX*>O2cTzKEYet759shYq7KQ^Y8I-G^lSAsN&MWw`ob&|NQwV z>WK~)XqJjQefl&J5fLFJe{fLH1I`GrOEtkMXUSqWI90W@er75r7;7R;8i|0IIN2Dc zAPRWcd_)$7<4;f(5C?S*7e`Q_@ZMeLCMnRPdnhc-=IdQyS@!nrx$C)(XT7>UJ+|BW z5V#rQP>qvv{eHH=D%%^#nTXX{jn%L_;>FG;-`)iwR^{VgXJzGCswOQg>fy3dh_WM9 zV`GdhqJIwoU>I3YP!Jh0+Pk#5?3=<2e~eEty06fWjrO-!fSO#|^^0+}XLm9)4R^DQ z=;GK%7dz#Fv*y^^1NdTTrpW=&DYNX5p{FoOWE0`Gn! z4b#)^O{Nqttvrp5;bBR8`z>YhgG>CBak+Ix;+VXDsk=DeDv;!;5ej zzsB#w&C19qdYKl|*!S(~nA~PpIaIaHU{9vvf|a-@(WO7(=PG`4y&{iCnJGNQVfa98 zB>3nIkb+%EIqzhl85r8Edol`bfcqf$lEQcRCd5ZeVE5@) zBg97ZIJ_bie~)#b39x#ok$+O1IH_Bw?IR!x8-o6(mjok=p&mY(n!kA?1+Q2@peZV+ zG?ERCF&%lLSfsFiPwnA`zyl!1zyH%BpH)%nMcc>X?5ew1I$Z`Vqi$?sV(-fhnQqFB;%o`AH?>th?-in}!b11y{)0vA!x+R<@*ml(g*RSkDI zdMZ7wY-(z1gE$gkGJQjhCUg^NGrW0VdV2aV)Y0nU{yR0Y@AIW>AAW);jRj zxVAy}Q|t$~rLiiU9B+=-&gZ~D9%->PtOHgq|Yp1k!!Ba}r z)42ePgq{5WUH%^SLY*9R3H)81sBR~t!Y>X8`|GmCpYieo>*{3gEZ>o zgH0OWDhU<~shR->tV$cSh5>8P@y{#R5Z~hiWEg?5 zeVcs@DoJerbGm_jG_PN!D^pwI&eH;&(2kDHqN<)h7*w&8iOmSc;$U}az9hw0tO_tu z@xzDGo{y?0Ws!LrE*g9JdOO2iso$hEJoYxRYMIFOje?nJb#4AG*(nX3-_DWMyw=a1 z$iMZLmOMA@**4$^@0*;QR8v*GkAD`^kFnj|1<63gXyua8s}V`l7dBKwj#PUl`5v(4 zT6%_2yIdjg59rK15k|z^9b21Zedo9k68;zulzz``Kvl>L;d^w!q(MZV z7p*MjUTh$$=}^F9^XHfJ+6<~+yJ*zGXAg(7NY07-_)}tjmBqyhRQt3`F-X{g=Uh)$ zQ7v`OrP1xtUM%NCtmxf@qQ!r5)O5+m@&MDvYZ%bvO-o`UekBV&m|D(C%`$u-g?;z# z#|Ap6EDG=i6RJClQ56hBvmMoieP&Yl;mr@Yi25E9A`aUH=s^KLQvf~Fo>!zhsK~mr zV?U7nOg*hfyC~HmV`+Bwq0y6UFZmI-*;l}CA@@=a#o;OEN!&!lU8AXxi~ITWCnz^% zA`oU~W*}^gi$kjekj~9bIFN~yaun)s!j29-FTZ7&%q#OkHTr(IU1xw1NQBwuZ#2Y2 z^ei+p=`R6^W3}~h5Sh$PedwaHHEbMfv;JgQZQ<>`0dxoT2f4`uP?osGrt!WU6eCG) z+;CkU1+?o>0I7DpX^YpZ^iL6VBfjN=caX8l%W)k<(A^>=wV+3#p95*Via z@gp2*3?ax-Z+%wn+{NkF(A-Y7IH~5cHZxmYou^l|ow?HS&p#d|Fo|$Bu5(e$Sf7{w zL$Wx6h{lT?OfnC`4Csor`;Kp;EK)Yq>bpu1o$GT2Mf#MwD);ks%f9XnY-Y$A&6mQo z!wpuVc>T7wd~<&ZH8dCGaCsi+NXR;OYR**HKsI1Ov`kkyXUEr56D z=dHP2Et-{eBo9B&d`@?Yj;5r2wDgGcT9{i zn%OVbesKc@%m~P66)yT_8l~n2x@DXErYOY(4|#qsuz@_HGhGw^Ih&$WBch6<=dN9q zm!M}mz?GFTDP4_Rgti|^)k+f?UY;rbtrS^(^HzUb?^=;SGa_yDKG!W+P%dgY(!$() z0>3^EV?oivR?vNwFWgo8(9vnI@E;oAVI22SZx#qY5LnQEy&C4rdjvE^u#mB3MSLFX zy=Lh!xlj`mVLr=Wg@;79!Lr9kIow3+-m>_@q+0)v^qvp2Awh?v0}?&xtEzc1lf@=Q zf5N0sr0@S-jKN>-M4s9F(ak^2&3-(Fpgy#`w6wIN<0>!)+#n%|Xp2)!XmjNaIs<+J z*1plrj_nLHJIZSE{8#XqSCQba{pI`$3JL1%RWI;Cz%>A$DCy26Fl{|b^swR zyxuUxV>kQ-;7fX{e6T`0g#3(waR-OBuMLSl##?~rlv;EO2@6}8ntn`sM6g z1U9x5W$4@MFp+9=_09I`RBm}3U!vkyI5NUd{(|{T;EiHi~c%38lZVCgJ z1`Q33j7+1L_im!7=RlK=GaB>75VD`xwE{FpM^B%Wn3$_kfGIYSnI3yZO4Y8?=C^4i zE$vrS#AS>xKCm3{wy|7WpD3*TDkvkl+NRg`% zw*7e@3S08)j znEQ(Zzmd`=`-D}nj?`tsEZNW}>0!c4E@X4h+KYJh2^yu2Lv zNq}?5Px=A1Q+M;u+N>)MbL-@mg2boGzkC7J7+vosoie+Tx;n|6B2c}5{N_T-@(s#o zp|phJTf;GQ&PBAB&DS(DB9iHq?msE_(8^@^Y-6& zeor}BuRw>do~~2;Xdf(^er>hPr*pkood;IJz>Sj6b*oue$`kPTfByL=Z3;Z&Q5QIT zw9GzbH)i<#b^5BRDj+wRcP7K86j#B|E}lEr6iC5udz*~R!ixi=tMB|2H}*J|?bJs5 zCa}vW=-31O6Nlr-d%zpN^3mnq*3w98iY_FIQNj|sS&A}(X~b*vE8$@QA*Kp6v*LE~ zzEj%)2ucD@JzzQm2kGXb$BUmJlO`}mn%fB$Wy=!ykJDkFPdV9_5Jso-)|d9Ct`Kb8 z8#|^grylTZk!Eem-O_0(jyEMkmu_53n!;DgL2$KmZKaP`)NLN$)hs6JR;qy_fln1u zF;z=XF88O-RCPVygtM^_6k1HNr_AeeP}r!1DOFV)xZ=iK2}*kQ#|=__kr)Lup543} z5)>XfjDsm}DisdHJDt*%3Wr)s@k{x1TknvO9NY*qhtj43ga<@Au(^H1W7p?qtjDE# zZI1cTEBA2UDYwmFt?D_xWA^h^_4wgH+!*rH;0yS8Y6=zjPngW#^Zg}i{|@5{kPm3a zExsqz8mwOZ3TKuCm$)732c2B446)y!6mWt`<>~wW? zJp!^eKC|40wGN^6toNWC$)uDIOo?yaya@{fk<6QjMoh252drj&Fq`cy?=siRc8LcD z29lGLxoY=U2MvAoS$|L%CgtVX02PyZUcJNn9Lwb`pq?N56T-Bxz^3Uj86qG_;tECy zG&z7~C7wKy-|7Q3O)jtfmByHC7-CiYXiZ$RP>&o`Z-Bd%CKnyu5+;6ln~I8xl+;`U zV^C_Y2(UPG($806x>mH2fJ%EVS8w%|FW2nZfgl}i2R0~J$Pwj0|6U$0Fa51T_Zegb z;MJT%TXxeSia&jNvG}uiN#j;hQqo0YYOl2!#f?RloZR)I=Q*mDmcN0^E_xFb+d^-1 zf3L4+1VL@6#0>K~4rpE}DTdt+8ijf+%}9n&A*R!p)k*BCXD`yHM>FSwr$$CfYR&do z_;$J8ZBY+oY;3G4QZ-x-fd$GLum^Lb3knH|9qq3Hc~2d4v^_~&9A8va#9jgd|0Jzk z<@(CjR_slFR#tgB-cdVA{gPNyL-GK{y}RYLD*Y}aUmPU#V^i(Fb&U>OuKa0Y0v>~7 ze#<_xHFq_@#^inS2R1z!M9NiT1q7#-`V!( zM)!a$xBmY9d;6N5nT4WSB4T1qJnhn%V`1Go1u8Ub z07+O*Uf=nSphC-KP2QXlX7{$yYR~8!5KsWZCI?t27G4O>8MV%Yo+AJ)hAR-%75Kh=$)^Q zb36T-hUB$!oT(bQ&o4uhPT+$U1!W4WL|&_iR51@4fBUf-eI7?9CMI$5+U;#O#Iw3g z?*VjiF}Kf^n8OHa7MC2$S(IPH4p3r4QV1EcZgNzwwy@3HTa%L)NIvTaY5UWsN0qhk z`$7uM8LHww`)=S#O2xEbfE6}qQy_yJ=08U@tWufE?Ylkh&!UzoE7D-ndpSKLwIyqj zd`J_Rfk!J`@>ZY~%gHr`cycPdg%=iQy3T+ZtD!IpgE!et36!rRX#f0ss+w93m2B+pwXKH9|-zCm~vo- z(4%mA6KipM{+kIMd1o03)nWUCwTFx{vH9IiWiKr4ZN{)4kTpL|Bp!2;MM>x^`!N?~ zRPV30LJ?Ri7_G0T2VWj7=$qDKXZrRZx!!SJpb&uv z97tzW(?10UAZUq5L1VDpM zouZm)-GL$UR|ioxni8WC>>A!^4wgkgN}}PMm@WpJghXQ?eP3bMzk<`@G29wBz`|60T>$z}G5a9uiu$$7ok1U&h6c7B@pwVC& z8`V)yO)2WJ4xnVl)Iu5#6n+;*uB9r(a<&lI;BVg#4a#wNa8U!mMO9*WzcwAUU~_yn zOs;ETu!xK0nTpDX#Ea1luV23g(7{u8NHh0&4ybMczyAZDKr*ANl>AQ%85q=~PooTz zbxKU-z+MUtOCN!P0hys!v2+Lpi7Jrk)6>!z0@$IcWlVnQ)or_aGF^Dj8U>XYIyUq{5TT z)H4#?l)Pz1Rct}@a&~mASXMz{rzwi9dNZYK&P#ae7P3sQJ`rj)LW2cn92NqWGr!%i z*3O7OFbY&t0^hgP;qVna&x>uxv@YbBk5;*Z$M174%RX<_>kL;ZNA$ANY*@nR?(SYu znd#x)rv7m@{{$oWsoD+C^~_p~y=T~28;q9&F94gLF6+#w=?&IKn8(+(cWi2Sd;?Vk zPB}ept48R2xlpPImi=8X3`F@CkR4(ldiTD$Y27%K1M0Om=Dz4B*My7F%Il@!mi z;s9+9byCe(N)U93A^un7D?k2;Ou*@~eIK3QU*uor#Clg4xVhqR_Pz{QJQMl4p9s5A zQLVAn9*w1*JLo8@3|+N7HXS-S!yVtnG(!N#?AvklInp=}NNy)+1khl88qxO-k ztme>Qt=k;JH^=9E4dh%1cH!COiDJ690zUcwyK;Q2iI>X}eN}Ws5)L!S{L97%a2+1! z{qK^pPX;8~PE-;97aH_yRL5!@0q-xSy}iI@un-Vn7I-*N)hdKEH3vEMD=c{_)*%25 z(Rao!v?>cTb5lo$52#H8*mtM`XtNK%Ss(;`{(S%62M(aWgRKer45D}c2VHFhFwoLs zcK&q)_g2a3=Elax>Z(qKb58E8%fi6<^XEZrl!cdf9hBpxTzn=Q2rpi|h&Y509MAwZ zfdL2(rv?!lgpz1@0PVwvS{408hQ2@i2qrsj;-ulV>{8&W&KO(=xtp*g zS3^r{8$ekIs`8lu<_lbQ+L=%R;fyn%(K7NpPlyVtK@9~+inePG_Ef?@QIZ7y2APE9QaV9VsU-yruf9E0@x z=@!^4p`oF`md=Xd;Nns)(W=H2s^<-y$@7!4_Y10z-UZR4FJH$~-&pZkP7i17B{D99 z=ksIUTS>t=eW2l`+_X6Y*dpXDUY37UnFm#2f=icF<|BOba1p9`IKS83!e3Hlj|(&} z3T5gPFE?P7JO$xd{?9H^3}_{uwuRk$pGtFz+LRAK4ZpFf##fW1aA zS5{OEd=C_Owt?wN7N3x3^lx^1d3aPfYg{aUIsRHTb{!LC zi)q>jw0KfjnLN4MY0o0)-AEK{^)_D2~tf$WE*hnVj zLXCoWA@{i+M6$*x@HraS09}H_N_Tw>w7(ryZyjy|p8-gWpo0McvZg&2tvL4?Y!5(v zgdaX^s+R;V385yCeZYq0AiL(6`l*JS^9(hgb=0hA1`t#QZU&EjBa{N3Ehu)E2>TIq z7&N*27R>a;`6D-ZL{TeDKcSiL*i$h;dEbLAFL56IoWRg>&Gq`1!B6<&t37c)#%eJk zdj9^bpf-7rwCNvCfFgnI)6M_&faZt1N7A}vUd9!po|wh^T)`|7wErURRlfkxC^*&)yvjRy!ngC7!n-3Q6Vy(v%%>k&j)2{LZ&1<@vr= z87gSp{&_fpi(yoE_+3j)+yYxm6KoGRw_>bu9k^yX?1j5fZZY4n)m3|GJlCKsagj+_ zq!eL|EdeE`lR=)P`eD7AWO}0n8lSb@-}~x$=yw$9mG#BN&28qS-g$w!-9#&5dGR-5 zda@|vG(JeACfX!e;6;M}M-F`)8&4{J_$%Y(NJz}2-k|_wVPbaH)i+$EZ@blJ_FrWD zK|fl@3M6-1+r@+W^fF&7*v;d-1}pnw|BhKpR~Phgy5KMyU@;ThQ8_iCK_^v2N{YRo zxfDL$r{hS8vp)a?1k_!KDM$3Jrs6IJ{7nGuI@l{&$|-2b9t9wFP;fASBe_6SYYe4Q zPJEb}l$1wc%^MH2P!CVf%F4>lL_sww{(JZC-M`=U?Hh=zZXjTIXo`Y30{jH316lw~ zf)WA{rS^7q)bq8Tkpe~=)}zQX19d}Q0Pg3=!1zaoRc;O-?vW$|3L2d56sgCggv*Nr zSao-M`=OA~=3MVAi0ch951eUgeio=zG!x}$mu6lSe);Q>_PdUU9$TMN^{ts5PlSbd z7>zL8_IFn*Fm3(s3S%Z&Y!lty)^cn;_^W7ht}FEiXhW;#Jm#Zuqw4@A`1-#>i$>lIIqAMF`X)9MfReCy{IO2H>@b<`++NQ(njMS0I|3oOPVcjMR^raIsV zdg+oyJ&DM9>b5QYkot=QEF)3e=W~LoMRhFhNO)>3bMoC3#82JcN)MVpq8Z@IJj*g5 zZ*g&P;Ul(2x3k*rd&j6jHESd3_Crq0g$ZfpE~7Lv-oV z&q~)K)g#~%vF^(@^5T`iaKaa&?B+j~X!bX45EWMYZ!D=wgdZa{n%;z|#xR<&z(CN1 zr*;ptD`areX$ITb*yIfgfnrz{h>|Ow%b8UlVV0_%UO7z}qwpIjj==t!r^SG?iN({x zD6I#5yzbtr4$q^yiZiON>{FS7LNva&_&aEB>>GYo^5f&jkHf=yA<|B(Q*x**6biLT z=i}q!QHty7L9+~@r%LWwW5>phetml*^@f{QCjbd!z{bdT+hV(@2Pzc^r9$yQ&F3kK z9}kNu;$M~V62 z;hK}$bSzO#t6?H4 z2gNbBQ7p*Tv+voG55nt1D2U@@S3){EdR?fwW@CmGh7cVZCr^Za1=&w#5HP-KrNZ^6 z5&GH4U$#)51ql(jP*38mZ9x>ne}&t=oCFvZex4PuWJJ`$_e}@DNd+|R*2>CZPs$#m z#LJk1o)R^InFBEkW`H@Y%w%~8%uyb_r<-r-GO1s}n%;N~+Axmob9$%)n!YJvU^8$+ zy@VPtjz^UT!%gi)nx9o27J301Dvk$b53^w$8|qfrVY7m=1$6q;$0+_dhyRxtiCfLZ z8!N)@;G0ySr1jV4acl``-~IpU#Q0x4CZOlsE9v6bo-iXM;dBp=Wr8i`E6`2_j36gI zl#{j@z*SDcbY~VsPR00KO@}eZ#yp1fUhoJ14L)yx{D2=dG=40P;q2+p>n68h66MIN zkO$;07P>tC7^lcPuM)uS`yzL|U;dNWKwI)e$Hno;sJ$A;3mo@h3cOK`=&8cd*Hb4S z9IZ48M>T$K(oC@>3&cAu9${~`cEQJU2alZ#z{Nq1iBWD3)Qq&1aBb6$h>muds0#pU z780rr0}IK=9JIPqmSm%;x%t1m9Qj~=o&0T+d1(GROJ)j>tsjqe`q0tQx4wI3Bic4! zb{A3qJ%BX*uQ!AfG-6GDynN%y*|SN&YT)ISySb#l_3PW={BUoUVNmdTk?M2+Xu5J< z)Y_YB9(+pNK*Gh-Vw--|d*dUt_JK7l|9!CkRjb65BY=h`2?+cP^sxm}1ombF^+EO( zm6|%AKOKN(z9^C#y`$~zKzGJwj4%Zavzi%0iDivJ# zTbh!`s`sYvDT*-qd2Hwn7A(MNR6$!D=vMxQx8`8@ZD7Qdr0fIXEz_;^pm$_qHf1cL z;*xY&<-&-E4UHAy<{kGMFO) zyWrk9M(HYNt894C(?FJGdTyw$pW>crZfs1P1I&Vm`npRN?bf~Mf}(sK9v`VqRmrJN zu>~eiIr&O>jiYF4A<}8~)ExWSj^q*PO`|fqe<<3=B4L=m=atP9phxIQ;PXsHn}Qpq z;rh8$38{7f_R7i6?<^EePCXf)j(xKvfdvZk5B%MItVEaD;SMH(HV{1a2D+|W@(vE+ zs$>P3nVTAkwDYPS2DM@u5rPPVvciR(HQ%wy!3QFuD!1qEqpYHm!Jh`jfqsuBP@M%2HCn1(wlR*hz z6bM~fo83`hmJ16(-K|3yUiFd&V^D1T?xQeh8Q|u20DvRnrHLd2uwV&EXaukjC4Brj zIhU2Epf6TtJJ(;BROE13bSFc>X#BHry;aM?bIqdA9G#8Jh|#j5WDD%s3%vIocx@Jr zdT9k*KZhyzwCU+b6>;AOINx%yzL|72)x#Ru$F=gR``9emEygr zExytPd`e-|RDA`ypXO9;yT1$5rshdMSRW`@Ie-4gs9U<6X+|y20@(S(!*`605BKhs z2L)AoD1Xs{RaR=M_PgOQKeLh@JDYwpZ6J|i-@YZcq&ut8CE}ZtNK!XTadMI0`z)}a zlk1+cowh2>bx}*#9X*mN0mI2mHn`?&N|VnG0DsqHqhYj&0a71yy6tss0zw9QFgY;> z<&MB4ud1OzDlG}6IkcelUR}U9IGq1z`6wh}H<~RygVR0N;qpke@MB)XvO*h#{H=Jt zHB1GnY!1`WLh%Rm;7vXGHaZH*XAawcGRJ(yb&@<+r^af_I9laN3#HT6bWoeE{g@&f zK|%HBn2SVS);p`RE-`witAN?&2L1kt*`}3jWvmB5)h`Ts4%$UTk31svSm&widr=1< z2$^|KKau1-k2I7Ns8K)(w8)l$>9g-^`tj{sq1=tmYR|n#%|J$c#S!~CRoIuX!1mCO0(=9dHE|U|59bfDn$v8W(vbHF`V90doqVgt{%6bs zcv0|j6ux5#K!Cp{n=AN(08HSghp91p5a9p!>msrsB0M}iF|i8lpSN${GPD}^-dm$; z|NaY@$CNhaS=!fJg8OnZPQb9OV}?i)DB5mf~$dNxr@&_V@yWYRuDpVBh96G&D4ed3}zY zni}+5K;0ITyd@9XA7g=$#TKaS2v6B*pz`pUMYTs5GUfkTI~5Wjz{VqqFdFD z~zM%^uKaSs5eWpV!lI(ERTX|j>a*fK644p9I>4>Hqi z9JyKZQk0AR+1!lzX7)I#8x6zG%*~NgQ5D?C3Qx-6)KyWr>+R*`b%%;-Z+*@QbY@M4 zH@!Zt4+FbH$ryvl09&7N8B!*U`e?gR97#qFniqj|sDLEJfp96%34ALUOvUev@MWn= z%HzqX%MuYRrhfSH(K7pr1{g``Fg-8t^N!W|W&sfqFCZhOB?8S#y--h(j_%2YXZN5Y zH%76D7U;9j5%SUC1x9nGCrw72{NK}V&Bb{W<2kBTk!H~-ASyuI^aZb70d|q<^&SQJ z%8%-FQNI|up^mCeOE%5IC=otR&O*yXX?P|@z*HrWETil|73QW5#C+WtN?rg{3lX&l z^B(NDD~e46>rwXHII_LlK=NLFKBG?1=KK^kO8s{d?`pNB>#?~ac00<##>R8}`}yip z1`}+xXW6dSb8jy;rZIp-SfRpbbH*3Nq~~fzy_Iu(lEAPPc~3D=%xL4!pKFyj!E6O$ z)b^es`9TJaxWu{v*jf&G7CO(o?IW!_+}b*K^kK;2Q@#lBswz;wq6_!s`ddX`0&$!%0I2^s4@vkF1=xZ`>NGZle%_a?3OaL}+uQpsdBGe!-vg4%Tsu1Kjv!sYHfR*s zXs|N1$fe)g+skt$Kyu#q_Y;KTiWx zD(SEYkri+7hO@q`f)X5P(*& zkq{qGas9fH2#LgAfp)1m&}9TYH}ltH9|w-~fH?!6J_TFQP0UjclyU!5*bRv0fxxzT zkobQ}ITN6!?mUje(ux>?rBzWtLODG^^C-{1B5 zPC0!6bVE7xHG#2%PJ<>IO&J{2FptRONK2eTf*ZTYE`i!~bqD+h%N3j88Bo3&PuFLoR<91@DFEVDR_2F)_DQPg@Xt zq9g6KA;0v~vW)whbVe!hogew3#TACcSn~(u0Som@rl(qYo@tg?Tl5@DO>9YtdTLW^ zfLdG=NEVw#wDP#Z58x!zeHXg?3JyE)Hc51OmtT3Z#=RCNivV6ws_e!Rw~gSaikyA* zUXdL~0yH?P|NK@aEbK_*cI#cGq>1m(H+n1xXcb+ogY29XB&VNpu{p^qi;$K{O9?H3%Uhg85iwo$Xc$1=3lQ`@Su_d zBFg&0;vnUfCiRG%Sn#I9k5g)SVy^*o?4%V@xqQ`G*Nfp@Bxb>WX$A+XWVm)|R!Mq* zr84pVFLHPY%uxY-%}PD;(SAqoF*Va*CUt?3}yM~o@3y|b>RNcgInp0 z=9!Rgi}lZhI@bDVCj8)VRs@GjY|lY!R~Hf|c0IX)U9j8xgx~noD?)wsK~a`QcE;`O>ANhuD8eP>FEI_Xl6DIQ7`-hkE4*h5MT*8QnYsV8eKTyty@u1#e(Q^gf4FfwQXchxu@Z~ z7#MbzBH&;U7j&Pn0m&Ov%`~?UYffDotqs?0(QfMV-|iN84XxcHgjptNis%iqBFvXV zmc>=Dv=}}HE+^05Mj&jF`qMz;@aGrxIw_~cTm7grlap_r8*saekH`IL{2-M6ephl( z5M|ZcH#|t>Wcw;MZs}1?rwwDOk@x7|5wyp*bz}CCSA&leg8h)Oo=qgNHLdP4JdNY* zk}>?+p^HhA&T}p3E(L`IkMHBA!;Zar;;p3lLh`J%+BP1O-PN~$Tm^iFSmf(q6Go$5 zPSrRb-l-e^dW~?TfbQYTQ8uP8T4)I$GifP~&A^-7j34UAH!1hb3y)PR#k0y)fK+nf zc+8Fz3=|bru#GA!zt%-T#)-qc;@((i%W(4`+i%Xh9Ua_oz`2@Dg=vM6juhTbzXpUu zvob8tx*oDtSvgNu&L)W7fJRdIk+w#NZ&lcPf#AoR0ew-Dxq2I{lC$?jWSUDAIlSxd z@$sWu{I;}tH-Y!n`Z<+sKl6h-YY#LVh$V+eR>TEKolTB7XZFunUlpg<7#XaLGu6Wv ziyuz>A*-s_Qr}xu)z?1e!m5Zxt(wX+iF#pW2!#^+zS<~o;X^TLbu^#`1srVMF@ewi z{m~PHT|ZO^01<3%N|%h4$xVH&kKJ5ob9O9{2lqGYETNrwH~)S}%U$c3YrRFo!`_T8 zwSJ)_B5|B`_Q(;O@x5+95o$9u+&{d^`24kD{VLI<>j1&QU-v6lj?WXzw#4_5luo3d zTB!kKauoBndu7!Q7;6f@23AnKSe;G`k&wbRvb3ChATNv;L5-!yV$V(5ve#D2BN7cF zFfnttZJNY4m9(<0Z7&7au6^p%-#XF|^El2&%uACVPBA^C8zPCd0FXG#N~qcwdeiC%WVI zfOh><;>3ktpBp$3xl$4{Ud3HpU3z4WaVVlOl+EhI@tkhWT(17*7t3yo9R7b|r;@9# zkYX3_y%sD8|I(#X3npcP$^Oikn9u3-0j%U5xGwO$IRJbP-Ag!?g_fWF?)m-ZIIh#R zLAm+qu6Qe(T|FWlee4NFt{1o;Pz0HoWh`=mPFXXjHY2bhhgBQJ+e(RjPP`-hFZNzJ zrL@x4xTOD;UA43}DMmJhvvYrh#*fq&td;rpd$YEw>C%{Q(oit=iF?KmO18h0rJ}lq zN9p{G&p3@0XKu#gJ@aspgLZBc89xuw8G22}8|+TriA(r|n?iz(E%7u1-( zQFHF@Zu!g-_rQa&#EYuQVy8_H3iS+xis~a{8cZ(xlw4z7qphduMm4t2wq)jB?kf^0 z6B6CVdTZ!h@2{_5igakn3m(34{C1_zPNq%l%7I#AB_wa&D_VySzH%oL>ec2D6DqRF z8JTYJl7$D|YypoS#MucK$!&x@L)o!}nnLnvVT=HfB27z|?Q;4Lo24$S zv~Im5_sgm2@&TQS24dJv9FOU+r0- z`j|wUqvvaKLU+P$&g?YaxEiGjwC82oyFuCQ5D{)7z zWH}ySQ?MGXe(m$6P$||gQgi>E58u<1v~f7AULKhDch{Hp*J=*M%95RRBto~ zk0+Jmc5S(nD|1oPKDG8`7CIYGcz7K-e&&izoJTaPxAz-s{XYMdpWXh%Fxc}gOdKZ= zY6>Bf$j#MnbeuX_XX>=<<0TbwzqhR!ZKfEIe>(?rH7^|Q;wGO RootTxn : client.NewTxn(RootTxnn) +... +note over SQL: during query execution +SQL -> RootTxn : txn.Run(client.Batch) +RootTxn -> cluster : txn.sender.Send(roachpb.BatchRequest) +... +cluster --> RootTxn : BatchResponse +RootTxn --> SQL : Batch modified in-place +... +note over SQL: when SQL txn completes +SQL -> RootTxn : Commit/Rollback/CleanupOnError +@enduml diff --git a/docs/tech-notes/txn_coord_sender/txncoordsender.png b/docs/tech-notes/txn_coord_sender/txncoordsender.png new file mode 100644 index 0000000000000000000000000000000000000000..6df87138bdfb96504b6002138c8bbfd7017505ee GIT binary patch literal 55505 zcmd43by!qw*FKDL8z^BTttg-(picfN(xBJkkZUBbf-v3D>8Hl zl0!*1{MHcOKJWW{_1||N$9)Jho4v1n?RBknuJb&v?fytcoa_ko5h5ZYGKu@5vP4A0 z)$sq&;r;N*XOhF$;GfIpVhZM3CNFFZwROyi#I=pJO`n=;KRc&sbMCph`3rsyju(bc zjm<5L4B53zjE-O9qa`BRKcM$S!F=y?qJ3~0>#)ElQYM|Zj+LZVIGufY;}FdQl-u$n zvGaOy0dG&;mkC5y(rYMZj!!s?J2~hYouIvgo+s+9=IyKJ^(np^ZPu~%kWWLR#mHA2mNaqFPTo!v20Q# z4>$jAIl}bmls@%U+*c9>{`rr;Gu7|jd9pvwYwCsc+ABva|1#oIZ&U_`?xYxxm(HNf zm=RY;;jqnt!%A`5J%eZ6KYTe7;VCj|MxnoYRIW?o@u}kPANeMw<_8~{SPh+XI#l3x zk(!TcVqVoWlwHY!jQ&?lLvhP2N;3_u}tv3GA?i4rwQM69&GYIti;m3pu+>euXs))HUwC5l|) zC|a7WXBCfgPcB6-U(Ac^30JzkL^_~$qPj+pbYq-kdF{;6l3t3>mlEWzcFB6`@wu^C zmdn#VUrtug++V(br~#KdLR|Yz@;Os5SJ1@pj3vd*wNq+Gj{cfudsp4%F_M0S^;4eh z;}PAZSQ= zQLHxQMU|@Y<+K3QqsEigAMHu(3_lNb&+tjys#*0Y8fEM8>QFDrI`zq|X_SwM=oOKK z= z=tkM{0CK|17j=UgeJW?{988Me! zyx3`gEpAe>E7eWESv|2uSXM-zd!AM?5q=>0jNbkrO@I5hMd#vHUhck|^y1!t4suZammZSA3f z0md1BmWP@+3JtN3{6t7H1#TqS?^k}o!EGt30(N$5lU=#NK|!^c3OYKvP_zQC{`E@q zzBK*M=yd&8hXT^Isy!0flye?PD;Ah1#R|^VbE24YH7k!eIXThLguZ(9%3*DBjMZp! zeZ{Zi%a<=THDfCyk)<#C-@Lwaz3#V{~1FnW1-c(~e|?#JZ-{?Nw=MD~mR zJ0c%SD5FQd0lJY6NJi5s6y;X_NCc-! zOvyoF4IB~Sz8_hMI*FCag!9M6-&c`UQBmR2Hm?78eWWQ;z1TLpLTukc;9{$?37wGb z{OnMj)A}z4rjZKwlVp^$gZ>XC?Q8s)By)_qa%yz$-**#EXz6(G=61Nleq|_FyDe72 z|0264c0#GdK3`U&jP7FeSh6Q22eS6+<;x+$4h0oMC*KKU&a}1Z;UXd;PWX6v1>S*M zJxxoS+=RB^QLB*WIYdSs;N)7(;MD#3ZB{FLek0=m5{Yzoceh)b%FfENpZ_jiS64^Y zmY7-wcSD5qLJ+smJ-+_(s#xWzi|m&N^ij+#Usz^X7!i;cd#RA$0)H`-ii)Z>UX-iyiKyH_ zM@PpA$~CUvuWwt{yld9W&B@7WKygb+NPI7H;m0cJ{$j8hJl^qSWO#_veFn)kI5>Ep zI4nGT=!b83xT>0(8kQSxa18?P6+#O~v>+nS79Ia%ko2aytZehEeZ&)!lVU#u4AC`I zmtvi^Hc#~Bny4!(hIn~N2eYfbKQ7rIZD3&F?kONy6diqMK-N1$iV+n_#jakGY@;zZ z+>jlFNQ`89pNnUb3d%C z-nn$Lz~!+p%cZ((rc$IU6ZAJ#C z_4Mo0sK&n{&(zFpaykuG%TY@9OMUyAQ&bC2vdDbY$EBpCWEeo4GchsMuIgxI$sRs@ z_<3)UjSsyLOegD=D-W{qd}hO!#?d(23yh2bsel7?oj zUCU=ISv7smV!Sn6#7RxqQ|iIZ zq$*Es44%o$$arU54y+wTXXkviqEx-%pKB0^TlQ%$N6_gocLB`n5w)FuiHVo6rme|}KVX2ZSY-izeis)P4`Y0?LtlwQ^roz1f=syZ z@=xN1JC@VE8_U0Za$(^~hw{9B{rV6EJy#7{B!7-0Q2%J4&ps4Jn#&i1|M@eDK4m@p zc6g6G^BU9l7hbE>q^}UL^cA>o{koUc-r9=H%E-%GK3{rjY-}vLbhhJ_+CDgdu#R4l zJdu~LuB!5ol!*{r=`Y&ve24w^7L-}6YZ_o?T-`|oHcQQBOC!03CF30p9BTG ztUg2p6P<=<*2G0yonYazzvTXK$eV)jXBs%^S~M$C+{wnCIy4I5 ziQkysinze_V42a514*`XyaM+p`sCFZx6=|55_)=iu#gWNH~3PamI$5)#ML*{@s~Ae0`$^%S_vxG=r5B1bhfHFJ!6u4sk_1tA|j`b-Sp?^K32 zdPuq@%fTuy8nNVG0|S13ezUW))`Zh<7Z~Yl4d|0+?wRx!DOT|F@bCx=3kwK@!YvYM z;0Wp9pPf5^JMM#GLm1O%G)*(wY5CKOjtr`O`-It)mF%qZ-qTRp4v2;oEU9nRu0?Ly ze6Op^R^(*02^X^a(xx8yXbBY(BJ{Pki>RS|=PVGan@kVFZVu#+xTqBb%x^5cbJ{fP zOi`60CT2ibIxML=`q2A4h*+NwWWAhg93IprueM^8svM0)0s zzLI?&$!+ziiccbiRgLQuph;qbirmTk)|N0Ou@gRWdFLG$O6Fin@(r0s`8SV)yE z8dN1DSmT=O>)UB~Wxir6DCkvUhDk|m9w=3+?B2VDYVzvm*MZ*NmQBJmIXVpj2{Mz* zV+o8F!PypbP!RKwjuPqVdQW2!t}X{7_%fAy)+aK}MMNIr8hd)e#iGQLnj(7S3^t>p zrrh0G-zyG(YC=0Z+kLIkmt1_cPY4&r#6*%z=p`TmFez(RuT)AM?W?^jIdufv3Ui&R zy{8TwV4%*y!&;)&!wrlF1)VwZkC~ZS4aZB1=_;>&zmFe&zrMaAHe|q4brF-3YqExu z=flTWFo`wTjJN9T#8sYn_huWfIz5MRZ2w|gpC*pkRmd*LY7%c?eZVc22F^RwmO}R< z*Xnz85)v6C74(3YxzyPidmXtk*CbY@oE=~jyw&E0fOtoeRh=`wGOcxi%&F;e6cxT;q8(>lt=-AM$Zrf|Tv;ElF z!%l3!a_5p z&H@MtmAnr(ja3r;*vaTcHj@n6ydBmilaiC$UlB26BZP&g#>QwIH}mhS-k%&Ax-w)x z*C1gNm^bXis)+;qj3_V8!QWdlaPYuu;O@PH5Z;=yddis0!^6gJF^l4zKJBY#F%9u( zy*4Sq7zGbDIs$6PdRN~KE_wCQ_5A`3FV>gOT>pi{8cs}hxx8Cws(&9iTWB?%WjaL5 z-Q-Q@X}36$+@j1`iZa^Td_E+#6ia2dcvYVA-NUe(06cqJb1b$tdqe|pI`+?lQ&Xw!!WE?SHsFVx-1!W5 z9MhX`rVBY2P@lFo)4$(QV@mpv*y%&aDLvJl^C^toeU77ebMz^0sSiC}!T@QDkh+l$ zPC`d8C0ldVgu@8A*y3as{!h}ut?vtH2)(eAat!FRjk*{a7_9ivaxAH^KH9=CDanF2 zZOz^M{T;?z*__`LB*dp+SH`ccW>6`<)4BFYMzU6ah*vW(OtWsine?l~Bv|oDa{M;O?Q?dobk4jmTrP-M=9Fq5On2ZarL(4GOq8TI5Z9 zXc0N2{3hVhT#wO@c${uzT-&PEA%~9lNnbvi-nmQM6xM-V;r!s}7{UGU+Vz)W3JOjQ zF^iL)H#T4Fq@zSD4EYf}yN71;R%XcV3^=_hc2Hh97Go@_BHJQGNL}G=-^t7*s0%z0goKMRElEQAq8dLwh4)Y!Wijreq$)cC!p_J4K)!q!1wQj-lI|^8&by)v9)gI zo5XCngmy-373{yrXA?Ml=V-&}bE_#aNp+mUD;IICIoT(2Gcwkpe1uo(>u_ctaTxkK zX4rk!_<=f~g}(SmWA6#ywl{Amq=oiI-G)9yF@?D*fnzy8+1cSZBNmTRdveMI?cP%i zS*4Ucc5&9Up|Ll4Q!Wbf4!+PEfOdk?R^X%6+mOOaw}g~u4co7!n)DGd^;B022{}01 z2i*v3_;@9&AiUA^#tojDk)p}23#sZNn?u2hvWEA+YOU7V4EmDKO30|;PWhAXu6x@o z{m195taPu3{hJnp-WL7-k~Z@zvcPh(J-08Sw$_8}@bBxmiYYaVG`zaA{jHgWg%1?$ z+@*?5pX0hSv|lTjWa=OxF8E6qWqM=3efykaq|C?P=|v|XDtz(s*xHh~)x@Z=p5CXy zmS7kom-A~=(S|suEPZV)lRloS$xb>t$t9*iS3}rwH5i@f8CPheAkp3xQp#qOscq}E z7;AoPeqZX2=sOk}+Wx_3CI(#vPFu!@C{p%DRYgf%=~GI`4%>gW;0&ven*exsftnYg zH-qoA0Q|tmypw>3cfb0RnWzEx?&6c}p9n!swEGhQ67NCEovRSwbXykQ1M%H-^Q{Ys zY4*;amfgn~n@7Z>8_S>L-nvligp$U41g{#-CNH@lBR2V>FoJ|ZEV-RW_j}Nm;;|*m zSe3(Hipjb}Dt0bG+`YNlRXTEtwf`%c4U+N{DtEnnDTw0 z@M79`{HnZiO1{cUQIQ}$4EGhnS$KACj)IQ=6azzP%;WQ9K{`^uQq_uAk11LxDWSqv z0C1d6cd_$Q(arBPZ5XPhJay_f{2?r*`IkU$Bqt{i4nDi?8t(Ov9-2!5#pYEC-TU2V zH~`m|FJCs@eEITa`d`F1DlJK2_X%!-b%}+7MKwUh24UJ_G;e z&HnF&Sy@>y{N<|h6RwUkUMKS#O|yqkb%7jgY(qPceDqgmw!uI=@Dxq(A6ld$CS!`K z`H5Z{mol}}uPQIsuV}Fe4-b!u-vGliXebOq7aEPjzFGuf*xFQ5>}_oLpp$^|cbh>o z4hk5hP_gZTzV^m>dW22Whj>ZnxQ^y!PSQDl$xG<7#WTs2KUG4v#leH?){AZZ!^J90-VyB6mN zEexSgT6-bpxH_hK=f~k$V`ha#@#H^Y*d2&a@3?s&eLD>TLxy@Gk7P>?-kLXkqpg8& zwy?3W5fc*|8X6KjzjJnK451q;bz3M1K6!P&KVx)eX6A8&9kft|Cb3A@M^w9pYJI5zfZr#N!Ak3-u{%H z-l)F8N#eKwgFsHEvvW4q$n$#)!%A15W2#P9iNmvCc5gOio};AZte01sCYHa)G%Zbx zF=7{%OqTjxV>ahFY;BLM^t)OOH&HCk>W||)@eiG-(Wyd`qQWxuVIA_O1$kM1Zidvx z?mCGbXw+${n{o?pS_CCWR#1c8u$rKEb>rS?7=iPopc$T<$lQcUbDV3S#j0wC$fp#2 z3SkV@jI;>F1T`c?PR$3GK9J`rv@omvYJL04>4Zog>=JA0{G-f`#oTZ-Cc$xIVdC9U zr<1f3pJH=O`uNX3ozLxS%(tGwjYLvhQ~5ejNqmTG*00rZbLBm6?`RxhsOxL)EJdkm z;ml8eYjCU#YUDAYZtG&QuH1MLkD?SEzZA|C`V2?IYb?6^_zPYquVPYl+>w^kQ-(%H zxb&EwiH?nyBm*favvmH3`e+yE{aQNE7))PFh+OB-{_po1n6D&Pm~?+M($b2n5EAN2 zQ9QXiB2t=^%HMFu-eJ-}hHX4kpKtsqWqjiJR|oqE(p*u_QNzI3tT7hbV@OnLYCDVZ zc6aXp@F*!M3Cx?G-en3{@C3M9B9FwnFgS%NEAeYL$BMgBEOB3xpPcHAZ$ttWl^Xi) z9ml)3=4rXP$RHuNG8)T1PF=OlNfUu=1KnREx{;{W?39}&bWTZp>}tnF46s$1ZWe}yhW7UBz@Y&tWlbOh0DmqB1u(0LJmlsObXR*q;!kM0(FsGI)Qsg zWXa39;VuM1PTdYWlJ-;>U)li$SX6v#mF^rld228`)f(M785y>^S~Z+*oTkUxk`G|X z+&Fx?mOz_6?x9G}*hEKg>4v4Zg3j6^mW(pvSq;ZHn(48bYIezDeJEvY3Wl6y>=-@0 zVG&~7Fc2wrFCwY+F~1ZK5LBz9F+f@$~w)l1^0gsi4S`sCS;w8u-&4Gh%qmk!;Xt7C0BcAC4QG3A}GJ9PEk z7kqAV8R+y6E!6EFP{P{wm0a&B6oAySFfr!mJM*UhPRZ70_)_oD+27AE2p2ppe^fqs z+$>zU*vGSuqi-MMaSLv@>qu-SLg_S>P5kaNJmcT0@{eD5ctbhlIReW=^ZbBY$qSgi zefzxa76lu8Ztw)DtL;|f2Yy{ZG6QpTYCU^WlT@LsB{^t-1LK9~``r%_M zxpT6EZrO1B9XWo=U&rHMEx3DnN=Zr%P}c2~g@On^==PwkGBwR`TaekfN_YLGmK@=9 zGj6gbJq3u~A*jKBg^+6G6C^T%&+J~s>bFmK-B%Tl9Xr$YZh)_DLj0*od^gRE}43&b(cw;zI*lxQVa?%AhGSr`$z=I z=D1S2WjA%6RQ=iXwgv>_Bmg{2${~z5JvZN#EBYI&3(%0uNJ}>~H6>{Mxf~JF2!-IF zx&9Iofv>MGMn3gSdSiWk?Eg3-)p{{|Y3iINw9Tt6El>Cl5;6n4mwG+Ds8 z!h__r3aYBA9Neg}FD-YXV`A>!y*r+MOS`~!vNL-d;q1_&gw$f&lK2=Xe8V(q18pGu zL2XIy^GpLpj87V->mcFT>L;&$O2gOI)>3imMMXs|^^YGvmK4R~ zzkPC+A)yy>TtCEMA7u#`~)fOH{U7w>Jo*jpj#3(vS13yEk zS5@fgP=wX!x7%heh)BfR>Z;DOXP0JvBs?rEEW}WWRtM&}CEko9`SFCaZhsu5X*_x;}bp~Z0q|pO@K(pJ=+TyNq_TizSx?qEB{TZ1$sc#hYybo#31yHjg3!?b?47P3jsOtDTX7#_rEonG`?hp*A<=7< zLj!lzSEW#pU17Rj41PT>B2wDi+^ocUgp^bz*H{(wrHkyTp*%+TzS7dyL9uah`2_`4 z6%|lV6RVO_v&&s6Vp5w$^#i_UIe-66JCB3nOTe zjrla%z|cbJft3{jpvU7ZnUrt6@(EaJ;ALUbUSCcC{_^af`E^iQ2;(yoRWLI(B~8V6 zySbgcrtuj9o@f={~KL40sN*&=v`^_FN7u-6a6+)l;`+gbXcuz5E(s!(NE2>5__I|7x_~iBtP}L}mrt^zpj{SWy){31LTVF8Ag$!pUp`wcpYE;= z3|RB>CX~CjK>^y!5%)8*6#hH<{~n#{j>|j#c`S_wv*A>vK3|B>4nMJnU@bA_C;lT+)XGG5D*CC&%O(>x;RNEvU2hCUNWY}xVW4g z|HpH)Sz=M(ZTtf2Tq?e0p+|Ya_X3UW(qfy+(!?K{cz&|8R0Jd#D=Dd?<$cM(E+z!B zjwD_du)GSoPiE|&0k@5)I^Vh`e9_Lk6UJPl_k6M9z`7t2b(k>pS2qQE`!Yn1TQ7mR(Nkde-YmmoZ$OI_kgWH>r@uM)H%)q`;B@ zd;90kmdeonevXPAVIfp@zp6+;epu!F6^-2?kDo#gf;BweTkK!taYP56oqFpjoP71!OX94Lsn20+8+zi`-{&u{f^hbv=FG957mPNCL;d}3 zIIhca6kQ`F2$OK@P}p78G;n|CjVF^iu@}phPoMkr>9ua-1d>-|?P^a`Q>IQ1P|N3o zpdFAp7H4T?bw-`?ubg7{C#Mk3yp7155nWxm8+Z(@h1`H&wEj5MB(Ge3ad82NK$15w ztAoCI8RK}I4@d_CgU?cvmmdA_%F-vH;dPM;At7OSuCul>60Jl5z&dPCP&M#PQ#gx5eC#rIb!FMz;_O}-j@HN^NF&EFGUO_1#_ zvZOYz@7DuXmsv5JHV?8)<2qKUgP{B{MfhiePhxZVmx#sax01Y@y$uaxzxx;d3L!xS zWOzVUb6p)bh)fEpJ8M=t1ONWLG&y-rSFX5PspEPbN2chzyLag2EEn)YrKUq+(4%r%ObIH|*xxD=qS@2) zm)+7L=+PkxYn|C{(|MV z=oCvG9iBZKw6jw9mPTlc${fV_4EP-K4<{uhneU=mfVGJi0643GN$TTJk4=_J5G4ZR5w)@@EuWKG*8DV*0B`R;% z*X{ZL6cgf>9 z+NqHNMBiS2da0bP(6hBU&G*G&x21w|?0bTU|C$Ewt!)~9^s#?TbaX4gW`Kp4`HpNG*F2D(Nq07ffBEdj*ilC;)%Y} zzOJsWk6|ySpT8PccfA6z^yG;X(G}n}n{O1D26xG4G;hgu(89$uaerb2vIz{9aIx_+ z^xEPi&B>GHL&c>h)`^rXz@}AP>;S@fhRsC#^+ShTrGnX?;=Fpm@*xV8x^3B;1JfmI z#xUrqp2D5+!m#jg$AFN7gM&rRF+Pv?bbZRM$W|H~`wZWb)Z5z&-2xb)9ttf4+z4#{ zop21?h}6{RkQ>klf<&viy9l9IBu1)6mRLU&o4c*-+pK{U0*}*3q?0f-Ji_?F;o%ge zY_x7u#M00y7Yc?F(;>-*fYYqV_te}+Fs-Df2%5*Bpl&ZUvPV*fQW6sZLca+Q4+lHs zK&2-)Rw)7n_Ph*(feLrOhR@w;hA@ldG&~P(<;(J3EQj+!UuO_NzOt zXxIE49ldniVpI9XRrQi?91hpry`El8|08Td@a4eU%uF+&G_gk#72RH2O6H7?J_q|F zG$JrzaR~{mV}+KJM-CrO%gG5+9XKgD&=Zz?T^F0lbWkZpG}f1&!|f3Q!ELu_2yL6P za@WgWRp6Aq1=vJ4o{yAhJu@>wY354 zAhg2kya1nQs?2wNHnJ9VL9O)k?7ynsMch2@x<~PQB@=ILE|oG|)41j6xC!0U>e6)T zWhcpkbH2H~wsmQxt7~i19hoD|aZ-$y{>;+0Q~4tS=Dum4Ki_(p3>HvIb^FH&%O38M zrxT{m8GYn4i$}3>aXnd)=?1H*=ej%Ya-ri=VP6BZYaiVa4R=Vh~Xmf z#f^CwDJ)r#x8_1d18zTih%k$s1!Eax57&!Uer3(5A^a83;}`B-WM*zvDWj#Qr>CV2 z^Y=HK>duGe;bF*)$``7uw(nk>a&crVZ&Z3xa?_j}k{S>~VM^?@Po{da!P^B8~%xBAGxF2DCd07$hd2zCfLnZI`V2%GOdsGq&#+dr1K{(s>Z>HRj9~8sd zVpGcc@I~EjlG_5`+1AE;;lf>aXwEx(dV2o7Rjym(J-&p#_Fx(wD(Ez>&S`G=Ft>JB zj+n#R!R{i(a~ZBm=MSRY+(V|v$G<0s1Qu0Q zm6+7CC&{4t`};MGjZ<#&H%e=h517GV`)#^RZ(?;h5YGHcU>XGCMG6| zii^Q`HnsT-Crzx2Q+CR-p@4#C*tHcc}Ll1$;*$(i(dYE z3qZy^r&U!}71ZPAv)Y`XGUw)Y&HPf1U7VO`fF?yhQ^3+(#KyF+@edxYO?X#Z>%8Mo zP-Ro^8#|j=_u@5*j6kmu^6aaECWNQQSQsYgKGqE9cb=cWATLiLc?k$mJiG6A)Q{X+ zyt39Xnmf}ycV%s@6kOyG`UgwwL?bU#8z=O~#Nwef=jY?=*{GrBl;4>U0_h3<>wr9a zY62UusWdiz`d$#H{qk>cU+&~ZTE2`jaDE#LnFUYR9AxX_i{S6bFH^3>Y4#3K$nvCwMVIClgj$ ziB|Oh6=%eFPgg1U`@r)MXr6Agjqt-7!%H74wAk_=#NSYf(sAA^n4h!AuLkiru3JN2 z3oI7+oJZHmj~y3n$bTBnTwv72!@$76%F4>ioBQr4CAu(0CI5y|Czrgu{A1b6>Jp?B z6ee9cSd@5}MRTl#x77!?g2kwK1YnKcuCBt2jEqJkHZ^?J+r!;`tU1mzJKN&ctncY|L=#=YU9l<*fb~blvHtuV_j(l)koh{<9i$;*OS6<*c{#(V5Q;u!UJb zLL5w1;sjEp$Ut5lv6QMRpfjk5xeODLbn8#r&7fO$XNgeA)FCq&b#Cf1DHXE*t--{8 zJiJta9a&tesxW2X$w6O-QUBk)NVfiHbKDQI^<2#tFMI|E5vZeAxa41rcEusWUn8iL zZkv5kmh8&A-u(o_^C(#2faZ3}x=(WJ22ggLMgYNOEia z=I^-7am%)_f)r8^j zY=iAUJYHXqgQE>n1YTrIZ!@9o zlI#fdO|K~{vs_#JI^O2vw4xn!>cojz1uW7iirxRn4n*@k@rt?q^^os$f4{Gv-6GoistO?3?n)NU|o{gI2&u~5#rdaT2!wI1wha&ui%G7;UIg?;bx31U;$XiJSj zZ;^a#HvTbVHDd|%h3#FrWJkv%L#UK72%5yY*!_E86+pSygXEJ_^$JqQa(ml&M8gY= zzmzKU=I!5u%j*fmCl7mFx|lT$&?ZD##`PK; z|ESTS9v;?OOZfQDSD%l?k+_~zCT~H7?Ev4sGhQ7^XdLG7NaEuo5bA%9kBTV!`@)3J zw`rLG==pwYY&UV~{DAyt!C6fBa=T5T&PAN0|FF})5_*^)uRlGOb)0>0yTX=9)CW~$ zSW$7hsU!y!k7%p6YsYe&z=R5zce~x;GLDXp9&QMQR&E*~!E^3zmcfBK<#Q%)i|9b{ z`l~0opanV&XdTl%Pj6p%HsAA(;10uQ$jpJCnNI)A*(XG{IryJ)3|ppYFN$ z_06j^D;g2gnIX0hgri|e|9!MCbal+RyItMAui#ek^6ONZGSs#d>Gm@jk7xovzu(+=w~9*yMG16Zi9ZGR zXL;n)?Em(9-&vPE(7)5}5qMGG6F*;)@0|`+cpjqT)zh8nHsk#VBJLXLUMk+{DG7b} z1*XrxXo>btXLT2PIim7Ml+Yx2RifjP5?SCFhy!yCqQtdp$vB4v?Q5H#?GoCSI??Gy zJ8Bx-!=tA1Z%_DA#6w!;Tb%jiy*vpV!*(l5xYuoNZ|5Y}UY^xY`Qhg6y$G6OdioeV zDqGtEor9cYS>*Ng^|U;OA7QTzQ1~w7bV?wNWj^DU%?v}M(H)t(cGlLgMar9oHwC;) zI!C!FsCG4@h&I@=&}%;@U8yay{<|EuZ~+OuINhgSn>2PX*SN=a9Gj~GYM6@3MaEF{ zA(QsBvg+yt)j~@sxLY&T4B6|;eb6BIF-ZXio1C144MgibR(+Spfd7{h6@3dGUchCh zORkAgnS@&*w8H!7?-9105VR$RU2SQ;CcQPjv~&w<4rp~xWMyyYHOG!b*fsus{Ut-Y z&V`gJx}v%Sy>59ChSb& zjQXY97&bFABk|xt%T;k%E+%CbB}BrWVALMUor*@^=tYy&BcNkbbeD2Lnc>0(tO&w( z!+rq^{1dYjB_+?VgxKB=^Ur9D1c943X@zXFbPlqxB!EpKFgSQ*aFCRSr~VaU36Fzq zPm-8@$B!Q;K1lk(Z$}fNrKVQO^hF^&D?NTyRaZ;dr*vF^-4&o-OnYVFL3ziFqg_d< zX5n{CNTmI>qDesQ27$%B;TJ2CZLTYxeS*^aeB?d?>P!tDlwmuGB>BevU8QL{cng*L50pDfV zWH}VGU(y4@6WB-s|Aj9wTbxi_C=@0nx1F>HJ1mYuR{$pK3x6NEy`}4(Bq6IU@RClO z>}ZGE#1v8a?|oRh-PHW5nhz+py}I)Ml!*z<%wB@rtK8euxtB+$TnKs5rQa#>N};_D zuCaEGa%VHqcHt&srYJvpnM(0jIY?7M%(OZw|JMFjB;{%73Df@OhNPZ|U{#WaH;qO; zeP>O8f7|2Xh`J-6CO(jvHsD--G@`qhO&6v?^@KLG{r-O(#5H>D2}^=MZ8^E6qfV;! zmby;Tbg}GCQQuzWQDda#YcxB<+U63!opR}qd+#rPC!U$>C{lXq)wtZDu*f3otErD8 zcS;r{IM6i7gh^R3vm>bwEg4GM1RtI7r@cdyt+01y;Dld@xPIcd2nQ!#sp|6!rw#tC zt3>p#o?7eTSEKscFIJ~jU25ICE5XKnKlf3)1e+b5yTH>O(}&zceA`v$?-IGkVD3EB z-4gkqH4RKX+tqUKUnnqpK9Ze}AXjg^>gLX~*bmV}sBkR}B!tueho$Ljq!6qP+s2i> zj|lQ2;VbwTvpKEmtjq#=o=`#W_JkdS5CrE9)OG?mgCBN6Z092a6HE9C^bh=2@Bg1J zsN=nF7dpJo2glNX!2~dc+vWKJ69Gv4K^%`bz^pp*2e}m7PCf8R!ezi{&8B5$51Ck$ zt9nK{+6I5CcvFNW3EDVrKtDpX2UAF_v~h8DeWJ-W|DqpeaI)5oha5<~sPIobyDF4b zc_{H}YK`SSurT*l73{=g>o>2cvt#>XUH$vK;9ezpH@(H!x;C78>p#xk@*T}|s@{nt zeRvNB;nxgtfMl%oV{-H>P=BsL6N%93w>pU*!jIW2|21k@pfbFB ze6dx~;KuD^MzjOo`jjyxHgl6na&~+-71oy9vFajEaKda`KOXpa(GB;1uj=l8zzTju zU)6$}IfflquYJSD4xVP(D4gyE*QR>ybo=)yf_qesRobZBN2Jl~y?xQ$L&|n`B zoO5weXF)-Z-$|}-rU0ihY7i*C%+!FN1I?0*8G6g;zf7qA-o{<;KaC&mp@@WBvVQ%u&*A(VASXpELQ($7y7`Pv(h22e*ys(oo$YJsc)8Y`WBGWTsl^ z*Doj5m!NgVo=r5L8F=k{&_*?XwrV7@>Ct-EZ`_X%b~SeSTQchT`@zKf`8R^Q2F+{S z?WQod7kUfD+4>9&g=^mN_FWZ`l0-WA2gMk9@r9H}RG z!$5~C89tSfgk+PKm^f(N|*hhC^}%Z+oy|b~>JnF&KrpeD!L*MGo6#y$NtUb{B9jt&9-2CX-+lQ%g)Q z7Hw@3Nhqc>#w%>dR<`UD02g%uNW~6gxxQ?fq?;8S=l2;j#&S7VIk#(^_8-_u9oZb zKlDySSbtGo$oZ+^1D51Dh+qsl-kW+zUKd<$9QRe_*-nqm-;wJ|)5o3klNME}&yLdY zn`mYU2O}n%dnm6++>e_O+Bl)rL^))uwR=MB`*B>|P^if;gh-8_dMiR6xfyAJTUDAG zY?0kqkFp4CvA~@_PxqzVdHnl-v!;$P{t5Fom`g|TY8SFn$E|x3HLR~+8-vq(ZpkFq zVeKow@&dS~Z5D75utUDLH|P8JyD09=yfvQnw%Z#4}v z#~Op9wRply{8in73UOW|>iA(_p3-?)UgAKk(@A0r?o2|>)d_1T%b=#tc z#r!c`4z>)G;E5#`WyI@$uua#$uGb)T4t$q_lz-A>K?Jtod^x!0-IRF;gZX`E$8rtY z5?$Aq>eKcA|PZI|PdZ%UVX+gd^mZlF4 z<)4JEzzsd=UhBGDGya^Mo3OL*&mF98svrGw_iWm~!!2QX2HXFbnL*Mp1JR!(J7drM zdRgP&YpQMyj*Kwyo8SM(!1egQ{#XBF|KWfAbD09in}1ng8#@*FH3zD;9kcKky#*GS zq8`{P4*SIao?Lp^!Y>GZCAS6O-A@>3-M*z@xNT~csHm)TmrVfmPFI)m^y!(uSM^WW ztHRj2w^zlBiF{)q^oaDB$^%Al&9bpc2NXm_fnXBwcf=7x!pd{pD9BrY7MImP%P6zA zL;793IXkm&a4?Io>XRoICE=YCjb^@o%F=ZC>A`J&;Qs~q`Ok)a2Z-)Kl63vpf1o4Z zdu;zn7wq=;dq}H24d6V>#fufU4svp4-m}_R$hP?wb$hyKj{G6uxqsP%?uTSeLaQA%e%12)=a5G*c~(6k1BL~T$C2o!$!aI9R(@YU$(=(lg*3aIcs z0x84pA45ao)d=3Se6uY{a#nrx#5M=XsIx35mFU|6_wVu~RcI%+guFcKs`pf2B1J3n zfpt$k-*jjrTc1f5fe4I@+`?m0L1tQtP+IUf&3ey^x{{a%^M_gSS}=Yd|Ju--L1fdJ zLio|ro~bm@@Q*n)x(?o%;DVj`^6n@owfsi1W-~60H-vBMhhEtQ1Pl;Q>Hf>W= zkN5AB72WhdXX-Y=8#dCiv*~z^RX{O*t(4W+nYI}e5Rf?H7ZSn;+w|a`;XN*-bo^O~ ziT8E&Ry&iYsdcoq!TtOnb96G~@0NeN4qmbWX$KmzN~CTB3YYMZq1+4y?8G!EfLQRg zKX}{@EIxXrGLV>?f;LG$!|n z15PesuY+@Jh21Xyh&&&tg(4MTd|Kj;1r@8%CrhMtyI`QJ}Sx*Gj1gi}{?0L&NjL)X(@M1$E5 z=FbGI|JSdBzq8Ri-RAObY08=s`xR3xlFbjD^ySQ%Gt9V@>50Y&ApwWg`KYLK+MG;G zVgmyml5Fq_3w|li?qO6YxTZ~81=9eg@50$0w`mtJjiN#!AAlr3PyrGEx`)~XcZ5YI z?1iam0leI&wcG6J)2CRZ(2kbG-9xY#3sGmThZ}km_1|Gcm&52E7k$$8Pt1PmrHV%^KYEp}8Q#85nT~vOKZpPCKNY!T zMYjLb_jknX<%>%54^62(cD#v%b{(>6%uPPKIL04qoo{KUL{>Bs(rf51$TX^|K2~_} zpO;4_iN%PJqpl(yCXzx%au|T6`fL1Wr5r0(-|F0UZ*a!d=>6x&NC@7`Hl&>?3dmry zfA;ufjeptv|Gd-xFNE)2LHL`U-eqC-xRpO;DqSDS4aI{)Je-^cS1HlQxfjk0(*0>1 zj}c*v`KfDN|M_dq=|`4s7wKc8$&Ie9alk_K_Vxy&fuIGxg;=wmVsk8!u;=vM+qVD` zK5Yt$?~oOQizFdfeS!O3UtgD3cgTj+;MIwynKf^j2RI5l5$5r@qnP1;pKYSuA)X(% zq!rdqM4$l%~9Dk{C@B6B2`)2zXxwyuug znn|ks{{1myhE3l?@yr6)SdnNkRyce5^y!n#4?gf%+t{$F6)DUe8C^4s$_zvG#0oEk=Jb8>NUiHN9o%r7nVLD7km3a*%AmPXBYn>8g7-be;zLhPf?PMKip0poVL z3mJ&Q!j=>5U@yo*Uweeoq51OPnC5P_t5@xRdGYQiCYDBvWj!S4gY6Zxn**L4)WRGN za1)tGF~&k5C~sQ0Zf>l>Cf~V|Nbg~lh54}`cJrf8mum?=nqavPYM;XHKwV0uE$<;F zFeC6kDE`daI4E`eW{`>t3JTC26}BSMPiOo|MUPbzaZP z%BnnbVz+KQK*w#@mgt4>U|~#ah$4m&q~aKHpDQnu;AJ`EFdQ9K&4TD2m(-#x#ko`Z z+4>GcXZwWq_&IR3%p-y7#~MR}_RoO>?t_;P>Ok68NFaxu(r3=-K;oL5q@$!v$Zq@E zEV{{+-2^lGBC~0`qL^Rqh*bh1>LqNsP)c=qis4~a&17`{I_w}q$HZu24yrr6%SlU9 zlT_UXE0?%%WR$h~#S+2xx?_EfI&P(*y^$qMG54QkLIukocdj7EVjRiB!h$VcmHeUa z3}iXKo&KQfzJ|)Od##L`31+5CQ&=U~4LCxGajqZW-Z?)V(DM9etwAG#fzMP_NX2js zEM1aCX=#t7dDqGHEhw-j|CQ{b9$RW0I&=u}QQ!7sVpF_yD1;-x7jPXRF^}L?RPpig zYQSY@)j%;fNG)m3>O6M*_;roXL=wlfq-11*Zjc!ywo)pdRjh+u3=knqKCO2K3v>0c z9`O$&Q&QlK4@dkO)*Shj#=z6o0$wRH6&t-K*bFc?eU7T7d~JS=)1WmuBjZMr{0+?5 zYVG$9p)8qy{*8`%9j`(D0QUL+F!vrnQD$A&sA+Tz7*Ih$MG-InVj!p}n3Z5cn$RRc zB?m>x(NR%BM3JCKP)X8Fh6c$hpaKFaG&u+oq)C#Kz+Ddwiu28P>(;IMZ{4YydgrC{ z^PF?`-fOS5HnlKudQeEXT=$lIYbPIH2$(@g#b(Z&30suU71Su<2vKZWO}J0o(SqMEM&X{!j4l*C?|MKnFN6o`DPp}4*0&}=560AXw>Wc zLZ{F%vebJFGSiC4b)dy<>lb>0o&@w!26pqz%>THa0M3^D_4DG!->o}TN74GfPdYq7 zN#ysV)8J(0(fPZnW~s#0Jh1=Aw%_S(jQ+FPyZ)K6>MtfVYH|4Mm4d%#onOK1tfDap z81nZg0b88P)09L`0dzjLbVeZl2%r)j#eD^<56-wZYbt~R(VLe1?<-sTiqh8oge1;i zsBwPI% zG=3X66^74YeqROGvUFynkL8-34n0O^r1R3}pIgt5c);)@NTENMw;R zU6f)e;3hL?dgzm$7a6{_;>YLrcANMT22X1B%p4)JK)Gyr|JQY7lwGHXK`QZjbn)r+ zB{QZ)7qj-)79o*K=4H4y2(O5lo+G@XF1Fp1xLr~!bN{f>(O)FsaE?FgDz~dduk^n{ z=f4`e{|2#t>~7}%1$Rz15q`CLziKjG1G8N6_2?%Rh^{NhhxnH~;wNO&#{;qSe2j=J?Kt4lBNMp{LMMgXf{S znVB6UER=Qm^43HW>LJjId|DR#5tpk2o~7^8H^X@cnePC(ZwTRdv-k_)>_8SPHt>zu>!Z$PJ0VYhSS4R zTe}H#te~5}E4Qn$%KM{ZW95F9<>%+;QC+`m&YZj_de)VG zK38(QhQ=MNT0hzaG)pX5(ll#u)V||=VN&y+fvT$=`|}3&sCVo#@$ZvsIc?G1AZ4~q zh88Ek@R3%jb$-4{yw%Y;LlPS|y~x|{#$4a`oMxCAyav}8#UYr#WpO%gx<*9e5MQ07%6=7qRHdK^WN|D#{f@s%C0_}KV7_q`2RFF?Vz!1)va;#_DVUh{% z>MD*1rSLV}A=_?4gI8f4)@wiqGy<49PF-Av7JZTK)x9J2bCUsGj?6p!dLJ_z-^DSj ztilli;pxMNT^%W%jE#p69qR9>rrvT}vj=XH;=G#N_sL3G@-2#>a)EhGyA1B8sR;kj znthg^qW3|Aj5Xa37$HtO|L5o^mrdIa+BLffY2H0gm1yh0BSVXSHQUGc@86#}hPs(9 z7cZfbDc4fS8^T{)rQX0)}*dTXv@mJ+L3W->|~VbJLH;KA39 zj&&C`@nE(l#P{@UJ43-eHJ>~x#Fl$=LJ+t z&YqaJHl1muD5ninC^KizO&zRv@mgFe*($K-qP8};ZrBLkz$!fW+-|spazv6;V)R|1 zNIo8!^-!g89Uee1s-!XwXhf%7(hLoKGr#Im~)&fPBYW#`s{fLdPT=$iH|RL8hUj9UvX$a67&%)cWuT@PWbX0LLGkiN@mLjK9&053bmcx*nj6t*vdA zLgNv$ScWIRqIt`gY~p%zi-{?=X|RIQkvnj6@)vxlNwXPRxvFWh9{s%3ajqf1o8*`! z_A7^0PE_k@(Oi)__Pkc<#^)E_vqwdQy?o1*a}n&PX#H2#SroB^`-Pl+7l-_BAPt#l zU=u-|MNz1vbq#k=pi%qowlaPyi+AKy*1BaT$MzY;+e%vfY@{?)Y6XtXYLY3Pit`>b zSXRw2WNl^1zYv@<`l|f+#agyNjpV7EKWGle;n~_3E~M;vYM1q{Wd5d_3Mld>I73Pz zpB!gCGnbCD&|h->gID`1+UJN)zNjnIJ$CIwF(Gm7@i_{74zOjt{J zWb#{KGd`O+d+H&e6LUd42;{6NFBjQO_1Y<>I-a!dmdPh}BKv*FdVcRbL~48GJ`wLc zC#}~L(D`iQ+N7)Wk#rOH1|1A1jk{Mx)|Vx((~_}flB)0xnZ&$_J9uFBT*uZbmZOis zmTT_p?Cj_usyzy2t6+L;Z1?BmTz#-f>h1M#Y$KmG;eM~L_!lZJ>~@~xP(btX@j+d$ zIwJp^ml_uK1oDM|fPkk@D}U5wLTmhSVfX{)%*k>;O0iE9O(kmF2aGr|yNOX@#!vgq-HgaE$dIscs#sOs~rhshCp znE?s=eD9U|Bc`U^cb08jwB$b`VMG+k0n%Z}ngKcN?90nw^;&guPU8>M+0Na$r8Blr zlD6KF02jx>@)l&{K<&2P^%!(eJvfV?*xa3y(fdUPRK$_5HOLPIG$w-DMden z$ODw3A^7@=BqgSQq9*ERe(_mj5cZnSii2$ox}=YJc`24%^*}Os#Vp?5obMDGEq{wF z#5Zw+X9%BnRH>#Q^hVsWfQk0frO$4Sj*gOS5%HW8J<@@_o+FbP0MZ_>2dvjwlSUqd zSuzwIamMtW-yFJgHbINx00vlt#(e%|D};*Qg28p!++0-g*4MA+V{8Rg6%_bF72Z<@ zySq*8QrRAu{0&AhS*c*vs#Pyoj}r zD|$pSEUXvGJX4O-CM(Ty>hk&mYZ-($7&^c@prU6k*3*uCpK@}f#+ClLapN$~Zg?3e zazpJUm=ylJA~y*Pxu9FT>r=MI1X~KOC+o?b=vmaNPfy5F*&3Y__R6!d4|5lIef z)o9(qxVB@G)HQQcO;?uO+{QIXjw$Bx% zZRiR{rs1vzbf;V1uVD@WPoF%j&N1|`yiRE<{q<{~o=G9WXj#8d3M?;~{gQs}?l1hQ z_nn>H)mmGCnh+e*LtX&^fRas&=^&yJg=%=X#~g^Kw$@~39@a~KfJ(I4bJdkzFsDu^ zG+k1-eIgAK`dXTSV$QjcG2Gc&yMbzWPsE>{)BN#C;JR}i;8&eHy+4?>7D2xvFb8zm z;pPb1YyPNI?s?9s>aQ->LDvJ%=aTvWyZ&+syWtaWdYACZWP3K1&?zVH)8x-Ty1=rq z^V_$Lr9Dqg(jV5dfT;Fm%OWcC)FBRK{Y3q~Q*P)$P%Z54=<#maWTSp?79(tlObZff zh$lPR+CX4~W3iqwJ?0B^^XESdKjIB4qQ1Je z*_~pUTxdO_!pa>09OH34Xdrh)N2)kLG?zx{Tf5$=i|;*oYxI!#fsYHv`zuXwOJnqk zLI5bpS;~ZN51%y=m$21lysW0QdeF^ABeJM3OIdoHabMNbS$|6EK0bA4f**+^2V66D zptNN>`XZMQ@DA8h%s>(41j=RVFoO!i8G;{1ykWBD#_&nPukPQn&HsuuETNn=RV8P3$+WgMpD5Tt&44?J)D?4l_lbpnfpho!{b|pScJ^&cvv*g!{lT@vE^lCs;itYstBETx zA#(ldWRc16`Q}pE@m<2&F79y~-$Nua9Pxi}(RQUgCnpyqxRrg{8F}6L*Ovy5t?wD9 zzD%iwTY1BiPfN3L8L@9skhiRv!8j`xzIpRjSq)o1dzEF=@6Pnp;1}XCk*NIZDl^&D zC}gl+{&;9onM?35kv2?a3datg-y8ZnahS*{UPlzMw70b}q@EaU$#G5O-Qu}Ga-=N| z6!m6kt>h0tgQL_^*nYo#b#%lqX!~|;G4HcPU|{hS5p=9k$l>mt zJFgPv4k13K!$0|?UY#Nxla z&@2;>Ke(r_Qc{pH5c4^jPCMQaH}9w_M=dTa+}r4_Q6eFyB)t&|;13Ow4XD%Dor823VxwhDsTopyKji2?u-p3m3a>BI5*XO<$D37kPj z5F<2@@H7gLf5IsT7DDxPb;}@8LAQD94a3^>QPg?@PlA2ba-ZEo#NUsmZ_dG7hw;$? zhq0mN;cRWQuNJj-o9qS52D-YGQ!QhNQhp%HVN}L1PYy}{+n}=H&bdxQ;K!aee&QR6 z0v1DvvqMCL1{!sVj23Ue> zb0lhep|oB(@Xj%m;KtSFg1cERQT)$6J^~8opSuS(KU?my1w4GXjEigB4L<3}?C!a^ zoa@-EZB_U7uftr#1Oo@CUi&(#Ra}yQOab!_;(1YlU*re!`SZ`)m`RZq!*mcCWrVl% z+7aA2iN4}a`l~{>lg}FSgmZXCR zuUqYuQ7{VQQRt5O>;)CA13EwYbx7YRo@y#TMPO94LSv!Jv(|C)6~KX$fJ8W%#>f-O z_z7Nnp`UoDoX@_++x+!kDHvr_M8^6PGlm8SKdg=@#)a+W?|>n43{qe>r?Q`T+lC;g zm7mARU3Is)h@R80GXo_7I@WCJ8~FI%RZ+fu`-bds+bOluB4PdMfhF+d?U8 zKpI4^>=GEF`i|n*8-2gB;lN6v7yEQStv@_UugP1sdUaU4vz>GdjzZ2Yg#Z-grH6)} z%MOgI0mC5DFOEme6gY2b;+#SP7$XN2D7-R5>+UFXXcj}~HW2$BlhK7m71m01wf&0C9OVR9X<~K)5{oN@XCQ7zr1$R$& zjK)4aq9>h}uGh<#n7+w=pjn2?;SG7B;GM)BqDf(;dyb5~I#Mlctx-HUZa1>Da3VY1 z?l+r&BAoUD{0Oy)ZMR{thvxec9QBFsFacAzNT#C<8*+F*PuS`6nPP6Az&S-d7kn_wc64E1XH+vw=u;@u&hzIDRwpX}|u2cbj;K zC{`rhI2QQYWzR?fqgo~=fBUHO#Z&kUm6ai=`rY)SU*G({2w{v z)I<2Slb4V9dH0-NBh~*GT9ryPIyC8G#Xs8Gj2lgvr8!=NI)c%vOYO@~_!U2e3ygIK zb|J$)=N_?X_ihnIz z<8+e2bsrxeBqQjtz#B_LkyFV|Z)WTcPbryo6a2IjVf7h@A4>UDwbM zbZcM&jcNc(jKT1E{MZuhmr<>_z0t|Z_XGUr`;R`ZgWvz9cw? zqV~y-3r?_I05ggQNW&-$XW*~fR$tG$di9nk23>Lodp?3Xe;Yb0Xzn3%(uZXoSe{~^ z-H=w`r|H!JuGOEJb8aj3*1SclqVn#I41ITupvm8(}PX$Ia_vCuIBZ8=O`yl9cU%;#alqp`HhR#r*mMm4Z8(7d7=PIyr4m*BMDmu^P^=#_wxJ2Q*-9bNsCwSC&4Fy` z;0VG-ovnXhK&((TT?WL2iPZ6kzRn9StUFJkRjigM0n$=aAwK1+yS>g@7a^SWfzaIU zwSfw&J$=zO8P8gN`a=e5g8Hfb3JhJ#lf_nobbevsV%xJn8Q~StGlqlh_B_-qK)Ol; zMEkEFSPIWe02*+HhS!CW*rrX#-`1D=%n{T3lJ$x56#k}X&ua3jj7GBUzS<-Umvm4vayQ(Zzzj4{lqQ%z31mgsG9B+d|}SiZ-W06g24?vDeAI|e@zhsy8Y;CGXA1rhieFkNjE;@ll_<-e^*vo2hjbC`vEcZ` z#8NKS6)P?>lfW9Q=CwjXLVE&lVpr3XX;Nty-;EbKjBOkP1R9tnvrm0tz+SUlFPc8T zx~{twIOV8X45|+d_qIIp^Q(?)cDwY*A<+=y{@6q9cv9q+^&D<^(P9BgO!qb$uPaj!m- ze`r75-!&vL$Pwi4wHf38{I!q=z~8#%%Qu!gTiy)<=xi|mZ9DnyCk@bbvxw<%(qB8$ z2GrY^)nsKI=`-vC8=!dCdt6tN!8(e%kO^o&a+p9wJsbn4UN|qN{-uN zO=CIp)JcN*T>su-Fq4Z%z`QK>l1%#sz2yyBTa)&IJCBU8(OC*w<95`n13>cu0P~v( zr}AxGF4Je-U#`VXcBzmWxaH9UGcu@9vOu2k!7jHY%jU9IU`0+_D zSQcv8`uq9=;F_HakzFQ*_M*R}gS}t4H3|pJRk9k-t(DH*0zL$QjO^XWWrrWdR+@b; ziFn%2{SfhMw3QD%IU#!??w2(;HrVe2q#>&Kz|s20YxfA_n$+@rJzVyK2e0jc1SdP2 zci%qS){=;(x!p+Tb1n_F#TD{ku1kRQAk2SRAuz?*oY^eFjX3Sx^QeCCp&6o;s-NQ$ zRDE?BVM3Hl^W4j!zI-%Kcu^~F?aT(JT=v;R>JQ;Gp-x^7vOLi>nFvqnX%!7?*Ejw3 z;59K&Q?A0jA?gbN%t~-=4MAPP4qG8RuIM@Z`?4XcGCm@UV?7#W}Uo zBI*3DCNnT#(H1ockA?B6uDW5|F6`E%l$(4jX`DSstsJ1#9XaS z=p}Q>QFAZShj070&c;rO>;19_$0j;>s7uet5RhJJ;`cP#=!3pfX|Yd_iJb{HzO0~D ztNZ!It$zJUjX$1x-at&=rHFHXed6E}_s9IyViu*aT9d~aO7In&?9y;t7E|r4;!lxk zu=ZBU4|S+o!<`*h`=sk1ocU5EU*C3{3>D?YJ6vLHk}cbQZ({2p;rr@imgzkEw#7hz zzn#j*y(0nP))(?#o%EmIXtOw$nf@XDBD#cMSf~ch{%T&!ckiy;Ajd5x>}K>yYYKrx zPWU>%&1pEe)pgw{$II#P;F%@Kb)Lbk5o_0#6(=8z{Phs>=mYy1PAmU9daK1H-(eC2 zWL}w1A5tbcIsXCZ{wp10HYBH-hs@S#;D*U8od6vFOr!tkB~E_YtK{c6xM;Bq%z8Bk z?8b7G441O;k=pTfquD5QCtnl2PML6IhfD#LI;V5|Z{%+_Q4UW$>%fgRC$p91dmGP{ zel$2~;{MuQFan-zKfb;ZJK=xtRdOSP@Sl7A>#o7M9m8~i0wHCx3KY!kV7DsfG)dio z(70&TL`?xe@jzy)({LlHe65InG1u6Sd3`Nt(@9$Q>Iy1@t)#2_5)~#e8gvHG#$kOd zX5ycbk&%)zQWkRoP5*ey`S(`uI>tSF)+~Rkrq!PIpi!dz!P@yN9qjqCz@Xjwf1q+r zGfanu2Li(09>)g3>Xy{y7sjznW@^hAd%?E>if=ZLj=J5T~h8S5-2>}7w z)P68hAS^g>>eRX&S~u&I-&Oq8+gjQQAhe zu+E#{vQ^uErQmZ-=x@Qc05FA1!80zc6uf2Y)-P{C=K)rrs7Xf?r{pa{1#J30y=6Vq z1 zU09WG5)9sPP2>3S_ZX01Cku!SF@Nt7bCNA+J;+XayS6riy@8}hOiZlZ$_-TLRsO@G z;`2r?Y%ty1HwQEj*BT1~;(DoeML9DNBR~OXO-pmu+O&M(MTXO5RvAF`7q^J)|8Xx^ zc(@vxQ3RlAOC0+Kb<%J(j%IkogN;r@#xEpPXiivupQ-I>lhH)ESN(@eQ$fFX4UChkvT1{LN7MY$s^TDBeiS70GfiHT4<4PP>$O!8xhB7e|CEHw!( z#lOzt?@oQaZOi~7Hw%440s$cYg_i@-)%!2(;Md3e?rcr5;o(;CuWr+f>K~i*bGP2# zrb-&Ekz#frCpwD<{8H+2eA7mb2OD)u3*^`2O!9Mx-{83STBc8|^NYWtj5>#-Z9VK% zdTsbc#j*}EDacHY&eI|mT6POdq+NoUy>8%fTk}UR>w{Z4zpp3QM+7E7#AlXvC;cnd zk1XmF#E*q%2oAp|FK{$U)BmujI{Ih(Z1K(~5877+%UZ;p9?!2C z@ao)C_N?{RPNJhb`OckAbdgSp^vhSD2ATCQV@=B0n10Y@dLvLQj*}AHYB+MoHR2oR zf29Y${f9+qr*!Z9$|wlnrSB!D!ip6uLxj>#WkfJ9|76GaS9ZX>%fvM&8jI+o{%3aZ zM>m(rI{M#Qi`)H*rneIhY&3WcCT`UV0#PFt!$dX&O876b;`Np)?NK^r9u4&Nt`XKn zOt@{Iv_nNY@Km$(-OpEN%pmDW9XTiuA|pF{qQkg-$J(imBy50j60^Mw3xhHpra^}+ zEeFkL`hsUvRSA6#NS9YTr0BBiQ=m~d4Lf(5BB?J~c;RtU zu=^B|j5NC_NrYPRt*I%&pmbZ$MG#jpKiWxQEvPpN4t?d$7Y*vb%AxT^NP3@$cpS;b z#mX8Bb811`h7S)hluxe?dI^kAAO^nhJpuPeWuc57QmOc8DOCpsA$B`ZS6p z^O~sUFYU^yvi{?Duu(;=gl^wyf;$&dB8(z>G|3O;jd~ld9LLSg&EuMLl@7(|rwoU2 z_+p({kL;L_?^vv$kWeiuUR&E#uLH9tNvmC;z$-k-BIr7VDbv5`I{)J;R{4-xYc~k8 zT77*zvpo1$KV3=T%C$NQW8jo%Z*F&{>og{H)y=u8kGmL=H@7@ZWtEAu5d9}ti$4AFS94Xb zJiBb6l{;MdqOk~s3y$0H37Y<#@2Qjhn~bCh@GRwkL<v|po)3X_ducau7Z2)JObm=(%`yjgl?ZCpKP85gkuNb2Y0=Ras3d;G_*$n*dh>ekzo&iVOPSe`k$ciDjXL%(7ZrA^Zu5Ip zuwhorw4-lkWb908a4Se5oK;H=X8dN(ex-0mCtO=QQpOi`By0N57(B}I>cz8{+wHDw z9E7^#_bVppJxnp%|7#v74(ExWG!3;MW?My2JdB4|!xUiEiWS$)c#|R ze}Qf$9=HI=I)of*YF3MiG^o;=#6dsFI5jp5d|q8xR#}PcW-*5_e7g^4s4i&DB%7gG zhkeodd~l+FG4=GD3K4(l>+7S9w@LOfhPl=34j=aRm`T;%JMUK%>7#Sj0Aj|Hi~j*_ z297#xc5GXuc!UOCqf%iW=q0E9_R{|Yp>IZLXl`oa@E?An&Qpd|*9oI;pVD;o<@*^6AnEVK*(EBbfk7o1c<23ORGg-&Q1?)AdouYL|1G9tH zBp-t9d%RvzJmG>TW2>q8oXuoRm6q83?l5 zYmQObW|ug)T)uocn)r-_6;TeN$7VV4D$W;B)6A674{Mmlc5C$}ErfZZJbx6v>60f< z&>@0pJn?zKpx=x=fM7q9#xS@daP$so64TFI>1lKbI#;1_>mY$EzC=h?rm~AyH#(obic5@&^%0?(|J+ zuUBMr9WPDZbfRO-%(eR7QdXjERj)s9@tRK(I{u zv}dyJ)x1APCpQh&)5JG&sdZKv9PTM=3;BFUgaFf+Tz@`2m(N~uj&aO@j@~b(P0yY+ z=@>^OoN#<&I>r4mksG#l{r_bndg9CDR!sfx|LIPcilOoHzp={b)8W`SU$Nvy=`O`)D=u_n%8QQWN6h4p&6BzFV1|pfDPzbey{cr{QGz z2h=yk8fJ38zIEbFl0(-~MqByA=g%X2{nn2DRK3m8s&r;M+0ME?E9#N^KPvIBBCL8s z?%r*ek+wIo(d2mXTr^j@`r7n&ajm6fd%kz;R8Ssz+SAM>c~$!hg*1RUsZ=|8QXeNQ zK-e|fM(S>#yOj5_t+k~^^=#KFg?t6Y>FmSND7;&klCzQZx@rc;8csj0`$yu3{> zFS?UP`a2?2#83O%4%}Mk_EbLihr5q_Zmwr-+DKhWhyKP(e=jt^L2j4mLsF@1+bFut zn&tPs($Jv#3x}ngsB>FrqZ9fnxdwDM6JCL_O--T7)eknTE%1H9)9>~0%IRzI0p`Un zLV&cyqC>R5=mg}i+Mg_=&xrEaNc?aqsW(d`Rwj`>NWsVP*Zi$sT-%({Fn`_wU&dAf=q72$# zY~1a!k=nSdZ(R1Tb}cNkZdQ-c%7)s1sln7j?lX&L z_9U2eN?g@GO}kbdbT`fNxkSuki=vRliUZL%R|stWVAS7sVo33Ag7q9mc()Qd!5}%zN}%_bSyMe28Vg zzj+^x;V{;dICG|?Px)F4OQY(&`?ga2Zt@$i`m(BNOR@>~0N+E_2Wl!i}@ z&b4-D(BamS0_?*C6NjZwL}M6W)j_e;*w6rNX*9gN?mlT*!vsF;+Q-Q;7M+fXH5}bk z4Vq{4&y*|QX4?(VQL@OYG#S`^Ra-7X-5^h+!fwopLTxo7TQ;1y(M)-NuSM&Eaex2# zrh`|yc1uc@vT$|`^ckJS9<7wwtVIr`>2pRThqs05g!6LsWD2UuJ(5?Tag{C=eADbso0XZYRZyzYkI!k(5&wQ#&PhV@uN+(7>2T|IFy%HGA)XH z-^?uSq-R#ctyRONTHnXr3#-$fut}P?t(EcO33EI0iOVX8>dKpH+2!DG$t|6(cE^94 zy6?+%l_#WHK8NJHbU=2?1-hWV5ZG6G9R&RNYE&Ms6hOfp8#M5AD z+?h2{dgm1sNLc<@zkC;RjfvW9xI)Ejv}&8GW$IO(LJ@3fYbZA`nyFx5wr`msWJ;)i3H8c0Jt+=NU z;J>@8h&qbm$>(uddQd}YX;g^VK+5&8+eP^r8fD3}9>Oan`{;y4L1fo@Z)uI%x#MXsTt zSC3|2aQ#BjutxvczNlL6p~TR|Vc$KUA3ki4)TG;M+99a@_>t*1?I-W2F`lhYEDGI3 zQM;0_SM=7-*8XSG^7-E#q8ePq=9)1YTI#0_f z(Yp+vxH_%SzQfxz{P&xvN4-9fs1V`1&MK%?Q6pbCDk?56Q0h&wsJ_RW(;;qa%FB;! z@!+@{YHhijX3|MSJ4$bTuLo!(q9su1bfXQQ*qMDHXygP)zzS32)yHKMA4k1efH;|2i;w$y;8j-pKm#w#lhJ6*#uZn7nomnU8kGr#&OQY?aePZgWzdZajvwi-LF z=3uk`64Zfn-7TYYGkZS0PjrXce3w`R^2A_Yqa=1BYUX+6ON{TMe=!^DEm|$-r!h*2 zz|U^*frWhToZDRQhR^bB3h_q=066qhU3LWl2j}OkdbJ$^FroH!aJu$WYH8?4A(k zg*8^8iSnw>!bH%!Ux}bMyv5XLxu#J?@tb2*ud3Xd{yH+}jke@h!r9X{#lZXr6iV%PUX>@oD8q*z;; zp{>?D+qOlM_%}t!*LM&%W-MU*r_q7eZq|11*Q%=|E_QPs?mDP*_n{^Ko>FTef7_i~ z1qCZ`Q~E2pxOTYR?W&Bfvu_Lb*d%a=O=WdpRJkw8(+6)ExC2bYy z!@9=w7g}m5*7-gwm#z99((vUS4fLmqni@VTo2PspRn$1Z!Jomfe#*9P&zG-Bu$)i+ z`Ie?1*5bT!w&dtFC*2MtU}w7cJY(etYtm-dU={Rlcls583zJH0HVid1y2frx@63=a ze;=CE|4~Y$Nat5e_9-b#((kblQ+BStgAhM?(9L$+wbNky` zQcN44F&KB*x9w9ux2q-QeTTI40tE+!?=3C9PljqO94#$(hlTEhPgU46>&U%3x-u{p z>#6esm+D+8F3V?u&SGI6!EYzSIE}a2TjM_a^3l=G5v9WN;k2dW#^te3`q#vnb*&W* z|3;0fF=(t&#%p}6nl$W0dhUy-_IHkdZo6vg1c>OE&Bto#3#+!TKVlR9*R)@GoXU2) zC_}QNcHH07{Y&7Pzt!Dr6f0 z;gIGcQ?zbB==tlk{poLRT);UXyN{levjgd0cS8zsn}*|q8bSw1GX9Z~i7VON;<{@t z^(;8`__88Z;goyCswC_6HkAo=SiKukX3i80wqzVYuCw)L*SBwL)~+2Ja0rze40H>T zQu54|D=6taYTkcFMlx6r^bT<@)>@irr$F{o`TTu7F&8?OVtJ(6a~rK29!4s3gxe<8 zDV}En6_CZL&_V%6jwqiX4<<*Z}ZLX7kWptbr5)sUQL2B9;GGVTU&N35> zGBo`C-be!QAj0SbN>g4IJ(Uk!hERgDG9qE(Z6<0_hJhj78G07X%Cd(+4~PjLBb&oj zysLF}WzF=?oF_hy`NUQ&ItCY_-@iOlNx5_A+xSPjh?q~08DFB81{dV!l9{Cjs^es$jkh=R()8Z7eRBaCt`#<1>&FLDS|tEgLrq9}aYrV2!cwOd9~aJy2@W zWYcR)Wqw*2fLh2=gH zv9O4Xi_2U$aBpXkU<1nZ>uHwDuPrC`_0(VN0*_}xOjiXBI50|L;?d{8_b_5;y{is* zl^;r+krRg(te$B4j#*!#Q+yS+Y1>6Js;a8M+E0Pr8ie@x;9NeSYeWm4_*cQmaTO&r zwD}Qqad0gF8wh#Ic_&%|YhsbJBMKrkes3N413O!n-oSgLI6WETOQS|}7AK&kq6 zp%DE0Fy%zn)&dMCvy;K%?Cmf91cDr;s43Q8@oD#N{ zz?J*lKU@cZ4Qs4|f&$ZOST3^IC6@TWll3Av#>0nEaEn4ovDfdYLozYr2OdMPVX`gF zwfns3!rpn4dTz$MT^!B>aJA;N=OM3NzWnO<1sGz*t(ofz152V0eQY;+74Y=H%Vqc~ zjE@{SqV{O@f+}lOn(? J=Y*z$Zu zI(p!}-#-smfrs8yR~G}Oe~&3S=`n?eDHe4P2jM!%9Pn3(4EH>~31g@aW-=2!aF`7n zDV^K$RQ|Zboi^vek`{sCqMT}vbE`VYNkYRHv3Cht3;wOPJeCO}7W{%%**A1{V&1Q@ zpzqVC)OBmw#P-?_Mzd`L>Y#S^G>yvUZgg6E^JSB?z7jQy>R<^uI5W8Xq~pdpN{2Zt zBA?dN3xgLT8^P6YrCEF0LWWQouNQYIk1dn=wP8U<4E1_@{Qerf(n!BP_hRd6!|*Vx z@~p4Cu_bYQV+(VK&SFiC(J_EMr6mxMp*-VVPKvw%3`92(y)sRyQi zjUIOp@8j)t;9aS4+H=ZLfeO+=Dv77k61ZWSynij{%9WR5O>OfdDqunp_E`54T8XCuB#bN#$Hzw51(eRq z%>*VSs@FveFhNT*DMp5SrY)|Q#}4mE z#q?~fV9>WPJ;)XsS+ecXqeo3mW1Cm?+c_gW8K?F7_!nqhasD{i|R9vX?s-%yKwbbxlyq)1{2Hg0aJi$yg_I^%}JpwKKUl zEto(5in)0Q{33^`$DzZ$Mr?i%CAU2}A%z=d5E1qRsz#T~2Ax!|*N8QSN#b7q)CAM3 z_z4`Ix9_jjNKFWNv0!2v%!IeXu}lL%qZ{cqt($fHbni&kZ&pYPGduL=LZL_J9-!dA z&D=^CjowJ|hy%O+N5{8?rRB#@@Y{*kKfK^FrbQ?ar#;h=RJp!nmj-K0&8HU@r@nr5 zHMTqH6{+`n#>Yu>z`zZT5n6tBm$I$Z9-X+-%3^#)zr|*;3oWQi0hkEWvX`)4_;hn-Jnh>tW9%UM(3h=3BuWrBh%K5F>E6(ZmSgo}GTV35qgf?Al z@E%Q@?r1#cH-F+W=Vdc9+KIcMj$+6+qF?+6W;m7C!W?Iq>g2A&LPc7SDd5=`uzG2O zVzst$5Ojg{+x;2BYs1)eVfXEz2Zu2kA&~yt%gQP|O4sLz|L!Y?oS*-fTraa~QIun_ z4`b4o@X9d6S`hYvF$|~2)bMaw7fej!*rk2XW5fFOUlGkeYp}@FvMlO}XgR)DDvVO__61a*p?W*i{2+1Ny^&*9fe0G?QwK;G%^g|y=Kz#z`Ct49o8vH zd2gWR$v&ANwDoah7nws$4S!e=k8nCeKeZPQvV!m}ZZXAV+zQWdQjs~>`s{5%y8Q&bVJjV@Kw4XNUGfj0O~X*Okahg4bdhQ*ugS-Y|?Cu zlqqadT@bW^A_VE#>(Gbk-CheAcP9pRGnK?3sr~j>V`{?c3C_u~g4YoB*dg}~&FVc`R zvp{#pYy(TI2p)kQQEB}?W>nGeSWjw1VeStl8M>H&W3eJf4FoK3mlFGjSi=*kB1__f zUsSRn;1tRK82-!Hz>5_O-w*|=sDPWG0sSqm3C74k+8AH$>$_&Yi;h7Dop3{}GTak* z_omcQxR}AX8lFD!mNEO&n)mG9y-GcYe5pWAB0*L2DDpC_`8j*d+Xmr+NLV7ly{mnP z6FKiZ?^VuPE-g#3K6;`DvWFrb(8>c0*ZjGP_Pi`2(^m4cW-q%9vx+a~wB}GjUbeg~ zm|^=Cv4wFiv-|PIOVH45Opt4X6ctGtIO6W^te#(Z#4>l0z4uHQCT^wf*I0&Bz)Pm@ zyY~sWbp@B6%w(wKzW3q}-^9(Wg}YmEby;^O7;8`gour%}ovXH-njmkN83@xh7^rQu zuY!fzmBx2hHZpt2?(0T@ign%BN6V`aSFHF-)Ha7O6Heb|@r7g>zqMzYW1xk;+^rNap zg}N>u=c54Bh(3RP4y;$8`bv1AOSm9#=*2RF1^iVx;6F3aHRxd|#N6=LILpTu(G z!Hfgd7<%n!hXo%g+Wto5U6KkVedw&|7UCw9sF+Aumd9R#I179m`HioZO_7-{Q?!YAz|4~Ko<9)IWr^EGx2B!+ zr;6??k?YHMOP*w{)5MW8)@9!QWBogmGF=(UtUV z&Vkt>^Jad+q+#OlJVy8pP5-}O%TBR*i4fg6SM}T!qwvf04?N(<`D20&E`i)2M%ZT4 z{=Xg7nM7{lbN*06N>h8 zs@mv?xz>EI8A#qLvj1-;Y5X1K$}HV|EJ}i>OfKKeH@{cx_0q)gUvKio(M?F5f4nBk zHRX3x{T=1`4&kreiA&m2bq*z5@Yq?sw)(wcztq;+s}JA~{o7l%IC!RFY3j(}+k7MU z668`pODeezgiqrwI#LZ^>$mK!+@SSHkz0#p+Dlc9W~awqOB=m^d)Xh9LT}Np&i}l4 z+nZh5lDM*NJkes?d6I|KY;TwOHLo_`F?~2gF*ud?)@$oGug@6lqRH|P@VDlXDrMU% zV?{cRww6^ya;q;D+ont&(B)0`Od6EcE$-35wV^M`jwXgIyK_N#krb&>l~{V z|Fkb0@zr^}3&uLT#>Yp-3byMk#(hWWMOaZBm@#{X9+N>)#5pT-&&)*!IG7MouhWb< z+owIY^!1D&CWVS%%K!Uy^JY2P*>PT#n`!BbFLR!5mPiQ?{|K+9UAvZ3BeNkuf?%^S zK=O3%_DNrEd`YhfPii9Wz>^xG_fdAH#{gL4?3)NmG%0p;0caY$$= ze-9|@#MD-3T!n<90BPy&r_1j*N$ojy+rsjYY~WD~GgexV+uinUsUYE`6+7Tb2f0Lizg|o_)^~{f6GG|qzVH%Qq#4N8;#XG zsaSYU!HBXha8tz(GA&+VrQP|-;VxL8A6dQvwSrMpFTALV3*a<}+y8Q>9gMOuLxS}7 z$Qn<*W#>p4_^OM!B*JB(6QD+hoI4E*&CmV7u?n7wnLt_T2aZc`uPt(&VwqA``qr={ zFVg?j*qOjXo&SGa$C4r<7Kvmfce!%pNC^>2!^ydS=X-rV@ALJ1JHee$g1l2>Fc=?RTqCr@ z5@V7mR6ZA+7n(k9V$sjeQO3ZWlI9zhV1Jy1Pl1PgUpDM}v{fXexw2VFK|$f@(N4f| zb#$nL6szbF38w)BbN~CCM$n3GmSU^rhxl+#jn3f;cnr~gg)8(*rmJtx%tDD|CxCLe zMG^)(fx2qX?%mIpW22+@H}|&Ja0B8z%=Jv6kjWcVZLsYeffafy;YT*kh>BUiSq(%$ zYxd1Y{(45oHzWT5&Xfy)6KfJdZ!n~A&coo0pBmyEG9GhTO$9Vq^oa}iI*7pZNJr<@ zw+zwz;t9bZjReOB7;NCIlJr{(_MSgV9i%w!C7CM0%msdr_N03Ha~kEEgEn84i2O!Q zm#bDwFd|Lx+P7n8gWWcG)v$#;w_*Us)?N^VL2cQlmP$p$=G#<9Lo zNM|$djH!ov)EX&XC$_P&B1&?$|NR()@;5<60w(=wLKq0Mvmk?Edl`$J0ir%|onbUM z6OY{0d*)e2cLFED4vrcL1%K| za7Ll$Y<-L6v((hkmD)ETKb%y4T5s(-eWHyP1+mhk#Gx5N?}>|zZ|8`$5|D3&{s2&b zgxZXTgM{@SuiRS>boVoj_Jfr-tt2rFm#BKupWPGCZezOAf5VX*y2*Te@6xJX&5^f= zG^%e~#(wG;=PROT9o7wwE;IXI&;q$PZn-SCOYxXq{lz8!1EW}&O9kj7dQ&qUJp?x6 zucKMHtkItOuZ3+&dW3shX}^%9HPPEHY`@jCY@7LcWgN{kHoD6)I&uREN!K)ir7}K3 zC}bPM7O#okU(G{^(pDx1%XWwx?RF<`j7|o56$2xK2hP{1?Xbo|4&q}#qMm5ms6E?v zTevO-(@Yk_CauWyJEaB1`)<}A!yW_YMw zarez@+eX}P6{CvZiO3S44ZlW)mB2H4GiaxTx9zS4)M8iNFsD z?)qVrVI#tSkX*RUx(A<-V40dt3aq&) z|5tOf36Va10yGP(Z8&e>)umW?Lnk~0y#Ozx3#R3b+=4J5bguZ_$`jP&R8ovVRbq%B ziN~=F6q+f3WE>fh^joYFjsV_`J(s~(`%^LwdSw`d zu7+%Ft0hA93=%rgz!c@p=m6gaWxCRCmnimm6;9xs#LUFR85yb29#UIS-Rt_&k-9L%$p%` z>Jbh{uxJ}pYiJsQs~2mQ6$)&1pjR>BjmEmWpPkb!tbw|p6kdiC@K+$W*Z~YXhgjh$ zY%4Cg1tent1-S_6oJ&nL^nhRQ{(%{QDnX?bQYdTB!C>N@ zsJ%XbyAF1`YYce3kbNOLKyi$6dW%I-W_>*%7uD!QmAyEPnz_l$*-A#6r6L{|n7Y84 zJ+;Rs7GCoJs6y^9wYoNO3x!iT3{-VcHS~D*7zG^21jG}>y^F)Epl63x#BY$TlVmc_ z&qHf>bEv{V%H{y{1U&c3pbbeeY`K!MtbqCkjQ7L%c$ewoAxP2#5bDn75T;Naq7el! ze(jKxg6u&$&#ql(WJm()*XCitXnWaOq!|~Efyny75)o=E&^0N;lOZ7vy>TB@$n@4j z6G(CQ!BpoSi77%mJd=}@V_VI!7-VcnE}wGpuD}YfkBvT;6E(pZ$q|J=wGWjeh4(0<k3bwW^(oQP*7FM_qrE&;_n?koIs zm8YKG-PM9O09b{WdJaH8BoBHcuuTmQHATX-0IAHMbFByDW%$`Kc1y4*XTi)uQ>pS=%7>MqG#DHAKirjiU zV0oevFt3H<#c2Om0`e-CsMXSJHuqxv`;fafbYxd{GCaGuC&#gUFXT1fywU2+ORMnB ztJ`+><`;oaA|*@Bb5=bY_*@nzGi3wJyb#agJ|`y86LIZNKn*JL(7Ha5E1B zJ+6DlF}yp5Y1UIfOz7$EQ34EmrE8ZN7=upg9~Y~%Ixo>YNxPhaTT#gyNwrdWV3yJ` z`<}^Es6tPkiW8< z0baWg6r+lLNQv!wm65_YU`e3nVUR6sMOmo z1kFAU-Yb7?c@6XBPFUFV)YMB@9E#3JD7eXLfVJJf|4n0K7Hollr~r?ENO(iLi^A=I zwFGoE5cfi;_fQ4v2p({NAM7|;FbQR*3vANVUD1RF#Y_{XVVL8it9D2Dr&E-JccP+1 zcS4D@x_Z~ec#XhSXyCxnt>B%A2>5hOPp=x*PwMH%murX!V5_Kse{SphFQ#+41Lnuy z0`>AZ?E82JADlyMvP*aU&P{Y|JTvn#{-3b#CkiMm?4Pp!W1Y(2f|qgW>+5%Qc22Kj zXu(GS{~EFSACce6?7>hUUthp(pe=0+Os8y0VD$1u3?@S9Cmul4DaSFn_SyPKKu9Vd zJmLTpmBI9@^WWfm2gt%)sbjFnU$U0v>J=Y!&EKM2zJ3-Nf1~6 z0KNx%Fub@*^7^qcYlEL~;>68x1TX=eq2xfxrL2cZk@TQN)ms`Y4Iryts6k3JJ4n_7 zhRK4}x_NVJEDX6GhMGEdb=l|FLpzhYi~S3srcNfiz}y(Tv<1v_MLH@A-7rjxq!&-m-kSUPjNEfX&61TIUW2LeJ|R` zCY{D?gYElY7Zdo?n||n8g+=IpQ-=$dLT_XTfwDvQg3(`Fe-Q6s*ZNVeUSBi+4=?-K z*8k&=zFD1%!6l+fUJku8{{guGQ~9LZ0m+fLc&wq>&+lf;n0nPu`aSG?zRxaV&3)>p zNV6(kG8hrsO7_5tlk4s9HOVsO?kH!yIPr&rw=^nml0uR;=!T0uO}*Pm3{6q zAZyyiXq?`rRZAV|owcXpFxr;Nv+4w&Q+nPaJ?Uc9aNA~M{P?|XdO;nMQX%q)1CLZ7#A=Nei8uk7 zpsA?|DjP8A3id7DXKtI}sQXR{_sfDQ&vQl!ph^?~V+ork>>a?UDh^mtF+J@NZQN0O z#T@9EP{=%WD#Y_T0K z@&7sltO{_q^h6&*heu-K`+fH~*>7typ5Z$j19r2(wrn!qtS2(az|fF8OC6#LNF8Q5 zp1OY%HImc^aMWnSL(ty^<<28VVvg~N?%Vk%OOpI0xWhr0(Zi-lmWU{N{N^*ab%G%FvN@o>b(R6ic$<&RRfNi`p?zM zcv$&nrWfhO*HhXeY;=L`l(^VrykEC&)b=qmE!?(R#t)jNxMFi zZ#djJ{^ALXv{M*9pu#!LpYEL1cbtqr}o^#D_%CYJgB=+w6t z7q_@}R#g(Sc=hufZM8NpjkK3D32a*&UrXpbF>#V#tE1{kM}*A;?c+xw*_1fdJhQXo z_l=rN&UW)DyCbJqwyNCNyU|;vWa@(H`5G$Y9#%|oZXdbJsnhyM%)>POq4g^BeM67I zhShkLE~q0)O9aeWJgq8tzJQm~Z?VnASaGv|%&aQ-G)W@@UdbLVac#AGytt5dZqEbd zlQN_eIwk$X!;M+VFA>A~;bI+8XAu-D+1iSVG^e(D4ieK& z=R*qOU6+xUsMr|QanH4HWqVKd6fC)xbav9#_SIImt6#mMAxB=EpFo-;EeSjDFV$Vo zgL2!$cGYUt-PkbUoJg~ek80mYNb8`ajCRs9@9~7Rv^_9G&=U1X$+l(%`_`PC2ZMQR zl6iR#w`B^(n8o6Y&sxaJPg^>0wQcJlBlPWT44)6`L@9U3axG^+EGda-S)iH;*2F;q z4F*&lS!QyLb{5a+TpOII^9s3#e2~9+Txhd}UEn zDNKGTwXz)>yeGY&3x$o3PPSH^>y+Ck>s~sLXs3T&_muiK&B#WCtA=FOP(f|6{~0wu z74=2?dBL^$>}l_wK&xlPB7zTBSLbMEwKn0`dXF>{%ie4@iuW~JnQ9_w(wM zH*(7AJd($1U~+Oo;;hjMcQum0Vz0-Ix{c+Et(%2Ag)CARb9EC#3Wv&v%A^jJ3LQ z?}B_{Vu^kakxKG~*x2);qGIdWtxy#)w@jgWA8|oR$^quax_&ZgGWcFv3_?{I+vYVW zE9KE5Qjzgu$jg?B@|_tHx={Ki@ktqD@R$C3mVu!brbLZEbls^LTAc#KW5Q4mSPt zf^HG@)q90wlT;X2)c;s{NMDesRvJ9#(m|u8pI2^+>m#hmRuTzIay>*seaGu$!N*xJ z;qnNEJMQR0{gYz>wJ~dN7@3$DE_^$4$fx#)jj$>)sXf6i?9%sy(eztgtYxwHIJ4J=5fVH`qaNW zqw$B$V9M#xd&M)QOo2lkdERbrT8Zk?Ol+(9^?j9L%rN;g{?oF68h?%%Z)gZdi>aPf z=8B-zm2Z(>zRb|pTB!>qe$FZJon|YII!u2=W>{%h+C3k9jgaW}^p!|1jX>O}Fvt1} z`%sDbDm|vo$ZjevuHJ%I<>!4Qn%x4MP8L>n{A=(-c<#03iA_wmGfwId)=zhggXij8 zh6Sly+_g*m)fFMz#pyd%Yl}jod9M?F`acngD{J$WVm|Q-PEGON1%V5;8L`Q$jmeMu zKDUMMt#xt?x5$qcv$$eESQJwJsz62Jcd5xqFPIT&(yM9b1A#ZXv&*8nshfL7*-!C| zdZgGCp%gleVj}4KI}8vA|r`2D-M z_O77fpC(P2lCL%&zPf(Rw&vI^`xpA_uJ00NtX6U#>$K!G0jliRYTw?Jy!+l1MbYHo zGz|L?i!lh>B_4Eei5 zH+Ex1g`RqxUZ9i(nZ{ynslMf)lo@6d1W1uanBe!9-w-?U5rQf?6A6Bd_T$_NeBvq? z-nWk%2clxP2c+{!RuT8`S08cxB4e?*9JHys>3(7U%<~zEL#$P*WpyrHiv?A#mEpMo z0SUC!uCHufDZv6OU56=TUj%(hmP&LoV_5j;9qHS*e)AgdCfFHOR^P7_PsH2{Kb6Oe zM`Eq<`Y}xePWc&|{Q+WpkF)lX4(!}iwXe2@?mVHf!}O&WrH`U@z~ky)Q~K?@KfwOm zder;fU9kD*$N%O%u;5(?mu3r1~blp6v+X@_uc9h+wMb|*;a{YmKX#?{UA4+d5 zRg<=bJe+&wc8p|Nr_fo-B*?!oR!E}Kx_#`ZnE?v4Woj1h4(#>diju}{EzDf8iLt!G z;>&$8`BVL;{bt*BJcrU0O!)SZqa#fNmd&rNWCp6GOZJbC3%lUAMk&L>2AIlqrJDz8 zp47}lap*tbmUQ3#Vn`7~<&Lp;1Q%gV9IV^yXlT=^lXUauDI?bI(xQFxUV#pdtxw-9 zxTb@Qj)6CC_yHr_@R!UUL(DsN0L{)~^u6Zdf)C%=g-gA(Z{K#&D6=ExcB?}l!ft^p z)7xXeb?9bD?)gHm^OEh?V9Pi)IyNR=42lpWPo}i#`M7V1S78)Oguu>nA^9_mXHa_J z&*UP|uIr}+D_3LfD0;;t#Wq~U8D>oCC7XU48TnT6yE2W!f}6vcC5*K_imRFEp zU0b*#=5UDeQ2FtpRDoCrYilWO7&sk`wWVEQ0@pWEgi8iI6w_{553KWBEpoxd35@(^ ztX5xd1=|55f#KfXXPx6uS%u^zdU71HTJrto_(m^jEE}I22q_opJOD8h1V9fayQ3)` zze{&4HVtQm;`NQspq`9;xP{2mD=wU!p^jW4@IyG_X5ONjS)4VDqv9cwq3Ds=ny~4aWnQk9-7p=Q>d8mc{F4{oJJDa*yQr8g_gX!s z2QS;%-QKH;Z)@Q%bQdupva%Mnol`5B*Y0`#{MIC~KCKO2U9j8m>JFk+p2|9ZV7N=d z6jw_~h6yTttM@gZ0nh=o3gaWL)#P!3ghEV^FU!$>*B|cCgZanJOV-xbzufVpr2TNm z&ma(heH1$krL~q5Kd;8g(UF#kkjQghT=e}wqw%Pl8&vY;x%ykA=UMBZ!4pl6DB`T% zwqwUgcai@4gz)>K5r}h|HQhx~-rZkat*yzD4vvRC6wlN;7*K^Br30D?p+)yi=3oti zH=Z&?e;b?)mjY*f$>DX#IK&aj)zd4cp=riq(uS`F5;l2vHl9iMFAH8f+(XnZXJQ*m z?XLghyJ56dhjh15KtSBAZcmLM6yXtOa^BEK>1d*pEZB;Dk>bpTXH28T< zAcpJCfDuq3+RO7h6s_IBpG|bzVQf|Y3F?<8`k`(59U)VciHWc;J;f>h z^FUClL__5I_A=ehFi5hc^P!~Z`HwpK&bv-E!gf#`of%&^_OgF(k#lBzTuQuqoK8{# zPf+Xwi|J|p&EN4x>mN97LmGvSX0nzGLU7&&`p|kAUhd~NkrVvA~2;==FH>&&iONg3i|5_(Ec9?ap>yNa*0x5ijTSmd04 zi#pYe-&^=17Juub=uUb&VR}JhI!YhzC8Q)gJ#$#H&`-WbeU{Wghv)Mzz{+F{;(n&a z1c?whtqat9&QxgKWM*cDUK5kf&5!t$>)mtu9@a*|Ts}Jn`(reZpQlx28jr)_pt0qK z6C9dFUd=tO&%X9>jIhqCrx@TI zo1XmU@V?ZDADYb=kXT_fh2=SX?wDnSyGQ?Lef|%9hR`pc;9}{Y;3w(-{PWYs e7d1ORuQ6!(4$Ru!Mat2?q^E6oB17xK_5T7iZpGOE literal 0 HcmV?d00001 diff --git a/docs/tech-notes/txn_coord_sender/txncoordsender.puml b/docs/tech-notes/txn_coord_sender/txncoordsender.puml new file mode 100644 index 000000000000..81607dc2a92e --- /dev/null +++ b/docs/tech-notes/txn_coord_sender/txncoordsender.puml @@ -0,0 +1,37 @@ +@startuml +box "On any node using txns" + participant "SQL executor\nor distsql server" as SQL + participant "RootTxn\nor LeafTxn" as Txn + participant TxnCoordSender +end box +participant cluster + +note over SQL: SQL transaction starts +create Txn +SQL -> Txn : client.NewTxn(Txn) +create TxnCoordSender +Txn -> TxnCoordSender : db.factory.TransactionalSender() +TxnCoordSender -> TxnCoordSender : initialize txn object +... +note over SQL: during query execution +SQL -> Txn : txn.Run(client.Batch) +Txn -> Txn : translate Batch into BatchRequest +Txn -> TxnCoordSender: sender.Send(BatchRequest) +TxnCoordSender -> TxnCoordSender: populate txn object into batch +TxnCoordSender -> cluster : distsender.Send(BatchRequest) +... +cluster --> TxnCoordSender : BatchResponse w/ txn object update +TxnCoordSender -> TxnCoordSender: txn.Update(resp.Txn) +TxnCoordSender --> Txn : BatchResponse +Txn --> SQL : Batch modified in-place +... +note over SQL + when SQL txn completes + (only on the SQL gateway / RootTxn) +end note +SQL -> Txn : Commit/Rollback/CleanupOnError +Txn -> Txn : construct BatchRequest with EndTxnRequest +Txn -> TxnCoordSender : sender.Send(BatchRequest) +TxnCoordSender --> cluster : clean up (not always) +TxnCoordSender -> TxnCoordSender : finalize txn +@enduml diff --git a/docs/tech-notes/txn_coord_sender/zombie.png b/docs/tech-notes/txn_coord_sender/zombie.png new file mode 100644 index 0000000000000000000000000000000000000000..324f6f979f573d207247bd5054dad5b4d27b5714 GIT binary patch literal 34467 zcmd431yogA+cs<$A_qZHKv8-tNQ1OWmx!AX5Ky{7Qd&JIu#xVTZjf#;K&3-Er5mIh zzIy>G=e*DR{ont7;~(QcV;ps}7jvyO=RNPZuIrvI5AF+LpCLSR=+Gf-5n=v^hYlT8 zJ9OxX$EhRm6Qiitc<>j>OyH53nvt=UzJ{jRAt4Pz4U?y48tT_nt*&XCnHh7?(HZMM zH8e9f(7&Z-WPnT0P7EVZ>PSB_`}OlfhhdoKp>AkJqfUB!`#j$r<1flDP`EF1qP$vV zA1Juo9X~a!5klz``cTiMmN=x^e#`wUg;{KqGUd(-WsF4E9IU_VpS&7GL6~?y=#}Hg zvhLaIuWTMSMk#O}?uon{)qk!F$By*2^0CYE!;V+${DMxqHMUn6#K}4xbJ7;#;>aNi zxVv1aPI&C2i*%Y{&S!~vt}iMVu}q0)&wJNg)C}lndFHXM(x)e0R`yr!L$xSsf(VUM z2{}!(%EAHCS^gQ%Qh5qA>ZQ29;F0+Y1z^{ywhg2<#WT({?$DWvr?knnKK^LrSAVR~ zOm3<0Yufl~-%bR#`XUkcxs!L~&qR_t-p#_z{rkPlm8KbG+DNN&PdU$g9;0zIG&*uF zJ!7Kh^Pw{u=C3?4pR^gW-W=0SsShCjB1`C{b}agM)pqUZ)*)*XRJJ#}=;!tCPepF; zj9DHlNtn(NcTx-+XO-nI=)5~9l-086wP7%MG3(3gzV}S$b0fRxy_>r8sfdjd9Ik%6 zBV42zo!`ukrEvE%4&5fZXfx?kw$*Y2P3lKc-3uN+o|#PMb-bu97&c9M;cb4QU<~^y z!^Tl6?Z=g`%YOWQ=+KKpBK&uypLb4$oqlwottY<&-xZ{D?SJbbrCBKshZXFfV^`T6Z4~`q( ziT`e7ZCbE`IrDA4w4Q5)-BfcVG{|OTv#z?p@)qUJ39!sV4Nj+x5EDK01cUtf0)<%r z9n;`}!#+atj~@9B^Kf1@x`u^Sy(M)1#YHVl+o2zwIf4uVy@H2GPhUI|NSaM2ZpddF z&vqAU7Hc{tSrv0AaG=C^q$Vgn!oKt4$B&;rMG1La_2#izreO#c`;6apqqbpqvTrSr z^aLw$_)ja&xcJKl$+S?~~Ym%SC-#b=B3Akxo|$ zZyP^({5bUfPrE@kHuaX8nws9;?GCMo65F**g;GrBXiKu(u`7v#&id5rElWk(_?HOv`GcCyhUWJT|jOsb)`1_(I_Y4eD&j!WC#f^`T zTRUNw$SKLm<(Z8?ED)~@FjdS{cNt1JS*1f{ee$dl{@fL`eymdO1C@rZL^piB7(Hqs z5%-~xkPsq9uEHB$#}1+OZ}7*E^Nfs+CbN6{hhF8j);>9K8jtdEp_zym)9G|;SMMjZ zihTF(oiSWk9rTzC95Aw92!ZF2k&!7`8wv5|wOcpibEu_LDSe!notRNR@5jNnP;ZX(nO2s%TYtJ)6-&1W>V#r;u(GxR%CV`8$zRcRo%?6hp96jb*QV~ z@=OyYKl}kd^|Pg-qT>4-Q*K>ZSy$(u&2PoBKHHwb*F+}do|2N{%e%rt=$4aXE+Fta>V>NP zmq}3(5ihec`z>FI!7pw;EhRQ9goO5s$~p1z@f;1|+%~dJ-qHFK-{U<}G}YCqlf4^C znfV9=Y`@|w>IASg5u6SB_1IqQgAs(^Wn9P)1*)1#(%-&)d(3TBu(Xz}DUSOHOsmeU z?ds3|ckixoa&o#1ik1>KOa^phs0oGPy!U-79`k_J53b-7 zwX@8uF5NA>5qIhZij@|RQe696fIlIlKD8-EUS8gM6kqA_WAYNa4U5y=sya($A7Yez zlarG9|pa}qN1>5eR+ropcsa-~7!>6V`RV z%P5AfdA9kBD@!xW%vb#Jq(IU`aqUqeVq!kuZXPa-4GVlr!ph3Zc#k^mG<*e~80?YW zcGYY8Mf!>nwb8mTzBgwxa&u?8 z3$g+ST2mDjJ1=u^a)NW=o&QGm#y%W-|6*8F+(YnhWV(*+{Rv~hzmbsy4qfx!y*KHh zzeM)0a)zxzdT1`J_Za*Q&sA<|jR)62l=!O;Oa$@E>iS3b#t`?)!2XZJ?hXE*?)*ad z2n+?|-a#A(Ji_m&^6Q@2ROplvvjVdnmy`L;{{DEX?+gM5j?PNZ+D2v_^@>Sc+cr|M z3r2T~2F#YNa#FUPmyr6D7en&nVIi;IUq=l-!eoln{$I-vobSpJ6?i|I~t@OH3g@c#A8#S806+&0V3G*@};i{arv78)Ar=O?46=-U1ibN1>B6yyl- zP`K!F{L5U2&?$M~3cv>Vy-ylqaJVi$#iX|2mu=-5wbuppPy6NTHH?@hud>AMK`xL?c?+dt+Jl{{NWVgu$_2Q&fR{c$I5D}CEb2zvTJrS zz5XfKd9Q(^!D6Xq%+bNg74pQe?HM3H)l#NTIbhAAqDO*gXrJ zmbVsu8qYL*jFKeDv)hnO?ulvK2=+Rn!ES99<{_yfR_$)W_^@uJk04RNpzGrFK-1^u z=4?eaA{jvquWUiJh4FZKY_3pPH*bF(Op}AtxoIPfmJB16l@c4daYwK9IsV$jRBl## z+k}!5rO`T^ZS2);5AYIP~~UZ z9CeS6WNj=Ohn(ADYH)&@w7O_E&2}Qu^9{*;WDj653p5sma~F~DmiD{S)%p5S@NO;Q zVY5|S=UcC+sNI>$>ZU8>BD+3TNp*Dbs!7E4BWQh%xaUK{wktEa^;~$R_FJDeH@SB< zM+MA%HreOf%0|Yf=-69Z{Sl92ZM6F1$Mvu7sckbVx@&p7djb2uKXdgsHn@lHXWyR51hr9w0BZTqj15wlFN zuTy6`Mv2%=mZ#)-EN95q)S@hnMomY18F;MCXEy)Zn07{r}Qo%-olrS zk&WKQVkk+*lp@1qI5a@ndc<(=#bSe^HcPDK;#NQ75agJOiuv?fPqKM=Jw0DLFi0@Q zJ;cLzEUvv!Lh1x-0lTl6c#W8KnFh%fZY=aM@i{|YJHrXM#ghk~_5QR{*e-)>Yc}i4 z<^hhyNoa3u-aW(9#C&$bL6-`uCM;M?YA#e#1?g^AxNa;hPwKTjG|=a`se^@OIz5}$ zz56;rsP(n2sW3}f<&xm_>Acr+X~bb%>;nO1#PQoEPU%KvDq9nyIXPdq=RC0HIZQ`Z zdrQKQeZ0TGD}6g9kyHwVCFz$@^kizuf@)bdmNjlf>=E|C6j~Pu*PSDyuG;6z*=`=O zk};i`*qdcvl<3ei3ya&&R0ji8!V7HK*%=wd;}=9z0vR=$mnO#@c5LirWd~jyzB)mf zA==hRSL1?Y*xnn9&lHPS z@Fx4^1uB)neV;dxg&QAz}{v_)SMQ z{_`&PIKt!>4c8YLmX|G9S;MxrN<6M8rW>9=eimzCvNHz_y9JNRJ>P;>hW5d>g-q1< zEhl(zbBA?(^|jSOW}&Z!Z|wd4{&E;GAbo_iKxVW6xe~ zSQejqxfpk`w4vjra8qbp`$Uu{At`-@za9q1`039VteO}aWLU08X#EnzTe!qMQdCeb zd&=Gk7U2zwXSKKa69KaQf&(EF*ibQX^&1}QoUW`M z=u;|Q4{MXl@&D^3DwiMKJo2;uGsIa@VPVe)rKy*_CSo2YD%}6EIR2fVSUWbZ{k%0A z%L#G6n$ZyKR_6xiEc==Qf(_*??nqgZmXu-~QAN&;22~3rt~% zqj9V6eh)DSksdO4h`UXLy@nLu601&E>r_`lvUw##?^#@6SeV`7n3N9Hnk#R{??)M^ zaQ*i<(Yd+HpI&3BDt+}Jk^D1GU;eYB$T-vv17fTGWlK0DJjFYPk0PzicI>XTh{Qh3 z3B4mPe;M~=CxkLnBVDtZEF0Ea$m}u`LH0q=R|w_m-Me>X2d`hJNUrom`{DV2_xIV& zN{XP`Kc7FGZtLrlD8d+u=&G8X-|*077xXpf;@BAypX&TF;DmaW@;w++Z^|`BqxLdZ$f(89DHq?Sq|}6RkkmCODCp0P z;T{9D=*J5+DQ!k%nfW^3eLcN62?Y{*>x{gb=e+6mt0PM?;~H@+U%epUQgwX3T? zts-Sy{`0wk_V)InA@x7M;Be2%svs;(NmkZBGdMgvtx@~N&6|Lk7-2-jkUAhwz0$i) z*@hM#X12HF*K@31lV~h2C%=7rb)qGiBWOj|%Y|nq*E2sq|NVRV=}=EkPi!JsSa_lR z)nLs-xf8J&jyJ`+sbbt7euDgTaBwikv^7PZjLKzq=JwVKrtdji1qDF#b0Q&Us(<5y5nNLMad3?A@Bp&^3NnlII|^@YMOW=aBL z6A%ypDRO#xy6?NG9}^S7K0eWQ-msM(qW0`#cg0jqL!ydj`pWvcx;Co(ZgPYZT)1%I z(xt$V5XFZN8SOV$gP62LN3wOk`_rl1a@pM6ydTD)RB9{V#F!=-!K0Lp63y`7Ss(l2 zMnpjiA^e87-KWo=7v|^r?~K-lu$hfL&~UhUGZ}+9GwtJhg*?kIIBFeL9IGKDDH#Cs zIy37~U$(z*q%7%PHdo29q&k&SJRz-mm`^v;i%El^FlWv7D7?WM6c(1cFrKBJYk`*b z=iOP8_PE-UZF>HE@6ScE@rIB0l=1K!Y`#heQl86^gZ%L4(sZ7IVs>zr>1ZZIRNJj3 z_KxZHjHSbe|JK*%;JmFJCqZO4D0psf|028bgoM;>tNECK06BU2py2WTelPgv4mB@t z890n{fR!}O7n)DXIygA+Rh*%^A8Kr7CN3-tw&tY!@{Rbj#j(bTHj)c4w@emO?W}5; zFizf>w5vSCvja_JOiXjtboJkoV=DS*@)KX0Vo2;3UlRKTK_Jzmvdn9x2yKVNFM0cNMt)RkvMa@+Xb{Gjt2afVyB49lHyJ45dA@d;e(tckCO zfc+rv#EEk}CgjSB#DBlCg@@~p;Blpd>O7P^ON==qK|7*Q@!8z=jzT9@P)}h8vh5Im zawP8gNp!LGk}iU-WYtIg9UdOu+S=ON+w0=uB5JRzn^aI>B_-t*1F*`qaXhN~mu^@9 z_>nA|`cSU#`JmgkZ^C48)Uvu-TJf9pD%Z)#_(lWrk{4WW8QQ0(CB@|gMzL~Ro8d8` z^Ex}sx2Cclo69tLn|_tBT;i^vRg4msknr*DZR(wNBDs~DKzAD|He|nj*wNOZPA^@b zfDr1JO+K;1#>-Igd6{fMlQvzLw?|o*q8C5ZaZnam~di+pODs_GO zWLtq@7qs~NJG;n`cy`^e{xB17LH5d)y?pjFqb<}I|26K4WI1$^dDhx|HPeJp4GN{# zk}BlBIrLqNFQj~Pd#&CTo1p@-M!R%nZdcb8PWBFmpC2K6*-KnL;CEF&Fx$4|^sYI2 zPVbYu)mvV5&U;#z42|F@8px7QpEgS9<1sTet*Pr>ZjsX(mTBsg%)Zn))ZZWKpDRC# zr(PmE@#9@(eP1+JV^fya3Z%RYkO1w48htvxMq2soQ=VAjGndS2^j@t-aa6RcO3V#a z*7Chs{JwoPGCtn*^=qpwl}d)}@rKBJNOqyZdb=Z9_wKG8Bnw<3C!2v12Zu^P8{A|x`#IBk-B_$#5r2=j1nAymNkLn)#r83xVp{0L9veF z)vm}B>_vuCfvQ<15pmk8fi^dPh8UeuQ&lFU|M0mgNv%G-N65pThSNrIp}i}w?_!z) zi(HcZ*!7zP0asxOwCbil{WbzzG?(#?zv+&T>45IbmwzM0I0nPlXJ@(X;ULawZPqjB zvBBzbpo*`M-TCQ+H(OfNR6ncrXAU7W>0{W8F0QUGcjeM5)OHk@cD#Rjv~+ttbP7Di z`imELA+Nqd%=|sht@Ih0P=^)(41-^t2op)K_hW-chYWsVXFE4i;$c|m8&@$LG#b4m z&czj~QO{%7y*#a$8lIbzGdvnGdO1lUwB>7jIs^iC3#k0dc1ne5`x=HXC00Lw$wBoN zW*kXlV0sio(3?;?^IOi(Tin>8khC4Eci@SgVtw*LNKTrP85Lo8U_xc9P#9PdZxa^lTSbkOR7E5J2iSa%Ag}$uaNKM+^ zzt_JdUotkB&hR2TDw`G>?Qb+@^0rx{_|7!mu8HjC^BD&A>omb*Q&ZP?$b%nAXG`{^ z=0D$Ym>%`1G3?Rz&GwV|TF+C>B5nWV#}6{0ft(5k>Tjd&Au)wbP2}citNnJO?X08t8uGdNxvyWZVJ?`y zor)W|w6j=HdJco>dFFH9JTo2_-K-32%WeEFKIwg1dOX|Dq!wZ_^B$R(e=IBaF4}fy zt8ZnZB`g1VnUBxbNfQ0SXob|r>|ej((SpsQAGYc`1SW5dXGKbIv|L`}f27qI9_ZVY ztM5{%h-QA+rgQ-*nSL1s6XM^%5UYrv-u>9cZDUbpLEDnAV7DL)zQ;VnXYxa@YW~#aR#UZ*&=9r9Wopurev9b6~&U z;p>$l6P?jf!a4HAEmV@X*83iSI0ext!oJyrO|X+uksvbPc6 zt=U}e_+HMBM7#DT39r{X8*le&QX~pu{fW+J1z;`UDoDozN%?T$pZoFX-=B8B`IY7( zSJ4ELM&I6xr_QL%19b#i{UiQ(`2K=a@7AZP*CFZm()@$~;>Z+H0bYB7&$A{YOi!nHtr~4qHn=dwhk= zmx^lFkMC6&L;9WBC`!tar7jvBycA+4I}t^G9Wosnoy>pK0wJNmYo#gX-QQH38>mRn zIqiS;ltU^lDK>d6xf_Lqe^8Li@5)2f$0s=W#z+R#6Hp-pvVJ^M8_i&Vt8-0<5*hc^ zt0R9(h7jUs?lm>yULHe3LnQ{Uu&=zCGH?|>D#9e9CgKBt^`BbkRJIP@7t8$+aT!A| z06futLBNQAcyB$BIX)|JC*=2f#GT4U{klpOgM9J3bZfv8_lnzv79d!ae8ATc-LL8X z!xw-tz)~Y-AaVK*`9}nk0E3M|?WMW`Z%D8D81p~VG9cjC?u+Mt*r|MqZ3n7%JE%hf30$RHlNhfr~ObeJbwI` z!))xw*w|`0o@DV#ugw=MW(I~v`K9^!LhGd@gxqZGn06n&M0)GiEgBkePAYP8_0Mmx z14#k-K#Bpdq7Y;18`RXs78dsXuL+>`kdcylw!Sz{tC)NC3df?ML}OQ%O;@ggZXFbB z6!96mS?=82TqnC;YszIdL*lptZ$?Rs3AY*v0vxcip|!BEaBX34ymfYdpOZ7)6kGkxiwtVhgrTCs!jYjN?TANE zI06Ixue%Tv6PMSYHE@Hn=j_><+FGE#F7w)N)&wzO*P<|3g+)aqT+biKuFUq)K`mja z_kYBEVC+1Ci~#L1Il3O0Es{G%nHT8NI0_Rh`DZe|dW&&kdGa&WmE z`MgR>?6%kiWPSiyoXtzd%EZK^ih;`W+_`g;ZRr7WB7J=9pg_Prb?SXahLi=KtIWKT zk`mNoIV&|)Ro>#iObM}NiQD)Rp|_twA@)YR@$+XgI=b|2DIoZ*S5)$iBgcB81YBoQ ziP_llpjaLl80hTm^!7f#^s_Nq2==+|Y=LHNaE97fRSb~i2+;;Mp#Ldy44@@$>%|A* zc4;Xo_`F@Ec120rjg5^^XH=;*0yH!s2^1t@Wr`8z&D*S(!|h#NMm>c&g@qh!Y;2^Y z_qZP~K11pX3a2-3z$2VPlb8PTC#v7i324=Yf>~&l6VTGogocGh*v?n-Ih{j=OiWDp z-%^?wAJ;K3n4X>v3k*yc5)u$FpYD{|=+GLBYyV(kG1Coo*!B4Sp&{~(i`|}fR8`(onNWfMmCnxi_G5|6A;K`{%XVlduE-yAc zD}i;LPW)e+fI(pO^zp_JoIsu%*p{ke*->sbladU(zJC{G`1YW=RHLzmx03XS^VYY& zmEKH>|MS~k4Tj1Zm0(GKzFbM-Dg3DW2fl9C=5~xVpb@!p1SfEfB$ot%BlaJNPpW9- zQ)eDdf+9w{;n%%qSQ=|~oEOH1oL+q=8}XmwX*$<+;Mo(> zUF=0hwkOC{iww}eR<#_zzs#4y&a)fEueF;n|Be|Y=r9%O_cW2)LH;D-gUt^6VK-}tH;W5!vU^)txBFJtZY`XJ!n+KTJ-Bgohi(oC%9M+p??iL8=$H_1w!KRcUGI zCr_Rj8XAg-R08q}rkyynGTCO-S84~b3E&VW&6+@Forha4cSK+}fD?1x1b6=qWH3*k zK7Ht$IWan_ReUClU4W0};4ZYKj~g2w5AyPgjH!>{ot>I`>@b>{$tWZwR1)`mrkkBY zNlNN^^6S^HCz9Ykb2r}FuBRxG9rTMQ)i495og~LX08&}|<4)$Gv|9>}>SZK)^tx{Q zjqbrvvijt=4><>~^z7L)0Kx#%qM)Fd*j{;pd6PuX!eSuocF;MJyhRBL37tHCJPdHK zs3=mn7D7v9US3{bp}C&9xj78P&tJY37!`Sgnb1u=X@e*U_n<@YIV)g3EJM&*s$?^s zc^#dSoUB-AmI%@qxN%;drK)O_cbVzveHJU zXyWy8u59%n$-HuX&HD}>tndD>HKa*Mh4FK>vy)fOGgRI^@XZU9auMxo4~n2Q)T;y< z5zTD^Rm+IAT0)WPaV}-LDe zYsPuT>PR4mWMwxeiMpCJe+Daxt!;;d_67L9z1MYIHiYT;^YTEJ{K<6=q)tyDSvt7d zmod8=e?Q;E)jr@~Ub0r~ZCyxxz?J?+y)6O%IsNkN?~RQ-G(Y$d@J#TSkt~6#u*D#e z2T4k5N(%9nD?)CBexae=F|u0oXnFzgFhlkyko~3|8Fz`8cylVNUQZ}ameVVCcgc*lp4 z;NcBUOz7(AaoVjf0u1>kFzVE#X+5`=kx^fXjit!9B=y3A0x)XPj_a(fvLZ3Dv7pMq zI%v6QY=P_7ud97|D{}un1j^kOxDB%hh8gY@R(QEwzjqdgM+4~rh2-% zL+0~Cl`!{Jabsg+Aa|Q%#r^2(>kDiEg-Uc|0wWb9t7g5$R^aeW@~5H=Fd*x!TEQm) zdD$?LlO=NnyXiS2B-EPQ*>GCNE%f(X~16hhsoDMl6JROB>TtDx*< zAN;FCLA(um^R$3oAN!LLDypB`WK0;=nHJ6p2;fNSu&Z7bLQAVNB=WtZ5wdpYXlKdZ z#wXi@^ct>8AWlHauVTpkr3qmsaW09~dsF9Se6l^xs{dmMDHfJkwY)dSv8(h;oXxbw zW;>_%Ht|b#fhYGfw(3y+6n4h%^~Me}?d)hrS$+6;>pDAA0HiP9eD`4$6ky_cltR0uEOXmz*prlXDs~0D@B`A;{Gav;eDn% z@c|(|<4-Xb#{b*j0BX3x{;v$Bi?Jmh3(LMJ2&!YKh>@9K#D5BN5_hWhW#GxOL!E{G zri~%=Ac#SnAIziU2vbF{`Ll=-;wL)&Vzr&O;b)-!*>!)B1rky;Ltjb69x1HOi!2{L z@D-@b7_>3Z2E#qv+@PTO4#X0Ga#&~192goY42v}|$zQ}(il09d7q0;lf&U^D0f-)F z|MvUW=!FL@tsC%PWHadW4GzXY{`BpEBa-8&GG#pZ_zDED6b7oQCPuG`In2KG^lXLW zl%2eH@7~2*kDXDuC(+&W15p~){w}Cn!0$l{#Lj*hDNB=gvzH8ZYC5{StgPy)szOOL zF{}O~_)XJr&=miwF}sRUl(a62d4@gS6k^IY(>Wk}f*K!ze^oEQ+_eKi!bS5RS-JNc zyg`Vc1NU1+)c(`-)ala*w7Cm+>?hytg5u9$&x4;Zz(?!*o)`4EdWD3fkGiw-kY(D{;*``>J3G7BV;U#r+bt3$&XHd{(T->L6dR?(% z0bXQ6%Eiy6v@Fo_K)|hmhD={WqXmfNm>6;?4ogc*P>v>YaC7$vUsxIyT53)3^z);b z#feLx5ppjt%3t(c!A1fS{0|#5BPaMDw4cev{~9WGX}|J9a<7PpzC?6Ud6N|dPl(!v6h^6R?Ls7S)}o^9{gPzOD>iZ3RdI5?vKjNLp= zxEoS;kqrIuIO>{UmdV%v<)+HT#l@O$-ySrL)<;-cSorg+styD)sF7!AH=jy7YGR20LJw7P1xxB8%3D(31oWL zS(J;m_^Y8ApeqIy&Yq>$@Q!ExhP9lcjsDQL7k)QKsYA$lZ{CzzoH26XtkG3$-x z>7yslN!veb@Jw@Ybi2e5WnUWUlaZH4Ad{MwR##i==IoqLf@YR}Eg0QsKk)T|+dagX z`9qj1{wb|faLA)8y-haKW{ma~U~cT_xJxX3fU;cg_)S?NAwU&_(}WL%J|Qu2%2s~H z*t>s(@`&JIT|+}!4D7g9)R5`U&u!<`%Zz=VXkGP0nd9u>b?1c7#_t zB_r%wZ5Tn0z4-Iz`}gmSO-#J6fOr~P26rzhFlXaq0e%12=DdZuePc!KY>P}w_F?o? zZ%->IwJv^aZeCpeI-^+?hr?^@Fi?&QvZ|exXT=>#K~Xe{x%?tQA|ggQv&$f%Q~W_d zyc_MyZ-6L8D6XB+YNYGiHdAjKA%!%h))`h)W zaLw}iWc|*#Bo%N#V?cqmd?qr)RQuQFzNbgzUS2&&_@futE#@<^P4?JMo!5WZPT8lJ zyLWlufyR5v&{UJ4$V`qgHMQ(cMV0Ps&)O_a$UiRbtE}Aq8;2E%uzM5Dtg$h$Icq^4 zbdtE$@yfy_hi?{+b@*z@^c8R4rUK@_K?0Bo0@jNe=7A^LTbnac7|$@E;y@dJKJ?pp5J4)+Y)?h zIud|DIrIyh-Lzab>*qnHR#xox=jJL^-aNO}IY~PvGIj!^#-qLKo?Abau$NNYDy!e~ zT>C6Q!vhKNiA&F`aceg{mV%6*PIpbn?&c4_0OPK7${T2x3G%4AbNj(>4>HM>lbGV# zd|oCzEyDlLU*P-~-)XwbckTh*h>bjPEDFOBq`*&GS^6c`wA;_T1&nG@|m6gNhb zRM+*kaxE`|2r^l=)-Q>~`dbxjdv*2pAf}=Nrf~lMn)3`KeTTp}E-2PaRBy5TkI62h zu3T*vt6=0BSI#5#GQmhHwNSA$i~I3 z1mulpTX}gOtGbUgReb-h8d;hf4+lr9BWqv49B*J?u(md8^(N_}*3#U#o=%69yHe@q z@N5~~!)i9OJ=U#mmvw`L>KL*>1_t-%qCI&2>Zz>mO59B6nu`QQiEm~Xs4;=KIH19B zG;z+$3z#okTD;|i^HuDn$qx{7ddZ2nMgc!3O#s@@(6^bv{<$4JJt!)zqUP{Dxia@O zu=8}a4$Cd0%zwtN9qgM;W;(8jV$EBrL5_##hD^6pqk_0azBxKDAf$WzGTD z{+6rp{H5EsDKt`-oeDZ5c!)Y<&|zMZj^-DqT|`g@)&2=ZECRpi)ZgPN{{Mgh{k15a zzR3Qe*M1POx_bP#zbeJe@g9bZgH~x>wak9T-~Y;$D@Eg*^UI)_4iCR|@#4sm1I5>d zxb+r;zEb{UX?!QK=;-98RnpjPELQ03ac8Eco){Q(#CvoNRFrvV$#|Ua#|xS~;5G-Y zNHhv7S;|~HXrI4H(701$#{t;gB{26BHGpjY;lYmV{vx6daYi-y?o|PM_!9i!*O$M_ zvOPK#;p&hJU!Z^r-5U&`XsCi-Aw>{uTj1`#ZFSE}WM~?MJ~{>B$e5tr@=cu@7(^)I zSOxrLv!7sAFaFx+wfsvRSff#FsOm4Lv+UV4*C*obFKH+(Vdt@% z%pPB|1!7*t1R+D8`9~G$NGTKwsTg{QjGf)eDC7yB({-CJ(xy92;!Tg5)PdMK@VrNi zH&~wTk&-}(0{rgfxCh1q>$4AXA`gMvx63N#ksu40@nZEpLPO?n?M^89MMNHb?8h>q zg%bCAbO>QhKTPid?t2{SKRyYs95dvFC4h3%8K_E$)7|#SdeQbA=-WEjn>8-6s~b9X zXOH?kChkS0@PgIAGH!1mkACuFF&q7VeAD>)J!Tr2|Fy--^a9s_QDde+o)lSUWY%;<5sdr4 z|Acg8eZVIo!o$O3DcJ=N8Z?vBa{fAhKGV4W9@vF*Ix=U@sCyeM6@ZgMe{)I-jkuH( zv=q7tqUGg9{+NS86w2Su#%Ik5Qm{$fk`dMbn2pZ0Gr&SZgER@Jg}%N0 z_Lky1sJ|xk4Gj4B_(BvO%`THeXE3XN`|g7885xWUEGqBDRMvnb(4kgVR(fq|OLKEN zk8cu!_74EStxYH=U%h%op#1!4`}?O7x){-@{Cwa;3Q9{$X{+$tKx=8(eH-EZ_pGtu zyhLMFRZ@1g^eiPq!M_OT93=C)o04|P$;;tFej#|(y*47#TA(V8XV3&Bl zlu41ORCiZIOH)&jpI`ViwB}r9HyHx8L=M&QmZN%ldKGI>eojqKtEsBKjqtND2OL*+ zlF`ikP1?QD>>N8eBlg+SjhPpRkH&DY$C^*Jj+gv(mssfrDMF0yZ;eKLn2|4UFZ+as zKHljvfpY%k%a;WO1;D64D{KV2Y1|Ew{{DWL7FB_Tky;`Svjp#*NtGQAYU%{QKzVul zpoe$qdPXeJ;3e)dj~=ZKxYDh!t)ceW?ubyAvB7<=yRE6bX6UaBNp{$)kRqR@nTUQ1+E1WlMb|gi7V7Hil4hKvP&!g~WX!s|yFpVB8}q%XN&2R@x4F0! z5#aJZRFV_i3<$OL3M9sK=2Z`h9+hatDh0HMB*e@mGnrigZuM1PjDJ@RwU59U#Y}erVcSLXi)r;Ue_RV=|0zf7s zL^p))o;8PN1z2lnvB3Oz3rk3( zCMBImn;9E7*jz^n^!Z|}12H3`B4_~4IsiTS_h=!UVH*rIZW7p_Sz)KsDit<=|Nb4s z30+;BpjgTGap{)`VytLwm93J`P<$VpXt!l&^cb#_ut1mSpR zPI963aFt(xiaqGRq|S=pD7=a8&eEy}sL1VcSY)!Wm>6EPUsy?Tak*ViM0KO$eLy^S zwzq(GI&t>Wd!s&{5&~~8uaAg|HIQ_FQG+ETLY(EG0DxB0REB)!@iKd_m6&aZh2%QA z!u++8!X&~!?j}&Rh`yq=`sfT1Ozi&PH}$w7t2y?&O-(YBVB3;6iOKR;nX39a`KEq&f$Yf=ToM@Nqw zQSe2jfj0(-{U3uu-~az}IGhH_!~ZZIj$Yf1htSYH37tPiM!>U1+sWRdY{lVT^eihY z3r%%Lol&t{@(a!K-fk<~&QYy617JF28>1kk9;pox-B$JyS_D{HM{gso*K!7SPBy9?>5(8-7b{oUqKiW1YURazts1C88=lPOZ-p5W~bJYjnBsc+b9ncN{%>W zrM8yy%Ip>!cN{BLS68XRpAXU|7>33{OF z2wNl2zAFzdXcSWIRymM;XQ!+x%gVA^&U9~YZ73=zASik;i&&WIKcRP7b0K^Q5!WM0 zQG+8R78A`ViZEC<1ux$eHB2W6?LxV%9?rV3b8t*tNKr1aPQ(l9tf-PlNkOxCne6Df zoXLrftWoF?Oc-+YYi(|(9+%`<*=v;OH+dOm76!kdPxpU+N)K2Xy$?p1Eqt5;{hDc`o<*_%4v21 zulv(f3Q$vFU$I}C*O->CJg9~PlPMYmH@c;fmLmPohz5rjywA&Pv$@`8$lJ(hG4<%cla#N{3=ZOvkVHUV7aC2L z4k-?_$VssZ>`G-^SY58i#>6C)nW=rlhn+%1O#IlGi2cEaoQdGuOOb8(0%wM)Ra8_o zWZvq6%|nDVp~S_-od_p%6U@oe%X9eJRbMZ;+_)AFMfqH7+h1g4v1de5RLYdo5|WZ8 zU~x;0`#+&2xnrm38_PvzW&Y6N57UVup|B1h_W}X#uq%oKj*uz&>pG}Ut$JV2{4UsO zZ16YzX7s1O1`g+}x?Dsn9Y5wS-$p)TLhNhl9*MHWWEXWG(6e zi1MJ3YhwFGX4WOYb@fGH0=c=kXh7{38XBWJ9WPjDF`bmATwpRh9H3%hW`=*%2QyQJ zFVGC#Uh}hE^*l!1`5Ay!fy-!k|MWAofyGVzOsgcbPA3ID!&}+jGjw!x&s3&IO%_YV zAWH)hAultN5Drsezu%0PfU~RMKi}a8@t)+SJJIgZyLI<5%_D(G*{#=RifQv0rjS2I z?H<^Doj!Ksn;N$eoTIdyx3eJ!zVLT}u2&Y9pgVvo(SCRG2NSQul7nRUpbg|R8HbQU z{7{~X4O@j`u=YrR`b_$L(pHb>0FDUL9NG)vZ8PF?UxJ$W!ki|>Ns+fd_*>~ai#pSB ze=kwt?m+;EX7oju2YbaEK`vWR#{3&1*!v!5j{)7)3cct$@X2dt9~1~AMb2gkKEIbJ z{Hkq{dKF!pYp<24OLZDsY&+Gh{JGSW!0*bSm!|WOiWBnf%e9KcY=1L=H`L`wE)@^;mjTe z1_n`0bMqz83{kNoCxJPpTqGdq>gezT0*>k$J2cb!5c5$~pVsdyRhl0@7NhqV41PR5 zqE(wYv&-=`o?&->Bw;^9JlGpC*`P?-939iI&h9%F*0LUuxI&Fg!XA>M zkb!#R=|;Ru@p$&O!~ zHuVlTYTeRHF)kND_bFN$8><09_r6kRkT)oQt^oG&Y*0`{zx{oGVthEGa%PMJM4=o# z5OM(v`Khk%en)M0_hXh%MrB*FAcFeWP5A;6vqFCK_uoB`jS%%!sNJ}6!`BQz5I%9K zujbHuIYBTvGt>KlUEpbbI5!{_-ktz7NDop>awQFnkFPCv896L>Xz@Y=IwT8ehJKEZ zg1tddmckpaov`@^N*A_KcT?y~_?;U{%-Q`e!0Kyjv9Yj<6wzMxe5-{Kkksfg2SZbVi1b5XSN-}t zlbH-LZ*Jl3>J)yZggds2V{$%}TThG}xKS>vbvxRJks4ypTNp`@_C6|@x54pn2npQO z)@V7w3`pfZwh6HDhQW75A7@EkT8q{6x-f^ShJKaYTmwE(e*URa^1rgmP~q+@HSqB& zJlgCkA5tN8H4y&iB>}8k>I)-)fNka_CZ38a+-#5htfH#MyHjgKPMe;or6(B>a=WjP zdDtQC+hk9CtfRlJTa&LrVu^6xm+h9TX)CRMy?SSLlyr7;-TpFz^>}oA>FIN_ns|Q0 zoGN_|931wc;dSDNOc4+XP)=~<*Gg43_zb4C(o2?MBK_7X9Jn@(6}=DmM{!)jNQoWJ z3T@5hzf0`8JpcJss)Qfv%j^kkT_QNo@K4oxZ!mA1WMO`r@Ji|euL8Zm8(pMd1Yp`o z=RY{{2Yi>~<(3DF!C&D6!4(m+*;SSAqKyCi1f>*CjD&=^_)LL6D0cwkDJm+0ehf%m z-3eK4u(PkFVM-!8iO%=MZraR9P%b@5IPt3n3smJheeRq+>>xO2Dk(V`y5U>XlqLrU zzx%`E5hChX#53_pZMe;H;Ax`c#H}V%^O{qC>4AC<>VW{6^1T9EPgG27_EE}z5(Lc} zf?Y3+)>}S%2E>5N+qX1ybY&5%X-cIj0EpT^W;)Rdhq9F}Hi=akTEjssDA8?jLO{m= z)H9imVdh=wHih`H7cwTeF@LhIRQ>yNa45im27iCxKY#xGSzOE|l?S$ueT751gBMCg zNX!6j1y}fmYG{3F5`+{y9TGJr2zp2<(Nv`PUuEe!kNM0WH>`#Ztn@jM5EF;``jYEd z`DG7LZ;X|2ki}iPcv1GjgLjtZ5Z|E9V=ZrPWyN9GPi}EW@Q-to)x>-P3>BJDwCcl= z;|EqyG&Isp5PD=~Wkm~9QpC#>9Co$=8qVBFkGo=_b@f_*>qHUs3bvpj#q?!MLFy=M(obvnWOYqsXcQPaq0 z@~`E~+mMvVX{q0zgN1n`D-fKVl;oRDt}Zicc)-aXpHmvuGd9F>sj2j>aG z;3mZsP}+Wz2WJO-vxta`!mB#Nt5>fAKx--8^~PbHW>_yB=Atk^pP1dGys}bcYB^k@ z@mmszl@F8(nX7OLhIAH+j&2r?Jb-eKqQq`v8BRsGp>l_LNbo-(d!FkRXTX3!m>^=QJSV8G9)ruq@%-lxP~-Zpz=O7&Y(PL+RJ8J-8K`5gm4{X~ATaQG#GDd81P9rmXQ2#gUqBt!ECJ}g z-S*ljR1lG~Nacl#PE6P^Adj>^k0>h;hXEYRT**>}<+ji+pY#8kma^=LqcuSKCV#!& z9b2Q;1?bNEjY%O!_0Lb^;d|7|(Mj zWseXNvh}oKC|Sn7WZ#v2r?L(qj6IYr*|YDw=N_rv=l6Tx|9?K8C(l@B?z!*p{XOSg z*LALw@5IY!CQG=p>I3Zw=MvyWwir%?e}lkuEd9&FV$0a^uzT|wEq9?==C9k268Qzi zTVeOsm!P|0ul)LvhlkBo6e|AufS;Sc7YTdVzpMK>p!gwg<5X6#>jjhl>s9vBk7&(2 zLHc=!iTQifrmN%dE;Uu0fW=WgNQOP~Ddoa_@J3^@U#PDKWGUkjh)t`lA7g_I&wLHCH|1 zgn;&*%7TRp%Fr{h(*)rhk#%$(ERG<{Y77&yfzH3Oj)@JFzFq=33!5iYMD^P>30(+! zBe3P9TV?{Mi}Ys99;Aoi*D`#vKUA;eB~m}ZZ$s46*jiBAe+x&7ybbZV;jDV>L4Lgb z3_&U(a|qdvfe6OaLrFX~SkQz_binGMITAE4&u%3sgqR?g@)k@ppGeW$P(dB%0dhV( zg{Fwj&?oR=VP;_=f~W9u$nEeZ;=Ez4zukIyZ`)}(3CiOf8t+`zSN#1^e_rs}qgqgj z8f{Q$W`aNyjC3k0#>emVi?Gn zgLe!eJtJ^Jx1Bi8NmlXyi{^dIs`ch!4Bd1%tJV(WtSJGZzvI*Ha|uXq?;k4nFOqia zK;YktO}lbv=G~{@D{^ujqW55S-)2FWWJI#~|1$#p>TxWw2RaajJ&qw)?%zzZ4fcAc z^O~K!m+wQZ2Wqpj=i7@u0aXki%s>&rC8JDz8a`gtC3;5uy6_cNUH4mC+KyE%^y$5){1l&wabTk_mHyRCI%5(x2I^J}b=#tTF!N3)|ySsx^F*OcR7Os}9_Dr3V{M~0A z&2yv`B<&Dvh1EUqCG&0C(4!38(+Lo=LqVJ@0{wnakL2d>eexsdgqhIvVIc5n)vrw<8$GX1*T@s>c&C<(#-h z7hrE8Wuz+O6oA$k$Q8kE38BYpUY`Mrnq)Q^h@e25>PbRoYP2}ngBwHwFH}XD(EFPS zUGYzD&{JpwyBDii4UOKP0q3JTaUzieJX-+%r-@~W_SOfM5Sm^Uk2_%9#|=fLN5y(2|=?QYlBq*ewXL3Q*3p(b$NLi zkVn7~AaN z=PA$%fTJza@>938WRLe0VY`-{nc3_hG4aowjD}D-2n-ryj{vX<4odt^4_6wl>MpO; z%1BG^1CPiD5AH!jmb|=|Ci=}~qEF)#z{w_`Pyg%Ag!=^&vqz5}fr--EDIseEq{UPf zj&%%7-htz7UYO`^)h%oqHh>ETR~29CJ)XFZ3uBFRQb}mWInrr*nF2ZvV33L*WSJ@2 zVCj^yu&}(|H+|&LA%F8)XQd1k3<{XgbH)v{B*qC-9EEmM8$+R%$55ymD?@$Tf>LWH zKS<>uL+6o%q&%FJ0p#ds8;kzhXe483lJo9$Qqa*s`;`sg2uRbEBSsBCF}S!r)z#H) zZMl%x*9CB17&p+fkCA$5{osMWwNYb48rSjX*4EZwmad%FzEKrLgpNUMU0hN^145B# z$%8`K3rRkpj!Nc+NPgkY$K602@thO7o+EDscQnX(dGEv3WDpY4^itzrn4hOwY!wZR z$qM`N!vuJ2kQ%tUZh|oX$3-7ly&-elmdA}w)oZTF;71sw_Wpv|Do|sU35{ePct2LL z=?cpD4t}~(pnT2rMiEstPeYDwm06bJB;Me`mg*fr9xI`b?Q1FpRu4jr5g4Bs6x4Im6=Juv%}oZZ1`eZ3C!~(Oq5Vs|0@$R@MZ>y(EJc{vjkfr;@=4C$hjn<6 zjEtL$D_u_P-Md7=v?V-iYHA7_zLZoIsAPbzHEl!fJwkhe zib~bWK}JUA;lqV|tM5>0O&JaA;hm50!vv5rr8_+fd6 z0!$E|QZWG5glXWw2-MCY8vfKbZ!X(rg1Z{sL1O33xXzo6UIG&?!p3mtNi?0>bT3CW``6EmjUr$l@Iy~9XQ{= zXBprrY!|J46G{o{ufjZaXobR{?Fph-c2;Ot^nlL5<*kJS%cc3!hZCW?u-lcFmqS*9 zG*Vi$0%q*EFdiHc;WFNlaYp<_@ycjQvL9SbLjwc%m&kYtcsKQcvgs-sI9s)#LaME$ zP{&j%O8gwa`w0)D$iU8IW$f)i>L0e2oZ2(lKk!-q@2 z{1U_jkdd+RZN;{G;E(~p5=?0 zNo@TF9$Mm49ndeo0r;rA)20GTz!afg>=Bi=7=aGf(FE1}tHHEA2!||X(ux5yR7|3tl^}mtv!$*uf9x~6AvIjAZC`JYb08;#%={uEI*@b|Bk5$B= z(EcR|M_T;A))useckkSZBE&*+IvRk;>7Kf{Y}yr*kdPRLTK8d638?3H%3t8mFlB*` zQ;5;$>U0@^_lLlrvh#Ao3J^}E_Vk^Xdu}#J@0tSI2vA{mnv}YE-0Y_djqc+CIGhLi zk-h!dgM+(O53BAYBU6vCL*zAE@$mhRkUrQ;B-BC#EYkt=0pL;3vNjp~F@Qx_U0Uks z>H^)t-OfzI7YfRUEI}d$XrD?0^z*k4GK4_AfwX@uuw`bL1OO5IuV}9I5pLs9e)Htg zi%{GGB?cD&+*SuJx1N{eX|U)iL^pDP{i78FhGvx?kvi(;3JngQSe^2G_Gx2yx-Bm#uDBL3WgL^S<+t#d+RTM%R(k)frwGVVL^QVsW2s+N z<@_VX=?)rY>}dzkrnMQePpkNznhTj5=JdGLz`&UeVIjwl%( zUhN+2*}ELET~R`{*=mR32+HNiRc_>LDQXN5jzplZ`2Y#B`uy7jX{@ z(RBuD`FJ{tyGzB2XfpE~2pK}9P*4Bl{4c->5*xQ+l&vNAcxYQy4j}*{Cb517nBu=Z z`*j77OKOlv`mbk5WlNo<)id0lR!MBE-o`}e1NZ)7FpT0vD7x)VaM@)4-)V~NZA6CI&kia~ z-@bQuo(>3Dd^roUU|Rg@%F3ue(|lYdH(GrO`ic#rfMnh|ot(ewEw-@){3dMupiQ;? z@%h_RGSSz}T{mMRAe+sG0V#x}7En(Ne+q7a{1@W)`}dcVav#>g-{k}Ts}RELW385| z5!JE`duhU8U7(yYAj6rRVEbes6PJTyep@_)HHb~FB?ZPJFc3ZhMgW9BwjeqgdK0W6Y1JUM)5Ua&`UdG{q5s(m9LQ-a|vKHA-ji(J2UZ8n;druQwo?j z9D}tcS={G2=3{V5=D_W&o+c zdqeRjRE5?DAa@_{Em;M2g}@100Xr3DO}k>T{U+^p-Uhf~MuR*LY|R&kK#v7LdM*@e zKRm<3VPaPvl#Xn5AgM}{^#zxoRs}04Q8M2_$qcV`K_>%r?n3E;egp~4<|y&Kor}PV z`-*`P@_?bof(C;tHW46t@FM#pAzKRQM;9j&K?8l0e)!1VqE(y$mA3Ek48P*CGFK`w zXBZxq3e$Q(1q?h^R@UrsU8krXm81?>K-p9&ZIy^Y90J=SsMJ_9=zTy>BY8B>yqjeu z)2y9qxd;baYw#^ z=7(JOrN33HJrtRY6Rz`V;a@W{Q}k}nt*!Y2SQ(U@xRwnPhEnL!ht(0QN)Hw-_`1r}N&SztU!oSy9)L7OwKTzgqKglTNNB)a zvyDGNN4EiXajP?fbx%nT`v!t{aB_a!Oim3sqm+L@0O*OBMMX#OLxANz9vImt2x{jaC1~l zK&Ehlk`goNaTFH2F3bmsaNgKhokDI$I0Hgq`GM~D3XCIDEU;jI2gS8#O_+a-vtryA z#C_apJfk`U5)1%ejUyabpjXHEd0qGjv|=-7L7Xy7Gw@H)*t+#zm_n(7@8mwBk zplqZrga6LoKn5TX%SkZEAS!r%ZO6ZY&^C_!A8;no7zprLJq6FM#lD^mJ?EK{6ai!b zKm|!OfP=epT1-d2=HkyU-P4mKvWvq!2}plx0>Umc%)M`uP31 z;Qy#|(6>55;t%MBGd)9ae57_lJ~j1JSbx3EX2a%Kv|x2Yu8QlQ?zuDfUc@nyV4z2{_7{d zPr>oC7_Zfcu%lC=JAkCvHd{h=f=%^d)c4@$p>)k4SNL(16?!mN+O&Zh@l;kEda@0h)h=6Z$&fG5~jWXZ(b`8QNF^XZto|lVI6L+C&?(`zT#IK=B?P|^>aqctDh%=GDP7Nv}I&QTfD z&nSg-A@uQM$er!K3OHo7sgQGUF8+{wh9cLgBvKmfm!S*_AJ>FMe4gDn=o3q-&@>R@ zXMG29U1htvC4;V2pmL$~1|4NA`uFw&gU6gP5RieV>+(q+NCqiwQqo#Kz*uDXbQS*+596)p%r?Bj?{PwBja<@4p(@XWE* zr;iK(Bbd)@SG-2vo%Q}oC5UqIHz`i7+rfuMp%F7&YA6c_wCaBqc(fp3)kPoLId#5ofBzkG;)TqVUCNhk|ptn%{wy-K<1WZ zbR^kJ))&S&l;hZwRKJ|U@QPswc72}MTo9mOnlHwdiLKtNQP+*b9b_CL7ynd+8Wepv zHEFEhlJC6AKi1yJX{XE9>uf7yHT~+0_*I>o;UT;ke%gLPOaGKk_g}fX8*J3$J8~yn zMMV;W=$r`x7qZseG*6yP5w^2%L$?*6Wd+o9t18#mRHOX;4cp&N1(x{+E@_V>vo)Yy zrYpHR;va-Yi-kiRs&JXlSz6zn@;GC>J1NHB{MX zd`FtLTTM+~=`zdB7l#TAmBdb^+63-BL>?6F5!|0^uFS>ts%+D_7?gQjMD(Q zW5E=zw}p88BbQcnvJ$D=1&kjc$fNlulDZL|}(wW6d-EjzWgDW4-L!z^SHVi4_ zucgUX_+CvGDk5<8ybbIE<@7GPJv}^Wt7S}ykAGwR`m`@Oks7woQcu36^&1xdW8=0LsREZEiB}M)AIG1krDP3Gi*GC z_@{D|hy{6OEW&T1pH_vBez%BZ{CRgsGQ@~DaHW-lSS6ds-{#FGeM=%FbcR$S=n#kZDFidkv`T)A4@t}l&y4^srfq0arrvj zo~spaa+RmzqP#MLcS3@;O0yg6A!1exYNVdlf$DmdiOM9fK4yFDG?S9**49f}RrP{1 zzr%hevs=5^UKVy!;cCzA+ZJL(q^qdB7A`H#w&>w4Onfmz#^u%eLA+met%t&0w z-R^aPi_}X6M)DbLA6F)OJd13{-mD*a?X4u|(5*hLZatUQRmi8JBThdib*6`@w zqu>%7Om+10tpI)ON^*u?Wf6lX?D2E+=5fHhQBRq@IEd25b(mXPjtvlaIXKQT8wQtl z^Exg_B&99re=wbZOPYut_3B-jbhMb1+Aj!$WqesyKJ0Z(D@2E=9pVYq#rynB|21!I zkn8P_o*FJ6`jwn2NA`IheuHR%naDtk;BRi?Y)ARf1C?z(HK0uVHjp2z0!X zHZRZOCnxLY*4g&jV3}MTisGB!j+w0qh%xNnkKy$==!p(VL(9s_u3YcEt~OS!%p*w= zuOZ?L=8(RxA$KPq?Lduv`+CKRwUkE565)s_hzYf{6YG`&0r?grqf<;o>_uj)5{Z{4 zvPnKh;>1Qhq}Iy25n84_{kR(W0VIYMHH=$B*@E<##k7pR=CP*uTJ(wX+6>Xdr8K`s z?P#BWh<3?dt81pqqrlM6ZK&bpYjF_QXW#2|+DahK!J$YqA?e4&&yKmTmPyPX zLE8X*C3_<;7R)zL^$o}FnZXtUGq~gYXe!@qOMChsA^Ocq%%d48r<&JC zN!;9DZ4`Mc)j<$bTT|j(QNw?WrJiZ%g=8opj_6GR|`l-7gW!CG1RFodD$5WQ0@ z@)`Z4x=tB#Pi{z6B}Yzfa^LXKJI?Krd~axKD%>}7yy_)MrU4;wC*!pR^@k$_y!vuq zk}yU`#~*c8;EkgL0V$?;)lE~hNbq~rX7 z#0NN9E{J=D$x4wHrauCi9FtlDieaTsxzgv}|E6mYw7juOYn%1?^Lxf?ZJV*n#t%wM z)7UPZBB?&F^CtIWjwRN%FVyQ|@I5Fij>^baqNZ-HPNvN1Fi%93F+7wa6r!in*1L5u z`-vYQIsh8Oz}Eu^O)v^S1$Qq_F=#Ou9uEA4IYm`=U{{)U`fU47_gQqQ;> zz4e1h6TwmuA->jhfXR{`;7~5Z4R>sDSy@jPo}TGc(FP|_XS23XSoagXkXPiIUm~N^ zN#O|0hY_ileNQJa7D<;jxA5HODBoksuCp3%a;OAf!_(BD2rD}Xi#7#Oj3UhaqfX~R z$9_~XPg(=_>!Ttu7b?Mpus|GYh8r*Pw8W3F?!}4A?SYw8HS&*rjXs^|dl%I@-bs1* zUOX4iTCj5EX|A2w&8mY(ShcC$r@FH5lsldhSxF>mjFWRrwr?AyAgNAP`^VM;m5I;0wNY zFHrXkisD5!40GIR6_2pF1ss5Z4M+Q!C%!C0<0f4}0gZO{{hk~%<+z>THQ+&7>eicO z95~TEY$o=k$;BwvQnjG`cT zU}tEP9%^<*T`=yy_G%m^rj}vUCVGkpot<~)M}yw0AycXC+R`+Xls71^B(qTkn92=;^ypQ`$#Asn*8sa^iVxrm+G4uB z5YKXO4CHY_f_)s4kDHlB^{I~^FJykXeA)lOSlfq@7Tu}OQ7@rxvRgD7bm3O-y|lWN zeXrwla=)ea4{g=y`|N7P?052%V_v{bl3gZX_4WPea3tC6+=Hx~9E&q?e72UN?`8-B8jyIL^l1;`heE_)5!Yx`MO^uDPpb97@Vkm zc7x2XUj!M<4Qq=yEaNEH&CIc!M&b<8%8j_qK!Y|+GXY^9V<{!{bp86-$!>QfzPK~N zKRWl;dQjH;2z&R96Ve=x>~=l8-=Nd1erMcV3J5PT##?ka)BZ7=#>$CdX5!I5YHx zPViVi9pJ~;`)Y3uGx6E&#pDaB$02zY6)3czy(^BQ|AUDS4O5pEc8k RootTxn : NewTxn() +SQL -> RootTxn : GetMeta() +RootTxn --> SQL : leafmeta +SQL -> dSQLServer : SetupFlow +create LeafTxn +dSQLServer -> LeafTxn : NewTxn(leafmeta) +... +SQL -> RootTxn : Send(BatchRequest) +RootTxn -> cluster : Send(BatchRequest) +cluster -[#red]-> RootTxn : unrecoverable error! +RootTxn -> RootTxn : txnState = txnError +activate RootTxn #red +... +dSQLServer -> LeafTxn : Send(BatchRequest) +LeafTxn -> cluster : Send(BatchRequest) +cluster -> cluster : **wut? zombie txn?** +@enduml