Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix(proto): use range of OffsetCommitRequest vers #2555

Merged
merged 2 commits into from
Aug 3, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 11 additions & 1 deletion client.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,6 @@ type client struct {
cachedPartitionsResults map[string][maxPartitionIndex][]int32

lock sync.RWMutex // protects access to the maps that hold cluster state.

}

// NewClient creates a new Client. It connects to one of the given broker addresses
Expand Down Expand Up @@ -899,9 +898,20 @@ func (client *client) getOffset(topic string, partitionID int32, time int64) (in
}

request := &OffsetRequest{}
// Version 1 removes MaxNumOffsets. From this version forward, only a single
// offset can be returned.
if client.conf.Version.IsAtLeast(V0_10_1_0) {
request.Version = 1
}
// Version 2 adds the isolation level, which is used for transactional reads.
if client.conf.Version.IsAtLeast(V0_11_0_0) {
request.Version = 2
}
// Version 3 is the same as version 2.
if client.conf.Version.IsAtLeast(V2_0_0_0) {
request.Version = 3
}

request.AddBlock(topic, partitionID, time, 1)

response, err := broker.GetAvailableOffsets(request)
Expand Down
22 changes: 11 additions & 11 deletions offset_commit_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,20 +207,20 @@ func (r *OffsetCommitRequest) isValidVersion() bool {

func (r *OffsetCommitRequest) requiredVersion() KafkaVersion {
switch r.Version {
case 1:
return V0_8_2_0
case 2:
return V0_9_0_0
case 3:
return V0_11_0_0
case 4:
return V2_0_0_0
case 5, 6:
return V2_1_0_0
case 7:
return V2_3_0_0
case 5, 6:
return V2_1_0_0
case 4:
return V2_0_0_0
case 3:
return V0_11_0_0
case 2:
return V0_9_0_0
case 0, 1:
return V0_8_2_0
default:
return MinVersion
return V2_4_0_0
}
}

Expand Down
20 changes: 11 additions & 9 deletions offset_commit_response.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,18 +106,20 @@ func (r *OffsetCommitResponse) isValidVersion() bool {

func (r *OffsetCommitResponse) requiredVersion() KafkaVersion {
switch r.Version {
case 1:
return V0_8_2_0
case 2:
return V0_9_0_0
case 3:
return V0_11_0_0
case 7:
return V2_3_0_0
case 5, 6:
return V2_1_0_0
case 4:
return V2_0_0_0
case 5, 6, 7:
return V2_3_0_0
case 3:
return V0_11_0_0
case 2:
return V0_9_0_0
case 0, 1:
return V0_8_2_0
default:
return MinVersion
return V2_4_0_0
}
}

Expand Down
69 changes: 45 additions & 24 deletions offset_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -277,24 +277,50 @@ func (om *offsetManager) flushToBroker() {
}

func (om *offsetManager) constructRequest() *OffsetCommitRequest {
var r *OffsetCommitRequest
var perPartitionTimestamp int64
if om.conf.Consumer.Offsets.Retention == 0 {
perPartitionTimestamp = ReceiveTime
r = &OffsetCommitRequest{
Version: 1,
ConsumerGroup: om.group,
ConsumerID: om.memberID,
ConsumerGroupGeneration: om.generation,
}
} else {
r = &OffsetCommitRequest{
Version: 2,
RetentionTime: int64(om.conf.Consumer.Offsets.Retention / time.Millisecond),
ConsumerGroup: om.group,
ConsumerID: om.memberID,
ConsumerGroupGeneration: om.generation,
}
r := &OffsetCommitRequest{
Version: 1,
ConsumerGroup: om.group,
ConsumerID: om.memberID,
ConsumerGroupGeneration: om.generation,
}
// Version 1 adds timestamp and group membership information, as well as the commit timestamp.
//
// Version 2 adds retention time. It removes the commit timestamp added in version 1.
if om.conf.Version.IsAtLeast(V0_9_0_0) {
r.Version = 2
}
// Version 3 and 4 are the same as version 2.
if om.conf.Version.IsAtLeast(V0_11_0_0) {
r.Version = 3
}
if om.conf.Version.IsAtLeast(V2_0_0_0) {
r.Version = 4
}
// Version 5 removes the retention time, which is now controlled only by a broker configuration.
//
// Version 6 adds the leader epoch for fencing.
if om.conf.Version.IsAtLeast(V2_1_0_0) {
r.Version = 6
}
// version 7 adds a new field called groupInstanceId to indicate member identity across restarts.
if om.conf.Version.IsAtLeast(V2_3_0_0) {
r.Version = 7
r.GroupInstanceId = om.groupInstanceId
}

// commit timestamp was only briefly supported in V1 where we set it to
// ReceiveTime (-1) to tell the broker to set it to the time when the commit
// request was received
var commitTimestamp int64
if r.Version == 1 {
commitTimestamp = ReceiveTime
}

// request controlled retention was only supported from V2-V4 (it became
// broker-only after that) so if the user has set the config options then
// flow those through as retention time on the commit request
if r.Version >= 2 && r.Version < 5 && om.conf.Consumer.Offsets.Retention > 0 {
r.RetentionTime = int64(om.conf.Consumer.Offsets.Retention / time.Millisecond)
}

om.pomsLock.RLock()
Expand All @@ -304,17 +330,12 @@ func (om *offsetManager) constructRequest() *OffsetCommitRequest {
for _, pom := range topicManagers {
pom.lock.Lock()
if pom.dirty {
r.AddBlockWithLeaderEpoch(pom.topic, pom.partition, pom.offset, pom.leaderEpoch, perPartitionTimestamp, pom.metadata)
r.AddBlockWithLeaderEpoch(pom.topic, pom.partition, pom.offset, pom.leaderEpoch, commitTimestamp, pom.metadata)
}
pom.lock.Unlock()
}
}

if om.groupInstanceId != nil {
r.Version = 7
r.GroupInstanceId = om.groupInstanceId
}

if len(r.blocks) > 0 {
return r
}
Expand Down
10 changes: 4 additions & 6 deletions offset_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,23 +138,21 @@ func (r *OffsetRequest) headerVersion() int16 {
}

func (r *OffsetRequest) isValidVersion() bool {
return r.Version >= 0 && r.Version <= 5
return r.Version >= 0 && r.Version <= 3
}

func (r *OffsetRequest) requiredVersion() KafkaVersion {
switch r.Version {
case 5:
return V2_2_0_0
case 4:
return V2_1_0_0
case 3:
return V2_0_0_0
case 2:
return V0_11_0_0
case 1:
return V0_10_1_0
case 0:
return V0_8_2_0
default:
return MinVersion
return V2_0_0_0
}
}

Expand Down
10 changes: 4 additions & 6 deletions offset_response.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,23 +168,21 @@ func (r *OffsetResponse) headerVersion() int16 {
}

func (r *OffsetResponse) isValidVersion() bool {
return r.Version >= 0 && r.Version <= 5
return r.Version >= 0 && r.Version <= 3
}

func (r *OffsetResponse) requiredVersion() KafkaVersion {
switch r.Version {
case 5:
return V2_2_0_0
case 4:
return V2_1_0_0
case 3:
return V2_0_0_0
case 2:
return V0_11_0_0
case 1:
return V0_10_1_0
case 0:
return V0_8_2_0
default:
return MinVersion
return V2_0_0_0
}
}

Expand Down