William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 1 | package sarama |
| 2 | |
| 3 | import ( |
| 4 | "math/rand" |
| 5 | "sort" |
| 6 | "sync" |
| 7 | "time" |
| 8 | ) |
| 9 | |
| 10 | // Client is a generic Kafka client. It manages connections to one or more Kafka brokers. |
| 11 | // You MUST call Close() on a client to avoid leaks, it will not be garbage-collected |
| 12 | // automatically when it passes out of scope. It is safe to share a client amongst many |
| 13 | // users, however Kafka will process requests from a single client strictly in serial, |
| 14 | // so it is generally more efficient to use the default one client per producer/consumer. |
| 15 | type Client interface { |
| 16 | // Config returns the Config struct of the client. This struct should not be |
| 17 | // altered after it has been created. |
| 18 | Config() *Config |
| 19 | |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 20 | // Controller returns the cluster controller broker. It will return a |
| 21 | // locally cached value if it's available. You can call RefreshController |
| 22 | // to update the cached value. Requires Kafka 0.10 or higher. |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 23 | Controller() (*Broker, error) |
| 24 | |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 25 | // RefreshController retrieves the cluster controller from fresh metadata |
| 26 | // and stores it in the local cache. Requires Kafka 0.10 or higher. |
| 27 | RefreshController() (*Broker, error) |
| 28 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 29 | // Brokers returns the current set of active brokers as retrieved from cluster metadata. |
| 30 | Brokers() []*Broker |
| 31 | |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 32 | // Broker returns the active Broker if available for the broker ID. |
| 33 | Broker(brokerID int32) (*Broker, error) |
| 34 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 35 | // Topics returns the set of available topics as retrieved from cluster metadata. |
| 36 | Topics() ([]string, error) |
| 37 | |
| 38 | // Partitions returns the sorted list of all partition IDs for the given topic. |
| 39 | Partitions(topic string) ([]int32, error) |
| 40 | |
| 41 | // WritablePartitions returns the sorted list of all writable partition IDs for |
| 42 | // the given topic, where "writable" means "having a valid leader accepting |
| 43 | // writes". |
| 44 | WritablePartitions(topic string) ([]int32, error) |
| 45 | |
| 46 | // Leader returns the broker object that is the leader of the current |
| 47 | // topic/partition, as determined by querying the cluster metadata. |
| 48 | Leader(topic string, partitionID int32) (*Broker, error) |
| 49 | |
| 50 | // Replicas returns the set of all replica IDs for the given partition. |
| 51 | Replicas(topic string, partitionID int32) ([]int32, error) |
| 52 | |
| 53 | // InSyncReplicas returns the set of all in-sync replica IDs for the given |
| 54 | // partition. In-sync replicas are replicas which are fully caught up with |
| 55 | // the partition leader. |
| 56 | InSyncReplicas(topic string, partitionID int32) ([]int32, error) |
| 57 | |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 58 | // OfflineReplicas returns the set of all offline replica IDs for the given |
| 59 | // partition. Offline replicas are replicas which are offline |
| 60 | OfflineReplicas(topic string, partitionID int32) ([]int32, error) |
| 61 | |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 62 | // RefreshBrokers takes a list of addresses to be used as seed brokers. |
| 63 | // Existing broker connections are closed and the updated list of seed brokers |
| 64 | // will be used for the next metadata fetch. |
| 65 | RefreshBrokers(addrs []string) error |
| 66 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 67 | // RefreshMetadata takes a list of topics and queries the cluster to refresh the |
| 68 | // available metadata for those topics. If no topics are provided, it will refresh |
| 69 | // metadata for all topics. |
| 70 | RefreshMetadata(topics ...string) error |
| 71 | |
| 72 | // GetOffset queries the cluster to get the most recent available offset at the |
| 73 | // given time (in milliseconds) on the topic/partition combination. |
| 74 | // Time should be OffsetOldest for the earliest available offset, |
| 75 | // OffsetNewest for the offset of the message that will be produced next, or a time. |
| 76 | GetOffset(topic string, partitionID int32, time int64) (int64, error) |
| 77 | |
| 78 | // Coordinator returns the coordinating broker for a consumer group. It will |
| 79 | // return a locally cached value if it's available. You can call |
| 80 | // RefreshCoordinator to update the cached value. This function only works on |
| 81 | // Kafka 0.8.2 and higher. |
| 82 | Coordinator(consumerGroup string) (*Broker, error) |
| 83 | |
| 84 | // RefreshCoordinator retrieves the coordinator for a consumer group and stores it |
| 85 | // in local cache. This function only works on Kafka 0.8.2 and higher. |
| 86 | RefreshCoordinator(consumerGroup string) error |
| 87 | |
| 88 | // InitProducerID retrieves information required for Idempotent Producer |
| 89 | InitProducerID() (*InitProducerIDResponse, error) |
| 90 | |
| 91 | // Close shuts down all broker connections managed by this client. It is required |
| 92 | // to call this function before a client object passes out of scope, as it will |
| 93 | // otherwise leak memory. You must close any Producers or Consumers using a client |
| 94 | // before you close the client. |
| 95 | Close() error |
| 96 | |
| 97 | // Closed returns true if the client has already had Close called on it |
| 98 | Closed() bool |
| 99 | } |
| 100 | |
| 101 | const ( |
| 102 | // OffsetNewest stands for the log head offset, i.e. the offset that will be |
| 103 | // assigned to the next message that will be produced to the partition. You |
| 104 | // can send this to a client's GetOffset method to get this offset, or when |
| 105 | // calling ConsumePartition to start consuming new messages. |
| 106 | OffsetNewest int64 = -1 |
| 107 | // OffsetOldest stands for the oldest offset available on the broker for a |
| 108 | // partition. You can send this to a client's GetOffset method to get this |
| 109 | // offset, or when calling ConsumePartition to start consuming from the |
| 110 | // oldest offset that is still available on the broker. |
| 111 | OffsetOldest int64 = -2 |
| 112 | ) |
| 113 | |
| 114 | type client struct { |
| 115 | conf *Config |
| 116 | closer, closed chan none // for shutting down background metadata updater |
| 117 | |
| 118 | // the broker addresses given to us through the constructor are not guaranteed to be returned in |
| 119 | // the cluster metadata (I *think* it only returns brokers who are currently leading partitions?) |
| 120 | // so we store them separately |
| 121 | seedBrokers []*Broker |
| 122 | deadSeeds []*Broker |
| 123 | |
| 124 | controllerID int32 // cluster controller broker id |
| 125 | brokers map[int32]*Broker // maps broker ids to brokers |
| 126 | metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata |
| 127 | metadataTopics map[string]none // topics that need to collect metadata |
| 128 | coordinators map[string]int32 // Maps consumer group names to coordinating broker IDs |
| 129 | |
| 130 | // If the number of partitions is large, we can get some churn calling cachedPartitions, |
| 131 | // so the result is cached. It is important to update this value whenever metadata is changed |
| 132 | cachedPartitionsResults map[string][maxPartitionIndex][]int32 |
| 133 | |
| 134 | lock sync.RWMutex // protects access to the maps that hold cluster state. |
| 135 | } |
| 136 | |
| 137 | // NewClient creates a new Client. It connects to one of the given broker addresses |
| 138 | // and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot |
| 139 | // be retrieved from any of the given broker addresses, the client is not created. |
| 140 | func NewClient(addrs []string, conf *Config) (Client, error) { |
| 141 | Logger.Println("Initializing new client") |
| 142 | |
| 143 | if conf == nil { |
| 144 | conf = NewConfig() |
| 145 | } |
| 146 | |
| 147 | if err := conf.Validate(); err != nil { |
| 148 | return nil, err |
| 149 | } |
| 150 | |
| 151 | if len(addrs) < 1 { |
| 152 | return nil, ConfigurationError("You must provide at least one broker address") |
| 153 | } |
| 154 | |
| 155 | client := &client{ |
| 156 | conf: conf, |
| 157 | closer: make(chan none), |
| 158 | closed: make(chan none), |
| 159 | brokers: make(map[int32]*Broker), |
| 160 | metadata: make(map[string]map[int32]*PartitionMetadata), |
| 161 | metadataTopics: make(map[string]none), |
| 162 | cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32), |
| 163 | coordinators: make(map[string]int32), |
| 164 | } |
| 165 | |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 166 | client.randomizeSeedBrokers(addrs) |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 167 | |
| 168 | if conf.Metadata.Full { |
| 169 | // do an initial fetch of all cluster metadata by specifying an empty list of topics |
| 170 | err := client.RefreshMetadata() |
| 171 | switch err { |
| 172 | case nil: |
| 173 | break |
| 174 | case ErrLeaderNotAvailable, ErrReplicaNotAvailable, ErrTopicAuthorizationFailed, ErrClusterAuthorizationFailed: |
| 175 | // indicates that maybe part of the cluster is down, but is not fatal to creating the client |
| 176 | Logger.Println(err) |
| 177 | default: |
| 178 | close(client.closed) // we haven't started the background updater yet, so we have to do this manually |
| 179 | _ = client.Close() |
| 180 | return nil, err |
| 181 | } |
| 182 | } |
| 183 | go withRecover(client.backgroundMetadataUpdater) |
| 184 | |
| 185 | Logger.Println("Successfully initialized new client") |
| 186 | |
| 187 | return client, nil |
| 188 | } |
| 189 | |
| 190 | func (client *client) Config() *Config { |
| 191 | return client.conf |
| 192 | } |
| 193 | |
| 194 | func (client *client) Brokers() []*Broker { |
| 195 | client.lock.RLock() |
| 196 | defer client.lock.RUnlock() |
| 197 | brokers := make([]*Broker, 0, len(client.brokers)) |
| 198 | for _, broker := range client.brokers { |
| 199 | brokers = append(brokers, broker) |
| 200 | } |
| 201 | return brokers |
| 202 | } |
| 203 | |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 204 | func (client *client) Broker(brokerID int32) (*Broker, error) { |
| 205 | client.lock.RLock() |
| 206 | defer client.lock.RUnlock() |
| 207 | broker, ok := client.brokers[brokerID] |
| 208 | if !ok { |
| 209 | return nil, ErrBrokerNotFound |
| 210 | } |
| 211 | _ = broker.Open(client.conf) |
| 212 | return broker, nil |
| 213 | } |
| 214 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 215 | func (client *client) InitProducerID() (*InitProducerIDResponse, error) { |
| 216 | var err error |
| 217 | for broker := client.any(); broker != nil; broker = client.any() { |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 218 | req := &InitProducerIDRequest{} |
| 219 | |
| 220 | response, err := broker.InitProducerID(req) |
| 221 | switch err.(type) { |
| 222 | case nil: |
| 223 | return response, nil |
| 224 | default: |
| 225 | // some error, remove that broker and try again |
| 226 | Logger.Printf("Client got error from broker %d when issuing InitProducerID : %v\n", broker.ID(), err) |
| 227 | _ = broker.Close() |
| 228 | client.deregisterBroker(broker) |
| 229 | } |
| 230 | } |
| 231 | return nil, err |
| 232 | } |
| 233 | |
| 234 | func (client *client) Close() error { |
| 235 | if client.Closed() { |
| 236 | // Chances are this is being called from a defer() and the error will go unobserved |
| 237 | // so we go ahead and log the event in this case. |
| 238 | Logger.Printf("Close() called on already closed client") |
| 239 | return ErrClosedClient |
| 240 | } |
| 241 | |
| 242 | // shutdown and wait for the background thread before we take the lock, to avoid races |
| 243 | close(client.closer) |
| 244 | <-client.closed |
| 245 | |
| 246 | client.lock.Lock() |
| 247 | defer client.lock.Unlock() |
| 248 | Logger.Println("Closing Client") |
| 249 | |
| 250 | for _, broker := range client.brokers { |
| 251 | safeAsyncClose(broker) |
| 252 | } |
| 253 | |
| 254 | for _, broker := range client.seedBrokers { |
| 255 | safeAsyncClose(broker) |
| 256 | } |
| 257 | |
| 258 | client.brokers = nil |
| 259 | client.metadata = nil |
| 260 | client.metadataTopics = nil |
| 261 | |
| 262 | return nil |
| 263 | } |
| 264 | |
| 265 | func (client *client) Closed() bool { |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 266 | client.lock.RLock() |
| 267 | defer client.lock.RUnlock() |
| 268 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 269 | return client.brokers == nil |
| 270 | } |
| 271 | |
| 272 | func (client *client) Topics() ([]string, error) { |
| 273 | if client.Closed() { |
| 274 | return nil, ErrClosedClient |
| 275 | } |
| 276 | |
| 277 | client.lock.RLock() |
| 278 | defer client.lock.RUnlock() |
| 279 | |
| 280 | ret := make([]string, 0, len(client.metadata)) |
| 281 | for topic := range client.metadata { |
| 282 | ret = append(ret, topic) |
| 283 | } |
| 284 | |
| 285 | return ret, nil |
| 286 | } |
| 287 | |
| 288 | func (client *client) MetadataTopics() ([]string, error) { |
| 289 | if client.Closed() { |
| 290 | return nil, ErrClosedClient |
| 291 | } |
| 292 | |
| 293 | client.lock.RLock() |
| 294 | defer client.lock.RUnlock() |
| 295 | |
| 296 | ret := make([]string, 0, len(client.metadataTopics)) |
| 297 | for topic := range client.metadataTopics { |
| 298 | ret = append(ret, topic) |
| 299 | } |
| 300 | |
| 301 | return ret, nil |
| 302 | } |
| 303 | |
| 304 | func (client *client) Partitions(topic string) ([]int32, error) { |
| 305 | if client.Closed() { |
| 306 | return nil, ErrClosedClient |
| 307 | } |
| 308 | |
| 309 | partitions := client.cachedPartitions(topic, allPartitions) |
| 310 | |
| 311 | if len(partitions) == 0 { |
| 312 | err := client.RefreshMetadata(topic) |
| 313 | if err != nil { |
| 314 | return nil, err |
| 315 | } |
| 316 | partitions = client.cachedPartitions(topic, allPartitions) |
| 317 | } |
| 318 | |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 319 | // no partitions found after refresh metadata |
| 320 | if len(partitions) == 0 { |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 321 | return nil, ErrUnknownTopicOrPartition |
| 322 | } |
| 323 | |
| 324 | return partitions, nil |
| 325 | } |
| 326 | |
| 327 | func (client *client) WritablePartitions(topic string) ([]int32, error) { |
| 328 | if client.Closed() { |
| 329 | return nil, ErrClosedClient |
| 330 | } |
| 331 | |
| 332 | partitions := client.cachedPartitions(topic, writablePartitions) |
| 333 | |
| 334 | // len==0 catches when it's nil (no such topic) and the odd case when every single |
| 335 | // partition is undergoing leader election simultaneously. Callers have to be able to handle |
| 336 | // this function returning an empty slice (which is a valid return value) but catching it |
| 337 | // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers |
| 338 | // a metadata refresh as a nicety so callers can just try again and don't have to manually |
| 339 | // trigger a refresh (otherwise they'd just keep getting a stale cached copy). |
| 340 | if len(partitions) == 0 { |
| 341 | err := client.RefreshMetadata(topic) |
| 342 | if err != nil { |
| 343 | return nil, err |
| 344 | } |
| 345 | partitions = client.cachedPartitions(topic, writablePartitions) |
| 346 | } |
| 347 | |
| 348 | if partitions == nil { |
| 349 | return nil, ErrUnknownTopicOrPartition |
| 350 | } |
| 351 | |
| 352 | return partitions, nil |
| 353 | } |
| 354 | |
| 355 | func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) { |
| 356 | if client.Closed() { |
| 357 | return nil, ErrClosedClient |
| 358 | } |
| 359 | |
| 360 | metadata := client.cachedMetadata(topic, partitionID) |
| 361 | |
| 362 | if metadata == nil { |
| 363 | err := client.RefreshMetadata(topic) |
| 364 | if err != nil { |
| 365 | return nil, err |
| 366 | } |
| 367 | metadata = client.cachedMetadata(topic, partitionID) |
| 368 | } |
| 369 | |
| 370 | if metadata == nil { |
| 371 | return nil, ErrUnknownTopicOrPartition |
| 372 | } |
| 373 | |
| 374 | if metadata.Err == ErrReplicaNotAvailable { |
| 375 | return dupInt32Slice(metadata.Replicas), metadata.Err |
| 376 | } |
| 377 | return dupInt32Slice(metadata.Replicas), nil |
| 378 | } |
| 379 | |
| 380 | func (client *client) InSyncReplicas(topic string, partitionID int32) ([]int32, error) { |
| 381 | if client.Closed() { |
| 382 | return nil, ErrClosedClient |
| 383 | } |
| 384 | |
| 385 | metadata := client.cachedMetadata(topic, partitionID) |
| 386 | |
| 387 | if metadata == nil { |
| 388 | err := client.RefreshMetadata(topic) |
| 389 | if err != nil { |
| 390 | return nil, err |
| 391 | } |
| 392 | metadata = client.cachedMetadata(topic, partitionID) |
| 393 | } |
| 394 | |
| 395 | if metadata == nil { |
| 396 | return nil, ErrUnknownTopicOrPartition |
| 397 | } |
| 398 | |
| 399 | if metadata.Err == ErrReplicaNotAvailable { |
| 400 | return dupInt32Slice(metadata.Isr), metadata.Err |
| 401 | } |
| 402 | return dupInt32Slice(metadata.Isr), nil |
| 403 | } |
| 404 | |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 405 | func (client *client) OfflineReplicas(topic string, partitionID int32) ([]int32, error) { |
| 406 | if client.Closed() { |
| 407 | return nil, ErrClosedClient |
| 408 | } |
| 409 | |
| 410 | metadata := client.cachedMetadata(topic, partitionID) |
| 411 | |
| 412 | if metadata == nil { |
| 413 | err := client.RefreshMetadata(topic) |
| 414 | if err != nil { |
| 415 | return nil, err |
| 416 | } |
| 417 | metadata = client.cachedMetadata(topic, partitionID) |
| 418 | } |
| 419 | |
| 420 | if metadata == nil { |
| 421 | return nil, ErrUnknownTopicOrPartition |
| 422 | } |
| 423 | |
| 424 | if metadata.Err == ErrReplicaNotAvailable { |
| 425 | return dupInt32Slice(metadata.OfflineReplicas), metadata.Err |
| 426 | } |
| 427 | return dupInt32Slice(metadata.OfflineReplicas), nil |
| 428 | } |
| 429 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 430 | func (client *client) Leader(topic string, partitionID int32) (*Broker, error) { |
| 431 | if client.Closed() { |
| 432 | return nil, ErrClosedClient |
| 433 | } |
| 434 | |
| 435 | leader, err := client.cachedLeader(topic, partitionID) |
| 436 | |
| 437 | if leader == nil { |
| 438 | err = client.RefreshMetadata(topic) |
| 439 | if err != nil { |
| 440 | return nil, err |
| 441 | } |
| 442 | leader, err = client.cachedLeader(topic, partitionID) |
| 443 | } |
| 444 | |
| 445 | return leader, err |
| 446 | } |
| 447 | |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 448 | func (client *client) RefreshBrokers(addrs []string) error { |
| 449 | if client.Closed() { |
| 450 | return ErrClosedClient |
| 451 | } |
| 452 | |
| 453 | client.lock.Lock() |
| 454 | defer client.lock.Unlock() |
| 455 | |
| 456 | for _, broker := range client.brokers { |
| 457 | _ = broker.Close() |
| 458 | delete(client.brokers, broker.ID()) |
| 459 | } |
| 460 | |
| 461 | client.seedBrokers = nil |
| 462 | client.deadSeeds = nil |
| 463 | |
| 464 | client.randomizeSeedBrokers(addrs) |
| 465 | |
| 466 | return nil |
| 467 | } |
| 468 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 469 | func (client *client) RefreshMetadata(topics ...string) error { |
| 470 | if client.Closed() { |
| 471 | return ErrClosedClient |
| 472 | } |
| 473 | |
| 474 | // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper |
| 475 | // error. This handles the case by returning an error instead of sending it |
| 476 | // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310 |
| 477 | for _, topic := range topics { |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 478 | if topic == "" { |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 479 | return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return |
| 480 | } |
| 481 | } |
| 482 | |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 483 | deadline := time.Time{} |
| 484 | if client.conf.Metadata.Timeout > 0 { |
| 485 | deadline = time.Now().Add(client.conf.Metadata.Timeout) |
| 486 | } |
| 487 | return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max, deadline) |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 488 | } |
| 489 | |
| 490 | func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) { |
| 491 | if client.Closed() { |
| 492 | return -1, ErrClosedClient |
| 493 | } |
| 494 | |
| 495 | offset, err := client.getOffset(topic, partitionID, time) |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 496 | if err != nil { |
| 497 | if err := client.RefreshMetadata(topic); err != nil { |
| 498 | return -1, err |
| 499 | } |
| 500 | return client.getOffset(topic, partitionID, time) |
| 501 | } |
| 502 | |
| 503 | return offset, err |
| 504 | } |
| 505 | |
| 506 | func (client *client) Controller() (*Broker, error) { |
| 507 | if client.Closed() { |
| 508 | return nil, ErrClosedClient |
| 509 | } |
| 510 | |
| 511 | if !client.conf.Version.IsAtLeast(V0_10_0_0) { |
| 512 | return nil, ErrUnsupportedVersion |
| 513 | } |
| 514 | |
| 515 | controller := client.cachedController() |
| 516 | if controller == nil { |
| 517 | if err := client.refreshMetadata(); err != nil { |
| 518 | return nil, err |
| 519 | } |
| 520 | controller = client.cachedController() |
| 521 | } |
| 522 | |
| 523 | if controller == nil { |
| 524 | return nil, ErrControllerNotAvailable |
| 525 | } |
| 526 | |
| 527 | _ = controller.Open(client.conf) |
| 528 | return controller, nil |
| 529 | } |
| 530 | |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 531 | // deregisterController removes the cached controllerID |
| 532 | func (client *client) deregisterController() { |
| 533 | client.lock.Lock() |
| 534 | defer client.lock.Unlock() |
| 535 | delete(client.brokers, client.controllerID) |
| 536 | } |
| 537 | |
| 538 | // RefreshController retrieves the cluster controller from fresh metadata |
| 539 | // and stores it in the local cache. Requires Kafka 0.10 or higher. |
| 540 | func (client *client) RefreshController() (*Broker, error) { |
| 541 | if client.Closed() { |
| 542 | return nil, ErrClosedClient |
| 543 | } |
| 544 | |
| 545 | client.deregisterController() |
| 546 | |
| 547 | if err := client.refreshMetadata(); err != nil { |
| 548 | return nil, err |
| 549 | } |
| 550 | |
| 551 | controller := client.cachedController() |
| 552 | if controller == nil { |
| 553 | return nil, ErrControllerNotAvailable |
| 554 | } |
| 555 | |
| 556 | _ = controller.Open(client.conf) |
| 557 | return controller, nil |
| 558 | } |
| 559 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 560 | func (client *client) Coordinator(consumerGroup string) (*Broker, error) { |
| 561 | if client.Closed() { |
| 562 | return nil, ErrClosedClient |
| 563 | } |
| 564 | |
| 565 | coordinator := client.cachedCoordinator(consumerGroup) |
| 566 | |
| 567 | if coordinator == nil { |
| 568 | if err := client.RefreshCoordinator(consumerGroup); err != nil { |
| 569 | return nil, err |
| 570 | } |
| 571 | coordinator = client.cachedCoordinator(consumerGroup) |
| 572 | } |
| 573 | |
| 574 | if coordinator == nil { |
| 575 | return nil, ErrConsumerCoordinatorNotAvailable |
| 576 | } |
| 577 | |
| 578 | _ = coordinator.Open(client.conf) |
| 579 | return coordinator, nil |
| 580 | } |
| 581 | |
| 582 | func (client *client) RefreshCoordinator(consumerGroup string) error { |
| 583 | if client.Closed() { |
| 584 | return ErrClosedClient |
| 585 | } |
| 586 | |
| 587 | response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max) |
| 588 | if err != nil { |
| 589 | return err |
| 590 | } |
| 591 | |
| 592 | client.lock.Lock() |
| 593 | defer client.lock.Unlock() |
| 594 | client.registerBroker(response.Coordinator) |
| 595 | client.coordinators[consumerGroup] = response.Coordinator.ID() |
| 596 | return nil |
| 597 | } |
| 598 | |
| 599 | // private broker management helpers |
| 600 | |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 601 | func (client *client) randomizeSeedBrokers(addrs []string) { |
| 602 | random := rand.New(rand.NewSource(time.Now().UnixNano())) |
| 603 | for _, index := range random.Perm(len(addrs)) { |
| 604 | client.seedBrokers = append(client.seedBrokers, NewBroker(addrs[index])) |
| 605 | } |
| 606 | } |
| 607 | |
| 608 | func (client *client) updateBroker(brokers []*Broker) { |
| 609 | currentBroker := make(map[int32]*Broker, len(brokers)) |
| 610 | |
| 611 | for _, broker := range brokers { |
| 612 | currentBroker[broker.ID()] = broker |
| 613 | if client.brokers[broker.ID()] == nil { // add new broker |
| 614 | client.brokers[broker.ID()] = broker |
| 615 | Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr()) |
| 616 | } else if broker.Addr() != client.brokers[broker.ID()].Addr() { // replace broker with new address |
| 617 | safeAsyncClose(client.brokers[broker.ID()]) |
| 618 | client.brokers[broker.ID()] = broker |
| 619 | Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr()) |
| 620 | } |
| 621 | } |
| 622 | |
| 623 | for id, broker := range client.brokers { |
| 624 | if _, exist := currentBroker[id]; !exist { // remove old broker |
| 625 | safeAsyncClose(broker) |
| 626 | delete(client.brokers, id) |
| 627 | Logger.Printf("client/broker remove invalid broker #%d with %s", broker.ID(), broker.Addr()) |
| 628 | } |
| 629 | } |
| 630 | } |
| 631 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 632 | // registerBroker makes sure a broker received by a Metadata or Coordinator request is registered |
| 633 | // in the brokers map. It returns the broker that is registered, which may be the provided broker, |
| 634 | // or a previously registered Broker instance. You must hold the write lock before calling this function. |
| 635 | func (client *client) registerBroker(broker *Broker) { |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 636 | if client.brokers == nil { |
| 637 | Logger.Printf("cannot register broker #%d at %s, client already closed", broker.ID(), broker.Addr()) |
| 638 | return |
| 639 | } |
| 640 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 641 | if client.brokers[broker.ID()] == nil { |
| 642 | client.brokers[broker.ID()] = broker |
| 643 | Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr()) |
| 644 | } else if broker.Addr() != client.brokers[broker.ID()].Addr() { |
| 645 | safeAsyncClose(client.brokers[broker.ID()]) |
| 646 | client.brokers[broker.ID()] = broker |
| 647 | Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr()) |
| 648 | } |
| 649 | } |
| 650 | |
| 651 | // deregisterBroker removes a broker from the seedsBroker list, and if it's |
| 652 | // not the seedbroker, removes it from brokers map completely. |
| 653 | func (client *client) deregisterBroker(broker *Broker) { |
| 654 | client.lock.Lock() |
| 655 | defer client.lock.Unlock() |
| 656 | |
| 657 | if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] { |
| 658 | client.deadSeeds = append(client.deadSeeds, broker) |
| 659 | client.seedBrokers = client.seedBrokers[1:] |
| 660 | } else { |
| 661 | // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever, |
| 662 | // but we really shouldn't have to; once that loop is made better this case can be |
| 663 | // removed, and the function generally can be renamed from `deregisterBroker` to |
| 664 | // `nextSeedBroker` or something |
| 665 | Logger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr()) |
| 666 | delete(client.brokers, broker.ID()) |
| 667 | } |
| 668 | } |
| 669 | |
| 670 | func (client *client) resurrectDeadBrokers() { |
| 671 | client.lock.Lock() |
| 672 | defer client.lock.Unlock() |
| 673 | |
| 674 | Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds)) |
| 675 | client.seedBrokers = append(client.seedBrokers, client.deadSeeds...) |
| 676 | client.deadSeeds = nil |
| 677 | } |
| 678 | |
| 679 | func (client *client) any() *Broker { |
| 680 | client.lock.RLock() |
| 681 | defer client.lock.RUnlock() |
| 682 | |
| 683 | if len(client.seedBrokers) > 0 { |
| 684 | _ = client.seedBrokers[0].Open(client.conf) |
| 685 | return client.seedBrokers[0] |
| 686 | } |
| 687 | |
| 688 | // not guaranteed to be random *or* deterministic |
| 689 | for _, broker := range client.brokers { |
| 690 | _ = broker.Open(client.conf) |
| 691 | return broker |
| 692 | } |
| 693 | |
| 694 | return nil |
| 695 | } |
| 696 | |
| 697 | // private caching/lazy metadata helpers |
| 698 | |
| 699 | type partitionType int |
| 700 | |
| 701 | const ( |
| 702 | allPartitions partitionType = iota |
| 703 | writablePartitions |
| 704 | // If you add any more types, update the partition cache in update() |
| 705 | |
| 706 | // Ensure this is the last partition type value |
| 707 | maxPartitionIndex |
| 708 | ) |
| 709 | |
| 710 | func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata { |
| 711 | client.lock.RLock() |
| 712 | defer client.lock.RUnlock() |
| 713 | |
| 714 | partitions := client.metadata[topic] |
| 715 | if partitions != nil { |
| 716 | return partitions[partitionID] |
| 717 | } |
| 718 | |
| 719 | return nil |
| 720 | } |
| 721 | |
| 722 | func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 { |
| 723 | client.lock.RLock() |
| 724 | defer client.lock.RUnlock() |
| 725 | |
| 726 | partitions, exists := client.cachedPartitionsResults[topic] |
| 727 | |
| 728 | if !exists { |
| 729 | return nil |
| 730 | } |
| 731 | return partitions[partitionSet] |
| 732 | } |
| 733 | |
| 734 | func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 { |
| 735 | partitions := client.metadata[topic] |
| 736 | |
| 737 | if partitions == nil { |
| 738 | return nil |
| 739 | } |
| 740 | |
| 741 | ret := make([]int32, 0, len(partitions)) |
| 742 | for _, partition := range partitions { |
| 743 | if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable { |
| 744 | continue |
| 745 | } |
| 746 | ret = append(ret, partition.ID) |
| 747 | } |
| 748 | |
| 749 | sort.Sort(int32Slice(ret)) |
| 750 | return ret |
| 751 | } |
| 752 | |
| 753 | func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) { |
| 754 | client.lock.RLock() |
| 755 | defer client.lock.RUnlock() |
| 756 | |
| 757 | partitions := client.metadata[topic] |
| 758 | if partitions != nil { |
| 759 | metadata, ok := partitions[partitionID] |
| 760 | if ok { |
| 761 | if metadata.Err == ErrLeaderNotAvailable { |
| 762 | return nil, ErrLeaderNotAvailable |
| 763 | } |
| 764 | b := client.brokers[metadata.Leader] |
| 765 | if b == nil { |
| 766 | return nil, ErrLeaderNotAvailable |
| 767 | } |
| 768 | _ = b.Open(client.conf) |
| 769 | return b, nil |
| 770 | } |
| 771 | } |
| 772 | |
| 773 | return nil, ErrUnknownTopicOrPartition |
| 774 | } |
| 775 | |
| 776 | func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) { |
| 777 | broker, err := client.Leader(topic, partitionID) |
| 778 | if err != nil { |
| 779 | return -1, err |
| 780 | } |
| 781 | |
| 782 | request := &OffsetRequest{} |
| 783 | if client.conf.Version.IsAtLeast(V0_10_1_0) { |
| 784 | request.Version = 1 |
| 785 | } |
| 786 | request.AddBlock(topic, partitionID, time, 1) |
| 787 | |
| 788 | response, err := broker.GetAvailableOffsets(request) |
| 789 | if err != nil { |
| 790 | _ = broker.Close() |
| 791 | return -1, err |
| 792 | } |
| 793 | |
| 794 | block := response.GetBlock(topic, partitionID) |
| 795 | if block == nil { |
| 796 | _ = broker.Close() |
| 797 | return -1, ErrIncompleteResponse |
| 798 | } |
| 799 | if block.Err != ErrNoError { |
| 800 | return -1, block.Err |
| 801 | } |
| 802 | if len(block.Offsets) != 1 { |
| 803 | return -1, ErrOffsetOutOfRange |
| 804 | } |
| 805 | |
| 806 | return block.Offsets[0], nil |
| 807 | } |
| 808 | |
| 809 | // core metadata update logic |
| 810 | |
| 811 | func (client *client) backgroundMetadataUpdater() { |
| 812 | defer close(client.closed) |
| 813 | |
| 814 | if client.conf.Metadata.RefreshFrequency == time.Duration(0) { |
| 815 | return |
| 816 | } |
| 817 | |
| 818 | ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency) |
| 819 | defer ticker.Stop() |
| 820 | |
| 821 | for { |
| 822 | select { |
| 823 | case <-ticker.C: |
| 824 | if err := client.refreshMetadata(); err != nil { |
| 825 | Logger.Println("Client background metadata update:", err) |
| 826 | } |
| 827 | case <-client.closer: |
| 828 | return |
| 829 | } |
| 830 | } |
| 831 | } |
| 832 | |
| 833 | func (client *client) refreshMetadata() error { |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 834 | var topics []string |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 835 | |
| 836 | if !client.conf.Metadata.Full { |
| 837 | if specificTopics, err := client.MetadataTopics(); err != nil { |
| 838 | return err |
| 839 | } else if len(specificTopics) == 0 { |
| 840 | return ErrNoTopicsToUpdateMetadata |
| 841 | } else { |
| 842 | topics = specificTopics |
| 843 | } |
| 844 | } |
| 845 | |
| 846 | if err := client.RefreshMetadata(topics...); err != nil { |
| 847 | return err |
| 848 | } |
| 849 | |
| 850 | return nil |
| 851 | } |
| 852 | |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 853 | func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int, deadline time.Time) error { |
| 854 | pastDeadline := func(backoff time.Duration) bool { |
| 855 | if !deadline.IsZero() && time.Now().Add(backoff).After(deadline) { |
| 856 | // we are past the deadline |
| 857 | return true |
| 858 | } |
| 859 | return false |
| 860 | } |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 861 | retry := func(err error) error { |
| 862 | if attemptsRemaining > 0 { |
| 863 | backoff := client.computeBackoff(attemptsRemaining) |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 864 | if pastDeadline(backoff) { |
| 865 | Logger.Println("client/metadata skipping last retries as we would go past the metadata timeout") |
| 866 | return err |
| 867 | } |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 868 | Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining) |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 869 | if backoff > 0 { |
| 870 | time.Sleep(backoff) |
| 871 | } |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 872 | return client.tryRefreshMetadata(topics, attemptsRemaining-1, deadline) |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 873 | } |
| 874 | return err |
| 875 | } |
| 876 | |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 877 | broker := client.any() |
| 878 | for ; broker != nil && !pastDeadline(0); broker = client.any() { |
| 879 | allowAutoTopicCreation := true |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 880 | if len(topics) > 0 { |
| 881 | Logger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr) |
| 882 | } else { |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 883 | allowAutoTopicCreation = false |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 884 | Logger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr) |
| 885 | } |
| 886 | |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 887 | req := &MetadataRequest{Topics: topics, AllowAutoTopicCreation: allowAutoTopicCreation} |
| 888 | if client.conf.Version.IsAtLeast(V1_0_0_0) { |
| 889 | req.Version = 5 |
| 890 | } else if client.conf.Version.IsAtLeast(V0_10_0_0) { |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 891 | req.Version = 1 |
| 892 | } |
| 893 | response, err := broker.GetMetadata(req) |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 894 | switch err := err.(type) { |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 895 | case nil: |
| 896 | allKnownMetaData := len(topics) == 0 |
| 897 | // valid response, use it |
| 898 | shouldRetry, err := client.updateMetadata(response, allKnownMetaData) |
| 899 | if shouldRetry { |
| 900 | Logger.Println("client/metadata found some partitions to be leaderless") |
| 901 | return retry(err) // note: err can be nil |
| 902 | } |
| 903 | return err |
| 904 | |
| 905 | case PacketEncodingError: |
| 906 | // didn't even send, return the error |
| 907 | return err |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 908 | |
| 909 | case KError: |
| 910 | // if SASL auth error return as this _should_ be a non retryable err for all brokers |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 911 | if err == ErrSASLAuthenticationFailed { |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 912 | Logger.Println("client/metadata failed SASL authentication") |
| 913 | return err |
| 914 | } |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 915 | |
| 916 | if err == ErrTopicAuthorizationFailed { |
| 917 | Logger.Println("client is not authorized to access this topic. The topics were: ", topics) |
| 918 | return err |
| 919 | } |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 920 | // else remove that broker and try again |
| 921 | Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err) |
| 922 | _ = broker.Close() |
| 923 | client.deregisterBroker(broker) |
| 924 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 925 | default: |
| 926 | // some other error, remove that broker and try again |
| 927 | Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err) |
| 928 | _ = broker.Close() |
| 929 | client.deregisterBroker(broker) |
| 930 | } |
| 931 | } |
| 932 | |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 933 | if broker != nil { |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 934 | Logger.Printf("client/metadata not fetching metadata from broker %s as we would go past the metadata timeout\n", broker.addr) |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 935 | return retry(ErrOutOfBrokers) |
| 936 | } |
| 937 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 938 | Logger.Println("client/metadata no available broker to send metadata request to") |
| 939 | client.resurrectDeadBrokers() |
| 940 | return retry(ErrOutOfBrokers) |
| 941 | } |
| 942 | |
| 943 | // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable |
| 944 | func (client *client) updateMetadata(data *MetadataResponse, allKnownMetaData bool) (retry bool, err error) { |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 945 | if client.Closed() { |
| 946 | return |
| 947 | } |
| 948 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 949 | client.lock.Lock() |
| 950 | defer client.lock.Unlock() |
| 951 | |
| 952 | // For all the brokers we received: |
| 953 | // - if it is a new ID, save it |
| 954 | // - if it is an existing ID, but the address we have is stale, discard the old one and save it |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 955 | // - if some brokers is not exist in it, remove old broker |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 956 | // - otherwise ignore it, replacing our existing one would just bounce the connection |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 957 | client.updateBroker(data.Brokers) |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 958 | |
| 959 | client.controllerID = data.ControllerID |
| 960 | |
| 961 | if allKnownMetaData { |
| 962 | client.metadata = make(map[string]map[int32]*PartitionMetadata) |
| 963 | client.metadataTopics = make(map[string]none) |
| 964 | client.cachedPartitionsResults = make(map[string][maxPartitionIndex][]int32) |
| 965 | } |
| 966 | for _, topic := range data.Topics { |
| 967 | // topics must be added firstly to `metadataTopics` to guarantee that all |
| 968 | // requested topics must be recorded to keep them trackable for periodically |
| 969 | // metadata refresh. |
| 970 | if _, exists := client.metadataTopics[topic.Name]; !exists { |
| 971 | client.metadataTopics[topic.Name] = none{} |
| 972 | } |
| 973 | delete(client.metadata, topic.Name) |
| 974 | delete(client.cachedPartitionsResults, topic.Name) |
| 975 | |
| 976 | switch topic.Err { |
| 977 | case ErrNoError: |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 978 | // no-op |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 979 | case ErrInvalidTopic, ErrTopicAuthorizationFailed: // don't retry, don't store partial results |
| 980 | err = topic.Err |
| 981 | continue |
| 982 | case ErrUnknownTopicOrPartition: // retry, do not store partial partition results |
| 983 | err = topic.Err |
| 984 | retry = true |
| 985 | continue |
| 986 | case ErrLeaderNotAvailable: // retry, but store partial partition results |
| 987 | retry = true |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 988 | default: // don't retry, don't store partial results |
| 989 | Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err) |
| 990 | err = topic.Err |
| 991 | continue |
| 992 | } |
| 993 | |
| 994 | client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions)) |
| 995 | for _, partition := range topic.Partitions { |
| 996 | client.metadata[topic.Name][partition.ID] = partition |
| 997 | if partition.Err == ErrLeaderNotAvailable { |
| 998 | retry = true |
| 999 | } |
| 1000 | } |
| 1001 | |
| 1002 | var partitionCache [maxPartitionIndex][]int32 |
| 1003 | partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions) |
| 1004 | partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions) |
| 1005 | client.cachedPartitionsResults[topic.Name] = partitionCache |
| 1006 | } |
| 1007 | |
| 1008 | return |
| 1009 | } |
| 1010 | |
| 1011 | func (client *client) cachedCoordinator(consumerGroup string) *Broker { |
| 1012 | client.lock.RLock() |
| 1013 | defer client.lock.RUnlock() |
| 1014 | if coordinatorID, ok := client.coordinators[consumerGroup]; ok { |
| 1015 | return client.brokers[coordinatorID] |
| 1016 | } |
| 1017 | return nil |
| 1018 | } |
| 1019 | |
| 1020 | func (client *client) cachedController() *Broker { |
| 1021 | client.lock.RLock() |
| 1022 | defer client.lock.RUnlock() |
| 1023 | |
| 1024 | return client.brokers[client.controllerID] |
| 1025 | } |
| 1026 | |
| 1027 | func (client *client) computeBackoff(attemptsRemaining int) time.Duration { |
| 1028 | if client.conf.Metadata.Retry.BackoffFunc != nil { |
| 1029 | maxRetries := client.conf.Metadata.Retry.Max |
| 1030 | retries := maxRetries - attemptsRemaining |
| 1031 | return client.conf.Metadata.Retry.BackoffFunc(retries, maxRetries) |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 1032 | } |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 1033 | return client.conf.Metadata.Retry.Backoff |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 1034 | } |
| 1035 | |
| 1036 | func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*FindCoordinatorResponse, error) { |
| 1037 | retry := func(err error) (*FindCoordinatorResponse, error) { |
| 1038 | if attemptsRemaining > 0 { |
| 1039 | backoff := client.computeBackoff(attemptsRemaining) |
| 1040 | Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining) |
| 1041 | time.Sleep(backoff) |
| 1042 | return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1) |
| 1043 | } |
| 1044 | return nil, err |
| 1045 | } |
| 1046 | |
| 1047 | for broker := client.any(); broker != nil; broker = client.any() { |
| 1048 | Logger.Printf("client/coordinator requesting coordinator for consumergroup %s from %s\n", consumerGroup, broker.Addr()) |
| 1049 | |
| 1050 | request := new(FindCoordinatorRequest) |
| 1051 | request.CoordinatorKey = consumerGroup |
| 1052 | request.CoordinatorType = CoordinatorGroup |
| 1053 | |
| 1054 | response, err := broker.FindCoordinator(request) |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 1055 | if err != nil { |
| 1056 | Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err) |
| 1057 | |
| 1058 | switch err.(type) { |
| 1059 | case PacketEncodingError: |
| 1060 | return nil, err |
| 1061 | default: |
| 1062 | _ = broker.Close() |
| 1063 | client.deregisterBroker(broker) |
| 1064 | continue |
| 1065 | } |
| 1066 | } |
| 1067 | |
| 1068 | switch response.Err { |
| 1069 | case ErrNoError: |
| 1070 | Logger.Printf("client/coordinator coordinator for consumergroup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr()) |
| 1071 | return response, nil |
| 1072 | |
| 1073 | case ErrConsumerCoordinatorNotAvailable: |
| 1074 | Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup) |
| 1075 | |
| 1076 | // This is very ugly, but this scenario will only happen once per cluster. |
| 1077 | // The __consumer_offsets topic only has to be created one time. |
| 1078 | // The number of partitions not configurable, but partition 0 should always exist. |
| 1079 | if _, err := client.Leader("__consumer_offsets", 0); err != nil { |
| 1080 | Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n") |
| 1081 | time.Sleep(2 * time.Second) |
| 1082 | } |
| 1083 | |
| 1084 | return retry(ErrConsumerCoordinatorNotAvailable) |
khenaidoo | 106c61a | 2021-08-11 18:05:46 -0400 | [diff] [blame] | 1085 | case ErrGroupAuthorizationFailed: |
| 1086 | Logger.Printf("client was not authorized to access group %s while attempting to find coordinator", consumerGroup) |
| 1087 | return retry(ErrGroupAuthorizationFailed) |
| 1088 | |
William Kurkian | ea86948 | 2019-04-09 15:16:11 -0400 | [diff] [blame] | 1089 | default: |
| 1090 | return nil, response.Err |
| 1091 | } |
| 1092 | } |
| 1093 | |
| 1094 | Logger.Println("client/coordinator no available broker to send consumer metadata request to") |
| 1095 | client.resurrectDeadBrokers() |
| 1096 | return retry(ErrOutOfBrokers) |
| 1097 | } |
Abhilash S.L | 3b49463 | 2019-07-16 15:51:09 +0530 | [diff] [blame] | 1098 | |
| 1099 | // nopCloserClient embeds an existing Client, but disables |
| 1100 | // the Close method (yet all other methods pass |
| 1101 | // through unchanged). This is for use in larger structs |
| 1102 | // where it is undesirable to close the client that was |
| 1103 | // passed in by the caller. |
| 1104 | type nopCloserClient struct { |
| 1105 | Client |
| 1106 | } |
| 1107 | |
| 1108 | // Close intercepts and purposely does not call the underlying |
| 1109 | // client's Close() method. |
| 1110 | func (ncc *nopCloserClient) Close() error { |
| 1111 | return nil |
| 1112 | } |