errors.go 9.9 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197
  1. package sarama
  2. import (
  3. "errors"
  4. "fmt"
  5. )
  6. // ErrOutOfBrokers is the error returned when the client has run out of brokers to talk to because all of them errored
  7. // or otherwise failed to respond.
  8. var ErrOutOfBrokers = errors.New("kafka: client has run out of available brokers to talk to (Is your cluster reachable?)")
  9. // ErrClosedClient is the error returned when a method is called on a client that has been closed.
  10. var ErrClosedClient = errors.New("kafka: tried to use a client that was closed")
  11. // ErrIncompleteResponse is the error returned when the server returns a syntactically valid response, but it does
  12. // not contain the expected information.
  13. var ErrIncompleteResponse = errors.New("kafka: response did not contain all the expected topic/partition blocks")
  14. // ErrInvalidPartition is the error returned when a partitioner returns an invalid partition index
  15. // (meaning one outside of the range [0...numPartitions-1]).
  16. var ErrInvalidPartition = errors.New("kafka: partitioner returned an invalid partition index")
  17. // ErrAlreadyConnected is the error returned when calling Open() on a Broker that is already connected or connecting.
  18. var ErrAlreadyConnected = errors.New("kafka: broker connection already initiated")
  19. // ErrNotConnected is the error returned when trying to send or call Close() on a Broker that is not connected.
  20. var ErrNotConnected = errors.New("kafka: broker not connected")
  21. // ErrInsufficientData is returned when decoding and the packet is truncated. This can be expected
  22. // when requesting messages, since as an optimization the server is allowed to return a partial message at the end
  23. // of the message set.
  24. var ErrInsufficientData = errors.New("kafka: insufficient data to decode packet, more bytes expected")
  25. // ErrShuttingDown is returned when a producer receives a message during shutdown.
  26. var ErrShuttingDown = errors.New("kafka: message received by producer in process of shutting down")
  27. // ErrMessageTooLarge is returned when the next message to consume is larger than the configured Consumer.Fetch.Max
  28. var ErrMessageTooLarge = errors.New("kafka: message is larger than Consumer.Fetch.Max")
  29. // PacketEncodingError is returned from a failure while encoding a Kafka packet. This can happen, for example,
  30. // if you try to encode a string over 2^15 characters in length, since Kafka's encoding rules do not permit that.
  31. type PacketEncodingError struct {
  32. Info string
  33. }
  34. func (err PacketEncodingError) Error() string {
  35. return fmt.Sprintf("kafka: error encoding packet: %s", err.Info)
  36. }
  37. // PacketDecodingError is returned when there was an error (other than truncated data) decoding the Kafka broker's response.
  38. // This can be a bad CRC or length field, or any other invalid value.
  39. type PacketDecodingError struct {
  40. Info string
  41. }
  42. func (err PacketDecodingError) Error() string {
  43. return fmt.Sprintf("kafka: error decoding packet: %s", err.Info)
  44. }
  45. // ConfigurationError is the type of error returned from a constructor (e.g. NewClient, or NewConsumer)
  46. // when the specified configuration is invalid.
  47. type ConfigurationError string
  48. func (err ConfigurationError) Error() string {
  49. return "kafka: invalid configuration (" + string(err) + ")"
  50. }
  51. // KError is the type of error that can be returned directly by the Kafka broker.
  52. // See https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-ErrorCodes
  53. type KError int16
  54. // Numeric error codes returned by the Kafka server.
  55. const (
  56. ErrNoError KError = 0
  57. ErrUnknown KError = -1
  58. ErrOffsetOutOfRange KError = 1
  59. ErrInvalidMessage KError = 2
  60. ErrUnknownTopicOrPartition KError = 3
  61. ErrInvalidMessageSize KError = 4
  62. ErrLeaderNotAvailable KError = 5
  63. ErrNotLeaderForPartition KError = 6
  64. ErrRequestTimedOut KError = 7
  65. ErrBrokerNotAvailable KError = 8
  66. ErrReplicaNotAvailable KError = 9
  67. ErrMessageSizeTooLarge KError = 10
  68. ErrStaleControllerEpochCode KError = 11
  69. ErrOffsetMetadataTooLarge KError = 12
  70. ErrNetworkException KError = 13
  71. ErrOffsetsLoadInProgress KError = 14
  72. ErrConsumerCoordinatorNotAvailable KError = 15
  73. ErrNotCoordinatorForConsumer KError = 16
  74. ErrInvalidTopic KError = 17
  75. ErrMessageSetSizeTooLarge KError = 18
  76. ErrNotEnoughReplicas KError = 19
  77. ErrNotEnoughReplicasAfterAppend KError = 20
  78. ErrInvalidRequiredAcks KError = 21
  79. ErrIllegalGeneration KError = 22
  80. ErrInconsistentGroupProtocol KError = 23
  81. ErrInvalidGroupId KError = 24
  82. ErrUnknownMemberId KError = 25
  83. ErrInvalidSessionTimeout KError = 26
  84. ErrRebalanceInProgress KError = 27
  85. ErrInvalidCommitOffsetSize KError = 28
  86. ErrTopicAuthorizationFailed KError = 29
  87. ErrGroupAuthorizationFailed KError = 30
  88. ErrClusterAuthorizationFailed KError = 31
  89. ErrInvalidTimestamp KError = 32
  90. ErrUnsupportedSASLMechanism KError = 33
  91. ErrIllegalSASLState KError = 34
  92. ErrUnsupportedVersion KError = 35
  93. ErrUnsupportedForMessageFormat KError = 43
  94. )
  95. func (err KError) Error() string {
  96. // Error messages stolen/adapted from
  97. // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol
  98. switch err {
  99. case ErrNoError:
  100. return "kafka server: Not an error, why are you printing me?"
  101. case ErrUnknown:
  102. return "kafka server: Unexpected (unknown?) server error."
  103. case ErrOffsetOutOfRange:
  104. return "kafka server: The requested offset is outside the range of offsets maintained by the server for the given topic/partition."
  105. case ErrInvalidMessage:
  106. return "kafka server: Message contents does not match its CRC."
  107. case ErrUnknownTopicOrPartition:
  108. return "kafka server: Request was for a topic or partition that does not exist on this broker."
  109. case ErrInvalidMessageSize:
  110. return "kafka server: The message has a negative size."
  111. case ErrLeaderNotAvailable:
  112. return "kafka server: In the middle of a leadership election, there is currently no leader for this partition and hence it is unavailable for writes."
  113. case ErrNotLeaderForPartition:
  114. return "kafka server: Tried to send a message to a replica that is not the leader for some partition. Your metadata is out of date."
  115. case ErrRequestTimedOut:
  116. return "kafka server: Request exceeded the user-specified time limit in the request."
  117. case ErrBrokerNotAvailable:
  118. return "kafka server: Broker not available. Not a client facing error, we should never receive this!!!"
  119. case ErrReplicaNotAvailable:
  120. return "kafka server: Replica information not available, one or more brokers are down."
  121. case ErrMessageSizeTooLarge:
  122. return "kafka server: Message was too large, server rejected it to avoid allocation error."
  123. case ErrStaleControllerEpochCode:
  124. return "kafka server: StaleControllerEpochCode (internal error code for broker-to-broker communication)."
  125. case ErrOffsetMetadataTooLarge:
  126. return "kafka server: Specified a string larger than the configured maximum for offset metadata."
  127. case ErrNetworkException:
  128. return "kafka server: The server disconnected before a response was received."
  129. case ErrOffsetsLoadInProgress:
  130. return "kafka server: The broker is still loading offsets after a leader change for that offset's topic partition."
  131. case ErrConsumerCoordinatorNotAvailable:
  132. return "kafka server: Offset's topic has not yet been created."
  133. case ErrNotCoordinatorForConsumer:
  134. return "kafka server: Request was for a consumer group that is not coordinated by this broker."
  135. case ErrInvalidTopic:
  136. return "kafka server: The request attempted to perform an operation on an invalid topic."
  137. case ErrMessageSetSizeTooLarge:
  138. return "kafka server: The request included message batch larger than the configured segment size on the server."
  139. case ErrNotEnoughReplicas:
  140. return "kafka server: Messages are rejected since there are fewer in-sync replicas than required."
  141. case ErrNotEnoughReplicasAfterAppend:
  142. return "kafka server: Messages are written to the log, but to fewer in-sync replicas than required."
  143. case ErrInvalidRequiredAcks:
  144. return "kafka server: The number of required acks is invalid (should be either -1, 0, or 1)."
  145. case ErrIllegalGeneration:
  146. return "kafka server: The provided generation id is not the current generation."
  147. case ErrInconsistentGroupProtocol:
  148. return "kafka server: The provider group protocol type is incompatible with the other members."
  149. case ErrInvalidGroupId:
  150. return "kafka server: The provided group id was empty."
  151. case ErrUnknownMemberId:
  152. return "kafka server: The provided member is not known in the current generation."
  153. case ErrInvalidSessionTimeout:
  154. return "kafka server: The provided session timeout is outside the allowed range."
  155. case ErrRebalanceInProgress:
  156. return "kafka server: A rebalance for the group is in progress. Please re-join the group."
  157. case ErrInvalidCommitOffsetSize:
  158. return "kafka server: The provided commit metadata was too large."
  159. case ErrTopicAuthorizationFailed:
  160. return "kafka server: The client is not authorized to access this topic."
  161. case ErrGroupAuthorizationFailed:
  162. return "kafka server: The client is not authorized to access this group."
  163. case ErrClusterAuthorizationFailed:
  164. return "kafka server: The client is not authorized to send this request type."
  165. case ErrInvalidTimestamp:
  166. return "kafka server: The timestamp of the message is out of acceptable range."
  167. case ErrUnsupportedSASLMechanism:
  168. return "kafka server: The broker does not support the requested SASL mechanism."
  169. case ErrIllegalSASLState:
  170. return "kafka server: Request is not valid given the current SASL state."
  171. case ErrUnsupportedVersion:
  172. return "kafka server: The version of API is not supported."
  173. case ErrUnsupportedForMessageFormat:
  174. return "kafka server: The requested operation is not supported by the message format version."
  175. }
  176. return fmt.Sprintf("Unknown error, how did this happen? Error code = %d", err)
  177. }