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

CDC as Export / Job Never Completes #86828

Closed
nollenr opened this issue Aug 24, 2022 · 3 comments · Fixed by #87464
Closed

CDC as Export / Job Never Completes #86828

nollenr opened this issue Aug 24, 2022 · 3 comments · Fixed by #87464
Assignees
Labels
C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. T-cdc

Comments

@nollenr
Copy link
Collaborator

nollenr commented Aug 24, 2022

Describe the problem
Attempting to perform CDC as an EXPORT with format=csv, initial_scan_only, compression=gzip. The job is created, output is produced in the S3 bucket, but the job never completes.

To Reproduce
Create a table:

  CREATE TABLE public.table0006 (
      column1 UUID NOT NULL DEFAULT gen_random_uuid(),
      create_time TIMESTAMPTZ NOT NULL DEFAULT now():::TIMESTAMPTZ,
      region STRING NOT NULL DEFAULT crdb_internal.locality_value('region':::STRING),
      az STRING NOT NULL DEFAULT crdb_internal.locality_value('zone':::STRING),
      state STRING(2) NOT NULL,
      column_2 STRING NULL,
      column_3 STRING NULL,
      column_4 STRING NULL,
      column_5 STRING NULL,
      column_6 STRING NULL,
      column_7 STRING NULL,
      column_8 STRING NULL,
      column_9 STRING NULL,
      column_10 STRING NULL,
      column_11 STRING NULL,
      column_12 STRING NULL,
      column_13 STRING NULL,
      column_14 STRING NULL,
      column_15 STRING NULL,
      column_16 STRING NULL,
      column_17 STRING NULL,
      column_18 STRING NULL,
      column_19 STRING NULL,
      column_20 STRING NULL,
      column_21 STRING NULL,
      column_22 STRING NULL,
      column_23 STRING NULL,
      column_24 STRING NULL,
      column_25 STRING NULL,
      CONSTRAINT "primary" PRIMARY KEY (column1 ASC)
  );
  ALTER TABLE nvidia.public.table0006 CONFIGURE ZONE USING
      gc.ttlseconds = 90000;

Imported data

IMPORT INTO table0006
  CSV DATA (
    's3://nollen-cluster-backup-bucket/export-table0006-86GB/*.csv?AWS_ACCESS_KEY_ID=AKIAX4BORNLV2ZEKHCPV&AWS_SECRET_ACCESS_KEY=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'
  );
 select count(*) from table0006;
    count
-------------
  111042010

Created a CDC Change feed:

CREATE CHANGEFEED FOR table0006
INTO 's3://nollen-cluster-backup-bucket/cdc-avro-export?AWS_ACCESS_KEY_ID={ID}&AWS_SECRET_ACCESS_KEY={secret ID}'
with format=csv, compression=gzip, initial_scan_only;

The change feed runs successfully and produces the S3 output:

root@:26257/nvidia> show job 790720365774700545;
        job_id       |  job_type  |                                                                                                            description                                                                                                            | statement | user_name | status  | running_status |          created           |          started           | finished |          modified          | fraction_completed | error | coordinator_id |      trace_id       |          last_run          |          next_run          | num_runs | execution_errors
---------------------+------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+-----------+---------+----------------+----------------------------+----------------------------+----------+----------------------------+--------------------+-------+----------------+---------------------+----------------------------+----------------------------+----------+-------------------
  790720365774700545 | CHANGEFEED | CREATE CHANGEFEED FOR TABLE table0006 INTO 's3://nollen-cluster-backup-bucket/cdc-avro-export?AWS_ACCESS_KEY_ID=AKIAX4BORNLV2ZEKHCPV&AWS_SECRET_ACCESS_KEY=redacted' WITH compression = 'gzip', format = 'csv', initial_scan_only |           | root      | running | NULL           | 2022-08-24 22:11:45.375601 | 2022-08-24 22:11:45.406135 | NULL     | 2022-08-24 22:11:45.387582 |                  0 |       |              1 | 3429844606177434773 | 2022-08-24 22:11:45.406136 | 2022-08-24 22:12:15.406136 |        1 | {}

The size and number of files in the S3 bucket has stopped changing which indicates to me, the process export of data is complete.
Size of S3 Exports: 205.2MB
Number of Files: 89

Size of table in CRDB: 83.3GiB
Number of Ranges: 692

I have attempted the process several times and the number of files and size of bucket is similar between executions. I am able to view the data in the csv files in S3 (using S3 Query) and the data is valid.

Expected behavior
I expect the "CDC Export" to write the contents of the table, in csv format, to my S3 bucket. Since the with intitial_scan_only was supplied, I expect the CDC job to complete when the data has been output to the S3 bucket.

Environment:

[ec2-user@ip-192-168-2-68 ~]$ cockroach version
Build Tag:        v22.1.5
Build Time:       2022/07/28 14:58:04
Distribution:     CCL
Platform:         linux amd64 (x86_64-pc-linux-gnu)
Go Version:       go1.17.11
C Compiler:       gcc 6.5.0
Build Commit ID:  a30a663cbd9323d34d50f343dd038af64671e25f
Build Type:       release
  • Server OS: AWS ec2 /m5a.large instance type / running amazon linux amzn2-ami-kernel-5.10-hvm-2.0.20220805.0-x86_64-gp2
  • Cluster: 3 nodes / 200GB storage per node
  • Client app: cockroach sql

debug zip is being upload to GoogleDrive.

Jira issue: CRDB-18949

Epic CRDB-19123

@nollenr nollenr added the C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. label Aug 24, 2022
@HonoreDB HonoreDB self-assigned this Aug 26, 2022
@HonoreDB
Copy link
Contributor

Logs showed changefeed processors waiting hours for their memory quota, a high count of ranges whose backfill never completed, and some messages still in the buffer. The issue goes away if you increase changefeed.backfill.concurrent_scan_requests and changefeed.memory.per_changefeed_limit before running the changefeed. This is at minimum an observability bug but also seems likely to indicate something more serious, as changing those settings should only result in a linear decrease in runtime but instead seemed to have superlinear effects.

@blathers-crl blathers-crl bot added the T-cdc label Aug 29, 2022
@blathers-crl
Copy link

blathers-crl bot commented Aug 29, 2022

cc @cockroachdb/cdc

@miretskiy
Copy link
Contributor

I think I know what the underlying root cause is; and I suspect that the issue is pretty serious, and most likely
triggered by the recent reduction of default memory limits for changefeed.

goroutine 1763350 [select, 32 minutes]:
github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent.(*blockingBuffer).Get(0xc00b7611f0, {0x6355138, 0xc01be46ed0})
        github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent/blocking_buffer.go:140 +0xf3
github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent.(*throttlingBuffer).Get(0xc04243f398, {0x6355138, 0xc01be46ed0})
        github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent/throttling_buffer.go:30 +0xa3
github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*changeAggregator).tick(0xc002409000)
        github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeed_processors.go:540 +0x72
github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*changeAggregator).Next(0xc002409000)
        github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeed_processors.go:508 +0x6e
github.com/cockroachdb/cockroach/pkg/sql/execinfra.Run({0x6355138, 0xc01be46ed0}, {0x637ccb8, 0xc002409000}, {0x62d1e68, 0xc00106e700})
        github.com/cockroachdb/cockroach/pkg/sql/execinfra/base.go:187 +0x55
github.com/cockroachdb/cockroach/pkg/sql/execinfra.(*ProcessorBaseNoHelper).Run(0xc002409000, {0x6355138, 0xc004ce6e40})
        github.com/cockroachdb/cockroach/pkg/sql/execinfra/processorsbase.go:754 +0x5b
github.com/cockroachdb/cockroach/pkg/sql/flowinfra.(*FlowBase).StartInternal.func1(0xc003e5e790)
        github.com/cockroachdb/cockroach/pkg/sql/flowinfra/flow.go:419 +0x48
created by github.com/cockroachdb/cockroach/pkg/sql/flowinfra.(*FlowBase).StartInternal
        github.com/cockroachdb/cockroach/pkg/sql/flowinfra/flow.go:418 +0x36f

// Multiple go routines stuck w/ the stacktrace below...
goroutine 1763338 [select, 32 minutes]:
github.com/cockroachdb/cockroach/pkg/util/quotapool.(*AbstractPool).Acquire(0xc0053a3e40, {0x6355090, 0xc0073f7780}, {0x62d04f0, 0xc003b9d7a0})
        github.com/cockroachdb/cockroach/pkg/util/quotapool/quotapool.go:281 +0x77c
github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent.(*blockingBuffer).Add(0xc00b7611f0, {0x6355090, 0xc0073f7780}, {{{0xc064db001f, 0x16, 0x111fe1}, {{0xc064db003f, 0x9c3, 0x111fc1}, {0x170e16efa6428485, ...}}}, ...})
        github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent/blocking_buffer.go:212 +0x23f
github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed.slurpScanResponse({0x6355090, 0xc0073f7780}, {0x7fcf52c33128, 0xc04243f398}, 0x45dbad, {0xc0016b9aa8, 0x62f5020, 0x0}, 0x0, {{0xc002821c50, ...}, ...})
        github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed/scanner.go:251 +0x38e
github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed.(*scanRequestScanner).exportSpan(0xc009ccb3e0, {0x6355090, 0xc0073f7780}, {{0xc002821c50, 0x15, 0x18}, {0xc0405dd0e0, 0x15, 0x18}}, {0x170e5bfdc7a44c4b, ...}, ...)
        github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed/scanner.go:153 +0x576
github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed.(*scanRequestScanner).Scan.func1({0x6355090, 0xc0073f7780})
        github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed/scanner.go:100 +0x1de
github.com/cockroachdb/cockroach/pkg/util/ctxgroup.Group.GoCtx.func1()
        github.com/cockroachdb/cockroach/pkg/util/ctxgroup/ctxgroup.go:169 +0x25
golang.org/x/sync/errgroup.(*Group).Go.func1()
        golang.org/x/sync/errgroup/external/org_golang_x_sync/errgroup/errgroup.go:57 +0x67
created by golang.org/x/sync/errgroup.(*Group).Go
        golang.org/x/sync/errgroup/external/org_golang_x_sync/errgroup/errgroup.go:54 +0x92

miretskiy pushed a commit to miretskiy/cockroach that referenced this issue Sep 6, 2022
Ensure that out of quota events are not lost and propagated
if necessary to the consumer.

Prior to this change, it was possible for an out of quota
notification to be "lost" because "blocked" bit would be cleared
out when an event was enqueued.
Instead of relying on a boolean bit, we now keep track of the
number of consumers currently blocked, and issue flush request
if there are non-zero blocked consumers with zero events
currently queued.

Fixes cockroachdb#86828

Release justification: bug fix
Release note: None
miretskiy pushed a commit to miretskiy/cockroach that referenced this issue Sep 7, 2022
Ensure that out of quota events are not lost and propagated
if necessary to the consumer.

Prior to this change, it was possible for an out of quota
notification to be "lost" because "blocked" bit would be cleared
out when an event was enqueued.
Instead of relying on a boolean bit, we now keep track of the
number of consumers currently blocked, and issue flush request
if there are non-zero blocked consumers with zero events
currently queued.

Fixes cockroachdb#86828

Release justification: bug fix
Release note: None
craig bot pushed a commit that referenced this issue Sep 8, 2022
86734: kvserver: avoid race in preSplitApply r=erikgrinaker a=tbg

When `splitPreApply` has to handle a right-hand side replica that is
newer than the split, the split needs to throw the "snapshot" it was
going to install into the right-hand side away. It does so by deleting
all data in the RHS and replacing the raft state bits. It is using
the RHS replica's stateloader to that effect, but didn't actually
hold the raftMu to make this safe. The mutex acquisition has been
added.

Fixes #86669.
Fixes #86734.

No release note since the bug shouldn't be visible to end users (it is
very rare in the first place, and having noticeable effect even rarer),
and if so it would likely look like unspecific Raft corruption that will
be hard to trace back to this race.

Release justification: this will merge on master only after branch cut.
Release note: None


87385: roachtest: update a comment r=renatolabs a=tbg

Release justification: changes a comment in testing code.
Release note: None


87464: kvevent: Ensure out of quota events correctly handled r=miretskiy a=miretskiy

Ensure that out of quota events are not lost and propagated if necessary to the consumer.

Prior to this change, it was possible for an out of quota notification to be "lost" because "blocked" bit would be cleared out when an event was enqueued.
Instead of relying on a boolean bit, we now keep track of the number of consumers currently blocked, and issue flush request if there are non-zero blocked consumers with zero events currently queued.

Fixes #86828

Release justification: bug fix
Release note: None

87511: authors: add angeladietz to authors r=angeladietz a=angeladietz

Release note: None
Release justification: non-production code change

Co-authored-by: Tobias Grieger <[email protected]>
Co-authored-by: Yevgeniy Miretskiy <[email protected]>
Co-authored-by: Angela Dietz <[email protected]>
@craig craig bot closed this as completed in 12a1b04 Sep 8, 2022
miretskiy pushed a commit to miretskiy/cockroach that referenced this issue Sep 8, 2022
Ensure that out of quota events are not lost and propagated
if necessary to the consumer.

Prior to this change, it was possible for an out of quota
notification to be "lost" because "blocked" bit would be cleared
out when an event was enqueued.
Instead of relying on a boolean bit, we now keep track of the
number of consumers currently blocked, and issue flush request
if there are non-zero blocked consumers with zero events
currently queued.

Fixes cockroachdb#86828

Release justification: bug fix
Release note: None
miretskiy pushed a commit to miretskiy/cockroach that referenced this issue Sep 10, 2022
Ensure that out of quota events are not lost and propagated
if necessary to the consumer.

Prior to this change, it was possible for an out of quota
notification to be "lost" because "blocked" bit would be cleared
out when an event was enqueued.
Instead of relying on a boolean bit, we now keep track of the
number of consumers currently blocked, and issue flush request
if there are non-zero blocked consumers with zero events
currently queued.

Fixes cockroachdb#86828

Release justification: bug fix
Release note: None
miretskiy pushed a commit to miretskiy/cockroach that referenced this issue Oct 19, 2022
Ensure that out of quota events are not lost and propagated
if necessary to the consumer.

Prior to this change, it was possible for an out of quota
notification to be "lost" because "blocked" bit would be cleared
out when an event was enqueued.
Instead of relying on a boolean bit, we now keep track of the
number of consumers currently blocked, and issue flush request
if there are non-zero blocked consumers with zero events
currently queued.

Fixes cockroachdb#86828

Release justification: bug fix
Release note: None
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. T-cdc
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants