瀏覽代碼

Group related client functions together

Also add some missing godoc. No functional changes.
Evan Huus 10 年之前
父節點
當前提交
2778be4164
共有 1 個文件被更改,包括 163 次插入154 次删除
  1. 163 154
      client.go

+ 163 - 154
client.go

@@ -14,6 +14,39 @@ type ClientConfig struct {
 	BackgroundRefreshFrequency time.Duration // How frequently the client will refresh the cluster metadata in the background. Defaults to 10 minutes. Set to 0 to disable.
 }
 
+// NewClientConfig creates a new ClientConfig instance with sensible defaults
+func NewClientConfig() *ClientConfig {
+	return &ClientConfig{
+		MetadataRetries:            3,
+		WaitForElection:            250 * time.Millisecond,
+		BackgroundRefreshFrequency: 10 * time.Minute,
+	}
+}
+
+// Validate checks a ClientConfig instance. This will return a
+// ConfigurationError if the specified values don't make sense.
+func (config *ClientConfig) Validate() error {
+	if config.MetadataRetries < 0 {
+		return ConfigurationError("Invalid MetadataRetries, must be >= 0")
+	}
+
+	if config.WaitForElection <= time.Duration(0) {
+		return ConfigurationError("Invalid WaitForElection, must be > 0")
+	}
+
+	if config.DefaultBrokerConf != nil {
+		if err := config.DefaultBrokerConf.Validate(); err != nil {
+			return err
+		}
+	}
+
+	if config.BackgroundRefreshFrequency < time.Duration(0) {
+		return ConfigurationError("Invalid BackgroundRefreshFrequency, must be >= 0")
+	}
+
+	return nil
+}
+
 // Client is a generic Kafka client. It manages connections to one or more Kafka brokers.
 // You MUST call Close() on a client to avoid leaks, it will not be garbage-collected
 // automatically when it passes out of scope. A single client can be safely shared by
@@ -115,6 +148,29 @@ func (client *Client) Close() error {
 	return nil
 }
 
