functional_producer_test.go 12 KB

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