github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/config.go (about) 1 /** 2 * Copyright 2016 Confluent Inc. 3 * 4 * Licensed under the Apache License, Version 2.0 (the "License"); 5 * you may not use this file except in compliance with the License. 6 * You may obtain a copy of the License at 7 * 8 * http://www.apache.org/licenses/LICENSE-2.0 9 * 10 * Unless required by applicable law or agreed to in writing, software 11 * distributed under the License is distributed on an "AS IS" BASIS, 12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 * See the License for the specific language governing permissions and 14 * limitations under the License. 15 */ 16 17 package kafka 18 19 import ( 20 "fmt" 21 "reflect" 22 "strings" 23 "unsafe" 24 ) 25 26 /* 27 #include <stdlib.h> 28 #include "select_rdkafka.h" 29 */ 30 import "C" 31 32 // ConfigValue supports the following types: 33 // bool, int, string, any type with the standard String() interface 34 type ConfigValue interface{} 35 36 // ConfigMap is a map containing standard librdkafka configuration properties as documented in: 37 // https://github.com/edenhill/librdkafka/tree/master/CONFIGURATION.md 38 // 39 // The special property "default.topic.config" (optional) is a ConfigMap 40 // containing default topic configuration properties. 41 // 42 // The use of "default.topic.config" is deprecated, 43 // topic configuration properties shall be specified in the standard ConfigMap. 44 // For backwards compatibility, "default.topic.config" (if supplied) 45 // takes precedence. 46 type ConfigMap map[string]ConfigValue 47 48 // SetKey sets configuration property key to value. 49 // 50 // For user convenience a key prefixed with {topic}. will be 51 // set on the "default.topic.config" sub-map, this use is deprecated. 52 func (m ConfigMap) SetKey(key string, value ConfigValue) error { 53 if strings.HasPrefix(key, "{topic}.") { 54 _, found := m["default.topic.config"] 55 if !found { 56 m["default.topic.config"] = ConfigMap{} 57 } 58 m["default.topic.config"].(ConfigMap)[strings.TrimPrefix(key, "{topic}.")] = value 59 } else { 60 m[key] = value 61 } 62 63 return nil 64 } 65 66 // Set implements flag.Set (command line argument parser) as a convenience 67 // for `-X key=value` config. 68 func (m ConfigMap) Set(kv string) error { 69 i := strings.Index(kv, "=") 70 if i == -1 { 71 return newErrorFromString(ErrInvalidArg, "Expected key=value") 72 } 73 74 k := kv[:i] 75 v := kv[i+1:] 76 77 return m.SetKey(k, v) 78 } 79 80 func value2string(v ConfigValue) (ret string, errstr string) { 81 82 switch x := v.(type) { 83 case bool: 84 if x { 85 ret = "true" 86 } else { 87 ret = "false" 88 } 89 case int: 90 ret = fmt.Sprintf("%d", x) 91 case string: 92 ret = x 93 case fmt.Stringer: 94 ret = x.String() 95 default: 96 return "", fmt.Sprintf("Invalid value type %T", v) 97 } 98 99 return ret, "" 100 } 101 102 // rdkAnyconf abstracts rd_kafka_conf_t and rd_kafka_topic_conf_t 103 // into a common interface. 104 type rdkAnyconf interface { 105 set(cKey *C.char, cVal *C.char, cErrstr *C.char, errstrSize int) C.rd_kafka_conf_res_t 106 } 107 108 func anyconfSet(anyconf rdkAnyconf, key string, val ConfigValue) (err error) { 109 value, errstr := value2string(val) 110 if errstr != "" { 111 return newErrorFromString(ErrInvalidArg, fmt.Sprintf("%s for key %s (expected string,bool,int,ConfigMap)", errstr, key)) 112 } 113 cKey := C.CString(key) 114 defer C.free(unsafe.Pointer(cKey)) 115 cVal := C.CString(value) 116 defer C.free(unsafe.Pointer(cVal)) 117 cErrstr := (*C.char)(C.malloc(C.size_t(128))) 118 defer C.free(unsafe.Pointer(cErrstr)) 119 120 if anyconf.set(cKey, cVal, cErrstr, 128) != C.RD_KAFKA_CONF_OK { 121 return newErrorFromCString(C.RD_KAFKA_RESP_ERR__INVALID_ARG, cErrstr) 122 } 123 124 return nil 125 } 126 127 // we need these typedefs to workaround a crash in golint 128 // when parsing the set() methods below 129 type rdkConf C.rd_kafka_conf_t 130 type rdkTopicConf C.rd_kafka_topic_conf_t 131 132 func (cConf *rdkConf) set(cKey *C.char, cVal *C.char, cErrstr *C.char, errstrSize int) C.rd_kafka_conf_res_t { 133 return C.rd_kafka_conf_set((*C.rd_kafka_conf_t)(cConf), cKey, cVal, cErrstr, C.size_t(errstrSize)) 134 } 135 136 func (ctopicConf *rdkTopicConf) set(cKey *C.char, cVal *C.char, cErrstr *C.char, errstrSize int) C.rd_kafka_conf_res_t { 137 return C.rd_kafka_topic_conf_set((*C.rd_kafka_topic_conf_t)(ctopicConf), cKey, cVal, cErrstr, C.size_t(errstrSize)) 138 } 139 140 func configConvertAnyconf(m ConfigMap, anyconf rdkAnyconf) (err error) { 141 // set plugins first, any plugin-specific configuration depends on 142 // the plugin to have already been set 143 pluginPaths, ok := m["plugin.library.paths"] 144 if ok { 145 err = anyconfSet(anyconf, "plugin.library.paths", pluginPaths) 146 if err != nil { 147 return err 148 } 149 } 150 for k, v := range m { 151 if k == "plugin.library.paths" { 152 continue 153 } 154 switch v.(type) { 155 case ConfigMap: 156 /* Special sub-ConfigMap, only used for default.topic.config */ 157 158 if k != "default.topic.config" { 159 return newErrorFromString(ErrInvalidArg, fmt.Sprintf("Invalid type for key %s", k)) 160 } 161 162 var cTopicConf = C.rd_kafka_topic_conf_new() 163 164 err = configConvertAnyconf(v.(ConfigMap), 165 (*rdkTopicConf)(cTopicConf)) 166 if err != nil { 167 C.rd_kafka_topic_conf_destroy(cTopicConf) 168 return err 169 } 170 171 C.rd_kafka_conf_set_default_topic_conf( 172 (*C.rd_kafka_conf_t)(anyconf.(*rdkConf)), 173 (*C.rd_kafka_topic_conf_t)((*rdkTopicConf)(cTopicConf))) 174 175 default: 176 err = anyconfSet(anyconf, k, v) 177 if err != nil { 178 return err 179 } 180 } 181 } 182 183 return nil 184 } 185 186 // convert ConfigMap to C rd_kafka_conf_t * 187 func (m ConfigMap) convert() (cConf *C.rd_kafka_conf_t, err error) { 188 cConf = C.rd_kafka_conf_new() 189 190 // Set the client.software.name and .version (use librdkafka version). 191 _, librdkafkaVersion := LibraryVersion() 192 anyconfSet((*rdkConf)(cConf), "client.software.name", "confluent-kafka-go") 193 anyconfSet((*rdkConf)(cConf), "client.software.version", librdkafkaVersion) 194 195 err = configConvertAnyconf(m, (*rdkConf)(cConf)) 196 if err != nil { 197 C.rd_kafka_conf_destroy(cConf) 198 return nil, err 199 } 200 return cConf, nil 201 } 202 203 // get finds key in the configmap and returns its value. 204 // If the key is not found defval is returned. 205 // If the key is found but the type is mismatched an error is returned. 206 func (m ConfigMap) get(key string, defval ConfigValue) (ConfigValue, error) { 207 if strings.HasPrefix(key, "{topic}.") { 208 defconfCv, found := m["default.topic.config"] 209 if !found { 210 return defval, nil 211 } 212 return defconfCv.(ConfigMap).get(strings.TrimPrefix(key, "{topic}."), defval) 213 } 214 215 v, ok := m[key] 216 if !ok { 217 return defval, nil 218 } 219 220 if defval != nil && reflect.TypeOf(defval) != reflect.TypeOf(v) { 221 return nil, newErrorFromString(ErrInvalidArg, fmt.Sprintf("%s expects type %T, not %T", key, defval, v)) 222 } 223 224 return v, nil 225 } 226 227 // extract performs a get() and if found deletes the key. 228 func (m ConfigMap) extract(key string, defval ConfigValue) (ConfigValue, error) { 229 230 v, err := m.get(key, defval) 231 if err != nil { 232 return nil, err 233 } 234 235 delete(m, key) 236 237 return v, nil 238 } 239 240 // extractLogConfig extracts generic go.logs.* configuration properties. 241 func (m ConfigMap) extractLogConfig() (logsChanEnable bool, logsChan chan LogEvent, err error) { 242 v, err := m.extract("go.logs.channel.enable", false) 243 if err != nil { 244 return 245 } 246 247 logsChanEnable = v.(bool) 248 249 v, err = m.extract("go.logs.channel", nil) 250 if err != nil { 251 return 252 } 253 254 if v != nil { 255 logsChan = v.(chan LogEvent) 256 } 257 258 if logsChanEnable { 259 // Tell librdkafka to forward logs to the log queue 260 m.Set("log.queue=true") 261 } 262 263 return 264 } 265 266 func (m ConfigMap) clone() ConfigMap { 267 m2 := make(ConfigMap) 268 for k, v := range m { 269 m2[k] = v 270 } 271 return m2 272 } 273 274 // Get finds the given key in the ConfigMap and returns its value. 275 // If the key is not found `defval` is returned. 276 // If the key is found but the type does not match that of `defval` (unless nil) 277 // an ErrInvalidArg error is returned. 278 func (m ConfigMap) Get(key string, defval ConfigValue) (ConfigValue, error) { 279 return m.get(key, defval) 280 }