1 | /* |
2 | * librdkafka - The Apache Kafka C/C++ library |
3 | * |
4 | * Copyright (c) 2017 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 | |
30 | /** |
31 | * Builtin SASL PLAIN support when Cyrus SASL is not available |
32 | */ |
33 | #include "rdkafka_int.h" |
34 | #include "rdkafka_transport.h" |
35 | #include "rdkafka_transport_int.h" |
36 | #include "rdkafka_sasl.h" |
37 | #include "rdkafka_sasl_int.h" |
38 | |
39 | |
40 | /** |
41 | * @brief Handle received frame from broker. |
42 | */ |
43 | static int rd_kafka_sasl_plain_recv (struct rd_kafka_transport_s *rktrans, |
44 | const void *buf, size_t size, |
45 | char *errstr, size_t errstr_size) { |
46 | if (size) |
47 | rd_rkb_dbg(rktrans->rktrans_rkb, SECURITY, "SASLPLAIN" , |
48 | "Received non-empty SASL PLAIN (builtin) " |
49 | "response from broker (%" PRIusz" bytes)" , size); |
50 | |
51 | rd_kafka_sasl_auth_done(rktrans); |
52 | |
53 | return 0; |
54 | } |
55 | |
56 | |
57 | /** |
58 | * @brief Initialize and start SASL PLAIN (builtin) authentication. |
59 | * |
60 | * Returns 0 on successful init and -1 on error. |
61 | * |
62 | * @locality broker thread |
63 | */ |
64 | int rd_kafka_sasl_plain_client_new (rd_kafka_transport_t *rktrans, |
65 | const char *hostname, |
66 | char *errstr, size_t errstr_size) { |
67 | rd_kafka_broker_t *rkb = rktrans->rktrans_rkb; |
68 | rd_kafka_t *rk = rkb->rkb_rk; |
69 | /* [authzid] UTF8NUL authcid UTF8NUL passwd */ |
70 | char *buf; |
71 | int of = 0; |
72 | int zidlen = 0; |
73 | int cidlen = rk->rk_conf.sasl.username ? |
74 | (int)strlen(rk->rk_conf.sasl.username) : 0; |
75 | int pwlen = rk->rk_conf.sasl.password ? |
76 | (int)strlen(rk->rk_conf.sasl.password) : 0; |
77 | |
78 | |
79 | buf = rd_alloca(zidlen + 1 + cidlen + 1 + pwlen + 1); |
80 | |
81 | /* authzid: none (empty) */ |
82 | /* UTF8NUL */ |
83 | buf[of++] = 0; |
84 | /* authcid */ |
85 | memcpy(&buf[of], rk->rk_conf.sasl.username, cidlen); |
86 | of += cidlen; |
87 | /* UTF8NUL */ |
88 | buf[of++] = 0; |
89 | /* passwd */ |
90 | memcpy(&buf[of], rk->rk_conf.sasl.password, pwlen); |
91 | of += pwlen; |
92 | |
93 | rd_rkb_dbg(rkb, SECURITY, "SASLPLAIN" , |
94 | "Sending SASL PLAIN (builtin) authentication token" ); |
95 | |
96 | if (rd_kafka_sasl_send(rktrans, buf, of, |
97 | errstr, errstr_size)) |
98 | return -1; |
99 | |
100 | /* PLAIN is appearantly done here, but we still need to make sure |
101 | * the PLAIN frame is sent and we get a response back (empty) */ |
102 | rktrans->rktrans_sasl.complete = 1; |
103 | return 0; |
104 | } |
105 | |
106 | |
107 | /** |
108 | * @brief Validate PLAIN config |
109 | */ |
110 | static int rd_kafka_sasl_plain_conf_validate (rd_kafka_t *rk, |
111 | char *errstr, |
112 | size_t errstr_size) { |
113 | if (!rk->rk_conf.sasl.username || !rk->rk_conf.sasl.password) { |
114 | rd_snprintf(errstr, errstr_size, |
115 | "sasl.username and sasl.password must be set" ); |
116 | return -1; |
117 | } |
118 | |
119 | return 0; |
120 | } |
121 | |
122 | |
123 | const struct rd_kafka_sasl_provider rd_kafka_sasl_plain_provider = { |
124 | .name = "PLAIN (builtin)" , |
125 | .client_new = rd_kafka_sasl_plain_client_new, |
126 | .recv = rd_kafka_sasl_plain_recv, |
127 | .conf_validate = rd_kafka_sasl_plain_conf_validate |
128 | }; |
129 | |