-
Notifications
You must be signed in to change notification settings - Fork 172
/
copartition_strategy_test.go
162 lines (157 loc) · 3.76 KB
/
copartition_strategy_test.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
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
package goka
import (
"reflect"
"testing"
"github.com/IBM/sarama"
"github.com/stretchr/testify/require"
)
func TestCopartitioningStrategy(t *testing.T) {
t.Run("name", func(t *testing.T) {
require.Equal(t, "copartition", CopartitioningStrategy.Name())
})
for _, ttest := range []struct {
name string
members map[string]sarama.ConsumerGroupMemberMetadata
topics map[string][]int32
hasError bool
useStrict bool
expected sarama.BalanceStrategyPlan
}{
{
name: "inconsistent-topic-members",
members: map[string]sarama.ConsumerGroupMemberMetadata{
"M1": {Topics: []string{"T1"}},
},
// topics are inconsistent with members
topics: map[string][]int32{
"T2": {0, 1, 2},
},
hasError: true,
useStrict: true,
},
{
name: "not-copartitioned",
members: map[string]sarama.ConsumerGroupMemberMetadata{
"M1": {Topics: []string{"T1", "T2"}},
},
// topics are inconsistent with members
topics: map[string][]int32{
"T1": {0, 1, 2},
"T2": {0, 1},
},
hasError: true,
},
{
name: "inconsistent-members",
members: map[string]sarama.ConsumerGroupMemberMetadata{
"M1": {Topics: []string{"T1", "T2"}},
"M2": {Topics: []string{"T2"}},
},
// topics are inconsistent with members
topics: map[string][]int32{
"T1": {0, 1, 2},
"T2": {0, 1, 2},
},
hasError: true,
useStrict: true,
},
{
name: "tolerate-inconsistent-members",
members: map[string]sarama.ConsumerGroupMemberMetadata{
"M1": {Topics: []string{"T1", "T2"}},
"M2": {Topics: []string{"T2"}},
},
// topics are inconsistent with members
topics: map[string][]int32{
"T1": {0, 1, 2},
"T2": {0, 1, 2},
},
expected: sarama.BalanceStrategyPlan{
"M1": map[string][]int32{
"T1": {0, 1},
"T2": {0, 1},
},
"M2": map[string][]int32{
"T2": {2},
},
},
},
{
name: "single-member",
members: map[string]sarama.ConsumerGroupMemberMetadata{
"M1": {Topics: []string{"T1"}},
},
// topics are inconsistent with members
topics: map[string][]int32{
"T1": {0, 1, 2},
},
expected: sarama.BalanceStrategyPlan{
"M1": map[string][]int32{
"T1": {0, 1, 2},
},
},
},
{
name: "multi-member",
members: map[string]sarama.ConsumerGroupMemberMetadata{
"M1": {Topics: []string{"T1"}},
"M2": {Topics: []string{"T1"}},
},
// topics are inconsistent with members
topics: map[string][]int32{
"T1": {0, 1, 2},
},
expected: sarama.BalanceStrategyPlan{
"M1": map[string][]int32{
"T1": {0, 1},
},
"M2": map[string][]int32{
"T1": {2},
},
},
},
{
name: "multi-member-multitopic",
members: map[string]sarama.ConsumerGroupMemberMetadata{
"M1": {Topics: []string{"T1", "T2", "T3"}},
"M2": {Topics: []string{"T2", "T3", "T1"}},
"M3": {Topics: []string{"T2", "T3", "T1"}},
},
// topics are inconsistent with members
topics: map[string][]int32{
"T1": {0, 1, 2, 3, 4, 5},
"T2": {0, 1, 2, 3, 4, 5},
"T3": {0, 1, 2, 3, 4, 5},
},
expected: sarama.BalanceStrategyPlan{
"M1": map[string][]int32{
"T1": {0, 1},
"T2": {0, 1},
"T3": {0, 1},
},
"M2": map[string][]int32{
"T1": {2, 3},
"T2": {2, 3},
"T3": {2, 3},
},
"M3": map[string][]int32{
"T1": {4, 5},
"T2": {4, 5},
"T3": {4, 5},
},
},
},
} {
t.Run(ttest.name, func(t *testing.T) {
var strategy sarama.BalanceStrategy = CopartitioningStrategy
if ttest.useStrict {
strategy = StrictCopartitioningStrategy
}
plan, err := strategy.Plan(ttest.members, ttest.topics)
require.Equal(t, ttest.hasError, err != nil)
if err == nil {
require.True(t, reflect.DeepEqual(ttest.expected, plan), "expected", ttest.expected, "actual", plan)
}
})
}
}