From 9461f5ba55b10381552fbbecc30e73dfde04bd56 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 16 Jun 2021 10:30:36 +0800 Subject: [PATCH] docs/design: add proposal for common table expression (#24147) --- .../2021-04-18-common-table-expression.md | 334 ++++++++++++++++++ docs/design/imgs/cte_computation.png | Bin 0 -> 12131 bytes docs/design/imgs/executors.png | Bin 0 -> 9328 bytes docs/design/imgs/logical_plan.png | Bin 0 -> 8734 bytes docs/design/imgs/physical_plan.png | Bin 0 -> 9447 bytes 5 files changed, 334 insertions(+) create mode 100644 docs/design/2021-04-18-common-table-expression.md create mode 100755 docs/design/imgs/cte_computation.png create mode 100755 docs/design/imgs/executors.png create mode 100755 docs/design/imgs/logical_plan.png create mode 100755 docs/design/imgs/physical_plan.png diff --git a/docs/design/2021-04-18-common-table-expression.md b/docs/design/2021-04-18-common-table-expression.md new file mode 100644 index 0000000000000..b61e4bb5a918f --- /dev/null +++ b/docs/design/2021-04-18-common-table-expression.md @@ -0,0 +1,334 @@ + + +# Proposal: Support Common Table Expression + +- Author(s): [@guo-shaoge](https://github.com/guo-shaoge), [@wjhuang2016](https://github.com/wjhuang2016) +- Discussion PR: https://github.com/pingcap/tidb/pull/24147 +- Tracking Issue: https://github.com/pingcap/tidb/issues/17472 + +## Table of Contents + +* [Introduction](#introduction) +* [Motivation or Background](#motivation-or-background) +* [Detailed Design](#detailed-design) +* [Test Design](#test-design) + * [Functional Tests](#functional-tests) + * [Scenario Tests](#scenario-tests) + * [Compatibility Tests](#compatibility-tests) + * [Benchmark Tests](#benchmark-tests) +* [Impacts & Risks](#impacts--risks) +* [Investigation & Alternatives](#investigation--alternatives) +* [Unresolved Questions](#unresolved-questions) +* [Future Work](#future-work) + +## Introduction + +This proposal describes the basic implementation of Common Table Expression(CTE) using the `Materialization` method. +`Merge` is another way to implement CTE and will be supported later. + +## Motivation or Background + +CTE was introduced in SQL:1999. It's a temporary result set which exists within a statement. You can reference it later in the statement. +It's similar to derived tables in some way. But CTE has extra advantages over derived tables. + +1. CTE can be referenced multiple times. +2. CTE is easier to read. +3. You can use CTE to do hierarchical queries. + +There are two kinds of CTE: +1. non-recursive CTE. + + ```SQL + WITH + cte1 AS (SELECT c1 FROM t1) + cte2 AS (SELECT c2 FROM t2) + SELECT cte1.c1, cte2.c2 FROM cte1, cte2 WHERE cte1.c1 = cte2.c2 AND cte1.c1 = 100; + ``` +2. recursive CTE, which can be used to do hierarchical queries. Recursive CTE normally consists of the seed part and the recursive part. + Seed part will generate the origin data, the remaining computation will be done by the recursive part. + + ```SQL + WITH RECURSIVE cte1 AS ( + SELECT part, sub_part FROM t WHERE part = 'human' + UNION ALL + SELECT t.part, t.sub_part FROM t, cte1 WHERE cte1.sub_part = t.part + ) + SELECT * FROM cte1; + ``` + +## Detailed Design +### Overview +There are two ways to implement CTE: +1. Merge: just like view, CTE is expanded on where it's used. +2. Materialization: use temporary storage to store the result of CTE, and read the temporary storage when using CTE. + +`Merge` is normally a better way to implement CTE, because optimizer can pushdown predicates from outer query to inner query. +But when CTE is referenced multiple times, `Materialization` will be better. And recursive CTE can only be implemented by `Materialization`. + +For simplicity, this design doc only uses `Materialization` to implement both non-recursive and recursive CTE. +So there is no need to consider how to choose between these two methods, so as to reduce the chance of bugs. +And in the near future, `Merge` will be definitely supported to implement non-recursive CTE. + +`RowContainer` will be used to store the materialized result. +The result will first reside in memory, and when the memory usage exceeds `tidb_mem_quota_query`, intermediate results will be spilled to disk. + +For recursive CTE, the optimizer has no way to accurately know the number of iterations. So we use the seed part's cost as CTE's cost. +Also we will add a system variable `cte_max_recursion_depth` to control the max iteration number. +If the maximum iteration number is reached, an error will be returned. + +A major advantage of `Materialization` is that data will only be materialized once even if there are many references to the same CTE. +So we will add a map to record all CTE's subplan, these subplans will only be optimized and executed once. + +### New Data Structures +#### Logical Operator +```Go +type LogicalCTE struct { + logicalSchemaProducer + + cte *CTEClass + cteAsName model.CIStr +} + +type CTEClass struct { + IsDistinct bool + seedPartLogicalPlan LogicalPlan + recursivePartLogicalPlan LogicalPlan + IDForStorage int + LimitBeg uint64 + LimitEnd uint64 +} +``` + +`LogicalCTE` is the logical operator for CTE. `cte` is a member of `CTEClass` type. +The content of `CTEClass` will be the same if different CTE references point to the same CTE. + +The meanings of main members in CTEClass are as follows: +1. `IsDistinct`: True if UNION [DISTINCT] and false if UNION ALL. +2. `seedPartLogicalPlan`: Logical Plan of seed part. +3. `recursivePartLogicalPlan`: Logical Plan of recursive part. It will be nil if CTE is non-recursive. +4. `IDForStorage`: ID of intermediate storage. +5. `LimitBeg` and `LimitEnd`: Useful when limit is used in recursive CTE. + +```Go +type LogicalCTETable struct { + logicalSchemaProducer + + name string + idForStorage int +} +``` +`LogicalCTETable` will read the temporary result set of CTE. `idForStorage` points to the intermediate storage. + +#### Physical Operator +`PhysicalCTE` and `PhysicalCTETable` will be added corresponding to `LogicalCTE` and `LogicalCTETable`. + +#### Executor + +```Go +type CTEExec struct { + baseExecutor + + seedExec Executor + recursiveExec Executor + + resTbl cteutil.Storage + iterInTbl cteutil.Storage + iterOutTbl cteutil.Storage + + hashTbl baseHashTable +} +``` + +`CTEExec` will handle the main execution. Detailed execution process will be explained later. +1. `seedExec` and `recursiveExec`: Executors of seed part and recursive part. +2. `resTbl`: The final output is stored in this storage. +3. `iterInTbl`: Input data of each iteration. +4. `iterOutTbl`: Output data of each iteration. +5. `hashTbl`: Data will be deduplicated if UNION [DISTINCT] is used. + +```Go +type CTETableReaderExec struct { + baseExecutor + + iterInTbl cteutil.Storage + chkIdx int + curIter int +} +``` + +`CTETableReaderExec` is used in recursive CTE. It will read `iterInTbl` and the output chunk will be processed in `CTEExec`. + +#### Storage +```Go +type Storage interface { + OpenAndRef() bool + DerefAndClose() error + Add(chk *chunk.Chunk) error + GetChunk(chkdIdx int) (*chunk.Chunk, error) + Lock() + Unlock() +} +``` + +`Storage` is used to store the intermediate data of CTE. Check the `resTbl`, `iterInTbl` and `iterOutTbl` in `CTEExec`. + +Since there will be multiple executors using one `Storage`, we use a ref count to record how many users currently there are. +When the last user calls Close(), the `Storage` will really be closed. + +1. `OpenAndRef()`: Open this `Storage`, if already opened, add ref count by one. +2. `DerefAndClose()`: Deref and check if ref count is zero, if true, the underlying storage will be truly closed. +3. `Add()`: Add chunk into storage. +4. `GetChunk()`: Get chunk by chunk id. +5. `Lock()` and `Unlock`: `Storage` may be used concurrently. So we need to add a lock. + +### Life of a CTE +#### Parsing +In parsing phase, definition of CTE will be parsed as a subtree of the outermost select stmt. + +#### Logical Plan +The parsing phase will generate an AST, which will be used to generate `LogicalCTE`. This stage will complete the following steps: +1. Distinguish between seed part and recursive part of the definition of CTE. And build logical plans for them. +2. Do some validation checks. + + 1. Mutual recursive(cte1 -> cte2 -> cte1) is not supported. + + 2. Column number of the seed part and the recursive part must be same. + + 3. All seed parts should follow recursive parts. + + 4. recursive parts cannot include: `ORDER BY`, `Aggregate Function`, `Window Function`, `DISTINCT`. + +3. Recognize the same CTE. If there are multiple references to the same CTE. + +We use the following SQL to illustrate: +```SQL +WITH RECURSIVE cte1 AS (SELECT c1 FROM t1 UNION ALL SELECT c1 FROM cte1 WHERE cte1.c1 < 10) SELECT * FROM t2 JOIN cte1; +``` + +The logical plan of above SQL will be like: + +![logical_plan](./imgs/logical_plan.png) + +#### Physical Plan +In this stage, the `LogicalCTE` will be converted to `PhysicalCTE`. We just convert logical plans of the seed part and the recursive part to its physical plan. +Also `LogicalCTETable` will be converted to `PhysicalCTETable`. + +The Physical Plan will be like: + +![physical_plan](./imgs/physical_plan.png) + +#### Build Executor +Three structures will be constructed: +1. `CTEExec`: Evaluate seed part and recursive part iteratively. +2. `CTETableReaderExec`: Read result of previous iteration and return result to parent operator. +3. `Storage`: This is where the materialized results are stored. `CTEExec` will write it and `CTETableReaderExec` will read it. + +The executor tree will be like: + +![executors](./imgs/executors.png) + +#### Execution +The following pseudo code describes the execution of `CTEExec`: + +```Go +func (e *CTEExec) Next(req *Chunk) { + // 1. The first executed CTEExec will be responsible for filling storage. + e.storage.Lock() + defer e.storage.Unlock() + if !e.storage.Done() { + // 1.1 Compute seed part and store data into e.iterInTbl. + for { + // 1.2 Compute recursive part iteratively and break if reaches termination conditions. + } + e.storage.SetDone() + } + // 2. Return chunk in e.resTbl. +} +``` + +Only the first `CTEExec` will fill the storage. Others will be waiting until the filling process is done. +So we use `Done()` to check at the beginning of execution. +If the filling process is already done, we just read the chunk from storage. + +The filling of `Storage` is done by `CTEExec` and `CTETableReaderExec` together. The following figure describes the process. + +![cte_computation](./imgs/cte_computation.png) + +1. Compute the `SeedExec`(step 1) and all output chunks will be stored into `iterInTbl`(step 2), which will be the input data of next iteration. +2. Compute the `RecursiveExec` iteratively. `iterInTbl` will be read by `CTETableReaderExec`(step 3) and its output will be processed by executors in `RecursiveExec`(step 4,5). And finally the data will be put into `iterOutTbl`(step 5). +3. At the end of each iteration, we copy all data from `iterOutTbl` to `iterInTbl` and `resTbl`(step 7). So the output of the previous iteration will be the input of the next iteration. +4. Iteration will be terminated if: + + 1. No data is generated in this iteration or + 2. Iteration number reaches `@@cte_max_recursion_depth` or + 3. Execution time reaches `@@max_execution_time`. + +Data in storage will be spilled to disk if memory usage reaches `@@tidb_mem_quota_query`. `MemTracker` and `RowContainer` will handle all the spilling process. + +Also we use a hash table to de-duplicate data in `resTbl`. Before copying data from `iterOutTbl` to `resTbl`, we use this hash table to check if there are duplications. + +We also support use LIMIT in recursive CTE. By using LIMIT, users don’t have to worry about infinite recursion. +Because if the number of output rows reaches the LIMIT requirement, the iteration will be terminated early. + +## Test Design + +### Functional Tests + +1. Basic usage of non-recursive and recursive CTE. +2. Define a new CTE within a CTE. +3. Use CTE in subquery. +4. Use CTE in UPDATE/DELETE/INSERT statements. +5. CTE name conflicts with other table names. +6. Join CTE with other tables/CTE. +7. Use expressions with CTE. + +### Scenario Tests + +We should test CTE used together with other features: +1. Use CTE with PREPARE/EXECUTE/PlanCache. +2. Use CTE with a partition table. +3. Stale read. +4. Clustered index. +5. SPM/Hint/Binding. + +### Compatibility Tests + +None + +### Benchmark Tests + +A basic performance test should be given in a specific scenario. +The memory usage, disk usage and QPS should be reported. + +## Impacts & Risks + +CTE is a new feature and it will not affect the overall performance. +But for now we only use `Materialization` to implement non-recursive CTE. +In some scenarios, the performance may not be as good as implementations which use `Merge`. + +## Investigation & Alternatives + +### Choose between Merge and Materialization +Most mainstream DBMS use `Merge` and `Materialization` to implement non-recursive cte, while recursive cte can only be implemented with `Materialization`. +`Merge` is preferred when there are no side effects, such as random()/cur_timestamp() is used in subquery. +But when CTE is referenced multiple times, `Materialization` may be better. +Also users can control which method to use explicitly by using hints. + +### Materialization +For `Materialization`, most DBMS use some kind of container to store materialized result, +which will be spilled to disk if the size is too large. The computation steps for recursive CTE are all similar. + +But different system use different ways to try to optimize: +1. Try to postpone materialization. +3. Pushdown predicates to reduce the size of the temporary table. + +## Unresolved Questions + +None + +## Future Work +1. Support `Merge` and related hints(merge/no_merge). +2. Optimize `Materialization`, pushdown predicates to the materialized table. +2. MPP support. `CTEExec` will be implemented in TiFlash later. But SQL used in CTE definition still can be pushed to TiFlash. diff --git a/docs/design/imgs/cte_computation.png b/docs/design/imgs/cte_computation.png new file mode 100755 index 0000000000000000000000000000000000000000..938ef23f7cd6b98469c44905b0669a578b03c7ed GIT binary patch literal 12131 zcmb7qcR1B={P(dlOIg`+jO>+TWv|GV>_kS!!9n)QEc+nICYw~|u|lDf9kLFQV`s%N z!*lEVd#>ktuIu^dab3~*ocnW+_x+x)*ZYOOt_CF;BN+q&p}eQ5Y6yYgr-J{VU%>~z z*TQd-fIoO%h8oI{cZ1Ap-~*wfl8zDt@*#=*?_(nHnbbqm%nJgchp@KV#nlf9VwB{h2;-=eBw!&Fe2n!~2_%G>X5E<9RiQeqM* zM9A+xZ&dLY%jUzXdXsTF&H4s^VUcApbs}2wk+=)5#*DynWg%}=+f1Ke8gj*V@OZO7 zQ(XA8Fj(60;1uFF&(K;_R1|o)rK6)W(&krexX=*Jpt&VqxR|tcusWF3C*tbuqW19Q zIEZySkD$P%iT~ld*1bVly*&9PNciI-ac>0TPoMB7ton0a&8?7vj*z_pm}j6PNApE) zZmztX99AqIDnnr%xLF@_l?`DVrW10u@#xdV`6qy+}YWga)F|+pCxhMFfFas z!v!5wiW*5rcN~45oPU=R-T^B3^k90L5yn8sfeNfKul_E`VmpJVH&1x;0^Zt0!5J|) z0QQuo#g%yFJ?*UjqT8OP<`!#_E=NWm$JK1bLk!KFWmw8CnkR4gj5&tD^!MuODv!c3 z=`6WtpVAFvOYf?3yQu&Iuj(N$&5qUg6vN8#!8`@Ml#T_+vzE|{^R6y@7^?(W`vttI zc1ocAZ0TBa_Rz|nWf=AQ_V)IMhMo@)p|4AbrcFt9RNzy0cXvDjFV>+OoRY_){od3-dAz zyTyUSNC;63z>*vWj~rNoKQHD9?v=e?awJX7(snZId^+Vy_o!3vW-|?W{`gcI}0g$#0L2|y)cabqfMImyHB6gc~Q9Ko0I{5MpT|O6luBs;eS(a zPu=fPR5Bhld_bdelb@)BknSCA!miFPnx+1MDEZAR&_+D;thkEKmM&7bVw-MHgH+Xp zCRC}A*h+RdG&~dE<(G`y6Zn^%czwj_z?a`oqt17`vaE$QIu+M49jQo=qj^sEzps!X z(Tw#Tzi90olJwxgu3nV8GsB^p@Jm(6*4z4WJY$Ae8t%Knma8%66t^}P#-h%HT1uQ? zLVtX<%0xT2|4o{VSZhem&6I*TVl?S7C-h)>5iy0!e$+}y|&L89^J_3J2#FTx?a7pY&Syg`nB7pe+$3}2AU!>b5?9(g;{JrtE*$_E|wn zwY2K6Q>@{04&o8gf}GGzA0Dod)gIn??<38PKLjGm=xSKfb;cAiYuC~DT{i^8DOM0WvQrcCgy&vO3>bN zvJ^)>f4wd&cJ1@Wc~YpV&3FA;yENpC(?wYkhuZ!mw|9Oyl%Yy-8O^(&ni7E|a#s8D z; zxs|JgXmPC=(j2XgI;FzWlZEXY9lhRPwm%=Q`W}iiTsipp#p0^a&2L=aU zX>rjK&nF)r`)fUdcMo!)IM9=mrbQ$3_)@=)j-)dBfOB*6_dg!Yk>#W4;7CnPP1lOD z_~4}p*EHm%IXFB-K@#(`-rIioN~C#5=I#Su5-MpaDJh_bu*VORG0D*2-d^60T?HA& z1Y-M+k&%@9gdU%JdU^)3U?|d-=h$c{F5qq|8u-1 z92}^wUPYZXgMoAnP@yo~YEVh-8*&l+v0g@xna!7m8eF+nSP*|NJ40Gu=MG?yN z<^z|JBIU!sNKx_}Pe3EXY+-xJmkdo=Ju;Y`mGwDr`Bmk_#^~s%H%X5@!%c6vb$gJ% zukS;3g2|1x!@1MHdqb!xT|6|U+wreBVIiyLFGTN$KAq!gG9HEaJ35kVC&rhgrrwZI zIXpNpRt}4jgu!a8T6!p-$!Nlz!nYPyuletG)8_?lp{AzHF#Z;?TMdsuG~Nzcku@?h z%DnI3>Kd;bZu`MhD8utse4=x`)X%q@&DWsu+?qv%N z+FRBkiw5!Sw?(DC_|o3-`1ttYTwoG$nG5@vda>bCd;6%tC~?QZx4)q=Qr;`U$A6aU z>+b~F46uS&=gnwX5^_CeX+Uk<1dLAzQB`Pa|Acai*aN1?j=lL!?niC7$jJ z#;5>7XKH8e^%{D5`tn%E_Yf8a??B)4zsE`PC#!3trF?QWyw7}nS5{VpEt`h-R?L9f zJFm4jrSb`&=$vCTPshJ9lGw4H;)Bhlv$=!^T&EpA#aLT`7g zqN!DbrEplwFm|9@??Ix7LZOLnV!dbFFgu+dAm0=e5<(~$h8`^Aheae~NRu(c_)B%W z4M~T~(x%Uas^cH6)l6)lEC)|ph{L@84M@iB?0CT8iR3!t&OYRNVvVHr21iBbt4AGA zrNVBoqc}+CynXPypJ9Ir?-USv&?9B>kW97kKHSDg-{CoyHkNM*@xc2xXJPQ) z-Cl1G4?JdH1OnN&G#0qr+uQ5?Y<`yJJ@da2Us}pj*w`<&LofPN=G#AAoNZtHT~Q#9 z>dNGngVJ^wXu_}QnG}S`rJ`No@B^G#i-;vKCUD!uhUMnf6`!K7POF_u@4`-c2vu0df}czAf05SiqR>pwVqBW|n;a*9z8D5|GKg_ zr*X5))6Mj(mXzg!MvqmV`8!}Gdms%J>N?T{TsoLH+gUQpufny>57gfe20aN?Gs4dEyL={ zjQP@jcvt42pENWyjQKVJp@xn=eNIvtKCq+i&kThft>#kXN$3I*s+KnI>L@hpgIhm3 zVVl$lI=z!vViLR@7jWW+x+B|Lfu1CE#rmGK5;j?1NE3kL1U!R+;N|({vRaM2bO5;)^*$<%91M7^QKH7YqRc@VhrI4J$Z3W zmNsnMnh!-ilah?F;-iZzHhe)YBIm6Et_&q&?3Y+Iu*%*^fMQlgOTb_-!B3hmCaF{q zz7g;y@l1-j(Ar4K##wKXC$(m+BzDy)XEP?nhEvni&Hj7KRW2br5o~gY7%vb@EuZ!F z^r)jOw>F4KeD3tF1?&&ABbnUW&FufDC{%f_(iP(U^p1?L`&jM&6>JIxZc94>3y;ZG z4$Kw0F_fpEAS$YKmjFfzWmH8N>CQ`EJ;*^Zr$=MfvHMRjQ9bzTy;OD`&#j>81btkkq+iA5!Us~L3rU&v3HI* z|K9v7T~u;{T+#V{h%FI^W(^?lev4qRB@qc!(n4X~Wp=*`ybEMeeHp@w@R^WpkYNG% zWN_dIW0bY{6@t%v>}!vNsino*w9MTEb-F#IT-rnz;@xy9^{BhFN4bVvK1!n2?7uw4 zRea%3s)D|n3dj(#Bbd&uVd`uMXkh*zRxABw|dw>CFbb#!n<#`-dBh( zXAGw{xU?;IroT4-wLsdR*XqZ_#KeyuPb9)N5h%+Uwr>Ta9kLC$$Nv2aqeCgn#^*oF zs`h%cjSXZ+$}0Gr_P$aVIP08i>iuc;v}{_TtARG1B43=O2wVLw{LvEmHMo2tjZOGDedzQ~e1xkKi25kFiRwPrM)sblx1 zsMTne=^9SOC>XuRt%T5st)OoiHWL3qXO3bhV5oGji7Jvx!IU~hm6^@>yz%F(Z|j(h z)}^_d;Xg-W{Krkb_j(aTgw_D_5CF$CeJlkA> z1?SujW0^|Ur#z{?)BcXHVflJERo!uo9xqwc(mT6z7h2pnmHj}bnon@X*O&J#!ry%A zemA;;Qs z>`KK|ClhM!^s3{ESMkeiSws$paS`fI6ACM{U(CR%Y7mI$>9io1Mwg(z_{!zWjJ?b*`)SXhWZp{?o?uaWyx-FYv3viE-4GWL3Np$}gA&)lcG_^HjE9 zHO`QQKmSzwYlDHy{o%T-IotB)2+tjIL=&oP-Znw)OSQ3Ge}2l|i3??{sg6gddQ{n+ zn2~ROyn>)Q`k#6ze?t(ioQ%M{knXD<2h~Zhps@k^0y&8T=fX z_Gk+6^AT_$g+420w0peSuY60A6Hdmb!yD_U*iuS1^>am z=*Ra!CosVp8XHlxoeS8+)f-vhSPO8ijkfDD?{Poj#8i>s>z;c6aIsDyV~G+Fjj z+H2wx^!Y<}{Wat`#7l)31~J%I(_G%3YHi_YdV6E|OJaUT_jy37_z?Gttz-!A6si;oWZr^x!h-QUc^&G0uU z?n^E|D9)VRBz-r*%Q!lAC3rxvsDS3)RMCNiURzQ)0e9w@#sWUZ;ZC_~Oq! zMCG3UY1(37O|oYSYf_Z?n(yS+10CJrPeZeqyAFpXg5eCZ20}xQa77zyX_wyzuQw>^ z1`?d#O`1L~W2APGXsi_6`kRyGt8Qe=e{{<8zV^DgG$vMc=At0yom*nVZS{H^(YpAr z=_ks#nd^5l8VbwKe)OcMc>U2gALd6Y+3>s^0s!OsUH3$T%m#sL-6xSAtK87T3WXvQ z&Th2eAmzQ_zfS&B!d?lAEUZQ3I>k#zpYAOuxS7PA%sq4Bu>O#+C-RaUor5SOMf zRZy4z5E{{;drQ8)_0CK4)`tG6BkYgfZ`97xMX8tpqY<_rLnMrngX3l8)O{n4V?6vP zW6SuT>t({R4NCNGWo&vZDu}B;)&@sdH|D=8A+DN;$g6Z#b5*~7MN9_MD)m(|ADuEzi(=4YHaL{3jf@leZz3-{uscj-Wy7xI3@VM zGQ-7GV3MfM8TguTamR?l5_JKMni4&e{c?%l%7|2uJO#8361`XaWFdD`XgOQ|Kd>s9 zjs^ilMWef=f1;j3%|?%A%*2KJ9Xu5MhTgO(;tEgi zKawakxo@H-`d00|0dyC=Igu9U@@~+w-c2_Hb6X-73pf@4BQ4@nWKkX(N2s@CI6f{v;xW^ z`VKKr@BuD_Lyh;H@%1o^WK1*|d(TBYVp`7*W}dda`8_y~%qkn=;%JQlyvSv?`6GWrnrwR^M2%98X2#op z)=8o%TjpVn_3YG?!D9%O|3-D=$se5k7zk=`NYyIesH@nI}?g^QxzUCl9SVId*& zQ)vR6!~K)?Ff5&nCMbAU;j**zKiEQ!7S`7*$Ot?%g)%n!@{XtDzgu(pZd1shUNjFB zvUd&~Lelz{TqRxng}$dKPSANXoAA-H5)r7zkl^Brc6?RWS4&Dtp3sGfuO78}Ea1Z) zR}B~G#>U?N!ORp?^|F7g?B`jUlxG5$#$6*d`QWJD*Zkw_d-xHYY6gPPzkLNCqLVR^ zLOa^UhTN$JdV!j|Enc6MDYYil)22Xn|1L=Lbweh+eXSz#R%EoN@jfjJE#MBl{f3p4 zOBZ5;ro*Tz0F_70X6V(IBkRj+g(9`nzaGuXx1O-q6V3gSdO*wEXZb;rd6|N)P$Z8= zm4t0a4UMU#)05T(Xrr7sflWj0(TI;%fj%hg ziw;OTaX1`6?`TQqBPHkfCTbI-vqaZZIok+vt9I8*PErd^XmV4c_;$kDqXvl9$zNFI zyT3+LCR2Hr?=G@k9Mw-)f6l=Ce)x}73-$JXYltV{dvSgi6-5MZHBAeZxk!AF#hyCT zl%X@I?H;di+gYvPCINr(#}C@P5ERA9 zGZN5+azRJ>-2|o^D+(9K6OTd%++LO444Y2QjFK>6DkdI~>1^HaaY~&wO_3LHiF`HM zscZ{&(v0nv+zq*Q6-*J(7IOT@UM{`c;nV5t_SRfmALS25`~0p&L0$`LhA%+Ctoz#tvmU{tK) z89t865&{8~1s*%tc8B@p(=aUYdpaZ&1W=NARZ*9(fu}BDwf~PHVWSlj8{W5Q$%tfj zV*MszrglL=mi2w#zL7N(X_9~|JzN#Z%E~Ho&98L|4b6Kgwd1NX@u0%$+maZLqT+*d zJ)<76$2A~Grg7&7nBKPI<%I=~Bz*cZnC8bRSAZryW+k7e0rT{()@1@fRdo5CdXWKN z`k!ODkgJrIKuW6=vy~&#Dpm>^on`<@^qV03_q5~Nw{HgrlJqhM14TE_C(}gsR+(5Y zTr)MI1QoPj3%nY(oijyP*gD7Ba}QP#KGIFS_!yw(s=J+0F}8tIE(qHp280jZv|8pR zxE!U6jvU|P0YvRnr$kp6s&F}Byb>$9j(#}a)0*RSi1RHUak{Ukx9H~OaRnSk@X|4P z^+f5a^}c0xQ8hF^X->nu=-V{RO9fbZw!YsY!OA=tIF*aH#j62_maQo}PTW*&nZ$N5 zB92SKv!Wt07<&GoPI>_Q$|uuAN3bi1sOLvdKfkK$gH*{004aoX^#~->l{h2eLd?28 z_QWElK-+%k;^h7Bcd6a*NeC0K6)_oWRULYay8V%g#HO}gA$-+~ur!i7SF#wh=_=N_mWX?z*i$v1~U(lO*#GkL#= z{`4&3>~cH(0|Pl*7Ah(Jv?^ni(MHBLYH@EezK-+v4Yj(FK0N4ky3I?>`L%rRBeb6J z49vt-`(Rm_Ya7QSs6Vr8ls}hzq6&~yKu{!O$K2D&ZzbF+dZ=!|?p|0CfF4)II>$rz zU-tO@s!`YOeCrcK$MGofk?iBSrs`YM-!dDs-zJa_{h(}CEh4}3KeN0wuU_8s7xETl zPJPbo0!?9Xj5ac~Su0$=b3=#5Ub+t91`p1SR85P!qMI6b_=-`XMpSLEXKC25UQlQ9 zb>ux3DQRi6LoI-ZX^~9q_lDqIPo$pSyS>MIEeH83k5$yA%r>&jKSGHYrYv~B`e&tZ z>FRhZO&aZ4w{s2W`&DkYfUZ_g3ofE%GTr!}45ChO?G?s^p2O2@Oc+gKX_DZq!Ho}`%@PWO%tnG5bRK;=+ zNl8y=o`pp|YQTz#jjh)`Hhc+Oz&$|jbzJ%CPz#I;Y#R7Tc2Z>E7(1#n1gIPoAjH7q zoC2?b1kke18)S<~WqRLI<-?l6!9n?pzk7eDAgiV1^g_dP?EoJHa4SrO4OrH{(K5&^ z8O3i&^wxeLxc>hAJ58$?_o5W zzO!6gXrxR5XG+Et^~_?=Pc~{oPe$`tnVHKvAeA5ndN??gJX}ZsVnn{dCI)B6UButG z#;gNxo++8^VXJjJSju?R4qhw$OHD`wR0{e4osbNVh(qfz1+9HA25l-8kr0>QgDJ;} zcUFrIvP59hNW*et0Xo#Y7}Ph*^8Qv-RMhF=cBLz%&pycl)0!QUp@ zJF7jJ=cAlbH0Y3e3Dx>e)N+Y-LapOag&gGJa5|lpyXTE(;Go0G+^K85M#;ogId5rw zV$a)s{UrTsszX>M1l^NQzb*X%4xipq@$S*zzpdX0Ji13lsIA;{mXpN5>FCP%OQ~|m z2c0)q&D4F_lcf7~#89KYeVf)3o_os!2?>-pN#>E|<+o2V{;}6#bgQO?oG-x(oT%>E z{Xb&kqaS7Al0GXr?*g|1+H%oPFeiS~zkdtlzFsc;KkPW3IbdW!!bT2v&cxh`Q;S-@ zxZh8oKI~>m+T;KPeA!Zz*yOR$H8+{I2KA&}RL(lzXbF&U?7Wy-zNG$V_3K;GWVR4m@Ja0#r z(Wt!J0T{!H10i9f$7B~;rSmR};*|k}r%(Da zC;8QWtO+^I1)sye!unJD{JQJ?Eg{>>lt4z2hat+BtbLA>><+IF{oTI{SiB++!+RpG z0Yb!LhS5*iKW7nTezUH7iOl_iaQ3=yBhMv|I(t(vKJ^MS+bs!9&Rc!fU-k^q@|+@??iDTtVM zz`M}Q0V6o7Z7eEl>0jA9^RqOsHZii=vzg4rIIwP*F~%F#I(Sr=-PhVuyc7DNfQw?h zWxM?pG}~?e{_TNM0ci%%Hu&$&P5WB2Z|;m8Y@c!zADVmT={&UZuAD6uUL8fQy$Ng1 z?*vdMh!~x`w8gnlUyrv6+;x!!Z!$A)GBE`%Ccv6FWHn}~*u2Wau+ia$3IsCp^12!< zl&;0y?jWrK>MGQfWFCW`{;qCSzNIE~jlp9@=cm9!p(F%z{h&?m^B8O* znmStK62g;F?K<@Dyy0bM2>u94-hd%A7X zh#hW$MMZC;&X&HWuUXKVcIo^7BvbkK+E)4GyNM0Za~1&lDVx)EIZ&v4BHoqTelr`&Q3Ff4Ep`SqAP3Kzv*20m*gj7W^V?qL|iSGb_5+XV3MT+0+5*x6KHO2QMB(} zHkQo!udrlZnGXZqpnEGri?DOb%S~O*yOX0=G~%~boN?LaQeR(>!}-AFoB%n!4?(ub zs;mW_@0Ti^tv~#PL_thsrKJ3p(jR(!26PJ8CmVPRD%X-x>M#V;^|{vwutH2mP(6*ema^ zb{_!YJUahF4;Q&|xxQjEa^~y0V&dVCYqEI(ZMsvSAyX(7UjX6Zq3AmkyGeL?1Wnve z)MzH=8;F`?W5(si$P_#k^~UM3TQ#kJ$SA$Yl`hlmt8W293@_cBMXalFQazKqsp^H2xEL8U|RdH!DhLeQh{s;_O0Kx5;oDyXSwBF<}>@ zRjiz)diVvFxKDBJo#f$rVeS;VX~(Nfo_r>pC?trB>-gZCg|STP`z^VRa4on#)x0HBTe>GC1!l)7r&lErs$3uCE) zYPl4*FDs#Z58b#xIANpDezpnPEk3diNH^^EQ4~A{dv==4=-sR7B#a{MlEs0wuN54l z5O|F<4Wy2>xfsHQ95^VWAib{c?o8#bP*}l^wWVd%#D--w4e*l?DdB8PvhC%F6%LdJ z87RLx{57{yfRBfGup$HB0(T6_j={3t{(lV7WoN^migeV(3#(7Ya1xf z_EJKgUB0Qi1v(ojuWHqsKM0H||0$bxI0O3C`0*g75hotd!_{l(v!h)*XT%OooLaqa zrfa35~ltC{e^7%*}(IBRzyjmq-mDB{F~# zqWAcH-@VU$?!CY7k9lC9v-jESthHwC_kGtS80u@15;G8EVPTPKYpEGyVPTg6pL>LN zfGb7ICG5Z-Y=2`-6|B0CO#8q9_o=d;G8R@t8p$tvJYY=ZqxIMy3+o=}?+07knCm+h z78jnjnzCsy^f*r-`EcTXEO`}Ka$=FK8Lc4&tva4rfD$eJoMHi+unS{lK_zjA@rV8x z89`xw;=2`1M$`Q}AJU(+!g+#*l z>?LzgKiZ^hz;SF(j*s%no+KUD z0ypnKptwxLZL}+Fq~V+@fA2P|9`ByDOKRarv!o4d8Zt)HV1tArS1ZbV!Fl*m5&y1< zzV45c9NQWB>22F1*X1Ixv9mdz?Y$o6a%idc%_771m)Xdug=M=TMr+d$!@&#Fi-K$Z zvo*?{0U>8Bjp&NUhvX9y#ly!xvn#hDx=xHtBWs4LBoBo7rSMFdHTS6?4_P2Ad`u9i z+APZT81BFpbehm_O^9m$*qB}QoYHeRu7 z%%7U8u{-}r2GNm&(rCy)LPf4{9%?=oXbqEwF3l^WSILobwS@jT%VJ1tx&Pk6Svx;- z@U5!zsfW6<Lmr8ZNOc1i?>xRU$YIYeAe9(@D|7W(i4kmy@y&#a^k_9jN zzHtm{bj#db?S5A7vWd<^>i?V-1J@K){?{V~XbIAPui}F0UH`9Ge0VGMzk+cas+c?3 zq|DHFb93V&K7-G$u7pYoe(?gxTOczwe57^DO*|=#ae=r^O-+UN{y9A@^fZ=AOG^WW zD{Li%ys&#oLk^v5Zc%fScEZwm`u|=0tSv$C0yQzS`?%dv2d=s4 zDurm!9Zm8V0@n;?e$q_ABxwj@kWk8+mlP9O!_>SaA8A`)4vLfK)VYG1T z>T$5#aLeh{)%~8TRJM|8LyV<%MZl7~>fFN~>6daWtgNh?=gRlVAn!C<)%hNu~Bf4Hk%*e~j zlY%H-bN>n?$W!neF~l5TzrPb<0wF-T@5ZSD>u#dpkN6D19+nM(F{yo-Ze=qv(faEi zl2L>QhAJz--ozUmo|25E0oC(WrL3AOCqI`CRBj5w^_Ie^;W!Xz0&i)qE_ZsujvtKm zZNz}K{acdb;rwiUR##?3*VfhLrEu0as3J8GGhnj?Wt6~O-nD0)d6UlF(38}xj}gE` zIz*S1k*$Ccq}4(Ye3PdG_=46DxmnZbp#ny>LeCwD%MNMGGpT@mK>?D#Dw(!E1hcR2$2n@%ze}l;NnXF~_Z4UnqH@rXwd^VSV zXTJcs8`gdl^!c8vbd2!09eQhzT6M9f`J=0*dcD);CjonZr1m8;^Rg z=M5BZ&*Kz-DHBMLpL1PjhP_8JS`W{4X8PQ2U;#mSpTP@TqIZB)D)O1>oHw5~m1Z1J z61_a;^;1`{(w=_Rsa=N!Xv8_1`~Q|6GnJ{=QlNa}bYI zS}zh6L=E8ii!+=e!!8P@80XJ?vN!fs<9oF}Ny1p)pB_l54u5)12yo4_&2lngL%9zx%04Nf969{5(C)vcZC4nO56$Tiw4nV1Mm0S z>rf6Y3gZQt20wk$ir0OZ>&W z(pTqjRs=PkWPneCl(X`#Z4!lS)7WVY&3D|H8_!)~gI)Rkb2ztXiB$|=^&?x#dOy9F z;K`%nni8B7gi4OPOD_yvk4JjL*FMteKG%Kenc*y)qdOK#N)P1<(e^Z*lz`j28HbOa z49sw`8Yy$&snPv;oKy2UfZ-)tvWVfYLAeNCl)CV-=lBf2*p z<J+`XoHt@TZ|RIkPG-ZsqQ=_@ay-vG>f_U}sk$t*frZ zZd5!G1Q`p*5D}`%wHG20N*{A8XlIf+sV-Lx=>|5_!)TwA9B0becDN%y1-@!tmKZr` z!imo6RHxN~;+d7OJs**EFZpJ1yQ{-*T&mxZM0AcNFFlZ(KEkr{yfSFxTR0WS;3HH| z+-C0R>fvuQ+#Gd3e{)ncLviGI)!Q9Hqp(VmmzxT?0Ou9zCmliq!Ai&17d#Va~*QmtIqfX$(pqt+L$5ijxF+_cB{ z!5zq#6hxSi(5siv;TF`oMLY$U263w<#uQVfM0$qwOvEnD|PM$`MUJ`Wm$Kn`EVT9 zdMwqWdnK|9&>393Fe&3yw2XuO>$=qFF*`y?qWNH>fGILNdI|Ak^o_>7S>1%0Q8;|1 zYDEpjsjdQS{z1fN|dNeODkaV_;C$5#YOhXr&Pp`r3-KwGfz#0 zWqE?{5&97TsI(0+UL}z)U>xRxVOhGBX{XTqO10SYS0F4g{80=?4Zg+wsE~wX-^1p~ zHd=IJE*({~8rQp(KsXQgM*kSsM zZmi(GZvj7}$eMjt3-0}0#tWlEu|IFEOL?f3KVNJ7Z9*T3ZO=+vjSXKUoGwPM5%y*; zs)5;H$iyoyJuL@{C3@8O;W-^4u=LUl&29YQQHDEGPG?XzFo~U00=Zf)()BPp^yasX zy86(4*IG!8s9IeGmv>56*ZnD7oNX9<$de%DnF~xo9a;; zl{ha{P$e1baSShy93h2FCNU+p#)`$4L?CJ7)4O*3+Lt{LdTvi!S10%UJrNo&doOU9 zXnhHXgiKjDUt0SXrO!F2mndOj!OK{R5c9 z??f??%84uRzvRelyFw#!SG}KxxpEM>r52Ne;?fTnE@iGZ#OI>fO8>OvRehGgXx=h>Rc;&3qW1>uD-d}?pl-ck}-8hcMZ?T zN4I)uvcp0!EQeO>hc~x9e%F1QK7FOJpW_d7J&d1se{pDV=~u_YN#S;l62Ds-6e#b~ zp-2T1;<7HC-KW85dZ`6E&w7pZ%>o#yD|cG_-E~FKkl$mBmgds&vjeZ#7VX-;_5pLO zMXhg7lK?8jF}HS?aNVtWi`8%84ub3vb3C&~#MIOAaJZ)Ry92<70pMHA4#~azm9*9v z#oK*@fz9tyy;r0^qQDjM6=Wl3aSo4IfNwB9+CU+|75mTCRM_uddx`)e>W#JdbL<@! zg}%7C?ftbQM7YSukoGOWm>VHC;{`B=f0#h5f82vY*q%;%H6a1uM&Ur*ePQ6fdFn{< z^E9^xi;}P#az4OM4Ns#*F`l!j0F`~fIaJ$8h%l-|dq}FXhwzn00mRLH^xJ}sWAA!# zI<$VoJ2L>Fb*|vnw!r-Y#|u30MYeEA_$NGGSV=q0*M(A?`!QAe+|+Z;mbI5e&9VT= zWYC0pt*L+9JA2;ElA=*@y}zG84m%+S9zv1T9!RIG!v>4`fQ1=}>-vccN}DrC0cO8L zneofz=}(VpG^d~ecOBr$j?BBwYcpFlc+hCK|Yl4N5}{ zb$0~PZK{6Z20VZ_%M5H1GP3h3Iq5SI44YaL@7s$bp%28n-Nn2$V5&(6-dQ;4P{ z4>)9qd_I~~O*VW-)>Z-C%?x2_2=AM&b!VebnsgRpND&q{FfbU|HquNN0On(x(}ooW--r$Fn%xHKJ_Js2hX@hFXa29a~UEW#RJpFD_$&yjvmdI^i3MV+DIx- zTOyDUm<$A$2)0^kJvmN0l93Gt(Ut95W(mpcYhIm0O0qJ#`BZc8f1&n@V!V^!1${% zuVXtxVG2^CoN1TAo1jVbN?{?oy=L){dtKp#hCZJAv;GdkMfB!2R5I6<69*wj7kn?| zs({^&O_*t=BJG2nQhhJVT0#jSmlRtgLS=kIOPI^;kq0OvD6>v%NWMA_E6uYvD?eKd zs65j(FxS`M9Ni{gB1k^ox}GaTvu!k>Ains6+r|la|NObUr*$O!X@iAe%W5qks8!s0 zN-I=*;@S3wPG0)Fg+ImqNw5F@?+qKw^=;@#Z_6C25LoYy zVo%r^50w*ijA=`p&A&%3{@x)ml;xMieQhq6FRdqvO}-DBby&crtTt>Tyw(z93PtQ~ z?Yt?n!LHVslKr^?2Xg*Z)*}AAFq!uGaq0osB<9|JcO&=NU8MhzE*}m?M;AOCpyyUt zv!0-A@#S8#D#kBW#hBF@J$%GcM=1hJe6)Bn4(gBW5lzFM@DPiGXJ;;G=L^|56iPi= z*sXpX&G+V&I})xZz$J9Tdx^36kg~k(_)yrlz~pq#Z!vi-zUH4b7%m|OuOOvfxe&TP z{E-vi(3wOM8hHF+E6KuQula6|r!O=z%$MzpyJ|CXqG`FsC}e&y`$tG>ec3U>>QV52 zF}7MyECxaQW)6dG$m05WA)ur!hsQNot;R7E$F*Yk#;^7c#zBX|I=*+JUEe5CA-dim zxL~`y5&D3d^kqJ^7xEktCDn0P~ zjgdZ27j1PUGs_68;%d=!4%{uCo{Q!s2umjHmiFClGQ@~)MVl#(7E6nu&e$BNN2W); zSUNfJ`Ypsrv_588R7RM*Qo_YM*6%~n){j0gU!r>oXsSG=TKP{!@ye+2&Mz4T@Es%r*DD}Gz>}u42-XwpPl_5FMux^6p4{(cc2}* zqZXU=Pm&pPP2GQDkKe)s6;m|~&ly2^Q{}P{=c)eM%MtkB7yH zsswR5LEJw(RaIEzjEy<7EBsO`_KY`**t{(zArC|M=zhOcKgNe$f zs5v#!FXJS;k+MR`RN+q#?pu()6UD!xTHRi7&-E3Uc)EpPr<52raW0Vmi3te9ZF_pi zsN7m^tg$ScI!rMLUUNyErs2u)93J~_&OoX2mdN&DF^6rh(<>QZlh6FHApIU0GQX!D zr`t*Q;9;*IobH)GwJ1cFaz1RAS-{=(j3sHNgFv&;OzW6Yxxba95N*RTQq-pWofl>{ z%RAlaunT0+OvSoBsNflDya%b30*oW)Vo zc&WB~cIDMNi|@*{chA$PIyRaOI|)MI|ky?B%{i1OttLSJqEXn`_(HS6x~{&0wnc_#xXQ2^|7=M+P9 zsVy8z+=KpDEh?e-wE6Xia}Pz&>hbWl$RGORXFi)bQ+rp?r4n>0wkD3ro1Yi_+dk3- z1GyoKM}W92Yi&)|tjDjZ2mgE$XM%a?BgfK^m!n^<2aOBD`g;by_W$#c=(4bS-c6X^ zLz3~!TJ_!FXX!G{^uC+Zg^NIuz*FQvQb z+qX8$Gm9Vk`|9avljKU@*_nTYR5mXxIv{C9QQt%eA*a%T2TG%uVF#|IlL2%&I)D(u z!d{|kh#_QW2Cy69J_RsIE3N=dsR9N-=KrUuTq$Q)zxaWsp26|n-rFz${jRvkxLE`E zo2BY5VB4ubIIqBW`?PKS#DJ@M9wO8%dG-?!J?q16t0gT{8L|t}r>CbthmIU73fOPG zvG5>!{DU1<%8M5-W(!L`^i97A4UPQ?&eNZbNrP*~7^N< zOK}w5^b(X{d5+o2{ICTwKu{FyXlc&|o!0k{?nCl*t3}|I9G}H}EOXNzZMd3+XB@8n zJTXro!YdNaqC%TmpOvwI(gI!siE3HfuQ zg^^B}yUyxUAv4YGFgHXGDUG=3kZPAf*b;yb9spf6Wq$uY}eaW(- zr(_e}8{W#GsiHDVL|`Nl8&WdQETEFFj{t+bjjO|r+PMXJYDG>xWcGjF?49NOHN#U1 z!{z&ua@6oCQIc?wS$S;$wJr1!aWvnt#g2xTm_Z_9l;cSHafpSRACyPKdG=Y2X>loE zigrQ=*~Moh@B|X`VucR8_A1hZ43gy;rUZ65b`E#JLIPW;m9itJseRRN;qGLS%oq$i zwZ;&`o;@wI-C#Ld&n;_k%t@15N`tAx69sgi0R~ix&Pe@NrIz$k7qZoB@*^F5{fzU- zgZml87Ru(N!tT(l+ZAtk)-9(hWPK$J!Lm=jn~F+?Ftu>9d#hAO4qG*#PIm6&b$EVE zxn^Vp5|K9sps?6L#9RR$26ULgq@a3tfE<<=q67aXsyL#n4yZ+WvG7F+fJ@FAVmk2u zCY>9X(CK97*TAMbl-@1N_3vaZ7&C;(W4ex>iMiPGy`+o-9;Fuue>9z#N|cZ2v>cpp3UGfnHhP{H6cO(i39*yRJ^uLuu=SzEi|Y#UXxx$q zyDkl~<4|HXJH}XAFB+lLF3h%Bcd%cMG@%G7%tL%)!c}V6M1wvz0L+lFM9hm5Y91cI z?BUVSJ+C42HHLLyfoK%>x7cEcE>4jn)KiNvdy9%9K0}F)j%{YB$^=^Hk@);K@#mau8FwhG4p~aX2j}}gmZ>R+vJUZ(rbZ8DT;XNltM6|o zQpa#**Q_!R4pS#mJhlm@*jk`BFcJo;+~5AFF-9w>eV@*wF~gNFNv*|lG+o*l6EZX; zArMi(e;S7Pk}0oiS3c)T8xyi;L&^Oh>HDGwAFQMkP;)|i>r?zj-yvem<>Z$7;fTp% z0pg4g#0HQ`otD`VYEAY^K0UF8=&yqPQECs%+bF*>o01d!4M*AR{-)0-4I*pnF%@yO z$c->PYegFEouL6{U*b=AvOBWPxjV~zc_ICGk1S$D`yOe%i5|X(SLX>R9-Wpjqq7par*lHlf|-Z0(WULdyGU~&)z#ju9&eB}Szox_ zldtrnT(&eW+5lvi5nGiaG-Xl5!G}fJw^J+JwRqVN8c5brzi)r&~yYGnynA1~m1hnfmwLZ`Qi&up59r+$UMEC4{A|uCG?7Vi);80E_I(wEzGB literal 0 HcmV?d00001 diff --git a/docs/design/imgs/logical_plan.png b/docs/design/imgs/logical_plan.png new file mode 100755 index 0000000000000000000000000000000000000000..f7827f5232866f036dea4ccdd444b7ed1f89d429 GIT binary patch literal 8734 zcmch7XH=6-*DfF+y-Dva^e#;L=@>kM354Q)DWcirqV=&P=a(J0wPVC z^dL6l z34EC-NPwOj;BP9xAL2l$p$<{qC+;nvLF%n-s!c@HkV<`in+#}E`hy(;iHKN1f4{^= zP~m+dB7sIDJ#FhS*lvNOzrq5~2O*)HhY}V%m8CIy!P%h_+2$+AVe8w!S?z=?uUU3m z|5Wqvm{fUm`*m7+dV-yvfuK;Q#>U(B4JzvIv`43j9+w;MHwfP^zxM2?FC;lK{e3WwTQ^bPhO1=FgMw0DkomSYRV}L$}FoUx|cRKycZT?(UhP) z23J?t=l!MAJGIlk6h(9({VT3}D=WlyXbeV$M-P0%f>0U}5kZ6KT@RpKnb0mbSzKBg zTdAIc%*UdMK(aJ1`10w_r<7kIPa`x6%$l^pV-^GwvCEX3vuzfb(|6k75(|R!n`EA3 z1MDkn*$fE%}uPK6ykKOgW7h zoONhDpuzrb%;d+&&b^TKpkLqy3xZU9ufX1_65?kVcJ@B-21W~lf`*1hct@c!_)1uK z`07>Lzrz%o=XdWaj+6TkaWg_y(O8<_%%(AsD&pJ_7nw5Qkm&M zvSqxuYh#?EnJfoF()}puS2mRKUL5d+ou z#jSnD1i!NBYnn3MaR(RH486AJ@`7;`GW*(D3jgz(q>mu;yKyWTB1*zw0VRV9(+W{r zxt?UV>X(Z3y18PD!jknaP1AFw#s3&-Ee(GUEf;vNcsib8cvsI+8#OFzmyQlDHnN3P z<=Z!8>EMLoba6v3T?tIP34JFhHYqz#yNIVvACW#*C^2XQjr@VydLOcb`Co7@P+9RO zoTyAx9~s_gHP*??wyr>;;as7Sq0!XrjJr`z3|{Up2IsIbE<|6G6fxa4N~L)NlC^zP zrrBfSC-muKL9_6paJBORyQ2JAhnlq>Cv=Ed@{$)m9 zuy*cgDVqN}6=g@``3h3@uk#eagqjg?hG!cmQGQV}0#GJ2mgp;VYG{6b{;uMx4FXLJk{u3cjm_$| zvAgy8X4q`RJ#EM{6q{!Utatk4rs1X#RMbyOuTuORg1O1WoUZ=9jLt4f>4)gy@gKm!#MFUBFy*% zs340|M6=HvO+_-!Asf0?sqcV|IboDpH?gHvOLh83wm@UtQ4CH?rts_h{Ct1kqImo? z+l)_BNN0`+RPT@tL3FBqVe~A`f8Xs!)JcC;E|^q6kuKL;Waj4Z+-LG{p&_Sz(lVlWIh|t(O43Ftv4$u4)ZN}GnPYm9F z<>B21d$9_qT3gKTTn4o#c+L_$3%T&}&!3F%>Fs5*q~QXnwOvV6B*_2Po1>ROQ-CY1O&)_rt<}!^<#mF% zU$IVXt$`3|2p7n_qtr}VPQsU^yhMN;qRzOT&uxTR>~* zh9ErKzIuv^KujLf&C^Qrt7LnaD>4-}3z(e6kg0y3ra?izVOep7egAnyR*Z&q!7Jyn z75{`gJceemW_&kdqVT)NXE+a%j!}V;0O_R%Eil06E6ORi--rkIM^qI0bm zSW@mMwXdJv|Fs8ve*BOFp(VpvBF>I>&7c0BD7rlN_H(75DJjF8UA%A0EWaAVY$^iJ z-pu}rot*5C>_pVt?H!W)*-Nq72RF{r6l!#4L2tD$nB9`=^C;fW)E0q%zWK+@ro*l6 zPU$pA@pDVzdAMd~9VNE2Q}<-q_n`NnzZ68z#@aHmojplmKzyepnM%9D*@c#LT7ZIY zPl+WVP2G9w5q#rCGo@D0oBEnCsp3ZLvJ&h156eTO4tTLK0)cvKv^(t8co<%rOwe|D zX~|Wd{yn4}*b01DhFHAz-br-jw>Pu*KddzGX8XX`J~uNYDBshEFs2jVfdBM?*3c##x5#d7ScK=9{Ix z_63>zF3QS9`xUpCob?%F&o8vp2d zP{n>oU!1cPoS%Dp`EH+pj!{Jte%O`OtNpujcx~gA*kI_qWSV+p=RSlZb$}Lr(U%F0 z#lt0G!p02U3a~pKp&35v7yI%7YU?*@!6w{K&wlEzkKgtx!)_~mX{}t>w_;D|bI2cL zH0CHJ!Bu}NkrU6~3i|!)S2=BfS{%5LrBLEZ+G?@ju$#7Uj$YsFD^rjAOg@}uES2IJ z97;1vNPAEFmB`YXpeh+QhoTAAvfp>C`QdkqG$b;vjh=}awq5XdV*`?`f~?jzm05!IG_y=SG{ij$sjJ9S;06w%@l3_mb6OZ`bT7ve#BFj zfs(B{c}}-}hZV0_m-sp(+lSspxLgsY8QNPWTp-!4C2*1ih$AKp!Fgxp>!~S$iX<~O z(VF>IJ)s2eGU$VvelV;Q$Rz4rp5=XLYYSmw%a9WNs!$uCZ}R50V5SubXs@yuUl5gj z866x#Qc|f#zNO|ChT2H55_J2A9ag9RG}!U*s%jM+rb|z#iC&f{2CP~D z$dSM5B&!d(3JA`dW6M6462&h6&1LpGg;?UaaEU>&81zNi$7X^E`yr8QKOGpBt+nb7 ztcPL=`Bm@t#h(Y534j0+6@74PpVPYO&oQ}+Al|w&Q6nTl|GhoM` zRP96q9&}$kxTTZq-nkU2#^Z-JD7(6_s50ISXatvpH0)`_d&gR|>AkDhI42yrDlEHq zb)e+jE!2#t0Yr0hY3}qQ;drhdP4(koQTcZ~o$Jdb28pv;=*i*!k0$N9*n1DczbZQt zB%FNka+w!Y^hJ}dLQ`LVwfMcm6EBWc7a&CypZ?s~UVeDtY$gH+WZS02B1bLt%h9`v zuL+`%6|IC4x<-KE}L9fu!zU{`&9_)Zj?Ha?3y<8 z$nbU|$%T5+sC-`a^i7KnU!zYkGH}ZlR8>^j^vlSYM)hfJbp^E6^tT3TW%$F_koQEk`}xiz ztu6IHvSd$as~l@>pcMW!xko-w=S6^g?s+LDS5;Mw-R*lMp0$ zB|4}1-cUIq-mwSxE1jmMZtC9DZir|G<+;W^*TdzWnCg_ayuO=To?18|;$`vL_fxu8XKs5QF2}qIKWdOBDyv)6;*- zYItLA$JZTQrdD89BBl219Xo`#)t2Aee?7coF`!jFVxBSOb@(qz&-~GipjUG=pj2lL z3zfUv=ue?LDbKLgQ}^W90mB9c3>OuA1pr%=w3hh;_b18R0I@Psp@sP&_R?3a;Pn819JQnm9a?={ z6%xEd*Neca(1TJH6C@VediMqPYJt&*R%K41%o_L(c|h@)<*t3?f_naw%i8Z zu6q0{^bRi{U*6I9m)r)3svVm5P0Ybh)me#m&~ht1Y1DDaU&kMc+k%yP5LgfZD)HtQrK=ivX;ug?uGneX<0K8kAO_b1z;s;JU8F9-H*vNFPgC;| z3IPdF@wv901AstS48Hs9!2&G+ExZ`tMAjC(8%+Qsc1(yYPvN^>s2q1<3JzlA14Gu-2Li1Bo zQ*j=pb!js^U602Z@$R2zJ2bP zDXDJHIC*^A&Weyy%zL@!Ugrnp%>J@@FQ}zCk4hxww;1=sBwPB3TPCP}_mbU-61;wQ zZ2VGpjtl7HV;zi2^XQm*SjhQzr)0MWQd!2|Lq_K}poE@cr?IO2o9|2|NEhfFe2TY| z2iCsbERiN|UpTN%MX`A)=KVo-Hf@T&yYhMk#i$eomE9 zvz|hJ&}tt<%g~yP&0=98evNqk`Bjzy1QtbJhhZImS^a5fh??8nPmf`!4UB6ib#e#j z8B*vFHJIdJmvZ5YF>W&0I^C5zI8eL)2O`?DGh17F4y7M zRuJK<2;1{1Ym#fNX&FipeP9S)4ORC9%k#~cI|{^s9Ts#Rzop^0ah_*V&1k?CKUuXlcBcF5QIg0jg-2d&s*lxFeoCSy)=~U1JAU`!B`) zyIOW;ze(*sSD(2je?wyx`&_31M&qm{B5c;;$+`aFZ8h?36-dKCz@hrSdt4rzEOD!9 zR7cQ(kbV#;S&Vl`GAP>(sRm%XmxVp|6&Int=inso12re{h=SzRor1elL@2H|ji|bM z39BVqArm=Rj755MMR&{1N5bs%8u5acR)&zmej}9Vhd!Y_eupAcO{CS7xP!5tiR%J- zW{cfOST+4KNupIdzNGzdOumo5-^`58ZJg1N#+8JI$3#j`_!4AmcD-IpKi4lqK%z_L zYf=MMB8VDe`oa6vNU4We^{j!OV}Um>Ko&$qjSJ;pliXOnLB5S%QS^?}BUfdaj#9N- zsF~Rse4!>!akFCAP6W<7+sZj+c9mUz`&;gI!*`08ITc1a$y%1>Qihp+>zVAC6g49D zao(q2bp&xeeiMgV;Xs@=v+w%94)oQe0n!69nd%I5!K$}ab&b}!hctO0-S_QYCJUOZ z^d)@QczDKQ1e4E~Qv(0ZBv7TPo2R~Qs(&`(TN;>bEElk0p(5n{-d7}L>WOc0@2~-U z%C{!abcnh#&6MfI1XbeT?abf=t}OM?F#V_NKe$jp&LKlkc$7*d3!A?v(PQ%rtdN*+ z5G(`)XL{v|_3d1R9LpG&vILuWmLH-YA)(s}6dyd4xHpbE)W^6uRvris04&i+043V5 zZR#Q{5A|S7Vd_JH2GdBd8|c}a4$k52(sLaI6Yo@ptvGN*Gu4n+k={AKmNe0pEPaC- zvY5jx4JQ{2er=CX5;Xmmfh?+-sTZey^jgmJri6%=VCIwXRw@kTz=^&ga6zeC5I&Xf zC;LfQy0068gkDG?2l>o$UVI143_*(thj0?>8DL;lGBN&16czY%k`lnQ?o?TBpFtBR z%F_Q=N9>G8_xSb5Bak7$ZTO$biE!6-3&>0lL&%rA3PN2q;P_1< z`2F>%_*rK7eNp2SvXJV=FEcB!CI`lzkK;KlHb$$sCt`4McUA2BB>n-3(Ct_YnZP&s z(J?up?8bJshP7P{15j3Eqg?2Unt)qD-l0VX~uWCez6)s`qgtF|e3f z4DPLvV*-Q0-f44c?V_@tN@pfY>%(I|FxKbK1--_TK_{;OAM8oM}5Zp&&&=8=g6<_d4h0N=_f%ne@_9;Og)L>BSv^wSXgA_7X!Z;NML-HH-DfZ zT9zMft0r5_h`M@72P;%g(Z)4);Xg1>1GR)ENweLXzKLa0H)2I(;FRE%%$08e3s{Tu z^m`J^$uy37A=nFgmkNL?PKmbLsmPZkEJSXYn;m@?Iystp4Y%rjHpI%S{) zXP^XgH6~gaT#LWd?uYt(K&zVfLMj+d)ebRL;&9^4O{M7gmN2AKVXvLE zsjjl=pM;v5o6l*GK^6rj2~ILuhmmHbpyG_NDca6qxBBFM<)YEN#;pe|*}~ORub1|U zo~)GMd5EVo`rV^t3=AjMBq@VxatAiIzb%v6gEzcpcO~z1RWR zeU&ojXqdXeQb_wkw_B8H*ZLV3>SP!zkxk0$}98TLIcSb4Gf-$yZ>7qox7KWkU!{_{%=9-GDru*%qX6cAp4a(xQ|g@E)klp%!=BU)Vt z{2Umh<46ihWf&Ly|2Gr?q_V50wtxP7)L@@fR}|T~v>Vd&Xk>UJkcO&@X!rQ>El1!e zW@PeeobBQQSohYpqfC~)%k-zqKgXR*k!Pk{(@AY~#54@?J_zpoJg~|e-7QMspTS>WeQ~(CTS2TKhd#_;Q|5HA!LynG`Q!Kg3{Vh;v zD+6dK`fu`wjE;`ZiP#ODo!vYSI-hDL2B0C%Cut!M`tg9g<*`M_h*#MzDF9_pXYXGH zIwzviJw2VZw_{15IDMkdd-j8Q zSRA7dOt1d%YgHUxKGoG3lT}3>B+AIo8W|Y$v(MY=GPI)^IiU`)~Rd2s1CQr z80l2yLDR;+i9|9wKqqyWR891{nhC1(0&=Z>x)(6^9a~NG%rt6ncWgIAQ&aQrb3tka zLl5yCk)7>lM{8BhGKT2m3mYy`rs1=L0Zcp1f_1vWGa3He)p?ah^}IW@S$>JaZTP-G zd3G9LF9Q=){Fg3~^g*Pd1etCt+E-3qJ6Ix+E|8K6hNWXI?`5hVTCzZReZ;p$3aD`) zT9(_@2KKHVFK3s(fD7A#jd4QK6@vdqCDQQ}D#%20OAcVVwu^4PFjwi@;)aACX=~K6 z&`n99z+Zi-&4QwUV7&J_O}+I;hG#_IKFD0=?A=|)AC(jp4em^941%|pN#Fja^XGr~ zCT`q+xNlJ5)PW>8pAG;amjE)CvIS~>5?Kx6hJprFrgo7vZn0e5ww6AyC)^)@tSDIt z5D;%r@dqEA#8O9S3_Rw->eqdS|4OuKaTmgHMIBi8Tqz$YE}qUl`A@M@N=5P~MQN#4 zkMpSz{CYm?Z_!Mf48_HIRdYK$P`7h5rA1}cY=e_J2y6EQ8ztX&I(mZOWYLfwJqcuC zmXYwocX5%wrG$O3Cw%;)uebp<>p0RB4hP7k<2o|ityfQ?DpsqlM*k|sw_BfeM5#_h z3#(THfy>->;^kwwLKp0`;n_r3oSfBs%E6(&n{~y0?yCgrhu4N57snwa%o}fax1pACtl%p&tY#G`j2e6@Lw!@O@XZtr|CM+JAMX` z^>NsLv+t%EDN28X-T4$qr9bU!rY+~a1++O2na#QI2J%mf3jipRPs={xZU1S^eY&p< z9J|0yLCv6_`*>9{@nAKO#Ei2wiq literal 0 HcmV?d00001 diff --git a/docs/design/imgs/physical_plan.png b/docs/design/imgs/physical_plan.png new file mode 100755 index 0000000000000000000000000000000000000000..cd20d44b17231591e8cb401677a21653c21d05e2 GIT binary patch literal 9447 zcmch7XEe`4MNE^wbJ>ndYmHKh{&eS!yE6MBIye6g@-LAM`lEkn-V zSXi8)T58J1LD2mIey_a+nmBTj<0W3<-5?^_IF0*{%X@v(El9=GxgL?$6l-x8^%|iE z?%+iKiWxF;q^E!8rDRw1WOcuXbvg{v2C{(H-Qa9L+eiL^Ue3YM-&Vr7HIJ^ZuaA$9 zm8u;&mw%mxJnD(bXN7!!qM)dFaBy&Wb^S~{cXB(p8P4}whpQV(egiz%+u=gz>tiry z7Z;X-i0Ho6!$Zpc3L~xA2VXloI|pbs_xE+VDScRYU^$_op~)i~v#xk+Gg|2UuAUx; z{<*!@HbNPk%r5C~VFa?j8``vN1Zb1aL+eRZs^j2v%0q!&8SX?UuOBnT7!!de?;@Kr zP%-3LxlWRK-CbSJqSJPZ(C4*tZcA1!(c-u&Kl|)@D=JHUcwl7TTuP?mkuB~KNFj1*B{v|G~UdkKHK+L_d*jN-^rMTr@oRmIO8 ztgwfM&(4dkm=E#J>$O}=vLZ*sEoXVr|05&-Ln?;3YO_tl2W5}f3C6(C`}!F0CZObL zw!B;xJdiq5-cTk}eqbIn8K3Yam$l!RC-#sUO#N8bLJFkyM`E1U$tm845ch6-G0 zGB3;BSuZQO86`=q@*g|$Z7@33hIegLT$ANZr`e|c-bu>Sz+1~0jD@>93P7>_qU9DT zi#a>vVm~`XAITR?|F3>qA)ODD4BDD%_J)IyfF`jeNm4KpeqTAM%?;6bmrbO=F>v3{ zfC$2Ji$bwIiny_5V2075T(kaLrUe;|XU1aXuT7TiB&ge541ab&<))mIl>P-s z0H)`MPOqaxv0Vo;rP*hseQD)O{2PQYoj7QCl?WTqHwWeSC$=Qo%(-zdJg4M=-Tshq zrr<(p4c(Il$RTB!f4zwa`(rAhM_e!u%5kDaabDB8Pa?lRuK&A-{WN(tJ|xnDGLQYC zr`g9JV$)eh>dbN`n^-MRShaPuI&CiLgPgtpT{!?;NL2bRM#KJNqrIh$<*4>GA|7M9 zq2=FMZUmk4s&*UrwV;4sXutkc{B4!K zh!09NCId}2&Oa}osk(K?kPxQSk%b4PL_9EjkH)q`3`R6>qD2`@1334^)%Esvg(P@p zX=O!GLBaJOSC;WgZ6$(6-Z~fa_mvgRl5|OM$?y8?d>lw2c^iXdo+Q|Jbs?HRO_HFB z08%*VfgEtYyttT_bh#5|7oSfEf#QBQlo(%JTyzv&GkFQT2F;&!jpd2%tPd*+jDL6j zHR@J~D>XGWUII8aExMm3gS&_GjSfc7i_n{q@eb0=f{5{}u!f9L*f$R(!7th*JPtP> zQy$peUa~=5kA}HCy^8qDMzP{(+0;y|VrDMi=6PTT{mF zPH6f{Nv^_JGpYnRLxYuEqv3(1N}^x)Bs@qc1=qe)tWo_MWCOMHzf@9u^yttdodOSZ z5Y^XR96SQ&EBMsd>wXrhpb3tp@JBB07^`#;fetX>EIq>cfX-QldEp-%YRafi`Uh6} zHS^B{x~5}+JNIA0=K>7N_D+KJYarLx2xZnCfy}P#I}E{LVIMj>YxH+ieio&^p=7g( zfX@j&FB|BQ{>reehWDamiolzd%x;%NrJf7|wNhe!8NqMTq2VDJ*m=MI5$+3mLPc-g zdMXHXpcEpNH(`k}Cw1rr4{wM>EBL(7$K*cgTxKtXWYrhE>n}>RX3gxfG=SWt%+cXO zP$`N0;F!Y&HIMmPJ&<5+1|TdjXkq#ECN?vUjD%!?3&qX4%>J%-itX(6r3Lrrv)?N>ha#^B=4^f zlht56US3{ymt-n{fX9o_G=tZPvP;7OraLNyZ{AeQ@RsA5-NAoR>aLH`!wLCzDXLeX zZ0)7hn-!tV>cW)SRT44%h|AVMvrLWJ6RYu)|y!?z4k_s_XB)I@j_?cn;fmkE)6UT}D-4|Wx zwB~7bl##7Lg(Q7ey@{QSGv+k8DB`m+V6VR)7}F)RfnFIv(k`v7E6@dYwr|H0>>8_7 zuU>$Jt`cSWmWP_=pHn1={otVSX@zgUVBls=u@wX~#aic~%We(tQ~*%KFsI2&e+}-A z3yqk1cAS!l<6Fsfl-Q$ruml+CG@2>DL~sAJzllP8js7>e_IR|6aYmKO)L~GvLxHAV zj4~^)k{agE<<)b^=b2 zm=e)ecyoQ)({p{XC)D#7bJ}xrgLko8QQyuYC(StPLxvnaH1$`XLetJmpWTy;!b+O7Ne+DTHI5vxGHMXgkx{q)6-KR z<}FAo{vdwj8r=e4_K;#pWr#M5N|XO#(?l-L&@B?*q9(l*T6ITsWSTaezWU|Mg0(gm zrH-uU7k89911jQv(xRJe`o?sa$nk0YLOAm#oMj7KH$FZ`S{)C)4BW5KYiRd~K`$>Y zQTqlrRpHdSAGinK=GikK&*n5PG-dG+&nELao0zqCYy7q(kfXsfCf3^E0mSf_c`oC*1X-CWS|yW$ss3OU(Ju5t;)^hqbYyQ)~T7MN7oZa zN%*{q`S()ur&I$wr_b=Cl}RT(8p9itT9hqQFT}9j)82*-eC@}d`I->#&N9auTOrYw z;PA!xrka}7U?GFXjz2vG@#}R!4_v=rNskyxOQGB0AjV6h|M>S1#lgB=ky)Xq)%gkb zf^53#;?fM`VrI236KR{_G$vpuS%Jew#-z531U2rW3@NnfbA3=pdu({8cJ#esT*u;m z+(5D9XzcIfWAQhWt;$4GnP^@HZ?}*M2C>@Lv>CJ*_zmc7*KE6yjU8_`TLoUuX4`hA z@yNSO7t?H_V3}8poSvk01K%cphJIKx&=m}wUiMowl_Dr16@{EB~jt7$>BfP79)InNfw$l1`lLI;-= zF3_{WoZO>>IKVZB#*6V$HQQyvRr%+kKpkC5>LaTQF6_7}S#>gsMnAzjHkci~dk1?< zP%-F}ej0o~W_EdmYFKfDn{$fPF0tO?1JlK7Di`*gTwb~0R$86B97xU>b_Tuwo%CJlJtp%KXp`h)y?~0 z%dI-1kDhgf!VQdD)KQ%jJTQ0kZb?^5%okqRb51a~n`*YOA;?WztkN#nUrw7S*2(eJ zoU-8C=L=DN$AMuyFiF$*(pjger9)aepv;%ELGWoYI;)fom2v&_vQkGHF?SaAw8J#G;?g9JHzCUZ zylG@s9VYqbFV46QS#-i2e+hWkl140(z+9sFgYAgcBP9*Zw{;!Xv4ADcoRgTZnMU#l z;$7dn>ei2UxCr(HIAql{L64bB$}h~y&f=$K=c-?C%0W9Ea(CHf7K(pZ z34i%eX>1odJb}kk;+L^mj;U07B!QYw+B5Rqzxo&mW`BR*U7p3QMlaLICHX+>oi)ME z2dlAMkMrKC<#M&exi=c<_Sk~*-I8X&sj1%!{^Qh3#s3zkziUF42OqP^%J-D(wqzLs z3mWcPrSr4yCtTP zWewpSLPQX#R?G@=@UmGDG|jp~#Am$kas+I&M~qo7uRhG}xokXd z`88hRgNQ7>eTUPt$sB0yoaR*GN*NxGQ7}HPjj8$x*07J$m5H6zXx{B`Og^sdEnfS& zAk8EpFUYwaL2O3F1|#ii$#UHUr|W3lEB(?=8#6-rU^a#iUHSTHFpN)GYCqs!ADxNfKoM`45Z`Q1OL>DedcK!7+EE)ac#fv8@4acZw*i$uDzl#(yP^F*V z@Tf~C6d5v=o>-sMnZKY%;apy^n1w0K*0WZUACnz5?02lDzJ*f!-C2Wpagsk6YtP8P zENN7Wgh2&pe{Wr!kbPmbgxd>qHW1ljs0ohoqx-I|9;rqgH7)LEk8gTOKmS9?_m$C1 z+C9xQ$~}Hj2%5vkCJ4hix;#6JXJUQEDkwx=D*Hf0U$)V*j*;+}yfCNr_x6&7go}0Z z>rcTZju0r(7Nc_OLQN?Zg?a1#OlfCF36G~Iu|cV7O7Y&| z;pF?5fA*~!<_5ZF%Y<1{Z_Y*(uh+=K>B(M(U+vlYyY49~Ra`GnjrBs6?^9M*o4wH=9&@7qqk`4_DTIFaa1DZHG99%NdmtfMscc58U5rI-$5JUqV(M1ga0|JdU7OX9O z1a!80(r)Gma$5}a5%4J7+$RZUEme%7`W!{>f3>i&@=eyI{>mHAKfQ53V*br*B&}1IR@|q+$H_8Cs6q0)_4&VqgP?{0bNH5VUeMY^WTmZg1JS*HOU4V>m>pE zmjDa6NcI?GgkY1nfYaES&#|1rk(NBL!QRIDiQgbkpvQ=hc(Pp9i|7Kfm_K8jKZ4j6 zLmLgBGob%!Lkhq~mD%M1AeabX<4}b5$BXk{jue9V7NKoqNcK8NY%y{An0i3BS|6fI zGPBFUKxlMTiU9E;J+ar(9wO32ck%gZa}$qUG)D*Ej1)kIyzP%hj@eD?c6T^I2g0nZ ztj+T}Hf-L35kIt^rcK+4ij=)J*AZH}F#O`@0A7@7rE6?3{t) zbuybr1%o}Hid=Xg*2x7cov(XFGd6PA*rC65*>sg;x5O({*ite6WR-J#({Qmu6*Z1a zmeL+wXv57H+~0x>_#`5)7V!6{RdlbEX${S2SK@ z-mrUaL6;j9ZPe3Ee)bY>*12}{D*b=Gw+#^JNTUYXr<(wRkrk2xf}YEpKYp^!4@{JO ze4p>eLiF4O;3)(d0ZQ@9O9^`|RisGOwk75sVce8GYJkd)ZNV!fZ-a^s`Kt+g^I^4m zj<3vye6mJKhJp(ibz~uksF{T@X>w$X`@PuE_CbXc7!_x)_~IJKOE@iOa?S_dugJ_| z8v_NQ0{kr$1!0G@Ul=U7?M(t0k#}hiRteso!cUr^4RlGdlGcMVw$mMX>6>;^P`ejb zP3nmq6%M4bKR%(IQe^qDCZpin(R;8+^6U64LYtM+>Z}=1Dfy^FYc0F(ZGrVWR3%p# zZ>DC^!jMP6As3HbZbHJb?E&ee4v7(OE!EBjK_i7ZnY*}V!`yhVjtAHBaA{&mOH2-7 zNSM<*Nk_tFrFU)O(M08D$mUXgbB*m$$vTD5{KQvJ7*T@vcFY+6e2!2oTP0Su_8r{~6@Ia9@huOqgU;a55ztd(*&ru`|?UsE^1ZGMs4}xVkWJn>6I4E+l@8FtPyQ(s-1Aj_;0w<0A(+>1 z;#onQ{&k7$ir>pjjxK99Ouh??GNGQc33wXTiY2TahltBePVGLCE!YaZn;<=R#?I44 zh}3aw>nGIJ0t(6j5V+Bm#WIjCy$-W!0M6wm^BHq|HNm80FB@%K3KFXx1nBOL_wT2M z*#MxA&j0T0XaPXemcHIg`k$uAa@&%I;}`qk^pTC?Zi&ppAPXh&omUg3##q@nNpDlQ zM#I^Hepq)CB6a^;jecK>@*<>`smV5-tmel!Y3?Y;l&0s4TI^T&O8`5T;$<)q9ff`0 zRYv&&1pMN@DN{w-20lFhlR5n5#Ii}NnJhD*Ak7RUfpa59>S&VS``1bnQmA?l zUa=(Xwg&pALZ$Ky=(5a2G6q%?-$C3%w>)TG`jZdeHP05E7&BBB8kuTGKlyRN6DQr; zB_`&}?yVz%&~tB)afqXtu}8UrZnV{4HQqHHV@)AhydJ(>i(}#`jtCTeo`5}{xo*|i zr^dQu&ig^- zUOad-B!E=u$P%zhlUT-Z-H*{tpl(o97N2*i=Hh`-d9L;Qlf>5+ffLUVDQ5gXN3>n~ z|3L@fx}i{L%M?k~8R(UNvYE^$X79zp<9AB0*%P>vzTn8!xiTWUHcqEyyyrH|5sit= zRl3kRufD6yMm4cpbPL}k(bkvdKbCe|sxtJa2BkdKTr$n0>^UXk9?Vd8ow&AHMds9$ zHTA9xTeUw&2&=5VP-cEP~Fz{$zUg=ZvzkKS4Lb2&VTtSu-Gh1M07JSfIS5GaSf zujONpm2TAfUMBHi_sq-!-Gwti*VNV|((h6FS zL?p+yDkT(=_GFv;bm|NAcpwO5TvC7tWNT}CJC1YhWQk#YFDPI%jk{x3k0<2Yr7KZP z1oPN=4K0K;hJ3}{#AR@r)D#` zEYd5r>FxQ0Uf2514KBTdA_&g^{vEX0Kb%tsh6|UMm(RqfhiMF-^G5(pQpQbmV(Szc z;+fqw`BWlX!NuZCW*0$Sd-WSqZzsXk3aVttmOS7PW2Z6GO2 z55M&>WD$EPpZ9~*4sRI_3kY>;F%CNZm#kcsHd6E?zYY!>&LqqhmEuWHds`4)-}N!C z);BhA?xj2;^h)pP>@*4|P#=qM8FZ|+7&?Z2br2<3+1S{4E_u5=aRh1AAbXS*TBd>}^GLVa`bP*dgsSQnFWI;LFOj+X;4h()!WSy?!ai z6kyWj2PX#5AiX0SzkGDm+=d#$TwlMEzJKGo2%r{^rHrkDgCA&y)BmXQyL0nYz#TxC za^*?Lt%GLNtnTgcC!YX7{#7W@8}N#-#wPe)92|evqKo2i1gxhec2BF>*xUq0+%7;~ zktM6Y0ZQQ2m+wc_9W)Pt-p++$g>7sG|GnXVhI+E)==5JvtMw}fM+3t)cJeGWbGK49 zUd_=~EWlFE|B#=+2pk-kGr4^pp08FP2hi7jb`dxbmL0)wI2PgAYW1mMJvZm)S}2a? zvbjB)khIf_E324;23e~urW$F`B4N$VU+d_+uQsfcw*P+IBPFf02b7}JoYnp5 zj)>%roG<*a-4Z^)T7&7Aa(M9v;Te`!0S{v&lH1!yjStl7%`jg|tN<%{X0;3Fdwn{u zDauvcZ9MNKtyl5xm2;x5+X5jL3~#NY5Vbl6BGu{*4W>u%1iNwxt*KKm_^pKVHa_mwMp^_9S8C70+fV;5x{w^N4} zK5IbeN8>@Dq|h_EN~vR;uV)-@k@Rt7<-RR$=Uwe*|6cEoDqCJrr;q*sw`2v5!2wV? zNXGN1-OBBmVUm)!?vk4>@12~xYn!#s0SQ@T`38BiQ6^Vj#p0Tz5wInmXJAbIk=aM3 z#p4q}w}2g>G~9SZ6FA}pP723*l{T1(GR9r zjLD`tWH^jf!8M~gou#tbWjf8*FlKU9sD@{lsrK3jCRZxA6tXJEczmYvBjvgq6LM z76T6Bu5w@qA|xbqdn!CUJY274iTy(~`rHUkhJXnka8ewS7OmXV+%z9b0q@aWhq?!X z-#FYJ7+7#HKw2$4Fw^b$`j6!1!Rh5+%0GjPk2Th*>%A|i zf)f{l7@wKBCun04ZU#^-gV^?WQA}hnYi1J<#03ci@~Jw{LkpX|0Hy4qVPU{v9*6)B zaq&W7uz7e}6khIxGf+a