forked from cockroachdb/cockroach
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
changefeedccl: fix initial scan checkpointing
Initially, all span initial resolved timestamps are kept as zero upon resuming a job since initial resolved timestamps are set as initial high water which remains zero until initial scan is completed. However, since cockroachdb@afb95b1, we began reloading checkpoint timestamps instead of setting them all to zero at the start. In PR cockroachdb#102717, we introduced a mechanism to reduce message duplicates by re-loading job progress upon resuming which largely increased the likelihood of this bug. These errors could lead to incorrect frontier and missing events during initial scans. This patches changes how we initialize initial high water and frontier by initializing it as zero if there are any zero initial high water in initial resolved timestamps. Fixes: cockroachdb#123371 Release note (enterprise change): Fixed a bug in v22.2+ where long running initial scans may incorrectly restore checkpoint job progress and drop events during node / changefeed restart. This issue was most likely to occur in clusters with: 1. changefeed.shutdown_checkpoint.enabled (v23.2) is set 2. Multiple table targets in a changefeed 3. Low changefeed.frontier_checkpoint_frequency or low changefeed.frontier_highwater_lag_checkpoint_threshold.
- Loading branch information
Showing
3 changed files
with
142 additions
and
3 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
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,129 @@ | ||
// Copyright 2024 The Cockroach Authors. | ||
// | ||
// Licensed as a CockroachDB Enterprise file under the Cockroach Community | ||
// License (the "License"); you may not use this file except in compliance with | ||
// the License. You may obtain a copy of the License at | ||
// | ||
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt | ||
|
||
package changefeedccl | ||
|
||
import ( | ||
"github.com/cockroachdb/cockroach/pkg/roachpb" | ||
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" | ||
"github.com/cockroachdb/cockroach/pkg/util/hlc" | ||
"github.com/stretchr/testify/require" | ||
"testing" | ||
) | ||
|
||
// TestSetupSpansAndFrontier tests that the setupSpansAndFrontier function | ||
// correctly sets up frontier for the changefeed aggregator frontier. | ||
func TestSetupSpansAndFrontier(t *testing.T) { | ||
for _, tc := range []struct { | ||
name string | ||
expectedFrontier hlc.Timestamp | ||
watches []execinfrapb.ChangeAggregatorSpec_Watch | ||
}{ | ||
{ | ||
name: "new initial scan", | ||
expectedFrontier: hlc.Timestamp{}, | ||
watches: []execinfrapb.ChangeAggregatorSpec_Watch{ | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, | ||
InitialResolved: hlc.Timestamp{}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, | ||
InitialResolved: hlc.Timestamp{}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("d")}, | ||
InitialResolved: hlc.Timestamp{}, | ||
}, | ||
}, | ||
}, | ||
{ | ||
name: "incomplete initial scan with non-empty initial resolved in the end", | ||
expectedFrontier: hlc.Timestamp{}, | ||
watches: []execinfrapb.ChangeAggregatorSpec_Watch{ | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, | ||
InitialResolved: hlc.Timestamp{WallTime: 5}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, | ||
InitialResolved: hlc.Timestamp{}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("d")}, | ||
InitialResolved: hlc.Timestamp{WallTime: 20}, | ||
}, | ||
}, | ||
}, | ||
{ | ||
name: "incomplete initial scan with non-empty initial resolved in the beginning", | ||
expectedFrontier: hlc.Timestamp{}, | ||
watches: []execinfrapb.ChangeAggregatorSpec_Watch{ | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, | ||
InitialResolved: hlc.Timestamp{}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, | ||
InitialResolved: hlc.Timestamp{WallTime: 10}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("d")}, | ||
InitialResolved: hlc.Timestamp{WallTime: 20}, | ||
}, | ||
}, | ||
}, | ||
{ | ||
name: "incomplete initial scan with empty initial resolved in the end", | ||
expectedFrontier: hlc.Timestamp{}, | ||
watches: []execinfrapb.ChangeAggregatorSpec_Watch{ | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, | ||
InitialResolved: hlc.Timestamp{WallTime: 10}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, | ||
InitialResolved: hlc.Timestamp{WallTime: 20}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("d")}, | ||
InitialResolved: hlc.Timestamp{}, | ||
}, | ||
}, | ||
}, | ||
{ | ||
name: "complete initial scan", | ||
expectedFrontier: hlc.Timestamp{WallTime: 5}, | ||
watches: []execinfrapb.ChangeAggregatorSpec_Watch{ | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b")}, | ||
InitialResolved: hlc.Timestamp{WallTime: 10}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}, | ||
InitialResolved: hlc.Timestamp{WallTime: 20}, | ||
}, | ||
{ | ||
Span: roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("d")}, | ||
InitialResolved: hlc.Timestamp{WallTime: 5}, | ||
}, | ||
}, | ||
}, | ||
} { | ||
t.Run(tc.name, func(t *testing.T) { | ||
ca := &changeAggregator{ | ||
spec: execinfrapb.ChangeAggregatorSpec{ | ||
Watches: tc.watches, | ||
}, | ||
} | ||
_, err := ca.setupSpansAndFrontier() | ||
require.NoError(t, err) | ||
require.Equal(t, tc.expectedFrontier, ca.frontier.Frontier()) | ||
}) | ||
} | ||
} |