From 80e687ea2168e537959cdde16910f32859c8cdfd Mon Sep 17 00:00:00 2001 From: pankajmahato-visa <154867659+pankajmahato-visa@users.noreply.github.com> Date: Mon, 10 Jun 2024 19:04:20 +0530 Subject: [PATCH 1/8] feat(platform): added db2 platform (#10601) --- datahub-web-react/src/images/db2logo.png | Bin 0 -> 39262 bytes .../src/main/resources/boot/data_platforms.json | 10 ++++++++++ 2 files changed, 10 insertions(+) create mode 100644 datahub-web-react/src/images/db2logo.png diff --git a/datahub-web-react/src/images/db2logo.png b/datahub-web-react/src/images/db2logo.png new file mode 100644 index 0000000000000000000000000000000000000000..bc842aa418b4e6088e993346576d415e5b6fb5fc GIT binary patch literal 39262 zcmeFYXHb+&)HV9d3_0f0RfeqK>?Kvl4S-Jkc=Wp$ta*8K_sUkql7_` zAaTe+qNE|tZO+m2zJG4j{dw!FuZr^cc$l82cX#i#*IK*7^=@iXlChCN5JY+X+SOYS z1P6bG;Yo!}fyrroI74>o~K@bmg{i^C6U(1y=V&6M$6YY~;YK!%ImW+)0 zdL=R{s)a-Tt3#Ngp^1QXMs@TPI9xc>x+UVwwe|WT0{GcypO1f2)lQI*{_IKj+)W+# z3H(hnwUUxD*`4l(>27RQowFvGhR(wCDKcN;1wIb1e>p+w3hoLZ7Zz|iXW(ae;AgND zFm4nas)l|;cM-#+*2E}nJjMmzDA#``X((HL8Hu>cWL{UDp9jy!;>*}jxeyzazkq-L z3m>128p_2LFJxRlIjjm?98jR8fzdt}hBH9sM`e&(mM(D#6IvK5fL$OiX~*vcKpPIC zs6gn*shg1$g{PEb4ied=gn7aGxK!k!kH^RkYnN#^h3nxL1}k) zO-i18IEv&s%9B8l2$zfDgmY1DIFuoctYN5qhzUN7om3R`D!TuoKq!$(ayMaNZ>(V< zF-Qwv8DQdBh9BFLGbV!sW23pz4c!FxXb92sm(11aPb zFInOCLot<)Pd0p9_B^Mk>!#$z7VdbAK(KgSg-=c7==QO4+3_x- z_NaqI-~B30G#r(9WR9&~`^ z&m<0BP$bU24)r+}ha;!fNi*e92SHsyC+s1?5&ym^$ul)n`%|xrW6OJNI7)OeG=Jbs ztX7>=P_$D6VGWf(Zr|@4>S^o&GiBTT=8)kMV7Z=Cz#qlNclxH{H&B|0Z7%FL5Tz3W zF-oYO#E}n;y^=0$Y01Hw7Te^v*f2z{Lfz=j9euD}vHt#ketX1efbvM(G*RM&CvY?; zECNQvkHKJ50wiKZ=-e}iEz%%% z7!j*&Ak3elYC;435RNiC7@JT=s3_wrlpi0?Rj#+*G2%Q8B@g_NJ~N9amUy1=XS&V> ze_qaCvC^H5o{u&oAgmdZ57LA${*u|A!Fvj*xE>d!+dfD=4NNNmY=l_sB~LVBNqi$s znK}6346&~&^kdg`llzzQuZ2NnzFl`26?KBEDyKhVYN!Q#w;Y-SG0Dt%TX%DvXd@Z@ zsO=*hwQ6gx$hOZpr@WAtW#!3=t9GG!ulnzc5z}SrV=+*}W-aA+$3BA&xC;X`_C0S> zi4W&qi-b^AkmFAAzbBRlZpn*&{zQQeSJ$cK{7iWriX||oz1|1*!@>c@w5N$aN&D@i z;J{hTdcsnX_dg*fAt33aN4Xwr%SttLK7U-L%qSj9dd6~$A7bfSc$&LYwy|IlTvos>aSlXK&}LdoCyCr zg)=l430Q9WhAQWwji5yJ?;9nlkq!^)%ki<;~Z`7C=+hfA|24X#;}`Y(7|sRk>#^j4lFGhSf=y{_r4u5|Jk8@@WI0^ zVv0=xCfh>DOdWL_j{0_tn@4^^ZaNrSNQJP_sSt7iN{_k3!NQxPULm#-) z!^+!{V)1*a2rL}wglyk8`n&OIaon(m&2{)jxbg#@ScKCkVTl>E(WA@=VGc1nVYrXE z-~N4Ja7+YWK$M94^faB^8(|iyD-2Mb%;eD_Lz=Vs+}>VNQ-%fK`@?@X$W?M!4T;Dm zF}eRFw|Jiiv#LBHG`UTUaPxd&(;^-;)ab~{v2*3m7GF85LR-RuCO)xNWrxk0nq*nP zQ4U9%Oy}frGAON&P9fb+DX4*<88U@*Ac@d2kRbd*8EJ_YBv!Ys9+cBKA}*n~ zockyD7+**2LIIGr=;d3;>cwl1$xsvp!X>8AMv(SWK)7-RGN5g@@d}Ej{06I?yT{+5S4A@%FEesNuH5j<}?_QH9DTJflnz}jBEHE490s?6Lb{qHj zx=3R!w<2`s-xCY|Y!*PzKq5zckeDgf8WtTUXg7AlF>#O(TO!b#WoIqm1rGvOXlY6dmpP*{k zb;p4+kN|zF@r8Bua}!?3ac;8MxP)5+UpCz^w;ZQ*_D|6CwNXTKJ&l zMSk@Tk^>b9+cxrU6ioM6e*N#bW+Zw9M}$~LygoYdx6}m=7+U6-A1c~%#a9aIBDViQ zg41Lf97nPth*EvB$aJk3;-@70d8wohj>?!BRhB%`hrEb0J@Ikvy3GF`iRUWD1c@+T z%sL~@ZxIzgLP+xHlBnr6_xX{aMZ(O)-s2XwtB;NU>;$!QB|}Jp4H1-kR!~(PJkv%$ zH=8_?g0|@5E$(R8*$~sxqQA$1mLw92g%9qR5h)Q`BTz@y>&9F#G)h@9Lbw`}v07|f zlCX67Z-8N1q+(aFsoBBgH^c#J0ino zhCg{i3)S(raj&l9BgQDHL?xs$dHzE=S~uq46o?4HvW8(Ye(($Y>gS{4sYyfY50eLL z%LS(YAn$4Z#R{m%pmif8wi+m`o0{?o_@EE_n4=5=^ykLUg#T+eZrBK1dnb%INZA~Q zKC3*$z{+esPrcv83HzIzX<$Bd#_QB%UdR`tK^phJ!Ma9Z)~}RJk0h%g3n&HgAJAPz z*ApZQxe-BRsQd4J!e5<(UNeTUa2U`2-V=i&VD-nfKIj37KhY$i$<*SBhNTEPKD^Ea zyGIJyAJnp;+iZpZt{?y#dFTj+WrDQX168aupOT_(oFwCApvCDFk%Fv07{dcM-(aO! zl!k5}Wj(>G&RN4Vqm@@guiy=twfT%>QgCTNH%Bv(lq#q0i)E_wa3L zc-=@mdKVeb0_BMOo3fv)NYPvC@vK30rVPwHWl}KoDqcff9B;K11|=R^N$HaA==?Jp z4--irmG#wcFHUZ&2OlRU@k1XNOvtsx#Hp#D!PYvm7KI;n$P!6+=pvF%9hW3x-rtnY{Z76Fe4s&_0ggI zPFSEZNgfykq=R@!=rDsn>dzFfG6$DK zp%4y2kAQCtMJljY@zfj?O#Juq8y~sRN$RBd38*B$3c~K=DWRJT{D1dfa6TVe42{G& zALlYML(GV(@eD|C+5aQeKXioTxk?nh^)+mpdhm%Q@rI5&%q?N^a-0ZCSem}-_t*2^ zsy!Xf3iMA%NJn56uH3vy)gH?oYf@&<%f~l^iW&M~Kt5jcsKvw~Eu>2JXBH8}$i#F; zUY>ol!ntL>=S^wQv45d?H66Q*iy3D&?CfQ03VJ#^^!gVanen^n zyq6yQ*pTqp(ym%a57_u3plwoYD;Zo5NyBgq=&&|Iw2r#pllAjo35>hui57PujaaVe zhR28PZy{f7FtM0@H%pYma7k@FrNHgMB*-b-Mk?*Pmy> z)~+{{`9Hr;WN(22seSFU*4x((P5!@UJEHRecaInr$)i+F$?MmL>MU% z_3JEj7HUpbx)&rRDvDXD=rC};FUupR?u$vv+GU>vGipUdpSUEKIbfuTo*DX*Uws2*I-bcpTyNywTEJ93 zdtqRw!}qhGxPh&5+?#!ol7>%_f}5eP19_ueIqeDh;#`zuuNXWNi6nA~jyVIv z;$&|@Vw@YRAPQxZM!b~Ki$vGXPZ$et>r=N=f@6{qb?!pUPC_Wke0xdcnl8CNxmcB+ zYzQ1>IL2(@^U!;Sm`b6H71%=moEjzahGT#g)kkDS-vA`FqYoO)^Tii z)U|Oc$ttKRl#;#q@bEA6f}iuX!rl?Jq7;(oA{m!lcWm2X+^Z)|6OTqum^ zWF##)TKT$voeb3jxx9n~1)AKH+shO}H)iG1-$X?b7mt;oba} z4)L5RZaD#d{?JSIS%c0)J9m5dnA!fIav-i5}#1 zF@nT52QY~1f2akIH_0`qwb!^H?+-w3$iPS;9il&A@mv*72Z_rrE`R!iJ=Y*dsy{fR zLO~Y?J>Smkw-7c}^9%j@QfbkvO!&)Al#8miD;wq7-7F9H{+;0NCrwwI$g3DVlr$fX z5%n54O^Y)d(ui z%c#aM`rGJe?U(WKl;rjXcPYpPw0d4&q8w_WR<67AZRKu2VNs-QMldSqU`ZhU?Tt2C zFwGcirEBLfihCRf+e>z8FwId!*;|*}P)PyWE(w>S|L5y; zvv=_|KI<-O7q;Y^{L>0+3;k@{8n>#^%)xo-{$Jk~8oECN=aes*`qmeaoF8x{zkmP! zg-e9p&el|$sJtgEiqp(P8V%mW4vm8Og1ujkzz%nE7}NSZz_n}fw%=)HaCw*Ds7>@4 zX~isR8AYvpM8mKau($T;F7N!!?o8FEZhPZCZ%U@VnF>m~)-3T;uuESstzE1Lx(;3S zznV-L$69Zz1BtDyJf&9h)q}YWnm8O^gtB+E<-Tu;CLb7mGz|f#f|VP2Bwpb%Zn~V! zOb$npJ_K(RR=ZZywPG%3LynGX>}FMi>Aky=U>({WKXeU}RdNC`=9?^THr6KY4&x_8 zH+_eB5~JYQ5N1m;jhA%Li#PAPhs5rEWhjF#npG6u;rj^WOAf+Z{rRzb@ORcxd@Y=E z{KwVvuOGU7IhS;G$0hkYEKL5k+DRRON@wSV6BL2Uk|sn4G3i!#OvFT*`}<$+RmKN; z67^E}ol7vLg`@~(rFXPM>j}7C3|olQZ{I3I^c5J1))qu89-o6aIz(YuS&vmH+4=S= zh{Qahm9n#Ra_J^m{EpUedM&Au*>PKSsE_4$7Nb2N`GE(W}cpdphs>#c$ zET@k=$DqKlYQ5nwDotQR<-QmF@Aiw+ja{3Ik8G3tz_ZzXZTb) zG>zbB_t`@OS&!LksqXEp{8TE1uT(ftC}#BXKmqllKbjSFsGV}PCFQ1IM)b=>xE1gFN+b61K2DeMFN355< z^5xkWjz69z2o}$P_57R|2{@WGDf`-9r7utBR7)7U)-qURj=sW?@`pJNnDnZ(nqMbW z2^`K%x@Q!bng%iAGniC4&nW7nC|1cEM}j#6;@&L)5GYQ#qG1n*-CcSl?_P{mn+Q5| zzF<-F^CQ31d*!lo*`0h)FET#j`TxTJB(kXq>p$7YAxP(hGf~MJzqk1x&;TxU1H)0sCoEP?t~*xWioZysA?yUcrkJ7qnsK$qIhG!gRQF=z((Lf;PGd5caLVQr0)pO?=ESY^<}jh0p^s(@2~)S^ z_h{pwjRS3LHnNj93_ED%dP&uoMXa?~!gF3baQmAkKl^JVvi@O9wV>B} zNz`geM=57RuY0nHb@a-d8T2{Y?3fwz#1L+^g_SK5NX8V#QX@kfIpw`C$WpHer^)*e z-BzV$lX9HA{k3D^RV{fuT)cem87X}?m7ZT$X$Mzi8Agd6a1ZHk%;SA~=;5S_f;Q$S zu8qe`qIDZ;ezhMl>Y*U*+^sS*I^Pl^ac6N0qZ|u(&Zr0!bi89!cl5i} zG2l1*LUa04mx$z&cl|9e&8O}&?ZNvOS-w{&gvdLsY|SNE#hhZALS@O0M8r(Lyk{)$*a~R(f5pyKJ5h4*^Z+ z0M%)%-`4z3LjfNsnV>@pjv`lbhN}9d08fc?KI^#MnG+kZUeBDD>Jf3BU^d~C_o^u$ zB)e{)psoT(-)=bv(<_<+^%!#nLg{KtJD_1y-P2Wp`yLNnOq(t01fk*R?QO5qQZL`Ej0*+eBEolUG8wpGXtE zU7%9*we(*)XV+(zqY?dyrh0ku9AtipBTGsH&ck5pQYTHPf%+`7@sbjXhWw--R#HNg zP1yq^ad4D3rO)JJ872IYr&%MJB_p)`63gMcQrZz^5LnUyf#qwX)#M01<$oYOO9i9q z?wp0{q)bO(8!L_pKQ>ShNfm5ll~6@S1?;8Et3-=$X?bQ~!xmLlj{%$GZ*S67KEC_# z5d*Xh-+0Mx?as6PDl_YBH0{kx$7@!1BS5eI5Pw(mz1nvwN-3uj}-$x$qu10T8vO1)~ zG*q`uvmk2<0SwO`de-0ZW^X7+ed!2qG z%vLB_=F{)p^E711swhVqdmkkzwCi1?FDNaYSDF3d*spL%QivVeJ4_|YPDc2Ntd-N@ zV#LH~_+}F>P5LS_Kd#yE;25VnkzrVqlatFWPTl)2-M%Z0;%O%)#$WgqOn(!J;8m^e z`&nNV(%*ZmtR;O^D*5>Vb|Sb{&^zWzCh}%%^25gC6W1n1t)qKql_~areeNJ$kVKfe zcf;Er&KQnePIrliQAemCWtnf^zTG5-_V&*kR!oAYtELL1ivAa?tkOW^@5rj#c{kfL zhI_+ExlPR~T`sA*NV|+JvU+Wo&g&)!8*o^4D16t`!|2}1PdizOSTz-?{?*vY0kAKFOF3|y-llg$V;|wLbdHSSS*E0=I5jmtD$7oC{ll1 zduv!^S_pV5@7DT)N~YXU#_z|jW}#MR*JkVeTDNB7b*KAvmD@hu>q;#g`O((42|#7b zNdyInkwLk7Tyh>~K|z_&I2k82!n%xm{jf+}-p7&xx)XS-(6XKla9ru7sK1%iAOcz( zC`h{AcLjinM5a~He%p^rK%v59dJ3~=I-oe-W~HR0B#YaH{?vbc>0zP?I#uN6Jp{#7 z=UHjRMj7uwA7ANSiWh*Fhy|VOJ8U?HkRm%PT_+nVLgnr4n^B4(X}qCAkf6uO;UxaD zqQM<2Uohe38RmUm;!iLSV${Nzw?b>g-b-q6QG3L!2x@!@#iz{P{17&I#C*XRB{Lb= zPogmYxWwN~A(`p&Imj`&L;a=M@XoIx&9uvpp4_**HYCO&HZyx0a(fd9v>tnbi0}kJ z9m#@_A4NV3V2vdlh*vF#ML~IBLQRPBh6=f^kFE>AtmSRroR5P9*~OTVq^PH!(`p!* zYS9Ku7YnMx`JQsS^fwma6kpPw0-Bm>Dl0wDQSp+IS`2=Bl(tM^Ko|_AAmQG6! z{Q|E%0_d>&M?0TJ6nZHyBL;v(Zz5sJwlYZPbpZ7F?IzwOSw^tC*3kkIaX!HMt2<$= zd-BWViOU@vo>AgbZx@0s)a4=X7v;wdbWjCfMKzoh^$Dg0fXuP^87YfWpe(=6YXT9T z7Z+}lp@5cnDlv6{d>ITw5TV{bUllaR`F)mf=;bIg=(T{_VI?_Y_qG7+I+%N7F8{i) zy*557ZPT)=1r3Ww#EjjBqiMUP4W~atnY28WlJ)b1nS7|10=$@}``emBH$_gi4`aRD zZ$N!GS5POmvt_vAKX<#1*OXrz&l5j)aZZS!sPV+V>{u6}SU~PfwOH*A)ZEuw$ldVA zWo4ZteK%_=Cd}afV21*h=u~r)pC&|a-C~2qu*>uYPng^$Nea%(Ka7uR--*%m$t|QlB^Ryh8|2rA#AK?~hKS=#%d{Nh$|8FRF=`dqk3#Zt@cIpb!k(tg0XXm;Sf_{P3n9JpKbd=nya&b6LR$FaL;6 z!#bdXb7`(={PSED2k@GlSY79@XH`pj4I0)l-uvrfG6QAGZ7TEGkF)dpx$7QaF`OeW zayiocvc23N%`W5rbs@>dXul=%x{Fr;`F-@oy5y5-if1i6D%;#XS~gx1;)GY+dTBB`s7fihE;rw~)W5?sc@J?%n%Mr@L_Uu%=&(-n9CC9@M!(z^B$dfJZ zc)8efVi?xz>#uI#RDDWBiAG&_xd2l}22FAYtcV8!iCH6+_(o&UF~t=nZr_ibGhLZF zcRQWJFiMAo?Befr-hrxM<6wp4EW&@SdZDk-MAB^180U8P!*G(N{}<0MC|q2gO2kfE zBQDV4j%K^N0~aY-&2*&Z*+%AWLX{@(xdfxv_QT@#<}`>b6d2U0`;E-{x`=1GmCyL2 zT*@#GkLMS6%-UpRWQNYV)$Gr|>G6rzzl;Dj2@p?0fUWcO!!lJBwH(lZztNU)eE6R_ zjEDfD-Hq zIhxewLHl!6fSx!zlMzBQ!;<%0fU16De;4ED`TL7N?d4}uv9q|eSt@LE2zgJDMy41w z6w>`oliT^c<~_t`#^N-YB&a05hfuQWnU!4jy$zDD&DLs$GSI6p9!|5=+YS|3;Fb%DqaJsFQosAnTTZj} zChVi9P-Z=O8NqLsY`cz_CaKZQhq4-x%%bfGY6TQ`5N-+;6$7+2>hK}qSJ1}D^VLa9 zFtDZ`Jkv>|?uj(S0Mou$|LE5s(n)~WhxlQU$5Y5!w@MOiN5Vy1RVs#<1df%@l*YhNv6%qaqToh-lGbez zjJ|-~cI!jxsQ$L=FctD$fH54_2(;2rp$R?UXChpI8>;k4==a;LO}}$D=i_*ddSefpWC7>6$d54Eb zMmwf&1eE7&qI(uSVYhac2fk%pA)YB}#aI4{OixaJuJcAZrSJXG_keeO?+U=O<{N%| z9sbghn7g=yBsb0cf^6*KWIZ6Z9kDQ$CrRQLWnF?xKlyjVUu=H*^ofcj^LDDfsVN(q zjEj>3tO{ZMNd5_H>~XQpX!ZG)(yVpleP>%)I|+&^@sV%dz4Q`yS+H`R^RM#c542}b zNkL5grAx$IgN(v=#HgL+F+zT>q3UyQ;>}ue60auIo;h;wXDYgs<2q>}JNQ|6!I+s^ z#Lg-jUn{vF-Vy1JTNN3*u>dzT#X0q+ynTCibj#`4h;GH38PAd#*55PKA+>2C&Qn{c zUB187{w=$|sK5pi>!8~Yi@&JxrwV>h_y-k?uK<)_i>ZnH^ZX~9Xhk#rX=8f`Uf;My z-{)_qn(nE@^usNo&$ve0pvasR&^W|XcluF%~f%{tX-Vu<$F$oGls>Mia(02 zCO3H5YjGtyUSYQWcRW6g8oVs>pwRDv^t{?W83LgAu=5PQgV4!{D^5Dlq zpG$^&7in0+T)evTlhV>G?2{I6b@;VbZ2HvBe?Orl-N+l|Mlg#jc#3&Op5Rjt4L;l{ zU9Cntz%A5M6e8UD?);H24AktF@EEDeBy*2&KUsYmH}R614c@EBMWBhK(7Z*S?B@*_ z40|ivd&-}BC?z&mM&U3*CWhPrPEgd%_ohX5cWaLxw6br(kbv$#;kByfF|NNx#Kvum zN)~k7Ui_9U=b3xa^MW4tQ|X)MRDhjQO5^euh`vgWsE0nP&-5Xp~2P?3My0A3#fJc;V-o(~Do@5F7)uvl-Xq_C6KRW;!nxiSa;Hf|7y4l_9Ga?i>K^58fWb`Ox7~;>014&&1#; z`E%Y(8u2q{{C<6`d~Hs8lN8m#jU4NBIezTwlC6gIl87AInIzciNq@w)hr8p63}aVwWFZLpJjQBEw`2xLCXpd?2q;u@#W=Q z<6|D+BqZ`Z!i9b%GM0AB`vkiMz0J#;sTg&Mya7X3_E9V0AYdgF{rnB8IU$IpB?!Qj z>MB&y*iGsN45K#|Tx%6SS^aBeCqW)hQBO=EeOkV!C#_Oz@AzX3SOmYW2*Lj9Tf^rY z4wlbTa48HAa`MQKqdpiEur+M}X^zECCGWS^5m;GYp#1b029&77d5mv~2c!|8hQci2 znJcuf0C4D+7@caFzQ8QgzTsFK@rDpO3_R6kfxOxRv?iN?Y|g=s)e~K%{nj7XV?(SX z2~ple z2)^KM!O?3kHjjX(Q>XOoKh$|rZ~NutIeTJMXr^~nLMf*UGvoz(8+nHLUJcQRMb)wH z{1_ogTN{}E|9rS@9Ac&DbP+{?Y(AiHR6k2}O%#eiNjr}WUmlZ!zYUS5Y|Z{gj@oeC z)Zs$|muHXwRhmOE2Hkgdb+9>P9bPOf>m?f@iCdjALoW2Di34T#v(ryHVCi{F_Ya%b z?M(!L04l3}2(yI7-pL*;=3F^2&u8|$*-0sO=Rr}GB^=GzLZiXQGQPIaLf!FrZ#)d( z#~W)!@46JiGcXB=hC48xUxzGbpf~81>PuWdFi$ft=3jB<-3M1IHVP9Be}mQ=@*Qbj z3k+HVK|~?8W|URO8hYn?Y2;5$jqI70bl1ivTy|1k(G@B=0#HOYgE|Jk*YPpEe}#Mo z0)EmL+*eNrzG1bNMo@=hi)z;-ly(8TB!8{^>z+cebse!UrG@DIBh4nAQo0*9fv0qa z2J`@t&)M->XfZ{-6SHY1P}uo#oAJPr5LmLBejuImJOTM zn#GvbHs_`%gfFbdpu?soBoLZ@6}4&@t00t6DFQx2JIzNV z{={CqtX6D!ba!=Bhv&7>?OfF3@*Hr4Ny0e2ljX=ry6+WqT;LgIsSKuF@$Kchk+ore z2;5gHej}1~VU5$|iw%gRulk_9jdY*5zO$;-aq6-W4 zt`%m=p!beVT5}ZwdbyVSvi)CAh!Pz72^N9!TYpTY3+F&ri_pz+?LW_TssR4jXWknA z>Apj~1F7~x+xs_v+PY9Or`*jh^~a4rvR%n5klfzMcDYL)xDNqvt9l=@9C0UR*?fz( z#EDJg49V`QwzQ?nxdx9oe6GgVA!`gr^!V-M9)XZ|sXrmZT?*wSG#( z(~wvqjBjOith#KBFDWt6{)t;TFa@hbLi=4SB1@TAEL=g&H|IjH+$nv@b%Nv4!$FE) zN;d;##Tr4w3?kR@e;MEPdlL_z{1AKQDoY)6ZIAEO!B1U{7v#@m?w!8SQ?{LzpZ!=w8CE=fZ3xPMR7+>s?^<8Ay+HQ?HrD`x!UZkHIY^ONNAU*s(` zE&nifr3j5DW6B@8Ftq3)r_Y#;+Vdi-#uHTT=4wdf7tl(wl4#%I zB`<4#-l#~*PTG5An2y;ewBsHZL3`%HTJ$KnCrm8rBA<|lLrz^tq*7x`al?V|@{jlQ ziPq;FLXRHE?G>gI@rO#TnM^$EU*cz5Wn7XarVr|AC3WS$fi>SHI4@ zQ_bf>uhI*UBQg4;cSF78(jC7omMAX8k+=9c7p*F3RRDrPgxQ<+APQUt|G4d#rccWY zqsl%`Rpod<4cQJUSD~YL&>>##GCo2A`4_!%U`B-|0Lku5C^ff>!rf-j4m3pE!RSuhiHh0dJd{ z3gd1E|72fB|Nk~xsE2F)4pS%v>l5IqysEjx*#$>YfSnNqBw>1>84G}3kvmL-$Rqj5 zlT+72AS3WQi^+Et@v$>78j+@aJZNxu8zyTAz{q#55r6EO;(`*b-+l>)sk^1UNoZ5{>3#}Q#}v_#0k6o-*YyVNufxJP%ltHun*Nf82CU~AJ=~z3%C9~`-T5| zHe<*PdWIx|u97*1cwy+&Tm|Y~ZGH8(?@GYNk3E}{D{k&k_zXVJoCZg^Do}VeuijHS z+7*?`KErHkc%CkEtL)Xx)isKN?n=A#<0JzXx=Nt8@}Ct!QV&Jc7alb4@2Ln{H#`Dn z?NDcDC&_~Md|9109@iWQ4NbkGkN_x!ck+n)!yH})J)|0BSFXRI`mA`&A-f1FCl=UhLVct>RX~XPlfwm7!IP4X>VzHl|Uu zx1vlb^f?w?{SIyS7KW9S=nX%b8rWKk)&7e8c6fS_uppZ~@LC-i*rC=wFSn`M8TSbY zIO0?HQiucb&FZtKvKR`6U?3a5foof1G-d$&D*Adnzs)K5Uwp>k1P;I_(x+XMQET`)0mSrEPtbbm3qEY;;?o-L zhb9yRn$=2k6i|`hX1-%Nf8JxfX`})v!6T!YZii>}i+Qm}M8)BFT?BRfb3=X{@bfzS z$sxZ4k573-5^z8nK>01w0Q!^7viXT#!t_aeZHQ4Af_XUB{LX*vU8LM_&`}e49p*M< z;gbM5_BeI2P#-hiMJ-`ukZa9i0*LtOXJxpu5X)|&YcRA0|M{1&RplELX4yrQoRV^%09|6!^7VgqW!WzM4dTEJPDn`sTuvj( zv89AH?BjvDpNZp;_b&WNl*g%2<+@_Y0ClU|3{gP~7ci4}(4aH}ydc$D@6~STR-nGG z!qG+x|0d=B=;`&xDumlskIbpm&w8UFj{xOas8w_w4yus%axBKa4ad7?@sDE8pvd*B zfGT`U2NB2I4IyLftMzdm148*}zjyRU%tU>_CUB;LYt0%@4hrNVN^^jet9t3fnh@v* zoO%TTfjxlgcj24Fd95Uowo_3oMUwL5w6*#E02p%mZlbPzdElv#B(4Ia5}lDz(42_{ zF1vUr%A1iCs_oS1d3=giujoK)n0I#*Dvxmdg5RJr=pzA>p~j9XV@y}w!&VMl5EiE7Xb%QSxs4b4lzojc(H-uP!A5MnW60k4yr+ zQ4FA!>yy;4JDQK4nQTLN5^-X6P2byR(?i>Yh?vxTf> zK0yzEn~Nsz1r6p8$0o-t1)`0MjAi6p`ix7TGFtQ9kEw-#4?=VgTZs|xWc{9Zz zd^k`0A_1u$e*;GqH35^WWF@`kn<_|(SN)%Cn3sUsOq`iCE|P^A*-+{sV~xL+qe1Dl z{DV{=&Fke)Sf2>gy{kkBF+U=MM0myq^GL92?f8YsBnH!;;eI!hWEz~gJUV&#>&i#T zO`!pinFIlKsl}{ zRzUZscy?n3&py#%|J`tk0alP3L7DJhamfdWOKwBH`Ir8t>i<&)pw^{V{M!rge|kIw z)uGlYTFnne=l-lB5BvoWX!J3Ijj`$dH3ms~g{_?yl1k*e)EX@OhznAUSKywE+y;M)@@2AB< zl@OCCAn}g&SLlqau}T*5t%t8lNeM%W zq7^_H-5DD!V}`x*AW&l2FhAVTfQ?sNsp--9MpGEvikFOxJkBaEcC0yYmxB2~8p=+?du53o#4$am_PI3~ao#xJ@gDFVzbPHu~}(!Ev?b>`JbQFNP@3|7c&L zjA`o`7rF9oHsWwlCy7DHtQuPFnT$-W<5=nK(Xd|=xx4bEHFw-h%x6fru-i1&S&p^y zYgc>kSlaNLMq)p~tGu|6BqT6m`Bn|)`KeZa&6&wnj0T2ioQ8S*O#aE7a3d-*NW|m! z?nRvZ-dgi+D(CR4ddl-YA3F-u&Qs`~jINE1cYYlWIcipzZDU+mE($ufdv#1MX~sg$ zE$BQ$8`g@u`}XYHY&BcLvjS)Ngp^?6o8i}<>#8WIt8x?zPZU3vC&1HWGvtKqhjJ74 zPaHYTGS9QA>Y=9=b~H?`>OH$GJ7-|1-D7-iy@uczijp5qg;|*NhJ`4MQH}zc6Kt7XG1mD6G@>*#q(8 z`o~~lp#*(7A__vq4Ied@>-9wxN0Bf?`4&CJ8P82 zB@Y=59h|TF_j=!*T|o~jj>cIfBU(EUn_qf+m6LjOJINW<@GeOs+xzoug2CUS0jcmY;Zw@cng0 zrz;o>1}+0<3^-PbOT)?Kgk2It))Tw$jN?jK$bE`b?I&CqHc;p_IY%?mM|MW1?I?34 zRgL=kf)Ko~2O;NL>_>j+J?@(PY0Bp)p_GJu*J<3L*M7*B@f|!9cQ?0lhbuJq+h@GjoTxx{ApA@YMz zg0sujrOLeEbE^i1>*ld3W=zPA#*WTyqhd} zl)jy~-m!t*nx_AjX=|~a?@1!^aGZO5*;{Rg%peZis;^6;c;Ujf&CFU4OXCAFC97k_ zlVNG%lQ~Vf5VwN@rbI$@8ZoXlk*)}-fge1 z-Ml8TfOEOPpcO#2Yui+D4;MJ+Hz|G{r#fEg*_JK6n+W5=DZ7 z*xRXxb2yUv9WHM@a?lwV@^TD&Ms6xg&RjASbxP7|s;uv-M8 zkeWlTt{Zh=20QkrPId=LNum2a^WbQcQ8AacN|tI{x?Gnx_X2A!%Ew0Q!&KHrqezm& zljf;ftd_`2_(ZL}Oa}7_w$=~f`#-L|4^`TUZJl%OZGG`bKF9mPj##Pld|HJzn>F#b6M7lc!1(8Oj zyE~=pNF$|ygmjA1edz9%?hfhh_-}vLecku-;(2-YXP>=h&CHtl&a7Fp0zjGiEfgzU znOvJ-=n4^`a}#BM(e75W(d;e-XtZy7qP1=a7xk`~Hup3>r?iQncYMQkY?2&*Qyfg? zg5A!V%H9eNn5C-vTHPmA(h?&Q;5W@RzKUT#{&NzRm|O;2yOQPw|K`x$6$K5|B00Bu zr9dzuaLv9{hLp~l9{9toIUx0V=6UHVgklW}Ml4HZ_7&=JR*G}O^T$|%4c>! z3&G)*vbB{l)&V$X7qTJ<-OUevTsd@Zb%`?X*-n!X11T8<5b>g0eEXqWl=PBoDz8Wt z0}oM!ZxX)bjuC)MU_4SYZ9d5lc2o4yaUbBOyeiRp4t4G)rkzH?9e7?<$jdNxDJ z4OlO*Uq z#+JMnY*6zLN$nYPgP0$Qnwjk@s`_o!1nJxD-JuQeT#z!G&CQoVuz}Uy-3pyoX-%__ zaEBv3fdh1xoLf}|3WjNBe z9W61+L<1}PR=vIX`nf%%q`eSC!C+2vlc1}o?-hTA_Vrrk$LOGbi~0Q}&Y-cMwjHHp zrWV)j?t#U3kq4913@f+{-&BxkVWbLNePFTAfnDMt)WF$Qh|lzgvAW(Qgmm(Az)FA^ zCmVziFGiV@b3(*{`6(B-c1+k#d{3cOrX6)cz#Vr*u+jD^rv}^y^^5XQNXa0SM%W&E za(^+x){7gv?hdTfUH!mG+sRRsI6wer3O1L#6XQG@uB_Pr)VgFyxUNFZ#~f~{hm8%p zps{ZSwgcsxBy*mry$ac(JbIZPGFqpF-;ao)E14wiX}YO(SLH#qX{YV~EUBPO$cy{; zX~h=;;%K4XiIHEbPd*lo;)P892`sLf%~sS>c-qORQ9&4)l1>U$X53WVp+o@S;v(y6w?BMMY51iF#6oN)Zn;Pyf5=~wW;u=!CoKr(e-Cp zjpQn#dh`pLsV%8AS^maD5*ki6D%nt}x85SEvA>pckVHB2H+lu~T37N-o-{8_4R=&H zhZwEarGGJY`SL;zb;qpuZ*=@U^GHp~h6p%hAqjHZif&2QHlt(vx&FSL0PrT>2#62U zRUOc_=2FFvkw>bL{zkSoqD47Up~wI6*9=(|^JHlxr*nf}={mZIxH*ZMX%YH^ZIydC z_`QlT;FkZygnvA(Z8?v!)UdY^T)y&ESbsu z&+XQ3r5{i@7TdP^n>*Vpmbf&ZT&9*JJcD|_iIi&_t*T$HrE<&C9gH`-Y%59jI(o-reP+SQrq`<9nNWeX=|W|cr^x2K!dA!%4|OI4UZ5WuWG?8o}$~< zqHbc7B^%rA`AeIgSq!`Sgj#gv#1Mv`=MH9L_{SR2$%Dz&-!9OZT?=_74`Vm*OD(#& zVCFZiIorGxr^=$WWKl&^lr=6*1Ty2hHa`VchB``LuW`7qJin`U$wQD3Z0hE`1ZxHp z$(^i)Z0Z4$pi5AdD_L7nV0d*VRB%LwYq__QXjmmPq~`qMqybu~pM$^cYSleQu-aNZ znkkosyLE*#aB26<%u?V0N&{C*SQ6X8M)G;FGTc5O4D+|+6<+Cc-C7RplF3x-+==HN zWbSD{8K@$;S}4*;tDPm80<-%k&LfR#<~lP6B03X)=F3a4Pv&smpq(GD9+NaUJk4{` zV|W_aH1_UGNY_snj{MF)@U;;?aA_9>%)$|78k*T}ZSOHK^aRMH)V*;VMosd+ZQuOo z6weJz!h;;-SU&1c52Y$$0nR_s=rwld`(@z;39HmmRDRf(AF!dDAHq2 zY8ug@5kQg1#d*%5z0bsiizCKzSGaoi)_j63iFSLaKE{wJsmzn=7)(*_a9qNwpHs>bGx65 z@Ojven!CGiY99r0oU=U1CHDviU06+5KX1`_48S2%7PKi}Z<;F&FY5mwkUhhhuU(dx zD-bDXaKm-Pdb{f)PQyKZcaeq*AU$9821Iv^Sf31gOEAB$02 zJymPl*DYzQEmGn|Dp32jP&-4?w2nS6xR1EU)`Qh%*7obW3$+pm-H+BwUpI~F;R9xa zFC?_Q@uCWy5e`Jp-co4Rgy-7p&ok&%c;N@Lk%dfcHH8#;)Ju8Eoy7q(N>ml=*_z)r z;WfM81WGn~^HWr3e%JN}U2Zf78(n>7FBP%Ae_1{6Bk_{$yP8d4{~T6_OX?8++^!ly z)b35;pu5%097%;I9^d933j3b3q1d(*XX!)|ysAG}pwiWg`Mc#o>t5WNx?(H+n0|NR zK<5#nak_!#e%;rtzm*<#J@camN@y_2(~A#Gnb3nB&&V=l3*E+_xgs?ido6C(k1bEe z(z-tvV)kq;1c)LXfo-U8WejO!@Khb7ZciOe=PE{<;$7?)UUg%9YTt@(-3=G(YD+D^ zE%t$6E9+5s<~at1nF1u?mAnX_~fiqA)P)~nGb3k$bKS;|g~v307$ z^e{bFKD8Hf>%Bm&WGu~4F1#!E71H}IO0YKR6CzpX89b(jKAWQrPXyH?9JzZ?;WP{2 zs0V^1^!RT0L{X<)>u?zP|Ckz~YJ4a^bt>gCCbUbvA$X8a5750Hz{Xj>7i9!4jw6^^ zUM+R`ID80k9xbV@nzaB8++LCj4-Tg8@)k@bJENU=0m3&^wn_Sb` zzUx{YH<@-vF)zAr8llZcRZziqT`_~G5~PkO@jz8sG>N8ODLj>8sa=(-V1%JB;@FAp zZge4>*vrw6ObBRhPu&sdEF4Ro*?QXGv;@?fSa!XQ&=8@D9`qviFrW(J7~J&NVw2$Awo9C*wm+svAF^FbT36rcjJ!hui=CfnI5Q9jcmpJ(! z!>0b~fNh7)Z@m(9Kp*lwm$#!?>!Lki;n5hZ_Gc|?)_9=QOW~(@S2HC2DGV9s6@zdI z8qB}ZB4rXFuG)#7tJjGlvz%-kJeY$d`ZJm}`+T8<`HZ5i`5eRZ_66Y$3jkOI|BK6Z z%%*|MT0l&fn~5iT=r@@J1SUBy_tQ8E7_#z(?x_8;{oq{Yx#bz5%Sl?JBD_P=Vv5_(Sh_by~Cw-frg1)vGIK6iJX?IEP^XE4&!zT5zbpt`Nbph z3xZ9AOPmXko3GnHpF0@#H<=-4D4OmPanAw~+FYc=176t9Z>u2SR!7X2z}>mj5V^V3 zAgP^?rAE&OMSZm%8PmRVlEm`ge}>v^1PUvGc^Kcfx}emB&g>mUcJeg4ZT5>JNorQf z=TCO&pTAi@h;;>6k}*bI=}}=8r$|+h`@h-DAaJU|2`94;Ki)yH=)UBjvAb#v!f| z#ZR@WDccJ_I-k{dJgijbu%6Q)aANF6gDf+@iv5fk32e{p35>t37{YN`I8eAU+Hiv( zZy`SesX|oo1gUyXsZn1~hzNxV?t6{zMcmfw1g#}s^32Q>K0oVJ9A1nu;hgbB$LzaLy&wliLs7;T*D(2ju3OX`Z3Ot_bEw4xs&rE$|8P z z!8ms?+uC?ZgdMneUB0VRg;;Fu%*M7Wg7PzL5+REAywStUaX6iQt=&}Lpg+xalodF+ zmxkFQ?Qug3m7Q)jPX@a71(V>9H3=Mc3zr({6-9BKHHpA7mPGPH`~eW5droPWcEdAy zpktf_yg4JxQrIBsoCgZxk(I*IHLILOyI9f#A0Hk9^#KG!Kcl~n=Re28=EtTjaDX60 zix@R*bADk$Y-$hD7mS+bjR(vu9gSo9og^_>jZ|+ok`t-z$l*!z7eLHl=^xEolckhG&L$dH2v@bdu zq4YZ+xuF?K?FVmU->- zAK3)6Mh#}a3JI4V!J@VRS#$1im=V1M^(q>=gH^k+!t-Sat754Q)NFIF9CE5QgQia59WYMR;0lV2IEsUQyvZGq`&uz?Z zpe=$%DLb)5WA^c~;bZIQc@e0XM8it@sw{XJP>3cUZT7J=$U1>vSZ7Rc5hONpJ_}^I zs(X*z%8nPkM@QXIJoThG9gg=7YDJeZ{+;);)inBa9=D~*M1m0SouhF*`q825laXpO zxqgPFzJaPM_t6A*-&Tjco~3>LAvwR>&F1%Vzx;Q=kdaet1`h)6u?nA7lrb?>IvwCZFT(D1JcgH_3tqW}RE!oW%Jef( z)I2upouVJ4K5(fWYK42VwY*Fy?|DQ4tT+klVKcL?Y^7u zHCn93{9eACjdk}dvf_BMO~u{*dg}r4XT<9cW??`KN#zAX=ez!2?#|hn|JDMq9-ZOV zEcbvJP~J{5`nOHB!!vax)a=)ICG1x?Gfg7+)l7R8Zv5Wue=~aNf$`sxSI)B!Ue?r6 zst9%p)yU>$KUn;6pTF)7pH534r5u>&up6C40{sD{X7Vi|CbUHzHhyJNSulxKatTLI zwXUhLK~Fy0veOK#A&TlqmXmG|X)9mqXrKcltuJKhCa#K90%x8S3)>v;<;vJnZC@IS ziRto@+7H~~G~1=1jfjqwv$`iY6Pqq9s>60ZL*;=W;f=H zP#S1x*f!=6aY)iGE5-#L_D`GLMhBVTWtCghwlBaS_GZD+^K@skS*%x1O77ue+sv` zH&k!OZsRHtWlq_?1$zo*SgNo=RcHaUuM`O|qHoE|we9wuqnP!Xo>o;(E6@K(F}oFZ zYNJb&F}pILe69q$1=!+Jr9KET6s3f+ux%c9Vp83s+blN-vn@6xG0@pKelX4@U?_fkCNPZOZ&q)&E(<26(kQBX_=P}Ts_H<$=kd0@ z3t0$V1p8aW2XW3!fjM4&V(-u9n~jHP5Ds=@CcVUSF<+fi%JN9yO5Z=L1OD`sD4if7 zG%k=JhvB_M=lbW2WRr>YzI^pEKB4!P!EC&9SsoKz`Ffe3bM5a_Ms)5T=XsEa2dVti z`G+}*{X(P8&dm6giu zA1;c^N%d=*@NeRT6Jyc(RU8T|5bno~a=;uRz81$RK@v2Oy(c9>)V#sNU9Ro6)(exn4k0>@h@()7D|b}AnKAsDV?vhW~NF# zClb>WqY}PAic(}r9;#E}`P-haOT{HaX_OAYiN5*ia@0dRwOUE!qdeT!us2s;vo;u% zzWg=G9+~)6ml-)=SGEOi7G9t{8^*mEIjpGRZcYMtj~~SMzkg_wq$V<9X|D&3dryqY z^oK9_rU%wR625FhcAjX(dk$2JR!)^0^^6qBVa$FV%%@^4m&tjw(J#aApu@$EV?Nn1 za1W0F!{})p4lk4j>zsMKRVqX;d@T9$<^5Y1mh4&|C5ACSssu0e1lfKmg%z044U6L?aF2r?HFU`iDJ(EGtoX~ z%Imbw<^pbsUWY=5Yp4Cm&aT({J_ir!;F(tDZSF@bzNhbBCv}LENq;0CvhUI#*1T7& zEMijBlb+U8gNc_U4l&o-SJZkk&e~SH(RH}B$!e%h@}q_VUr0yUpty9vz?7}0w^Cl@ zBk2;gcU?BrK*^Xl+SdDn!T9Rd>y|@3p4+Z~odEV~v8ws%q>OU1q9Q^J}d~x^O7E zL_l*`U)PHhD|F5GmEWUJh|24X=$~DrHKShO(jiIY}5Ndy>Y%th#a*I}3L zFm_1VzJKVN5A5Hj~wXsXqHV)d$4S$H2jOc~Jo97n}# zqU$rhg@M5dP)v34q?pVVSE}y9jc0c0Ayi?#8QYC*IYIyKn^gfWzAc?m7K4xoaV|Un z2M0nkmZgh8_XBpA|-dJ}fn|i&nNh&lAh}dFP zAo7&xu*`M3VeLGY8LLf^>$TS_aBLjh_UYr)Py$E>?}G8WTH^sFxcAjHhWwyIgy|`f z#7A@hA3CD}=_9|KR`lD5{6;YT?TdC}e>Fm3@qnqD6zciI&8BkuEh3|n4UJG14JA!4 zVfVSeu?}kKtXY6Fp*G zZG;!Tcm=$gba&OTr+WjOh3@y+YgGeCITtP%nF5gND6ZwpvId@HH`k%9_SE#4ca5n+ zlr$rF5&7qMQAd=sbT4a@Mtpc;D~8s-uMp3n`pN^j$~s+!zp0NZG7JzOwH`6j_w?km zHxSQVG`5t-nN6kt>M*WQY36}iAPQ{r9iUt5ymx-*FfDi{C+ilGf&yDszF`sH!2DLo zG}8k@B~6Wn2rhebY*XvDTP++E2d(rqxNk?a0Q>3tx=uILnN@oWvh>ecvv|IZ>-qyr z_lUU)gxfe2tQdCd>&2yvG*FtokfWZDG+78RK_qudak8C4Am8p+nGbNFbsdTTwK9$W z-Xg)fsO9v+?Pxxp|Jq;*T0Kd`bBhqTZC!WIu6VkdzV~@+T@N2_w;nCP=>u)V_gb#5 zVV86!{n1SM;aa5eSiGU(obrUc?`THmxs?!dn#+QCK8nvP6QWjOU!q zNsX&@&PKz>I^26k3>Tu`sZbXrUB`zrxQ_|s%X0*U#Wu?J5~lTPk{M!+w4e_NoE7Ww z_Rnab5)3l+D6ay6(!E#w3z+lF8&KC zcM~BRXvT+w&Z^vM;Ocai5rxFr)_P_}G6mhh_L}ISdKGnZ|0*}{f<~9!A`vvKzaCX>KDKv zSA<^UglqheR0M3$O;RdJ8?j5f4jLJyXAAl*Tb{k$!Hs#!&oJbpqDy*N;t@YpKRB)LC0tvj*O@Is|oSw9z(&^zT4ajZh$$feWlmwagsPA3tH zi+;|X;#*MX87UKo77I~u0V;X`Z>;umTU*e1`Kjl#TtR8y?lkW?=#j*QbE*^&XTQm= zFE->QES52JL501!2S*h)ZdlZgYWzgGoM&zGLQ3_KVdLKky z#0C~7ok4vyv7xM(594#@i^4|C86L-%86Wft59?SGeFxSq@#rUh1QN17N(_>(^(T@A~tEaTa$z zrOv6Z=5)GCQw09!wfviTwZSRcMX1KRrX(+Qk)aeyYE|>(7~|Q|iIPqjuF}u6_r)Ym zVhLucgPe`hU)PUTEr_J>l~#tCsmw1XFK5*1(dRS6iXJgZwAMz6l6px~<{k-(NiUv6 z9f-cjNqfli?MIs?#oyXwUEL@;5GpBKLhfDv2>V5C%hm;YdUV)6u^II-c_< z$=+YhaXxl3-AB{2mL(f!*JOS~kG4O}i4Sij>+FI!TZ(LhoBJ>TBS z2Iw#nZ?WY1CtZFEpwPIkSi5)i?*7#+B40Vau5R@ z@D{z*>U)SCr5EoOhRDpZ1Dsc(N~HPP_MveFvHa2o$<6=tcBN~MdK*8zYa?K2?X%$) zch;>zcMUI=_fmr|d##r%7xR{=-jZN~ZPoR-&&Do|*TepePfUnEJ?L%(6rH9Zf8onh zxDn~-vv3*3E7mp7NE$)5-m=UGlPx;o>ifNh=m71)whsV_(I+Ac5DTzz zMupC|{q`F}8)1iow*o+mX5Jfny+EIiMu$+%(z4XHKS}$%jqQ z#e=o`ZoOb>G>ksTNBFP2gh6@XA;1V}dhv%5#6?^DO}(+?`o=GR zqArki=-CON18Zvrh$L}wL*k4NQSZ2;T*lMe21W-r=G>o|V_g;{%f^~Ubuu9r0)ydx z4WCz5A?8jCp>hJ&5*GA#NwJul?rWQ$_eIJN5EqxuNf9HTQ99RU>ay315R`$Od9^nZ zTx|>j`n#uIcmjD_8fl^%?5=b)(ZY!p$E}(7Q{T5dJjbjH`MoAvDQeNJAqGRG*MG>T zBK0Gh;+bXiUM;QbwSGc7VT(n&+}}Ul+xxtsK$uu55@L>cC6s$8NczKJY`b3|`w{hN zw@?k{$I{{n&s`1uwEiV^@y?i4!Z^MUt2NFnJqdna-o3PQ22URVKi)`d3WDBI)d)|p zc*iH>MI66R?AtR3#i=5V9!k{yp!FErN=;?0J3X=9Yf$M?Al!j5(m90tFS)&2E(*g##JkWk|vwi{1ztdwg4S^}f5Q?G7T| zto=YUZ0`}9bwe{)Z^#a?)Vbpc6m6Bwd##q(VBbhFQdrH+jcr>S>9yk7X?s(9@CqMO zMgb#4WqpIoalSg@He3nVq6uf~X;IaeEsKOx-#Gm)oy_cSL$OcAe=O=Bj_^s(3G^*U zE;B>v028jta0w7D>K!i#6fE`L-LBVf#GpcT6w zWh%d;OeK1bO)+7NaA4p~?VkeMEWGny(04i|_A}Z|+rJuc-oJBlwGM~K{sAp9&%t+& z2`s==k|Jc4ON9nR*jFY~MtVIy3DIusd7LB99q*+e+I=?UAYW|Pp1IZFgd%{v#Zc^9 zE+06(+IM_M8GdY+(t!30Evfp1^P$3-&8y)<>-}^{JtCk`_?ebIlb=29nm=lv7b!Xb z+A`2EjIMrCF?>k-gA2KY74UZ`7CFkTy8C$k)cF|3ebouK7Wn11opqPg+1#bwJ>M|w z;jrNBty!HW-H&{55oj2%PfZJ+oV-5=E(+k)NNX= z8F|qBcOSB?oM;fdQ(_&trlPKJi7~~vb1Z$(ci$$~8`D14uXs59a+`+ue$<}Z>nV(r z3~YIc0tV-ne}i)xfZ%hOXqV#@oLrFp^pz3iTQ1||U#IcMmoD5%>XjHJe58B+8u@e* zpd-T;HTL4z^}2L!umyYK7dGWv5c2yx$bfX%)g!jNx~aG`$u)7>O=QyHx?;J>J+quT zvp=Veq+B=*Xb2KXxaA{%Q8Qw2JyN~o?|xg;r)0NH0&0s;uye-*9xk+{GwDzF>F}Cw zXAhNT6sww8|8PO_*3p}ba2EDCn5nQk^2!D!1lp4FV_1H#yTW|2KQJvjWeNIRt8);% zK7%YF*nMLqcqPGUFqE33sT)P-;#hdA#MaC8cqqzaVDW-BynoW5s#|NtyCUZm?UBM? zMU@5>6#)e%piV)IA&}~)bi*o&S0}g$Il#Pv`whycE!|P`eXz9`gpzcq>w}Bf_Um4h zv1Gzmk&iovq_KPyHSOqoZEfG?&#pynehyeVLNVIPjTqq7Z$5T&inyR$Qa>_B?TZG- z37DsdUHbV!g(N)Msm!(-hUK)Es<>|uO^)rSlp;jNA)tBt2&}&;q?<>C5oYKc3H?Rs zjonL|`8A%|&0&w4H%g21LLm)xMktdtG*AQyekHo4UD2;z3=jT>3rK>Vd5h&NL`22Q zd9VGR);KYX3G4VeKqV~02Rg*&g`CjTBkMxbP!}lum?KZkt7_tFoWwUR zdP|pqY$>#VIgJhj8jar7T8#wV-ECWwBz|8(_+8S+p?SKw#lE%`erOs(iA>XD42w^v zD$^C9_o2g1aYt^9^~A%6vU}sNZ=L!R%4;8U)ba}dI010Pbph&Ab=U|oFaJ&1o-8!$)w7`b6;{j*vLK)6OEIe@mDjIlS7 zyHT8TGhmBGi^+;otXZM`vi)UzP|ObZ`6J#_!rVd~2aLcA0md_uZWsE2#m7$gf?YSl zQWKUM|3rC)!!zt+zp&S@<@U0DSC6)~kK_z9X+Wqz@IE_Zpp+0`u#8L>D_f}bI3v{I z-%;g?@e<}UU|0@ERXeSJ_DEC1M_qw|Kv~G}m+m6o_L$q#Xneo!oU`mde?hVOJ7o`D zL}ypuE_*eRSF%#5pA5hr<=N^s>THzWY&8){Z9SE=QPsT!W?5ov?0UoKf`?q(mkMN0 zFb8oTD9T?5E*dZ?qwIm$MPgI39wo2Aen6bl$s4UrE&09*`4s zzS{cL{rqCGl66@QbkCGD6RABNdNQZWvd!iORzk^E@+CNd&&%;g~WIgX=IE&R@6B zA3XQi6UW`4vSndWk#WZl`dXF2k9uePeDAkLG6v};f}J;4UH~lAVZ{PNLx~;i-S>gA zq)U!_Z#EtRPGwuU^6cG%rks|fy;iXX)wJO()a`W`rn8(i4lg$?p9%>~=IJ)UqVK-b z9q3#%53pXPa2--P2-+Ju*!aJFDY-21*U~70mIjUr3IK`t9^cdXbep-BXP4sU$Jxs& z(@%DYalIE8i$hA+8^o8Y)*h(S$mp{Ubu@bjFw`FuFjyz5XtYsZX*k?_`tlFeQkCwz z9c66N-M+t|^QdksvDq5Z@sv_1E3Nk;B%B#3eVEm_csX&=n4;#*dp@<}lTAU9_Ol_WkmtG4CV& zu&rl+UvWCB-D%KlaOcuEt|GMMO4ULk!il@Q-@sR7U4)J5PFLF1ZLxPK#p1Y9|%E9a=i&2DRc;N2Ug1H$#^NN(D(OZ}UDws)SrPgq!W2dVsY zM>tXa3tk@JK`AA`s1#Wus^f)F6)-Z`q5XjWL+4CsZ!pP+(P{sP+Ce_uF)AegTpO}D z!xxP_CDv`8+pit<@Hz&#Vi?od$kLXldlGYVx>k4!r#bx?>_R#1I*PwuUsmABuVga3 z*{e{&Wb?xIxaf+gX_l^)vir-pe46+MD>*47s@{) zb3&%sLUFM7lW2cU(reHptwRlnluVo3%40s@;LG<+`k=q>_ZwugiM<{?;e5&)&ZJ3v zv{vX$@-=><_@Mei*~8~qv}tqS(v+A&b)lS{-0dac>JX{Hy|*eKI!cOe?{m9NvU z4s_-ZH4V_i9s%_2%7QY7&TEt-0N;32a<)Ev| zZiC5r6w$TNLfQ`K^5{AH@%Rm~hGxU+2((4dVh~&RR6N#)ZE}OQH}*|)j#$drtp>xn z@X*D@VeE8(`PhulvJS`g3H^*{sE- z&?&4pPZMyii2g1$D{!fWBDNIN9-iJz3YsLC2W)AJoPP@^A+%2RdN+Uk)GKBicPUaY z%o5}n9epmgW-sZP!hw{NP1c&3U;7}%6YC?slai|8W66U#n_sg;-s{azT0^Fxr9YkO z_g^>X8_;4_BQ7W7qA3WN)G!AN5Xv<$2eX0cu1E!1)+T0A{INZpx+i#EW9+rqjre14 zA{AVDLfTq4GjmygkrmW>UDG#|-P50t%V|~VbSxVh4Lgc>{?;{dA%Bz7evBfo=-*H$ z*~GFS@h;=%vu&mN#jGzPIjjnnKDDV~4n7+UR!A<0|K)z)(m!=tqNa&|&9< zf;Rcnl8%knt>Q-(ANu(tSplQ(h%n|T6D7KZHPt^~M^Y;T?Q*)1D_k7hk3grlvubmo z%Nuo%7$*oY-1HUSP^_<*dk%&GzEb(bw$I~%Zwr`bz#W~MD z*04f0o1*H-o+z-eX>_K8@*vK&wkThFr0lz+&T!KqTRQY3yG(aS{^{5YpR1r8Q=!Co zDJl&fNR92*?WbB@^B;cBudQkuCWGtGC;K%u4zxvzLL8}$nzTefYU6F5&*tv9iJzDI zAbLaGEm5-d$p~9)YH=#MNhqR(0LMaeWLJc)nttmOW^*pQR|O8W10FEfC>c1!+OkCU%{T4Z*!l$Vfq8$#_n_vFecgkZg#+6_&sU8g6kx|2 z(}rcKZ!rCFl&^|Uqv;~~asM76_D4%UqL0#6b$0RQ00f^h@*R%dSSbkaf30H9UtNpt z^z)NXyXD0a?Z= zl!vBq=s*!`_4|Y6C2RfFZ?=cZ2PK_C^9UStK-2d;hNIz~as`pnP0hu{YuqY`J&xBt z*~X=_;U?*w(;a5Ycd^!EOKL2{&)}Icm7S-y1dtI~B)a+UxEoLdFBh>xyY%6J-Lz*V znI!3&4cKn>ItVT5_Mc|jeSPM1sAi9#L9UzP&V@yU@0vD#gbRoDR}5`6cj>l=9t5=Q z7dhoL(t%CBNGO9sfzT*N*M0J>Oo5AB!7bmcrePstl9cDcP7pbhWzHdVm;U*6>&7hG9vNv18_AyE^v<+bkJ z&P9DI(j@~C!!Fc@NgL*4s1`Gq{39EbMjrX#4UaZL%hwDlh>LV4aD?Q| z!mKkWJ~^;HmSy^WZgFBsM9xoxo_qt^f>3-d;wN}n?cE zpC6#w>DRH+6Ba=wqzrXYPhDA8$`neWtI3aV zlEK&lWwR&$6?jcggmK6vT^k~7Cc4NstC0c6Pgp1T-gl=({w7s3hmJg_)d8J2c}vO1BMNEwwbRUV6j4X-?mu@!5a}+ z=y8$u!T13wV4O6WBoo>ac)`139@*<>dW^I<%Nae4mQdcv*6blZ`={jLzMBsu!!e7e z*D@ko-_eIOeUo)#ut7-!44TXHL=99gs$ItqGYIVoW;nDn6W;@x*eqQuSBKkO&9J zGWJ#P&*@gi>u?YHL;T4&`T{8A>NB1d9Q1F3I5->7-${@^*z3GZr z_Ro{0St7|PJCt!e@$a_mJIYU)kZ{+xzQ}`{{IVD>{YU+9vyM*C*_StF<6I1f0Vub$ zdvL5!U|V1k_d_h!m!y%9-Y3JGmyGjo-`c4D*iX|8jrddByJV7_&g{+mp6`RojK*t2 zTh!W#`aKK}VQci>0`|-S?Vw!k@*zq)_oZrOOI(2X@|LY8vi|$n4fRrH(&g;$@sb1H zABRw}1_(Rm1B2E5d7PJZzSXa+hNhVPs`}pISw$5Eu{kSrCY!?2L(3*Cx0R(Sw8-DH zLq_hm+XYEndr;|%9xHV>x)oaE-dVk>@cMYMj*#oqXXFM?F|6`$z@X{934Mc(?I?ef zu=$19rve#$NhA3!)HjjN%JO-WN!tYE3%)*le1p8!n<h}u}f&3mv83t0o^`1Q`ui^*xP zYmf8JxTyq{S>=~{x~Oxz;YZ0ly$l)AD|2ed&1y_M=r9e!{%--pYhF(jv;3?)3zS~( zV4B%{-sy8UYPX4x^OpVA+&be<_!W(f?DiSh>QZJQOw1Zjr|e92yLE)^$q~gQRWbhl zbsYvhAg}mw&GMk)jt8;fVyYRH$N2YUJkNeB*Gun+?poBf0wbYe?40Ut!l~4OIE>$T zbwW+=sn{$lw#ijYJHB!9CNIA+Bk#N`IOU)A?oWQvYGviR&3#HYJ{0TvqjLYrm!vpF zY1I-%+pj|NoEG~Y^gu)|^N$zN7F7YS!>!b?cF)N85+F zGW2(IB>civGe4k0`{%4SwuTeO=#T1&d$rR}ETv=RyRhIH==3xF2TGVwVF5_yWIJOd zF;r3l?6p8UA_juB*=~h2y#GAFnFO;Sf#*lEYe6s^oWBn%qyI)JjEiQuu?x1i2V)I4c`|l{e_x>Z3_DI_g`T5>B&+=U@D(=1dEA2d?IT!ab%s%Q z_WtkiG~j=50qsxv;lP*Pzd!yC3EujX$O^Mrb_1QY`AF`Me<-CvU?E^0-N6R2CD(uc zi4%$h-9&)B20H1K{T7vgh+6z0K|}+8&;DosrR5b71fRbicaRZ2aSwthQe)qG{g8!nw)Ccjz*RP^&c9+_i&{pR$jmRyefWG zWPWi8x!PDv`R)c;peyYEP5FQG`1`Gv#7A1{!Em_yxi#cyeOi0;Mv=!SFy_C*;5Z`e z?2`a&yibZY_YR*NaT7GZwHGU!4|uwlI!|kFRfj&{7wnaV!k>ZAo1b*oIJ9V zWnupG{$I)Cd_t%WI7iWH>?%#Gzg&VQhHC4FYj=g`?`xia1WEa zFBHD%v!5wNQ@xhZ4co*bpsEaoy&-k^6Y|geUWr1eOVRf0BCRH!yJzb_Mpnp}j;z-s zY}_z^M?giTIFEe|{3AMvb)5XyyFu=E{L4ygraNd$Q_$t5?3gfhbA07!s3lC;2}}aw zLjRFG&IAG)tOo9QUzC_5u{6D#9vt3etSE7e&4UlS%9nnY`rj!rA@JR(5IBfN=<^OY0-uP*XKs&EQruKxt9W3d5dQ9T`7Y0w?lUR0cgcdtGF zqsX|DEl(j_n|{qM;{2xU!f+r_tPt5W9zvd(xjf?soT{;B(qUVU<4 zG-$dqUv?C2cd=GKSW0@NKDo>+6j;R{M(Q^~5}^u3yW0Q zZa&mwvjTako0;mJI3hQ81Y21yQ_lk#bcz1u{s98?d3>hsi=gkON`@Mh&^0V^xZ$_FVu7~c*IUS9~O9`X4}y4jC=WK3?;`- zUgSrOf8K6Oh9SQHw0n6~=ms~vFJ4A~)$aDHbQ{*@LDk=AEFYyI{`UMw6GMfdE?z}f z$))4_2vFwQXZ3vUkfV;oMU5M>xH?i25|{0q_J6HJ+c#Ji%AhtJ>i5jWSS%J5fgKp& zlW@84(AZI1_FCzam}N!ykDaMQ$p_xrBRm8XUShpde@|KXh5)zW2)GSNbcpwdMsA3aCCvPSqMF$4*O$p2Lj4mFAe5D!v17I zo6#%%C&AL@23dUg{DAu2-VY&!2pEeFgDh#pQ4(y6(C}(q$%_9gl|X99La6E?BpTGB z+7PS9OhlN6(@oDuWn$Q_9}MzbWCi(^7R$e+zd+eTxM#6@{8+qn7un;7e9`0r;o#;@ z%gbqn$I0T_NIsD>X~|EN`Yrnp2T{18L~ngwNr=B3jb1dF<-tc^)-IU@VGqgKp^B`# zPBXOl6(`RC?2Yrk^t=*+;9oXV4c=P}hu_(_C|XluNdvHB#F51?_*u!Pu}>Y7Xz|VH zkbi9jrF09j6>9$N@%0uZ%aPJt4KRSuAl?#T{M&(d!GDv>;8lo=?2G@IIjq+$i8lqK zA}BkAXj8U^WFb6{$!-1>K%(#u1?N%wi_21Y(+txW0aEOLN03|&#m|3Th9$Qot2&03 zx2}dfmbHusW7;n>B!wI2WsECWpkh;C7}U!AtBokAiR}0L7HP& z|4hiBG;u@hUy7peIrMLsM{i!^&2vvzSJNhf8KokNdLWVaMLZcVUh}S*&K*$XRoR=+ za{M6o>0cGiijhD)A`?0yc*kFC!~Cfl9eCHCZCt-H$+)_z$YcCzy;_as@_(G&f0}We zeML}Y9<%HlUNALTRh4sr3X^kmUc4e``1JVF5p|oTWn!{#!}njhBr^sdM2R%;WgX7G{pD$%;iRYOgd>}^GAd@3Omq3pOBa9)+BNOKU6 z2f{OxEL68)|DM2e075$3@~s3Wl+TD-AG$p zXeP>{R033~rj8{rYE0rb@lE$(GKs;8;P(ICCV%D0-Et?Yu*{md7T$2}B& zV$f5X@HC9+sksgvfc=L@K>PpzbZ|dOUOy}SgZZ*ea@a7;w!^)&bQh@dsEepemtr*7 z(H~i=?h3gn`>WPAjA<#rU8;`H{n|Z z5)>rcUHECH6$7}FuE|rvge|;NUrgQGS93;rEEwR~RxN7+>T3`Bq$_nAn=z1C_+=*+ z8zpdPOFX(Dt^n*@9~x5E+R+t7M@Llp#BDj4G|pMxBYwuD(hN{-OsyKB!44z!+zw=+Yx^mPPu@ zHKgy=Sm!nB&5@!>a(m%;BHCrj}ir6_$o*-(5iF+obi(iB0tKx0H_YLo8vDu zjMwZxb;+~jSpe?-Y`jup);Q!q!;D4`I&sF0HM`g>Dq@S8crIjLtg=lF-sR0R8S#7O zhA3rSD8pNyj^&&AI_OU;n1YF3CMUb~Nn4=Xb#Fg?51N7&5v}c7oVy5l(L2R&x(QrZ z&^|QZPkQjXf##z7VX=Jx^tv|Y=u{BfFLm6?4aWF@uPFC6`BzTD}}C_yJ0K*cGM|Lk}ybh1{BxK zs`^)qrb(N>e-I$q)T`)$f+a}RB_9`bDeAPL_MliZVyS?o-Sw;^y4yigHlf8J)Qbo; z>H2aHr{Htenv#Dv(NlQ~;}*`gr|u$nnjm-SO9<#e4ZL5&)5U}O=Uy1{F-NL^4N^vt zI=GsMQ3dXm(_=m0m6RJBV$fIZQ0Ajy%dkuSkVl5(8dv$GV{g@BR&tAGwW5+qIit?q!QV4T3PF6Q6%TaZB|89m0uG;XGyx5&= z4*|0?QTCSGTu~**5KX=Mo+x-+8RNF8u^)GNrAZ|(UhUER7TIuE zDAvF$khQHgjKjNiNaYFIX4uCXjxZ3*{b7nB zcUx`}mX;wCwqqE9*JM>Vda!!7$msCDS|aWVHHwpJ-iQ8C!m2ZEtF;|m80%X6E;{*o zY()a}cZ%#5u$p#3oNHAT>U{-(s{|VeU-gfF99U^@&ZiJFa-{@n71-thk4XuS$2uML zJ7vlqDcFm6Cg_eaG(^Uq$)525M@f3!63qZdd;dT7IE)HPoviP*+KM4!maxd``vv&M z*di3@M0ip;DCkgd!Y`LO*Q(AhB;GqjGnJ@a2B~0gFvu&5-bK#Jr|bUBzET2w5PVs) zS++Ki>nV&GuWEK|Kgsy+J?b~vX@5qlKO0|REhcnUrwXo_1-678kcVKg{Uznx7&@i*ksVtk|Hsd96V49HI~-3Bpn(`Bi+y&D zQn#h4t7eF(>k%`d8_DTv7iFwb_L`eTkv`5^9(QDPEv4{^*=}~!-DF!K$0?Zn67<^a z&$66wWCnZ~Px=kzRsTz~EdKmKPc`mQ(ekMM7uV@CrSYSt-vrIkxjad373xLrI+&P` z%c~K4F(mT`9O+btd`3c5wBFBA!WCigIJUl6Cax*<&*LYy z+|QV>*$8&b^m6y~JL|#%U%mjLH?#XVp#Obm0j(g#h(KPK4?ZJ^8gSxWkjpbjow+K(vAVful(2MWO2HsQ{ zKh5#F38ZDxpDSjMTkR)iiM{y$Pu{0Dz`{C;=!ejisR6(1HEg_QPrxp~)rq<9u pI+-_j9R2)xC?B3m(8op0EwGl&t$lZl4YWbL$HDfnO&yMy@DC}&+GGF# literal 0 HcmV?d00001 diff --git a/metadata-service/war/src/main/resources/boot/data_platforms.json b/metadata-service/war/src/main/resources/boot/data_platforms.json index 4135280cb1ac86..ab040ad2854ddb 100644 --- a/metadata-service/war/src/main/resources/boot/data_platforms.json +++ b/metadata-service/war/src/main/resources/boot/data_platforms.json @@ -187,6 +187,16 @@ "logoUrl": "/assets/platforms/mysqllogo.png" } }, + { + "urn": "urn:li:dataPlatform:db2", + "aspect": { + "datasetNameDelimiter": ".", + "name": "db2", + "displayName": "DB2", + "type": "RELATIONAL_DB", + "logoUrl": "/assets/platforms/db2logo.png" + } + }, { "urn": "urn:li:dataPlatform:mariadb", "aspect": { From 8a905774f7090abf5edaacc0dae622b327303b20 Mon Sep 17 00:00:00 2001 From: aabharti-visa <145495867+aabharti-visa@users.noreply.github.com> Date: Tue, 11 Jun 2024 17:30:12 +0530 Subject: [PATCH 2/8] feat(ingestion/kafka)-Add support for ingesting schemas from schema registry (#10612) --- .../source/confluent_schema_registry.py | 58 +- .../src/datahub/ingestion/source/kafka.py | 80 +- .../source/kafka_schema_registry_base.py | 14 +- .../source/snowflake/snowflake_assertion.py | 2 - .../source/snowflake/snowflake_query.py | 1 - .../integration/kafka/kafka_mces_golden.json | 889 +++++++++++++++++- .../tests/unit/test_kafka_source.py | 71 +- 7 files changed, 1059 insertions(+), 56 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/confluent_schema_registry.py b/metadata-ingestion/src/datahub/ingestion/source/confluent_schema_registry.py index 54475cb509621d..fba71240282c43 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/confluent_schema_registry.py +++ b/metadata-ingestion/src/datahub/ingestion/source/confluent_schema_registry.py @@ -231,16 +231,25 @@ def get_schemas_from_confluent_ref_json( return all_schemas def _get_schema_and_fields( - self, topic: str, is_key_schema: bool + self, topic: str, is_key_schema: bool, is_subject: bool ) -> Tuple[Optional[Schema], List[SchemaField]]: schema: Optional[Schema] = None - schema_type_str: str = "key" if is_key_schema else "value" - topic_subject: Optional[str] = self._get_subject_for_topic( - topic=topic, is_key_schema=is_key_schema - ) + kafka_entity = "subject" if is_subject else "topic" + + # if provided schema as topic, assuming it as value subject + schema_type_str: Optional[str] = "value" + topic_subject: Optional[str] = None + if not is_subject: + schema_type_str = "key" if is_key_schema else "value" + topic_subject = self._get_subject_for_topic( + topic=topic, is_key_schema=is_key_schema + ) + else: + topic_subject = topic + if topic_subject is not None: logger.debug( - f"The {schema_type_str} schema subject:'{topic_subject}' is found for topic:'{topic}'." + f"The {schema_type_str} schema subject:'{topic_subject}' is found for {kafka_entity}:'{topic}'." ) try: registered_schema = self.schema_registry_client.get_latest_version( @@ -249,7 +258,7 @@ def _get_schema_and_fields( schema = registered_schema.schema except Exception as e: logger.warning( - f"For topic: {topic}, failed to get {schema_type_str} schema from schema registry using subject:'{topic_subject}': {e}." + f"For {kafka_entity}: {topic}, failed to get {schema_type_str} schema from schema registry using subject:'{topic_subject}': {e}." ) self.report.report_warning( topic, @@ -257,21 +266,23 @@ def _get_schema_and_fields( ) else: logger.debug( - f"For topic: {topic}, the schema registry subject for the {schema_type_str} schema is not found." + f"For {kafka_entity}: {topic}, the schema registry subject for the {schema_type_str} schema is not found." ) if not is_key_schema: # Value schema is always expected. Report a warning. self.report.report_warning( topic, f"The schema registry subject for the {schema_type_str} schema is not found." - f" The topic is either schema-less, or no messages have been written to the topic yet.", + f" The {kafka_entity} is either schema-less, or no messages have been written to the {kafka_entity} yet.", ) # Obtain the schema fields from schema for the topic. fields: List[SchemaField] = [] if schema is not None: fields = self._get_schema_fields( - topic=topic, schema=schema, is_key_schema=is_key_schema + topic=topic, + schema=schema, + is_key_schema=is_key_schema, ) return (schema, fields) @@ -352,16 +363,21 @@ def _get_schema_fields( return fields def _get_schema_metadata( - self, topic: str, platform_urn: str + self, topic: str, platform_urn: str, is_subject: bool ) -> Optional[SchemaMetadata]: + # Process the value schema schema, fields = self._get_schema_and_fields( - topic=topic, is_key_schema=False + topic=topic, + is_key_schema=False, + is_subject=is_subject, ) # type: Tuple[Optional[Schema], List[SchemaField]] # Process the key schema key_schema, key_fields = self._get_schema_and_fields( - topic=topic, is_key_schema=True + topic=topic, + is_key_schema=True, + is_subject=is_subject, ) # type:Tuple[Optional[Schema], List[SchemaField]] # Create the schemaMetadata aspect. @@ -388,17 +404,22 @@ def _get_schema_metadata( return None def get_schema_metadata( - self, topic: str, platform_urn: str + self, topic: str, platform_urn: str, is_subject: bool ) -> Optional[SchemaMetadata]: - logger.debug(f"Inside _get_schema_metadata {topic} {platform_urn}") + logger.debug(f"Inside get_schema_metadata {topic} {platform_urn}") + # Process the value schema schema, fields = self._get_schema_and_fields( - topic=topic, is_key_schema=False + topic=topic, + is_key_schema=False, + is_subject=is_subject, ) # type: Tuple[Optional[Schema], List[SchemaField]] # Process the key schema key_schema, key_fields = self._get_schema_and_fields( - topic=topic, is_key_schema=True + topic=topic, + is_key_schema=True, + is_subject=is_subject, ) # type:Tuple[Optional[Schema], List[SchemaField]] # Create the schemaMetadata aspect. @@ -423,3 +444,6 @@ def get_schema_metadata( fields=key_fields + fields, ) return None + + def get_subjects(self) -> List[str]: + return self.known_schema_registry_subjects diff --git a/metadata-ingestion/src/datahub/ingestion/source/kafka.py b/metadata-ingestion/src/datahub/ingestion/source/kafka.py index 99ef737206ab0c..0d718e509d5c58 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/kafka.py +++ b/metadata-ingestion/src/datahub/ingestion/source/kafka.py @@ -303,34 +303,63 @@ def get_workunits_internal(self) -> Iterable[MetadataWorkUnit]: ).topics extra_topic_details = self.fetch_extra_topic_details(topics.keys()) - for t, t_detail in topics.items(): - self.report.report_topic_scanned(t) - if self.source_config.topic_patterns.allowed(t): + for topic, topic_detail in topics.items(): + self.report.report_topic_scanned(topic) + if self.source_config.topic_patterns.allowed(topic): try: yield from self._extract_record( - t, t_detail, extra_topic_details.get(t) + topic, False, topic_detail, extra_topic_details.get(topic) ) except Exception as e: - logger.warning(f"Failed to extract topic {t}", exc_info=True) + logger.warning(f"Failed to extract topic {topic}", exc_info=True) self.report.report_warning( - "topic", f"Exception while extracting topic {t}: {e}" + "topic", f"Exception while extracting topic {topic}: {e}" ) else: - self.report.report_dropped(t) + self.report.report_dropped(topic) + + # Get all subjects from schema registry and ingest them as SCHEMA DatasetSubTypes + for subject in self.schema_registry_client.get_subjects(): + try: + yield from self._extract_record( + subject, True, topic_detail=None, extra_topic_config=None + ) + except Exception as e: + logger.warning(f"Failed to extract subject {subject}", exc_info=True) + self.report.report_warning( + "subject", f"Exception while extracting topic {subject}: {e}" + ) def _extract_record( self, topic: str, + is_subject: bool, topic_detail: Optional[TopicMetadata], extra_topic_config: Optional[Dict[str, ConfigEntry]], ) -> Iterable[MetadataWorkUnit]: - logger.debug(f"topic = {topic}") - AVRO = "AVRO" - # 1. Create the default dataset snapshot for the topic. - dataset_name = topic + kafka_entity = "subject" if is_subject else "topic" + + logger.debug(f"extracting schema metadata from kafka entity = {kafka_entity}") + platform_urn = make_data_platform_urn(self.platform) + + # 1. Create schemaMetadata aspect (pass control to SchemaRegistry) + schema_metadata = self.schema_registry_client.get_schema_metadata( + topic, platform_urn, is_subject + ) + + # topic can have no associated subject, but still it can be ingested without schema + # for schema ingestion, ingest only if it has valid schema + if is_subject: + if schema_metadata is None: + return + dataset_name = schema_metadata.schemaName + else: + dataset_name = topic + + # 2. Create the default dataset snapshot for the topic. dataset_urn = make_dataset_urn_with_platform_instance( platform=self.platform, name=dataset_name, @@ -342,10 +371,6 @@ def _extract_record( aspects=[Status(removed=False)], # we append to this list later on ) - # 2. Attach schemaMetadata aspect (pass control to SchemaRegistry) - schema_metadata = self.schema_registry_client.get_schema_metadata( - topic, platform_urn - ) if schema_metadata is not None: dataset_snapshot.aspects.append(schema_metadata) @@ -356,9 +381,19 @@ def _extract_record( browse_path = BrowsePathsClass([browse_path_str]) dataset_snapshot.aspects.append(browse_path) - custom_props = self.build_custom_properties( - topic, topic_detail, extra_topic_config - ) + # build custom properties for topic, schema properties may be added as needed + custom_props: Dict[str, str] = {} + if not is_subject: + custom_props = self.build_custom_properties( + topic, topic_detail, extra_topic_config + ) + schema_name: Optional[ + str + ] = self.schema_registry_client._get_subject_for_topic( + topic, is_key_schema=False + ) + if schema_name is not None: + custom_props["Schema Name"] = schema_name # 4. Set dataset's description, tags, ownership, etc, if topic schema type is avro description: Optional[str] = None @@ -414,7 +449,7 @@ def _extract_record( ) dataset_properties = DatasetPropertiesClass( - name=topic, customProperties=custom_props, description=description + name=dataset_name, customProperties=custom_props, description=description ) dataset_snapshot.aspects.append(dataset_properties) @@ -431,12 +466,13 @@ def _extract_record( # 6. Emit the datasetSnapshot MCE mce = MetadataChangeEvent(proposedSnapshot=dataset_snapshot) - yield MetadataWorkUnit(id=f"kafka-{topic}", mce=mce) + yield MetadataWorkUnit(id=f"kafka-{kafka_entity}", mce=mce) - # 7. Add the subtype aspect marking this as a "topic" + # 7. Add the subtype aspect marking this as a "topic" or "schema" + typeName = DatasetSubTypes.SCHEMA if is_subject else DatasetSubTypes.TOPIC yield MetadataChangeProposalWrapper( entityUrn=dataset_urn, - aspect=SubTypesClass(typeNames=[DatasetSubTypes.TOPIC]), + aspect=SubTypesClass(typeNames=[typeName]), ).as_workunit() domain_urn: Optional[str] = None diff --git a/metadata-ingestion/src/datahub/ingestion/source/kafka_schema_registry_base.py b/metadata-ingestion/src/datahub/ingestion/source/kafka_schema_registry_base.py index 34ff76f44d1dd3..59f174a9a50458 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/kafka_schema_registry_base.py +++ b/metadata-ingestion/src/datahub/ingestion/source/kafka_schema_registry_base.py @@ -1,5 +1,5 @@ from abc import ABC, abstractmethod -from typing import Optional +from typing import List, Optional from datahub.metadata.com.linkedin.pegasus2avro.schema import SchemaMetadata @@ -7,6 +7,16 @@ class KafkaSchemaRegistryBase(ABC): @abstractmethod def get_schema_metadata( - self, topic: str, platform_urn: str + self, topic: str, platform_urn: str, is_subject: bool ) -> Optional[SchemaMetadata]: pass + + @abstractmethod + def get_subjects(self) -> List[str]: + pass + + @abstractmethod + def _get_subject_for_topic( + self, dataset_subtype: str, is_key_schema: bool + ) -> Optional[str]: + pass diff --git a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_assertion.py b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_assertion.py index 8abb656e30e73e..a28a81cc5b955d 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_assertion.py +++ b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_assertion.py @@ -59,7 +59,6 @@ def __init__( def get_assertion_workunits( self, discovered_datasets: List[str] ) -> Iterable[MetadataWorkUnit]: - self.connection = self.create_connection() if self.connection is None: return @@ -80,7 +79,6 @@ def get_assertion_workunits( yield self._gen_platform_instance_wu(mcp.entityUrn) def _gen_platform_instance_wu(self, urn: str) -> MetadataWorkUnit: - # Construct a MetadataChangeProposalWrapper object for assertion platform return MetadataChangeProposalWrapper( entityUrn=urn, diff --git a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_query.py b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_query.py index 205490a6d29c6f..8187fce78e5e47 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_query.py +++ b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_query.py @@ -1019,7 +1019,6 @@ def table_upstreams_only( @staticmethod def dmf_assertion_results(start_time_millis: int, end_time_millis: int) -> str: - pattern = r"datahub\\_\\_%" escape_pattern = r"\\" return f""" diff --git a/metadata-ingestion/tests/integration/kafka/kafka_mces_golden.json b/metadata-ingestion/tests/integration/kafka/kafka_mces_golden.json index 7dd328168e84c0..7df790b74e829c 100644 --- a/metadata-ingestion/tests/integration/kafka/kafka_mces_golden.json +++ b/metadata-ingestion/tests/integration/kafka/kafka_mces_golden.json @@ -270,7 +270,8 @@ "retention.ms": "604800000", "cleanup.policy": "delete", "max.message.bytes": "1048588", - "unclean.leader.election.enable": "false" + "unclean.leader.election.enable": "false", + "Schema Name": "key_value_topic-value" }, "name": "key_value_topic", "description": "Value schema for kafka topic", @@ -472,7 +473,8 @@ "retention.ms": "604800000", "cleanup.policy": "delete", "max.message.bytes": "1048588", - "unclean.leader.election.enable": "false" + "unclean.leader.election.enable": "false", + "Schema Name": "value_topic-value" }, "name": "value_topic", "description": "Value schema for kafka topic", @@ -522,6 +524,889 @@ "lastRunId": "no-run-id-provided" } }, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_topic-key,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "key_topic-key", + "platform": "urn:li:dataPlatform:kafka", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "5e468f7aa532c2f2ed9686ff3ec943ec", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.KafkaSchema": { + "documentSchema": "{\"type\":\"record\",\"name\":\"UserKey\",\"namespace\":\"io.codebrews.createuserrequest\",\"doc\":\"Key schema for kafka topic\",\"fields\":[{\"name\":\"id\",\"type\":\"long\"},{\"name\":\"namespace\",\"type\":\"string\"}]}", + "documentSchemaType": "AVRO", + "keySchema": "{\"type\":\"record\",\"name\":\"UserKey\",\"namespace\":\"io.codebrews.createuserrequest\",\"doc\":\"Key schema for kafka topic\",\"fields\":[{\"name\":\"id\",\"type\":\"long\"},{\"name\":\"namespace\",\"type\":\"string\"}]}", + "keySchemaType": "AVRO" + } + }, + "fields": [ + { + "fieldPath": "[version=2.0].[key=True].[type=UserKey].[type=long].id", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "id", + "recursive": false, + "isPartOfKey": true + }, + { + "fieldPath": "[version=2.0].[key=True].[type=UserKey].[type=string].namespace", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "namespace", + "recursive": false, + "isPartOfKey": true + }, + { + "fieldPath": "[version=2.0].[type=UserKey].[type=long].id", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "id", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "[version=2.0].[type=UserKey].[type=string].namespace", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "namespace", + "recursive": false, + "isPartOfKey": false + } + ] + } + }, + { + "com.linkedin.pegasus2avro.common.BrowsePaths": { + "paths": [ + "/prod/kafka" + ] + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": {}, + "name": "key_topic-key", + "description": "Key schema for kafka topic", + "tags": [] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_topic-key,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Schema" + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_topic-key,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_value_topic-key,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "key_value_topic-key", + "platform": "urn:li:dataPlatform:kafka", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "5e468f7aa532c2f2ed9686ff3ec943ec", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.KafkaSchema": { + "documentSchema": "{\"type\":\"record\",\"name\":\"UserKey\",\"namespace\":\"io.codebrews.createuserrequest\",\"doc\":\"Key schema for kafka topic\",\"fields\":[{\"name\":\"id\",\"type\":\"long\"},{\"name\":\"namespace\",\"type\":\"string\"}]}", + "documentSchemaType": "AVRO", + "keySchema": "{\"type\":\"record\",\"name\":\"UserKey\",\"namespace\":\"io.codebrews.createuserrequest\",\"doc\":\"Key schema for kafka topic\",\"fields\":[{\"name\":\"id\",\"type\":\"long\"},{\"name\":\"namespace\",\"type\":\"string\"}]}", + "keySchemaType": "AVRO" + } + }, + "fields": [ + { + "fieldPath": "[version=2.0].[key=True].[type=UserKey].[type=long].id", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "id", + "recursive": false, + "isPartOfKey": true + }, + { + "fieldPath": "[version=2.0].[key=True].[type=UserKey].[type=string].namespace", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "namespace", + "recursive": false, + "isPartOfKey": true + }, + { + "fieldPath": "[version=2.0].[type=UserKey].[type=long].id", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "id", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "[version=2.0].[type=UserKey].[type=string].namespace", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "namespace", + "recursive": false, + "isPartOfKey": false + } + ] + } + }, + { + "com.linkedin.pegasus2avro.common.BrowsePaths": { + "paths": [ + "/prod/kafka" + ] + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": {}, + "name": "key_value_topic-key", + "description": "Key schema for kafka topic", + "tags": [] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_value_topic-key,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Schema" + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_value_topic-key,PROD)", + "changeType": "UPSERT", + "aspectName": "domains", + "aspect": { + "json": { + "domains": [ + "urn:li:domain:sales" + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_value_topic-key,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_value_topic-value,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "key_value_topic-value", + "platform": "urn:li:dataPlatform:kafka", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "c9b692583e304b9cb703ffa748a9f37d", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.KafkaSchema": { + "documentSchema": "{\"type\":\"record\",\"name\":\"CreateUserRequest\",\"namespace\":\"io.codebrews.createuserrequest\",\"doc\":\"Value schema for kafka topic\",\"fields\":[{\"name\":\"email\",\"type\":\"string\",\"tags\":[\"Email\"]},{\"name\":\"firstName\",\"type\":\"string\",\"tags\":[\"Name\"]},{\"name\":\"lastName\",\"type\":\"string\",\"tags\":[\"Name\"]}],\"tags\":[\"PII\"]}", + "documentSchemaType": "AVRO", + "keySchema": "{\"type\":\"record\",\"name\":\"CreateUserRequest\",\"namespace\":\"io.codebrews.createuserrequest\",\"doc\":\"Value schema for kafka topic\",\"fields\":[{\"name\":\"email\",\"type\":\"string\",\"tags\":[\"Email\"]},{\"name\":\"firstName\",\"type\":\"string\",\"tags\":[\"Name\"]},{\"name\":\"lastName\",\"type\":\"string\",\"tags\":[\"Name\"]}],\"tags\":[\"PII\"]}", + "keySchemaType": "AVRO" + } + }, + "fields": [ + { + "fieldPath": "[version=2.0].[key=True].[type=CreateUserRequest].[type=string].email", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "email", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Email" + } + ] + }, + "isPartOfKey": true, + "jsonProps": "{\"tags\": [\"Email\"]}" + }, + { + "fieldPath": "[version=2.0].[key=True].[type=CreateUserRequest].[type=string].firstName", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "firstName", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Name" + } + ] + }, + "isPartOfKey": true, + "jsonProps": "{\"tags\": [\"Name\"]}" + }, + { + "fieldPath": "[version=2.0].[key=True].[type=CreateUserRequest].[type=string].lastName", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "lastName", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Name" + } + ] + }, + "isPartOfKey": true, + "jsonProps": "{\"tags\": [\"Name\"]}" + }, + { + "fieldPath": "[version=2.0].[type=CreateUserRequest].[type=string].email", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "email", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Email" + } + ] + }, + "isPartOfKey": false, + "jsonProps": "{\"tags\": [\"Email\"]}" + }, + { + "fieldPath": "[version=2.0].[type=CreateUserRequest].[type=string].firstName", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "firstName", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Name" + } + ] + }, + "isPartOfKey": false, + "jsonProps": "{\"tags\": [\"Name\"]}" + }, + { + "fieldPath": "[version=2.0].[type=CreateUserRequest].[type=string].lastName", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "lastName", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Name" + } + ] + }, + "isPartOfKey": false, + "jsonProps": "{\"tags\": [\"Name\"]}" + } + ] + } + }, + { + "com.linkedin.pegasus2avro.common.BrowsePaths": { + "paths": [ + "/prod/kafka" + ] + } + }, + { + "com.linkedin.pegasus2avro.common.GlobalTags": { + "tags": [ + { + "tag": "urn:li:tag:PII" + } + ] + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": {}, + "name": "key_value_topic-value", + "description": "Value schema for kafka topic", + "tags": [] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_value_topic-value,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Schema" + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_value_topic-value,PROD)", + "changeType": "UPSERT", + "aspectName": "domains", + "aspect": { + "json": { + "domains": [ + "urn:li:domain:sales" + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,key_value_topic-value,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:kafka,value_topic-key,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "value_topic-key", + "platform": "urn:li:dataPlatform:kafka", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "c088cd2eb2de57e32c00b32d4871ec72", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.KafkaSchema": { + "documentSchema": "\"string\"", + "documentSchemaType": "AVRO", + "keySchema": "\"string\"", + "keySchemaType": "AVRO" + } + }, + "fields": [ + { + "fieldPath": "[version=2.0].[key=True].[type=string]", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "isPartOfKey": true + }, + { + "fieldPath": "[version=2.0].[type=string]", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "string", + "recursive": false, + "isPartOfKey": false + } + ] + } + }, + { + "com.linkedin.pegasus2avro.common.BrowsePaths": { + "paths": [ + "/prod/kafka" + ] + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": {}, + "name": "value_topic-key", + "tags": [] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,value_topic-key,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Schema" + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,value_topic-key,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:kafka,value_topic-value,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "value_topic-value", + "platform": "urn:li:dataPlatform:kafka", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "c9b692583e304b9cb703ffa748a9f37d", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.KafkaSchema": { + "documentSchema": "{\"type\":\"record\",\"name\":\"CreateUserRequest\",\"namespace\":\"io.codebrews.createuserrequest\",\"doc\":\"Value schema for kafka topic\",\"fields\":[{\"name\":\"email\",\"type\":\"string\",\"tags\":[\"Email\"]},{\"name\":\"firstName\",\"type\":\"string\",\"tags\":[\"Name\"]},{\"name\":\"lastName\",\"type\":\"string\",\"tags\":[\"Name\"]}],\"tags\":[\"PII\"]}", + "documentSchemaType": "AVRO", + "keySchema": "{\"type\":\"record\",\"name\":\"CreateUserRequest\",\"namespace\":\"io.codebrews.createuserrequest\",\"doc\":\"Value schema for kafka topic\",\"fields\":[{\"name\":\"email\",\"type\":\"string\",\"tags\":[\"Email\"]},{\"name\":\"firstName\",\"type\":\"string\",\"tags\":[\"Name\"]},{\"name\":\"lastName\",\"type\":\"string\",\"tags\":[\"Name\"]}],\"tags\":[\"PII\"]}", + "keySchemaType": "AVRO" + } + }, + "fields": [ + { + "fieldPath": "[version=2.0].[key=True].[type=CreateUserRequest].[type=string].email", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "email", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Email" + } + ] + }, + "isPartOfKey": true, + "jsonProps": "{\"tags\": [\"Email\"]}" + }, + { + "fieldPath": "[version=2.0].[key=True].[type=CreateUserRequest].[type=string].firstName", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "firstName", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Name" + } + ] + }, + "isPartOfKey": true, + "jsonProps": "{\"tags\": [\"Name\"]}" + }, + { + "fieldPath": "[version=2.0].[key=True].[type=CreateUserRequest].[type=string].lastName", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "lastName", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Name" + } + ] + }, + "isPartOfKey": true, + "jsonProps": "{\"tags\": [\"Name\"]}" + }, + { + "fieldPath": "[version=2.0].[type=CreateUserRequest].[type=string].email", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "email", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Email" + } + ] + }, + "isPartOfKey": false, + "jsonProps": "{\"tags\": [\"Email\"]}" + }, + { + "fieldPath": "[version=2.0].[type=CreateUserRequest].[type=string].firstName", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "firstName", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Name" + } + ] + }, + "isPartOfKey": false, + "jsonProps": "{\"tags\": [\"Name\"]}" + }, + { + "fieldPath": "[version=2.0].[type=CreateUserRequest].[type=string].lastName", + "nullable": false, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "lastName", + "recursive": false, + "globalTags": { + "tags": [ + { + "tag": "urn:li:tag:Name" + } + ] + }, + "isPartOfKey": false, + "jsonProps": "{\"tags\": [\"Name\"]}" + } + ] + } + }, + { + "com.linkedin.pegasus2avro.common.BrowsePaths": { + "paths": [ + "/prod/kafka" + ] + } + }, + { + "com.linkedin.pegasus2avro.common.GlobalTags": { + "tags": [ + { + "tag": "urn:li:tag:PII" + } + ] + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": {}, + "name": "value_topic-value", + "description": "Value schema for kafka topic", + "tags": [] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,value_topic-value,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Schema" + ] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:kafka,value_topic-value,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [] + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "kafka-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "tag", "entityUrn": "urn:li:tag:Email", diff --git a/metadata-ingestion/tests/unit/test_kafka_source.py b/metadata-ingestion/tests/unit/test_kafka_source.py index 5ad9ac45534aa1..b4e37d288a3041 100644 --- a/metadata-ingestion/tests/unit/test_kafka_source.py +++ b/metadata-ingestion/tests/unit/test_kafka_source.py @@ -334,7 +334,9 @@ def mock_get_latest_version(subject_name: str) -> Optional[RegisteredSchema]: mock_kafka_consumer.assert_called_once() mock_kafka_instance.list_topics.assert_called_once() - assert len(workunits) == 8 + # Along with with 4 topics(3 with schema and 1 schemaless) which constitutes to 8 workunits, + # there will be 6 schemas (1 key and 1 value schema for 3 topics) which constitutes to 12 workunits + assert len(workunits) == 20 i: int = -1 for wu in workunits: assert isinstance(wu, MetadataWorkUnit) @@ -343,6 +345,8 @@ def mock_get_latest_version(subject_name: str) -> Optional[RegisteredSchema]: mce: MetadataChangeEvent = wu.metadata i += 1 + # Only topic (named schema_less_topic) does not have schema metadata but other workunits (that are created + # for schema) will have corresponding SchemaMetadata aspect if i < len(topic_subject_schema_map.keys()): # First 3 workunits (topics) must have schemaMetadata aspect assert isinstance(mce.proposedSnapshot.aspects[1], SchemaMetadataClass) @@ -380,11 +384,18 @@ def mock_get_latest_version(subject_name: str) -> Optional[RegisteredSchema]: ) # Make sure we have 2 fields, one from the key schema & one from the value schema. assert len(schemaMetadataAspect.fields) == 2 - else: + elif i == len(topic_subject_schema_map.keys()): # Last topic('schema_less_topic') has no schema defined in the registry. # The schemaMetadata aspect should not be present for this. for aspect in mce.proposedSnapshot.aspects: assert not isinstance(aspect, SchemaMetadataClass) + else: + # Last 2 workunits (schemas) must have schemaMetadata aspect + assert isinstance(mce.proposedSnapshot.aspects[1], SchemaMetadataClass) + schemaMetadataAspectObj: SchemaMetadataClass = mce.proposedSnapshot.aspects[ + 1 + ] + assert isinstance(schemaMetadataAspectObj.platformSchema, KafkaSchemaClass) @pytest.mark.parametrize( @@ -465,7 +476,7 @@ def mock_get_latest_version(subject_name: str) -> Optional[RegisteredSchema]: workunits = list(kafka_source.get_workunits()) - assert len(workunits) == 2 + assert len(workunits) == 6 if ignore_warnings_on_schema_type: assert not kafka_source.report.warnings else: @@ -643,8 +654,10 @@ def mock_get_latest_version(subject_name: str) -> Optional[RegisteredSchema]: }, ctx, ) + # Along with with 1 topics(and 5 meta mapping) it constitutes to 6 workunits, + # there will be 2 schemas which constitutes to 4 workunits (1 mce and 1 mcp each) workunits = [w for w in kafka_source.get_workunits()] - assert len(workunits) == 6 + assert len(workunits) == 10 mce = workunits[0].metadata assert isinstance(mce, MetadataChangeEvent) @@ -677,11 +690,49 @@ def mock_get_latest_version(subject_name: str) -> Optional[RegisteredSchema]: "urn:li:glossaryTerm:double_meta_property", ] ) - assert isinstance(workunits[2].metadata, MetadataChangeProposalWrapper) + assert isinstance(workunits[1].metadata, MetadataChangeProposalWrapper) + mce = workunits[2].metadata + assert isinstance(mce, MetadataChangeEvent) assert isinstance(workunits[3].metadata, MetadataChangeProposalWrapper) - assert isinstance(workunits[4].metadata, MetadataChangeProposalWrapper) + + mce = workunits[4].metadata + assert isinstance(mce, MetadataChangeEvent) + ownership_aspect = [ + asp for asp in mce.proposedSnapshot.aspects if isinstance(asp, OwnershipClass) + ][0] + assert ownership_aspect == make_ownership_aspect_from_urn_list( + [ + make_owner_urn("charles", OwnerType.USER), + make_owner_urn("jdoe.last@gmail.com", OwnerType.USER), + ], + "SERVICE", + ) + + tags_aspect = [ + asp for asp in mce.proposedSnapshot.aspects if isinstance(asp, GlobalTagsClass) + ][0] + assert tags_aspect == make_global_tag_aspect_with_tag_list( + ["has_pii_test", "int_meta_property"] + ) + + terms_aspect = [ + asp + for asp in mce.proposedSnapshot.aspects + if isinstance(asp, GlossaryTermsClass) + ][0] + assert terms_aspect == make_glossary_terms_aspect_from_urn_list( + [ + "urn:li:glossaryTerm:Finance_test", + "urn:li:glossaryTerm:double_meta_property", + ] + ) + assert isinstance(workunits[5].metadata, MetadataChangeProposalWrapper) - assert workunits[2].metadata.aspectName == "glossaryTermKey" - assert workunits[3].metadata.aspectName == "glossaryTermKey" - assert workunits[4].metadata.aspectName == "tagKey" - assert workunits[5].metadata.aspectName == "tagKey" + assert isinstance(workunits[6].metadata, MetadataChangeProposalWrapper) + assert isinstance(workunits[7].metadata, MetadataChangeProposalWrapper) + assert isinstance(workunits[8].metadata, MetadataChangeProposalWrapper) + assert isinstance(workunits[9].metadata, MetadataChangeProposalWrapper) + assert workunits[6].metadata.aspectName == "glossaryTermKey" + assert workunits[7].metadata.aspectName == "glossaryTermKey" + assert workunits[8].metadata.aspectName == "tagKey" + assert workunits[9].metadata.aspectName == "tagKey" From e123c25d82f10872c09de85753951212381c1f64 Mon Sep 17 00:00:00 2001 From: Davi Arnaut Date: Tue, 11 Jun 2024 10:19:04 -0700 Subject: [PATCH 3/8] fix(azure_ad): print request URL on error (#10677) --- .../src/datahub/ingestion/source/identity/azure_ad.py | 1 + 1 file changed, 1 insertion(+) diff --git a/metadata-ingestion/src/datahub/ingestion/source/identity/azure_ad.py b/metadata-ingestion/src/datahub/ingestion/source/identity/azure_ad.py index 2bd05ca11e234e..20b313474d174b 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/identity/azure_ad.py +++ b/metadata-ingestion/src/datahub/ingestion/source/identity/azure_ad.py @@ -505,6 +505,7 @@ def _get_azure_ad_data(self, kind: str) -> Iterable[List]: yield json_data["value"] else: error_str = ( + f"Request URL: {url}. " f"Response status code: {str(response.status_code)}. " f"Response content: {str(response.content)}" ) From 46dbb109403717c8302953162b7c0f6b342111ba Mon Sep 17 00:00:00 2001 From: Andrew Sikowitz Date: Tue, 11 Jun 2024 11:44:13 -0700 Subject: [PATCH 4/8] docs(ingest): Rename csv / s3 / file source and sink (#10675) --- .gitignore | 3 ++ docs-website/build.gradle | 4 +++ docs-website/filterTagIndexes.json | 4 +-- docs/cli.md | 16 +++++----- docs/how/updating-datahub.md | 2 +- docs/troubleshooting/quickstart.md | 2 +- metadata-ingestion/cli-ingestion.md | 2 +- .../csv-enricher_recipe.yml | 0 .../metadata-file_recipe.yml} | 0 metadata-ingestion/docs/sources/s3/README.md | 30 ++++++++++--------- .../docs/sources/s3/s3_recipe.yml | 11 +++++-- metadata-ingestion/setup.py | 4 ++- .../sink_docs/{file.md => metadata-file.md} | 5 ++-- metadata-ingestion/sink_overview.md | 2 +- .../datahub/ingestion/source/csv_enricher.py | 9 ++++-- .../src/datahub/ingestion/source/file.py | 21 +++++++++---- .../src/datahub/ingestion/source/s3/source.py | 2 +- metadata-integration/java/as-a-library.md | 4 +-- 18 files changed, 75 insertions(+), 46 deletions(-) rename metadata-ingestion/docs/sources/{csv => csv-enricher}/csv-enricher_recipe.yml (100%) rename metadata-ingestion/docs/sources/{file/file_recipe.yml => metadata-file/metadata-file_recipe.yml} (100%) rename metadata-ingestion/sink_docs/{file.md => metadata-file.md} (93%) diff --git a/.gitignore b/.gitignore index 1fcca8751131fc..43c627f9ed244f 100644 --- a/.gitignore +++ b/.gitignore @@ -126,3 +126,6 @@ metadata-service/war/bin/ metadata-utils/bin/ test-models/bin/ +datahub-executor/ +datahub-integrations-service/ +metadata-ingestion-modules/acryl-cloud diff --git a/docs-website/build.gradle b/docs-website/build.gradle index f3bedd2516319a..798047a562ffd2 100644 --- a/docs-website/build.gradle +++ b/docs-website/build.gradle @@ -148,8 +148,12 @@ clean { delete 'tmp' delete 'build' delete 'just' + delete 'sphinx/venv' + delete 'sphinx/_build' + delete 'versioned_docs' delete fileTree(dir: 'genDocs', exclude: '.gitignore') delete fileTree(dir: 'docs', exclude: '.gitignore') + delete fileTree(dir: 'genStatic', exclude: '.gitignore') delete 'graphql/combined.graphql' yarnClear } diff --git a/docs-website/filterTagIndexes.json b/docs-website/filterTagIndexes.json index 0c1f541cf53d34..8caff3497a2002 100644 --- a/docs-website/filterTagIndexes.json +++ b/docs-website/filterTagIndexes.json @@ -67,7 +67,7 @@ } }, { - "Path": "docs/generated/ingestion/sources/csv", + "Path": "docs/generated/ingestion/sources/csv-enricher", "imgPath": "img/datahub-logo-color-mark.svg", "Title": "CSV", "Description": "An ingestion source for enriching metadata provided in CSV format provided by DataHub", @@ -177,7 +177,7 @@ } }, { - "Path": "docs/generated/ingestion/sources/file", + "Path": "docs/generated/ingestion/sources/metadata-file", "imgPath": "img/datahub-logo-color-mark.svg", "Title": "File", "Description": "An ingestion source for single files provided by DataHub", diff --git a/docs/cli.md b/docs/cli.md index 411cb2d1ab77f0..32036a11dfb3d0 100644 --- a/docs/cli.md +++ b/docs/cli.md @@ -655,8 +655,8 @@ We use a plugin architecture so that you can install only the dependencies you a Please see our [Integrations page](https://datahubproject.io/integrations) if you want to filter on the features offered by each source. | Plugin Name | Install Command | Provides | -| ---------------------------------------------------------------------------------------------- | ---------------------------------------------------------- | --------------------------------------- | -| [file](./generated/ingestion/sources/file.md) | _included by default_ | File source and sink | +|------------------------------------------------------------------------------------------------| ---------------------------------------------------------- | --------------------------------------- | +| [metadata-file](./generated/ingestion/sources/metadata-file.md) | _included by default_ | File source and sink | | [athena](./generated/ingestion/sources/athena.md) | `pip install 'acryl-datahub[athena]'` | AWS Athena source | | [bigquery](./generated/ingestion/sources/bigquery.md) | `pip install 'acryl-datahub[bigquery]'` | BigQuery source | | [datahub-lineage-file](./generated/ingestion/sources/file-based-lineage.md) | _no additional dependencies_ | Lineage File source | @@ -696,12 +696,12 @@ Please see our [Integrations page](https://datahubproject.io/integrations) if yo ### Sinks -| Plugin Name | Install Command | Provides | -| ----------------------------------------------------------- | -------------------------------------------- | -------------------------- | -| [file](../metadata-ingestion/sink_docs/file.md) | _included by default_ | File source and sink | -| [console](../metadata-ingestion/sink_docs/console.md) | _included by default_ | Console sink | -| [datahub-rest](../metadata-ingestion/sink_docs/datahub.md) | `pip install 'acryl-datahub[datahub-rest]'` | DataHub sink over REST API | -| [datahub-kafka](../metadata-ingestion/sink_docs/datahub.md) | `pip install 'acryl-datahub[datahub-kafka]'` | DataHub sink over Kafka | +| Plugin Name | Install Command | Provides | +|-------------------------------------------------------------------| -------------------------------------------- | -------------------------- | +| [metadata-file](../metadata-ingestion/sink_docs/metadata-file.md) | _included by default_ | File source and sink | +| [console](../metadata-ingestion/sink_docs/console.md) | _included by default_ | Console sink | +| [datahub-rest](../metadata-ingestion/sink_docs/datahub.md) | `pip install 'acryl-datahub[datahub-rest]'` | DataHub sink over REST API | +| [datahub-kafka](../metadata-ingestion/sink_docs/datahub.md) | `pip install 'acryl-datahub[datahub-kafka]'` | DataHub sink over Kafka | These plugins can be mixed and matched as desired. For example: diff --git a/docs/how/updating-datahub.md b/docs/how/updating-datahub.md index c29c20e7d48a3c..bd559d1a24d273 100644 --- a/docs/how/updating-datahub.md +++ b/docs/how/updating-datahub.md @@ -484,7 +484,7 @@ Helm with `--atomic`: In general, it is recommended to not use the `--atomic` se ### Breaking Changes -- The `should_overwrite` flag in `csv-enricher` has been replaced with `write_semantics` to match the format used for other sources. See the [documentation](https://datahubproject.io/docs/generated/ingestion/sources/csv/) for more details +- The `should_overwrite` flag in `csv-enricher` has been replaced with `write_semantics` to match the format used for other sources. See the [documentation](https://datahubproject.io/docs/generated/ingestion/sources/csv-enricher/) for more details - Closing an authorization hole in creating tags adding a Platform Privilege called `Create Tags` for creating tags. This is assigned to `datahub` root user, along with default All Users policy. Notice: You may need to add this privilege (or `Manage Tags`) to existing users that need the ability to create tags on the platform. - #5329 Below profiling config parameters are now supported in `BigQuery`: diff --git a/docs/troubleshooting/quickstart.md b/docs/troubleshooting/quickstart.md index 9da5aa443069e1..cafc1e30c50520 100644 --- a/docs/troubleshooting/quickstart.md +++ b/docs/troubleshooting/quickstart.md @@ -246,7 +246,7 @@ ALTER TABLE metadata_aspect_v2 CONVERT TO CHARACTER SET utf8mb4 COLLATE utf8mb4_ ## I've modified the default user.props file to include a custom username and password, but I don't see the new user(s) inside the Users & Groups tab. Why not? Currently, `user.props` is a file used by the JAAS PropertyFileLoginModule solely for the purpose of **Authentication**. The file is not used as an source from which to -ingest additional metadata about the user. For that, you'll need to ingest some custom information about your new user using the Rest.li APIs or the [File-based ingestion source](../generated/ingestion/sources/file.md). +ingest additional metadata about the user. For that, you'll need to ingest some custom information about your new user using the Rest.li APIs or the [Metadata File ingestion source](../generated/ingestion/sources/metadata-file.md). For an example of a file that ingests user information, check out [single_mce.json](https://github.com/datahub-project/datahub/blob/master/metadata-ingestion/examples/mce_files/single_mce.json), which ingests a single user object into DataHub. Notice that the "urn" field provided will need to align with the custom username you've provided in user.props file. For example, if your user.props file contains: diff --git a/metadata-ingestion/cli-ingestion.md b/metadata-ingestion/cli-ingestion.md index b15dd2a5019959..64a2b370dd93cf 100644 --- a/metadata-ingestion/cli-ingestion.md +++ b/metadata-ingestion/cli-ingestion.md @@ -58,7 +58,7 @@ Please refer the following pages for advanced guids on CLI ingestion. - [Reference for `datahub ingest` command](../docs/cli.md#ingest) - [UI Ingestion Guide](../docs/ui-ingestion.md) -:::Tip Compatibility +:::tip Compatibility DataHub server uses a 3 digit versioning scheme, while the CLI uses a 4 digit scheme. For example, if you're using DataHub server version 0.10.0, you should use CLI version 0.10.0.x, where x is a patch version. We do this because we do CLI releases at a much higher frequency than server releases, usually every few days vs twice a month. diff --git a/metadata-ingestion/docs/sources/csv/csv-enricher_recipe.yml b/metadata-ingestion/docs/sources/csv-enricher/csv-enricher_recipe.yml similarity index 100% rename from metadata-ingestion/docs/sources/csv/csv-enricher_recipe.yml rename to metadata-ingestion/docs/sources/csv-enricher/csv-enricher_recipe.yml diff --git a/metadata-ingestion/docs/sources/file/file_recipe.yml b/metadata-ingestion/docs/sources/metadata-file/metadata-file_recipe.yml similarity index 100% rename from metadata-ingestion/docs/sources/file/file_recipe.yml rename to metadata-ingestion/docs/sources/metadata-file/metadata-file_recipe.yml diff --git a/metadata-ingestion/docs/sources/s3/README.md b/metadata-ingestion/docs/sources/s3/README.md index 7944f78280a428..b0d354a9b3c2ac 100644 --- a/metadata-ingestion/docs/sources/s3/README.md +++ b/metadata-ingestion/docs/sources/s3/README.md @@ -1,19 +1,11 @@ -This connector ingests S3 datasets into DataHub. It allows mapping an individual file or a folder of files to a dataset in DataHub. +This connector ingests AWS S3 datasets into DataHub. It allows mapping an individual file or a folder of files to a dataset in DataHub. To specify the group of files that form a dataset, use `path_specs` configuration in ingestion recipe. Refer section [Path Specs](https://datahubproject.io/docs/generated/ingestion/sources/s3/#path-specs) for more details. -### Concept Mapping - -This ingestion source maps the following Source System Concepts to DataHub Concepts: - -| Source Concept | DataHub Concept | Notes | -| ---------------------------------------- |--------------------------------------------------------------------------------------------| ------------------- | -| `"s3"` | [Data Platform](https://datahubproject.io/docs/generated/metamodel/entities/dataplatform/) | | -| s3 object / Folder containing s3 objects | [Dataset](https://datahubproject.io/docs/generated/metamodel/entities/dataset/) | | -| s3 bucket | [Container](https://datahubproject.io/docs/generated/metamodel/entities/container/) | Subtype `S3 bucket` | -| s3 folder | [Container](https://datahubproject.io/docs/generated/metamodel/entities/container/) | Subtype `Folder` | +:::tip +This connector can also be used to ingest local files. +Just replace `s3://` in your path_specs with an absolute path to files on the machine running ingestion. +::: -This connector supports both local files as well as those stored on AWS S3 (which must be identified using the prefix `s3://`). -[a] ### Supported file types Supported file types are as follows: @@ -30,6 +22,16 @@ Schemas for schemaless formats (CSV, TSV, JSONL, JSON) are inferred. For CSV, TS JSON file schemas are inferred on the basis of the entire file (given the difficulty in extracting only the first few objects of the file), which may impact performance. We are working on using iterator-based JSON parsers to avoid reading in the entire JSON object. +### Concept Mapping + +This ingestion source maps the following Source System Concepts to DataHub Concepts: + +| Source Concept | DataHub Concept | Notes | +| ---------------------------------------- |--------------------------------------------------------------------------------------------| ------------------- | +| `"s3"` | [Data Platform](https://datahubproject.io/docs/generated/metamodel/entities/dataplatform/) | | +| s3 object / Folder containing s3 objects | [Dataset](https://datahubproject.io/docs/generated/metamodel/entities/dataset/) | | +| s3 bucket | [Container](https://datahubproject.io/docs/generated/metamodel/entities/container/) | Subtype `S3 bucket` | +| s3 folder | [Container](https://datahubproject.io/docs/generated/metamodel/entities/container/) | Subtype `Folder` | ### Profiling @@ -42,4 +44,4 @@ This plugin extracts: - histograms or frequencies of unique values Note that because the profiling is run with PySpark, we require Spark 3.0.3 with Hadoop 3.2 to be installed (see [compatibility](#compatibility) for more details). If profiling, make sure that permissions for **s3a://** access are set because Spark and Hadoop use the s3a:// protocol to interface with AWS (schema inference outside of profiling requires s3:// access). -Enabling profiling will slow down ingestion runs. \ No newline at end of file +Enabling profiling will slow down ingestion runs. diff --git a/metadata-ingestion/docs/sources/s3/s3_recipe.yml b/metadata-ingestion/docs/sources/s3/s3_recipe.yml index 693b9528373ab2..301e811b769260 100644 --- a/metadata-ingestion/docs/sources/s3/s3_recipe.yml +++ b/metadata-ingestion/docs/sources/s3/s3_recipe.yml @@ -1,9 +1,9 @@ +# Ingest data from S3 source: type: s3 config: path_specs: - - - include: "s3://covid19-lake/covid_knowledge_graph/csv/nodes/*.*" + - include: "s3://covid19-lake/covid_knowledge_graph/csv/nodes/*.*" aws_config: aws_access_key_id: ***** @@ -13,4 +13,9 @@ source: profiling: enabled: false -# sink configs +# Ingest data from local filesystem +source: + type: s3 + config: + path_specs: + - include: "/absolute/path/*.csv" diff --git a/metadata-ingestion/setup.py b/metadata-ingestion/setup.py index 4702c9d540ec0e..ade1e1a6ee5ba4 100644 --- a/metadata-ingestion/setup.py +++ b/metadata-ingestion/setup.py @@ -259,7 +259,9 @@ delta_lake = { *s3_base, - "deltalake>=0.6.3, != 0.6.4", + # Version 0.18.0 broken on ARM Macs: https://github.com/delta-io/delta-rs/issues/2577 + "deltalake>=0.6.3, != 0.6.4, < 0.18.0; platform_system == 'Darwin' and platform_machine == 'arm64'", + "deltalake>=0.6.3, != 0.6.4; platform_system != 'Darwin' or platform_machine != 'arm64'", } powerbi_report_server = {"requests", "requests_ntlm"} diff --git a/metadata-ingestion/sink_docs/file.md b/metadata-ingestion/sink_docs/metadata-file.md similarity index 93% rename from metadata-ingestion/sink_docs/file.md rename to metadata-ingestion/sink_docs/metadata-file.md index 2991afacbd93d2..7cac8d55422438 100644 --- a/metadata-ingestion/sink_docs/file.md +++ b/metadata-ingestion/sink_docs/metadata-file.md @@ -1,4 +1,4 @@ -# File +# Metadata File For context on getting started with ingestion, check out our [metadata ingestion guide](../README.md). @@ -10,7 +10,7 @@ Works with `acryl-datahub` out of the box. Outputs metadata to a file. This can be used to decouple metadata sourcing from the process of pushing it into DataHub, and is particularly useful for debugging purposes. -Note that the [file source](../../docs/generated/ingestion/sources/file.md) can read files generated by this sink. +Note that the [file source](../../docs/generated/ingestion/sources/metadata-file.md) can read files generated by this sink. ## Quickstart recipe @@ -35,4 +35,3 @@ Note that a `.` is used to denote nested fields in the YAML recipe. | Field | Required | Default | Description | | -------- | -------- | ------- | ------------------------- | | filename | ✅ | | Path to file to write to. | - diff --git a/metadata-ingestion/sink_overview.md b/metadata-ingestion/sink_overview.md index c71ba1f97932cf..95f18a0a6cb944 100644 --- a/metadata-ingestion/sink_overview.md +++ b/metadata-ingestion/sink_overview.md @@ -25,7 +25,7 @@ When configuring ingestion for DataHub, you're likely to be sending the metadata For debugging purposes or troubleshooting, the following sinks can be useful: -- [File](sink_docs/file.md) +- [Metadata File](sink_docs/metadata-file.md) - [Console](sink_docs/console.md) ## Default Sink diff --git a/metadata-ingestion/src/datahub/ingestion/source/csv_enricher.py b/metadata-ingestion/src/datahub/ingestion/source/csv_enricher.py index d998c37d32ed2a..feee89ba579837 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/csv_enricher.py +++ b/metadata-ingestion/src/datahub/ingestion/source/csv_enricher.py @@ -93,13 +93,18 @@ class CSVEnricherReport(SourceReport): num_domain_workunits_produced: int = 0 -@platform_name("CSV") +@platform_name("CSV Enricher") @config_class(CSVEnricherConfig) @support_status(SupportStatus.INCUBATING) class CSVEnricherSource(Source): """ + :::tip Looking to ingest a CSV data file into DataHub, as an asset? + Use the [Local File](./s3.md) ingestion source. + The CSV enricher is used for enriching entities already ingested into DataHub. + ::: + This plugin is used to bulk upload metadata to Datahub. - It will apply glossary terms, tags, decription, owners and domain at the entity level. It can also be used to apply tags, + It will apply glossary terms, tags, description, owners and domain at the entity level. It can also be used to apply tags, glossary terms, and documentation at the column level. These values are read from a CSV file. You have the option to either overwrite or append existing values. diff --git a/metadata-ingestion/src/datahub/ingestion/source/file.py b/metadata-ingestion/src/datahub/ingestion/source/file.py index 49cc314426eb55..3e8c88b725de50 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/file.py +++ b/metadata-ingestion/src/datahub/ingestion/source/file.py @@ -56,11 +56,17 @@ class FileSourceConfig(ConfigModel): message="filename is deprecated. Use path instead.", ) path: str = Field( - description="File path to folder or file to ingest, or URL to a remote file. If pointed to a folder, all files with extension {file_extension} (default json) within that folder will be processed." + description=( + "File path to folder or file to ingest, or URL to a remote file. " + "If pointed to a folder, all files with extension {file_extension} (default json) within that folder will be processed." + ) ) file_extension: str = Field( ".json", - description="When providing a folder to use to read files, set this field to control file extensions that you want the source to process. * is a special value that means process every file regardless of extension", + description=( + "When providing a folder to use to read files, set this field to control file extensions that you want the source to process. " + "* is a special value that means process every file regardless of extension" + ), ) read_mode: FileReadMode = FileReadMode.AUTO aspect: Optional[str] = Field( @@ -69,7 +75,10 @@ class FileSourceConfig(ConfigModel): ) count_all_before_starting: bool = Field( default=True, - description="When enabled, counts total number of records in the file before starting. Used for accurate estimation of completion time. Turn it off if startup time is too high.", + description=( + "When enabled, counts total number of records in the file before starting. " + "Used for accurate estimation of completion time. Turn it off if startup time is too high." + ), ) _minsize_for_streaming_mode_in_bytes: int = ( @@ -163,12 +172,14 @@ def compute_stats(self) -> None: self.percentage_completion = f"{percentage_completion:.2f}%" -@platform_name("File") +@platform_name("Metadata File") @config_class(FileSourceConfig) @support_status(SupportStatus.CERTIFIED) class GenericFileSource(TestableSource): """ - This plugin pulls metadata from a previously generated file. The [file sink](../../../../metadata-ingestion/sink_docs/file.md) can produce such files, and a number of samples are included in the [examples/mce_files](../../../../metadata-ingestion/examples/mce_files) directory. + This plugin pulls metadata from a previously generated file. + The [metadata file sink](../../../../metadata-ingestion/sink_docs/metadata-file.md) can produce such files, and a number of + samples are included in the [examples/mce_files](../../../../metadata-ingestion/examples/mce_files) directory. """ def __init__(self, ctx: PipelineContext, config: FileSourceConfig): diff --git a/metadata-ingestion/src/datahub/ingestion/source/s3/source.py b/metadata-ingestion/src/datahub/ingestion/source/s3/source.py index 8bc075f720cc55..c35f500df1b8c7 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/s3/source.py +++ b/metadata-ingestion/src/datahub/ingestion/source/s3/source.py @@ -217,7 +217,7 @@ class TableData: number_of_files: int -@platform_name("S3 Data Lake", id="s3") +@platform_name("S3 / Local Files", id="s3") @config_class(DataLakeSourceConfig) @support_status(SupportStatus.INCUBATING) @capability(SourceCapability.DATA_PROFILING, "Optionally enabled via configuration") diff --git a/metadata-integration/java/as-a-library.md b/metadata-integration/java/as-a-library.md index 59ee6595132fa1..e38cfef080f5f1 100644 --- a/metadata-integration/java/as-a-library.md +++ b/metadata-integration/java/as-a-library.md @@ -169,7 +169,7 @@ If you're interested in looking at the Kafka emitter code, it is available [here ## File Emitter -The File emitter writes metadata change proposal events (MCPs) into a JSON file that can be later handed off to the Python [File source](docs/generated/ingestion/sources/file.md) for ingestion. This works analogous to the [File sink](../../metadata-ingestion/sink_docs/file.md) in Python. This mechanism can be used when the system producing metadata events doesn't have direct connection to DataHub's REST server or Kafka brokers. The generated JSON file can be transferred later and then ingested into DataHub using the [File source](docs/generated/ingestion/sources/file.md). +The File emitter writes metadata change proposal events (MCPs) into a JSON file that can be later handed off to the Python [Metadata File source](docs/generated/ingestion/sources/metadata-file.md) for ingestion. This works analogous to the [Metadata File sink](../../metadata-ingestion/sink_docs/metadata-file.md) in Python. This mechanism can be used when the system producing metadata events doesn't have direct connection to DataHub's REST server or Kafka brokers. The generated JSON file can be transferred later and then ingested into DataHub using the [Metadata File source](docs/generated/ingestion/sources/metadata-file.md). ### Usage @@ -223,5 +223,3 @@ The File emitter only supports writing to the local filesystem currently. If you Emitter API-s are also supported for: - [Python](../../metadata-ingestion/as-a-library.md) - - From b9e71a61b1d4c7c49fd224fea60ca289a2e7dc42 Mon Sep 17 00:00:00 2001 From: skrydal Date: Tue, 11 Jun 2024 20:50:46 +0200 Subject: [PATCH 5/8] feat(ingest/glue): database parameters extraction (#10665) --- .../src/datahub/ingestion/source/aws/glue.py | 8 ++++++++ .../tests/unit/glue/glue_delta_mces_golden.json | 3 ++- .../unit/glue/glue_malformed_delta_mces_golden.json | 3 ++- .../tests/unit/glue/glue_mces_golden.json | 9 +++++++-- .../glue/glue_mces_platform_instance_golden.json | 12 ++++++++++-- .../tests/unit/test_glue_source_stubs.py | 2 ++ 6 files changed, 31 insertions(+), 6 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/aws/glue.py b/metadata-ingestion/src/datahub/ingestion/source/aws/glue.py index a6393aa9d0ced6..d65d17f223361d 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/aws/glue.py +++ b/metadata-ingestion/src/datahub/ingestion/source/aws/glue.py @@ -1,3 +1,4 @@ +import datetime import json import logging from collections import defaultdict @@ -895,6 +896,12 @@ def gen_database_containers( ) -> Iterable[MetadataWorkUnit]: domain_urn = self._gen_domain_urn(database["Name"]) database_container_key = self.gen_database_key(database["Name"]) + parameters = database.get("Parameters", {}) + if database.get("LocationUri") is not None: + parameters["LocationUri"] = database["LocationUri"] + if database.get("CreateTime") is not None: + create_time: datetime.datetime = database["CreateTime"] + parameters["CreateTime"] = create_time.strftime("%B %-d, %Y at %H:%M:%S") yield from gen_containers( container_key=database_container_key, name=database["Name"], @@ -904,6 +911,7 @@ def gen_database_containers( qualified_name=self.get_glue_arn( account_id=database["CatalogId"], database=database["Name"] ), + extra_properties=parameters, ) def add_table_to_database_container( diff --git a/metadata-ingestion/tests/unit/glue/glue_delta_mces_golden.json b/metadata-ingestion/tests/unit/glue/glue_delta_mces_golden.json index f3d4812b791243..992ec338dea300 100644 --- a/metadata-ingestion/tests/unit/glue/glue_delta_mces_golden.json +++ b/metadata-ingestion/tests/unit/glue/glue_delta_mces_golden.json @@ -10,7 +10,8 @@ "platform": "glue", "instance": "delta_platform_instance", "env": "PROD", - "database": "delta-database" + "database": "delta-database", + "CreateTime": "June 9, 2021 at 14:14:19" }, "name": "delta-database", "qualifiedName": "arn:aws:glue:us-west-2:123412341234:database/delta-database" diff --git a/metadata-ingestion/tests/unit/glue/glue_malformed_delta_mces_golden.json b/metadata-ingestion/tests/unit/glue/glue_malformed_delta_mces_golden.json index 015daaa27162f2..b8e3445eea5094 100644 --- a/metadata-ingestion/tests/unit/glue/glue_malformed_delta_mces_golden.json +++ b/metadata-ingestion/tests/unit/glue/glue_malformed_delta_mces_golden.json @@ -10,7 +10,8 @@ "platform": "glue", "instance": "delta_platform_instance", "env": "PROD", - "database": "delta-database" + "database": "delta-database", + "CreateTime": "June 9, 2021 at 14:14:19" }, "name": "delta-database", "qualifiedName": "arn:aws:glue:us-west-2:123412341234:database/delta-database" diff --git a/metadata-ingestion/tests/unit/glue/glue_mces_golden.json b/metadata-ingestion/tests/unit/glue/glue_mces_golden.json index 7460e21e887ebe..f180185f67ead8 100644 --- a/metadata-ingestion/tests/unit/glue/glue_mces_golden.json +++ b/metadata-ingestion/tests/unit/glue/glue_mces_golden.json @@ -9,7 +9,11 @@ "customProperties": { "platform": "glue", "env": "PROD", - "database": "flights-database" + "database": "flights-database", + "param1": "value1", + "param2": "value2", + "LocationUri": "s3://test-bucket/test-prefix", + "CreateTime": "June 9, 2021 at 14:14:19" }, "name": "flights-database", "qualifiedName": "arn:aws:glue:us-west-2:123412341234:database/flights-database" @@ -288,7 +292,8 @@ "customProperties": { "platform": "glue", "env": "PROD", - "database": "test-database" + "database": "test-database", + "CreateTime": "June 1, 2021 at 14:55:02" }, "name": "test-database", "qualifiedName": "arn:aws:glue:us-west-2:123412341234:database/test-database" diff --git a/metadata-ingestion/tests/unit/glue/glue_mces_platform_instance_golden.json b/metadata-ingestion/tests/unit/glue/glue_mces_platform_instance_golden.json index f0d506cd59f2ea..4b64ee1bf08d41 100644 --- a/metadata-ingestion/tests/unit/glue/glue_mces_platform_instance_golden.json +++ b/metadata-ingestion/tests/unit/glue/glue_mces_platform_instance_golden.json @@ -10,7 +10,11 @@ "platform": "glue", "instance": "some_instance_name", "env": "PROD", - "database": "flights-database" + "database": "flights-database", + "param1": "value1", + "param2": "value2", + "LocationUri": "s3://test-bucket/test-prefix", + "CreateTime": "June 9, 2021 at 14:14:19" }, "name": "flights-database", "qualifiedName": "arn:aws:glue:us-west-2:123412341234:database/flights-database" @@ -235,6 +239,7 @@ "type": "DATAOWNER" } ], + "ownerTypes": {}, "lastModified": { "time": 0, "actor": "urn:li:corpuser:unknown" @@ -292,7 +297,8 @@ "platform": "glue", "instance": "some_instance_name", "env": "PROD", - "database": "test-database" + "database": "test-database", + "CreateTime": "June 1, 2021 at 14:55:02" }, "name": "test-database", "qualifiedName": "arn:aws:glue:us-west-2:123412341234:database/test-database" @@ -474,6 +480,7 @@ "type": "DATAOWNER" } ], + "ownerTypes": {}, "lastModified": { "time": 0, "actor": "urn:li:corpuser:unknown" @@ -660,6 +667,7 @@ "type": "DATAOWNER" } ], + "ownerTypes": {}, "lastModified": { "time": 0, "actor": "urn:li:corpuser:unknown" diff --git a/metadata-ingestion/tests/unit/test_glue_source_stubs.py b/metadata-ingestion/tests/unit/test_glue_source_stubs.py index c971001f97072d..80d16b93907f5b 100644 --- a/metadata-ingestion/tests/unit/test_glue_source_stubs.py +++ b/metadata-ingestion/tests/unit/test_glue_source_stubs.py @@ -61,6 +61,8 @@ } ], "CatalogId": "123412341234", + "LocationUri": "s3://test-bucket/test-prefix", + "Parameters": {"param1": "value1", "param2": "value2"}, }, { "Name": "test-database", From 52ac3143a417be3ccfa53ee4b7b1c1d09a9cfe93 Mon Sep 17 00:00:00 2001 From: Davi Arnaut Date: Tue, 11 Jun 2024 13:41:36 -0700 Subject: [PATCH 6/8] fix(azure_ad): fix infinite loop on request error (#10679) --- .../datahub/ingestion/source/identity/azure_ad.py | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/identity/azure_ad.py b/metadata-ingestion/src/datahub/ingestion/source/identity/azure_ad.py index 20b313474d174b..885b6514779cc4 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/identity/azure_ad.py +++ b/metadata-ingestion/src/datahub/ingestion/source/identity/azure_ad.py @@ -9,6 +9,7 @@ import click import requests from pydantic.fields import Field +from requests.adapters import HTTPAdapter, Retry from datahub.configuration.common import AllowDenyPattern from datahub.configuration.source_common import DatasetSourceConfigMixin @@ -268,6 +269,14 @@ def __init__(self, config: AzureADConfig, ctx: PipelineContext): self.report = AzureADSourceReport( filtered_tracking=self.config.filtered_tracking ) + session = requests.Session() + retries = Retry( + total=5, backoff_factor=1, status_forcelist=[429, 500, 502, 503, 504] + ) + adapter = HTTPAdapter(max_retries=retries) + session.mount("http://", adapter) + session.mount("https://", adapter) + self.session = session self.token_data = { "grant_type": "client_credentials", "client_id": self.config.client_id, @@ -494,7 +503,7 @@ def _get_azure_ad_data(self, kind: str) -> Iterable[List]: while True: if not url: break - response = requests.get(url, headers=headers) + response = self.session.get(url, headers=headers) if response.status_code == 200: json_data = json.loads(response.text) try: @@ -512,7 +521,7 @@ def _get_azure_ad_data(self, kind: str) -> Iterable[List]: logger.debug(f"URL = {url}") logger.error(error_str) self.report.report_failure("_get_azure_ad_data_", error_str) - continue + raise Exception(f"Unable to get {url}, error {response.status_code}") def _map_identity_to_urn(self, func, id_to_extract, mapping_identifier, id_type): result, error_str = None, None From 05aee03f3f7f232872c462f30b78f82b8fe4ab85 Mon Sep 17 00:00:00 2001 From: Shubham Jagtap <132359390+shubhamjagtap639@users.noreply.github.com> Date: Wed, 12 Jun 2024 08:49:57 +0530 Subject: [PATCH 7/8] perf(ingestion/fivetran): Connector performance optimization (#10556) --- .../datahub/ingestion/api/source_helpers.py | 4 +- .../ingestion/source/fivetran/config.py | 22 +++++ .../ingestion/source/fivetran/data_classes.py | 4 +- .../ingestion/source/fivetran/fivetran.py | 7 +- .../source/fivetran/fivetran_log_api.py | 96 +++++++++++-------- .../source/fivetran/fivetran_query.py | 12 +-- .../state/stale_entity_removal_handler.py | 14 +++ .../integration/fivetran/test_fivetran.py | 6 +- .../state/golden_test_checkpoint_state.json | 2 +- ...n_test_checkpoint_state_after_deleted.json | 2 +- .../state/golden_test_stateful_ingestion.json | 22 +++++ ...test_stateful_ingestion_after_deleted.json | 22 +++++ .../state/test_stateful_ingestion.py | 42 +++++++- 13 files changed, 198 insertions(+), 57 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/api/source_helpers.py b/metadata-ingestion/src/datahub/ingestion/api/source_helpers.py index 7226258515155f..8cc2cc565db85c 100644 --- a/metadata-ingestion/src/datahub/ingestion/api/source_helpers.py +++ b/metadata-ingestion/src/datahub/ingestion/api/source_helpers.py @@ -102,7 +102,6 @@ def auto_status_aspect( """ all_urns: Set[str] = set() status_urns: Set[str] = set() - skip_urns: Set[str] = set() for wu in stream: urn = wu.get_urn() all_urns.add(urn) @@ -127,14 +126,13 @@ def auto_status_aspect( yield wu - for urn in sorted(all_urns - status_urns - skip_urns): + for urn in sorted(all_urns - status_urns): entity_type = guess_entity_type(urn) if not entity_supports_aspect(entity_type, StatusClass): # If any entity does not support aspect 'status' then skip that entity from adding status aspect. # Example like dataProcessInstance doesn't suppport status aspect. # If not skipped gives error: java.lang.RuntimeException: Unknown aspect status for entity dataProcessInstance continue - yield MetadataChangeProposalWrapper( entityUrn=urn, aspect=StatusClass(removed=False), diff --git a/metadata-ingestion/src/datahub/ingestion/source/fivetran/config.py b/metadata-ingestion/src/datahub/ingestion/source/fivetran/config.py index a689e9ee642aef..f55d9f89ad97f1 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/fivetran/config.py +++ b/metadata-ingestion/src/datahub/ingestion/source/fivetran/config.py @@ -9,6 +9,7 @@ from datahub.configuration.common import AllowDenyPattern, ConfigModel from datahub.configuration.source_common import DEFAULT_ENV, DatasetSourceConfigMixin from datahub.configuration.validate_field_rename import pydantic_renamed_field +from datahub.ingestion.api.report import Report from datahub.ingestion.source.bigquery_v2.bigquery_config import ( BigQueryConnectionConfig, ) @@ -20,6 +21,7 @@ StatefulIngestionConfigBase, ) from datahub.ingestion.source_config.sql.snowflake import BaseSnowflakeConfig +from datahub.utilities.perf_timer import PerfTimer logger = logging.getLogger(__name__) @@ -110,10 +112,26 @@ def validate_destination_platfrom_and_config(cls, values: Dict) -> Dict: return values +@dataclass +class MetadataExtractionPerfReport(Report): + connectors_metadata_extraction_sec: PerfTimer = dataclass_field( + default_factory=PerfTimer + ) + connectors_lineage_extraction_sec: PerfTimer = dataclass_field( + default_factory=PerfTimer + ) + connectors_jobs_extraction_sec: PerfTimer = dataclass_field( + default_factory=PerfTimer + ) + + @dataclass class FivetranSourceReport(StaleEntityRemovalSourceReport): connectors_scanned: int = 0 filtered_connectors: List[str] = dataclass_field(default_factory=list) + metadata_extraction_perf: MetadataExtractionPerfReport = dataclass_field( + default_factory=MetadataExtractionPerfReport + ) def report_connectors_scanned(self, count: int = 1) -> None: self.connectors_scanned += count @@ -163,3 +181,7 @@ class FivetranSourceConfig(StatefulIngestionConfigBase, DatasetSourceConfigMixin default={}, description="A mapping of destination dataset to platform instance. Use destination id as key.", ) + history_sync_lookback_period: int = pydantic.Field( + 7, + description="The number of days to look back when extracting connectors' sync history.", + ) diff --git a/metadata-ingestion/src/datahub/ingestion/source/fivetran/data_classes.py b/metadata-ingestion/src/datahub/ingestion/source/fivetran/data_classes.py index 4ae71b990e5cde..18de2b01edd3b7 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/fivetran/data_classes.py +++ b/metadata-ingestion/src/datahub/ingestion/source/fivetran/data_classes.py @@ -1,5 +1,5 @@ from dataclasses import dataclass -from typing import List, Optional +from typing import List @dataclass @@ -23,7 +23,7 @@ class Connector: paused: bool sync_frequency: int destination_id: str - user_email: Optional[str] + user_id: str table_lineage: List[TableLineage] jobs: List["Job"] diff --git a/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran.py b/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran.py index c8ae779b602b8a..56a80a2fd963e7 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran.py +++ b/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran.py @@ -173,11 +173,12 @@ def _generate_datajob_from_connector(self, connector: Connector) -> DataJob: env=self.config.env, platform_instance=self.config.platform_instance, ) + owner_email = self.audit_log.get_user_email(connector.user_id) datajob = DataJob( id=connector.connector_id, flow_urn=dataflow_urn, name=connector.connector_name, - owners={connector.user_email} if connector.user_email else set(), + owners={owner_email} if owner_email else set(), ) job_property_bag: Dict[str, str] = {} @@ -281,7 +282,9 @@ def get_workunits_internal(self) -> Iterable[MetadataWorkUnit]: """ logger.info("Fivetran plugin execution is started") connectors = self.audit_log.get_allowed_connectors_list( - self.config.connector_patterns, self.report + self.config.connector_patterns, + self.report, + self.config.history_sync_lookback_period, ) for connector in connectors: logger.info(f"Processing connector id: {connector.connector_id}") diff --git a/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran_log_api.py b/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran_log_api.py index a9eb59f9297992..51ef45c500c350 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran_log_api.py +++ b/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran_log_api.py @@ -1,3 +1,4 @@ +import functools import json import logging from typing import Any, Dict, List, Optional, Tuple @@ -151,9 +152,14 @@ def _get_table_lineage( return table_lineage_list - def _get_all_connector_sync_logs(self) -> Dict[str, Dict]: + def _get_all_connector_sync_logs(self, syncs_interval: int) -> Dict[str, Dict]: sync_logs = {} - for row in self._query(self.fivetran_log_query.get_sync_logs_query()): + for row in self._query( + self.fivetran_log_query.get_sync_logs_query().format( + db_clause=self.fivetran_log_query.db_clause, + syncs_interval=syncs_interval, + ) + ): if row[Constant.CONNECTOR_ID] not in sync_logs: sync_logs[row[Constant.CONNECTOR_ID]] = { row[Constant.SYNC_ID]: { @@ -208,50 +214,62 @@ def _get_jobs_list( ) return jobs - def _get_user_email(self, user_id: Optional[str]) -> Optional[str]: + @functools.lru_cache() + def _get_users(self) -> Dict[str, str]: + users = self._query(self.fivetran_log_query.get_users_query()) + if not users: + return {} + return {user[Constant.USER_ID]: user[Constant.EMAIL] for user in users} + + def get_user_email(self, user_id: str) -> Optional[str]: if not user_id: return None - user_details = self._query( - self.fivetran_log_query.get_user_query(user_id=user_id) - ) + return self._get_users().get(user_id) - if not user_details: - return None + def _fill_connectors_table_lineage(self, connectors: List[Connector]) -> None: + table_lineage_metadata = self._get_connectors_table_lineage_metadata() + column_lineage_metadata = self._get_column_lineage_metadata() + for connector in connectors: + connector.table_lineage = self._get_table_lineage( + column_lineage_metadata=column_lineage_metadata, + table_lineage_result=table_lineage_metadata.get(connector.connector_id), + ) - return f"{user_details[0][Constant.EMAIL]}" + def _fill_connectors_jobs( + self, connectors: List[Connector], syncs_interval: int + ) -> None: + sync_logs = self._get_all_connector_sync_logs(syncs_interval) + for connector in connectors: + connector.jobs = self._get_jobs_list(sync_logs.get(connector.connector_id)) def get_allowed_connectors_list( - self, connector_patterns: AllowDenyPattern, report: FivetranSourceReport + self, + connector_patterns: AllowDenyPattern, + report: FivetranSourceReport, + syncs_interval: int, ) -> List[Connector]: connectors: List[Connector] = [] - sync_logs = self._get_all_connector_sync_logs() - table_lineage_metadata = self._get_connectors_table_lineage_metadata() - column_lineage_metadata = self._get_column_lineage_metadata() - connector_list = self._query(self.fivetran_log_query.get_connectors_query()) - for connector in connector_list: - if not connector_patterns.allowed(connector[Constant.CONNECTOR_NAME]): - report.report_connectors_dropped(connector[Constant.CONNECTOR_NAME]) - continue - connectors.append( - Connector( - connector_id=connector[Constant.CONNECTOR_ID], - connector_name=connector[Constant.CONNECTOR_NAME], - connector_type=connector[Constant.CONNECTOR_TYPE_ID], - paused=connector[Constant.PAUSED], - sync_frequency=connector[Constant.SYNC_FREQUENCY], - destination_id=connector[Constant.DESTINATION_ID], - user_email=self._get_user_email( - connector[Constant.CONNECTING_USER_ID] - ), - table_lineage=self._get_table_lineage( - column_lineage_metadata=column_lineage_metadata, - table_lineage_result=table_lineage_metadata.get( - connector[Constant.CONNECTOR_ID] - ), - ), - jobs=self._get_jobs_list( - sync_logs.get(connector[Constant.CONNECTOR_ID]) - ), + with report.metadata_extraction_perf.connectors_metadata_extraction_sec: + connector_list = self._query(self.fivetran_log_query.get_connectors_query()) + for connector in connector_list: + if not connector_patterns.allowed(connector[Constant.CONNECTOR_NAME]): + report.report_connectors_dropped(connector[Constant.CONNECTOR_NAME]) + continue + connectors.append( + Connector( + connector_id=connector[Constant.CONNECTOR_ID], + connector_name=connector[Constant.CONNECTOR_NAME], + connector_type=connector[Constant.CONNECTOR_TYPE_ID], + paused=connector[Constant.PAUSED], + sync_frequency=connector[Constant.SYNC_FREQUENCY], + destination_id=connector[Constant.DESTINATION_ID], + user_id=connector[Constant.CONNECTING_USER_ID], + table_lineage=[], + jobs=[], + ) ) - ) + with report.metadata_extraction_perf.connectors_lineage_extraction_sec: + self._fill_connectors_table_lineage(connectors) + with report.metadata_extraction_perf.connectors_jobs_extraction_sec: + self._fill_connectors_jobs(connectors, syncs_interval) return connectors diff --git a/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran_query.py b/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran_query.py index 8f621bc3ffd06e..0c8ade26943490 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran_query.py +++ b/metadata-ingestion/src/datahub/ingestion/source/fivetran/fivetran_query.py @@ -21,24 +21,24 @@ def get_connectors_query(self) -> str: FROM {self.db_clause}connector WHERE _fivetran_deleted = FALSE""" - def get_user_query(self, user_id: str) -> str: + def get_users_query(self) -> str: return f""" SELECT id as user_id, given_name, family_name, email - FROM {self.db_clause}user - WHERE id = '{user_id}'""" + FROM {self.db_clause}user""" def get_sync_logs_query(self) -> str: - return f""" + return """ SELECT connector_id, sync_id, message_event, message_data, time_stamp - FROM {self.db_clause}log - WHERE message_event in ('sync_start', 'sync_end')""" + FROM {db_clause}log + WHERE message_event in ('sync_start', 'sync_end') + and time_stamp > CURRENT_TIMESTAMP - INTERVAL '{syncs_interval} days'""" def get_table_lineage_query(self) -> str: return f""" diff --git a/metadata-ingestion/src/datahub/ingestion/source/state/stale_entity_removal_handler.py b/metadata-ingestion/src/datahub/ingestion/source/state/stale_entity_removal_handler.py index 9154a555f23090..97c9dd9e245ddf 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/state/stale_entity_removal_handler.py +++ b/metadata-ingestion/src/datahub/ingestion/source/state/stale_entity_removal_handler.py @@ -6,6 +6,7 @@ import pydantic from datahub.emitter.mcp import MetadataChangeProposalWrapper +from datahub.emitter.mcp_builder import entity_supports_aspect from datahub.ingestion.api.common import PipelineContext from datahub.ingestion.api.ingestion_job_checkpointing_provider_base import JobId from datahub.ingestion.api.source_helpers import auto_stale_entity_removal @@ -23,6 +24,7 @@ ) from datahub.metadata.schema_classes import StatusClass from datahub.utilities.lossy_collections import LossyList +from datahub.utilities.urns.urn import guess_entity_type logger: logging.Logger = logging.getLogger(__name__) @@ -48,10 +50,14 @@ class StatefulStaleMetadataRemovalConfig(StatefulIngestionConfig): @dataclass class StaleEntityRemovalSourceReport(StatefulIngestionReport): soft_deleted_stale_entities: LossyList[str] = field(default_factory=LossyList) + last_state_non_deletable_entities: LossyList[str] = field(default_factory=LossyList) def report_stale_entity_soft_deleted(self, urn: str) -> None: self.soft_deleted_stale_entities.append(urn) + def report_last_state_non_deletable_entities(self, urn: str) -> None: + self.last_state_non_deletable_entities.append(urn) + class StaleEntityRemovalHandler( StatefulIngestionUsecaseHandlerBase["GenericCheckpointState"] @@ -272,11 +278,19 @@ def gen_removed_entity_workunits(self) -> Iterable[MetadataWorkUnit]: self.add_entity_to_state("", urn) return + report = self.source.get_report() + assert isinstance(report, StaleEntityRemovalSourceReport) + # Everything looks good, emit the soft-deletion workunits for urn in last_checkpoint_state.get_urns_not_in( type="*", other_checkpoint_state=cur_checkpoint_state ): + if not entity_supports_aspect(guess_entity_type(urn), StatusClass): + # If any entity does not support aspect 'status' then skip that entity urn + report.report_last_state_non_deletable_entities(urn) + continue if urn in self._urns_to_skip: + report.report_last_state_non_deletable_entities(urn) logger.debug( f"Not soft-deleting entity {urn} since it is in urns_to_skip" ) diff --git a/metadata-ingestion/tests/integration/fivetran/test_fivetran.py b/metadata-ingestion/tests/integration/fivetran/test_fivetran.py index de1e5543f4be69..642d4ca992ca03 100644 --- a/metadata-ingestion/tests/integration/fivetran/test_fivetran.py +++ b/metadata-ingestion/tests/integration/fivetran/test_fivetran.py @@ -89,7 +89,7 @@ def default_query_results( "destination_column_name": "name", }, ] - elif query == fivetran_log_query.get_user_query("reapply_phone"): + elif query == fivetran_log_query.get_users_query(): return [ { "user_id": "reapply_phone", @@ -98,7 +98,9 @@ def default_query_results( "email": "abc.xyz@email.com", } ] - elif query == fivetran_log_query.get_sync_logs_query(): + elif query == fivetran_log_query.get_sync_logs_query().format( + db_clause=fivetran_log_query.db_clause, syncs_interval=7 + ): return [ { "connector_id": "calendar_elected", diff --git a/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_checkpoint_state.json b/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_checkpoint_state.json index fcf73d9614f242..ce03804279097f 100644 --- a/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_checkpoint_state.json +++ b/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_checkpoint_state.json @@ -17,7 +17,7 @@ "state": { "formatVersion": "1.0", "serde": "utf-8", - "payload": "{\"urns\": [\"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset1,PROD)\", \"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset2,PROD)\", \"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset3,PROD)\"]}" + "payload": "{\"urns\": [\"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset1,PROD)\", \"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset2,PROD)\", \"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset3,PROD)\", \"urn:li:dataProcessInstance:478810e859f870a54f72c681f41af619\"]}" }, "runId": "dummy-test-stateful-ingestion" } diff --git a/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_checkpoint_state_after_deleted.json b/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_checkpoint_state_after_deleted.json index 5477af72a1939c..6a00e67a2ca216 100644 --- a/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_checkpoint_state_after_deleted.json +++ b/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_checkpoint_state_after_deleted.json @@ -17,7 +17,7 @@ "state": { "formatVersion": "1.0", "serde": "utf-8", - "payload": "{\"urns\": [\"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset1,PROD)\", \"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset2,PROD)\"]}" + "payload": "{\"urns\": [\"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset1,PROD)\", \"urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset2,PROD)\", \"urn:li:dataProcessInstance:7f26c3b4d2d82ace47f4b9dd0c9dea26\"]}" }, "runId": "dummy-test-stateful-ingestion" } diff --git a/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_stateful_ingestion.json b/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_stateful_ingestion.json index 4a77651c930667..c5d0df1aeb59b5 100644 --- a/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_stateful_ingestion.json +++ b/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_stateful_ingestion.json @@ -46,5 +46,27 @@ "runId": "dummy-test-stateful-ingestion", "lastRunId": "no-run-id-provided" } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:478810e859f870a54f72c681f41af619", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": {}, + "name": "job1", + "type": "BATCH_SCHEDULED", + "created": { + "time": 1586847600000, + "actor": "urn:li:corpuser:datahub" + } + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "dummy-test-stateful-ingestion", + "lastRunId": "no-run-id-provided" + } } ] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_stateful_ingestion_after_deleted.json b/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_stateful_ingestion_after_deleted.json index 9d6f755374462b..c1bdc8ffeee052 100644 --- a/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_stateful_ingestion_after_deleted.json +++ b/metadata-ingestion/tests/unit/stateful_ingestion/state/golden_test_stateful_ingestion_after_deleted.json @@ -31,6 +31,28 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:7f26c3b4d2d82ace47f4b9dd0c9dea26", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": {}, + "name": "job2", + "type": "BATCH_SCHEDULED", + "created": { + "time": 1586847600000, + "actor": "urn:li:corpuser:datahub" + } + } + }, + "systemMetadata": { + "lastObserved": 1586847600000, + "runId": "dummy-test-stateful-ingestion", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:postgres,dummy_dataset3,PROD)", diff --git a/metadata-ingestion/tests/unit/stateful_ingestion/state/test_stateful_ingestion.py b/metadata-ingestion/tests/unit/stateful_ingestion/state/test_stateful_ingestion.py index 50d9b86b3a0171..e3a2a6cccea794 100644 --- a/metadata-ingestion/tests/unit/stateful_ingestion/state/test_stateful_ingestion.py +++ b/metadata-ingestion/tests/unit/stateful_ingestion/state/test_stateful_ingestion.py @@ -1,3 +1,4 @@ +import time from dataclasses import dataclass, field as dataclass_field from typing import Any, Dict, Iterable, List, Optional, cast from unittest import mock @@ -7,6 +8,7 @@ from freezegun import freeze_time from pydantic import Field +from datahub.api.entities.dataprocess.dataprocess_instance import DataProcessInstance from datahub.configuration.common import AllowDenyPattern from datahub.configuration.source_common import DEFAULT_ENV, DatasetSourceConfigMixin from datahub.emitter.mcp import MetadataChangeProposalWrapper @@ -24,7 +26,10 @@ StatefulIngestionConfigBase, StatefulIngestionSourceBase, ) -from datahub.metadata.schema_classes import StatusClass +from datahub.metadata.com.linkedin.pegasus2avro.dataprocess import ( + DataProcessInstanceProperties, +) +from datahub.metadata.schema_classes import AuditStampClass, StatusClass from datahub.utilities.urns.dataset_urn import DatasetUrn from tests.test_helpers import mce_helpers from tests.test_helpers.state_helpers import ( @@ -62,6 +67,10 @@ class DummySourceConfig(StatefulIngestionConfigBase, DatasetSourceConfigMixin): default=False, description="Should this dummy source report a failure.", ) + dpi_id_to_ingest: Optional[str] = Field( + default=None, + description="Data process instance id to ingest.", + ) class DummySource(StatefulIngestionSourceBase): @@ -109,6 +118,24 @@ def get_workunits_internal(self) -> Iterable[MetadataWorkUnit]: aspect=StatusClass(removed=False), ).as_workunit() + if self.source_config.dpi_id_to_ingest: + dpi = DataProcessInstance( + id=self.source_config.dpi_id_to_ingest, + orchestrator="dummy", + ) + + yield MetadataChangeProposalWrapper( + entityUrn=str(dpi.urn), + aspect=DataProcessInstanceProperties( + name=dpi.id, + created=AuditStampClass( + time=int(time.time() * 1000), + actor="urn:li:corpuser:datahub", + ), + type=dpi.type, + ), + ).as_workunit() + if self.source_config.report_failure: self.reporter.report_failure("Dummy error", "Error") @@ -152,6 +179,7 @@ def test_stateful_ingestion(pytestconfig, tmp_path, mock_time): "stateful_ingestion": { "enabled": True, "remove_stale_metadata": True, + "fail_safe_threshold": 100, "state_provider": { "type": "file", "config": { @@ -159,6 +187,7 @@ def test_stateful_ingestion(pytestconfig, tmp_path, mock_time): }, }, }, + "dpi_id_to_ingest": "job1", }, }, "sink": { @@ -207,6 +236,7 @@ def test_stateful_ingestion(pytestconfig, tmp_path, mock_time): pipeline_run2_config["source"]["config"]["dataset_patterns"] = { "allow": ["dummy_dataset1", "dummy_dataset2"], } + pipeline_run2_config["source"]["config"]["dpi_id_to_ingest"] = "job2" pipeline_run2_config["sink"]["config"][ "filename" ] = f"{tmp_path}/{output_file_name_after_deleted}" @@ -253,6 +283,16 @@ def test_stateful_ingestion(pytestconfig, tmp_path, mock_time): ] assert sorted(deleted_dataset_urns) == sorted(difference_dataset_urns) + report = pipeline_run2.source.get_report() + assert isinstance(report, StaleEntityRemovalSourceReport) + # assert report last ingestion state non_deletable entity urns + non_deletable_urns: List[str] = [ + "urn:li:dataProcessInstance:478810e859f870a54f72c681f41af619", + ] + assert sorted(non_deletable_urns) == sorted( + report.last_state_non_deletable_entities + ) + @freeze_time(FROZEN_TIME) def test_stateful_ingestion_failure(pytestconfig, tmp_path, mock_time): From 1d4977cbb7a5427d49f80ac7c34e9e7b526fe356 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 12 Jun 2024 01:57:36 -0500 Subject: [PATCH 8/8] feat(ingest): make query formatting more robust (#10678) --- metadata-ingestion/src/datahub/sql_parsing/sqlglot_utils.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/metadata-ingestion/src/datahub/sql_parsing/sqlglot_utils.py b/metadata-ingestion/src/datahub/sql_parsing/sqlglot_utils.py index b494dde4013a46..f74a915c9fe821 100644 --- a/metadata-ingestion/src/datahub/sql_parsing/sqlglot_utils.py +++ b/metadata-ingestion/src/datahub/sql_parsing/sqlglot_utils.py @@ -315,8 +315,8 @@ def try_format_query( try: dialect = get_dialect(platform) - expression = parse_statement(expression, dialect=dialect) - return expression.sql(dialect=dialect, pretty=True) + parsed_expression = parse_statement(expression, dialect=dialect) + return parsed_expression.sql(dialect=dialect, pretty=True) except Exception as e: if raises: raise