Commit Graph

376 Commits (354d0f1e0e44388d31901450e78b477fb3c1a60f)

Author SHA1 Message Date
kodiakhq[bot] 54e68637dc
Merge branch 'main' into dom/partition-cache 2022-09-28 15:22:40 +00:00
Dom Dwyer 82b7479f97 refactor(write_buffer): seek error at seek time
Moves the "you've tried to seek into the future!" error to the point at
which the seek attempt was made.

This makes more sense than deferring the seek error until read time, and
is easier to determine this is the case rather than at read time (where
the read response error contains an invalid high_watermark value of -1,
making it impossible to conclusively determine what has happened).
2022-09-28 16:44:59 +02:00
Dom Dwyer 5f2f735c7e fix: spurious watermark < read offset panic
In staging we observed an ingester panic due to the write buffer stream
yielding an WriteBufferErrorKind::SequenceNumberAfterWatermark,
suggesting the ingester was attempting to read from an offset that
exceeds the current max write offset in Kafka (high watermark offset).

This turned out not to be the case - the partition had a single write at
offset 2, and the ingester was attempting to seek to offset 1. The first
read would fail (offset 1 does not exist) and the error handling did not
account for the high watermark not being correctly set (-1 in the
response).

I have no idea why rskafka returns this watermark / doesn't retry / etc
but this change will allow the ingesters to recover.
2022-09-28 15:22:34 +02:00
Dom Dwyer 8cf81f457a perf(ingester): amortise Partition cache memory
Remove each cache hit from the partition cache, as each partition should
be looked up at most once.

This amortises the memory usage of the cache, as it should be "drained"
of hot partitions.
2022-09-27 17:16:18 +02:00
Dom Dwyer 1311a8746d refactor(ingester): use Partition cache
Cache the 10,000 most recent partitions at startup, and share them
across all shards.

At commit time, there are approx ~8,000 partitions per day, per
ingester, so this should cache all of the partitions for a given day so
far at startup.
2022-09-27 17:15:59 +02:00
Dom Dwyer 2068ff394b perf(ingester): cache Partition
This commit implements a PartitionCache decorator over the
PartitionProvider abstraction.

When an ingester starts up, the internal data structures are empty and
are lazily initialised for each namespace / table / partition as they
are observed in the stream of DML ops.

This lazy initialisation includes resolving the Partition ID and last
persisted sequence number offset value from the catalog for each
partition in each table in each namespace for which an op is observed -
this occurs in the hot path, while blocking ingest for a shard.
resolving each partition will cause a catalog query, this can cause a
spike in queries against the catalog, also resulting in unnecessarily
slow ingester recovery - we're effectively lazily warming a cache of
PartitionData in the hot path!

Instead this cache can be used to pre-warm the N most recently created
partitions (which are likely to have ongoing writes) at startup to
eliminate the hot-path overhead and associated catalog queries.

NOTE: unlike most of the other hot-path queries, partition persist
offset resolution cannot be eliminated by changes to the Kafka wire
format.
2022-09-27 17:15:57 +02:00
Dom Dwyer a3d6e7a45a refactor(ingester): server-wide PartitionProvider
Lifts the PartitionProvider initialisation higher in the stack to a
point where a single instance can be used across all shards an ingester
manages.

