github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/librdkafka_vendor/rdkafka_mock.h (about)

     1  /*
     2   * librdkafka - Apache Kafka C library
     3   *
     4   * Copyright (c) 2019 Magnus Edenhill
     5   * All rights reserved.
     6   *
     7   * Redistribution and use in source and binary forms, with or without
     8   * modification, are permitted provided that the following conditions are met:
     9   *
    10   * 1. Redistributions of source code must retain the above copyright notice,
    11   *    this list of conditions and the following disclaimer.
    12   * 2. Redistributions in binary form must reproduce the above copyright notice,
    13   *    this list of conditions and the following disclaimer in the documentation
    14   *    and/or other materials provided with the distribution.
    15   *
    16   * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
    17   * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
    18   * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
    19   * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
    20   * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
    21   * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
    22   * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
    23   * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
    24   * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
    25   * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
    26   * POSSIBILITY OF SUCH DAMAGE.
    27   */
    28  
    29  #ifndef _RDKAFKA_MOCK_H_
    30  #define _RDKAFKA_MOCK_H_
    31  
    32  #ifndef _RDKAFKA_H_
    33  #error "rdkafka_mock.h must be included after rdkafka.h"
    34  #endif
    35  
    36  #ifdef __cplusplus
    37  extern "C" {
    38  #if 0
    39  } /* Restore indent */
    40  #endif
    41  #endif
    42  
    43  
    44  /**
    45   * @name Mock cluster
    46   *
    47   * Provides a mock Kafka cluster with a configurable number of brokers
    48   * that support a reasonable subset of Kafka protocol operations,
    49   * error injection, etc.
    50   *
    51   * There are two ways to use the mock clusters, the most simple approach
    52   * is to configure `test.mock.num.brokers` (to e.g. 3) on the rd_kafka_t
    53   * in an existing application, which will replace the configured
    54   * `bootstrap.servers` with the mock cluster brokers.
    55   * This approach is convenient to easily test existing applications.
    56   *
    57   * The second approach is to explicitly create a mock cluster on an
    58   * rd_kafka_t instance by using rd_kafka_mock_cluster_new().
    59   *
    60   * Mock clusters provide localhost listeners that can be used as the bootstrap
    61   * servers by multiple rd_kafka_t instances.
    62   *
    63   * Currently supported functionality:
    64   *  - Producer
    65   *  - Idempotent Producer
    66   *  - Transactional Producer
    67   *  - Low-level consumer
    68   *  - High-level balanced consumer groups with offset commits
    69   *  - Topic Metadata and auto creation
    70   *
    71   * @remark High-level consumers making use of the balanced consumer groups
    72   *         are not supported.
    73   *
    74   * @remark This is an experimental public API that is NOT covered by the
    75   *         librdkafka API or ABI stability guarantees.
    76   *
    77   *
    78   * @warning THIS IS AN EXPERIMENTAL API, SUBJECT TO CHANGE OR REMOVAL.
    79   *
    80   * @{
    81   */
    82  
    83  typedef struct rd_kafka_mock_cluster_s rd_kafka_mock_cluster_t;
    84  
    85  
    86  /**
    87   * @brief Create new mock cluster with \p broker_cnt brokers.
    88   *
    89   * The broker ids will start at 1 up to and including \p broker_cnt.
    90   *
    91   * The \p rk instance is required for internal book keeping but continues
    92   * to operate as usual.
    93   */
    94  RD_EXPORT
    95  rd_kafka_mock_cluster_t *rd_kafka_mock_cluster_new (rd_kafka_t *rk,
    96                                                      int broker_cnt);
    97  
    98  
    99  /**
   100   * @brief Destroy mock cluster.
   101   */
   102  RD_EXPORT
   103  void rd_kafka_mock_cluster_destroy (rd_kafka_mock_cluster_t *mcluster);
   104  
   105  
   106  
   107  /**
   108   * @returns the rd_kafka_t instance for a cluster as passed to
   109   *          rd_kafka_mock_cluster_new().
   110   */
   111  RD_EXPORT rd_kafka_t *
   112  rd_kafka_mock_cluster_handle (const rd_kafka_mock_cluster_t *mcluster);
   113  
   114  
   115  /**
   116   * @returns the rd_kafka_mock_cluster_t instance as created by
   117   *          setting the `test.mock.num.brokers` configuration property,
   118   *          or NULL if no such instance.
   119   */
   120  RD_EXPORT rd_kafka_mock_cluster_t *
   121  rd_kafka_handle_mock_cluster (const rd_kafka_t *rk);
   122  
   123  
   124  
   125  /**
   126   * @returns the mock cluster's bootstrap.servers list
   127   */
   128  RD_EXPORT const char *
   129  rd_kafka_mock_cluster_bootstraps (const rd_kafka_mock_cluster_t *mcluster);
   130  
   131  
   132  /**
   133   * @brief Clear the cluster's error state for the given \p ApiKey.
   134   */
   135  RD_EXPORT
   136  void rd_kafka_mock_clear_request_errors (rd_kafka_mock_cluster_t *mcluster,
   137                                           int16_t ApiKey);
   138  
   139  
   140  /**
   141   * @brief Push \p cnt errors in the \p ... va-arg list onto the cluster's
   142   *        error stack for the given \p ApiKey.
   143   *
   144   * \p ApiKey is the Kafka protocol request type, e.g., ProduceRequest (0).
   145   *
   146   * The following \p cnt protocol requests matching \p ApiKey will fail with the
   147   * provided error code and removed from the stack, starting with
   148   * the first error code, then the second, etc.
   149   *
   150   * Passing \c RD_KAFKA_RESP_ERR__TRANSPORT will make the mock broker
   151   * disconnect the client which can be useful to trigger a disconnect on certain
   152   * requests.
   153   */
   154  RD_EXPORT
   155  void rd_kafka_mock_push_request_errors (rd_kafka_mock_cluster_t *mcluster,
   156                                          int16_t ApiKey, size_t cnt, ...);
   157  
   158  
   159  /**
   160   * @brief Same as rd_kafka_mock_push_request_errors() but takes
   161   *        an array of errors.
   162   */
   163  RD_EXPORT void
   164  rd_kafka_mock_push_request_errors_array (rd_kafka_mock_cluster_t *mcluster,
   165                                           int16_t ApiKey,
   166                                           size_t cnt,
   167                                           const rd_kafka_resp_err_t *errors);
   168  
   169  
   170  /**
   171   * @brief Push \p cnt errors and RTT tuples in the \p ... va-arg list onto
   172   *        the broker's error stack for the given \p ApiKey.
   173   *
   174   * \p ApiKey is the Kafka protocol request type, e.g., ProduceRequest (0).
   175   *
   176   * Each entry is a tuple of:
   177   *   rd_kafka_resp_err_t err - error to return (or 0)
   178   *   int rtt_ms              - response RTT/delay in milliseconds (or 0)
   179   *
   180   * The following \p cnt protocol requests matching \p ApiKey will fail with the
   181   * provided error code and removed from the stack, starting with
   182   * the first error code, then the second, etc.
   183   *
   184   * @remark The broker errors take precedence over the cluster errors.
   185   */
   186  RD_EXPORT rd_kafka_resp_err_t
   187  rd_kafka_mock_broker_push_request_error_rtts (rd_kafka_mock_cluster_t *mcluster,
   188                                                int32_t broker_id,
   189                                                int16_t ApiKey, size_t cnt, ...);
   190  
   191  
   192  /**
   193   * @brief Set the topic error to return in protocol requests.
   194   *
   195   * Currently only used for TopicMetadataRequest and AddPartitionsToTxnRequest.
   196   */
   197  RD_EXPORT
   198  void rd_kafka_mock_topic_set_error (rd_kafka_mock_cluster_t *mcluster,
   199                                      const char *topic,
   200                                      rd_kafka_resp_err_t err);
   201  
   202  
   203  /**
   204   * @brief Creates a topic.
   205   *
   206   * This is an alternative to automatic topic creation as performed by
   207   * the client itself.
   208   *
   209   * @remark The Topic Admin API (CreateTopics) is not supported by the
   210   *         mock broker.
   211   */
   212  RD_EXPORT rd_kafka_resp_err_t
   213  rd_kafka_mock_topic_create (rd_kafka_mock_cluster_t *mcluster,
   214                              const char *topic, int partition_cnt,
   215                              int replication_factor);
   216  
   217  
   218  /**
   219   * @brief Sets the partition leader.
   220   *
   221   * The topic will be created if it does not exist.
   222   *
   223   * \p broker_id needs to be an existing broker, or -1 to make the
   224   * partition leader-less.
   225   */
   226  RD_EXPORT rd_kafka_resp_err_t
   227  rd_kafka_mock_partition_set_leader (rd_kafka_mock_cluster_t *mcluster,
   228                                      const char *topic, int32_t partition,
   229                                      int32_t broker_id);
   230  
   231  /**
   232   * @brief Sets the partition's preferred replica / follower.
   233   *
   234   * The topic will be created if it does not exist.
   235   *
   236   * \p broker_id does not need to point to an existing broker.
   237   */
   238  RD_EXPORT rd_kafka_resp_err_t
   239  rd_kafka_mock_partition_set_follower (rd_kafka_mock_cluster_t *mcluster,
   240                                        const char *topic, int32_t partition,
   241                                        int32_t broker_id);
   242  
   243  /**
   244   * @brief Sets the partition's preferred replica / follower low and high
   245   *        watermarks.
   246   *
   247   * The topic will be created if it does not exist.
   248   *
   249   * Setting an offset to -1 will revert back to the leader's corresponding
   250   * watermark.
   251   */
   252  RD_EXPORT rd_kafka_resp_err_t
   253  rd_kafka_mock_partition_set_follower_wmarks (rd_kafka_mock_cluster_t *mcluster,
   254                                               const char *topic,
   255                                               int32_t partition,
   256                                               int64_t lo, int64_t hi);
   257  
   258  
   259  /**
   260   * @brief Disconnects the broker and disallows any new connections.
   261   *        This does NOT trigger leader change.
   262   */
   263  RD_EXPORT rd_kafka_resp_err_t
   264  rd_kafka_mock_broker_set_down (rd_kafka_mock_cluster_t *mcluster,
   265                                 int32_t broker_id);
   266  
   267  /**
   268   * @brief Makes the broker accept connections again.
   269   *        This does NOT trigger leader change.
   270   */
   271  RD_EXPORT rd_kafka_resp_err_t
   272  rd_kafka_mock_broker_set_up (rd_kafka_mock_cluster_t *mcluster,
   273                               int32_t broker_id);
   274  
   275  
   276  /**
   277   * @brief Set broker round-trip-time delay in milliseconds.
   278   */
   279  RD_EXPORT rd_kafka_resp_err_t
   280  rd_kafka_mock_broker_set_rtt (rd_kafka_mock_cluster_t *mcluster,
   281                                int32_t broker_id, int rtt_ms);
   282  
   283  /**
   284   * @brief Sets the broker's rack as reported in Metadata to the client.
   285   */
   286  RD_EXPORT rd_kafka_resp_err_t
   287  rd_kafka_mock_broker_set_rack (rd_kafka_mock_cluster_t *mcluster,
   288                                 int32_t broker_id, const char *rack);
   289  
   290  
   291  
   292  /**
   293   * @brief Explicitly sets the coordinator. If this API is not a standard
   294   *        hashing scheme will be used.
   295   *
   296   * @param key_type  "transaction" or "group"
   297   * @param key       The transactional.id or group.id
   298   * @param broker_id The new coordinator, does not have to be a valid broker.
   299   */
   300  RD_EXPORT rd_kafka_resp_err_t
   301  rd_kafka_mock_coordinator_set (rd_kafka_mock_cluster_t *mcluster,
   302                                 const char *key_type, const char *key,
   303                                 int32_t broker_id);
   304  
   305  
   306  
   307  /**
   308   * @brief Set the allowed ApiVersion range for \p ApiKey.
   309   *
   310   *        Set \p MinVersion and \p MaxVersion to -1 to disable the API
   311   *        completely.
   312   *
   313   *        \p MaxVersion MUST not exceed the maximum implemented value,
   314   *        see rdkafka_mock_handlers.c.
   315   *
   316   * @param ApiKey Protocol request type/key
   317   * @param MinVersion Minimum version supported (or -1 to disable).
   318   * @param MinVersion Maximum version supported (or -1 to disable).
   319   */
   320  RD_EXPORT rd_kafka_resp_err_t
   321  rd_kafka_mock_set_apiversion (rd_kafka_mock_cluster_t *mcluster,
   322                                int16_t ApiKey,
   323                                int16_t MinVersion, int16_t MaxVersion);
   324  
   325  
   326  /**@}*/
   327  
   328  #ifdef __cplusplus
   329  }
   330  #endif
   331  #endif /* _RDKAFKA_MOCK_H_ */