/
mockcluster.go
134 lines (116 loc) · 4.15 KB
/
mockcluster.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
/**
* Copyright 2023 Confluent Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka
import (
"time"
"unsafe"
)
/*
#include <stdlib.h>
#include "select_rdkafka.h"
#include "glue_rdkafka.h"
*/
import "C"
// MockCluster represents a Kafka mock cluster instance which can be used
// for testing.
type MockCluster struct {
rk *C.rd_kafka_t
mcluster *C.rd_kafka_mock_cluster_t
}
// NewMockCluster provides a mock Kafka cluster with a configurable
// number of brokers that support a reasonable subset of Kafka protocol
// operations, error injection, etc.
//
// The broker ids will start at 1 up to and including brokerCount.
//
// Mock clusters provide localhost listeners that can be used as the bootstrap
// servers by multiple Kafka client instances.
//
// Currently supported functionality:
// - Producer
// - Idempotent Producer
// - Transactional Producer
// - Low-level consumer
// - High-level balanced consumer groups with offset commits
// - Topic Metadata and auto creation
//
// Warning THIS IS AN EXPERIMENTAL API, SUBJECT TO CHANGE OR REMOVAL.
func NewMockCluster(brokerCount int) (*MockCluster, error) {
mc := &MockCluster{}
cErrstr := (*C.char)(C.malloc(C.size_t(512)))
defer C.free(unsafe.Pointer(cErrstr))
cConf := C.rd_kafka_conf_new()
mc.rk = C.rd_kafka_new(C.RD_KAFKA_PRODUCER, cConf, cErrstr, 256)
if mc.rk == nil {
C.rd_kafka_conf_destroy(cConf)
return nil, newErrorFromCString(C.RD_KAFKA_RESP_ERR__INVALID_ARG, cErrstr)
}
mc.mcluster = C.rd_kafka_mock_cluster_new(mc.rk, C.int(brokerCount))
if mc.mcluster == nil {
C.rd_kafka_destroy(mc.rk)
return nil, newErrorFromCString(C.RD_KAFKA_RESP_ERR__INVALID_ARG, cErrstr)
}
return mc, nil
}
// BootstrapServers returns the bootstrap.servers property for this MockCluster
func (mc *MockCluster) BootstrapServers() string {
return C.GoString(C.rd_kafka_mock_cluster_bootstraps(mc.mcluster))
}
// SetRoundtripDuration sets the broker round-trip-time delay for the given broker.
// Use brokerID -1 for all brokers, or >= 0 for a specific broker.
func (mc *MockCluster) SetRoundtripDuration(brokerID int, duration time.Duration) error {
durationInMillis := C.int(duration.Milliseconds())
cError := C.rd_kafka_mock_broker_set_rtt(mc.mcluster, C.int(brokerID), durationInMillis)
if cError != C.RD_KAFKA_RESP_ERR_NO_ERROR {
return newError(cError)
}
return nil
}
// SetBrokerDown disconnects the broker and disallows any new connections.
// This does NOT trigger leader change.
// Use brokerID -1 for all brokers, or >= 0 for a specific broker.
func (mc *MockCluster) SetBrokerDown(brokerID int) error {
cError := C.rd_kafka_mock_broker_set_down(mc.mcluster, C.int(brokerID))
if cError != C.RD_KAFKA_RESP_ERR_NO_ERROR {
return newError(cError)
}
return nil
}
// SetBrokerUp makes the broker accept connections again.
// This does NOT trigger leader change.
// Use brokerID -1 for all brokers, or >= 0 for a specific broker.
func (mc *MockCluster) SetBrokerUp(brokerID int) error {
cError := C.rd_kafka_mock_broker_set_up(mc.mcluster, C.int(brokerID))
if cError != C.RD_KAFKA_RESP_ERR_NO_ERROR {
return newError(cError)
}
return nil
}
// CreateTopic creates a topic without having to use a producer
func (mc *MockCluster) CreateTopic(topic string, partitions, replicationFactor int) error {
topicStr := C.CString(topic)
defer C.free(unsafe.Pointer(topicStr))
cError := C.rd_kafka_mock_topic_create(mc.mcluster, topicStr, C.int(partitions), C.int(replicationFactor))
if cError != C.RD_KAFKA_RESP_ERR_NO_ERROR {
return newError(cError)
}
return nil
}
// Close and destroy the MockCluster
func (mc *MockCluster) Close() {
C.rd_kafka_mock_cluster_destroy(mc.mcluster)
C.rd_kafka_destroy(mc.rk)
}