hdfs.c 100 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354135513561357135813591360136113621363136413651366136713681369137013711372137313741375137613771378137913801381138213831384138513861387138813891390139113921393139413951396139713981399140014011402140314041405140614071408140914101411141214131414141514161417141814191420142114221423142414251426142714281429143014311432143314341435143614371438143914401441144214431444144514461447144814491450145114521453145414551456145714581459146014611462146314641465146614671468146914701471147214731474147514761477147814791480148114821483148414851486148714881489149014911492149314941495149614971498149915001501150215031504150515061507150815091510151115121513151415151516151715181519152015211522152315241525152615271528152915301531153215331534153515361537153815391540154115421543154415451546154715481549155015511552155315541555155615571558155915601561156215631564156515661567156815691570157115721573157415751576157715781579158015811582158315841585158615871588158915901591159215931594159515961597159815991600160116021603160416051606160716081609161016111612161316141615161616171618161916201621162216231624162516261627162816291630163116321633163416351636163716381639164016411642164316441645164616471648164916501651165216531654165516561657165816591660166116621663166416651666166716681669167016711672167316741675167616771678167916801681168216831684168516861687168816891690169116921693169416951696169716981699170017011702170317041705170617071708170917101711171217131714171517161717171817191720172117221723172417251726172717281729173017311732173317341735173617371738173917401741174217431744174517461747174817491750175117521753175417551756175717581759176017611762176317641765176617671768176917701771177217731774177517761777177817791780178117821783178417851786178717881789179017911792179317941795179617971798179918001801180218031804180518061807180818091810181118121813181418151816181718181819182018211822182318241825182618271828182918301831183218331834183518361837183818391840184118421843184418451846184718481849185018511852185318541855185618571858185918601861186218631864186518661867186818691870187118721873187418751876187718781879188018811882188318841885188618871888188918901891189218931894189518961897189818991900190119021903190419051906190719081909191019111912191319141915191619171918191919201921192219231924192519261927192819291930193119321933193419351936193719381939194019411942194319441945194619471948194919501951195219531954195519561957195819591960196119621963196419651966196719681969197019711972197319741975197619771978197919801981198219831984198519861987198819891990199119921993199419951996199719981999200020012002200320042005200620072008200920102011201220132014201520162017201820192020202120222023202420252026202720282029203020312032203320342035203620372038203920402041204220432044204520462047204820492050205120522053205420552056205720582059206020612062206320642065206620672068206920702071207220732074207520762077207820792080208120822083208420852086208720882089209020912092209320942095209620972098209921002101210221032104210521062107210821092110211121122113211421152116211721182119212021212122212321242125212621272128212921302131213221332134213521362137213821392140214121422143214421452146214721482149215021512152215321542155215621572158215921602161216221632164216521662167216821692170217121722173217421752176217721782179218021812182218321842185218621872188218921902191219221932194219521962197219821992200220122022203220422052206220722082209221022112212221322142215221622172218221922202221222222232224222522262227222822292230223122322233223422352236223722382239224022412242224322442245224622472248224922502251225222532254225522562257225822592260226122622263226422652266226722682269227022712272227322742275227622772278227922802281228222832284228522862287228822892290229122922293229422952296229722982299230023012302230323042305230623072308230923102311231223132314231523162317231823192320232123222323232423252326232723282329233023312332233323342335233623372338233923402341234223432344234523462347234823492350235123522353235423552356235723582359236023612362236323642365236623672368236923702371237223732374237523762377237823792380238123822383238423852386238723882389239023912392239323942395239623972398239924002401240224032404240524062407240824092410241124122413241424152416241724182419242024212422242324242425242624272428242924302431243224332434243524362437243824392440244124422443244424452446244724482449245024512452245324542455245624572458245924602461246224632464246524662467246824692470247124722473247424752476247724782479248024812482248324842485248624872488248924902491249224932494249524962497249824992500250125022503250425052506250725082509251025112512251325142515251625172518251925202521252225232524252525262527252825292530253125322533253425352536253725382539254025412542254325442545254625472548254925502551255225532554255525562557255825592560256125622563256425652566256725682569257025712572257325742575257625772578257925802581258225832584258525862587258825892590259125922593259425952596259725982599260026012602260326042605260626072608260926102611261226132614261526162617261826192620262126222623262426252626262726282629263026312632263326342635263626372638263926402641264226432644264526462647264826492650265126522653265426552656265726582659266026612662266326642665266626672668266926702671267226732674267526762677267826792680268126822683268426852686268726882689269026912692269326942695269626972698269927002701270227032704270527062707270827092710271127122713271427152716271727182719272027212722272327242725272627272728272927302731273227332734273527362737273827392740274127422743274427452746274727482749275027512752275327542755275627572758275927602761276227632764276527662767276827692770277127722773277427752776277727782779278027812782278327842785278627872788278927902791279227932794279527962797279827992800280128022803280428052806280728082809281028112812281328142815281628172818281928202821282228232824282528262827282828292830283128322833283428352836283728382839284028412842284328442845284628472848284928502851285228532854285528562857285828592860286128622863286428652866286728682869287028712872287328742875287628772878287928802881288228832884288528862887288828892890289128922893289428952896289728982899290029012902290329042905290629072908290929102911291229132914291529162917291829192920292129222923292429252926292729282929293029312932293329342935293629372938293929402941294229432944294529462947294829492950295129522953295429552956295729582959296029612962296329642965296629672968296929702971297229732974297529762977297829792980298129822983298429852986298729882989299029912992299329942995299629972998299930003001300230033004300530063007300830093010301130123013301430153016301730183019302030213022302330243025302630273028302930303031303230333034303530363037303830393040304130423043304430453046304730483049305030513052305330543055305630573058305930603061306230633064306530663067306830693070307130723073307430753076307730783079308030813082308330843085308630873088308930903091309230933094309530963097309830993100310131023103310431053106310731083109311031113112311331143115311631173118311931203121312231233124312531263127312831293130313131323133313431353136313731383139314031413142314331443145314631473148314931503151315231533154315531563157315831593160316131623163316431653166316731683169317031713172317331743175317631773178317931803181318231833184318531863187318831893190319131923193319431953196319731983199320032013202320332043205320632073208320932103211321232133214321532163217321832193220322132223223322432253226322732283229323032313232323332343235323632373238323932403241324232433244324532463247324832493250325132523253325432553256325732583259326032613262326332643265326632673268326932703271327232733274327532763277327832793280328132823283328432853286328732883289329032913292329332943295329632973298329933003301330233033304330533063307330833093310331133123313331433153316331733183319332033213322332333243325332633273328332933303331333233333334333533363337333833393340334133423343334433453346334733483349335033513352335333543355335633573358335933603361336233633364336533663367336833693370337133723373337433753376337733783379338033813382338333843385338633873388338933903391339233933394339533963397339833993400340134023403340434053406340734083409341034113412341334143415341634173418341934203421342234233424342534263427342834293430343134323433343434353436343734383439344034413442344334443445344634473448344934503451345234533454345534563457345834593460346134623463346434653466346734683469347034713472347334743475347634773478347934803481348234833484348534863487348834893490349134923493349434953496349734983499350035013502350335043505350635073508350935103511351235133514351535163517351835193520352135223523352435253526352735283529353035313532
  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. jvalue jVal;
  1263. jthrowable jthr;
  1264. JNIEnv* env;
  1265. if (length == 0) {
  1266. return 0;
  1267. } else if (length < 0) {
  1268. errno = EINVAL;
  1269. return -1;
  1270. }
  1271. if (f->flags & HDFS_FILE_SUPPORTS_DIRECT_READ) {
  1272. return readDirect(fs, f, buffer, length);
  1273. }
  1274. // JAVA EQUIVALENT:
  1275. // byte [] bR = new byte[length];
  1276. // fis.read(bR);
  1277. //Get the JNIEnv* corresponding to current thread
  1278. env = getJNIEnv();
  1279. if (env == NULL) {
  1280. errno = EINTERNAL;
  1281. return -1;
  1282. }
  1283. //Parameters
  1284. if (readPrepare(env, fs, f, &jInputStream) == -1) {
  1285. return -1;
  1286. }
  1287. //Read the requisite bytes
  1288. jbRarray = (*env)->NewByteArray(env, length);
  1289. if (!jbRarray) {
  1290. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1291. "hdfsRead: NewByteArray");
  1292. return -1;
  1293. }
  1294. jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream, HADOOP_ISTRM,
  1295. "read", "([B)I", jbRarray);
  1296. if (jthr) {
  1297. destroyLocalReference(env, jbRarray);
  1298. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1299. "hdfsRead: FSDataInputStream#read");
  1300. return -1;
  1301. }
  1302. if (jVal.i < 0) {
  1303. // EOF
  1304. destroyLocalReference(env, jbRarray);
  1305. return 0;
  1306. } else if (jVal.i == 0) {
  1307. destroyLocalReference(env, jbRarray);
  1308. errno = EINTR;
  1309. return -1;
  1310. }
  1311. // We only copy the portion of the jbRarray that was actually filled by
  1312. // the call to FsDataInputStream#read; #read is not guaranteed to fill the
  1313. // entire buffer, instead it returns the number of bytes read into the
  1314. // buffer; we use the return value as the input in GetByteArrayRegion to
  1315. // ensure don't copy more bytes than necessary
  1316. (*env)->GetByteArrayRegion(env, jbRarray, 0, jVal.i, buffer);
  1317. destroyLocalReference(env, jbRarray);
  1318. if ((*env)->ExceptionCheck(env)) {
  1319. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1320. "hdfsRead: GetByteArrayRegion");
  1321. return -1;
  1322. }
  1323. return jVal.i;
  1324. }
  1325. // Reads using the read(ByteBuffer) API, which does fewer copies
  1326. tSize readDirect(hdfsFS fs, hdfsFile f, void* buffer, tSize length)
  1327. {
  1328. // JAVA EQUIVALENT:
  1329. // ByteBuffer bbuffer = ByteBuffer.allocateDirect(length) // wraps C buffer
  1330. // fis.read(bbuffer);
  1331. jobject jInputStream;
  1332. jvalue jVal;
  1333. jthrowable jthr;
  1334. jobject bb;
  1335. //Get the JNIEnv* corresponding to current thread
  1336. JNIEnv* env = getJNIEnv();
  1337. if (env == NULL) {
  1338. errno = EINTERNAL;
  1339. return -1;
  1340. }
  1341. if (readPrepare(env, fs, f, &jInputStream) == -1) {
  1342. return -1;
  1343. }
  1344. //Read the requisite bytes
  1345. bb = (*env)->NewDirectByteBuffer(env, buffer, length);
  1346. if (bb == NULL) {
  1347. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1348. "readDirect: NewDirectByteBuffer");
  1349. return -1;
  1350. }
  1351. jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream,
  1352. HADOOP_ISTRM, "read", "(Ljava/nio/ByteBuffer;)I", bb);
  1353. destroyLocalReference(env, bb);
  1354. if (jthr) {
  1355. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1356. "readDirect: FSDataInputStream#read");
  1357. return -1;
  1358. }
  1359. return (jVal.i < 0) ? 0 : jVal.i;
  1360. }
  1361. tSize hdfsPread(hdfsFS fs, hdfsFile f, tOffset position,
  1362. void* buffer, tSize length)
  1363. {
  1364. JNIEnv* env;
  1365. jbyteArray jbRarray;
  1366. jvalue jVal;
  1367. jthrowable jthr;
  1368. if (length == 0) {
  1369. return 0;
  1370. } else if (length < 0) {
  1371. errno = EINVAL;
  1372. return -1;
  1373. }
  1374. if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
  1375. errno = EBADF;
  1376. return -1;
  1377. }
  1378. env = getJNIEnv();
  1379. if (env == NULL) {
  1380. errno = EINTERNAL;
  1381. return -1;
  1382. }
  1383. //Error checking... make sure that this file is 'readable'
  1384. if (f->type != HDFS_STREAM_INPUT) {
  1385. fprintf(stderr, "Cannot read from a non-InputStream object!\n");
  1386. errno = EINVAL;
  1387. return -1;
  1388. }
  1389. // JAVA EQUIVALENT:
  1390. // byte [] bR = new byte[length];
  1391. // fis.read(pos, bR, 0, length);
  1392. jbRarray = (*env)->NewByteArray(env, length);
  1393. if (!jbRarray) {
  1394. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1395. "hdfsPread: NewByteArray");
  1396. return -1;
  1397. }
  1398. jthr = invokeMethod(env, &jVal, INSTANCE, f->file, HADOOP_ISTRM,
  1399. "read", "(J[BII)I", position, jbRarray, 0, length);
  1400. if (jthr) {
  1401. destroyLocalReference(env, jbRarray);
  1402. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1403. "hdfsPread: FSDataInputStream#read");
  1404. return -1;
  1405. }
  1406. if (jVal.i < 0) {
  1407. // EOF
  1408. destroyLocalReference(env, jbRarray);
  1409. return 0;
  1410. } else if (jVal.i == 0) {
  1411. destroyLocalReference(env, jbRarray);
  1412. errno = EINTR;
  1413. return -1;
  1414. }
  1415. (*env)->GetByteArrayRegion(env, jbRarray, 0, jVal.i, buffer);
  1416. destroyLocalReference(env, jbRarray);
  1417. if ((*env)->ExceptionCheck(env)) {
  1418. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1419. "hdfsPread: GetByteArrayRegion");
  1420. return -1;
  1421. }
  1422. return jVal.i;
  1423. }
  1424. tSize hdfsWrite(hdfsFS fs, hdfsFile f, const void* buffer, tSize length)
  1425. {
  1426. // JAVA EQUIVALENT
  1427. // byte b[] = str.getBytes();
  1428. // fso.write(b);
  1429. jobject jOutputStream;
  1430. jbyteArray jbWarray;
  1431. jthrowable jthr;
  1432. //Get the JNIEnv* corresponding to current thread
  1433. JNIEnv* env = getJNIEnv();
  1434. if (env == NULL) {
  1435. errno = EINTERNAL;
  1436. return -1;
  1437. }
  1438. //Sanity check
  1439. if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
  1440. errno = EBADF;
  1441. return -1;
  1442. }
  1443. jOutputStream = f->file;
  1444. if (length < 0) {
  1445. errno = EINVAL;
  1446. return -1;
  1447. }
  1448. //Error checking... make sure that this file is 'writable'
  1449. if (f->type != HDFS_STREAM_OUTPUT) {
  1450. fprintf(stderr, "Cannot write into a non-OutputStream object!\n");
  1451. errno = EINVAL;
  1452. return -1;
  1453. }
  1454. if (length < 0) {
  1455. errno = EINVAL;
  1456. return -1;
  1457. }
  1458. if (length == 0) {
  1459. return 0;
  1460. }
  1461. //Write the requisite bytes into the file
  1462. jbWarray = (*env)->NewByteArray(env, length);
  1463. if (!jbWarray) {
  1464. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1465. "hdfsWrite: NewByteArray");
  1466. return -1;
  1467. }
  1468. (*env)->SetByteArrayRegion(env, jbWarray, 0, length, buffer);
  1469. if ((*env)->ExceptionCheck(env)) {
  1470. destroyLocalReference(env, jbWarray);
  1471. errno = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1472. "hdfsWrite(length = %d): SetByteArrayRegion", length);
  1473. return -1;
  1474. }
  1475. jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
  1476. HADOOP_OSTRM, "write", "([B)V", jbWarray);
  1477. destroyLocalReference(env, jbWarray);
  1478. if (jthr) {
  1479. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1480. "hdfsWrite: FSDataOutputStream#write");
  1481. return -1;
  1482. }
  1483. // Unlike most Java streams, FSDataOutputStream never does partial writes.
  1484. // If we succeeded, all the data was written.
  1485. return length;
  1486. }
  1487. int hdfsSeek(hdfsFS fs, hdfsFile f, tOffset desiredPos)
  1488. {
  1489. // JAVA EQUIVALENT
  1490. // fis.seek(pos);
  1491. jobject jInputStream;
  1492. jthrowable jthr;
  1493. //Get the JNIEnv* corresponding to current thread
  1494. JNIEnv* env = getJNIEnv();
  1495. if (env == NULL) {
  1496. errno = EINTERNAL;
  1497. return -1;
  1498. }
  1499. //Sanity check
  1500. if (!f || f->type != HDFS_STREAM_INPUT) {
  1501. errno = EBADF;
  1502. return -1;
  1503. }
  1504. jInputStream = f->file;
  1505. jthr = invokeMethod(env, NULL, INSTANCE, jInputStream,
  1506. HADOOP_ISTRM, "seek", "(J)V", desiredPos);
  1507. if (jthr) {
  1508. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1509. "hdfsSeek(desiredPos=%" PRId64 ")"
  1510. ": FSDataInputStream#seek", desiredPos);
  1511. return -1;
  1512. }
  1513. return 0;
  1514. }
  1515. tOffset hdfsTell(hdfsFS fs, hdfsFile f)
  1516. {
  1517. // JAVA EQUIVALENT
  1518. // pos = f.getPos();
  1519. jobject jStream;
  1520. const char *interface;
  1521. jvalue jVal;
  1522. jthrowable jthr;
  1523. //Get the JNIEnv* corresponding to current thread
  1524. JNIEnv* env = getJNIEnv();
  1525. if (env == NULL) {
  1526. errno = EINTERNAL;
  1527. return -1;
  1528. }
  1529. //Sanity check
  1530. if (!f || f->type == HDFS_STREAM_UNINITIALIZED) {
  1531. errno = EBADF;
  1532. return -1;
  1533. }
  1534. //Parameters
  1535. jStream = f->file;
  1536. interface = (f->type == HDFS_STREAM_INPUT) ?
  1537. HADOOP_ISTRM : HADOOP_OSTRM;
  1538. jthr = invokeMethod(env, &jVal, INSTANCE, jStream,
  1539. interface, "getPos", "()J");
  1540. if (jthr) {
  1541. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1542. "hdfsTell: %s#getPos",
  1543. ((f->type == HDFS_STREAM_INPUT) ? "FSDataInputStream" :
  1544. "FSDataOutputStream"));
  1545. return -1;
  1546. }
  1547. return jVal.j;
  1548. }
  1549. int hdfsFlush(hdfsFS fs, hdfsFile f)
  1550. {
  1551. // JAVA EQUIVALENT
  1552. // fos.flush();
  1553. jthrowable jthr;
  1554. //Get the JNIEnv* corresponding to current thread
  1555. JNIEnv* env = getJNIEnv();
  1556. if (env == NULL) {
  1557. errno = EINTERNAL;
  1558. return -1;
  1559. }
  1560. //Sanity check
  1561. if (!f || f->type != HDFS_STREAM_OUTPUT) {
  1562. errno = EBADF;
  1563. return -1;
  1564. }
  1565. jthr = invokeMethod(env, NULL, INSTANCE, f->file,
  1566. HADOOP_OSTRM, "flush", "()V");
  1567. if (jthr) {
  1568. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1569. "hdfsFlush: FSDataInputStream#flush");
  1570. return -1;
  1571. }
  1572. return 0;
  1573. }
  1574. int hdfsHFlush(hdfsFS fs, hdfsFile f)
  1575. {
  1576. jobject jOutputStream;
  1577. jthrowable jthr;
  1578. //Get the JNIEnv* corresponding to current thread
  1579. JNIEnv* env = getJNIEnv();
  1580. if (env == NULL) {
  1581. errno = EINTERNAL;
  1582. return -1;
  1583. }
  1584. //Sanity check
  1585. if (!f || f->type != HDFS_STREAM_OUTPUT) {
  1586. errno = EBADF;
  1587. return -1;
  1588. }
  1589. jOutputStream = f->file;
  1590. jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
  1591. HADOOP_OSTRM, "hflush", "()V");
  1592. if (jthr) {
  1593. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1594. "hdfsHFlush: FSDataOutputStream#hflush");
  1595. return -1;
  1596. }
  1597. return 0;
  1598. }
  1599. int hdfsHSync(hdfsFS fs, hdfsFile f)
  1600. {
  1601. jobject jOutputStream;
  1602. jthrowable jthr;
  1603. //Get the JNIEnv* corresponding to current thread
  1604. JNIEnv* env = getJNIEnv();
  1605. if (env == NULL) {
  1606. errno = EINTERNAL;
  1607. return -1;
  1608. }
  1609. //Sanity check
  1610. if (!f || f->type != HDFS_STREAM_OUTPUT) {
  1611. errno = EBADF;
  1612. return -1;
  1613. }
  1614. jOutputStream = f->file;
  1615. jthr = invokeMethod(env, NULL, INSTANCE, jOutputStream,
  1616. HADOOP_OSTRM, "hsync", "()V");
  1617. if (jthr) {
  1618. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1619. "hdfsHSync: FSDataOutputStream#hsync");
  1620. return -1;
  1621. }
  1622. return 0;
  1623. }
  1624. int hdfsAvailable(hdfsFS fs, hdfsFile f)
  1625. {
  1626. // JAVA EQUIVALENT
  1627. // fis.available();
  1628. jobject jInputStream;
  1629. jvalue jVal;
  1630. jthrowable jthr;
  1631. //Get the JNIEnv* corresponding to current thread
  1632. JNIEnv* env = getJNIEnv();
  1633. if (env == NULL) {
  1634. errno = EINTERNAL;
  1635. return -1;
  1636. }
  1637. //Sanity check
  1638. if (!f || f->type != HDFS_STREAM_INPUT) {
  1639. errno = EBADF;
  1640. return -1;
  1641. }
  1642. //Parameters
  1643. jInputStream = f->file;
  1644. jthr = invokeMethod(env, &jVal, INSTANCE, jInputStream,
  1645. HADOOP_ISTRM, "available", "()I");
  1646. if (jthr) {
  1647. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1648. "hdfsAvailable: FSDataInputStream#available");
  1649. return -1;
  1650. }
  1651. return jVal.i;
  1652. }
  1653. static int hdfsCopyImpl(hdfsFS srcFS, const char *src, hdfsFS dstFS,
  1654. const char *dst, jboolean deleteSource)
  1655. {
  1656. //JAVA EQUIVALENT
  1657. // FileUtil#copy(srcFS, srcPath, dstFS, dstPath,
  1658. // deleteSource = false, conf)
  1659. //Parameters
  1660. jobject jSrcFS = (jobject)srcFS;
  1661. jobject jDstFS = (jobject)dstFS;
  1662. jobject jConfiguration = NULL, jSrcPath = NULL, jDstPath = NULL;
  1663. jthrowable jthr;
  1664. jvalue jVal;
  1665. int ret;
  1666. //Get the JNIEnv* corresponding to current thread
  1667. JNIEnv* env = getJNIEnv();
  1668. if (env == NULL) {
  1669. errno = EINTERNAL;
  1670. return -1;
  1671. }
  1672. jthr = constructNewObjectOfPath(env, src, &jSrcPath);
  1673. if (jthr) {
  1674. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1675. "hdfsCopyImpl(src=%s): constructNewObjectOfPath", src);
  1676. goto done;
  1677. }
  1678. jthr = constructNewObjectOfPath(env, dst, &jDstPath);
  1679. if (jthr) {
  1680. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1681. "hdfsCopyImpl(dst=%s): constructNewObjectOfPath", dst);
  1682. goto done;
  1683. }
  1684. //Create the org.apache.hadoop.conf.Configuration object
  1685. jthr = constructNewObjectOfClass(env, &jConfiguration,
  1686. HADOOP_CONF, "()V");
  1687. if (jthr) {
  1688. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1689. "hdfsCopyImpl: Configuration constructor");
  1690. goto done;
  1691. }
  1692. //FileUtil#copy
  1693. jthr = invokeMethod(env, &jVal, STATIC,
  1694. NULL, "org/apache/hadoop/fs/FileUtil", "copy",
  1695. "(Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;"
  1696. "Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;"
  1697. "ZLorg/apache/hadoop/conf/Configuration;)Z",
  1698. jSrcFS, jSrcPath, jDstFS, jDstPath, deleteSource,
  1699. jConfiguration);
  1700. if (jthr) {
  1701. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1702. "hdfsCopyImpl(src=%s, dst=%s, deleteSource=%d): "
  1703. "FileUtil#copy", src, dst, deleteSource);
  1704. goto done;
  1705. }
  1706. if (!jVal.z) {
  1707. ret = EIO;
  1708. goto done;
  1709. }
  1710. ret = 0;
  1711. done:
  1712. destroyLocalReference(env, jConfiguration);
  1713. destroyLocalReference(env, jSrcPath);
  1714. destroyLocalReference(env, jDstPath);
  1715. if (ret) {
  1716. errno = ret;
  1717. return -1;
  1718. }
  1719. return 0;
  1720. }
  1721. int hdfsCopy(hdfsFS srcFS, const char *src, hdfsFS dstFS, const char *dst)
  1722. {
  1723. return hdfsCopyImpl(srcFS, src, dstFS, dst, 0);
  1724. }
  1725. int hdfsMove(hdfsFS srcFS, const char *src, hdfsFS dstFS, const char *dst)
  1726. {
  1727. return hdfsCopyImpl(srcFS, src, dstFS, dst, 1);
  1728. }
  1729. int hdfsDelete(hdfsFS fs, const char *path, int recursive)
  1730. {
  1731. // JAVA EQUIVALENT:
  1732. // Path p = new Path(path);
  1733. // bool retval = fs.delete(p, recursive);
  1734. jobject jFS = (jobject)fs;
  1735. jthrowable jthr;
  1736. jobject jPath;
  1737. jvalue jVal;
  1738. jboolean jRecursive;
  1739. //Get the JNIEnv* corresponding to current thread
  1740. JNIEnv* env = getJNIEnv();
  1741. if (env == NULL) {
  1742. errno = EINTERNAL;
  1743. return -1;
  1744. }
  1745. jthr = constructNewObjectOfPath(env, path, &jPath);
  1746. if (jthr) {
  1747. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1748. "hdfsDelete(path=%s): constructNewObjectOfPath", path);
  1749. return -1;
  1750. }
  1751. jRecursive = recursive ? JNI_TRUE : JNI_FALSE;
  1752. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1753. "delete", "(Lorg/apache/hadoop/fs/Path;Z)Z",
  1754. jPath, jRecursive);
  1755. destroyLocalReference(env, jPath);
  1756. if (jthr) {
  1757. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1758. "hdfsDelete(path=%s, recursive=%d): "
  1759. "FileSystem#delete", path, recursive);
  1760. return -1;
  1761. }
  1762. if (!jVal.z) {
  1763. errno = EIO;
  1764. return -1;
  1765. }
  1766. return 0;
  1767. }
  1768. int hdfsRename(hdfsFS fs, const char *oldPath, const char *newPath)
  1769. {
  1770. // JAVA EQUIVALENT:
  1771. // Path old = new Path(oldPath);
  1772. // Path new = new Path(newPath);
  1773. // fs.rename(old, new);
  1774. jobject jFS = (jobject)fs;
  1775. jthrowable jthr;
  1776. jobject jOldPath = NULL, jNewPath = NULL;
  1777. int ret = -1;
  1778. jvalue jVal;
  1779. //Get the JNIEnv* corresponding to current thread
  1780. JNIEnv* env = getJNIEnv();
  1781. if (env == NULL) {
  1782. errno = EINTERNAL;
  1783. return -1;
  1784. }
  1785. jthr = constructNewObjectOfPath(env, oldPath, &jOldPath );
  1786. if (jthr) {
  1787. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1788. "hdfsRename: constructNewObjectOfPath(%s)", oldPath);
  1789. goto done;
  1790. }
  1791. jthr = constructNewObjectOfPath(env, newPath, &jNewPath);
  1792. if (jthr) {
  1793. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1794. "hdfsRename: constructNewObjectOfPath(%s)", newPath);
  1795. goto done;
  1796. }
  1797. // Rename the file
  1798. // TODO: use rename2 here? (See HDFS-3592)
  1799. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS, "rename",
  1800. JMETHOD2(JPARAM(HADOOP_PATH), JPARAM(HADOOP_PATH), "Z"),
  1801. jOldPath, jNewPath);
  1802. if (jthr) {
  1803. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1804. "hdfsRename(oldPath=%s, newPath=%s): FileSystem#rename",
  1805. oldPath, newPath);
  1806. goto done;
  1807. }
  1808. if (!jVal.z) {
  1809. errno = EIO;
  1810. goto done;
  1811. }
  1812. ret = 0;
  1813. done:
  1814. destroyLocalReference(env, jOldPath);
  1815. destroyLocalReference(env, jNewPath);
  1816. return ret;
  1817. }
  1818. char* hdfsGetWorkingDirectory(hdfsFS fs, char* buffer, size_t bufferSize)
  1819. {
  1820. // JAVA EQUIVALENT:
  1821. // Path p = fs.getWorkingDirectory();
  1822. // return p.toString()
  1823. jobject jPath = NULL;
  1824. jstring jPathString = NULL;
  1825. jobject jFS = (jobject)fs;
  1826. jvalue jVal;
  1827. jthrowable jthr;
  1828. int ret;
  1829. const char *jPathChars = NULL;
  1830. //Get the JNIEnv* corresponding to current thread
  1831. JNIEnv* env = getJNIEnv();
  1832. if (env == NULL) {
  1833. errno = EINTERNAL;
  1834. return NULL;
  1835. }
  1836. //FileSystem#getWorkingDirectory()
  1837. jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
  1838. HADOOP_FS, "getWorkingDirectory",
  1839. "()Lorg/apache/hadoop/fs/Path;");
  1840. if (jthr) {
  1841. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1842. "hdfsGetWorkingDirectory: FileSystem#getWorkingDirectory");
  1843. goto done;
  1844. }
  1845. jPath = jVal.l;
  1846. if (!jPath) {
  1847. fprintf(stderr, "hdfsGetWorkingDirectory: "
  1848. "FileSystem#getWorkingDirectory returned NULL");
  1849. ret = -EIO;
  1850. goto done;
  1851. }
  1852. //Path#toString()
  1853. jthr = invokeMethod(env, &jVal, INSTANCE, jPath,
  1854. "org/apache/hadoop/fs/Path", "toString",
  1855. "()Ljava/lang/String;");
  1856. if (jthr) {
  1857. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1858. "hdfsGetWorkingDirectory: Path#toString");
  1859. goto done;
  1860. }
  1861. jPathString = jVal.l;
  1862. jPathChars = (*env)->GetStringUTFChars(env, jPathString, NULL);
  1863. if (!jPathChars) {
  1864. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  1865. "hdfsGetWorkingDirectory: GetStringUTFChars");
  1866. goto done;
  1867. }
  1868. //Copy to user-provided buffer
  1869. ret = snprintf(buffer, bufferSize, "%s", jPathChars);
  1870. if (ret >= bufferSize) {
  1871. ret = ENAMETOOLONG;
  1872. goto done;
  1873. }
  1874. ret = 0;
  1875. done:
  1876. if (jPathChars) {
  1877. (*env)->ReleaseStringUTFChars(env, jPathString, jPathChars);
  1878. }
  1879. destroyLocalReference(env, jPath);
  1880. destroyLocalReference(env, jPathString);
  1881. if (ret) {
  1882. errno = ret;
  1883. return NULL;
  1884. }
  1885. return buffer;
  1886. }
  1887. int hdfsSetWorkingDirectory(hdfsFS fs, const char *path)
  1888. {
  1889. // JAVA EQUIVALENT:
  1890. // fs.setWorkingDirectory(Path(path));
  1891. jobject jFS = (jobject)fs;
  1892. jthrowable jthr;
  1893. jobject jPath;
  1894. //Get the JNIEnv* corresponding to current thread
  1895. JNIEnv* env = getJNIEnv();
  1896. if (env == NULL) {
  1897. errno = EINTERNAL;
  1898. return -1;
  1899. }
  1900. //Create an object of org.apache.hadoop.fs.Path
  1901. jthr = constructNewObjectOfPath(env, path, &jPath);
  1902. if (jthr) {
  1903. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1904. "hdfsSetWorkingDirectory(%s): constructNewObjectOfPath",
  1905. path);
  1906. return -1;
  1907. }
  1908. //FileSystem#setWorkingDirectory()
  1909. jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
  1910. "setWorkingDirectory",
  1911. "(Lorg/apache/hadoop/fs/Path;)V", jPath);
  1912. destroyLocalReference(env, jPath);
  1913. if (jthr) {
  1914. errno = printExceptionAndFree(env, jthr, NOPRINT_EXC_ILLEGAL_ARGUMENT,
  1915. "hdfsSetWorkingDirectory(%s): FileSystem#setWorkingDirectory",
  1916. path);
  1917. return -1;
  1918. }
  1919. return 0;
  1920. }
  1921. int hdfsCreateDirectory(hdfsFS fs, const char *path)
  1922. {
  1923. // JAVA EQUIVALENT:
  1924. // fs.mkdirs(new Path(path));
  1925. jobject jFS = (jobject)fs;
  1926. jobject jPath;
  1927. jthrowable jthr;
  1928. jvalue jVal;
  1929. //Get the JNIEnv* corresponding to current thread
  1930. JNIEnv* env = getJNIEnv();
  1931. if (env == NULL) {
  1932. errno = EINTERNAL;
  1933. return -1;
  1934. }
  1935. //Create an object of org.apache.hadoop.fs.Path
  1936. jthr = constructNewObjectOfPath(env, path, &jPath);
  1937. if (jthr) {
  1938. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1939. "hdfsCreateDirectory(%s): constructNewObjectOfPath", path);
  1940. return -1;
  1941. }
  1942. //Create the directory
  1943. jVal.z = 0;
  1944. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1945. "mkdirs", "(Lorg/apache/hadoop/fs/Path;)Z",
  1946. jPath);
  1947. destroyLocalReference(env, jPath);
  1948. if (jthr) {
  1949. errno = printExceptionAndFree(env, jthr,
  1950. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  1951. NOPRINT_EXC_UNRESOLVED_LINK | NOPRINT_EXC_PARENT_NOT_DIRECTORY,
  1952. "hdfsCreateDirectory(%s): FileSystem#mkdirs", path);
  1953. return -1;
  1954. }
  1955. if (!jVal.z) {
  1956. // It's unclear under exactly which conditions FileSystem#mkdirs
  1957. // is supposed to return false (as opposed to throwing an exception.)
  1958. // It seems like the current code never actually returns false.
  1959. // So we're going to translate this to EIO, since there seems to be
  1960. // nothing more specific we can do with it.
  1961. errno = EIO;
  1962. return -1;
  1963. }
  1964. return 0;
  1965. }
  1966. int hdfsSetReplication(hdfsFS fs, const char *path, int16_t replication)
  1967. {
  1968. // JAVA EQUIVALENT:
  1969. // fs.setReplication(new Path(path), replication);
  1970. jobject jFS = (jobject)fs;
  1971. jthrowable jthr;
  1972. jobject jPath;
  1973. jvalue jVal;
  1974. //Get the JNIEnv* corresponding to current thread
  1975. JNIEnv* env = getJNIEnv();
  1976. if (env == NULL) {
  1977. errno = EINTERNAL;
  1978. return -1;
  1979. }
  1980. //Create an object of org.apache.hadoop.fs.Path
  1981. jthr = constructNewObjectOfPath(env, path, &jPath);
  1982. if (jthr) {
  1983. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1984. "hdfsSetReplication(path=%s): constructNewObjectOfPath", path);
  1985. return -1;
  1986. }
  1987. //Create the directory
  1988. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  1989. "setReplication", "(Lorg/apache/hadoop/fs/Path;S)Z",
  1990. jPath, replication);
  1991. destroyLocalReference(env, jPath);
  1992. if (jthr) {
  1993. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  1994. "hdfsSetReplication(path=%s, replication=%d): "
  1995. "FileSystem#setReplication", path, replication);
  1996. return -1;
  1997. }
  1998. if (!jVal.z) {
  1999. // setReplication returns false "if file does not exist or is a
  2000. // directory." So the nearest translation to that is ENOENT.
  2001. errno = ENOENT;
  2002. return -1;
  2003. }
  2004. return 0;
  2005. }
  2006. int hdfsChown(hdfsFS fs, const char *path, const char *owner, const char *group)
  2007. {
  2008. // JAVA EQUIVALENT:
  2009. // fs.setOwner(path, owner, group)
  2010. jobject jFS = (jobject)fs;
  2011. jobject jPath = NULL;
  2012. jstring jOwner = NULL, jGroup = NULL;
  2013. jthrowable jthr;
  2014. int ret;
  2015. //Get the JNIEnv* corresponding to current thread
  2016. JNIEnv* env = getJNIEnv();
  2017. if (env == NULL) {
  2018. errno = EINTERNAL;
  2019. return -1;
  2020. }
  2021. if (owner == NULL && group == NULL) {
  2022. return 0;
  2023. }
  2024. jthr = constructNewObjectOfPath(env, path, &jPath);
  2025. if (jthr) {
  2026. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2027. "hdfsChown(path=%s): constructNewObjectOfPath", path);
  2028. goto done;
  2029. }
  2030. jthr = newJavaStr(env, owner, &jOwner);
  2031. if (jthr) {
  2032. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2033. "hdfsChown(path=%s): newJavaStr(%s)", path, owner);
  2034. goto done;
  2035. }
  2036. jthr = newJavaStr(env, group, &jGroup);
  2037. if (jthr) {
  2038. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2039. "hdfsChown(path=%s): newJavaStr(%s)", path, group);
  2040. goto done;
  2041. }
  2042. //Create the directory
  2043. jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
  2044. "setOwner", JMETHOD3(JPARAM(HADOOP_PATH),
  2045. JPARAM(JAVA_STRING), JPARAM(JAVA_STRING), JAVA_VOID),
  2046. jPath, jOwner, jGroup);
  2047. if (jthr) {
  2048. ret = printExceptionAndFree(env, jthr,
  2049. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  2050. NOPRINT_EXC_UNRESOLVED_LINK,
  2051. "hdfsChown(path=%s, owner=%s, group=%s): "
  2052. "FileSystem#setOwner", path, owner, group);
  2053. goto done;
  2054. }
  2055. ret = 0;
  2056. done:
  2057. destroyLocalReference(env, jPath);
  2058. destroyLocalReference(env, jOwner);
  2059. destroyLocalReference(env, jGroup);
  2060. if (ret) {
  2061. errno = ret;
  2062. return -1;
  2063. }
  2064. return 0;
  2065. }
  2066. int hdfsChmod(hdfsFS fs, const char *path, short mode)
  2067. {
  2068. int ret;
  2069. // JAVA EQUIVALENT:
  2070. // fs.setPermission(path, FsPermission)
  2071. jthrowable jthr;
  2072. jobject jPath = NULL, jPermObj = NULL;
  2073. jobject jFS = (jobject)fs;
  2074. jshort jmode = mode;
  2075. //Get the JNIEnv* corresponding to current thread
  2076. JNIEnv* env = getJNIEnv();
  2077. if (env == NULL) {
  2078. errno = EINTERNAL;
  2079. return -1;
  2080. }
  2081. // construct jPerm = FsPermission.createImmutable(short mode);
  2082. jthr = constructNewObjectOfClass(env, &jPermObj,
  2083. HADOOP_FSPERM,"(S)V",jmode);
  2084. if (jthr) {
  2085. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2086. "constructNewObjectOfClass(%s)", HADOOP_FSPERM);
  2087. return -1;
  2088. }
  2089. //Create an object of org.apache.hadoop.fs.Path
  2090. jthr = constructNewObjectOfPath(env, path, &jPath);
  2091. if (jthr) {
  2092. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2093. "hdfsChmod(%s): constructNewObjectOfPath", path);
  2094. goto done;
  2095. }
  2096. //Create the directory
  2097. jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
  2098. "setPermission",
  2099. JMETHOD2(JPARAM(HADOOP_PATH), JPARAM(HADOOP_FSPERM), JAVA_VOID),
  2100. jPath, jPermObj);
  2101. if (jthr) {
  2102. ret = printExceptionAndFree(env, jthr,
  2103. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  2104. NOPRINT_EXC_UNRESOLVED_LINK,
  2105. "hdfsChmod(%s): FileSystem#setPermission", path);
  2106. goto done;
  2107. }
  2108. ret = 0;
  2109. done:
  2110. destroyLocalReference(env, jPath);
  2111. destroyLocalReference(env, jPermObj);
  2112. if (ret) {
  2113. errno = ret;
  2114. return -1;
  2115. }
  2116. return 0;
  2117. }
  2118. int hdfsUtime(hdfsFS fs, const char *path, tTime mtime, tTime atime)
  2119. {
  2120. // JAVA EQUIVALENT:
  2121. // fs.setTimes(src, mtime, atime)
  2122. jthrowable jthr;
  2123. jobject jFS = (jobject)fs;
  2124. jobject jPath;
  2125. static const tTime NO_CHANGE = -1;
  2126. jlong jmtime, jatime;
  2127. //Get the JNIEnv* corresponding to current thread
  2128. JNIEnv* env = getJNIEnv();
  2129. if (env == NULL) {
  2130. errno = EINTERNAL;
  2131. return -1;
  2132. }
  2133. //Create an object of org.apache.hadoop.fs.Path
  2134. jthr = constructNewObjectOfPath(env, path, &jPath);
  2135. if (jthr) {
  2136. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2137. "hdfsUtime(path=%s): constructNewObjectOfPath", path);
  2138. return -1;
  2139. }
  2140. jmtime = (mtime == NO_CHANGE) ? -1 : (mtime * (jlong)1000);
  2141. jatime = (atime == NO_CHANGE) ? -1 : (atime * (jlong)1000);
  2142. jthr = invokeMethod(env, NULL, INSTANCE, jFS, HADOOP_FS,
  2143. "setTimes", JMETHOD3(JPARAM(HADOOP_PATH), "J", "J", JAVA_VOID),
  2144. jPath, jmtime, jatime);
  2145. destroyLocalReference(env, jPath);
  2146. if (jthr) {
  2147. errno = printExceptionAndFree(env, jthr,
  2148. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  2149. NOPRINT_EXC_UNRESOLVED_LINK,
  2150. "hdfsUtime(path=%s): FileSystem#setTimes", path);
  2151. return -1;
  2152. }
  2153. return 0;
  2154. }
  2155. /**
  2156. * Zero-copy options.
  2157. *
  2158. * We cache the EnumSet of ReadOptions which has to be passed into every
  2159. * readZero call, to avoid reconstructing it each time. This cache is cleared
  2160. * whenever an element changes.
  2161. */
  2162. struct hadoopRzOptions
  2163. {
  2164. JNIEnv *env;
  2165. int skipChecksums;
  2166. jobject byteBufferPool;
  2167. jobject cachedEnumSet;
  2168. };
  2169. struct hadoopRzOptions *hadoopRzOptionsAlloc(void)
  2170. {
  2171. struct hadoopRzOptions *opts;
  2172. JNIEnv *env;
  2173. env = getJNIEnv();
  2174. if (!env) {
  2175. // Check to make sure the JNI environment is set up properly.
  2176. errno = EINTERNAL;
  2177. return NULL;
  2178. }
  2179. opts = calloc(1, sizeof(struct hadoopRzOptions));
  2180. if (!opts) {
  2181. errno = ENOMEM;
  2182. return NULL;
  2183. }
  2184. return opts;
  2185. }
  2186. static void hadoopRzOptionsClearCached(JNIEnv *env,
  2187. struct hadoopRzOptions *opts)
  2188. {
  2189. if (!opts->cachedEnumSet) {
  2190. return;
  2191. }
  2192. (*env)->DeleteGlobalRef(env, opts->cachedEnumSet);
  2193. opts->cachedEnumSet = NULL;
  2194. }
  2195. int hadoopRzOptionsSetSkipChecksum(
  2196. struct hadoopRzOptions *opts, int skip)
  2197. {
  2198. JNIEnv *env;
  2199. env = getJNIEnv();
  2200. if (!env) {
  2201. errno = EINTERNAL;
  2202. return -1;
  2203. }
  2204. hadoopRzOptionsClearCached(env, opts);
  2205. opts->skipChecksums = !!skip;
  2206. return 0;
  2207. }
  2208. int hadoopRzOptionsSetByteBufferPool(
  2209. struct hadoopRzOptions *opts, const char *className)
  2210. {
  2211. JNIEnv *env;
  2212. jthrowable jthr;
  2213. jobject byteBufferPool = NULL;
  2214. env = getJNIEnv();
  2215. if (!env) {
  2216. errno = EINTERNAL;
  2217. return -1;
  2218. }
  2219. if (className) {
  2220. // Note: we don't have to call hadoopRzOptionsClearCached in this
  2221. // function, since the ByteBufferPool is passed separately from the
  2222. // EnumSet of ReadOptions.
  2223. jthr = constructNewObjectOfClass(env, &byteBufferPool, className, "()V");
  2224. if (jthr) {
  2225. printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2226. "hadoopRzOptionsSetByteBufferPool(className=%s): ", className);
  2227. errno = EINVAL;
  2228. return -1;
  2229. }
  2230. }
  2231. if (opts->byteBufferPool) {
  2232. // Delete any previous ByteBufferPool we had.
  2233. (*env)->DeleteGlobalRef(env, opts->byteBufferPool);
  2234. }
  2235. opts->byteBufferPool = byteBufferPool;
  2236. return 0;
  2237. }
  2238. void hadoopRzOptionsFree(struct hadoopRzOptions *opts)
  2239. {
  2240. JNIEnv *env;
  2241. env = getJNIEnv();
  2242. if (!env) {
  2243. return;
  2244. }
  2245. hadoopRzOptionsClearCached(env, opts);
  2246. if (opts->byteBufferPool) {
  2247. (*env)->DeleteGlobalRef(env, opts->byteBufferPool);
  2248. opts->byteBufferPool = NULL;
  2249. }
  2250. free(opts);
  2251. }
  2252. struct hadoopRzBuffer
  2253. {
  2254. jobject byteBuffer;
  2255. uint8_t *ptr;
  2256. int32_t length;
  2257. int direct;
  2258. };
  2259. static jthrowable hadoopRzOptionsGetEnumSet(JNIEnv *env,
  2260. struct hadoopRzOptions *opts, jobject *enumSet)
  2261. {
  2262. jthrowable jthr = NULL;
  2263. jobject enumInst = NULL, enumSetObj = NULL;
  2264. jvalue jVal;
  2265. if (opts->cachedEnumSet) {
  2266. // If we cached the value, return it now.
  2267. *enumSet = opts->cachedEnumSet;
  2268. goto done;
  2269. }
  2270. if (opts->skipChecksums) {
  2271. jthr = fetchEnumInstance(env, READ_OPTION,
  2272. "SKIP_CHECKSUMS", &enumInst);
  2273. if (jthr) {
  2274. goto done;
  2275. }
  2276. jthr = invokeMethod(env, &jVal, STATIC, NULL,
  2277. "java/util/EnumSet", "of",
  2278. "(Ljava/lang/Enum;)Ljava/util/EnumSet;", enumInst);
  2279. if (jthr) {
  2280. goto done;
  2281. }
  2282. enumSetObj = jVal.l;
  2283. } else {
  2284. jclass clazz = (*env)->FindClass(env, READ_OPTION);
  2285. if (!clazz) {
  2286. jthr = newRuntimeError(env, "failed "
  2287. "to find class for %s", READ_OPTION);
  2288. goto done;
  2289. }
  2290. jthr = invokeMethod(env, &jVal, STATIC, NULL,
  2291. "java/util/EnumSet", "noneOf",
  2292. "(Ljava/lang/Class;)Ljava/util/EnumSet;", clazz);
  2293. enumSetObj = jVal.l;
  2294. }
  2295. // create global ref
  2296. opts->cachedEnumSet = (*env)->NewGlobalRef(env, enumSetObj);
  2297. if (!opts->cachedEnumSet) {
  2298. jthr = getPendingExceptionAndClear(env);
  2299. goto done;
  2300. }
  2301. *enumSet = opts->cachedEnumSet;
  2302. jthr = NULL;
  2303. done:
  2304. (*env)->DeleteLocalRef(env, enumInst);
  2305. (*env)->DeleteLocalRef(env, enumSetObj);
  2306. return jthr;
  2307. }
  2308. static int hadoopReadZeroExtractBuffer(JNIEnv *env,
  2309. const struct hadoopRzOptions *opts, struct hadoopRzBuffer *buffer)
  2310. {
  2311. int ret;
  2312. jthrowable jthr;
  2313. jvalue jVal;
  2314. uint8_t *directStart;
  2315. void *mallocBuf = NULL;
  2316. jint position;
  2317. jarray array = NULL;
  2318. jthr = invokeMethod(env, &jVal, INSTANCE, buffer->byteBuffer,
  2319. "java/nio/ByteBuffer", "remaining", "()I");
  2320. if (jthr) {
  2321. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2322. "hadoopReadZeroExtractBuffer: ByteBuffer#remaining failed: ");
  2323. goto done;
  2324. }
  2325. buffer->length = jVal.i;
  2326. jthr = invokeMethod(env, &jVal, INSTANCE, buffer->byteBuffer,
  2327. "java/nio/ByteBuffer", "position", "()I");
  2328. if (jthr) {
  2329. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2330. "hadoopReadZeroExtractBuffer: ByteBuffer#position failed: ");
  2331. goto done;
  2332. }
  2333. position = jVal.i;
  2334. directStart = (*env)->GetDirectBufferAddress(env, buffer->byteBuffer);
  2335. if (directStart) {
  2336. // Handle direct buffers.
  2337. buffer->ptr = directStart + position;
  2338. buffer->direct = 1;
  2339. ret = 0;
  2340. goto done;
  2341. }
  2342. // Handle indirect buffers.
  2343. // The JNI docs don't say that GetDirectBufferAddress throws any exceptions
  2344. // when it fails. However, they also don't clearly say that it doesn't. It
  2345. // seems safest to clear any pending exceptions here, to prevent problems on
  2346. // various JVMs.
  2347. (*env)->ExceptionClear(env);
  2348. if (!opts->byteBufferPool) {
  2349. fputs("hadoopReadZeroExtractBuffer: we read through the "
  2350. "zero-copy path, but failed to get the address of the buffer via "
  2351. "GetDirectBufferAddress. Please make sure your JVM supports "
  2352. "GetDirectBufferAddress.\n", stderr);
  2353. ret = ENOTSUP;
  2354. goto done;
  2355. }
  2356. // Get the backing array object of this buffer.
  2357. jthr = invokeMethod(env, &jVal, INSTANCE, buffer->byteBuffer,
  2358. "java/nio/ByteBuffer", "array", "()[B");
  2359. if (jthr) {
  2360. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2361. "hadoopReadZeroExtractBuffer: ByteBuffer#array failed: ");
  2362. goto done;
  2363. }
  2364. array = jVal.l;
  2365. if (!array) {
  2366. fputs("hadoopReadZeroExtractBuffer: ByteBuffer#array returned NULL.",
  2367. stderr);
  2368. ret = EIO;
  2369. goto done;
  2370. }
  2371. mallocBuf = malloc(buffer->length);
  2372. if (!mallocBuf) {
  2373. fprintf(stderr, "hadoopReadZeroExtractBuffer: failed to allocate %d bytes of memory\n",
  2374. buffer->length);
  2375. ret = ENOMEM;
  2376. goto done;
  2377. }
  2378. (*env)->GetByteArrayRegion(env, array, position, buffer->length, mallocBuf);
  2379. jthr = (*env)->ExceptionOccurred(env);
  2380. if (jthr) {
  2381. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2382. "hadoopReadZeroExtractBuffer: GetByteArrayRegion failed: ");
  2383. goto done;
  2384. }
  2385. buffer->ptr = mallocBuf;
  2386. buffer->direct = 0;
  2387. ret = 0;
  2388. done:
  2389. free(mallocBuf);
  2390. (*env)->DeleteLocalRef(env, array);
  2391. return ret;
  2392. }
  2393. static int translateZCRException(JNIEnv *env, jthrowable exc)
  2394. {
  2395. int ret;
  2396. char *className = NULL;
  2397. jthrowable jthr = classNameOfObject(exc, env, &className);
  2398. if (jthr) {
  2399. fputs("hadoopReadZero: failed to get class name of "
  2400. "exception from read().\n", stderr);
  2401. destroyLocalReference(env, exc);
  2402. destroyLocalReference(env, jthr);
  2403. ret = EIO;
  2404. goto done;
  2405. }
  2406. if (!strcmp(className, "java.lang.UnsupportedOperationException")) {
  2407. ret = EPROTONOSUPPORT;
  2408. goto done;
  2409. }
  2410. ret = printExceptionAndFree(env, exc, PRINT_EXC_ALL,
  2411. "hadoopZeroCopyRead: ZeroCopyCursor#read failed");
  2412. done:
  2413. free(className);
  2414. return ret;
  2415. }
  2416. struct hadoopRzBuffer* hadoopReadZero(hdfsFile file,
  2417. struct hadoopRzOptions *opts, int32_t maxLength)
  2418. {
  2419. JNIEnv *env;
  2420. jthrowable jthr = NULL;
  2421. jvalue jVal;
  2422. jobject enumSet = NULL, byteBuffer = NULL;
  2423. struct hadoopRzBuffer* buffer = NULL;
  2424. int ret;
  2425. env = getJNIEnv();
  2426. if (!env) {
  2427. errno = EINTERNAL;
  2428. return NULL;
  2429. }
  2430. if (file->type != HDFS_STREAM_INPUT) {
  2431. fputs("Cannot read from a non-InputStream object!\n", stderr);
  2432. ret = EINVAL;
  2433. goto done;
  2434. }
  2435. buffer = calloc(1, sizeof(struct hadoopRzBuffer));
  2436. if (!buffer) {
  2437. ret = ENOMEM;
  2438. goto done;
  2439. }
  2440. jthr = hadoopRzOptionsGetEnumSet(env, opts, &enumSet);
  2441. if (jthr) {
  2442. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2443. "hadoopReadZero: hadoopRzOptionsGetEnumSet failed: ");
  2444. goto done;
  2445. }
  2446. jthr = invokeMethod(env, &jVal, INSTANCE, file->file, HADOOP_ISTRM, "read",
  2447. "(Lorg/apache/hadoop/io/ByteBufferPool;ILjava/util/EnumSet;)"
  2448. "Ljava/nio/ByteBuffer;", opts->byteBufferPool, maxLength, enumSet);
  2449. if (jthr) {
  2450. ret = translateZCRException(env, jthr);
  2451. goto done;
  2452. }
  2453. byteBuffer = jVal.l;
  2454. if (!byteBuffer) {
  2455. buffer->byteBuffer = NULL;
  2456. buffer->length = 0;
  2457. buffer->ptr = NULL;
  2458. } else {
  2459. buffer->byteBuffer = (*env)->NewGlobalRef(env, byteBuffer);
  2460. if (!buffer->byteBuffer) {
  2461. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  2462. "hadoopReadZero: failed to create global ref to ByteBuffer");
  2463. goto done;
  2464. }
  2465. ret = hadoopReadZeroExtractBuffer(env, opts, buffer);
  2466. if (ret) {
  2467. goto done;
  2468. }
  2469. }
  2470. ret = 0;
  2471. done:
  2472. (*env)->DeleteLocalRef(env, byteBuffer);
  2473. if (ret) {
  2474. if (buffer) {
  2475. if (buffer->byteBuffer) {
  2476. (*env)->DeleteGlobalRef(env, buffer->byteBuffer);
  2477. }
  2478. free(buffer);
  2479. }
  2480. errno = ret;
  2481. return NULL;
  2482. } else {
  2483. errno = 0;
  2484. }
  2485. return buffer;
  2486. }
  2487. int32_t hadoopRzBufferLength(const struct hadoopRzBuffer *buffer)
  2488. {
  2489. return buffer->length;
  2490. }
  2491. const void *hadoopRzBufferGet(const struct hadoopRzBuffer *buffer)
  2492. {
  2493. return buffer->ptr;
  2494. }
  2495. void hadoopRzBufferFree(hdfsFile file, struct hadoopRzBuffer *buffer)
  2496. {
  2497. jvalue jVal;
  2498. jthrowable jthr;
  2499. JNIEnv* env;
  2500. env = getJNIEnv();
  2501. if (env == NULL) {
  2502. errno = EINTERNAL;
  2503. return;
  2504. }
  2505. if (buffer->byteBuffer) {
  2506. jthr = invokeMethod(env, &jVal, INSTANCE, file->file,
  2507. HADOOP_ISTRM, "releaseBuffer",
  2508. "(Ljava/nio/ByteBuffer;)V", buffer->byteBuffer);
  2509. if (jthr) {
  2510. printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2511. "hadoopRzBufferFree: releaseBuffer failed: ");
  2512. // even on error, we have to delete the reference.
  2513. }
  2514. (*env)->DeleteGlobalRef(env, buffer->byteBuffer);
  2515. }
  2516. if (!buffer->direct) {
  2517. free(buffer->ptr);
  2518. }
  2519. memset(buffer, 0, sizeof(*buffer));
  2520. free(buffer);
  2521. }
  2522. char***
  2523. hdfsGetHosts(hdfsFS fs, const char *path, tOffset start, tOffset length)
  2524. {
  2525. // JAVA EQUIVALENT:
  2526. // fs.getFileBlockLoctions(new Path(path), start, length);
  2527. jobject jFS = (jobject)fs;
  2528. jthrowable jthr;
  2529. jobject jPath = NULL;
  2530. jobject jFileStatus = NULL;
  2531. jvalue jFSVal, jVal;
  2532. jobjectArray jBlockLocations = NULL, jFileBlockHosts = NULL;
  2533. jstring jHost = NULL;
  2534. char*** blockHosts = NULL;
  2535. int i, j, ret;
  2536. jsize jNumFileBlocks = 0;
  2537. jobject jFileBlock;
  2538. jsize jNumBlockHosts;
  2539. const char *hostName;
  2540. //Get the JNIEnv* corresponding to current thread
  2541. JNIEnv* env = getJNIEnv();
  2542. if (env == NULL) {
  2543. errno = EINTERNAL;
  2544. return NULL;
  2545. }
  2546. //Create an object of org.apache.hadoop.fs.Path
  2547. jthr = constructNewObjectOfPath(env, path, &jPath);
  2548. if (jthr) {
  2549. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2550. "hdfsGetHosts(path=%s): constructNewObjectOfPath", path);
  2551. goto done;
  2552. }
  2553. jthr = invokeMethod(env, &jFSVal, INSTANCE, jFS,
  2554. HADOOP_FS, "getFileStatus", "(Lorg/apache/hadoop/fs/Path;)"
  2555. "Lorg/apache/hadoop/fs/FileStatus;", jPath);
  2556. if (jthr) {
  2557. ret = printExceptionAndFree(env, jthr, NOPRINT_EXC_FILE_NOT_FOUND,
  2558. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2559. "FileSystem#getFileStatus", path, start, length);
  2560. destroyLocalReference(env, jPath);
  2561. goto done;
  2562. }
  2563. jFileStatus = jFSVal.l;
  2564. //org.apache.hadoop.fs.FileSystem#getFileBlockLocations
  2565. jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
  2566. HADOOP_FS, "getFileBlockLocations",
  2567. "(Lorg/apache/hadoop/fs/FileStatus;JJ)"
  2568. "[Lorg/apache/hadoop/fs/BlockLocation;",
  2569. jFileStatus, start, length);
  2570. if (jthr) {
  2571. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2572. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2573. "FileSystem#getFileBlockLocations", path, start, length);
  2574. goto done;
  2575. }
  2576. jBlockLocations = jVal.l;
  2577. //Figure out no of entries in jBlockLocations
  2578. //Allocate memory and add NULL at the end
  2579. jNumFileBlocks = (*env)->GetArrayLength(env, jBlockLocations);
  2580. blockHosts = calloc(jNumFileBlocks + 1, sizeof(char**));
  2581. if (blockHosts == NULL) {
  2582. ret = ENOMEM;
  2583. goto done;
  2584. }
  2585. if (jNumFileBlocks == 0) {
  2586. ret = 0;
  2587. goto done;
  2588. }
  2589. //Now parse each block to get hostnames
  2590. for (i = 0; i < jNumFileBlocks; ++i) {
  2591. jFileBlock =
  2592. (*env)->GetObjectArrayElement(env, jBlockLocations, i);
  2593. if (!jFileBlock) {
  2594. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  2595. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2596. "GetObjectArrayElement(%d)", path, start, length, i);
  2597. goto done;
  2598. }
  2599. jthr = invokeMethod(env, &jVal, INSTANCE, jFileBlock, HADOOP_BLK_LOC,
  2600. "getHosts", "()[Ljava/lang/String;");
  2601. if (jthr) {
  2602. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2603. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2604. "BlockLocation#getHosts", path, start, length);
  2605. goto done;
  2606. }
  2607. jFileBlockHosts = jVal.l;
  2608. if (!jFileBlockHosts) {
  2609. fprintf(stderr,
  2610. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"):"
  2611. "BlockLocation#getHosts returned NULL", path, start, length);
  2612. ret = EINTERNAL;
  2613. goto done;
  2614. }
  2615. //Figure out no of hosts in jFileBlockHosts, and allocate the memory
  2616. jNumBlockHosts = (*env)->GetArrayLength(env, jFileBlockHosts);
  2617. blockHosts[i] = calloc(jNumBlockHosts + 1, sizeof(char*));
  2618. if (!blockHosts[i]) {
  2619. ret = ENOMEM;
  2620. goto done;
  2621. }
  2622. //Now parse each hostname
  2623. for (j = 0; j < jNumBlockHosts; ++j) {
  2624. jHost = (*env)->GetObjectArrayElement(env, jFileBlockHosts, j);
  2625. if (!jHost) {
  2626. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  2627. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64"): "
  2628. "NewByteArray", path, start, length);
  2629. goto done;
  2630. }
  2631. hostName =
  2632. (const char*)((*env)->GetStringUTFChars(env, jHost, NULL));
  2633. if (!hostName) {
  2634. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  2635. "hdfsGetHosts(path=%s, start=%"PRId64", length=%"PRId64", "
  2636. "j=%d out of %d): GetStringUTFChars",
  2637. path, start, length, j, jNumBlockHosts);
  2638. goto done;
  2639. }
  2640. blockHosts[i][j] = strdup(hostName);
  2641. (*env)->ReleaseStringUTFChars(env, jHost, hostName);
  2642. if (!blockHosts[i][j]) {
  2643. ret = ENOMEM;
  2644. goto done;
  2645. }
  2646. destroyLocalReference(env, jHost);
  2647. jHost = NULL;
  2648. }
  2649. destroyLocalReference(env, jFileBlockHosts);
  2650. jFileBlockHosts = NULL;
  2651. }
  2652. ret = 0;
  2653. done:
  2654. destroyLocalReference(env, jPath);
  2655. destroyLocalReference(env, jFileStatus);
  2656. destroyLocalReference(env, jBlockLocations);
  2657. destroyLocalReference(env, jFileBlockHosts);
  2658. destroyLocalReference(env, jHost);
  2659. if (ret) {
  2660. errno = ret;
  2661. if (blockHosts) {
  2662. hdfsFreeHosts(blockHosts);
  2663. }
  2664. return NULL;
  2665. }
  2666. return blockHosts;
  2667. }
  2668. void hdfsFreeHosts(char ***blockHosts)
  2669. {
  2670. int i, j;
  2671. for (i=0; blockHosts[i]; i++) {
  2672. for (j=0; blockHosts[i][j]; j++) {
  2673. free(blockHosts[i][j]);
  2674. }
  2675. free(blockHosts[i]);
  2676. }
  2677. free(blockHosts);
  2678. }
  2679. tOffset hdfsGetDefaultBlockSize(hdfsFS fs)
  2680. {
  2681. // JAVA EQUIVALENT:
  2682. // fs.getDefaultBlockSize();
  2683. jobject jFS = (jobject)fs;
  2684. jvalue jVal;
  2685. jthrowable jthr;
  2686. //Get the JNIEnv* corresponding to current thread
  2687. JNIEnv* env = getJNIEnv();
  2688. if (env == NULL) {
  2689. errno = EINTERNAL;
  2690. return -1;
  2691. }
  2692. //FileSystem#getDefaultBlockSize()
  2693. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  2694. "getDefaultBlockSize", "()J");
  2695. if (jthr) {
  2696. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2697. "hdfsGetDefaultBlockSize: FileSystem#getDefaultBlockSize");
  2698. return -1;
  2699. }
  2700. return jVal.j;
  2701. }
  2702. tOffset hdfsGetDefaultBlockSizeAtPath(hdfsFS fs, const char *path)
  2703. {
  2704. // JAVA EQUIVALENT:
  2705. // fs.getDefaultBlockSize(path);
  2706. jthrowable jthr;
  2707. jobject jFS = (jobject)fs;
  2708. jobject jPath;
  2709. tOffset blockSize;
  2710. JNIEnv* env = getJNIEnv();
  2711. if (env == NULL) {
  2712. errno = EINTERNAL;
  2713. return -1;
  2714. }
  2715. jthr = constructNewObjectOfPath(env, path, &jPath);
  2716. if (jthr) {
  2717. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2718. "hdfsGetDefaultBlockSize(path=%s): constructNewObjectOfPath",
  2719. path);
  2720. return -1;
  2721. }
  2722. jthr = getDefaultBlockSize(env, jFS, jPath, &blockSize);
  2723. (*env)->DeleteLocalRef(env, jPath);
  2724. if (jthr) {
  2725. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2726. "hdfsGetDefaultBlockSize(path=%s): "
  2727. "FileSystem#getDefaultBlockSize", path);
  2728. return -1;
  2729. }
  2730. return blockSize;
  2731. }
  2732. tOffset hdfsGetCapacity(hdfsFS fs)
  2733. {
  2734. // JAVA EQUIVALENT:
  2735. // FsStatus fss = fs.getStatus();
  2736. // return Fss.getCapacity();
  2737. jobject jFS = (jobject)fs;
  2738. jvalue jVal;
  2739. jthrowable jthr;
  2740. jobject fss;
  2741. //Get the JNIEnv* corresponding to current thread
  2742. JNIEnv* env = getJNIEnv();
  2743. if (env == NULL) {
  2744. errno = EINTERNAL;
  2745. return -1;
  2746. }
  2747. //FileSystem#getStatus
  2748. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  2749. "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
  2750. if (jthr) {
  2751. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2752. "hdfsGetCapacity: FileSystem#getStatus");
  2753. return -1;
  2754. }
  2755. fss = (jobject)jVal.l;
  2756. jthr = invokeMethod(env, &jVal, INSTANCE, fss, HADOOP_FSSTATUS,
  2757. "getCapacity", "()J");
  2758. destroyLocalReference(env, fss);
  2759. if (jthr) {
  2760. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2761. "hdfsGetCapacity: FsStatus#getCapacity");
  2762. return -1;
  2763. }
  2764. return jVal.j;
  2765. }
  2766. tOffset hdfsGetUsed(hdfsFS fs)
  2767. {
  2768. // JAVA EQUIVALENT:
  2769. // FsStatus fss = fs.getStatus();
  2770. // return Fss.getUsed();
  2771. jobject jFS = (jobject)fs;
  2772. jvalue jVal;
  2773. jthrowable jthr;
  2774. jobject fss;
  2775. //Get the JNIEnv* corresponding to current thread
  2776. JNIEnv* env = getJNIEnv();
  2777. if (env == NULL) {
  2778. errno = EINTERNAL;
  2779. return -1;
  2780. }
  2781. //FileSystem#getStatus
  2782. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  2783. "getStatus", "()Lorg/apache/hadoop/fs/FsStatus;");
  2784. if (jthr) {
  2785. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2786. "hdfsGetUsed: FileSystem#getStatus");
  2787. return -1;
  2788. }
  2789. fss = (jobject)jVal.l;
  2790. jthr = invokeMethod(env, &jVal, INSTANCE, fss, HADOOP_FSSTATUS,
  2791. "getUsed", "()J");
  2792. destroyLocalReference(env, fss);
  2793. if (jthr) {
  2794. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  2795. "hdfsGetUsed: FsStatus#getUsed");
  2796. return -1;
  2797. }
  2798. return jVal.j;
  2799. }
  2800. /**
  2801. * We cannot add new fields to the hdfsFileInfo structure because it would break
  2802. * binary compatibility. The reason is because we return an array
  2803. * of hdfsFileInfo structures from hdfsListDirectory. So changing the size of
  2804. * those structures would break all programs that relied on finding the second
  2805. * element in the array at <base_offset> + sizeof(struct hdfsFileInfo).
  2806. *
  2807. * So instead, we add the new fields to the hdfsExtendedFileInfo structure.
  2808. * This structure is contained in the mOwner string found inside the
  2809. * hdfsFileInfo. Specifically, the format of mOwner is:
  2810. *
  2811. * [owner-string] [null byte] [padding] [hdfsExtendedFileInfo structure]
  2812. *
  2813. * The padding is added so that the hdfsExtendedFileInfo structure starts on an
  2814. * 8-byte boundary.
  2815. *
  2816. * @param str The string to locate the extended info in.
  2817. * @return The offset of the hdfsExtendedFileInfo structure.
  2818. */
  2819. static size_t getExtendedFileInfoOffset(const char *str)
  2820. {
  2821. int num_64_bit_words = ((strlen(str) + 1) + 7) / 8;
  2822. return num_64_bit_words * 8;
  2823. }
  2824. static struct hdfsExtendedFileInfo *getExtendedFileInfo(hdfsFileInfo *fileInfo)
  2825. {
  2826. char *owner = fileInfo->mOwner;
  2827. return (struct hdfsExtendedFileInfo *)(owner +
  2828. getExtendedFileInfoOffset(owner));
  2829. }
  2830. static jthrowable
  2831. getFileInfoFromStat(JNIEnv *env, jobject jStat, hdfsFileInfo *fileInfo)
  2832. {
  2833. jvalue jVal;
  2834. jthrowable jthr;
  2835. jobject jPath = NULL;
  2836. jstring jPathName = NULL;
  2837. jstring jUserName = NULL;
  2838. jstring jGroupName = NULL;
  2839. jobject jPermission = NULL;
  2840. const char *cPathName;
  2841. const char *cUserName;
  2842. const char *cGroupName;
  2843. struct hdfsExtendedFileInfo *extInfo;
  2844. size_t extOffset;
  2845. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2846. HADOOP_STAT, "isDir", "()Z");
  2847. if (jthr)
  2848. goto done;
  2849. fileInfo->mKind = jVal.z ? kObjectKindDirectory : kObjectKindFile;
  2850. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2851. HADOOP_STAT, "getReplication", "()S");
  2852. if (jthr)
  2853. goto done;
  2854. fileInfo->mReplication = jVal.s;
  2855. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2856. HADOOP_STAT, "getBlockSize", "()J");
  2857. if (jthr)
  2858. goto done;
  2859. fileInfo->mBlockSize = jVal.j;
  2860. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2861. HADOOP_STAT, "getModificationTime", "()J");
  2862. if (jthr)
  2863. goto done;
  2864. fileInfo->mLastMod = jVal.j / 1000;
  2865. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2866. HADOOP_STAT, "getAccessTime", "()J");
  2867. if (jthr)
  2868. goto done;
  2869. fileInfo->mLastAccess = (tTime) (jVal.j / 1000);
  2870. if (fileInfo->mKind == kObjectKindFile) {
  2871. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2872. HADOOP_STAT, "getLen", "()J");
  2873. if (jthr)
  2874. goto done;
  2875. fileInfo->mSize = jVal.j;
  2876. }
  2877. jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT,
  2878. "getPath", "()Lorg/apache/hadoop/fs/Path;");
  2879. if (jthr)
  2880. goto done;
  2881. jPath = jVal.l;
  2882. if (jPath == NULL) {
  2883. jthr = newRuntimeError(env, "org.apache.hadoop.fs.FileStatus#"
  2884. "getPath returned NULL!");
  2885. goto done;
  2886. }
  2887. jthr = invokeMethod(env, &jVal, INSTANCE, jPath, HADOOP_PATH,
  2888. "toString", "()Ljava/lang/String;");
  2889. if (jthr)
  2890. goto done;
  2891. jPathName = jVal.l;
  2892. cPathName =
  2893. (const char*) ((*env)->GetStringUTFChars(env, jPathName, NULL));
  2894. if (!cPathName) {
  2895. jthr = getPendingExceptionAndClear(env);
  2896. goto done;
  2897. }
  2898. fileInfo->mName = strdup(cPathName);
  2899. (*env)->ReleaseStringUTFChars(env, jPathName, cPathName);
  2900. jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT,
  2901. "getOwner", "()Ljava/lang/String;");
  2902. if (jthr)
  2903. goto done;
  2904. jUserName = jVal.l;
  2905. cUserName =
  2906. (const char*) ((*env)->GetStringUTFChars(env, jUserName, NULL));
  2907. if (!cUserName) {
  2908. jthr = getPendingExceptionAndClear(env);
  2909. goto done;
  2910. }
  2911. extOffset = getExtendedFileInfoOffset(cUserName);
  2912. fileInfo->mOwner = malloc(extOffset + sizeof(struct hdfsExtendedFileInfo));
  2913. if (!fileInfo->mOwner) {
  2914. jthr = newRuntimeError(env, "getFileInfo: OOM allocating mOwner");
  2915. goto done;
  2916. }
  2917. strcpy(fileInfo->mOwner, cUserName);
  2918. (*env)->ReleaseStringUTFChars(env, jUserName, cUserName);
  2919. extInfo = getExtendedFileInfo(fileInfo);
  2920. memset(extInfo, 0, sizeof(*extInfo));
  2921. jthr = invokeMethod(env, &jVal, INSTANCE, jStat,
  2922. HADOOP_STAT, "isEncrypted", "()Z");
  2923. if (jthr) {
  2924. goto done;
  2925. }
  2926. if (jVal.z == JNI_TRUE) {
  2927. extInfo->flags |= HDFS_EXTENDED_FILE_INFO_ENCRYPTED;
  2928. }
  2929. jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT,
  2930. "getGroup", "()Ljava/lang/String;");
  2931. if (jthr)
  2932. goto done;
  2933. jGroupName = jVal.l;
  2934. cGroupName = (const char*) ((*env)->GetStringUTFChars(env, jGroupName, NULL));
  2935. if (!cGroupName) {
  2936. jthr = getPendingExceptionAndClear(env);
  2937. goto done;
  2938. }
  2939. fileInfo->mGroup = strdup(cGroupName);
  2940. (*env)->ReleaseStringUTFChars(env, jGroupName, cGroupName);
  2941. jthr = invokeMethod(env, &jVal, INSTANCE, jStat, HADOOP_STAT,
  2942. "getPermission",
  2943. "()Lorg/apache/hadoop/fs/permission/FsPermission;");
  2944. if (jthr)
  2945. goto done;
  2946. if (jVal.l == NULL) {
  2947. jthr = newRuntimeError(env, "%s#getPermission returned NULL!",
  2948. HADOOP_STAT);
  2949. goto done;
  2950. }
  2951. jPermission = jVal.l;
  2952. jthr = invokeMethod(env, &jVal, INSTANCE, jPermission, HADOOP_FSPERM,
  2953. "toShort", "()S");
  2954. if (jthr)
  2955. goto done;
  2956. fileInfo->mPermissions = jVal.s;
  2957. jthr = NULL;
  2958. done:
  2959. if (jthr)
  2960. hdfsFreeFileInfoEntry(fileInfo);
  2961. destroyLocalReference(env, jPath);
  2962. destroyLocalReference(env, jPathName);
  2963. destroyLocalReference(env, jUserName);
  2964. destroyLocalReference(env, jGroupName);
  2965. destroyLocalReference(env, jPermission);
  2966. destroyLocalReference(env, jPath);
  2967. return jthr;
  2968. }
  2969. static jthrowable
  2970. getFileInfo(JNIEnv *env, jobject jFS, jobject jPath, hdfsFileInfo **fileInfo)
  2971. {
  2972. // JAVA EQUIVALENT:
  2973. // fs.isDirectory(f)
  2974. // fs.getModificationTime()
  2975. // fs.getAccessTime()
  2976. // fs.getLength(f)
  2977. // f.getPath()
  2978. // f.getOwner()
  2979. // f.getGroup()
  2980. // f.getPermission().toShort()
  2981. jobject jStat;
  2982. jvalue jVal;
  2983. jthrowable jthr;
  2984. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_FS,
  2985. "exists", JMETHOD1(JPARAM(HADOOP_PATH), "Z"),
  2986. jPath);
  2987. if (jthr)
  2988. return jthr;
  2989. if (jVal.z == 0) {
  2990. *fileInfo = NULL;
  2991. return NULL;
  2992. }
  2993. jthr = invokeMethod(env, &jVal, INSTANCE, jFS,
  2994. HADOOP_FS, "getFileStatus",
  2995. JMETHOD1(JPARAM(HADOOP_PATH), JPARAM(HADOOP_STAT)), jPath);
  2996. if (jthr)
  2997. return jthr;
  2998. jStat = jVal.l;
  2999. *fileInfo = calloc(1, sizeof(hdfsFileInfo));
  3000. if (!*fileInfo) {
  3001. destroyLocalReference(env, jStat);
  3002. return newRuntimeError(env, "getFileInfo: OOM allocating hdfsFileInfo");
  3003. }
  3004. jthr = getFileInfoFromStat(env, jStat, *fileInfo);
  3005. destroyLocalReference(env, jStat);
  3006. return jthr;
  3007. }
  3008. hdfsFileInfo* hdfsListDirectory(hdfsFS fs, const char *path, int *numEntries)
  3009. {
  3010. // JAVA EQUIVALENT:
  3011. // Path p(path);
  3012. // Path []pathList = fs.listPaths(p)
  3013. // foreach path in pathList
  3014. // getFileInfo(path)
  3015. jobject jFS = (jobject)fs;
  3016. jthrowable jthr;
  3017. jobject jPath = NULL;
  3018. hdfsFileInfo *pathList = NULL;
  3019. jobjectArray jPathList = NULL;
  3020. jvalue jVal;
  3021. jsize jPathListSize = 0;
  3022. int ret;
  3023. jsize i;
  3024. jobject tmpStat;
  3025. //Get the JNIEnv* corresponding to current thread
  3026. JNIEnv* env = getJNIEnv();
  3027. if (env == NULL) {
  3028. errno = EINTERNAL;
  3029. return NULL;
  3030. }
  3031. //Create an object of org.apache.hadoop.fs.Path
  3032. jthr = constructNewObjectOfPath(env, path, &jPath);
  3033. if (jthr) {
  3034. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  3035. "hdfsListDirectory(%s): constructNewObjectOfPath", path);
  3036. goto done;
  3037. }
  3038. jthr = invokeMethod(env, &jVal, INSTANCE, jFS, HADOOP_DFS, "listStatus",
  3039. JMETHOD1(JPARAM(HADOOP_PATH), JARRPARAM(HADOOP_STAT)),
  3040. jPath);
  3041. if (jthr) {
  3042. ret = printExceptionAndFree(env, jthr,
  3043. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  3044. NOPRINT_EXC_UNRESOLVED_LINK,
  3045. "hdfsListDirectory(%s): FileSystem#listStatus", path);
  3046. goto done;
  3047. }
  3048. jPathList = jVal.l;
  3049. //Figure out the number of entries in that directory
  3050. jPathListSize = (*env)->GetArrayLength(env, jPathList);
  3051. if (jPathListSize == 0) {
  3052. ret = 0;
  3053. goto done;
  3054. }
  3055. //Allocate memory
  3056. pathList = calloc(jPathListSize, sizeof(hdfsFileInfo));
  3057. if (pathList == NULL) {
  3058. ret = ENOMEM;
  3059. goto done;
  3060. }
  3061. //Save path information in pathList
  3062. for (i=0; i < jPathListSize; ++i) {
  3063. tmpStat = (*env)->GetObjectArrayElement(env, jPathList, i);
  3064. if (!tmpStat) {
  3065. ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  3066. "hdfsListDirectory(%s): GetObjectArrayElement(%d out of %d)",
  3067. path, i, jPathListSize);
  3068. goto done;
  3069. }
  3070. jthr = getFileInfoFromStat(env, tmpStat, &pathList[i]);
  3071. destroyLocalReference(env, tmpStat);
  3072. if (jthr) {
  3073. ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  3074. "hdfsListDirectory(%s): getFileInfoFromStat(%d out of %d)",
  3075. path, i, jPathListSize);
  3076. goto done;
  3077. }
  3078. }
  3079. ret = 0;
  3080. done:
  3081. destroyLocalReference(env, jPath);
  3082. destroyLocalReference(env, jPathList);
  3083. if (ret) {
  3084. hdfsFreeFileInfo(pathList, jPathListSize);
  3085. errno = ret;
  3086. return NULL;
  3087. }
  3088. *numEntries = jPathListSize;
  3089. errno = 0;
  3090. return pathList;
  3091. }
  3092. hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char *path)
  3093. {
  3094. // JAVA EQUIVALENT:
  3095. // File f(path);
  3096. // fs.isDirectory(f)
  3097. // fs.lastModified() ??
  3098. // fs.getLength(f)
  3099. // f.getPath()
  3100. jobject jFS = (jobject)fs;
  3101. jobject jPath;
  3102. jthrowable jthr;
  3103. hdfsFileInfo *fileInfo;
  3104. //Get the JNIEnv* corresponding to current thread
  3105. JNIEnv* env = getJNIEnv();
  3106. if (env == NULL) {
  3107. errno = EINTERNAL;
  3108. return NULL;
  3109. }
  3110. //Create an object of org.apache.hadoop.fs.Path
  3111. jthr = constructNewObjectOfPath(env, path, &jPath);
  3112. if (jthr) {
  3113. errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  3114. "hdfsGetPathInfo(%s): constructNewObjectOfPath", path);
  3115. return NULL;
  3116. }
  3117. jthr = getFileInfo(env, jFS, jPath, &fileInfo);
  3118. destroyLocalReference(env, jPath);
  3119. if (jthr) {
  3120. errno = printExceptionAndFree(env, jthr,
  3121. NOPRINT_EXC_ACCESS_CONTROL | NOPRINT_EXC_FILE_NOT_FOUND |
  3122. NOPRINT_EXC_UNRESOLVED_LINK,
  3123. "hdfsGetPathInfo(%s): getFileInfo", path);
  3124. return NULL;
  3125. }
  3126. if (!fileInfo) {
  3127. errno = ENOENT;
  3128. return NULL;
  3129. }
  3130. return fileInfo;
  3131. }
  3132. static void hdfsFreeFileInfoEntry(hdfsFileInfo *hdfsFileInfo)
  3133. {
  3134. free(hdfsFileInfo->mName);
  3135. free(hdfsFileInfo->mOwner);
  3136. free(hdfsFileInfo->mGroup);
  3137. memset(hdfsFileInfo, 0, sizeof(*hdfsFileInfo));
  3138. }
  3139. void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries)
  3140. {
  3141. //Free the mName, mOwner, and mGroup
  3142. int i;
  3143. for (i=0; i < numEntries; ++i) {
  3144. hdfsFreeFileInfoEntry(hdfsFileInfo + i);
  3145. }
  3146. //Free entire block
  3147. free(hdfsFileInfo);
  3148. }
  3149. int hdfsFileIsEncrypted(hdfsFileInfo *fileInfo)
  3150. {
  3151. struct hdfsExtendedFileInfo *extInfo;
  3152. extInfo = getExtendedFileInfo(fileInfo);
  3153. return !!(extInfo->flags & HDFS_EXTENDED_FILE_INFO_ENCRYPTED);
  3154. }
  3155. char* hdfsGetLastExceptionRootCause()
  3156. {
  3157. return getLastTLSExceptionRootCause();
  3158. }
  3159. char* hdfsGetLastExceptionStackTrace()
  3160. {
  3161. return getLastTLSExceptionStackTrace();
  3162. }
  3163. /**
  3164. * vim: ts=4: sw=4: et:
  3165. */