+// Closed returns true if the client has already had Close called on it
+func (client *Client) Closed() bool {
+	return client.brokers == nil
+}
+
+// Topics returns the set of available topics as retrieved from the cluster metadata.
+func (client *Client) Topics() ([]string, error) {
+	// Check to see whether the client is closed
+	if client.Closed() {
+		return nil, ClosedClient
+	}
+
+	client.lock.RLock()
+	defer client.lock.RUnlock()
+
+	ret := make([]string, 0, len(client.metadata))
+	for topic := range client.metadata {
+		ret = append(ret, topic)
+	}
+
+	return ret, nil
+}
+
 // Partitions returns the sorted list of all partition IDs for the given topic.
 func (client *Client) Partitions(topic string) ([]int32, error) {
 	// Check to see whether the client is closed
@@ -170,42 +226,7 @@ func (client *Client) WritablePartitions(topic string) ([]int32, error) {
 	return partitions, nil
 }
 
-// Topics returns the set of available topics as retrieved from the cluster metadata.
-func (client *Client) Topics() ([]string, error) {
-	// Check to see whether the client is closed
-	if client.Closed() {
-		return nil, ClosedClient
-	}
-
-	client.lock.RLock()
-	defer client.lock.RUnlock()
-
-	ret := make([]string, 0, len(client.metadata))
-	for topic := range client.metadata {
-		ret = append(ret, topic)
-	}
-
-	return ret, nil
-}
-
-func (client *Client) getMetadata(topic string, partitionID int32) (*PartitionMetadata, error) {
-	metadata := client.cachedMetadata(topic, partitionID)
-
-	if metadata == nil {
-		err := client.RefreshTopicMetadata(topic)
-		if err != nil {
-			return nil, err
-		}
-		metadata = client.cachedMetadata(topic, partitionID)
-	}
-
-	if metadata == nil {
-		return nil, UnknownTopicOrPartition
-	}
-
-	return metadata, nil
-}
-
+// Replicas returns the set of all replica IDs for the given partition.
 func (client *Client) Replicas(topic string, partitionID int32) ([]int32, error) {
 	if client.Closed() {
 		return nil, ClosedClient
@@ -223,6 +244,9 @@ func (client *Client) Replicas(topic string, partitionID int32) ([]int32, error)
 	return dupeAndSort(metadata.Replicas), nil
 }
 
+// ReplicasInSync returns the set of all in-sync replica IDs for the given partition.
+// Note: kafka's metadata here is known to be stale in many cases, and should not generally be trusted.
+// This method should be considered effectively deprecated.
 func (client *Client) ReplicasInSync(topic string, partitionID int32) ([]int32, error) {
 	if client.Closed() {
 		return nil, ClosedClient
@@ -298,7 +322,7 @@ func (client *Client) GetOffset(topic string, partitionID int32, where OffsetTim
 	return block.Offsets[0], nil
 }
 
-// misc private helper functions
+// private broker management helpers
 
 // XXX: see https://github.com/Shopify/sarama/issues/15
 //      and https://github.com/Shopify/sarama/issues/23
@@ -328,69 +352,6 @@ func (client *Client) disconnectBroker(broker *Broker) {
 	safeAsyncClose(broker)
 }
 
-func (client *Client) Closed() bool {
-	return client.brokers == nil
-}
-
-func (client *Client) refreshMetadata(topics []string, retriesRemaining int) error {
-	// This function is a sort of central point for most functions that create new
-	// resources.  Check to see if we're dealing with a closed Client and error
-	// out immediately if so.
-	if client.Closed() {
-		return ClosedClient
-	}
-
-	// Kafka will throw exceptions on an empty topic and not return a proper
-	// error. This handles the case by returning an error instead of sending it
-	// off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
-	for _, topic := range topics {
-		if len(topic) == 0 {
-			return UnknownTopicOrPartition
-		}
-	}
-
-	for broker := client.any(); broker != nil; broker = client.any() {
-		Logger.Printf("Fetching metadata for %v from broker %s\n", topics, broker.addr)
-		response, err := broker.GetMetadata(client.id, &MetadataRequest{Topics: topics})
-
-		switch err {
-		case nil:
-			// valid response, use it
-			retry, err := client.update(response)
-
-			if len(retry) > 0 {
-				if retriesRemaining <= 0 {
-					Logger.Println("Some partitions are leaderless, but we're out of retries")
-					return nil
-				}
-				Logger.Printf("Some partitions are leaderless, waiting %dms for election... (%d retries remaining)\n", client.config.WaitForElection/time.Millisecond, retriesRemaining)
-				time.Sleep(client.config.WaitForElection) // wait for leader election
-				return client.refreshMetadata(retry, retriesRemaining-1)
-			}
-
-			return err
-		case EncodingError:
-			// didn't even send, return the error
-			return err
-		default:
-			// some other error, remove that broker and try again
-			Logger.Println("Error from broker while fetching metadata:", err)
-			client.disconnectBroker(broker)
-		}
-	}
-
-	Logger.Println("Out of available brokers.")
-
-	if retriesRemaining > 0 {
-		Logger.Printf("Resurrecting dead brokers after %dms... (%d retries remaining)\n", client.config.WaitForElection/time.Millisecond, retriesRemaining)
-		time.Sleep(client.config.WaitForElection)
-		client.resurrectDeadBrokers()
-		return client.refreshMetadata(topics, retriesRemaining-1)
-	}
-
-	return OutOfBrokers
-}
-
 func (client *Client) resurrectDeadBrokers() {
 	client.lock.Lock()
 	defer client.lock.Unlock()
@@ -424,26 +385,29 @@ func (client *Client) any() *Broker {
 	return nil
 }
 
-func (client *Client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
-	client.lock.RLock()
-	defer client.lock.RUnlock()
+// private caching/lazy metadata helpers
 
-	partitions := client.metadata[topic]
-	if partitions != nil {
-		metadata, ok := partitions[partitionID]
-		if ok {
-			if metadata.Err == LeaderNotAvailable {
-				return nil, LeaderNotAvailable
-			}
-			b := client.brokers[metadata.Leader]
-			if b == nil {
-				return nil, LeaderNotAvailable
-			}
-			return b, nil
+const (
+	allPartitions = iota
+	writablePartitions
+)
+
+func (client *Client) getMetadata(topic string, partitionID int32) (*PartitionMetadata, error) {
+	metadata := client.cachedMetadata(topic, partitionID)
+
+	if metadata == nil {
+		err := client.RefreshTopicMetadata(topic)
+		if err != nil {
+			return nil, err
 		}
+		metadata = client.cachedMetadata(topic, partitionID)
 	}
 
-	return nil, UnknownTopicOrPartition
+	if metadata == nil {
+		return nil, UnknownTopicOrPartition
+	}
+
+	return metadata, nil
 }
 
 func (client *Client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
@@ -458,11 +422,6 @@ func (client *Client) cachedMetadata(topic string, partitionID int32) *Partition
 	return nil
 }
 
-const (
-	allPartitions = iota
-	writablePartitions
-)
-
 func (client *Client) cachedPartitions(topic string, partitionSet int) []int32 {
 	client.lock.RLock()
 	defer client.lock.RUnlock()
@@ -484,6 +443,30 @@ func (client *Client) cachedPartitions(topic string, partitionSet int) []int32 {
 	return ret
 }
 
+func (client *Client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
+	client.lock.RLock()
+	defer client.lock.RUnlock()
+
+	partitions := client.metadata[topic]
+	if partitions != nil {
+		metadata, ok := partitions[partitionID]
+		if ok {
+			if metadata.Err == LeaderNotAvailable {
+				return nil, LeaderNotAvailable
+			}
+			b := client.brokers[metadata.Leader]
+			if b == nil {
+				return nil, LeaderNotAvailable
+			}
+			return b, nil
+		}
+	}
+
+	return nil, UnknownTopicOrPartition
+}
+
+// core metadata update logic
+
 func (client *Client) backgroundMetadataUpdater() {
 	if client.config.BackgroundRefreshFrequency == time.Duration(0) {
 		return
@@ -503,6 +486,65 @@ func (client *Client) backgroundMetadataUpdater() {
 	}
 }
 
+func (client *Client) refreshMetadata(topics []string, retriesRemaining int) error {
+	// This function is a sort of central point for most functions that create new
+	// resources.  Check to see if we're dealing with a closed Client and error
+	// out immediately if so.
+	if client.Closed() {
+		return ClosedClient
+	}
+
+	// Kafka will throw exceptions on an empty topic and not return a proper
+	// error. This handles the case by returning an error instead of sending it
+	// off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
+	for _, topic := range topics {
+		if len(topic) == 0 {
+			return UnknownTopicOrPartition
+		}
+	}
+
+	for broker := client.any(); broker != nil; broker = client.any() {
+		Logger.Printf("Fetching metadata for %v from broker %s\n", topics, broker.addr)
+		response, err := broker.GetMetadata(client.id, &MetadataRequest{Topics: topics})
+
+		switch err {
+		case nil:
+			// valid response, use it
+			retry, err := client.update(response)
+
+			if len(retry) > 0 {
+				if retriesRemaining <= 0 {
+					Logger.Println("Some partitions are leaderless, but we're out of retries")
+					return nil
+				}
+				Logger.Printf("Some partitions are leaderless, waiting %dms for election... (%d retries remaining)\n", client.config.WaitForElection/time.Millisecond, retriesRemaining)
+				time.Sleep(client.config.WaitForElection) // wait for leader election
+				return client.refreshMetadata(retry, retriesRemaining-1)
+			}
+
+			return err
+		case EncodingError:
+			// didn't even send, return the error
+			return err
+		default:
+			// some other error, remove that broker and try again
+			Logger.Println("Error from broker while fetching metadata:", err)
+			client.disconnectBroker(broker)
+		}
+	}
+
+	Logger.Println("Out of available brokers.")
+
+	if retriesRemaining > 0 {
+		Logger.Printf("Resurrecting dead brokers after %dms... (%d retries remaining)\n", client.config.WaitForElection/time.Millisecond, retriesRemaining)
+		time.Sleep(client.config.WaitForElection)
+		client.resurrectDeadBrokers()
+		return client.refreshMetadata(topics, retriesRemaining-1)
+	}
+
+	return OutOfBrokers
+}
+
 // if no fatal error, returns a list of topics that need retrying due to LeaderNotAvailable
 func (client *Client) update(data *MetadataResponse) ([]string, error) {
 	client.lock.Lock()
@@ -560,36 +602,3 @@ func (client *Client) update(data *MetadataResponse) ([]string, error) {
 	}
 	return ret, nil
 }
-
-// NewClientConfig creates a new ClientConfig instance with sensible defaults
-func NewClientConfig() *ClientConfig {
-	return &ClientConfig{
-		MetadataRetries:            3,
-		WaitForElection:            250 * time.Millisecond,
-		BackgroundRefreshFrequency: 10 * time.Minute,
-	}
-}
-
-// Validate checks a ClientConfig instance. This will return a
-// ConfigurationError if the specified values don't make sense.
-func (config *ClientConfig) Validate() error {
-	if config.MetadataRetries < 0 {
-		return ConfigurationError("Invalid MetadataRetries, must be >= 0")
-	}
-
-	if config.WaitForElection <= time.Duration(0) {
-		return ConfigurationError("Invalid WaitForElection, must be > 0")
-	}
-
-	if config.DefaultBrokerConf != nil {
-		if err := config.DefaultBrokerConf.Validate(); err != nil {
-			return err
-		}
-	}
-
-	if config.BackgroundRefreshFrequency < time.Duration(0) {
-		return ConfigurationError("Invalid BackgroundRefreshFrequency, must be >= 0")
-	}
-
-	return nil
-}