github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/producer_test.go (about) 1 /** 2 * Copyright 2016 Confluent Inc. 3 * 4 * Licensed under the Apache License, Version 2.0 (the "License"); 5 * you may not use this file except in compliance with the License. 6 * You may obtain a copy of the License at 7 * 8 * http://www.apache.org/licenses/LICENSE-2.0 9 * 10 * Unless required by applicable law or agreed to in writing, software 11 * distributed under the License is distributed on an "AS IS" BASIS, 12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 * See the License for the specific language governing permissions and 14 * limitations under the License. 15 */ 16 17 package kafka 18 19 import ( 20 "bytes" 21 "context" 22 "encoding/binary" 23 "encoding/json" 24 "reflect" 25 "strings" 26 "testing" 27 "time" 28 ) 29 30 // TestProducerAPIs dry-tests all Producer APIs, no broker is needed. 31 func TestProducerAPIs(t *testing.T) { 32 33 // expected message dr count on events channel 34 expMsgCnt := 0 35 p, err := NewProducer(&ConfigMap{ 36 "socket.timeout.ms": 10, 37 "message.timeout.ms": 10, 38 "go.delivery.report.fields": "key,value,headers"}) 39 if err != nil { 40 t.Fatalf("%s", err) 41 } 42 43 t.Logf("Producer %s", p) 44 45 drChan := make(chan Event, 10) 46 47 topic1 := "gotest" 48 topic2 := "gotest2" 49 50 // Produce with function, DR on passed drChan 51 err = p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic1, Partition: 0}, 52 Value: []byte("Own drChan"), Key: []byte("This is my key")}, 53 drChan) 54 if err != nil { 55 t.Errorf("Produce failed: %s", err) 56 } 57 58 // Produce with function, use default DR channel (Events) 59 err = p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic2, Partition: 0}, 60 Value: []byte("Events DR"), Key: []byte("This is my key")}, 61 nil) 62 if err != nil { 63 t.Errorf("Produce failed: %s", err) 64 } 65 expMsgCnt++ 66 67 // Produce with function and timestamp, 68 // success depends on librdkafka version 69 err = p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic2, Partition: 0}, Timestamp: time.Now()}, nil) 70 numver, strver := LibraryVersion() 71 t.Logf("Produce with timestamp on %s returned: %s", strver, err) 72 if numver < 0x00090400 { 73 if err == nil || err.(Error).Code() != ErrNotImplemented { 74 t.Errorf("Expected Produce with timestamp to fail with ErrNotImplemented on %s (0x%x), got: %s", strver, numver, err) 75 } 76 } else { 77 if err != nil { 78 t.Errorf("Produce with timestamp failed on %s: %s", strver, err) 79 } 80 } 81 if err == nil { 82 expMsgCnt++ 83 } 84 85 // Produce through ProducerChannel, uses default DR channel (Events), 86 // pass Opaque object. 87 myOpq := "My opaque" 88 p.ProduceChannel() <- &Message{TopicPartition: TopicPartition{Topic: &topic2, Partition: 0}, 89 Opaque: &myOpq, 90 Value: []byte("ProducerChannel"), Key: []byte("This is my key")} 91 expMsgCnt++ 92 93 // Len() will not report messages on private delivery report chans (our drChan for example), 94 // so expect at least 2 messages, not 3. 95 // And completely ignore the timestamp message. 96 if p.Len() < 2 { 97 t.Errorf("Expected at least 2 messages (+requests) in queue, only %d reported", p.Len()) 98 } 99 100 // Message Headers 101 varIntHeader := make([]byte, binary.MaxVarintLen64) 102 varIntLen := binary.PutVarint(varIntHeader, 123456789) 103 104 myHeaders := []Header{ 105 {"thisHdrIsNullOrNil", nil}, 106 {"empty", []byte("")}, 107 {"MyVarIntHeader", varIntHeader[:varIntLen]}, 108 {"mystring", []byte("This is a simple string")}, 109 } 110 111 p.ProduceChannel() <- &Message{TopicPartition: TopicPartition{Topic: &topic2, Partition: 0}, 112 Value: []byte("Headers"), 113 Headers: myHeaders} 114 expMsgCnt++ 115 116 // 117 // Now wait for messages to time out so that delivery reports are triggered 118 // 119 120 // drChan (1 message) 121 ev := <-drChan 122 m := ev.(*Message) 123 if string(m.Value) != "Own drChan" { 124 t.Errorf("DR for wrong message (wanted 'Own drChan'), got %s", 125 string(m.Value)) 126 } else if m.TopicPartition.Error == nil { 127 t.Errorf("Expected error for message") 128 } else { 129 t.Logf("Message %s", m.TopicPartition) 130 } 131 close(drChan) 132 133 // Events chan (3 messages and possibly events) 134 for msgCnt := 0; msgCnt < expMsgCnt; { 135 ev = <-p.Events() 136 switch e := ev.(type) { 137 case *Message: 138 msgCnt++ 139 if (string)(e.Value) == "ProducerChannel" { 140 s := e.Opaque.(*string) 141 if s != &myOpq { 142 t.Errorf("Opaque should point to %v, not %v", &myOpq, s) 143 } 144 if *s != myOpq { 145 t.Errorf("Opaque should be \"%s\", not \"%v\"", 146 myOpq, *s) 147 } 148 t.Logf("Message \"%s\" with opaque \"%s\"\n", 149 (string)(e.Value), *s) 150 151 } else if (string)(e.Value) == "Headers" { 152 if e.Opaque != nil { 153 t.Errorf("Message opaque should be nil, not %v", e.Opaque) 154 } 155 if !reflect.DeepEqual(e.Headers, myHeaders) { 156 t.Errorf("Message headers should be %v, not %v", myHeaders, e.Headers) 157 } 158 } else { 159 if e.Opaque != nil { 160 t.Errorf("Message opaque should be nil, not %v", e.Opaque) 161 } 162 } 163 default: 164 t.Logf("Ignored event %s", e) 165 } 166 } 167 168 r := p.Flush(2000) 169 if r > 0 { 170 t.Errorf("Expected empty queue after Flush, still has %d", r) 171 } 172 173 // OffsetsForTimes 174 offsets, err := p.OffsetsForTimes([]TopicPartition{{Topic: &topic2, Offset: 12345}}, 100) 175 t.Logf("OffsetsForTimes() returned Offsets %s and error %s\n", offsets, err) 176 if err == nil { 177 t.Errorf("OffsetsForTimes() should have failed\n") 178 } 179 if offsets != nil { 180 t.Errorf("OffsetsForTimes() failed but returned non-nil Offsets: %s\n", offsets) 181 } 182 } 183 184 // TestPurgeAPI test if messages are purged successfully 185 func TestPurgeAPI(t *testing.T) { 186 topic := "sometopic" 187 unreachableProducer, _ := NewProducer(&ConfigMap{ 188 "bootstrap.servers": "127.0.0.1:65533", 189 }) 190 purgeDrChan := make(chan Event) 191 err := unreachableProducer.Produce(&Message{ 192 TopicPartition: TopicPartition{ 193 Topic: &topic, 194 Partition: 0, 195 }, 196 Value: []byte("somevalue"), 197 }, purgeDrChan) 198 if err != nil { 199 t.Errorf("Produce failed: %s", err) 200 } 201 202 err = unreachableProducer.Purge(PurgeInFlight | PurgeQueue) 203 if err != nil { 204 t.Errorf("Failed to purge message: %s", err) 205 } 206 207 select { 208 case e := <-purgeDrChan: 209 purgedMessage := e.(*Message) 210 err = purgedMessage.TopicPartition.Error 211 if err != nil { 212 if err.(Error).Code() != ErrPurgeQueue { 213 t.Errorf("Unexpected error after purge api is called: %s", e) 214 } 215 } else { 216 t.Errorf("Purge should have triggered error report") 217 } 218 case <-time.After(1 * time.Second): 219 t.Errorf("No delivery report after purge api is called") 220 } 221 222 close(purgeDrChan) 223 } 224 225 // TestProducerBufferSafety verifies issue #24, passing any type of memory backed buffer 226 // (JSON in this case) to Produce() 227 func TestProducerBufferSafety(t *testing.T) { 228 229 p, err := NewProducer(&ConfigMap{ 230 "socket.timeout.ms": 10, 231 // Use deprecated default.topic.config here to verify 232 // it still works. 233 "default.topic.config": ConfigMap{"message.timeout.ms": 10}}) 234 if err != nil { 235 t.Fatalf("%s", err) 236 } 237 238 topic := "gotest" 239 value, _ := json.Marshal(struct{ M string }{M: "Hello Go!"}) 240 empty := []byte("") 241 242 // Try combinations of Value and Key: json value, empty, nil 243 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: value, Key: nil}, nil) 244 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: value, Key: value}, nil) 245 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: nil, Key: value}, nil) 246 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: nil, Key: nil}, nil) 247 248 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: empty, Key: nil}, nil) 249 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: empty, Key: empty}, nil) 250 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: nil, Key: empty}, nil) 251 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: value, Key: empty}, nil) 252 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: value, Key: value}, nil) 253 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: empty, Key: value}, nil) 254 255 // And Headers 256 p.Produce(&Message{TopicPartition: TopicPartition{Topic: &topic}, Value: empty, Key: value, 257 Headers: []Header{{"hdr", value}, {"hdr2", empty}, {"hdr3", nil}}}, nil) 258 259 p.Flush(100) 260 261 p.Close() 262 } 263 264 // TestProducerInvalidConfig verifies that invalid configuration is handled correctly. 265 func TestProducerInvalidConfig(t *testing.T) { 266 267 _, err := NewProducer(&ConfigMap{ 268 "delivery.report.only.error": true, 269 }) 270 if err == nil { 271 t.Fatalf("Expected NewProducer() to fail with delivery.report.only.error set") 272 } 273 } 274 275 func TestProducerOAuthBearerConfig(t *testing.T) { 276 myOAuthConfig := "scope=myscope principal=gotest" 277 278 p, err := NewProducer(&ConfigMap{ 279 "security.protocol": "SASL_PLAINTEXT", 280 "sasl.mechanisms": "OAUTHBEARER", 281 "sasl.oauthbearer.config": myOAuthConfig, 282 }) 283 if err != nil { 284 t.Fatalf("NewProducer failed: %s", err) 285 } 286 287 // Wait for initial OAuthBearerTokenRefresh and check 288 // that its SerializerConfig string is identical to myOAuthConfig 289 for { 290 ev := <-p.Events() 291 oatr, ok := ev.(OAuthBearerTokenRefresh) 292 if !ok { 293 continue 294 } 295 296 t.Logf("Got %s with SerializerConfig \"%s\"", oatr, oatr.Config) 297 298 if oatr.Config != myOAuthConfig { 299 t.Fatalf("%s: Expected .SerializerConfig to be %s, not %s", 300 oatr, myOAuthConfig, oatr.Config) 301 } 302 303 // Verify that we can set a token 304 err = p.SetOAuthBearerToken(OAuthBearerToken{ 305 TokenValue: "aaaa", 306 Expiration: time.Now().Add(time.Second * time.Duration(60)), 307 Principal: "gotest", 308 }) 309 if err != nil { 310 t.Fatalf("Failed to set token: %s", err) 311 } 312 313 // Verify that we can set a token refresh failure 314 err = p.SetOAuthBearerTokenFailure("A token failure test") 315 if err != nil { 316 t.Fatalf("Failed to set token failure: %s", err) 317 } 318 319 break 320 } 321 322 p.Close() 323 } 324 325 func TestProducerLog(t *testing.T) { 326 p, err := NewProducer(&ConfigMap{ 327 "debug": "all", 328 "go.logs.channel.enable": true, 329 "socket.timeout.ms": 10, 330 "default.topic.config": ConfigMap{"message.timeout.ms": 10}}) 331 if err != nil { 332 t.Fatalf("%s", err) 333 } 334 335 expectedLogs := map[struct { 336 tag string 337 message string 338 }]bool{ 339 {"INIT", "librdkafka"}: false, 340 } 341 342 go func() { 343 for { 344 select { 345 case log, ok := <-p.Logs(): 346 if !ok { 347 return 348 } 349 350 t.Log(log.String()) 351 352 for expectedLog, found := range expectedLogs { 353 if found { 354 continue 355 } 356 if log.Tag != expectedLog.tag { 357 continue 358 } 359 if strings.Contains(log.Message, expectedLog.message) { 360 expectedLogs[expectedLog] = true 361 } 362 } 363 } 364 } 365 }() 366 367 <-time.After(time.Second * 5) 368 p.Close() 369 370 for expectedLog, found := range expectedLogs { 371 if !found { 372 t.Errorf( 373 "Expected to find log with tag `%s' and message containing `%s',"+ 374 " but didn't find any.", 375 expectedLog.tag, 376 expectedLog.message) 377 } 378 } 379 } 380 381 // TestTransactionalAPI test the transactional producer API 382 func TestTransactionalAPI(t *testing.T) { 383 p, err := NewProducer(&ConfigMap{ 384 "bootstrap.servers": "127.0.0.1:65533", 385 "transactional.id": "test", 386 "transaction.timeout.ms": "4000", 387 }) 388 if err != nil { 389 t.Fatalf("Failed to create transactional producer: %v", err) 390 } 391 392 // 393 // Call InitTransactions() with explicit timeout and check that 394 // it times out accordingly. 395 // 396 maxDuration, err := time.ParseDuration("2s") 397 if err != nil { 398 t.Fatalf("%s", err) 399 } 400 ctx, cancel := context.WithTimeout(context.Background(), maxDuration) 401 defer cancel() 402 403 start := time.Now() 404 err = p.InitTransactions(ctx) 405 duration := time.Now().Sub(start).Seconds() 406 407 t.Logf("InitTransactions(%v) returned '%v' in %.2fs", 408 maxDuration, err, duration) 409 if err.(Error).Code() != ErrTimedOut { 410 t.Errorf("Expected ErrTimedOut, not %v", err) 411 } else if duration < maxDuration.Seconds()*0.8 || 412 duration > maxDuration.Seconds()*1.2 { 413 t.Errorf("InitTransactions() should have finished within "+ 414 "%.2f +-20%%, not %.2f", 415 maxDuration.Seconds(), duration) 416 } 417 418 // 419 // Call InitTransactions() without timeout, which makes it 420 // default to the transaction.timeout.ms. 421 // NOTE: cancelling the context currently does not work. 422 // 423 maxDuration, err = time.ParseDuration("4s") // transaction.tiemout.ms 424 if err != nil { 425 t.Fatalf("%s", err) 426 } 427 ctx = context.TODO() 428 429 start = time.Now() 430 err = p.InitTransactions(ctx) 431 duration = time.Now().Sub(start).Seconds() 432 433 t.Logf("InitTransactions() returned '%v' in %.2fs", err, duration) 434 if err.(Error).Code() != ErrTimedOut { 435 t.Errorf("Expected ErrTimedOut, not %v", err) 436 } else if duration < maxDuration.Seconds()*0.8 || 437 duration > maxDuration.Seconds()*1.2 { 438 t.Errorf("InitTransactions() should have finished within "+ 439 "%.2f +-20%%, not %.2f", 440 maxDuration.Seconds(), duration) 441 } 442 443 // And again with a nil context 444 start = time.Now() 445 err = p.InitTransactions(nil) 446 duration = time.Now().Sub(start).Seconds() 447 448 t.Logf("InitTransactions() returned '%v' in %.2fs", err, duration) 449 if err.(Error).Code() != ErrTimedOut { 450 t.Errorf("Expected ErrTimedOut, not %v", err) 451 } else if duration < maxDuration.Seconds()*0.8 || 452 duration > maxDuration.Seconds()*1.2 { 453 t.Errorf("InitTransactions() should have finished within "+ 454 "%.2f +-20%%, not %.2f", 455 maxDuration.Seconds(), duration) 456 } 457 458 // 459 // All sub-sequent APIs should fail (unless otherwise noted) 460 // since InitTransactions() has not succeeded. 461 // 462 maxDuration, err = time.ParseDuration("1s") // Should fail quickly, not by timeout 463 if err != nil { 464 t.Fatalf("%s", err) 465 } 466 ctx = context.TODO() 467 468 // Perform multiple iterations to make sure API behaviour is consistent. 469 for iter := 0; iter < 5; iter++ { 470 471 if iter == 4 { 472 // Last iteration, pass context as nil 473 ctx = nil 474 } 475 476 // BeginTransaction 477 what := "BeginTransaction" 478 start = time.Now() 479 err = p.BeginTransaction() 480 duration = time.Now().Sub(start).Seconds() 481 482 t.Logf("%s() returned '%v' in %.2fs", what, err, duration) 483 if err == nil || err.(Error).Code() == ErrTimedOut { 484 t.Errorf("Expected %s() to fail due to state, not %v", what, err) 485 } 486 487 // SendOffsetsToTransaction 488 what = "SendOffsetsToTransaction" 489 topic := "myTopic" 490 start = time.Now() 491 cgmd, err := NewTestConsumerGroupMetadata("myConsumerGroup") 492 if err != nil { 493 t.Fatalf("Failed to create group metadata: %v", err) 494 } 495 err = p.SendOffsetsToTransaction(ctx, 496 []TopicPartition{ 497 {Topic: &topic, Partition: 1, Offset: 123}, 498 {Topic: &topic, Partition: 0, Offset: 4567890}, 499 }, 500 cgmd) 501 duration = time.Now().Sub(start).Seconds() 502 503 t.Logf("%s() returned '%v' in %.2fs", what, err, duration) 504 if err == nil || err.(Error).Code() == ErrTimedOut { 505 t.Errorf("Expected %s() to fail due to state, not %v", what, err) 506 } 507 508 what = "SendOffsetsToTransaction(nil offsets)" 509 start = time.Now() 510 err = p.SendOffsetsToTransaction(ctx, nil, cgmd) 511 duration = time.Now().Sub(start).Seconds() 512 513 t.Logf("%s() returned '%v' in %.2fs", what, err, duration) 514 if err == nil || err.(Error).Code() != ErrInvalidArg { 515 t.Errorf("Expected %s() to fail due to bad args, not %v", what, err) 516 } 517 518 what = "SendOffsetsToTransaction(empty offsets, empty group)" 519 cgmdEmpty, err := NewTestConsumerGroupMetadata("") 520 if err != nil { 521 t.Fatalf("Failed to create group metadata: %v", err) 522 } 523 start = time.Now() 524 err = p.SendOffsetsToTransaction(ctx, []TopicPartition{}, cgmdEmpty) 525 duration = time.Now().Sub(start).Seconds() 526 527 t.Logf("%s() returned '%v' in %.2fs", what, err, duration) 528 if err != nil { 529 t.Errorf("Expected %s() to succeed as a no-op, but got %v", what, err) 530 } 531 532 what = "SendOffsetsToTransaction(empty offsets)" 533 start = time.Now() 534 err = p.SendOffsetsToTransaction(ctx, []TopicPartition{}, cgmd) 535 duration = time.Now().Sub(start).Seconds() 536 537 t.Logf("%s() returned '%v' in %.2fs", what, err, duration) 538 if err != nil { 539 t.Errorf("Expected %s() to succeed as a no-op, but got %v", what, err) 540 } 541 542 // AbortTransaction 543 what = "AbortTransaction" 544 start = time.Now() 545 err = p.AbortTransaction(ctx) 546 duration = time.Now().Sub(start).Seconds() 547 548 t.Logf("%s() returned '%v' in %.2fs", what, err, duration) 549 if err == nil || err.(Error).Code() == ErrTimedOut { 550 t.Errorf("Expected %s() to fail due to state, not %v", what, err) 551 } 552 553 // CommitTransaction 554 what = "CommitTransaction" 555 start = time.Now() 556 err = p.CommitTransaction(ctx) 557 duration = time.Now().Sub(start).Seconds() 558 559 t.Logf("%s() returned '%v' in %.2fs", what, err, duration) 560 if err == nil || err.(Error).Code() == ErrTimedOut { 561 t.Errorf("Expected %s() to fail due to state, not %v", what, err) 562 } 563 } 564 565 p.Close() 566 } 567 568 // TestProducerDeliveryReportFields tests the `go.delivery.report.fields` config setting 569 func TestProducerDeliveryReportFields(t *testing.T) { 570 t.Run("none", func(t *testing.T) { 571 runProducerDeliveryReportFieldTest(t, &ConfigMap{ 572 "socket.timeout.ms": 10, 573 "message.timeout.ms": 10, 574 "go.delivery.report.fields": "", 575 }, func(expected, actual *Message) { 576 if len(actual.Key) > 0 { 577 t.Errorf("key should not be set") 578 } 579 if len(actual.Value) > 0 { 580 t.Errorf("value should not be set") 581 } 582 if s, ok := actual.Opaque.(*string); ok { 583 if *s != *(expected.Opaque.(*string)) { 584 t.Errorf("Opaque should be \"%v\", not \"%v\"", expected.Opaque, actual.Opaque) 585 } 586 } else { 587 t.Errorf("opaque value should be a string, not \"%v\"", actual.Opaque) 588 } 589 }) 590 }) 591 t.Run("single", func(t *testing.T) { 592 runProducerDeliveryReportFieldTest(t, &ConfigMap{ 593 "socket.timeout.ms": 10, 594 "message.timeout.ms": 10, 595 "go.delivery.report.fields": "key", 596 }, func(expected, actual *Message) { 597 if !bytes.Equal(expected.Key, actual.Key) { 598 t.Errorf("key should be \"%s\", not \"%s\"", expected.Key, actual.Key) 599 } 600 if len(actual.Value) > 0 { 601 t.Errorf("value should not be set") 602 } 603 if s, ok := actual.Opaque.(*string); ok { 604 if *s != *(expected.Opaque.(*string)) { 605 t.Errorf("Opaque should be \"%v\", not \"%v\"", expected.Opaque, actual.Opaque) 606 } 607 } else { 608 t.Errorf("opaque value should be a string, not \"%v\"", actual.Opaque) 609 } 610 }) 611 }) 612 t.Run("multiple", func(t *testing.T) { 613 runProducerDeliveryReportFieldTest(t, &ConfigMap{ 614 "socket.timeout.ms": 10, 615 "message.timeout.ms": 10, 616 "go.delivery.report.fields": "key,value", 617 }, func(expected, actual *Message) { 618 if !bytes.Equal(expected.Key, actual.Key) { 619 t.Errorf("key should be \"%s\", not \"%s\"", expected.Key, actual.Key) 620 } 621 if !bytes.Equal(expected.Value, actual.Value) { 622 t.Errorf("value should be \"%s\", not \"%s\"", expected.Value, actual.Value) 623 } 624 if s, ok := actual.Opaque.(*string); ok { 625 if *s != *(expected.Opaque.(*string)) { 626 t.Errorf("Opaque should be \"%v\", not \"%v\"", expected.Opaque, actual.Opaque) 627 } 628 } else { 629 t.Errorf("opaque value should be a string, not \"%v\"", actual.Opaque) 630 } 631 }) 632 }) 633 t.Run("default", func(t *testing.T) { 634 runProducerDeliveryReportFieldTest(t, &ConfigMap{ 635 "socket.timeout.ms": 10, 636 "message.timeout.ms": 10, 637 }, func(expected, actual *Message) { 638 if !bytes.Equal(expected.Key, actual.Key) { 639 t.Errorf("key should be \"%s\", not \"%s\"", expected.Key, actual.Key) 640 } 641 if !bytes.Equal(expected.Value, actual.Value) { 642 t.Errorf("value should be \"%s\", not \"%s\"", expected.Value, actual.Value) 643 } 644 if actual.Headers != nil { 645 t.Errorf("Did not expect Headers") 646 } 647 if s, ok := actual.Opaque.(*string); ok { 648 if *s != *(expected.Opaque.(*string)) { 649 t.Errorf("Opaque should be \"%v\", not \"%v\"", expected.Opaque, actual.Opaque) 650 } 651 } else { 652 t.Errorf("opaque value should be a string, not \"%v\"", actual.Opaque) 653 } 654 }) 655 }) 656 t.Run("all", func(t *testing.T) { 657 runProducerDeliveryReportFieldTest(t, &ConfigMap{ 658 "socket.timeout.ms": 10, 659 "message.timeout.ms": 10, 660 "go.delivery.report.fields": "all", 661 }, func(expected, actual *Message) { 662 if !bytes.Equal(expected.Key, actual.Key) { 663 t.Errorf("key should be \"%s\", not \"%s\"", expected.Key, actual.Key) 664 } 665 if !bytes.Equal(expected.Value, actual.Value) { 666 t.Errorf("value should be \"%s\", not \"%s\"", expected.Value, actual.Value) 667 } 668 if actual.Headers == nil { 669 t.Errorf("Expected headers") 670 } 671 if !reflect.DeepEqual(expected.Headers, actual.Headers) { 672 t.Errorf("Headers mismatch: Expected %v, got %v", 673 expected.Headers, actual.Headers) 674 } 675 if s, ok := actual.Opaque.(*string); ok { 676 if *s != *(expected.Opaque.(*string)) { 677 t.Errorf("Opaque should be \"%v\", not \"%v\"", expected.Opaque, actual.Opaque) 678 } 679 } else { 680 t.Errorf("opaque value should be a string, not \"%v\"", actual.Opaque) 681 } 682 }) 683 }) 684 } 685 686 func runProducerDeliveryReportFieldTest(t *testing.T, config *ConfigMap, fn func(expected, actual *Message)) { 687 p, err := NewProducer(config) 688 if err != nil { 689 t.Fatalf("%s", err) 690 } 691 692 topic1 := "gotest" 693 694 myOpq := "My opaque" 695 expected := &Message{ 696 TopicPartition: TopicPartition{Topic: &topic1, Partition: 0}, 697 Opaque: &myOpq, 698 Value: []byte("ProducerChannel"), 699 Key: []byte("This is my key"), 700 Headers: []Header{ 701 {"hdr1", []byte("value1")}, 702 {"hdr2", []byte("value2")}, 703 {"hdr2", []byte("headers are not unique")}}, 704 } 705 p.ProduceChannel() <- expected 706 707 // We should expect a single message and possibly events 708 for msgCnt := 0; msgCnt < 1; { 709 ev := <-p.Events() 710 switch e := ev.(type) { 711 case *Message: 712 msgCnt++ 713 fn(expected, e) 714 default: 715 t.Logf("Ignored event %s", e) 716 } 717 } 718 719 r := p.Flush(2000) 720 if r > 0 { 721 t.Errorf("Expected empty queue after Flush, still has %d", r) 722 } 723 }