@@ -5,43 +5,90 @@ import (
5
5
"sync"
6
6
7
7
"github.com/coder/quartz"
8
- "github.com/go-kit/log"
9
- "github.com/twmb/franz-go/pkg/kgo"
10
8
)
11
9
10
+ // PartitionState is the state of a partition in [PartitionManager].
11
+ type PartitionState int
12
+
13
+ const (
14
+ PartitionPending PartitionState = iota
15
+ PartitionReplaying
16
+ PartitionReady
17
+ )
18
+
19
+ // String implements the [fmt.Stringer] interface.
20
+ func (s PartitionState ) String () string {
21
+ switch s {
22
+ case PartitionPending :
23
+ return "pending"
24
+ case PartitionReplaying :
25
+ return "replaying"
26
+ case PartitionReady :
27
+ return "ready"
28
+ default :
29
+ return "unknown"
30
+ }
31
+ }
32
+
12
33
// PartitionManager keeps track of the partitions assigned and for
13
- // each partition a timestamp of when it was last updated .
34
+ // each partition a timestamp of when it was assigned .
14
35
type PartitionManager struct {
15
- // partitions maps partitionID to last updated (unix nanoseconds).
16
- partitions map [int32 ]int64
36
+ partitions map [int32 ]partitionEntry
17
37
mtx sync.Mutex
18
- logger log.Logger
19
38
20
39
// Used for tests.
21
40
clock quartz.Clock
22
41
}
23
42
43
+ // partitionEntry contains metadata about an assigned partition.
44
+ type partitionEntry struct {
45
+ assignedAt int64
46
+ targetOffset int64
47
+ state PartitionState
48
+ }
49
+
24
50
// NewPartitionManager returns a new [PartitionManager].
25
- func NewPartitionManager (logger log. Logger ) * PartitionManager {
51
+ func NewPartitionManager () * PartitionManager {
26
52
return & PartitionManager {
27
- partitions : make (map [int32 ]int64 ),
28
- logger : log .With (logger , "component" , "limits.PartitionManager" ),
53
+ partitions : make (map [int32 ]partitionEntry ),
29
54
clock : quartz .NewReal (),
30
55
}
31
56
}
32
57
33
58
// Assign assigns the partitions and sets the last updated timestamp for each
34
59
// partition to the current time.
35
- func (m * PartitionManager ) Assign (_ context.Context , _ * kgo. Client , topicPartitions map [ string ] []int32 ) {
60
+ func (m * PartitionManager ) Assign (_ context.Context , partitions []int32 ) {
36
61
m .mtx .Lock ()
37
62
defer m .mtx .Unlock ()
38
- for _ , partitions := range topicPartitions {
39
- for _ , partition := range partitions {
40
- m .partitions [partition ] = m .clock .Now ().UnixNano ()
63
+ for _ , partition := range partitions {
64
+ m .partitions [partition ] = partitionEntry {
65
+ assignedAt : m .clock .Now ().UnixNano (),
66
+ state : PartitionPending ,
41
67
}
42
68
}
43
69
}
44
70
71
+ // GetState returns the current state of the partition. It returns false
72
+ // if the partition does not exist.
73
+ func (m * PartitionManager ) GetState (partition int32 ) (PartitionState , bool ) {
74
+ m .mtx .Lock ()
75
+ defer m .mtx .Unlock ()
76
+ entry , ok := m .partitions [partition ]
77
+ return entry .state , ok
78
+ }
79
+
80
+ // TargetOffsetReached returns true if the partition is replaying and the
81
+ // target offset has been reached.
82
+ func (m * PartitionManager ) TargetOffsetReached (partition int32 , offset int64 ) bool {
83
+ m .mtx .Lock ()
84
+ defer m .mtx .Unlock ()
85
+ entry , ok := m .partitions [partition ]
86
+ if ok {
87
+ return entry .state == PartitionReplaying && entry .targetOffset <= offset
88
+ }
89
+ return false
90
+ }
91
+
45
92
// Has returns true if the partition is assigned, otherwise false.
46
93
func (m * PartitionManager ) Has (partition int32 ) bool {
47
94
m .mtx .Lock ()
@@ -50,23 +97,66 @@ func (m *PartitionManager) Has(partition int32) bool {
50
97
return ok
51
98
}
52
99
53
- // List returns a map of all assigned partitions and their last updated timestamps.
100
+ // List returns a map of all assigned partitions and their last updated
101
+ // timestamps.
54
102
func (m * PartitionManager ) List () map [int32 ]int64 {
55
103
m .mtx .Lock ()
56
104
defer m .mtx .Unlock ()
57
- v := make (map [int32 ]int64 )
58
- for partition , lastUpdated := range m .partitions {
59
- v [partition ] = lastUpdated
105
+ result := make (map [int32 ]int64 )
106
+ for partition , entry := range m .partitions {
107
+ result [partition ] = entry . assignedAt
60
108
}
61
- return v
109
+ return result
62
110
}
63
111
64
- func (m * PartitionManager ) Remove (_ context.Context , _ * kgo.Client , topicPartitions map [string ][]int32 ) {
112
+ // ListByState returns all partitions with the specified state and their last
113
+ // updated timestamps.
114
+ func (m * PartitionManager ) ListByState (state PartitionState ) map [int32 ]int64 {
65
115
m .mtx .Lock ()
66
116
defer m .mtx .Unlock ()
67
- for _ , partitions := range topicPartitions {
68
- for _ , partition := range partitions {
69
- delete (m .partitions , partition )
117
+ result := make (map [int32 ]int64 )
118
+ for partition , entry := range m .partitions {
119
+ if entry .state == state {
120
+ result [partition ] = entry .assignedAt
70
121
}
71
122
}
123
+ return result
124
+ }
125
+
126
+ // SetReplaying sets the partition as replaying and the offset that must
127
+ // be consumed for it to become ready. It returns false if the partition
128
+ // does not exist.
129
+ func (m * PartitionManager ) SetReplaying (partition int32 , offset int64 ) bool {
130
+ m .mtx .Lock ()
131
+ defer m .mtx .Unlock ()
132
+ entry , ok := m .partitions [partition ]
133
+ if ok {
134
+ entry .state = PartitionReplaying
135
+ entry .targetOffset = offset
136
+ m .partitions [partition ] = entry
137
+ }
138
+ return ok
139
+ }
140
+
141
+ // SetReady sets the partition as ready. It returns false if the partition
142
+ // does not exist.
143
+ func (m * PartitionManager ) SetReady (partition int32 ) bool {
144
+ m .mtx .Lock ()
145
+ defer m .mtx .Unlock ()
146
+ entry , ok := m .partitions [partition ]
147
+ if ok {
148
+ entry .state = PartitionReady
149
+ entry .targetOffset = 0
150
+ m .partitions [partition ] = entry
151
+ }
152
+ return ok
153
+ }
154
+
155
+ // Revoke deletes the partitions.
156
+ func (m * PartitionManager ) Revoke (_ context.Context , partitions []int32 ) {
157
+ m .mtx .Lock ()
158
+ defer m .mtx .Unlock ()
159
+ for _ , partition := range partitions {
160
+ delete (m .partitions , partition )
161
+ }
72
162
}
0 commit comments