github.com/Jeffail/benthos/v3@v3.65.0/lib/test/integration/kafka_test.go (about) 1 package integration 2 3 import ( 4 "context" 5 "errors" 6 "fmt" 7 "strconv" 8 "testing" 9 "time" 10 11 "github.com/Jeffail/benthos/v3/internal/integration" 12 "github.com/Jeffail/benthos/v3/lib/log" 13 "github.com/Jeffail/benthos/v3/lib/message" 14 "github.com/Jeffail/benthos/v3/lib/metrics" 15 "github.com/Jeffail/benthos/v3/lib/output/writer" 16 "github.com/Jeffail/benthos/v3/lib/types" 17 "github.com/Shopify/sarama" 18 "github.com/ory/dockertest/v3" 19 "github.com/ory/dockertest/v3/docker" 20 "github.com/stretchr/testify/assert" 21 "github.com/stretchr/testify/require" 22 ) 23 24 var _ = registerIntegrationTest("kafka_redpanda", func(t *testing.T) { 25 t.Parallel() 26 27 pool, err := dockertest.NewPool("") 28 require.NoError(t, err) 29 30 pool.MaxWait = time.Second * 30 31 32 kafkaPort, err := getFreePort() 33 require.NoError(t, err) 34 35 kafkaPortStr := strconv.Itoa(kafkaPort) 36 37 options := &dockertest.RunOptions{ 38 Repository: "docker.vectorized.io/vectorized/redpanda", 39 Tag: "latest", 40 Hostname: "redpanda", 41 ExposedPorts: []string{"9092"}, 42 PortBindings: map[docker.Port][]docker.PortBinding{ 43 "9092/tcp": {{HostIP: "", HostPort: kafkaPortStr}}, 44 }, 45 Cmd: []string{ 46 "redpanda", "start", "--smp 1", "--overprovisioned", 47 "--kafka-addr 0.0.0.0:9092", 48 fmt.Sprintf("--advertise-kafka-addr localhost:%v", kafkaPort), 49 }, 50 } 51 resource, err := pool.RunWithOptions(options) 52 require.NoError(t, err) 53 t.Cleanup(func() { 54 assert.NoError(t, pool.Purge(resource)) 55 }) 56 57 resource.Expire(900) 58 require.NoError(t, pool.Retry(func() error { 59 outConf := writer.NewKafkaConfig() 60 outConf.TargetVersion = "2.1.0" 61 outConf.Addresses = []string{"localhost:" + kafkaPortStr} 62 outConf.Topic = "pls_ignore_just_testing_connection" 63 tmpOutput, serr := writer.NewKafka(outConf, types.NoopMgr(), log.Noop(), metrics.Noop()) 64 if serr != nil { 65 return serr 66 } 67 defer tmpOutput.CloseAsync() 68 if serr = tmpOutput.Connect(); serr != nil { 69 return serr 70 } 71 return tmpOutput.Write(message.New([][]byte{ 72 []byte("foo message"), 73 })) 74 })) 75 76 template := ` 77 output: 78 kafka: 79 addresses: [ localhost:$PORT ] 80 topic: topic-$ID 81 max_in_flight: $MAX_IN_FLIGHT 82 retry_as_batch: $VAR3 83 metadata: 84 exclude_prefixes: [ $OUTPUT_META_EXCLUDE_PREFIX ] 85 batching: 86 count: $OUTPUT_BATCH_COUNT 87 88 input: 89 kafka: 90 addresses: [ localhost:$PORT ] 91 topics: [ topic-$ID$VAR1 ] 92 consumer_group: "$VAR4" 93 checkpoint_limit: $VAR2 94 start_from_oldest: true 95 batching: 96 count: $INPUT_BATCH_COUNT 97 ` 98 99 suite := integration.StreamTests( 100 integration.StreamTestOpenClose(), 101 integration.StreamTestMetadata(), 102 integration.StreamTestMetadataFilter(), 103 integration.StreamTestSendBatch(10), 104 integration.StreamTestStreamSequential(1000), 105 integration.StreamTestStreamParallel(1000), 106 integration.StreamTestStreamParallelLossy(1000), 107 integration.StreamTestSendBatchCount(10), 108 ) 109 // In some modes include testing input level batching 110 suiteExt := append(suite, integration.StreamTestReceiveBatchCount(10)) 111 112 // Only for checkpointed tests 113 suiteSingleCheckpointedStream := append(suite, integration.StreamTestCheckpointCapture()) 114 115 t.Run("balanced", func(t *testing.T) { 116 t.Parallel() 117 suite.Run( 118 t, template, 119 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 120 vars.Var4 = "group" + testID 121 require.NoError(t, createKafkaTopic("localhost:"+kafkaPortStr, testID, 4)) 122 }), 123 integration.StreamTestOptPort(kafkaPortStr), 124 integration.StreamTestOptVarTwo("1"), 125 integration.StreamTestOptVarThree("false"), 126 ) 127 128 t.Run("only one partition", func(t *testing.T) { 129 t.Parallel() 130 suiteExt.Run( 131 t, template, 132 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 133 vars.Var4 = "group" + testID 134 }), 135 integration.StreamTestOptPort(kafkaPortStr), 136 integration.StreamTestOptVarTwo("1"), 137 integration.StreamTestOptVarThree("false"), 138 ) 139 }) 140 141 t.Run("checkpointed", func(t *testing.T) { 142 t.Parallel() 143 suite.Run( 144 t, template, 145 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 146 vars.Var4 = "group" + testID 147 require.NoError(t, createKafkaTopic("localhost:"+kafkaPortStr, testID, 4)) 148 }), 149 integration.StreamTestOptPort(kafkaPortStr), 150 integration.StreamTestOptVarTwo("1000"), 151 integration.StreamTestOptVarThree("false"), 152 ) 153 }) 154 155 t.Run("retry as batch", func(t *testing.T) { 156 t.Parallel() 157 suite.Run( 158 t, template, 159 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 160 vars.Var4 = "group" + testID 161 require.NoError(t, createKafkaTopic("localhost:"+kafkaPortStr, testID, 4)) 162 }), 163 integration.StreamTestOptPort(kafkaPortStr), 164 integration.StreamTestOptVarTwo("1"), 165 integration.StreamTestOptVarThree("true"), 166 ) 167 }) 168 }) 169 170 t.Run("explicit partitions", func(t *testing.T) { 171 t.Parallel() 172 suite.Run( 173 t, template, 174 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 175 vars.Var4 = "group" + testID 176 topicName := "topic-" + testID 177 vars.Var1 = fmt.Sprintf(":0,%v:1,%v:2,%v:3", topicName, topicName, topicName) 178 require.NoError(t, createKafkaTopic("localhost:"+kafkaPortStr, testID, 4)) 179 }), 180 integration.StreamTestOptPort(kafkaPortStr), 181 integration.StreamTestOptSleepAfterInput(time.Second*3), 182 integration.StreamTestOptVarTwo("1"), 183 integration.StreamTestOptVarThree("false"), 184 ) 185 186 t.Run("range of partitions", func(t *testing.T) { 187 t.Parallel() 188 suite.Run( 189 t, template, 190 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 191 vars.Var4 = "group" + testID 192 require.NoError(t, createKafkaTopic("localhost:"+kafkaPortStr, testID, 4)) 193 }), 194 integration.StreamTestOptPort(kafkaPortStr), 195 integration.StreamTestOptSleepAfterInput(time.Second*3), 196 integration.StreamTestOptVarOne(":0-3"), 197 integration.StreamTestOptVarTwo("1"), 198 integration.StreamTestOptVarThree("false"), 199 ) 200 }) 201 202 t.Run("checkpointed", func(t *testing.T) { 203 t.Parallel() 204 suiteSingleCheckpointedStream.Run( 205 t, template, 206 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 207 vars.Var4 = "group" + testID 208 require.NoError(t, createKafkaTopic("localhost:"+kafkaPortStr, testID, 1)) 209 }), 210 integration.StreamTestOptPort(kafkaPortStr), 211 integration.StreamTestOptSleepAfterInput(time.Second*3), 212 integration.StreamTestOptVarOne(":0"), 213 integration.StreamTestOptVarTwo("1000"), 214 integration.StreamTestOptVarThree("false"), 215 ) 216 }) 217 }) 218 219 t.Run("without consumer group", func(t *testing.T) { 220 t.Parallel() 221 suite.Run( 222 t, template, 223 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 224 require.NoError(t, createKafkaTopic("localhost:"+kafkaPortStr, testID, 4)) 225 }), 226 integration.StreamTestOptPort(kafkaPortStr), 227 integration.StreamTestOptSleepAfterInput(time.Second*3), 228 integration.StreamTestOptVarOne(":0-3"), 229 integration.StreamTestOptVarTwo("1"), 230 integration.StreamTestOptVarThree("false"), 231 ) 232 }) 233 234 templateManualPartitioner := ` 235 output: 236 kafka: 237 addresses: [ localhost:$PORT ] 238 topic: topic-$ID 239 max_in_flight: $MAX_IN_FLIGHT 240 retry_as_batch: $VAR3 241 metadata: 242 exclude_prefixes: [ $OUTPUT_META_EXCLUDE_PREFIX ] 243 batching: 244 count: $OUTPUT_BATCH_COUNT 245 partitioner: manual 246 partition: '${! random_int() % 4 }' 247 248 input: 249 kafka: 250 addresses: [ localhost:$PORT ] 251 topics: [ topic-$ID$VAR1 ] 252 consumer_group: "$VAR4" 253 checkpoint_limit: $VAR2 254 start_from_oldest: true 255 batching: 256 count: $INPUT_BATCH_COUNT 257 ` 258 259 t.Run("manual_partitioner", func(t *testing.T) { 260 t.Parallel() 261 suite.Run( 262 t, templateManualPartitioner, 263 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 264 vars.Var4 = "group" + testID 265 require.NoError(t, createKafkaTopic("localhost:"+kafkaPortStr, testID, 4)) 266 }), 267 integration.StreamTestOptPort(kafkaPortStr), 268 integration.StreamTestOptVarTwo("1"), 269 integration.StreamTestOptVarThree("false"), 270 ) 271 }) 272 273 }) 274 275 func createKafkaTopic(address, id string, partitions int32) error { 276 topicName := fmt.Sprintf("topic-%v", id) 277 278 b := sarama.NewBroker(address) 279 defer b.Close() 280 281 if err := b.Open(sarama.NewConfig()); err != nil { 282 return err 283 } 284 285 req := &sarama.CreateTopicsRequest{ 286 TopicDetails: map[string]*sarama.TopicDetail{ 287 topicName: { 288 NumPartitions: partitions, 289 ReplicationFactor: 1, 290 }, 291 }, 292 } 293 294 res, err := b.CreateTopics(req) 295 if err != nil { 296 return err 297 } 298 if len(res.TopicErrors) > 0 { 299 if errStr := res.TopicErrors[topicName].ErrMsg; errStr != nil { 300 return errors.New(*errStr) 301 } 302 } 303 304 var meta *sarama.MetadataResponse 305 for i := 0; i < 20; i++ { 306 meta, err = b.GetMetadata(&sarama.MetadataRequest{ 307 Topics: []string{topicName}, 308 }) 309 if err == nil && len(meta.Topics) == 1 && len(meta.Topics[0].Partitions) == int(partitions) { 310 break 311 } 312 <-time.After(time.Millisecond * 100) 313 } 314 if err != nil { 315 return err 316 } 317 if len(meta.Topics) == 0 || len(meta.Topics[0].Partitions) != int(partitions) { 318 return fmt.Errorf("failed to create topic: %v", topicName) 319 } 320 321 return nil 322 } 323 324 var _ = registerIntegrationTest("kafka_old", func(t *testing.T) { 325 t.Parallel() 326 327 pool, err := dockertest.NewPool("") 328 require.NoError(t, err) 329 330 pool.MaxWait = time.Minute 331 332 networks, _ := pool.Client.ListNetworks() 333 hostIP := "" 334 for _, network := range networks { 335 if network.Name == "bridge" { 336 hostIP = network.IPAM.Config[0].Gateway 337 } 338 } 339 340 zkResource, err := pool.RunWithOptions(&dockertest.RunOptions{ 341 Repository: "wurstmeister/zookeeper", 342 Tag: "latest", 343 }) 344 require.NoError(t, err) 345 t.Cleanup(func() { 346 require.NoError(t, pool.Purge(zkResource)) 347 }) 348 zkResource.Expire(900) 349 zkAddr := fmt.Sprintf("%v:2181", zkResource.Container.NetworkSettings.IPAddress) 350 351 kafkaPort, err := getFreePort() 352 require.NoError(t, err) 353 354 kafkaPortStr := strconv.Itoa(kafkaPort) 355 env := []string{ 356 "KAFKA_ADVERTISED_HOST_NAME=" + hostIP, 357 "KAFKA_BROKER_ID=1", 358 "KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=OUTSIDE:PLAINTEXT,INSIDE:PLAINTEXT", 359 "KAFKA_LISTENERS=OUTSIDE://:" + kafkaPortStr + ",INSIDE://:9092", 360 "KAFKA_ADVERTISED_LISTENERS=OUTSIDE://" + hostIP + ":" + kafkaPortStr + ",INSIDE://:9092", 361 "KAFKA_INTER_BROKER_LISTENER_NAME=INSIDE", 362 "KAFKA_ZOOKEEPER_CONNECT=" + zkAddr, 363 } 364 365 kafkaResource, err := pool.RunWithOptions(&dockertest.RunOptions{ 366 Repository: "wurstmeister/kafka", 367 Tag: "latest", 368 ExposedPorts: []string{kafkaPortStr + "/tcp"}, 369 PortBindings: map[docker.Port][]docker.PortBinding{ 370 docker.Port(kafkaPortStr + "/tcp"): {{HostIP: "", HostPort: kafkaPortStr}}, 371 }, 372 Env: env, 373 }) 374 require.NoError(t, err) 375 t.Cleanup(func() { 376 require.NoError(t, pool.Purge(kafkaResource)) 377 }) 378 kafkaResource.Expire(900) 379 380 address := fmt.Sprintf("%v:%v", hostIP, kafkaPortStr) 381 382 require.NoError(t, pool.Retry(func() error { 383 outConf := writer.NewKafkaConfig() 384 outConf.TargetVersion = "2.1.0" 385 outConf.Addresses = []string{address} 386 outConf.Topic = "pls_ignore_just_testing_connection" 387 tmpOutput, serr := writer.NewKafka(outConf, types.NoopMgr(), log.Noop(), metrics.Noop()) 388 if serr != nil { 389 return serr 390 } 391 defer tmpOutput.CloseAsync() 392 if serr = tmpOutput.Connect(); serr != nil { 393 return serr 394 } 395 return tmpOutput.Write(message.New([][]byte{ 396 []byte("foo message"), 397 })) 398 })) 399 400 template := fmt.Sprintf(` 401 output: 402 kafka: 403 addresses: [ %v ] 404 topic: topic-$ID 405 max_in_flight: $MAX_IN_FLIGHT 406 retry_as_batch: $VAR3 407 batching: 408 count: $OUTPUT_BATCH_COUNT 409 410 input: 411 kafka: 412 addresses: [ %v ] 413 topics: [ topic-$ID$VAR1 ] 414 consumer_group: consumer-$ID 415 checkpoint_limit: $VAR2 416 batching: 417 count: $INPUT_BATCH_COUNT 418 `, address, address) 419 420 suite := integration.StreamTests( 421 integration.StreamTestOpenClose(), 422 integration.StreamTestMetadata(), 423 integration.StreamTestSendBatch(10), 424 integration.StreamTestStreamSequential(1000), 425 integration.StreamTestStreamParallel(1000), 426 integration.StreamTestStreamParallelLossy(1000), 427 integration.StreamTestSendBatchCount(10), 428 ) 429 // In some tests include testing input level batching 430 suiteExt := append(suite, integration.StreamTestReceiveBatchCount(10)) 431 432 // Only for checkpointed tests 433 suiteSingleCheckpointedStream := append(suiteExt, integration.StreamTestCheckpointCapture()) 434 435 t.Run("balanced", func(t *testing.T) { 436 t.Parallel() 437 suiteExt.Run( 438 t, template, 439 integration.StreamTestOptVarOne(""), 440 integration.StreamTestOptVarTwo("1"), 441 integration.StreamTestOptVarThree("false"), 442 ) 443 444 t.Run("checkpointed", func(t *testing.T) { 445 t.Parallel() 446 suiteSingleCheckpointedStream.Run( 447 t, template, 448 integration.StreamTestOptVarOne(""), 449 integration.StreamTestOptVarTwo("1000"), 450 integration.StreamTestOptVarThree("false"), 451 ) 452 }) 453 454 t.Run("retry as batch", func(t *testing.T) { 455 t.Parallel() 456 suiteExt.Run( 457 t, template, 458 integration.StreamTestOptVarOne(""), 459 integration.StreamTestOptVarTwo("1"), 460 integration.StreamTestOptVarThree("true"), 461 ) 462 }) 463 464 t.Run("with four partitions", func(t *testing.T) { 465 t.Parallel() 466 suite.Run( 467 t, template, 468 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 469 require.NoError(t, createKafkaTopic(address, testID, 4)) 470 }), 471 integration.StreamTestOptVarOne(""), 472 integration.StreamTestOptVarTwo("1"), 473 integration.StreamTestOptVarThree("false"), 474 ) 475 476 t.Run("checkpointed", func(t *testing.T) { 477 t.Parallel() 478 suite.Run( 479 t, template, 480 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 481 require.NoError(t, createKafkaTopic(address, testID, 4)) 482 }), 483 integration.StreamTestOptVarOne(""), 484 integration.StreamTestOptVarTwo("1000"), 485 integration.StreamTestOptVarThree("false"), 486 ) 487 }) 488 }) 489 }) 490 491 t.Run("partitions", func(t *testing.T) { 492 t.Parallel() 493 suiteExt.Run( 494 t, template, 495 integration.StreamTestOptVarOne(":0"), 496 integration.StreamTestOptVarTwo("1"), 497 integration.StreamTestOptVarThree("false"), 498 ) 499 500 t.Run("checkpointed", func(t *testing.T) { 501 t.Parallel() 502 suiteSingleCheckpointedStream.Run( 503 t, template, 504 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 505 require.NoError(t, createKafkaTopic("localhost:"+kafkaPortStr, testID, 1)) 506 }), 507 integration.StreamTestOptVarOne(":0"), 508 integration.StreamTestOptVarTwo("1000"), 509 integration.StreamTestOptVarThree("false"), 510 ) 511 }) 512 513 t.Run("with four partitions", func(t *testing.T) { 514 t.Parallel() 515 suite.Run( 516 t, template, 517 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 518 topicName := "topic-" + testID 519 vars.Var1 = fmt.Sprintf(":0,%v:1,%v:2,%v:3", topicName, topicName, topicName) 520 require.NoError(t, createKafkaTopic(address, testID, 4)) 521 }), 522 integration.StreamTestOptSleepAfterInput(time.Second*3), 523 integration.StreamTestOptVarTwo("1"), 524 integration.StreamTestOptVarThree("false"), 525 ) 526 527 t.Run("checkpointed", func(t *testing.T) { 528 t.Parallel() 529 suite.Run( 530 t, template, 531 integration.StreamTestOptPreTest(func(t testing.TB, ctx context.Context, testID string, vars *integration.StreamTestConfigVars) { 532 topicName := "topic-" + testID 533 vars.Var1 = fmt.Sprintf(":0,%v:1,%v:2,%v:3", topicName, topicName, topicName) 534 require.NoError(t, createKafkaTopic(address, testID, 4)) 535 }), 536 integration.StreamTestOptSleepAfterInput(time.Second*3), 537 integration.StreamTestOptVarTwo("1000"), 538 integration.StreamTestOptVarThree("false"), 539 ) 540 }) 541 }) 542 }) 543 544 deprecatedTemplate := fmt.Sprintf(` 545 output: 546 kafka: 547 addresses: [ %v ] 548 topic: topic-$ID 549 max_in_flight: $MAX_IN_FLIGHT 550 batching: 551 count: $OUTPUT_BATCH_COUNT 552 553 input: 554 kafka: 555 addresses: [ %v ] 556 topic: topic-$ID 557 partition: 0 558 consumer_group: consumer-$ID 559 batching: 560 count: $INPUT_BATCH_COUNT 561 `, address, address) 562 563 t.Run("deprecated", func(t *testing.T) { 564 t.Parallel() 565 suiteExt.Run(t, deprecatedTemplate) 566 }) 567 })