protoc-gen-hrpc.cc 15 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467
  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 <ctype.h>
  19. #include <google/protobuf/compiler/code_generator.h>
  20. #include <google/protobuf/compiler/plugin.h>
  21. #include <google/protobuf/descriptor.h>
  22. #include <google/protobuf/io/printer.h>
  23. #include <google/protobuf/io/zero_copy_stream.h>
  24. #include <google/protobuf/stubs/common.h>
  25. #include <cstddef>
  26. #ifdef _LIBCPP_VERSION
  27. #include <memory>
  28. #else
  29. #include <tr1/memory>
  30. #endif
  31. #include <iostream>
  32. #include <map>
  33. #include <stdio.h>
  34. #include <string>
  35. #define PROTO_EXTENSION ".proto"
  36. #define APACHE_HEADER \
  37. "/**\n" \
  38. " * Licensed to the Apache Software Foundation (ASF) under one\n" \
  39. " * or more contributor license agreements. See the NOTICE file\n" \
  40. " * distributed with this work for additional information\n" \
  41. " * regarding copyright ownership. The ASF licenses this file\n" \
  42. " * to you under the Apache License, Version 2.0 (the\n" \
  43. " * \"License\"); you may not use this file except in compliance\n" \
  44. " * with the License. You may obtain a copy of the License at\n" \
  45. " *\n" \
  46. " * http://www.apache.org/licenses/LICENSE-2.0\n" \
  47. " *\n" \
  48. " * Unless required by applicable law or agreed to in writing, software\n" \
  49. " * distributed under the License is distributed on an \"AS IS\" BASIS,\n" \
  50. " * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n" \
  51. " * See the License for the specific language governing permissions and\n" \
  52. " * limitations under the License.\n" \
  53. " */\n"
  54. using google::protobuf::FileDescriptor;
  55. using google::protobuf::MethodDescriptor;
  56. using google::protobuf::ServiceDescriptor;
  57. using google::protobuf::compiler::GeneratorContext;
  58. using google::protobuf::io::Printer;
  59. using std::map;
  60. using std::string;
  61. #ifdef _LIBCPP_VERSION
  62. using std::shared_ptr;
  63. #else
  64. using std::tr1::shared_ptr;
  65. #endif
  66. typedef map<string, string> string_map_t;
  67. static string camel_case_to_uscore(const string &in)
  68. {
  69. string out;
  70. bool prev_lower = false;
  71. for (size_t i = 0; i < in.size(); i++) {
  72. char c = in[i];
  73. if (isupper(c)) {
  74. if (prev_lower) {
  75. out += "_";
  76. }
  77. prev_lower = false;
  78. } else if (islower(c) || isdigit(c)) {
  79. prev_lower = true;
  80. } else {
  81. prev_lower = false;
  82. }
  83. out += tolower(c);
  84. }
  85. return out;
  86. }
  87. static bool try_strip_suffix(const string &str, const char *suffix,
  88. string *out)
  89. {
  90. size_t suffix_len = strlen(suffix);
  91. if (str.size() < suffix_len) {
  92. return false;
  93. }
  94. *out = str.substr(0, str.size() - suffix_len);
  95. return true;
  96. }
  97. static void get_base_name(const string &path, string *base)
  98. {
  99. size_t last_slash = path.find_last_of("/");
  100. if (last_slash != string::npos) {
  101. *base = path.substr(last_slash + 1);
  102. } else {
  103. *base = path;
  104. }
  105. }
  106. static string set_path_substitutions(const FileDescriptor *file,
  107. string_map_t *map)
  108. {
  109. string path = file->name();
  110. (*map)["path"] = path;
  111. // Initialize path_
  112. // If path = /foo/bar/baz_stuff.proto, path_ = /foo/bar/baz_stuff
  113. string path_without_extension;
  114. if (!try_strip_suffix(path, PROTO_EXTENSION, &path_without_extension)) {
  115. return string("file name " + path + " did not end in " +
  116. PROTO_EXTENSION);
  117. }
  118. (*map)["path_without_extension"] = path_without_extension;
  119. // If path = /foo/bar/baz_stuff.proto, base_ = baz_stuff
  120. string base;
  121. get_base_name(path_without_extension, &base);
  122. (*map)["path_base"] = base;
  123. (*map)["function_prefix"] = base;
  124. return "";
  125. }
  126. static string shorten_service_prefix(const string &prefix)
  127. {
  128. if (prefix == "ClientNamenodeProtocol") {
  129. return "cnn";
  130. } else if (prefix == "ClientDatanodeProtocolService") {
  131. return "cdn";
  132. } else if (prefix == "NamenodeProtocolService") {
  133. return "nnp";
  134. } else if (prefix == "DatanodeProtocolService") {
  135. return "dn";
  136. } else {
  137. return prefix;
  138. }
  139. }
  140. static void set_service_substitutions(const ServiceDescriptor *service,
  141. string_map_t *map)
  142. {
  143. // Service prefix.
  144. // example: cnn
  145. (*map)["service_prefix"] = shorten_service_prefix(service->name());
  146. }
  147. /**
  148. * Process a dot-separated type name into a protobuf-c type name.
  149. *
  150. * @param input The input type name.
  151. *
  152. * @return The protobuf-c type name.
  153. */
  154. static string get_pbc_type_name(string input)
  155. {
  156. char *word, *ptr = NULL;
  157. string output, prefix;
  158. char line[input.size() + 1];
  159. strcpy(line, input.c_str());
  160. for (word = strtok_r(line, ".", &ptr); word;
  161. word = strtok_r(NULL, ".", &ptr)) {
  162. //fprintf(stderr, "word = %s\n", word);
  163. if (!isupper(word[0])) {
  164. word[0] = toupper(word[0]);
  165. }
  166. output += prefix;
  167. prefix = "__";
  168. output += word;
  169. }
  170. return output;
  171. }
  172. static string replace(string input, char target,
  173. const char *replacement)
  174. {
  175. string output;
  176. for (size_t i = 0; i < input.size(); i++) {
  177. if (input[i] == target) {
  178. output += replacement;
  179. } else {
  180. output += input[i];
  181. }
  182. }
  183. return output;
  184. }
  185. static void set_method_substitutions(const MethodDescriptor *method,
  186. string_map_t *map)
  187. {
  188. // Request type, in camelcase.
  189. // example: Hadoop__Hdfs__SetReplicationRequestProto
  190. (*map)["req_ty_camel"] =
  191. get_pbc_type_name(method->input_type()->full_name());
  192. // Request type, in underscore-separated lowercase.
  193. // example: hadoop__hdfs__set_replication_request_proto
  194. (*map)["req_ty_uscore"] = camel_case_to_uscore((*map)["req_ty_camel"]);
  195. // Response type, in camelcase.
  196. // example: Hadoop__Hdfs__SetReplicationResponseProto
  197. (*map)["resp_ty_camel"] =
  198. get_pbc_type_name(method->output_type()->full_name());
  199. // Response type, in underscore-separated lowercase.
  200. // example: hadoop__hdfs__set_replication_response_proto
  201. (*map)["resp_ty_uscore"] = camel_case_to_uscore((*map)["resp_ty_camel"]);
  202. // RPC name, in camelcase.
  203. // example: setReplication
  204. (*map)["rpc_camel"] = method->name();
  205. // RPC name, in underscore-separated lowercase.
  206. // example: setReplication
  207. (*map)["rpc_uscore"] = camel_case_to_uscore((*map)["rpc_camel"]);
  208. // sync stub function name.
  209. // example: cnn_set_replication
  210. (*map)["sync_call"] =
  211. (*map)["service_prefix"] + "_" + (*map)["rpc_uscore"];
  212. // async stub function name.
  213. // example: cnn_async_set_replication
  214. (*map)["async_call"] =
  215. (*map)["service_prefix"] + "_async_" + (*map)["rpc_uscore"];
  216. // async callback adaptor function name.
  217. // example: cnn_async_adaptor_set_replication
  218. (*map)["async_adaptor"] =
  219. (*map)["service_prefix"] + "_async_adaptor_" + (*map)["rpc_uscore"];
  220. }
  221. class HrpcCodeGenerator
  222. : public ::google::protobuf::compiler::CodeGenerator
  223. {
  224. public:
  225. HrpcCodeGenerator()
  226. {
  227. }
  228. ~HrpcCodeGenerator()
  229. {
  230. }
  231. bool Generate(const google::protobuf::FileDescriptor *file,
  232. const string &, GeneratorContext *gen_context,
  233. string *error) const
  234. {
  235. string_map_t path_map;
  236. string ret = set_path_substitutions(file, &path_map);
  237. if (!ret.empty()) {
  238. *error = ret;
  239. return false;
  240. }
  241. generate_call_header(gen_context, &path_map, file);
  242. generate_call_body(gen_context, &path_map, file);
  243. return true;
  244. }
  245. private:
  246. void generate_call_header(GeneratorContext *gen_context,
  247. string_map_t *path_map,
  248. const FileDescriptor *file) const
  249. {
  250. shared_ptr<google::protobuf::io::ZeroCopyOutputStream> output(
  251. gen_context->Open((*path_map)["path_without_extension"] +
  252. ".call.h"));
  253. Printer printer(output.get(), '$');
  254. printer.Print(APACHE_HEADER);
  255. printer.Print(*path_map,
  256. "\n"
  257. "// This header file was auto-generated from $path$\n"
  258. "\n"
  259. "#ifndef HADOOP_NATIVE_CORE_$path_base$_CALL_H\n"
  260. "#define HADOOP_NATIVE_CORE_$path_base$_CALL_H\n"
  261. "\n"
  262. "#include \"protobuf/$path_base$.pb-c.h\"\n"
  263. "#include \"protobuf/$path_base$.pb-c.h.s\"\n"
  264. "\n"
  265. "struct hadoop_err;\n"
  266. "struct hrpc_proxy;\n"
  267. "\n");
  268. for (int service_idx = 0; service_idx < file->service_count();
  269. ++service_idx) {
  270. string_map_t service_map = *path_map;
  271. const ServiceDescriptor *service = file->service(service_idx);
  272. set_service_substitutions(service, &service_map);
  273. for (int method_idx = 0; method_idx < service->method_count();
  274. ++method_idx) {
  275. const MethodDescriptor *method = service->method(method_idx);
  276. string_map_t map = service_map;
  277. set_method_substitutions(method, &map);
  278. printer.Print(map,
  279. "struct hadoop_err *$sync_call$(struct hrpc_proxy *proxy,\n"
  280. " const $req_ty_camel$ *req,\n"
  281. " $resp_ty_camel$ **resp);\n"
  282. "\n"
  283. "void $async_call$(struct hrpc_proxy *proxy,\n"
  284. " const $req_ty_camel$ *req,\n"
  285. " void (*cb)($resp_ty_camel$ *,\n"
  286. " struct hadoop_err *, void *cb_data),\n"
  287. " void *cb_data);\n"
  288. "\n");
  289. }
  290. }
  291. printer.Print("#endif\n");
  292. }
  293. void generate_call_body(GeneratorContext *gen_context,
  294. string_map_t *path_map,
  295. const FileDescriptor *file) const
  296. {
  297. shared_ptr<google::protobuf::io::ZeroCopyOutputStream> output(
  298. gen_context->Open((*path_map)["path_without_extension"] +
  299. ".call.c"));
  300. Printer printer(output.get(), '$');
  301. printer.Print(APACHE_HEADER);
  302. printer.Print(*path_map, "\n"
  303. "#include \"common/hadoop_err.h\"\n"
  304. "#include \"protobuf/$path_base$.call.h\"\n"
  305. "#include \"rpc/messenger.h\"\n"
  306. "#include \"rpc/proxy.h\"\n");
  307. printer.Print("\n"
  308. "#include <errno.h>\n"
  309. "#include <netinet/in.h>\n"
  310. "#include <stdio.h>\n"
  311. "#include <stdlib.h>\n"
  312. "#include <string.h>\n"
  313. "#include <uv.h>\n"
  314. "\n");
  315. for (int service_idx = 0; service_idx < file->service_count();
  316. ++service_idx) {
  317. string_map_t service_map = *path_map;
  318. const ServiceDescriptor *service = file->service(service_idx);
  319. set_service_substitutions(service, &service_map);
  320. for (int method_idx = 0; method_idx < service->method_count();
  321. ++method_idx) {
  322. const MethodDescriptor *method = service->method(method_idx);
  323. string_map_t map = service_map;
  324. set_method_substitutions(method, &map);
  325. printer.Print(map,
  326. "struct hadoop_err *$sync_call$(struct hrpc_proxy *proxy,\n"
  327. " const $req_ty_camel$ *req,\n"
  328. " $resp_ty_camel$ **out)\n"
  329. "{\n"
  330. " struct hadoop_err *err;\n"
  331. " struct hrpc_sync_ctx *ctx;\n"
  332. " $resp_ty_camel$ *resp;\n"
  333. "\n"
  334. " err = hrpc_proxy_activate(proxy);\n"
  335. " if (err) {\n"
  336. " return err;\n"
  337. " }\n"
  338. " ctx = hrpc_proxy_alloc_sync_ctx(proxy);\n"
  339. " if (!ctx) {\n"
  340. " hrpc_proxy_deactivate(proxy);\n"
  341. " return hadoop_lerr_alloc(ENOMEM, \"$sync_call$: \"\n"
  342. " \"failed to allocate sync_ctx\");\n"
  343. " }\n"
  344. " hrpc_proxy_start(proxy, \"$rpc_camel$\", req,\n"
  345. " $req_ty_uscore$__get_packed_size(req),\n"
  346. " (hrpc_pack_cb_t)$req_ty_uscore$__pack,\n"
  347. " hrpc_proxy_sync_cb, ctx);\n"
  348. " uv_sem_wait(&ctx->sem);\n"
  349. " if (ctx->err) {\n"
  350. " err = ctx->err;\n"
  351. " hrpc_release_sync_ctx(ctx);\n"
  352. " return err;\n"
  353. " }\n"
  354. " resp = $resp_ty_uscore$__unpack(NULL, ctx->resp.pb_len,\n"
  355. " ctx->resp.pb_base);\n"
  356. " hrpc_release_sync_ctx(ctx);\n"
  357. " if (!resp) {\n"
  358. " return hadoop_lerr_alloc(EINVAL,\n"
  359. " \"$sync_call$: failed to parse response from server\");\n"
  360. " }\n"
  361. " *out = resp;\n"
  362. " return NULL;\n"
  363. "}\n");
  364. printer.Print(map,
  365. "struct $async_call$_cb_data {\n"
  366. " void (*cb)($resp_ty_camel$ *,\n"
  367. " struct hadoop_err *, void *);\n"
  368. " void *cb_data;\n"
  369. "};\n"
  370. "\n"
  371. "void $async_adaptor$(struct hrpc_response *resp,\n"
  372. " struct hadoop_err *err, void *cb_data)\n"
  373. "{\n"
  374. " struct $async_call$_cb_data *wrapped = cb_data;\n"
  375. " $resp_ty_camel$ *msg;\n"
  376. "\n"
  377. " if (err) {\n"
  378. " wrapped->cb(NULL, err, wrapped->cb_data);\n"
  379. " return;\n"
  380. " }\n"
  381. " msg = $resp_ty_uscore$__unpack(NULL, resp->pb_len,\n"
  382. " resp->pb_base);\n"
  383. " free(resp->base);\n"
  384. " if (!msg) {\n"
  385. " wrapped->cb(NULL, hadoop_lerr_alloc(EIO,\n"
  386. " \"$async_adaptor$: \"\n"
  387. " \"failed to parse response from server.\"), wrapped->cb_data);\n"
  388. " return;\n"
  389. " }\n"
  390. " wrapped->cb(msg, NULL, wrapped->cb_data);\n"
  391. "}\n");
  392. printer.Print(map,
  393. "void $async_call$(struct hrpc_proxy *proxy,\n"
  394. " const $req_ty_camel$ *req,\n"
  395. " void (*cb)($resp_ty_camel$ *,\n"
  396. " struct hadoop_err *, void *),\n"
  397. " void *cb_data)\n"
  398. "{\n"
  399. " struct $async_call$_cb_data *wrapped;\n"
  400. " struct hadoop_err *err;\n"
  401. "\n"
  402. " err = hrpc_proxy_activate(proxy);\n"
  403. " if (err) {\n"
  404. " cb(NULL, err, cb_data);\n"
  405. " return;\n"
  406. " }\n"
  407. " wrapped = hrpc_proxy_alloc_userdata(proxy, sizeof(*wrapped));\n"
  408. " if (!wrapped) {\n"
  409. " hrpc_proxy_deactivate(proxy);\n"
  410. " cb(NULL, hadoop_lerr_alloc(ENOMEM, \"$async_call$: failed \"\n"
  411. " \"to allocate sync_ctx\"), cb_data);\n"
  412. " return;\n"
  413. " }\n"
  414. " wrapped->cb = cb;\n"
  415. " wrapped->cb_data = cb_data;\n"
  416. " hrpc_proxy_start(proxy, \"$rpc_camel$\", req, \n"
  417. " $req_ty_uscore$__get_packed_size(req),\n"
  418. " (hrpc_pack_cb_t)$req_ty_uscore$__pack,\n"
  419. " $async_adaptor$, wrapped);\n"
  420. "}\n"
  421. "\n");
  422. }
  423. }
  424. printer.Print("// vim: ts=4:sw=4:tw=79:et\n");
  425. }
  426. };
  427. int main(int argc, char *argv[])
  428. {
  429. HrpcCodeGenerator generator;
  430. return google::protobuf::compiler::PluginMain(argc, argv, &generator);
  431. }
  432. // vim: ts=4:sw=4:tw=79:et