This is a pre-requisite for sharing a cache of Partitions across all
shards.
2022-09-27 17:15:31 +02:00
Dom Dwyer 38ebd5fb20 test: simplify partition provider mock
Removes redundant fields from the MockPartitionProvider.
2022-09-27 17:11:13 +02:00
Dom 2ef04e99da
Merge branch 'main' into dom/non-pub-shard 2022-09-27 14:21:23 +01:00
Andrew Lamb 66dbb9541f
chore: Update datafusion and `arrow`/`parquet`/`arrow-flight` to 23.0.0, `thrift` to 0.16.0 (#5694)
* chore: Update datafusion and `arrow`/`parquet`/`arrow-flight`  to 23.0.0

* chore: Update thrift / remove parquet_format

* fix: Update APIs

* chore: Update lock + Run cargo hakari tasks

* fix: use patched version of arrow-rs to work around https://github.com/apache/arrow-rs/issues/2779

* chore: Run cargo hakari tasks

Co-authored-by: CircleCI[bot] <circleci@influxdata.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2022-09-27 12:50:54 +00:00
Dom Dwyer b873297fad refactor(ingester): limit visibility
Marks many internal data structures as non-pub.

Many remain as they're used across tests / from multiple callers
"peeking", but this limits the scope of false sharing in the future.
2022-09-27 14:27:32 +02:00
Dom Dwyer 11be746dc0 refactor: internalise ShardData init
Move the initialisation of ShardData (an internal ingester data
structure) into the ingester itself.

Previously callers would initialise the ingester state, and pass it into
the IngesterData constructor.
2022-09-27 14:26:17 +02:00
Dom Dwyer 61aecc3044 refactor: decouple partition init from table
Removes the "how" of initialising a per-partition buffer structure
(PartitionData) from the per-table buffer (TableData).

This is a cleaner separation of concerns - a table buffer is responsible
for addressing and initialising per-table partitions as necessary, and
buffering of ops for them. It does not have to be concerned with the
series of steps necessary to look up the various bits of data in order
to construct a PartitionData.

This abstract provider can be layered up to provide more complex
behaviours - I intend to add a read-through cache impl that decorates
the catalog impl in this commit, which should eliminate most partition
queries at ingester startup utilising the indirection added here.
2022-09-26 14:35:15 +02:00
Carol (Nichols || Goulding) c8108f01e7
chore: Upgrade to Rust 1.64 (#5727)
* chore: Upgrade to Rust 1.64

* fix: Use iter find instead of a for loop, thanks clippy

* fix: Remove some needless borrows, thanks clippy

* fix: Use then_some rather than then with a closure, thanks clippy

* fix: Use iter retain rather than filter collect, thanks clippy

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2022-09-22 18:04:00 +00:00
Marco Neumann 55ef272920
refactor: acquire table locks concurrently (#5722)
Waiting for one after the other (one per shard) in serial fashion
likely increases latency too much.
2022-09-22 10:56:22 +00:00
Marco Neumann 365a246f8d
refactor: do not run de-dup in ingester for querier requests (#5626)
* refactor: do not run de-dup in ingester for querier requests

This removes the entire de-dup logic from the inegster for querier
requests. Furthermore, it even removes the entire datafusion execution
from the querier and just dumps the in-memory record batches as quickly
as possible. No filters are applied. Note that even prior to this PR,
we've never applied projections (tracked by #5624).

**Pros:**

- speed up query planning within the querier (since we need the ingester
  response for state reconciling)
- lowered ingester CPU load

**Cons:**

- more querier<>ingester network traffic

Closes #5602.

* test: extend query test case

* fix: ingester tests

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2022-09-22 07:33:54 +00:00
Marco Neumann c66f16e4af
fix: ingester retries (#5708)
* fix: retry ingester requests faster

The retries introduced in #5695 are too slow and block the entire
querier for minutes (until the very long gRPC timeout kicks in).

* fix: add error details on why the query planning failed

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2022-09-21 09:27:47 +00:00
Dom Dwyer c6fe0dab3e refactor(ingester): reduced internal visibility
Changes many pub fields / methods to be pub(super), or if necessary,
pub(crate).

This helps maintain an internal API boundary for code hygiene, and helps
identify functions that are unused / only used in tests (which I've
annotated with cfg(test) and intend to remove - we should be driving
code under test via the public API rather than using test-only state
mutation, otherwise we're just testing our tests!)
2022-09-20 16:24:27 +01:00
Dom Dwyer 6d00d6b683 test(ingester): refactor querier API tests
This commit changes the prepare_data_to_querier() tests to drive the
ingester state by applying DML ops, therefore driving the prod code
paths (and testing them!) rather than having the tests set up what the
tests believe is the correct internal ingester state, and then asserting
on that state.

This gives us much better coverage of prod code paths, decouples the
tests from the internal state/representation of ingesters (making the
tests less fragile), and removes a bunch of special-cased, test-only
functions that are functionally similar, but not the same as, the prod
functions.

Unblocks #5658, further clean-up to come.
2022-09-20 16:24:27 +01:00
dependabot[bot] 4fbb32eed6
chore(deps): Bump tokio-stream from 0.1.9 to 0.1.10 (#5667)
Bumps [tokio-stream](https://github.com/tokio-rs/tokio) from 0.1.9 to 0.1.10.
- [Release notes](https://github.com/tokio-rs/tokio/releases)
- [Commits](https://github.com/tokio-rs/tokio/compare/tokio-stream-0.1.9...tokio-stream-0.1.10)

---
updated-dependencies:
- dependency-name: tokio-stream
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
2022-09-19 07:36:11 +00:00
Dom Dwyer b0eb85ddd5 refactor: store ShardId in child nodes
Instead of passing the ShardId into each function for child nodes of the
Shard, store it. This avoids the possibility of mistakenly passing the
wrong value.
2022-09-16 18:00:11 +02:00
Dom Dwyer 07b08fa9cb refactor: add table name in PartitionData
A partition belongs to a table - this commit stores the table name in
the PartitionData (which was readily available at construction time)
instead of redundantly passing it into various functions at the risk of
getting it wrong.
2022-09-16 17:59:22 +02:00
Dom Dwyer c7ba0bea91 refactor: ShardId & TableId in PartitionData
When we construct a PartitionData we have the ShardId and TableId. This
commit stores them in the PartitionData for later use, rather than
repeatedly passing them in again when constructing snapshots, at the
risk of passing the wrong IDs.
2022-09-16 17:17:16 +02:00
Dom Dwyer 85d6efafe1 refactor: snapshot_to_persisting redundant ID
Partition::snapshot_to_persisting() passes the ID of the partition it is
calling `snapshot_to_persisting()` on. The partition already knows what
its ID is, so at best it's redundant, and at worst, inconsistent with
the actual ID.
2022-09-16 17:08:08 +02:00
Dom Dwyer ce0d189260 perf: O(1) partition persist mark discovery
Changes the ingest code path to eliminate scanning the parquet_files
table to discover the last persisted offset per partition, instead
utilising the new persisted_sequence_number field on the Partition
itself to read the same value.

This lookup blocks ingest for the shard, so removing the expensive query
from the ingest hot path should improve catch-up time after a
restart/deployment.
2022-09-16 14:06:42 +02:00
Dom Dwyer 66bf0ff272 refactor(db): NULLable persisted_sequence_number
Makes the partition.persisted_sequence_number column in the catalog DB
NULLable. 0 is a valid persisted sequence number.
2022-09-15 18:19:39 +02:00
Dom Dwyer 234d460fcb chore: rename update_persisted_sequence_number fn 2022-09-15 16:10:35 +02:00
Dom Dwyer f91d802107 feat: store per-partition persist markers
Changes the ingester to record the per-partition, maximum persisted
sequencer offsets to the catalog. This will enable quick O(1) lookup in
the future, but the currently persisted value is only used to assert the
per-partition monotonic persist ordering invariant.
2022-09-15 16:10:35 +02:00
Dom Dwyer 300938f858 refactor: assert partition persistence ordering
Assert the per-shard / per-partition persistence watermarks
monotonically increase, and document the invariant.

NOTE: this is not a new invariant, just a new assertion to validate it.
2022-09-15 16:10:35 +02:00
Dom Dwyer d199a83355 feat(catalog): per-partition persist mark API
Adds the "persisted_sequence_number" field to the Partition model, and
updates the catalog API to read & update it.
2022-09-15 16:10:35 +02:00
Dom Dwyer fc17f2ec2d refactor: hoist persistence watermark from buffer
The maximum persisted sequence number is tracked to answer "up to where
has this partition been persisted", used for querying and skipping
writes that have already been applied (though I suspect this is
redundant).

This is a property of the partition, not the actual data buffer, so this
commit hoists it up out of the data buffer and onto the per-partition
data structure, internalising the field in the process (not pub).
2022-09-14 18:07:45 +02:00
Dom Dwyer ee8cdb48af style(ingester): fmt imports & long strings
Rewrite the imports to be a consistent order; std, external, crate and
merge all crate-level imports into one use statement.
2022-09-14 14:20:19 +02:00
Dom Dwyer 074722eb3e refactor(ingester): split data.rs into modules
Breaks the gigantic data.rs file into sub-modules for Shard, Namespace,
Table, Partition, and finally the actual data buffer used to store
writes.
2022-09-14 14:20:19 +02:00
Andrew Lamb f86d3e31da
chore: Update datafusion + object_store (#5619)
* chore: Update datafusion pin

* chore: update object_store to 0.5.0

* chore: Run cargo hakari tasks

Co-authored-by: CircleCI[bot] <circleci@influxdata.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2022-09-13 12:34:54 +00:00
Marco Neumann e4a66f69c7
refactor: dedicated query path for querier tombstone materialization (#5618)
I would like to remove de-dup from the querier<>ingester query path in #5602,
but the tombstone application and parquet write path can still do the full work.
2022-09-13 07:16:38 +00:00
Andrew Lamb 1fd31ee3bf
chore: Update datafusion / `arrow` / `arrow-flight` / `parquet` to version 22.0.0 (#5591)
* chore: Update datafusion / `arrow` / `arrow-flight` / `parquet` to version 22.0.0

* fix: enable dynamic comparison flag

* chore: derive Eq for clippy

* chore: update explain plans

* chore: Update sizes for ReadBuffer encoding

* chore: update more tests

Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2022-09-12 17:45:03 +00:00
Dom Dwyer 6342be2fa9 refactor: consistent logging
Changes the ingester to log ALL reasons for a partition being marked for
persistence, rather than just one of the 4 previously. Fields are
consistent, but verbose / repetitive.

Also cleans up some misleading messages like "updating ..." to be logged
only when the update actually takes place.
2022-09-12 15:57:27 +02:00
Marco Neumann 8933f47ec1
refactor: make `QueryChunk::partition_id` non-optional (#5614)
In our data model, a chunk always belongs to a partition[^1], so let's
not make this attribute optional. The optional value only leads to
-- mostly surprising -- conditional behavior, ranging from "do not equalize
the partition sort key" (querier) to "always consider the chunk overlapping"
(iox_query when dealing with ingester chunks).

[^1]: This is even true when the chunk belongs to a parquet file that is not
      yet added to the catalog, contrary to what a comment in the ingester
      stated. The catalog and data model used by the querier are two totally
      different things.
2022-09-12 13:52:51 +00:00
Marco Neumann caa0dfd1e0
refactor: query code clean ups (#5612)
* refactor: remove dead code

* refactor: `Deduplicator::build_scan_plan` consumes `self`

There is no good reason to use the same `Deduplicator` twice. In
contrast I'm quite sure that this would lead to nasty bugs, because
`split_overlapped_chunks` exists early in some cases so the 2nd plan
would have old and new chunks mixed together.
2022-09-12 13:00:56 +00:00
dependabot[bot] 786ce75e26
chore(deps): Bump tokio-util from 0.7.3 to 0.7.4 (#5596)
Bumps [tokio-util](https://github.com/tokio-rs/tokio) from 0.7.3 to 0.7.4.
- [Release notes](https://github.com/tokio-rs/tokio/releases)
- [Commits](https://github.com/tokio-rs/tokio/compare/tokio-util-0.7.3...tokio-util-0.7.4)

---
updated-dependencies:
- dependency-name: tokio-util
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
2022-09-09 07:40:16 +00:00
YIXIAO SHI 52ae60bf2e
chore: fix comment typo (#5551)
Co-authored-by: Dom <dom@itsallbroken.com>
2022-09-07 08:49:29 +00:00
Marco Neumann adeacf416c
ci: fix (#5569)
* ci: use same feature set in `build_dev` and `build_release`

* ci: also enable unstable tokio for `build_dev`

* chore: update tokio to 1.21 (to fix console-subscriber 0.1.8

* fix: "must use"
2022-09-06 14:13:28 +00:00
Marco Neumann 87772a6aec
refactor: debug log improvements (#5553)
* feat: extend log output for ingester responses

* feat: add debug log for parquet `read_filter` calls

* feat: add debug log to `get_write_info`

* feat: add debug log parquet cache invalidation
2022-09-05 13:54:13 +00:00
dependabot[bot] 7c61bdcf35
chore(deps): Bump paste from 1.0.8 to 1.0.9 (#5526)
Bumps [paste](https://github.com/dtolnay/paste) from 1.0.8 to 1.0.9.
- [Release notes](https://github.com/dtolnay/paste/releases)
- [Commits](https://github.com/dtolnay/paste/compare/1.0.8...1.0.9)

---
updated-dependencies:
- dependency-name: paste
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
2022-09-01 12:07:53 +00:00
Dom ed2490deb2
Merge branch 'main' into dom/ingester-row-limit 2022-08-31 14:56:42 +01:00
Dom Dwyer 2a19606456 feat(ingester): restrict partition row count
This limit restricts a single partition to containing at most N rows
before it is marked for persistence (note: being marked for persistence
does not currently prevent further ingest for that partition.)
2022-08-31 15:48:18 +02:00
Andrew Lamb 6669d85fb4
chore: Update datafusion + arrow/parquet to `21.0.0` (#5519)
* chore: Update arrow/arrow-flight/parquet to 21.0.0

* chore: Update datafusion pin

* chore: Fix arrow update script

* chore: Update Cargo.lock

* chore: Update for new API
2022-08-31 13:30:47 +00:00
Nga Tran cb10a7c6d8
feat: More accurate memory estimate for compaction (#5471)
* feat: initial implementation of memory estimation for a compaction

* feat: estimate size of files and have the right actions for the needed budget

* feat: run candidates in parallel

* fix: have the right name for the column field of the output struct

* feat: add metrics for estimated budgets

* chore: cleanup

* chore: Apply suggestions from code review

Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>

* fix: fix syntax after applying review's suggestions

* refactor: Convert a Vec to VecDeque to go well with pop and push

* chore: remove max_concurrent_size_bytes and input_size_threshold_bytes

* chore: remove input_file_count_threshold

* test: tests for estimate_arrow_bytes_for_file

Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
2022-08-30 13:44:44 +00:00
Carol (Nichols || Goulding) dbd27f648f
refactor: Rename more mentions of Kafka to their other name where appropriate 2022-08-29 14:27:02 -04:00
Carol (Nichols || Goulding) 58f0b63cdc
refactor: Rename KafkaTopic to Topic or TopicMetadata or topic name as appropriate 2022-08-29 14:27:02 -04:00