jni_helper.c 20 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689
  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 "config.h"
  19. #include "exception.h"
  20. #include "jclasses.h"
  21. #include "jni_helper.h"
  22. #include "platform.h"
  23. #include "os/mutexes.h"
  24. #include "os/thread_local_storage.h"
  25. #include <stdio.h>
  26. #include <string.h>
  27. /** The Native return types that methods could return */
  28. #define JVOID 'V'
  29. #define JOBJECT 'L'
  30. #define JARRAYOBJECT '['
  31. #define JBOOLEAN 'Z'
  32. #define JBYTE 'B'
  33. #define JCHAR 'C'
  34. #define JSHORT 'S'
  35. #define JINT 'I'
  36. #define JLONG 'J'
  37. #define JFLOAT 'F'
  38. #define JDOUBLE 'D'
  39. /**
  40. * Length of buffer for retrieving created JVMs. (We only ever create one.)
  41. */
  42. #define VM_BUF_LENGTH 1
  43. void destroyLocalReference(JNIEnv *env, jobject jObject)
  44. {
  45. if (jObject)
  46. (*env)->DeleteLocalRef(env, jObject);
  47. }
  48. static jthrowable validateMethodType(JNIEnv *env, MethType methType)
  49. {
  50. if (methType != STATIC && methType != INSTANCE) {
  51. return newRuntimeError(env, "validateMethodType(methType=%d): "
  52. "illegal method type.\n", methType);
  53. }
  54. return NULL;
  55. }
  56. jthrowable newJavaStr(JNIEnv *env, const char *str, jstring *out)
  57. {
  58. jstring jstr;
  59. if (!str) {
  60. /* Can't pass NULL to NewStringUTF: the result would be
  61. * implementation-defined. */
  62. *out = NULL;
  63. return NULL;
  64. }
  65. jstr = (*env)->NewStringUTF(env, str);
  66. if (!jstr) {
  67. /* If NewStringUTF returns NULL, an exception has been thrown,
  68. * which we need to handle. Probaly an OOM. */
  69. return getPendingExceptionAndClear(env);
  70. }
  71. *out = jstr;
  72. return NULL;
  73. }
  74. jthrowable newCStr(JNIEnv *env, jstring jstr, char **out)
  75. {
  76. const char *tmp;
  77. if (!jstr) {
  78. *out = NULL;
  79. return NULL;
  80. }
  81. tmp = (*env)->GetStringUTFChars(env, jstr, NULL);
  82. if (!tmp) {
  83. return getPendingExceptionAndClear(env);
  84. }
  85. *out = strdup(tmp);
  86. (*env)->ReleaseStringUTFChars(env, jstr, tmp);
  87. return NULL;
  88. }
  89. /**
  90. * Does the work to actually execute a Java method. Takes in an existing jclass
  91. * object and a va_list of arguments for the Java method to be invoked.
  92. */
  93. static jthrowable invokeMethodOnJclass(JNIEnv *env, jvalue *retval,
  94. MethType methType, jobject instObj, jclass cls, const char *className,
  95. const char *methName, const char *methSignature, va_list args)
  96. {
  97. jmethodID mid;
  98. jthrowable jthr;
  99. const char *str;
  100. char returnType;
  101. jthr = methodIdFromClass(cls, className, methName, methSignature, methType,
  102. env, &mid);
  103. if (jthr)
  104. return jthr;
  105. str = methSignature;
  106. while (*str != ')') str++;
  107. str++;
  108. returnType = *str;
  109. if (returnType == JOBJECT || returnType == JARRAYOBJECT) {
  110. jobject jobj = NULL;
  111. if (methType == STATIC) {
  112. jobj = (*env)->CallStaticObjectMethodV(env, cls, mid, args);
  113. }
  114. else if (methType == INSTANCE) {
  115. jobj = (*env)->CallObjectMethodV(env, instObj, mid, args);
  116. }
  117. retval->l = jobj;
  118. }
  119. else if (returnType == JVOID) {
  120. if (methType == STATIC) {
  121. (*env)->CallStaticVoidMethodV(env, cls, mid, args);
  122. }
  123. else if (methType == INSTANCE) {
  124. (*env)->CallVoidMethodV(env, instObj, mid, args);
  125. }
  126. }
  127. else if (returnType == JBOOLEAN) {
  128. jboolean jbool = 0;
  129. if (methType == STATIC) {
  130. jbool = (*env)->CallStaticBooleanMethodV(env, cls, mid, args);
  131. }
  132. else if (methType == INSTANCE) {
  133. jbool = (*env)->CallBooleanMethodV(env, instObj, mid, args);
  134. }
  135. retval->z = jbool;
  136. }
  137. else if (returnType == JSHORT) {
  138. jshort js = 0;
  139. if (methType == STATIC) {
  140. js = (*env)->CallStaticShortMethodV(env, cls, mid, args);
  141. }
  142. else if (methType == INSTANCE) {
  143. js = (*env)->CallShortMethodV(env, instObj, mid, args);
  144. }
  145. retval->s = js;
  146. }
  147. else if (returnType == JLONG) {
  148. jlong jl = -1;
  149. if (methType == STATIC) {
  150. jl = (*env)->CallStaticLongMethodV(env, cls, mid, args);
  151. }
  152. else if (methType == INSTANCE) {
  153. jl = (*env)->CallLongMethodV(env, instObj, mid, args);
  154. }
  155. retval->j = jl;
  156. }
  157. else if (returnType == JINT) {
  158. jint ji = -1;
  159. if (methType == STATIC) {
  160. ji = (*env)->CallStaticIntMethodV(env, cls, mid, args);
  161. }
  162. else if (methType == INSTANCE) {
  163. ji = (*env)->CallIntMethodV(env, instObj, mid, args);
  164. }
  165. retval->i = ji;
  166. }
  167. jthr = (*env)->ExceptionOccurred(env);
  168. if (jthr) {
  169. (*env)->ExceptionClear(env);
  170. return jthr;
  171. }
  172. return NULL;
  173. }
  174. jthrowable findClassAndInvokeMethod(JNIEnv *env, jvalue *retval,
  175. MethType methType, jobject instObj, const char *className,
  176. const char *methName, const char *methSignature, ...)
  177. {
  178. jclass cls = NULL;
  179. jthrowable jthr = NULL;
  180. va_list args;
  181. va_start(args, methSignature);
  182. jthr = validateMethodType(env, methType);
  183. if (jthr) {
  184. goto done;
  185. }
  186. cls = (*env)->FindClass(env, className);
  187. if (!cls) {
  188. jthr = getPendingExceptionAndClear(env);
  189. goto done;
  190. }
  191. jthr = invokeMethodOnJclass(env, retval, methType, instObj, cls,
  192. className, methName, methSignature, args);
  193. done:
  194. va_end(args);
  195. destroyLocalReference(env, cls);
  196. return jthr;
  197. }
  198. jthrowable invokeMethod(JNIEnv *env, jvalue *retval, MethType methType,
  199. jobject instObj, CachedJavaClass class,
  200. const char *methName, const char *methSignature, ...)
  201. {
  202. jthrowable jthr;
  203. va_list args;
  204. va_start(args, methSignature);
  205. jthr = invokeMethodOnJclass(env, retval, methType, instObj,
  206. getJclass(class), getClassName(class), methName, methSignature,
  207. args);
  208. va_end(args);
  209. return jthr;
  210. }
  211. static jthrowable constructNewObjectOfJclass(JNIEnv *env,
  212. jobject *out, jclass cls, const char *className,
  213. const char *ctorSignature, va_list args) {
  214. jmethodID mid;
  215. jobject jobj;
  216. jthrowable jthr;
  217. jthr = methodIdFromClass(cls, className, "<init>", ctorSignature, INSTANCE,
  218. env, &mid);
  219. if (jthr)
  220. return jthr;
  221. jobj = (*env)->NewObjectV(env, cls, mid, args);
  222. if (!jobj)
  223. return getPendingExceptionAndClear(env);
  224. *out = jobj;
  225. return NULL;
  226. }
  227. jthrowable constructNewObjectOfClass(JNIEnv *env, jobject *out,
  228. const char *className, const char *ctorSignature, ...)
  229. {
  230. va_list args;
  231. jclass cls;
  232. jthrowable jthr = NULL;
  233. cls = (*env)->FindClass(env, className);
  234. if (!cls) {
  235. jthr = getPendingExceptionAndClear(env);
  236. goto done;
  237. }
  238. va_start(args, ctorSignature);
  239. jthr = constructNewObjectOfJclass(env, out, cls, className,
  240. ctorSignature, args);
  241. va_end(args);
  242. done:
  243. destroyLocalReference(env, cls);
  244. return jthr;
  245. }
  246. jthrowable constructNewObjectOfCachedClass(JNIEnv *env, jobject *out,
  247. CachedJavaClass cachedJavaClass, const char *ctorSignature, ...)
  248. {
  249. jthrowable jthr = NULL;
  250. va_list args;
  251. va_start(args, ctorSignature);
  252. jthr = constructNewObjectOfJclass(env, out,
  253. getJclass(cachedJavaClass), getClassName(cachedJavaClass),
  254. ctorSignature, args);
  255. va_end(args);
  256. return jthr;
  257. }
  258. jthrowable methodIdFromClass(jclass cls, const char *className,
  259. const char *methName, const char *methSignature, MethType methType,
  260. JNIEnv *env, jmethodID *out)
  261. {
  262. jthrowable jthr;
  263. jmethodID mid = 0;
  264. jthr = validateMethodType(env, methType);
  265. if (jthr)
  266. return jthr;
  267. if (methType == STATIC) {
  268. mid = (*env)->GetStaticMethodID(env, cls, methName, methSignature);
  269. }
  270. else if (methType == INSTANCE) {
  271. mid = (*env)->GetMethodID(env, cls, methName, methSignature);
  272. }
  273. if (mid == NULL) {
  274. fprintf(stderr, "could not find method %s from class %s with "
  275. "signature %s\n", methName, className, methSignature);
  276. return getPendingExceptionAndClear(env);
  277. }
  278. *out = mid;
  279. return NULL;
  280. }
  281. jthrowable classNameOfObject(jobject jobj, JNIEnv *env, char **name)
  282. {
  283. jthrowable jthr;
  284. jclass cls, clsClass = NULL;
  285. jmethodID mid;
  286. jstring str = NULL;
  287. const char *cstr = NULL;
  288. char *newstr;
  289. cls = (*env)->GetObjectClass(env, jobj);
  290. if (cls == NULL) {
  291. jthr = getPendingExceptionAndClear(env);
  292. goto done;
  293. }
  294. clsClass = (*env)->FindClass(env, "java/lang/Class");
  295. if (clsClass == NULL) {
  296. jthr = getPendingExceptionAndClear(env);
  297. goto done;
  298. }
  299. mid = (*env)->GetMethodID(env, clsClass, "getName", "()Ljava/lang/String;");
  300. if (mid == NULL) {
  301. jthr = getPendingExceptionAndClear(env);
  302. goto done;
  303. }
  304. str = (*env)->CallObjectMethod(env, cls, mid);
  305. jthr = (*env)->ExceptionOccurred(env);
  306. if (jthr) {
  307. (*env)->ExceptionClear(env);
  308. goto done;
  309. }
  310. if (str == NULL) {
  311. jthr = getPendingExceptionAndClear(env);
  312. goto done;
  313. }
  314. cstr = (*env)->GetStringUTFChars(env, str, NULL);
  315. if (!cstr) {
  316. jthr = getPendingExceptionAndClear(env);
  317. goto done;
  318. }
  319. newstr = strdup(cstr);
  320. if (newstr == NULL) {
  321. jthr = newRuntimeError(env, "classNameOfObject: out of memory");
  322. goto done;
  323. }
  324. *name = newstr;
  325. jthr = NULL;
  326. done:
  327. destroyLocalReference(env, cls);
  328. destroyLocalReference(env, clsClass);
  329. if (str) {
  330. if (cstr)
  331. (*env)->ReleaseStringUTFChars(env, str, cstr);
  332. (*env)->DeleteLocalRef(env, str);
  333. }
  334. return jthr;
  335. }
  336. /**
  337. * Get the global JNI environemnt.
  338. *
  339. * We only have to create the JVM once. After that, we can use it in
  340. * every thread. You must be holding the jvmMutex when you call this
  341. * function.
  342. *
  343. * @return The JNIEnv on success; error code otherwise
  344. */
  345. static JNIEnv* getGlobalJNIEnv(void)
  346. {
  347. JavaVM* vmBuf[VM_BUF_LENGTH];
  348. JNIEnv *env;
  349. jint rv = 0;
  350. jint noVMs = 0;
  351. jthrowable jthr;
  352. char *hadoopClassPath;
  353. const char *hadoopClassPathVMArg = "-Djava.class.path=";
  354. size_t optHadoopClassPathLen;
  355. char *optHadoopClassPath;
  356. int noArgs = 1;
  357. char *hadoopJvmArgs;
  358. char jvmArgDelims[] = " ";
  359. char *str, *token, *savePtr;
  360. JavaVMInitArgs vm_args;
  361. JavaVM *vm;
  362. JavaVMOption *options;
  363. rv = JNI_GetCreatedJavaVMs(&(vmBuf[0]), VM_BUF_LENGTH, &noVMs);
  364. if (rv != 0) {
  365. fprintf(stderr, "JNI_GetCreatedJavaVMs failed with error: %d\n", rv);
  366. return NULL;
  367. }
  368. if (noVMs == 0) {
  369. //Get the environment variables for initializing the JVM
  370. hadoopClassPath = getenv("CLASSPATH");
  371. if (hadoopClassPath == NULL) {
  372. fprintf(stderr, "Environment variable CLASSPATH not set!\n");
  373. return NULL;
  374. }
  375. optHadoopClassPathLen = strlen(hadoopClassPath) +
  376. strlen(hadoopClassPathVMArg) + 1;
  377. optHadoopClassPath = malloc(sizeof(char)*optHadoopClassPathLen);
  378. snprintf(optHadoopClassPath, optHadoopClassPathLen,
  379. "%s%s", hadoopClassPathVMArg, hadoopClassPath);
  380. // Determine the # of LIBHDFS_OPTS args
  381. hadoopJvmArgs = getenv("LIBHDFS_OPTS");
  382. if (hadoopJvmArgs != NULL) {
  383. hadoopJvmArgs = strdup(hadoopJvmArgs);
  384. for (noArgs = 1, str = hadoopJvmArgs; ; noArgs++, str = NULL) {
  385. token = strtok_r(str, jvmArgDelims, &savePtr);
  386. if (NULL == token) {
  387. break;
  388. }
  389. }
  390. free(hadoopJvmArgs);
  391. }
  392. // Now that we know the # args, populate the options array
  393. options = calloc(noArgs, sizeof(JavaVMOption));
  394. if (!options) {
  395. fputs("Call to calloc failed\n", stderr);
  396. free(optHadoopClassPath);
  397. return NULL;
  398. }
  399. options[0].optionString = optHadoopClassPath;
  400. hadoopJvmArgs = getenv("LIBHDFS_OPTS");
  401. if (hadoopJvmArgs != NULL) {
  402. hadoopJvmArgs = strdup(hadoopJvmArgs);
  403. for (noArgs = 1, str = hadoopJvmArgs; ; noArgs++, str = NULL) {
  404. token = strtok_r(str, jvmArgDelims, &savePtr);
  405. if (NULL == token) {
  406. break;
  407. }
  408. options[noArgs].optionString = token;
  409. }
  410. }
  411. //Create the VM
  412. vm_args.version = JNI_VERSION_1_2;
  413. vm_args.options = options;
  414. vm_args.nOptions = noArgs;
  415. vm_args.ignoreUnrecognized = 1;
  416. rv = JNI_CreateJavaVM(&vm, (void*)&env, &vm_args);
  417. if (hadoopJvmArgs != NULL) {
  418. free(hadoopJvmArgs);
  419. }
  420. free(optHadoopClassPath);
  421. free(options);
  422. if (rv != 0) {
  423. fprintf(stderr, "Call to JNI_CreateJavaVM failed "
  424. "with error: %d\n", rv);
  425. return NULL;
  426. }
  427. // We use findClassAndInvokeMethod here because the jclasses in
  428. // jclasses.h have not loaded yet
  429. jthr = findClassAndInvokeMethod(env, NULL, STATIC, NULL, HADOOP_FS,
  430. "loadFileSystems", "()V");
  431. if (jthr) {
  432. printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
  433. "FileSystem: loadFileSystems failed");
  434. return NULL;
  435. }
  436. } else {
  437. //Attach this thread to the VM
  438. vm = vmBuf[0];
  439. rv = (*vm)->AttachCurrentThread(vm, (void*)&env, 0);
  440. if (rv != 0) {
  441. fprintf(stderr, "Call to AttachCurrentThread "
  442. "failed with error: %d\n", rv);
  443. return NULL;
  444. }
  445. }
  446. return env;
  447. }
  448. /**
  449. * getJNIEnv: A helper function to get the JNIEnv* for the given thread.
  450. * If no JVM exists, then one will be created. JVM command line arguments
  451. * are obtained from the LIBHDFS_OPTS environment variable.
  452. *
  453. * Implementation note: we rely on POSIX thread-local storage (tls).
  454. * This allows us to associate a destructor function with each thread, that
  455. * will detach the thread from the Java VM when the thread terminates. If we
  456. * failt to do this, it will cause a memory leak.
  457. *
  458. * However, POSIX TLS is not the most efficient way to do things. It requires a
  459. * key to be initialized before it can be used. Since we don't know if this key
  460. * is initialized at the start of this function, we have to lock a mutex first
  461. * and check. Luckily, most operating systems support the more efficient
  462. * __thread construct, which is initialized by the linker.
  463. *
  464. * @param: None.
  465. * @return The JNIEnv* corresponding to the thread.
  466. */
  467. JNIEnv* getJNIEnv(void)
  468. {
  469. struct ThreadLocalState *state = NULL;
  470. THREAD_LOCAL_STORAGE_GET_QUICK(&state);
  471. if (state) return state->env;
  472. mutexLock(&jvmMutex);
  473. if (threadLocalStorageGet(&state)) {
  474. mutexUnlock(&jvmMutex);
  475. return NULL;
  476. }
  477. if (state) {
  478. mutexUnlock(&jvmMutex);
  479. // Free any stale exception strings.
  480. free(state->lastExceptionRootCause);
  481. free(state->lastExceptionStackTrace);
  482. state->lastExceptionRootCause = NULL;
  483. state->lastExceptionStackTrace = NULL;
  484. return state->env;
  485. }
  486. /* Create a ThreadLocalState for this thread */
  487. state = threadLocalStorageCreate();
  488. if (!state) {
  489. mutexUnlock(&jvmMutex);
  490. fprintf(stderr, "getJNIEnv: Unable to create ThreadLocalState\n");
  491. return NULL;
  492. }
  493. if (threadLocalStorageSet(state)) {
  494. mutexUnlock(&jvmMutex);
  495. goto fail;
  496. }
  497. THREAD_LOCAL_STORAGE_SET_QUICK(state);
  498. state->env = getGlobalJNIEnv();
  499. mutexUnlock(&jvmMutex);
  500. jthrowable jthr = NULL;
  501. jthr = initCachedClasses(state->env);
  502. if (jthr) {
  503. printExceptionAndFree(state->env, jthr, PRINT_EXC_ALL,
  504. "initCachedClasses failed");
  505. goto fail;
  506. }
  507. if (!state->env) {
  508. goto fail;
  509. }
  510. return state->env;
  511. fail:
  512. fprintf(stderr, "getJNIEnv: getGlobalJNIEnv failed\n");
  513. hdfsThreadDestructor(state);
  514. return NULL;
  515. }
  516. char* getLastTLSExceptionRootCause()
  517. {
  518. struct ThreadLocalState *state = NULL;
  519. THREAD_LOCAL_STORAGE_GET_QUICK(&state);
  520. if (!state) {
  521. mutexLock(&jvmMutex);
  522. if (threadLocalStorageGet(&state)) {
  523. mutexUnlock(&jvmMutex);
  524. return NULL;
  525. }
  526. mutexUnlock(&jvmMutex);
  527. }
  528. return state->lastExceptionRootCause;
  529. }
  530. char* getLastTLSExceptionStackTrace()
  531. {
  532. struct ThreadLocalState *state = NULL;
  533. THREAD_LOCAL_STORAGE_GET_QUICK(&state);
  534. if (!state) {
  535. mutexLock(&jvmMutex);
  536. if (threadLocalStorageGet(&state)) {
  537. mutexUnlock(&jvmMutex);
  538. return NULL;
  539. }
  540. mutexUnlock(&jvmMutex);
  541. }
  542. return state->lastExceptionStackTrace;
  543. }
  544. void setTLSExceptionStrings(const char *rootCause, const char *stackTrace)
  545. {
  546. struct ThreadLocalState *state = NULL;
  547. THREAD_LOCAL_STORAGE_GET_QUICK(&state);
  548. if (!state) {
  549. mutexLock(&jvmMutex);
  550. if (threadLocalStorageGet(&state)) {
  551. mutexUnlock(&jvmMutex);
  552. return;
  553. }
  554. mutexUnlock(&jvmMutex);
  555. }
  556. free(state->lastExceptionRootCause);
  557. free(state->lastExceptionStackTrace);
  558. state->lastExceptionRootCause = (char*)rootCause;
  559. state->lastExceptionStackTrace = (char*)stackTrace;
  560. }
  561. int javaObjectIsOfClass(JNIEnv *env, jobject obj, const char *name)
  562. {
  563. jclass clazz;
  564. int ret;
  565. clazz = (*env)->FindClass(env, name);
  566. if (!clazz) {
  567. printPendingExceptionAndFree(env, PRINT_EXC_ALL,
  568. "javaObjectIsOfClass(%s)", name);
  569. return -1;
  570. }
  571. ret = (*env)->IsInstanceOf(env, obj, clazz);
  572. (*env)->DeleteLocalRef(env, clazz);
  573. return ret == JNI_TRUE ? 1 : 0;
  574. }
  575. jthrowable hadoopConfSetStr(JNIEnv *env, jobject jConfiguration,
  576. const char *key, const char *value)
  577. {
  578. jthrowable jthr;
  579. jstring jkey = NULL, jvalue = NULL;
  580. jthr = newJavaStr(env, key, &jkey);
  581. if (jthr)
  582. goto done;
  583. jthr = newJavaStr(env, value, &jvalue);
  584. if (jthr)
  585. goto done;
  586. jthr = invokeMethod(env, NULL, INSTANCE, jConfiguration,
  587. JC_CONFIGURATION, "set", "(Ljava/lang/String;Ljava/lang/String;)V",
  588. jkey, jvalue);
  589. if (jthr)
  590. goto done;
  591. done:
  592. (*env)->DeleteLocalRef(env, jkey);
  593. (*env)->DeleteLocalRef(env, jvalue);
  594. return jthr;
  595. }
  596. jthrowable fetchEnumInstance(JNIEnv *env, const char *className,
  597. const char *valueName, jobject *out)
  598. {
  599. jclass clazz;
  600. jfieldID fieldId;
  601. jobject jEnum;
  602. char prettyClass[256];
  603. clazz = (*env)->FindClass(env, className);
  604. if (!clazz) {
  605. return getPendingExceptionAndClear(env);
  606. }
  607. if (snprintf(prettyClass, sizeof(prettyClass), "L%s;", className)
  608. >= sizeof(prettyClass)) {
  609. return newRuntimeError(env, "fetchEnum(%s, %s): class name too long.",
  610. className, valueName);
  611. }
  612. fieldId = (*env)->GetStaticFieldID(env, clazz, valueName, prettyClass);
  613. if (!fieldId) {
  614. return getPendingExceptionAndClear(env);
  615. }
  616. jEnum = (*env)->GetStaticObjectField(env, clazz, fieldId);
  617. if (!jEnum) {
  618. return getPendingExceptionAndClear(env);
  619. }
  620. *out = jEnum;
  621. return NULL;
  622. }