From 0512da094bad2f3bcd2ddddc29e8abfec175dcfe Mon Sep 17 00:00:00 2001 From: Prasanna Rajaperumal Date: Fri, 16 Dec 2016 14:03:59 -0800 Subject: [PATCH] Import from Hoodie private repo: Part 1 --- hoodie-cli/hoodie-cli.sh | 4 + .../utils/textutils/0.3.3/textutils-0.3.3.jar | Bin 0 -> 18856 bytes hoodie-cli/pom.xml | 208 + .../java/com/uber/hoodie/cli/HoodieCLI.java | 54 + .../cli/HoodieHistoryFileNameProvider.java | 37 + .../uber/hoodie/cli/HoodiePrintHelper.java | 34 + .../com/uber/hoodie/cli/HoodiePrompt.java | 49 + .../uber/hoodie/cli/HoodieSplashScreen.java | 55 + .../main/java/com/uber/hoodie/cli/Main.java | 33 + .../hoodie/cli/commands/CommitsCommand.java | 244 ++ .../hoodie/cli/commands/DatasetsCommand.java | 42 + .../cli/commands/HoodieSyncCommand.java | 106 + .../hoodie/cli/commands/RecordsCommand.java | 78 + .../uber/hoodie/cli/commands/SparkMain.java | 94 + .../hoodie/cli/commands/StatsCommand.java | 136 + .../hoodie/cli/commands/UtilsCommand.java | 34 + .../com/uber/hoodie/cli/utils/CommitUtil.java | 38 + .../com/uber/hoodie/cli/utils/HiveUtil.java | 125 + .../hoodie/cli/utils/InputStreamConsumer.java | 56 + .../com/uber/hoodie/cli/utils/SparkUtil.java | 75 + .../META-INF/spring/spring-shell-plugin.xml | 26 + .../com/uber/hoodie/cli/DedupeSparkJob.scala | 179 + .../com/uber/hoodie/cli/SparkHelpers.scala | 141 + hoodie-common/pom.xml | 94 + .../hoodie/avro/HoodieAvroWriteSupport.java | 54 + .../hoodie/avro/MercifulJsonConverter.java | 166 + .../com/uber/hoodie/common/BloomFilter.java | 100 + .../hoodie/common/file/HoodieAppendLog.java | 3826 +++++++++++++++++ .../common/model/HoodieCommitMetadata.java | 190 + .../hoodie/common/model/HoodieCommits.java | 191 + .../uber/hoodie/common/model/HoodieFile.java | 57 + .../uber/hoodie/common/model/HoodieKey.java | 73 + .../hoodie/common/model/HoodieRecord.java | 153 + .../common/model/HoodieRecordLocation.java | 69 + .../common/model/HoodieRecordPayload.java | 57 + .../common/model/HoodieTableMetadata.java | 480 +++ .../hoodie/common/model/HoodieTableType.java | 35 + .../hoodie/common/model/HoodieWriteStat.java | 158 + .../com/uber/hoodie/common/util/FSUtils.java | 117 + .../hoodie/common/util/HoodieAvroUtils.java | 140 + .../uber/hoodie/common/util/NumericUtils.java | 26 + .../uber/hoodie/common/util/ParquetUtils.java | 138 + .../hoodie/common/util/ReflectionUtils.java | 43 + .../exception/DatasetNotFoundException.java | 32 + .../hoodie/exception/HoodieException.java | 55 + .../hoodie/exception/HoodieIOException.java | 37 + .../exception/HoodieIndexException.java | 34 + .../HoodieRecordMissingException.java | 35 + .../exception/InvalidDatasetException.java | 32 + .../uber/hoodie/common/TestBloomFilter.java | 45 + .../hoodie/common/model/HoodieTestUtils.java | 96 + .../common/model/TestHoodieCommits.java | 48 + .../common/model/TestHoodieTableMetadata.java | 242 ++ .../uber/hoodie/common/util/TestFSUtils.java | 70 + .../hoodie/common/util/TestNumericUtils.java | 35 + .../hoodie/common/util/TestParquetUtils.java | 92 + 56 files changed, 8868 insertions(+) create mode 100755 hoodie-cli/hoodie-cli.sh create mode 100644 hoodie-cli/lib/dnl/utils/textutils/0.3.3/textutils-0.3.3.jar create mode 100644 hoodie-cli/pom.xml create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieHistoryFileNameProvider.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RecordsCommand.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java create mode 100644 hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java create mode 100644 hoodie-cli/src/main/resources/META-INF/spring/spring-shell-plugin.xml create mode 100644 hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala create mode 100644 hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala create mode 100644 hoodie-common/pom.xml create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/BloomFilter.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/file/HoodieAppendLog.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommits.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFile.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableMetadata.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/NumericUtils.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/exception/DatasetNotFoundException.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieException.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieIOException.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieIndexException.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieRecordMissingException.java create mode 100644 hoodie-common/src/main/java/com/uber/hoodie/exception/InvalidDatasetException.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommits.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieTableMetadata.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/TestNumericUtils.java create mode 100644 hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java diff --git a/hoodie-cli/hoodie-cli.sh b/hoodie-cli/hoodie-cli.sh new file mode 100755 index 000000000..05adb9496 --- /dev/null +++ b/hoodie-cli/hoodie-cli.sh @@ -0,0 +1,4 @@ +#!/usr/bin/env bash +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +HOODIE_JAR=`ls $DIR/target/hoodie-cli-*-SNAPSHOT.jar` +java -cp /etc/hadoop/conf:/etc/spark/conf:$DIR/target/lib/*:$HOODIE_JAR org.springframework.shell.Bootstrap diff --git a/hoodie-cli/lib/dnl/utils/textutils/0.3.3/textutils-0.3.3.jar b/hoodie-cli/lib/dnl/utils/textutils/0.3.3/textutils-0.3.3.jar new file mode 100644 index 0000000000000000000000000000000000000000..42a9c6fa4ce334d113eba5af341d8a06b1a651e7 GIT binary patch literal 18856 zcmb7s1z254(l#F43GS}JA-KD1@Zj$5?i$?P;o`2r-QC@TdjjMqJ2RV^?9A@>AD#pE zK6Je;r%%<})m1Gg2?_=c^y4x$O%eRphu?oez5bOJR^q1>ml1g<_j@r&Ad}Z(fS|X* z=CA(_c)d~oTuho@MqET#QHf4k-^mT7a(j%sX_a>}u^ zPC*QqN^EQ>7_#8dt-!!PKKTT*H!b;nk21$w5;Eg_st2;ff_+Mi6W=jg z8j0~i8cA9K4E&flb$Ru2pt#Mw?L82nKN<%FG-2}BK7hS8&cMpz-_8He5qlBY_ zsfGQ2DTnZvat?+r4u2}V`AfBE|5?~USI@%mPa?mz6!V`%^zEJgi`b9R|JnS1HOs-y z@MlGkKx@CZ4dk`tkLyQ8bXJBAdWO1I_By(b4%U{s4yO7#dXATa@QdX5kNRXi zZ1P^T9?x*m!PqhgNLySMYyDyG%4Eg%N#?|4vO!c0zg5h zd>gkLUvs818o%^NU4|YPn7cZuptZ3G zcg+P~bnWjYPcPlcLSIaP#z{|g?JHT_5flctEL9;>QH`R!zU^~e$*A9_o^?1dBP~;? z(-b`uEZE<$cd1gU&;%6sW*)7KbR0?6s$JowDy|FeaY`>j$DJq*8-Qe_#lc`CB}v}J zp3jzMMy-Ifmz*9XiCPc~RV5UeO;u(_wpcBzEIIiP8=xl7%d} zTb+L5I&+C3gG6VKtvxm>X)qb*y*ovgWH6U>Ql3+#2ztNxfz4r~nWd9Yw5w49p%6ei z2gmX(D~@I7-vGXnWj4w8HWSxOS(R%OdLon57Q>H1 z+^Lh{4T}YP{_Yi~Xg&Xu_(DCNMV zn3`wX0w6F=SaLMn72?`K2n!H>9s{ z%?z6&)VJMcq**f&1MFN!0Fa{WLUaQX38@1Nj+13_I_h?o6SYU3Y;&<2$09D{Nf9Bj zBr9U*MEWu3H9Q^A#81^z65B2h601eK&mQ2=?`?jz9k-N1x2Z?Oo?BYNg) zBG|ITSK~tkAt)Tfq}yVM0+HhOF@WF|m0vz4g2>hbEs(gBgFgWK;Xk2Y5au`fBD~q$ zHWsS(PfWPM+Lp?m0O$@V*0uW(D%Q(}w+)&srz{RKk?b=ro8ufsKKoQSD7c439i}WE zf6Y45nM|C`%wzJ?C_C|vXzP!hS~xTmTA-pcs+-EcgKbr@<(P~xtjkZ7HLGN_ZGpFF zax%V|%CFFLED`V5;pUIXiXz&7pgbCI^7uZatm^?vcWiv*D#)dz6yKC-MruJAiYzh1}mDO;6+R*vzbMcToj&dDK!!}mMT zNo7x!wW#e&2r=O?JX4P1eV@f<_vMW~gvxPYsyY~=9!yoA@5;gj^&SD?+56+V?No7) zUErO99M}?qaTMGcB;tIrcYt|n0zWD5mH^WxO#Ivj(H)9NNEdx&XMQ83=+K759V)4N zgX?DQ;WYB}Ei;eYZ=;Hnx23bB(C0P|=Z-Wm5uOOk(NSMwjBuAvAz1*ha z3G@$x@|zMI;U{H%4-N#>^J)>_{a=)zf}xdxp`D@K?|M*WG(Gf&7E;vmOyT)qR0!70wZOSGL zidOO{?^a@<%+_`Ye?LPF#hihk%3v(r!*H!AvP`}FSkklVT>cc#Hun7aF`lotcE8=$ ze!3cLIk?7kfWo~&kuiViB7Y4-=axYITDr$--VV&^+tam(Y1C}lxcRpmc--pi24|&g z>PwpAGbrGKEOuX7L8o;r$^$Zh9!s^QpjE#E*=R29f^y!r^a_`-wdKB)>HOilv9BHa zDm>6FWO z{tvL-h10c#iVab&jNfm=F*4Gt8SzYeVJ9{(if_^zq>M<<4PJcszA)Th9_4bXqVaYM zD*MSz5!PB5tfn)#F4Z#5j@iJdtn)g$<)%XEe=xU@qG#ct&#H#bJl)@0Pv_g&-vvLO z$xLZxws`At^wd3EJXj;7=1iro=1AjTr$J{_zSGbavS^fXFRp-_QRVPOBrPnQqmUku zhl{J!kyxH^fvkUW7(_UEIeIOAj#*f^gZv>fGA&$76y6n=4w{3g(aL-6AqATk!J4gjhUr7xWVUw>83f{=pPwf4Q0fQ>|p64rNoIim(?q3UuS%!wO- z2JQ!?me88D9?{n&!evS|H6XO1skE!yzVIFA8hlIH7quvWr%<^nn2AT?);|0t z3LO8N6Cw1h344|xAl}annQ4gJ?QDpLM_NJ`5b-B<5Ep){#BX<12k{BNsa&1s6g2zw zhKm8cNQrE7`n1n0?(vE%#pnx8ARmw4tqYkf5Y0MTjUAW>a@WSCpQ4?!j6>XPliCd} zX@ALUG+?9p;%hyqJ<-0GX^r%8nOG8Xgy2&?QLLl8H9yJ&@RR|?Vp0FB%i$dh^MD|O zqqS5!IJeIYnlerK(_7W;9O2n-GGHu_nmwO6IB>p%_Yx)CveY5rgs*ld62z)=;e_|r zpHC)tAQR418bHSl6=_9K-`j9#J0oyz4-+-eeWb?>TNsj)hGAEz$IYdU&%c#nqyol` z25lgB1aZJeytBMK%mB9#r;-fGh@@tl@;KlSctF0t2`koR5^4$g!6Ne_=|s?@L*fk3t;pkVvlmC zQ~c1}u%hRfGQRfS7ca-cYnuC%jgz+fU;~NmDxI_u^V^yTVwaxEq&>6tfXiT=NB&Hx z(Ola=@j&Ial5N_Y?c}gV<(o2>;B3*w=7^97Y+Ac#NcWcjudvoB6c5;|RzI@Cv}&)9 z@1c)M%jt`2&Onct5DjGpo=bTz|IKP|yeyaI!;qiM4h8UE%Bf*0KK?`+#Wmh-y$w8W zYBCC1H1R|MzabWOd>V7{al#|(Y}L9||MA@WAvb6+ss`=h36-uH~Tt(+bf zo}{uan6B93s#~Nr2A&B@zzzi&as(Z^25hjy=~VENI<2wdC|17rOIxLeYr^WT#i^~E zp#dP-heh3Sj3-WQX$8RmSCG?aRBCAP`}85Hx~5B9rC+Z;m#)}@ftE2yCD9Hbn>%nh zo4G!c3FaQ3DOfT&j%;bP%Y?h@Ivbr>tyHB28H;l|koBU3*JzcVb2MbBsiHse9SZQw z;|q~YK%H3LVT za-X@*SH$au<#LA^;ZE$CE+0|&PXc1|Oc%Me35ny%;7IG%+$0KuQZEOPhv@5u_x7JJ zLCrf(_|DH1xwj~&02LPz4H$UNa?>rO#_Vj205tJjU0@9?I$fErhM+|RS)eGM+tuwGxlZpR)IzK-3!9hMDia~3)pVL(qBf&#Z$2{rQVg145D!WB zpjRLd3FETtCu?BUp$?IGCJxLZ$NCGLayc%6a=9Uy3OuxWvu z1MK%8L%i8!c=&|3t@z(KJazm9HdJG>i^ao|a$Hm?w;6?7MO(3NZbT5nYrNntocYH` zl=^6C6l5BMk19WuRTo|+CD#SJBT5JITOqjimz;~to)Fek z!5k5f(GTn3Y^n}C%!NK#Z;r)$hl&(qWv4FU@Q%R5H14wgfRDb7VY#&CJ7t)cXn5p3 z*7lt%ywMaLdys&W&_6Dx-QW57cD?hj^zGY|*HbDm5YYMS3Yg)a6o-_qy@P_av!bDm zuAQ!fwVj-`g{i*lZv-wuenp~(50Tq^MrCHX^j6y%4c|=odNYqGga9=+1Py6po^^IE zS*)i1*80h(1Jdu%AAeIcl*zO>o2bdR*>)n;e*ozBwLc4)fXK>0o_T2n0EC z>ahnTs#uwr7Uj{7b<%1CeO?`wv5|d|FyH zXnJR#H9u}yxAC1R1U>V#rp70fl^rlEZ8m}nDSvD#G<^Qbx86TJmbw9bJl?UL|Ij)M zca|!QF3-^4dTXNvcuy*C_Hj48)x$3Y_(E}%RN8NdwA0|hxR z;Y6{_2`9s&<{IRZ{}G5xEshJ)2$5b;zTpiKuT&Ik)~MryyCWZqN`O zmCdR zrqZ0_yNAS#?TrgiPL=glK5bZxbXlY#=0bP4TgI|qAq>`2_3O_3MOd?%f z0pAof2(;hg+j=hoDgtQbI`O_-xPK#CL}9V+dad(H!W2qXf8Tc<4! z(}ctduW(5x?$!@U=Q8*Qcm_2LpS8;3#uAFDRyase>SoD^r(}DInWevdzE4FG#>-t* z<-#d(VOOI=zfE<>H|I2CQ>eq15#L~qH#c^Z?kE-0V-I{|aiznZ7DbGZP>IE=zbFvJ z>R24PMTl3@a*DBy5lWb&9c>$h)#^y?Lk^PkQhN?^?a8ELvz*!IRMU4k?*5scb7rPj zMYP-cwkKBtL$YAy$v7k}?zAyXwc8+4SK~#i68qeAORp5Y9sz0pK^)BG2lD&%^LPdROX#J+*fe-ci=g07qR@(8Js>CdN0D8-Ty;&Cj{yy&#-a#Clj znT;aR1;XUfhA@h`*;G94;D^2}e>g*aSe8Q_$IOZKStpY{1A-AJ>2T68+ImUU?2`5> z<^G0d8`-$@8byq7Y#9@_+AS0pxx8aqskU3*6dtVAvw7VqmoJ>qr)&(x@H?pf9zMoG z+uSYhU*9plxyV@GS7o~Ss!acph5fA-jr^n`3Br#$@Lm5THSKZDhfZmuRxx~hodbPWbzF3_WPsnzh86Ouk&he0DaM2^YgVs zhWQ}=k;Rt?`SZ3AN27^_d*sJH<;=4*zu+xQ}bMCz-#5G@3ff+rGlDQVdPRJAN zQX)?H%#gCI3qP}9&fZ<~%wFc~{iJEJZ=0O>d1su+wRM;ajNXlJogNcW zl2j!N;E?sqd#CCR9&xQ~1vt%PhjuVhF~j*63Od#!tJ&6wC*K*+vxtPZ-J(~CGU`>C zG#-M1V;InWN=Sv6N`dc19t;4H4x6n`d|Q0ZMSNEfo3qWakX=r)R;S>Kg-c~Bsc#xW zW0zBi%g;@Chd7kt%k-UCjt9gnfNvjXw5|YoFW#7M6tXwnZ_*g83diKF|LXKF%5o~( zVkzfUNshhV6#t|w|D`1V>djD&s)p|7LliRBXHJ#q9iEoJuB@s<2P6C_bRteEHrc2^UdGN|x!YNOn*Df!1a`$LZzL`( z)=%*W2@)^Wg(n9T-^99+>~=qh_~91^Hl9$$%;u{-uz&U6ll)^hU%mPN>gw+f8&y+P z6jcnbAh7-benj8|fI0vUIAZ?ngL>GU2%)+_wMipk%JyPR4iieu*|&~&P8;`of-8z! zA102Dio6at4m%HzOuwzBoy7DOdtjbjb(~+foxpj&d^@oQa@zhz!Ae}DPZox?OVotn zdTHBhtTNy!e`s^E)fKHr42?9Z%QM`%!|^uPo(p2M&;&0BonpkH_{k2AyEl&y=2#}Be>%!ndCLKJLN$o z>O&gw9be)PY4T2}M{<%sw&Z+Jgjx~DK}{O1H5`W(3pO_jol(EdoT|E3qn(lFOU{JblpKZ|A%vgryR<@%^&!9<2liQ`9 zL2nOk9->{>7dw<`hK9S1gCxUoEpCQq99rbbWPe`#zHGbcYw;2I98>;qAZT&_dN(WO zkV5PcA|oh=Ho4m}WfP`(G_55V;WF7SoP}sATvIvcZp*%(SgkLhY4-D5O+A7rZMd`v ztID-`ZX-g~>EP~lY>sv2i?=cuF4&w7ZTg#95g3d9o)$CqHzsl--QDn@o`(yUnL!oT z^QnmI#+#lV**k)u_`2PnG5VL>si_fizE}IoUQnia_F=MIVDc|ld#Y;(eCs7yaV(5&{5pFK&bc_tKe>pD7>?mYn*DJB@aRg)T-!=taGX)(%(#-YtG zhJ!85=lFX}nd9*~DC;>>mNf{O8lHsa%S=^qM9uXRD(kaVx9#1!t^t_H4$%3{j{=eJ z=1#)={i4s_yJ^E`G(^N@)oOZ6v8*=bwV_z{b*;E|90Wv~y~(jww3ZdVw=JyESIj3f zCfLCkACZ_|OQ zs9hvSW+pb~yi|xA&awEk5OTYB=SG+IE^TA3}Lfg2! zd0t2&q)Wi%dJc(W!XWlYs}cE(jj}23vMNkf8O^52s5D7cUrDf(rxl)yrkIze8Ut8_ zMJc8hJ41|ov8X81PEwk&tDvw_IEqi-n-wKI=xM2XK`W-A46&xK;_I8S^rY=bB~2MIE55AaZ;FT0 z^vllBYgpczb@sNOf;Ord=49>C)`cw}f-Ap&qrMRo#$4-~?}&@rqe~1BS5X9RrQ26? z5QN-@QCRZ9*c6YCh0@1o$|1$0mG2b|WAz zlX6*mPdfQNXO%U_k;nlwM?dA;*Sh_kBeUojUX>g>@&Jpx_XIAMC}INjWdg34(ZP(z z;WfL_b&wm@}m;m0mRL zE#^#_t|L+rLAqV>3w5VZY=v{9h!VRN#wLH&De`x}q>%W)_o?bR+MrvehT--!K=B}U zezHX@!fU9F*5H90xSNb?7vq}CQ0`-5EPF-x=tD$r@`^VwcTp_z!c&A$in-~!svJ0% z*ZV}pa?(NLTL<&KU&bD1d^-#V76@qT?SI)`SNemU{FR7BszQ0h?O}fJZ0=xwPEDKy zGKEQORaFO8t;g|$to;C;hXZ1dyo`V)6-Oto(MD#q$#4KrOeV_pvmZ5A-~)3E=5WS8 z>p0lomvN>7UHa~NO=?LP!h86Qd*hnx+#X*?+w*>@2dI!T2_d1W0I1fc45#tYhVu;Z zPzKcbakFbp+haM{-KH{n5FoHbHfaoPi+^^n%ZYcfh4Y2}A{&9UsV_X^%0nzFxZOo4 zv$bvUv@g%@v{svojE$E(_a1Wp8ibq9E{pyme=-w!bVl2wE8I9h-x}ER7~Lf6aOK@3 zu&##+Gycu?dwlqSP^C`ET7*)8u=%1$}-N_ zhdlYWFN_LpoYFamNQ2ns{$ZJP6Da@}N%FqO#tfokFHqYybZt7@p#S2%; zt&lAPJ_`#q>NL<#0|a{@X3*}rK{t%wQ^k`3l!)fd$E17RTDEDIi0oCwZPMF6Ey}UI z1+0CnFY42ct&5Yv8n5L>(x;}3U%Xm|Xv2}OZ)xVxp;(0#UQrL66z{!u&t>4C&%l*I zA;!ghZ##CPPFvP`Zi zj21MCtfJ=<-IZgw@(6J?9$76{XZ6@vi_)5ks`J+?DQ;%T)X?KG&6}0W^ekM2QCRBf zPkO7vjPXm;MU1zNcI|osrODd*LAxr_=CIAtAx;*3mB?DmOD^ zj7UVwY}+DRF(!lZ7QNyCfgowvl zzjcn*r#t_iEeWlE&lqP|Db?DGN~vu@WV_$Z8k>E0UgM|>ovN%jH7N;jjA2F9m4_EsXs&%*TNw^gehI_V+8D|sqZY>EZB*y_0@ z4l)w(f}*)4t|@@ep`8+m9wZ4XetON`7wV|Jn;o582@Q*?<3iDhL^abv7Ih$2sjNq? ziK~NzfEp6XZ27J>FbDuQ(s_NX!myhbt@NTNd3X(`{$sT!RN(bykR%yJ=T|Mpf!nwh zKHg@?2ZzD1&DxkJ7#pOCw;ceo^{#`#R8KjU@phZp*|Wgf!Up_`b{pP;cXYe&{it>w za!|oBpm;?tX#-Ru=ZLBx!-3&K%?i*w2USqXqvIyRp2EziLNF&)`thOongIa6Wf8>@ z488G5ENZ42J1lC8NRZGp_-@I`#HmxwQAatDVs~?$0Ap%sa~+=|N48P_i}LcdH+l8@ znCzyaanzfD#b`e=w#b_~M2#i{qd4+i4j-aG5n_`s!JYVhk?Z~xYRw2eAiiF(2yWlx zu^^F;2SN({XiN>Fn1{(8DOhJeU{Juk4I8wZw>rR@G7b_v^9mxvp;`={+KO35;DbGu)=R10@Dk{}r3hH4o z_%~#Om{h~v`stA`*&BC`swyvI8SC3tu^T^Jq}oj4(-@VPv7yh5vax=1U8iw%K5nxfFvvtnt_-ChAO>WiK>;fdc7qKC0s+RO?ZFe(J_?h z0)tM0uutiQSpw7yXRxMW6s=bH4IsakPfz z$1|440`(G`Jxu{?P7Pt3!~tB%0J{EyHr-JKJa{RC-gKZveh*czX7&WMdXO>dQPbl~AGf0!GrLQwa?5W|<_S^Wc^UG)mHQYZeFa2hU6l7AZ7gchinxLbM=$t6vy2i*u&6j(ue@a3UVj3y2Z`gR`2`<{nP z5)wgDU2=H=48Ly_K_z(?W%`iVVhrro#POlfotdYu_bqrsI8}G-9}d%5+~nAeqSqmG zGV0bY(CSm^)f?&80Js280L&~(tPB+*S(Rs3FL>%(AL>jY^#K_aq`sW1`bZ) z7;YaCdOeYiVQi>3tn`JW)nggV)89rPJm2mw&+} z@9T4=yLKnW;bKo(-;%H!w61g&>Wxp{LeBi5;7&olo15^T-YY|>a& zjc&}anK|i*%-&=tU5@bi*)b2`)UhV>WR}`xG3vo{pK&mibPtbwHU@QH17|($P}%+t zji{()xv0s`Pym@{w|(K;C0La(2?3aZfQxAh z>w#cFN|o5-*>vK2<&=F#fK?d%Mw%vD1LI!kT0JG0Q9wSux>Lop3de-l0_fEnSynK_ z+&qTWv$~o@TEepe1!~1%dlmaL$g=MhE=SU!9_EMGJm46Q*dM>#&&nQp7t3Dm{t}!= z>k_MK_PQAX{~Dmj|6iA{|L(2QAD{kqpQ1o%MP`-{EQ9(rk{R;Es}x?5 zH(@2`ip|7|M8eX<+V^t8pF&IrI>Hq|g31PNCwUo#e5egYR6oDCp(5TuK3R*bYS=^> z$M@|Z{5_o5$80ryE!;t&Lfb1cc>iNj;6`L&Dm7@s_6m&0TyUJglTzEO(J4K17V+rifzVt)1}Lf}oT$4|VWB(D(6%V-BkgKf($gmCK^^;C!8w zMFqv!b`LJ*-foi|>HQeAROOXxp&!!8#eVB=KROb@avM-^McF%tf{-Q5+^VK&TjL3X zzz}0G|4%{tYEiJPMJaUkj+E9L8#53{6ibj)Nr>;$D1SiVK~C z&;Ud+(nja2%?Ih6_v>rC-rmm;RelIm-IHisN?ScOVR&4~nP}Bq@Kbl8ennJ%E`{6g zwJBUm2=YEA0K;xMuI$UqGg6^x>td`svTv|N(RB2T1nD=C8n+sxJf6c&p~P_p#qly+ z2PM#|&E<}Lp2q|!E3|AehgCnOcQ*52aTuwJzQtW&INcf5McH`5^XJu|vI$yGH@zM+ zyA0t}NOZ6xMP6jWQT)PPw870l*=|trQNwMH1rT<2#iZ)oSKT3LM0^P=D63uAXo}jG zp(o8q(C@c}1^Ya9rJpPzex9QJC=w-!S#2e^tg_Fi$?u2_9~3$dR9YubG5byeSg9E_ zNC+E^kWp^ytcWb(zG&VM-Z0m`ohown(V;fq8iq|P7`rb5_9meO6;yhnB(P9xvtb0$ zfn5#W8AE~AjJAA2o1|+lLt$>{I=b-7hWFYBWIx^Bi!j2O4%ea9eObBkz)LLa02ac; zBl@q11=Z%V?kQzH?aCK)5j5+HNh`!9)qoK`!KJSHM9xQ!hA0`=^#rXA6Xoard}qWs z%|s7*tPTyGneQj!aZCX`Gf zIkqj6P4SrJJy?tkvi;E{MqF1cVMy}8mS4!@sd<#$$V1=RMp931LJv;Q@WRw4UpIBg z!F#!h(jl%cm_b_77496!SsT6gN6c}9B2TX8DD1-FhrX`CK;j5TW&{H_r9qqz79Oe> zG8wFOXNC;{8#>N$+4yBAT2z0nmFe|;(SA)&q5nTA{$KoO3R>nr0zTrtSfx6~vCgST z$EA8LlC3PyL3@Ux<`aPBhCsO2PSCA2qPT5=jBt(euQpEii7Ly z7US&QqG^e?OLrAF_%~`Kd>BfD48k(~e%O;4)PDMV^z7ENb@o)3;L%=zU7 zX4i1u1OD_-+Gfs@;$7cvMRw%MWZ}fdx(kqCO;g&8Kf(#Dkr2%moX2#%efIr27-LSk z?)fbpp&z^jU_jBjtyUblT`;yd<$blOw;~9PfT2>Uy1PYY<+{kv;QzTtkn5_-M;OBU zt7P`sGw*}m=CX1g6K2}5BR?}=yPOzZYkSQ8Awg)+bF%K^F>{bdbXHxjk7Nb%qteu% zGIqUS`Q)r|P8M_VOcQ^Z=sb_Y4MkCvY7g6Q3a@zh!l`UMPD{zo8Z%@VXE-sI7@K+h`P$bM$_6>4 zsi2x_?_*nJeq`*9<>Tx_{2(4(Oj8?4&!K54y-dsFZV0=+X;eEw305F&u-{^Z?U#h4 z$~wWWE|--kG6n%!3FxWR+N~p3_D8?I!Nm@;U{ z&6C>9)r`A_ptQ7J?yl*{?aKmluX*G#B;JS1M0ML~2HZzVdqNXVv9BRGh4T617#6cca&u^ z3WId0I1HettzEo(m-8sijUp3^boz_^;!|upsOih3`QGR%>|O6l$h`W%kq|j%pdqLB zWZ;_3Hya_;n=?0ybiabGVwWpZB(_BOBzzFD5!ZIfG)FD%WF?3Y`O?Fmo_`>>ho${F z+^0^6Ln124X84Wl@C0AREer^+5~1Mq5|QqxgbzXdFW#mL^tHg?abmF10_l|_S2r5hxqPLZoZc69SgWJ zn*?Qy)5DnZhfR+r*Lea(jZXeF&A^TbdQLP=Q2ZI6gb*xwK?f@|vc?POV{bQsL{+hC zExtWsa<0$K?Ps7a6G(4AnvY!1*`HFKCQi@kmU1UmKH=(^4O?Qd=Nd zk{R{)iNBavNCxLsI~X8|*oPE;iu!;|Y*fA_hsbaXb2zq4be!5V9fj2$Rs3Z|G^nQO zaxK;aG1n5s0`dY2+a@J$wP(OCkoKs6U3Dsw=jGidhsEk=?wcJ#Ue5@Ak03&ip5QBP zkqZpA>fX-t5sU$>sh8bHo#c4H_jKz`hOCA6`4sojql?ni-L2fH#}o%j^Y z0b>h!8(CCgRlS;$%8Tzl74r}wU|Zar{7olU1#1uJe^A~3%rbtB%lVmQY^m#HX!Y;7 z|NkufBX{-RDapSTe$7aw`!#p>*Gy)z|0ZTe`y*?X_OBR_KmPzQ2yOUVAKpl6y;{M5)`<;sz6#zbFX|IGP|FO=kp;PbD7rK zur8{2iK{=F;eConf?jyO(tFfSK5QXb4WEE054uouW!0=}<|ObE0ELtQXA%sPWnZS@ z+c80dErYxmX2>S4-K*}V7%p!5jLCI}F0z~OA?{{b6DE^)YEVb$^HqBYU^*{CHoD*+ z{aLpa%(U|PU4p(qsfMBRF)B}oMTs}h_N}7r$@z}*8-?u2L1Zq>pCu5>mE?y)lRb(( zS&(?AvnKMMu&jR?6WgN6LC>&e`t&h$2_^H9214P|4k%eoUU3J@p=f(Re;+! ze9CP~2lUV8Z4FSdgkEvm%aOZZJFB>66&9h)UcWB~c{GiW_*E;WLp0!X9k`bYZ*h{a zml5EWLRA?DH@ZVTln>?DqSzW}4h=G22~3Jj5gaMZZT1cgNUAwVef$_~tcb?au@hC% zSYZpdGFVwC$1J>9NrO`)ftpRMV;kvFxT}4k-YCh)e#xN(VB!;Yk>6^CQ7e4^0Uvq; zI83{%TWikz7fWNx0e%^QR*IBMi|`TldPRQN5^UjfTq#{DrIzb3u>#GOU_2i*Ub0`n{Uuc;P4;oFh_0sfy7FMfsoHNyNS zG~sKM%-@d4-}-TX#r-wl`6uo%`aj_Q%Q^nf&+o@L{~BNX6LkvXpHTmZ=lN|>_-i|V zjmP?lc#Z$R5dRdN^(*YJp+-MptqK1Z?0+AEU-uh+BKN+MroVk)e=LjtvitCBi+}a( z|HQ4K`#<6S+fn*|njgRVUw>BT{+c!Rzq|4GPxDtV)lby2SEKyDQUCkU|LTJHiB7}z zpXmSTkoY;||7mysY?Qu#H{$w}-Ti~6|1<@^8tp$@l<(i;{!{q1=l`bd{|xhkxc_z` z0)78J{NG^yLcxFb>+gyDAE^1h@ZA5=ub)l(3+C^%{?E`q6u_@E>nHRG@Ba_ + + + + + hoodie + com.uber.hoodie + 0.2.5-SNAPSHOT + + 4.0.0 + + hoodie-cli + jar + + + 1.2.0.RELEASE + org.springframework.shell.Bootstrap + 1.2.17 + 4.10 + + + + + libs-milestone + http://repo.spring.io/libs-milestone/ + + + libs-release + http://repo.spring.io/libs-release/ + + + scala-tools.org + Scala-tools Maven2 Repository + http://scala-tools.org/repo-releases + + + + + + + + net.alchim31.maven + scala-maven-plugin + 3.2.1 + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.5 + 1.5 + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + prepare-package + + copy-dependencies + + + ${project.build.directory}/lib + true + true + true + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + true + false + lib/ + ${jar.mainclass} + + + ${project.version} + + + + + + net.alchim31.maven + scala-maven-plugin + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.apache.rat + apache-rat-plugin + + + + + + + + + org.scala-lang + scala-library + 2.10.5 + + + + org.springframework.shell + spring-shell + ${spring.shell.version} + + + de.vandermeer + asciitable + 0.2.5 + + + org.apache.spark + spark-core_2.10 + + + org.apache.spark + spark-sql_2.10 + ${spark.version} + provided + + + + + dnl.utils + textutils + 0.3.3 + system + ${basedir}/lib/dnl/utils/textutils/0.3.3/textutils-0.3.3.jar + + + + log4j + log4j + ${log4j.version} + + + com.uber.hoodie + hoodie-client + ${project.version} + + + com.uber.hoodie + hoodie-common + ${project.version} + + + com.uber.hoodie + hoodie-mr + ${project.version} + + + com.uber.hoodie + hoodie-tools + ${project.version} + + + junit + junit-dep + ${junit.version} + test + + + + diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java new file mode 100644 index 000000000..5c8e6e9ca --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli; + +import com.uber.hoodie.common.model.HoodieTableMetadata; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +import java.io.IOException; + +public class HoodieCLI { + public static Configuration conf; + public static FileSystem fs; + public static CLIState state = CLIState.INIT; + public static HoodieTableMetadata tableMetadata; + public static HoodieTableMetadata syncTableMetadata; + + + public enum CLIState { + INIT, DATASET, SYNC + } + + public static boolean initConf() { + if (HoodieCLI.conf == null) { + HoodieCLI.conf = new Configuration(); + return true; + } + return false; + } + + public static void initFS(boolean force) throws IOException { + if(fs == null || force) { + fs = FileSystem.get(conf); + } + } + + public static void setTableMetadata(HoodieTableMetadata tableMetadata) { + HoodieCLI.tableMetadata = tableMetadata; + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieHistoryFileNameProvider.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieHistoryFileNameProvider.java new file mode 100644 index 000000000..aecf1de49 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieHistoryFileNameProvider.java @@ -0,0 +1,37 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli; + +import org.springframework.core.Ordered; +import org.springframework.core.annotation.Order; +import org.springframework.shell.plugin.support.DefaultHistoryFileNameProvider; +import org.springframework.stereotype.Component; + +@Component +@Order(Ordered.HIGHEST_PRECEDENCE) +public class HoodieHistoryFileNameProvider extends DefaultHistoryFileNameProvider { + + public String getHistoryFileName() { + return "hoodie-cmd.log"; + } + + @Override + public String getProviderName() { + return "Hoodie file name provider"; + } + +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java new file mode 100644 index 000000000..34b2c2414 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java @@ -0,0 +1,34 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli; + +import dnl.utils.text.table.TextTable; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.Charset; + +public class HoodiePrintHelper { + + public static String print(String[] header, String[][] rows) { + TextTable textTable = new TextTable(header, rows); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintStream ps = new PrintStream(baos); + textTable.printTable(ps, 4); + return new String(baos.toByteArray(), Charset.forName("utf-8")); + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java new file mode 100644 index 000000000..e44c62dfd --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrompt.java @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli; + +import org.springframework.core.Ordered; +import org.springframework.core.annotation.Order; +import org.springframework.shell.plugin.support.DefaultPromptProvider; +import org.springframework.stereotype.Component; + +@Component +@Order(Ordered.HIGHEST_PRECEDENCE) +public class HoodiePrompt extends DefaultPromptProvider { + + @Override + public String getPrompt() { + switch (HoodieCLI.state) { + case INIT: + return "hoodie->"; + case DATASET: + return "hoodie:" + HoodieCLI.tableMetadata.getTableName() + "->"; + case SYNC: + return "hoodie:" + HoodieCLI.tableMetadata.getTableName() + " <==> " + + HoodieCLI.syncTableMetadata.getTableName() + "->"; + } + if (HoodieCLI.tableMetadata != null) + return "hoodie:" + HoodieCLI.tableMetadata.getTableName() + "->"; + return "hoodie->"; + } + + @Override + public String getProviderName() { + return "Hoodie provider"; + } + +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java new file mode 100644 index 000000000..fa27d5749 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieSplashScreen.java @@ -0,0 +1,55 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli; + +import org.springframework.core.Ordered; +import org.springframework.core.annotation.Order; +import org.springframework.shell.plugin.support.DefaultBannerProvider; +import org.springframework.shell.support.util.OsUtils; +import org.springframework.stereotype.Component; + +@Component @Order(Ordered.HIGHEST_PRECEDENCE) public class HoodieSplashScreen + extends DefaultBannerProvider { + private static String screen = "============================================" + OsUtils.LINE_SEPARATOR + + "* *" + OsUtils.LINE_SEPARATOR + + "* _ _ _ _ *" + OsUtils.LINE_SEPARATOR + + "* | | | | | (_) *" + OsUtils.LINE_SEPARATOR + + "* | |__| | ___ ___ __| |_ ___ *" + OsUtils.LINE_SEPARATOR + + "* | __ |/ _ \\ / _ \\ / _` | |/ _ \\ *" + + OsUtils.LINE_SEPARATOR + + "* | | | | (_) | (_) | (_| | | __/ *" + OsUtils.LINE_SEPARATOR + + "* |_| |_|\\___/ \\___/ \\__,_|_|\\___| *" + + OsUtils.LINE_SEPARATOR + + "* *" + OsUtils.LINE_SEPARATOR + + "============================================" + OsUtils.LINE_SEPARATOR; + + public String getBanner() { + return screen; + } + + public String getVersion() { + return "1.0"; + } + + public String getWelcomeMessage() { + return "Welcome to Hoodie CLI. Please type help if you are looking for help. "; + } + + @Override public String getProviderName() { + return "Hoodie Banner"; + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java new file mode 100644 index 000000000..779df13f0 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Main.java @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli; + +import org.springframework.shell.Bootstrap; + +import java.io.IOException; + +public class Main { + /** + * Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging inside an IDE + * + * @param args + * @throws IOException + */ + public static void main(String[] args) throws IOException { + Bootstrap.main(args); + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java new file mode 100644 index 000000000..e2f415c08 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java @@ -0,0 +1,244 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.commands; + +import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.cli.utils.InputStreamConsumer; +import com.uber.hoodie.cli.utils.SparkUtil; +import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.model.HoodieCommits; +import com.uber.hoodie.common.model.HoodieTableMetadata; +import com.uber.hoodie.common.model.HoodieWriteStat; +import com.uber.hoodie.common.util.NumericUtils; + +import org.apache.spark.launcher.SparkLauncher; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliAvailabilityIndicator; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; + +@Component +public class CommitsCommand implements CommandMarker { + @CliAvailabilityIndicator({"commits show"}) + public boolean isShowAvailable() { + return HoodieCLI.tableMetadata != null; + } + + @CliAvailabilityIndicator({"commits refresh"}) + public boolean isRefreshAvailable() { + return HoodieCLI.tableMetadata != null; + } + + @CliAvailabilityIndicator({"commit rollback"}) + public boolean isRollbackAvailable() { + return HoodieCLI.tableMetadata != null; + } + + @CliAvailabilityIndicator({"commit show"}) + public boolean isCommitShowAvailable() { + return HoodieCLI.tableMetadata != null; + } + + @CliCommand(value = "commits show", help = "Show the commits") + public String showCommits( + @CliOption(key = { + "limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10") + final Integer limit) throws IOException { + SortedMap map = + HoodieCLI.tableMetadata.getAllCommitMetadata(); + int arraySize = + Math.min(limit, HoodieCLI.tableMetadata.getAllCommits().getCommitList().size()); + String[][] rows = new String[arraySize][]; + ArrayList commitList = + new ArrayList(HoodieCLI.tableMetadata.getAllCommits().getCommitList()); + Collections.reverse(commitList); + for (int i = 0; i < arraySize; i++) { + String commit = commitList.get(i); + HoodieCommitMetadata commitMetadata = map.get(commit); + rows[i] = new String[] {commit, + NumericUtils.humanReadableByteCount(commitMetadata.fetchTotalBytesWritten()), + String.valueOf(commitMetadata.fetchTotalFilesInsert()), + String.valueOf(commitMetadata.fetchTotalFilesUpdated()), + String.valueOf(commitMetadata.fetchTotalPartitionsWritten()), + String.valueOf(commitMetadata.fetchTotalRecordsWritten()), + String.valueOf(commitMetadata.fetchTotalUpdateRecordsWritten()), + String.valueOf(commitMetadata.fetchTotalWriteErrors())}; + } + return HoodiePrintHelper.print( + new String[] {"CommitTime", "Total Written (B)", "Total Files Added", + "Total Files Updated", "Total Partitions Written", "Total Records Written", + "Total Update Records Written", "Total Errors"}, rows); + } + + @CliCommand(value = "commits refresh", help = "Refresh the commits") + public String refreshCommits() throws IOException { + HoodieTableMetadata metadata = + new HoodieTableMetadata(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); + HoodieCLI.setTableMetadata(metadata); + return "Metadata for table " + metadata.getTableName() + " refreshed."; + } + + @CliCommand(value = "commit rollback", help = "Rollback a commit") + public String rollbackCommit( + @CliOption(key = {"commit"}, help = "Commit to rollback") + final String commitTime, + @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") + final String sparkPropertiesPath) throws Exception { + if (!HoodieCLI.tableMetadata.getAllCommits().contains(commitTime)) { + return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata + .getAllCommits(); + } + SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); + sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(), + commitTime, + HoodieCLI.tableMetadata.getBasePath()); + Process process = sparkLauncher.launch(); + InputStreamConsumer.captureOutput(process); + int exitCode = process.waitFor(); + // Refresh the current + refreshCommits(); + if (exitCode != 0) { + return "Commit " + commitTime + " failed to roll back"; + } + return "Commit " + commitTime + " rolled back"; + } + + @CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit") + public String showCommitPartitions( + @CliOption(key = {"commit"}, help = "Commit to show") + final String commitTime) throws Exception { + if (!HoodieCLI.tableMetadata.getAllCommits().contains(commitTime)) { + return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata + .getAllCommits(); + } + HoodieCommitMetadata meta = HoodieCLI.tableMetadata.getAllCommitMetadata().get(commitTime); + List rows = new ArrayList(); + for (Map.Entry> entry : meta.getPartitionToWriteStats() + .entrySet()) { + String path = entry.getKey(); + List stats = entry.getValue(); + long totalFilesAdded = 0; + long totalFilesUpdated = 0; + long totalRecordsUpdated = 0; + long totalRecordsInserted = 0; + long totalBytesWritten = 0; + long totalWriteErrors = 0; + for (HoodieWriteStat stat : stats) { + if (stat.getPrevCommit().equals(HoodieWriteStat.NULL_COMMIT)) { + totalFilesAdded += 1; + totalRecordsInserted += stat.getNumWrites(); + } else { + totalFilesUpdated += 1; + totalRecordsUpdated += stat.getNumUpdateWrites(); + } + totalBytesWritten += stat.getTotalWriteBytes(); + totalWriteErrors += stat.getTotalWriteErrors(); + } + rows.add(new String[] {path, String.valueOf(totalFilesAdded), + String.valueOf(totalFilesUpdated), String.valueOf(totalRecordsInserted), + String.valueOf(totalRecordsUpdated), + NumericUtils.humanReadableByteCount(totalBytesWritten), + String.valueOf(totalWriteErrors)}); + + } + return HoodiePrintHelper.print( + new String[] {"Partition Path", "Total Files Added", "Total Files Updated", + "Total Records Inserted", "Total Records Updated", "Total Bytes Written", + "Total Errors"}, rows.toArray(new String[rows.size()][])); + } + + @CliCommand(value = "commit showfiles", help = "Show file level details of a commit") + public String showCommitFiles( + @CliOption(key = {"commit"}, help = "Commit to show") + final String commitTime) throws Exception { + if (!HoodieCLI.tableMetadata.getAllCommits().contains(commitTime)) { + return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata + .getAllCommits(); + } + HoodieCommitMetadata meta = HoodieCLI.tableMetadata.getAllCommitMetadata().get(commitTime); + List rows = new ArrayList(); + for (Map.Entry> entry : meta.getPartitionToWriteStats() + .entrySet()) { + String path = entry.getKey(); + List stats = entry.getValue(); + for (HoodieWriteStat stat : stats) { + rows.add(new String[] {path, stat.getFileId(), stat.getPrevCommit(), + String.valueOf(stat.getNumUpdateWrites()), String.valueOf(stat.getNumWrites()), + String.valueOf(stat.getTotalWriteBytes()), + String.valueOf(stat.getTotalWriteErrors())}); + } + } + return HoodiePrintHelper.print( + new String[] {"Partition Path", "File ID", "Previous Commit", "Total Records Updated", + "Total Records Written", "Total Bytes Written", "Total Errors"}, + rows.toArray(new String[rows.size()][])); + } + + @CliAvailabilityIndicator({"commits compare"}) + public boolean isCompareCommitsAvailable() { + return HoodieCLI.tableMetadata != null; + } + + @CliCommand(value = "commits compare", help = "Compare commits with another Hoodie dataset") + public String compareCommits( + @CliOption(key = {"path"}, help = "Path of the dataset to compare to") + final String path) throws Exception { + HoodieTableMetadata target = new HoodieTableMetadata(HoodieCLI.fs, path); + HoodieTableMetadata source = HoodieCLI.tableMetadata; + String targetLatestCommit = + target.isCommitsEmpty() ? "0" : target.getAllCommits().lastCommit(); + String sourceLatestCommit = + source.isCommitsEmpty() ? "0" : source.getAllCommits().lastCommit(); + + if (sourceLatestCommit != null && HoodieCommits + .isCommit1After(targetLatestCommit, sourceLatestCommit)) { + // source is behind the target + List commitsToCatchup = target.findCommitsSinceTs(sourceLatestCommit); + return "Source " + source.getTableName() + " is behind by " + commitsToCatchup.size() + + " commits. Commits to catch up - " + commitsToCatchup; + } else { + List commitsToCatchup = source.findCommitsSinceTs(targetLatestCommit); + return "Source " + source.getTableName() + " is ahead by " + commitsToCatchup.size() + + " commits. Commits to catch up - " + commitsToCatchup; + } + } + + @CliAvailabilityIndicator({"commits sync"}) + public boolean isSyncCommitsAvailable() { + return HoodieCLI.tableMetadata != null; + } + + @CliCommand(value = "commits sync", help = "Compare commits with another Hoodie dataset") + public String syncCommits( + @CliOption(key = {"path"}, help = "Path of the dataset to compare to") + final String path) throws Exception { + HoodieCLI.syncTableMetadata = new HoodieTableMetadata(HoodieCLI.fs, path); + HoodieCLI.state = HoodieCLI.CLIState.SYNC; + return "Load sync state between " + HoodieCLI.tableMetadata.getTableName() + " and " + + HoodieCLI.syncTableMetadata.getTableName(); + } + +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java new file mode 100644 index 000000000..5646566eb --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java @@ -0,0 +1,42 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.commands; + +import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.common.model.HoodieTableMetadata; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +import java.io.IOException; + +@Component +public class DatasetsCommand implements CommandMarker { + @CliCommand(value = "connect", help = "Connect to a hoodie dataset") + public String connect( + @CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") + final String path) throws IOException { + boolean initialized = HoodieCLI.initConf(); + HoodieCLI.initFS(initialized); + HoodieCLI.setTableMetadata(new HoodieTableMetadata(HoodieCLI.fs, path)); + HoodieCLI.state = HoodieCLI.CLIState.DATASET; + return "Metadata for table " + HoodieCLI.tableMetadata.getTableName() + " loaded"; + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java new file mode 100644 index 000000000..3a3767042 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieSyncCommand.java @@ -0,0 +1,106 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.commands; + +import com.uber.hoodie.cli.utils.CommitUtil; +import com.uber.hoodie.cli.utils.HiveUtil; +import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.common.model.HoodieCommits; +import com.uber.hoodie.common.model.HoodieTableMetadata; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliAvailabilityIndicator; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +import java.util.List; + +@Component +public class HoodieSyncCommand implements CommandMarker { + @CliAvailabilityIndicator({"sync validate"}) + public boolean isSyncVerificationAvailable() { + return HoodieCLI.tableMetadata != null && HoodieCLI.syncTableMetadata != null; + } + + @CliCommand(value = "sync validate", help = "Validate the sync by counting the number of records") + public String validateSync( + @CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode") + final String mode, + @CliOption(key = { + "sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database") + final String srcDb, + @CliOption(key = { + "targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database") + final String tgtDb, + @CliOption(key = { + "partitionCount"}, unspecifiedDefaultValue = "5", help = "total number of recent partitions to validate") + final int partitionCount, + @CliOption(key = { + "hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to") + final String hiveServerUrl, + @CliOption(key = { + "hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to") + final String hiveUser, + @CliOption(key = { + "hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to") + final String hivePass) throws Exception { + HoodieTableMetadata target = HoodieCLI.syncTableMetadata; + HoodieTableMetadata source = HoodieCLI.tableMetadata; + long sourceCount = 0; + long targetCount = 0; + if ("complete".equals(mode)) { + sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, hiveUser, hivePass); + targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, hiveUser, hivePass); + } else if ("latestPartitions".equals(mode)) { + sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass); + targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass); + } + + String targetLatestCommit = + target.isCommitsEmpty() ? "0" : target.getAllCommits().lastCommit(); + String sourceLatestCommit = + source.isCommitsEmpty() ? "0" : source.getAllCommits().lastCommit(); + + if (sourceLatestCommit != null && HoodieCommits + .isCommit1After(targetLatestCommit, sourceLatestCommit)) { + // source is behind the target + List commitsToCatchup = target.findCommitsSinceTs(sourceLatestCommit); + if (commitsToCatchup.isEmpty()) { + return "Count difference now is (count(" + target.getTableName() + ") - count(" + + source.getTableName() + ") == " + (targetCount - sourceCount); + } else { + long newInserts = CommitUtil.countNewRecords(target, commitsToCatchup); + return "Count difference now is (count(" + target.getTableName() + ") - count(" + + source.getTableName() + ") == " + (targetCount - sourceCount) + + ". Catch up count is " + newInserts; + } + } else { + List commitsToCatchup = source.findCommitsSinceTs(targetLatestCommit); + if (commitsToCatchup.isEmpty()) { + return "Count difference now is (count(" + source.getTableName() + ") - count(" + + target.getTableName() + ") == " + (sourceCount - targetCount); + } else { + long newInserts = CommitUtil.countNewRecords(source, commitsToCatchup); + return "Count difference now is (count(" + source.getTableName() + ") - count(" + + target.getTableName() + ") == " + (sourceCount - targetCount) + + ". Catch up count is " + newInserts; + } + + } + } + +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RecordsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RecordsCommand.java new file mode 100644 index 000000000..f48611197 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/RecordsCommand.java @@ -0,0 +1,78 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.commands; + +import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.cli.utils.InputStreamConsumer; +import com.uber.hoodie.cli.utils.SparkUtil; +import org.apache.spark.launcher.SparkLauncher; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliAvailabilityIndicator; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +@Component +public class RecordsCommand implements CommandMarker { + + @CliAvailabilityIndicator({"records deduplicate"}) + public boolean isRecordsDeduplicateAvailable() { + return HoodieCLI.tableMetadata != null; + } + + @CliCommand(value = "records deduplicate", help = "De-duplicate a partition path contains duplicates & produce repaired files to replace with") + public String deduplicate( + @CliOption(key = { + "duplicatedPartitionPath"}, help = "Partition Path containing the duplicates") + final String duplicatedPartitionPath, + @CliOption(key = {"repairedOutputPath"}, help = "Location to place the repaired files") + final String repairedOutputPath, + @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") + final String sparkPropertiesPath) throws Exception { + SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); + sparkLauncher + .addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath, + repairedOutputPath, HoodieCLI.tableMetadata.getBasePath()); + Process process = sparkLauncher.launch(); + InputStreamConsumer.captureOutput(process); + int exitCode = process.waitFor(); + + if (exitCode != 0) { + return "Deduplicated files placed in: " + repairedOutputPath; + } + return "Deduplication failed "; + } + +// @CliCommand(value = "records find", help = "Find Records in a hoodie dataset") +// public String findRecords( +// @CliOption(key = {"keys"}, help = "Keys To Find (Comma seperated)") +// final String hoodieKeys, +// @CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path") +// final String sparkPropertiesPath) throws Exception { +// SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath); +// sparkLauncher +// .addAppArgs(SparkMain.RECORD_FIND, hoodieKeys, HoodieCLI.tableMetadata.getBasePath()); +// Process process = sparkLauncher.launch(); +// InputStreamConsumer.captureOutput(process); +// int exitCode = process.waitFor(); +// +// if (exitCode != 0) { +// return "Deduplicated files placed in: " + repairedOutputPath; +// } +// return "Deduplication failed "; +// } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java new file mode 100644 index 000000000..c6f4c1327 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java @@ -0,0 +1,94 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.commands; + +import com.uber.hoodie.HoodieWriteClient; +import com.uber.hoodie.cli.DedupeSparkJob; +import com.uber.hoodie.cli.utils.SparkUtil; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieIndexConfig; +import com.uber.hoodie.config.HoodieWriteConfig; +import com.uber.hoodie.index.HoodieIndex; + +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SQLContext; + +public class SparkMain { + + protected final static Logger LOG = Logger.getLogger(SparkMain.class); + + + /** + * Commands + */ + enum SparkCommand { + ROLLBACK, + DEDUPLICATE + } + + public static void main(String[] args) throws Exception { + String command = args[0]; + LOG.info("Invoking SparkMain:" + command); + + SparkCommand cmd = SparkCommand.valueOf(command); + + JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + command); + int returnCode = 0; + if (SparkCommand.ROLLBACK.equals(cmd)) { + assert (args.length == 3); + returnCode = rollback(jsc, args[1], args[2]); + } else if(SparkCommand.DEDUPLICATE.equals(cmd)) { + assert (args.length == 4); + returnCode = deduplicatePartitionPath(jsc, args[1], args[2], args[3]); + } + + System.exit(returnCode); + } + + private static int deduplicatePartitionPath(JavaSparkContext jsc, + String duplicatedPartitionPath, + String repairedOutputPath, + String basePath) + throws Exception { + DedupeSparkJob job = new DedupeSparkJob(basePath, + duplicatedPartitionPath,repairedOutputPath,new SQLContext(jsc), FSUtils.getFs()); + job.fixDuplicates(true); + return 0; + } + + private static int rollback(JavaSparkContext jsc, String commitTime, String basePath) + throws Exception { + HoodieWriteClient client = createHoodieClient(jsc, basePath); + if (client.rollback(commitTime)) { + LOG.info(String.format("The commit \"%s\" rolled back.", commitTime)); + return -1; + } else { + LOG.info(String.format("The commit \"%s\" failed to roll back.", commitTime)); + } + return 0; + } + + private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) + throws Exception { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withIndexConfig( + HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .build(); + return new HoodieWriteClient(jsc, config); + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java new file mode 100644 index 000000000..8a4e68e57 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/StatsCommand.java @@ -0,0 +1,136 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.commands; + + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Snapshot; +import com.codahale.metrics.UniformReservoir; +import com.uber.hoodie.cli.HoodieCLI; +import com.uber.hoodie.cli.HoodiePrintHelper; +import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.NumericUtils; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliAvailabilityIndicator; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +import java.io.IOException; +import java.text.DecimalFormat; +import java.util.HashMap; +import java.util.Map; + +@Component +public class StatsCommand implements CommandMarker { + @CliAvailabilityIndicator({"stats wa"}) + public boolean isWriteAmpAvailable() { + return HoodieCLI.tableMetadata != null; + } + + @CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many records were actually written") + public String writeAmplificationStats() throws IOException { + long totalRecordsUpserted = 0; + long totalRecordsWritten = 0; + + String[][] rows = new String[HoodieCLI.tableMetadata.getAllCommitMetadata().size() + 1][]; + int i = 0; + DecimalFormat df = new DecimalFormat("#.00"); + for (Map.Entry commit : HoodieCLI.tableMetadata + .getAllCommitMetadata().entrySet()) { + String waf = "0"; + if (commit.getValue().fetchTotalUpdateRecordsWritten() > 0) { + waf = df.format( + (float) commit.getValue().fetchTotalRecordsWritten() / commit.getValue() + .fetchTotalUpdateRecordsWritten()); + } + rows[i++] = new String[] {commit.getKey(), + String.valueOf(commit.getValue().fetchTotalUpdateRecordsWritten()), + String.valueOf(commit.getValue().fetchTotalRecordsWritten()), waf}; + totalRecordsUpserted += commit.getValue().fetchTotalUpdateRecordsWritten(); + totalRecordsWritten += commit.getValue().fetchTotalRecordsWritten(); + } + String waf = "0"; + if (totalRecordsUpserted > 0) { + waf = df.format((float) totalRecordsWritten / totalRecordsUpserted); + } + rows[i] = new String[] {"Total", String.valueOf(totalRecordsUpserted), + String.valueOf(totalRecordsWritten), waf}; + return HoodiePrintHelper.print( + new String[] {"CommitTime", "Total Upserted", "Total Written", + "Write Amplifiation Factor"}, rows); + + } + + + private String[] printFileSizeHistogram(String commitTime, Snapshot s) { + return new String[]{ + commitTime, + NumericUtils.humanReadableByteCount(s.getMin()), + NumericUtils.humanReadableByteCount(s.getValue(0.1)), + NumericUtils.humanReadableByteCount(s.getMedian()), + NumericUtils.humanReadableByteCount(s.getMean()), + NumericUtils.humanReadableByteCount(s.get95thPercentile()), + NumericUtils.humanReadableByteCount(s.getMax()), + String.valueOf(s.size()), + NumericUtils.humanReadableByteCount(s.getStdDev()) + }; + } + + @CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files") + public String fileSizeStats( + @CliOption(key = {"partitionPath"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*") + final String globRegex) throws IOException { + + FileSystem fs = HoodieCLI.fs; + String globPath = String.format("%s/%s/*", + HoodieCLI.tableMetadata.getBasePath(), + globRegex); + FileStatus[] statuses = fs.globStatus(new Path(globPath)); + + // max, min, #small files < 10MB, 50th, avg, 95th + final int MAX_FILES = 1000000; + Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES)); + HashMap commitHistoMap = new HashMap(); + for (FileStatus fileStatus: statuses) { + String commitTime = FSUtils.getCommitTime(fileStatus.getPath().getName()); + long sz = fileStatus.getLen(); + if (!commitHistoMap.containsKey(commitTime)) { + commitHistoMap.put(commitTime, new Histogram(new UniformReservoir(MAX_FILES))); + } + commitHistoMap.get(commitTime).update(sz); + globalHistogram.update(sz); + } + + String[][] rows = new String[commitHistoMap.size() + 1][]; + int ind = 0; + for (String commitTime: commitHistoMap.keySet()) { + Snapshot s = commitHistoMap.get(commitTime).getSnapshot(); + rows[ind++] = printFileSizeHistogram(commitTime, s); + } + Snapshot s = globalHistogram.getSnapshot(); + rows[ind++] = printFileSizeHistogram("ALL", s); + + return HoodiePrintHelper.print( + new String[] {"CommitTime", "Min", "10th", "50th", "avg", "95th", "Max", "NumFiles", "StdDev"}, rows); + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java new file mode 100644 index 000000000..b5abb6a6e --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/UtilsCommand.java @@ -0,0 +1,34 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.commands; + +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +@Component +public class UtilsCommand implements CommandMarker { + @CliCommand(value = "utils loadClass", help = "Load a class" ) + public String loadClass( + @CliOption(key = {"class"}, help = "Check mode" ) final String clazz + ) throws Exception { + Class klass = Class.forName(clazz); + return klass.getProtectionDomain().getCodeSource().getLocation().toExternalForm(); + } + +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java new file mode 100644 index 000000000..a92036402 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/CommitUtil.java @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.utils; + +import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.model.HoodieTableMetadata; + +import java.io.IOException; +import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; + +public class CommitUtil { + public static long countNewRecords(HoodieTableMetadata target, List commitsToCatchup) + throws IOException { + long totalNew = 0; + SortedMap meta = target.getAllCommitMetadata(); + for(String commit:commitsToCatchup) { + HoodieCommitMetadata c = meta.get(commit); + totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten(); + } + return totalNew; + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java new file mode 100644 index 000000000..1b7267d4a --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/HiveUtil.java @@ -0,0 +1,125 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.utils; + +import com.uber.hoodie.common.model.HoodieTableMetadata; +import com.uber.hoodie.hadoop.HoodieInputFormat; +import org.apache.commons.dbcp.BasicDataSource; +import org.joda.time.DateTime; + +import javax.sql.DataSource; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; + +public class HiveUtil { + private static String driverName = "org.apache.hive.jdbc.HiveDriver"; + + static { + try { + Class.forName(driverName); + } catch (ClassNotFoundException e) { + throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e); + } + } + + private static Connection connection; + + private static Connection getConnection(String jdbcUrl, String user, String pass) throws SQLException { + DataSource ds = getDatasource(jdbcUrl, user, pass); + return ds.getConnection(); + } + + private static DataSource getDatasource(String jdbcUrl, String user, String pass) { + BasicDataSource ds = new BasicDataSource(); + ds.setDriverClassName(driverName); + ds.setUrl(jdbcUrl); + ds.setUsername(user); + ds.setPassword(pass); + return ds; + } + + public static long countRecords(String jdbcUrl, HoodieTableMetadata source, String dbName, String user, String pass) throws SQLException { + Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass); + ResultSet rs = null; + Statement stmt = conn.createStatement(); + try { + //stmt.execute("set mapred.job.queue.name="); + stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat" ); + stmt.execute("set hive.stats.autogather=false" ); + System.out.println("Class " + HoodieInputFormat.class.getName()); + rs = stmt.executeQuery( + "select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + source + .getTableName()); + long count = -1; + if(rs.next()) { + count = rs.getLong("cnt"); + } + System.out.println("Total records in " + source.getTableName() + " is " + count); + return count; + } finally { + if (rs != null) { + rs.close(); + } + if (stmt != null) { + stmt.close(); + } + } + } + + public static long countRecords(String jdbcUrl, HoodieTableMetadata source, String srcDb, + int partitions, String user, String pass) throws SQLException { + DateTime dateTime = DateTime.now(); + String endDateStr = + dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + + String.format("%02d", dateTime.getDayOfMonth()); + dateTime = dateTime.minusDays(partitions); + String startDateStr = + dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" + + String.format("%02d", dateTime.getDayOfMonth()); + System.out.println("Start date " + startDateStr + " and end date " + endDateStr); + return countRecords(jdbcUrl, source, srcDb, startDateStr, endDateStr, user, pass); + } + + private static long countRecords(String jdbcUrl, HoodieTableMetadata source, String srcDb, String startDateStr, + String endDateStr, String user, String pass) throws SQLException { + Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass); + ResultSet rs = null; + Statement stmt = conn.createStatement(); + try { + //stmt.execute("set mapred.job.queue.name="); + stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat"); + stmt.execute("set hive.stats.autogather=false"); + rs = stmt.executeQuery( + "select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source + .getTableName() + " where datestr>'" + startDateStr + "' and datestr<='" + + endDateStr + "'"); + if(rs.next()) { + return rs.getLong("cnt"); + } + return -1; + } finally { + if (rs != null) { + rs.close(); + } + if (stmt != null) { + stmt.close(); + } + } + } +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java new file mode 100644 index 000000000..8da872ef3 --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/InputStreamConsumer.java @@ -0,0 +1,56 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.utils; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.logging.Logger; + +public class InputStreamConsumer extends Thread { + protected final static Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName()); + private InputStream is; + public InputStreamConsumer(InputStream is) { + this.is = is; + } + + @Override + public void run() { + try { + InputStreamReader isr = new InputStreamReader(is); + BufferedReader br = new BufferedReader(isr); + String line; + while ( (line = br.readLine()) != null) + LOG.info(line); + } catch (IOException ioe) { + LOG.severe(ioe.toString()); + ioe.printStackTrace(); + } + } + + public static void captureOutput(Process p) { + InputStreamConsumer stdout; + InputStreamConsumer errout; + errout = new InputStreamConsumer(p.getErrorStream()); + stdout = new InputStreamConsumer(p.getInputStream()); + errout.start(); + stdout.start(); + } + + +} diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java new file mode 100644 index 000000000..d2d5e4c8f --- /dev/null +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java @@ -0,0 +1,75 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli.utils; + +import com.uber.hoodie.HoodieWriteClient; +import com.uber.hoodie.cli.commands.SparkMain; + +import org.apache.log4j.Logger; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.launcher.SparkLauncher; + +import java.io.File; +import java.net.URISyntaxException; + +public class SparkUtil { + + public static Logger logger = Logger.getLogger(SparkUtil.class); + + /** + * + * TODO: Need to fix a bunch of hardcoded stuff here eg: history server, spark distro + * + * @return + * @throws URISyntaxException + */ + public static SparkLauncher initLauncher(String propertiesFile) throws URISyntaxException { + String currentJar = new File( + SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath()) + .getAbsolutePath(); + SparkLauncher sparkLauncher = + new SparkLauncher().setAppResource(currentJar) + .setMainClass(SparkMain.class.getName()) + .setPropertiesFile(propertiesFile); + File libDirectory = new File(new File(currentJar).getParent(), "lib"); + for (String library : libDirectory.list()) { + sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath()); + } + return sparkLauncher; + } + + public static JavaSparkContext initJavaSparkConf(String name) { + SparkConf sparkConf = new SparkConf().setAppName(name); + sparkConf.setMaster("yarn-client"); + sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + sparkConf.set("spark.driver.maxResultSize", "2g"); + sparkConf.set("spark.eventLog.overwrite", "true"); + sparkConf.set("spark.eventLog.enabled", "true"); + + // Configure hadoop conf + sparkConf.set("spark.hadoop.mapred.output.compress", "true"); + sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true"); + sparkConf.set("spark.hadoop.mapred.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec"); + sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK"); + + sparkConf = HoodieWriteClient.registerClasses(sparkConf); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false); + return jsc; + } +} diff --git a/hoodie-cli/src/main/resources/META-INF/spring/spring-shell-plugin.xml b/hoodie-cli/src/main/resources/META-INF/spring/spring-shell-plugin.xml new file mode 100644 index 000000000..900c41dd7 --- /dev/null +++ b/hoodie-cli/src/main/resources/META-INF/spring/spring-shell-plugin.xml @@ -0,0 +1,26 @@ + + + + + + + + diff --git a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala new file mode 100644 index 000000000..7bacade3a --- /dev/null +++ b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala @@ -0,0 +1,179 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli + +import com.uber.hoodie.common.model.{HoodieRecord, HoodieTableMetadata} +import com.uber.hoodie.common.util.FSUtils +import com.uber.hoodie.exception.HoodieException +import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.apache.log4j.Logger +import org.apache.spark.sql.{DataFrame, SQLContext} + +import scala.collection.JavaConversions._ +import scala.collection.mutable._ + + +/** + * Spark job to de-duplicate data present in a partition path + */ +class DedupeSparkJob (basePath: String, + duplicatedPartitionPath: String, + repairOutputPath: String, + sqlContext: SQLContext, + fs: FileSystem) { + + + val sparkHelper = new SparkHelper(sqlContext, fs) + val LOG = Logger.getLogger(this.getClass) + + + /** + * + * @param tblName + * @return + */ + def getDupeKeyDF(tblName: String) : DataFrame = { + val dupeSql = s""" + select `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` as dupe_key, + count(*) as dupe_cnt + from ${tblName} + group by `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` + having dupe_cnt > 1 + """ + return sqlContext.sql(dupeSql) + } + + + /** + * + * Check a given partition for duplicates and suggest the deletions that need to be done in each file, + * in order to set things right. + * + * @return + */ + private def planDuplicateFix() : HashMap[String, HashSet[String]] = { + + val tmpTableName = s"htbl_${System.currentTimeMillis()}" + val dedupeTblName = s"${tmpTableName}_dupeKeys" + + val metadata = new HoodieTableMetadata(fs, basePath) + val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}")) + val filteredStatuses = metadata.getLatestVersions(allFiles).map(f => f.getPath.toString); + LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}") + + val df = sqlContext.parquetFile(filteredStatuses:_*) + df.registerTempTable(tmpTableName) + val dupeKeyDF = getDupeKeyDF(tmpTableName) + dupeKeyDF.registerTempTable(dedupeTblName) + + // Obtain necessary satellite information for duplicate rows + val dupeDataSql = s""" + SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time` + FROM ${tmpTableName} h + JOIN ${dedupeTblName} d + ON h.`_hoodie_record_key` = d.dupe_key + """ + val dupeMap = sqlContext.sql(dupeDataSql).collectAsList().groupBy(r => r.getString(0)) + val fileToDeleteKeyMap = new HashMap[String, HashSet[String]]() + + // Mark all files except the one with latest commits for deletion + dupeMap.foreach(rt => { + val key = rt._1 + val rows = rt._2 + var maxCommit = -1L + + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c > maxCommit) + maxCommit = c + }) + + rows.foreach(r => { + val c = r(3).asInstanceOf[String].toLong + if (c != maxCommit){ + val f = r(2).asInstanceOf[String].split("_")(0) + if (!fileToDeleteKeyMap.contains(f)){ + fileToDeleteKeyMap(f) = HashSet[String]() + } + fileToDeleteKeyMap(f).add(key) + } + }) + }) + return fileToDeleteKeyMap + } + + + def fixDuplicates(dryRun: Boolean = true) = { + val metadata = new HoodieTableMetadata(fs, basePath) + val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}")) + val fileNameToPathMap = metadata.getLatestVersions(allFiles).map(f => (FSUtils.getFileId(f.getPath.getName), f.getPath)).toMap; + val dupeFixPlan = planDuplicateFix() + + // 1. Copy all latest files into the temp fix path + fileNameToPathMap.foreach{ case(fileName, filePath) => { + val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else "" + val dstPath = new Path(s"${repairOutputPath}/${filePath.getName}${badSuffix}") + LOG.info(s"Copying from ${filePath} to ${dstPath}") + FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf) + }} + + // 2. Remove duplicates from the bad files + dupeFixPlan.foreach{case(fileName, keysToSkip) => { + val commitTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName) + val badFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}.bad") + val newFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}") + LOG.info(" Skipping and writing new file for : " + fileName) + SparkHelpers.skipKeysAndWriteNewFile(commitTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName)) + fs.delete(badFilePath, false) + }} + + // 3. Check that there are no duplicates anymore. + val df = sqlContext.read.parquet(s"${repairOutputPath}/*.parquet") + df.registerTempTable("fixedTbl") + val dupeKeyDF = getDupeKeyDF("fixedTbl") + val dupeCnt = dupeKeyDF.count(); + if (dupeCnt != 0) { + dupeKeyDF.show() + throw new HoodieException("Still found some duplicates!!.. Inspect output") + } + + // 4. Additionally ensure no record keys are left behind. + val sourceDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => t._2.toString).toList) + val fixedDF = sparkHelper.getDistinctKeyDF(fileNameToPathMap.map(t => s"${repairOutputPath}/${t._2.getName}").toList) + val missedRecordKeysDF = sourceDF.except(fixedDF) + val missedCnt = missedRecordKeysDF.count() + if (missedCnt != 0) { + missedRecordKeysDF.show() + throw new HoodieException("Some records in source are not found in fixed files. Inspect output!!") + } + + + println("No duplicates found & counts are in check!!!! ") + // 4. Prepare to copy the fixed files back. + fileNameToPathMap.foreach { case (fileName, filePath) => { + val srcPath = new Path(s"${repairOutputPath}/${filePath.getName}") + val dstPath = new Path(s"${basePath}/${duplicatedPartitionPath}/${filePath.getName}") + if (dryRun) { + LOG.info(s"[JUST KIDDING!!!] Copying from ${srcPath} to ${dstPath}") + } else { + // for real + LOG.info(s"[FOR REAL!!!] Copying from ${srcPath} to ${dstPath}") + FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf) + } + }} + } +} diff --git a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala new file mode 100644 index 000000000..0eccd9bf6 --- /dev/null +++ b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala @@ -0,0 +1,141 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.cli + +import com.uber.hoodie.avro.HoodieAvroWriteSupport +import com.uber.hoodie.common.BloomFilter +import com.uber.hoodie.common.model.HoodieRecord +import com.uber.hoodie.common.util.ParquetUtils +import com.uber.hoodie.config.{HoodieIndexConfig, HoodieStorageConfig} +import com.uber.hoodie.io.storage.{HoodieParquetConfig, HoodieParquetWriter} +import com.uber.hoodie.stream.GenericHoodiePayload +import org.apache.avro.Schema +import org.apache.avro.generic.IndexedRecord +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.parquet.avro.AvroSchemaConverter +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.hadoop.metadata.CompressionCodecName +import org.apache.spark.sql.{DataFrame, SQLContext} + +import scala.collection.JavaConversions._ +import scala.collection.mutable._ + + +object SparkHelpers { + @throws[Exception] + def skipKeysAndWriteNewFile(commitTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) { + val sourceRecords = ParquetUtils.readAvroRecords(sourceFile) + val schema: Schema = sourceRecords.get(0).getSchema + val filter: BloomFilter = new BloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble) + val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter) + val parquetConfig: HoodieParquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf) + val writer = new HoodieParquetWriter[GenericHoodiePayload, IndexedRecord](commitTime, destinationFile, parquetConfig, schema) + for (rec <- sourceRecords) { + val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString + if (!keysToSkip.contains(key)) { + writer.writeAvro(key, rec) + } + } + writer.close + } + + def getBloomFilter(file: String, conf: Configuration): String = { + val footer = ParquetFileReader.readFooter(conf, new Path(file)); + return footer.getFileMetaData().getKeyValueMetaData().get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY) + } +} + + +/** + * Bunch of Spark Shell/Scala stuff useful for debugging + */ +class SparkHelper(sqlContext: SQLContext, fs: FileSystem) { + + + /** + * Print keys from a file + * + * @param file + */ + def printKeysFromFile(file: String) = { + getRowKeyDF(file).collect().foreach(println(_)) + } + + /** + * + * @param file + * @return + */ + def getRowKeyDF(file: String): DataFrame = { + sqlContext.read.parquet(file).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`") + } + + + /** + * Does the rowKey actually exist in the file. + * + * @param rowKey + * @param file + * @return + */ + def isFileContainsKey(rowKey: String, file: String): Boolean = { + println(s"Checking ${file} for key ${rowKey}") + val ff = getRowKeyDF(file).filter(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}` = '${rowKey}'") + if (ff.count() > 0) + return true + else + return false + } + + /** + * Number of keys in a given file + * + * @param file + * @param sqlContext + */ + def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) ={ + println(getRowKeyDF(file).collect().size) + } + + + /** + * + * Checks that all the keys in the file, have been added to the bloom filter + * in the footer + * + * @param conf + * @param sqlContext + * @param file + * @return + */ + def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String) : Boolean = { + val bfStr = SparkHelpers.getBloomFilter(file, conf) + val bf = new com.uber.hoodie.common.BloomFilter(bfStr) + val foundCount = sqlContext.parquetFile(file) + .select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`") + .collect(). + filter(r => !bf.mightContain(r.getString(0))).size + val totalCount = getKeyCount(file, sqlContext) + s"totalCount: ${totalCount}, foundCount: ${foundCount}" + totalCount == foundCount + } + + def getDistinctKeyDF(paths: List[String]) : DataFrame = { + sqlContext.read.parquet(paths:_*).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`").distinct() + } +} diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml new file mode 100644 index 000000000..83158bfce --- /dev/null +++ b/hoodie-common/pom.xml @@ -0,0 +1,94 @@ + + + + + + hoodie + com.uber.hoodie + 0.2.5-SNAPSHOT + + 4.0.0 + + hoodie-common + + + + + org.codehaus.mojo + cobertura-maven-plugin + + + org.apache.maven.plugins + maven-jar-plugin + 2.5 + + + + test-jar + + + + + + org.apache.rat + apache-rat-plugin + + + + + + + org.apache.avro + avro + + + org.apache.hadoop + hadoop-client + + + javax.servlet + * + + + + + junit + junit + ${junit.version} + test + + + com.fasterxml.jackson.core + jackson-annotations + + + org.codehaus.jackson + jackson-mapper-asl + + + org.apache.parquet + parquet-avro + ${parquet.version} + + + org.mockito + mockito-all + 1.10.19 + test + + + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java b/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java new file mode 100644 index 000000000..5a5191655 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/avro/HoodieAvroWriteSupport.java @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.avro; + +import com.uber.hoodie.common.BloomFilter; + +import org.apache.avro.Schema; +import org.apache.parquet.avro.AvroWriteSupport; +import org.apache.parquet.hadoop.api.WriteSupport; +import org.apache.parquet.schema.MessageType; + +import java.io.*; +import java.util.HashMap; + +/** + * Wrap AvroWriterSupport for plugging in the bloom filter. + */ +public class HoodieAvroWriteSupport extends AvroWriteSupport { + private BloomFilter bloomFilter; + public final static String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY = + "com.uber.hoodie.bloomfilter"; + + public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, BloomFilter bloomFilter) { + super(schema, avroSchema); + this.bloomFilter = bloomFilter; + } + + @Override public WriteSupport.FinalizedWriteContext finalizeWrite() { + HashMap extraMetaData = new HashMap<>(); + if (bloomFilter != null) { + extraMetaData + .put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString()); + } + return new WriteSupport.FinalizedWriteContext(extraMetaData); + } + + public void add(String recordKey) { + this.bloomFilter.add(recordKey); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java new file mode 100644 index 000000000..90e5257a0 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java @@ -0,0 +1,166 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.avro; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.codehaus.jackson.map.ObjectMapper; + +/** + * Marjority of this is copied from + * https://github.com/jwills/avro-json/blob/master/src/main/java/com/cloudera/science/avro/common/JsonConverter.java + * Adjusted for expected behavior of our use cases + */ +public class MercifulJsonConverter { + private final ObjectMapper mapper = new ObjectMapper(); + private final Schema baseSchema; + + public MercifulJsonConverter(Schema schema) { + this.baseSchema = schema; + } + + + public GenericRecord convert(String json) throws IOException { + try { + return convert(mapper.readValue(json, Map.class), baseSchema); + } catch (IOException e) { + throw new IOException("Failed to parse as Json: " + json + "\n\n" + e.getMessage()); + } + } + + private GenericRecord convert(Map raw, Schema schema) + throws IOException { + GenericRecord result = new GenericData.Record(schema); + for (Schema.Field f : schema.getFields()) { + String name = f.name(); + Object rawValue = raw.get(name); + if (rawValue != null) { + result.put(f.pos(), typeConvert(rawValue, name, f.schema())); + } + } + + return result; + } + + private Object typeConvert(Object value, String name, Schema schema) throws IOException { + if (isOptional(schema)) { + if (value == null) { + return null; + } else { + schema = getNonNull(schema); + } + } else if (value == null) { + // Always fail on null for non-nullable schemas + throw new JsonConversionException(null, name, schema); + } + + switch (schema.getType()) { + case BOOLEAN: + if (value instanceof Boolean) { + return (Boolean) value; + } + break; + case DOUBLE: + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + break; + case FLOAT: + if (value instanceof Number) { + return ((Number) value).floatValue(); + } + break; + case INT: + if (value instanceof Number) { + return ((Number) value).intValue(); + } + break; + case LONG: + if (value instanceof Number) { + return ((Number) value).longValue(); + } + break; + case STRING: + return value.toString(); + case ENUM: + if (schema.getEnumSymbols().contains(value.toString())) { + return new GenericData.EnumSymbol(schema, value.toString()); + } + throw new JsonConversionException(String.format("Symbol %s not in enum", value.toString()), + schema.getFullName(), schema); + case RECORD: + return convert((Map) value, schema); + case ARRAY: + Schema elementSchema = schema.getElementType(); + List listRes = new ArrayList(); + for (Object v : (List) value) { + listRes.add(typeConvert(v, name, elementSchema)); + } + return listRes; + case MAP: + Schema valueSchema = schema.getValueType(); + Map mapRes = new HashMap(); + for (Map.Entry v : ((Map) value).entrySet()) { + mapRes.put(v.getKey(), typeConvert(v.getValue(), name, valueSchema)); + } + return mapRes; + default: + throw new IllegalArgumentException( + "JsonConverter cannot handle type: " + schema.getType()); + } + throw new JsonConversionException(value, name, schema); + } + + private boolean isOptional(Schema schema) { + return schema.getType().equals(Schema.Type.UNION) && + schema.getTypes().size() == 2 && + (schema.getTypes().get(0).getType().equals(Schema.Type.NULL) || + schema.getTypes().get(1).getType().equals(Schema.Type.NULL)); + } + + private Schema getNonNull(Schema schema) { + List types = schema.getTypes(); + return types.get(0).getType().equals(Schema.Type.NULL) ? types.get(1) : types.get(0); + } + + public static class JsonConversionException extends RuntimeException { + + private Object value; + private String fieldName; + private Schema schema; + + public JsonConversionException(Object value, String fieldName, Schema schema) { + this.value = value; + this.fieldName = fieldName; + this.schema = schema; + } + + @Override + public String toString() { + return String.format("Type conversion error for field %s, %s for %s", + fieldName, value, schema); + } + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/BloomFilter.java b/hoodie-common/src/main/java/com/uber/hoodie/common/BloomFilter.java new file mode 100644 index 000000000..d81e31df3 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/BloomFilter.java @@ -0,0 +1,100 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common; + +import com.uber.hoodie.exception.HoodieIndexException; +import org.apache.commons.io.output.ByteArrayOutputStream; +import org.apache.hadoop.util.bloom.Key; +import org.apache.hadoop.util.hash.Hash; + +import javax.xml.bind.DatatypeConverter; + +import java.io.*; +import java.nio.charset.StandardCharsets; + +/** + * A Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}. + */ +public class BloomFilter { + /** + * Used in computing the optimal Bloom filter size. This approximately equals 0.480453. + */ + public static final double LOG2_SQUARED = Math.log(2) * Math.log(2); + + private org.apache.hadoop.util.bloom.BloomFilter filter = null; + + public BloomFilter(int numEntries, double errorRate) { + this(numEntries, errorRate, Hash.MURMUR_HASH); + } + + /** + * Create a new Bloom filter with the given configurations. + */ + public BloomFilter(int numEntries, double errorRate, int hashType) { + // Bit size + int bitSize = (int) Math.ceil(numEntries * (-Math.log(errorRate) / LOG2_SQUARED)); + // Number of the hash functions + int numHashs = (int) Math.ceil(Math.log(2) * bitSize / numEntries); + // The filter + this.filter = new org.apache.hadoop.util.bloom.BloomFilter(bitSize, numHashs, hashType); + } + + /** + * Create the bloom filter from serialized string. + */ + public BloomFilter(String filterStr) { + this.filter = new org.apache.hadoop.util.bloom.BloomFilter(); + byte[] bytes = DatatypeConverter.parseBase64Binary(filterStr); + DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes)); + try { + this.filter.readFields(dis); + dis.close(); + } catch (IOException e) { + throw new HoodieIndexException("Could not deserialize BloomFilter instance", e); + } + } + + public void add(String key) { + if (key == null) { + throw new NullPointerException("Key cannot by null"); + } + filter.add(new Key(key.getBytes(StandardCharsets.UTF_8))); + } + + public boolean mightContain(String key) { + if (key == null) { + throw new NullPointerException("Key cannot by null"); + } + return filter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8))); + } + + /** + * Serialize the bloom filter as a string. + */ + public String serializeToString() { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos); + try { + filter.write(dos); + byte[] bytes = baos.toByteArray(); + dos.close(); + return DatatypeConverter.printBase64Binary(bytes); + } catch (IOException e) { + throw new HoodieIndexException("Could not serialize BloomFilter instance", e); + } + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/file/HoodieAppendLog.java b/hoodie-common/src/main/java/com/uber/hoodie/common/file/HoodieAppendLog.java new file mode 100644 index 000000000..6eab1be12 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/file/HoodieAppendLog.java @@ -0,0 +1,3826 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.file; + + +import java.io.*; +import java.util.*; +import java.rmi.server.UID; +import java.security.MessageDigest; + +import org.apache.commons.logging.*; +import org.apache.hadoop.io.DataInputBuffer; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.RawComparator; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.UTF8; +import org.apache.hadoop.io.VersionMismatchException; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.io.WritableComparator; +import org.apache.hadoop.io.WritableName; +import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.util.Options; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.fs.Options.CreateOpts; +import org.apache.hadoop.io.compress.CodecPool; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.Decompressor; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.io.compress.zlib.ZlibFactory; +import org.apache.hadoop.io.serializer.Deserializer; +import org.apache.hadoop.io.serializer.Serializer; +import org.apache.hadoop.io.serializer.SerializationFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.*; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.Progress; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.NativeCodeLoader; +import org.apache.hadoop.util.MergeSort; +import org.apache.hadoop.util.PriorityQueue; +import org.apache.hadoop.util.Time; + +/** + * SequenceFiles are flat files consisting of binary key/value + * pairs. + * + *

