Skip to content

Commit

Permalink
Update checkpointer to commit periodically when there's no data
Browse files Browse the repository at this point in the history
  • Loading branch information
colmsnowplow committed Feb 1, 2022
1 parent 9b7ab41 commit 4e19415
Show file tree
Hide file tree
Showing 4 changed files with 157 additions and 40 deletions.
26 changes: 13 additions & 13 deletions checkpoints.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@ type checkpointer struct {
finalSequenceNumber string
updateSequencer chan struct{}
lastUpdate int64
commitIntervalCounter time.Duration
lastRecordPassed time.Time
}

type checkpointRecord struct {
Expand Down Expand Up @@ -59,7 +61,6 @@ func capture(
maxAgeForClientRecord time.Duration,
stats StatReceiver) (*checkpointer, error) {

// TODO: solve duplicate problem: If capture can grab shards as soon as maxAgeForClientRecord is reached, there are more chances for duplicates.
cutoff := time.Now().Add(-maxAgeForClientRecord).UnixNano()

// Grab the entry from dynamo assuming there is one
Expand Down Expand Up @@ -146,12 +147,20 @@ func capture(
// commit writes the latest SequenceNumber consumed to dynamo and updates LastUpdate.
// Returns true if we set Finished in dynamo because the library user finished consuming the shard.
// Once that has happened, the checkpointer should be released and never grabbed again.
func (cp *checkpointer) commit() (bool, error) {
func (cp *checkpointer) commit(commitFrequency time.Duration) (bool, error) {
cp.mutex.Lock()
defer cp.mutex.Unlock()

if !cp.dirty && !cp.finished {
return false, nil
cp.commitIntervalCounter += commitFrequency

// If we have recently passed a record to the user, don't update the table when we don't have a new sequence number
// If we haven't, update at a rate of maxAgeForClientRecord/2
if (time.Now().Sub(cp.lastRecordPassed) < cp.maxAgeForClientRecord/2) || (cp.commitIntervalCounter < cp.maxAgeForClientRecord/2) {
return false, nil
}
}
cp.commitIntervalCounter = 0 // Reset the counter if we're registering a commit
now := time.Now()

sn := &cp.sequenceNumber
Expand Down Expand Up @@ -193,12 +202,7 @@ func (cp *checkpointer) commit() (bool, error) {
ConditionExpression: aws.String("OwnerID = :ownerID"),
ExpressionAttributeValues: attrVals,
}); err != nil {
// TODO: Add logging for these cases
if awsErr, ok := err.(awserr.Error); ok && awsErr.Code() == conditionalFail && cp.lastUpdate < time.Now().Add(-cp.maxAgeForClientRecord).UnixNano() {

// TODO: investigate if not marking cp.dirty = false here causes duplicates

// If we failed conditional check, and the record has expired, assume ownership has legitimately changed, and don't return the error.
return false, nil
}

Expand Down Expand Up @@ -238,12 +242,8 @@ func (cp *checkpointer) release() error {
ConditionExpression: aws.String("OwnerID = :ownerID"),
ExpressionAttributeValues: attrVals,
}); err != nil {
// TODO: Add logging for these cases
if awsErr, ok := err.(awserr.Error); ok && awsErr.Code() == conditionalFail && cp.lastUpdate < time.Now().Add(-cp.maxAgeForClientRecord).UnixNano() {

// TODO: Investigate if not marking cp.captured = false here causes duplicates

// If we failed conditional check, and the record has expired, assume ownership has legitimately changed, and don't return the error.
// If we failed conditional check, and the record has expired, assume that another client has legitimately siezed the shard.
return nil
}
return fmt.Errorf("error releasing checkpoint: %s", err)
Expand Down
60 changes: 56 additions & 4 deletions checkpoints_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ func TestCheckpointer(t *testing.T) {

// Now actually commit.
mocks.AssertRequestMade(t, mock.(*mocks.MockDynamo), "commit(seq1)", func() {
if _, err = cp.commit(); err != nil {
if _, err = cp.commit(50 * time.Millisecond); err != nil {
t.Errorf("commit seq1 err=%q", err)
}
})
Expand All @@ -44,7 +44,7 @@ func TestCheckpointer(t *testing.T) {

// Since the sequence number hasn't changed, committing shouldn't make a request.
mocks.AssertNoRequestsMade(t, mock.(*mocks.MockDynamo), "commit unchanged sequence number", func() {
if _, err = cp.commit(); err != nil {
if _, err = cp.commit(50 * time.Millisecond); err != nil {
t.Errorf("commit unchanged err=%q", err)
}
})
Expand All @@ -54,7 +54,7 @@ func TestCheckpointer(t *testing.T) {

// committing should trigger a request
mocks.AssertRequestMade(t, mock.(*mocks.MockDynamo), "commit(seq2)", func() {
if _, err = cp.commit(); err != nil {
if _, err = cp.commit(50 * time.Millisecond); err != nil {
t.Errorf("commit seq2 err=%q", err)
}
})
Expand All @@ -65,7 +65,7 @@ func TestCheckpointer(t *testing.T) {

// This should still trigger an update
mocks.AssertRequestMade(t, mock.(*mocks.MockDynamo), "commit(seq3)", func() {
if _, err = cp.commit(); err != nil {
if _, err = cp.commit(50 * time.Millisecond); err != nil {
t.Errorf("commit seq3 err=%q", err)
}
})
Expand Down Expand Up @@ -104,3 +104,55 @@ func TestCheckpointer(t *testing.T) {
}
*/
}

func TestCheckpointer2(t *testing.T) {
table := "checkpoints"
mock := mocks.NewMockDynamo([]string{table})
stats := &NoopStatReceiver{}

cp, err := capture("shard", table, mock, "ownerName", "ownerId", 3*time.Minute, stats)

// Initially, we expect that there is no record, so our new record should have no sequence number
if err != nil {
t.Errorf("current 1 err=%q", err)
}
if cp == nil {
t.Errorf("Should always be able to capture the shard if there is no entry in dynamo")
}
if cp.sequenceNumber != "" {
t.Errorf("sequence number should initially be an empty string")
}

// Update the sequence number. This shouldn't cause any external request.
mocks.AssertNoRequestsMade(t, mock.(*mocks.MockDynamo), "update(seq1)", func() {
cp.update("seq1")
})

// Now actually commit.
mocks.AssertRequestMade(t, mock.(*mocks.MockDynamo), "commit(seq1)", func() {
if _, err = cp.commit(50 * time.Millisecond); err != nil {
t.Errorf("commit seq1 err=%q", err)
}
})

// Call update, but keep the same sequence number
cp.update("seq1")

// Since the sequence number hasn't changed, committing shouldn't make a request.
mocks.AssertNoRequestsMade(t, mock.(*mocks.MockDynamo), "commit unchanged sequence number", func() {
if _, err = cp.commit(50 * time.Millisecond); err != nil {
t.Errorf("commit unchanged err=%q", err)
}
})

// Set cp information to mimic conditions to retain the shard with no data.
cp.maxAgeForClientRecord = 1 * time.Second
cp.lastRecordPassed = time.Now().Add(-501 * time.Millisecond)

// committing should trigger a request
mocks.AssertRequestMade(t, mock.(*mocks.MockDynamo), "commit(seq2)", func() {
if _, err = cp.commit(501 * time.Millisecond); err != nil {
t.Errorf("commit seq2 err=%q", err)
}
})
}
28 changes: 23 additions & 5 deletions shard_consumer.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,6 +156,9 @@ func (k *Kinsumer) consume(shardID string) {

retryCount := 0

// lastSeqToCheckp is used to check if we have more data to checkpoint before we exit
var lastSeqToCheckp string
// lastSeqNum is used to check if a batch of data is the last in the stream
var lastSeqNum string
mainloop:
for {
Expand All @@ -170,7 +173,7 @@ mainloop:
case <-k.stop:
break mainloop
case <-commitTicker.C:
finishCommitted, err := checkpointer.commit()
finishCommitted, err := checkpointer.commit(k.config.commitFrequency)
if err != nil {
k.shardErrors <- shardConsumerError{shardID: shardID, action: "checkpointer.commit", err: err}
return
Expand Down Expand Up @@ -226,7 +229,7 @@ mainloop:
for {
select {
case <-commitTicker.C:
finishCommitted, err := checkpointer.commit()
finishCommitted, err := checkpointer.commit(k.config.commitFrequency)
if err != nil {
k.shardErrors <- shardConsumerError{shardID: shardID, action: "checkpointer.commit", err: err}
return
Expand All @@ -241,6 +244,8 @@ mainloop:
checkpointer: checkpointer,
retrievedAt: retrievedAt,
}:
checkpointer.lastRecordPassed = time.Now() // Mark the time so we don't retain shards when we're too slow to do so
lastSeqToCheckp = aws.StringValue(record.SequenceNumber)
break RecordLoop
}
}
Expand All @@ -255,24 +260,37 @@ mainloop:

// commit first in case the checkpointer has been updates since the last commit.
checkpointer.commitIntervalCounter = 0 // Reset commitIntervalCounter to avoid retaining ownership if there's no new sequence number
checkpointer.commit()

_, err1 := checkpointer.commit(0 * time.Millisecond)
if err1 != nil {
k.shardErrors <- shardConsumerError{shardID: shardID, action: "checkpointer.commit", err: err}
return
}
// Resume commit loop for some time, ensuring that we don't retain ownership unless there's a new sequence number.
timeoutCounter := 0

// If we have committed the last sequence number returned to the user, just return. Otherwise, keep committing until we reach that state
if !checkpointer.dirty && checkpointer.sequenceNumber == lastSeqToCheckp {
return
}

for {
select {
case <-commitTicker.C:
timeoutCounter += int(k.config.commitFrequency)
checkpointer.commitIntervalCounter = 0
finishCommitted, err := checkpointer.commit()
// passing 0 to commit ensures we no longer retain the shard.
finishCommitted, err := checkpointer.commit(0 * time.Millisecond)
if err != nil {
k.shardErrors <- shardConsumerError{shardID: shardID, action: "checkpointer.commit", err: err}
return
}
if finishCommitted {
return
}
// Once we have committed the last sequence Number we passed to the user, return.
if !checkpointer.dirty && checkpointer.sequenceNumber == lastSeqToCheckp {
return
}
if timeoutCounter >= int(k.maxAgeForClientRecord/2) {
return
}
Expand Down
83 changes: 65 additions & 18 deletions shard_consumer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,8 +85,10 @@ func TestForcefulOwnershipChange(t *testing.T) {
maxAge2 := 500 * time.Millisecond
config2 := config.WithClientRecordMaxAge(&maxAge2)

kinsumer1, err := NewWithInterfaces(k, dynamo, streamName, *applicationName, "client_1", config1)
kinsumer2, err := NewWithInterfaces(k, dynamo, streamName, *applicationName, "client_2", config2)
kinsumer1, err1 := NewWithInterfaces(k, dynamo, streamName, *applicationName, "client_1", config1)
kinsumer2, err2 := NewWithInterfaces(k, dynamo, streamName, *applicationName, "client_2", config2)
require.NoError(t, err1)
require.NoError(t, err2)

desc, err := k.DescribeStream(&kinesis.DescribeStreamInput{
StreamName: &streamName,
Expand All @@ -103,33 +105,83 @@ func TestForcefulOwnershipChange(t *testing.T) {

kinsumer1ResultBeforeOwnerChange := readEventsToSlice(kinsumer1.records, 5*time.Second)

assert.Equal(t, 2000, len(kinsumer1ResultBeforeOwnerChange)) // Assert that we get 100 results
assert.Equal(t, 2000, len(kinsumer1ResultBeforeOwnerChange))

kinsumer2.waitGroup.Add(1) // consume will mark waitgroup as done on exit, so we add to it to avoid a panic
go kinsumer2.consume(shard)

time.Sleep(1500 * time.Millisecond) // Sleep for long enough that an ownership change will happen
// Because we retain the shard if no data is coming through, we mimic a stale client scenario by sending data but not acking
lastK1Record := &consumedRecord{
checkpointer: &checkpointer{},
}
OwnerChangeLoop:
for {
spamStreamModified(t, k, 1, streamName, 9999)
getEventLoop:
select {
case k1record := <-kinsumer1.records: // if kinsumer1 gets it, don't ack
lastK1Record = k1record
break getEventLoop
case k2record := <-kinsumer2.records: // if kisumer2 gets it, ownership has changed. Ack then move on to the test.
k2record.checkpointer.update(aws.StringValue(k2record.record.SequenceNumber))
// because this may be called with no genuine record to k1, we use the k2 sequence number.
// this shouldn't make a difference since this commit will fail.
lastK1Record.checkpointer.update(aws.StringValue(k2record.record.SequenceNumber)) // Ack the last k1 record we have, to instigate behaviour we would see for that client
break OwnerChangeLoop
}
time.Sleep(120 * time.Millisecond)
}

go spamStreamModified(t, k, 2000, streamName, 4000)
time.Sleep(300 * time.Millisecond)

go spamStreamModified(t, k, 1000, streamName, 5000)

resultsAfterOwnerChange := readMultipleToSlice([]chan *consumedRecord{kinsumer1.records, kinsumer2.records}, 5*time.Second)
kinsumer1ResultAfterOwnerChange := resultsAfterOwnerChange[0]
kinsumer2ResultAfterOwnerChange := resultsAfterOwnerChange[1]
kinsumer1ResultAfterOwnerChangePreClean := resultsAfterOwnerChange[0]
kinsumer2ResultAfterOwnerChangePreClean := resultsAfterOwnerChange[1]

// clean out the records we just used to instigate a change in ownership
kinsumer1ResultAfterOwnerChange := make([]*consumedRecord, 0)
for _, val := range kinsumer1ResultAfterOwnerChangePreClean {
if string(val.record.Data) != "9999" {
kinsumer1ResultAfterOwnerChange = append(kinsumer1ResultAfterOwnerChange, val)
}
}

kinsumer2ResultAfterOwnerChange := make([]*consumedRecord, 0)
for _, val := range kinsumer2ResultAfterOwnerChangePreClean {
if string(val.record.Data) != "9999" {
kinsumer2ResultAfterOwnerChange = append(kinsumer2ResultAfterOwnerChange, val)
}
}

/*
// Leaving this here but commented out since it's useful in inspecting the behaviour when something does look off.
investigationSlice := make([]string, 0)
for _, record := range kinsumer1ResultAfterOwnerChange {
investigationSlice = append(investigationSlice, string(record.record.Data))
// Leaving this here but commented out since it's useful in inspecting the behaviour when something does look off.
if len(resultsAfterOwnerChange) > 0 {
investigationSlice := make([]string, 0)
for _, record := range kinsumer1ResultAfterOwnerChange {
investigationSlice = append(investigationSlice, string(record.record.Data))
}
fmt.Println(investigationSlice)
}
*/

assert.Equal(t, 0, len(kinsumer1ResultAfterOwnerChange))
assert.Equal(t, 2000, len(kinsumer2ResultAfterOwnerChange))
assert.Equal(t, 1000, len(kinsumer2ResultAfterOwnerChange))

dupes := make([]string, 0)

for _, val1 := range kinsumer1ResultAfterOwnerChange {
for _, val2 := range kinsumer2ResultAfterOwnerChange {
if string(val1.record.Data) == string(val2.record.Data) {
dupes = append(dupes, string(val1.record.Data))
}
}
}

// Check that every expected value is present in the results
missingIntegers := make([]int, 0)
for i := 4000; i < 6000; i++ {
for i := 5000; i < 6000; i++ {
present := false
for _, val := range kinsumer2ResultAfterOwnerChange {
if string(val.record.Data) == fmt.Sprint(i) {
Expand Down Expand Up @@ -519,8 +571,6 @@ func TestConsumerStopStart(t *testing.T) {

}

// TODO: For some reason, using the readMultipleToSlice function in this test causes us to receive no data for the first two consumers. Figure out why and resolve.
// (ideally we start to read the data first then we start to stop and consumers)
// TestMultipleConsumerStopStart tests the same thing as TestConsumerStopStart, but for the scenario where there are multiple clients vying for control of the same shard.
// This is a common scenario when shards are merged, because the reported shard count will change relatively slowly over time (seconds), and for a period different clients will report different shard counts
// The aim of this test is to give us some more robust assurance that there are no additional issues for multiple consumers on a shard which aren't caught when we only have one consumer at a time.
Expand Down Expand Up @@ -609,7 +659,6 @@ func TestMultipleConsumerStopStart(t *testing.T) {
}
}()

// TODO: Investigate why starting this process earlier results in an empty result set for one of the consumers.
results := readMultipleToSlice([]chan *consumedRecord{kinsumer1.records, kinsumer2.records, kinsumer3.records}, 10*time.Second)

kinsumer1Result := results[0]
Expand All @@ -621,8 +670,6 @@ func TestMultipleConsumerStopStart(t *testing.T) {
assert.NotEqual(t, 0, len(kinsumer2Result))
assert.NotEqual(t, 0, len(kinsumer3Result))

fmt.Println("Lengths: ", len(kinsumer1Result), len(kinsumer2Result), len(kinsumer3Result)) // TODO: Remove this

// Check for dupes within each client's results
kinsumer1Dupes := getDupesFromSlice(kinsumer1Result)
kinsumer2Dupes := getDupesFromSlice(kinsumer2Result)
Expand Down

0 comments on commit 4e19415

Please sign in to comment.