proxy.h 5.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179
  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. #ifndef HADOOP_CORE_RPC_PROXY_H
  19. #define HADOOP_CORE_RPC_PROXY_H
  20. #include <stdint.h> /* for uint8_t */
  21. #include <uv.h> /* for uv_buf_t */
  22. struct hadoop_err;
  23. struct hrpc_messenger;
  24. struct hrpc_proxy;
  25. struct hrpc_proxy_builder;
  26. struct hrpc_response {
  27. uint8_t *pb_base;
  28. int pb_len;
  29. void *base;
  30. };
  31. struct hrpc_sync_ctx {
  32. uv_sem_t sem;
  33. struct hadoop_err *err;
  34. struct hrpc_response resp;
  35. };
  36. typedef void (*hrpc_raw_cb_t)(struct hrpc_response *,
  37. struct hadoop_err *, void *);
  38. typedef size_t (*hrpc_pack_cb_t)(const void *, uint8_t *);
  39. /**
  40. * Allocate a Hadoop proxy builder.
  41. *
  42. * @param msgr The Hadoop messenger that this proxy will be associated
  43. * with.
  44. * @return A Hadoop proxy builder, or NULL on OOM.
  45. */
  46. struct hrpc_proxy_builder *hrpc_proxy_builder_alloc(
  47. struct hrpc_messenger *msgr);
  48. /**
  49. * Free a Hadoop proxy builder.
  50. *
  51. * @param bld The Hadoop proxy builder to free.
  52. */
  53. void hrpc_proxy_builder_free(struct hrpc_proxy_builder *bld);
  54. /**
  55. * Set the protocol used by a proxy.
  56. *
  57. * @param bld The Hadoop proxy builder.
  58. * @param proto The protocol string to use. Will be deep-copied.
  59. */
  60. void hrpc_proxy_builder_set_protocol(struct hrpc_proxy_builder *bld,
  61. const char *proto);
  62. /**
  63. * Set the remote that the proxy should connect to.
  64. *
  65. * @param bld The Hadoop proxy builder.
  66. * @param remote The remote. Will be deep-copied.
  67. */
  68. void hrpc_proxy_builder_set_remote(struct hrpc_proxy_builder *bld,
  69. const struct sockaddr_in *remote);
  70. /**
  71. * Create a Hadoop proxy
  72. *
  73. * @param bld The Hadoop proxy builder to use.
  74. * The builder will be freed, even on failure.
  75. * @param out (out param) On success, the Hadoop proxy.
  76. *
  77. * @return On success, NULL. On error, the error message.
  78. */
  79. struct hadoop_err *hrpc_proxy_create(struct hrpc_proxy_builder *bld,
  80. struct hrpc_proxy **out);
  81. /**
  82. * Free a Hadoop proxy.
  83. *
  84. * @param proxy The Hadoop proxy to free. You must not attempt to free a
  85. * proxy with a call in progress.
  86. */
  87. void hrpc_proxy_free(struct hrpc_proxy *proxy);
  88. /**
  89. * Mark the proxy as active.
  90. *
  91. * @param proxy The proxy
  92. *
  93. * @return NULL on success. If the proxy is already
  94. * active, an error will be returned.
  95. */
  96. struct hadoop_err *hrpc_proxy_activate(struct hrpc_proxy *proxy);
  97. /**
  98. * Mark the proxy as inactive.
  99. *
  100. * This function should not be called after hrpc_proxy_start, since a proxy
  101. * that has been started will mark itself as inactive when appropriate.
  102. *
  103. * @param proxy The proxy.
  104. */
  105. void hrpc_proxy_deactivate(struct hrpc_proxy *proxy);
  106. /**
  107. * Allocate some data in the proxy's userdata area.
  108. *
  109. * This will overwrite anything previously allocated in the proxy's userdata
  110. * area. It is not necessary to free this memory later; it will be freed when
  111. * the proxy is freed.
  112. *
  113. * @param proxy The proxy
  114. *
  115. * @return NULL on OOM; a pointer to the userdata
  116. * otherwise.
  117. */
  118. void *hrpc_proxy_alloc_userdata(struct hrpc_proxy *proxy, size_t size);
  119. /**
  120. * Allocate a sync context from a proxy via hrpc_proxy_alloc_userdata.
  121. *
  122. * @param proxy The proxy
  123. *
  124. * @return NULL on OOM; the sync context otherwise.
  125. */
  126. struct hrpc_sync_ctx *hrpc_proxy_alloc_sync_ctx(struct hrpc_proxy *proxy);
  127. /**
  128. * Free a sync context allocated from a proxy.
  129. *
  130. * @param proxy The sync context.
  131. */
  132. void hrpc_free_sync_ctx(struct hrpc_sync_ctx *ctx);
  133. /**
  134. * A callback which synchronous RPCs can use.
  135. */
  136. void hrpc_proxy_sync_cb(struct hrpc_response *resp, struct hadoop_err *err,
  137. void *cb_data);
  138. /**
  139. * Start an outgoing RPC from the proxy.
  140. *
  141. * This method will return after queuing up the RPC to be sent.
  142. *
  143. * @param proxy The Hadoop proxy to use. A single proxy can
  144. * only make one call at once.
  145. * @param method The method we're calling.
  146. * @param payload The protobuf message we're sending.
  147. * @param payload_packed_len Length of payload when serialized.
  148. * @param payload_pack_cb Function used to pack the payload.
  149. * @param cb Callback invoked when the message is done.
  150. * @param cb_data Data provided along with cb.
  151. */
  152. void hrpc_proxy_start(struct hrpc_proxy *proxy,
  153. const char *method, const void *payload, int payload_packed_len,
  154. hrpc_pack_cb_t payload_pack_cb,
  155. hrpc_raw_cb_t cb, void *cb_data);
  156. #endif
  157. // vim: ts=4:sw=4:tw=79:et