SequenceFile provides {@link HoodieAppendLog.Writer}, + * {@link HoodieAppendLog.Reader} and {@link Sorter} classes for writing, + * reading and sorting respectively.

+ * + * There are three SequenceFile Writers based on the + * {@link CompressionType} used to compress key/value pairs: + *
    + *
  1. + * Writer : Uncompressed records. + *
  2. + *
  3. + * RecordCompressWriter : Record-compressed files, only compress + * values. + *
  4. + *
  5. + * BlockCompressWriter : Block-compressed files, both keys & + * values are collected in 'blocks' + * separately and compressed. The size of + * the 'block' is configurable. + *
+ * + *

The actual compression algorithm used to compress key and/or values can be + * specified by using the appropriate {@link CompressionCodec}.

+ * + *

The recommended way is to use the static createWriter methods + * provided by the SequenceFile to chose the preferred format.

+ * + *

The {@link HoodieAppendLog.Reader} acts as the bridge and can read any of the + * above SequenceFile formats.

+ * + *

SequenceFile Formats

+ * + *

Essentially there are 3 different formats for SequenceFiles + * depending on the CompressionType specified. All of them share a + * common header described below. + * + *

+ *
    + *
  • + * version - 3 bytes of magic header SEQ, followed by 1 byte of actual + * version number (e.g. SEQ4 or SEQ6) + *
  • + *
  • + * keyClassName -key class + *
  • + *
  • + * valueClassName - value class + *
  • + *
  • + * compression - A boolean which specifies if compression is turned on for + * keys/values in this file. + *
  • + *
  • + * blockCompression - A boolean which specifies if block-compression is + * turned on for keys/values in this file. + *
  • + *
  • + * compression codec - CompressionCodec class which is used for + * compression of keys and/or values (if compression is + * enabled). + *
  • + *
  • + * metadata - {@link Metadata} for this file. + *
  • + *
  • + * sync - A sync marker to denote end of the header. + *
  • + *
