diff --git a/changelog/20.0/20.0.0/summary.md b/changelog/20.0/20.0.0/summary.md
index 9421018bc9c..6bf0b021cf0 100644
--- a/changelog/20.0/20.0.0/summary.md
+++ b/changelog/20.0/20.0.0/summary.md
@@ -8,6 +8,7 @@
- [vitess/base and vitess/k8s Docker images](#base-k8s-images)
- [`gh-ost` binary and endtoend tests](#gh-ost-binary-tests-removal)
- **[Breaking changes](#breaking-changes)**
+ - [ENUM and SET column handling in VTGate VStream API](#enum-set-vstream)
- [`shutdown_grace_period` Default Change](#shutdown-grace-period-default)
- [New `unmanaged` Flag and `disable_active_reparents` deprecation](#unmanaged-flag)
- [`recovery-period-block-duration` Flag deprecation](#recovery-block-deprecation)
@@ -93,6 +94,73 @@ Vitess' endtoend tests no longer use nor test `gh-ost` migrations.
### Breaking Changes
+#### ENUM and SET column handling in VTGate VStream API
+
+The [VTGate VStream API](https://vitess.io/docs/reference/vreplication/vstream/) now returns [`ENUM`](https://dev.mysql.com/doc/refman/en/enum.html) and [`SET`](https://dev.mysql.com/doc/refman/en/set.html) column type values in [`VEvent`](https://pkg.go.dev/vitess.io/vitess/go/vt/proto/binlogdata#VEvent) messages (in the embedded [`RowChange`](https://pkg.go.dev/vitess.io/vitess/go/vt/proto/binlogdata#RowChange) messages) as their string values instead of the integer based ones — in both the copy/snapshot phase and the streaming phase. This change was done to make the `VStream` API more user-friendly, intuitive, and to align the behavior across both phases. Before [this change](https://github.com/vitessio/vitess/pull/15723) the values for [`ENUM`](https://dev.mysql.com/doc/refman/en/enum.html) and [`SET`](https://dev.mysql.com/doc/refman/en/set.html) columns were string values in the copy phase but integer values (which only have an internal meaning to MySQL) in the streaming phase. This inconsistency led to various [challenges and issues](https://github.com/vitessio/vitess/issues/15750) for each `VStream` client/consumer (e.g. the [`Debezium` Vitess connector](https://debezium.io/documentation/reference/stable/connectors/vitess.html) failed to properly perform a snapshot for tables containing these column types). Now the behavior is intuitive — clients need the string values as the eventual sink is often not MySQL so each consumer needed to perform the mappings themselves — and consistent. While this is a (potentially) breaking change, a new boolean field has been added to the [`FieldEvent`](https://pkg.go.dev/vitess.io/vitess/go/vt/proto/binlogdata#FieldEvent) message called `EnumSetStringValues`. When that field is `false` (in Vitess v19 and older) then the consumer will need to perform the mappings during streaming phase, but not during copy phase. When this field is `true`, then no mapping is required. This will help to ensure a smooth transition for all consumers over time. To demonstrate, let's look at the textual output (printing the received `VEvents` as strings) when streaming a single `enum_set_test` table from the unsharded `commerce` keyspace so that we can see what the VStream looks like before and after when we start a new VStream in copy/snapshot mode and then transition to streaming mode for the following table:
+
+```sql
+CREATE TABLE `enum_set_test` (
+ `id` int NOT NULL AUTO_INCREMENT,
+ `name` varchar(120) DEFAULT NULL,
+ `shirt_size` enum('small','medium','large','xlarge','xxlarge') DEFAULT NULL,
+ `hobbies` set('knitting','cooking','pickleball','biking','hiking','motorcycle','video games','reading') DEFAULT NULL,
+ PRIMARY KEY (`id`)
+)
+```
+
+And with the table having this data when we start our `VStream` and begin the copy/snapshot phase:
+
+```sql
+mysql> select * from enum_set_test;
++----+-----------+------------+-------------------------+
+| id | name | shirt_size | hobbies |
++----+-----------+------------+-------------------------+
+| 1 | Billy Bob | xlarge | cooking,reading |
+| 2 | Sally Mae | medium | knitting,cooking,hiking |
++----+-----------+------------+-------------------------+
+2 rows in set (0.00 sec)
+```
+
+And finally we will perform the following inserts and updates to the table during the streaming phase:
+
+```sql
+insert into enum_set_test values (3, "Matt Lord", 'medium', 'pickleball,biking,hiking,motorcycle,video games,reading');
+insert into enum_set_test values (4, "Jerry Badyellow", 'large', '');
+update enum_set_test set shirt_size = 'small', hobbies = 'knitting,cooking,hiking,reading' where id = 2;
+```
+
+Vitess v19 and older:
+
+```text
+[type:BEGIN keyspace:"commerce" shard:"0" type:FIELD field_event:{table_name:"commerce.enum_set_test" fields:{name:"id" type:INT32 table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"id" column_length:11 charset:63 flags:49667 column_type:"int"} fields:{name:"name" type:VARCHAR table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"name" column_length:480 charset:255 column_type:"varchar(120)"} fields:{name:"shirt_size" type:ENUM table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"shirt_size" column_length:28 charset:255 flags:256 column_type:"enum('small','medium','large','xlarge','xxlarge')"} fields:{name:"hobbies" type:SET table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"hobbies" column_length:288 charset:255 flags:2048 column_type:"set('knitting','cooking','pickleball','biking','hiking','motorcycle','video games','reading')"} keyspace:"commerce" shard:"0"} keyspace:"commerce" shard:"0"]
+[type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/ce357206-0d49-11ef-8fd1-a74564279579:1-35"}} keyspace:"commerce" shard:"0"]
+[type:ROW row_event:{table_name:"commerce.enum_set_test" row_changes:{after:{lengths:1 lengths:9 lengths:6 lengths:15 values:"1Billy Bobxlargecooking,reading"}} keyspace:"commerce" shard:"0"} keyspace:"commerce" shard:"0" type:ROW row_event:{table_name:"commerce.enum_set_test" row_changes:{after:{lengths:1 lengths:9 lengths:6 lengths:23 values:"2Sally Maemediumknitting,cooking,hiking"}} keyspace:"commerce" shard:"0"} keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/ce357206-0d49-11ef-8fd1-a74564279579:1-35" table_p_ks:{table_name:"enum_set_test" lastpk:{fields:{name:"id" type:INT32 charset:63 flags:49667} rows:{lengths:1 values:"2"}}}}} keyspace:"commerce" shard:"0" type:COMMIT keyspace:"commerce" shard:"0"]
+[type:BEGIN keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/ce357206-0d49-11ef-8fd1-a74564279579:1-35"}} keyspace:"commerce" shard:"0" type:COMMIT keyspace:"commerce" shard:"0"]
+[type:COPY_COMPLETED keyspace:"commerce" shard:"0" type:COPY_COMPLETED]
+[type:BEGIN timestamp:1715179728 current_time:1715179728532658000 keyspace:"commerce" shard:"0" type:FIELD timestamp:1715179728 field_event:{table_name:"commerce.enum_set_test" fields:{name:"id" type:INT32 table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"id" column_length:11 charset:63 flags:49667 column_type:"int"} fields:{name:"name" type:VARCHAR table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"name" column_length:480 charset:255 column_type:"varchar(120)"} fields:{name:"shirt_size" type:ENUM table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"shirt_size" column_length:28 charset:255 flags:256 column_type:"enum('small','medium','large','xlarge','xxlarge')"} fields:{name:"hobbies" type:SET table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"hobbies" column_length:288 charset:255 flags:2048 column_type:"set('knitting','cooking','pickleball','biking','hiking','motorcycle','video games','reading')"} keyspace:"commerce" shard:"0"} current_time:1715179728535652000 keyspace:"commerce" shard:"0" type:ROW timestamp:1715179728 row_event:{table_name:"commerce.enum_set_test" row_changes:{after:{lengths:1 lengths:9 lengths:1 lengths:3 values:"3Matt Lord2252"}} keyspace:"commerce" shard:"0" flags:1} current_time:1715179728535739000 keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/ce357206-0d49-11ef-8fd1-a74564279579:1-36"}} keyspace:"commerce" shard:"0" type:COMMIT timestamp:1715179728 current_time:1715179728535754000 keyspace:"commerce" shard:"0"]
+[type:BEGIN timestamp:1715179735 current_time:1715179735538607000 keyspace:"commerce" shard:"0" type:ROW timestamp:1715179735 row_event:{table_name:"commerce.enum_set_test" row_changes:{after:{lengths:1 lengths:15 lengths:1 lengths:1 values:"4Jerry Badyellow30"}} keyspace:"commerce" shard:"0" flags:1} current_time:1715179735538659000 keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/ce357206-0d49-11ef-8fd1-a74564279579:1-37"}} keyspace:"commerce" shard:"0" type:COMMIT timestamp:1715179735 current_time:1715179735538672000 keyspace:"commerce" shard:"0"]
+[type:BEGIN timestamp:1715179741 current_time:1715179741728690000 keyspace:"commerce" shard:"0" type:ROW timestamp:1715179741 row_event:{table_name:"commerce.enum_set_test" row_changes:{before:{lengths:1 lengths:9 lengths:1 lengths:2 values:"2Sally Mae219"} after:{lengths:1 lengths:9 lengths:1 lengths:3 values:"2Sally Mae1147"}} keyspace:"commerce" shard:"0" flags:1} current_time:1715179741728730000 keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/ce357206-0d49-11ef-8fd1-a74564279579:1-38"}} keyspace:"commerce" shard:"0" type:COMMIT timestamp:1715179741 current_time:1715179741728744000 keyspace:"commerce" shard:"0"]
+```
+
+Vitess v20 and newer:
+
+```text
+[type:BEGIN keyspace:"commerce" shard:"0" type:FIELD field_event:{table_name:"commerce.enum_set_test" fields:{name:"id" type:INT32 table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"id" column_length:11 charset:63 flags:49667 column_type:"int"} fields:{name:"name" type:VARCHAR table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"name" column_length:480 charset:255 column_type:"varchar(120)"} fields:{name:"shirt_size" type:ENUM table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"shirt_size" column_length:28 charset:255 flags:256 column_type:"enum('small','medium','large','xlarge','xxlarge')"} fields:{name:"hobbies" type:SET table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"hobbies" column_length:288 charset:255 flags:2048 column_type:"set('knitting','cooking','pickleball','biking','hiking','motorcycle','video games','reading')"} keyspace:"commerce" shard:"0" enum_set_string_values:true} keyspace:"commerce" shard:"0"]
+[type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/156f702a-0d47-11ef-8723-653d045ab990:1-50"}} keyspace:"commerce" shard:"0"]
+[type:ROW row_event:{table_name:"commerce.enum_set_test" row_changes:{after:{lengths:1 lengths:9 lengths:6 lengths:15 values:"1Billy Bobxlargecooking,reading"}} keyspace:"commerce" shard:"0"} keyspace:"commerce" shard:"0" type:ROW row_event:{table_name:"commerce.enum_set_test" row_changes:{after:{lengths:1 lengths:9 lengths:6 lengths:23 values:"2Sally Maemediumknitting,cooking,hiking"}} keyspace:"commerce" shard:"0"} keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/156f702a-0d47-11ef-8723-653d045ab990:1-50" table_p_ks:{table_name:"enum_set_test" lastpk:{fields:{name:"id" type:INT32 charset:63 flags:49667} rows:{lengths:1 values:"2"}}}}} keyspace:"commerce" shard:"0" type:COMMIT keyspace:"commerce" shard:"0"]
+[type:BEGIN keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/156f702a-0d47-11ef-8723-653d045ab990:1-50"}} keyspace:"commerce" shard:"0" type:COMMIT keyspace:"commerce" shard:"0"]
+[type:COPY_COMPLETED keyspace:"commerce" shard:"0" type:COPY_COMPLETED]
+[type:BEGIN timestamp:1715179399 current_time:1715179399817221000 keyspace:"commerce" shard:"0" type:FIELD timestamp:1715179399 field_event:{table_name:"commerce.enum_set_test" fields:{name:"id" type:INT32 table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"id" column_length:11 charset:63 flags:49667 column_type:"int"} fields:{name:"name" type:VARCHAR table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"name" column_length:480 charset:255 column_type:"varchar(120)"} fields:{name:"shirt_size" type:ENUM table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"shirt_size" column_length:28 charset:255 flags:256 column_type:"enum('small','medium','large','xlarge','xxlarge')"} fields:{name:"hobbies" type:SET table:"enum_set_test" org_table:"enum_set_test" database:"vt_commerce" org_name:"hobbies" column_length:288 charset:255 flags:2048 column_type:"set('knitting','cooking','pickleball','biking','hiking','motorcycle','video games','reading')"} keyspace:"commerce" shard:"0" enum_set_string_values:true} current_time:1715179399821735000 keyspace:"commerce" shard:"0" type:ROW timestamp:1715179399 row_event:{table_name:"commerce.enum_set_test" row_changes:{after:{lengths:1 lengths:9 lengths:6 lengths:55 values:"3Matt Lordmediumpickleball,biking,hiking,motorcycle,video games,reading"}} keyspace:"commerce" shard:"0" flags:1} current_time:1715179399821762000 keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/156f702a-0d47-11ef-8723-653d045ab990:1-51"}} keyspace:"commerce" shard:"0" type:COMMIT timestamp:1715179399 current_time:1715179399821801000 keyspace:"commerce" shard:"0"]
+[type:BEGIN timestamp:1715179399 current_time:1715179399822310000 keyspace:"commerce" shard:"0" type:ROW timestamp:1715179399 row_event:{table_name:"commerce.enum_set_test" row_changes:{after:{lengths:1 lengths:15 lengths:5 lengths:0 values:"4Jerry Badyellowlarge"}} keyspace:"commerce" shard:"0" flags:1} current_time:1715179399822355000 keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/156f702a-0d47-11ef-8723-653d045ab990:1-52"}} keyspace:"commerce" shard:"0" type:COMMIT timestamp:1715179399 current_time:1715179399822360000 keyspace:"commerce" shard:"0"]
+[type:BEGIN timestamp:1715179400 current_time:1715179400512056000 keyspace:"commerce" shard:"0" type:ROW timestamp:1715179400 row_event:{table_name:"commerce.enum_set_test" row_changes:{before:{lengths:1 lengths:9 lengths:6 lengths:23 values:"2Sally Maemediumknitting,cooking,hiking"} after:{lengths:1 lengths:9 lengths:5 lengths:31 values:"2Sally Maesmallknitting,cooking,hiking,reading"}} keyspace:"commerce" shard:"0" flags:1} current_time:1715179400512094000 keyspace:"commerce" shard:"0" type:VGTID vgtid:{shard_gtids:{keyspace:"commerce" shard:"0" gtid:"MySQL56/156f702a-0d47-11ef-8723-653d045ab990:1-53"}} keyspace:"commerce" shard:"0" type:COMMIT timestamp:1715179400 current_time:1715179400512108000 keyspace:"commerce" shard:"0"]
+```
+
+An example key difference there being that `after:{lengths:1 lengths:9 lengths:1 lengths:3 values:"2Sally Mae1147"}` from Vitess v19 and older becomes `after:{lengths:1 lengths:9 lengths:5 lengths:31 values:"2Sally Maesmallknitting,cooking,hiking,reading"}` from Vitess v20 and newer. So `1` -> `small` and `147` -> `knitting,cooking,hiking,reading` for the `ENUM` and `SET` column values respectively. This also demonstrates why this mapping is necessary in consumers/clients, as `147` has no logical meaning/value for this column outside of MySQL internals.
+
+If you're using the [`Debezium` Vitess connector](https://debezium.io/documentation/reference/stable/connectors/vitess.html), you should upgrade your connector to 2.7 (the next release) — which should contain [the relevant necessary changes](https://issues.redhat.com/browse/DBZ-7792) — *prior to upgrading Vitess* to v20.0.1 or later. If you're using any of the PlanetScale connectors ([`AirByte`](https://github.com/planetscale/airbyte-source/), [`FiveTran`](https://github.com/planetscale/fivetran-source), or [`singer-tap`](https://github.com/planetscale/singer-tap)) then no actions are required.
+
+If you're using a custom `VStream` client/consumer, then you will need to build a new client with the updated v20 [binlogdata protos](https://pkg.go.dev/vitess.io/vitess/go/vt/proto/binlogdata) ([source](https://github.com/vitessio/vitess/blob/main/proto/binlogdata.proto) for which would be in `main` or the `release-20.0` branch) before needing to support Vitess v20.0.1 or later. Your client will then be able to handle old and new messages, with older messages always having this new field set to `false`.
+
#### `shutdown_grace_period` Default Change
The `--shutdown_grace_period` flag, which was introduced in v2 with a default of `0 seconds`, has now been changed to default to `3 seconds`.
diff --git a/go/test/endtoend/vreplication/config_test.go b/go/test/endtoend/vreplication/config_test.go
index 843fa364985..a37ebe77b94 100644
--- a/go/test/endtoend/vreplication/config_test.go
+++ b/go/test/endtoend/vreplication/config_test.go
@@ -16,6 +16,11 @@ limitations under the License.
package vreplication
+import (
+ "fmt"
+ "strings"
+)
+
// The product, customer, Lead, Lead-1 tables are used to exercise and test most Workflow variants.
// We violate the NO_ZERO_DATES and NO_ZERO_IN_DATE sql_modes that are enabled by default in
// MySQL 5.7+ and MariaDB 10.2+ to ensure that vreplication still works everywhere and the
@@ -40,9 +45,10 @@ package vreplication
// default collation as it has to work across versions and the 8.0 default does not exist in 5.7.
var (
// All standard user tables should have a primary key and at least one secondary key.
- initialProductSchema = `
+ customerTypes = []string{"'individual'", "'soho'", "'enterprise'"}
+ initialProductSchema = fmt.Sprintf(`
create table product(pid int, description varbinary(128), date1 datetime not null default '0000-00-00 00:00:00', date2 datetime not null default '2021-00-01 00:00:00', primary key(pid), key(date1,date2)) CHARSET=utf8mb4;
-create table customer(cid int auto_increment, name varchar(128) collate utf8mb4_bin, meta json default null, typ enum('individual','soho','enterprise'), sport set('football','cricket','baseball'),
+create table customer(cid int auto_increment, name varchar(128) collate utf8mb4_bin, meta json default null, typ enum(%s), sport set('football','cricket','baseball'),
ts timestamp not null default current_timestamp, bits bit(2) default b'11', date1 datetime not null default '0000-00-00 00:00:00',
date2 datetime not null default '2021-00-01 00:00:00', dec80 decimal(8,0), blb blob, primary key(cid,typ), key(name)) CHARSET=utf8mb4;
create table customer_seq(id int, next_id bigint, cache bigint, primary key(id)) comment 'vitess_sequence';
@@ -51,19 +57,19 @@ create table orders(oid int, cid int, pid int, mname varchar(128), price int, qt
create table order_seq(id int, next_id bigint, cache bigint, primary key(id)) comment 'vitess_sequence';
create table customer2(cid int, name varchar(128), typ enum('individual','soho','enterprise'), sport set('football','cricket','baseball'),ts timestamp not null default current_timestamp, primary key(cid), key(ts)) CHARSET=utf8;
create table customer_seq2(id int, next_id bigint, cache bigint, primary key(id)) comment 'vitess_sequence';
-create table ` + "`Lead`(`Lead-id`" + ` binary(16), name varbinary(16), date1 datetime not null default '0000-00-00 00:00:00', date2 datetime not null default '2021-00-01 00:00:00', primary key (` + "`Lead-id`" + `), key (date1));
-create table ` + "`Lead-1`(`Lead`" + ` binary(16), name varbinary(16), date1 datetime not null default '0000-00-00 00:00:00', date2 datetime not null default '2021-00-01 00:00:00', primary key (` + "`Lead`" + `), key (date2));
+create table `+"`Lead`(`Lead-id`"+` binary(16), name varbinary(16), date1 datetime not null default '0000-00-00 00:00:00', date2 datetime not null default '2021-00-01 00:00:00', primary key (`+"`Lead-id`"+`), key (date1));
+create table `+"`Lead-1`(`Lead`"+` binary(16), name varbinary(16), date1 datetime not null default '0000-00-00 00:00:00', date2 datetime not null default '2021-00-01 00:00:00', primary key (`+"`Lead`"+`), key (date2));
create table _vt_PURGE_4f9194b43b2011eb8a0104ed332e05c2_20221210194431(id int, val varbinary(128), primary key(id), key(val));
create table db_order_test (c_uuid varchar(64) not null default '', created_at datetime not null, dstuff varchar(128), dtstuff text, dbstuff blob, cstuff char(32), primary key (c_uuid,created_at), key (dstuff)) CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci;
create table vdiff_order (order_id varchar(50) collate utf8mb4_unicode_ci not null, primary key (order_id), key (order_id)) charset=utf8mb4 COLLATE=utf8mb4_unicode_ci;
create table datze (id int, dt1 datetime not null default current_timestamp, dt2 datetime not null, ts1 timestamp default current_timestamp, primary key (id), key (dt1));
create table json_tbl (id int, j1 json, j2 json, j3 json not null, primary key(id));
create table geom_tbl (id int, g geometry, p point, ls linestring, pg polygon, mp multipoint, mls multilinestring, mpg multipolygon, gc geometrycollection, primary key(id));
-create table ` + "`blüb_tbl`" + ` (id int, val1 varchar(20), ` + "`blöb1`" + ` blob, val2 varbinary(20), ` + "`bl@b2`" + ` longblob, txt1 text, blb3 tinyblob, txt2 longtext, blb4 mediumblob, primary key(id));
+create table `+"`blüb_tbl`"+` (id int, val1 varchar(20), `+"`blöb1`"+` blob, val2 varbinary(20), `+"`bl@b2`"+` longblob, txt1 text, blb3 tinyblob, txt2 longtext, blb4 mediumblob, primary key(id));
create table reftable (id int, val1 varchar(20), primary key(id), key(val1));
create table loadtest (id int, name varchar(256), primary key(id), key(name));
create table nopk (name varchar(128), age int unsigned);
-`
+`, strings.Join(customerTypes, ","))
// These should always be ignored in vreplication
internalSchema = `
create table _1e275eef_3b20_11eb_a38f_04ed332e05c2_20201210204529_gho(id int, val varbinary(128), primary key(id));
@@ -152,7 +158,7 @@ create table nopk (name varchar(128), age int unsigned);
}
]
},
- "customer_type": {
+ "enterprise_customer": {
"column_vindexes": [
{
"column": "cid",
@@ -434,13 +440,13 @@ create table nopk (name varchar(128), age int unsigned);
materializeCustomerTypeSpec = `
{
- "workflow": "customer_type",
+ "workflow": "enterprise_customer",
"source_keyspace": "customer",
"target_keyspace": "customer",
"table_settings": [{
- "target_table": "customer_type",
- "source_expression": "select cid, typ from customer",
- "create_ddl": "create table if not exists customer_type (cid bigint not null, typ enum('individual','soho','enterprise'), primary key(cid), key(typ))"
+ "target_table": "enterprise_customer",
+ "source_expression": "select cid, name, typ from customer where typ = 'enterprise'",
+ "create_ddl": "create table if not exists enterprise_customer (cid bigint not null, name varchar(128), typ varchar(64), primary key(cid), key(typ))"
}]
}
`
diff --git a/go/test/endtoend/vreplication/resharding_workflows_v2_test.go b/go/test/endtoend/vreplication/resharding_workflows_v2_test.go
index 0491b2edd9d..34a4b269b53 100644
--- a/go/test/endtoend/vreplication/resharding_workflows_v2_test.go
+++ b/go/test/endtoend/vreplication/resharding_workflows_v2_test.go
@@ -20,6 +20,7 @@ import (
"context"
"encoding/json"
"fmt"
+ "math/rand/v2"
"net"
"strconv"
"strings"
@@ -539,7 +540,7 @@ func testReshardV2Workflow(t *testing.T) {
// Generate customer records in the background for the rest of the test
// in order to confirm that no writes are lost in either the customer
- // table or the customer_name and customer_type materializations
+ // table or the customer_name and enterprise_customer materializations
// against it during the Reshard and all of the traffic switches.
dataGenCtx, dataGenCancel := context.WithCancel(context.Background())
defer dataGenCancel()
@@ -555,7 +556,9 @@ func testReshardV2Workflow(t *testing.T) {
case <-dataGenCtx.Done():
return
default:
- _ = execVtgateQuery(t, dataGenConn, "customer", fmt.Sprintf("insert into customer (cid, name) values (%d, 'tempCustomer%d')", id, id))
+ // Use a random customer type for each record.
+ _ = execVtgateQuery(t, dataGenConn, "customer", fmt.Sprintf("insert into customer (cid, name, typ) values (%d, 'tempCustomer%d', %s)",
+ id, id, customerTypes[rand.IntN(len(customerTypes))]))
}
time.Sleep(1 * time.Millisecond)
id++
@@ -591,17 +594,17 @@ func testReshardV2Workflow(t *testing.T) {
cnres := execVtgateQuery(t, dataGenConn, "customer", "select count(*) from customer_name")
require.Len(t, cnres.Rows, 1)
require.EqualValues(t, cres.Rows, cnres.Rows)
- waitForNoWorkflowLag(t, vc, "customer", "customer_type")
- ctres := execVtgateQuery(t, dataGenConn, "customer", "select count(*) from customer_type")
- require.Len(t, ctres.Rows, 1)
- require.EqualValues(t, cres.Rows, ctres.Rows)
if debugMode {
- t.Logf("Done inserting customer data. Record counts in customer: %s, customer_name: %s, customer_type: %s",
- cres.Rows[0][0].ToString(), cnres.Rows[0][0].ToString(), ctres.Rows[0][0].ToString())
+ // We expect the row count to differ in enterprise_customer because it is
+ // using a `where typ='enterprise'` filter. So the count is only for debug
+ // info.
+ ecres := execVtgateQuery(t, dataGenConn, "customer", "select count(*) from enterprise_customer")
+ t.Logf("Done inserting customer data. Record counts in customer: %s, customer_name: %s, enterprise_customer: %s",
+ cres.Rows[0][0].ToString(), cnres.Rows[0][0].ToString(), ecres.Rows[0][0].ToString())
}
// We also do a vdiff on the materialize workflows for good measure.
doVtctldclientVDiff(t, "customer", "customer_name", "", nil)
- doVtctldclientVDiff(t, "customer", "customer_type", "", nil)
+ doVtctldclientVDiff(t, "customer", "enterprise_customer", "", nil)
}
func testMoveTablesV2Workflow(t *testing.T) {
@@ -669,7 +672,7 @@ func testMoveTablesV2Workflow(t *testing.T) {
output, err = vc.VtctldClient.ExecuteCommandWithOutput(listAllArgs...)
require.NoError(t, err)
- require.True(t, listOutputContainsWorkflow(output, "customer_name") && listOutputContainsWorkflow(output, "customer_type") && !listOutputContainsWorkflow(output, "wf1"))
+ require.True(t, listOutputContainsWorkflow(output, "customer_name") && listOutputContainsWorkflow(output, "enterprise_customer") && !listOutputContainsWorkflow(output, "wf1"))
testVSchemaForSequenceAfterMoveTables(t)
@@ -684,14 +687,14 @@ func testMoveTablesV2Workflow(t *testing.T) {
createMoveTablesWorkflow(t, "Lead,Lead-1")
output, err = vc.VtctldClient.ExecuteCommandWithOutput(listAllArgs...)
require.NoError(t, err)
- require.True(t, listOutputContainsWorkflow(output, "wf1") && listOutputContainsWorkflow(output, "customer_name") && listOutputContainsWorkflow(output, "customer_type"))
+ require.True(t, listOutputContainsWorkflow(output, "wf1") && listOutputContainsWorkflow(output, "customer_name") && listOutputContainsWorkflow(output, "enterprise_customer"))
err = tstWorkflowCancel(t)
require.NoError(t, err)
output, err = vc.VtctldClient.ExecuteCommandWithOutput(listAllArgs...)
require.NoError(t, err)
- require.True(t, listOutputContainsWorkflow(output, "customer_name") && listOutputContainsWorkflow(output, "customer_type") && !listOutputContainsWorkflow(output, "wf1"))
+ require.True(t, listOutputContainsWorkflow(output, "customer_name") && listOutputContainsWorkflow(output, "enterprise_customer") && !listOutputContainsWorkflow(output, "wf1"))
}
func testPartialSwitches(t *testing.T) {
@@ -812,7 +815,7 @@ func testRestOfWorkflow(t *testing.T) {
// fully switch and complete
waitForLowLag(t, "customer", "wf1")
waitForLowLag(t, "customer", "customer_name")
- waitForLowLag(t, "customer", "customer_type")
+ waitForLowLag(t, "customer", "enterprise_customer")
tstWorkflowSwitchReadsAndWrites(t)
validateReadsRoute(t, "rdonly", targetRdonlyTab1)
validateReadsRouteToTarget(t, "replica")
diff --git a/go/vt/proto/binlogdata/binlogdata.pb.go b/go/vt/proto/binlogdata/binlogdata.pb.go
index 601f15e8e03..87cddda1f51 100644
--- a/go/vt/proto/binlogdata/binlogdata.pb.go
+++ b/go/vt/proto/binlogdata/binlogdata.pb.go
@@ -1500,6 +1500,14 @@ type FieldEvent struct {
Fields []*query.Field `protobuf:"bytes,2,rep,name=fields,proto3" json:"fields,omitempty"`
Keyspace string `protobuf:"bytes,3,opt,name=keyspace,proto3" json:"keyspace,omitempty"`
Shard string `protobuf:"bytes,4,opt,name=shard,proto3" json:"shard,omitempty"`
+ // Are ENUM and SET field values already mapped to strings in the ROW
+ // events? This allows us to transition VTGate VStream consumers from
+ // the pre v20 behavior of having to do this mapping themselves to the
+ // v20+ behavior of not having to do this anymore and to expect string
+ // values directly.
+ // NOTE: because this is the use case, this is ONLY ever set today in
+ // vstreams managed by the vstreamManager.
+ EnumSetStringValues bool `protobuf:"varint,25,opt,name=enum_set_string_values,json=enumSetStringValues,proto3" json:"enum_set_string_values,omitempty"`
}
func (x *FieldEvent) Reset() {
@@ -1562,6 +1570,13 @@ func (x *FieldEvent) GetShard() string {
return ""
}
+func (x *FieldEvent) GetEnumSetStringValues() bool {
+ if x != nil {
+ return x.EnumSetStringValues
+ }
+ return false
+}
+
// ShardGtid contains the GTID position for one shard.
// It's used in a request for requesting a starting position.
// It's used in a response to transmit the current position
@@ -3184,7 +3199,7 @@ var file_binlogdata_proto_rawDesc = []byte{
0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72,
0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14,
0x0a, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x66,
- 0x6c, 0x61, 0x67, 0x73, 0x22, 0x83, 0x01, 0x0a, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76,
+ 0x6c, 0x61, 0x67, 0x73, 0x22, 0xb8, 0x01, 0x0a, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76,
0x65, 0x6e, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d,
0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61,
0x6d, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03,
@@ -3192,187 +3207,119 @@ var file_binlogdata_proto_rawDesc = []byte{
0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73,
0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73,
0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x04, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0x88, 0x01, 0x0a, 0x09, 0x53,
- 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73,
- 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73,
- 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74,
- 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x35,
- 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x5f, 0x6b, 0x73, 0x18, 0x04, 0x20, 0x03,
- 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e,
- 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, 0x08, 0x74, 0x61, 0x62,
- 0x6c, 0x65, 0x50, 0x4b, 0x73, 0x22, 0x3f, 0x0a, 0x05, 0x56, 0x47, 0x74, 0x69, 0x64, 0x12, 0x36,
- 0x0a, 0x0b, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x67, 0x74, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20,
- 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61,
- 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x52, 0x0a, 0x73, 0x68, 0x61, 0x72,
- 0x64, 0x47, 0x74, 0x69, 0x64, 0x73, 0x22, 0x41, 0x0a, 0x0d, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61,
- 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70,
- 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70,
- 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01,
- 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0xbc, 0x02, 0x0a, 0x07, 0x4a, 0x6f,
- 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
- 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x40, 0x0a, 0x0e, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69,
- 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e,
- 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x69, 0x67, 0x72, 0x61,
- 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, 0x6d, 0x69, 0x67, 0x72, 0x61, 0x74,
- 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65,
- 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12,
- 0x25, 0x0a, 0x0e, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f,
- 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x50, 0x6f,
- 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x36, 0x0a, 0x0b, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f,
- 0x67, 0x74, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x62, 0x69,
- 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74,
- 0x69, 0x64, 0x52, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x73, 0x12, 0x3d,
- 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, 0x6e, 0x74, 0x73, 0x18, 0x06,
- 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74,
- 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x52,
- 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, 0x6e, 0x74, 0x73, 0x12, 0x29, 0x0a,
- 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77,
- 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x57,
- 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x22, 0x8b, 0x04, 0x0a, 0x06, 0x56, 0x45, 0x76,
- 0x65, 0x6e, 0x74, 0x12, 0x2a, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
- 0x0e, 0x32, 0x16, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56,
- 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12,
- 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01,
- 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x12, 0x0a,
- 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69,
- 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04,
- 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12,
- 0x31, 0x0a, 0x09, 0x72, 0x6f, 0x77, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01,
- 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e,
- 0x52, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x72, 0x6f, 0x77, 0x45, 0x76, 0x65,
- 0x6e, 0x74, 0x12, 0x37, 0x0a, 0x0b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e,
- 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67,
- 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52,
- 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x27, 0x0a, 0x05, 0x76,
- 0x67, 0x74, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x62, 0x69, 0x6e,
- 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x47, 0x74, 0x69, 0x64, 0x52, 0x05, 0x76,
- 0x67, 0x74, 0x69, 0x64, 0x12, 0x2d, 0x0a, 0x07, 0x6a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x18,
- 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61,
- 0x74, 0x61, 0x2e, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52, 0x07, 0x6a, 0x6f, 0x75, 0x72,
- 0x6e, 0x61, 0x6c, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x6d, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x03, 0x64, 0x6d, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74,
- 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x63, 0x75, 0x72,
- 0x72, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74,
- 0x5f, 0x70, 0x5f, 0x6b, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b,
- 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4c, 0x61,
- 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x50,
- 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61,
- 0x63, 0x65, 0x18, 0x16, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61,
- 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x17, 0x20, 0x01, 0x28,
- 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x6f,
- 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x18, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x74, 0x68, 0x72,
- 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0x8d, 0x01, 0x0a, 0x0c, 0x4d, 0x69, 0x6e, 0x69, 0x6d,
- 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66,
- 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75,
- 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64,
- 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x70, 0x5f, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73,
- 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x70, 0x4b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,
- 0x73, 0x12, 0x23, 0x0a, 0x0e, 0x70, 0x5f, 0x6b, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x6e,
- 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x4b, 0x49, 0x6e, 0x64,
- 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x41, 0x0a, 0x0d, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61,
- 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x30, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65,
- 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67,
- 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c,
- 0x65, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0xc7, 0x02, 0x0a, 0x0e, 0x56, 0x53,
- 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13,
- 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72,
- 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70,
- 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65,
- 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a,
- 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65,
- 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65,
- 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49,
- 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c,
- 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72,
- 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x70,
- 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70,
- 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65,
- 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67,
- 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c,
- 0x74, 0x65, 0x72, 0x12, 0x3e, 0x0a, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73,
- 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62,
- 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c,
- 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74,
- 0x50, 0x4b, 0x73, 0x22, 0x3d, 0x0a, 0x0f, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65,
- 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73,
- 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64,
- 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e,
- 0x74, 0x73, 0x22, 0x85, 0x02, 0x0a, 0x12, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f,
- 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66,
- 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64,
- 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43,
- 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69,
- 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d,
- 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69,
- 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e,
- 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11,
- 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49,
- 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28,
- 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74,
- 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72,
- 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x2a,
- 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12,
- 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75,
- 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22, 0xf9, 0x01, 0x0a, 0x13, 0x56,
- 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
- 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03,
- 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64,
- 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x28, 0x0a, 0x08, 0x70, 0x6b, 0x66, 0x69,
- 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65,
- 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c,
- 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04,
- 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77,
- 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x12, 0x22, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b,
- 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52,
- 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68,
- 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x74,
- 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x68, 0x65, 0x61, 0x72,
- 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x68, 0x65, 0x61,
- 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x22, 0xc5, 0x01, 0x0a, 0x14, 0x56, 0x53, 0x74, 0x72, 0x65,
- 0x61, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
- 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c,
- 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76,
- 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65,
- 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64,
- 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f, 0x63, 0x61,
- 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e,
- 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c,
- 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x43,
- 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65,
- 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e,
- 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0xde,
- 0x01, 0x0a, 0x15, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73,
- 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c,
- 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61,
- 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64,
- 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e,
- 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x28, 0x0a,
- 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32,
- 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x08, 0x70,
- 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18,
- 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72,
- 0x6f, 0x77, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72,
- 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x12, 0x22, 0x0a, 0x06, 0x6c,
- 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75,
- 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22,
- 0x69, 0x0a, 0x0b, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x3c,
- 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x5f, 0x6b,
- 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64,
- 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52,
- 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, 0x1c, 0x0a, 0x09,
- 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52,
- 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x22, 0x58, 0x0a, 0x0b, 0x54, 0x61,
- 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62,
- 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74,
- 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74,
- 0x70, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79,
- 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61,
- 0x73, 0x74, 0x70, 0x6b, 0x22, 0xdc, 0x01, 0x0a, 0x15, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,
- 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f,
+ 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x65, 0x6e,
+ 0x75, 0x6d, 0x5f, 0x73, 0x65, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61,
+ 0x6c, 0x75, 0x65, 0x73, 0x18, 0x19, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13, 0x65, 0x6e, 0x75, 0x6d,
+ 0x53, 0x65, 0x74, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22,
+ 0x88, 0x01, 0x0a, 0x09, 0x53, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x12, 0x1a, 0x0a,
+ 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+ 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61,
+ 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12,
+ 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67,
+ 0x74, 0x69, 0x64, 0x12, 0x35, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x5f, 0x6b,
+ 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67,
+ 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b,
+ 0x52, 0x08, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x4b, 0x73, 0x22, 0x3f, 0x0a, 0x05, 0x56, 0x47,
+ 0x74, 0x69, 0x64, 0x12, 0x36, 0x0a, 0x0b, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x67, 0x74, 0x69,
+ 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f,
+ 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x52,
+ 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x73, 0x22, 0x41, 0x0a, 0x0d, 0x4b,
+ 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08,
+ 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
+ 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72,
+ 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x22, 0xbc,
+ 0x02, 0x0a, 0x07, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x40, 0x0a, 0x0e, 0x6d, 0x69,
+ 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01,
+ 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e,
+ 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, 0x6d,
+ 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06,
+ 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61,
+ 0x62, 0x6c, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x70, 0x6f,
+ 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6c, 0x6f,
+ 0x63, 0x61, 0x6c, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x36, 0x0a, 0x0b, 0x73,
+ 0x68, 0x61, 0x72, 0x64, 0x5f, 0x67, 0x74, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b,
+ 0x32, 0x15, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x68,
+ 0x61, 0x72, 0x64, 0x47, 0x74, 0x69, 0x64, 0x52, 0x0a, 0x73, 0x68, 0x61, 0x72, 0x64, 0x47, 0x74,
+ 0x69, 0x64, 0x73, 0x12, 0x3d, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61,
+ 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x62, 0x69, 0x6e, 0x6c,
+ 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x53,
+ 0x68, 0x61, 0x72, 0x64, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x63, 0x69, 0x70, 0x61, 0x6e,
+ 0x74, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x77, 0x6f, 0x72,
+ 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x6f,
+ 0x75, 0x72, 0x63, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x22, 0x8b, 0x04,
+ 0x0a, 0x06, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x2a, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x16, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64,
+ 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04,
+ 0x74, 0x79, 0x70, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d,
+ 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
+ 0x6d, 0x70, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,
+ 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x6d,
+ 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65,
+ 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x31, 0x0a, 0x09, 0x72, 0x6f, 0x77, 0x5f, 0x65, 0x76, 0x65, 0x6e,
+ 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67,
+ 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x72,
+ 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x37, 0x0a, 0x0b, 0x66, 0x69, 0x65, 0x6c, 0x64,
+ 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x62,
+ 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x45,
+ 0x76, 0x65, 0x6e, 0x74, 0x52, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74,
+ 0x12, 0x27, 0x0a, 0x05, 0x76, 0x67, 0x74, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x11, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x47, 0x74,
+ 0x69, 0x64, 0x52, 0x05, 0x76, 0x67, 0x74, 0x69, 0x64, 0x12, 0x2d, 0x0a, 0x07, 0x6a, 0x6f, 0x75,
+ 0x72, 0x6e, 0x61, 0x6c, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x62, 0x69, 0x6e,
+ 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52,
+ 0x07, 0x6a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x10, 0x0a, 0x03, 0x64, 0x6d, 0x6c, 0x18,
+ 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x64, 0x6d, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75,
+ 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x03,
+ 0x52, 0x0b, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x3c, 0x0a,
+ 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18,
+ 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61,
+ 0x74, 0x61, 0x2e, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x0b,
+ 0x6c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b,
+ 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x16, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b,
+ 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64,
+ 0x18, 0x17, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x1c, 0x0a,
+ 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x18, 0x20, 0x01, 0x28, 0x08,
+ 0x52, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0x8d, 0x01, 0x0a, 0x0c,
+ 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x12, 0x0a, 0x04,
+ 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65,
+ 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b,
+ 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06,
+ 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x70, 0x5f, 0x6b, 0x5f, 0x63, 0x6f,
+ 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x70, 0x4b, 0x43,
+ 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x23, 0x0a, 0x0e, 0x70, 0x5f, 0x6b, 0x5f, 0x69, 0x6e,
+ 0x64, 0x65, 0x78, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b,
+ 0x70, 0x4b, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x41, 0x0a, 0x0d, 0x4d,
+ 0x69, 0x6e, 0x69, 0x6d, 0x61, 0x6c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x30, 0x0a, 0x06,
+ 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x62,
+ 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x61,
+ 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0xc7,
+ 0x02, 0x0a, 0x0e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+ 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63,
+ 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f,
+ 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52,
+ 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72,
+ 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, 0x5f,
+ 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65, 0x43, 0x61,
+ 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74,
+ 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74, 0x61, 0x72,
+ 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72,
+ 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74,
+ 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01,
+ 0x28, 0x09, 0x52, 0x08, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x0a, 0x06,
+ 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62,
+ 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72,
+ 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x3e, 0x0a, 0x0f, 0x74, 0x61, 0x62, 0x6c,
+ 0x65, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28,
+ 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54,
+ 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x52, 0x0c, 0x74, 0x61, 0x62, 0x6c,
+ 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x73, 0x22, 0x3d, 0x0a, 0x0f, 0x56, 0x53, 0x74, 0x72,
+ 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x65,
+ 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69,
+ 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52,
+ 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x85, 0x02, 0x0a, 0x12, 0x56, 0x53, 0x74, 0x72,
+ 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f,
0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c,
0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74,
0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x65, 0x66,
@@ -3385,60 +3332,131 @@ var file_binlogdata_proto_rawDesc = []byte{
0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54,
0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x14, 0x0a,
0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75,
- 0x65, 0x72, 0x79, 0x22, 0x72, 0x0a, 0x16, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65,
- 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a,
- 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e,
- 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65,
- 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28,
- 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18,
- 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f,
- 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x2a, 0x3e, 0x0a, 0x0b, 0x4f, 0x6e, 0x44, 0x44, 0x4c,
- 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45,
- 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x54, 0x4f, 0x50, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04,
- 0x45, 0x58, 0x45, 0x43, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x45, 0x58, 0x45, 0x43, 0x5f, 0x49,
- 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x10, 0x03, 0x2a, 0x7b, 0x0a, 0x18, 0x56, 0x52, 0x65, 0x70, 0x6c,
- 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54,
- 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x74, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69,
- 0x7a, 0x65, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x6f, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c,
- 0x65, 0x73, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4c, 0x6f,
- 0x6f, 0x6b, 0x75, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x4d,
- 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x65, 0x73, 0x68,
- 0x61, 0x72, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x4f, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x44,
- 0x44, 0x4c, 0x10, 0x05, 0x2a, 0x44, 0x0a, 0x1b, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61,
- 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54,
- 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a,
- 0x07, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x41, 0x74,
- 0x6f, 0x6d, 0x69, 0x63, 0x43, 0x6f, 0x70, 0x79, 0x10, 0x02, 0x2a, 0x71, 0x0a, 0x19, 0x56, 0x52,
- 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c,
- 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f,
- 0x77, 0x6e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, 0x12, 0x0b,
- 0x0a, 0x07, 0x53, 0x74, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x43,
- 0x6f, 0x70, 0x79, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x75, 0x6e, 0x6e,
- 0x69, 0x6e, 0x67, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x10, 0x05,
- 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x61, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x2a, 0x8d, 0x02,
- 0x0a, 0x0a, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07,
- 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x47, 0x54, 0x49,
- 0x44, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x45, 0x47, 0x49, 0x4e, 0x10, 0x02, 0x12, 0x0a,
- 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x4f,
- 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x44, 0x44, 0x4c, 0x10,
- 0x05, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x06, 0x12, 0x0b, 0x0a,
- 0x07, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x07, 0x12, 0x0a, 0x0a, 0x06, 0x55, 0x50,
- 0x44, 0x41, 0x54, 0x45, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45,
- 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x45, 0x54, 0x10, 0x0a, 0x12, 0x09, 0x0a, 0x05, 0x4f,
- 0x54, 0x48, 0x45, 0x52, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x52, 0x4f, 0x57, 0x10, 0x0c, 0x12,
- 0x09, 0x0a, 0x05, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x0d, 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x45,
- 0x41, 0x52, 0x54, 0x42, 0x45, 0x41, 0x54, 0x10, 0x0e, 0x12, 0x09, 0x0a, 0x05, 0x56, 0x47, 0x54,
- 0x49, 0x44, 0x10, 0x0f, 0x12, 0x0b, 0x0a, 0x07, 0x4a, 0x4f, 0x55, 0x52, 0x4e, 0x41, 0x4c, 0x10,
- 0x10, 0x12, 0x0b, 0x0a, 0x07, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f, 0x4e, 0x10, 0x11, 0x12, 0x0a,
- 0x0a, 0x06, 0x4c, 0x41, 0x53, 0x54, 0x50, 0x4b, 0x10, 0x12, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x41,
- 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x13, 0x12, 0x12, 0x0a, 0x0e, 0x43, 0x4f, 0x50,
- 0x59, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x14, 0x2a, 0x27, 0x0a,
- 0x0d, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a,
- 0x0a, 0x06, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x48,
- 0x41, 0x52, 0x44, 0x53, 0x10, 0x01, 0x42, 0x29, 0x5a, 0x27, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73,
- 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74,
- 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74,
- 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+ 0x65, 0x72, 0x79, 0x12, 0x2a, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x05, 0x20,
+ 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72,
+ 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22,
+ 0xf9, 0x01, 0x0a, 0x13, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64,
+ 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e,
+ 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x28, 0x0a,
+ 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32,
+ 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x08, 0x70,
+ 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64, 0x18,
+ 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x72,
+ 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72,
+ 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x12, 0x22, 0x0a, 0x06, 0x6c,
+ 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75,
+ 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x12,
+ 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01,
+ 0x28, 0x08, 0x52, 0x09, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x1c, 0x0a,
+ 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08,
+ 0x52, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x22, 0xc5, 0x01, 0x0a, 0x14,
+ 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76,
+ 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72,
+ 0x49, 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c,
+ 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61,
+ 0x74, 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,
+ 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74,
+ 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64,
+ 0x69, 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06,
+ 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71,
+ 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72,
+ 0x67, 0x65, 0x74, 0x22, 0xde, 0x01, 0x0a, 0x15, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x54,
+ 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a,
+ 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x06,
+ 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71,
+ 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c,
+ 0x64, 0x73, 0x12, 0x28, 0x0a, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x03,
+ 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65,
+ 0x6c, 0x64, 0x52, 0x08, 0x70, 0x6b, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04,
+ 0x67, 0x74, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64,
+ 0x12, 0x1e, 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a,
+ 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73,
+ 0x12, 0x22, 0x0a, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b,
+ 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x06, 0x6c, 0x61,
+ 0x73, 0x74, 0x70, 0x6b, 0x22, 0x69, 0x0a, 0x0b, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x45, 0x76,
+ 0x65, 0x6e, 0x74, 0x12, 0x3c, 0x0a, 0x0e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6c, 0x61, 0x73,
+ 0x74, 0x5f, 0x70, 0x5f, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x62, 0x69,
+ 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61,
+ 0x73, 0x74, 0x50, 0x4b, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50,
+ 0x4b, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x18, 0x02,
+ 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x22,
+ 0x58, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4c, 0x61, 0x73, 0x74, 0x50, 0x4b, 0x12, 0x1d,
+ 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01,
+ 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a,
+ 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e,
+ 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c,
+ 0x74, 0x52, 0x06, 0x6c, 0x61, 0x73, 0x74, 0x70, 0x6b, 0x22, 0xdc, 0x01, 0x0a, 0x15, 0x56, 0x53,
+ 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75,
+ 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x13, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65,
+ 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
+ 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49,
+ 0x44, 0x52, 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x43, 0x61, 0x6c, 0x6c,
+ 0x65, 0x72, 0x49, 0x64, 0x12, 0x45, 0x0a, 0x13, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74,
+ 0x65, 0x5f, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x0b, 0x32, 0x15, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x54, 0x47, 0x61, 0x74, 0x65,
+ 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x11, 0x69, 0x6d, 0x6d, 0x65, 0x64, 0x69,
+ 0x61, 0x74, 0x65, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x06, 0x74,
+ 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x71, 0x75,
+ 0x65, 0x72, 0x79, 0x2e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67,
+ 0x65, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28,
+ 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x22, 0x72, 0x0a, 0x16, 0x56, 0x53, 0x74, 0x72,
+ 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03,
+ 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64,
+ 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x67, 0x74, 0x69, 0x64,
+ 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x67, 0x74, 0x69, 0x64, 0x12, 0x1e, 0x0a, 0x04,
+ 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x71, 0x75, 0x65,
+ 0x72, 0x79, 0x2e, 0x52, 0x6f, 0x77, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x2a, 0x3e, 0x0a, 0x0b,
+ 0x4f, 0x6e, 0x44, 0x44, 0x4c, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0a, 0x0a, 0x06, 0x49,
+ 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x54, 0x4f, 0x50, 0x10,
+ 0x01, 0x12, 0x08, 0x0a, 0x04, 0x45, 0x58, 0x45, 0x43, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x45,
+ 0x58, 0x45, 0x43, 0x5f, 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x10, 0x03, 0x2a, 0x7b, 0x0a, 0x18,
+ 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b,
+ 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x74, 0x65,
+ 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x6f, 0x76,
+ 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x43, 0x72, 0x65,
+ 0x61, 0x74, 0x65, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02,
+ 0x12, 0x0b, 0x0a, 0x07, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x65, 0x10, 0x03, 0x12, 0x0b, 0x0a,
+ 0x07, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x4f, 0x6e,
+ 0x6c, 0x69, 0x6e, 0x65, 0x44, 0x44, 0x4c, 0x10, 0x05, 0x2a, 0x44, 0x0a, 0x1b, 0x56, 0x52, 0x65,
+ 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f,
+ 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65,
+ 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x10, 0x01, 0x12,
+ 0x0e, 0x0a, 0x0a, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x43, 0x6f, 0x70, 0x79, 0x10, 0x02, 0x2a,
+ 0x71, 0x0a, 0x19, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57,
+ 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07,
+ 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x6e, 0x69,
+ 0x74, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x74, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x10, 0x02,
+ 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x6f, 0x70, 0x79, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0b, 0x0a,
+ 0x07, 0x52, 0x75, 0x6e, 0x6e, 0x69, 0x6e, 0x67, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x72,
+ 0x72, 0x6f, 0x72, 0x10, 0x05, 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x61, 0x67, 0x67, 0x69, 0x6e, 0x67,
+ 0x10, 0x06, 0x2a, 0x8d, 0x02, 0x0a, 0x0a, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70,
+ 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x08,
+ 0x0a, 0x04, 0x47, 0x54, 0x49, 0x44, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x45, 0x47, 0x49,
+ 0x4e, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, 0x12,
+ 0x0c, 0x0a, 0x08, 0x52, 0x4f, 0x4c, 0x4c, 0x42, 0x41, 0x43, 0x4b, 0x10, 0x04, 0x12, 0x07, 0x0a,
+ 0x03, 0x44, 0x44, 0x4c, 0x10, 0x05, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54,
+ 0x10, 0x06, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x07, 0x12,
+ 0x0a, 0x0a, 0x06, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, 0x44,
+ 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x09, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x45, 0x54, 0x10, 0x0a,
+ 0x12, 0x09, 0x0a, 0x05, 0x4f, 0x54, 0x48, 0x45, 0x52, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x52,
+ 0x4f, 0x57, 0x10, 0x0c, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x49, 0x45, 0x4c, 0x44, 0x10, 0x0d, 0x12,
+ 0x0d, 0x0a, 0x09, 0x48, 0x45, 0x41, 0x52, 0x54, 0x42, 0x45, 0x41, 0x54, 0x10, 0x0e, 0x12, 0x09,
+ 0x0a, 0x05, 0x56, 0x47, 0x54, 0x49, 0x44, 0x10, 0x0f, 0x12, 0x0b, 0x0a, 0x07, 0x4a, 0x4f, 0x55,
+ 0x52, 0x4e, 0x41, 0x4c, 0x10, 0x10, 0x12, 0x0b, 0x0a, 0x07, 0x56, 0x45, 0x52, 0x53, 0x49, 0x4f,
+ 0x4e, 0x10, 0x11, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x41, 0x53, 0x54, 0x50, 0x4b, 0x10, 0x12, 0x12,
+ 0x0d, 0x0a, 0x09, 0x53, 0x41, 0x56, 0x45, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x13, 0x12, 0x12,
+ 0x0a, 0x0e, 0x43, 0x4f, 0x50, 0x59, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44,
+ 0x10, 0x14, 0x2a, 0x27, 0x0a, 0x0d, 0x4d, 0x69, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54,
+ 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x53, 0x10, 0x00, 0x12,
+ 0x0a, 0x0a, 0x06, 0x53, 0x48, 0x41, 0x52, 0x44, 0x53, 0x10, 0x01, 0x42, 0x29, 0x5a, 0x27, 0x76,
+ 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f,
+ 0x67, 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x62, 0x69, 0x6e, 0x6c,
+ 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
diff --git a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go
index f5a77cd964c..1332681a976 100644
--- a/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go
+++ b/go/vt/proto/binlogdata/binlogdata_vtproto.pb.go
@@ -371,9 +371,10 @@ func (m *FieldEvent) CloneVT() *FieldEvent {
return (*FieldEvent)(nil)
}
r := &FieldEvent{
- TableName: m.TableName,
- Keyspace: m.Keyspace,
- Shard: m.Shard,
+ TableName: m.TableName,
+ Keyspace: m.Keyspace,
+ Shard: m.Shard,
+ EnumSetStringValues: m.EnumSetStringValues,
}
if rhs := m.Fields; rhs != nil {
tmpContainer := make([]*query.Field, len(rhs))
@@ -1802,6 +1803,18 @@ func (m *FieldEvent) MarshalToSizedBufferVT(dAtA []byte) (int, error) {
i -= len(m.unknownFields)
copy(dAtA[i:], m.unknownFields)
}
+ if m.EnumSetStringValues {
+ i--
+ if m.EnumSetStringValues {
+ dAtA[i] = 1
+ } else {
+ dAtA[i] = 0
+ }
+ i--
+ dAtA[i] = 0x1
+ i--
+ dAtA[i] = 0xc8
+ }
if len(m.Shard) > 0 {
i -= len(m.Shard)
copy(dAtA[i:], m.Shard)
@@ -3555,6 +3568,9 @@ func (m *FieldEvent) SizeVT() (n int) {
if l > 0 {
n += 1 + l + sov(uint64(l))
}
+ if m.EnumSetStringValues {
+ n += 3
+ }
n += len(m.unknownFields)
return n
}
@@ -6813,6 +6829,26 @@ func (m *FieldEvent) UnmarshalVT(dAtA []byte) error {
}
m.Shard = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
+ case 25:
+ if wireType != 0 {
+ return fmt.Errorf("proto: wrong wireType = %d for field EnumSetStringValues", wireType)
+ }
+ var v int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflow
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ v |= int(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ m.EnumSetStringValues = bool(v != 0)
default:
iNdEx = preIndex
skippy, err := skip(dAtA[iNdEx:])
diff --git a/go/vt/schema/parser.go b/go/vt/schema/parser.go
index 7ed820a3687..78ec4ec36e6 100644
--- a/go/vt/schema/parser.go
+++ b/go/vt/schema/parser.go
@@ -19,7 +19,6 @@ package schema
import (
"fmt"
"regexp"
- "strconv"
"strings"
"vitess.io/vitess/go/sqltypes"
@@ -171,12 +170,13 @@ func parseEnumOrSetTokens(enumOrSetValues string) []string {
}
// ParseEnumOrSetTokensMap parses the comma delimited part of an enum column definition
-// and returns a map where ["1"] is the first token, and [""] is th elast token
-func ParseEnumOrSetTokensMap(enumOrSetValues string) map[string]string {
+// and returns a map where [1] is the first token, and [] is the last.
+func ParseEnumOrSetTokensMap(enumOrSetValues string) map[int]string {
tokens := parseEnumOrSetTokens(enumOrSetValues)
- tokensMap := map[string]string{}
+ tokensMap := map[int]string{}
for i, token := range tokens {
- tokensMap[strconv.Itoa(i+1)] = token
+ // SET and ENUM values are 1 indexed.
+ tokensMap[i+1] = token
}
return tokensMap
}
diff --git a/go/vt/schema/parser_test.go b/go/vt/schema/parser_test.go
index d251a195d1d..fe9264c29b9 100644
--- a/go/vt/schema/parser_test.go
+++ b/go/vt/schema/parser_test.go
@@ -167,12 +167,12 @@ func TestParseEnumTokensMap(t *testing.T) {
input := `'x-small','small','medium','large','x-large'`
enumTokensMap := ParseEnumOrSetTokensMap(input)
- expect := map[string]string{
- "1": "x-small",
- "2": "small",
- "3": "medium",
- "4": "large",
- "5": "x-large",
+ expect := map[int]string{
+ 1: "x-small",
+ 2: "small",
+ 3: "medium",
+ 4: "large",
+ 5: "x-large",
}
assert.Equal(t, expect, enumTokensMap)
}
@@ -183,7 +183,7 @@ func TestParseEnumTokensMap(t *testing.T) {
}
for _, input := range inputs {
enumTokensMap := ParseEnumOrSetTokensMap(input)
- expect := map[string]string{}
+ expect := map[int]string{}
assert.Equal(t, expect, enumTokensMap)
}
}
diff --git a/go/vt/sqlparser/ast.go b/go/vt/sqlparser/ast.go
index cfef9923530..1611619f950 100644
--- a/go/vt/sqlparser/ast.go
+++ b/go/vt/sqlparser/ast.go
@@ -1824,7 +1824,7 @@ type ColumnType struct {
// Text field options
Charset ColumnCharset
- // Enum values
+ // Enum and Set column definition values
EnumValues []string
}
diff --git a/go/vt/vtgate/endtoend/vstream_test.go b/go/vt/vtgate/endtoend/vstream_test.go
index 871e6cf98c3..246d17f88b5 100644
--- a/go/vt/vtgate/endtoend/vstream_test.go
+++ b/go/vt/vtgate/endtoend/vstream_test.go
@@ -617,9 +617,9 @@ func TestVStreamSharded(t *testing.T) {
received bool
}
expectedEvents := []*expectedEvent{
- {`type:FIELD field_event:{table_name:"ks.t1_sharded" fields:{name:"id1" type:INT64 table:"t1_sharded" org_table:"t1_sharded" database:"vt_ks_-80" org_name:"id1" column_length:20 charset:63 flags:53251 column_type:"bigint(20)"} fields:{name:"id2" type:INT64 table:"t1_sharded" org_table:"t1_sharded" database:"vt_ks_-80" org_name:"id2" column_length:20 charset:63 flags:32768 column_type:"bigint(20)"} keyspace:"ks" shard:"-80"}`, false},
+ {`type:FIELD field_event:{table_name:"ks.t1_sharded" fields:{name:"id1" type:INT64 table:"t1_sharded" org_table:"t1_sharded" database:"vt_ks_-80" org_name:"id1" column_length:20 charset:63 flags:53251 column_type:"bigint(20)"} fields:{name:"id2" type:INT64 table:"t1_sharded" org_table:"t1_sharded" database:"vt_ks_-80" org_name:"id2" column_length:20 charset:63 flags:32768 column_type:"bigint(20)"} keyspace:"ks" shard:"-80" enum_set_string_values:true}`, false},
{`type:ROW row_event:{table_name:"ks.t1_sharded" row_changes:{after:{lengths:1 lengths:1 values:"11"}} keyspace:"ks" shard:"-80"}`, false},
- {`type:FIELD field_event:{table_name:"ks.t1_sharded" fields:{name:"id1" type:INT64 table:"t1_sharded" org_table:"t1_sharded" database:"vt_ks_80-" org_name:"id1" column_length:20 charset:63 flags:53251 column_type:"bigint(20)"} fields:{name:"id2" type:INT64 table:"t1_sharded" org_table:"t1_sharded" database:"vt_ks_80-" org_name:"id2" column_length:20 charset:63 flags:32768 column_type:"bigint(20)"} keyspace:"ks" shard:"80-"}`, false},
+ {`type:FIELD field_event:{table_name:"ks.t1_sharded" fields:{name:"id1" type:INT64 table:"t1_sharded" org_table:"t1_sharded" database:"vt_ks_80-" org_name:"id1" column_length:20 charset:63 flags:53251 column_type:"bigint(20)"} fields:{name:"id2" type:INT64 table:"t1_sharded" org_table:"t1_sharded" database:"vt_ks_80-" org_name:"id2" column_length:20 charset:63 flags:32768 column_type:"bigint(20)"} keyspace:"ks" shard:"80-" enum_set_string_values:true}`, false},
{`type:ROW row_event:{table_name:"ks.t1_sharded" row_changes:{after:{lengths:1 lengths:1 values:"44"}} keyspace:"ks" shard:"80-"}`, false},
}
for {
diff --git a/go/vt/vtgate/evalengine/api_compare_test.go b/go/vt/vtgate/evalengine/api_compare_test.go
index 106b111cafc..aa05f5d2787 100644
--- a/go/vt/vtgate/evalengine/api_compare_test.go
+++ b/go/vt/vtgate/evalengine/api_compare_test.go
@@ -31,11 +31,12 @@ import (
"vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/sqltypes"
- querypb "vitess.io/vitess/go/vt/proto/query"
- vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vterrors"
+
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
type testCase struct {
diff --git a/go/vt/vttablet/grpctmclient/client.go b/go/vt/vttablet/grpctmclient/client.go
index 1d054eb7b9c..45b72e54272 100644
--- a/go/vt/vttablet/grpctmclient/client.go
+++ b/go/vt/vttablet/grpctmclient/client.go
@@ -235,12 +235,10 @@ func (client *grpcClient) dialDedicatedPool(ctx context.Context, dialPoolGroup D
invalidator := func() {
client.mu.Lock()
defer client.mu.Unlock()
- if tm, ok := m[addr]; ok {
- if tm != nil && tm.cc != nil {
- tm.cc.Close()
- }
- delete(m, addr)
+ if tm := m[addr]; tm != nil && tm.cc != nil {
+ tm.cc.Close()
}
+ delete(m, addr)
}
return m[addr].client, invalidator, nil
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go
index d4b733b4c0b..3bef997d0be 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go
@@ -200,7 +200,6 @@ type TablePlan struct {
Delete *sqlparser.ParsedQuery
MultiDelete *sqlparser.ParsedQuery
Fields []*querypb.Field
- EnumValuesMap map[string](map[string]string)
ConvertIntToEnum map[string]bool
// PKReferences is used to check if an event changed
// a primary key column (row move).
@@ -335,34 +334,6 @@ func (tp *TablePlan) bindFieldVal(field *querypb.Field, val *sqltypes.Value) (*q
// An integer converted to an enum. We must write the textual value of the int. i.e. 0 turns to '0'
return sqltypes.StringBindVariable(val.ToString()), nil
}
- if enumValues, ok := tp.EnumValuesMap[field.Name]; ok && !val.IsNull() {
- // The fact that this field has a EnumValuesMap entry, means we must
- // use the enum's text value as opposed to the enum's numerical value.
- // This may be needed in Online DDL, when the enum column could be modified:
- // - Either from ENUM to a text type (VARCHAR/TEXT)
- // - Or from ENUM to another ENUM with different value ordering,
- // e.g. from `('red', 'green', 'blue')` to `('red', 'blue')`.
- // By applying the textual value of an enum we eliminate the ordering concern.
- // In non-Online DDL this shouldn't be a concern because the schema is static,
- // and so passing the enum's numerical value is sufficient.
- enumValue, enumValueOK := enumValues[val.ToString()]
- if !enumValueOK {
- return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "Invalid enum value: %v for field %s", val, field.Name)
- }
- // get the enum text for this val
- return sqltypes.StringBindVariable(enumValue), nil
- }
- if field.Type == querypb.Type_ENUM {
- // This is an ENUM w/o a values map, which means that we are most likely using
- // the index value -- what is stored and binlogged vs. the list of strings
- // defined in the table schema -- and we must use an int bindvar or we'll have
- // invalid/incorrect predicates like WHERE enumcol='2'.
- // This will be the case when applying binlog events.
- enumIndexVal := sqltypes.MakeTrusted(querypb.Type_UINT64, val.Raw())
- if enumIndex, err := enumIndexVal.ToUint64(); err == nil {
- return sqltypes.Uint64BindVariable(enumIndex), nil
- }
- }
return sqltypes.ValueBindVariable(*val), nil
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go b/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go
index 17afe030d11..8f156101a7d 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go
@@ -27,7 +27,6 @@ import (
"vitess.io/vitess/go/textutil"
"vitess.io/vitess/go/vt/binlog/binlogplayer"
"vitess.io/vitess/go/vt/key"
- "vitess.io/vitess/go/vt/schema"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet"
@@ -231,12 +230,6 @@ func buildTablePlan(tableName string, rule *binlogdatapb.Rule, colInfos []*Colum
Match: fromTable,
}
- enumValuesMap := map[string](map[string]string){}
- for k, v := range rule.ConvertEnumToText {
- tokensMap := schema.ParseEnumOrSetTokensMap(v)
- enumValuesMap[k] = tokensMap
- }
-
if expr, ok := sel.SelectExprs[0].(*sqlparser.StarExpr); ok {
// If it's a "select *", we return a partial plan, and complete
// it when we get back field info from the stream.
@@ -252,7 +245,6 @@ func buildTablePlan(tableName string, rule *binlogdatapb.Rule, colInfos []*Colum
SendRule: sendRule,
Lastpk: lastpk,
Stats: stats,
- EnumValuesMap: enumValuesMap,
ConvertCharset: rule.ConvertCharset,
ConvertIntToEnum: rule.ConvertIntToEnum,
CollationEnv: collationEnv,
@@ -335,7 +327,6 @@ func buildTablePlan(tableName string, rule *binlogdatapb.Rule, colInfos []*Colum
tablePlan := tpb.generate()
tablePlan.SendRule = sendRule
- tablePlan.EnumValuesMap = enumValuesMap
tablePlan.ConvertCharset = rule.ConvertCharset
tablePlan.ConvertIntToEnum = rule.ConvertIntToEnum
return tablePlan, nil
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go
index 9796f7ef77c..5beb85e8450 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go
@@ -1525,8 +1525,8 @@ func TestPlayerTypes(t *testing.T) {
fmt.Sprintf("create table %s.vitess_ints(tiny tinyint, tinyu tinyint unsigned, small smallint, smallu smallint unsigned, medium mediumint, mediumu mediumint unsigned, normal int, normalu int unsigned, big bigint, bigu bigint unsigned, y year, primary key(tiny))", vrepldb),
"create table vitess_fracts(id int, deci decimal(5,2), num numeric(5,2), f float, d double, primary key(id))",
fmt.Sprintf("create table %s.vitess_fracts(id int, deci decimal(5,2), num numeric(5,2), f float, d double, primary key(id))", vrepldb),
- "create table vitess_strings(vb varbinary(16), c char(16), vc varchar(16), b binary(5), tb tinyblob, bl blob, ttx tinytext, tx text, en enum('a','b'), s set('a','b'), primary key(vb))",
- fmt.Sprintf("create table %s.vitess_strings(vb varbinary(16), c char(16), vc varchar(16), b binary(5), tb tinyblob, bl blob, ttx tinytext, tx text, en enum('a','b'), s set('a','b'), primary key(vb))", vrepldb),
+ "create table vitess_strings(vb varbinary(16), c char(16), vc varchar(16), b binary(5), tb tinyblob, bl blob, ttx tinytext, tx text, en enum('a','b'), s set('a','b','c','d','e','f','g','h','i','j','k','l','m','n','o','p','q','r','s','t','u','v','w','x','y','z'), primary key(vb))",
+ fmt.Sprintf("create table %s.vitess_strings(vb varbinary(16), c char(16), vc varchar(16), b binary(5), tb tinyblob, bl blob, ttx tinytext, tx text, en enum('a','b'), s set('a','b','c','d','e','f','g','h','i','j','k','l','m','n','o','p','q','r','s','t','u','v','w','x','y','z'), primary key(vb))", vrepldb),
"create table vitess_misc(id int, b bit(8), d date, dt datetime, t time, g geometry, primary key(id))",
fmt.Sprintf("create table %s.vitess_misc(id int, b bit(8), d date, dt datetime, t time, g geometry, primary key(id))", vrepldb),
"create table vitess_null(id int, val varbinary(128), primary key(id))",
@@ -1597,11 +1597,11 @@ func TestPlayerTypes(t *testing.T) {
{"1", "1.99", "2.99", "3.99", "4.99"},
},
}, {
- input: "insert into vitess_strings values('a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'a', 'a,b')",
- output: "insert into vitess_strings(vb,c,vc,b,tb,bl,ttx,tx,en,s) values ('a','b','c','d\\0\\0\\0\\0','e','f','g','h',1,'3')",
+ input: "insert into vitess_strings values('a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'a', 'a,b,f,l,q,s,v,z')",
+ output: "insert into vitess_strings(vb,c,vc,b,tb,bl,ttx,tx,en,s) values ('a','b','c','d\\0\\0\\0\\0','e','f','g','h','a','a,b,f,l,q,s,v,z')",
table: "vitess_strings",
data: [][]string{
- {"a", "b", "c", "d\000\000\000\000", "e", "f", "g", "h", "a", "a,b"},
+ {"a", "b", "c", "d\000\000\000\000", "e", "f", "g", "h", "a", "a,b,f,l,q,s,v,z"},
},
}, {
input: "insert into vitess_misc values(1, '\x01', '2012-01-01', '2012-01-01 15:45:45', '15:45:45', point(1, 2))",
diff --git a/go/vt/vttablet/tabletserver/vstreamer/copy.go b/go/vt/vttablet/tabletserver/vstreamer/copy.go
index 585be09dec3..2f54ea235fe 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/copy.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/copy.go
@@ -258,6 +258,13 @@ func (uvs *uvstreamer) copyTable(ctx context.Context, tableName string) error {
Fields: uvs.fields,
Keyspace: uvs.vse.keyspace,
Shard: uvs.vse.shard,
+ // In the copy phase the values for ENUM and SET fields are always strings.
+ // We are including this extra uint8 in the message even though there may
+ // not be an ENUM or SET column in the table because we only have one field
+ // event for each batch of ROWs being copied so it's negligible overhead
+ // and less costly and intrusive than iterating over the fields to see if
+ // we do indeed have any ENUM or SET columns in the table.
+ EnumSetStringValues: true,
}
if err := uvs.sendFieldEvent(ctx, rows.Gtid, fieldEvent); err != nil {
log.Infof("sendFieldEvent returned error %v", err)
diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine.go b/go/vt/vttablet/tabletserver/vstreamer/engine.go
index 7a0d1aaaf3d..d0202eeba9e 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/engine.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go
@@ -28,17 +28,16 @@ import (
"sync/atomic"
"time"
- "vitess.io/vitess/go/vt/dbconfigs"
- "vitess.io/vitess/go/vt/mysqlctl"
- "vitess.io/vitess/go/vt/servenv"
- "vitess.io/vitess/go/vt/vterrors"
-
"vitess.io/vitess/go/acl"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/stats"
+ "vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/log"
+ "vitess.io/vitess/go/vt/mysqlctl"
+ "vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/srvtopo"
"vitess.io/vitess/go/vt/topo"
+ "vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/vindexes"
"vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
@@ -234,7 +233,6 @@ func (vse *Engine) validateBinlogRowImage(ctx context.Context, db dbconfigs.Conn
// Stream starts a new stream.
// This streams events from the binary logs
func (vse *Engine) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error) error {
-
if err := vse.validateBinlogRowImage(ctx, vse.se.GetDBConnector()); err != nil {
return err
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go b/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go
index 8c4ac20cb10..ee67e482aa1 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go
@@ -38,7 +38,6 @@ import (
"context"
"fmt"
"slices"
- "strconv"
"strings"
"testing"
@@ -60,7 +59,14 @@ const (
lengthInt = 11
lengthBlob = 65535
lengthText = 262140
- lengthSet = 56
+
+ // We have to hardcode the set lengths as we don't yet have an encoded way
+ // to calculate the length for the TableMap event,
+ // This is the expected length of the only SET column in the test schema.
+ lengthSet = 204
+ // This is the expected length of the only SET column using a binary collation
+ // in the test schema.
+ lengthSetBinary = 428
)
var (
@@ -80,14 +86,16 @@ type TestColumn struct {
// TestFieldEvent has all the attributes of a table required for creating a field event.
type TestFieldEvent struct {
- table, db string
- cols []*TestColumn
+ table, db string
+ cols []*TestColumn
+ enumSetStrings bool
}
func (tfe *TestFieldEvent) String() string {
var fe binlogdatapb.FieldEvent
var field *query.Field
fe.TableName = tfe.table
+ fe.EnumSetStringValues = tfe.enumSetStrings
for _, col := range tfe.cols {
if col.skip {
continue
@@ -157,14 +165,33 @@ func (s *TestRowEventSpec) String() string {
if c.before != nil && len(c.before) > 0 {
rowChange.Before = &query.Row{}
for _, val := range c.before {
+ if val == sqltypes.NullStr {
+ val = ""
+ }
rowChange.Before.Lengths = append(rowChange.Before.Lengths, int64(len(val)))
rowChange.Before.Values = append(rowChange.Before.Values, []byte(val)...)
}
}
if c.after != nil && len(c.after) > 0 {
rowChange.After = &query.Row{}
- for _, val := range c.after {
- rowChange.After.Lengths = append(rowChange.After.Lengths, int64(len(val)))
+ for i, val := range c.after {
+ if val == sqltypes.NullStr {
+ val = ""
+ }
+ l := int64(len(val))
+ if strings.HasPrefix(val, "\x00") {
+ // The null byte hex representation is used when printing NULL ENUM/SET values.
+ // The length is 0, however, rather than the string representation of those
+ // null bytes.
+ l = 0
+ // The previous column's length increases by 1 for some reason. No idea why MySQL
+ // does this, but it does. It may be including the backslash, for example:
+ // row_changes:{after:{lengths:1 lengths:4 lengths:0 lengths:0 values:\"5mmm\\x00\"}}}"
+ if i > 0 {
+ rowChange.After.Lengths[i-1]++
+ }
+ }
+ rowChange.After.Lengths = append(rowChange.After.Lengths, l)
rowChange.After.Values = append(rowChange.After.Values, []byte(val)...)
}
}
@@ -312,10 +339,8 @@ func (ts *TestSpec) getBindVarsForInsert(stmt sqlparser.Statement) (string, map[
v.Format(bufV)
s := bufV.String()
switch fe.cols[i].dataTypeLowered {
- case "varchar", "char", "binary", "varbinary", "blob", "text":
+ case "varchar", "char", "binary", "varbinary", "blob", "text", "enum", "set":
s = strings.Trim(s, "'")
- case "set", "enum":
- s = ts.getMetadataMap(table, fe.cols[i], s)
}
bv[fe.cols[i].name] = s
}
@@ -404,6 +429,7 @@ func (ts *TestSpec) Run() {
isRowEvent = true
del := stmt.(*sqlparser.Delete)
table = del.TableExprs[0].(*sqlparser.AliasedTableExpr).As.String()
+ case *sqlparser.Set:
default:
_, ok := stmt.(sqlparser.DDLStatement)
if !ok {
@@ -504,13 +530,23 @@ func (ts *TestSpec) getFieldEvent(table *schemadiff.CreateTableEntity) *TestFiel
tc.len = lengthText
tc.colType = "text"
case "set":
- tc.len = lengthSet
+ if collation.IsBinary() {
+ tc.len = lengthSetBinary
+ tc.dataType = "BINARY"
+ } else {
+ tc.len = lengthSet
+ }
tc.colType = fmt.Sprintf("%s(%s)", tc.dataTypeLowered, strings.Join(col.Type.EnumValues, ","))
ts.metadata[getMetadataKey(table.Name(), tc.name)] = col.Type.EnumValues
+ tfe.enumSetStrings = true
case "enum":
tc.len = int64(len(col.Type.EnumValues) + 1)
+ if collation.IsBinary() {
+ tc.dataType = "BINARY"
+ }
tc.colType = fmt.Sprintf("%s(%s)", tc.dataTypeLowered, strings.Join(col.Type.EnumValues, ","))
ts.metadata[getMetadataKey(table.Name(), tc.name)] = col.Type.EnumValues
+ tfe.enumSetStrings = true
default:
log.Infof(fmt.Sprintf("unknown sqlTypeString %s", tc.dataTypeLowered))
}
@@ -530,28 +566,6 @@ func (ts *TestSpec) setMetadataMap(table, col, value string) {
ts.metadata[getMetadataKey(table, col)] = valuesReversed
}
-func (ts *TestSpec) getMetadataMap(table string, col *TestColumn, value string) string {
- var bits int64
- value = strings.Trim(value, "'")
- meta := ts.metadata[getMetadataKey(table, col.name)]
- values := strings.Split(value, ",")
- for _, v := range values {
- v2 := strings.Trim(v, "'")
- for i, m := range meta {
- m2 := strings.Trim(m, "'")
- if m2 == v2 {
- switch col.dataTypeLowered {
- case "set":
- bits |= 1 << uint(i)
- case "enum":
- bits = int64(i) + 1
- }
- }
- }
- }
- return strconv.FormatInt(bits, 10)
-}
-
func (ts *TestSpec) getRowEvent(table string, bv map[string]string, fe *TestFieldEvent, stmt sqlparser.Statement, flags uint32) string {
ev := &binlogdatapb.RowEvent{
TableName: table,
@@ -583,8 +597,12 @@ func (ts *TestSpec) getRowEvent(table string, bv map[string]string, fe *TestFiel
l++
}
}
- row.Values = append(row.Values, val...)
+ if slices.Equal(val, sqltypes.NullBytes) {
+ l = -1
+ val = []byte{}
+ }
row.Lengths = append(row.Lengths, l)
+ row.Values = append(row.Values, val...)
}
ev.RowChanges = ts.getRowChanges(table, stmt, &row)
vEvent := &binlogdatapb.VEvent{
diff --git a/go/vt/vttablet/tabletserver/vstreamer/main_test.go b/go/vt/vttablet/tabletserver/vstreamer/main_test.go
index 8ec639d47c4..aade1511060 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/main_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/main_test.go
@@ -21,23 +21,26 @@ import (
"fmt"
"io"
"os"
+ "slices"
+ "strings"
"sync"
"testing"
"time"
- "vitess.io/vitess/go/mysql/replication"
- "vitess.io/vitess/go/vt/log"
- binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
-
"github.com/stretchr/testify/require"
_flag "vitess.io/vitess/go/internal/flag"
"vitess.io/vitess/go/mysql"
+ "vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/vt/dbconfigs"
+ "vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
"vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/testenv"
+
+ binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+ querypb "vitess.io/vitess/go/vt/proto/query"
)
var (
@@ -152,6 +155,22 @@ func expectLog(ctx context.Context, t *testing.T, input any, ch <-chan []*binlog
defer timer.Stop()
for _, wantset := range output {
var evs []*binlogdatapb.VEvent
+ inCopyPhase := false
+ haveEnumOrSetField := func(fields []*querypb.Field) bool {
+ return slices.ContainsFunc(fields, func(f *querypb.Field) bool {
+ // We can't simply use querypb.Type_ENUM or querypb.Type_SET here
+ // because if a binary collation is used then the field Type will
+ // be BINARY. And we don't have the binlog event metadata from the
+ // original event any longer that we could use to get the MySQL type
+ // (which would still be ENUM or SET). So we instead look at the column
+ // type string value which will be e.g enum('s','m','l').
+ colTypeStr := strings.ToLower(f.GetColumnType())
+ if strings.HasPrefix(colTypeStr, "enum(") || strings.HasPrefix(colTypeStr, "set(") {
+ return true
+ }
+ return false
+ })
+ }
for {
select {
case allevs, ok := <-ch:
@@ -160,18 +179,27 @@ func expectLog(ctx context.Context, t *testing.T, input any, ch <-chan []*binlog
}
for _, ev := range allevs {
// Ignore spurious heartbeats that can happen on slow machines.
- if ev.Type == binlogdatapb.VEventType_HEARTBEAT {
+ if ev.Throttled || ev.Type == binlogdatapb.VEventType_HEARTBEAT {
continue
}
- if ev.Throttled {
- continue
+ switch ev.Type {
+ case binlogdatapb.VEventType_OTHER:
+ if strings.Contains(ev.Gtid, copyPhaseStart) {
+ inCopyPhase = true
+ }
+ case binlogdatapb.VEventType_COPY_COMPLETED:
+ inCopyPhase = false
+ case binlogdatapb.VEventType_FIELD:
+ // This is always set in the copy phase. It's also set in the
+ // running phase when the table has an ENUM or SET field.
+ ev.FieldEvent.EnumSetStringValues = inCopyPhase || haveEnumOrSetField(ev.FieldEvent.Fields)
}
evs = append(evs, ev)
}
case <-ctx.Done():
- t.Fatalf("expectLog: Done(), stream ended early")
+ require.Fail(t, "expectLog: Done(), stream ended early")
case <-timer.C:
- t.Fatalf("expectLog: timed out waiting for events: %v", wantset)
+ require.Fail(t, "expectLog: timed out waiting for events: %v", wantset)
}
if len(evs) != 0 {
break
diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go
index ad2f218f8d1..2c768d7d3c6 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go
@@ -58,6 +58,11 @@ type Plan struct {
// of the table.
Filters []Filter
+ // Convert any integer values seen in the binlog events for ENUM or SET
+ // columns to the string values. The map is keyed on the column number, with
+ // the value being the map of ordinal values to string values.
+ EnumSetValuesMap map[int](map[int]string)
+
env *vtenv.Environment
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go
index c515357a4ec..19c27e8a725 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go
@@ -69,6 +69,7 @@ const (
bulkInsertQuery = "insert into %s (id%d1, id%d2) values "
insertQuery = "insert into %s (id%d1, id%d2) values (%d, %d)"
numInitialRows = 10
+ copyPhaseStart = "Copy Start"
)
type TestState struct {
@@ -206,7 +207,7 @@ func TestVStreamCopyCompleteFlow(t *testing.T) {
}
// Test event called after t1 copy is complete
- callbacks["OTHER.*Copy Start t2"] = func() {
+ callbacks[fmt.Sprintf("OTHER.*%s t2", copyPhaseStart)] = func() {
conn, err := env.Mysqld.GetDbaConnection(ctx)
require.NoError(t, err)
defer conn.Close()
@@ -220,7 +221,7 @@ func TestVStreamCopyCompleteFlow(t *testing.T) {
}
- callbacks["OTHER.*Copy Start t3"] = func() {
+ callbacks[fmt.Sprintf("OTHER.*%s t3", copyPhaseStart)] = func() {
conn, err := env.Mysqld.GetDbaConnection(ctx)
require.NoError(t, err)
defer conn.Close()
@@ -298,18 +299,29 @@ func TestVStreamCopyCompleteFlow(t *testing.T) {
}
func validateReceivedEvents(t *testing.T) {
+ inCopyPhase := false
for i, ev := range allEvents {
ev.Timestamp = 0
- if ev.Type == binlogdatapb.VEventType_FIELD {
+ switch ev.Type {
+ case binlogdatapb.VEventType_OTHER:
+ if strings.Contains(ev.Gtid, copyPhaseStart) {
+ inCopyPhase = true
+ }
+ case binlogdatapb.VEventType_FIELD:
for j := range ev.FieldEvent.Fields {
ev.FieldEvent.Fields[j].Flags = 0
ev.FieldEvent.Keyspace = ""
ev.FieldEvent.Shard = ""
+ // We always set this in the copy phase. In the
+ // running phase we only set it IF the table has
+ // an ENUM or SET column.
+ ev.FieldEvent.EnumSetStringValues = inCopyPhase
}
- }
- if ev.Type == binlogdatapb.VEventType_ROW {
+ case binlogdatapb.VEventType_ROW:
ev.RowEvent.Keyspace = ""
ev.RowEvent.Shard = ""
+ case binlogdatapb.VEventType_COPY_COMPLETED:
+ inCopyPhase = false
}
got := ev.String()
want := env.RemoveAnyDeprecatedDisplayWidths(expectedEvents[i])
@@ -469,9 +481,9 @@ func startVStreamCopy(ctx context.Context, t *testing.T, filter *binlogdatapb.Fi
}
var expectedEvents = []string{
- "type:OTHER gtid:\"Copy Start t1\"",
+ fmt.Sprintf("type:OTHER gtid:\"%s t1\"", copyPhaseStart),
"type:BEGIN",
- "type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"}}",
+ "type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"} enum_set_string_values:true}",
"type:GTID",
"type:ROW row_event:{table_name:\"t1\" row_changes:{after:{lengths:1 lengths:2 values:\"110\"}}}",
"type:ROW row_event:{table_name:\"t1\" row_changes:{after:{lengths:1 lengths:2 values:\"220\"}}}",
@@ -489,18 +501,18 @@ var expectedEvents = []string{
"type:LASTPK last_p_k_event:{table_last_p_k:{table_name:\"t1\"} completed:true}",
"type:COMMIT",
"type:BEGIN",
- "type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"}}",
+ "type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"} enum_set_string_values:true}",
"type:ROW row_event:{table_name:\"t1\" row_changes:{after:{lengths:2 lengths:3 values:\"11110\"}}}",
"type:GTID",
"type:COMMIT", //insert for t2 done along with t1 does not generate an event since t2 is not yet copied
- "type:OTHER gtid:\"Copy Start t2\"",
+ fmt.Sprintf("type:OTHER gtid:\"%s t2\"", copyPhaseStart),
"type:BEGIN",
- "type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"}}",
+ "type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"} enum_set_string_values:true}",
"type:ROW row_event:{table_name:\"t1\" row_changes:{after:{lengths:2 lengths:3 values:\"12120\"}}}",
"type:GTID",
"type:COMMIT",
"type:BEGIN",
- "type:FIELD field_event:{table_name:\"t2\" fields:{name:\"id21\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id21\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id22\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id22\" column_length:11 charset:63 column_type:\"int(11)\"}}",
+ "type:FIELD field_event:{table_name:\"t2\" fields:{name:\"id21\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id21\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id22\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id22\" column_length:11 charset:63 column_type:\"int(11)\"} enum_set_string_values:true}",
"type:ROW row_event:{table_name:\"t2\" row_changes:{after:{lengths:1 lengths:2 values:\"120\"}}}",
"type:ROW row_event:{table_name:\"t2\" row_changes:{after:{lengths:1 lengths:2 values:\"240\"}}}",
"type:ROW row_event:{table_name:\"t2\" row_changes:{after:{lengths:1 lengths:2 values:\"360\"}}}",
@@ -517,19 +529,19 @@ var expectedEvents = []string{
"type:BEGIN",
"type:LASTPK last_p_k_event:{table_last_p_k:{table_name:\"t2\"} completed:true}",
"type:COMMIT",
- "type:OTHER gtid:\"Copy Start t3\"",
+ fmt.Sprintf("type:OTHER gtid:\"%s t3\"", copyPhaseStart),
"type:BEGIN",
- "type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"}}",
+ "type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"} enum_set_string_values:true}",
"type:ROW row_event:{table_name:\"t1\" row_changes:{after:{lengths:2 lengths:3 values:\"13130\"}}}",
"type:GTID",
"type:COMMIT",
"type:BEGIN",
- "type:FIELD field_event:{table_name:\"t2\" fields:{name:\"id21\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id21\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id22\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id22\" column_length:11 charset:63 column_type:\"int(11)\"}}",
+ "type:FIELD field_event:{table_name:\"t2\" fields:{name:\"id21\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id21\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id22\" type:INT32 table:\"t2\" org_table:\"t2\" database:\"vttest\" org_name:\"id22\" column_length:11 charset:63 column_type:\"int(11)\"} enum_set_string_values:true}",
"type:ROW row_event:{table_name:\"t2\" row_changes:{after:{lengths:2 lengths:3 values:\"12240\"}}}",
"type:GTID",
"type:COMMIT",
"type:BEGIN",
- "type:FIELD field_event:{table_name:\"t3\" fields:{name:\"id31\" type:INT32 table:\"t3\" org_table:\"t3\" database:\"vttest\" org_name:\"id31\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id32\" type:INT32 table:\"t3\" org_table:\"t3\" database:\"vttest\" org_name:\"id32\" column_length:11 charset:63 column_type:\"int(11)\"}}",
+ "type:FIELD field_event:{table_name:\"t3\" fields:{name:\"id31\" type:INT32 table:\"t3\" org_table:\"t3\" database:\"vttest\" org_name:\"id31\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id32\" type:INT32 table:\"t3\" org_table:\"t3\" database:\"vttest\" org_name:\"id32\" column_length:11 charset:63 column_type:\"int(11)\"} enum_set_string_values:true}",
"type:ROW row_event:{table_name:\"t3\" row_changes:{after:{lengths:1 lengths:2 values:\"130\"}}}",
"type:ROW row_event:{table_name:\"t3\" row_changes:{after:{lengths:1 lengths:2 values:\"260\"}}}",
"type:ROW row_event:{table_name:\"t3\" row_changes:{after:{lengths:1 lengths:2 values:\"390\"}}}",
diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
index 3aede20f650..5c40f321324 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
@@ -21,6 +21,7 @@ import (
"context"
"fmt"
"io"
+ "strings"
"time"
"google.golang.org/protobuf/encoding/prototext"
@@ -35,6 +36,7 @@ import (
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/logutil"
+ vtschema "vitess.io/vitess/go/vt/schema"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet"
@@ -45,7 +47,6 @@ import (
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
querypb "vitess.io/vitess/go/vt/proto/query"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
- vtschema "vitess.io/vitess/go/vt/schema"
)
const (
@@ -749,6 +750,9 @@ func (vs *vstreamer) buildTablePlan(id uint64, tm *mysql.TableMap) (*binlogdatap
vs.plans[id] = nil
return nil, nil
}
+ if err := addEnumAndSetMappingstoPlan(plan, cols, tm.Metadata); err != nil {
+ return nil, vterrors.Wrapf(err, "failed to build ENUM and SET column integer to string mappings")
+ }
vs.plans[id] = &streamerPlan{
Plan: plan,
TableMap: tm,
@@ -760,6 +764,9 @@ func (vs *vstreamer) buildTablePlan(id uint64, tm *mysql.TableMap) (*binlogdatap
Fields: plan.fields(),
Keyspace: vs.vse.keyspace,
Shard: vs.vse.shard,
+ // This mapping will be done, if needed, in the vstreamer when we process
+ // and build ROW events.
+ EnumSetStringValues: len(plan.EnumSetValuesMap) > 0,
},
}, nil
}
@@ -827,7 +834,7 @@ func (vs *vstreamer) buildTableColumns(tm *mysql.TableMap) ([]*querypb.Field, er
// initially using collations for the column types based on the *connection
// collation* and not the actual *column collation*.
// But because we now get the correct collation for the actual column from
- // mysqld in getExtColsInfo we know this is the correct one for the vstream
+ // mysqld in getExtColInfos we know this is the correct one for the vstream
// target and we use that rather than any that were in the binlog events,
// which were for the source and which can be using a different collation
// than the target.
@@ -835,7 +842,6 @@ func (vs *vstreamer) buildTableColumns(tm *mysql.TableMap) ([]*querypb.Field, er
if err != nil {
return nil, err
}
-
return fieldsCopy, nil
}
@@ -1046,6 +1052,30 @@ func (vs *vstreamer) extractRowAndFilter(plan *streamerPlan, data []byte, dataCo
}
pos += l
+ if !value.IsNull() { // ENUMs and SETs require no special handling if they are NULL
+ // If the column is a CHAR based type with a binary collation (e.g. utf8mb4_bin) then the
+ // actual column type is included in the second byte of the event metadata while the
+ // event's type for the field is BINARY. This is true for ENUM and SET types.
+ var mysqlType uint16
+ if sqltypes.IsQuoted(plan.Table.Fields[colNum].Type) {
+ mysqlType = plan.TableMap.Metadata[colNum] >> 8
+ }
+ // Convert the integer values in the binlog event for any SET and ENUM fields into their
+ // string representations.
+ if plan.Table.Fields[colNum].Type == querypb.Type_ENUM || mysqlType == mysqlbinlog.TypeEnum {
+ value, err = buildEnumStringValue(plan, colNum, value)
+ if err != nil {
+ return false, nil, false, vterrors.Wrapf(err, "failed to perform ENUM column integer to string value mapping")
+ }
+ }
+ if plan.Table.Fields[colNum].Type == querypb.Type_SET || mysqlType == mysqlbinlog.TypeSet {
+ value, err = buildSetStringValue(plan, colNum, value)
+ if err != nil {
+ return false, nil, false, vterrors.Wrapf(err, "failed to perform SET column integer to string value mapping")
+ }
+ }
+ }
+
charsets[colNum] = collations.ID(plan.Table.Fields[colNum].Charset)
values[colNum] = value
valueIndex++
@@ -1055,6 +1085,109 @@ func (vs *vstreamer) extractRowAndFilter(plan *streamerPlan, data []byte, dataCo
return ok, filtered, partial, err
}
+// addEnumAndSetMappingstoPlan sets up any necessary ENUM and SET integer to string mappings.
+func addEnumAndSetMappingstoPlan(plan *Plan, cols []*querypb.Field, metadata []uint16) error {
+ plan.EnumSetValuesMap = make(map[int]map[int]string)
+ for i, col := range cols {
+ // If the column is a CHAR based type with a binary collation (e.g. utf8mb4_bin) then
+ // the actual column type is included in the second byte of the event metadata while
+ // the event's type for the field is BINARY. This is true for ENUM and SET types.
+ var mysqlType uint16
+ if sqltypes.IsQuoted(col.Type) {
+ mysqlType = metadata[i] >> 8
+ }
+ if col.Type == querypb.Type_ENUM || mysqlType == mysqlbinlog.TypeEnum ||
+ col.Type == querypb.Type_SET || mysqlType == mysqlbinlog.TypeSet {
+ // Strip the enum() / set() parts out.
+ begin := strings.Index(col.ColumnType, "(")
+ end := strings.LastIndex(col.ColumnType, ")")
+ if begin == -1 || end == -1 {
+ return fmt.Errorf("enum or set column %s does not have valid string values: %s",
+ col.Name, col.ColumnType)
+ }
+ plan.EnumSetValuesMap[i] = vtschema.ParseEnumOrSetTokensMap(col.ColumnType[begin+1 : end])
+ }
+ }
+ return nil
+}
+
+// buildEnumStringValue takes the integer value of an ENUM column and returns the string value.
+func buildEnumStringValue(plan *streamerPlan, colNum int, value sqltypes.Value) (sqltypes.Value, error) {
+ if value.IsNull() { // No work is needed
+ return value, nil
+ }
+ // Add the mappings just-in-time in case we haven't properly received and processed a
+ // table map event to initialize it.
+ if plan.EnumSetValuesMap == nil {
+ if err := addEnumAndSetMappingstoPlan(plan.Plan, plan.Table.Fields, plan.TableMap.Metadata); err != nil {
+ return sqltypes.Value{}, err
+ }
+ }
+ // ENUM columns are stored as an unsigned 16-bit integer as they can contain a maximum
+ // of 65,535 elements (https://dev.mysql.com/doc/refman/en/enum.html) with the 0 element
+ // reserved for any integer value that has no string mapping.
+ iv, err := value.ToUint16()
+ if err != nil {
+ return sqltypes.Value{}, fmt.Errorf("no valid integer value found for column %s in table %s, bytes: %b",
+ plan.Table.Fields[colNum].Name, plan.Table.Name, iv)
+ }
+ var strVal string
+ // Match the MySQL behavior of returning an empty string for invalid ENUM values.
+ // This is what the 0 position in an ENUM is reserved for.
+ if iv != 0 {
+ var ok bool
+ strVal, ok = plan.EnumSetValuesMap[colNum][int(iv)]
+ if !ok {
+ // The integer value was NOT 0 yet we found no mapping. This should never happen.
+ return sqltypes.Value{}, fmt.Errorf("no string value found for ENUM column %s in table %s -- with available values being: %v -- using the found integer value: %d",
+ plan.Table.Fields[colNum].Name, plan.Table.Name, plan.EnumSetValuesMap[colNum], iv)
+ }
+ }
+ return sqltypes.MakeTrusted(plan.Table.Fields[colNum].Type, []byte(strVal)), nil
+}
+
+// buildSetStringValue takes the integer value of a SET column and returns the string value.
+func buildSetStringValue(plan *streamerPlan, colNum int, value sqltypes.Value) (sqltypes.Value, error) {
+ if value.IsNull() { // No work is needed
+ return value, nil
+ }
+ // Add the mappings just-in-time in case we haven't properly received and processed a
+ // table map event to initialize it.
+ if plan.EnumSetValuesMap == nil {
+ if err := addEnumAndSetMappingstoPlan(plan.Plan, plan.Table.Fields, plan.TableMap.Metadata); err != nil {
+ return sqltypes.Value{}, err
+ }
+ }
+ // A SET column can have 64 unique values: https://dev.mysql.com/doc/refman/en/set.html
+ // For this reason the binlog event contains the values encoded as an unsigned 64-bit
+ // integer which is really a bitmap.
+ val := bytes.Buffer{}
+ iv, err := value.ToUint64()
+ if err != nil {
+ return value, fmt.Errorf("no valid integer value found for column %s in table %s, bytes: %b",
+ plan.Table.Fields[colNum].Name, plan.Table.Name, iv)
+ }
+ idx := 1
+ // See what bits are set in the bitmap using bitmasks.
+ for b := uint64(1); b < 1<<63; b <<= 1 {
+ if iv&b > 0 { // This bit is set and the SET's string value needs to be provided.
+ strVal, ok := plan.EnumSetValuesMap[colNum][idx]
+ // When you insert values not found in the SET (which requires disabling STRICT mode) then
+ // they are effectively pruned and ignored (not actually saved). So this should never happen.
+ if !ok {
+ return sqltypes.Value{}, fmt.Errorf("no valid integer value found for SET column %s in table %s, bytes: %b",
+ plan.Table.Fields[colNum].Name, plan.Table.Name, iv)
+ }
+ if val.Len() > 0 {
+ val.WriteByte(',')
+ }
+ val.WriteString(strVal)
+ }
+ idx++
+ }
+ return sqltypes.MakeTrusted(plan.Table.Fields[colNum].Type, val.Bytes()), nil
+}
+
func wrapError(err error, stopPos replication.Position, vse *Engine) error {
if err != nil {
vse.vstreamersEndedWithErrors.Add(1)
diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go
index ffa757d277f..8628aeeba70 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go
@@ -118,18 +118,40 @@ func TestSetAndEnum(t *testing.T) {
ts := &TestSpec{
t: t,
ddls: []string{
- "create table t1(id int, val binary(4), color set('red','green','blue'), size enum('S','M','L'), primary key(id))",
+ "create table t1(id int, val binary(4), color set('red','green','blue','black','white','pink','purple','yellow','brown'), size enum('S','M','L'), primary key(id))",
+ "create table t2(id int, val binary(4), color set('red','green','blue','black','white','pink','purple','yellow','brown','eggshell','mint','tan','fuschia','teal','babyblue','grey','bulletgrey') collate utf8mb4_bin, size enum('S','M','L') collate utf8mb4_bin, primary key(id)) charset=utf8mb4",
},
}
defer ts.Close()
ts.Init()
- ts.tests = [][]*TestQuery{{
- {"begin", nil},
- {"insert into t1 values (1, 'aaa', 'red,blue', 'S')", nil},
- {"insert into t1 values (2, 'bbb', 'green', 'M')", nil},
- {"insert into t1 values (3, 'ccc', 'red,blue,green', 'L')", nil},
- {"commit", nil},
- }}
+ ts.tests = [][]*TestQuery{
+ {
+ {"begin", nil},
+ {"insert into t1 values (1, 'aaa', 'red,blue', 'S')", nil},
+ {"insert into t1 values (2, 'bbb', 'green,pink,purple,yellow,brown', 'M')", nil},
+ {"insert into t1 values (3, 'ccc', 'red,green,blue', 'L')", nil},
+ {"commit", nil},
+ },
+ {
+ {"begin", nil},
+ {"insert into t2 values (1, 'xxx', 'red,blue,black,grey', 'S')", nil},
+ {"insert into t2 values (2, 'yyy', 'green,black,pink,purple,yellow,brown,mint,tan,bulletgrey', 'M')", nil},
+ {"insert into t2 values (3, 'zzz', 'red,green,blue', 'L')", nil},
+ {"commit", nil},
+ },
+ {
+ {"begin", nil},
+ // This query fails with the following error when SQL mode includes STRICT:
+ // failed: Data truncated for column 'size' at row 1 (errno 1265) (sqlstate 01000) during query: insert into t2 values (4, 'lll', '', '')
+ {"set @@session.sql_mode = ''", nil},
+ {"insert into t2 values (4, 'lll', '', '')", nil},
+ {"insert into t2 values (5, 'mmm', 'invalid', 'invalid,invalid,mint,invalid')", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t2", changes: []TestRowChange{{after: []string{"5", "mmm", "\x00", ""}}}}},
+ }},
+ {"insert into t2 values (6, 'nnn', NULL, NULL)", nil},
+ {"commit", nil},
+ },
+ }
ts.Run()
}
@@ -421,6 +443,7 @@ func TestVStreamCopySimpleFlow(t *testing.T) {
{name: "id11", dataType: "INT32", colType: "int(11)", len: 11, collationID: 63},
{name: "id12", dataType: "INT32", colType: "int(11)", len: 11, collationID: 63},
},
+ enumSetStrings: true,
}
t2FieldEvent := &TestFieldEvent{
table: "t2",
@@ -429,6 +452,7 @@ func TestVStreamCopySimpleFlow(t *testing.T) {
{name: "id21", dataType: "INT32", colType: "int(11)", len: 11, collationID: 63},
{name: "id22", dataType: "INT32", colType: "int(11)", len: 11, collationID: 63},
},
+ enumSetStrings: true,
}
t1Events := []string{}
@@ -442,6 +466,9 @@ func TestVStreamCopySimpleFlow(t *testing.T) {
t1Events = append(t1Events, "lastpk", "commit")
t2Events = append(t2Events, "lastpk", "commit")
+ // Now we're past the copy phase and have no ENUM or SET columns.
+ t1FieldEvent.enumSetStrings = false
+ t2FieldEvent.enumSetStrings = false
insertEvents1 := []string{
"begin",
t1FieldEvent.String(),
@@ -511,6 +538,7 @@ func TestVStreamCopyWithDifferentFilters(t *testing.T) {
{name: "id1", dataType: "INT32", colType: "int(11)", len: 11, collationID: 63},
{name: "id2", dataType: "INT32", colType: "int(11)", len: 11, collationID: 63},
},
+ enumSetStrings: true,
}
execStatements(t, []string{
@@ -520,6 +548,12 @@ func TestVStreamCopyWithDifferentFilters(t *testing.T) {
"insert into t2b(id1, id2) values ('a', 5)",
})
+ // All field events in this test are in the copy phase so they should all
+ // have the enum_set_string_values field set.
+ for _, fe := range ts.fieldEvents {
+ fe.enumSetStrings = true
+ }
+
var expectedEvents = []string{
"begin",
t1FieldEvent.String(),
@@ -572,14 +606,17 @@ func TestVStreamCopyWithDifferentFilters(t *testing.T) {
log.Infof("Got %d events as expected", len(allEvents))
for i, ev := range allEvents {
ev.Timestamp = 0
- if ev.Type == binlogdatapb.VEventType_FIELD {
+ switch ev.Type {
+ case binlogdatapb.VEventType_FIELD:
for j := range ev.FieldEvent.Fields {
ev.FieldEvent.Fields[j].Flags = 0
}
ev.FieldEvent.Keyspace = ""
ev.FieldEvent.Shard = ""
- }
- if ev.Type == binlogdatapb.VEventType_ROW {
+ // All events in this test are in the copy phase so they should
+ // all have the enum_set_string_values field set.
+ ev.FieldEvent.EnumSetStringValues = true
+ case binlogdatapb.VEventType_ROW:
ev.RowEvent.Keyspace = ""
ev.RowEvent.Shard = ""
}
@@ -1549,9 +1586,9 @@ func TestTypes(t *testing.T) {
},
output: [][]string{{
`begin`,
- fmt.Sprintf(`type:FIELD field_event:{table_name:"vitess_strings" fields:{name:"vb" type:VARBINARY table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"vb" column_length:16 charset:63 column_type:"varbinary(16)"} fields:{name:"c" type:CHAR table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"c" column_length:64 charset:%d column_type:"char(16)"} fields:{name:"vc" type:VARCHAR table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"vc" column_length:64 charset:%d column_type:"varchar(16)"} fields:{name:"b" type:BINARY table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"b" column_length:4 charset:63 column_type:"binary(4)"} fields:{name:"tb" type:BLOB table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"tb" column_length:255 charset:63 column_type:"tinyblob"} fields:{name:"bl" type:BLOB table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"bl" column_length:65535 charset:63 column_type:"blob"} fields:{name:"ttx" type:TEXT table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"ttx" column_length:1020 charset:%d column_type:"tinytext"} fields:{name:"tx" type:TEXT table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"tx" column_length:262140 charset:%d column_type:"text"} fields:{name:"en" type:ENUM table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"en" column_length:4 charset:%d column_type:"enum('a','b')"} fields:{name:"s" type:SET table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"s" column_length:12 charset:%d column_type:"set('a','b')"}}`, testenv.DefaultCollationID, testenv.DefaultCollationID, testenv.DefaultCollationID, testenv.DefaultCollationID, testenv.DefaultCollationID, testenv.DefaultCollationID),
- `type:ROW row_event:{table_name:"vitess_strings" row_changes:{after:{lengths:1 lengths:1 lengths:1 lengths:4 lengths:1 lengths:1 lengths:1 lengths:1 lengths:1 lengths:1 ` +
- `values:"abcd\x00\x00\x00efgh13"}}}`,
+ fmt.Sprintf(`type:FIELD field_event:{table_name:"vitess_strings" fields:{name:"vb" type:VARBINARY table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"vb" column_length:16 charset:63 column_type:"varbinary(16)"} fields:{name:"c" type:CHAR table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"c" column_length:64 charset:%d column_type:"char(16)"} fields:{name:"vc" type:VARCHAR table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"vc" column_length:64 charset:%d column_type:"varchar(16)"} fields:{name:"b" type:BINARY table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"b" column_length:4 charset:63 column_type:"binary(4)"} fields:{name:"tb" type:BLOB table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"tb" column_length:255 charset:63 column_type:"tinyblob"} fields:{name:"bl" type:BLOB table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"bl" column_length:65535 charset:63 column_type:"blob"} fields:{name:"ttx" type:TEXT table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"ttx" column_length:1020 charset:%d column_type:"tinytext"} fields:{name:"tx" type:TEXT table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"tx" column_length:262140 charset:%d column_type:"text"} fields:{name:"en" type:ENUM table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"en" column_length:4 charset:%d column_type:"enum('a','b')"} fields:{name:"s" type:SET table:"vitess_strings" org_table:"vitess_strings" database:"vttest" org_name:"s" column_length:12 charset:%d column_type:"set('a','b')"} enum_set_string_values:true}`, testenv.DefaultCollationID, testenv.DefaultCollationID, testenv.DefaultCollationID, testenv.DefaultCollationID, testenv.DefaultCollationID, testenv.DefaultCollationID),
+ `type:ROW row_event:{table_name:"vitess_strings" row_changes:{after:{lengths:1 lengths:1 lengths:1 lengths:4 lengths:1 lengths:1 lengths:1 lengths:1 lengths:1 lengths:3 ` +
+ `values:"abcd\x00\x00\x00efghaa,b"}}}`,
`gtid`,
`commit`,
}},
diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto
index 7dbfff5258d..1e70275e8b5 100644
--- a/proto/binlogdata.proto
+++ b/proto/binlogdata.proto
@@ -352,6 +352,14 @@ message FieldEvent {
repeated query.Field fields = 2;
string keyspace = 3;
string shard = 4;
+ // Are ENUM and SET field values already mapped to strings in the ROW
+ // events? This allows us to transition VTGate VStream consumers from
+ // the pre v20 behavior of having to do this mapping themselves to the
+ // v20+ behavior of not having to do this anymore and to expect string
+ // values directly.
+ // NOTE: because this is the use case, this is ONLY ever set today in
+ // vstreams managed by the vstreamManager.
+ bool enum_set_string_values = 25;
}
// ShardGtid contains the GTID position for one shard.
diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts
index 9ef0ce1dd76..cf987da09dd 100644
--- a/web/vtadmin/src/proto/vtadmin.d.ts
+++ b/web/vtadmin/src/proto/vtadmin.d.ts
@@ -32124,6 +32124,9 @@ export namespace binlogdata {
/** FieldEvent shard */
shard?: (string|null);
+
+ /** FieldEvent enum_set_string_values */
+ enum_set_string_values?: (boolean|null);
}
/** Represents a FieldEvent. */
@@ -32147,6 +32150,9 @@ export namespace binlogdata {
/** FieldEvent shard. */
public shard: string;
+ /** FieldEvent enum_set_string_values. */
+ public enum_set_string_values: boolean;
+
/**
* Creates a new FieldEvent instance using the specified properties.
* @param [properties] Properties to set
diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js
index 14d9ac922fc..bd77386d58c 100644
--- a/web/vtadmin/src/proto/vtadmin.js
+++ b/web/vtadmin/src/proto/vtadmin.js
@@ -75112,6 +75112,7 @@ export const binlogdata = $root.binlogdata = (() => {
* @property {Array.|null} [fields] FieldEvent fields
* @property {string|null} [keyspace] FieldEvent keyspace
* @property {string|null} [shard] FieldEvent shard
+ * @property {boolean|null} [enum_set_string_values] FieldEvent enum_set_string_values
*/
/**
@@ -75162,6 +75163,14 @@ export const binlogdata = $root.binlogdata = (() => {
*/
FieldEvent.prototype.shard = "";
+ /**
+ * FieldEvent enum_set_string_values.
+ * @member {boolean} enum_set_string_values
+ * @memberof binlogdata.FieldEvent
+ * @instance
+ */
+ FieldEvent.prototype.enum_set_string_values = false;
+
/**
* Creates a new FieldEvent instance using the specified properties.
* @function create
@@ -75195,6 +75204,8 @@ export const binlogdata = $root.binlogdata = (() => {
writer.uint32(/* id 3, wireType 2 =*/26).string(message.keyspace);
if (message.shard != null && Object.hasOwnProperty.call(message, "shard"))
writer.uint32(/* id 4, wireType 2 =*/34).string(message.shard);
+ if (message.enum_set_string_values != null && Object.hasOwnProperty.call(message, "enum_set_string_values"))
+ writer.uint32(/* id 25, wireType 0 =*/200).bool(message.enum_set_string_values);
return writer;
};
@@ -75247,6 +75258,10 @@ export const binlogdata = $root.binlogdata = (() => {
message.shard = reader.string();
break;
}
+ case 25: {
+ message.enum_set_string_values = reader.bool();
+ break;
+ }
default:
reader.skipType(tag & 7);
break;
@@ -75300,6 +75315,9 @@ export const binlogdata = $root.binlogdata = (() => {
if (message.shard != null && message.hasOwnProperty("shard"))
if (!$util.isString(message.shard))
return "shard: string expected";
+ if (message.enum_set_string_values != null && message.hasOwnProperty("enum_set_string_values"))
+ if (typeof message.enum_set_string_values !== "boolean")
+ return "enum_set_string_values: boolean expected";
return null;
};
@@ -75331,6 +75349,8 @@ export const binlogdata = $root.binlogdata = (() => {
message.keyspace = String(object.keyspace);
if (object.shard != null)
message.shard = String(object.shard);
+ if (object.enum_set_string_values != null)
+ message.enum_set_string_values = Boolean(object.enum_set_string_values);
return message;
};
@@ -75353,6 +75373,7 @@ export const binlogdata = $root.binlogdata = (() => {
object.table_name = "";
object.keyspace = "";
object.shard = "";
+ object.enum_set_string_values = false;
}
if (message.table_name != null && message.hasOwnProperty("table_name"))
object.table_name = message.table_name;
@@ -75365,6 +75386,8 @@ export const binlogdata = $root.binlogdata = (() => {
object.keyspace = message.keyspace;
if (message.shard != null && message.hasOwnProperty("shard"))
object.shard = message.shard;
+ if (message.enum_set_string_values != null && message.hasOwnProperty("enum_set_string_values"))
+ object.enum_set_string_values = message.enum_set_string_values;
return object;
};