Commit Graph

941 Commits (e3e801d29aa31b019b8e3ebaff6875617b9a01a6)

Author SHA1 Message Date
Andrew Lamb 3ea84c6be4
feat: expose null_counts in system.chunk_columns (#2105)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-27 11:05:23 +00:00
kodiakhq[bot] 5551dd3a87
Merge branch 'main' into devnull 2021-07-27 09:57:16 +00:00
kodiakhq[bot] 119b913fa3
Merge branch 'main' into crepererum/improve_replay_tests 2021-07-27 07:27:58 +00:00
Andrew Lamb 5fb3e00f2a
fix: Properly record total_count and null_count in statistics (#2103)
* fix: Properly record total_count and null_count in statistics

* fix: fix statistics calculation in mutable_buffer

* refactor: expose null counts in read_buffer

* refactor: expose null_count in parquet_file

* fix: update server crate tests

* fix: update query_tests tests

* docs: tweak comments

* refactor: Use storage_stats rather than adding `null_count`

* refactor: rename test data field for clarity

* fix: fixup merge conflicts

* refactor: rename initial_non_null_count to initial_total_count

* refactor: caculate null_count as row_count - to_add
2021-07-26 18:13:36 +00:00
Marko Mikulicic 094945a72d
feat: Add '/dev/null' sink 2021-07-26 19:19:11 +02:00
Marco Neumann d7e0b03064 refactor: use `drop` instead of `Option` 2021-07-26 17:43:03 +02:00
Marco Neumann 2d5a095d2d refactor: rename `ActionOrTest` to `Step` 2021-07-26 17:34:13 +02:00
Marco Neumann 5787fbdb21 refactor: rename framework tests 2021-07-26 17:32:46 +02:00
Marco Neumann aa61eb2732 refactor: improve replay test naming and add more docs 2021-07-26 17:31:13 +02:00
Marco Neumann 43cb148566 fix: docstring
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
2021-07-26 17:14:01 +02:00
Marco Neumann 43f29422f8 refactor: isolate replay code and improve tests
This puts all the replay logic under `server::db::replay` as well as its
error variants and tests.

The tests are reworked using a more generic
test framework which allows us to specify a number of steps instead of
filling pre-defined ones with variables. Each step is either an action
(e.g. restart DB, perform replay, ingest data into the write buffer
state) or a check (e.g. assert that these partitions exists, await until
the background workers has ingested these partitions). The entire
framework is kept generic so it should be easy to create more checks and
actions in the future. The resulting tests are more verbose, but (at
least in my opinion) easier to follow along since the reader can see
what's happening at which step and does not jump back and forth between
the test config and the "driver" that uses the config.
2021-07-26 17:14:01 +02:00
kodiakhq[bot] 009c77d864
Merge branch 'main' into cn/parquet-first-last 2021-07-26 14:59:54 +00:00
Raphael Taylor-Davies 0b88deea43
refactor: don't pass sequence to MUB (#2107)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-26 14:40:39 +00:00
Marko Mikulicic e5ee252876
feat: Add kafka sink variant 2021-07-26 11:08:02 +02:00
Marko Mikulicic d58a3ccbc7
refactor: Add sink to routing config
This deprecates the "target" field in the RoutingConfig and replaces it with the "sink"
field, which has a variant that accepts a node group.

This commit is backward compatible in that it will accept existing configs.
The configs will roundtrip to the new format though (i.e. `database get` will render
the sink field).
2021-07-26 11:08:01 +02:00
Marko Mikulicic 16a82ba350
refactor: Generailize sinks: Rename Shard to Sink
The ShardConfig applies matchers that resolve to a shard number.
The config then applies a mapping between shard numbers to targets.
The type that encapsulated the target that a shard points to was also called
a "Shard". This is confusing. This commit changes it to "Sink", i.e. a destination
for traffic to go to. Subsequent commits will expand the definition of a Sink to
encompass different kinds of sinks (like kafka write buffer, "devnull", ...)

This changes only the name of the protobuf message and the related rust types,
it doesn't change any name of the json-rendered protobuf configs.
2021-07-26 11:08:00 +02:00
Raphael Taylor-Davies c595039c81
feat: add row timestamp metrics (#2101)
* feat: add row timestamp metrics

* chore: review feedback
2021-07-23 19:17:11 +00:00
Jake Goulding d928bc84e6 feat: Thread time_of_{first,last}_write through Parquet metadata 2021-07-23 14:07:35 -04:00
Raphael Taylor-Davies 446af5eb15
fix: consistent write timestamps (#2104)
* fix: consistent write timestamps

* chore: fix benchmarks
2021-07-23 18:04:15 +00:00
Carol (Nichols || Goulding) 3c794153dd refactor: Organize uses 2021-07-23 13:48:15 -04:00
Carol (Nichols || Goulding) 7de946c534 fix: ChunkStage::WrittenToObjectStore is now called ChunkStage::Persisted 2021-07-23 13:11:42 -04:00
Raphael Taylor-Davies 844a025c7c
feat: drop based on LRU (#2075) (#2092)
* feat: drop based on LRU (#2075)

* chore: review feedback

Co-authored-by: Andrew Lamb <alamb@influxdata.com>

Co-authored-by: Andrew Lamb <alamb@influxdata.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-23 08:31:28 +00:00
Marco Neumann 53b00ec4e0 test: split replay tests 2021-07-23 10:17:02 +02:00
Marco Neumann be1bc7025c refactor: use a single seek loop during replay 2021-07-23 10:05:11 +02:00
Marco Neumann ace247d5c2 feat: add replay logging 2021-07-23 10:03:02 +02:00
Marco Neumann 0c89930b7c feat: check that replay plan and write buffer are in-sync 2021-07-23 09:39:46 +02:00
Marco Neumann db0f501b02 feat: implement naive replay 2021-07-23 09:24:04 +02:00
Marco Neumann 6ef3680554 feat: collect replay plan during catalog loading 2021-07-23 09:23:06 +02:00
kodiakhq[bot] 71f3f1aba2
Merge branch 'main' into cn/refactorings 2021-07-22 19:44:18 +00:00
Andrew Lamb 01c79f1a1a
fix: Print all timestamps using RFC3339 format (#2098)
* fix: Use IOx pretty printer rather than arrow pretty printer

* chore: update tests in the query crate

* chore: update influxdb_iox tests

* chore: Update end to end tests

* chore: update query_tests

* chore: update mutable_buffer tests

* refactor: update parquet_file tests

* refactor: update db tests

* chore: update kafka integration test output

* fix: merge conflict
2021-07-22 19:04:52 +00:00
Raphael Taylor-Davies 20d06e3225
feat: include more information in system.operations table (#2097)
* feat: include more information in system.operations table

* chore: review feedback

Co-authored-by: Andrew Lamb <alamb@influxdata.com>

Co-authored-by: Andrew Lamb <alamb@influxdata.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-22 17:16:09 +00:00
Carol (Nichols || Goulding) 14cb2a6bef test: Add assertions for first/last write times as chunks move 2021-07-22 11:35:23 -04:00
Carol (Nichols || Goulding) 37f24ebfc7 feat: Record first/last write times for creation of read_buffer::Chunk 2021-07-22 11:35:23 -04:00
Carol (Nichols || Goulding) 0c44179aa9 feat: Add first/last write time on DbChunk
To eventually be used in collect_rub
2021-07-22 11:35:23 -04:00
Carol (Nichols || Goulding) 8d1d877196 feat: Record first/last write times for RUB chunks 2021-07-22 11:35:22 -04:00
Carol (Nichols || Goulding) 28fc01ecee test: Make test failure messages easier to read 2021-07-22 11:15:19 -04:00
Carol (Nichols || Goulding) 6feea3b2d5 feat: Require at least one RecordBatch to create a read_buffer::Chunk::new
In the signature only for the moment.
2021-07-22 11:15:18 -04:00
Carol (Nichols || Goulding) d347750366 refactor: Make collect_rub create the RBChunk
Which gets rid of the need for new_rub_chunk.

This will enable creating RBChunks that are guaranteed to have data.
2021-07-22 11:15:18 -04:00
Carol (Nichols || Goulding) 0a724878e6 refactor: Organize uses 2021-07-22 11:15:18 -04:00
Carol (Nichols || Goulding) 7371b0aabf refactor: Use existing new_rub_chunk function that has the same code 2021-07-22 11:15:18 -04:00
Carol (Nichols || Goulding) eadcb3265a refactor: Use some TryStreamExt adapters in collect_rub 2021-07-22 11:15:18 -04:00
Raphael Taylor-Davies 38e375d11a
feat: add chunk storage metrics (#2069)
* feat: add chunk storage metrics

* chore: review feedback

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-22 15:13:09 +00:00
Raphael Taylor-Davies 8c974beba0
feat: add access timestamps to CatalogChunk (#2075) (#2081)
* feat: add access timestamps to CatalogChunk (#2075)

* chore: review feedback

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-22 12:19:30 +00:00
kodiakhq[bot] 8c4f5cb237
Merge branch 'main' into crepererum/fix_db_checkpoints 2021-07-21 16:46:13 +00:00
Marco Neumann cddf94653c refactor: use `write_buffer` subsystem for ingest metrics 2021-07-21 15:07:59 +02:00
Marco Neumann fd00206fbb refactor: increase watermark update frequence to once per 10s 2021-07-21 15:02:48 +02:00
Marco Neumann 2f1efcf517 docs: clarify difference 2021-07-21 15:00:53 +02:00
Marco Neumann 4d5f209030 docs: do not repeat unix that often 2021-07-21 14:59:07 +02:00
Marco Neumann ec866de193 fix: collect checkpoint data from all tables 2021-07-21 14:48:29 +02:00
Marco Neumann 7d597d1d5c refactor: make ingest metrics easier to understand 2021-07-21 13:57:53 +02:00
Marco Neumann fb931bb1ca feat: write buffer ingestion metrics 2021-07-21 11:59:52 +02:00
Raphael Taylor-Davies 091837420f
feat: add PersistenceWindows sytem table (#2030) (#2062)
* feat: add PersistenceWindows sytem table (#2030)

* chore: update log

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-20 13:10:57 +00:00
Raphael Taylor-Davies e4d2c51e8b
fix: update PersistenceWindows on rules update (#2018) (#2060)
* fix: update PersistenceWindows on rules update (#2018)

* chore: review feedback

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-20 12:44:47 +00:00
kodiakhq[bot] 58dd7e9532
Merge branch 'main' into crepererum/writer_buffer_seek 2021-07-20 12:29:18 +00:00
Raphael Taylor-Davies cf8a60252d
refactor: split system_tables module into smaller modules (#2061)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-20 12:19:20 +00:00
Marco Neumann ec7ebdff29 refactor: use lifetimes to ensure single stream / no seek while streaming 2021-07-20 13:52:33 +02:00
Marco Neumann b0663a0337 feat: disallow multiple write buffer streams and seeking while streams
Multiple streams will mess up ordering. Seeking while streaming is
likely a bug and should not work.
2021-07-20 12:35:20 +02:00
Raphael Taylor-Davies 767c2a6fe1
refactor: explicit server startup state machine (#2040)
* refactor: explicit server startup state machine

* chore: update `ServerStage` docs

* chore: further docs

* chore: more logging

* chore: format
2021-07-20 10:11:18 +00:00
kodiakhq[bot] 5bf68c4a57
Merge branch 'main' into jg/snafu-driveby 2021-07-19 20:20:30 +00:00
Raphael Taylor-Davies 1c8c227668
refactor: push database rules update into Db (#2052)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-19 16:05:21 +00:00
kodiakhq[bot] 1d1ac12522
Merge branch 'main' into crepererum/write_buffer_multiple_streams 2021-07-19 15:50:42 +00:00
Andrew Lamb 4da8a16c18
chore: update to arrow 5.0 and master datafusion (#2049)
* chore: update to arrow 5.0 and master datafusion

* fix: Update test for change in object size
2021-07-19 12:49:51 +00:00
Raphael Taylor-Davies e2a23c7ac3
fix: persist deadlock (#2045) (#2046) 2021-07-19 11:52:48 +00:00
Marco Neumann 592424c896 refactor: use one stream per sequencer/partition
Advantages are:

- for large DBs w/ many partitions we can ingest data in-parallel
- on top of this change we can implement per-sequencer seeking, which is
  required for replay
2021-07-19 12:26:58 +02:00
kodiakhq[bot] a1d47a8a7a
Merge branch 'main' into crepererum/simplify_testdb_lifecycle_rules 2021-07-19 09:53:35 +00:00
Raphael Taylor-Davies 5fc98c7c56
feat: add failure reporting to TaskTracker (#2031)
* feat: add failure reporting to TaskTracker

* chore: review feedback

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-19 09:17:20 +00:00
Marco Neumann 2263189e09 test: make TestDb lifecycle better for testing
This is a leftover from #1972.
2021-07-19 09:50:44 +02:00
Jake Goulding 449ba46b22 refactor: Make more use of SNAFU's context methods and ensure! macro 2021-07-16 16:31:50 -04:00
Edd Robinson 54ad69ed86 fix: ensure correct table meta size used 2021-07-16 10:48:45 -04:00
Marco Neumann f57ba6afdb
fix: use fixed-size timestamps for parquet metadata (#2032)
This fixes flaky tests that rely on predictable files sizes.

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-16 13:14:02 +00:00
Marco Neumann 2498642c00 fix: `persist_partition` docstring 2021-07-16 12:46:07 +02:00
Marco Neumann 1ef2bc1887 refactor: `Db::{write_chunk_to_object_store => Db::persist_partition}`
The previous method allowed to persist any chunk -- even ones that
should not be persisted yet and w/o any order of peristence. That will
break our persistence windows. So instead offer a sane higher-level
interface that can trigger persistence of a partition within the
boundaries of the lifecycle rules. This needs some adjustments for our
test suite.
2021-07-16 12:07:58 +02:00
Marco Neumann 9683d91f32 refactor: adjust to upstream changes 2021-07-16 11:45:34 +02:00
Marco Neumann 2b0a4bbe0a feat: persist real (non-fake) part.+DB checkpoints 2021-07-16 11:45:34 +02:00
Marco Neumann 8276511bd3 feat: allow to construct partition checkpoint from partition 2021-07-16 11:45:34 +02:00
Marco Neumann a9ea8e9ced docs: add docstring to some `Partition` methods 2021-07-16 11:45:34 +02:00
Marco Neumann 71b5030fc0 refactor: remove unused `LockableChunk::write_to_object_store` 2021-07-16 11:45:34 +02:00
Raphael Taylor-Davies 00b89cd751
fix: freeze chunks in write path (#2021) (#2022)
* fix: freeze chunks in write path (#2021)

* chore: review feedback

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-16 08:51:37 +00:00
kodiakhq[bot] 50aa1f857d
Merge branch 'main' into ntran/refactor_use_sort_key 2021-07-15 21:17:26 +00:00
kodiakhq[bot] 76d9b8f7cc
Merge branch 'main' into debugkafka 2021-07-15 21:07:35 +00:00
Edd Robinson d5dcb40438
refactor: track future execution (#2014)
* refactor: track future execution

* refactor: update server/src/db/lifecycle/compact.rs

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-15 20:58:52 +00:00
Marko Mikulicic 06399e88e0
chore: Add some debug logs to write buffer 2021-07-15 22:18:03 +02:00
Nga Tran cfe0bfa88b refactor: address review comments and add useful log info to catch resort 2021-07-15 15:39:12 -04:00
Andrew Lamb 3fd6430fb6
fix: rename `estimated_bytes` to `memory_bytes` and expose `object_store_bytes` in ChunkSummary and system.chunks (#2017)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-15 16:00:24 +00:00
Marco Neumann b5428e53a5 refactor: write buffer testing + better mocking
This refactors the write buffer a bit for:

- **Testing:** Add generic tests for the Kafka and the mocking
  implementation. The same interface can be used easily add new
  implementations (e.g. via Redis, filesystem, ...).
- **Partition on Write:** The caller of the writer operation must now
  specify the partition/sequencer ID. The implicit partitioning of the
  Kafka writer would have lead to broken data since we must never spill
  entries w/ the same primary key over multiple partitions. At the
  moment we will only use partition 0 but we can easily implement
  better logic in the future.
- **Improved Mocking:** The mocked implementation now simulates a system
  that feels more real. Especially the handling around multiple streams
  and "write while read" has been improved. This will be helpful for
  testing and for new features like seeking (during replay). A solid
  realistic mock also helps us to ensure that the tests using the mock
  do not rely on unrealistic behavior too much.
2021-07-15 17:20:45 +02:00
Raphael Taylor-Davies d71f38f27c
feat: compute PartitionCheckpoint from PersistenceWindows (#2011)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-15 12:17:23 +00:00
Andrew Lamb 0c86d1dccf
feat: Record parquet bytes size in catalog / parquet_file (#2006)
* feat: Store object store size in parquet_file

* fix: update TRANSACTION_VERSION to 8

* refactor: rename os_bytes --> file_size_bytes
2021-07-15 12:07:11 +00:00
Marco Neumann 4741483f72 docs: explain why we update memory metrics when lifecycle action is cleared 2021-07-15 12:07:56 +02:00
Marco Neumann 924b0db542 fix: account for memory size in drop lifecycle action 2021-07-15 12:07:56 +02:00
Marco Neumann cccdd8a43f fix: correct code comment 2021-07-15 12:07:56 +02:00
Marco Neumann 77a9191a11 fix: chunk dropping over lifecycle policy should also respect the preserved catalog 2021-07-15 12:07:56 +02:00
Marco Neumann 71cb15f017 refactor: use lifecycle action to drop chunks
This avoids holding partition locks while the preserved catalog IO is
done.
2021-07-15 12:07:56 +02:00
Marco Neumann e570c66697 feat: add "dropping" chunk lifecycle action 2021-07-15 12:07:56 +02:00
Marco Neumann 68e20779a2 test: add test for clearing lifecycle actions from chunks 2021-07-15 12:07:56 +02:00
Marco Neumann d89fca00be feat: persist "drop chunk" 2021-07-15 12:07:56 +02:00
Raphael Taylor-Davies 3e0d1eb560
refactor: introduce PartitionAddr (#2010) 2021-07-15 10:01:33 +00:00
Nga Tran 0b1f2b1fd0 chore: merge main to branch 2021-07-14 16:17:14 -04:00
Nga Tran 552e3fb691 fix: Padd stats compute deterministic order of sort key and update tests that got changed by the use of sort key 2021-07-14 14:06:41 -04:00
kodiakhq[bot] 833debd5b5
Merge branch 'main' into cn/exploration 2021-07-14 17:30:55 +00:00
Raphael Taylor-Davies cbeeb97cff
feat: flush open window on persist (#1985)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-14 16:58:20 +00:00
Raphael Taylor-Davies 1d00fa2fd8
refactor: track memory metrics in catalog (#1995)
* refactor: track memory metrics in catalog

* chore: update comment
2021-07-14 16:23:00 +00:00
Carol (Nichols || Goulding) 8070065e2f fix: Change RUB chunk table_summaries to table_summary
Because chunks now have only one table.

Connects to #1718, #1613, #1295
2021-07-14 11:18:02 -04:00
Carol (Nichols || Goulding) 649b467adb fix: CatalogChunk no longer needs to record a write when created from a MUB chunk 2021-07-14 10:28:12 -04:00
Carol (Nichols || Goulding) 7ccbab8c90 feat: Make a TableSummaryAndTimes to use to slowly replace TableSummary
And use TableSummaryAndTimes with the mutable buffer chunks when turning
them into catalog chunks.

It's proving too big to switch over everything using TableSummary at
once, so this will let us switch over more incrementally.
2021-07-14 10:28:12 -04:00
Edd Robinson 4dedb657f2
Merge branch 'main' into alamb/go_go_go_go 2021-07-14 14:04:13 +01:00
Raphael Taylor-Davies f1c1620c84
feat: make persistence windows interface harder to use incorrectly (#1977)
* feat: make persistence windows interface harder to use incorrectly

* chore: review feedback

* chore: update comment

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-14 13:03:18 +00:00
Edd Robinson 0e5276ed20
Merge branch 'main' into alamb/go_go_go_go 2021-07-14 13:56:35 +01:00
Marco Neumann 9cb9ae0874 chore: move write buffer into its own crate 2021-07-14 14:09:18 +02:00
Marko Mikulicic d427fed9dc
fix: Remove bad max.request.size config param 2021-07-14 13:54:18 +02:00
Nga Tran 8fd0df04f2 feat: continue buidling and using sort_key if available 2021-07-13 16:25:58 -04:00
Andrew Lamb 781c4fa666 fix: update server tests 2021-07-13 15:44:57 -04:00
Marko Mikulicic 239c931f26
fix: Raise max message to 10M
And log message size on kafka write error.

Turns out the kafka partition message size limit default is 1MB, but also the
client side "max request size" default is also 1MB.
The error message we get from our kafka client is misleading: it says

```
KafkaError (Message production error: MessageSizeTooLarge (Broker: Message size too large)) }
```

which to my mind it seemed like if ("Broker:") the broker said "Message size too large".
That was a lie; I killed the broker and the client kept saying the same error message which means
it didn't even try to send the message out.

TODO: make this a proper parameter. (but let's unblock)
2021-07-13 17:47:36 +02:00
kodiakhq[bot] 6a09678f34
Merge branch 'main' into crepererum/update_deps 2021-07-13 14:18:57 +00:00
Raphael Taylor-Davies 6c8b2b4fa7
feat: add integration test of compaction freezing (#1938) (#1975)
* feat: add integration test of compaction freezing (#1938)

* chore: update server/src/db/lifecycle/compact.rs

Co-authored-by: Andrew Lamb <alamb@influxdata.com>

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
2021-07-13 14:11:10 +00:00
Marco Neumann 157a0cc98c chore: update flatbuffers to 2.0 2021-07-13 15:44:45 +02:00
Marko Mikulicic bf20641d78
chore: Log whether the write buffer is enabled 2021-07-13 14:15:52 +02:00
Raphael Taylor-Davies 5a0caeab44
feat: skip over fully persisted partitions (#1962) (#1973)
* feat: skip over fully persisted partitions (#1962)

* chore: review feedback
2021-07-13 10:40:45 +00:00
Andrew Lamb d35b74c226
fix: Fix doc build warnings (#1945)
* fix: Fix doc build warnings

* refactor: add deny bare_urls to crates

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-13 08:03:42 +00:00
Paul Dix 708aebaeb3
fix: freeze open chunk when compacting (#1971)
Closes #1938. Unfortunately, this contains only a unit test to ensure that open chunks are frozen when set_compacting is called. It would be better to have a more end-to-end integration test that ensurest his behavior, but I've confirmed by hand (with some sleeps and a hacked up end-to-end test) that this fixes it.
2021-07-13 07:44:02 +00:00
Nga Tran 5418a1fe6b refactor: remove unused comments 2021-07-12 18:14:38 -04:00
Nga Tran 23895e6673 feat: Using sort_key to avoid resorts 2021-07-12 18:08:45 -04:00
Carol (Nichols || Goulding) 6764a2d68e fix: Write Buffer errors are known, not UnknownDatabaseErrors
Fixes #1956.
2021-07-12 11:21:31 -04:00
Carol (Nichols || Goulding) 3bd7486016 test: Rename a test type alias to not shadow super::Error 2021-07-12 10:46:29 -04:00
Andrew Lamb 670826daf9
refactor: make object_store construction interface consistent (#1944)
* refactor: make object_store construction interface consistent

* fix: benchmarks

* fix: doc build

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-12 12:56:36 +00:00
Andrew Lamb 9534220035
feat: Add any lifecycle_action to system.chunks and API (#1947) 2021-07-09 17:38:29 +00:00
Raphael Taylor-Davies 7af560aa99
feat: Persist lifecycle action (#1888)
* feat: add split and persist operation

* docs: Improve doc strings

* refactor: use for loop rather than map

* refactor: Make it clear that the lifecycle policy picks the split timestamp

* fix: race condition

* docs: improve comments

* fix: logical merge conflict

* fix: clippy

Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
2021-07-09 13:21:46 +00:00
Andrew Lamb 1a79bf7e99
refactor: Make aws/azure/gcs optional features and stop compiling 100 dependencies during dev (#1933)
* feat: make aws, gcp, azure dependencies optional

* fix: only run object store tests if the features are enabled

* fix: clean up testing

* fix: rename step

* fix: add to list of jobs

* fix: remove test with object store

* fix: review comments
2021-07-09 11:38:30 +00:00
Andrew Lamb 3cb8f297b1
refactor: encapsulate the ObjectStore implementations in the object store crate (#1932)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-09 10:38:32 +00:00
Marco Neumann bc958e2ff0 refactor: use Arcs to pass schemas around 2021-07-09 09:45:12 +02:00
Marco Neumann 09e611deb7 refactor: lift query schema generation up to caller
Do no longer scan chunks during query planning to determine the schema
(except for the lifetime jobs where we have a good reason to do so).
Instead pass the schema down to from whoever is triggering the query.
For real SQL queries, we then just use the the table-wide schemas
introduced in #1913.

Apart from avoiding schema merges we now also don't crash any longer
when no chunks are left in the table (aka columns are present but all
rows are gone).

Fixes #1768.
Fixes #1884.
2021-07-09 09:24:21 +02:00
kodiakhq[bot] c37053ad46
Merge branch 'main' into cn/chunk-times 2021-07-08 20:58:54 +00:00
kodiakhq[bot] a2726c7e92
Merge branch 'main' into cn/kafka-read-metrics-and-e2e-tests 2021-07-08 20:40:19 +00:00
Carol (Nichols || Goulding) 22495dd355 fix: Take a TableBatch in the MBChunk constructor
Thus ensuring all MBChunks will have data in them.
2021-07-08 16:39:35 -04:00
Carol (Nichols || Goulding) 548c64539e fix: Wrap lines at 100 chars 2021-07-08 16:39:33 -04:00
Carol (Nichols || Goulding) 74c0a6cb00 fix: Arrange use statements so rustfmt can manage their order 2021-07-08 16:39:02 -04:00
kodiakhq[bot] c8126784a8
Merge branch 'main' into ntran/avoid_sort_in_scan 2021-07-08 20:22:18 +00:00
Andrew Lamb 72928aab3d
refactor: Move ChunkLifecycleAction to the data_types crate (#1939) 2021-07-08 20:18:33 +00:00
Andrew Lamb dd3eff7748
refactor: Always use `row_count` for count of rows in system.* tables (#1937) 2021-07-08 19:28:11 +00:00
Carol (Nichols || Goulding) c6bf0a26f4 feat: Add metrics for when ingesting from the write buffer fails
So that we have some way of figuring out what might be going on.
2021-07-08 09:57:51 -04:00
Carol (Nichols || Goulding) 80e1dcafe0 feat: Support reading from all Kafka partitions
When reading from the Kafka write buffer, subscribe to all partitions in
a topic and start from the smallest offset available, instead of
assuming there will only be 1 partition per topic.
2021-07-08 09:30:59 -04:00
Carol (Nichols || Goulding) c90ef7b14b fix: Create one consumer group per server+database
This hasn't caused any problems for me yet, but seemed like a good idea
because we want to be sure we don't get any of Kafka's consumer
rebalancing if we have multiple partitions.
2021-07-08 09:28:34 -04:00
Carol (Nichols || Goulding) e5168936f5 feat: Better error messages through to gRPC API + e2e Kafka Read tests 2021-07-08 09:28:34 -04:00
Carol (Nichols || Goulding) c53ae41d57 fix: Remove unneeded Option from the reading mock 2021-07-08 09:28:34 -04:00
Carol (Nichols || Goulding) 854c28c41a feat: Stream messages from Kafka into the database 2021-07-08 09:28:34 -04:00
Carol (Nichols || Goulding) ee500f5bda feat: Support configuring a write buffer for writing OR reading 2021-07-08 09:28:34 -04:00
Carol (Nichols || Goulding) 63d26f6f3f refactor: Rename KafkaBuffer to KafkaBufferProducer 2021-07-08 09:28:34 -04:00
Carol (Nichols || Goulding) e5de73133c feat: Change write buffer connection rule to take either Writing or Reading connection info
A database on one IOx server can, exclusively:

- Not interact with Kafka at all
- Send writes to Kafka
- Read writes from Kafka

Notably, a database on a particular server will never write *and* read from Kafka at the same time.
2021-07-08 09:28:34 -04:00
Carol (Nichols || Goulding) fd4bcc2fa5 refactor: Rename the WriteBuffer trait to be WriteBufferWriting 2021-07-08 09:28:34 -04:00
Carol (Nichols || Goulding) 83e50cfba4 refactor: Rename field to not contain the type 2021-07-08 09:28:34 -04:00
kodiakhq[bot] 69e4786fc7
Merge branch 'main' into crepererum/str_arcs 2021-07-08 13:20:49 +00:00
Marco Neumann 18893e76e0 refactor: convert some table name and part. key String to Arcs
This has the (somewhat nice) side effect that it shrinks the in-mem
catalog a bit as well because nw `ParquetChunk` is a bit smaller making
the chunk stage enum smaller as well.
2021-07-08 14:34:28 +02:00
Edd Robinson 7ff8ae4ce5 refactor: tidy up sort key rep 2021-07-08 12:48:41 +01:00
Edd Robinson f811bf1e5e refactor: log compaction activity 2021-07-08 12:48:41 +01:00
Andrew Lamb 33bc85ad18
feat: Infrastructure for persistence (#1925)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-08 11:14:38 +00:00
Andrew Lamb 7602bde850
chore: Update datafusion deps (#1799)
* chore: Update datafusion deps + rework code

* refactor: remove workaround as it has been contributed upstream

* fix: Update query/src/exec/split.rs

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-08 10:58:32 +00:00
Marco Neumann 24056d7bfc test: ensure that table schemas are recovered from pres. catalog 2021-07-08 10:01:42 +02:00
Marco Neumann a746cd45c5 test: check for schema change errors 2021-07-08 09:51:49 +02:00
Marco Neumann bd22dd38ea docs: fix typos
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
2021-07-08 09:18:09 +02:00
Marco Neumann b528ac2b55 feat: store schemas per table
This way we can:

- check for schema matches even for writes going into different
  partitions
- solve #1768 and #1884 in some future PR

Closes #1897.
2021-07-08 09:18:09 +02:00
Marco Neumann 5ca9760c94 test: make partioning in DB tests consistent w/ DB rules 2021-07-08 09:18:09 +02:00
Marco Neumann ed3ebdcbd2 refactor: use sync locks w/ better metrics 2021-07-08 09:18:09 +02:00
Marco Neumann 5936452895 feat: add infra to check table-wide schemas 2021-07-08 09:18:09 +02:00
Nga Tran 5c722af0fa fix: remove comments 2021-07-07 16:50:53 -04:00
Nga Tran d3c4f8c249 fix: store sort key correctly inthe schema. Update tests to reflect it 2021-07-07 15:55:23 -04:00
Paul Dix cc350bb1ea fix: don't update last write time on failed writes
Fixes #1905
2021-07-07 14:50:03 -04:00
Andrew Lamb e6d995cbd8
chore: Update to Rust 1.53.0 (#1922)
* chore: Update to Rust 1.53.0

* fix: Update to latest clippy standards

* fix: bad refactor

* fix: Update escaping

* test: update test output

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-07 18:02:03 +00:00
Andrew Lamb 957c6245e3
docs: Note that rollover_partition is not automatically called (#1910)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-07 12:14:31 +00:00
Marko Mikulicic 25e3a304ed
chore: Log partition rollover (#1907)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-07 11:48:16 +00:00
Nga Tran 8dfc3bb6bc fix: Thanks Andrew for helping fic the compile problem and avoid using Arc<Mutex> 2021-07-06 18:05:59 -04:00
Nga Tran 76789e5902 feat: store sotkey into the chunk schema of RUB 2021-07-06 17:00:35 -04:00
Marco Neumann b6185982f7 refactor: make `ProviderBuilder` a build-time-checked builder
It's safer and also avoids cloning / copying state around.
2021-07-06 18:20:05 +02:00
Marco Neumann 4f5fe62428
feat: add DB name to lifecycle logs (#1900)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-06 16:14:28 +00:00
Marco Neumann 09b7405b20
docs: spelling fixes
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
2021-07-06 17:46:36 +02:00
Marco Neumann 3d644b63a1 feat: add `Replay` state to DB init 2021-07-06 14:24:39 +02:00
Marco Neumann 4ca2d3e148 chore: move persistence windows related code into own crate
The entire persistence windows data structures (including the
checkpoints) have nothing to do with the mutable buffer per se. So lets
move them into their own crate. This also makes `parquet_file` not
longer depend on `mutable_buffer`.
2021-07-05 10:23:58 +02:00
Marco Neumann cdab1bed05 feat: persist part+db checkpoint in parquets and catalog
This will be required for replay on server startup.
2021-07-05 09:42:46 +02:00
kodiakhq[bot] bcf43a3de5
Merge branch 'main' into crepererum/db_state_in_grpc 2021-07-05 07:21:48 +00:00
Nga Tran 405a6a691b feat: intial implementation of #1886: avoid resort if appropriate 2021-07-02 17:57:48 -04:00
Raphael Taylor-Davies b4534883fe
refactor: remove table name from upsert_table (#1882)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-02 15:22:41 +00:00
Marco Neumann 54fbb60740 feat: expose DB state in gRPC interface 2021-07-02 11:24:36 +02:00
kodiakhq[bot] 404da38d6f
Merge branch 'main' into pd-remove-mb-size-limit-checks 2021-07-01 20:01:32 +00:00
Raphael Taylor-Davies 5b00bc69e6
refactor: use Arc<Db> in lifecycle actions (#1873)
* refactor: use Arc<Db> in lifecycle actions

* chore: review feedback
2021-07-01 19:56:33 +00:00
Paul Dix 61917c107f chore: add test for can_move on row count 2021-07-01 15:49:44 -04:00
Paul Dix 91f5478012 feat: remove MUB size threshold
Removes the MUB chunk close based on size. Also add a check in lifecycle policy to move if the MUB chunk crosses a default row count threshold.
2021-07-01 14:58:29 -04:00
Andrew Lamb 56c8c8d428
feat: Use separate executor for queries and compactions/moves (#1870)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-01 16:47:50 +00:00
Raphael Taylor-Davies f1a100c6ae
refactor: remove now unused chunk sort order (#1854)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-01 16:39:45 +00:00
Andrew Lamb 07826306ed
fix: Always deduplicate data prior to insertion into the ReadBuffer (#1863)
* fix: mark ReadBuffer as always deduplicated

* fix: Use compact plans during merge

* docs: Update server/src/db/chunk.rs

Co-authored-by: Nga Tran <ntran@influxdata.com>

Co-authored-by: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com>
Co-authored-by: Nga Tran <ntran@influxdata.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-01 16:23:37 +00:00
Jacob Marble 0779b0d9bd
feat: add gRPC listener for new write protocol (#1842)
* feat: add gRPC listener for new write protocol

* chore: clippy happy

* chore: lint

* chore: cargo fmt --all

* chore: cargo clippy

* chore: protobuf-lint

* chore: more formatting

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-07-01 16:15:12 +00:00
Marco Neumann e1e3163752 refactor: rework DB init state machine
Since adding new features like "sequencer replay" or init retries would
make the current code too complex, a refactor is required:

Config:
The config struct now holds a `DatabaseState` which is a simple linear
state machine representing the different stages of the database init.

Init:
The init module now has a fixpoint-loop which looks at the state,
decides what to do based on it and repeats until either the DB is
initialized or an error occured. This also makes it easier to continue
the init process "in the middle", e.g. when the preserved catalog is
broken or the sequencer (e.g. Kafka) could not be reached.
2021-07-01 13:47:51 +02:00
Andrew Lamb cfa06e1497
chore: Add query tests for compacted chunks (#1861)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-06-30 20:59:29 +00:00
Raphael Taylor-Davies 99a15cd452
refactor: single lifecycle error enumeration (#1859)
* refactor: single lifecycle error enumeration

* fix: fmt

Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
2021-06-30 18:35:57 +00:00
Andrew Lamb 817a480cde
refactor: move lifecycle implementations out of db.rs and into their own modules (#1858)
* refactor: move lifecycle implementations out of db.rs and into their own modules

* fix: clippy
2021-06-30 17:24:04 +00:00
Andrew Lamb 9e1723620c
refactor: rename load_chunk_to_read_buffer to move_chunk_to_read_buffer (#1857)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2021-06-30 16:53:18 +00:00
Marco Neumann 043890369f refactor: make `MinMaxSequence` safer to use 2021-06-30 16:37:48 +02:00
kodiakhq[bot] 983062f6fa
Merge branch 'main' into crepererum/no_catalog_on_db_creation 2021-06-30 10:04:00 +00:00
Edd Robinson 2e430ac7f0 refactor: remove table name from read_filter schema 2021-06-30 09:50:53 +01:00
Edd Robinson 62f274cc1b refactor: remove table name from column_values 2021-06-30 09:46:54 +01:00
Edd Robinson 5737c9d962 refactor: remove table name from column_names 2021-06-30 09:43:41 +01:00
Marco Neumann c4e054f909 feat: do NOT load preserved catalogs on late DB creation
When a DB is created AFTER the server is initialized, then we can assume
it is a new DB (because the rules file did not exist beforehand). We
shall treat it as a new DB with no data and should not try to load some
leftover / stale / whatever preserved catalog for it. How this catalog
came into existence we do not know and it was certainly not properly
managed by IOx. So we error if there is a catalog.

Furthermore the old implementation was kinda broken since it loaded the
perserved catalog "in-sync" with the gRPC call that issued the DB
creation (we only have a delayed init concept for DBs that are loaded on
instance startup). In production that would very likely provoke nasty
timeouts.

On top of that this new behavior will also be somewhat more sane when we
think about sequencer (e.g. Kafka) replays. We certainly do not wanna do
any replays for newly created DBs.

TLDR: New behavior for DBs created via gRPC is "new empty DB". This does
NOT affect DBs loaded on instance startup (aka existing DBs).
2021-06-30 10:12:38 +02:00
Marco Neumann 58310abfee refactor: de-duplicate code in `server::db::load` 2021-06-30 10:08:25 +02:00