github.com/confluentinc/confluent-kafka-go@v1.9.2/kafka/mockcluster.go (about) 1 /** 2 * Copyright 2022 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 "unsafe" 20 21 /* 22 #include <stdlib.h> 23 #include "select_rdkafka.h" 24 #include "glue_rdkafka.h" 25 */ 26 import "C" 27 28 // MockCluster represents a Kafka mock cluster instance which can be used 29 // for testing. 30 type MockCluster struct { 31 rk *C.rd_kafka_t 32 mcluster *C.rd_kafka_mock_cluster_t 33 } 34 35 // NewMockCluster provides a mock Kafka cluster with a configurable 36 // number of brokers that support a reasonable subset of Kafka protocol 37 // operations, error injection, etc. 38 // 39 // Mock clusters provide localhost listeners that can be used as the bootstrap 40 // servers by multiple Kafka client instances. 41 // 42 // Currently supported functionality: 43 // - Producer 44 // - Idempotent Producer 45 // - Transactional Producer 46 // - Low-level consumer 47 // - High-level balanced consumer groups with offset commits 48 // - Topic Metadata and auto creation 49 // 50 // Warning THIS IS AN EXPERIMENTAL API, SUBJECT TO CHANGE OR REMOVAL. 51 func NewMockCluster(brokerCount int) (*MockCluster, error) { 52 53 mc := &MockCluster{} 54 55 cErrstr := (*C.char)(C.malloc(C.size_t(512))) 56 defer C.free(unsafe.Pointer(cErrstr)) 57 58 cConf := C.rd_kafka_conf_new() 59 60 mc.rk = C.rd_kafka_new(C.RD_KAFKA_PRODUCER, cConf, cErrstr, 256) 61 if mc.rk == nil { 62 C.rd_kafka_conf_destroy(cConf) 63 return nil, newErrorFromCString(C.RD_KAFKA_RESP_ERR__INVALID_ARG, cErrstr) 64 } 65 66 mc.mcluster = C.rd_kafka_mock_cluster_new(mc.rk, C.int(brokerCount)) 67 if mc.mcluster == nil { 68 C.rd_kafka_destroy(mc.rk) 69 return nil, newErrorFromCString(C.RD_KAFKA_RESP_ERR__INVALID_ARG, cErrstr) 70 } 71 72 return mc, nil 73 } 74 75 // BootstrapServers returns the bootstrap.servers property for this MockCluster 76 func (mc *MockCluster) BootstrapServers() string { 77 return C.GoString(C.rd_kafka_mock_cluster_bootstraps(mc.mcluster)) 78 } 79 80 // Close and destroy the MockCluster 81 func (mc *MockCluster) Close() { 82 C.rd_kafka_mock_cluster_destroy(mc.mcluster) 83 C.rd_kafka_destroy(mc.rk) 84 }