functional_producer_test.go 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400
  1. package sarama
  2. import (
  3. "fmt"
  4. "os"
  5. "strings"
  6. "sync"
  7. "testing"
  8. "time"
  9. toxiproxy "github.com/Shopify/toxiproxy/client"
  10. "github.com/rcrowley/go-metrics"
  11. )
  12. const TestBatchSize = 1000
  13. func TestFuncProducing(t *testing.T) {
  14. config := NewConfig()
  15. testProducingMessages(t, config)
  16. }
  17. func TestFuncProducingGzip(t *testing.T) {
  18. config := NewConfig()
  19. config.Producer.Compression = CompressionGZIP
  20. testProducingMessages(t, config)
  21. }
  22. func TestFuncProducingSnappy(t *testing.T) {
  23. config := NewConfig()
  24. config.Producer.Compression = CompressionSnappy
  25. testProducingMessages(t, config)
  26. }
  27. func TestFuncProducingNoResponse(t *testing.T) {
  28. config := NewConfig()
  29. config.Producer.RequiredAcks = NoResponse
  30. testProducingMessages(t, config)
  31. }
  32. func TestFuncProducingFlushing(t *testing.T) {
  33. config := NewConfig()
  34. config.Producer.Flush.Messages = TestBatchSize / 8
  35. config.Producer.Flush.Frequency = 250 * time.Millisecond
  36. testProducingMessages(t, config)
  37. }
  38. func TestFuncMultiPartitionProduce(t *testing.T) {
  39. setupFunctionalTest(t)
  40. defer teardownFunctionalTest(t)
  41. config := NewConfig()
  42. config.ChannelBufferSize = 20
  43. config.Producer.Flush.Frequency = 50 * time.Millisecond
  44. config.Producer.Flush.Messages = 200
  45. config.Producer.Return.Successes = true
  46. producer, err := NewSyncProducer(kafkaBrokers, config)
  47. if err != nil {
  48. t.Fatal(err)
  49. }
  50. var wg sync.WaitGroup
  51. wg.Add(TestBatchSize)
  52. for i := 1; i <= TestBatchSize; i++ {
  53. go func(i int) {
  54. defer wg.Done()
  55. msg := &ProducerMessage{Topic: "test.64", Key: nil, Value: StringEncoder(fmt.Sprintf("hur %d", i))}
  56. if _, _, err := producer.SendMessage(msg); err != nil {
  57. t.Error(i, err)
  58. }
  59. }(i)
  60. }
  61. wg.Wait()
  62. if err := producer.Close(); err != nil {
  63. t.Error(err)
  64. }
  65. }
  66. func TestFuncProducingToInvalidTopic(t *testing.T) {
  67. setupFunctionalTest(t)
  68. defer teardownFunctionalTest(t)
  69. producer, err := NewSyncProducer(kafkaBrokers, nil)
  70. if err != nil {
  71. t.Fatal(err)
  72. }
  73. if _, _, err := producer.SendMessage(&ProducerMessage{Topic: "in/valid"}); err != ErrUnknownTopicOrPartition {
  74. t.Error("Expected ErrUnknownTopicOrPartition, found", err)
  75. }
  76. if _, _, err := producer.SendMessage(&ProducerMessage{Topic: "in/valid"}); err != ErrUnknownTopicOrPartition {
  77. t.Error("Expected ErrUnknownTopicOrPartition, found", err)
  78. }
  79. safeClose(t, producer)
  80. }
  81. func TestFuncProducingIdempotentWithBrokerFailure(t *testing.T) {
  82. setupFunctionalTest(t)
  83. defer teardownFunctionalTest(t)
  84. config := NewConfig()
  85. config.Producer.Flush.Frequency = 250 * time.Millisecond
  86. config.Producer.Idempotent = true
  87. config.Producer.Timeout = 500 * time.Millisecond
  88. config.Producer.Retry.Max = 1
  89. config.Producer.Retry.Backoff = 500 * time.Millisecond
  90. config.Producer.Return.Successes = true
  91. config.Producer.Return.Errors = true
  92. config.Producer.RequiredAcks = WaitForAll
  93. config.Net.MaxOpenRequests = 1
  94. config.Version = V0_11_0_0
  95. producer, err := NewSyncProducer(kafkaBrokers, config)
  96. if err != nil {
  97. t.Fatal(err)
  98. }
  99. defer safeClose(t, producer)
  100. // Successfully publish a few messages
  101. for i := 0; i < 10; i++ {
  102. _, _, err = producer.SendMessage(&ProducerMessage{
  103. Topic: "test.1",
  104. Value: StringEncoder(fmt.Sprintf("%d message", i)),
  105. })
  106. if err != nil {
  107. t.Fatal(err)
  108. }
  109. }
  110. // break the brokers.
  111. for proxyName, proxy := range Proxies {
  112. if !strings.Contains(proxyName, "kafka") {
  113. continue
  114. }
  115. if err := proxy.Disable(); err != nil {
  116. t.Fatal(err)
  117. }
  118. }
  119. // This should fail hard now
  120. for i := 10; i < 20; i++ {
  121. _, _, err = producer.SendMessage(&ProducerMessage{
  122. Topic: "test.1",
  123. Value: StringEncoder(fmt.Sprintf("%d message", i)),
  124. })
  125. if err == nil {
  126. t.Fatal(err)
  127. }
  128. }
  129. // Now bring the proxy back up
  130. for proxyName, proxy := range Proxies {
  131. if !strings.Contains(proxyName, "kafka") {
  132. continue
  133. }
  134. if err := proxy.Enable(); err != nil {
  135. t.Fatal(err)
  136. }
  137. }
  138. // We should be able to publish again (once everything calms down)
  139. // (otherwise it times out)
  140. for {
  141. _, _, err = producer.SendMessage(&ProducerMessage{
  142. Topic: "test.1",
  143. Value: StringEncoder("comeback message"),
  144. })
  145. if err == nil {
  146. break
  147. }
  148. }
  149. }
  150. func testProducingMessages(t *testing.T, config *Config) {
  151. setupFunctionalTest(t)
  152. defer teardownFunctionalTest(t)
  153. // Configure some latency in order to properly validate the request latency metric
  154. for _, proxy := range Proxies {
  155. if _, err := proxy.AddToxic("", "latency", "", 1, toxiproxy.Attributes{"latency": 10}); err != nil {
  156. t.Fatal("Unable to configure latency toxicity", err)
  157. }
  158. }
  159. config.Producer.Return.Successes = true
  160. config.Consumer.Return.Errors = true
  161. client, err := NewClient(kafkaBrokers, config)
  162. if err != nil {
  163. t.Fatal(err)
  164. }
  165. // Keep in mind the current offset
  166. initialOffset, err := client.GetOffset("test.1", 0, OffsetNewest)
  167. if err != nil {
  168. t.Fatal(err)
  169. }
  170. producer, err := NewAsyncProducerFromClient(client)
  171. if err != nil {
  172. t.Fatal(err)
  173. }
  174. expectedResponses := TestBatchSize
  175. for i := 1; i <= TestBatchSize; {
  176. msg := &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder(fmt.Sprintf("testing %d", i))}
  177. select {
  178. case producer.Input() <- msg:
  179. i++
  180. case ret := <-producer.Errors():
  181. t.Fatal(ret.Err)
  182. case <-producer.Successes():
  183. expectedResponses--
  184. }
  185. }
  186. for expectedResponses > 0 {
  187. select {
  188. case ret := <-producer.Errors():
  189. t.Fatal(ret.Err)
  190. case <-producer.Successes():
  191. expectedResponses--
  192. }
  193. }
  194. safeClose(t, producer)
  195. // Validate producer metrics before using the consumer minus the offset request
  196. validateMetrics(t, client)
  197. master, err := NewConsumerFromClient(client)
  198. if err != nil {
  199. t.Fatal(err)
  200. }
  201. consumer, err := master.ConsumePartition("test.1", 0, initialOffset)
  202. if err != nil {
  203. t.Fatal(err)
  204. }
  205. for i := 1; i <= TestBatchSize; i++ {
  206. select {
  207. case <-time.After(10 * time.Second):
  208. t.Fatal("Not received any more events in the last 10 seconds.")
  209. case err := <-consumer.Errors():
  210. t.Error(err)
  211. case message := <-consumer.Messages():
  212. if string(message.Value) != fmt.Sprintf("testing %d", i) {
  213. t.Fatalf("Unexpected message with index %d: %s", i, message.Value)
  214. }
  215. }
  216. }
  217. safeClose(t, consumer)
  218. safeClose(t, client)
  219. }
  220. func validateMetrics(t *testing.T, client Client) {
  221. // Get the broker used by test1 topic
  222. var broker *Broker
  223. if partitions, err := client.Partitions("test.1"); err != nil {
  224. t.Error(err)
  225. } else {
  226. for _, partition := range partitions {
  227. if b, err := client.Leader("test.1", partition); err != nil {
  228. t.Error(err)
  229. } else {
  230. if broker != nil && b != broker {
  231. t.Fatal("Expected only one broker, got at least 2")
  232. }
  233. broker = b
  234. }
  235. }
  236. }
  237. metricValidators := newMetricValidators()
  238. noResponse := client.Config().Producer.RequiredAcks == NoResponse
  239. compressionEnabled := client.Config().Producer.Compression != CompressionNone
  240. // We are adding 10ms of latency to all requests with toxiproxy
  241. minRequestLatencyInMs := 10
  242. if noResponse {
  243. // but when we do not wait for a response it can be less than 1ms
  244. minRequestLatencyInMs = 0
  245. }
  246. // We read at least 1 byte from the broker
  247. metricValidators.registerForAllBrokers(broker, minCountMeterValidator("incoming-byte-rate", 1))
  248. // in at least 3 global requests (1 for metadata request, 1 for offset request and N for produce request)
  249. metricValidators.register(minCountMeterValidator("request-rate", 3))
  250. metricValidators.register(minCountHistogramValidator("request-size", 3))
  251. metricValidators.register(minValHistogramValidator("request-size", 1))
  252. // and at least 2 requests to the registered broker (offset + produces)
  253. metricValidators.registerForBroker(broker, minCountMeterValidator("request-rate", 2))
  254. metricValidators.registerForBroker(broker, minCountHistogramValidator("request-size", 2))
  255. metricValidators.registerForBroker(broker, minValHistogramValidator("request-size", 1))
  256. metricValidators.registerForBroker(broker, minValHistogramValidator("request-latency-in-ms", minRequestLatencyInMs))
  257. // We send at least 1 batch
  258. metricValidators.registerForGlobalAndTopic("test_1", minCountHistogramValidator("batch-size", 1))
  259. metricValidators.registerForGlobalAndTopic("test_1", minValHistogramValidator("batch-size", 1))
  260. if compressionEnabled {
  261. // We record compression ratios between [0.50,-10.00] (50-1000 with a histogram) for at least one "fake" record
  262. metricValidators.registerForGlobalAndTopic("test_1", minCountHistogramValidator("compression-ratio", 1))
  263. metricValidators.registerForGlobalAndTopic("test_1", minValHistogramValidator("compression-ratio", 50))
  264. metricValidators.registerForGlobalAndTopic("test_1", maxValHistogramValidator("compression-ratio", 1000))
  265. } else {
  266. // We record compression ratios of 1.00 (100 with a histogram) for every TestBatchSize record
  267. metricValidators.registerForGlobalAndTopic("test_1", countHistogramValidator("compression-ratio", TestBatchSize))
  268. metricValidators.registerForGlobalAndTopic("test_1", minValHistogramValidator("compression-ratio", 100))
  269. metricValidators.registerForGlobalAndTopic("test_1", maxValHistogramValidator("compression-ratio", 100))
  270. }
  271. // We send exactly TestBatchSize messages
  272. metricValidators.registerForGlobalAndTopic("test_1", countMeterValidator("record-send-rate", TestBatchSize))
  273. // We send at least one record per request
  274. metricValidators.registerForGlobalAndTopic("test_1", minCountHistogramValidator("records-per-request", 1))
  275. metricValidators.registerForGlobalAndTopic("test_1", minValHistogramValidator("records-per-request", 1))
  276. // We receive at least 1 byte from the broker
  277. metricValidators.registerForAllBrokers(broker, minCountMeterValidator("outgoing-byte-rate", 1))
  278. if noResponse {
  279. // in exactly 2 global responses (metadata + offset)
  280. metricValidators.register(countMeterValidator("response-rate", 2))
  281. metricValidators.register(minCountHistogramValidator("response-size", 2))
  282. // and exactly 1 offset response for the registered broker
  283. metricValidators.registerForBroker(broker, countMeterValidator("response-rate", 1))
  284. metricValidators.registerForBroker(broker, minCountHistogramValidator("response-size", 1))
  285. metricValidators.registerForBroker(broker, minValHistogramValidator("response-size", 1))
  286. } else {
  287. // in at least 3 global responses (metadata + offset + produces)
  288. metricValidators.register(minCountMeterValidator("response-rate", 3))
  289. metricValidators.register(minCountHistogramValidator("response-size", 3))
  290. // and at least 2 for the registered broker
  291. metricValidators.registerForBroker(broker, minCountMeterValidator("response-rate", 2))
  292. metricValidators.registerForBroker(broker, minCountHistogramValidator("response-size", 2))
  293. metricValidators.registerForBroker(broker, minValHistogramValidator("response-size", 1))
  294. }
  295. // There should be no requests in flight anymore
  296. metricValidators.registerForAllBrokers(broker, counterValidator("requests-in-flight", 0))
  297. // Run the validators
  298. metricValidators.run(t, client.Config().MetricRegistry)
  299. }
  300. // Benchmarks
  301. func BenchmarkProducerSmall(b *testing.B) {
  302. benchmarkProducer(b, nil, "test.64", ByteEncoder(make([]byte, 128)))
  303. }
  304. func BenchmarkProducerMedium(b *testing.B) {
  305. benchmarkProducer(b, nil, "test.64", ByteEncoder(make([]byte, 1024)))
  306. }
  307. func BenchmarkProducerLarge(b *testing.B) {
  308. benchmarkProducer(b, nil, "test.64", ByteEncoder(make([]byte, 8192)))
  309. }
  310. func BenchmarkProducerSmallSinglePartition(b *testing.B) {
  311. benchmarkProducer(b, nil, "test.1", ByteEncoder(make([]byte, 128)))
  312. }
  313. func BenchmarkProducerMediumSnappy(b *testing.B) {
  314. conf := NewConfig()
  315. conf.Producer.Compression = CompressionSnappy
  316. benchmarkProducer(b, conf, "test.1", ByteEncoder(make([]byte, 1024)))
  317. }
  318. func benchmarkProducer(b *testing.B, conf *Config, topic string, value Encoder) {
  319. setupFunctionalTest(b)
  320. defer teardownFunctionalTest(b)
  321. metricsDisable := os.Getenv("METRICS_DISABLE")
  322. if metricsDisable != "" {
  323. previousUseNilMetrics := metrics.UseNilMetrics
  324. Logger.Println("Disabling metrics using no-op implementation")
  325. metrics.UseNilMetrics = true
  326. // Restore previous setting
  327. defer func() {
  328. metrics.UseNilMetrics = previousUseNilMetrics
  329. }()
  330. }
  331. producer, err := NewAsyncProducer(kafkaBrokers, conf)
  332. if err != nil {
  333. b.Fatal(err)
  334. }
  335. b.ResetTimer()
  336. for i := 1; i <= b.N; {
  337. msg := &ProducerMessage{Topic: topic, Key: StringEncoder(fmt.Sprintf("%d", i)), Value: value}
  338. select {
  339. case producer.Input() <- msg:
  340. i++
  341. case ret := <-producer.Errors():
  342. b.Fatal(ret.Err)
  343. }
  344. }
  345. safeClose(b, producer)
  346. }