1 | /* |
2 | * librdkafka - Apache Kafka C library |
3 | * |
4 | * Copyright (c) 2016, 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 | #ifndef _RDKAFKA_FEATURE_H_ |
29 | #define _RDKAFKA_FEATURE_H_ |
30 | |
31 | |
32 | /** |
33 | * @brief Kafka protocol features |
34 | */ |
35 | |
36 | /* Message version 1 (MagicByte=1): |
37 | * + relative offsets (KIP-31) |
38 | * + timestamps (KIP-32) */ |
39 | #define RD_KAFKA_FEATURE_MSGVER1 0x1 |
40 | |
41 | /* ApiVersionQuery support (KIP-35) */ |
42 | #define RD_KAFKA_FEATURE_APIVERSION 0x2 |
43 | |
44 | /* >= 0.9: Broker-based Balanced Consumer */ |
45 | #define RD_KAFKA_FEATURE_BROKER_BALANCED_CONSUMER 0x4 |
46 | |
47 | /* >= 0.9: Produce/Fetch ThrottleTime reporting */ |
48 | #define RD_KAFKA_FEATURE_THROTTLETIME 0x8 |
49 | |
50 | /* >= 0.9: SASL GSSAPI support */ |
51 | #define RD_KAFKA_FEATURE_SASL_GSSAPI 0x10 |
52 | |
53 | /* >= 0.10: SaslMechanismRequest (KIP-43) */ |
54 | #define RD_KAFKA_FEATURE_SASL_HANDSHAKE 0x20 |
55 | |
56 | /* >= 0.8.2.0: Broker-based Group coordinator */ |
57 | #define RD_KAFKA_FEATURE_BROKER_GROUP_COORD 0x40 |
58 | |
59 | /* >= 0.8.2.0: LZ4 compression (with bad and proper HC checksums) */ |
60 | #define RD_KAFKA_FEATURE_LZ4 0x80 |
61 | |
62 | /* >= 0.10.1.0: Time-based Offset fetch (KIP-79) */ |
63 | #define RD_KAFKA_FEATURE_OFFSET_TIME 0x100 |
64 | |
65 | /* >= 0.11.0.0: Message version 2 (MagicByte=2): |
66 | * + EOS message format KIP-98 */ |
67 | #define RD_KAFKA_FEATURE_MSGVER2 0x200 |
68 | |
69 | /* >= 0.11.0.0: Idempotent Producer support */ |
70 | #define RD_KAFKA_FEATURE_IDEMPOTENT_PRODUCER 0x400 |
71 | |
72 | /* >= 2.1.0-IV2: ZSTD compression */ |
73 | #define RD_KAFKA_FEATURE_ZSTD 0x800 |
74 | |
75 | /* All features (except UNITTEST) */ |
76 | #define RD_KAFKA_FEATURE_ALL 0xfff |
77 | |
78 | /* Unit-test mock broker: broker supports everything. |
79 | * Should be used with RD_KAFKA_FEATURE_ALL, but not be included in bitmask */ |
80 | #define RD_KAFKA_FEATURE_UNITTEST 0x100000 |
81 | |
82 | |
83 | int rd_kafka_get_legacy_ApiVersions (const char *broker_version, |
84 | struct rd_kafka_ApiVersion **apisp, |
85 | size_t *api_cntp, const char *fallback); |
86 | int rd_kafka_ApiVersion_is_queryable (const char *broker_version); |
87 | void rd_kafka_ApiVersions_copy (const struct rd_kafka_ApiVersion *src, size_t src_cnt, |
88 | struct rd_kafka_ApiVersion **dstp, size_t *dst_cntp); |
89 | int rd_kafka_features_check (rd_kafka_broker_t *rkb, |
90 | struct rd_kafka_ApiVersion *broker_apis, |
91 | size_t broker_api_cnt); |
92 | |
93 | const char *rd_kafka_features2str (int features); |
94 | |
95 | #endif /* _RDKAFKA_FEATURE_H_ */ |
96 | |