github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/docs/RFCS/20180501_change_data_capture.md (about) 1 - Feature Name: Change Data Capture (CDC) 2 - Status: draft 3 - Start Date: 2018-05-01 4 - Authors: Arjun Narayan, Daniel Harrison, Nathan VanBenschoten, Tobias Schottdorf 5 - RFC PR: (PR # after acceptance of initial draft) 6 - Cockroach Issue: [#2656], [#6130] 7 8 9 # Summary 10 11 Change Data Capture (CDC) provides efficient, distributed, row-level change 12 subscriptions. 13 14 15 # Motivation 16 17 CockroachDB is an excellent system of record, but no technology exists in a 18 vacuum. Users would like to keep their data mirrored in full-text indexes, 19 analytics engines, and big data pipelines, among others. A pull model can 20 currently be used to do this, but it’s inefficient, overly manual, and doesn’t 21 scale; the push model described in this RFC addresses these limitations. 22 23 Anecdotally, CDC is one of the more frequently requested features for 24 CockroachDB. 25 26 27 # Guide-level explanation 28 29 The core primitive of CDC is the `CHANGEFEED`. Changefeeds target a whitelist of 30 databases, tables, partitions, rows, or a combination of these, called the 31 "watched rows". Every change to a watched row is emitted as a record in a 32 configurable format (initially JSON or Avro) to a configurable sink (initially 33 Kafka). Changefeeds can be paused, unpaused, and cancelled. 34 35 Changefeeds scale to any size CockroachDB cluster and are designed to impact 36 production traffic as little as possible. 37 38 By default, when a new changefeed is created, an initial timestamp is chosen and 39 the current value of each watched row as of that timestamp is emitted (similar 40 to `SELECT ... AS OF SYSTEM TIME`). After this "initial scan", updates to 41 watched rows are emitted. The `WITH cursor=<timestamp>` syntax can be used to 42 start a new changefeed that skips the initial scan and only emits all changes at 43 or after the user-given timestamp. 44 45 Kafka was chosen as our initial sink because of customer demand and its status 46 as the clear frontrunner in the category. Most of these customers also use the 47 associated Confluent Platform ecosystem and Avro format; our changefeeds are 48 designed to integrate well with them. Other sinks will be added as demand 49 dictates, but the details of this are out of scope. The rest of this document is 50 written with Kafka specifics, like topics and partitions, to make it easier to 51 understand, but these same processes will work for other sinks. 52 53 54 ## Row Ordering 55 56 Each emitted record contains the data of a changed row along with the timestamp 57 associated with the transaction that updated the row. Rows are sharded between 58 Kafka partitions by the row’s primary key. Tables with more than one [column 59 family] will require some buffering and are more expensive in a changefeed. If a 60 row is modified more than once in the same transaction, only the last will be 61 emitted. 62 63 Once a row has been emitted with some timestamp, no _previously unseen_ versions 64 of that row will be emitted with a lower timestamp. In the common case, each 65 version of a row will be emitted once, but some (infrequent) conditions will 66 cause them to be repeated, giving our changefeeds an at-least-once delivery 67 guarantee. See [Full-text index example: Elasticsearch]’s "external versions" 68 for one way to be resilient to these repetitions. 69 70 Many of the sinks we could support have limited exactly-once guarantees on the 71 consumer side, so there's not much to be gained by offering exactly-once 72 delivery on the producer side, when that's even possible. Kafka, however, does 73 have the necessary primitives for us to build an option to provide [exactly-once 74 delivery] at the cost of performance and scalability. 75 76 Cross-row and cross-table order guarantees are not given. 77 78 These particular ordering and delivery guarantees were selected to allow simple 79 cases to be easy and low-latency (after the initial catch-up and in the absence 80 of rebalancing, perhaps single or double digit milliseconds). 81 82 83 ## Cross-Row and Cross-Table Ordering 84 85 Some users require stronger order guarantees, so we provide the pieces necessary 86 to reconstruct them. (See [Transaction grouped changes] for why CockroachDB 87 doesn’t do this ordering itself.) 88 89 As mentioned above, each emitted record contains the timestamp of the 90 transaction that updated the row. This can be used with the in-stream _timestamp 91 resolved_ notifications on every kafka partition. The timestamp included in this 92 record is a guarantee that no previously unseen version of _any_ row will be 93 emitted to that partition afterward will have a higher timestamp. 94 95 Together, these can be used to give strong ordering and global consistency 96 guarantees by buffering records in between timestamp closures. See [Data 97 warehouse example: Amazon Redshift] below for a concrete example of how this 98 works. 99 100 101 ## Syntax Examples 102 103 Creating a changefeed is accomplished via `CREATE CHANGEFEED`: 104 105 ```sql 106 CREATE CHANGEFEED <name> FOR TABLE tweets INTO 'kafka://host:port' WITH <...> 107 CREATE CHANGEFEED <name> FOR TABLE tweets VALUES FROM (1) TO (2) INTO <...> 108 CREATE CHANGEFEED <name> FOR TABLE tweets PARTITION north_america INTO <...> 109 CREATE CHANGEFEED <name> FOR DATABASE db INTO <...> 110 ``` 111 112 - INTO `<sink>` is a URI that contains all the configuration information needed 113 to connect to a given sink. Any necessary configuration is done through query 114 parameters. This is similar to how the URIs work in BACKUP/RESTORE/IMPORT. 115 - `kafka://host:port` Kafka is used as the sink. The bootstrap server is 116 specified as the host and port. 117 - `?kafka_topic_prefix=<...>` A string to prepend to the topic names used by 118 this changefeed. 119 - `?schema_topic=<...>` A Kafka topic to emit all schema changes to. 120 - `?confluent_schema_registry=<address>` The address of a schema registry 121 instance. Only allowed with the avro format. When unspecified, no schema 122 registry is used. 123 - `experimental-sql:///` A [SQL table sink] in the same CockroachDB cluster. 124 - `WITH <...>` 125 - `WITH envelope=<...>` Records have a key and a value. The key is always set 126 to the primary key of the changed row and the value’s contents are 127 controlled with this option. 128 - `envelope='none'` [DEFAULT] The new values in the row (or empty for a 129 deletion). This works with Kafka log compaction. The system may internally 130 need to read additional column families. 131 - `envelope='key_only'` The value is always empty. No additional reads are 132 ever needed. 133 - `envelope='diff'` The value is a composite with the old row value and the 134 new row value. The old row value is empty for INSERTs and the new row 135 value is empty for DELETEs. The system will internally need additional 136 reads. 137 - `WITH format=<...>` 138 - `format='json'` [DEFAULT] The record key is a serialized JSON array. The 139 record value is a serialized JSON object mapping column names to column 140 values. 141 - `format='avro'` The record key is an [Avro] array, serialized in the 142 binary format. The record value is an Avro record, mapping column names to 143 column values, serialized in the binary format. 144 - `WITH cursor=<timestamp>` can be used to set the initial high-water mark. If 145 this option is used, the changefeed will emit any changes after the given 146 timestamp and no initial scan. 147 148 See [other CDC syntaxes] for alternatives that were considered. 149 150 151 ## Altering the changefeed 152 153 `PAUSE CHANGEFEED <name>` and `RESUME CHANGEFEED <name>` can be used to pause 154 and unpause changefeeds. 155 156 `DROP CHANGEFEED <name>` can be used to remove a changefeed. 157 158 `SHOW CREATE CHANGEFEED <name>` works similarly to the other `SHOW CREATE` 159 command and can be used view the changefeed definition. 160 161 The following `ALTER CHANGEFEED` commands can be used to change the 162 configuration. Only paused changefeed jobs can be altered, so the user may have 163 to `PAUSE CHANGEFEED` and `RESUME CHANGEFEED` as necessary. 164 165 `ALTER CHANGEFEED <name> INTO <...>` can be used to migrate Kafka clusters. 166 167 `ALTER CHANGEFEED <name> SET key=<...>` can be used to change the `WITH` 168 options. 169 170 `ALTER CHANGEFEED <name> SET cursor=<timestamp>` can be used to manually adjust 171 the high-water mark forward or back. When resumed, the changefeed will emit any 172 changes after the given timestamp, but will not emit the initial catch-up. To do 173 this, the changefeed must be cancelled and replaced with a new one. 174 175 176 ## Full-text index example: Elasticsearch 177 178 MovR is a fictional ride sharing company. Their users are quite particular about 179 which vehicle picks them up, so MovR has added a search box for unstructured 180 text when booking a ride. A full-text index is needed to make this work, but 181 CockroachDB doesn't (yet) offer one, so MovR uses CDC to keep vehicle 182 descriptions and their current availability in [ElasticSearch]. 183 184 The authoritative store for vehicles is a table in CockroachDB: 185 186 ```sql 187 CREATE TABLE vehicles ( 188 id UUID NOT NULL, 189 type STRING NULL, 190 status STRING NULL, 191 <...> 192 ) 193 ``` 194 195 The `type` field is unstructured and can be anything from "BMW Z4" to "Vespa 196 Scooter" to "1980s military tank". The `status` field is updated when the 197 vehicle becomes available or unavilable. 198 199 This is watched by a changefeed and emitted to Kafka: 200 201 ```sql 202 CREATE CHANGEFEED feed_vehicles FOR TABLE movr.vehicles INTO 'kafka:/...' 203 ``` 204 205 When initially run, the `CREATE CHANGEFEED` takes a snapshot of all the initial 206 states, consistent as of a timestamp and emits them to Kafka. All subsequest 207 changes are emitted as they happen. The default options are suitable for use 208 with the Confluent [ElasticsearchSinkConnector], so Movr uses this to tail the 209 Kafka `vehicles` topic and load it into ElasticSearch. 210 211 This is everything MovR needs for an ElasticSearch query over the `type` and 212 `status` fields to power the available vehicle search in the MovR app. 213 214 215 ## Data warehouse example: Amazon Redshift 216 217 MovR uses real-time analytics to determine whether there is more demand or 218 supply and adjust prices accordingly. The algorithm used is much easier to write 219 if it's always operating on a consistent view of the `rides` tables, but it 220 needs to run once per minute to keep prices up to date. CockroachDB is not (yet) 221 tuned for analytics, so MovR has decided to use Amazon Redshift in the meantime. 222 This Redshift instance is also used by MovR's Tableau dashboards. 223 224 A changefeed watches the `rides` table and continually emits changes to Kafka: 225 226 ```sql 227 CREATE CHANGEFEED feed_analytics FOR DATABASE movr.rides INTO 'kafka:/' WITH 228 format='avro', 229 confluent_schema_registry='<address>' 230 ``` 231 232 The entire history of rides must be available in Redshift for the pricing 233 algorithm to work, but occasionally new columns are added, so the Avro format 234 and the Confluent Schema Registry are used to deal with schema changes. 235 236 MovR is popular, so the rides don't fit in one Kafka partition and are sharded 237 between `N` of them. A set of `N` distributed processes are run, each tailing 238 one partition and writing every ride to s3 as it comes in. Each ride has a 239 timestamp and is written to s3 bucketed first by minute then by Kafka partition. 240 241 ``` 242 s3://movr/rides/2018-05-06-07-05/partition-0 243 s3://movr/rides/2018-05-06-07-05/partition-1 244 ... 245 s3://movr/rides/2018-05-06-07-05/partition-N 246 s3://movr/rides/2018-05-06-07-06/partition-0 247 ... 248 s3://movr/rides/2018-05-06-07-06/partition-N 249 ``` 250 251 When the `partition-0` process receives a resolved timestamp after 2018-05-06 252 07:06, then it's guaranteed that no previously unseen values for the 253 `2018-05-06-07-05` bucket will be emitted on that partition. (These 254 notifications typically trail realtime by about 10 seconds.) Once this has 255 happened for all of the partitions, then this bucket contains every value that 256 it ever will and MovR uses the Amazon Redshift `COPY` command to load it in from 257 s3. Redshift now contains a consistent snapshot of the table. 258 259 260 ## Failures 261 262 If Kafka is unavailable or underprovisioned the changefeed will buffer records 263 as necessary. Buffering is limited by the `--max-changefeed-memory` and 264 `--max-disk-changefeed-storage` flags. As with any changefeed, degraded sinks 265 minimally affect foreground traffic. Further, one table failing will not affect 266 other tables, even ones watched by the same changefeed. 267 268 It is crucial that Kafka recovers before the watched rows (and the system 269 tables) exit their respective [garbage collection] TTL (default 25 hours) with 270 enough time for the changefeed to catch up. This is similar to the restriction 271 on [incremental backups]. A metric is exported to monitor how close to the GC 272 TTL each changefeed’s most behind timestamp is. Production users should monitor 273 and alert on this metric. 274 275 If the sink is down for too long, some data may be lost and the changefeed will 276 be marked as failed. User intervention is required. Either a new changefeed can 277 be created with the existing Kafka topics and an appropriate `WITH cursor=<...>` 278 (which will leave a hole of missing data) or a new changefeed can be restarted 279 with an empty sink (a clean slate). 280 281 If Kafka recovery won't happen in time, the relevant garbage collection TTLs can 282 be manually increased by the user. This affects disk usage and query 283 performance, so the system will not do it automatically. Similarly, additional 284 partitions are added by the user to underprovisioned sinks because the 285 key-to-partition sharding would change. This would break any consumers that 286 assume all keys are in the same partition, a common assumption. 287 288 Records are first buffered in memory, but this is limited to at most 289 `--max-changefeed-memory` bytes across all changefeeds on a node. If this limit 290 is reached, records are then spilled to disk, up to the 291 `--max-disk-changefeed-storage` flag. If this limit is also reached, the 292 changefeed will enter a "stalled" state and will tear down its internal 293 connections to reduce load on the cluster. When the backlog clears out, the 294 changefeed will restart from where it left off. 295 296 There are also scenarios where this is concern even when the sink is healthy. 297 Particularly, if a changefeed is started on a large set of watched rows, the 298 initial scan may take longer than the garbage collection TTL. In this case, the 299 user will have to temporarily raise the GC TTL while the initial scan runs. 300 301 302 ## AdminUI 303 304 A new page is used in the AdminUI to show the status of changefeeds, distinct 305 from the "Jobs" page because changefeeds don't have a finite lifespan. 306 Incrementally updated materialized views, for example, would also appear on this 307 page. Similarly, a counterpart to `SHOW JOBS` is introduced. 308 309 310 # Reference-level explanation 311 312 Each changefeed is structured as a single coordinator which sets up a 313 long-running [DistSQL] flow with a new processor described below. 314 315 Each changefeed is given an entry in the [system jobs], which provides this 316 fault-resistent coordinator. Briefly, this works by a time based lease, which 317 must be continually extended; if it is not, a new coordinator is started. Some 318 failure scenarios result in a split-brain coordinator, but our ordering and 319 duplicate delivery guarantees were chosen to account for this. 320 321 In addition to the 0-100% progress tracking, the `system.jobs` tracking is 322 extended with an option to show the minimum of all resolved timestamps. 323 324 325 ## DistSQL flow 326 327 ```proto 328 // ChangeAggregatorSpec is the specification for a processor that watches for 329 // changes in a set of spans. Each span may cross multiple ranges. 330 message ChangeAggregatorSpec { 331 message Watch { 332 util.hlc.Timestamp timestamp = 1; 333 roachpb.Span span = 2; 334 } 335 repeated Watch watches = 1; 336 bool initial_scan = 2; 337 } 338 ``` 339 340 Each flow is made of a set of `ChangeAggregator` processors. Progress 341 information in the form of `(span, timestamp)` pairs is passed back to the job 342 coordinator via DistSQL metadata. 343 344 DistSQL is currently optimized for short-running queries and doesn’t yet have a 345 facility for resilience to processor errors. Instead, it shuts down the whole 346 flow. This is unfortunate for CDC, but failures should be sufficiently 347 infrequent and the amount of work duplicated when the flow is restarted 348 sufficiently small that it should be workable in practice. The largest impact is 349 on tail latencies. If/when DistSQL's capabilities change, this will be 350 revisited. 351 352 `ChangeAggregator`s are scheduled such that each is responsible for data local 353 to the node it’s running on. If the data it’s watching moves, the local data 354 will become remote, which will continue to work, but will use bandwidth and add 355 a network hop to latency. DistSQL doesn’t yet have a mechanism for moving or 356 reconfiguring processors in a running flow, but leaving this state indefinitely 357 is unfortunate. Instead, the job coordinator will periodically run some 358 heuristics and when it’s advantageous, stop and restart the flow such that all 359 processors will be doing local watches again. If the heuristics are not 360 aggressive enough, the user always has an escape hatch of manually pausing and 361 resuming the job. This restarting of the flow is costly in proportion to the 362 size of the changefeed and will affect tail latencies, so something should be 363 done here in future work. 364 365 GDPR will also require that processors be scheduled in a certain region, once 366 that's supported by DistSQL. 367 368 369 ## ChangeAggregator processor 370 371 A `ChangeAggregator` is responsible for watching for changes in a set of 372 disjoint spans, buffering during the initial catch up, translating kvs into sql 373 rows, and emitting to a sink. Additionally, it bookkeeps and emits the timestamp 374 resolved notifications described in [Cross-Row and Cross-Table Ordering]. 375 376 If the `initial_scan` option is set, then the current value of every watched row 377 at `timestamp` is emitted. `ChangeAggregator` then proceeds to set up 378 RangeFeeds. 379 380 381 ### RangeFeed 382 383 The [RangeFeed] (previously called ChangeFeed) command sets up a stream of all 384 changes to keys in a given span. The changes are returned exactly as they are 385 [proposed and sent through raft]: in RocksDB WriteBatch format, possibly with 386 unrelated data (parts of the range’s keyspace not being watched by the 387 `RangeFeed` and range-local keys). Notably, the range feed will simply stream 388 intents upstream (which means these intents may or may not commit or move 389 timestamps), and the distributed architecture has to deduplicate/track/resolve 390 them. 391 392 Passing the raft proposal through unchanged minimizes cpu load on the replica 393 serving the `RangeFeed`. This is not initially critical, since we’ll just be 394 scheduling the processors on the same node, but gives us flexibility later to 395 make changefeeds extremely low-impact on production traffic. (Note that a 396 `RangeFeed` can be thought of as a slightly lighter weight additional Raft 397 follower.) Using the same format as proposer-evaluated kv also lesses the chance 398 that we’ll have tricky rpc migration issues in the future. 399 400 The amount of unrelated data returned will be low if most or all of the range’s 401 keyspace is being watched, but in the worst case (changes touching many keys in 402 the range), may be high if only a small subset is watched. This means table and 403 partition watches will be efficient, but the efficiency of single-row watches 404 will depend on the workload. We may support single-row watches, but are not 405 optimizing for them yet, so this is okay. 406 407 `RangeFeed`s are run on the leaseholder for now. It should be possible to run 408 `RangeFeed`s on followers in the future, which becomes especially important when 409 ranges are geographically distributed. However, there are complications to work 410 through. Importantly, this follower could be the slowest one, increasing 411 commit-to-kafka changefeed latency. We also don’t currently have common code for 412 geography-aware scheduling; this would have to be built. 413 414 A new `RangeFeed` first registers itself to emit any relevant raft commands as 415 they commit. These are sent to the (ordered) rpc stream in the same order they 416 are in the raft log. These are buffered by `ChangeAggregator`. 417 418 After the `RangeFeed` is set up, the `ChangeAggregator` uses `ExportRequest` to 419 catch up on any changes between `watch.timestamp` and some timestamp after the 420 raft hook was registered. These partially overlap the `RangeFeed` output and 421 must be deduplicated with it. `ExportRequest` already uses the time-bounded 422 iterator to minimize the data read, but support for `READ_UNCOMMITTED` is added 423 so we don't need to block on intent resolution. The potential move to larger 424 range sizes may dictate that this becomes a streaming process so we don't have 425 to hold everything in memory, but details of that are out of scope. 426 427 There is an ongoing discussion about how `RangeFeed` will handle splits, merges, 428 leaseholder moves, node failures, etc. The tradeoffs will be worked out in an 429 update to the `RangeFeed` RFC and are out of scope here. However, it's important 430 to note that the `RangeFeed` will occasionally need to disconnect as part of 431 normal operation (e.g. if it receives a snapshot) and so the reconnection should 432 be handled as cheaply as possible, likely by using resolved timestamps as lower 433 bounds on the catch up scans. 434 435 436 ### Intent tracking 437 438 Once the catch up scans finish, the buffered data is processed. Single range 439 fastpath transations and committed intents are passed on to the next stage. 440 Aborted intents and unrelated data are filtered. This is guaranteed to emit 441 individual keys in mvcc timestamp order. Opened intents are tracked as follows. 442 443 The `RangeFeed` also returns in-stream _close notifications_ (the same ones as 444 [follower reads]) which are a guarantee that no new intents or transactions will 445 be written below a timestamp. Notably, this does not imply that all the intents 446 below it have been resolved. An in-memory structure is bootstrapped with all 447 open intents returned by the `ExportRequest` catch up scans. As intents are 448 resolved and follower read notifications arrive, `min(earliest unresolved 449 intent, latest follower read notification)` will advance. These advances 450 correspond to our resolved timestamp guarantees and are passed to later stages. 451 452 453 ### KV to row conversion 454 455 `TableDescriptor`s are required to interpret kvs as sql rows. The correctness of 456 our online schema changes depends on each query holding a "lease" for the 457 TableDescriptor that is valid for the timestamp it’s executing at. In contrast 458 to queries, which execute with a distinct timestamp, changefeeds operate 459 continuously. 460 461 Rows are stored using one kv per [column family]. For tables with one column 462 family, which is the common case and the default if the user doesn’t specify, 463 each entry in the `RangeFeed`'s WriteBatch is the entire data for the row. For 464 tables with more than one column family, a followup kv scan will be needed to 465 reconstruct the row. A similar fetch will be required if the user requests a 466 changefeed with both the old and new values. `RangeFeed` could instead send this 467 along when necessary, eliminating a network hop and the cost of serving a 468 request, but it’s preferable to avoid pushing knowledge of column families and 469 sql rows down to kv. 470 471 472 ### Sink emitter 473 474 The sink emitter receives an ordered stream of sql rows, schema changes, and 475 resolved timestamps. The sql rows are sent to the relevant kafka topic, sharded 476 into partitions by primary key. When a schema registry is provided, the schema 477 changes are forwarded. 478 479 Resolved timestamps are forwarded when requested by the user. We guarantee that 480 no records with a timestamp less than the resolved notification will be after 481 it, so progress must be synchronously written to the job entry before emitting a 482 resolved timestamp notification. 483 484 For easy development and testing, a 1 partition topic will be auto-created on 485 demand when missing. For production deployments, the user should set the Kafka 486 option to disallow auto-creation of topics and manually create the topic with 487 the correct number of partitions. A user may manually add partitions at any time 488 and this transparently works, though the row to partition mapping will change 489 and consumers are responsible for handling this. Kafka does not support removing 490 partitions. 491 492 493 ### Buffering 494 495 For simplicity and to even out cpu utilization, there is backpressure from Kafka 496 all the way back to the buffer between `RangeFeed` and the intent tracker. The 497 buffer starts in-memory but spills to disk if it exceeds its budget. If the disk 498 buffer also exceeds its budget, the `RangeFeed`s are temporarily shut down until 499 the emitting process has caught up. 500 501 502 ## SQL Table Sink 503 504 ```sql 505 CREATE TABLE <name> ( 506 table STRING, 507 message_id INT, 508 key BYTES, 509 value BYTES, 510 PRIMARY KEY (table, key, message_id) 511 ) 512 ``` 513 514 The SQL Table Sink is initially only for internal testing. A single table is 515 created and contains all updates for a changefeed, even if that changefeed is 516 watching multiple tables. The `key` and `value` are encoded using the `format` 517 specified by the changefeed. `message_id`s are only comparable between rows with 518 equal `key`s. 519 520 521 ## Truncate 522 523 For efficiency, in CockroachDB and in other databases, truncate is implemented 524 as a schema change instead of a large number of row deletions. This has 525 consequences for changefeeds. Consider the [Elasticsearch example] above. It’s 526 not using schema changes for anything, so rows in a truncated table would 527 continue to exist in Elasticsearch after truncate. 528 529 Postgres’s logical decoding is not aware of truncate or schema changes. We adopt 530 the SQL Server behavior, which prevents a table with an active changefeed from 531 being truncated. 532 533 534 ## Permissions 535 536 Changefeeds will initially be admin-only. Eventually we should allow non-admin 537 users to create changefeeds, and admins should be able to revoke those users' 538 permissions (breaking/cancelling the feeds). 539 540 541 542 # Drawbacks 543 544 ## Lots of configuration options 545 546 There are a number of options described above for format, envelope, sink 547 configuration, schema topics, etc. This is a lot, but they all seem necessary. 548 Simple cases like mirroring into Elasticsearch should work without the overhead 549 of avro, schema registries, and message envelopes. At the same time, we don’t 550 want to limit power by not having the ability for users to subscribe to the diff 551 of a changed row. Connectors in the Confluent platform and other CDC 552 implementations also seem to have a large number of configuration options, so 553 maybe this is just unavoidable. 554 555 556 ## No transaction grouped changes 557 558 One could easily imagine that a user would want to subscribe to a totally 559 ordered feed of all transactions that happen in the database. However, unlike 560 sharded PostgreSQL (for example), CockroachDB supports transactions touching any 561 part of the cluster. Which means that in the general case, there is no way to 562 horizontally divide up the transaction log of the cluster such that each piece 563 is independent. Our only options are (a) a single transaction log, which limits 564 scalability, (b) encoding the transaction dependency graph and respecting it 565 when replaying transactions, which is complex, or (c) taking advantage of 566 special cases (such and such tables are never used in the same transactions as 567 these other ones), which will not always work. 568 569 As a result, we’ve decided to give CockroachDB users the information necessary 570 to reconstruct transactions, but will not immediately be building anything to do 571 it in the general case. 572 573 574 # Rationale and alternatives 575 576 ## Alternative sinks 577 578 Our customers have overwhelmingly asked for Kafka as the sink they want for CDC 579 changes, so it will be the first we support but Kafka won’t work for everyone. 580 581 Some changefeeds will be low-throughput and for this a streaming SQL connection 582 (with the same wire format as PostgreSQL’s `LISTEN`) would be sufficient and 583 much simpler operationally. This lends itself well to single-row watches (in the 584 style of RethinkDB). 585 586 Other users will want their changefeeds to emit to a cloud-hosted pubsub. 587 588 Kafka is not the only sink we’ll need, but to limit scope these other sinks not 589 be in the initial version. 590 591 592 ## Write a Kafka connector 593 594 The Confluent platform ships with a [JDBC source connector], so we’re done, 595 right? It works by periodically scanning an incrementing id column or a last 596 updated timestamp column or both. The id column only works with append-only 597 tables and the last updated timestamp column must be maintained correctly by the 598 user. It works well for prototyping, but ultimately we feel that latency, 599 cluster load, and developer burden means that this doesn’t fit with our mission 600 to make data easy. 601 602 We could also write our own Confluent Platform connector. While Kafka will be 603 our initial sink, we don’t want to tie ourselves to it too closely. It’s also 604 not clear how we could make this work with end-to-end push to keep latencies and 605 performance impact low. 606 607 608 ## Exactly-once delivery 609 610 TODO: Description of kafka transactions and how they could be used for this. 611 612 613 # Unresolved Questions 614 615 - Flow control/throttling of the RangeFeed streams 616 - Lots of small changefeeds will require some coalescing 617 - A column whitelist, which could be useful for GDPR compliance or performance 618 - Backup/restore of changefeeds 619 - Details of recovering a failed changefeed using `revision_history` backups 620 621 622 # Appendix 623 624 ## Appendix: Kafka background 625 626 [Kafka] is used for all sorts of things, including as a pub/sub. The fundamental 627 abstraction is a named _topic_, which is a stream of _records_. 628 629 A topic is subdivided into a number of (essentially totally independant) 630 _partitions_. Each partition is a distributed, replicated, ordered, immutable 631 sequence of records that is continually appended to. Records are a timestamped 632 key/value byte pair. Keys are not required. If the timestamp is not specified, 633 it will be assigned by Kafka. Once appended to a partition, an _offset_ is given 634 to the record, so topic+partition+offset will uniquely identify a record. 635 636 The number of partitions in a topic can be increased, but never decreased. There 637 are no ordering guarantees across partitions. 638 639 Any producers for a topic may append new records to any partition, by default 640 hash of key or round-robin if no key is specified. Two records sent from a 641 producer to the same partition are guaranteed to retain their order in the log, 642 but there are no ordering guarantees across-producers. 643 644 Any number of _consumer groups_ may tail a topic. A consumer group is made up of 645 independent consumer processes. Each partition in a topic is assigned to exactly 646 one consumer in the group. By default, the consumer works beginning to end and 647 Kafka keeps track of the current position in each partition, handling consumer 648 failures as needed, but consumers may seek to positions at will. 649 650 Old log entries are cleaned up according to time/space/etc policies. 651 652 653 ## Appendix: Avro background 654 655 Avro is a serialization format, similar to protocol buffers. Notably, schemas 656 are designed to be serialized and stored alongside the serialized records. The 657 spec defines clear "[schema resolution]" rules for asserting whether a 658 progression of schemas is forward or backward compatible and for reading data 659 written with an older or newer version of the schema. The [Confluent Schema 660 Registry] integrates Kafka and Avro and allows consumers to declare which schema 661 evolutions are supported. 662 663 664 ## Appendix: Other CDC syntaxes 665 666 Whenever possible, we reuse PostgreSQL syntax in an attempt to minimize user 667 friction. For CDC, there are two potentially relevant feature: logical decoding 668 and logical replication. 669 670 Logical decoding allows a dynamically loaded plugin to access the WAL. This is 671 how most Postgres CDC works. For example, [Debezium] uses logical decoding to 672 hook PostgreSQL up to Kafka. Unfortunately, the syntax is tightly tied to 673 PostgreSQL's notion of replication slots, which don't map cleanly onto 674 CockroachDB's distributed ranges and replicas. 675 676 Logical replication is used to stream changes from one PostgreSQL instance to 677 another to keep them in sync. Its syntax is in terms of producers and 678 subscribers. The biggest reason to have drop-in compatibility is so tooling, 679 such as ORMs, will work out of the box, but it's unlikely that ORMs would be 680 creating changefeeds. Additionally, our model of push to a sink is different 681 enough from a subscriber pulling from a producer that trying to reuse syntax is 682 likely to be more confusing than helpful. This syntax could always be added as 683 an alias if we build support for CockroachDB to CockroachDB replication. 684 685 686 687 [#2656]: https://github.com/cockroachdb/cockroach/issues/2656 688 [#6130]: https://github.com/cockroachdb/cockroach/issues/6130 689 [amazon redshift]: https://aws.amazon.com/redshift/ 690 [avro]: #appendix-avro-background 691 [rangefeed]: https://github.com/cockroachdb/cockroach/blob/381e4dafa596c5f3621a48fcb5fce1f62b18c186/docs/RFCS/20170613_change_feeds_storage_primitive.md 692 [column family]: https://www.cockroachlabs.com/docs/stable/column-families.html 693 [confluent schema registry]: https://docs.confluent.io/current/schema-registry/docs/index.html 694 [cross-row and cross-table ordering]: #cross-row-and-cross-table-ordering 695 [data warehouse example: amazon redshift]: #data-warehouse-example-amazon-redshift 696 [debezium]: http://debezium.io 697 [distsql]: https://github.com/cockroachdb/cockroach/blob/381e4dafa596c5f3621a48fcb5fce1f62b18c186/docs/RFCS/20160421_distributed_sql.md 698 [elasticsearch]: https://www.elastic.co/products/elasticsearch 699 [elasticsearch-example]: #fulltext-index-example-elasticsearch 700 [elasticsearchsinkconnector]: https://docs.confluent.io/current/connect/connect-elasticsearch/docs/elasticsearch_connector.html 701 [exactly-once delivery]: #exactly-once-delivery 702 [follower reads]: https://github.com/cockroachdb/cockroach/pull/26362 703 [full-text index example: elasticsearch]: #full-text-index-example-elasticsearch 704 [garbage collection]: https://www.cockroachlabs.com/docs/stable/architecture/storage-layer.html#garbage-collection 705 [incremental backups]: https://www.cockroachlabs.com/docs/stable/backup.html 706 [jdbc source connector]: https://docs.confluent.io/current/connect/connect-jdbc/docs/source_connector.html 707 [kafka]: https://kafka.apache.org/intro 708 [other cdc syntaxes]: #appendix-other-cdc-syntaxes 709 [proposed and sent through raft]: https://github.com/cockroachdb/cockroach/blob/381e4dafa596c5f3621a48fcb5fce1f62b18c186/docs/RFCS/20160420_proposer_evaluated_kv.md 710 [schema resolution]: http://avro.apache.org/docs/current/spec.html#Schema+Resolution 711 [sql table sink]: #sql-table-sink 712 [system jobs]: https://github.com/cockroachdb/cockroach/blob/381e4dafa596c5f3621a48fcb5fce1f62b18c186/docs/RFCS/20170215_system_jobs.md 713 [transaction grouped changes]: #no-transaction-grouped-changes 714 [follower read]: https://github.com/cockroachdb/cockroach/pull/19222