hdfs.c 100 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583258425852586258725882589259025912592259325942595259625972598259926002601260226032604260526062607260826092610261126122613261426152616261726182619262026212622262326242625262626272628262926302631263226332634263526362637263826392640264126422643264426452646264726482649265026512652265326542655265626572658265926602661266226632664266526662667266826692670267126722673267426752676267726782679268026812682268326842685268626872688268926902691269226932694269526962697269826992700270127022703270427052706270727082709271027112712271327142715271627172718271927202721272227232724272527262727272827292730273127322733273427352736273727382739274027412742274327442745274627472748274927502751275227532754275527562757275827592760276127622763276427652766276727682769277027712772277327742775277627772778277927802781278227832784278527862787278827892790279127922793279427952796279727982799280028012802280328042805280628072808280928102811281228132814281528162817281828192820282128222823282428252826282728282829283028312832283328342835283628372838283928402841284228432844284528462847284828492850285128522853285428552856285728582859286028612862286328642865286628672868286928702871287228732874287528762877287828792880288128822883288428852886288728882889289028912892289328942895289628972898289929002901290229032904290529062907290829092910291129122913291429152916291729182919292029212922292329242925292629272928292929302931293229332934293529362937293829392940294129422943294429452946294729482949295029512952295329542955295629572958295929602961296229632964296529662967296829692970297129722973297429752976297729782979298029812982298329842985298629872988298929902991299229932994299529962997299829993000300130023003300430053006300730083009301030113012301330143015301630173018301930203021302230233024302530263027302830293030303130323033303430353036303730383039304030413042304330443045304630473048304930503051305230533054305530563057305830593060306130623063306430653066306730683069307030713072307330743075307630773078307930803081308230833084308530863087308830893090309130923093309430953096309730983099310031013102310331043105310631073108310931103111311231133114311531163117311831193120312131223123312431253126312731283129313031313132313331343135313631373138313931403141314231433144314531463147314831493150315131523153315431553156315731583159316031613162316331643165316631673168316931703171317231733174317531763177317831793180318131823183318431853186318731883189319031913192319331943195319631973198319932003201320232033204320532063207320832093210321132123213321432153216321732183219322032213222322332243225322632273228322932303231323232333234323532363237323832393240324132423243324432453246324732483249325032513252325332543255325632573258325932603261326232633264326532663267326832693270327132723273327432753276327732783279328032813282328332843285328632873288328932903291329232933294329532963297329832993300330133023303330433053306330733083309331033113312331333143315331633173318331933203321332233233324332533263327332833293330333133323333333433353336333733383339334033413342334333443345334633473348334933503351335233533354335533563357335833593360336133623363336433653366336733683369337033713372337333743375337633773378337933803381338233833384338533863387338833893390339133923393339433953396339733983399340034013402340334043405340634073408340934103411341234133414341534163417341834193420342134223423342434253426342734283429343034313432343334343435343634373438343934403441344234433444344534463447344834493450345134523453345434553456345734583459346034613462346334643465346634673468346934703471347234733474347534763477347834793480348134823483348434853486348734883489349034913492349334943495349634973498349935003501350235033504350535063507350835093510351135123513351435153516351735183519352035213522352335243525352635273528
  1. /**
  2. * Licensed to the Apache Software Foundation (ASF) under one
  3. * or more contributor license agreements. See the NOTICE file
  4. * distributed with this work for additional information
  5. * regarding copyright ownership. The ASF licenses this file
  6. * to you under the Apache License, Version 2.0 (the
  7. * "License"); you may not use this file except in compliance
  8. * with the License. You may obtain a copy of the License at
  9. *
  10. * http://www.apache.org/licenses/LICENSE-2.0
  11. *
  12. * Unless required by applicable law or agreed to in writing, software
  13. * distributed under the License is distributed on an "AS IS" BASIS,
  14. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  15. * See the License for the specific language governing permissions and
  16. * limitations under the License.
  17. */
  18. #include "exception.h"
  19. #include "hdfs/hdfs.h"
  20. #include "jni_helper.h"
  21. #include "platform.h"
  22. #include <fcntl.h>
  23. #include <inttypes.h>
  24. #include <stdio.h>
  25. #include <string.h>
  26. /* Some frequently used Java paths */
  27. #define HADOOP_CONF "org/apache/hadoop/conf/Configuration"
  28. #define HADOOP_PATH "org/apache/hadoop/fs/Path"
  29. #define HADOOP_LOCALFS "org/apache/hadoop/fs/LocalFileSystem"
  30. #define HADOOP_FS "org/apache/hadoop/fs/FileSystem"
  31. #define HADOOP_FSSTATUS "org/apache/hadoop/fs/FsStatus"
  32. #define HADOOP_BLK_LOC "org/apache/hadoop/fs/BlockLocation"
  33. #define HADOOP_DFS "org/apache/hadoop/hdfs/DistributedFileSystem"
  34. #define HADOOP_ISTRM "org/apache/hadoop/fs/FSDataInputStream"
  35. #define HADOOP_OSTRM "org/apache/hadoop/fs/FSDataOutputStream"
  36. #define HADOOP_STAT "org/apache/hadoop/fs/FileStatus"
  37. #define HADOOP_FSPERM "org/apache/hadoop/fs/permission/FsPermission"
  38. #define JAVA_NET_ISA "java/net/InetSocketAddress"
  39. #define JAVA_NET_URI "java/net/URI"
  40. #define JAVA_STRING "java/lang/String"
  41. #define READ_OPTION "org/apache/hadoop/fs/ReadOption"
  42. #define JAVA_VOID "V"
  43. /* Macros for constructing method signatures */
  44. #define JPARAM(X) "L" X ";"
  45. #define JARRPARAM(X) "[L" X ";"
  46. #define JMETHOD1(X, R) "(" X ")" R
  47. #define JMETHOD2(X, Y, R) "(" X Y ")" R
  48. #define JMETHOD3(X, Y, Z, R) "(" X Y Z")" R
  49. #define KERBEROS_TICKET_CACHE_PATH "hadoop.security.kerberos.ticket.cache.path"
  50. // Bit fields for hdfsFile_internal flags
  51. #define HDFS_FILE_SUPPORTS_DIRECT_READ (1<<0)
  52. tSize readDirect(hdfsFS fs, hdfsFile f, void* buffer, tSize length);
  53. static void hdfsFreeFileInfoEntry(hdfsFileInfo *hdfsFileInfo);
  54. /**
  55. * The C equivalent of org.apache.org.hadoop.FSData(Input|Output)Stream .
  56. */
  57. enum hdfsStreamType
  58. {
  59. HDFS_STREAM_UNINITIALIZED = 0,
  60. HDFS_STREAM_INPUT = 1,
  61. HDFS_STREAM_OUTPUT = 2,
  62. };
  63. /**
  64. * The 'file-handle' to a file in hdfs.
  65. */
  66. struct hdfsFile_internal {
  67. void* file;
  68. enum hdfsStreamType type;
  69. int flags;
  70. };
  71. #define HDFS_EXTENDED_FILE_INFO_ENCRYPTED 0x1
  72. /**
  73. * Extended file information.
  74. */
  75. struct hdfsExtendedFileInfo {
  76. int flags;
  77. };
  78. int hdfsFileIsOpenForRead(hdfsFile file)
  79. {
  80. return (file->type == HDFS_STREAM_INPUT);
  81. }
  82. int hdfsGetHedgedReadMetrics(hdfsFS fs, struct hdfsHedgedReadMetrics **metrics)
  83. {
  84. jthrowable jthr;
  85. jobject hedgedReadMetrics = NULL;
  86. jvalue jVal;
  87. struct hdfsHedgedReadMetrics *m = NULL;
  88. int ret;
  89. jobject jFS = (jobject)fs;
  90. JNIEnv* env = getJNIEnv();
  91. if (env == NULL) {
  92. errno = EINTERNAL;
  93. return -1;
  94. }
  95. jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
  96. HADOOP_DFS,
  97. "getHedgedReadMetrics",
  98. "()Lorg/apache/hadoop/hdfs/DFSHedgedReadMetrics;");
  99. if (jthr) {
  100. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  101. "hdfsGetHedgedReadMetrics: getHedgedReadMetrics failed");
  102. goto done;
  103. }
  104. hedgedReadMetrics = jVal.l;
  105. m = malloc(sizeof(struct hdfsHedgedReadMetrics));
  106. if (!m) {
  107. ret = ENOMEM;
  108. goto done;
  109. }
  110. jthr = invokeMethod(env, &jVal, INSTANCE, hedgedReadMetrics,
  111. "org/apache/hadoop/hdfs/DFSHedgedReadMetrics",
  112. "getHedgedReadOps", "()J");
  113. if (jthr) {
  114. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  115. "hdfsGetHedgedReadStatistics: getHedgedReadOps failed");
  116. goto done;
  117. }
  118. m->hedgedReadOps = jVal.j;
  119. jthr = invokeMethod(env, &jVal, INSTANCE, hedgedReadMetrics,
  120. "org/apache/hadoop/hdfs/DFSHedgedReadMetrics",
  121. "getHedgedReadWins", "()J");
  122. if (jthr) {
  123. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  124. "hdfsGetHedgedReadStatistics: getHedgedReadWins failed");
  125. goto done;
  126. }
  127. m->hedgedReadOpsWin = jVal.j;
  128. jthr = invokeMethod(env, &jVal, INSTANCE, hedgedReadMetrics,
  129. "org/apache/hadoop/hdfs/DFSHedgedReadMetrics",
  130. "getHedgedReadOpsInCurThread", "()J");
  131. if (jthr) {
  132. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  133. "hdfsGetHedgedReadStatistics: getHedgedReadOpsInCurThread failed");
  134. goto done;
  135. }
  136. m->hedgedReadOpsInCurThread = jVal.j;
  137. *metrics = m;
  138. m = NULL;
  139. ret = 0;
  140. done:
  141. destroyLocalReference(env, hedgedReadMetrics);
  142. free(m);
  143. if (ret) {
  144. errno = ret;
  145. return -1;
  146. }
  147. return 0;
  148. }
  149. void hdfsFreeHedgedReadMetrics(struct hdfsHedgedReadMetrics *metrics)
  150. {
  151. free(metrics);
  152. }
  153. int hdfsFileGetReadStatistics(hdfsFile file,
  154. struct hdfsReadStatistics **stats)
  155. {
  156. jthrowable jthr;
  157. jobject readStats = NULL;
  158. jvalue jVal;
  159. struct hdfsReadStatistics *s = NULL;
  160. int ret;
  161. JNIEnv* env = getJNIEnv();
  162. if (env == NULL) {
  163. errno = EINTERNAL;
  164. return -1;
  165. }
  166. if (file->type != HDFS_STREAM_INPUT) {
  167. ret = EINVAL;
  168. goto done;
  169. }
  170. jthr = invokeMethod(env, &jVal, INSTANCE, file->file,
  171. "org/apache/hadoop/hdfs/client/HdfsDataInputStream",
  172. "getReadStatistics",
  173. "()Lorg/apache/hadoop/hdfs/ReadStatistics;");
  174. if (jthr) {
  175. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  176. "hdfsFileGetReadStatistics: getReadStatistics failed");
  177. goto done;
  178. }
  179. readStats = jVal.l;
  180. s = malloc(sizeof(struct hdfsReadStatistics));
  181. if (!s) {
  182. ret = ENOMEM;
  183. goto done;
  184. }
  185. jthr = invokeMethod(env, &jVal, INSTANCE, readStats,
  186. "org/apache/hadoop/hdfs/ReadStatistics",
  187. "getTotalBytesRead", "()J");
  188. if (jthr) {
  189. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  190. "hdfsFileGetReadStatistics: getTotalBytesRead failed");
  191. goto done;
  192. }
  193. s->totalBytesRead = jVal.j;
  194. jthr = invokeMethod(env, &jVal, INSTANCE, readStats,
  195. "org/apache/hadoop/hdfs/ReadStatistics",
  196. "getTotalLocalBytesRead", "()J");
  197. if (jthr) {
  198. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  199. "hdfsFileGetReadStatistics: getTotalLocalBytesRead failed");
  200. goto done;
  201. }
  202. s->totalLocalBytesRead = jVal.j;
  203. jthr = invokeMethod(env, &jVal, INSTANCE, readStats,
  204. "org/apache/hadoop/hdfs/ReadStatistics",
  205. "getTotalShortCircuitBytesRead", "()J");
  206. if (jthr) {
  207. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  208. "hdfsFileGetReadStatistics: getTotalShortCircuitBytesRead failed");
  209. goto done;
  210. }
  211. s->totalShortCircuitBytesRead = jVal.j;
  212. jthr = invokeMethod(env, &jVal, INSTANCE, readStats,
  213. "org/apache/hadoop/hdfs/ReadStatistics",
  214. "getTotalZeroCopyBytesRead", "()J");
  215. if (jthr) {
  216. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  217. "hdfsFileGetReadStatistics: getTotalZeroCopyBytesRead failed");
  218. goto done;
  219. }
  220. s->totalZeroCopyBytesRead = jVal.j;
  221. *stats = s;
  222. s = NULL;
  223. ret = 0;
  224. done:
  225. destroyLocalReference(env, readStats);
  226. free(s);
  227. if (ret) {
  228. errno = ret;
  229. return -1;
  230. }
  231. return 0;
  232. }
  233. int64_t hdfsReadStatisticsGetRemoteBytesRead(
  234. const struct hdfsReadStatistics *stats)
  235. {
  236. return stats->totalBytesRead - stats->totalLocalBytesRead;
  237. }
  238. int hdfsFileClearReadStatistics(hdfsFile file)
  239. {
  240. jthrowable jthr;
  241. int ret;
  242. JNIEnv* env = getJNIEnv();
  243. if (env == NULL) {
  244. errno = EINTERNAL;
  245. return EINTERNAL;
  246. }
  247. if (file->type != HDFS_STREAM_INPUT) {
  248. ret = EINVAL;
  249. goto done;
  250. }
  251. jthr = invokeMethod(env, NULL, INSTANCE, file->file,
  252. "org/apache/hadoop/hdfs/client/HdfsDataInputStream",
  253. "clearReadStatistics", "()V");
  254. if (jthr) {
  255. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  256. "hdfsFileClearReadStatistics: clearReadStatistics failed");
  257. goto done;
  258. }
  259. ret = 0;
  260. done:
  261. if (ret) {
  262. errno = ret;
  263. return ret;
  264. }
  265. return 0;
  266. }
  267. void hdfsFileFreeReadStatistics(struct hdfsReadStatistics *stats)
  268. {
  269. free(stats);
  270. }
  271. int hdfsFileIsOpenForWrite(hdfsFile file)
  272. {
  273. return (file->type == HDFS_STREAM_OUTPUT);
  274. }
  275. int hdfsFileUsesDirectRead(hdfsFile file)
  276. {
  277. return !!(file->flags & HDFS_FILE_SUPPORTS_DIRECT_READ);
  278. }
  279. void hdfsFileDisableDirectRead(hdfsFile file)
  280. {
  281. file->flags &= ~HDFS_FILE_SUPPORTS_DIRECT_READ;
  282. }
  283. int hdfsDisableDomainSocketSecurity(void)
  284. {
  285. jthrowable jthr;
  286. JNIEnv* env = getJNIEnv();
  287. if (env == NULL) {
  288. errno = EINTERNAL;
  289. return -1;
  290. }
  291. jthr = invokeMethod(env, NULL, STATIC, NULL,
  292. "org/apache/hadoop/net/unix/DomainSocket",
  293. "disableBindPathValidation", "()V");
  294. if (jthr) {
  295. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  296. "DomainSocket#disableBindPathValidation");
  297. return -1;
  298. }
  299. return 0;
  300. }
  301. /**
  302. * hdfsJniEnv: A wrapper struct to be used as 'value'
  303. * while saving thread -> JNIEnv* mappings
  304. */
  305. typedef struct
  306. {
  307. JNIEnv* env;
  308. } hdfsJniEnv;
  309. /**
  310. * Helper function to create a org.apache.hadoop.fs.Path object.
  311. * @param env: The JNIEnv pointer.
  312. * @param path: The file-path for which to construct org.apache.hadoop.fs.Path
  313. * object.
  314. * @return Returns a jobject on success and NULL on error.
  315. */
  316. static jthrowable constructNewObjectOfPath(JNIEnv *env, const char *path,
  317. jobject *out)
  318. {
  319. jthrowable jthr;
  320. jstring jPathString;
  321. jobject jPath;
  322. //Construct a java.lang.String object
  323. jthr = newJavaStr(env, path, &jPathString);
  324. if (jthr)
  325. return jthr;
  326. //Construct the org.apache.hadoop.fs.Path object
  327. jthr = constructNewObjectOfClass(env, &jPath, "org/apache/hadoop/fs/Path",
  328. "(Ljava/lang/String;)V", jPathString);
  329. destroyLocalReference(env, jPathString);
  330. if (jthr)
  331. return jthr;
  332. *out = jPath;
  333. return NULL;
  334. }
  335. static jthrowable hadoopConfGetStr(JNIEnv *env, jobject jConfiguration,
  336. const char *key, char **val)
  337. {
  338. jthrowable jthr;
  339. jvalue jVal;
  340. jstring jkey = NULL, jRet = NULL;
  341. jthr = newJavaStr(env, key, &jkey);
  342. if (jthr)
  343. goto done;
  344. jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration,
  345. HADOOP_CONF, "get", JMETHOD1(JPARAM(JAVA_STRING),
  346. JPARAM(JAVA_STRING)), jkey);
  347. if (jthr)
  348. goto done;
  349. jRet = jVal.l;
  350. jthr = newCStr(env, jRet, val);
  351. done:
  352. destroyLocalReference(env, jkey);
  353. destroyLocalReference(env, jRet);
  354. return jthr;
  355. }
  356. int hdfsConfGetStr(const char *key, char **val)
  357. {
  358. JNIEnv *env;
  359. int ret;
  360. jthrowable jthr;
  361. jobject jConfiguration = NULL;
  362. env = getJNIEnv();
  363. if (env == NULL) {
  364. ret = EINTERNAL;
  365. goto done;
  366. }
  367. jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V");
  368. if (jthr) {
  369. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  370. "hdfsConfGetStr(%s): new Configuration", key);
  371. goto done;
  372. }
  373. jthr = hadoopConfGetStr(env, jConfiguration, key, val);
  374. if (jthr) {
  375. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  376. "hdfsConfGetStr(%s): hadoopConfGetStr", key);
  377. goto done;
  378. }
  379. ret = 0;
  380. done:
  381. destroyLocalReference(env, jConfiguration);
  382. if (ret)
  383. errno = ret;
  384. return ret;
  385. }
  386. void hdfsConfStrFree(char *val)
  387. {
  388. free(val);
  389. }
  390. static jthrowable hadoopConfGetInt(JNIEnv *env, jobject jConfiguration,
  391. const char *key, int32_t *val)
  392. {
  393. jthrowable jthr = NULL;
  394. jvalue jVal;
  395. jstring jkey = NULL;
  396. jthr = newJavaStr(env, key, &jkey);
  397. if (jthr)
  398. return jthr;
  399. jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration,
  400. HADOOP_CONF, "getInt", JMETHOD2(JPARAM(JAVA_STRING), "I", "I"),
  401. jkey, (jint)(*val));
  402. destroyLocalReference(env, jkey);
  403. if (jthr)
  404. return jthr;
  405. *val = jVal.i;
  406. return NULL;
  407. }
  408. int hdfsConfGetInt(const char *key, int32_t *val)
  409. {
  410. JNIEnv *env;
  411. int ret;
  412. jobject jConfiguration = NULL;
  413. jthrowable jthr;
  414. env = getJNIEnv();
  415. if (env == NULL) {
  416. ret = EINTERNAL;
  417. goto done;
  418. }
  419. jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V");
  420. if (jthr) {
  421. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  422. "hdfsConfGetInt(%s): new Configuration", key);
  423. goto done;
  424. }
  425. jthr = hadoopConfGetInt(env, jConfiguration, key, val);
  426. if (jthr) {
  427. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  428. "hdfsConfGetInt(%s): hadoopConfGetInt", key);
  429. goto done;
  430. }
  431. ret = 0;
  432. done:
  433. destroyLocalReference(env, jConfiguration);
  434. if (ret)
  435. errno = ret;
  436. return ret;
  437. }
  438. struct hdfsBuilderConfOpt {
  439. struct hdfsBuilderConfOpt *next;
  440. const char *key;
  441. const char *val;
  442. };
  443. struct hdfsBuilder {
  444. int forceNewInstance;
  445. const char *nn;
  446. tPort port;
  447. const char *kerbTicketCachePath;
  448. const char *userName;
  449. struct hdfsBuilderConfOpt *opts;
  450. };
  451. struct hdfsBuilder *hdfsNewBuilder(void)
  452. {
  453. struct hdfsBuilder *bld = calloc(1, sizeof(struct hdfsBuilder));
  454. if (!bld) {
  455. errno = ENOMEM;
  456. return NULL;
  457. }
  458. return bld;
  459. }
  460. int hdfsBuilderConfSetStr(struct hdfsBuilder *bld, const char *key,
  461. const char *val)
  462. {
  463. struct hdfsBuilderConfOpt *opt, *next;
  464. opt = calloc(1, sizeof(struct hdfsBuilderConfOpt));
  465. if (!opt)
  466. return -ENOMEM;
  467. next = bld->opts;
  468. bld->opts = opt;
  469. opt->next = next;
  470. opt->key = key;
  471. opt->val = val;
  472. return 0;
  473. }
  474. void hdfsFreeBuilder(struct hdfsBuilder *bld)
  475. {
  476. struct hdfsBuilderConfOpt *cur, *next;
  477. cur = bld->opts;
  478. for (cur = bld->opts; cur; ) {
  479. next = cur->next;
  480. free(cur);
  481. cur = next;
  482. }
  483. free(bld);
  484. }
  485. void hdfsBuilderSetForceNewInstance(struct hdfsBuilder *bld)
  486. {
  487. bld->forceNewInstance = 1;
  488. }
  489. void hdfsBuilderSetNameNode(struct hdfsBuilder *bld, const char *nn)
  490. {
  491. bld->nn = nn;
  492. }
  493. void hdfsBuilderSetNameNodePort(struct hdfsBuilder *bld, tPort port)
  494. {
  495. bld->port = port;
  496. }
  497. void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName)
  498. {
  499. bld->userName = userName;
  500. }
  501. void hdfsBuilderSetKerbTicketCachePath(struct hdfsBuilder *bld,
  502. const char *kerbTicketCachePath)
  503. {
  504. bld->kerbTicketCachePath = kerbTicketCachePath;
  505. }
  506. hdfsFS hdfsConnect(const char *host, tPort port)
  507. {
  508. struct hdfsBuilder *bld = hdfsNewBuilder();
  509. if (!bld)
  510. return NULL;
  511. hdfsBuilderSetNameNode(bld, host);
  512. hdfsBuilderSetNameNodePort(bld, port);
  513. return hdfsBuilderConnect(bld);
  514. }
  515. /** Always return a new FileSystem handle */
  516. hdfsFS hdfsConnectNewInstance(const char *host, tPort port)
  517. {
  518. struct hdfsBuilder *bld = hdfsNewBuilder();
  519. if (!bld)
  520. return NULL;
  521. hdfsBuilderSetNameNode(bld, host);
  522. hdfsBuilderSetNameNodePort(bld, port);
  523. hdfsBuilderSetForceNewInstance(bld);
  524. return hdfsBuilderConnect(bld);
  525. }
  526. hdfsFS hdfsConnectAsUser(const char *host, tPort port, const char *user)
  527. {
  528. struct hdfsBuilder *bld = hdfsNewBuilder();
  529. if (!bld)
  530. return NULL;
  531. hdfsBuilderSetNameNode(bld, host);
  532. hdfsBuilderSetNameNodePort(bld, port);
  533. hdfsBuilderSetUserName(bld, user);
  534. return hdfsBuilderConnect(bld);
  535. }
  536. /** Always return a new FileSystem handle */
  537. hdfsFS hdfsConnectAsUserNewInstance(const char *host, tPort port,
  538. const char *user)
  539. {
  540. struct hdfsBuilder *bld = hdfsNewBuilder();
  541. if (!bld)
  542. return NULL;
  543. hdfsBuilderSetNameNode(bld, host);
  544. hdfsBuilderSetNameNodePort(bld, port);
  545. hdfsBuilderSetForceNewInstance(bld);
  546. hdfsBuilderSetUserName(bld, user);
  547. return hdfsBuilderConnect(bld);
  548. }
  549. /**
  550. * Calculate the effective URI to use, given a builder configuration.
  551. *
  552. * If there is not already a URI scheme, we prepend 'hdfs://'.
  553. *
  554. * If there is not already a port specified, and a port was given to the
  555. * builder, we suffix that port. If there is a port specified but also one in
  556. * the URI, that is an error.
  557. *
  558. * @param bld The hdfs builder object
  559. * @param uri (out param) dynamically allocated string representing the
  560. * effective URI
  561. *
  562. * @return 0 on success; error code otherwise
  563. */
  564. static int calcEffectiveURI(struct hdfsBuilder *bld, char ** uri)
  565. {
  566. const char *scheme;
  567. char suffix[64];
  568. const char *lastColon;
  569. char *u;
  570. size_t uriLen;
  571. if (!bld->nn)
  572. return EINVAL;
  573. scheme = (strstr(bld->nn, "://")) ? "" : "hdfs://";
  574. if (bld->port == 0) {
  575. suffix[0] = '\0';
  576. } else {
  577. lastColon = strrchr(bld->nn, ':');
  578. if (lastColon && (strspn(lastColon + 1, "0123456789") ==
  579. strlen(lastColon + 1))) {
  580. fprintf(stderr, "port %d was given, but URI '%s' already "
  581. "contains a port!\n", bld->port, bld->nn);
  582. return EINVAL;
  583. }
  584. snprintf(suffix, sizeof(suffix), ":%d", bld->port);
  585. }
  586. uriLen = strlen(scheme) + strlen(bld->nn) + strlen(suffix);
  587. u = malloc((uriLen + 1) * (sizeof(char)));
  588. if (!u) {
  589. fprintf(stderr, "calcEffectiveURI: out of memory");
  590. return ENOMEM;
  591. }
  592. snprintf(u, uriLen + 1, "%s%s%s", scheme, bld->nn, suffix);
  593. *uri = u;
  594. return 0;
  595. }
  596. static const char *maybeNull(const char *str)
  597. {
  598. return str ? str : "(NULL)";
  599. }
  600. static const char *hdfsBuilderToStr(const struct hdfsBuilder *bld,
  601. char *buf, size_t bufLen)
  602. {
  603. snprintf(buf, bufLen, "forceNewInstance=%d, nn=%s, port=%d, "
  604. "kerbTicketCachePath=%s, userName=%s",
  605. bld->forceNewInstance, maybeNull(bld->nn), bld->port,
  606. maybeNull(bld->kerbTicketCachePath), maybeNull(bld->userName));
  607. return buf;
  608. }
  609. hdfsFS hdfsBuilderConnect(struct hdfsBuilder *bld)
  610. {
  611. JNIEnv *env = 0;
  612. jobject jConfiguration = NULL, jFS = NULL, jURI = NULL, jCachePath = NULL;
  613. jstring jURIString = NULL, jUserString = NULL;
  614. jvalue jVal;
  615. jthrowable jthr = NULL;
  616. char *cURI = 0, buf[512];
  617. int ret;
  618. jobject jRet = NULL;
  619. struct hdfsBuilderConfOpt *opt;
  620. //Get the JNIEnv* corresponding to current thread
  621. env = getJNIEnv();
  622. if (env == NULL) {
  623. ret = EINTERNAL;
  624. goto done;
  625. }
  626. // jConfiguration = new Configuration();
  627. jthr = constructNewObjectOfClass(env, &jConfiguration, HADOOP_CONF, "()V");
  628. if (jthr) {
  629. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  630. "hdfsBuilderConnect(%s)", hdfsBuilderToStr(bld, buf, sizeof(buf)));
  631. goto done;
  632. }
  633. // set configuration values
  634. for (opt = bld->opts; opt; opt = opt->next) {
  635. jthr = hadoopConfSetStr(env, jConfiguration, opt->key, opt->val);
  636. if (jthr) {
  637. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  638. "hdfsBuilderConnect(%s): error setting conf '%s' to '%s'",
  639. hdfsBuilderToStr(bld, buf, sizeof(buf)), opt->key, opt->val);
  640. goto done;
  641. }
  642. }
  643. //Check what type of FileSystem the caller wants...
  644. if (bld->nn == NULL) {
  645. // Get a local filesystem.
  646. if (bld->forceNewInstance) {
  647. // fs = FileSytem#newInstanceLocal(conf);
  648. jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS,
  649. "newInstanceLocal", JMETHOD1(JPARAM(HADOOP_CONF),
  650. JPARAM(HADOOP_LOCALFS)), jConfiguration);
  651. if (jthr) {
  652. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  653. "hdfsBuilderConnect(%s)",
  654. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  655. goto done;
  656. }
  657. jFS = jVal.l;
  658. } else {
  659. // fs = FileSytem#getLocal(conf);
  660. jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS, "getLocal",
  661. JMETHOD1(JPARAM(HADOOP_CONF),
  662. JPARAM(HADOOP_LOCALFS)),
  663. jConfiguration);
  664. if (jthr) {
  665. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  666. "hdfsBuilderConnect(%s)",
  667. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  668. goto done;
  669. }
  670. jFS = jVal.l;
  671. }
  672. } else {
  673. if (!strcmp(bld->nn, "default")) {
  674. // jURI = FileSystem.getDefaultUri(conf)
  675. jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS,
  676. "getDefaultUri",
  677. "(Lorg/apache/hadoop/conf/Configuration;)Ljava/net/URI;",
  678. jConfiguration);
  679. if (jthr) {
  680. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  681. "hdfsBuilderConnect(%s)",
  682. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  683. goto done;
  684. }
  685. jURI = jVal.l;
  686. } else {
  687. // fs = FileSystem#get(URI, conf, ugi);
  688. ret = calcEffectiveURI(bld, &cURI);
  689. if (ret)
  690. goto done;
  691. jthr = newJavaStr(env, cURI, &jURIString);
  692. if (jthr) {
  693. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  694. "hdfsBuilderConnect(%s)",
  695. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  696. goto done;
  697. }
  698. jthr = invokeMethod(env, &jVal, STATIC, NULL, JAVA_NET_URI,
  699. "create", "(Ljava/lang/String;)Ljava/net/URI;",
  700. jURIString);
  701. if (jthr) {
  702. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  703. "hdfsBuilderConnect(%s)",
  704. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  705. goto done;
  706. }
  707. jURI = jVal.l;
  708. }
  709. if (bld->kerbTicketCachePath) {
  710. jthr = hadoopConfSetStr(env, jConfiguration,
  711. KERBEROS_TICKET_CACHE_PATH, bld->kerbTicketCachePath);
  712. if (jthr) {
  713. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  714. "hdfsBuilderConnect(%s)",
  715. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  716. goto done;
  717. }
  718. }
  719. jthr = newJavaStr(env, bld->userName, &jUserString);
  720. if (jthr) {
  721. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  722. "hdfsBuilderConnect(%s)",
  723. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  724. goto done;
  725. }
  726. if (bld->forceNewInstance) {
  727. jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS,
  728. "newInstance", JMETHOD3(JPARAM(JAVA_NET_URI),
  729. JPARAM(HADOOP_CONF), JPARAM(JAVA_STRING),
  730. JPARAM(HADOOP_FS)),
  731. jURI, jConfiguration, jUserString);
  732. if (jthr) {
  733. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  734. "hdfsBuilderConnect(%s)",
  735. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  736. goto done;
  737. }
  738. jFS = jVal.l;
  739. } else {
  740. jthr = invokeMethod(env, &jVal, STATIC, NULL, HADOOP_FS, "get",
  741. JMETHOD3(JPARAM(JAVA_NET_URI), JPARAM(HADOOP_CONF),
  742. JPARAM(JAVA_STRING), JPARAM(HADOOP_FS)),
  743. jURI, jConfiguration, jUserString);
  744. if (jthr) {
  745. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  746. "hdfsBuilderConnect(%s)",
  747. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  748. goto done;
  749. }
  750. jFS = jVal.l;
  751. }
  752. }
  753. jRet = (*env)->NewGlobalRef(env, jFS);
  754. if (!jRet) {
  755. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  756. "hdfsBuilderConnect(%s)",
  757. hdfsBuilderToStr(bld, buf, sizeof(buf)));
  758. goto done;
  759. }
  760. ret = 0;
  761. done:
  762. // Release unnecessary local references
  763. destroyLocalReference(env, jConfiguration);
  764. destroyLocalReference(env, jFS);
  765. destroyLocalReference(env, jURI);
  766. destroyLocalReference(env, jCachePath);
  767. destroyLocalReference(env, jURIString);
  768. destroyLocalReference(env, jUserString);
  769. free(cURI);
  770. hdfsFreeBuilder(bld);
  771. if (ret) {
  772. errno = ret;
  773. return NULL;
  774. }
  775. return (hdfsFS)jRet;
  776. }
  777. int hdfsDisconnect(hdfsFS fs)
  778. {
  779. // JAVA EQUIVALENT:
  780. // fs.close()
  781. //Get the JNIEnv* corresponding to current thread
  782. JNIEnv* env = getJNIEnv();
  783. int ret;
  784. jobject jFS;
  785. jthrowable jthr;
  786. if (env == NULL) {
  787. errno = EINTERNAL;
  788. return -1;
  789. }
  790. //Parameters
  791. jFS = (jobject)fs;
  792. //Sanity check
  793. if (fs == NULL) {
  794. errno = EBADF;
  795. return -1;
  796. }
  797. jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
  798. "close", "()V");
  799. if (jthr) {
  800. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  801. "hdfsDisconnect: FileSystem#close");
  802. } else {
  803. ret = 0;
  804. }
  805. (*env)->DeleteGlobalRef(env, jFS);
  806. if (ret) {
  807. errno = ret;
  808. return -1;
  809. }
  810. return 0;
  811. }
  812. /**
  813. * Get the default block size of a FileSystem object.
  814. *
  815. * @param env The Java env
  816. * @param jFS The FileSystem object
  817. * @param jPath The path to find the default blocksize at
  818. * @param out (out param) the default block size
  819. *
  820. * @return NULL on success; or the exception
  821. */
  822. static jthrowable getDefaultBlockSize(JNIEnv *env, jobject jFS,
  823. jobject jPath, jlong *out)
  824. {
  825. jthrowable jthr;
  826. jvalue jVal;
  827. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  828. "getDefaultBlockSize", JMETHOD1(JPARAM(HADOOP_PATH), "J"), jPath);
  829. if (jthr)
  830. return jthr;
  831. *out = jVal.j;
  832. return NULL;
  833. }
  834. hdfsFile hdfsOpenFile(hdfsFS fs, const char *path, int flags,
  835. int bufferSize, short replication, tSize blockSize)
  836. {
  837. struct hdfsStreamBuilder *bld = hdfsStreamBuilderAlloc(fs, path, flags);
  838. if (bufferSize != 0) {
  839. hdfsStreamBuilderSetBufferSize(bld, bufferSize);
  840. }
  841. if (replication != 0) {
  842. hdfsStreamBuilderSetReplication(bld, replication);
  843. }
  844. if (blockSize != 0) {
  845. hdfsStreamBuilderSetDefaultBlockSize(bld, blockSize);
  846. }
  847. return hdfsStreamBuilderBuild(bld);
  848. }
  849. struct hdfsStreamBuilder {
  850. hdfsFS fs;
  851. int flags;
  852. int32_t bufferSize;
  853. int16_t replication;
  854. int64_t defaultBlockSize;
  855. char path[1];
  856. };
  857. struct hdfsStreamBuilder *hdfsStreamBuilderAlloc(hdfsFS fs,
  858. const char *path, int flags)
  859. {
  860. int path_len = strlen(path);
  861. struct hdfsStreamBuilder *bld;
  862. // sizeof(hdfsStreamBuilder->path) includes one byte for the string
  863. // terminator
  864. bld = malloc(sizeof(struct hdfsStreamBuilder) + path_len);
  865. if (!bld) {
  866. errno = ENOMEM;
  867. return NULL;
  868. }
  869. bld->fs = fs;
  870. bld->flags = flags;
  871. bld->bufferSize = 0;
  872. bld->replication = 0;
  873. bld->defaultBlockSize = 0;
  874. memcpy(bld->path, path, path_len);
  875. bld->path[path_len] = '\0';
  876. return bld;
  877. }
  878. void hdfsStreamBuilderFree(struct hdfsStreamBuilder *bld)
  879. {
  880. free(bld);
  881. }
  882. int hdfsStreamBuilderSetBufferSize(struct hdfsStreamBuilder *bld,
  883. int32_t bufferSize)
  884. {
  885. if ((bld->flags & O_ACCMODE) != O_WRONLY) {
  886. errno = EINVAL;
  887. return -1;
  888. }
  889. bld->bufferSize = bufferSize;
  890. return 0;
  891. }
  892. int hdfsStreamBuilderSetReplication(struct hdfsStreamBuilder *bld,
  893. int16_t replication)
  894. {
  895. if ((bld->flags & O_ACCMODE) != O_WRONLY) {
  896. errno = EINVAL;
  897. return -1;
  898. }
  899. bld->replication = replication;
  900. return 0;
  901. }
  902. int hdfsStreamBuilderSetDefaultBlockSize(struct hdfsStreamBuilder *bld,
  903. int64_t defaultBlockSize)
  904. {
  905. if ((bld->flags & O_ACCMODE) != O_WRONLY) {
  906. errno = EINVAL;
  907. return -1;
  908. }
  909. bld->defaultBlockSize = defaultBlockSize;
  910. return 0;
  911. }
  912. static hdfsFile hdfsOpenFileImpl(hdfsFS fs, const char *path, int flags,
  913. int32_t bufferSize, int16_t replication, int64_t blockSize)
  914. {
  915. /*
  916. JAVA EQUIVALENT:
  917. File f = new File(path);
  918. FSData{Input|Output}Stream f{is|os} = fs.create(f);
  919. return f{is|os};
  920. */
  921. int accmode = flags & O_ACCMODE;
  922. jstring jStrBufferSize = NULL, jStrReplication = NULL;
  923. jobject jConfiguration = NULL, jPath = NULL, jFile = NULL;
  924. jobject jFS = (jobject)fs;
  925. jthrowable jthr;
  926. jvalue jVal;
  927. hdfsFile file = NULL;
  928. int ret;
  929. jint jBufferSize = bufferSize;
  930. jshort jReplication = replication;
  931. /* The hadoop java api/signature */
  932. const char *method = NULL;
  933. const char *signature = NULL;
  934. /* Get the JNIEnv* corresponding to current thread */
  935. JNIEnv* env = getJNIEnv();
  936. if (env == NULL) {
  937. errno = EINTERNAL;
  938. return NULL;
  939. }
  940. if (accmode == O_RDONLY || accmode == O_WRONLY) {
  941. /* yay */
  942. } else if (accmode == O_RDWR) {
  943. fprintf(stderr, "ERROR: cannot open an hdfs file in O_RDWR mode\n");
  944. errno = ENOTSUP;
  945. return NULL;
  946. } else {
  947. fprintf(stderr, "ERROR: cannot open an hdfs file in mode 0x%x\n", accmode);
  948. errno = EINVAL;
  949. return NULL;
  950. }
  951. if ((flags & O_CREAT) && (flags & O_EXCL)) {
  952. fprintf(stderr, "WARN: hdfs does not truly support O_CREATE && O_EXCL\n");
  953. }
  954. if (accmode == O_RDONLY) {
  955. method = "open";
  956. signature = JMETHOD2(JPARAM(HADOOP_PATH), "I", JPARAM(HADOOP_ISTRM));
  957. } else if (flags & O_APPEND) {
  958. method = "append";
  959. signature = JMETHOD1(JPARAM(HADOOP_PATH), JPARAM(HADOOP_OSTRM));
  960. } else {
  961. method = "create";
  962. signature = JMETHOD2(JPARAM(HADOOP_PATH), "ZISJ", JPARAM(HADOOP_OSTRM));
  963. }
  964. /* Create an object of org.apache.hadoop.fs.Path */
  965. jthr = constructNewObjectOfPath(env, path, &jPath);
  966. if (jthr) {
  967. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  968. "hdfsOpenFile(%s): constructNewObjectOfPath", path);
  969. goto done;
  970. }
  971. /* Get the Configuration object from the FileSystem object */
  972. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  973. "getConf", JMETHOD1("", JPARAM(HADOOP_CONF)));
  974. if (jthr) {
  975. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  976. "hdfsOpenFile(%s): FileSystem#getConf", path);
  977. goto done;
  978. }
  979. jConfiguration = jVal.l;
  980. jStrBufferSize = (*env)->NewStringUTF(env, "io.file.buffer.size");
  981. if (!jStrBufferSize) {
  982. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL, "OOM");
  983. goto done;
  984. }
  985. jStrReplication = (*env)->NewStringUTF(env, "dfs.replication");
  986. if (!jStrReplication) {
  987. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL, "OOM");
  988. goto done;
  989. }
  990. if (!bufferSize) {
  991. jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration,
  992. HADOOP_CONF, "getInt", "(Ljava/lang/String;I)I",
  993. jStrBufferSize, 4096);
  994. if (jthr) {
  995. ret = printExceptionAndFree(env, jthr, NOPRINT_EXC_FILE_NOT_FOUND |
  996. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_UNRESOLVED_LINK,
  997. "hdfsOpenFile(%s): Configuration#getInt(io.file.buffer.size)",
  998. path);
  999. goto done;
  1000. }
  1001. jBufferSize = jVal.i;
  1002. }
  1003. if ((accmode == O_WRONLY) && (flags & O_APPEND) == 0) {
  1004. if (!replication) {
  1005. jthr = invokeMethod(env, &jVal, INSTANCE, jConfiguration,
  1006. HADOOP_CONF, "getInt", "(Ljava/lang/String;I)I",
  1007. jStrReplication, 1);
  1008. if (jthr) {
  1009. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1010. "hdfsOpenFile(%s): Configuration#getInt(dfs.replication)",
  1011. path);
  1012. goto done;
  1013. }
  1014. jReplication = (jshort)jVal.i;
  1015. }
  1016. }
  1017. /* Create and return either the FSDataInputStream or
  1018. FSDataOutputStream references jobject jStream */
  1019. // READ?
  1020. if (accmode == O_RDONLY) {
  1021. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1022. method, signature, jPath, jBufferSize);
  1023. } else if ((accmode == O_WRONLY) && (flags & O_APPEND)) {
  1024. // WRITE/APPEND?
  1025. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1026. method, signature, jPath);
  1027. } else {
  1028. // WRITE/CREATE
  1029. jboolean jOverWrite = 1;
  1030. jlong jBlockSize = blockSize;
  1031. if (jBlockSize == 0) {
  1032. jthr = getDefaultBlockSize(env, jFS, jPath, &jBlockSize);
  1033. if (jthr) {
  1034. ret = EIO;
  1035. goto done;
  1036. }
  1037. }
  1038. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1039. method, signature, jPath, jOverWrite,
  1040. jBufferSize, jReplication, jBlockSize);
  1041. }
  1042. if (jthr) {
  1043. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1044. "hdfsOpenFile(%s): FileSystem#%s(%s)", path, method, signature);
  1045. goto done;
  1046. }
  1047. jFile = jVal.l;
  1048. file = calloc(1, sizeof(struct hdfsFile_internal));
  1049. if (!file) {
  1050. fprintf(stderr, "hdfsOpenFile(%s): OOM create hdfsFile\n", path);
  1051. ret = ENOMEM;
  1052. goto done;
  1053. }
  1054. file->file = (*env)->NewGlobalRef(env, jFile);
  1055. if (!file->file) {
  1056. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1057. "hdfsOpenFile(%s): NewGlobalRef", path);
  1058. goto done;
  1059. }
  1060. file->type = (((flags & O_WRONLY) == 0) ? HDFS_STREAM_INPUT :
  1061. HDFS_STREAM_OUTPUT);
  1062. file->flags = 0;
  1063. if ((flags & O_WRONLY) == 0) {
  1064. // Try a test read to see if we can do direct reads
  1065. char buf;
  1066. if (readDirect(fs, file, &buf, 0) == 0) {
  1067. // Success - 0-byte read should return 0
  1068. file->flags |= HDFS_FILE_SUPPORTS_DIRECT_READ;
  1069. } else if (errno != ENOTSUP) {
  1070. // Unexpected error. Clear it, don't set the direct flag.
  1071. fprintf(stderr,
  1072. "hdfsOpenFile(%s): WARN: Unexpected error %d when testing "
  1073. "for direct read compatibility\n", path, errno);
  1074. }
  1075. }
  1076. ret = 0;
  1077. done:
  1078. destroyLocalReference(env, jStrBufferSize);
  1079. destroyLocalReference(env, jStrReplication);
  1080. destroyLocalReference(env, jConfiguration);
  1081. destroyLocalReference(env, jPath);
  1082. destroyLocalReference(env, jFile);
  1083. if (ret) {
  1084. if (file) {
  1085. if (file->file) {
  1086. (*env)->DeleteGlobalRef(env, file->file);
  1087. }
  1088. free(file);
  1089. }
  1090. errno = ret;
  1091. return NULL;
  1092. }
  1093. return file;
  1094. }
  1095. hdfsFile hdfsStreamBuilderBuild(struct hdfsStreamBuilder *bld)
  1096. {
  1097. hdfsFile file = hdfsOpenFileImpl(bld->fs, bld->path, bld->flags,
  1098. bld->bufferSize, bld->replication, bld->defaultBlockSize);
  1099. int prevErrno = errno;
  1100. hdfsStreamBuilderFree(bld);
  1101. errno = prevErrno;
  1102. return file;
  1103. }
  1104. int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength)
  1105. {
  1106. jobject jFS = (jobject)fs;
  1107. jthrowable jthr;
  1108. jvalue jVal;
  1109. jobject jPath = NULL;
  1110. JNIEnv *env = getJNIEnv();
  1111. if (!env) {
  1112. errno = EINTERNAL;
  1113. return -1;
  1114. }
  1115. /* Create an object of org.apache.hadoop.fs.Path */
  1116. jthr = constructNewObjectOfPath(env, path, &jPath);
  1117. if (jthr) {
  1118. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1119. "hdfsTruncateFile(%s): constructNewObjectOfPath", path);
  1120. return -1;
  1121. }
  1122. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1123. "truncate", JMETHOD2(JPARAM(HADOOP_PATH), "J", "Z"),
  1124. jPath, newlength);
  1125. destroyLocalReference(env, jPath);
  1126. if (jthr) {
  1127. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1128. "hdfsTruncateFile(%s): FileSystem#truncate", path);
  1129. return -1;
  1130. }
  1131. if (jVal.z == JNI_TRUE) {
  1132. return 1;
  1133. }
  1134. return 0;
  1135. }
  1136. int hdfsUnbufferFile(hdfsFile file)
  1137. {
  1138. int ret;
  1139. jthrowable jthr;
  1140. JNIEnv *env = getJNIEnv();
  1141. if (!env) {
  1142. ret = EINTERNAL;
  1143. goto done;
  1144. }
  1145. if (file->type != HDFS_STREAM_INPUT) {
  1146. ret = ENOTSUP;
  1147. goto done;
  1148. }
  1149. jthr = invokeMethod(env, NULL, INSTANCE, file->file, HADOOP_ISTRM,
  1150. "unbuffer", "()V");
  1151. if (jthr) {
  1152. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1153. HADOOP_ISTRM "#unbuffer failed:");
  1154. goto done;
  1155. }
  1156. ret = 0;
  1157. done:
  1158. errno = ret;
  1159. return ret;
  1160. }
  1161. int hdfsCloseFile(hdfsFS fs, hdfsFile file)
  1162. {
  1163. int ret;
  1164. // JAVA EQUIVALENT:
  1165. // file.close
  1166. //The interface whose 'close' method to be called
  1167. const char *interface;
  1168. const char *interfaceShortName;
  1169. //Caught exception
  1170. jthrowable jthr;
  1171. //Get the JNIEnv* corresponding to current thread
  1172. JNIEnv* env = getJNIEnv();
  1173. if (env == NULL) {
  1174. errno = EINTERNAL;
  1175. return -1;
  1176. }
  1177. //Sanity check
  1178. if (!file || file->type == HDFS_STREAM_UNINITIALIZED) {
  1179. errno = EBADF;
  1180. return -1;
  1181. }
  1182. interface = (file->type == HDFS_STREAM_INPUT) ?
  1183. HADOOP_ISTRM : HADOOP_OSTRM;
  1184. jthr = invokeMethod(env, NULL, INSTANCE, file->file, interface,
  1185. "close", "()V");
  1186. if (jthr) {
  1187. interfaceShortName = (file->type == HDFS_STREAM_INPUT) ?
  1188. "FSDataInputStream" : "FSDataOutputStream";
  1189. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1190. "%s#close", interfaceShortName);
  1191. } else {
  1192. ret = 0;
  1193. }
  1194. //De-allocate memory
  1195. (*env)->DeleteGlobalRef(env, file->file);
  1196. free(file);
  1197. if (ret) {
  1198. errno = ret;
  1199. return -1;
  1200. }
  1201. return 0;
  1202. }
  1203. int hdfsExists(hdfsFS fs, const char *path)
  1204. {
  1205. JNIEnv *env = getJNIEnv();
  1206. jobject jPath;
  1207. jvalue jVal;
  1208. jobject jFS = (jobject)fs;
  1209. jthrowable jthr;
  1210. if (env == NULL) {
  1211. errno = EINTERNAL;
  1212. return -1;
  1213. }
  1214. if (path == NULL) {
  1215. errno = EINVAL;
  1216. return -1;
  1217. }
  1218. jthr = constructNewObjectOfPath(env, path, &jPath);
  1219. if (jthr) {
  1220. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1221. "hdfsExists: constructNewObjectOfPath");
  1222. return -1;
  1223. }
  1224. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1225. "exists", JMETHOD1(JPARAM(HADOOP_PATH), "Z"), jPath);
  1226. destroyLocalReference(env, jPath);
  1227. if (jthr) {
  1228. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1229. "hdfsExists: invokeMethod(%s)",
  1230. JMETHOD1(JPARAM(HADOOP_PATH), "Z"));
  1231. return -1;
  1232. }
  1233. if (jVal.z) {
  1234. return 0;
  1235. } else {
  1236. errno = ENOENT;
  1237. return -1;
  1238. }
  1239. }
  1240. // Checks input file for readiness for reading.
  1241. static int readPrepare(JNIEnv* env, hdfsFS fs, hdfsFile f,
  1242. jobject* jInputStream)
  1243. {
  1244. *jInputStream = (jobject)(f ? f->file : NULL);
  1245. //Sanity check
  1246. if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
  1247. errno = EBADF;
  1248. return -1;
  1249. }
  1250. //Error checking... make sure that this file is 'readable'
  1251. if (f->type != HDFS_STREAM_INPUT) {
  1252. fprintf(stderr, "Cannot read from a non-InputStream object!\n");
  1253. errno = EINVAL;
  1254. return -1;
  1255. }
  1256. return 0;
  1257. }
  1258. tSize hdfsRead(hdfsFS fs, hdfsFile f, void* buffer, tSize length)
  1259. {
  1260. jobject jInputStream;
  1261. jbyteArray jbRarray;
  1262. jint noReadBytes = length;
  1263. jvalue jVal;
  1264. jthrowable jthr;
  1265. JNIEnv* env;
  1266. if (length == 0) {
  1267. return 0;
  1268. } else if (length < 0) {
  1269. errno = EINVAL;
  1270. return -1;
  1271. }
  1272. if (f->flags & HDFS_FILE_SUPPORTS_DIRECT_READ) {
  1273. return readDirect(fs, f, buffer, length);
  1274. }
  1275. // JAVA EQUIVALENT:
  1276. // byte [] bR = new byte[length];
  1277. // fis.read(bR);
  1278. //Get the JNIEnv* corresponding to current thread
  1279. env = getJNIEnv();
  1280. if (env == NULL) {
  1281. errno = EINTERNAL;
  1282. return -1;
  1283. }
  1284. //Parameters
  1285. if (readPrepare(env, fs, f, &jInputStream) == -1) {
  1286. return -1;
  1287. }
  1288. //Read the requisite bytes
  1289. jbRarray = (*env)->NewByteArray(env, length);
  1290. if (!jbRarray) {
  1291. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1292. "hdfsRead: NewByteArray");
  1293. return -1;
  1294. }
  1295. jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream, HADOOP_ISTRM,
  1296. "read", "([B)I", jbRarray);
  1297. if (jthr) {
  1298. destroyLocalReference(env, jbRarray);
  1299. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1300. "hdfsRead: FSDataInputStream#read");
  1301. return -1;
  1302. }
  1303. if (jVal.i < 0) {
  1304. // EOF
  1305. destroyLocalReference(env, jbRarray);
  1306. return 0;
  1307. } else if (jVal.i == 0) {
  1308. destroyLocalReference(env, jbRarray);
  1309. errno = EINTR;
  1310. return -1;
  1311. }
  1312. (*env)->GetByteArrayRegion(env, jbRarray, 0, noReadBytes, buffer);
  1313. destroyLocalReference(env, jbRarray);
  1314. if ((*env)->ExceptionCheck(env)) {
  1315. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1316. "hdfsRead: GetByteArrayRegion");
  1317. return -1;
  1318. }
  1319. return jVal.i;
  1320. }
  1321. // Reads using the read(ByteBuffer) API, which does fewer copies
  1322. tSize readDirect(hdfsFS fs, hdfsFile f, void* buffer, tSize length)
  1323. {
  1324. // JAVA EQUIVALENT:
  1325. // ByteBuffer bbuffer = ByteBuffer.allocateDirect(length) // wraps C buffer
  1326. // fis.read(bbuffer);
  1327. jobject jInputStream;
  1328. jvalue jVal;
  1329. jthrowable jthr;
  1330. jobject bb;
  1331. //Get the JNIEnv* corresponding to current thread
  1332. JNIEnv* env = getJNIEnv();
  1333. if (env == NULL) {
  1334. errno = EINTERNAL;
  1335. return -1;
  1336. }
  1337. if (readPrepare(env, fs, f, &jInputStream) == -1) {
  1338. return -1;
  1339. }
  1340. //Read the requisite bytes
  1341. bb = (*env)->NewDirectByteBuffer(env, buffer, length);
  1342. if (bb == NULL) {
  1343. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1344. "readDirect: NewDirectByteBuffer");
  1345. return -1;
  1346. }
  1347. jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream,
  1348. HADOOP_ISTRM, "read", "(Ljava/nio/ByteBuffer;)I", bb);
  1349. destroyLocalReference(env, bb);
  1350. if (jthr) {
  1351. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1352. "readDirect: FSDataInputStream#read");
  1353. return -1;
  1354. }
  1355. return (jVal.i < 0) ? 0 : jVal.i;
  1356. }
  1357. tSize hdfsPread(hdfsFS fs, hdfsFile f, tOffset position,
  1358. void* buffer, tSize length)
  1359. {
  1360. JNIEnv* env;
  1361. jbyteArray jbRarray;
  1362. jvalue jVal;
  1363. jthrowable jthr;
  1364. if (length == 0) {
  1365. return 0;
  1366. } else if (length < 0) {
  1367. errno = EINVAL;
  1368. return -1;
  1369. }
  1370. if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
  1371. errno = EBADF;
  1372. return -1;
  1373. }
  1374. env = getJNIEnv();
  1375. if (env == NULL) {
  1376. errno = EINTERNAL;
  1377. return -1;
  1378. }
  1379. //Error checking... make sure that this file is 'readable'
  1380. if (f->type != HDFS_STREAM_INPUT) {
  1381. fprintf(stderr, "Cannot read from a non-InputStream object!\n");
  1382. errno = EINVAL;
  1383. return -1;
  1384. }
  1385. // JAVA EQUIVALENT:
  1386. // byte [] bR = new byte[length];
  1387. // fis.read(pos, bR, 0, length);
  1388. jbRarray = (*env)->NewByteArray(env, length);
  1389. if (!jbRarray) {
  1390. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1391. "hdfsPread: NewByteArray");
  1392. return -1;
  1393. }
  1394. jthr = invokeMethod(env, &jVal, INSTANCE, f->file, HADOOP_ISTRM,
  1395. "read", "(J[BII)I", position, jbRarray, 0, length);
  1396. if (jthr) {
  1397. destroyLocalReference(env, jbRarray);
  1398. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1399. "hdfsPread: FSDataInputStream#read");
  1400. return -1;
  1401. }
  1402. if (jVal.i < 0) {
  1403. // EOF
  1404. destroyLocalReference(env, jbRarray);
  1405. return 0;
  1406. } else if (jVal.i == 0) {
  1407. destroyLocalReference(env, jbRarray);
  1408. errno = EINTR;
  1409. return -1;
  1410. }
  1411. (*env)->GetByteArrayRegion(env, jbRarray, 0, jVal.i, buffer);
  1412. destroyLocalReference(env, jbRarray);
  1413. if ((*env)->ExceptionCheck(env)) {
  1414. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1415. "hdfsPread: GetByteArrayRegion");
  1416. return -1;
  1417. }
  1418. return jVal.i;
  1419. }
  1420. tSize hdfsWrite(hdfsFS fs, hdfsFile f, const void* buffer, tSize length)
  1421. {
  1422. // JAVA EQUIVALENT
  1423. // byte b[] = str.getBytes();
  1424. // fso.write(b);
  1425. jobject jOutputStream;
  1426. jbyteArray jbWarray;
  1427. jthrowable jthr;
  1428. //Get the JNIEnv* corresponding to current thread
  1429. JNIEnv* env = getJNIEnv();
  1430. if (env == NULL) {
  1431. errno = EINTERNAL;
  1432. return -1;
  1433. }
  1434. //Sanity check
  1435. if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
  1436. errno = EBADF;
  1437. return -1;
  1438. }
  1439. jOutputStream = f->file;
  1440. if (length < 0) {
  1441. errno = EINVAL;
  1442. return -1;
  1443. }
  1444. //Error checking... make sure that this file is 'writable'
  1445. if (f->type != HDFS_STREAM_OUTPUT) {
  1446. fprintf(stderr, "Cannot write into a non-OutputStream object!\n");
  1447. errno = EINVAL;
  1448. return -1;
  1449. }
  1450. if (length < 0) {
  1451. errno = EINVAL;
  1452. return -1;
  1453. }
  1454. if (length == 0) {
  1455. return 0;
  1456. }
  1457. //Write the requisite bytes into the file
  1458. jbWarray = (*env)->NewByteArray(env, length);
  1459. if (!jbWarray) {
  1460. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1461. "hdfsWrite: NewByteArray");
  1462. return -1;
  1463. }
  1464. (*env)->SetByteArrayRegion(env, jbWarray, 0, length, buffer);
  1465. if ((*env)->ExceptionCheck(env)) {
  1466. destroyLocalReference(env, jbWarray);
  1467. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1468. "hdfsWrite(length = %d): SetByteArrayRegion", length);
  1469. return -1;
  1470. }
  1471. jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
  1472. HADOOP_OSTRM, "write", "([B)V", jbWarray);
  1473. destroyLocalReference(env, jbWarray);
  1474. if (jthr) {
  1475. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1476. "hdfsWrite: FSDataOutputStream#write");
  1477. return -1;
  1478. }
  1479. // Unlike most Java streams, FSDataOutputStream never does partial writes.
  1480. // If we succeeded, all the data was written.
  1481. return length;
  1482. }
  1483. int hdfsSeek(hdfsFS fs, hdfsFile f, tOffset desiredPos)
  1484. {
  1485. // JAVA EQUIVALENT
  1486. // fis.seek(pos);
  1487. jobject jInputStream;
  1488. jthrowable jthr;
  1489. //Get the JNIEnv* corresponding to current thread
  1490. JNIEnv* env = getJNIEnv();
  1491. if (env == NULL) {
  1492. errno = EINTERNAL;
  1493. return -1;
  1494. }
  1495. //Sanity check
  1496. if (!f || f->type != HDFS_STREAM_INPUT) {
  1497. errno = EBADF;
  1498. return -1;
  1499. }
  1500. jInputStream = f->file;
  1501. jthr = invokeMethod(env, NULL, INSTANCE, jInputStream,
  1502. HADOOP_ISTRM, "seek", "(J)V", desiredPos);
  1503. if (jthr) {
  1504. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1505. "hdfsSeek(desiredPos=%" PRId64 ")"
  1506. ": FSDataInputStream#seek", desiredPos);
  1507. return -1;
  1508. }
  1509. return 0;
  1510. }
  1511. tOffset hdfsTell(hdfsFS fs, hdfsFile f)
  1512. {
  1513. // JAVA EQUIVALENT
  1514. // pos = f.getPos();
  1515. jobject jStream;
  1516. const char *interface;
  1517. jvalue jVal;
  1518. jthrowable jthr;
  1519. //Get the JNIEnv* corresponding to current thread
  1520. JNIEnv* env = getJNIEnv();
  1521. if (env == NULL) {
  1522. errno = EINTERNAL;
  1523. return -1;
  1524. }
  1525. //Sanity check
  1526. if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
  1527. errno = EBADF;
  1528. return -1;
  1529. }
  1530. //Parameters
  1531. jStream = f->file;
  1532. interface = (f->type == HDFS_STREAM_INPUT) ?
  1533. HADOOP_ISTRM : HADOOP_OSTRM;
  1534. jthr = invokeMethod(env, &jVal, INSTANCE, jStream,
  1535. interface, "getPos", "()J");
  1536. if (jthr) {
  1537. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1538. "hdfsTell: %s#getPos",
  1539. ((f->type == HDFS_STREAM_INPUT) ? "FSDataInputStream" :
  1540. "FSDataOutputStream"));
  1541. return -1;
  1542. }
  1543. return jVal.j;
  1544. }
  1545. int hdfsFlush(hdfsFS fs, hdfsFile f)
  1546. {
  1547. // JAVA EQUIVALENT
  1548. // fos.flush();
  1549. jthrowable jthr;
  1550. //Get the JNIEnv* corresponding to current thread
  1551. JNIEnv* env = getJNIEnv();
  1552. if (env == NULL) {
  1553. errno = EINTERNAL;
  1554. return -1;
  1555. }
  1556. //Sanity check
  1557. if (!f || f->type != HDFS_STREAM_OUTPUT) {
  1558. errno = EBADF;
  1559. return -1;
  1560. }
  1561. jthr = invokeMethod(env, NULL, INSTANCE, f->file,
  1562. HADOOP_OSTRM, "flush", "()V");
  1563. if (jthr) {
  1564. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1565. "hdfsFlush: FSDataInputStream#flush");
  1566. return -1;
  1567. }
  1568. return 0;
  1569. }
  1570. int hdfsHFlush(hdfsFS fs, hdfsFile f)
  1571. {
  1572. jobject jOutputStream;
  1573. jthrowable jthr;
  1574. //Get the JNIEnv* corresponding to current thread
  1575. JNIEnv* env = getJNIEnv();
  1576. if (env == NULL) {
  1577. errno = EINTERNAL;
  1578. return -1;
  1579. }
  1580. //Sanity check
  1581. if (!f || f->type != HDFS_STREAM_OUTPUT) {
  1582. errno = EBADF;
  1583. return -1;
  1584. }
  1585. jOutputStream = f->file;
  1586. jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
  1587. HADOOP_OSTRM, "hflush", "()V");
  1588. if (jthr) {
  1589. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1590. "hdfsHFlush: FSDataOutputStream#hflush");
  1591. return -1;
  1592. }
  1593. return 0;
  1594. }
  1595. int hdfsHSync(hdfsFS fs, hdfsFile f)
  1596. {
  1597. jobject jOutputStream;
  1598. jthrowable jthr;
  1599. //Get the JNIEnv* corresponding to current thread
  1600. JNIEnv* env = getJNIEnv();
  1601. if (env == NULL) {
  1602. errno = EINTERNAL;
  1603. return -1;
  1604. }
  1605. //Sanity check
  1606. if (!f || f->type != HDFS_STREAM_OUTPUT) {
  1607. errno = EBADF;
  1608. return -1;
  1609. }
  1610. jOutputStream = f->file;
  1611. jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
  1612. HADOOP_OSTRM, "hsync", "()V");
  1613. if (jthr) {
  1614. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1615. "hdfsHSync: FSDataOutputStream#hsync");
  1616. return -1;
  1617. }
  1618. return 0;
  1619. }
  1620. int hdfsAvailable(hdfsFS fs, hdfsFile f)
  1621. {
  1622. // JAVA EQUIVALENT
  1623. // fis.available();
  1624. jobject jInputStream;
  1625. jvalue jVal;
  1626. jthrowable jthr;
  1627. //Get the JNIEnv* corresponding to current thread
  1628. JNIEnv* env = getJNIEnv();
  1629. if (env == NULL) {
  1630. errno = EINTERNAL;
  1631. return -1;
  1632. }
  1633. //Sanity check
  1634. if (!f || f->type != HDFS_STREAM_INPUT) {
  1635. errno = EBADF;
  1636. return -1;
  1637. }
  1638. //Parameters
  1639. jInputStream = f->file;
  1640. jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream,
  1641. HADOOP_ISTRM, "available", "()I");
  1642. if (jthr) {
  1643. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1644. "hdfsAvailable: FSDataInputStream#available");
  1645. return -1;
  1646. }
  1647. return jVal.i;
  1648. }
  1649. static int hdfsCopyImpl(hdfsFS srcFS, const char *src, hdfsFS dstFS,
  1650. const char *dst, jboolean deleteSource)
  1651. {
  1652. //JAVA EQUIVALENT
  1653. // FileUtil#copy(srcFS, srcPath, dstFS, dstPath,
  1654. // deleteSource = false, conf)
  1655. //Parameters
  1656. jobject jSrcFS = (jobject)srcFS;
  1657. jobject jDstFS = (jobject)dstFS;
  1658. jobject jConfiguration = NULL, jSrcPath = NULL, jDstPath = NULL;
  1659. jthrowable jthr;
  1660. jvalue jVal;
  1661. int ret;
  1662. //Get the JNIEnv* corresponding to current thread
  1663. JNIEnv* env = getJNIEnv();
  1664. if (env == NULL) {
  1665. errno = EINTERNAL;
  1666. return -1;
  1667. }
  1668. jthr = constructNewObjectOfPath(env, src, &jSrcPath);
  1669. if (jthr) {
  1670. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1671. "hdfsCopyImpl(src=%s): constructNewObjectOfPath", src);
  1672. goto done;
  1673. }
  1674. jthr = constructNewObjectOfPath(env, dst, &jDstPath);
  1675. if (jthr) {
  1676. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1677. "hdfsCopyImpl(dst=%s): constructNewObjectOfPath", dst);
  1678. goto done;
  1679. }
  1680. //Create the org.apache.hadoop.conf.Configuration object
  1681. jthr = constructNewObjectOfClass(env, &jConfiguration,
  1682. HADOOP_CONF, "()V");
  1683. if (jthr) {
  1684. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1685. "hdfsCopyImpl: Configuration constructor");
  1686. goto done;
  1687. }
  1688. //FileUtil#copy
  1689. jthr = invokeMethod(env, &jVal, STATIC,
  1690. NULL, "org/apache/hadoop/fs/FileUtil", "copy",
  1691. "(Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;"
  1692. "Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;"
  1693. "ZLorg/apache/hadoop/conf/Configuration;)Z",
  1694. jSrcFS, jSrcPath, jDstFS, jDstPath, deleteSource,
  1695. jConfiguration);
  1696. if (jthr) {
  1697. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1698. "hdfsCopyImpl(src=%s, dst=%s, deleteSource=%d): "
  1699. "FileUtil#copy", src, dst, deleteSource);
  1700. goto done;
  1701. }
  1702. if (!jVal.z) {
  1703. ret = EIO;
  1704. goto done;
  1705. }
  1706. ret = 0;
  1707. done:
  1708. destroyLocalReference(env, jConfiguration);
  1709. destroyLocalReference(env, jSrcPath);
  1710. destroyLocalReference(env, jDstPath);
  1711. if (ret) {
  1712. errno = ret;
  1713. return -1;
  1714. }
  1715. return 0;
  1716. }
  1717. int hdfsCopy(hdfsFS srcFS, const char *src, hdfsFS dstFS, const char *dst)
  1718. {
  1719. return hdfsCopyImpl(srcFS, src, dstFS, dst, 0);
  1720. }
  1721. int hdfsMove(hdfsFS srcFS, const char *src, hdfsFS dstFS, const char *dst)
  1722. {
  1723. return hdfsCopyImpl(srcFS, src, dstFS, dst, 1);
  1724. }
  1725. int hdfsDelete(hdfsFS fs, const char *path, int recursive)
  1726. {
  1727. // JAVA EQUIVALENT:
  1728. // Path p = new Path(path);
  1729. // bool retval = fs.delete(p, recursive);
  1730. jobject jFS = (jobject)fs;
  1731. jthrowable jthr;
  1732. jobject jPath;
  1733. jvalue jVal;
  1734. jboolean jRecursive;
  1735. //Get the JNIEnv* corresponding to current thread
  1736. JNIEnv* env = getJNIEnv();
  1737. if (env == NULL) {
  1738. errno = EINTERNAL;
  1739. return -1;
  1740. }
  1741. jthr = constructNewObjectOfPath(env, path, &jPath);
  1742. if (jthr) {
  1743. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1744. "hdfsDelete(path=%s): constructNewObjectOfPath", path);
  1745. return -1;
  1746. }
  1747. jRecursive = recursive ? JNI_TRUE : JNI_FALSE;
  1748. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1749. "delete", "(Lorg/apache/hadoop/fs/Path;Z)Z",
  1750. jPath, jRecursive);
  1751. destroyLocalReference(env, jPath);
  1752. if (jthr) {
  1753. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1754. "hdfsDelete(path=%s, recursive=%d): "
  1755. "FileSystem#delete", path, recursive);
  1756. return -1;
  1757. }
  1758. if (!jVal.z) {
  1759. errno = EIO;
  1760. return -1;
  1761. }
  1762. return 0;
  1763. }
  1764. int hdfsRename(hdfsFS fs, const char *oldPath, const char *newPath)
  1765. {
  1766. // JAVA EQUIVALENT:
  1767. // Path old = new Path(oldPath);
  1768. // Path new = new Path(newPath);
  1769. // fs.rename(old, new);
  1770. jobject jFS = (jobject)fs;
  1771. jthrowable jthr;
  1772. jobject jOldPath = NULL, jNewPath = NULL;
  1773. int ret = -1;
  1774. jvalue jVal;
  1775. //Get the JNIEnv* corresponding to current thread
  1776. JNIEnv* env = getJNIEnv();
  1777. if (env == NULL) {
  1778. errno = EINTERNAL;
  1779. return -1;
  1780. }
  1781. jthr = constructNewObjectOfPath(env, oldPath, &jOldPath );
  1782. if (jthr) {
  1783. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1784. "hdfsRename: constructNewObjectOfPath(%s)", oldPath);
  1785. goto done;
  1786. }
  1787. jthr = constructNewObjectOfPath(env, newPath, &jNewPath);
  1788. if (jthr) {
  1789. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1790. "hdfsRename: constructNewObjectOfPath(%s)", newPath);
  1791. goto done;
  1792. }
  1793. // Rename the file
  1794. // TODO: use rename2 here? (See HDFS-3592)
  1795. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS, "rename",
  1796. JMETHOD2(JPARAM(HADOOP_PATH), JPARAM(HADOOP_PATH), "Z"),
  1797. jOldPath, jNewPath);
  1798. if (jthr) {
  1799. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1800. "hdfsRename(oldPath=%s, newPath=%s): FileSystem#rename",
  1801. oldPath, newPath);
  1802. goto done;
  1803. }
  1804. if (!jVal.z) {
  1805. errno = EIO;
  1806. goto done;
  1807. }
  1808. ret = 0;
  1809. done:
  1810. destroyLocalReference(env, jOldPath);
  1811. destroyLocalReference(env, jNewPath);
  1812. return ret;
  1813. }
  1814. char* hdfsGetWorkingDirectory(hdfsFS fs, char* buffer, size_t bufferSize)
  1815. {
  1816. // JAVA EQUIVALENT:
  1817. // Path p = fs.getWorkingDirectory();
  1818. // return p.toString()
  1819. jobject jPath = NULL;
  1820. jstring jPathString = NULL;
  1821. jobject jFS = (jobject)fs;
  1822. jvalue jVal;
  1823. jthrowable jthr;
  1824. int ret;
  1825. const char *jPathChars = NULL;
  1826. //Get the JNIEnv* corresponding to current thread
  1827. JNIEnv* env = getJNIEnv();
  1828. if (env == NULL) {
  1829. errno = EINTERNAL;
  1830. return NULL;
  1831. }
  1832. //FileSystem#getWorkingDirectory()
  1833. jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
  1834. HADOOP_FS, "getWorkingDirectory",
  1835. "()Lorg/apache/hadoop/fs/Path;");
  1836. if (jthr) {
  1837. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1838. "hdfsGetWorkingDirectory: FileSystem#getWorkingDirectory");
  1839. goto done;
  1840. }
  1841. jPath = jVal.l;
  1842. if (!jPath) {
  1843. fprintf(stderr, "hdfsGetWorkingDirectory: "
  1844. "FileSystem#getWorkingDirectory returned NULL");
  1845. ret = -EIO;
  1846. goto done;
  1847. }
  1848. //Path#toString()
  1849. jthr = invokeMethod(env, &jVal, INSTANCE, jPath,
  1850. "org/apache/hadoop/fs/Path", "toString",
  1851. "()Ljava/lang/String;");
  1852. if (jthr) {
  1853. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1854. "hdfsGetWorkingDirectory: Path#toString");
  1855. goto done;
  1856. }
  1857. jPathString = jVal.l;
  1858. jPathChars = (*env)->GetStringUTFChars(env, jPathString, NULL);
  1859. if (!jPathChars) {
  1860. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1861. "hdfsGetWorkingDirectory: GetStringUTFChars");
  1862. goto done;
  1863. }
  1864. //Copy to user-provided buffer
  1865. ret = snprintf(buffer, bufferSize, "%s", jPathChars);
  1866. if (ret >= bufferSize) {
  1867. ret = ENAMETOOLONG;
  1868. goto done;
  1869. }
  1870. ret = 0;
  1871. done:
  1872. if (jPathChars) {
  1873. (*env)->ReleaseStringUTFChars(env, jPathString, jPathChars);
  1874. }
  1875. destroyLocalReference(env, jPath);
  1876. destroyLocalReference(env, jPathString);
  1877. if (ret) {
  1878. errno = ret;
  1879. return NULL;
  1880. }
  1881. return buffer;
  1882. }
  1883. int hdfsSetWorkingDirectory(hdfsFS fs, const char *path)
  1884. {
  1885. // JAVA EQUIVALENT:
  1886. // fs.setWorkingDirectory(Path(path));
  1887. jobject jFS = (jobject)fs;
  1888. jthrowable jthr;
  1889. jobject jPath;
  1890. //Get the JNIEnv* corresponding to current thread
  1891. JNIEnv* env = getJNIEnv();
  1892. if (env == NULL) {
  1893. errno = EINTERNAL;
  1894. return -1;
  1895. }
  1896. //Create an object of org.apache.hadoop.fs.Path
  1897. jthr = constructNewObjectOfPath(env, path, &jPath);
  1898. if (jthr) {
  1899. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1900. "hdfsSetWorkingDirectory(%s): constructNewObjectOfPath",
  1901. path);
  1902. return -1;
  1903. }
  1904. //FileSystem#setWorkingDirectory()
  1905. jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
  1906. "setWorkingDirectory",
  1907. "(Lorg/apache/hadoop/fs/Path;)V", jPath);
  1908. destroyLocalReference(env, jPath);
  1909. if (jthr) {
  1910. errno = printExceptionAndFree(env, jthr, NOPRINT_EXC_ILLEGAL_ARGUMENT,
  1911. "hdfsSetWorkingDirectory(%s): FileSystem#setWorkingDirectory",
  1912. path);
  1913. return -1;
  1914. }
  1915. return 0;
  1916. }
  1917. int hdfsCreateDirectory(hdfsFS fs, const char *path)
  1918. {
  1919. // JAVA EQUIVALENT:
  1920. // fs.mkdirs(new Path(path));
  1921. jobject jFS = (jobject)fs;
  1922. jobject jPath;
  1923. jthrowable jthr;
  1924. jvalue jVal;
  1925. //Get the JNIEnv* corresponding to current thread
  1926. JNIEnv* env = getJNIEnv();
  1927. if (env == NULL) {
  1928. errno = EINTERNAL;
  1929. return -1;
  1930. }
  1931. //Create an object of org.apache.hadoop.fs.Path
  1932. jthr = constructNewObjectOfPath(env, path, &jPath);
  1933. if (jthr) {
  1934. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1935. "hdfsCreateDirectory(%s): constructNewObjectOfPath", path);
  1936. return -1;
  1937. }
  1938. //Create the directory
  1939. jVal.z = 0;
  1940. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1941. "mkdirs", "(Lorg/apache/hadoop/fs/Path;)Z",
  1942. jPath);
  1943. destroyLocalReference(env, jPath);
  1944. if (jthr) {
  1945. errno = printExceptionAndFree(env, jthr,
  1946. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  1947. NOPRINT_EXC_UNRESOLVED_LINK | NOPRINT_EXC_PARENT_NOT_DIRECTORY,
  1948. "hdfsCreateDirectory(%s): FileSystem#mkdirs", path);
  1949. return -1;
  1950. }
  1951. if (!jVal.z) {
  1952. // It's unclear under exactly which conditions FileSystem#mkdirs
  1953. // is supposed to return false (as opposed to throwing an exception.)
  1954. // It seems like the current code never actually returns false.
  1955. // So we're going to translate this to EIO, since there seems to be
  1956. // nothing more specific we can do with it.
  1957. errno = EIO;
  1958. return -1;
  1959. }
  1960. return 0;
  1961. }
  1962. int hdfsSetReplication(hdfsFS fs, const char *path, int16_t replication)
  1963. {
  1964. // JAVA EQUIVALENT:
  1965. // fs.setReplication(new Path(path), replication);
  1966. jobject jFS = (jobject)fs;
  1967. jthrowable jthr;
  1968. jobject jPath;
  1969. jvalue jVal;
  1970. //Get the JNIEnv* corresponding to current thread
  1971. JNIEnv* env = getJNIEnv();
  1972. if (env == NULL) {
  1973. errno = EINTERNAL;
  1974. return -1;
  1975. }
  1976. //Create an object of org.apache.hadoop.fs.Path
  1977. jthr = constructNewObjectOfPath(env, path, &jPath);
  1978. if (jthr) {
  1979. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1980. "hdfsSetReplication(path=%s): constructNewObjectOfPath", path);
  1981. return -1;
  1982. }
  1983. //Create the directory
  1984. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1985. "setReplication", "(Lorg/apache/hadoop/fs/Path;S)Z",
  1986. jPath, replication);
  1987. destroyLocalReference(env, jPath);
  1988. if (jthr) {
  1989. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1990. "hdfsSetReplication(path=%s, replication=%d): "
  1991. "FileSystem#setReplication", path, replication);
  1992. return -1;
  1993. }
  1994. if (!jVal.z) {
  1995. // setReplication returns false "if file does not exist or is a
  1996. // directory." So the nearest translation to that is ENOENT.
  1997. errno = ENOENT;
  1998. return -1;
  1999. }
  2000. return 0;
  2001. }
  2002. int hdfsChown(hdfsFS fs, const char *path, const char *owner, const char *group)
  2003. {
  2004. // JAVA EQUIVALENT:
  2005. // fs.setOwner(path, owner, group)
  2006. jobject jFS = (jobject)fs;
  2007. jobject jPath = NULL;
  2008. jstring jOwner = NULL, jGroup = NULL;
  2009. jthrowable jthr;
  2010. int ret;
  2011. //Get the JNIEnv* corresponding to current thread
  2012. JNIEnv* env = getJNIEnv();
  2013. if (env == NULL) {
  2014. errno = EINTERNAL;
  2015. return -1;
  2016. }
  2017. if (owner == NULL && group == NULL) {
  2018. return 0;
  2019. }
  2020. jthr = constructNewObjectOfPath(env, path, &jPath);
  2021. if (jthr) {
  2022. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2023. "hdfsChown(path=%s): constructNewObjectOfPath", path);
  2024. goto done;
  2025. }
  2026. jthr = newJavaStr(env, owner, &jOwner);
  2027. if (jthr) {
  2028. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2029. "hdfsChown(path=%s): newJavaStr(%s)", path, owner);
  2030. goto done;
  2031. }
  2032. jthr = newJavaStr(env, group, &jGroup);
  2033. if (jthr) {
  2034. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2035. "hdfsChown(path=%s): newJavaStr(%s)", path, group);
  2036. goto done;
  2037. }
  2038. //Create the directory
  2039. jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
  2040. "setOwner", JMETHOD3(JPARAM(HADOOP_PATH),
  2041. JPARAM(JAVA_STRING), JPARAM(JAVA_STRING), JAVA_VOID),
  2042. jPath, jOwner, jGroup);
  2043. if (jthr) {
  2044. ret = printExceptionAndFree(env, jthr,
  2045. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  2046. NOPRINT_EXC_UNRESOLVED_LINK,
  2047. "hdfsChown(path=%s, owner=%s, group=%s): "
  2048. "FileSystem#setOwner", path, owner, group);
  2049. goto done;
  2050. }
  2051. ret = 0;
  2052. done:
  2053. destroyLocalReference(env, jPath);
  2054. destroyLocalReference(env, jOwner);
  2055. destroyLocalReference(env, jGroup);
  2056. if (ret) {
  2057. errno = ret;
  2058. return -1;
  2059. }
  2060. return 0;
  2061. }
  2062. int hdfsChmod(hdfsFS fs, const char *path, short mode)
  2063. {
  2064. int ret;
  2065. // JAVA EQUIVALENT:
  2066. // fs.setPermission(path, FsPermission)
  2067. jthrowable jthr;
  2068. jobject jPath = NULL, jPermObj = NULL;
  2069. jobject jFS = (jobject)fs;
  2070. jshort jmode = mode;
  2071. //Get the JNIEnv* corresponding to current thread
  2072. JNIEnv* env = getJNIEnv();
  2073. if (env == NULL) {
  2074. errno = EINTERNAL;
  2075. return -1;
  2076. }
  2077. // construct jPerm = FsPermission.createImmutable(short mode);
  2078. jthr = constructNewObjectOfClass(env, &jPermObj,
  2079. HADOOP_FSPERM,"(S)V",jmode);
  2080. if (jthr) {
  2081. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2082. "constructNewObjectOfClass(%s)", HADOOP_FSPERM);
  2083. return -1;
  2084. }
  2085. //Create an object of org.apache.hadoop.fs.Path
  2086. jthr = constructNewObjectOfPath(env, path, &jPath);
  2087. if (jthr) {
  2088. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2089. "hdfsChmod(%s): constructNewObjectOfPath", path);
  2090. goto done;
  2091. }
  2092. //Create the directory
  2093. jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
  2094. "setPermission",
  2095. JMETHOD2(JPARAM(HADOOP_PATH), JPARAM(HADOOP_FSPERM), JAVA_VOID),
  2096. jPath, jPermObj);
  2097. if (jthr) {
  2098. ret = printExceptionAndFree(env, jthr,
  2099. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  2100. NOPRINT_EXC_UNRESOLVED_LINK,
  2101. "hdfsChmod(%s): FileSystem#setPermission", path);
  2102. goto done;
  2103. }
  2104. ret = 0;
  2105. done:
  2106. destroyLocalReference(env, jPath);
  2107. destroyLocalReference(env, jPermObj);
  2108. if (ret) {
  2109. errno = ret;
  2110. return -1;
  2111. }
  2112. return 0;
  2113. }
  2114. int hdfsUtime(hdfsFS fs, const char *path, tTime mtime, tTime atime)
  2115. {
  2116. // JAVA EQUIVALENT:
  2117. // fs.setTimes(src, mtime, atime)
  2118. jthrowable jthr;
  2119. jobject jFS = (jobject)fs;
  2120. jobject jPath;
  2121. static const tTime NO_CHANGE = -1;
  2122. jlong jmtime, jatime;
  2123. //Get the JNIEnv* corresponding to current thread
  2124. JNIEnv* env = getJNIEnv();
  2125. if (env == NULL) {
  2126. errno = EINTERNAL;
  2127. return -1;
  2128. }
  2129. //Create an object of org.apache.hadoop.fs.Path
  2130. jthr = constructNewObjectOfPath(env, path, &jPath);
  2131. if (jthr) {
  2132. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2133. "hdfsUtime(path=%s): constructNewObjectOfPath", path);
  2134. return -1;
  2135. }
  2136. jmtime = (mtime == NO_CHANGE) ? -1 : (mtime * (jlong)1000);
  2137. jatime = (atime == NO_CHANGE) ? -1 : (atime * (jlong)1000);
  2138. jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
  2139. "setTimes", JMETHOD3(JPARAM(HADOOP_PATH), "J", "J", JAVA_VOID),
  2140. jPath, jmtime, jatime);
  2141. destroyLocalReference(env, jPath);
  2142. if (jthr) {
  2143. errno = printExceptionAndFree(env, jthr,
  2144. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  2145. NOPRINT_EXC_UNRESOLVED_LINK,
  2146. "hdfsUtime(path=%s): FileSystem#setTimes", path);
  2147. return -1;
  2148. }
  2149. return 0;
  2150. }
  2151. /**
  2152. * Zero-copy options.
  2153. *
  2154. * We cache the EnumSet of ReadOptions which has to be passed into every
  2155. * readZero call, to avoid reconstructing it each time. This cache is cleared
  2156. * whenever an element changes.
  2157. */
  2158. struct hadoopRzOptions
  2159. {
  2160. JNIEnv *env;
  2161. int skipChecksums;
  2162. jobject byteBufferPool;
  2163. jobject cachedEnumSet;
  2164. };
  2165. struct hadoopRzOptions *hadoopRzOptionsAlloc(void)
  2166. {
  2167. struct hadoopRzOptions *opts;
  2168. JNIEnv *env;
  2169. env = getJNIEnv();
  2170. if (!env) {
  2171. // Check to make sure the JNI environment is set up properly.
  2172. errno = EINTERNAL;
  2173. return NULL;
  2174. }
  2175. opts = calloc(1, sizeof(struct hadoopRzOptions));
  2176. if (!opts) {
  2177. errno = ENOMEM;
  2178. return NULL;
  2179. }
  2180. return opts;
  2181. }
  2182. static void hadoopRzOptionsClearCached(JNIEnv *env,
  2183. struct hadoopRzOptions *opts)
  2184. {
  2185. if (!opts->cachedEnumSet) {
  2186. return;
  2187. }
  2188. (*env)->DeleteGlobalRef(env, opts->cachedEnumSet);
  2189. opts->cachedEnumSet = NULL;
  2190. }
  2191. int hadoopRzOptionsSetSkipChecksum(
  2192. struct hadoopRzOptions *opts, int skip)
  2193. {
  2194. JNIEnv *env;
  2195. env = getJNIEnv();
  2196. if (!env) {
  2197. errno = EINTERNAL;
  2198. return -1;
  2199. }
  2200. hadoopRzOptionsClearCached(env, opts);
  2201. opts->skipChecksums = !!skip;
  2202. return 0;
  2203. }
  2204. int hadoopRzOptionsSetByteBufferPool(
  2205. struct hadoopRzOptions *opts, const char *className)
  2206. {
  2207. JNIEnv *env;
  2208. jthrowable jthr;
  2209. jobject byteBufferPool = NULL;
  2210. env = getJNIEnv();
  2211. if (!env) {
  2212. errno = EINTERNAL;
  2213. return -1;
  2214. }
  2215. if (className) {
  2216. // Note: we don't have to call hadoopRzOptionsClearCached in this
  2217. // function, since the ByteBufferPool is passed separately from the
  2218. // EnumSet of ReadOptions.
  2219. jthr = constructNewObjectOfClass(env, &byteBufferPool, className, "()V");
  2220. if (jthr) {
  2221. printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2222. "hadoopRzOptionsSetByteBufferPool(className=%s): ", className);
  2223. errno = EINVAL;
  2224. return -1;
  2225. }
  2226. }
  2227. if (opts->byteBufferPool) {
  2228. // Delete any previous ByteBufferPool we had.
  2229. (*env)->DeleteGlobalRef(env, opts->byteBufferPool);
  2230. }
  2231. opts->byteBufferPool = byteBufferPool;
  2232. return 0;
  2233. }
  2234. void hadoopRzOptionsFree(struct hadoopRzOptions *opts)
  2235. {
  2236. JNIEnv *env;
  2237. env = getJNIEnv();
  2238. if (!env) {
  2239. return;
  2240. }
  2241. hadoopRzOptionsClearCached(env, opts);
  2242. if (opts->byteBufferPool) {
  2243. (*env)->DeleteGlobalRef(env, opts->byteBufferPool);
  2244. opts->byteBufferPool = NULL;
  2245. }
  2246. free(opts);
  2247. }
  2248. struct hadoopRzBuffer
  2249. {
  2250. jobject byteBuffer;
  2251. uint8_t *ptr;
  2252. int32_t length;
  2253. int direct;
  2254. };
  2255. static jthrowable hadoopRzOptionsGetEnumSet(JNIEnv *env,
  2256. struct hadoopRzOptions *opts, jobject *enumSet)
  2257. {
  2258. jthrowable jthr = NULL;
  2259. jobject enumInst = NULL, enumSetObj = NULL;
  2260. jvalue jVal;
  2261. if (opts->cachedEnumSet) {
  2262. // If we cached the value, return it now.
  2263. *enumSet = opts->cachedEnumSet;
  2264. goto done;
  2265. }
  2266. if (opts->skipChecksums) {
  2267. jthr = fetchEnumInstance(env, READ_OPTION,
  2268. "SKIP_CHECKSUMS", &enumInst);
  2269. if (jthr) {
  2270. goto done;
  2271. }
  2272. jthr = invokeMethod(env, &jVal, STATIC, NULL,
  2273. "java/util/EnumSet", "of",
  2274. "(Ljava/lang/Enum;)Ljava/util/EnumSet;", enumInst);
  2275. if (jthr) {
  2276. goto done;
  2277. }
  2278. enumSetObj = jVal.l;
  2279. } else {
  2280. jclass clazz = (*env)->FindClass(env, READ_OPTION);
  2281. if (!clazz) {
  2282. jthr = newRuntimeError(env, "failed "
  2283. "to find class for %s", READ_OPTION);
  2284. goto done;
  2285. }
  2286. jthr = invokeMethod(env, &jVal, STATIC, NULL,
  2287. "java/util/EnumSet", "noneOf",
  2288. "(Ljava/lang/Class;)Ljava/util/EnumSet;", clazz);
  2289. enumSetObj = jVal.l;
  2290. }
  2291. // create global ref
  2292. opts->cachedEnumSet = (*env)->NewGlobalRef(env, enumSetObj);
  2293. if (!opts->cachedEnumSet) {
  2294. jthr = getPendingExceptionAndClear(env);
  2295. goto done;
  2296. }
  2297. *enumSet = opts->cachedEnumSet;
  2298. jthr = NULL;
  2299. done:
  2300. (*env)->DeleteLocalRef(env, enumInst);
  2301. (*env)->DeleteLocalRef(env, enumSetObj);
  2302. return jthr;
  2303. }
  2304. static int hadoopReadZeroExtractBuffer(JNIEnv *env,
  2305. const struct hadoopRzOptions *opts, struct hadoopRzBuffer *buffer)
  2306. {
  2307. int ret;
  2308. jthrowable jthr;
  2309. jvalue jVal;
  2310. uint8_t *directStart;
  2311. void *mallocBuf = NULL;
  2312. jint position;
  2313. jarray array = NULL;
  2314. jthr = invokeMethod(env, &jVal, INSTANCE, buffer->byteBuffer,
  2315. "java/nio/ByteBuffer", "remaining", "()I");
  2316. if (jthr) {
  2317. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2318. "hadoopReadZeroExtractBuffer: ByteBuffer#remaining failed: ");
  2319. goto done;
  2320. }
  2321. buffer->length = jVal.i;
  2322. jthr = invokeMethod(env, &jVal, INSTANCE, buffer->byteBuffer,
  2323. "java/nio/ByteBuffer", "position", "()I");
  2324. if (jthr) {
  2325. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2326. "hadoopReadZeroExtractBuffer: ByteBuffer#position failed: ");
  2327. goto done;
  2328. }
  2329. position = jVal.i;
  2330. directStart = (*env)->GetDirectBufferAddress(env, buffer->byteBuffer);
  2331. if (directStart) {
  2332. // Handle direct buffers.
  2333. buffer->ptr = directStart + position;
  2334. buffer->direct = 1;
  2335. ret = 0;
  2336. goto done;
  2337. }
  2338. // Handle indirect buffers.
  2339. // The JNI docs don't say that GetDirectBufferAddress throws any exceptions
  2340. // when it fails. However, they also don't clearly say that it doesn't. It
  2341. // seems safest to clear any pending exceptions here, to prevent problems on
  2342. // various JVMs.
  2343. (*env)->ExceptionClear(env);
  2344. if (!opts->byteBufferPool) {
  2345. fputs("hadoopReadZeroExtractBuffer: we read through the "
  2346. "zero-copy path, but failed to get the address of the buffer via "
  2347. "GetDirectBufferAddress. Please make sure your JVM supports "
  2348. "GetDirectBufferAddress.\n", stderr);
  2349. ret = ENOTSUP;
  2350. goto done;
  2351. }
  2352. // Get the backing array object of this buffer.
  2353. jthr = invokeMethod(env, &jVal, INSTANCE, buffer->byteBuffer,
  2354. "java/nio/ByteBuffer", "array", "()[B");
  2355. if (jthr) {
  2356. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2357. "hadoopReadZeroExtractBuffer: ByteBuffer#array failed: ");
  2358. goto done;
  2359. }
  2360. array = jVal.l;
  2361. if (!array) {
  2362. fputs("hadoopReadZeroExtractBuffer: ByteBuffer#array returned NULL.",
  2363. stderr);
  2364. ret = EIO;
  2365. goto done;
  2366. }
  2367. mallocBuf = malloc(buffer->length);
  2368. if (!mallocBuf) {
  2369. fprintf(stderr, "hadoopReadZeroExtractBuffer: failed to allocate %d bytes of memory\n",
  2370. buffer->length);
  2371. ret = ENOMEM;
  2372. goto done;
  2373. }
  2374. (*env)->GetByteArrayRegion(env, array, position, buffer->length, mallocBuf);
  2375. jthr = (*env)->ExceptionOccurred(env);
  2376. if (jthr) {
  2377. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2378. "hadoopReadZeroExtractBuffer: GetByteArrayRegion failed: ");
  2379. goto done;
  2380. }
  2381. buffer->ptr = mallocBuf;
  2382. buffer->direct = 0;
  2383. ret = 0;
  2384. done:
  2385. free(mallocBuf);
  2386. (*env)->DeleteLocalRef(env, array);
  2387. return ret;
  2388. }
  2389. static int translateZCRException(JNIEnv *env, jthrowable exc)
  2390. {
  2391. int ret;
  2392. char *className = NULL;
  2393. jthrowable jthr = classNameOfObject(exc, env, &className);
  2394. if (jthr) {
  2395. fputs("hadoopReadZero: failed to get class name of "
  2396. "exception from read().\n", stderr);
  2397. destroyLocalReference(env, exc);
  2398. destroyLocalReference(env, jthr);
  2399. ret = EIO;
  2400. goto done;
  2401. }
  2402. if (!strcmp(className, "java.lang.UnsupportedOperationException")) {
  2403. ret = EPROTONOSUPPORT;
  2404. goto done;
  2405. }
  2406. ret = printExceptionAndFree(env, exc, PRINT_EXC_ALL,
  2407. "hadoopZeroCopyRead: ZeroCopyCursor#read failed");
  2408. done:
  2409. free(className);
  2410. return ret;
  2411. }
  2412. struct hadoopRzBuffer* hadoopReadZero(hdfsFile file,
  2413. struct hadoopRzOptions *opts, int32_t maxLength)
  2414. {
  2415. JNIEnv *env;
  2416. jthrowable jthr = NULL;
  2417. jvalue jVal;
  2418. jobject enumSet = NULL, byteBuffer = NULL;
  2419. struct hadoopRzBuffer* buffer = NULL;
  2420. int ret;
  2421. env = getJNIEnv();
  2422. if (!env) {
  2423. errno = EINTERNAL;
  2424. return NULL;
  2425. }
  2426. if (file->type != HDFS_STREAM_INPUT) {
  2427. fputs("Cannot read from a non-InputStream object!\n", stderr);
  2428. ret = EINVAL;
  2429. goto done;
  2430. }
  2431. buffer = calloc(1, sizeof(struct hadoopRzBuffer));
  2432. if (!buffer) {
  2433. ret = ENOMEM;
  2434. goto done;
  2435. }
  2436. jthr = hadoopRzOptionsGetEnumSet(env, opts, &enumSet);
  2437. if (jthr) {
  2438. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2439. "hadoopReadZero: hadoopRzOptionsGetEnumSet failed: ");
  2440. goto done;
  2441. }
  2442. jthr = invokeMethod(env, &jVal, INSTANCE, file->file, HADOOP_ISTRM, "read",
  2443. "(Lorg/apache/hadoop/io/ByteBufferPool;ILjava/util/EnumSet;)"
  2444. "Ljava/nio/ByteBuffer;", opts->byteBufferPool, maxLength, enumSet);
  2445. if (jthr) {
  2446. ret = translateZCRException(env, jthr);
  2447. goto done;
  2448. }
  2449. byteBuffer = jVal.l;
  2450. if (!byteBuffer) {
  2451. buffer->byteBuffer = NULL;
  2452. buffer->length = 0;
  2453. buffer->ptr = NULL;
  2454. } else {
  2455. buffer->byteBuffer = (*env)->NewGlobalRef(env, byteBuffer);
  2456. if (!buffer->byteBuffer) {
  2457. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  2458. "hadoopReadZero: failed to create global ref to ByteBuffer");
  2459. goto done;
  2460. }
  2461. ret = hadoopReadZeroExtractBuffer(env, opts, buffer);
  2462. if (ret) {
  2463. goto done;
  2464. }
  2465. }
  2466. ret = 0;
  2467. done:
  2468. (*env)->DeleteLocalRef(env, byteBuffer);
  2469. if (ret) {
  2470. if (buffer) {
  2471. if (buffer->byteBuffer) {
  2472. (*env)->DeleteGlobalRef(env, buffer->byteBuffer);
  2473. }
  2474. free(buffer);
  2475. }
  2476. errno = ret;
  2477. return NULL;
  2478. } else {
  2479. errno = 0;
  2480. }
  2481. return buffer;
  2482. }
  2483. int32_t hadoopRzBufferLength(const struct hadoopRzBuffer *buffer)
  2484. {
  2485. return buffer->length;
  2486. }
  2487. const void *hadoopRzBufferGet(const struct hadoopRzBuffer *buffer)
  2488. {
  2489. return buffer->ptr;
  2490. }
  2491. void hadoopRzBufferFree(hdfsFile file, struct hadoopRzBuffer *buffer)
  2492. {
  2493. jvalue jVal;
  2494. jthrowable jthr;
  2495. JNIEnv* env;
  2496. env = getJNIEnv();
  2497. if (env == NULL) {
  2498. errno = EINTERNAL;
  2499. return;
  2500. }
  2501. if (buffer->byteBuffer) {
  2502. jthr = invokeMethod(env, &jVal, INSTANCE, file->file,
  2503. HADOOP_ISTRM, "releaseBuffer",
  2504. "(Ljava/nio/ByteBuffer;)V", buffer->byteBuffer);
  2505. if (jthr) {
  2506. printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2507. "hadoopRzBufferFree: releaseBuffer failed: ");
  2508. // even on error, we have to delete the reference.
  2509. }
  2510. (*env)->DeleteGlobalRef(env, buffer->byteBuffer);
  2511. }
  2512. if (!buffer->direct) {
  2513. free(buffer->ptr);
  2514. }
  2515. memset(buffer, 0, sizeof(*buffer));
  2516. free(buffer);
  2517. }
  2518. char***
  2519. hdfsGetHosts(hdfsFS fs, const char *path, tOffset start, tOffset length)
  2520. {
  2521. // JAVA EQUIVALENT:
  2522. // fs.getFileBlockLoctions(new Path(path), start, length);
  2523. jobject jFS = (jobject)fs;
  2524. jthrowable jthr;
  2525. jobject jPath = NULL;
  2526. jobject jFileStatus = NULL;
  2527. jvalue jFSVal, jVal;
  2528. jobjectArray jBlockLocations = NULL, jFileBlockHosts = NULL;
  2529. jstring jHost = NULL;
  2530. char*** blockHosts = NULL;
  2531. int i, j, ret;
  2532. jsize jNumFileBlocks = 0;
  2533. jobject jFileBlock;
  2534. jsize jNumBlockHosts;
  2535. const char *hostName;
  2536. //Get the JNIEnv* corresponding to current thread
  2537. JNIEnv* env = getJNIEnv();
  2538. if (env == NULL) {
  2539. errno = EINTERNAL;
  2540. return NULL;
  2541. }
  2542. //Create an object of org.apache.hadoop.fs.Path
  2543. jthr = constructNewObjectOfPath(env, path, &jPath);
  2544. if (jthr) {
  2545. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2546. "hdfsGetHosts(path=%s): constructNewObjectOfPath", path);
  2547. goto done;
  2548. }
  2549. jthr = invokeMethod(env, &jFSVal, INSTANCE, jFS,
  2550. HADOOP_FS, "getFileStatus", "(Lorg/apache/hadoop/fs/Path;)"
  2551. "Lorg/apache/hadoop/fs/FileStatus;", jPath);
  2552. if (jthr) {
  2553. ret = printExceptionAndFree(env, jthr, NOPRINT_EXC_FILE_NOT_FOUND,
  2554. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2555. "FileSystem#getFileStatus", path, start, length);
  2556. destroyLocalReference(env, jPath);
  2557. goto done;
  2558. }
  2559. jFileStatus = jFSVal.l;
  2560. //org.apache.hadoop.fs.FileSystem#getFileBlockLocations
  2561. jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
  2562. HADOOP_FS, "getFileBlockLocations",
  2563. "(Lorg/apache/hadoop/fs/FileStatus;JJ)"
  2564. "[Lorg/apache/hadoop/fs/BlockLocation;",
  2565. jFileStatus, start, length);
  2566. if (jthr) {
  2567. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2568. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2569. "FileSystem#getFileBlockLocations", path, start, length);
  2570. goto done;
  2571. }
  2572. jBlockLocations = jVal.l;
  2573. //Figure out no of entries in jBlockLocations
  2574. //Allocate memory and add NULL at the end
  2575. jNumFileBlocks = (*env)->GetArrayLength(env, jBlockLocations);
  2576. blockHosts = calloc(jNumFileBlocks + 1, sizeof(char**));
  2577. if (blockHosts == NULL) {
  2578. ret = ENOMEM;
  2579. goto done;
  2580. }
  2581. if (jNumFileBlocks == 0) {
  2582. ret = 0;
  2583. goto done;
  2584. }
  2585. //Now parse each block to get hostnames
  2586. for (i = 0; i < jNumFileBlocks; ++i) {
  2587. jFileBlock =
  2588. (*env)->GetObjectArrayElement(env, jBlockLocations, i);
  2589. if (!jFileBlock) {
  2590. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  2591. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2592. "GetObjectArrayElement(%d)", path, start, length, i);
  2593. goto done;
  2594. }
  2595. jthr = invokeMethod(env, &jVal, INSTANCE, jFileBlock, HADOOP_BLK_LOC,
  2596. "getHosts", "()[Ljava/lang/String;");
  2597. if (jthr) {
  2598. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2599. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2600. "BlockLocation#getHosts", path, start, length);
  2601. goto done;
  2602. }
  2603. jFileBlockHosts = jVal.l;
  2604. if (!jFileBlockHosts) {
  2605. fprintf(stderr,
  2606. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2607. "BlockLocation#getHosts returned NULL", path, start, length);
  2608. ret = EINTERNAL;
  2609. goto done;
  2610. }
  2611. //Figure out no of hosts in jFileBlockHosts, and allocate the memory
  2612. jNumBlockHosts = (*env)->GetArrayLength(env, jFileBlockHosts);
  2613. blockHosts[i] = calloc(jNumBlockHosts + 1, sizeof(char*));
  2614. if (!blockHosts[i]) {
  2615. ret = ENOMEM;
  2616. goto done;
  2617. }
  2618. //Now parse each hostname
  2619. for (j = 0; j < jNumBlockHosts; ++j) {
  2620. jHost = (*env)->GetObjectArrayElement(env, jFileBlockHosts, j);
  2621. if (!jHost) {
  2622. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  2623. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"): "
  2624. "NewByteArray", path, start, length);
  2625. goto done;
  2626. }
  2627. hostName =
  2628. (const char*)((*env)->GetStringUTFChars(env, jHost, NULL));
  2629. if (!hostName) {
  2630. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  2631. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64", "
  2632. "j=%d out of %d): GetStringUTFChars",
  2633. path, start, length, j, jNumBlockHosts);
  2634. goto done;
  2635. }
  2636. blockHosts[i][j] = strdup(hostName);
  2637. (*env)->ReleaseStringUTFChars(env, jHost, hostName);
  2638. if (!blockHosts[i][j]) {
  2639. ret = ENOMEM;
  2640. goto done;
  2641. }
  2642. destroyLocalReference(env, jHost);
  2643. jHost = NULL;
  2644. }
  2645. destroyLocalReference(env, jFileBlockHosts);
  2646. jFileBlockHosts = NULL;
  2647. }
  2648. ret = 0;
  2649. done:
  2650. destroyLocalReference(env, jPath);
  2651. destroyLocalReference(env, jFileStatus);
  2652. destroyLocalReference(env, jBlockLocations);
  2653. destroyLocalReference(env, jFileBlockHosts);
  2654. destroyLocalReference(env, jHost);
  2655. if (ret) {
  2656. errno = ret;
  2657. if (blockHosts) {
  2658. hdfsFreeHosts(blockHosts);
  2659. }
  2660. return NULL;
  2661. }
  2662. return blockHosts;
  2663. }
  2664. void hdfsFreeHosts(char ***blockHosts)
  2665. {
  2666. int i, j;
  2667. for (i=0; blockHosts[i]; i++) {
  2668. for (j=0; blockHosts[i][j]; j++) {
  2669. free(blockHosts[i][j]);
  2670. }
  2671. free(blockHosts[i]);
  2672. }
  2673. free(blockHosts);
  2674. }
  2675. tOffset hdfsGetDefaultBlockSize(hdfsFS fs)
  2676. {
  2677. // JAVA EQUIVALENT:
  2678. // fs.getDefaultBlockSize();
  2679. jobject jFS = (jobject)fs;
  2680. jvalue jVal;
  2681. jthrowable jthr;
  2682. //Get the JNIEnv* corresponding to current thread
  2683. JNIEnv* env = getJNIEnv();
  2684. if (env == NULL) {
  2685. errno = EINTERNAL;
  2686. return -1;
  2687. }
  2688. //FileSystem#getDefaultBlockSize()
  2689. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  2690. "getDefaultBlockSize", "()J");
  2691. if (jthr) {
  2692. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2693. "hdfsGetDefaultBlockSize: FileSystem#getDefaultBlockSize");
  2694. return -1;
  2695. }
  2696. return jVal.j;
  2697. }
  2698. tOffset hdfsGetDefaultBlockSizeAtPath(hdfsFS fs, const char *path)
  2699. {
  2700. // JAVA EQUIVALENT:
  2701. // fs.getDefaultBlockSize(path);
  2702. jthrowable jthr;
  2703. jobject jFS = (jobject)fs;
  2704. jobject jPath;
  2705. tOffset blockSize;
  2706. JNIEnv* env = getJNIEnv();
  2707. if (env == NULL) {
  2708. errno = EINTERNAL;
  2709. return -1;
  2710. }
  2711. jthr = constructNewObjectOfPath(env, path, &jPath);
  2712. if (jthr) {
  2713. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2714. "hdfsGetDefaultBlockSize(path=%s): constructNewObjectOfPath",
  2715. path);
  2716. return -1;
  2717. }
  2718. jthr = getDefaultBlockSize(env, jFS, jPath, &blockSize);
  2719. (*env)->DeleteLocalRef(env, jPath);
  2720. if (jthr) {
  2721. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2722. "hdfsGetDefaultBlockSize(path=%s): "
  2723. "FileSystem#getDefaultBlockSize", path);
  2724. return -1;
  2725. }
  2726. return blockSize;
  2727. }
  2728. tOffset hdfsGetCapacity(hdfsFS fs)
  2729. {
  2730. // JAVA EQUIVALENT:
  2731. // FsStatus fss = fs.getStatus();
  2732. // return Fss.getCapacity();
  2733. jobject jFS = (jobject)fs;
  2734. jvalue jVal;
  2735. jthrowable jthr;
  2736. jobject fss;
  2737. //Get the JNIEnv* corresponding to current thread
  2738. JNIEnv* env = getJNIEnv();
  2739. if (env == NULL) {
  2740. errno = EINTERNAL;
  2741. return -1;
  2742. }
  2743. //FileSystem#getStatus
  2744. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  2745. "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
  2746. if (jthr) {
  2747. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2748. "hdfsGetCapacity: FileSystem#getStatus");
  2749. return -1;
  2750. }
  2751. fss = (jobject)jVal.l;
  2752. jthr = invokeMethod(env, &jVal, INSTANCE, fss, HADOOP_FSSTATUS,
  2753. "getCapacity", "()J");
  2754. destroyLocalReference(env, fss);
  2755. if (jthr) {
  2756. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2757. "hdfsGetCapacity: FsStatus#getCapacity");
  2758. return -1;
  2759. }
  2760. return jVal.j;
  2761. }
  2762. tOffset hdfsGetUsed(hdfsFS fs)
  2763. {
  2764. // JAVA EQUIVALENT:
  2765. // FsStatus fss = fs.getStatus();
  2766. // return Fss.getUsed();
  2767. jobject jFS = (jobject)fs;
  2768. jvalue jVal;
  2769. jthrowable jthr;
  2770. jobject fss;
  2771. //Get the JNIEnv* corresponding to current thread
  2772. JNIEnv* env = getJNIEnv();
  2773. if (env == NULL) {
  2774. errno = EINTERNAL;
  2775. return -1;
  2776. }
  2777. //FileSystem#getStatus
  2778. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  2779. "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
  2780. if (jthr) {
  2781. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2782. "hdfsGetUsed: FileSystem#getStatus");
  2783. return -1;
  2784. }
  2785. fss = (jobject)jVal.l;
  2786. jthr = invokeMethod(env, &jVal, INSTANCE, fss, HADOOP_FSSTATUS,
  2787. "getUsed", "()J");
  2788. destroyLocalReference(env, fss);
  2789. if (jthr) {
  2790. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2791. "hdfsGetUsed: FsStatus#getUsed");
  2792. return -1;
  2793. }
  2794. return jVal.j;
  2795. }
  2796. /**
  2797. * We cannot add new fields to the hdfsFileInfo structure because it would break
  2798. * binary compatibility. The reason is because we return an array
  2799. * of hdfsFileInfo structures from hdfsListDirectory. So changing the size of
  2800. * those structures would break all programs that relied on finding the second
  2801. * element in the array at <base_offset> + sizeof(struct hdfsFileInfo).
  2802. *
  2803. * So instead, we add the new fields to the hdfsExtendedFileInfo structure.
  2804. * This structure is contained in the mOwner string found inside the
  2805. * hdfsFileInfo. Specifically, the format of mOwner is:
  2806. *
  2807. * [owner-string] [null byte] [padding] [hdfsExtendedFileInfo structure]
  2808. *
  2809. * The padding is added so that the hdfsExtendedFileInfo structure starts on an
  2810. * 8-byte boundary.
  2811. *
  2812. * @param str The string to locate the extended info in.
  2813. * @return The offset of the hdfsExtendedFileInfo structure.
  2814. */
  2815. static size_t getExtendedFileInfoOffset(const char *str)
  2816. {
  2817. int num_64_bit_words = ((strlen(str) + 1) + 7) / 8;
  2818. return num_64_bit_words * 8;
  2819. }
  2820. static struct hdfsExtendedFileInfo *getExtendedFileInfo(hdfsFileInfo *fileInfo)
  2821. {
  2822. char *owner = fileInfo->mOwner;
  2823. return (struct hdfsExtendedFileInfo *)(owner +
  2824. getExtendedFileInfoOffset(owner));
  2825. }
  2826. static jthrowable
  2827. getFileInfoFromStat(JNIEnv *env, jobject jStat, hdfsFileInfo *fileInfo)
  2828. {
  2829. jvalue jVal;
  2830. jthrowable jthr;
  2831. jobject jPath = NULL;
  2832. jstring jPathName = NULL;
  2833. jstring jUserName = NULL;
  2834. jstring jGroupName = NULL;
  2835. jobject jPermission = NULL;
  2836. const char *cPathName;
  2837. const char *cUserName;
  2838. const char *cGroupName;
  2839. struct hdfsExtendedFileInfo *extInfo;
  2840. size_t extOffset;
  2841. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2842. HADOOP_STAT, "isDir", "()Z");
  2843. if (jthr)
  2844. goto done;
  2845. fileInfo->mKind = jVal.z ? kObjectKindDirectory : kObjectKindFile;
  2846. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2847. HADOOP_STAT, "getReplication", "()S");
  2848. if (jthr)
  2849. goto done;
  2850. fileInfo->mReplication = jVal.s;
  2851. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2852. HADOOP_STAT, "getBlockSize", "()J");
  2853. if (jthr)
  2854. goto done;
  2855. fileInfo->mBlockSize = jVal.j;
  2856. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2857. HADOOP_STAT, "getModificationTime", "()J");
  2858. if (jthr)
  2859. goto done;
  2860. fileInfo->mLastMod = jVal.j / 1000;
  2861. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2862. HADOOP_STAT, "getAccessTime", "()J");
  2863. if (jthr)
  2864. goto done;
  2865. fileInfo->mLastAccess = (tTime) (jVal.j / 1000);
  2866. if (fileInfo->mKind == kObjectKindFile) {
  2867. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2868. HADOOP_STAT, "getLen", "()J");
  2869. if (jthr)
  2870. goto done;
  2871. fileInfo->mSize = jVal.j;
  2872. }
  2873. jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT,
  2874. "getPath", "()Lorg/apache/hadoop/fs/Path;");
  2875. if (jthr)
  2876. goto done;
  2877. jPath = jVal.l;
  2878. if (jPath == NULL) {
  2879. jthr = newRuntimeError(env, "org.apache.hadoop.fs.FileStatus#"
  2880. "getPath returned NULL!");
  2881. goto done;
  2882. }
  2883. jthr = invokeMethod(env, &jVal, INSTANCE, jPath, HADOOP_PATH,
  2884. "toString", "()Ljava/lang/String;");
  2885. if (jthr)
  2886. goto done;
  2887. jPathName = jVal.l;
  2888. cPathName =
  2889. (const char*) ((*env)->GetStringUTFChars(env, jPathName, NULL));
  2890. if (!cPathName) {
  2891. jthr = getPendingExceptionAndClear(env);
  2892. goto done;
  2893. }
  2894. fileInfo->mName = strdup(cPathName);
  2895. (*env)->ReleaseStringUTFChars(env, jPathName, cPathName);
  2896. jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT,
  2897. "getOwner", "()Ljava/lang/String;");
  2898. if (jthr)
  2899. goto done;
  2900. jUserName = jVal.l;
  2901. cUserName =
  2902. (const char*) ((*env)->GetStringUTFChars(env, jUserName, NULL));
  2903. if (!cUserName) {
  2904. jthr = getPendingExceptionAndClear(env);
  2905. goto done;
  2906. }
  2907. extOffset = getExtendedFileInfoOffset(cUserName);
  2908. fileInfo->mOwner = malloc(extOffset + sizeof(struct hdfsExtendedFileInfo));
  2909. if (!fileInfo->mOwner) {
  2910. jthr = newRuntimeError(env, "getFileInfo: OOM allocating mOwner");
  2911. goto done;
  2912. }
  2913. strcpy(fileInfo->mOwner, cUserName);
  2914. (*env)->ReleaseStringUTFChars(env, jUserName, cUserName);
  2915. extInfo = getExtendedFileInfo(fileInfo);
  2916. memset(extInfo, 0, sizeof(*extInfo));
  2917. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2918. HADOOP_STAT, "isEncrypted", "()Z");
  2919. if (jthr) {
  2920. goto done;
  2921. }
  2922. if (jVal.z == JNI_TRUE) {
  2923. extInfo->flags |= HDFS_EXTENDED_FILE_INFO_ENCRYPTED;
  2924. }
  2925. jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT,
  2926. "getGroup", "()Ljava/lang/String;");
  2927. if (jthr)
  2928. goto done;
  2929. jGroupName = jVal.l;
  2930. cGroupName = (const char*) ((*env)->GetStringUTFChars(env, jGroupName, NULL));
  2931. if (!cGroupName) {
  2932. jthr = getPendingExceptionAndClear(env);
  2933. goto done;
  2934. }
  2935. fileInfo->mGroup = strdup(cGroupName);
  2936. (*env)->ReleaseStringUTFChars(env, jGroupName, cGroupName);
  2937. jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT,
  2938. "getPermission",
  2939. "()Lorg/apache/hadoop/fs/permission/FsPermission;");
  2940. if (jthr)
  2941. goto done;
  2942. if (jVal.l == NULL) {
  2943. jthr = newRuntimeError(env, "%s#getPermission returned NULL!",
  2944. HADOOP_STAT);
  2945. goto done;
  2946. }
  2947. jPermission = jVal.l;
  2948. jthr = invokeMethod(env, &jVal, INSTANCE, jPermission, HADOOP_FSPERM,
  2949. "toShort", "()S");
  2950. if (jthr)
  2951. goto done;
  2952. fileInfo->mPermissions = jVal.s;
  2953. jthr = NULL;
  2954. done:
  2955. if (jthr)
  2956. hdfsFreeFileInfoEntry(fileInfo);
  2957. destroyLocalReference(env, jPath);
  2958. destroyLocalReference(env, jPathName);
  2959. destroyLocalReference(env, jUserName);
  2960. destroyLocalReference(env, jGroupName);
  2961. destroyLocalReference(env, jPermission);
  2962. destroyLocalReference(env, jPath);
  2963. return jthr;
  2964. }
  2965. static jthrowable
  2966. getFileInfo(JNIEnv *env, jobject jFS, jobject jPath, hdfsFileInfo **fileInfo)
  2967. {
  2968. // JAVA EQUIVALENT:
  2969. // fs.isDirectory(f)
  2970. // fs.getModificationTime()
  2971. // fs.getAccessTime()
  2972. // fs.getLength(f)
  2973. // f.getPath()
  2974. // f.getOwner()
  2975. // f.getGroup()
  2976. // f.getPermission().toShort()
  2977. jobject jStat;
  2978. jvalue jVal;
  2979. jthrowable jthr;
  2980. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  2981. "exists", JMETHOD1(JPARAM(HADOOP_PATH), "Z"),
  2982. jPath);
  2983. if (jthr)
  2984. return jthr;
  2985. if (jVal.z == 0) {
  2986. *fileInfo = NULL;
  2987. return NULL;
  2988. }
  2989. jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
  2990. HADOOP_FS, "getFileStatus",
  2991. JMETHOD1(JPARAM(HADOOP_PATH), JPARAM(HADOOP_STAT)), jPath);
  2992. if (jthr)
  2993. return jthr;
  2994. jStat = jVal.l;
  2995. *fileInfo = calloc(1, sizeof(hdfsFileInfo));
  2996. if (!*fileInfo) {
  2997. destroyLocalReference(env, jStat);
  2998. return newRuntimeError(env, "getFileInfo: OOM allocating hdfsFileInfo");
  2999. }
  3000. jthr = getFileInfoFromStat(env, jStat, *fileInfo);
  3001. destroyLocalReference(env, jStat);
  3002. return jthr;
  3003. }
  3004. hdfsFileInfo* hdfsListDirectory(hdfsFS fs, const char *path, int *numEntries)
  3005. {
  3006. // JAVA EQUIVALENT:
  3007. // Path p(path);
  3008. // Path []pathList = fs.listPaths(p)
  3009. // foreach path in pathList
  3010. // getFileInfo(path)
  3011. jobject jFS = (jobject)fs;
  3012. jthrowable jthr;
  3013. jobject jPath = NULL;
  3014. hdfsFileInfo *pathList = NULL;
  3015. jobjectArray jPathList = NULL;
  3016. jvalue jVal;
  3017. jsize jPathListSize = 0;
  3018. int ret;
  3019. jsize i;
  3020. jobject tmpStat;
  3021. //Get the JNIEnv* corresponding to current thread
  3022. JNIEnv* env = getJNIEnv();
  3023. if (env == NULL) {
  3024. errno = EINTERNAL;
  3025. return NULL;
  3026. }
  3027. //Create an object of org.apache.hadoop.fs.Path
  3028. jthr = constructNewObjectOfPath(env, path, &jPath);
  3029. if (jthr) {
  3030. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  3031. "hdfsListDirectory(%s): constructNewObjectOfPath", path);
  3032. goto done;
  3033. }
  3034. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_DFS, "listStatus",
  3035. JMETHOD1(JPARAM(HADOOP_PATH), JARRPARAM(HADOOP_STAT)),
  3036. jPath);
  3037. if (jthr) {
  3038. ret = printExceptionAndFree(env, jthr,
  3039. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  3040. NOPRINT_EXC_UNRESOLVED_LINK,
  3041. "hdfsListDirectory(%s): FileSystem#listStatus", path);
  3042. goto done;
  3043. }
  3044. jPathList = jVal.l;
  3045. //Figure out the number of entries in that directory
  3046. jPathListSize = (*env)->GetArrayLength(env, jPathList);
  3047. if (jPathListSize == 0) {
  3048. ret = 0;
  3049. goto done;
  3050. }
  3051. //Allocate memory
  3052. pathList = calloc(jPathListSize, sizeof(hdfsFileInfo));
  3053. if (pathList == NULL) {
  3054. ret = ENOMEM;
  3055. goto done;
  3056. }
  3057. //Save path information in pathList
  3058. for (i=0; i < jPathListSize; ++i) {
  3059. tmpStat = (*env)->GetObjectArrayElement(env, jPathList, i);
  3060. if (!tmpStat) {
  3061. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  3062. "hdfsListDirectory(%s): GetObjectArrayElement(%d out of %d)",
  3063. path, i, jPathListSize);
  3064. goto done;
  3065. }
  3066. jthr = getFileInfoFromStat(env, tmpStat, &pathList[i]);
  3067. destroyLocalReference(env, tmpStat);
  3068. if (jthr) {
  3069. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  3070. "hdfsListDirectory(%s): getFileInfoFromStat(%d out of %d)",
  3071. path, i, jPathListSize);
  3072. goto done;
  3073. }
  3074. }
  3075. ret = 0;
  3076. done:
  3077. destroyLocalReference(env, jPath);
  3078. destroyLocalReference(env, jPathList);
  3079. if (ret) {
  3080. hdfsFreeFileInfo(pathList, jPathListSize);
  3081. errno = ret;
  3082. return NULL;
  3083. }
  3084. *numEntries = jPathListSize;
  3085. errno = 0;
  3086. return pathList;
  3087. }
  3088. hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char *path)
  3089. {
  3090. // JAVA EQUIVALENT:
  3091. // File f(path);
  3092. // fs.isDirectory(f)
  3093. // fs.lastModified() ??
  3094. // fs.getLength(f)
  3095. // f.getPath()
  3096. jobject jFS = (jobject)fs;
  3097. jobject jPath;
  3098. jthrowable jthr;
  3099. hdfsFileInfo *fileInfo;
  3100. //Get the JNIEnv* corresponding to current thread
  3101. JNIEnv* env = getJNIEnv();
  3102. if (env == NULL) {
  3103. errno = EINTERNAL;
  3104. return NULL;
  3105. }
  3106. //Create an object of org.apache.hadoop.fs.Path
  3107. jthr = constructNewObjectOfPath(env, path, &jPath);
  3108. if (jthr) {
  3109. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  3110. "hdfsGetPathInfo(%s): constructNewObjectOfPath", path);
  3111. return NULL;
  3112. }
  3113. jthr = getFileInfo(env, jFS, jPath, &fileInfo);
  3114. destroyLocalReference(env, jPath);
  3115. if (jthr) {
  3116. errno = printExceptionAndFree(env, jthr,
  3117. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  3118. NOPRINT_EXC_UNRESOLVED_LINK,
  3119. "hdfsGetPathInfo(%s): getFileInfo", path);
  3120. return NULL;
  3121. }
  3122. if (!fileInfo) {
  3123. errno = ENOENT;
  3124. return NULL;
  3125. }
  3126. return fileInfo;
  3127. }
  3128. static void hdfsFreeFileInfoEntry(hdfsFileInfo *hdfsFileInfo)
  3129. {
  3130. free(hdfsFileInfo->mName);
  3131. free(hdfsFileInfo->mOwner);
  3132. free(hdfsFileInfo->mGroup);
  3133. memset(hdfsFileInfo, 0, sizeof(*hdfsFileInfo));
  3134. }
  3135. void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries)
  3136. {
  3137. //Free the mName, mOwner, and mGroup
  3138. int i;
  3139. for (i=0; i < numEntries; ++i) {
  3140. hdfsFreeFileInfoEntry(hdfsFileInfo + i);
  3141. }
  3142. //Free entire block
  3143. free(hdfsFileInfo);
  3144. }
  3145. int hdfsFileIsEncrypted(hdfsFileInfo *fileInfo)
  3146. {
  3147. struct hdfsExtendedFileInfo *extInfo;
  3148. extInfo = getExtendedFileInfo(fileInfo);
  3149. return !!(extInfo->flags & HDFS_EXTENDED_FILE_INFO_ENCRYPTED);
  3150. }
  3151. char* hdfsGetLastExceptionRootCause()
  3152. {
  3153. return getLastTLSExceptionRootCause();
  3154. }
  3155. char* hdfsGetLastExceptionStackTrace()
  3156. {
  3157. return getLastTLSExceptionStackTrace();
  3158. }
  3159. /**
  3160. * vim: ts=4: sw=4: et:
  3161. */