forked from IBM/sarama
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request IBM#1008 from buyology/delete-topics
add DeleteTopicsRequest/Response
- Loading branch information
Showing
6 changed files
with
190 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,41 @@ | ||
package sarama | ||
|
||
import "time" | ||
|
||
type DeleteTopicsRequest struct { | ||
Topics []string | ||
Timeout time.Duration | ||
} | ||
|
||
func (d *DeleteTopicsRequest) encode(pe packetEncoder) error { | ||
if err := pe.putStringArray(d.Topics); err != nil { | ||
return err | ||
} | ||
pe.putInt32(int32(d.Timeout / time.Millisecond)) | ||
|
||
return nil | ||
} | ||
|
||
func (d *DeleteTopicsRequest) decode(pd packetDecoder, version int16) (err error) { | ||
if d.Topics, err = pd.getStringArray(); err != nil { | ||
return err | ||
} | ||
timeout, err := pd.getInt32() | ||
if err != nil { | ||
return err | ||
} | ||
d.Timeout = time.Duration(timeout) * time.Millisecond | ||
return nil | ||
} | ||
|
||
func (d *DeleteTopicsRequest) key() int16 { | ||
return 20 | ||
} | ||
|
||
func (d *DeleteTopicsRequest) version() int16 { | ||
return 0 | ||
} | ||
|
||
func (d *DeleteTopicsRequest) requiredVersion() KafkaVersion { | ||
return V0_10_1_0 | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,22 @@ | ||
package sarama | ||
|
||
import ( | ||
"testing" | ||
"time" | ||
) | ||
|
||
var deleteTopicsRequest = []byte{ | ||
0, 0, 0, 2, | ||
0, 5, 't', 'o', 'p', 'i', 'c', | ||
0, 5, 'o', 't', 'h', 'e', 'r', | ||
0, 0, 0, 100, | ||
} | ||
|
||
func TestDeleteTopicsRequest(t *testing.T) { | ||
req := &DeleteTopicsRequest{ | ||
Topics: []string{"topic", "other"}, | ||
Timeout: 100 * time.Millisecond, | ||
} | ||
|
||
testRequest(t, "", req, deleteTopicsRequest) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,78 @@ | ||
package sarama | ||
|
||
import "time" | ||
|
||
type DeleteTopicsResponse struct { | ||
Version int16 | ||
ThrottleTime time.Duration | ||
TopicErrorCodes map[string]KError | ||
} | ||
|
||
func (d *DeleteTopicsResponse) encode(pe packetEncoder) error { | ||
if d.Version >= 1 { | ||
pe.putInt32(int32(d.ThrottleTime / time.Millisecond)) | ||
} | ||
|
||
if err := pe.putArrayLength(len(d.TopicErrorCodes)); err != nil { | ||
return err | ||
} | ||
for topic, errorCode := range d.TopicErrorCodes { | ||
if err := pe.putString(topic); err != nil { | ||
return err | ||
} | ||
pe.putInt16(int16(errorCode)) | ||
} | ||
|
||
return nil | ||
} | ||
|
||
func (d *DeleteTopicsResponse) decode(pd packetDecoder, version int16) (err error) { | ||
if version >= 1 { | ||
throttleTime, err := pd.getInt32() | ||
if err != nil { | ||
return err | ||
} | ||
d.ThrottleTime = time.Duration(throttleTime) * time.Millisecond | ||
|
||
d.Version = version | ||
} | ||
|
||
n, err := pd.getArrayLength() | ||
if err != nil { | ||
return err | ||
} | ||
|
||
d.TopicErrorCodes = make(map[string]KError, n) | ||
|
||
for i := 0; i < n; i++ { | ||
topic, err := pd.getString() | ||
if err != nil { | ||
return err | ||
} | ||
errorCode, err := pd.getInt16() | ||
if err != nil { | ||
return err | ||
} | ||
|
||
d.TopicErrorCodes[topic] = KError(errorCode) | ||
} | ||
|
||
return nil | ||
} | ||
|
||
func (d *DeleteTopicsResponse) key() int16 { | ||
return 20 | ||
} | ||
|
||
func (d *DeleteTopicsResponse) version() int16 { | ||
return d.Version | ||
} | ||
|
||
func (d *DeleteTopicsResponse) requiredVersion() KafkaVersion { | ||
switch d.Version { | ||
case 1: | ||
return V0_11_0_0 | ||
default: | ||
return V0_10_1_0 | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,36 @@ | ||
package sarama | ||
|
||
import ( | ||
"testing" | ||
"time" | ||
) | ||
|
||
var ( | ||
deleteTopicsResponseV0 = []byte{ | ||
0, 0, 0, 1, | ||
0, 5, 't', 'o', 'p', 'i', 'c', | ||
0, 0, | ||
} | ||
|
||
deleteTopicsResponseV1 = []byte{ | ||
0, 0, 0, 100, | ||
0, 0, 0, 1, | ||
0, 5, 't', 'o', 'p', 'i', 'c', | ||
0, 0, | ||
} | ||
) | ||
|
||
func TestDeleteTopicsResponse(t *testing.T) { | ||
resp := &DeleteTopicsResponse{ | ||
TopicErrorCodes: map[string]KError{ | ||
"topic": ErrNoError, | ||
}, | ||
} | ||
|
||
testResponse(t, "version 0", resp, deleteTopicsResponseV0) | ||
|
||
resp.Version = 1 | ||
resp.ThrottleTime = 100 * time.Millisecond | ||
|
||
testResponse(t, "version 1", resp, deleteTopicsResponseV1) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters