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

Bug Report: Vttablet is writing invalid _vt.schema_version rows #12981

Closed
brendar opened this issue Apr 26, 2023 · 5 comments · Fixed by #13045
Closed

Bug Report: Vttablet is writing invalid _vt.schema_version rows #12981

brendar opened this issue Apr 26, 2023 · 5 comments · Fixed by #13045

Comments

@brendar
Copy link
Contributor

brendar commented Apr 26, 2023

Overview of the Issue

With vttablet schema tracking enabled (--watch_replication_stream=true and --track_schema_versions=true), we're seeing invalid schemax data being written to _vt.schema_version rows.

Symptoms
On v15.0.3 this causes vttablets to fail to start serving at startup with errors like this in their logs: Historian failed to open: proto: cannot parse invalid wire-format data. It also causes running tablets to silently stop loading new _vt.schema_version rows, which effectively stops schema tracking.

On main (tested on edb702b), which includes the switch from proto.Marshal() to MinimalSchema.MarshalVT(), the invalid schema_version row is never written because the call to MarshalVT() panics. I'm not sure what state this leaves the tablet in.

Cause
We believe the cause of the invalid schemax data is a race condition between marshaling the schema data in Tracker. saveCurrentSchemaToDb() and modifying Field.ColumnType in vstreamer.buildTableColumns()

Details
Using a mix of debugging and protoscope to unpack a corrupt schemax protobuf message, we found that the corruption occurred in the MinimalTable message for the table that was migrated, and that the message size did not match the data written. We also noticed that the ColumnType fields of its query.Field messages did not appear to be included in the message sizes. This led us to believe that something was setting ColumnType between when protobuf message sizes were calculated, and when data was actually written to the buffer.

We found that ColumnType is being set in vstreamer.buildTableColumns(), and we believe this leads to a race condition when:

  • Vttablet schema tracking is enabled
  • The schema is very large (hundreds of tables, thousands of columns), which means there is a longer window between calculating message sizes and writing data to the buffer.
  • The table being migrated has binlog row events (insert/update/delete). These are always preceded by table map events.
  • A client is vstreaming from the tablet, and that vstream's filter matches the table being migrated. This is what will cause vstreamer.buildTableColumns() to be invoked concurrently with Tracker.saveCurrentSchemaToDb() when the client's vstreamer encounters a table map event.

Reproduction Steps

We have been able to reproduce this race condition on both v15.0.3 and main (edb702b).

I can clean up our reproduction example and share it if that would be helpful, but the process was as follows:

v15.0.3

  • Modified historian.loadFromDB to log an error when it encounters a corrupt row
  • Enabled schema tracking on vttablets (--watch_replication_stream=true and --track_schema_versions=true)
  • Loaded an enormous schema (~1000 tables, ~10k cols) into the commerce keyspace so that marshaling snapshots would be slow
  • Started a vstream client on the commerce keyspace matching "/.*" (all tables)
  • Started repeatedly updating a row in the customer table
  • Started repeatedly issuing DDLs to add then remove a column from the customer table

We were able to see corrupt schema_version rows being created every minute or so.

After reproducing the issue, we confirmed that commenting out field.ColumnType = extColInfo.columnType in vstreamer.buildTableColumns() prevented reoccurrence (but obviously we're not proposing that as the fix).

main
We followed the same reproduction steps, but main includes the switch from proto.Marshal() to MinimalSchema.MarshalVT()), so the schema_version row is never written because the call to MarshalVT() panics:

