From 8eb0a938938ca12448ee7944a802589d86b8b66f Mon Sep 17 00:00:00 2001 From: Pavel Yaskevich Date: Fri, 2 Sep 2011 20:23:42 +0000 Subject: [PATCH] Streams Compression patch by Pavel Yaskevich; reviewed by Jonathan Ellis for CASSANDRA-3015 git-svn-id: https://svn.apache.org/repos/asf/cassandra/trunk@1164689 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 1 + lib/compress-lzf-0.8.4.jar | Bin 0 -> 25490 bytes lib/licenses/compress-lzf-0.8.4.txt | 11 +++++++++++ .../cassandra/net/IncomingTcpConnection.java | 3 ++- .../cassandra/streaming/FileStreamTask.java | 15 ++++++++++----- .../streaming/IncomingStreamReader.java | 10 ++++++---- 6 files changed, 30 insertions(+), 10 deletions(-) create mode 100644 lib/compress-lzf-0.8.4.jar create mode 100755 lib/licenses/compress-lzf-0.8.4.txt diff --git a/CHANGES.txt b/CHANGES.txt index d61609c9..7c12a5fd 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -58,6 +58,7 @@ * make the repair of a range repair all replica (CASSANDRA-2610) * expose the ability to repair the first range (as returned by the partitioner) of a node (CASSANDRA-2606) + * Streams Compression (CASSANDRA-3015) 0.8.5 * fix NPE when encryption_options is unspecified (CASSANDRA-3007) diff --git a/lib/compress-lzf-0.8.4.jar b/lib/compress-lzf-0.8.4.jar new file mode 100644 index 0000000000000000000000000000000000000000..a712c2489360e1a920a037d5b465a814c0005fb1 GIT binary patch literal 25490 zcmbTdW0b7nvL;yAW!tuEmu=g&ZQHhO+qP}nwySnw_PMuv`tBEQTp zRy>G!>@pU@NN79o(Q*plgmbfwi!H_Mg#!!vV^eb8kEDexM8)SXv^Zl*Rxb2OXQG4BX`d1 ztgQ|~742`K7e5{|Q^&MB`;00d))`S3x}(H3L)eAoEsM_6CNr*(zG+AI#|sMELKeJD zKLS~#Hztqg1#)TBYL<8!8Fr={KT>HQ4pRp;wP8{(2F^;gCKU`LEcwJBnBQy3{VTD$ zi$>@2LP|JCjzmVma$&$4XIT3?O`g_ZGr$_hUa-nh(mBSe^4-Y8K4qnkx3_O^t4}wz zC6RQ7-~PJ`zt1%z;h{>M%H`!JJIAj-zl?Y?r%ma+hlkhleL0d=GzwtYz}azQwVWQ> zAOB1}9miGo(y3-%M@YY?sn)Xq??Hi{J0o%4JQzS&n@K^EB+UT71Dl!4=3c>K?iZ$1 zBjm8zVMbs$J4Ir#U8n)7V`ianHpuqGy+5N6o&#_qH2pfLs<)Y|6ZJmZy&Tz|5j4Q|F22y{$8@3gRPyh zgOj;OJIz+851TT$aLOzS4`QH!igI61aEM93pZkuMLh-M0=bQX|Fr{ty2t z{tSaFq~cLA+xC_t!jv^2l6mcPy?&qyFz%?*$uh9C(__Hy1&yva*9mil2kKZtJDcMh z(oEKLauEHt1lE@CTKbzm1U}twfd9B@Z1a;mZhtR3;P2~y-8B3NsQ;}SQ2%#+n47iL zwYs|P77M&jj4#M#DvKYhQe_c;$$CNKa56%o4NT`!h9+41LI=?(YTA&~YbU7w-Xdi7~P^{7f$!r`MuYBLJR#e;n&%=^Pj z-0tI-;SlHr87xZrUxzS8)6(G$4MEd)cz8fM*rOF958gvrcj73 zh~>1f(Cr;(Mxt+pNPq0j_(HJYgQ=>Ymrn>O^B-e%?(M<dISyT zko_r9#N??>x@E~l7ik_Qjpri&u7A~n765p<9bcB<5R0B0FMG}84t8daeP19!0;yN< ziS2APN|RtV+Bd2Y@hU9PgR9UtQ989i!*tIP*KgHqS2*X3yYb7yL)Oz#gs{4=e#xwUqB#%ZPBW_zNe!sF zT{3q%%(X3ESa!KZ%FvzV(&0dVmrVuSU#yr<(A@1x0EgSSI7VLi02RH12i%)!0Z_m* z)7t4ei0OKNDv2tg1q_cfI*`A@!!^ySMKKh%SA^r_LCo8HD8OZ}_4?cQvBGrT??Eve zFhTm{1q-A(lPQ*pSO-l5f=r}OBo!d@&)mJb!Fqd9*W+7k5vQ0Z`kys zMiYCyY#C?)i@zGAF>UNV@<|^BSb$mi&H|s8>Cfk}gUEID3!E7+#{7uP+vTLi3gbc3 z!Fy%`^V^!4R4P|Ls)MERwxXMP3t76=uW&G5hvW<5x38W-(bxP!hwzxR0T z9<vkVi`cXA(#P6ji4a^vaSARxZ?thoG6RyIm$f~`;d|jyYTLLPc&^_y+zRhesBhQA zUTiEVo$&k%uXSACWGO-Vgkg~bQdwY8a7jH_ttW75t!vTSR-kAizV)pDwB*u7LGi5g zl4LTaPe#6<@`K*ar1DAi)bEuZ33C0pP9GQ|c66$CQbPNCOk(;n;<^+Uj#=UxKg|cU zsysbktLvILtp@38`rPD}M(+)$dLV>J8@O|)V67mc!zknY8X`^_`=nF6lVG>zyxk=m z7@#Tc0j|`}X7uu&Y5;2kHS!PlqV9>aKz7Y+U;k=bjC!bNG-)zj7wosptf>Wks- z4R5;r8c83>t*Aydv4GA8gOuCEo9~=UTt+^>=9X<{=qK#VkN`lZTzRwWIC}h*84x!g zijhNRELlhwZ-~eOzx6l~lhIbZYO>}(z7t-{VjA@Bz2>E8jc9NGB8Aay#D|(^~Zq_OHcp+W$b?jvHwbG zfBglqztH~QLiB&ooPWgpxA_I)KL`F#Lh;}FYSjP74F7MG#Q4APc*=j7MZnp_#MnW> z*wEe3%GiO{&`RIYu{vo}QBxUtq&JZdQAQS#mZoWos#zdUy0TfPPojPFWC5{{aKjB8 z3mgXGq@Xy4j>B7SbRG;xmotXZ+q&D22&Y8$I?6ZcuvJ!umaHfVEL8uxdsWA?`b%?n z{QGpb_Z3h}NJ^N|1aps%Pnc-O@gO6DgB?_v!}N%d*mZz`wBUflW^BERN#6XVDBab7 zUu2r?z9%3>4j(%cI#!evX#ad|?Q?1As0xTAua7%N#@Nw=vNezFaRAosmdY$)KmNV_ z^3lwqF05Ol&l0t9WDS9aJgsfO(@m)$wilhjLX2m8a)x(S}FqJt= zpfe;>O=-?POagKSTbMNc5g0~tAxoqc3D4pUyp`!GPjgW6Y;lgnwEIlOuHZyAb(`W3 z(IggOPGNy%zzZAi<^{J8D%l5|hjHq)y8D>o4%9h&gx>DbE&2cmC{R~Xpe*5@0H=l> z_S8r+Otn_4a+Pd>qD#ajtl{>!Zvy%wuHzN+=;Dc(R*X%KexdoS!|`dEUDfNgfV)vp zx4pHVuqE3=24AE;5>P64>R2MS*S z91k@hTM}}5DNZ1|9sP3G_1?-6W)Gt62W*6oj2$2O}5@B)~N^i8e_)|qxVuf zxSPrG@7gN%v5aFK&!{FvvjL__Fo_{94a(&ZR&_p!XBBCl>48;7i~U8|yI`}yqsX{h z(Dwq-6VFc4T$Q862rmuhUFip!hR+44w~+NW0sl=>U9`uJNiU00!)CD)9zN8uBnUk)wXO^hamZfn2RM2S$^-;PlDR5O8k-sS_q-`H71s zvYjW`$ZU|#nFGo)e2R0@!;DBPZk#8$Y@|D7o;PKiB-k*~EIR4>IN9}jHshP+y*uu< zpYC|hxb}F?wBL57>b3#s;qV*^5J^GR)$I+Ck7$|owLytrxhinr7bhi=9>t|BMkT>Y zoa?X~9fxN@xl2v#^R#>PCc(Dbp(`qI2t3w)>)d$q!QLhg%5&m29ex3c=+4H(zwC&O zzQp^FiHR23p4QBK`vA@p$!*7rpFY$(`Ov3iB;oHDE8w2(tn6|x>^8P0O57AXY~EBR zX_hxh8W&YYE*ck~*T}G=zyh9%twiC)lulLkP3*Xt6fkE?#WpLjr7=6UuPiN@S^haX zD@)&2-fETaV`jjejD{O}DVFlDATV&ElGY{tojw6S8kQboifFQ*x3xf}H1ak>nB3xX0D^sz=R|VG%QZ3NLHw^|aB%%AimnCA}#4gLStrZUU{3>WT}Q z749TaiiQ){0K14y2UbT1!gd)W)#p#5V$?Q|jz&`qX(Rhby*QJDrm})X;(0d4P8jLy zZ;Vlr%k@DVVW2a4e>xoYwj9q5k|X(odL!wWmhiRwfKa0j#_YHvsgq=RQ{f(V1!7_> zN%?Czv89*{NL}E1qlg{J?<7H0UlRr_*3N{5pjrlQ4Ps3Wjr#eRP-m_HbDYh2%h5s( z+(9s>DuKorxC22B6{0etU%fGKhxRy@bt7LcfEko)MiYJ z<86by<}>z8N8#*Lf=ROIs?+EWz1)XCO=fJ%(`xrVtx+D6%aW7xj8*`9n;VdqvV!w^ zXf7^~(p9>$4>@K|^!<}}Yxdm3N;W8GSpS^nTSLqYdcd~m-MOy;Io9O{3Qoj8X&hi@ z`SiM)Fa1V=v9t)xpYC^ty=k17^@yQ52U~-N(cC)`JRN4)3UV2lzE(W zz9q-sdsX2R*GEam!n&7{Q0Brpf#9v*0fnOpj6Cx3tbaxxb$OowkS6}*-G%r!;FoA z)?j6v{RZ)dL)PFiTj-=cEZSj`0@|b0VLJbYnq-SdS#rB08dcn^*nI}6n@o<@&Kmmm z+Sz7=Vx6Pc3|U=yxjkx4Ny%BsBGiwCM-BD_qVT37lS@=SwkQlwB!?Glqz`^Kby=bv;>2R(_l0&l+ zVB(JEpH20dZz+^N?=w?MV&+I_rMH`czQwn&Hm)ESMsNum#rJ}0`txI$9*tC%(&44I zm^C4j2RY$J)@prZd4Qa_zv~(_diuLR8oR`yt!MWRupU=R?bOy2_+*FQ!L+p4Hl*|n zSG>bg-5Zy@zowt?_A$N@^!VZJQ~N|czERNbnYuynj=tVO)$|19?kRtw*6urhLZ;qX z$_MX)gV92=`jA=NcbJXf4S{MQZod_tjba(V`yvXyHHID15ytjLgx;~<|N5qt*~2qH z{Xw2R*rgQDHTkGU4qY#CZS-Na|G_0K$HFl{_oO%%dXRL1;az-1%u)63BSLCPT1kp{ z7ATjaQ^2Hsf|AhIT269;b5>a!I$i$^G@2A8{#shE4{H-4XP+aFxmUkS9Ho43RIBPx zHRERq1&;)@UOg(6MrTRSMO$88Giq6fTsKh^|9D{b7<9qbQm3y{t*Hwp08WLl1$2%o z*12Q#Bd$e2ZGwbph2*>;-sc_&*GbT^$Zyz8y5Ll>!H!2nNQw}WqeSE{=L<>_QTl@t zMnxR{1N09S&7g2O{P@?!&--i9|1T@}zY;0De@UdIG(`SWMS81P8g4FO_{dOeMpU#6 zS!+lk$!Tt|Ia|LDD4Zk!h-b#rG6nC50pT>zkX>Mpt2rz-Kw(*Fwe+Lc@ps1bgC<35 zo3*mz=x?81L-p$Q>i_6%Iv%lwCz^cUU7T^>KI-H;9{GNsHfjRA9HzwZWG&t?SBu>Ehhv$$YWB%E8ptc$1WD~=B3PBXNZ>i$J>1-mApGCX+%E%?X*6$? zUX0|Nx5@{BE*&pyjB*fIJKdq`{vol(v_Fuq#25 zHTw8kn2EC8P&y~frinp@el@E&M@&k$jz<&3=uodE_vtT!7v&zr)5p6tFc0dat2ON{ zYmpL&q?I2skf{uNbt{awmd7R;wIb4tA+(YVVn`l5CF)db)L{jiTN#Q7Kp*jCwm9(* z%ffs@+fJqy_|LE=JIU!jsXBf6WioH`m*?S)iUW@t`OJ^5t0|J+|9&?(p#y2jmA}UV?j0th^fBFg zeG97Mn%6_@=Cs*ExC8r~?<;MiwL8^tO#N!p-;;f7^zR+}w&v1&2Fk>pS`fH)?wXZ( zF|r)Ol{$~QIki(9)`jsRg;_~I7hutbmS=eIYOJj!Gv9Okj83w!iH)LJ5X;hn{3F2iiX$GbAMP)Vo0mWrwQg?*E)mapp)WJsXB=IlKV zcZFfIri@tQ82?2UrfuLeRiPwC=>K#LRHZ-zpT z8{u7~ccr7g!uytPXCcsz+P*L<2X}RBNeVEB39f=e2xHiKEhdC&!R~-<*1YLLZ!cI6 zFKrwVf&8%A(lSFq9IWKbwm=^T2$CIn(ND;iZPYsP2v`c1R=!u_0>`tdVH-V3q$cG6 zgVu{S#vHY#>z*j*@HOczXP68xo5}J?d~L*nnh>khXKHljhch#)#sSNH494;O(r1?Kt014n2ov9L{=Rj0v8t$i=l_`zJj}@va!1u3=t&)%sLbD@@(F zi`}sZnC=Hf7Z5W*?+#K0rB2EKNg2^PwV^}a-U)9fZ+_d zAGarv8ltn;rg4qbPWfH%M<@QM|KL;MnM{+YQ2luUf82?lW*ZhCG6IP_wZb$FJN6FZ znsI22{+dmGXT09hW-E-^M9tAP#068I^)8xDO+hbsgm`hr znU^%(2KS6M#s$fN{w@3UA9Ha*Rl=Etzb5k~)W3sA;(rH^|6w)!Cq_mqTPtF!BJ-qb zyT<1&pj#=4MYos;gNJ*I7lw!9iyT0z#Jw$GGz?UzX;D*KPxanO@;wpXQ{Va-C{@zV`C|oqt(xK*r_bJ|{>`0!}O#CaVC+JpoiJ z#AH5FR{!p3>Zns+X&j#V2$%^o#&|BqQki6-?lfB3ED16)q<3h2dJwTGt}=C??{Ob2 z%f(7%go;deUX`FEB578hN@|U#9;VqeO+j3hhze2k;OSPNwW&6qX!okgnyIa+HiZnH zB6P(U(T^2;B{5u5tif#QA8SoKh%n}9#E3pm9+6+DQ2!$iJ}TX~EU}V%OY(eO{KUWc zVFQbPvs{(I9gA*F#tee)&>_!^0dvMYIn^BNGHo)yRz2vTZmSgKF>bcfc7tjBCvRn7 zlmfScNqUfF(T1jR;w&U(Tim!&@m92eC8ODUy9qyC#aNzL3BP(>S`^+DQee~h3H7^P z{V_#@CAtD)wk&1(B6I{!q+vzuWb?&-lXlI|nd$V2jCmsrw_`0>{*LcRy!rLTt%dJI zeVD7o61`hummbrJV~UyDHZgGn>}AbUi`61!G1F-ZEAgVMva%q(rpok7sL8dZutkcK zEQCA1yS4|twh`lS11wM3&w7^!K+GBvj&GaQ8myyA`kq;lGWsyl02t^tw^M8xH31AV z+NIHecKI#jDVwBgyc2wZSp#kXJi0;&Tp^wXUE7lt9`_+!q8Oh$e;QwyJVF>LY6)iO z$Q5oHG6FUyK55$3gfuR-V+)FmW|M1xILc&^Z6Ag{!u;di%W)k_NQ`8S&*RJH87jn?f>HMF3IJYPGr`R(GQM@3Z;lj{-IQLK95TE5j zN833l^%JdqEI$dAb>8C;A|M40#7TmiYl&!^Zn3o#6K8W~ai zvUy&ykI18H{dbh-uCu)GeA0QGx;JLEj>sO9`EBgEZKCL#!&Wa`RycBDIFJK6Y-%UL zA$xJ8uy!UAwLQdHN`g~D=|dF}UVDkXnAb-}Cyd?nU$#X3onm?6K@609WJ~hcHee9* zeAqb`w?$pYc8D0^DSpQYlr+HZs9HQC)hKr3@l`F7mpG-jw@}oA@>($G4o-x%shwsF)E>&knG)vCj@7 z1dX(gBjHm&wV8B?P)=z#D)F6Bm@@M0;l^J9Pxt^Z9Ty@Fh>2w|kr2tIPYI&+-A}X@ zOt#7L?w(a=Yk%@hzmjJ@sMI>hZ!mgZx4Fdgk)DTi5F z&wLZMaCgDq$|s3(J?g5jTLbL@1~$M)tip)5Joo*Fh6vwbiC+9`#(Mo7oc(JJLHh5S zq>%AHS+Uouo_0#h7(YFK*19{XiHWuRtx!`!6RZ#meyK_c_}#^c5evAD367I`CQ@77 zj7)3emz6P7z5!`gM-Yl#4#`V6JHZe0b6!g=x5TW-TsFx#Uq!aaSaZg3G$()0v}a0# z=O0Lw`Z~MLbnm)*oqF=#TCzo@>w*e-$NK%f-5J9>o7P ztK{GO)wV5*`5ZCG`RRnCO_zrSy>VWQ1=BH}mkINc!QUEo%oAvXa=QuLv7M*wvmxz@ z?o`U}is97C|BBu;n%Aw57~fD;V(FbQ~` zk)ZcE=p8$yzN=~0hm>b$vWdb_0$5vm(cqD8SNHUV3u0W*IDm*?;Lva4nPeFc0IMb| zQaaeZ1<b38-?5KQvLMithR4ty2Hqv@mqWY+0MQ}rsFzy+H3CquB4#*kWp4FvVX`XeGm_KsGTRSXz` zJ*Y*Ce=RUh8*AGMHtFl<@+h3!@p0H@^ZPU;;C37Sox~}Iq+Iqut(>{q&Zas>LtA}H zT6+i*g=823|vp{%F1 z6qHuG@Ey?O@ypFkPig=j`_Tj`hV@Bg%=d_A;H?j^WeB2bus`CJ+Hd-ePthddMNtu1 zg0sl2AZ1yUKut-QKuW@4`2*O|C0Q`(BXAj5MhGO&=|T#Ss9I?frHtD$Y8?)0X&qeD z*(6*ve7kvnh7Ntqc(o862WqtR+2qWf*>b`{ual5O@=CMsT7niaa_W7iqwHi024mRM zn^00nA?H}n0sBt_0WI>=QgI>Ycs$tnsLXKA4hy2PL&R`cvdD9aX##+aBxaVS5y0BU z#0M!5Ab?w)OsZJ};+BC02E_C&I|J1&0Q>FtRVrDHM+egcHP8t7s+UHZ-mfY!`B55+ zo&=NR36m5tZDdIj%dG>3byOwntGKu-1#n#pE1CFkg3vRW`R0$=nwJNbK)>$vijqy& z$J0rBxvoSe`VvxcA?ADn3O&$RX%N=@v~DESh)qa^4$vv+FrtAdIYllu8cH2gqY320CCA`oAp0ZRp19EBX-mj= zeC(Vd@Jra!%-t*#=8Dw?tkW#r&_n`;6J{I$Py(ce?HC;W+$f@pEyPI=3XI3W%QPqj z`I!BXvtA?@F6bAU)fXHa89qRXIk}|7B}2vzClMY1@1@)?wj|x@>_qiX#e|znM4Xu~ zAW&P)?#o!sJmBU7X9*{%~#iRxB=ok?6v?hiBwBDaRQx8)-_uH^Ek7>77P5CDim^kal2V zu*>{5%2bgUh}fj>Oy4j^Q@JUU@d!h7iffrWT=Nd-IHPDNX6ek&)@JfQ=@;_ZYKS@k zX7K9{Lj`z%*p~NJSrs+YUnvuy2%gFI&Dv5oLJGU;sEEQQS#3>q(BQ^{BO6lNvc~N|lyuqWwF{aMm2v<*m z$F3@F$)&91)(y~N8!Bd6+l7mVn+q1KA33g*;)_S>Z}XeQ5`xAF5h-27dO}4p;x=V` zK#-?1;mq5-)Z50=3&t`>+z!HaVU*d0wA=>mMc`?`%CQ48-lN()r_E}^;(VRc*$B+= zrRxq^*@xk?o%eoX*_Qn&08=aWzA6l5kBA`VGAFuP_y6%SzqNTgK3>?dJ#3G{tOvbZ zK&c@E)VT3L%p*M)hqUQHjEkTPr#xrIe0MK-!u(>YRR^qfE#!>S+-EitjiVb8WmaUw zEgWD_?iej&kmeX=Q`{8&+l|PZ^aAEo(sfXlC%Fc`aTRw(K@gg?n1AP*dF%lz5^AJZ zWDf9?rj##8PB>xTMVLLW1cMef#ys44VJh$hB7aj$7oX;>7Vcf6JIq0e`O^_L#0<{r z26U0Zx6%pj*5ae~hO&M7$kkZf!iiy4N3r@Hx`YOaJJ&b)xgGM>c z<{Err9CAms{`Lv8$3E;t-yLuRIFyOP`t3o_LC>Pcq)$E6_Nx#Y_|={uB<{{JDwf_S z@E$4(Probbo?3)=WDwnRLeQJCGpgo;_p>+j&wEih7N=0zk?c;ot(B`BF9Pm6dEOss z@44VQ4cvq@QirRG{JL}Zg=fPAohU6o4&ggvffRUqp@!r3J*ziel$+CaL9I>3C`rX% zuHU7hWo5I6RB(X6a6!JfIR)TeFS`bZT6&8}7J#9+a2W}4W)cOu>T8O%hY4?qHeEBb z7ivvrjmE5EN?x?9K~mTCvp*rHfEmA=-jHFCDY>HN^!z8E>$GBrq@R$AYFOv4yTx}) zatSHEp~Y`cpD4S5&r+Xwx+U=zYR?%xa+yE;9?x5~qILr_MeiEJqTr1!l~ zIMK?P)RimUxl1s|s-oWgI+n;OvpOOyP7IvTfu)GOGdhOIDc~AiOj3RGIv(W`bPttG z*0m3Q0Lv-&8bB)_YE(+DT;j^9jya56zV|BUh^}0gsFAm9Q|XR6t(ev-wK;IVK+UPO zIRv?&@k;N=Os(*2*4ACa#s^Jux{tTGsGewE*eTsT3|i zF4iMSaNt9d#!Fyp^ObGU!*zy}Xx-!@FfP1LBtf1NX$=DE&_D$N)Fz4y@M9S-pjQKk z7!vh-F)Si@ltKquS#cn%~G# zR)Ztp5^y=NH)K0J*pQba1nsZVGdkOIxxScImf(RLO5J|SchfiOfF?8Nv90(S=Y-v@ z7S-ND##a~IX5N7=mmj1P?$>E0+YLDAG^J3OYV6lN z-s%m+yTfHWUKpQ3WZ$a(jy-7Q+Yn1g7sd*gFcIE?8^Yjyf>2>#vgB$kGl>Px2pAwq zUlVoUN2>H9wwRpP$E9%=#>0_`5s@F1e#(IZ@gmT|bh2!TqHW3W1`6U_7k~8xuH17w z!Q2u+?Q&#$)grr7q)`WPsyBcEbh7Uwz1gO`6U1K=|601?fXM?^1`YhRO!K3DteIFF}>&l&U-#SAIgShs)PK<>qqWi|ElBCAuWfKO;=uS75JOnHUzIIk-szfS(yvRUeUT%zPn&!XXKUf(&&--|-Q4)q}r z%Do_UyQ31H@aXL*<&LPr;wRFraBc(#O5Ir-YA!JPg8FQCD5P{M{L`)DXlJ zZjZjcMg7|Z4@$=>B@EMH2lOsu=)JWWcDgV}j!Fzi3mt-(p!%pbPDY9_!Y6$%IZvao zf~Y9NSL@YsU$*5MtxaV%{Yo$4kFBJr>_zgl&ULd{r$*4iW$_Z@ff zETJMVfnur!a~3;0Ag#YlwPGo4EuMR;iHej@8&1d@TP3}54gq#aNbSOL-w0dh?)#^8 z@2vXc8OwozS=1#_7jzHAA7*cc~KwoiND zSR;$u4^Nvv?vumqVXugEuai^7xfwJi$-db2y8p(_SbR)}U}6aoL2+8j!ihuP`2*)g z!PYfz{`Rf6|JjsXN2x2AhYXfYUtM!IK12o(QBbTx&lq2-QD6KVKUH2dBST)C&>1*F zHe2ND7!NX#_`4QCM-PQhx#Wusz0alnt@mxP-83F^Z_S5?8XPnol8Z zFx!?t4H|?gOD=jQh3EM@Mos>1GO`xqXAUZo78|zo*4_RwrUuOopfsqp8y+!fLRZpB z6t_BpFq_1Bzb)5(6LcG90zF>dR2JQFivVXm9`&v2D zZPv1j>_z4skPb!8MQcu#k4?xJ!nfW7<_5}d^&9hmx=%fx;4%b>%Zrb ztib+V1Nqmxu75|9;x=~9PKr(r#`@O(!+}+SaYRx^=H9#_mF{RNCIL@TlM7-}t_Tx0 zqgfLmY4%c)g9JBkFh^S`A}vQmmNS)e8oB=y3CL@Q3>I0)*pwH=R>;V4p?e2+MmIN~ zpYz8hv#2@BspNLs`}Vr~G{^I1^V8uvvl8HDgb{En>Kc$qFEm#r7aDG|j?qusO{hxL zc#TG|7c4C+DVqDCO|{C~fSXk3Wzr7c4;>UJjEJB!1zOym*G#Iz*TIS$01M2U?MP4} z(ARe*0_}i;s#0IwmWy#Kp=B|UhDoGBg?2T8Bf;t%Tbi``3ieeiGlY_pn<3+N(zJT5 zE=2FPe+{V~m2(l1hURf;t3gq_Txv$bZ0>`3Y8_7&)+M$?fwM}Du(W@x7M4SvN$Ki5 zi4lS-M_es@MquY4{Q|JLlP0n8(S=l6RKDko`bFcs|1%EMwllls5eE}B$W(Z#NiuLp zxD?aEEwO5J$qOy5}nYmdL9UJ;g3wk>@8l7cGzR4Ddn8T|G< zN%_`qvLF^}zD#3+IMGR(=onEM1rJjiFQZ^OclOATl6ZTlScdW!u;*D6fX39<&GPoxy(1dC$ zYO;fN>SCvi;NQX3vEezJI7H$;meulx8f9|0=p@z@xM*h4=t)I2OvP-;O9je+Qj`zt zs1Qz${yc@Qyha6$^(%^lyQNxE`t%6o{gfEL;Lvg>N1gLt9?ER1u&H`PM8`#g%0$F* z6~ORLvZWf6C(_M%QcOSMP8dvA43#`tK|Zsr)1W%q zdD*)Pvuc>%l2<3EqKgG=<$92axX9bqRH^gb9FSMM$t;PPZh4@q9dy~T^4)>Xmbhh) zBQ(_<5V_=lWt(L1%3C&kE0uQZ>3NDSkX4wLfbb9_Y%FvKw8#-d*CC=Ec5=gBEOo-y znDE;L5KN(TXy%SR31Hua44l zZ{?XMl}KeY?N1e}aH;$+XzTZn!^MnBizRI(iq+YB-CORgMY+NI7*wvcw1Hrmx~?Izj83pF|c!8Bds+)$Tyg0|NHV8`{}q5!}HUVYJ>J2+2s`DIWPK zj%*2bUSXJCLNYT3fIKF%-UE}TiZ2K3CXX1i*`~C)NT=u+?*&pA+%_%DO8p~0UNhe>55H}4MzdNr^lM+aU8Dd=+-c%3!Ae$B7&QBOqW2@>#FIvnV+FkdY zEV`GvOTaWnRl|Lbm-)sY_r5=Wgd(;5>ouQH47Qr5{4uS3Xua0`2V1_K z7FbOFJ7~;={&!$U_3u1%S!bvJ1MRdO6_t^-TgR<8>@+1dl=7mOn6Gc#H2JQW`ZRxq4DN9>J8M6r!S>_v3F;xH7^v-hR5`fGFd-bAY|v$PVL ziiqI2O>-Sz*DbG`U)j(9LQ@CW9z3KCm%kb#hrI~&A?*-O-$w_Q%-=)KwH@}bNCO$j zgGp*KP?ufkI*yKM_J*jA2Sb;H8zB1H$UYGDMa;29P*AjIkX-lW&JP?^{9bNDh?t%cuczzU__F_ zmk(y^mu&$jLNjx!&WhOQn+IZTX);*tjxBg_&BP7E0}}Q0@?#j|#vs}ClH;ri{ZcIG zwd=L^)S_~8^k}GYRO?2r&bh8R?dx{#UbUk+QCjFImoCajft-oR2rQzy|$ z0V4z*XabNf@0WuwvQ%~=X}JN{^-;!AXsKvXTSgB3M=@k6%ZVpW45g6Gtdl zyM#Y$S8F9*P0{b3k0SXxh=4yL=ob+13Sp=^$7f@mGu^h`zdW&NRK z%vb%2m??4dcT6u9XM(jMs_+i`2_cj8%+!Mm^wa>NLvh0qP^wHveJq#WRafaE-1Y8M zxHh7P$uWD*zFC!$#|#hMbL9!XlN?+QyRO92ka)Xx$;vHdZPV5XG3iEe;mcnA_=6`# zw5eLljZ~EfkV6%)_ra1OI&8SrjU-s@-Jc%Wbqc4(+rkvcZ$^c--m9CAA?C!7BPKqM z`uEMv5zeF(H9>~#hTf5so+T!lzfrB&m#= z;xkvI?62eN)98QpPuFIboo`s}cu?DbN%qL|3HGS_5#BInFk(HxqDk6-T<4Vp9QE>k zx#HguWpH0$wj}34{G@Cq$od)LWw7Jw3DH$^n+eBZ!zD};5W*g?V~@zk06$$&7am}6 zL5W=gtUt!@rw6jMB7A_;spd(_edGfjrECr&6DT1~DMOqP>eWKCh>d|_NR_tUUFZYB zd@kN>8|*PRL2}X$POG1CE#7_@G-SkwwgZ&r?K5rIg``KV0=y^zc%~(nO+PA_f`}47 zK^dDF$IV3xSVDLXojXIqmJ?N|K5q-W^0gNA(d=qlr7C#^j9*`X)lSfD>p1(YohmC5 zoDU|Kp)%gNXSddc`qcQ_U~G1WuxJcyrbBwSe#hRDYD~^!BMy=cs5W4Wa*jewuuN0W zM*0G*F)~(-QCHgg6Y2{6g2+0O{r;8kBxb+rHx70RiCRBFc5Gc!(?4+PB@rZ$?=XTA zTyrQ~Q7U#x-fR)K@>>J;z$f6aDND!~Ot5JQM&cp8A<|6+6=iK&37ggE8q-E)Vywy4 zsz%#iTdl}DS$;QZs>$J3raH}x7|1DB=Oba%#si5!>;ZW5>K;PE2Xzdhxo4`?zic zNYqlkPr%Oc-z^^!1rVR-38snZS_@>kYkElF9<9^5!#~oYz15y>$c8OuL@)a(8qF2Z z8tUeL4f#RVZHh=$$MHc~2cr|Po`HG=386x31n)xZA>0?8`ZzyD)BMV2X$dK&})An<=%Px-HBAhG@>YB)QYTm9SfiT`yGl=F?73+%Y>CNNp5%=NO=l@m7Rfk2jeQiKe89JoHp%IYoMuefH zC6o~9?nYt=2^m`H5Rh(=W=2twZUm8TDJiLM=H5?1@73pb*ZebQo>}kOd!2pGS!b=i zUYU|>7&M3{@?M;GNhy4~SKzc5zt3NZ{AY*gI2k*@IUxRaOPYqV{h%bC?^?2;D+&Os z4+X%AXEEC+j75t>i*r*mf~bbzG5xY-oQ>X%&e{`X;@0qM1bA4$`rRD7D*YnakS7i6 zKCLO9jmNvAqnk*=_0}D28)(8KN<@8JeY{yY#vRgKZB$p*u#XP=!dGq48zpw~YnMDQ z_*tZwv){T+tF8mx3>we9ygxe1^#sAVAr4QevI z4O8K!ir5mOc{N}7^%URzz+?H2i0i7868fCk+aSqc?7Z})aEGpiyFlhG+K;GvM4`2P z@2Gj^$-Ak8j=ov|1IMb3#yEL_V zZ)F29Pq;|t3}<6O?^Wt~`dG4_n1PSaUD>d;hTm;(G%Po{ALNpXO-8O2H8P@)@(lOu zKgC@j^AF`eEkHTt(&hj;-5JbMC|%y^g}BDBN99`9h|5$xdg}`WL4|~OpW5d}$PJi@ zn`K45l!c-hYkOJ20J+s3|n=#oi*>QI=)dz z5-lY}RxwNSY0m6?JRkp0=b0~Dhoj?$?tKwX}QNA0$ zpbB}N!d$iB-1zuI$o7{?k&i<+l|A#6S43FV!y)tz2vu)x_Rrk>bz;2Bp|K+y1c*_`*so^b+7`N3e4D4 zkzF2agQ{x~&!u2x2jSEu4B2#aYdAx0dqKlUxlYdYW-%JGiG#a>dkw@k*H0rv5$n_XOr95+oCiWDr zR|i*`k|a`I++}ZAaGLOEPNFWJcd}b?@UW``1MauzV!a81SgYb5g;YKt&Q!M0l~|cq z!O`(KFSj;dd6TM%QM74oAuP^pURkQ~_SJZMiezTBO5v1DNF}L9h$!d#t=Frw19T2OV^BwQPVT1)2>tR`v_>r*YN=r`zri6H_@)hW{`{7N*LugOY5m@ei6!+ zqM-)@Y;*2^qAF5SZOCifV98*k%0cVgj~Qc7O^3>f2`=D!dD-fQN%eZ}_^ml1kJmB971D7t~9lSEjpC=BIR(Kq>Sc#FZVm;>4EJIE(gVv(#;G!C-mG0@SSD{w} z+A$NT&tK<mk=()oL!WeJ9c3K9l&P z#L6c~$${Wvs@h)l!ENch`Fg10Hib^8Y3LoJN(TuW`VidYL)zNsV~;n`srk0ad`4i} zP17*`@ncGH`)rKFaL%ydYanRJxejAQ=Pu*G;ZxmH?n&d>TcUQef!N1SZ#i;Cr(A!X z{Fq6}2Va%k<2`K9<2FesK-79xNVLrO_!*vB$Qz$~wQ=_;lG4Z_QSMR1fiuz!86|lY z;GVqG5AERLlFeLb`R8UIq4Hgv8%c_LO@p;VcE^70?|5rdEm#L{7@7x7OcV+C4y4EF znVyzb?5%4OkKT3i9pz|bGpb;(A1AZpz!=fv?731KFa2<$i;d%D-~@%0`wPnYLG`pe zh}axa1P^)fn*s9-hcqMiMRE`svcql-ORO?6L4UPEYv~t&g0Iz>!KO;o4kb;zUZ?jV z)#@T?1sRpPWVK5n9bf*?BrqX&@sq=&l|s+ZXCk@6g3nJJDnuLnu4UZ69Zyo;yJ{Hl zNN%wYC?ktL^(tbS(!_)!(G{&CdeO7`1KoXZT>*?iFT%B$dlM*nOMUyVb`J5QNoFZ= zx6Y{b>aqad<@lz{9a58#GB6iXyH4XsBg(p+d4Lw>GL%E|UfoO%%&T#>z__kss@IYD z%f36!K`e2748cJ&%r~;0DQrd1aoU|~(wj>%K5#$Uy0B?xN?xP^Wdkd_CfqH;f-1Y4 zpv0T18s2%RQy?0arM7^ap16Cf=*iQy`*T)&s%0TyebX9!)3L5pBET1zgU9IUjG;m$ zcT1$jAUh!KJl4j4^a&Szne^tfl7?kk%Fo7IbtHNYIB6c$?ONO}!wSqdfGnkBb21IB z?d}UhVPQY#rF@ub~m-^esi%Xu(Smh%%eULXqsSJp6{p5SXhn1Q? z*jey(&cz@3`4(z|AB69S$aIUhXoD=>JKE7BUGYBfoRmBuCb;e6^`g_r`-VrVTN%?? zPdn(Wx~7xIOuBD43Tbv#N8c^QIz`2rsziRWwCYq7*c!M^u;7*A)SNB)Y?wkirt6dv z^K>06&#J=fMzekQ9Qa$pt-Wuf6m4}wD3cvXW#|&x&6;aWhu7UX81{7VqCW6D72Fo} zBo^3Ti6Efy7GU~XG?qKYK-3tV?xPS=xWa9M<;$OSAm_OZOJCKh(CL5B%7)*4R<@R% zdYm0@v19$jo-x}s+7z2^OL_K`B+omI+mB)+*|aFloR)MfpauH0sr|z@4u)gukHZJ! zD1L1N=X8VT@*b!XkFGXFnQs&NU`c4{0W=#%t6#yu<}5ck3u{MniyJNEmd(NLcKKk8 zf*HY*Wp3roH)6(_emynJOWRgQ*C`wat@*IfA)F=@c`K`fgx_>Jl(WjriyB&8)_o4( zMdsq;j(`c}2>y4wWhHMnAE-qivRf6`0jn^o9-&d{)K}(rK<4hSY{UG5uCmIfvvM_zhC7PHym9gwfRAWZ zVcO4!Dqu&f{JkvNxoqCCti4-&GUhztL(rxutZOF;sb{yr{^p@47#iO)ibwsopC8&g z>A^T&mFjf2XlYf&`uZ1Zr@fjTv)Qv#;h=5Ewy3sH#y>LQT;Zd@7nSB5dp;$!tz}Tld*_`H&OwuqI@f)mMU-Fev}nVa5MHJ*+RMcz#6IQ5EpW z8XJ7h3XiNKqj7ptMT25tvU8so-DV4)&e2ZJe9jR%L=um$fDOOqw>@vQY-#xv!xV7c zl%^#n0B^nPg^Cu$QZCsJN1@0qs))SKy#^u|-}M}*q|G>-*Zsk)XtGB1=sG@$7ok}@;#^`^HtwsO4Pks)rHUB@ju6JC$P7YC5v^>J* zb){9AxJq}|(IQJdH;nRXR3;PCRPOC)+oK@u6E+|%!5xPsdAdgejB*8}&f~s3BD;$l zUBVLnzNa{uA+}WHBTJTrE6heRL4`7}!O_ChRvx)o?N4J=Qwt^@TDYN~jqOm{Kg7qB zrj|=QHnw1>nA1zEJ3fx?YE+;EZc9-*d7s34xl{IrXWX(GuH?6;>58v^bW{4Vk3F(AXn+$r+hNtMH^9J**%N8&v*h*3MAXx;E zrW;vMlgA-K8B15(o!Kf#u|@0V!#QBo|8=6@PM>7q_?-juLuUC#q9dp=%oK&=;P!Sn zIz7^rUK^zm{p#F=_xCi|Z-TqmQC+&PNMKDb-dYDBGQOT1%7NFdM|!WT z#&X1(Q!%g&*VJ#4LBrN~{DTPKIeKk*$JXJw-wm*w(EII@))+EY|ACD!HsoRp#Uxzw zRvm9{*WLz8TV(9R=A`1pt5AI1iZuyv@YS<0q07Vu*%r^~L)@l{{lO+GPmk}!26k01 z(osJFqjhAbzf_Z!D>ug1DOBWI&&3Y0SDRMiI+)AwliBA~wjS_%oahkMlaxmfUV}I@ zEUe#E$!oUo3siq#cpP}rz;38k!R?|QJG5Bsa@!CFC{}EcEAGdW%YGj0v^1wk@6N$By=b{S40o_+LF}@9EGAlP@?z+tY%VI4{dOcBW&`MucRe6iF zLrau_LN#;?J3S$)0V)03=Od*1eMkVW_mxT_cX{8NRTa~5SyAxtyeOedB_k8(;}#X? zf+*pnQ%l5=Qw(M1@m2#BwHr)y6RK`kSXp64Ki95}j}!487$0z%WYvvxRC#1mWe`nT zwm27F_|ny~PsmkPX1Ct<(Nd*cL#2Bf+m&ITPt~f8OCE)0O`0ViG}(H<=)FhC6}%d( zq^u#3JmZfZ{1Pebx*)Hk6JMN6<;iVJn7ixeigW`#;F9i&sa7|ErH^I7hd zVUVMeB)JgC)~pM=!LJqis~Opvh@9AfVuC>Ku@}o@@TNIN3Jl4E5AT$Z5^e%88pS1* z4w&1p4ysd}ZPBXK9W3LtX4sy2W&NZo%62!(Pt1!@R$d+-M6xcXp~tt59QT{(=19^KXg91DPhHHVo_S4*x? zM81@lss1#fRtRC&DlsV09xqQk&`5bTEOxG1l{=*h|&w@kjlR6h z)eK_LinHw*U9C^h*vs5Waw-Rk+7D$bE+ z)SHWj0Zfbf0RsCPhdm*uAPYT7S~tRWG~P}Vfj-K8PWtKqWzo!xCzc?Kdy;FeAuD*D zhO$>Fvxaek!(4fyokpHaE#HXt>xrHr6s_pqj({ElGaqHl-i*pet9auaSlKIgM6$8% z5BO#}MFkr3sWW2ilXZBG>3mIvHbP206{w8cznxF~;h3t9FZq;1DP=ECyO+GJQ}C12 zT4H0!L9DReU}_q$L?)L{W;1^zYflh8Ore~CT%l#p&R9ev)Hb;a-$VptlLOly6CP7} zx?KZdqEMu6(dX(NRX7u5ILBu24}`=J_Jp_s-HrGYPv{sUZg@(#1KT8v4b@s6I8&;3 z#YLLGj1HCTmO^5AMjUnM=$PZ*$v4u~Hu33=vAF2wz*qV&;Wz0zM%(CXw<}ntPC<;x zjE-?ZU*vr5=vvJcZm?C*%(wbxO6zD<&vJXW6L%z=p6I;m(s(#$=@w&t=iE7XPxOXQ z1-dJ#i0ZR6x$DcWQk*;vkk%~n^}pcCvK^fOV79lDcAPhruAUWqoa7I=&pWM(aVeE{oTNCk6#5lV@s@FAHh-2WMsiL z-W#EN0Ao6I4n4v5|4J_WO5~woVzz+egY{G;4lbs$G8@^G$(67}&aby`oEc;YLj@EJ z)bXMYOO5lHN2juE>Nv8~c!V~(_L|*~ar@o-S9Nz;Y3|;7@Wc+&maYi3+rYsu-AnpzzO&0v35IBk0oKsRd?buItCVMdMeay?QbG`TuM$l&eEI&w+e<9KS6$PR7ojn=BI(Q+p@SlI6{7DT#t$ZQ(F01`b zwEQP!1hMjkB>z(xo;LiOX8BJj2wLR}`TPfz?^(#dNS1$vyv$0B$X#9tAAA?s4`=*6 zs~-dV3G!!(<;w;T%`d#DaGminU_d zMrD2n{8wG%*AZT>10u?w7eX)lN1%TeSiYxhel>Bqo`WbTUI>ree>L$_2IN;mmvd1+ z-(q{@|D&OQdL40ZxSZHQq_i%?TlIf2aQVFc&RdsX#Stl@3yD$t&o=((uFC-c;?@2_ dq%{7krSJYigKzReLL!HM_xj literal 0 HcmV?d00001 diff --git a/lib/licenses/compress-lzf-0.8.4.txt b/lib/licenses/compress-lzf-0.8.4.txt new file mode 100755 index 00000000..1e22f87c --- /dev/null +++ b/lib/licenses/compress-lzf-0.8.4.txt @@ -0,0 +1,11 @@ +Copyright 2009-2010 Ning, Inc. + +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. \ No newline at end of file diff --git a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java index d4c1244a..86ffbe14 100644 --- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java +++ b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java @@ -22,6 +22,7 @@ import java.io.*; +import java.net.InetSocketAddress; import java.net.Socket; import org.apache.cassandra.gms.Gossiper; @@ -78,7 +79,7 @@ public void run() else { // streaming connections are per-session and have a fixed version. we can't do anything with a new-version stream connection, so drop it. - logger.error("Received untranslated stream from newer protcol version. Terminating connection!"); + logger.error("Received untranslated stream from newer protocol version. Terminating connection!"); } // We are done with this connection.... return; diff --git a/src/java/org/apache/cassandra/streaming/FileStreamTask.java b/src/java/org/apache/cassandra/streaming/FileStreamTask.java index a6611815..6e6e63d5 100644 --- a/src/java/org/apache/cassandra/streaming/FileStreamTask.java +++ b/src/java/org/apache/cassandra/streaming/FileStreamTask.java @@ -18,9 +18,9 @@ package org.apache.cassandra.streaming; -import java.io.DataOutputStream; import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.Socket; @@ -40,6 +40,8 @@ import org.apache.cassandra.utils.Throttle; import org.apache.cassandra.utils.WrappedRunnable; +import com.ning.compress.lzf.LZFOutputStream; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,7 +60,7 @@ public class FileStreamTask extends WrappedRunnable // communication socket private Socket socket; // socket's output stream - private DataOutputStream output; + private OutputStream output; // system encryption options if any private final EncryptionOptions encryptionOptions; // allocate buffer to use for transfers only once @@ -119,7 +121,7 @@ public void runMayThrow() throws IOException private void stream() throws IOException { ByteBuffer HeaderBuffer = MessagingService.instance().constructStreamHeader(header, false, Gossiper.instance.getVersion(to)); - // write header + // write header (this should not be compressed for compatibility with other messages) output.write(ByteBufferUtil.getArray(HeaderBuffer)); if (header.file == null) @@ -129,6 +131,9 @@ private void stream() throws IOException ? CompressedRandomAccessReader.open(header.file.getFilename(), true) : RandomAccessReader.open(new File(header.file.getFilename()), CHUNK_SIZE, true); + // setting up data compression stream + output = new LZFOutputStream(output); + try { // stream each of the required sections of the file @@ -234,12 +239,12 @@ protected void bind() throws IOException protected void connect() throws IOException { socket.connect(new InetSocketAddress(to, DatabaseDescriptor.getStoragePort())); - output = new DataOutputStream(socket.getOutputStream()); + output = socket.getOutputStream(); } protected void close() throws IOException { - socket.close(); + output.close(); } public String toString() diff --git a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java index 2688792b..a598587e 100644 --- a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java +++ b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java @@ -23,9 +23,6 @@ import java.net.Socket; import java.util.Collections; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamily; import org.apache.cassandra.db.ColumnFamilyStore; @@ -41,6 +38,11 @@ import org.apache.cassandra.utils.BytesReadTracker; import org.apache.cassandra.utils.Pair; +import com.ning.compress.lzf.LZFInputStream; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class IncomingStreamReader { private static final Logger logger = LoggerFactory.getLogger(IncomingStreamReader.class); @@ -79,7 +81,7 @@ public void read() throws IOException assert remoteFile.estimatedKeys > 0; SSTableReader reader = null; logger.debug("Estimated keys {}", remoteFile.estimatedKeys); - DataInputStream dis = new DataInputStream(socket.getInputStream()); + DataInputStream dis = new DataInputStream(new LZFInputStream(socket.getInputStream())); try { reader = streamIn(dis, localFile, remoteFile);