+ * + *
Uncompressed SequenceFile Format
+ *
    + *
  • + * Header + *
  • + *
  • + * Record + *
      + *
    • Record length
    • + *
    • Key length
    • + *
    • Key
    • + *
    • Value
    • + *
    + *
  • + *
  • + * A sync-marker every few 100 bytes or so. + *
  • + *
+ * + *
Record-Compressed SequenceFile Format
+ *
    + *
  • + * Header + *
  • + *
  • + * Record + *
      + *
    • Record length
    • + *
    • Key length
    • + *
    • Key
    • + *
    • Compressed Value
    • + *
    + *
  • + *
  • + * A sync-marker every few 100 bytes or so. + *
  • + *
+ * + *
Block-Compressed SequenceFile Format
+ *
    + *
  • + * Header + *
  • + *
  • + * Record Block + *
      + *
    • Uncompressed number of records in the block
    • + *
    • Compressed key-lengths block-size
    • + *
    • Compressed key-lengths block
    • + *
    • Compressed keys block-size
    • + *
    • Compressed keys block
    • + *
    • Compressed value-lengths block-size
    • + *
    • Compressed value-lengths block
    • + *
    • Compressed values block-size
    • + *
    • Compressed values block
    • + *
    + *
  • + *
  • + * A sync-marker every block. + *
  • + *
+ * + *

The compressed blocks of key lengths and value lengths consist of the + * actual lengths of individual keys/values encoded in ZeroCompressedInteger + * format.

+ * + * @see CompressionCodec + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class HoodieAppendLog { + private static final Log LOG = LogFactory.getLog(HoodieAppendLog.class); + + private HoodieAppendLog() { + } // no public ctor + + private static final byte BLOCK_COMPRESS_VERSION = (byte) 4; + private static final byte CUSTOM_COMPRESS_VERSION = (byte) 5; + private static final byte VERSION_WITH_METADATA = (byte) 6; + private static byte[] VERSION = new byte[]{ + (byte) 'S', (byte) 'E', (byte) 'Q', VERSION_WITH_METADATA + }; + + private static final int SYNC_ESCAPE = -1; // "length" of sync entries + private static final int SYNC_HASH_SIZE = 16; // number of bytes in hash + private static final int SYNC_SIZE = 4 + SYNC_HASH_SIZE; // escape + hash + + /** The number of bytes between sync points.*/ + public static final int SYNC_INTERVAL = 100 * SYNC_SIZE; + + /** + * The compression type used to compress key/value pairs in the + * {@link HoodieAppendLog}. + * + * @see HoodieAppendLog.Writer + */ + public static enum CompressionType { + /** Do not compress records. */ + NONE, + /** Compress values only, each separately. */ + RECORD, + /** Compress sequences of records together in blocks. */ + BLOCK + } + + /** + * Get the compression type for the reduce outputs + * @param job the job config to look in + * @return the kind of compression to use + */ + static public CompressionType getDefaultCompressionType(Configuration job) { + String name = job.get("io.seqfile.compression.type"); + return name == null ? CompressionType.RECORD : + CompressionType.valueOf(name); + } + + /** + * Set the default compression type for sequence files. + * @param job the configuration to modify + * @param val the new compression type (none, block, record) + */ + static public void setDefaultCompressionType(Configuration job, + CompressionType val) { + job.set("io.seqfile.compression.type", val.toString()); + } + + /** + * Create a new Writer with the given options. + * @param conf the configuration to use + * @param opts the options to create the file with + * @return a new Writer + * @throws IOException + */ + public static Writer createWriter(Configuration conf, Writer.Option... opts + ) throws IOException { + Writer.CompressionOption compressionOption = + Options.getOption(Writer.CompressionOption.class, opts); + CompressionType kind; + if (compressionOption != null) { + kind = compressionOption.getValue(); + } else { + kind = getDefaultCompressionType(conf); + opts = Options.prependOptions(opts, Writer.compression(kind)); + } + switch (kind) { + default: + case NONE: + return new Writer(conf, opts); + case RECORD: + return new RecordCompressWriter(conf, opts); + case BLOCK: + return new BlockCompressWriter(conf, opts); + } + } + + /** + * Construct the preferred type of SequenceFile Writer. + * @param fs The configured filesystem. + * @param conf The configuration. + * @param name The name of the file. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public static Writer + createWriter(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass) throws IOException { + return createWriter(conf, Writer.filesystem(fs), + Writer.file(name), Writer.keyClass(keyClass), + Writer.valueClass(valClass)); + } + + /** + * Construct the preferred type of SequenceFile Writer. + * @param fs The configured filesystem. + * @param conf The configuration. + * @param name The name of the file. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param compressionType The compression type. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public static Writer + createWriter(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass, + CompressionType compressionType) throws IOException { + return createWriter(conf, Writer.filesystem(fs), + Writer.file(name), Writer.keyClass(keyClass), + Writer.valueClass(valClass), + Writer.compression(compressionType)); + } + + /** + * Construct the preferred type of SequenceFile Writer. + * @param fs The configured filesystem. + * @param conf The configuration. + * @param name The name of the file. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param compressionType The compression type. + * @param progress The Progressable object to track progress. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public static Writer + createWriter(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass, CompressionType compressionType, + Progressable progress) throws IOException { + return createWriter(conf, Writer.file(name), + Writer.filesystem(fs), + Writer.keyClass(keyClass), + Writer.valueClass(valClass), + Writer.compression(compressionType), + Writer.progressable(progress)); + } + + /** + * Construct the preferred type of SequenceFile Writer. + * @param fs The configured filesystem. + * @param conf The configuration. + * @param name The name of the file. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param compressionType The compression type. + * @param codec The compression codec. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public static Writer + createWriter(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass, CompressionType compressionType, + CompressionCodec codec) throws IOException { + return createWriter(conf, Writer.file(name), + Writer.filesystem(fs), + Writer.keyClass(keyClass), + Writer.valueClass(valClass), + Writer.compression(compressionType, codec)); + } + + /** + * Construct the preferred type of SequenceFile Writer. + * @param fs The configured filesystem. + * @param conf The configuration. + * @param name The name of the file. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param compressionType The compression type. + * @param codec The compression codec. + * @param progress The Progressable object to track progress. + * @param metadata The metadata of the file. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public static Writer + createWriter(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass, + CompressionType compressionType, CompressionCodec codec, + Progressable progress, Metadata metadata) throws IOException { + return createWriter(conf, Writer.file(name), + Writer.filesystem(fs), + Writer.keyClass(keyClass), + Writer.valueClass(valClass), + Writer.compression(compressionType, codec), + Writer.progressable(progress), + Writer.metadata(metadata)); + } + + /** + * Construct the preferred type of SequenceFile Writer. + * @param fs The configured filesystem. + * @param conf The configuration. + * @param name The name of the file. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param bufferSize buffer size for the underlaying outputstream. + * @param replication replication factor for the file. + * @param blockSize block size for the file. + * @param compressionType The compression type. + * @param codec The compression codec. + * @param progress The Progressable object to track progress. + * @param metadata The metadata of the file. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public static Writer + createWriter(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass, int bufferSize, + short replication, long blockSize, + CompressionType compressionType, CompressionCodec codec, + Progressable progress, Metadata metadata) throws IOException { + return createWriter(conf, Writer.file(name), + Writer.filesystem(fs), + Writer.keyClass(keyClass), + Writer.valueClass(valClass), + Writer.bufferSize(bufferSize), + Writer.replication(replication), + Writer.blockSize(blockSize), + Writer.compression(compressionType, codec), + Writer.progressable(progress), + Writer.metadata(metadata)); + } + + /** + * Construct the preferred type of SequenceFile Writer. + * @param fs The configured filesystem. + * @param conf The configuration. + * @param name The name of the file. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param bufferSize buffer size for the underlaying outputstream. + * @param replication replication factor for the file. + * @param blockSize block size for the file. + * @param createParent create parent directory if non-existent + * @param compressionType The compression type. + * @param codec The compression codec. + * @param metadata The metadata of the file. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + */ + @Deprecated + public static Writer + createWriter(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass, int bufferSize, + short replication, long blockSize, boolean createParent, + CompressionType compressionType, CompressionCodec codec, + Metadata metadata) throws IOException { + return createWriter(FileContext.getFileContext(fs.getUri(), conf), + conf, name, keyClass, valClass, compressionType, codec, + metadata, EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), + CreateOpts.bufferSize(bufferSize), + createParent ? CreateOpts.createParent() + : CreateOpts.donotCreateParent(), + CreateOpts.repFac(replication), + CreateOpts.blockSize(blockSize) + ); + } + + /** + * Construct the preferred type of SequenceFile Writer. + * @param fc The context for the specified file. + * @param conf The configuration. + * @param name The name of the file. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param compressionType The compression type. + * @param codec The compression codec. + * @param metadata The metadata of the file. + * @param createFlag gives the semantics of create: overwrite, append etc. + * @param opts file creation options; see {@link CreateOpts}. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + */ + public static Writer + createWriter(FileContext fc, Configuration conf, Path name, + Class keyClass, Class valClass, + CompressionType compressionType, CompressionCodec codec, + Metadata metadata, + final EnumSet createFlag, CreateOpts... opts) + throws IOException { + return createWriter(conf, fc.create(name, createFlag, opts), + keyClass, valClass, compressionType, codec, metadata).ownStream(); + } + + /** + * Construct the preferred type of SequenceFile Writer. + * @param fs The configured filesystem. + * @param conf The configuration. + * @param name The name of the file. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param compressionType The compression type. + * @param codec The compression codec. + * @param progress The Progressable object to track progress. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public static Writer + createWriter(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass, + CompressionType compressionType, CompressionCodec codec, + Progressable progress) throws IOException { + return createWriter(conf, Writer.file(name), + Writer.filesystem(fs), + Writer.keyClass(keyClass), + Writer.valueClass(valClass), + Writer.compression(compressionType, codec), + Writer.progressable(progress)); + } + + /** + * Construct the preferred type of 'raw' SequenceFile Writer. + * @param conf The configuration. + * @param out The stream on top which the writer is to be constructed. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param compressionType The compression type. + * @param codec The compression codec. + * @param metadata The metadata of the file. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public static Writer + createWriter(Configuration conf, FSDataOutputStream out, + Class keyClass, Class valClass, + CompressionType compressionType, + CompressionCodec codec, Metadata metadata) throws IOException { + return createWriter(conf, Writer.stream(out), Writer.keyClass(keyClass), + Writer.valueClass(valClass), + Writer.compression(compressionType, codec), + Writer.metadata(metadata)); + } + + /** + * Construct the preferred type of 'raw' SequenceFile Writer. + * @param conf The configuration. + * @param out The stream on top which the writer is to be constructed. + * @param keyClass The 'key' type. + * @param valClass The 'value' type. + * @param compressionType The compression type. + * @param codec The compression codec. + * @return Returns the handle to the constructed SequenceFile Writer. + * @throws IOException + * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public static Writer + createWriter(Configuration conf, FSDataOutputStream out, + Class keyClass, Class valClass, CompressionType compressionType, + CompressionCodec codec) throws IOException { + return createWriter(conf, Writer.stream(out), Writer.keyClass(keyClass), + Writer.valueClass(valClass), + Writer.compression(compressionType, codec)); + } + + + /** The interface to 'raw' values of SequenceFiles. */ + public static interface ValueBytes { + + /** Writes the uncompressed bytes to the outStream. + * @param outStream : Stream to write uncompressed bytes into. + * @throws IOException + */ + public void writeUncompressedBytes(DataOutputStream outStream) + throws IOException; + + /** Write compressed bytes to outStream. + * Note: that it will NOT compress the bytes if they are not compressed. + * @param outStream : Stream to write compressed bytes into. + */ + public void writeCompressedBytes(DataOutputStream outStream) + throws IllegalArgumentException, IOException; + + /** + * Size of stored data. + */ + public int getSize(); + } + + private static class UncompressedBytes implements ValueBytes { + private int dataSize; + private byte[] data; + + private UncompressedBytes() { + data = null; + dataSize = 0; + } + + private void reset(DataInputStream in, int length) throws IOException { + if (data == null) { + data = new byte[length]; + } else if (length > data.length) { + data = new byte[Math.max(length, data.length * 2)]; + } + dataSize = -1; + in.readFully(data, 0, length); + dataSize = length; + } + + @Override + public int getSize() { + return dataSize; + } + + @Override + public void writeUncompressedBytes(DataOutputStream outStream) + throws IOException { + outStream.write(data, 0, dataSize); + } + + @Override + public void writeCompressedBytes(DataOutputStream outStream) + throws IllegalArgumentException, IOException { + throw + new IllegalArgumentException("UncompressedBytes cannot be compressed!"); + } + + } // UncompressedBytes + + private static class CompressedBytes implements ValueBytes { + private int dataSize; + private byte[] data; + DataInputBuffer rawData = null; + CompressionCodec codec = null; + CompressionInputStream decompressedStream = null; + + private CompressedBytes(CompressionCodec codec) { + data = null; + dataSize = 0; + this.codec = codec; + } + + private void reset(DataInputStream in, int length) throws IOException { + if (data == null) { + data = new byte[length]; + } else if (length > data.length) { + data = new byte[Math.max(length, data.length * 2)]; + } + dataSize = -1; + in.readFully(data, 0, length); + dataSize = length; + } + + @Override + public int getSize() { + return dataSize; + } + + @Override + public void writeUncompressedBytes(DataOutputStream outStream) + throws IOException { + if (decompressedStream == null) { + rawData = new DataInputBuffer(); + decompressedStream = codec.createInputStream(rawData); + } else { + decompressedStream.resetState(); + } + rawData.reset(data, 0, dataSize); + + byte[] buffer = new byte[8192]; + int bytesRead = 0; + while ((bytesRead = decompressedStream.read(buffer, 0, 8192)) != -1) { + outStream.write(buffer, 0, bytesRead); + } + } + + @Override + public void writeCompressedBytes(DataOutputStream outStream) + throws IllegalArgumentException, IOException { + outStream.write(data, 0, dataSize); + } + + } // CompressedBytes + + /** + * The class encapsulating with the metadata of a file. + * The metadata of a file is a list of attribute name/value + * pairs of Text type. + * + */ + public static class Metadata implements Writable { + + private TreeMap theMetadata; + + public Metadata() { + this(new TreeMap()); + } + + public Metadata(TreeMap arg) { + if (arg == null) { + this.theMetadata = new TreeMap(); + } else { + this.theMetadata = arg; + } + } + + public Text get(Text name) { + return this.theMetadata.get(name); + } + + public void set(Text name, Text value) { + this.theMetadata.put(name, value); + } + + public TreeMap getMetadata() { + return new TreeMap(this.theMetadata); + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(this.theMetadata.size()); + Iterator> iter = + this.theMetadata.entrySet().iterator(); + while (iter.hasNext()) { + Map.Entry en = iter.next(); + en.getKey().write(out); + en.getValue().write(out); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + int sz = in.readInt(); + if (sz < 0) throw new IOException("Invalid size: " + sz + " for file metadata object"); + this.theMetadata = new TreeMap(); + for (int i = 0; i < sz; i++) { + Text key = new Text(); + Text val = new Text(); + key.readFields(in); + val.readFields(in); + this.theMetadata.put(key, val); + } + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass() != this.getClass()) { + return false; + } else { + return equals((Metadata) other); + } + } + + public boolean equals(Metadata other) { + if (other == null) return false; + if (this.theMetadata.size() != other.theMetadata.size()) { + return false; + } + Iterator> iter1 = + this.theMetadata.entrySet().iterator(); + Iterator> iter2 = + other.theMetadata.entrySet().iterator(); + while (iter1.hasNext() && iter2.hasNext()) { + Map.Entry en1 = iter1.next(); + Map.Entry en2 = iter2.next(); + if (!en1.getKey().equals(en2.getKey())) { + return false; + } + if (!en1.getValue().equals(en2.getValue())) { + return false; + } + } + if (iter1.hasNext() || iter2.hasNext()) { + return false; + } + return true; + } + + @Override + public int hashCode() { + assert false : "hashCode not designed"; + return 42; // any arbitrary constant will do + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("size: ").append(this.theMetadata.size()).append("\n"); + Iterator> iter = + this.theMetadata.entrySet().iterator(); + while (iter.hasNext()) { + Map.Entry en = iter.next(); + sb.append("\t").append(en.getKey().toString()).append("\t").append(en.getValue().toString()); + sb.append("\n"); + } + return sb.toString(); + } + } + + /** Write key/value pairs to a sequence-format file. */ + public static class Writer implements java.io.Closeable, Syncable { + private Configuration conf; + FSDataOutputStream out; + boolean ownOutputStream = true; + DataOutputBuffer buffer = new DataOutputBuffer(); + + Class keyClass; + Class valClass; + + private final CompressionType compress; + CompressionCodec codec = null; + CompressionOutputStream deflateFilter = null; + DataOutputStream deflateOut = null; + Metadata metadata = null; + Compressor compressor = null; + + private boolean appendMode = false; + + protected Serializer keySerializer; + protected Serializer uncompressedValSerializer; + protected Serializer compressedValSerializer; + + // Insert a globally unique 16-byte value every few entries, so that one + // can seek into the middle of a file and then synchronize with record + // starts and ends by scanning for this value. + long lastSyncPos; // position of last sync + byte[] sync; // 16 random bytes + + { + try { + MessageDigest digester = MessageDigest.getInstance("MD5"); + long time = Time.now(); + digester.update((new UID() + "@" + time).getBytes()); + sync = digester.digest(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static interface Option { + } + + static class FileOption extends Options.PathOption + implements Option { + FileOption(Path path) { + super(path); + } + } + + /** + * @deprecated only used for backwards-compatibility in the createWriter methods + * that take FileSystem. + */ + @Deprecated + private static class FileSystemOption implements Option { + private final FileSystem value; + + protected FileSystemOption(FileSystem value) { + this.value = value; + } + + public FileSystem getValue() { + return value; + } + } + + static class StreamOption extends Options.FSDataOutputStreamOption + implements Option { + StreamOption(FSDataOutputStream stream) { + super(stream); + } + } + + static class BufferSizeOption extends Options.IntegerOption + implements Option { + BufferSizeOption(int value) { + super(value); + } + } + + static class BlockSizeOption extends Options.LongOption implements Option { + BlockSizeOption(long value) { + super(value); + } + } + + static class ReplicationOption extends Options.IntegerOption + implements Option { + ReplicationOption(int value) { + super(value); + } + } + + static class AppendIfExistsOption extends Options.BooleanOption implements + Option { + AppendIfExistsOption(boolean value) { + super(value); + } + } + + static class KeyClassOption extends Options.ClassOption implements Option { + KeyClassOption(Class value) { + super(value); + } + } + + static class ValueClassOption extends Options.ClassOption + implements Option { + ValueClassOption(Class value) { + super(value); + } + } + + static class MetadataOption implements Option { + private final Metadata value; + + MetadataOption(Metadata value) { + this.value = value; + } + + Metadata getValue() { + return value; + } + } + + static class ProgressableOption extends Options.ProgressableOption + implements Option { + ProgressableOption(Progressable value) { + super(value); + } + } + + private static class CompressionOption implements Option { + private final CompressionType value; + private final CompressionCodec codec; + + CompressionOption(CompressionType value) { + this(value, null); + } + + CompressionOption(CompressionType value, CompressionCodec codec) { + this.value = value; + this.codec = (CompressionType.NONE != value && null == codec) + ? new DefaultCodec() + : codec; + } + + CompressionType getValue() { + return value; + } + + CompressionCodec getCodec() { + return codec; + } + } + + public static Option file(Path value) { + return new FileOption(value); + } + + /** + * @deprecated only used for backwards-compatibility in the createWriter methods + * that take FileSystem. + */ + @Deprecated + private static Option filesystem(FileSystem fs) { + return new HoodieAppendLog.Writer.FileSystemOption(fs); + } + + public static Option bufferSize(int value) { + return new BufferSizeOption(value); + } + + public static Option stream(FSDataOutputStream value) { + return new StreamOption(value); + } + + public static Option replication(short value) { + return new ReplicationOption(value); + } + + public static Option appendIfExists(boolean value) { + return new AppendIfExistsOption(value); + } + + public static Option blockSize(long value) { + return new BlockSizeOption(value); + } + + public static Option progressable(Progressable value) { + return new ProgressableOption(value); + } + + public static Option keyClass(Class value) { + return new KeyClassOption(value); + } + + public static Option valueClass(Class value) { + return new ValueClassOption(value); + } + + public static Option metadata(Metadata value) { + return new MetadataOption(value); + } + + public static Option compression(CompressionType value) { + return new CompressionOption(value); + } + + public static Option compression(CompressionType value, + CompressionCodec codec) { + return new CompressionOption(value, codec); + } + + /** + * Construct a uncompressed writer from a set of options. + * @param conf the configuration to use + * @param opts the options used when creating the writer + * @throws IOException if it fails + */ + Writer(Configuration conf, + Option... opts) throws IOException { + BlockSizeOption blockSizeOption = + Options.getOption(BlockSizeOption.class, opts); + BufferSizeOption bufferSizeOption = + Options.getOption(BufferSizeOption.class, opts); + ReplicationOption replicationOption = + Options.getOption(ReplicationOption.class, opts); + ProgressableOption progressOption = + Options.getOption(ProgressableOption.class, opts); + FileOption fileOption = Options.getOption(FileOption.class, opts); + AppendIfExistsOption appendIfExistsOption = Options.getOption( + AppendIfExistsOption.class, opts); + FileSystemOption fsOption = Options.getOption(FileSystemOption.class, opts); + StreamOption streamOption = Options.getOption(StreamOption.class, opts); + KeyClassOption keyClassOption = + Options.getOption(KeyClassOption.class, opts); + ValueClassOption valueClassOption = + Options.getOption(ValueClassOption.class, opts); + MetadataOption metadataOption = + Options.getOption(MetadataOption.class, opts); + CompressionOption compressionTypeOption = + Options.getOption(CompressionOption.class, opts); + // check consistency of options + if ((fileOption == null) == (streamOption == null)) { + throw new IllegalArgumentException("file or stream must be specified"); + } + if (fileOption == null && (blockSizeOption != null || + bufferSizeOption != null || + replicationOption != null || + progressOption != null)) { + throw new IllegalArgumentException("file modifier options not " + + "compatible with stream"); + } + + FSDataOutputStream out; + boolean ownStream = fileOption != null; + if (ownStream) { + Path p = fileOption.getValue(); + FileSystem fs; + if (fsOption != null) { + fs = fsOption.getValue(); + } else { + fs = p.getFileSystem(conf); + } + int bufferSize = bufferSizeOption == null ? getBufferSize(conf) : + bufferSizeOption.getValue(); + short replication = replicationOption == null ? + fs.getDefaultReplication(p) : + (short) replicationOption.getValue(); + long blockSize = blockSizeOption == null ? fs.getDefaultBlockSize(p) : + blockSizeOption.getValue(); + Progressable progress = progressOption == null ? null : + progressOption.getValue(); + + if (appendIfExistsOption != null && appendIfExistsOption.getValue() + && fs.exists(p)) { + + // Read the file and verify header details + HoodieAppendLog.Reader reader = new HoodieAppendLog.Reader(conf, + HoodieAppendLog.Reader.file(p), new Reader.OnlyHeaderOption()); + try { + + if (keyClassOption.getValue() != reader.getKeyClass() + || valueClassOption.getValue() != reader.getValueClass()) { + throw new IllegalArgumentException( + "Key/value class provided does not match the file"); + } + + if (reader.getVersion() != VERSION[3]) { + throw new VersionMismatchException(VERSION[3], + reader.getVersion()); + } + + if (metadataOption != null) { + LOG.info("MetaData Option is ignored during append"); + } + metadataOption = (MetadataOption) HoodieAppendLog.Writer + .metadata(reader.getMetadata()); + + CompressionOption readerCompressionOption = new CompressionOption( + reader.getCompressionType(), reader.getCompressionCodec()); + + if (readerCompressionOption.value != compressionTypeOption.value + || !readerCompressionOption.codec.getClass().getName() + .equals(compressionTypeOption.codec.getClass().getName())) { + throw new IllegalArgumentException( + "Compression option provided does not match the file"); + } + + sync = reader.getSync(); + + } finally { + reader.close(); + } + + out = fs.append(p, bufferSize, progress); + this.appendMode = true; + } else { + out = fs + .create(p, true, bufferSize, replication, blockSize, progress); + } + } else { + out = streamOption.getValue(); + } + Class keyClass = keyClassOption == null ? + Object.class : keyClassOption.getValue(); + Class valueClass = valueClassOption == null ? + Object.class : valueClassOption.getValue(); + Metadata metadata = metadataOption == null ? + new Metadata() : metadataOption.getValue(); + this.compress = compressionTypeOption.getValue(); + final CompressionCodec codec = compressionTypeOption.getCodec(); + if (codec != null && + (codec instanceof GzipCodec) && + !NativeCodeLoader.isNativeCodeLoaded() && + !ZlibFactory.isNativeZlibLoaded(conf)) { + throw new IllegalArgumentException("SequenceFile doesn't work with " + + "GzipCodec without native-hadoop " + + "code!"); + } + init(conf, out, ownStream, keyClass, valueClass, codec, metadata); + } + + /** Create the named file. + * @deprecated Use + * {@link HoodieAppendLog#createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public Writer(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass) throws IOException { + this.compress = CompressionType.NONE; + init(conf, fs.create(name), true, keyClass, valClass, null, + new Metadata()); + } + + /** Create the named file with write-progress reporter. + * @deprecated Use + * {@link HoodieAppendLog#createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public Writer(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass, + Progressable progress, Metadata metadata) throws IOException { + this.compress = CompressionType.NONE; + init(conf, fs.create(name, progress), true, keyClass, valClass, + null, metadata); + } + + /** Create the named file with write-progress reporter. + * @deprecated Use + * {@link HoodieAppendLog#createWriter(Configuration, Writer.Option...)} + * instead. + */ + @Deprecated + public Writer(FileSystem fs, Configuration conf, Path name, + Class keyClass, Class valClass, + int bufferSize, short replication, long blockSize, + Progressable progress, Metadata metadata) throws IOException { + this.compress = CompressionType.NONE; + init(conf, + fs.create(name, true, bufferSize, replication, blockSize, progress), + true, keyClass, valClass, null, metadata); + } + + boolean isCompressed() { + return compress != CompressionType.NONE; + } + + boolean isBlockCompressed() { + return compress == CompressionType.BLOCK; + } + + Writer ownStream() { + this.ownOutputStream = true; + return this; + } + + /** Write and flush the file header. */ + private void writeFileHeader() + throws IOException { + out.write(VERSION); + Text.writeString(out, keyClass.getName()); + Text.writeString(out, valClass.getName()); + + out.writeBoolean(this.isCompressed()); + out.writeBoolean(this.isBlockCompressed()); + + if (this.isCompressed()) { + Text.writeString(out, (codec.getClass()).getName()); + } + this.metadata.write(out); + out.write(sync); // write the sync bytes + out.flush(); // flush header + } + + /** Initialize. */ + @SuppressWarnings("unchecked") + void init(Configuration conf, FSDataOutputStream out, boolean ownStream, + Class keyClass, Class valClass, + CompressionCodec codec, Metadata metadata) + throws IOException { + this.conf = conf; + this.out = out; + this.ownOutputStream = ownStream; + this.keyClass = keyClass; + this.valClass = valClass; + this.codec = codec; + this.metadata = metadata; + SerializationFactory serializationFactory = new SerializationFactory(conf); + this.keySerializer = serializationFactory.getSerializer(keyClass); + if (this.keySerializer == null) { + throw new IOException( + "Could not find a serializer for the Key class: '" + + keyClass.getCanonicalName() + "'. " + + "Please ensure that the configuration '" + + CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is " + + "properly configured, if you're using" + + "custom serialization."); + } + this.keySerializer.open(buffer); + this.uncompressedValSerializer = serializationFactory.getSerializer(valClass); + if (this.uncompressedValSerializer == null) { + throw new IOException( + "Could not find a serializer for the Value class: '" + + valClass.getCanonicalName() + "'. " + + "Please ensure that the configuration '" + + CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is " + + "properly configured, if you're using" + + "custom serialization."); + } + this.uncompressedValSerializer.open(buffer); + if (this.codec != null) { + ReflectionUtils.setConf(this.codec, this.conf); + this.compressor = CodecPool.getCompressor(this.codec); + this.deflateFilter = this.codec.createOutputStream(buffer, compressor); + this.deflateOut = + new DataOutputStream(new BufferedOutputStream(deflateFilter)); + this.compressedValSerializer = serializationFactory.getSerializer(valClass); + if (this.compressedValSerializer == null) { + throw new IOException( + "Could not find a serializer for the Value class: '" + + valClass.getCanonicalName() + "'. " + + "Please ensure that the configuration '" + + CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is " + + "properly configured, if you're using" + + "custom serialization."); + } + this.compressedValSerializer.open(deflateOut); + } + + if (appendMode) { + sync(); + } else { + writeFileHeader(); + } + } + + /** Returns the class of keys in this file. */ + public Class getKeyClass() { + return keyClass; + } + + /** Returns the class of values in this file. */ + public Class getValueClass() { + return valClass; + } + + /** Returns the compression codec of data in this file. */ + public CompressionCodec getCompressionCodec() { + return codec; + } + + /** create a sync point */ + public void sync() throws IOException { + if (sync != null && lastSyncPos != out.getPos()) { + out.writeInt(SYNC_ESCAPE); // mark the start of the sync + out.write(sync); // write sync + lastSyncPos = out.getPos(); // update lastSyncPos + } + } + + /** + * flush all currently written data to the file system + * @deprecated Use {@link #hsync()} or {@link #hflush()} instead + */ + @Deprecated + public void syncFs() throws IOException { + if (out != null) { + out.sync(); // flush contents to file system + } + } + + @Override + public void hsync() throws IOException { + if (out != null) { + out.hsync(); + } + } + + @Override + public void hflush() throws IOException { + if (out != null) { + out.hflush(); + } + } + + /** Returns the configuration of this file. */ + Configuration getConf() { + return conf; + } + + /** Close the file. */ + @Override + public synchronized void close() throws IOException { + keySerializer.close(); + uncompressedValSerializer.close(); + if (compressedValSerializer != null) { + compressedValSerializer.close(); + } + + CodecPool.returnCompressor(compressor); + compressor = null; + + if (out != null) { + + // Close the underlying stream iff we own it... + if (ownOutputStream) { + out.close(); + } else { + out.flush(); + } + out = null; + } + } + + synchronized void checkAndWriteSync() throws IOException { + if (sync != null && + out.getPos() >= lastSyncPos + SYNC_INTERVAL) { // time to emit sync + sync(); + } + } + + /** Append a key/value pair. */ + public void append(Writable key, Writable val) + throws IOException { + append((Object) key, (Object) val); + } + + /** Append a key/value pair. */ + @SuppressWarnings("unchecked") + public synchronized void append(Object key, Object val) + throws IOException { + if (key.getClass() != keyClass) + throw new IOException("wrong key class: " + key.getClass().getName() + + " is not " + keyClass); + if (val.getClass() != valClass) + throw new IOException("wrong value class: " + val.getClass().getName() + + " is not " + valClass); + + buffer.reset(); + + // Append the 'key' + keySerializer.serialize(key); + int keyLength = buffer.getLength(); + if (keyLength < 0) + throw new IOException("negative length keys not allowed: " + key); + + // Append the 'value' + if (compress == CompressionType.RECORD) { + deflateFilter.resetState(); + compressedValSerializer.serialize(val); + deflateOut.flush(); + deflateFilter.finish(); + } else { + uncompressedValSerializer.serialize(val); + } + + // Write the record out + checkAndWriteSync(); // sync + out.writeInt(buffer.getLength()); // total record length + out.writeInt(keyLength); // key portion length + out.write(buffer.getData(), 0, buffer.getLength()); // data + } + + public synchronized void appendRaw(byte[] keyData, int keyOffset, + int keyLength, ValueBytes val) throws IOException { + if (keyLength < 0) + throw new IOException("negative length keys not allowed: " + keyLength); + + int valLength = val.getSize(); + + checkAndWriteSync(); + + out.writeInt(keyLength + valLength); // total record length + out.writeInt(keyLength); // key portion length + out.write(keyData, keyOffset, keyLength); // key + val.writeUncompressedBytes(out); // value + } + + /** Returns the current length of the output file. + * + *

This always returns a synchronized position. In other words, + * immediately after calling {@link HoodieAppendLog.Reader#seek(long)} with a position + * returned by this method, {@link HoodieAppendLog.Reader#next(Writable)} may be called. However + * the key may be earlier in the file than key last written when this + * method was called (e.g., with block-compression, it may be the first key + * in the block that was being written when this method was called). + */ + public synchronized long getLength() throws IOException { + return out.getPos(); + } + + } // class Writer + + /** Write key/compressed-value pairs to a sequence-format file. */ + static class RecordCompressWriter extends Writer { + + RecordCompressWriter(Configuration conf, + Option... options) throws IOException { + super(conf, options); + } + + /** Append a key/value pair. */ + @Override + @SuppressWarnings("unchecked") + public synchronized void append(Object key, Object val) + throws IOException { + if (key.getClass() != keyClass) + throw new IOException("wrong key class: " + key.getClass().getName() + + " is not " + keyClass); + if (val.getClass() != valClass) + throw new IOException("wrong value class: " + val.getClass().getName() + + " is not " + valClass); + + buffer.reset(); + + // Append the 'key' + keySerializer.serialize(key); + int keyLength = buffer.getLength(); + if (keyLength < 0) + throw new IOException("negative length keys not allowed: " + key); + + // Compress 'value' and append it + deflateFilter.resetState(); + compressedValSerializer.serialize(val); + deflateOut.flush(); + deflateFilter.finish(); + + // Write the record out + checkAndWriteSync(); // sync + out.writeInt(buffer.getLength()); // total record length + out.writeInt(keyLength); // key portion length + out.write(buffer.getData(), 0, buffer.getLength()); // data + } + + /** Append a key/value pair. */ + @Override + public synchronized void appendRaw(byte[] keyData, int keyOffset, + int keyLength, ValueBytes val) throws IOException { + + if (keyLength < 0) + throw new IOException("negative length keys not allowed: " + keyLength); + + int valLength = val.getSize(); + + checkAndWriteSync(); // sync + out.writeInt(keyLength + valLength); // total record length + out.writeInt(keyLength); // key portion length + out.write(keyData, keyOffset, keyLength); // 'key' data + val.writeCompressedBytes(out); // 'value' data + } + + } // RecordCompressionWriter + + /** Write compressed key/value blocks to a sequence-format file. */ + static class BlockCompressWriter extends Writer { + + private int noBufferedRecords = 0; + + private DataOutputBuffer keyLenBuffer = new DataOutputBuffer(); + private DataOutputBuffer keyBuffer = new DataOutputBuffer(); + + private DataOutputBuffer valLenBuffer = new DataOutputBuffer(); + private DataOutputBuffer valBuffer = new DataOutputBuffer(); + + private final int compressionBlockSize; + + BlockCompressWriter(Configuration conf, + Option... options) throws IOException { + super(conf, options); + compressionBlockSize = + conf.getInt("io.seqfile.compress.blocksize", 1000000); + keySerializer.close(); + keySerializer.open(keyBuffer); + uncompressedValSerializer.close(); + uncompressedValSerializer.open(valBuffer); + } + + /** Workhorse to check and write out compressed data/lengths */ + private synchronized void writeBuffer(DataOutputBuffer uncompressedDataBuffer) + throws IOException { + deflateFilter.resetState(); + buffer.reset(); + deflateOut.write(uncompressedDataBuffer.getData(), 0, + uncompressedDataBuffer.getLength()); + deflateOut.flush(); + deflateFilter.finish(); + + WritableUtils.writeVInt(out, buffer.getLength()); + out.write(buffer.getData(), 0, buffer.getLength()); + } + + /** Compress and flush contents to dfs */ + @Override + public synchronized void sync() throws IOException { + if (noBufferedRecords > 0) { + super.sync(); + + // No. of records + WritableUtils.writeVInt(out, noBufferedRecords); + + // Write 'keys' and lengths + writeBuffer(keyLenBuffer); + writeBuffer(keyBuffer); + + // Write 'values' and lengths + writeBuffer(valLenBuffer); + writeBuffer(valBuffer); + + // Flush the file-stream + out.flush(); + + // Reset internal states + keyLenBuffer.reset(); + keyBuffer.reset(); + valLenBuffer.reset(); + valBuffer.reset(); + noBufferedRecords = 0; + } + + } + + /** Close the file. */ + @Override + public synchronized void close() throws IOException { + if (out != null) { + sync(); + } + super.close(); + } + + /** Append a key/value pair. */ + @Override + @SuppressWarnings("unchecked") + public synchronized void append(Object key, Object val) + throws IOException { + if (key.getClass() != keyClass) + throw new IOException("wrong key class: " + key + " is not " + keyClass); + if (val.getClass() != valClass) + throw new IOException("wrong value class: " + val + " is not " + valClass); + + // Save key/value into respective buffers + int oldKeyLength = keyBuffer.getLength(); + keySerializer.serialize(key); + int keyLength = keyBuffer.getLength() - oldKeyLength; + if (keyLength < 0) + throw new IOException("negative length keys not allowed: " + key); + WritableUtils.writeVInt(keyLenBuffer, keyLength); + + int oldValLength = valBuffer.getLength(); + uncompressedValSerializer.serialize(val); + int valLength = valBuffer.getLength() - oldValLength; + WritableUtils.writeVInt(valLenBuffer, valLength); + + // Added another key/value pair + ++noBufferedRecords; + + // Compress and flush? + int currentBlockSize = keyBuffer.getLength() + valBuffer.getLength(); + if (currentBlockSize >= compressionBlockSize) { + sync(); + } + } + + /** Append a key/value pair. */ + @Override + public synchronized void appendRaw(byte[] keyData, int keyOffset, + int keyLength, ValueBytes val) throws IOException { + + if (keyLength < 0) + throw new IOException("negative length keys not allowed"); + + int valLength = val.getSize(); + + // Save key/value data in relevant buffers + WritableUtils.writeVInt(keyLenBuffer, keyLength); + keyBuffer.write(keyData, keyOffset, keyLength); + WritableUtils.writeVInt(valLenBuffer, valLength); + val.writeUncompressedBytes(valBuffer); + + // Added another key/value pair + ++noBufferedRecords; + + // Compress and flush? + int currentBlockSize = keyBuffer.getLength() + valBuffer.getLength(); + if (currentBlockSize >= compressionBlockSize) { + sync(); + } + } + + } // BlockCompressionWriter + + /** Get the configured buffer size */ + private static int getBufferSize(Configuration conf) { + return conf.getInt("io.file.buffer.size", 4096); + } + + /** Reads key/value pairs from a sequence-format file. */ + public static class Reader implements java.io.Closeable { + private String filename; + private FSDataInputStream in; + private DataOutputBuffer outBuf = new DataOutputBuffer(); + + private byte version; + + private String keyClassName; + private String valClassName; + private Class keyClass; + private Class valClass; + + private CompressionCodec codec = null; + private Metadata metadata = null; + + private byte[] sync = new byte[SYNC_HASH_SIZE]; + private byte[] syncCheck = new byte[SYNC_HASH_SIZE]; + private boolean syncSeen; + + private long headerEnd; + private long end; + private int keyLength; + private int recordLength; + + private boolean decompress; + private boolean blockCompressed; + + private Configuration conf; + + private int noBufferedRecords = 0; + private boolean lazyDecompress = true; + private boolean valuesDecompressed = true; + + private int noBufferedKeys = 0; + private int noBufferedValues = 0; + + private DataInputBuffer keyLenBuffer = null; + private CompressionInputStream keyLenInFilter = null; + private DataInputStream keyLenIn = null; + private Decompressor keyLenDecompressor = null; + private DataInputBuffer keyBuffer = null; + private CompressionInputStream keyInFilter = null; + private DataInputStream keyIn = null; + private Decompressor keyDecompressor = null; + + private DataInputBuffer valLenBuffer = null; + private CompressionInputStream valLenInFilter = null; + private DataInputStream valLenIn = null; + private Decompressor valLenDecompressor = null; + private DataInputBuffer valBuffer = null; + private CompressionInputStream valInFilter = null; + private DataInputStream valIn = null; + private Decompressor valDecompressor = null; + + private Deserializer keyDeserializer; + private Deserializer valDeserializer; + + /** + * A tag interface for all of the Reader options + */ + public static interface Option { + } + + /** + * Create an option to specify the path name of the sequence file. + * @param value the path to read + * @return a new option + */ + public static Option file(Path value) { + return new FileOption(value); + } + + /** + * Create an option to specify the stream with the sequence file. + * @param value the stream to read. + * @return a new option + */ + public static Option stream(FSDataInputStream value) { + return new InputStreamOption(value); + } + + /** + * Create an option to specify the starting byte to read. + * @param value the number of bytes to skip over + * @return a new option + */ + public static Option start(long value) { + return new StartOption(value); + } + + /** + * Create an option to specify the number of bytes to read. + * @param value the number of bytes to read + * @return a new option + */ + public static Option length(long value) { + return new LengthOption(value); + } + + /** + * Create an option with the buffer size for reading the given pathname. + * @param value the number of bytes to buffer + * @return a new option + */ + public static Option bufferSize(int value) { + return new BufferSizeOption(value); + } + + private static class FileOption extends Options.PathOption + implements Option { + private FileOption(Path value) { + super(value); + } + } + + private static class InputStreamOption + extends Options.FSDataInputStreamOption + implements Option { + private InputStreamOption(FSDataInputStream value) { + super(value); + } + } + + private static class StartOption extends Options.LongOption + implements Option { + private StartOption(long value) { + super(value); + } + } + + private static class LengthOption extends Options.LongOption + implements Option { + private LengthOption(long value) { + super(value); + } + } + + private static class BufferSizeOption extends Options.IntegerOption + implements Option { + private BufferSizeOption(int value) { + super(value); + } + } + + // only used directly + private static class OnlyHeaderOption extends Options.BooleanOption + implements Option { + private OnlyHeaderOption() { + super(true); + } + } + + public Reader(Configuration conf, Option... opts) throws IOException { + // Look up the options, these are null if not set + FileOption fileOpt = Options.getOption(FileOption.class, opts); + InputStreamOption streamOpt = + Options.getOption(InputStreamOption.class, opts); + StartOption startOpt = Options.getOption(StartOption.class, opts); + LengthOption lenOpt = Options.getOption(LengthOption.class, opts); + BufferSizeOption bufOpt = Options.getOption(BufferSizeOption.class, opts); + OnlyHeaderOption headerOnly = + Options.getOption(OnlyHeaderOption.class, opts); + // check for consistency + if ((fileOpt == null) == (streamOpt == null)) { + throw new + IllegalArgumentException("File or stream option must be specified"); + } + if (fileOpt == null && bufOpt != null) { + throw new IllegalArgumentException("buffer size can only be set when" + + " a file is specified."); + } + // figure out the real values + Path filename = null; + FSDataInputStream file; + final long len; + if (fileOpt != null) { + filename = fileOpt.getValue(); + FileSystem fs = filename.getFileSystem(conf); + int bufSize = bufOpt == null ? getBufferSize(conf) : bufOpt.getValue(); + len = null == lenOpt + ? fs.getFileStatus(filename).getLen() + : lenOpt.getValue(); + file = openFile(fs, filename, bufSize, len); + } else { + len = null == lenOpt ? Long.MAX_VALUE : lenOpt.getValue(); + file = streamOpt.getValue(); + } + long start = startOpt == null ? 0 : startOpt.getValue(); + // really set up + initialize(filename, file, start, len, conf, headerOnly != null); + } + + /** + * Construct a reader by opening a file from the given file system. + * @param fs The file system used to open the file. + * @param file The file being read. + * @param conf Configuration + * @throws IOException + * @deprecated Use Reader(Configuration, Option...) instead. + */ + @Deprecated + public Reader(FileSystem fs, Path file, + Configuration conf) throws IOException { + this(conf, file(file.makeQualified(fs))); + } + + /** + * Construct a reader by the given input stream. + * @param in An input stream. + * @param buffersize unused + * @param start The starting position. + * @param length The length being read. + * @param conf Configuration + * @throws IOException + * @deprecated Use Reader(Configuration, Reader.Option...) instead. + */ + @Deprecated + public Reader(FSDataInputStream in, int buffersize, + long start, long length, Configuration conf) throws IOException { + this(conf, stream(in), start(start), length(length)); + } + + /** Common work of the constructors. */ + private void initialize(Path filename, FSDataInputStream in, + long start, long length, Configuration conf, + boolean tempReader) throws IOException { + if (in == null) { + throw new IllegalArgumentException("in == null"); + } + this.filename = filename == null ? "" : filename.toString(); + this.in = in; + this.conf = conf; + boolean succeeded = false; + try { + seek(start); + this.end = this.in.getPos() + length; + // if it wrapped around, use the max + if (end < length) { + end = Long.MAX_VALUE; + } + init(tempReader); + succeeded = true; + } finally { + if (!succeeded) { + IOUtils.cleanup(LOG, this.in); + } + } + } + + /** + * Override this method to specialize the type of + * {@link FSDataInputStream} returned. + * @param fs The file system used to open the file. + * @param file The file being read. + * @param bufferSize The buffer size used to read the file. + * @param length The length being read if it is >= 0. Otherwise, + * the length is not available. + * @return The opened stream. + * @throws IOException + */ + protected FSDataInputStream openFile(FileSystem fs, Path file, + int bufferSize, long length) throws IOException { + return fs.open(file, bufferSize); + } + + /** + * Initialize the {@link Reader} + * @param tempReader true if we are constructing a temporary + * reader {@link HoodieAppendLog.Sorter}, + * and hence do not initialize every component; + * false otherwise. + * @throws IOException + */ + private void init(boolean tempReader) throws IOException { + byte[] versionBlock = new byte[VERSION.length]; + in.readFully(versionBlock); + + if ((versionBlock[0] != VERSION[0]) || + (versionBlock[1] != VERSION[1]) || + (versionBlock[2] != VERSION[2])) + throw new IOException(this + " not a SequenceFile"); + + // Set 'version' + version = versionBlock[3]; + if (version > VERSION[3]) + throw new VersionMismatchException(VERSION[3], version); + + if (version < BLOCK_COMPRESS_VERSION) { + UTF8 className = new UTF8(); + + className.readFields(in); + keyClassName = className.toStringChecked(); // key class name + + className.readFields(in); + valClassName = className.toStringChecked(); // val class name + } else { + keyClassName = Text.readString(in); + valClassName = Text.readString(in); + } + + if (version > 2) { // if version > 2 + this.decompress = in.readBoolean(); // is compressed? + } else { + decompress = false; + } + + if (version >= BLOCK_COMPRESS_VERSION) { // if version >= 4 + this.blockCompressed = in.readBoolean(); // is block-compressed? + } else { + blockCompressed = false; + } + + // if version >= 5 + // setup the compression codec + if (decompress) { + if (version >= CUSTOM_COMPRESS_VERSION) { + String codecClassname = Text.readString(in); + try { + Class codecClass + = conf.getClassByName(codecClassname).asSubclass(CompressionCodec.class); + this.codec = ReflectionUtils.newInstance(codecClass, conf); + } catch (ClassNotFoundException cnfe) { + throw new IllegalArgumentException("Unknown codec: " + + codecClassname, cnfe); + } + } else { + codec = new DefaultCodec(); + ((Configurable) codec).setConf(conf); + } + } + + this.metadata = new Metadata(); + if (version >= VERSION_WITH_METADATA) { // if version >= 6 + this.metadata.readFields(in); + } + + if (version > 1) { // if version > 1 + in.readFully(sync); // read sync bytes + headerEnd = in.getPos(); // record end of header + } + + // Initialize... *not* if this we are constructing a temporary Reader + if (!tempReader) { + valBuffer = new DataInputBuffer(); + if (decompress) { + valDecompressor = CodecPool.getDecompressor(codec); + valInFilter = codec.createInputStream(valBuffer, valDecompressor); + valIn = new DataInputStream(valInFilter); + } else { + valIn = valBuffer; + } + + if (blockCompressed) { + keyLenBuffer = new DataInputBuffer(); + keyBuffer = new DataInputBuffer(); + valLenBuffer = new DataInputBuffer(); + + keyLenDecompressor = CodecPool.getDecompressor(codec); + keyLenInFilter = codec.createInputStream(keyLenBuffer, + keyLenDecompressor); + keyLenIn = new DataInputStream(keyLenInFilter); + + keyDecompressor = CodecPool.getDecompressor(codec); + keyInFilter = codec.createInputStream(keyBuffer, keyDecompressor); + keyIn = new DataInputStream(keyInFilter); + + valLenDecompressor = CodecPool.getDecompressor(codec); + valLenInFilter = codec.createInputStream(valLenBuffer, + valLenDecompressor); + valLenIn = new DataInputStream(valLenInFilter); + } + + SerializationFactory serializationFactory = + new SerializationFactory(conf); + this.keyDeserializer = + getDeserializer(serializationFactory, getKeyClass()); + if (this.keyDeserializer == null) { + throw new IOException( + "Could not find a deserializer for the Key class: '" + + getKeyClass().getCanonicalName() + "'. " + + "Please ensure that the configuration '" + + CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is " + + "properly configured, if you're using " + + "custom serialization."); + } + if (!blockCompressed) { + this.keyDeserializer.open(valBuffer); + } else { + this.keyDeserializer.open(keyIn); + } + this.valDeserializer = + getDeserializer(serializationFactory, getValueClass()); + if (this.valDeserializer == null) { + throw new IOException( + "Could not find a deserializer for the Value class: '" + + getValueClass().getCanonicalName() + "'. " + + "Please ensure that the configuration '" + + CommonConfigurationKeys.IO_SERIALIZATIONS_KEY + "' is " + + "properly configured, if you're using " + + "custom serialization."); + } + this.valDeserializer.open(valIn); + } + } + + @SuppressWarnings("unchecked") + private Deserializer getDeserializer(SerializationFactory sf, Class c) { + return sf.getDeserializer(c); + } + + /** Close the file. */ + @Override + public synchronized void close() throws IOException { + // Return the decompressors to the pool + CodecPool.returnDecompressor(keyLenDecompressor); + CodecPool.returnDecompressor(keyDecompressor); + CodecPool.returnDecompressor(valLenDecompressor); + CodecPool.returnDecompressor(valDecompressor); + keyLenDecompressor = keyDecompressor = null; + valLenDecompressor = valDecompressor = null; + + if (keyDeserializer != null) { + keyDeserializer.close(); + } + if (valDeserializer != null) { + valDeserializer.close(); + } + + // Close the input-stream + in.close(); + } + + /** Returns the name of the key class. */ + public String getKeyClassName() { + return keyClassName; + } + + /** Returns the class of keys in this file. */ + public synchronized Class getKeyClass() { + if (null == keyClass) { + try { + keyClass = WritableName.getClass(getKeyClassName(), conf); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return keyClass; + } + + /** Returns the name of the value class. */ + public String getValueClassName() { + return valClassName; + } + + /** Returns the class of values in this file. */ + public synchronized Class getValueClass() { + if (null == valClass) { + try { + valClass = WritableName.getClass(getValueClassName(), conf); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return valClass; + } + + /** Returns true if values are compressed. */ + public boolean isCompressed() { + return decompress; + } + + /** Returns true if records are block-compressed. */ + public boolean isBlockCompressed() { + return blockCompressed; + } + + /** Returns the compression codec of data in this file. */ + public CompressionCodec getCompressionCodec() { + return codec; + } + + private byte[] getSync() { + return sync; + } + + private byte getVersion() { + return version; + } + + /** + * Get the compression type for this file. + * @return the compression type + */ + public CompressionType getCompressionType() { + if (decompress) { + return blockCompressed ? CompressionType.BLOCK : CompressionType.RECORD; + } else { + return CompressionType.NONE; + } + } + + /** Returns the metadata object of the file */ + public Metadata getMetadata() { + return this.metadata; + } + + /** Returns the configuration used for this file. */ + Configuration getConf() { + return conf; + } + + /** Read a compressed buffer */ + private synchronized void readBuffer(DataInputBuffer buffer, + CompressionInputStream filter) throws IOException { + // Read data into a temporary buffer + DataOutputBuffer dataBuffer = new DataOutputBuffer(); + + try { + int dataBufferLength = WritableUtils.readVInt(in); + dataBuffer.write(in, dataBufferLength); + + // Set up 'buffer' connected to the input-stream + buffer.reset(dataBuffer.getData(), 0, dataBuffer.getLength()); + } finally { + dataBuffer.close(); + } + + // Reset the codec + filter.resetState(); + } + + /** Read the next 'compressed' block */ + private synchronized void readBlock() throws IOException { + // Check if we need to throw away a whole block of + // 'values' due to 'lazy decompression' + if (lazyDecompress && !valuesDecompressed) { + in.seek(WritableUtils.readVInt(in) + in.getPos()); + in.seek(WritableUtils.readVInt(in) + in.getPos()); + } + + // Reset internal states + noBufferedKeys = 0; + noBufferedValues = 0; + noBufferedRecords = 0; + valuesDecompressed = false; + + //Process sync + if (sync != null) { + in.readInt(); + in.readFully(syncCheck); // read syncCheck + if (!Arrays.equals(sync, syncCheck)) // check it + throw new IOException("File is corrupt!"); + } + syncSeen = true; + + // Read number of records in this block + noBufferedRecords = WritableUtils.readVInt(in); + + // Read key lengths and keys + readBuffer(keyLenBuffer, keyLenInFilter); + readBuffer(keyBuffer, keyInFilter); + noBufferedKeys = noBufferedRecords; + + // Read value lengths and values + if (!lazyDecompress) { + readBuffer(valLenBuffer, valLenInFilter); + readBuffer(valBuffer, valInFilter); + noBufferedValues = noBufferedRecords; + valuesDecompressed = true; + } + } + + /** + * Position valLenIn/valIn to the 'value' + * corresponding to the 'current' key + */ + private synchronized void seekToCurrentValue() throws IOException { + if (!blockCompressed) { + if (decompress) { + valInFilter.resetState(); + } + valBuffer.reset(); + } else { + // Check if this is the first value in the 'block' to be read + if (lazyDecompress && !valuesDecompressed) { + // Read the value lengths and values + readBuffer(valLenBuffer, valLenInFilter); + readBuffer(valBuffer, valInFilter); + noBufferedValues = noBufferedRecords; + valuesDecompressed = true; + } + + // Calculate the no. of bytes to skip + // Note: 'current' key has already been read! + int skipValBytes = 0; + int currentKey = noBufferedKeys + 1; + for (int i = noBufferedValues; i > currentKey; --i) { + skipValBytes += WritableUtils.readVInt(valLenIn); + --noBufferedValues; + } + + // Skip to the 'val' corresponding to 'current' key + if (skipValBytes > 0) { + if (valIn.skipBytes(skipValBytes) != skipValBytes) { + throw new IOException("Failed to seek to " + currentKey + + "(th) value!"); + } + } + } + } + + /** + * Get the 'value' corresponding to the last read 'key'. + * @param val : The 'value' to be read. + * @throws IOException + */ + public synchronized void getCurrentValue(Writable val) + throws IOException { + if (val instanceof Configurable) { + ((Configurable) val).setConf(this.conf); + } + + // Position stream to 'current' value + seekToCurrentValue(); + + if (!blockCompressed) { + val.readFields(valIn); + + if (valIn.read() > 0) { + LOG.info("available bytes: " + valIn.available()); + throw new IOException(val + " read " + (valBuffer.getPosition() - keyLength) + + " bytes, should read " + + (valBuffer.getLength() - keyLength)); + } + } else { + // Get the value + int valLength = WritableUtils.readVInt(valLenIn); + val.readFields(valIn); + + // Read another compressed 'value' + --noBufferedValues; + + // Sanity check + if ((valLength < 0) && LOG.isDebugEnabled()) { + LOG.debug(val + " is a zero-length value"); + } + } + + } + + /** + * Get the 'value' corresponding to the last read 'key'. + * @param val : The 'value' to be read. + * @throws IOException + */ + public synchronized Object getCurrentValue(Object val) + throws IOException { + if (val instanceof Configurable) { + ((Configurable) val).setConf(this.conf); + } + + // Position stream to 'current' value + seekToCurrentValue(); + + if (!blockCompressed) { + val = deserializeValue(val); + + if (valIn.read() > 0) { + LOG.info("available bytes: " + valIn.available()); + throw new IOException(val + " read " + (valBuffer.getPosition() - keyLength) + + " bytes, should read " + + (valBuffer.getLength() - keyLength)); + } + } else { + // Get the value + int valLength = WritableUtils.readVInt(valLenIn); + val = deserializeValue(val); + + // Read another compressed 'value' + --noBufferedValues; + + // Sanity check + if ((valLength < 0) && LOG.isDebugEnabled()) { + LOG.debug(val + " is a zero-length value"); + } + } + return val; + + } + + @SuppressWarnings("unchecked") + private Object deserializeValue(Object val) throws IOException { + return valDeserializer.deserialize(val); + } + + /** Read the next key in the file into key, skipping its + * value. True if another entry exists, and false at end of file. */ + public synchronized boolean next(Writable key) throws IOException { + if (key.getClass() != getKeyClass()) + throw new IOException("wrong key class: " + key.getClass().getName() + + " is not " + keyClass); + + if (!blockCompressed) { + outBuf.reset(); + + keyLength = next(outBuf); + if (keyLength < 0) + return false; + + valBuffer.reset(outBuf.getData(), outBuf.getLength()); + + key.readFields(valBuffer); + valBuffer.mark(0); + if (valBuffer.getPosition() != keyLength) + throw new IOException(key + " read " + valBuffer.getPosition() + + " bytes, should read " + keyLength); + } else { + //Reset syncSeen + syncSeen = false; + + if (noBufferedKeys == 0) { + try { + readBlock(); + } catch (EOFException eof) { + return false; + } + } + + int keyLength = WritableUtils.readVInt(keyLenIn); + + // Sanity check + if (keyLength < 0) { + return false; + } + + //Read another compressed 'key' + key.readFields(keyIn); + --noBufferedKeys; + } + + return true; + } + + /** Read the next key/value pair in the file into key and + * val. Returns true if such a pair exists and false when at + * end of file */ + public synchronized boolean next(Writable key, Writable val) + throws IOException { + if (val.getClass() != getValueClass()) + throw new IOException("wrong value class: " + val + " is not " + valClass); + + boolean more = next(key); + + if (more) { + getCurrentValue(val); + } + + return more; + } + + /** + * Read and return the next record length, potentially skipping over + * a sync block. + * @return the length of the next record or -1 if there is no next record + * @throws IOException + */ + private synchronized int readRecordLength() throws IOException { + if (in.getPos() >= end) { + return -1; + } + int length = in.readInt(); + if (version > 1 && sync != null && + length == SYNC_ESCAPE) { // process a sync entry + in.readFully(syncCheck); // read syncCheck + if (!Arrays.equals(sync, syncCheck)) // check it + throw new IOException("File is corrupt!"); + syncSeen = true; + if (in.getPos() >= end) { + return -1; + } + length = in.readInt(); // re-read length + } else { + syncSeen = false; + } + + return length; + } + + /** Read the next key/value pair in the file into buffer. + * Returns the length of the key read, or -1 if at end of file. The length + * of the value may be computed by calling buffer.getLength() before and + * after calls to this method. */ + /** @deprecated Call {@link #nextRaw(DataOutputBuffer, HoodieAppendLog.ValueBytes)}. */ + @Deprecated + synchronized int next(DataOutputBuffer buffer) throws IOException { + // Unsupported for block-compressed sequence files + if (blockCompressed) { + throw new IOException("Unsupported call for block-compressed" + + " SequenceFiles - use SequenceFile.Reader.next(DataOutputStream, ValueBytes)"); + } + try { + int length = readRecordLength(); + if (length == -1) { + return -1; + } + int keyLength = in.readInt(); + buffer.write(in, length); + return keyLength; + } catch (ChecksumException e) { // checksum failure + handleChecksumException(e); + return next(buffer); + } + } + + public ValueBytes createValueBytes() { + ValueBytes val = null; + if (!decompress || blockCompressed) { + val = new UncompressedBytes(); + } else { + val = new CompressedBytes(codec); + } + return val; + } + + /** + * Read 'raw' records. + * @param key - The buffer into which the key is read + * @param val - The 'raw' value + * @return Returns the total record length or -1 for end of file + * @throws IOException + */ + public synchronized int nextRaw(DataOutputBuffer key, ValueBytes val) + throws IOException { + if (!blockCompressed) { + int length = readRecordLength(); + if (length == -1) { + return -1; + } + int keyLength = in.readInt(); + int valLength = length - keyLength; + key.write(in, keyLength); + if (decompress) { + CompressedBytes value = (CompressedBytes) val; + value.reset(in, valLength); + } else { + UncompressedBytes value = (UncompressedBytes) val; + value.reset(in, valLength); + } + + return length; + } else { + //Reset syncSeen + syncSeen = false; + + // Read 'key' + if (noBufferedKeys == 0) { + if (in.getPos() >= end) + return -1; + + try { + readBlock(); + } catch (EOFException eof) { + return -1; + } + } + int keyLength = WritableUtils.readVInt(keyLenIn); + if (keyLength < 0) { + throw new IOException("zero length key found!"); + } + key.write(keyIn, keyLength); + --noBufferedKeys; + + // Read raw 'value' + seekToCurrentValue(); + int valLength = WritableUtils.readVInt(valLenIn); + UncompressedBytes rawValue = (UncompressedBytes) val; + rawValue.reset(valIn, valLength); + --noBufferedValues; + + return (keyLength + valLength); + } + + } + + /** + * Read 'raw' keys. + * @param key - The buffer into which the key is read + * @return Returns the key length or -1 for end of file + * @throws IOException + */ + public synchronized int nextRawKey(DataOutputBuffer key) + throws IOException { + if (!blockCompressed) { + recordLength = readRecordLength(); + if (recordLength == -1) { + return -1; + } + keyLength = in.readInt(); + key.write(in, keyLength); + return keyLength; + } else { + //Reset syncSeen + syncSeen = false; + + // Read 'key' + if (noBufferedKeys == 0) { + if (in.getPos() >= end) + return -1; + + try { + readBlock(); + } catch (EOFException eof) { + return -1; + } + } + int keyLength = WritableUtils.readVInt(keyLenIn); + if (keyLength < 0) { + throw new IOException("zero length key found!"); + } + key.write(keyIn, keyLength); + --noBufferedKeys; + + return keyLength; + } + + } + + /** Read the next key in the file, skipping its + * value. Return null at end of file. */ + public synchronized Object next(Object key) throws IOException { + if (key != null && key.getClass() != getKeyClass()) { + throw new IOException("wrong key class: " + key.getClass().getName() + + " is not " + keyClass); + } + + if (!blockCompressed) { + outBuf.reset(); + + keyLength = next(outBuf); + if (keyLength < 0) + return null; + + valBuffer.reset(outBuf.getData(), outBuf.getLength()); + + key = deserializeKey(key); + valBuffer.mark(0); + if (valBuffer.getPosition() != keyLength) + throw new IOException(key + " read " + valBuffer.getPosition() + + " bytes, should read " + keyLength); + } else { + //Reset syncSeen + syncSeen = false; + + if (noBufferedKeys == 0) { + try { + readBlock(); + } catch (EOFException eof) { + return null; + } + } + + int keyLength = WritableUtils.readVInt(keyLenIn); + + // Sanity check + if (keyLength < 0) { + return null; + } + + //Read another compressed 'key' + key = deserializeKey(key); + --noBufferedKeys; + } + + return key; + } + + @SuppressWarnings("unchecked") + private Object deserializeKey(Object key) throws IOException { + return keyDeserializer.deserialize(key); + } + + /** + * Read 'raw' values. + * @param val - The 'raw' value + * @return Returns the value length + * @throws IOException + */ + public synchronized int nextRawValue(ValueBytes val) + throws IOException { + + // Position stream to current value + seekToCurrentValue(); + + if (!blockCompressed) { + int valLength = recordLength - keyLength; + if (decompress) { + CompressedBytes value = (CompressedBytes) val; + value.reset(in, valLength); + } else { + UncompressedBytes value = (UncompressedBytes) val; + value.reset(in, valLength); + } + + return valLength; + } else { + int valLength = WritableUtils.readVInt(valLenIn); + UncompressedBytes rawValue = (UncompressedBytes) val; + rawValue.reset(valIn, valLength); + --noBufferedValues; + return valLength; + } + + } + + private void handleChecksumException(ChecksumException e) + throws IOException { + if (this.conf.getBoolean("io.skip.checksum.errors", false)) { + LOG.warn("Bad checksum at " + getPosition() + ". Skipping entries."); + sync(getPosition() + this.conf.getInt("io.bytes.per.checksum", 512)); + } else { + throw e; + } + } + + /** disables sync. often invoked for tmp files */ + synchronized void ignoreSync() { + sync = null; + } + + /** Set the current byte position in the input file. + * + *

The position passed must be a position returned by {@link + * HoodieAppendLog.Writer#getLength()} when writing this file. To seek to an arbitrary + * position, use {@link HoodieAppendLog.Reader#sync(long)}. + */ + public synchronized void seek(long position) throws IOException { + in.seek(position); + if (blockCompressed) { // trigger block read + noBufferedKeys = 0; + valuesDecompressed = true; + } + } + + /** Seek to the next sync mark past a given position.*/ + public synchronized void sync(long position) throws IOException { + if (position + SYNC_SIZE >= end) { + seek(end); + return; + } + + if (position < headerEnd) { + // seek directly to first record + in.seek(headerEnd); + // note the sync marker "seen" in the header + syncSeen = true; + return; + } + + try { + seek(position + 4); // skip escape + in.readFully(syncCheck); + int syncLen = sync.length; + for (int i = 0; in.getPos() < end; i++) { + int j = 0; + for (; j < syncLen; j++) { + if (sync[j] != syncCheck[(i + j) % syncLen]) + break; + } + if (j == syncLen) { + in.seek(in.getPos() - SYNC_SIZE); // position before sync + return; + } + syncCheck[i % syncLen] = in.readByte(); + } + } catch (ChecksumException e) { // checksum failure + handleChecksumException(e); + } + } + + /** Returns true iff the previous call to next passed a sync mark.*/ + public synchronized boolean syncSeen() { + return syncSeen; + } + + /** Return the current byte position in the input file. */ + public synchronized long getPosition() throws IOException { + return in.getPos(); + } + + /** Returns the name of the file. */ + @Override + public String toString() { + return filename; + } + + } + + /** Sorts key/value pairs in a sequence-format file. + * + *

For best performance, applications should make sure that the {@link + * Writable#readFields(DataInput)} implementation of their keys is + * very efficient. In particular, it should avoid allocating memory. + */ + public static class Sorter { + + private RawComparator comparator; + + private MergeSort mergeSort; //the implementation of merge sort + + private Path[] inFiles; // when merging or sorting + + private Path outFile; + + private int memory; // bytes + private int factor; // merged per pass + + private FileSystem fs = null; + + private Class keyClass; + private Class valClass; + + private Configuration conf; + private Metadata metadata; + + private Progressable progressable = null; + + /** Sort and merge files containing the named classes. */ + public Sorter(FileSystem fs, Class keyClass, + Class valClass, Configuration conf) { + this(fs, WritableComparator.get(keyClass, conf), keyClass, valClass, conf); + } + + /** Sort and merge using an arbitrary {@link RawComparator}. */ + public Sorter(FileSystem fs, RawComparator comparator, Class keyClass, + Class valClass, Configuration conf) { + this(fs, comparator, keyClass, valClass, conf, new Metadata()); + } + + /** Sort and merge using an arbitrary {@link RawComparator}. */ + public Sorter(FileSystem fs, RawComparator comparator, Class keyClass, + Class valClass, Configuration conf, Metadata metadata) { + this.fs = fs; + this.comparator = comparator; + this.keyClass = keyClass; + this.valClass = valClass; + this.memory = conf.getInt("io.sort.mb", 100) * 1024 * 1024; + this.factor = conf.getInt("io.sort.factor", 100); + this.conf = conf; + this.metadata = metadata; + } + + /** Set the number of streams to merge at once.*/ + public void setFactor(int factor) { + this.factor = factor; + } + + /** Get the number of streams to merge at once.*/ + public int getFactor() { + return factor; + } + + /** Set the total amount of buffer memory, in bytes.*/ + public void setMemory(int memory) { + this.memory = memory; + } + + /** Get the total amount of buffer memory, in bytes.*/ + public int getMemory() { + return memory; + } + + /** Set the progressable object in order to report progress. */ + public void setProgressable(Progressable progressable) { + this.progressable = progressable; + } + + /** + * Perform a file sort from a set of input files into an output file. + * @param inFiles the files to be sorted + * @param outFile the sorted output file + * @param deleteInput should the input files be deleted as they are read? + */ + public void sort(Path[] inFiles, Path outFile, + boolean deleteInput) throws IOException { + if (fs.exists(outFile)) { + throw new IOException("already exists: " + outFile); + } + + this.inFiles = inFiles; + this.outFile = outFile; + + int segments = sortPass(deleteInput); + if (segments > 1) { + mergePass(outFile.getParent()); + } + } + + /** + * Perform a file sort from a set of input files and return an iterator. + * @param inFiles the files to be sorted + * @param tempDir the directory where temp files are created during sort + * @param deleteInput should the input files be deleted as they are read? + * @return iterator the RawKeyValueIterator + */ + public RawKeyValueIterator sortAndIterate(Path[] inFiles, Path tempDir, + boolean deleteInput) throws IOException { + Path outFile = new Path(tempDir + Path.SEPARATOR + "all.2"); + if (fs.exists(outFile)) { + throw new IOException("already exists: " + outFile); + } + this.inFiles = inFiles; + //outFile will basically be used as prefix for temp files in the cases + //where sort outputs multiple sorted segments. For the single segment + //case, the outputFile itself will contain the sorted data for that + //segment + this.outFile = outFile; + + int segments = sortPass(deleteInput); + if (segments > 1) + return merge(outFile.suffix(".0"), outFile.suffix(".0.index"), + tempDir); + else if (segments == 1) + return merge(new Path[]{outFile}, true, tempDir); + else return null; + } + + /** + * The backwards compatible interface to sort. + * @param inFile the input file to sort + * @param outFile the sorted output file + */ + public void sort(Path inFile, Path outFile) throws IOException { + sort(new Path[]{inFile}, outFile, false); + } + + private int sortPass(boolean deleteInput) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("running sort pass"); + } + SortPass sortPass = new SortPass(); // make the SortPass + sortPass.setProgressable(progressable); + mergeSort = new MergeSort(sortPass.new SeqFileComparator()); + try { + return sortPass.run(deleteInput); // run it + } finally { + sortPass.close(); // close it + } + } + + private class SortPass { + private int memoryLimit = memory / 4; + private int recordLimit = 1000000; + + private DataOutputBuffer rawKeys = new DataOutputBuffer(); + private byte[] rawBuffer; + + private int[] keyOffsets = new int[1024]; + private int[] pointers = new int[keyOffsets.length]; + private int[] pointersCopy = new int[keyOffsets.length]; + private int[] keyLengths = new int[keyOffsets.length]; + private ValueBytes[] rawValues = new ValueBytes[keyOffsets.length]; + + private ArrayList segmentLengths = new ArrayList(); + + private Reader in = null; + private FSDataOutputStream out = null; + private FSDataOutputStream indexOut = null; + private Path outName; + + private Progressable progressable = null; + + public int run(boolean deleteInput) throws IOException { + int segments = 0; + int currentFile = 0; + boolean atEof = (currentFile >= inFiles.length); + CompressionType compressionType; + CompressionCodec codec = null; + segmentLengths.clear(); + if (atEof) { + return 0; + } + + // Initialize + in = new Reader(fs, inFiles[currentFile], conf); + compressionType = in.getCompressionType(); + codec = in.getCompressionCodec(); + + for (int i = 0; i < rawValues.length; ++i) { + rawValues[i] = null; + } + + while (!atEof) { + int count = 0; + int bytesProcessed = 0; + rawKeys.reset(); + while (!atEof && + bytesProcessed < memoryLimit && count < recordLimit) { + + // Read a record into buffer + // Note: Attempt to re-use 'rawValue' as far as possible + int keyOffset = rawKeys.getLength(); + ValueBytes rawValue = + (count == keyOffsets.length || rawValues[count] == null) ? + in.createValueBytes() : + rawValues[count]; + int recordLength = in.nextRaw(rawKeys, rawValue); + if (recordLength == -1) { + in.close(); + if (deleteInput) { + fs.delete(inFiles[currentFile], true); + } + currentFile += 1; + atEof = currentFile >= inFiles.length; + if (!atEof) { + in = new Reader(fs, inFiles[currentFile], conf); + } else { + in = null; + } + continue; + } + + int keyLength = rawKeys.getLength() - keyOffset; + + if (count == keyOffsets.length) + grow(); + + keyOffsets[count] = keyOffset; // update pointers + pointers[count] = count; + keyLengths[count] = keyLength; + rawValues[count] = rawValue; + + bytesProcessed += recordLength; + count++; + } + + // buffer is full -- sort & flush it + if (LOG.isDebugEnabled()) { + LOG.debug("flushing segment " + segments); + } + rawBuffer = rawKeys.getData(); + sort(count); + // indicate we're making progress + if (progressable != null) { + progressable.progress(); + } + flush(count, bytesProcessed, compressionType, codec, + segments == 0 && atEof); + segments++; + } + return segments; + } + + public void close() throws IOException { + if (in != null) { + in.close(); + } + if (out != null) { + out.close(); + } + if (indexOut != null) { + indexOut.close(); + } + } + + private void grow() { + int newLength = keyOffsets.length * 3 / 2; + keyOffsets = grow(keyOffsets, newLength); + pointers = grow(pointers, newLength); + pointersCopy = new int[newLength]; + keyLengths = grow(keyLengths, newLength); + rawValues = grow(rawValues, newLength); + } + + private int[] grow(int[] old, int newLength) { + int[] result = new int[newLength]; + System.arraycopy(old, 0, result, 0, old.length); + return result; + } + + private ValueBytes[] grow(ValueBytes[] old, int newLength) { + ValueBytes[] result = new ValueBytes[newLength]; + System.arraycopy(old, 0, result, 0, old.length); + for (int i = old.length; i < newLength; ++i) { + result[i] = null; + } + return result; + } + + private void flush(int count, int bytesProcessed, + CompressionType compressionType, + CompressionCodec codec, + boolean done) throws IOException { + if (out == null) { + outName = done ? outFile : outFile.suffix(".0"); + out = fs.create(outName); + if (!done) { + indexOut = fs.create(outName.suffix(".index")); + } + } + + long segmentStart = out.getPos(); + Writer writer = createWriter(conf, Writer.stream(out), + Writer.keyClass(keyClass), Writer.valueClass(valClass), + Writer.compression(compressionType, codec), + Writer.metadata(done ? metadata : new Metadata())); + + if (!done) { + writer.sync = null; // disable sync on temp files + } + + for (int i = 0; i < count; i++) { // write in sorted order + int p = pointers[i]; + writer.appendRaw(rawBuffer, keyOffsets[p], keyLengths[p], rawValues[p]); + } + writer.close(); + + if (!done) { + // Save the segment length + WritableUtils.writeVLong(indexOut, segmentStart); + WritableUtils.writeVLong(indexOut, (out.getPos() - segmentStart)); + indexOut.flush(); + } + } + + private void sort(int count) { + System.arraycopy(pointers, 0, pointersCopy, 0, count); + mergeSort.mergeSort(pointersCopy, pointers, 0, count); + } + + class SeqFileComparator implements Comparator { + @Override + public int compare(IntWritable I, IntWritable J) { + return comparator.compare(rawBuffer, keyOffsets[I.get()], + keyLengths[I.get()], rawBuffer, + keyOffsets[J.get()], keyLengths[J.get()]); + } + } + + /** set the progressable object in order to report progress */ + public void setProgressable(Progressable progressable) { + this.progressable = progressable; + } + + } // SequenceFile.Sorter.SortPass + + /** The interface to iterate over raw keys/values of SequenceFiles. */ + public static interface RawKeyValueIterator { + /** Gets the current raw key + * @return DataOutputBuffer + * @throws IOException + */ + DataOutputBuffer getKey() throws IOException; + + /** Gets the current raw value + * @return ValueBytes + * @throws IOException + */ + ValueBytes getValue() throws IOException; + + /** Sets up the current key and value (for getKey and getValue) + * @return true if there exists a key/value, false otherwise + * @throws IOException + */ + boolean next() throws IOException; + + /** closes the iterator so that the underlying streams can be closed + * @throws IOException + */ + void close() throws IOException; + + /** Gets the Progress object; this has a float (0.0 - 1.0) + * indicating the bytes processed by the iterator so far + */ + Progress getProgress(); + } + + /** + * Merges the list of segments of type SegmentDescriptor + * @param segments the list of SegmentDescriptors + * @param tmpDir the directory to write temporary files into + * @return RawKeyValueIterator + * @throws IOException + */ + public RawKeyValueIterator merge(List segments, + Path tmpDir) + throws IOException { + // pass in object to report progress, if present + MergeQueue mQueue = new MergeQueue(segments, tmpDir, progressable); + return mQueue.merge(); + } + + /** + * Merges the contents of files passed in Path[] using a max factor value + * that is already set + * @param inNames the array of path names + * @param deleteInputs true if the input files should be deleted when + * unnecessary + * @param tmpDir the directory to write temporary files into + * @return RawKeyValueIteratorMergeQueue + * @throws IOException + */ + public RawKeyValueIterator merge(Path[] inNames, boolean deleteInputs, + Path tmpDir) + throws IOException { + return merge(inNames, deleteInputs, + (inNames.length < factor) ? inNames.length : factor, + tmpDir); + } + + /** + * Merges the contents of files passed in Path[] + * @param inNames the array of path names + * @param deleteInputs true if the input files should be deleted when + * unnecessary + * @param factor the factor that will be used as the maximum merge fan-in + * @param tmpDir the directory to write temporary files into + * @return RawKeyValueIteratorMergeQueue + * @throws IOException + */ + public RawKeyValueIterator merge(Path[] inNames, boolean deleteInputs, + int factor, Path tmpDir) + throws IOException { + //get the segments from inNames + ArrayList a = new ArrayList(); + for (int i = 0; i < inNames.length; i++) { + SegmentDescriptor s = new SegmentDescriptor(0, + fs.getFileStatus(inNames[i]).getLen(), inNames[i]); + s.preserveInput(!deleteInputs); + s.doSync(); + a.add(s); + } + this.factor = factor; + MergeQueue mQueue = new MergeQueue(a, tmpDir, progressable); + return mQueue.merge(); + } + + /** + * Merges the contents of files passed in Path[] + * @param inNames the array of path names + * @param tempDir the directory for creating temp files during merge + * @param deleteInputs true if the input files should be deleted when + * unnecessary + * @return RawKeyValueIteratorMergeQueue + * @throws IOException + */ + public RawKeyValueIterator merge(Path[] inNames, Path tempDir, + boolean deleteInputs) + throws IOException { + //outFile will basically be used as prefix for temp files for the + //intermediate merge outputs + this.outFile = new Path(tempDir + Path.SEPARATOR + "merged"); + //get the segments from inNames + ArrayList a = new ArrayList(); + for (int i = 0; i < inNames.length; i++) { + SegmentDescriptor s = new SegmentDescriptor(0, + fs.getFileStatus(inNames[i]).getLen(), inNames[i]); + s.preserveInput(!deleteInputs); + s.doSync(); + a.add(s); + } + factor = (inNames.length < factor) ? inNames.length : factor; + // pass in object to report progress, if present + MergeQueue mQueue = new MergeQueue(a, tempDir, progressable); + return mQueue.merge(); + } + + /** + * Clones the attributes (like compression of the input file and creates a + * corresponding Writer + * @param inputFile the path of the input file whose attributes should be + * cloned + * @param outputFile the path of the output file + * @param prog the Progressable to report status during the file write + * @return Writer + * @throws IOException + */ + public Writer cloneFileAttributes(Path inputFile, Path outputFile, + Progressable prog) throws IOException { + Reader reader = new Reader(conf, + Reader.file(inputFile), + new Reader.OnlyHeaderOption()); + CompressionType compress = reader.getCompressionType(); + CompressionCodec codec = reader.getCompressionCodec(); + reader.close(); + + Writer writer = createWriter(conf, + Writer.file(outputFile), + Writer.keyClass(keyClass), + Writer.valueClass(valClass), + Writer.compression(compress, codec), + Writer.progressable(prog)); + return writer; + } + + /** + * Writes records from RawKeyValueIterator into a file represented by the + * passed writer + * @param records the RawKeyValueIterator + * @param writer the Writer created earlier + * @throws IOException + */ + public void writeFile(RawKeyValueIterator records, Writer writer) + throws IOException { + while (records.next()) { + writer.appendRaw(records.getKey().getData(), 0, + records.getKey().getLength(), records.getValue()); + } + writer.sync(); + } + + /** Merge the provided files. + * @param inFiles the array of input path names + * @param outFile the final output file + * @throws IOException + */ + public void merge(Path[] inFiles, Path outFile) throws IOException { + if (fs.exists(outFile)) { + throw new IOException("already exists: " + outFile); + } + RawKeyValueIterator r = merge(inFiles, false, outFile.getParent()); + Writer writer = cloneFileAttributes(inFiles[0], outFile, null); + + writeFile(r, writer); + + writer.close(); + } + + /** sort calls this to generate the final merged output */ + private int mergePass(Path tmpDir) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("running merge pass"); + } + Writer writer = cloneFileAttributes( + outFile.suffix(".0"), outFile, null); + RawKeyValueIterator r = merge(outFile.suffix(".0"), + outFile.suffix(".0.index"), tmpDir); + writeFile(r, writer); + + writer.close(); + return 0; + } + + /** Used by mergePass to merge the output of the sort + * @param inName the name of the input file containing sorted segments + * @param indexIn the offsets of the sorted segments + * @param tmpDir the relative directory to store intermediate results in + * @return RawKeyValueIterator + * @throws IOException + */ + private RawKeyValueIterator merge(Path inName, Path indexIn, Path tmpDir) + throws IOException { + //get the segments from indexIn + //we create a SegmentContainer so that we can track segments belonging to + //inName and delete inName as soon as we see that we have looked at all + //the contained segments during the merge process & hence don't need + //them anymore + SegmentContainer container = new SegmentContainer(inName, indexIn); + MergeQueue mQueue = new MergeQueue(container.getSegmentList(), tmpDir, progressable); + return mQueue.merge(); + } + + /** This class implements the core of the merge logic */ + private class MergeQueue extends PriorityQueue + implements RawKeyValueIterator { + private boolean compress; + private boolean blockCompress; + private DataOutputBuffer rawKey = new DataOutputBuffer(); + private ValueBytes rawValue; + private long totalBytesProcessed; + private float progPerByte; + private Progress mergeProgress = new Progress(); + private Path tmpDir; + private Progressable progress = null; //handle to the progress reporting object + private SegmentDescriptor minSegment; + + //a TreeMap used to store the segments sorted by size (segment offset and + //segment path name is used to break ties between segments of same sizes) + private Map sortedSegmentSizes = + new TreeMap(); + + @SuppressWarnings("unchecked") + public void put(SegmentDescriptor stream) throws IOException { + if (size() == 0) { + compress = stream.in.isCompressed(); + blockCompress = stream.in.isBlockCompressed(); + } else if (compress != stream.in.isCompressed() || + blockCompress != stream.in.isBlockCompressed()) { + throw new IOException("All merged files must be compressed or not."); + } + super.put(stream); + } + + /** + * A queue of file segments to merge + * @param segments the file segments to merge + * @param tmpDir a relative local directory to save intermediate files in + * @param progress the reference to the Progressable object + */ + public MergeQueue(List segments, + Path tmpDir, Progressable progress) { + int size = segments.size(); + for (int i = 0; i < size; i++) { + sortedSegmentSizes.put(segments.get(i), null); + } + this.tmpDir = tmpDir; + this.progress = progress; + } + + @Override + protected boolean lessThan(Object a, Object b) { + // indicate we're making progress + if (progress != null) { + progress.progress(); + } + SegmentDescriptor msa = (SegmentDescriptor) a; + SegmentDescriptor msb = (SegmentDescriptor) b; + return comparator.compare(msa.getKey().getData(), 0, + msa.getKey().getLength(), msb.getKey().getData(), 0, + msb.getKey().getLength()) < 0; + } + + @Override + public void close() throws IOException { + SegmentDescriptor ms; // close inputs + while ((ms = (SegmentDescriptor) pop()) != null) { + ms.cleanup(); + } + minSegment = null; + } + + @Override + public DataOutputBuffer getKey() throws IOException { + return rawKey; + } + + @Override + public ValueBytes getValue() throws IOException { + return rawValue; + } + + @Override + public boolean next() throws IOException { + if (size() == 0) + return false; + if (minSegment != null) { + //minSegment is non-null for all invocations of next except the first + //one. For the first invocation, the priority queue is ready for use + //but for the subsequent invocations, first adjust the queue + adjustPriorityQueue(minSegment); + if (size() == 0) { + minSegment = null; + return false; + } + } + minSegment = (SegmentDescriptor) top(); + long startPos = minSegment.in.getPosition(); // Current position in stream + //save the raw key reference + rawKey = minSegment.getKey(); + //load the raw value. Re-use the existing rawValue buffer + if (rawValue == null) { + rawValue = minSegment.in.createValueBytes(); + } + minSegment.nextRawValue(rawValue); + long endPos = minSegment.in.getPosition(); // End position after reading value + updateProgress(endPos - startPos); + return true; + } + + @Override + public Progress getProgress() { + return mergeProgress; + } + + private void adjustPriorityQueue(SegmentDescriptor ms) throws IOException { + long startPos = ms.in.getPosition(); // Current position in stream + boolean hasNext = ms.nextRawKey(); + long endPos = ms.in.getPosition(); // End position after reading key + updateProgress(endPos - startPos); + if (hasNext) { + adjustTop(); + } else { + pop(); + ms.cleanup(); + } + } + + private void updateProgress(long bytesProcessed) { + totalBytesProcessed += bytesProcessed; + if (progPerByte > 0) { + mergeProgress.set(totalBytesProcessed * progPerByte); + } + } + + /** This is the single level merge that is called multiple times + * depending on the factor size and the number of segments + * @return RawKeyValueIterator + * @throws IOException + */ + public RawKeyValueIterator merge() throws IOException { + //create the MergeStreams from the sorted map created in the constructor + //and dump the final output to a file + int numSegments = sortedSegmentSizes.size(); + int origFactor = factor; + int passNo = 1; + LocalDirAllocator lDirAlloc = new LocalDirAllocator("io.seqfile.local.dir"); + do { + //get the factor for this pass of merge + factor = getPassFactor(passNo, numSegments); + List segmentsToMerge = + new ArrayList(); + int segmentsConsidered = 0; + int numSegmentsToConsider = factor; + while (true) { + //extract the smallest 'factor' number of segment pointers from the + //TreeMap. Call cleanup on the empty segments (no key/value data) + SegmentDescriptor[] mStream = + getSegmentDescriptors(numSegmentsToConsider); + for (int i = 0; i < mStream.length; i++) { + if (mStream[i].nextRawKey()) { + segmentsToMerge.add(mStream[i]); + segmentsConsidered++; + // Count the fact that we read some bytes in calling nextRawKey() + updateProgress(mStream[i].in.getPosition()); + } else { + mStream[i].cleanup(); + numSegments--; //we ignore this segment for the merge + } + } + //if we have the desired number of segments + //or looked at all available segments, we break + if (segmentsConsidered == factor || + sortedSegmentSizes.size() == 0) { + break; + } + + numSegmentsToConsider = factor - segmentsConsidered; + } + //feed the streams to the priority queue + initialize(segmentsToMerge.size()); + clear(); + for (int i = 0; i < segmentsToMerge.size(); i++) { + put(segmentsToMerge.get(i)); + } + //if we have lesser number of segments remaining, then just return the + //iterator, else do another single level merge + if (numSegments <= factor) { + //calculate the length of the remaining segments. Required for + //calculating the merge progress + long totalBytes = 0; + for (int i = 0; i < segmentsToMerge.size(); i++) { + totalBytes += segmentsToMerge.get(i).segmentLength; + } + if (totalBytes != 0) //being paranoid + progPerByte = 1.0f / (float) totalBytes; + //reset factor to what it originally was + factor = origFactor; + return this; + } else { + //we want to spread the creation of temp files on multiple disks if + //available under the space constraints + long approxOutputSize = 0; + for (SegmentDescriptor s : segmentsToMerge) { + approxOutputSize += s.segmentLength + + ChecksumFileSystem.getApproxChkSumLength( + s.segmentLength); + } + Path tmpFilename = + new Path(tmpDir, "intermediate").suffix("." + passNo); + + Path outputFile = lDirAlloc.getLocalPathForWrite( + tmpFilename.toString(), + approxOutputSize, conf); + if (LOG.isDebugEnabled()) { + LOG.debug("writing intermediate results to " + outputFile); + } + Writer writer = cloneFileAttributes( + fs.makeQualified(segmentsToMerge.get(0).segmentPathName), + fs.makeQualified(outputFile), null); + writer.sync = null; //disable sync for temp files + writeFile(this, writer); + writer.close(); + + //we finished one single level merge; now clean up the priority + //queue + this.close(); + + SegmentDescriptor tempSegment = + new SegmentDescriptor(0, + fs.getFileStatus(outputFile).getLen(), outputFile); + //put the segment back in the TreeMap + sortedSegmentSizes.put(tempSegment, null); + numSegments = sortedSegmentSizes.size(); + passNo++; + } + //we are worried about only the first pass merge factor. So reset the + //factor to what it originally was + factor = origFactor; + } while (true); + } + + //Hadoop-591 + public int getPassFactor(int passNo, int numSegments) { + if (passNo > 1 || numSegments <= factor || factor == 1) + return factor; + int mod = (numSegments - 1) % (factor - 1); + if (mod == 0) + return factor; + return mod + 1; + } + + /** Return (& remove) the requested number of segment descriptors from the + * sorted map. + */ + public SegmentDescriptor[] getSegmentDescriptors(int numDescriptors) { + if (numDescriptors > sortedSegmentSizes.size()) + numDescriptors = sortedSegmentSizes.size(); + SegmentDescriptor[] SegmentDescriptors = + new SegmentDescriptor[numDescriptors]; + Iterator iter = sortedSegmentSizes.keySet().iterator(); + int i = 0; + while (i < numDescriptors) { + SegmentDescriptors[i++] = (SegmentDescriptor) iter.next(); + iter.remove(); + } + return SegmentDescriptors; + } + } // SequenceFile.Sorter.MergeQueue + + /** This class defines a merge segment. This class can be subclassed to + * provide a customized cleanup method implementation. In this + * implementation, cleanup closes the file handle and deletes the file + */ + public class SegmentDescriptor implements Comparable { + + long segmentOffset; //the start of the segment in the file + long segmentLength; //the length of the segment + Path segmentPathName; //the path name of the file containing the segment + boolean ignoreSync = true; //set to true for temp files + private Reader in = null; + private DataOutputBuffer rawKey = null; //this will hold the current key + private boolean preserveInput = false; //delete input segment files? + + /** Constructs a segment + * @param segmentOffset the offset of the segment in the file + * @param segmentLength the length of the segment + * @param segmentPathName the path name of the file containing the segment + */ + public SegmentDescriptor(long segmentOffset, long segmentLength, + Path segmentPathName) { + this.segmentOffset = segmentOffset; + this.segmentLength = segmentLength; + this.segmentPathName = segmentPathName; + } + + /** Do the sync checks */ + public void doSync() { + ignoreSync = false; + } + + /** Whether to delete the files when no longer needed */ + public void preserveInput(boolean preserve) { + preserveInput = preserve; + } + + public boolean shouldPreserveInput() { + return preserveInput; + } + + @Override + public int compareTo(Object o) { + SegmentDescriptor that = (SegmentDescriptor) o; + if (this.segmentLength != that.segmentLength) { + return (this.segmentLength < that.segmentLength ? -1 : 1); + } + if (this.segmentOffset != that.segmentOffset) { + return (this.segmentOffset < that.segmentOffset ? -1 : 1); + } + return (this.segmentPathName.toString()). + compareTo(that.segmentPathName.toString()); + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof SegmentDescriptor)) { + return false; + } + SegmentDescriptor that = (SegmentDescriptor) o; + if (this.segmentLength == that.segmentLength && + this.segmentOffset == that.segmentOffset && + this.segmentPathName.toString().equals( + that.segmentPathName.toString())) { + return true; + } + return false; + } + + @Override + public int hashCode() { + return 37 * 17 + (int) (segmentOffset ^ (segmentOffset >>> 32)); + } + + /** Fills up the rawKey object with the key returned by the Reader + * @return true if there is a key returned; false, otherwise + * @throws IOException + */ + public boolean nextRawKey() throws IOException { + if (in == null) { + int bufferSize = getBufferSize(conf); + Reader reader = new Reader(conf, + Reader.file(segmentPathName), + Reader.bufferSize(bufferSize), + Reader.start(segmentOffset), + Reader.length(segmentLength)); + + //sometimes we ignore syncs especially for temp merge files + if (ignoreSync) reader.ignoreSync(); + + if (reader.getKeyClass() != keyClass) + throw new IOException("wrong key class: " + reader.getKeyClass() + + " is not " + keyClass); + if (reader.getValueClass() != valClass) + throw new IOException("wrong value class: " + reader.getValueClass() + + " is not " + valClass); + this.in = reader; + rawKey = new DataOutputBuffer(); + } + rawKey.reset(); + int keyLength = + in.nextRawKey(rawKey); + return (keyLength >= 0); + } + + /** Fills up the passed rawValue with the value corresponding to the key + * read earlier + * @param rawValue + * @return the length of the value + * @throws IOException + */ + public int nextRawValue(ValueBytes rawValue) throws IOException { + int valLength = in.nextRawValue(rawValue); + return valLength; + } + + /** Returns the stored rawKey */ + public DataOutputBuffer getKey() { + return rawKey; + } + + /** closes the underlying reader */ + private void close() throws IOException { + this.in.close(); + this.in = null; + } + + /** The default cleanup. Subclasses can override this with a custom + * cleanup + */ + public void cleanup() throws IOException { + close(); + if (!preserveInput) { + fs.delete(segmentPathName, true); + } + } + } // SequenceFile.Sorter.SegmentDescriptor + + /** This class provisions multiple segments contained within a single + * file + */ + private class LinkedSegmentsDescriptor extends SegmentDescriptor { + + SegmentContainer parentContainer = null; + + /** Constructs a segment + * @param segmentOffset the offset of the segment in the file + * @param segmentLength the length of the segment + * @param segmentPathName the path name of the file containing the segment + * @param parent the parent SegmentContainer that holds the segment + */ + public LinkedSegmentsDescriptor(long segmentOffset, long segmentLength, + Path segmentPathName, SegmentContainer parent) { + super(segmentOffset, segmentLength, segmentPathName); + this.parentContainer = parent; + } + + /** The default cleanup. Subclasses can override this with a custom + * cleanup + */ + @Override + public void cleanup() throws IOException { + super.close(); + if (super.shouldPreserveInput()) return; + parentContainer.cleanup(); + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof LinkedSegmentsDescriptor)) { + return false; + } + return super.equals(o); + } + } //SequenceFile.Sorter.LinkedSegmentsDescriptor + + /** The class that defines a container for segments to be merged. Primarily + * required to delete temp files as soon as all the contained segments + * have been looked at */ + private class SegmentContainer { + private int numSegmentsCleanedUp = 0; //track the no. of segment cleanups + private int numSegmentsContained; //# of segments contained + private Path inName; //input file from where segments are created + + //the list of segments read from the file + private ArrayList segments = + new ArrayList(); + + /** This constructor is there primarily to serve the sort routine that + * generates a single output file with an associated index file */ + public SegmentContainer(Path inName, Path indexIn) throws IOException { + //get the segments from indexIn + FSDataInputStream fsIndexIn = fs.open(indexIn); + long end = fs.getFileStatus(indexIn).getLen(); + while (fsIndexIn.getPos() < end) { + long segmentOffset = WritableUtils.readVLong(fsIndexIn); + long segmentLength = WritableUtils.readVLong(fsIndexIn); + Path segmentName = inName; + segments.add(new LinkedSegmentsDescriptor(segmentOffset, + segmentLength, segmentName, this)); + } + fsIndexIn.close(); + fs.delete(indexIn, true); + numSegmentsContained = segments.size(); + this.inName = inName; + } + + public List getSegmentList() { + return segments; + } + + public void cleanup() throws IOException { + numSegmentsCleanedUp++; + if (numSegmentsCleanedUp == numSegmentsContained) { + fs.delete(inName, true); + } + } + } //SequenceFile.Sorter.SegmentContainer + + } // SequenceFile.Sorter + +} // SequenceFile diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java new file mode 100644 index 000000000..1e7448475 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommitMetadata.java @@ -0,0 +1,190 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.codehaus.jackson.annotate.JsonAutoDetect; +import org.codehaus.jackson.annotate.JsonMethod; +import org.codehaus.jackson.map.ObjectMapper; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * All the metadata that gets stored along with a commit. + */ +public class HoodieCommitMetadata implements Serializable { + private static volatile Logger log = LogManager.getLogger(HoodieCommitMetadata.class); + private HashMap> partitionToWriteStats; + + public HoodieCommitMetadata() { + partitionToWriteStats = new HashMap<>(); + } + + public void addWriteStat(String partitionPath, HoodieWriteStat stat) { + if (!partitionToWriteStats.containsKey(partitionPath)) { + partitionToWriteStats.put(partitionPath, new ArrayList()); + } + partitionToWriteStats.get(partitionPath).add(stat); + } + + public List getWriteStats(String partitionPath) { + return partitionToWriteStats.get(partitionPath); + } + + public HashMap> getPartitionToWriteStats() { + return partitionToWriteStats; + } + + public HashMap getFileIdAndFullPaths() { + HashMap filePaths = new HashMap<>(); + // list all partitions paths + for (Map.Entry> entry: getPartitionToWriteStats().entrySet()) { + for (HoodieWriteStat stat: entry.getValue()) { + filePaths.put(stat.getFileId(), stat.getFullPath()); + } + } + return filePaths; + } + + + public String toJsonString() throws IOException { + if(partitionToWriteStats.containsKey(null)) { + log.info("partition path is null for " + partitionToWriteStats.get(null)); + partitionToWriteStats.remove(null); + } + ObjectMapper mapper = new ObjectMapper(); + mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY); + return mapper.defaultPrettyPrintingWriter().writeValueAsString(this); + } + + public static HoodieCommitMetadata fromJsonString(String jsonStr) throws IOException { + if (jsonStr == null || jsonStr.isEmpty()) { + // For empty commit file (no data or somethings bad happen). + return new HoodieCommitMetadata(); + } + ObjectMapper mapper = new ObjectMapper(); + mapper.setVisibility(JsonMethod.FIELD, JsonAutoDetect.Visibility.ANY); + return mapper.readValue(jsonStr, HoodieCommitMetadata.class); + } + + // Here the functions are named "fetch" instead of "get", to get avoid of the json conversion. + public long fetchTotalPartitionsWritten() { + return partitionToWriteStats.size(); + } + + public long fetchTotalFilesInsert() { + long totalFilesInsert = 0; + for (List stats : partitionToWriteStats.values()) { + for (HoodieWriteStat stat : stats) { + if (stat.getPrevCommit() != null && stat.getPrevCommit().equals("null")) { + totalFilesInsert ++; + } + } + } + return totalFilesInsert; + } + + public long fetchTotalFilesUpdated() { + long totalFilesUpdated = 0; + for (List stats : partitionToWriteStats.values()) { + for (HoodieWriteStat stat : stats) { + if (stat.getPrevCommit() != null && !stat.getPrevCommit().equals("null")) { + totalFilesUpdated ++; + } + } + } + return totalFilesUpdated; + } + + public long fetchTotalUpdateRecordsWritten() { + long totalUpdateRecordsWritten = 0; + for (List stats : partitionToWriteStats.values()) { + for (HoodieWriteStat stat : stats) { + totalUpdateRecordsWritten += stat.getNumUpdateWrites(); + } + } + return totalUpdateRecordsWritten; + } + + public long fetchTotalInsertRecordsWritten() { + long totalInsertRecordsWritten = 0; + for (List stats : partitionToWriteStats.values()) { + for (HoodieWriteStat stat : stats) { + if (stat.getPrevCommit() != null && stat.getPrevCommit().equals("null")) { + totalInsertRecordsWritten += stat.getNumWrites(); + } + } + } + return totalInsertRecordsWritten; + } + + public long fetchTotalRecordsWritten() { + long totalRecordsWritten = 0; + for (List stats : partitionToWriteStats.values()) { + for (HoodieWriteStat stat : stats) { + totalRecordsWritten += stat.getNumWrites(); + } + } + return totalRecordsWritten; + } + + public long fetchTotalBytesWritten() { + long totalBytesWritten = 0; + for (List stats : partitionToWriteStats.values()) { + for (HoodieWriteStat stat : stats) { + totalBytesWritten += stat.getTotalWriteBytes(); + } + } + return totalBytesWritten; + } + + public long fetchTotalWriteErrors() { + long totalWriteErrors = 0; + for (List stats : partitionToWriteStats.values()) { + for (HoodieWriteStat stat : stats) { + totalWriteErrors += stat.getTotalWriteErrors(); + } + } + return totalWriteErrors; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + HoodieCommitMetadata that = (HoodieCommitMetadata) o; + + return partitionToWriteStats != null ? + partitionToWriteStats.equals(that.partitionToWriteStats) : + that.partitionToWriteStats == null; + + } + + @Override + public int hashCode() { + return partitionToWriteStats != null ? partitionToWriteStats.hashCode() : 0; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommits.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommits.java new file mode 100644 index 000000000..99ea51364 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieCommits.java @@ -0,0 +1,191 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Manages the commit meta and provides operations on the commit timeline + */ +public class HoodieCommits implements Serializable { + + private List commitList; + + public HoodieCommits(List commitList) { + this.commitList = new ArrayList<>(commitList); + Collections.sort(this.commitList); + this.commitList = Collections.unmodifiableList(this.commitList); + } + + /** + * Returns the commits which are in the range (startsTs, endTs]. + * + * @param startTs - exclusive start commit ts + * @param endTs - inclusive end commit ts + */ + public List findCommitsInRange(String startTs, String endTs) { + if (commitList.isEmpty()) { + return Collections.EMPTY_LIST; + } + int startIndex = 0; + if (startTs != null) { + startIndex = Collections.binarySearch(commitList, startTs); + // If startIndex is negative + if (startIndex < 0) { + startIndex = -(startIndex + 1); + } + } + + int endIndex = Collections.binarySearch(commitList, endTs); + // If endIndex is negative + if (endIndex < 0) { + endIndex = -(endIndex + 1); + } + + if (endIndex < startIndex) { + throw new IllegalArgumentException( + "Start Commit Ts " + startTs + " cannot be less than end commit ts" + endTs); + } + List returns = new ArrayList<>(commitList.subList(startIndex, endIndex)); + if(endIndex < commitList.size()) { + // Be inclusive of the endIndex + returns.add(commitList.get(endIndex)); + } + return Collections.unmodifiableList(returns); + } + + /** + * Finds the list of commits on or before asOfTs + */ + public List findCommitsAfter(String commitTimeStamp, int numCommits) { + if (commitList.isEmpty()) { + return null; + } + + int startIndex = Collections.binarySearch(commitList, commitTimeStamp); + if (startIndex < 0) { + startIndex = -(startIndex + 1); + } else { + // we found asOfTs at startIndex. We want to exclude it. + startIndex++; + } + + + List commits = new ArrayList<>(); + while (numCommits > 0 && startIndex < commitList.size()) { + commits.add(commitList.get(startIndex)); + startIndex++; + numCommits--; + } + + return Collections.unmodifiableList(commits); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HoodieCommits{"); + sb.append("commitList=").append(commitList); + sb.append('}'); + return sb.toString(); + } + + public boolean isEmpty() { + return commitList.isEmpty(); + } + + public int getNumCommits() { + return commitList.size(); + } + + public String firstCommit() { + return commitList.isEmpty() ? null : commitList.get(0); + } + + public String nthCommit(int n) { + return commitList.isEmpty() || n >= commitList.size() ? null : commitList.get(n); + } + + public String lastCommit() { + return commitList.isEmpty() ? null : commitList.get(commitList.size() - 1); + } + + /** + * Returns the nth commit from the latest commit such that lastCommit(0) => lastCommit() + */ + public String lastCommit(int n) { + if (commitList.size() < n + 1) { + return null; + } + return commitList.get(commitList.size() - 1 - n); + } + + public boolean contains(String commitTs) { + return commitList.contains(commitTs); + } + + public String max(String commit1, String commit2) { + if (commit1 == null && commit2 == null) { + return null; + } + if (commit1 == null) { + return commit2; + } + if (commit2 == null) { + return commit1; + } + return (isCommit1BeforeOrOn(commit1, commit2) ? commit2 : commit1); + } + + public static boolean isCommit1BeforeOrOn(String commit1, String commit2) { + return commit1.compareTo(commit2) <= 0; + } + + public static boolean isCommit1After(String commit1, String commit2) { + return commit1.compareTo(commit2) > 0; + } + + public List getCommitList() { + return commitList; + } + + public boolean isCommitBeforeEarliestCommit(String commitTs) { + return isCommit1BeforeOrOn(commitTs, firstCommit()); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + HoodieCommits that = (HoodieCommits) o; + + return commitList != null ? commitList.equals(that.commitList) : that.commitList == null; + + } + + @Override + public int hashCode() { + return commitList != null ? commitList.hashCode() : 0; + } + +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFile.java new file mode 100644 index 000000000..ca3f46b2d --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFile.java @@ -0,0 +1,57 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import com.uber.hoodie.common.util.FSUtils; + +import org.apache.hadoop.fs.FileStatus; + +public class HoodieFile { + + private final FileStatus fileStatus; + private String fileNameWithoutCommitTs; + private String commitTs; + + public HoodieFile(FileStatus fileStatus) { + this.fileStatus = fileStatus; + String fileName = fileStatus.getPath().getName(); + this.fileNameWithoutCommitTs = FSUtils.getFileId(fileName); + this.commitTs = FSUtils.getCommitTime(fileName); + } + + public String getFileNameWithoutCommitTs() { + return fileNameWithoutCommitTs; + } + + public String getCommitTs() { + return commitTs; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HoodieFile{"); + sb.append("fileStatus=").append(fileStatus); + sb.append(", fileNameWithoutCommitTs='").append(fileNameWithoutCommitTs).append('\''); + sb.append(", commitTs='").append(commitTs).append('\''); + sb.append('}'); + return sb.toString(); + } + + public FileStatus getFileStatus() { + return fileStatus; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java new file mode 100644 index 000000000..9dcc1528c --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieKey.java @@ -0,0 +1,73 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import com.google.common.base.Objects; + +import java.io.Serializable; + +/** + * HoodieKey consists of + * + * - recordKey : a recordKey that acts as primary key for a record - partitionPath : path to the + * partition that contains the record + */ +public class HoodieKey implements Serializable { + + + private final String recordKey; + + private final String partitionPath; + + public HoodieKey(String recordKey, String partitionPath) { + this.recordKey = recordKey; + this.partitionPath = partitionPath; + } + + public String getRecordKey() { + return recordKey; + } + + public String getPartitionPath() { + return partitionPath; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + HoodieKey otherKey = (HoodieKey) o; + return Objects.equal(recordKey, otherKey.recordKey) && + Objects.equal(partitionPath, otherKey.partitionPath); + } + + @Override + public int hashCode() { + return Objects.hashCode(recordKey, partitionPath); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HoodieKey {"); + sb.append(" recordKey=").append(recordKey); + sb.append(" partitionPath=").append(partitionPath); + sb.append('}'); + return sb.toString(); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java new file mode 100644 index 000000000..01f3fd42d --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java @@ -0,0 +1,153 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import com.google.common.base.Objects; + +import java.io.Serializable; + +/** + * A Single Record managed by Hoodie TODO - Make this generic + */ +public class HoodieRecord implements Serializable { + + public static String COMMIT_TIME_METADATA_FIELD = "_hoodie_commit_time"; + public static String COMMIT_SEQNO_METADATA_FIELD = "_hoodie_commit_seqno"; + public static String RECORD_KEY_METADATA_FIELD = "_hoodie_record_key"; + public static String PARTITION_PATH_METADATA_FIELD = "_hoodie_partition_path"; + public static String FILENAME_METADATA_FIELD = "_hoodie_file_name"; + + /** + * Identifies the record across the table + */ + private HoodieKey key; + + /** + * Actual payload of the record + */ + private T data; + + /** + * Current location of record on storage. Filled in by looking up index + */ + private HoodieRecordLocation currentLocation; + + /** + * New location of record on storage, after written + */ + private HoodieRecordLocation newLocation; + + public HoodieRecord(HoodieKey key, T data) { + this.key = key; + this.data = data; + this.currentLocation = null; + this.newLocation = null; + } + + public HoodieKey getKey() { + return key; + } + + public T getData() { + if (data == null) { + throw new IllegalStateException("Payload already deflated for record."); + } + return data; + } + + /** + * Release the actual payload, to ease memory pressure. To be called after the record + * has been written to storage. Once deflated, cannot be inflated. + */ + public void deflate() { + this.data = null; + } + + + /** + * Sets the current currentLocation of the record. This should happen exactly-once + */ + public HoodieRecord setCurrentLocation(HoodieRecordLocation location) { + assert currentLocation == null; + this.currentLocation = location; + return this; + } + + public HoodieRecordLocation getCurrentLocation() { + return currentLocation; + } + + /** + * Sets the new currentLocation of the record, after being written. This again should happen + * exactly-once. + */ + public HoodieRecord setNewLocation(HoodieRecordLocation location) { + assert newLocation == null; + this.newLocation = location; + return this; + } + + public HoodieRecordLocation getNewLocation() { + return this.newLocation; + } + + public boolean isCurrentLocationKnown() { + return this.currentLocation != null; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + HoodieRecord that = (HoodieRecord) o; + return Objects.equal(key, that.key) && + Objects.equal(data, that.data) && + Objects.equal(currentLocation, that.currentLocation) && + Objects.equal(newLocation, that.newLocation); + } + + @Override + public int hashCode() { + return Objects.hashCode(key, data, currentLocation, newLocation); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HoodieRecord{"); + sb.append("key=").append(key); + sb.append(", currentLocation='").append(currentLocation).append('\''); + sb.append(", newLocation='").append(newLocation).append('\''); + sb.append('}'); + return sb.toString(); + } + + public static String generateSequenceId(String commitTime, int partitionId, long recordIndex) { + return commitTime + "_" + partitionId + "_" + recordIndex; + } + + public String getPartitionPath() { + assert key != null; + return key.getPartitionPath(); + } + + public String getRecordKey() { + assert key != null; + return key.getRecordKey(); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java new file mode 100644 index 000000000..c84b028ff --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordLocation.java @@ -0,0 +1,69 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import com.google.common.base.Objects; + +import java.io.Serializable; + +/** + * Location of a HoodieRecord within the parition it belongs to. Ultimately, this points to an + * actual file on disk + */ +public class HoodieRecordLocation implements Serializable { + + private final String commitTime; + private final String fileId; + + public HoodieRecordLocation(String commitTime, String fileId) { + this.commitTime = commitTime; + this.fileId = fileId; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + HoodieRecordLocation otherLoc = (HoodieRecordLocation) o; + return Objects.equal(commitTime, otherLoc.commitTime) && + Objects.equal(fileId, otherLoc.fileId); + } + + @Override + public int hashCode() { + return Objects.hashCode(commitTime, fileId); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HoodieRecordLocation {"); + sb.append("commitTime=").append(commitTime).append(", "); + sb.append("fileId=").append(fileId); + sb.append('}'); + return sb.toString(); + } + + public String getCommitTime() { + return commitTime; + } + + public String getFileId() { + return fileId; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java new file mode 100644 index 000000000..28af727f1 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java @@ -0,0 +1,57 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; + +import java.io.IOException; +import java.io.Serializable; + +/** + * Every Hoodie dataset has an implementation of the HoodieRecordPayload + * This abstracts out callbacks which depend on record specific logic + */ +public interface HoodieRecordPayload extends Serializable { + /** + * When more than one HoodieRecord have the same HoodieKey, this function combines them + * before attempting to insert/upsert (if combining turned on in HoodieClientConfig) + */ + T preCombine(T another); + + /** + * + * This methods lets you write custom merging/combining logic to produce new values + * as a function of current value on storage and whats contained in this object. + * + * eg: + * 1) You are updating counters, you may want to add counts to currentValue and write back updated counts + * 2) You may be reading DB redo logs, and merge them with current image for a database row on storage + * + * @param currentValue Current value in storage, to merge/combine this payload with + * @param schema Schema used for record + * @return new combined/merged value to be written back to storage + */ + IndexedRecord combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException; + + /** + * Generates an avro record out of the given HoodieRecordPayload, to be written out to storage. + * Called when writing a new value for the given HoodieKey, wherein there is no existing record in + * storage to be combined against. (i.e insert) + */ + IndexedRecord getInsertValue(Schema schema) throws IOException; +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableMetadata.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableMetadata.java new file mode 100644 index 000000000..b87fd3fd4 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableMetadata.java @@ -0,0 +1,480 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import com.uber.hoodie.common.util.FSUtils; + +import com.uber.hoodie.exception.DatasetNotFoundException; +import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.exception.InvalidDatasetException; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.SortedMap; +import java.util.TreeMap; + +/** + * Manages all file system level interactions for the Hoodie tables. + */ +public class HoodieTableMetadata implements Serializable { + public static final String MAX_COMMIT_TS = String.valueOf(Long.MAX_VALUE); + public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name"; + public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type"; + public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE; + + public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties"; + private static final String HOODIE_HDRONE_PROFILE_DEFAULT_VALUE = "HOODIE"; + private static final java.lang.String HOODIE_HDRONE_PROFILE_PROP_NAME = + "hoodie.hdrone.dataset.profile"; + + private static Logger log = LogManager.getLogger(HoodieTableMetadata.class); + private transient final FileSystem fs; + private transient final Path metadataFolder; + private final Properties properties; + private HoodieCommits commits; + private List inflightCommits; + private String basePath; + + public static final String METAFOLDER_NAME = ".hoodie"; + public static final String COMMIT_FILE_SUFFIX = ".commit"; + public static final String INFLIGHT_FILE_SUFFIX = ".inflight"; + + /** + * Constructor which initializes the hoodie table metadata. It will initialize the meta-data if not already present. + * + * @param fs + * @param basePath + * @param tableName + * @throws IOException + */ + public HoodieTableMetadata(FileSystem fs, String basePath, String tableName) { + this(fs, basePath, tableName, true); + } + + /** + * Constructor which loads the hoodie table metadata, It requires the meta-data to be present already + * @param fs + * @param basePath + * @throws IOException + */ + public HoodieTableMetadata(FileSystem fs, String basePath) { + this(fs, basePath, null, false); + } + + private HoodieTableMetadata(FileSystem fs, String basePath, String tableName, + boolean initOnMissing) { + this.fs = fs; + this.basePath = basePath; + + try { + Path basePathDir = new Path(this.basePath); + if (!fs.exists(basePathDir)) { + if (initOnMissing) { + fs.mkdirs(basePathDir); + } else { + throw new DatasetNotFoundException(this.basePath); + } + } + + if (!fs.isDirectory(new Path(basePath))) { + throw new DatasetNotFoundException(this.basePath); + } + + // create .hoodie folder if it does not exist. + this.metadataFolder = new Path(this.basePath, METAFOLDER_NAME); + Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE); + if (!fs.exists(propertyPath)) { + if (initOnMissing) { + createHoodieProperties(metadataFolder, tableName); + } else { + throw new InvalidDatasetException(this.basePath); + } + } + + // Load meta data + this.commits = new HoodieCommits(scanCommits(COMMIT_FILE_SUFFIX)); + this.inflightCommits = scanCommits(INFLIGHT_FILE_SUFFIX); + this.properties = readHoodieProperties(); + log.info("All commits :" + commits); + } catch (IOException e) { + throw new HoodieIOException("Could not load HoodieMetadata from path " + basePath, e); + } + } + + /** + * Returns all the commit metadata for this table. Reads all the commit files from HDFS. + * Expensive operation, use with caution. + * + * @return SortedMap of CommitTime,HoodieCommitMetadata + * @throws IOException + */ + public SortedMap getAllCommitMetadata() { + try { + TreeMap metadataMap = new TreeMap<>(); + for (String commitTs : commits.getCommitList()) { + metadataMap.put(commitTs, getCommitMetadata(commitTs)); + } + return Collections.unmodifiableSortedMap(metadataMap); + } catch (IOException e) { + throw new HoodieIOException("Could not load all commits for table " + getTableName(), + e); + } + } + + public HoodieCommitMetadata getCommitMetadata(String commitTime) throws IOException { + FSDataInputStream is = fs.open(new Path(metadataFolder, FSUtils.makeCommitFileName(commitTime))); + try { + String jsonStr = IOUtils.toString(is); + return HoodieCommitMetadata.fromJsonString(jsonStr); + } finally { + is.close(); + } + } + + public HoodieTableType getTableType() { + return HoodieTableType.valueOf(properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME)); + } + + /** + * Lookup the file name for specified HoodieRecord + *

+ * TODO(vc): This metadata needs to be cached in each executor, statically, and used across, if + * we need to be nicer to the NameNode + */ + public String getFilenameForRecord(FileSystem fs, final HoodieRecord record) { + String fileId = record.getCurrentLocation().getFileId(); + return getFilenameForRecord(fs, record, fileId); + } + + + public String getFilenameForRecord(FileSystem fs, final HoodieRecord record, String fileId) { + try { + FileStatus[] files = fs.listStatus(new Path(basePath, record.getPartitionPath())); + Map> fileIdToVersions = + groupFilesByFileId(files, commits.lastCommit()); + // If the record is not found + if(!fileIdToVersions.containsKey(fileId)) { + throw new FileNotFoundException("Cannot find valid versions for fileId " + fileId); + } + + List statuses = fileIdToVersions.get(fileId); + return statuses.get(0).getPath().getName(); + } catch (IOException e) { + throw new HoodieIOException( + "Could not get Filename for record " + record, e); + } + } + + + + /** + * Get only the latest file in the partition with precondition commitTime(file) < maxCommitTime + * + * @param fs + * @param partitionPathStr + * @param maxCommitTime + * @return + */ + public FileStatus[] getLatestVersionInPartition(FileSystem fs, String partitionPathStr, + String maxCommitTime) { + try { + Path partitionPath = new Path(basePath, partitionPathStr); + if(!fs.exists(partitionPath)) { + return new FileStatus[0]; + } + FileStatus[] files = fs.listStatus(partitionPath); + Map> fileIdToVersions = + groupFilesByFileId(files, commits.lastCommit()); + HashMap validFiles = new HashMap<>(); + for (String fileId : fileIdToVersions.keySet()) { + List versions = fileIdToVersions.get(fileId); + for (FileStatus file : versions) { + String filename = file.getPath().getName(); + String commitTime = FSUtils.getCommitTime(filename); + if (HoodieCommits.isCommit1BeforeOrOn(commitTime, maxCommitTime)) { + validFiles.put(fileId, file); + break; + } + } + } + return validFiles.values().toArray(new FileStatus[validFiles.size()]); + } catch (IOException e) { + throw new HoodieIOException( + "Could not get latest versions in Partition " + partitionPathStr, e); + } + } + + /** + * Get ALL the data files in partition grouped by fileId and sorted by the commitTime + * Given a partition path, provide all the files with a list of their commits, sorted by commit time. + */ + public Map> getAllVersionsInPartition(FileSystem fs, String partitionPath) { + try { + FileStatus[] files = fs.listStatus(new Path(basePath, partitionPath)); + return groupFilesByFileId(files, commits.lastCommit()); + } catch (IOException e) { + throw new HoodieIOException( + "Could not load all file versions in partition " + partitionPath, e); + } + } + + /** + * Get all the versions of files, within the commit range provided. + * + * @param commitsToReturn - commits to include + */ + public FileStatus[] getLatestVersionInRange(FileStatus[] fileStatuses, List commitsToReturn) { + if (commitsToReturn.isEmpty()) { + return new FileStatus[0]; + } + try { + Map> fileIdToVersions = + groupFilesByFileId(fileStatuses, commits.lastCommit()); + + List statuses = new ArrayList<>(); + for (List entry : fileIdToVersions.values()) { + for (FileStatus status : entry) { + String commitTime = FSUtils.getCommitTime(status.getPath().getName()); + if (commitsToReturn.contains(commitTime)) { + statuses.add(status); + break; + } + } + } + return statuses.toArray(new FileStatus[statuses.size()]); + } catch (IOException e) { + throw new HoodieIOException("Could not filter files from commits " + commitsToReturn, e); + } + } + + /** + * + * Get the latest versions of all the files. + * + * @param fileStatuses + * @return + */ + public FileStatus[] getLatestVersions(FileStatus[] fileStatuses) { + try { + Map> fileIdToVersions = + groupFilesByFileId(fileStatuses, commits.lastCommit()); + + List statuses = new ArrayList<>(); + for(List entry:fileIdToVersions.values()) { + // first file is the latest one + statuses.add(entry.get(0)); + } + return statuses.toArray(new FileStatus[statuses.size()]); + } catch (IOException e) { + throw new HoodieIOException("Could not filter files for latest version ", e); + } + } + + + /** + * Get the base path for the Hoodie Table + * + * @return + */ + public String getBasePath() { + return basePath; + } + + + public boolean isCommitsEmpty() { + return commits.isEmpty(); + } + + public boolean isCommitTsSafe(String commitTs) { + return !isCommitsEmpty() && (commits.isCommitBeforeEarliestCommit(commitTs) || commits + .contains(commitTs)); + } + + public List findCommitsSinceTs(String startTs) { + return commits.findCommitsInRange(startTs, MAX_COMMIT_TS); + } + + public List findCommitsInRange(String startTs, String endTs) { + return commits.findCommitsInRange(startTs, endTs); + } + + public List findCommitsAfter(String startTs, Integer maxCommits) { + return commits.findCommitsAfter(startTs, maxCommits); + } + + public HoodieCommits getAllCommits() { + return commits; + } + + public List getAllInflightCommits() { + return inflightCommits; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HoodieTableMetadata{"); + sb.append("commits=").append(commits); + sb.append('}'); + return sb.toString(); + } + + public String getTableName() { + return properties.getProperty(HOODIE_TABLE_NAME_PROP_NAME); + } + + public String getHDroneDatasetProfile() { + return properties.getProperty(HOODIE_HDRONE_PROFILE_PROP_NAME, HOODIE_HDRONE_PROFILE_DEFAULT_VALUE); + } + + /** + * Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties) + * + * @param metadataFolder + * @param tableName + * @throws IOException + */ + private void createHoodieProperties(Path metadataFolder, String tableName) throws IOException { + if (!fs.exists(metadataFolder)) { + fs.mkdirs(metadataFolder); + } + Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE); + FSDataOutputStream outputStream = fs.create(propertyPath); + try { + Properties props = new Properties(); + props.setProperty(HOODIE_TABLE_NAME_PROP_NAME, tableName); + props.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name()); + props + .store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); + } finally { + outputStream.close(); + } + } + + /** + * Loads the hoodie table properties from the hoodie.properties file under the .hoodie path + */ + private Properties readHoodieProperties() throws IOException { + Properties props = new Properties(); + Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE); + FSDataInputStream inputStream = fs.open(propertyPath); + try { + props.load(inputStream); + } finally { + inputStream.close(); + } + return props; + } + + /** + * Scan the commit times (only choosing commit file with the given suffix) + */ + private List scanCommits(final String commitFileSuffix) throws IOException { + log.info("Attempting to load the commits under " + metadataFolder + " with suffix " + commitFileSuffix); + final List commitFiles = new ArrayList<>(); + fs.listStatus(metadataFolder, new PathFilter() { + @Override + public boolean accept(Path path) { + if (path.getName().endsWith(commitFileSuffix)) { + commitFiles.add(path.getName().split("\\.")[0]); + return true; + } + return false; + } + }); + return commitFiles; + } + + /** + * Takes a bunch of file versions, and returns a map keyed by fileId, with the necessary + * version safety checking. Returns a map of commitTime and Sorted list of FileStats + * ( by reverse commit time ) + * + * @param maxCommitTime maximum permissible commit time + * + * @return + */ + private Map> groupFilesByFileId(FileStatus[] files, + String maxCommitTime) throws IOException { + HashMap> fileIdtoVersions = new HashMap<>(); + for (FileStatus file : files) { + String filename = file.getPath().getName(); + String fileId = FSUtils.getFileId(filename); + String commitTime = FSUtils.getCommitTime(filename); + if (isCommitTsSafe(commitTime) && HoodieCommits + .isCommit1BeforeOrOn(commitTime, maxCommitTime)) { + if (!fileIdtoVersions.containsKey(fileId)) { + fileIdtoVersions.put(fileId, new ArrayList()); + } + fileIdtoVersions.get(fileId).add(file); + } + } + for (Map.Entry> entry : fileIdtoVersions.entrySet()) { + Collections.sort(fileIdtoVersions.get(entry.getKey()), new Comparator() { + @Override + public int compare(FileStatus o1, FileStatus o2) { + String o1CommitTime = FSUtils.getCommitTime(o1.getPath().getName()); + String o2CommitTime = FSUtils.getCommitTime(o2.getPath().getName()); + // Reverse the order + return o2CommitTime.compareTo(o1CommitTime); + } + }); + } + return fileIdtoVersions; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + HoodieTableMetadata metadata = (HoodieTableMetadata) o; + + if (commits != null ? !commits.equals(metadata.commits) : metadata.commits != null) + return false; + return basePath != null ? basePath.equals(metadata.basePath) : metadata.basePath == null; + + } + + @Override + public int hashCode() { + int result = commits != null ? commits.hashCode() : 0; + result = 31 * result + (basePath != null ? basePath.hashCode() : 0); + return result; + } + +} + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java new file mode 100644 index 000000000..d37252045 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieTableType.java @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +/** + * Type of the Hoodie Table. + * + * Currently, 1 type is supported + * + * COPY_ON_WRITE - Performs upserts by versioning entire files, with later versions containing newer + * value of a record. + * + * In the future, following might be added. + * + * MERGE_ON_READ - Speeds up upserts, by delaying merge until enough work piles up. + * + * SIMPLE_LSM - A simple 2 level LSM tree. + */ +public enum HoodieTableType { + COPY_ON_WRITE +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java new file mode 100644 index 000000000..a29d07428 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java @@ -0,0 +1,158 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.uber.hoodie.common.util.FSUtils; + +import java.io.Serializable; + +/** + * Statistics about a single Hoodie write operation. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class HoodieWriteStat implements Serializable { + + public static final String NULL_COMMIT = "null"; + + /** + * Id of the file being written + */ + private String fileId; + + /** + * Full path to the file on underlying file system + */ + private String fullPath; + + /** + * The previous version of the file. (null if this is the first version. i.e insert) + */ + private String prevCommit; + + /** + * Total number of records written for this file. + * - for updates, its the entire number of records in the file + * - for inserts, its the actual number of records inserted. + */ + private long numWrites; + + /** + * Total number of records actually changed. (0 for inserts) + */ + private long numUpdateWrites; + + /** + * Total size of file written + */ + private long totalWriteBytes; + + /** + * Total number of records, that were n't able to be written due to errors. + */ + private long totalWriteErrors; + + public HoodieWriteStat() { + // called by jackson json lib + } + + public void setFileId(String fileId) { + this.fileId = fileId; + } + + public void setFullPath(String fullFilePath) { + this.fullPath = fullFilePath; + } + + public void setPrevCommit(String prevCommit) { + this.prevCommit = prevCommit; + } + + public void setNumWrites(long numWrites) { + this.numWrites = numWrites; + } + + public void setNumUpdateWrites(long numUpdateWrites) { + this.numUpdateWrites = numUpdateWrites; + } + + public long getTotalWriteBytes() { + return totalWriteBytes; + } + + public void setTotalWriteBytes(long totalWriteBytes) { + this.totalWriteBytes = totalWriteBytes; + } + + public long getTotalWriteErrors() { return totalWriteErrors; } + + public void setTotalWriteErrors(long totalWriteErrors) { this.totalWriteErrors = totalWriteErrors; } + + public String getPrevCommit() { + return prevCommit; + } + + public long getNumWrites() { + return numWrites; + } + + public long getNumUpdateWrites() { + return numUpdateWrites; + } + + public String getFileId() { + return fileId; + } + + public String getFullPath() { + return fullPath; + } + + @Override + public String toString() { + return new StringBuilder() + .append("HoodieWriteStat {") + .append("fullPath='" + fullPath + '\'') + .append(", prevCommit='" + prevCommit + '\'') + .append(", numWrites=" + numWrites) + .append(", numUpdateWrites=" + numUpdateWrites) + .append(", numWriteBytes=" + totalWriteBytes) + .append('}') + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + HoodieWriteStat that = (HoodieWriteStat) o; + if (!fullPath.equals(that.fullPath)) + return false; + return prevCommit.equals(that.prevCommit); + + } + + @Override + public int hashCode() { + int result = fullPath.hashCode(); + result = 31 * result + prevCommit.hashCode(); + return result; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java new file mode 100644 index 000000000..1926946db --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -0,0 +1,117 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.util; + +import com.uber.hoodie.common.model.HoodieTableMetadata; +import com.uber.hoodie.exception.HoodieIOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Utility functions related to accessing the file storage + */ +public class FSUtils { + + private static final Logger LOG = LogManager.getLogger(FSUtils.class); + + public static FileSystem getFs() { + Configuration conf = new Configuration(); + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + FileSystem fs; + try { + fs = FileSystem.get(conf); + } catch (IOException e) { + throw new HoodieIOException("Failed to get instance of " + FileSystem.class.getName(), + e); + } + LOG.info(String.format("Hadoop Configuration: fs.defaultFS: [%s], Config:[%s], FileSystem: [%s]", + conf.getRaw("fs.defaultFS"), conf.toString(), fs.toString())); + return fs; + } + + public static String makeDataFileName(String commitTime, int taskPartitionId, String fileId) { + return String.format("%s_%d_%s.parquet", fileId, taskPartitionId, commitTime); + } + + public static String maskWithoutFileId(String commitTime, int taskPartitionId) { + return String.format("*_%s_%s.parquet", taskPartitionId, commitTime); + } + + public static String maskWithoutTaskPartitionId(String commitTime, String fileId) { + return String.format("%s_*_%s.parquet", fileId, commitTime); + } + + public static String maskWithOnlyCommitTime(String commitTime) { + return String.format("*_*_%s.parquet", commitTime); + } + + public static String makeInflightCommitFileName(String commitTime) { + return commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX; + } + + public static String makeCommitFileName(String commitTime) { + return commitTime + HoodieTableMetadata.COMMIT_FILE_SUFFIX; + } + + public static String getCommitFromCommitFile(String commitFileName) { + return commitFileName.split("\\.")[0]; + } + + public static String getCommitTime(String fullFileName) { + return fullFileName.split("_")[2].split("\\.")[0]; + } + + public static long getFileSize(FileSystem fs, Path path) throws IOException { + return fs.listStatus(path)[0].getLen(); + } + + public static String globAllFiles(String basePath) { + return String.format("%s/*/*/*/*", basePath); + } + + // TODO (weiy): rename the function for better readability + public static String getFileId(String fullFileName) { + return fullFileName.split("_")[0]; + } + + /** + * Obtain all the partition paths, that are present in this table. + */ + public static List getAllPartitionPaths(FileSystem fs, String basePath) throws IOException { + List partitionsToClean = new ArrayList<>(); + // TODO(vc): For now, assume partitions are two levels down from base path. + FileStatus[] folders = fs.globStatus(new Path(basePath + "/*/*/*")); + for (FileStatus status : folders) { + Path path = status.getPath(); + partitionsToClean.add(String.format("%s/%s/%s", + path.getParent().getParent().getName(), + path.getParent().getName(), + path.getName())); + } + return partitionsToClean; + } + +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java new file mode 100644 index 000000000..e59f77028 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/HoodieAvroUtils.java @@ -0,0 +1,140 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.util; + +import com.uber.hoodie.common.model.HoodieRecord; + +import org.apache.avro.Schema; +import org.apache.avro.generic.*; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.EncoderFactory; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Helper class to do common stuff across Avro. + */ +public class HoodieAvroUtils { + + // All metadata fields are optional strings. + private final static Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.STRING))); + + private final static Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); + + /** + * Convert a given avro record to bytes + */ + public static byte[] avroToBytes(GenericRecord record) throws IOException { + GenericDatumWriter writer = + new GenericDatumWriter<>(record.getSchema()); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null); + writer.write(record, encoder); + encoder.flush(); + out.close(); + return out.toByteArray(); + } + + /** + * Convert serialized bytes back into avro record + */ + public static GenericRecord bytesToAvro(byte[] bytes, Schema schema) throws IOException { + Decoder decoder = DecoderFactory.get().binaryDecoder(bytes, null); + GenericDatumReader reader = new GenericDatumReader(schema); + return reader.read(null, decoder); + } + + + /** + * Adds the Hoodie metadata fields to the given schema + */ + public static Schema addMetadataFields(Schema schema) { + List parentFields = new ArrayList<>(); + + Schema.Field commitTimeField = new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null); + Schema.Field commitSeqnoField = new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null); + Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null); + Schema.Field partitionPathField = new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null); + Schema.Field fileNameField = new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null); + + parentFields.add(commitTimeField); + parentFields.add(commitSeqnoField); + parentFields.add(recordKeyField); + parentFields.add(partitionPathField); + parentFields.add(fileNameField); + for (Schema.Field field : schema.getFields()) { + parentFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), null)); + } + + Schema mergedSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false); + mergedSchema.setFields(parentFields); + return mergedSchema; + } + + private static Schema initRecordKeySchema() { + Schema.Field recordKeyField = new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", null); + Schema recordKeySchema = Schema.createRecord("HoodieRecordKey", "", "", false); + recordKeySchema.setFields(Arrays.asList(recordKeyField)); + return recordKeySchema; + } + + public static Schema getRecordKeySchema() { + return RECORD_KEY_SCHEMA; + } + + public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String recordKey, String partitionPath, String fileName) { + record.put(HoodieRecord.FILENAME_METADATA_FIELD, fileName); + record.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, partitionPath); + record.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recordKey); + return record; + } + + /** + * Adds the Hoodie commit metadata into the provided Generic Record. + */ + public static GenericRecord addCommitMetadataToRecord(GenericRecord record, String commitTime, String commitSeqno) { + record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime); + record.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, commitSeqno); + return record; + } + + + /** + * Given a avro record with a given schema, rewrites it into the new schema + */ + public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema) + throws Exception { + GenericRecord newRecord = new GenericData.Record(newSchema); + for (Schema.Field f : record.getSchema().getFields()) { + newRecord.put(f.name(), record.get(f.name())); + } + if (!new GenericData().validate(newSchema, newRecord)) { + throw new Exception( + "Unable to validate the rewritten record " + record + " against schema " + + newSchema); + } + return newRecord; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/NumericUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/NumericUtils.java new file mode 100644 index 000000000..7828c1e73 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/NumericUtils.java @@ -0,0 +1,26 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.util; + +public class NumericUtils { + public static String humanReadableByteCount(double bytes) { + if (bytes < 1024) return String.format("%.1f B", bytes); + int exp = (int) (Math.log(bytes) / Math.log(1024)); + String pre = "KMGTPE".charAt(exp-1) + ""; + return String.format("%.1f %sB", bytes / Math.pow(1024, exp), pre); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java new file mode 100644 index 000000000..31f5ab3b1 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java @@ -0,0 +1,138 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.util; + +import com.uber.hoodie.avro.HoodieAvroWriteSupport; +import com.uber.hoodie.common.BloomFilter; +import com.uber.hoodie.common.model.HoodieRecord; + +import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.exception.HoodieIndexException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.avro.AvroReadSupport; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; + +import java.io.*; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Utility functions involving with parquet. + */ +public class ParquetUtils { + + /** + * Read the rowKey list from the given parquet file. + * + * @param filePath The parquet file path. + */ + public static Set readRowKeysFromParquet(Path filePath) { + Configuration conf = new Configuration(); + Schema readSchema = HoodieAvroUtils.getRecordKeySchema(); + AvroReadSupport.setAvroReadSchema(conf, readSchema); + AvroReadSupport.setRequestedProjection(conf, readSchema); + ParquetReader reader = null; + Set rowKeys = new HashSet<>(); + try { + reader = AvroParquetReader.builder(filePath).withConf(conf).build(); + Object obj = reader.read(); + while (obj != null) { + if (obj instanceof GenericRecord) { + rowKeys.add(((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()); + } + obj = reader.read(); + } + } catch (IOException e) { + throw new HoodieIOException("Failed to read row keys from Parquet " + filePath, e); + + } finally { + if (reader != null) { + try { + reader.close(); + } catch (IOException e) { + // ignore + } + } + } + return rowKeys; + } + + /** + * Read out the bloom filter from the parquet file meta data. + */ + public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) { + ParquetMetadata footer; + try { + footer = ParquetFileReader.readFooter(new Configuration(), parquetFilePath); + } catch (IOException e) { + throw new HoodieIndexException("Failed to read footer for parquet " + parquetFilePath, + e); + } + Map metadata = footer.getFileMetaData().getKeyValueMetaData(); + if (metadata.containsKey(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY)) { + return new BloomFilter(metadata.get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY)); + } else { + throw new HoodieIndexException("Could not find index in Parquet footer. Looked for key " + + HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY + " in " + + parquetFilePath); + } + } + + + /** + * + * NOTE: This literally reads the entire file contents, thus should be used with caution. + * + * @param filePath + * @return + */ + public static List readAvroRecords(Path filePath) { + ParquetReader reader = null; + List records = new ArrayList<>(); + try { + reader = AvroParquetReader.builder(filePath).build(); + Object obj = reader.read(); + while (obj != null) { + if (obj instanceof GenericRecord) { + records.add(((GenericRecord) obj)); + } + obj = reader.read(); + } + } catch (IOException e) { + throw new HoodieIOException("Failed to read avro records from Parquet " + filePath, e); + + } finally { + if (reader != null) { + try { + reader.close(); + } catch (IOException e) { + // ignore + } + } + } + return records; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java new file mode 100644 index 000000000..e32ddc558 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ReflectionUtils.java @@ -0,0 +1,43 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.util; + +import com.uber.hoodie.common.model.HoodieRecordPayload; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class ReflectionUtils { + private static Map> clazzCache = new HashMap<>(); + + public static T loadPayload(String recordPayloadClass) throws IOException { + try { + if(clazzCache.get(recordPayloadClass) == null) { + Class clazz = Class.forName(recordPayloadClass); + clazzCache.put(recordPayloadClass, clazz); + } + return (T) clazzCache.get(recordPayloadClass).newInstance(); + } catch (ClassNotFoundException e) { + throw new IOException("Could not load payload class " + recordPayloadClass, e); + } catch (InstantiationException e) { + throw new IOException("Could not load payload class " + recordPayloadClass, e); + } catch (IllegalAccessException e) { + throw new IOException("Could not load payload class " + recordPayloadClass, e); + } + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/DatasetNotFoundException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/DatasetNotFoundException.java new file mode 100644 index 000000000..3981529e4 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/DatasetNotFoundException.java @@ -0,0 +1,32 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.exception; + +/** + *

+ * Exception thrown to indicate that a hoodie dataset was not found on the path provided + *

+ */ +public class DatasetNotFoundException extends HoodieException { + public DatasetNotFoundException(String basePath) { + super(getErrorMessage(basePath)); + } + + private static String getErrorMessage(String basePath) { + return "Hoodie dataset not found in path " + basePath; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieException.java new file mode 100644 index 000000000..d8eb86c48 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieException.java @@ -0,0 +1,55 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.exception; + +/** + *

+ * Exception thrown for Hoodie failures. The root of + * the exception hierarchy. + *

+ *

+ * Hoodie Write/Read clients will throw this exception if + * any of its operations fail. This is a runtime (unchecked) exception. + *

+ * + */ +public class HoodieException extends RuntimeException { + public HoodieException() { + super(); + } + + public HoodieException(String message) { + super(message); + } + + public HoodieException(String message, Throwable t) { + super(message, t); + } + + public HoodieException(Throwable t) { + super(t); + } + + protected static String format(String message, Object... args) { + String[] argStrings = new String[args.length]; + for (int i = 0; i < args.length; i += 1) { + argStrings[i] = String.valueOf(args[i]); + } + return String.format(String.valueOf(message), (Object[]) argStrings); + } + +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieIOException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieIOException.java new file mode 100644 index 000000000..2e0b838b7 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieIOException.java @@ -0,0 +1,37 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.exception; + +import java.io.IOException; + +/** + *

+ * Exception thrown for dataset IO-related failures. + *

+ */ +public class HoodieIOException extends HoodieException { + private final IOException ioException; + + public HoodieIOException(String msg, IOException t) { + super(msg, t); + this.ioException = t; + } + + public IOException getIOException() { + return ioException; + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieIndexException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieIndexException.java new file mode 100644 index 000000000..3d030cb14 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieIndexException.java @@ -0,0 +1,34 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.exception; + +import java.io.IOException; + +/** + *

+ * Exception thrown for HoodieIndex related errors. + *

+ */ +public class HoodieIndexException extends HoodieException { + public HoodieIndexException(String msg) { + super(msg); + } + + public HoodieIndexException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieRecordMissingException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieRecordMissingException.java new file mode 100644 index 000000000..72b1d29a2 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieRecordMissingException.java @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.exception; + +import com.uber.hoodie.common.model.HoodieRecord; + +/** + *

+ * Exception throws when indexing fails to locate the hoodie record. + * HoodieRecord current location and partition path does not match. + * This is an unrecoverable error + *

+ */ +public class HoodieRecordMissingException extends HoodieException { + public HoodieRecordMissingException(HoodieRecord record) { + super( + "Record " + record.getRecordKey() + " with partition path " + record.getPartitionPath() + + " in current location " + record.getCurrentLocation() + + " is not found in the partition"); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/InvalidDatasetException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/InvalidDatasetException.java new file mode 100644 index 000000000..e80ae306a --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/InvalidDatasetException.java @@ -0,0 +1,32 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.exception; + +/** + *

+ * Exception thrown to indicate that a hoodie dataset is invalid + *

+ */ +public class InvalidDatasetException extends HoodieException { + public InvalidDatasetException(String basePath) { + super(getErrorMessage(basePath)); + } + + private static String getErrorMessage(String basePath) { + return "Invalid Hoodie Dataset. " + basePath; + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java b/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java new file mode 100644 index 000000000..fa90c3e73 --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/TestBloomFilter.java @@ -0,0 +1,45 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common; + +import org.junit.Test; + +import java.io.*; + +public class + +TestBloomFilter { + @Test + public void testAddKey() { + BloomFilter filter = new BloomFilter(100, 0.0000001); + filter.add("key1"); + assert (filter.mightContain("key1")); + } + + @Test + public void testSerialize() throws IOException, ClassNotFoundException { + BloomFilter filter = new BloomFilter(1000, 0.0000001); + filter.add("key1"); + filter.add("key2"); + String filterStr = filter.serializeToString(); + + // Rebuild + BloomFilter newFilter = new BloomFilter(filterStr); + assert (newFilter.mightContain("key1")); + assert (newFilter.mightContain("key2")); + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java new file mode 100644 index 000000000..b930ae240 --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -0,0 +1,96 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import com.uber.hoodie.common.util.FSUtils; + +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Properties; +import java.util.UUID; + +public class HoodieTestUtils { + + public static final String RAW_TRIPS_TEST_NAME = "raw_trips"; + + public static final void initializeHoodieDirectory(String basePath) throws IOException { + new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME).mkdirs(); + Properties properties = new Properties(); + properties.setProperty(HoodieTableMetadata.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME); + properties.setProperty(HoodieTableMetadata.HOODIE_TABLE_TYPE_PROP_NAME, HoodieTableMetadata.DEFAULT_TABLE_TYPE.name()); + FileWriter fileWriter = new FileWriter(new File(basePath + "/.hoodie/hoodie.properties")); + try { + properties.store(fileWriter, ""); + } finally { + fileWriter.close(); + } + } + + public static final String initializeTempHoodieBasePath() throws IOException { + // Create a temp folder as the base path + TemporaryFolder folder = new TemporaryFolder(); + folder.create(); + String basePath = folder.getRoot().getAbsolutePath(); + HoodieTestUtils.initializeHoodieDirectory(basePath); + return basePath; + } + + public static final String getNewCommitTime() { + return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); + } + + public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException { + for (String commitTime: commitTimes) { + new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + FSUtils.makeCommitFileName(commitTime)).createNewFile(); + } + } + + public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException { + for (String commitTime: commitTimes) { + new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + FSUtils.makeInflightCommitFileName(commitTime)).createNewFile(); + } + } + + public static final String createNewDataFile(String basePath, String partitionPath, String commitTime) throws IOException { + String fileID = UUID.randomUUID().toString(); + return createDataFile(basePath, partitionPath, commitTime, fileID); + } + + public static final String createDataFile(String basePath, String partitionPath, String commitTime, String fileID) throws IOException { + String folderPath = basePath + "/" + partitionPath + "/"; + new File(folderPath).mkdirs(); + new File(folderPath + FSUtils.makeDataFileName(commitTime, 1, fileID)).createNewFile(); + return fileID; + } + + public static final boolean doesDataFileExist(String basePath, String partitionPath, String commitTime, String fileID) throws IOException { + return new File(basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, 1, fileID)).exists(); + } + + public static final boolean doesCommitExist(String basePath, String commitTime) { + return new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetadata.COMMIT_FILE_SUFFIX).exists(); + } + + public static final boolean doesInflightExist(String basePath, String commitTime) { + return new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX).exists(); + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommits.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommits.java new file mode 100644 index 000000000..dbe92e4ce --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieCommits.java @@ -0,0 +1,48 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + + +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.*; + +/** + * + */ +public class TestHoodieCommits { + + @Test + public void testHoodieCommits() throws Exception { + HoodieCommits commits = new HoodieCommits(Arrays.asList("001", "005", "004", "002")); + assertFalse(commits.contains("003")); + assertTrue(commits.contains("002")); + assertEquals(Arrays.asList("004", "005"), commits.findCommitsAfter("003", 2)); + assertEquals(Arrays.asList("001", "002", "004"), commits.findCommitsInRange("000", "004")); + assertEquals(commits.lastCommit(), commits.lastCommit(0)); + assertEquals("001", commits.lastCommit(3)); + assertEquals(null, commits.lastCommit(4)); + + assertEquals(commits.max("001", "000"), "001"); + assertFalse(HoodieCommits.isCommit1After("001", "002")); + assertFalse(HoodieCommits.isCommit1After("001", "001")); + assertTrue(HoodieCommits.isCommit1After("003", "002")); + assertTrue(HoodieCommits.isCommit1BeforeOrOn("003", "003")); + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieTableMetadata.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieTableMetadata.java new file mode 100644 index 000000000..a07bd8699 --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/TestHoodieTableMetadata.java @@ -0,0 +1,242 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.model; + +import com.google.common.collect.Sets; + +import com.uber.hoodie.common.util.FSUtils; + +import com.uber.hoodie.exception.HoodieIOException; +import com.uber.hoodie.exception.HoodieRecordMissingException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.*; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestHoodieTableMetadata { + private String basePath = null; + private HoodieTableMetadata metadata = null; + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Before + public void init() throws Exception { + basePath = HoodieTestUtils.initializeTempHoodieBasePath(); + metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable"); + } + + @Test + public void testScanCommitTs() throws Exception { + // Empty commit dir + assertTrue(metadata.getAllCommits().isEmpty()); + + // Create some commit files + new File(basePath + "/.hoodie/20160504123032.commit").createNewFile(); + new File(basePath + "/.hoodie/20160503122032.commit").createNewFile(); + metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable"); + List list = metadata.getAllCommits().getCommitList(); + assertEquals(list.size(), 2); + assertTrue(list.contains("20160504123032")); + assertTrue(list.contains("20160503122032")); + + // Check the .inflight files + assertTrue(metadata.getAllInflightCommits().isEmpty()); + new File(basePath + "/.hoodie/20160505123032.inflight").createNewFile(); + new File(basePath + "/.hoodie/20160506122032.inflight").createNewFile(); + metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable"); + list = metadata.getAllInflightCommits(); + assertEquals(list.size(), 2); + assertTrue(list.contains("20160505123032")); + assertTrue(list.contains("20160506122032")); + } + + @Test + public void testGetLastValidFileNameForRecord() throws Exception { + FileSystem fs = FSUtils.getFs(); + String partitionPath = "2016/05/01"; + new File(basePath + "/" + partitionPath).mkdirs(); + String fileId = UUID.randomUUID().toString(); + HoodieRecord record = mock(HoodieRecord.class); + when(record.getPartitionPath()).thenReturn(partitionPath); + when(record.getCurrentLocation()).thenReturn(new HoodieRecordLocation("001", fileId)); + + // First, no commit for this record + exception.expect(HoodieIOException.class); + metadata.getFilenameForRecord(fs, record); + + // Only one commit, but is not safe + String commitTime1 = "20160501123212"; + String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId); + new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); + assertNull(metadata.getFilenameForRecord(fs, record)); + + // Make this commit safe + new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); + metadata = new HoodieTableMetadata(fs, basePath, "testTable"); + assertTrue(metadata.getFilenameForRecord(fs, record).equals(fileName1)); + + // Do another commit, but not safe + String commitTime2 = "20160502123012"; + String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId); + new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); + assertTrue(metadata.getFilenameForRecord(fs, record).equals(fileName1)); + + // Make it safe + new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); + metadata = new HoodieTableMetadata(fs, basePath, "testTable"); + assertTrue(metadata.getFilenameForRecord(fs, record).equals(fileName2)); + } + + @Test + public void testGetAllPartitionPaths() throws IOException { + FileSystem fs = FSUtils.getFs(); + + // Empty + List partitions = FSUtils.getAllPartitionPaths(fs, basePath); + assertEquals(partitions.size(), 0); + + // Add some dirs + new File(basePath + "/2016/04/01").mkdirs(); + new File(basePath + "/2015/04/01").mkdirs(); + partitions = FSUtils.getAllPartitionPaths(fs, basePath); + assertEquals(partitions.size(), 2); + assertTrue(partitions.contains("2016/04/01")); + assertTrue(partitions.contains("2015/04/01")); + } + + @Test + public void testGetFileVersionsInPartition() throws IOException { + // Put some files in the partition + String fullPartitionPath = basePath + "/2016/05/01/"; + new File(fullPartitionPath).mkdirs(); + + String commitTime1 = "20160501123032"; + String commitTime2 = "20160502123032"; + String commitTime3 = "20160503123032"; + String commitTime4 = "20160504123032"; + + HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2, commitTime3, commitTime4); + + String fileId1 = UUID.randomUUID().toString(); + String fileId2 = UUID.randomUUID().toString(); + String fileId3 = UUID.randomUUID().toString(); + + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); + + metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable"); + + Map> fileVersions = metadata.getAllVersionsInPartition(FSUtils.getFs(), "2016/05/01"); + assertEquals(fileVersions.get(fileId1).size(), 2); + assertEquals(fileVersions.get(fileId2).size(), 3); + assertEquals(fileVersions.get(fileId3).size(), 2); + String commitTs = FSUtils.getCommitTime(fileVersions.get(fileId1).get(fileVersions.get(fileId1).size() - 1).getPath().getName()); + assertTrue(commitTs.equals(commitTime1)); + commitTs = FSUtils.getCommitTime(fileVersions.get(fileId1).get(fileVersions.get(fileId1).size() - 2).getPath().getName()); + assertTrue(commitTs.equals(commitTime4)); + } + + @Test + public void testGetOnlyLatestVersionFiles() throws Exception { + // Put some files in the partition + String fullPartitionPath = basePath + "/2016/05/01/"; + new File(fullPartitionPath).mkdirs(); + String commitTime1 = "20160501123032"; + String commitTime2 = "20160502123032"; + String commitTime3 = "20160503123032"; + String commitTime4 = "20160504123032"; + String fileId1 = UUID.randomUUID().toString(); + String fileId2 = UUID.randomUUID().toString(); + String fileId3 = UUID.randomUUID().toString(); + + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile(); + new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile(); + + new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile(); + new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile(); + new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile(); + new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); + + // Now we list the entire partition + FileSystem fs = FSUtils.getFs(); + FileStatus[] statuses = fs.listStatus(new Path(fullPartitionPath)); + assertEquals(statuses.length, 7); + + metadata = new HoodieTableMetadata(fs, basePath, "testTable"); + FileStatus[] statuses1 = metadata + .getLatestVersionInPartition(fs, "2016/05/01", commitTime4); + assertEquals(statuses1.length, 3); + Set filenames = Sets.newHashSet(); + for (FileStatus status : statuses1) { + filenames.add(status.getPath().getName()); + } + assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1))); + assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2))); + assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3))); + + // Reset the max commit time + FileStatus[] statuses2 = metadata + .getLatestVersionInPartition(fs, "2016/05/01", commitTime3); + assertEquals(statuses2.length, 3); + filenames = Sets.newHashSet(); + for (FileStatus status : statuses2) { + filenames.add(status.getPath().getName()); + } + assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1))); + assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2))); + assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3))); + } + + @Test + public void testCommitTimeComparison() { + String commitTime1 = "20160504123032"; + String commitTime2 = "20151231203159"; + assertTrue(HoodieCommits.isCommit1After(commitTime1, commitTime2)); + assertTrue(HoodieCommits.isCommit1BeforeOrOn(commitTime1, commitTime1)); + assertTrue(HoodieCommits.isCommit1BeforeOrOn(commitTime2, commitTime1)); + } + + @After + public void cleanup() { + if (basePath != null) { + new File(basePath).delete(); + } + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java new file mode 100644 index 000000000..7156991c6 --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java @@ -0,0 +1,70 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.util; + +import org.junit.Test; + +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.UUID; + +import static org.junit.Assert.assertTrue; + +public class TestFSUtils { + + @Test + public void testMakeDataFileName() { + String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); + int taskPartitionId = 2; + String fileName = UUID.randomUUID().toString(); + assertTrue(FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName) + .equals(fileName + "_" + taskPartitionId + "_" + commitTime + ".parquet")); + } + + @Test + public void testMaskFileName() { + String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); + int taskPartitionId = 2; + assertTrue(FSUtils.maskWithoutFileId(commitTime, taskPartitionId) + .equals("*_" + taskPartitionId + "_" + commitTime + ".parquet")); + } + + @Test + public void testGetCommitTime() { + String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); + int taskPartitionId = 2; + String fileName = UUID.randomUUID().toString(); + String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName); + assertTrue(FSUtils.getCommitTime(fullFileName).equals(commitTime)); + } + + @Test + public void testGetCommitFromCommitFile() { + String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); + String commitFileName = FSUtils.makeCommitFileName(commitTime); + assertTrue(FSUtils.getCommitFromCommitFile(commitFileName).equals(commitTime)); + } + + @Test + public void testGetFileNameWithoutMeta() { + String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); + int taskPartitionId = 2; + String fileName = UUID.randomUUID().toString(); + String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName); + assertTrue(FSUtils.getFileId(fullFileName).equals(fileName)); + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestNumericUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestNumericUtils.java new file mode 100644 index 000000000..9a916d4da --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestNumericUtils.java @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.util; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class TestNumericUtils { + @Test + public void testHumanReadableByteCount() { + assertTrue(NumericUtils.humanReadableByteCount(0).equals("0.0 B")); + assertTrue(NumericUtils.humanReadableByteCount(27).equals("27.0 B")); + assertTrue(NumericUtils.humanReadableByteCount(1023).equals("1023.0 B")); + assertTrue(NumericUtils.humanReadableByteCount(1024).equals("1.0 KB")); + assertTrue(NumericUtils.humanReadableByteCount(110592).equals("108.0 KB")); + assertTrue(NumericUtils.humanReadableByteCount(28991029248L).equals("27.0 GB")); + assertTrue(NumericUtils.humanReadableByteCount(1855425871872L).equals("1.7 TB")); + assertTrue(NumericUtils.humanReadableByteCount(9223372036854775807L).equals("8.0 EB")); + + } +} diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java new file mode 100644 index 000000000..86efe691a --- /dev/null +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java @@ -0,0 +1,92 @@ +/* + * Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com) + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.uber.hoodie.common.util; + +import com.uber.hoodie.avro.HoodieAvroWriteSupport; +import com.uber.hoodie.common.BloomFilter; +import com.uber.hoodie.common.model.HoodieRecord; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.junit.Before; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; + +import static org.junit.Assert.*; + +public class TestParquetUtils { + + + private String basePath; + + @Before + public void setup() throws IOException { + // Create a temp folder as the base path + TemporaryFolder folder = new TemporaryFolder(); + folder.create(); + basePath = folder.getRoot().getAbsolutePath(); + } + + @Test + public void testHoodieWriteSupport() throws Exception { + + List rowKeys = new ArrayList<>(); + for (int i = 0; i < 1000; i++) { + rowKeys.add(UUID.randomUUID().toString()); + } + + // Write out a parquet file + Schema schema = HoodieAvroUtils.getRecordKeySchema(); + BloomFilter filter = new BloomFilter(1000, 0.0001); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter); + + + String filePath = basePath + "/test.parquet"; + ParquetWriter writer = new ParquetWriter(new Path(filePath), + writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); + for (String rowKey : rowKeys) { + GenericRecord rec = new GenericData.Record(schema); + rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey); + writer.write(rec); + filter.add(rowKey); + } + writer.close(); + + + // Read and verify + List rowKeysInFile = new ArrayList<>(ParquetUtils.readRowKeysFromParquet(new Path(filePath))); + Collections.sort(rowKeysInFile); + Collections.sort(rowKeys); + + assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile); + BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(new Path(filePath)); + for (String rowKey : rowKeys) { + assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey)); + } + } +}