protoc-gen-hrpc.cc 15 KB

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