E0426 15:32:10.827345   31637 tabletserver.go:305] Uncaught panic:
runtime error: slice bounds out of range [-19:]
runtime/panic.go:153 (0x436f5e)
vitess.io/vitess/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go:1405 (0xe7564f)
vitess.io/vitess/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go:1445 (0xe75bde)
vitess.io/vitess/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go:1419 (0xe75977)
vitess.io/vitess/go/vt/vttablet/tabletserver/schema/tracker.go:230 (0x143071b)
vitess.io/vitess/go/vt/vttablet/tabletserver/schema/tracker.go:219 (0x1430413)
vitess.io/vitess/go/vt/vttablet/tabletserver/schema/tracker.go:140 (0x142f907)
vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/uvstreamer.go:105 (0x14a796d)
vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go:238 (0x14ac548)
vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go:363 (0x14abac1)
vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go:192 (0x14ab00f)
vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go:171 (0x14aad53)
vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/uvstreamer.go:424 (0x14a9704)
vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/engine.go:237 (0x149a07c)
vitess.io/vitess/go/vt/vttablet/tabletserver/schema/tracker.go:132 (0x142f6a8)
runtime/asm_amd64.s:1598 (0x4704a0)

Binary Version

main

vttablet --version
Version: 17.0.0-SNAPSHOT (Git revision edb702b039f6c1c67446bfbe32aaf7b0c166693e branch 'main') built on Wed Apr 26 14:32:28 UTC 2023 by spin@localhost using go1.20.3 linux/amd64

Operating System and Environment details

$ cat /etc/os-release
PRETTY_NAME="Ubuntu 22.04.2 LTS"
NAME="Ubuntu"
VERSION_ID="22.04"
VERSION="22.04.2 LTS (Jammy Jellyfish)"
VERSION_CODENAME=jammy
ID=ubuntu
ID_LIKE=debian
HOME_URL="https://www.ubuntu.com/"
SUPPORT_URL="https://help.ubuntu.com/"
BUG_REPORT_URL="https://bugs.launchpad.net/ubuntu/"
PRIVACY_POLICY_URL="https://www.ubuntu.com/legal/terms-and-policies/privacy-policy"
UBUNTU_CODENAME=jammy

$ uname -sr
Linux 5.15.0-69-generic

$ uname -m
x86_64

Log Fragments

No response

@brendar brendar added Needs Triage This issue needs to be correctly labelled and triaged Type: Bug labels Apr 26, 2023
@mattlord
Copy link
Contributor

Thank you for the amazing bug report, @brendar ! 😍

Are you still able to repeat it with this patch against main?

diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
index 1f5d0af511..b3a5fb5db0 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
@@ -26,6 +26,7 @@ import (
        "vitess.io/vitess/go/vt/vttablet"

        "google.golang.org/protobuf/encoding/prototext"
+       "google.golang.org/protobuf/proto"

        "vitess.io/vitess/go/mysql"
        mysqlbinlog "vitess.io/vitess/go/mysql/binlog"
@@ -730,11 +731,17 @@ func (vs *vstreamer) buildTablePlan(id uint64, tm *mysql.TableMap) (*binlogdatap
                Plan:     plan,
                TableMap: tm,
        }
+       // Make a deep copy of the fields as they are pointers and
+       // may be changed during the life of the stream.
+       fields := make([]*querypb.Field, len(plan.fields()))
+       for i, field := range plan.fields() {
+               fields[i] = proto.Clone(field).(*querypb.Field)
+       }
        return &binlogdatapb.VEvent{
                Type: binlogdatapb.VEventType_FIELD,
                FieldEvent: &binlogdatapb.FieldEvent{
                        TableName: plan.Table.Name,
-                       Fields:    plan.fields(),
+                       Fields:    fields,
                        Keyspace:  vs.vse.keyspace,
                        Shard:     vs.vse.shard,
                },

@mattlord mattlord self-assigned this Apr 26, 2023
@GuptaManan100 GuptaManan100 added Component: VReplication and removed Needs Triage This issue needs to be correctly labelled and triaged labels Apr 27, 2023
@brendar
Copy link
Contributor Author

brendar commented Apr 27, 2023

@mattlord that doesn't fix the issue. The pointers are being modified in buildTableColumns().

I've put up a reproduction example here: Shopify@8f7b006

And a prototype fix here (copying the fields in buildTableColumns()): Shopify@f84bcf2
This seems to fix the issue, but I'm not sure if copying those fields here is the right thing to do.

@mattlord
Copy link
Contributor

mattlord commented May 2, 2023

Thanks, @brendar! I do think we'll need to that as there are bigger "race" related issues with the schema tracker that we need to address. Do you want to open a PR?

@brendar
Copy link
Contributor Author

brendar commented May 8, 2023

Opened a PR here: #13045

@mattlord
Copy link
Contributor

Opened a PR here: #13045

Thanks, @brendar ! ❤️

maxenglander added a commit to planetscale/debezium-connector-planetscale that referenced this issue Oct 16, 2024
Enable support for enums during vstream copy phase.

There are two reasons that the connector does not handle `enum` for PSDB branches.

 1. The upstream debezium-connector-vitess simply does not support
    `enum` during the VStream copy phase. It tries to cast the row value
    to an integer, but the value is a string. It seems support for
    `enum` landed in 2021
    debezium#20, and
    support for snapshots (VStream Copy) landed in 2022
    debezium#112,
    without taking the former into account. This is easily fixed by
    finding finding the index of the string
    value in the list of values obtained from `column_type` during the
    schema discovery phase at the beginning of the VStream.
 2. However, this isn't working on some PSDB branches which don't have
    the fix vitessio/vitess#13045 for this bug
    vitessio/vitess#12981. Fixable by
    backporting the bugfix or upgrading those branches.

Signed-off-by: Max Englander <[email protected]>
maxenglander added a commit to planetscale/debezium-connector-planetscale that referenced this issue Oct 16, 2024
Enable support for enums during vstream copy phase.

There are two reasons that the connector does not handle `enum` for PSDB branches.

 1. The upstream debezium-connector-vitess simply does not support
    `enum` during the VStream copy phase. It tries to cast the row value
    to an integer, but the value is a string. It seems support for
    `enum` landed in 2021
    debezium#20, and
    support for snapshots (VStream Copy) landed in 2022
    debezium#112,
    without taking the former into account. This is easily fixed by
    finding finding the index of the string
    value in the list of values obtained from `column_type` during the
    schema discovery phase at the beginning of the VStream.
 2. However, this isn't working on some PSDB branches which don't have
    the fix vitessio/vitess#13045 for this bug
    vitessio/vitess#12981. Fixable by
    backporting the bugfix or upgrading those branches.

Signed-off-by: Max Englander <[email protected]>
maxenglander added a commit to planetscale/debezium-connector-planetscale that referenced this issue Oct 18, 2024
Enable support for enums during vstream copy phase.

There are two reasons that the connector does not handle `enum` for PSDB branches.

 1. The upstream debezium-connector-vitess simply does not support
    `enum` during the VStream copy phase. It tries to cast the row value
    to an integer, but the value is a string. It seems support for
    `enum` landed in 2021
    debezium#20, and
    support for snapshots (VStream Copy) landed in 2022
    debezium#112,
    without taking the former into account. This is easily fixed by
    finding finding the index of the string
    value in the list of values obtained from `column_type` during the
    schema discovery phase at the beginning of the VStream.
 2. However, this isn't working on some PSDB branches which don't have
    the fix vitessio/vitess#13045 for this bug
    vitessio/vitess#12981. Fixable by
    backporting the bugfix or upgrading those branches.

Signed-off-by: Max Englander <[email protected]>
maxenglander added a commit to planetscale/debezium-connector-planetscale that referenced this issue Oct 18, 2024
Enable support for enums during vstream copy phase.

There are two reasons that the connector does not handle `enum` for PSDB branches.

 1. The upstream debezium-connector-vitess simply does not support
    `enum` during the VStream copy phase. It tries to cast the row value
    to an integer, but the value is a string. It seems support for
    `enum` landed in 2021
    debezium#20, and
    support for snapshots (VStream Copy) landed in 2022
    debezium#112,
    without taking the former into account. This is easily fixed by
    finding finding the index of the string
    value in the list of values obtained from `column_type` during the
    schema discovery phase at the beginning of the VStream.
 2. However, this isn't working on some PSDB branches which don't have
    the fix vitessio/vitess#13045 for this bug
    vitessio/vitess#12981. Fixable by
    backporting the bugfix or upgrading those branches.

Signed-off-by: Max Englander <[email protected]>
maxenglander added a commit to planetscale/debezium-connector-planetscale that referenced this issue Oct 18, 2024
Enable support for enums during vstream copy phase.

There are two reasons that the connector does not handle `enum` for PSDB branches.

 1. The upstream debezium-connector-vitess simply does not support
    `enum` during the VStream copy phase. It tries to cast the row value
    to an integer, but the value is a string. It seems support for
    `enum` landed in 2021
    debezium#20, and
    support for snapshots (VStream Copy) landed in 2022
    debezium#112,
    without taking the former into account. This is easily fixed by
    finding finding the index of the string
    value in the list of values obtained from `column_type` during the
    schema discovery phase at the beginning of the VStream.
 2. However, this isn't working on some PSDB branches which don't have
    the fix vitessio/vitess#13045 for this bug
    vitessio/vitess#12981. Fixable by
    backporting the bugfix or upgrading those branches.

Signed-off-by: Max Englander <[email protected]>
maxenglander added a commit to planetscale/debezium-connector-planetscale that referenced this issue Oct 21, 2024
Enable support for enums during vstream copy phase.

There are two reasons that the connector does not handle `enum` for PSDB branches.

 1. The upstream debezium-connector-vitess simply does not support
    `enum` during the VStream copy phase. It tries to cast the row value
    to an integer, but the value is a string. It seems support for
    `enum` landed in 2021
    debezium#20, and
    support for snapshots (VStream Copy) landed in 2022
    debezium#112,
    without taking the former into account. This is easily fixed by
    finding finding the index of the string
    value in the list of values obtained from `column_type` during the
    schema discovery phase at the beginning of the VStream.
 2. However, this isn't working on some PSDB branches which don't have
    the fix vitessio/vitess#13045 for this bug
    vitessio/vitess#12981. Fixable by
    backporting the bugfix or upgrading those branches.

Signed-off-by: Max Englander <[email protected]>
maxenglander added a commit to planetscale/debezium-connector-planetscale that referenced this issue Oct 23, 2024
Enable support for enums during vstream copy phase.

There are two reasons that the connector does not handle `enum` for PSDB branches.

 1. The upstream debezium-connector-vitess simply does not support
    `enum` during the VStream copy phase. It tries to cast the row value
    to an integer, but the value is a string. It seems support for
    `enum` landed in 2021
    debezium#20, and
    support for snapshots (VStream Copy) landed in 2022
    debezium#112,
    without taking the former into account. This is easily fixed by
    finding finding the index of the string
    value in the list of values obtained from `column_type` during the
    schema discovery phase at the beginning of the VStream.
 2. However, this isn't working on some PSDB branches which don't have
    the fix vitessio/vitess#13045 for this bug
    vitessio/vitess#12981. Fixable by
    backporting the bugfix or upgrading those branches.

Signed-off-by: Max Englander <[email protected]>
maxenglander added a commit to planetscale/debezium-connector-planetscale that referenced this issue Oct 25, 2024
Enable support for enums during vstream copy phase.

There are two reasons that the connector does not handle `enum` for PSDB branches.

 1. The upstream debezium-connector-vitess simply does not support
    `enum` during the VStream copy phase. It tries to cast the row value
    to an integer, but the value is a string. It seems support for
    `enum` landed in 2021
    debezium#20, and
    support for snapshots (VStream Copy) landed in 2022
    debezium#112,
    without taking the former into account. This is easily fixed by
    finding finding the index of the string
    value in the list of values obtained from `column_type` during the
    schema discovery phase at the beginning of the VStream.
 2. However, this isn't working on some PSDB branches which don't have
    the fix vitessio/vitess#13045 for this bug
    vitessio/vitess#12981. Fixable by
    backporting the bugfix or upgrading those branches.

Signed-off-by: Max Englander <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants