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

Added a delete records api #1141

Merged
merged 30 commits into from
Jul 10, 2024
Merged
Show file tree
Hide file tree
Changes from 11 commits
Commits
Show all changes
30 commits
Select commit Hold shift + click to select a range
c097770
Added a delete records api
PratRanj07 Feb 23, 2024
a989ce5
Corrected semaphore errors
PratRanj07 Feb 23, 2024
37b83f7
Resolved all the comments
PratRanj07 Feb 26, 2024
ab205ab
Resolved formatting errors
PratRanj07 Feb 26, 2024
18c5761
Merge branch 'master' into deleteRecords
PratRanj07 Apr 22, 2024
8c445a5
Minor Change
PratRanj07 May 2, 2024
0bd001d
Comments added
PratRanj07 May 2, 2024
ec93271
indentation check
PratRanj07 May 2, 2024
e15dea9
Merge branch 'master' into deleteRecords
PratRanj07 May 8, 2024
f717d46
Merge branch 'master' into deleteRecords
PratRanj07 May 22, 2024
71c9973
Merge branch 'master' into deleteRecords
PratRanj07 Jun 12, 2024
c7c93be
Update kafka/adminapi.go
milindl Jul 5, 2024
72eb3ce
Update kafka/adminapi.go
milindl Jul 5, 2024
2d9381a
Update kafka/adminapi.go
milindl Jul 5, 2024
3b8b4f2
Update tests and CHANGELOG.md per comments
milindl Jul 5, 2024
95a1cd2
Merge branch 'master' into deleteRecords
milindl Jul 5, 2024
8127d58
Update kafka/adminapi.go
milindl Jul 9, 2024
d9c0949
Address public API comments
milindl Jul 9, 2024
83dcaa0
Merge branch 'master' into deleteRecords
milindl Jul 9, 2024
4e25b7d
Update examples/admin_delete_records/admin_delete_records.go
milindl Jul 10, 2024
6db32fb
Update examples/admin_delete_records/admin_delete_records.go
milindl Jul 10, 2024
89070c7
Update examples/admin_delete_records/admin_delete_records.go
milindl Jul 10, 2024
4618e60
Update kafka/adminapi.go
milindl Jul 10, 2024
6d26d84
Update kafka/integration_test.go
milindl Jul 10, 2024
82d3df8
Update kafka/integration_test.go
milindl Jul 10, 2024
f840767
Address PR comments
milindl Jul 10, 2024
0eb4183
Merge branch 'master' into deleteRecords
milindl Jul 10, 2024
8785c3c
Regenerate api.html
milindl Jul 10, 2024
c5448e0
Run gofmt on example after comment addressal via Github UI
milindl Jul 10, 2024
7c6bdc3
Small changes within comments and strings
milindl Jul 10, 2024
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
7 changes: 7 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,13 @@ confluent-kafka-go is based on librdkafka v2.4.0, see the
for a complete list of changes, enhancements, fixes and upgrade considerations.


# v2.4.0

This is a feature release.

* Adds an AdminAPI `DeleteRecords()`
(#1141, @PratRanj07).

# v2.3.0

This is a feature release.
Expand Down
1 change: 1 addition & 0 deletions examples/.gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ admin_describe_topics/admin_describe_topics
admin_describe_cluster/admin_describe_cluster
admin_delete_acls/admin_delete_acls
admin_delete_consumer_groups/admin_delete_consumer_groups
admin_delete_records/admin_delete_records
admin_delete_topics/admin_delete_topics
admin_describe_acls/admin_describe_acls
admin_describe_config/admin_describe_config
Expand Down
2 changes: 2 additions & 0 deletions examples/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@ Examples

[admin_delete_consumer_groups](admin_delete_consumer_groups) - Delete consumer groups

[admin_delete_records](admin_delete_records) - Delete records before a specified offset

[admin_delete_topics](admin_delete_topics) - Delete topics

[admin_describe_acls](admin_describe_acls) - Find Access Control Lists using a filter
Expand Down
92 changes: 92 additions & 0 deletions examples/admin_delete_records/admin_delete_records.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/**
* Copyright 2024 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.
*/

// Delete Records before a particular offset in specified Topic Partition.

package main

import (
"context"
"fmt"
"os"
"strconv"
"time"

"github.com/confluentinc/confluent-kafka-go/v2/kafka"
)

func main() {
args := os.Args
PratRanj07 marked this conversation as resolved.
Show resolved Hide resolved

if len(args) < 5 {
fmt.Fprintf(os.Stderr,
"Usage: %s <bootstrap_servers> "+
"<topic1> <partition1> <offset1> ...\n",
args[0])
os.Exit(1)
}

// Create new AdminClient.
ac, err := kafka.NewAdminClient(&kafka.ConfigMap{
"bootstrap.servers": args[1],
})
if err != nil {
fmt.Printf("Failed to create Admin client: %s\n", err)
os.Exit(1)
}
defer ac.Close()

var topicPartitionOffsets []kafka.TopicPartition

argsCnt := len(os.Args)
i := 2
index := 0

for i < argsCnt {
if i+3 > argsCnt {
fmt.Printf("Expected %d arguments for partition %d, got %d\n", 3, index, argsCnt-i)
os.Exit(1)
}
topicName := os.Args[i]
partition, err := strconv.ParseInt(args[i+1], 10, 32)
if err != nil {
panic(err)
}
offset, err := strconv.ParseUint(args[i+2], 10, 64)
milindl marked this conversation as resolved.
Show resolved Hide resolved
if err != nil {
panic(err)
}

topicPartitionOffsets = append(topicPartitionOffsets, kafka.TopicPartition{
Topic: &topicName,
Partition: int32(partition),
Offset: kafka.Offset(offset),
})
milindl marked this conversation as resolved.
Show resolved Hide resolved
i += 3
index++
}

ctx, cancel := context.WithTimeout(context.Background(), time.Second*30)
defer cancel()

res, err := ac.DeleteRecords(ctx, topicPartitionOffsets)
if err != nil {
fmt.Printf("Failed to delete records: %s\n", err)
os.Exit(1)
}

fmt.Printf("Delete Records result: %+v\n", res)
milindl marked this conversation as resolved.
Show resolved Hide resolved
}
85 changes: 85 additions & 0 deletions kafka/adminapi.go
Original file line number Diff line number Diff line change
Expand Up @@ -258,6 +258,14 @@ type ListConsumerGroupsResult struct {
Errors []error
}

// DeleteRecordsResult represents the result of a DeleteRecords call
type DeleteRecordsResult struct {
// slice of TopicPartitions containing the minimum offsets available in that
// topic partition after the deletion operation has been performed. The error is
// set if any occurred for that topic partition.
milindl marked this conversation as resolved.
Show resolved Hide resolved
TopicPartitions []TopicPartition
}

// MemberAssignment represents the assignment of a consumer group member.
type MemberAssignment struct {
// Partitions assigned to current member.
Expand Down Expand Up @@ -3397,6 +3405,83 @@ func (a *AdminClient) AlterUserScramCredentials(
return result, nil
}

// DeleteRecords deletes records (messages) in topic partitions older than the offsets provided.
//
// Parameters:
// - `ctx` - context with the maximum amount of time to block, or nil for
// indefinite.
// - `recordsToDelete` - A slice of TopicPartitions with the offset field set. Those records
// will be deleted whose offset is smaller than given offset of
// corresponding partition. Additionally offset could be set to kafka.OffsetEnd
// to delete all the data in the partition.
milindl marked this conversation as resolved.
Show resolved Hide resolved
// - `options` - DeleteRecordsAdminOptions options.
//
// Returns DeleteRecordsResult, which contains a slice of
// TopicPartitions containing the minimum offsets available in that
// topic partition after the deletion operation has been performed or
// any error occured hile deleting the records in the particular TopicPartition.
// User has to check all the elements of the result to check any error occured per partition.
milindl marked this conversation as resolved.
Show resolved Hide resolved
func (a *AdminClient) DeleteRecords(ctx context.Context, recordsToDelete []TopicPartition, options ...DeleteRecordsAdminOption) (result DeleteRecordsResult, err error) {

if len(recordsToDelete) == 0 {
return result, newErrorFromString(ErrInvalidArg, "No records to delete")
}
result = DeleteRecordsResult{}
milindl marked this conversation as resolved.
Show resolved Hide resolved
err = a.verifyClient()
if err != nil {
return result, err
}
emasab marked this conversation as resolved.
Show resolved Hide resolved

// convert recordsToDelete to rd_kafka_DeleteRecords_t** required by implementation
cRecordsToDelete := newCPartsFromTopicPartitions(recordsToDelete)
defer C.rd_kafka_topic_partition_list_destroy(cRecordsToDelete)

cDelRecords := make([]*C.rd_kafka_DeleteRecords_t, 1)
defer C.rd_kafka_DeleteRecords_destroy_array(&cDelRecords[0], C.size_t(1))

cDelRecords[0] = C.rd_kafka_DeleteRecords_new(cRecordsToDelete)

// Convert Go AdminOptions (if any) to C AdminOptions.
genericOptions := make([]AdminOption, len(options))
for i := range options {
genericOptions[i] = options[i]
}
cOptions, err := adminOptionsSetup(
a.handle, C.RD_KAFKA_ADMIN_OP_DELETERECORDS, genericOptions)
if err != nil {
return result, err
}
defer C.rd_kafka_AdminOptions_destroy(cOptions)

// Create temporary queue for async operation.
cQueue := C.rd_kafka_queue_new(a.handle.rk)
defer C.rd_kafka_queue_destroy(cQueue)

// Call rd_kafka_DeleteRecords (asynchronous).
C.rd_kafka_DeleteRecords(
a.handle.rk,
&cDelRecords[0],
C.size_t(1),
cOptions,
cQueue)

// Wait for result, error or context timeout.
rkev, err := a.waitResult(
ctx, cQueue, C.RD_KAFKA_EVENT_DELETERECORDS_RESULT)
if err != nil {
return result, err
}
defer C.rd_kafka_event_destroy(rkev)

cRes := C.rd_kafka_event_DeleteRecords_result(rkev)
cDeleteRecordsResultList := C.rd_kafka_DeleteRecords_result_offsets(cRes)

// Convert result from C to Go.
result.TopicPartitions = newTopicPartitionsFromCparts(cDeleteRecordsResultList)

return result, nil
}

// NewAdminClient creats a new AdminClient instance with a new underlying client instance
func NewAdminClient(conf *ConfigMap) (*AdminClient, error) {

Expand Down
38 changes: 38 additions & 0 deletions kafka/adminapi_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -881,6 +881,43 @@ func testAdminAPIsUserScramCredentials(what string, a *AdminClient, expDuration
}
}

func testAdminAPIsDeleteRecords(what string, a *AdminClient, expDuration time.Duration, t *testing.T) {
topic := "test"
partition := int32(0)
offset := Offset(2)
ctx, cancel := context.WithTimeout(context.Background(), expDuration)
defer cancel()
topicPartitionOffset := []TopicPartition{{Topic: &topic, Partition: partition, Offset: offset}}
_, err := a.DeleteRecords(ctx, topicPartitionOffset, SetAdminRequestTimeout(time.Second))
if err == nil || ctx.Err() != context.DeadlineExceeded {
t.Fatalf("Expected context deadline exceeded, got %s and %s\n",
err, ctx.Err())
}

// Invalid option value
_, err = a.DeleteRecords(ctx, topicPartitionOffset, SetAdminRequestTimeout(-1))
if err == nil || err.(Error).Code() != ErrInvalidArg {
t.Fatalf("Expected ErrInvalidArg, not %v", err)
}

for _, options := range [][]DeleteRecordsAdminOption{
{},
{SetAdminRequestTimeout(time.Second)},
} {
// nil argument should fail, not being treated as empty
ctx, cancel = context.WithTimeout(context.Background(), expDuration)
defer cancel()
result, err := a.DeleteRecords(ctx, nil, options...)
if result.TopicPartitions != nil || err == nil {
t.Fatalf("Expected DeleteRecords to fail, but got result: %v, err: %v",
result, err)
}
if err.(Error).Code() != ErrInvalidArg {
t.Fatalf("Expected ErrInvalidArg, not %v", err)
}
}
}

func testAdminAPIs(what string, a *AdminClient, t *testing.T) {
t.Logf("AdminClient API testing on %s: %s", a, what)

Expand Down Expand Up @@ -1134,6 +1171,7 @@ func testAdminAPIs(what string, a *AdminClient, t *testing.T) {
testAdminAPIsListOffsets(what, a, expDuration, t)

testAdminAPIsUserScramCredentials(what, a, expDuration, t)
testAdminAPIsDeleteRecords(what, a, expDuration, t)
}

// TestAdminAPIs dry-tests most Admin APIs, no broker is needed.
Expand Down
12 changes: 12 additions & 0 deletions kafka/adminoptions.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,8 @@ func (ao AdminOptionOperationTimeout) supportsDeleteTopics() {
}
func (ao AdminOptionOperationTimeout) supportsCreatePartitions() {
}
func (ao AdminOptionOperationTimeout) supportsDeleteRecords() {
}

func (ao AdminOptionOperationTimeout) apply(cOptions *C.rd_kafka_AdminOptions_t) error {
if !ao.isSet {
Expand Down Expand Up @@ -143,6 +145,8 @@ func (ao AdminOptionRequestTimeout) supportsDescribeUserScramCredentials() {
}
func (ao AdminOptionRequestTimeout) supportsAlterUserScramCredentials() {
}
func (ao AdminOptionRequestTimeout) supportsDeleteRecords() {
}
func (ao AdminOptionRequestTimeout) apply(cOptions *C.rd_kafka_AdminOptions_t) error {
if !ao.isSet {
return nil
Expand Down Expand Up @@ -563,6 +567,14 @@ type ListOffsetsAdminOption interface {
apply(cOptions *C.rd_kafka_AdminOptions_t) error
}

// DeleteRecordsAdminOption - see setter.
//
// See SetAdminRequestTimeout, SetAdminOperationTimeout.
type DeleteRecordsAdminOption interface {
supportsDeleteRecords()
apply(cOptions *C.rd_kafka_AdminOptions_t) error
}

// AdminOption is a generic type not to be used directly.
//
// See CreateTopicsAdminOption et.al.
Expand Down
72 changes: 72 additions & 0 deletions kafka/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3245,6 +3245,78 @@ func (its *IntegrationTestSuite) TestAdminClient_ListOffsets() {

}

// Test DeleteRecords API which deletes all the records before the specified offset
// in the particular partition of the specified topic.
func (its *IntegrationTestSuite) TestAdminClient_DeleteRecords() {
t := its.T()
bootstrapServers := testconf.Brokers
assert := its.Assert()

// Create a new AdminClient.
a := createAdminClient(t)
defer a.Close()

ctx, cancel := context.WithCancel(context.Background())
defer cancel()

// Create a new topic to test the api and produce some messages to that topic
topic := "test-delete"
milindl marked this conversation as resolved.
Show resolved Hide resolved

topics := []TopicSpecification{TopicSpecification{Topic: topic, NumPartitions: 1, ReplicationFactor: 1}}
createTopicResult, createTopicError := a.CreateTopics(ctx, topics)
assert.Nil(createTopicError, "Create Topics should not fail.")
milindl marked this conversation as resolved.
Show resolved Hide resolved
assert.Equal(createTopicResult[0].Error.Code(), ErrNoError, "Create Topics Error Code should be ErrNoError.")
emasab marked this conversation as resolved.
Show resolved Hide resolved

p, err := NewProducer(&ConfigMap{"bootstrap.servers": bootstrapServers})
assert.Nil(err, "Unable to create Producer.")
defer p.Close()

p.Produce(&Message{
TopicPartition: TopicPartition{Topic: &topic, Partition: 0},
Value: []byte("Message-1"),
}, nil)

p.Produce(&Message{
TopicPartition: TopicPartition{Topic: &topic, Partition: 0},
Value: []byte("Message-2"),
}, nil)

p.Produce(&Message{
TopicPartition: TopicPartition{Topic: &topic, Partition: 0},
Value: []byte("Message-3"),
}, nil)

p.Flush(5 * 1000)

// Delete the records till offset 2 in partion 0 of the topic
milindl marked this conversation as resolved.
Show resolved Hide resolved
// The result will contain the minimum offset available after deletion in that Topic Partiton
delRecordsTopicPartitionOffsets := []TopicPartition{{Topic: &topic, Partition: int32(0), Offset: Offset(2)}}
deleteRes, err := a.DeleteRecords(ctx, delRecordsTopicPartitionOffsets)
assert.Nil(err, "Delete Records should not fail")
assert.Equal(len(deleteRes.TopicPartitions), 1, "Length of deleteRes.TopicPartitions should be 1")
assert.Nil(deleteRes.TopicPartitions[0].Error, "Error should not be set in deleteRes.TopicPartitions[0]")
offsetAfterDeletion := deleteRes.TopicPartitions[0].Offset

// Find the minimum uncommitted offset in that partition of the topic.
// It should be equal to the offset we get after the deletion operation
topicPartitionOffsets := make(map[TopicPartition]OffsetSpec)
tp1 := TopicPartition{Topic: &topic, Partition: 0}
topicPartitionOffsets[tp1] = EarliestOffsetSpec
var results ListOffsetsResult
results, err = a.ListOffsets(ctx, topicPartitionOffsets)
assert.Nil(err, "ListOffsets should not fail.")
assert.Equal(len(results.ResultInfos), 1, "Length of results.ResultInfos should be 1")
milindl marked this conversation as resolved.
Show resolved Hide resolved
for _, info := range results.ResultInfos {
PratRanj07 marked this conversation as resolved.
Show resolved Hide resolved
assert.Equal(info.Error.Code(), ErrNoError, "Error code should be ErrNoError.")
emasab marked this conversation as resolved.
Show resolved Hide resolved
assert.Equal(info.Offset, offsetAfterDeletion, "Offset should be equal to the offset obtained after deletion.")
}

delTopics := []string{topic}
_, err = a.DeleteTopics(ctx, delTopics)
assert.Nil(err, "DeleteTopics should not fail.")

}

func TestIntegration(t *testing.T) {
its := new(IntegrationTestSuite)
testconfInit()
Expand Down