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
changefeedccl: add changefeed range distribution strategies #115166
changefeedccl: add changefeed range distribution strategies #115166
Conversation
49f61a9
to
20ae679
Compare
4a78157
to
e1dfae5
Compare
1f0b87f
to
f41649b
Compare
f41649b
to
861c9e8
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 6 of 10 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @herkolategan, @jayshrivastava, @michae2, and @srosenberg)
pkg/ccl/changefeedccl/changefeed_dist.go
line 328 at r1 (raw file):
// set of nodes to distribute work to. However, changefeeds will try to // distribute work evenly across this set of nodes. DISTRIBUTION_STRATEGY_BALANCED_SIMPLE
it's a bit more idiomatic to have type rangeDistributionStrategy int
and then define distributionBlah constants.
(All caps is only used in proto enums, afaik)
pkg/ccl/changefeedccl/changefeed_dist.go
line 341 at r1 (raw file):
"none", map[int64]string{ int64(DISTRIBUTION_STRATEGY_NONE): "none",
maybe better name would be just "default" -- none reads like "no distribution at all".
pkg/ccl/changefeedccl/changefeed_dist.go
line 333 at r2 (raw file):
) var rangeDistributionStrategy = settings.RegisterEnumSetting(
// Let's add a todo to deprecate this setting in favor of explicit option.
pkg/ccl/changefeedccl/changefeed_dist.go
line 406 at r2 (raw file):
} if knobs, ok := maybeCfKnobs.(*TestingKnobs); ok && knobs != nil &&
why not (.TestingKnobs) abouve (maybeCfKnobs initialization)?
861c9e8
to
c23d322
Compare
ecd2986
to
89fc22b
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @herkolategan, @michae2, @miretskiy, and @srosenberg)
pkg/ccl/changefeedccl/changefeed_dist.go
line 328 at r1 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
it's a bit more idiomatic to have
type rangeDistributionStrategy int
and then define distributionBlah constants.
(All caps is only used in proto enums, afaik)
Done.
pkg/ccl/changefeedccl/changefeed_dist.go
line 341 at r1 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
maybe better name would be just "default" -- none reads like "no distribution at all".
Done.
pkg/ccl/changefeedccl/changefeed_dist.go
line 333 at r2 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
// Let's add a todo to deprecate this setting in favor of explicit option.
Done.
pkg/ccl/changefeedccl/changefeed_dist.go
line 406 at r2 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
why not (.TestingKnobs) abouve (maybeCfKnobs initialization)?
Done.
89fc22b
to
63121a7
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 2 of 10 files at r1, 7 of 7 files at r3.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @herkolategan, @jayshrivastava, @michae2, and @srosenberg)
pkg/ccl/changefeedccl/changefeed_dist.go
line 394 at r3 (raw file):
planCtx = dsp.NewPlanningCtxWithOracle(ctx, execCtx.ExtendedEvalContext(), nil, /* planner */ blankTxn, sql.DistributionType(distMode), oracle, locFilter) spanPartitions, err = dsp.PartitionSpans(ctx, planCtx, trackedSpans)
unless I'm misreading, looks like planCtx construction and dsp.PartitionSpans calls are identical between
this and the above case. Does it make sense to pull it out? Construct planCtx, and get the spans regardless of the rangeDistribution; and then just do rebalancing bit?
pkg/ccl/changefeedccl/changefeed_dist.go
line 407 at r3 (raw file):
} default: return nil, nil, errors.AssertionFailedf("could not reconcile dist strategy %d and dist mode %d",
nit: not sure about reconcile part... perhaps just return "unsupported distribution strategy XXX"
pkg/cmd/roachtest/tests/cdc.go
line 641 at r3 (raw file):
settings.ClusterSettings["changefeed.slow_span_log_threshold"] = "30s" settings.ClusterSettings["server.child_metrics.enabled"] = "true" settings.ClusterSettings["changefeed.range_distribution_strategy"] = "balanced_simple"
do you want to add some metamorphic goodness, add cdcFeatureFlags and change the strategy in the applySettings function instead ?
pkg/ccl/changefeedccl/changefeed_dist_test.go
line 94 at r3 (raw file):
input: partitions( mkPart(1, mkSpan("a", "c"), mkSpan("e", "p"), mkSpan("r", "z")), mkPart(2),
I'm not even sure sql planning could return a span partition w/out spans.
I'm actually not sure it can ; and thus this test case is probably unrealistic.
When you implement "balance_full" mode and you get "all nodes", then I think this would make sense.
It's probably okay to keep this test case -- but I just want to point out potential issue.
(And... I realize it's a code move)
pkg/ccl/changefeedccl/changefeed_dist_test.go
line 108 at r3 (raw file):
mkPart(3, mkSpan("c", "e"), mkSpan("p", "r")), ), },
I realize this is a code move fromt he previous test, but... no good deed goes unpunished.
Would love to see more test cases.
Quick snippet to generate span consisting of a single letter:
for r := 'A'; r <= 'Z'\-1; r++ {
var sp roachpb.Span
sp.Key = append(sp.Key, byte(r))
sp.EndKey = append(sp.EndKey, byte(r+1))
...
So, perhaps test cases such as:
- bunch of ranges on a single node (say A-B, B-C, ... W-X), each letter), and like 1 on the other nodes.
- A test case with just 1 node. Basically, idempotency test.
- A test case with 5 nodes.
- A test case with ... like 25 nodes -- all nodes having 1 range (you'll need a helper to generate that).
- Perhaps a 25 node test, but with imbalance.
- A test case where the span list is not sorted alphabetically (I know that sql will return sorted spans; but I want to make sure we don't depend on that).
pkg/ccl/changefeedccl/changefeed_dist_test.go
line 150 at r3 (raw file):
// nodes using the supplied localities. It creates a table 'x' with 64 ranges // and distributes the leaseholders across the first 6 nodes using an // exponential distribution.
can you specify what "exponential" distribution means here (I guess I'm just
looking for explicit "[]int{2, 2, 4, 8, 16, 32, 0, 0})" counts.
pkg/ccl/changefeedccl/changefeed_dist_test.go
line 230 at r3 (raw file):
// countRanges returns an array where each index i stores the ranges assigned to node i. func (rdt *rangeDistributionTester) countRanges(partitions []sql.SpanPartition) []int {
nit: countNodeRanges? Or countRangesPerNode?
pkg/ccl/changefeedccl/changefeed_dist_test.go
line 258 at r3 (raw file):
} func TestChangefeedDistributionStrategy(t *testing.T) {
this is so nice!
c80c8f4
to
5fc67eb
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @herkolategan, @michae2, @miretskiy, and @srosenberg)
pkg/ccl/changefeedccl/changefeed_dist.go
line 394 at r3 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
unless I'm misreading, looks like planCtx construction and dsp.PartitionSpans calls are identical between
this and the above case. Does it make sense to pull it out? Construct planCtx, and get the spans regardless of the rangeDistribution; and then just do rebalancing bit?
Done.
pkg/cmd/roachtest/tests/cdc.go
line 641 at r3 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
do you want to add some metamorphic goodness, add cdcFeatureFlags and change the strategy in the applySettings function instead ?
Done.
pkg/ccl/changefeedccl/changefeed_dist_test.go
line 94 at r3 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
I'm not even sure sql planning could return a span partition w/out spans.
I'm actually not sure it can ; and thus this test case is probably unrealistic.
When you implement "balance_full" mode and you get "all nodes", then I think this would make sense.It's probably okay to keep this test case -- but I just want to point out potential issue.
(And... I realize it's a code move)
I'll keep it here :)
pkg/ccl/changefeedccl/changefeed_dist_test.go
line 108 at r3 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
I realize this is a code move fromt he previous test, but... no good deed goes unpunished.
Would love to see more test cases.Quick snippet to generate span consisting of a single letter:
for r := 'A'; r <= 'Z'\-1; r++ { var sp roachpb.Span sp.Key = append(sp.Key, byte(r)) sp.EndKey = append(sp.EndKey, byte(r+1)) ...
So, perhaps test cases such as:
- bunch of ranges on a single node (say A-B, B-C, ... W-X), each letter), and like 1 on the other nodes.
- A test case with just 1 node. Basically, idempotency test.
- A test case with 5 nodes.
- A test case with ... like 25 nodes -- all nodes having 1 range (you'll need a helper to generate that).
- Perhaps a 25 node test, but with imbalance.
- A test case where the span list is not sorted alphabetically (I know that sql will return sorted spans; but I want to make sure we don't depend on that).
Done. Added these cases!
pkg/ccl/changefeedccl/changefeed_dist_test.go
line 150 at r3 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
can you specify what "exponential" distribution means here (I guess I'm just
looking for explicit "[]int{2, 2, 4, 8, 16, 32, 0, 0})" counts.
Done.
pkg/ccl/changefeedccl/changefeed_dist_test.go
line 258 at r3 (raw file):
Previously, miretskiy (Yevgeniy Miretskiy) wrote…
this is so nice!
Done.
5fc67eb
to
fe3e4b7
Compare
67c252f
to
09b9eeb
Compare
This commit introduces several changes. It makes `replicaoracle.BinPackingChoice` the explicit default for changefeeds. Previously, we would use `physicalplan.DefaultReplicaChooser`, which would point to the bin packing oracle. This change is better since it gives the changefeed package more control over its dependencies. A new cluster setting `changefeed.default_range_distribution_strategy` is added to specify how changefeeds should distribute work. In the default case, `none`, we defer to distsql to select nodes and distribute work among them. This is the same behavior as running a changefeed today. In the other case, `balanced_simple`, we still let distsql choose nodes for us, but we attempt to evently distribute ranges over them. This case is exactly the same as setting `changefeed.balance_range_distribution.enable = true`, but it is allowed to be used outside of initial scans (`changefeed.balance_range_distribution.enable` was only allowed with `initial_scan=only`. This change also deprecates `changefeed.balance_range_distribution.enable` in favor of `changefeed.default_range_distribution_strategy` because the latter can be expanded in the future to add more functionality. This change also moves changefeed distribution tests to a dedicated file: `changefeed_dist_test.go`. This file contains a tester which can be used to start a cluster with a given topology and range distribution. Changefeed tests can use the tester to perform various load balancing strategies. Right now, the new tests assert how ranges are distributed after planning changefeeds, which implicitly tests code outside of the changefeedccl package (ie. `dsp.PartitionSpans()`). However, this is important since changefeeds rely on the behavior of this black box to perform well. In the future, cdc will implement more distribution strategies and likely implement its own partitioning logic. Tests for these strategies can be added in this file. Over time, cdc can move away from distsql planning. Informs: cockroachdb#113898 Epic: None Release note (enterprise change): `changefeed.balance_range_distribution.enable` is now deprecated. Users should use a new cluster setting `changefeed.default_range_distribution_strategy` instead. `changefeed.default_range_distribution_strategy='balanced_simple'` has the same effect as setting `changefeed.balance_range_distribution.enable=true`. It does not require `initial_scan='only'`, which was required by the old setting.
This change adds a new metamorphic flag in cdc roachtests which choses a distribution strategy randomly. Release note: None
09b9eeb
to
0b1f895
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 2 of 5 files at r4, 1 of 3 files at r5, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @herkolategan, @michae2, and @srosenberg)
bors r+ |
Build succeeded: |
changefeedccl: add changefeed range distribution strategies
This commit introduces several changes.
It makes
replicaoracle.BinPackingChoice
the explicit default for changefeeds.Previously, we would use
physicalplan.DefaultReplicaChooser
, which would pointto the bin packing oracle. This change is better since it gives the changefeed package more control over its dependencies.
A new cluster setting
changefeed.default_range_distribution_strategy
is added tospecify how changefeeds should distribute work. In the default case,
none
, we defer to distsql to select nodes and distribute work among them. Thisis the same behavior as running a changefeed today.
In the other case,
balanced_simple
, we still let distsql choose nodes for us, but we attemptto evently distribute ranges over them. This case is exactly the same as setting
changefeed.balance_range_distribution.enable = true
, but it is allowed to be usedoutside of initial scans (
changefeed.balance_range_distribution.enable
was only allowedwith
initial_scan=only
.This change also deprecates
changefeed.balance_range_distribution.enable
in favor ofchangefeed.default_range_distribution_strategy
because the latter can be expandedin the future to add more functionality.
This change also moves changefeed distribution tests to a dedicated file:
changefeed_dist_test.go
. This file contains a tester which can be used tostart a cluster with a given topology and range distribution. Changefeed tests
can use the tester to perform various load balancing strategies. Right now, the new
tests assert how ranges are distributed after planning changefeeds, which implicitly tests
code outside of the changefeedccl package (ie.
dsp.PartitionSpans()
). However,this is important since changefeeds rely on the behavior of this black box to perform well.
In the future, cdc will implement more distribution strategies and likely implement its
own partitioning logic. Tests for these strategies can be added in this file. Over time,
cdc can move away from distsql planning.
Informs: #113898
Epic: None
Release note (enterprise change):
changefeed.balance_range_distribution.enable
is now deprecated.Users should use a new cluster setting
changefeed.default_range_distribution_strategy
instead.
changefeed.default_range_distribution_strategy='balanced_simple'
has the sameeffect as setting
changefeed.balance_range_distribution.enable=true
. It does not requireinitial_scan='only'
, which was required by the old setting.roachtest/cdc: metamorphically determine range distribution strategy
This change adds a new metamorphic flag in cdc roachtests which choses
a distribution strategy randomly.
Release note: None