Removes the need to leak the PartitionProvider outside of the ingester
crate.
This will allow the PartitionProvider to utilise a
DeferredLoad<TableName> without having to make the DeferredLoad and
TableName pub.
Removes reliance on string name identifiers for namespaces in the
ingester buffer tree, reducing the memory usage of the namespace index
and associated overhead.
The namespace name is required (though unused by IOx) in the IoxMetadata
embedded within a parquet file, and therefore the name is necessary at
persist time. For this reason, a DeferredLoad is used to query the
catalog (by ID) for the name, at some uniformly random duration of time
after initialisation of the NamespaceData, up to a maximum of 1 minute
later. This ensures the query remains off the hot ingest path, and the
jitter prevents spikes in catalog load during replay/ingester startup.
As an additional / easy optimisation, the persist code causes a
pre-fetch of the name in the background while compacting, hiding the
query latency should it not have already been resolved.
In order to keep the the ingester buffer & catalog decoupled / easily
testable, this commit uses a provider/factory trait
NamespaceNameProvider and corresponding implementation
(NamespaceNameResolver) in a similar fashion to the PartitionResolver,
allowing easy mocking for tests, and composition for prod code, allowing
future optimisations such as pre-fetching / caching the "hot" namespace
names at startup.
Internal string identifier removal is a pre-requisite for removing
string identifiers from the write wire format (#4880).
Changes the ingester's buffer tree to use the deferred loading primitive
to resolve the namespace name for NamespaceData.
Note that the loader is initialised with the name in the first place -
this commit just introduces the use of the deferred loading primitive,
and doesn't change where the name is sourced from.
This lets deferred loads be used in place of a non-differed T, such as
log context fields.
If the value has not been resolved, the display impl returns
"<unresolved>".
Allow a caller to signal to the DeferredLoad that the value it may or
may not have to materialise will be used imminently, optimistically
hiding the latency of resolving the value (typically a catalog query).
* refactor: generic deferred loader helper
Splits the DeferredSortKey loader introduced in #5807 into two parts - a
generic helper type that implements deferred/background loading of
values, and SortKey specific logic for use with it.
As this will be more widley used, this implementation features improved
behaviour of the deferred loader under concurrent demand requests
(multiple calls to get() do not attempt to concurrently resolve the
value), as well as complete cancellation safety (cancelling the get()
doesn't affect the liveness of the background task).
* docs: doc-link & minor comment amendments
Fixes naming, adds missing doc-links, and expands some code comments.
* test: bound wait times to avoid hangs
Adds timeouts to all .await of the code under test, ensuring tests don't
hang if something goes wrong.
I tracked down the source of the size difference to the difference in
`mem::size_of::<mutable_batch::column::ColumnData>`. I believe this enum
is now able to take advantage of this niche-filling optimization:
<https://github.com/rust-lang/rust/pull/94075/>
* feat: flag partition for delete
* fix: compare the right date and time
* chore: Run cargo hakari tasks
* chore: cleanup
* fix: typos
* chore: rust style tidy ups in catalog
Co-authored-by: CircleCI[bot] <circleci@influxdata.com>
Co-authored-by: Luke Bond <luke.n.bond@gmail.com>
* refactor: NS+table ID (instead of name) in querier<>ingester
* feat(ingester): use IDs for query API
Changes the ingester to utilise the ID fields (instead of names) sent
over the query wire message wrapped within the Flight API.
BREAKING: this changes the "query-ingester" CLI command arguments which
now expects the namespace & table IDs, rather than their names.
* refactor(ingester): add more query logging context
Updates the log messages during query execution to include more context
fields.
* style: remove unused import
Co-authored-by: Marco Neumann <marco@crepererum.net>
Now DML operations contain the table ID, the ingester has all necessary
data to initialise the TableData buffer node without having to query the
catalog.
This also removes the catalog from the buffer_operation() call path,
simplifying testing.
Now DML operations contain the namespace ID, the ingester has all
necessary data to initialise the NamespaceData buffer node without
having to query the catalog.
Expose the Table and Namespace IDs encoded within the serialised DML
write (added in #6036).
This makes the IDs available for use in the consumers, ending the
transition period. This commit DOES NOT remove the strings sent over the
wire.
This commit pushes the existing table-level mutex down to the partition.
This allows the ingester to gather data from multiple partitions within
a single table in parallel, and reduces contention between ingest/query
workloads.
This moves the logic that skips operations that do not need to be
applied to a partition during shard replay from the table level, to the
partition level.
Changes the bounds on the ArcMap to accept an owned key, avoiding an
extra allocation.
Cleans up the bounds on other fn to ensure the borrowed key impl Eq and
is the ref type of K.
This commit changes the ArcMap HashBuilder to use the same instance as
the underlying HashMap hasher.
This prevents divergent hashing across threads that MAY initialise a
hasher with a different seed.
This commit pushes the existing table-level mutex down to the partition.
This allows the ingester to gather data from multiple partitions within
a single table in parallel, and reduces contention between ingest/query
workloads.
This moves the logic that skips operations that do not need to be
applied to a partition during shard replay from the table level, to the
partition level.
It should be always clear from the context to which table a chunk
belongs.
I think having a table name bound to a chunk goes back to a time where
chunks had multiple tables.
Helps with #6049.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* chore: delete metric duplicate character
* fix: failure ci test case
* fix: failure ci test case
* fix: failure ci test case
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Changes the DmlDelete to contain the NamespaceId for which it should be
applied, propagating this value over the wire.
Like the existing IDs within the DmlWrite, these values are marked
unsafe to use due to avoid the consumers utilising them accidentally
during deployment. Unlike DmlWrite, the DmlDelete is completely unused,
so this is less of an issue.
Implements a map of K -> Arc<V> with exactly-once initialisation
semantics.
This map can be used to ensure a given key maps to singleton instances
of V; exactly what all the nodes in the ingester "buffer tree" of shard
-> namespace -> table -> partition require.
This impl contains unused funcs (silenced with an allow(dead_code)) due
to it being picked from a future branch.
This commit is part of a two-part change in order to add the table &
namespace IDs to the write buffer wire format. This commit forms the
first half; changing the producer to send the IDs.
In this commit the new ID values are never read on the consumer side,
ensuring there is no consumer dependency on them. This ensures they
remain operational during a rollout, where the consumer may be updated
to the latest code dependent on the IDs before the producer is updated
to send them. This also ensures we have a window of time where where the
consumers can be rolled back after being updated, and still handle
replaying messages in Kafka.
* refactor: simplify `QueryChunk` data access
We have only two types for chunks (now that the RUB is gone):
1. In-memory RecordBatches
2. Parquet files
Loads of logic is duplicated in the different `read_filter`
implementations. Also `read_filter` hides a solid amount of logic from
DataFusion, which will prevent certain (future) optimizations. To enable #5897
and to simplify the interface, let the chunks return the data (batches
or metadata for parquet files) directly and let `iox_query` perform the
actual heavy-lifting.
* docs: improve
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
* docs: improve
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>
With #5963 merged, all chunks now provide a summary (even though it may
not contain data for all columns). So let's make it mandatory, which
also removes a few 🙈-style `.except(...)` calls.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Use the table summary instead. This allows us to have a single mechanism
that both IOx and DataFusion understand. This basically lifts the "basic
table summary" mechanism that the querier uses to `iox_query` and let
the compactor and ingester use the same mechanism.
While not strictly necessary, simplifying the `QueryChunk[Meta]`
interface helps with #5897.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Changes the DmlWrite type to require a PartitionKey be specified,
instead of accepting an Option.
This requirement was already in place - the write buffer upheld an
invariant that all writes contained a partition key value (was not
"None") or it panicked at runtime when attempting to enqueue the write.
It is now possible to encode this invariant in the type system, which is
what this change does.
This commit makes use of the partition buffer state machine introduced
in https://github.com/influxdata/influxdb_iox/pull/5943.
This commit significantly changes the buffering, and querying, of data
from a partition, swapping out the existing "DataBuffer" for the new
state machine implementation (itself simplified due to temporary lack of
incremental snapshot generation, see #5944).
This commit simplifies the query path, removing multiple types that
wrapped one-another to pass around various state necessary to perform a
query, with various query functions needing different types or
combinations of types. The query path now operates using a single type
(named "QueryAdaptor") that provides a queryable interface over the set
of RecordBatch returned from a partition.
There is significantly increased testing of the PartitionData itself,
covering data in various states and the ordering of returned RecordBatch
(to ensure correct materialisation of updates). There are also
invariants upheld by the type system / compiler to minimise the
complexities of working with empty batches & states, and many asserts
that ensure (mostly existing!) invariants are upheld.
This reverts commit c63312ce12.
This change fixed a low-priority alert when there was no traffic flowing
through the system. The loss in TTBR value fidelity due to bucketing is
a greater concern as it affects live, high-volume clusters and hinders
operational insight.
This commit removes the on-demand, incremental snapshot generation
driven by queries.
This functionality is "on hold" due to concerns documented in:
https://github.com/influxdata/influxdb_iox/issues/5805
Incremental snapshots will be introduced alongside incremental
compactions of those same snapshots.
This commit introduces code that is intended to replace the current
implicit state machine used by PartitionData. The existing code is still
in use, the new code is NOT used in this commit. A follow-up commit will
switch over to minimise the diff.
This change has two main goals;
* encapsulation & simplification for callers
* robust implementation so developing correct additions is easier
This is a significant refactor of the partition buffering logic to
encapsulate the various states of data (buffering, snapshot, persisting
and the mixed states between them) within the Partition. This alleviates
the rest of the system from having to be concerned with the differences
between "buffering" data, and "unpersisted data", "snapshot data",
"persisting data", "persisting with snapshots" etc - callers now invoke
a method called get_query_data() and they are provided with all the
relevant data for a partition. This abstraction change alone
significantly reduces code and test complexity in the rest of the
ingester.
For the second goal, the new implementation leverages an explicit state
machine, encoded using typestates. Typestate ensures compile-time
correctness of transitions and method calls, and the explicit FSM itself
helps ensure the system progresses in the desired manner - this fixes
and helps prevent bugs caused by implicit states such as:
https://github.com/influxdata/influxdb_iox/issues/5805
This state machine makes the system states explicit and
self-descriptive, helping to reduce the cost of developer on-boarding
(no prior knowledge of "how this bit works") and reduces ongoing
developer burden. This explicit nature also de-risks adding new
functionality - it should be relatively easy to add concurrent snapshot
generation or incremental compaction without introducing bugs. The state
transition logic is abstracted away from callers, minimising the
overhead of this strategy.
Asserts write buffer seeking behaviour, including:
* Seeking past already persisted data correctly
* Skipping to next available op in non-contiguous offset stream
* Skipping to next available op for dropped ops due to retention
* Panics when seeking beyond available data (into the future)
Removes a pair of tests that covered some of the above due to their
tight coupling with ingester internals.
This commit adds a new test that exercises all major external APIs of
the ingester:
* Writing data via the write buffer
* Waiting for data to be readable via the progress API
* Querying data and and asserting the contents
This should provide basic integration coverage for the Ingester
internals. This commit also removes a similar test (though with less
coverage) that was tightly coupled to the existing buffering structures.
Adds a test helper type that maintains the in-memory state for a single
ingester integration test, and provides easy-to-use methods to
manipulate and inspect the ingester instance.
An existing function to map the complex IngesterQueryResponse type to a
simple set of RecordBatch existed in test code - this has been lifted
onto an inherent method on the response type itself for reuse.
* fix: only emit ttbr metric for applied ops
* fix: move DmlApplyAction to s/w accessible
* chore: test for skipped ingest; comments and log improvements
* fix: fixed ingester test re skipping write
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* fix: Avoid some allocations by collecting instead of inserting into a vec
* refactor: Encode that adding columns is for one table at a time
* test: Add another test of column limits
* test: Add below/above limit tests for create_or_get_many
* fix: Explicitly DO NOT check column limits when inserting many columns
* feat: Cache the max_columns_per_table on the NamespaceSchema
* feat: Add a function to validate column limits in-memory
* fix: Provide more useful information when over column limits
* fix: Swap types to remove intermediate allocation
* docs: Explain the interactions of the cache and the column limits
* test: Actually set up test that showcases column limit race condition
* fix: Allow writing to existing columns even if table is over column limit
Co-authored-by: Dom <dom@itsallbroken.com>
Copies the existing monotonic partition persistence check into the
partition too - this ensures that even if the partitions are persisted
in order, they are never marked as persisted OUT of order.
Removes the catalog queries previously used to look up various
information about the partition/table/namespace that was already in
memory.
As part of this change, the compaction helper function is changed to
accept the inputs it needs, rather than a struct of data from the
catalog - this significantly simplifies testing.
This commit also adds additional context to all log messages in the
persist() fn.
This commit changes the table ID lookup query from an expensive,
JOIN multi-query to a simple, single table, indexed lookup.
As this is on the hot path, this should help with the recovery rate of
the ingesters.
This commit removes tombstone support from the ingester, and deletes
associated code/helpers/tests. This commit does NOT remove tombstone
support from any other service, but MAY include removing overlapping
test coverage.
This also removes the tombstone support from the Ingester -> Querier RPC
response message.
This has the nice side effect of removing a whole lot of thread spawning
in the ingester tests for the Executor, speeding everything up!
Changes the get() code path to abort the background load task when the
caller will resolve the sort key.
Note that an aborted future will leave the DeferredSortKey without a
background task to fetch the key, and the next caller will have to query
the catalog. Given the rarity of aborted futures, and desire to minimise
catalog load, this seems like a decent trade-off.
This commit also documents the many-readers eager loading problem.
This commit carries the SortKey in the PartitionData, and configures the
ingester to use deferred sort key lookups, smearing the lookups across a
fixed period of time after initialising the PartitionData, instead of
querying for the sort key at persist time.
This allows large numbers of PartitionData to be initialised without
causing a equally large spike in catalog load to resolve the sort key -
instead this load is spread out randomly to reduce peak query rps.
Adds a new DeferredSortKey type that fetches a partition's sort key from
the catalog in the background, or on-demand if not yet pre-fetched.
From the caller's perspective, little has changed compared to reading it
from the catalog directly - the sort key is always returned when calling
get(), regardless of the mechanism, and retries are handled
transparently. Internally the sort key MAY have been pre-fetched in the
background between the DeferredSortKey being initialised, and the call
to get().
The background task waits a (uniformly) random duration of time before
issuing the catalog query to pre-fetch the sort key. This allows large
numbers of DeferredSortKey to (randomly) smear the lookup queries over a
large duration of time. This allows a large number of DeferredSortKey to
be initialised in a short period of time, without creating an equally
large spike in queries against the catalog in the same time period.
- treat OOM protection as "resource exhausted"
- use `DataFusionError` in more places instead of opaque `Box<dyn Error>`
- improve conversion from/into `DataFusionError` to preserve more
semantics
Overall, this improves our error handling. DF can now return errors like
"resource exhausted" and gRPC should now automatically generate a
sensible status code for it.
Fixes#5799.
Changes the ingester's NamespaceData to carry a ref-counted string
identifier as well as the ID.
The backing storage for the name in NamespaceData is shared with the
index map in ShardData, so it is effectively free!
This commit changes the persist() call so that it passes through all
relevant IDs so that the impl can locate the partition in the buffer
tree - this will enable elimination of many queries against the catalog
in the future.
This commit also cleans up the persist() impl, deferring queries until
the result will be used to avoid unnecessary load, improves logging &
error handling, and documents a TOCTOU bug in code:
https://github.com/influxdata/influxdb_iox/issues/5777
Changes the TableData to hold a map of partition key -> PartitionData,
and partition ID -> PartitionData simultaneously. This allows for cheap
lookups when the caller holds an ID.
This commit also manages to internalise the partition map within the
TableData - one less pub / peeking!
This commit also switches from a BTreeMap to a HashMap as the backing
collection, as maintaining key ordering doesn't appear to be necessary.
Changes the ShardData to hold a map of namespace name -> NamespaceData,
and namespace ID -> NamespaceData simultaneously.
This allows for cheap lookups when the caller holds an ID, and is part
of preparatory work to transition away from using string names in the
ingester for tables.
This commit also switches from a BTreeMap to a HashMap as the backing
collection, as maintaining key ordering doesn't appear to be necessary.
Changes the NamespaceData to hold a map of table name -> TableData, and
table ID -> TableData simultaneously.
This allows for cheap lookups when the caller holds an ID, and is part
of preparatory work to transition away from using string names in the
ingester for tables.
This commit also switches from a BTreeMap to a HashMap as the backing
collection, as maintaining key ordering doesn't appear to be necessary.
Fixes a case where the ingester may incorrectly record a write as having
been buffered in memory, when in fact the buffering failed.
This could cause the effective buffer size to be reduced over time as
more and more data is spuriously "added" to the buffer, but never
released back to the memory tracker as it is never persisted.
Changes the NoopLifecycleHandle to MockLifecycleCall, and adds code
causing it to log all calls made to the log_write() method.
This will allow tests to assert calls and their values in DML buffering
tests.
Changes the lifecycle handle to also track the namespace + table ID in
addition to the existing shard ID.
Adds asserts to ensure the values never vary for a given partition.
This commit changes the PartitionData buffer structure to carry the IDs
of all its parents - the table, namespace, and shard. Previously only
the table & shard were carried.
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).
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.
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.
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.
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.
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.
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.
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.
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.
* 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>
* 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>
* 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>
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!)
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.
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.
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.
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.
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.
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.
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.
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.
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).
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.
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.
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.
* 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.
* 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"
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.)
* 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>
* fix: do not loose data when Kafka reports that offset is above watermark
This can happen in certain cluster rebalance settings.
This is also linked to https://github.com/influxdata/rskafka/issues/147
but for the upstream issue I currently have no idea how to fix it, so
let's at least harden IOx against it.
Fixes#5128.
* refactor: panic for `SequenceNumberAfterWatermark`
* chore: return a struct with named and documented fields from `compact_persisting_batch`
* docs: Remove extra 'the' and fix a typo
Co-authored-by: Carol (Nichols || Goulding) <carol.nichols@gmail.com>
* feat: `QueryChunk::as_any`
* feat: allo `ChunkPruner::prune_chunks` to fail
* feat: limit per-table chunk data for every query
Closes#5211.
* fix: address review comments
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* refactor: remove min_sequnce_number
* fix: typos
* fix: remove min_sequencer_number from new files from merging main
* fix: add back throwing error if the compactor compacts files persisted by the ingester after the ingester sends max seq_num back to querier
* test: add test_compactor_collision back but modify the input to make it work woth new changes
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
- remove `IOxSessionContext::default()` because untracked contexts
should only be created by tests
- remove `Option<IOxSessionContext>` because it is a typed workaround
for `IOxSessionContext::default`
Tests should use `IOxSessionContext::testing` and all _normal_ users
should create proper contexts.
I suspect this will help tracing or at least prevent silent regressions.
See #5129.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* refactor: remove parquet chunk ID to `ChunkMeta`
* refactor: return `Arc` from `QueryChunk::summary`
This is similar to how we handle other chunk data like schemas. This
allows a chunk to change/refine its "believe" over its own payload while
it is passed around in the query stack.
Helps w/ #5032.
* test: add test for timestamps in kafka write buffer
* refactor: move timestamp batching test to generic tests
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Instead of using some hand-rolled timestamp-based logic (or just
"unknown") all over the place, just use logic introduced in #5017.
This requires slightly improved table summaries within the querier that
at least has min/max for the timestamp column. For that, the former
`IngesterChunk`-specific `calculate_summary` method was extended to
`create_basic_summary` to include that data and is now also used by
`QuerierParquetChunk`.
Note: `QuerierRBChunk` already has detailled metrics that are provided
by the read buffer implementation.
Should we ever need even better pruning for `QuerierParquetChunk` (or
`IngesterChunk`) then we _only_ need add extra data to the table
summaries.
Closes#4976.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Went through and remove all lazy_static uses with once_cell (while waiting for the project to compile). There are still dependencies using lazy_static so it is still in the crate graph but at least there isn't an explicit dependency on it (and it is easier to update to `std::lazy::Lazy` once that is stable).
* refactor: change level 1 to level 2 preparing for next design changes
* fix: make level-2 consistent everywhere
* chore: remove unused comments
* refactor: change all the name level_1 to level_2 to completely replace 1 with 2 to amke everything consistent
* chore: add correspinding constants for the comapction levels in the comments
Co-authored-by: Dom <dom@itsallbroken.com>
* fix: avoid using min_time, which can be negative, for ChunkId. Using object store id which is uuid instead
* chore: Apply suggestions from code review
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* chore: run fmt
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
The ingester maintains a rough "total memory in use" counter it uses to
try and limit the amount of memory the ingester is using overall.
When a partition is persisted, this total memory usage value is adjusted
to account for releasing the partition memory. Prior to this commit, the
ordering was:
* Writes increase the memory counter
* maybe_persist() is called to trigger persistence
* A partition is identified for persistence
* Partition memory usage is released back to the total memory counter
* Persistence starts
This meant that the partitions in the process of being persisted were
not accounted for in the ingester's total memory counter, and therefore
we could significantly overrun the configured memory limit.
After this commit, the ordering is:
* Writes increase the memory counter
* maybe_persist() is called to trigger persistence
* A partition is identified for persistence
* Persistence starts
* Persistence completes
* Partition memory usage is released back to the total memory counter
This ensures persisting partitions are sill tracked in the total memory
counter, causing pauses to correctly fire.
Changes the ingester to use the partition key derived in the router, and
transmitted over through the kafka API boundary.
This should have no observable behavioural change, but be more resilient
as we're no longer assuming the partitioning algorithm produces the same
value in both the router (where data is partitioned) and the ingester
(where data is persisted, segregated by partition key).
This is a pre-requisite to allowing the user to specify partitioning
schemes.
* refactor: store per-file column set in catalog
Together with the table-wide schema and the partition-wide sort key, this should
be everything we need to read a parquet file directly into memory
without peeking any file-level metadata.
The querier will use this to directly load parquet files into the read
buffer.
**WARNING: This requires a catalog wipe!**
Ref #4124.
* refactor: use proper `ColumnSet` type
* refactor: stream partitions from ingester
Ref #4849.
* refactor: do not collect record batched on the ingester side
Ref #4849.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* refactor: use new ingester<>querier wire protocol
Use and document the new and more flexible ingester<>querier wire
protocol.
Note that the ingester does NOT stream the response data yet, but the
internal data structures would allow that. A follow-up change will
adjust the ingester code to stream the data.
Ref #4849.
* fix: typos
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* refactor: clarify naming and public interface
* test: add schema assertion to `ingester_response_to_record_batches`
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* refactor: prepare new ingester<>querier protocol on the querier side
This changes the querier internals to work with the new protocol. The
wire protocol stays the same (for now). There's a (somewhat hackish)
adapter in place on the querier side that converts the old to the new
protocol on-the-fly. This is an intermediate step before we actually
change the wire protocol (and in a step after that also take advantage
of the new possibilites on the ingester side).
Ref #4849.
* docs: explain adapter
* chore: TEMP Update DataFusion to pre-release
* chore: update arrow et al to 16.0.0
* chore: Run cargo hakari tasks
* fix: update reader read_dictionary API
* chore: Update to real Datafusion release
* fix: Update parquet API
* fix: update test
Co-authored-by: CircleCI[bot] <circleci@influxdata.com>
This commit changes the code base to use a new reference-counted
PartitionKey type wrapper, instead of passing a bare String around.
This allows the compiler to type check & verify usage of the partition
key, instead of passing a bare string around. By reference counting the
underlying string, we reduce memory usage for some use cases.
* feat: Change data type of catalog Postgres partition's sort_key from a string to an array of string
* test: add column with comma
* fix: use new protonuf field to avoid incompactible
* fix: ensure sort_key is an empty array rather than NULL
* refactor: address review comments
* refactor: address more comments
* chore: clearer comments
* chore: Update iox_catalog/migrations/20220607102200_change_sort_key_type_to_array.sql
* chore: Update iox_catalog/migrations/20220607102200_change_sort_key_type_to_array.sql
* fix: Rename migration so it will be applied after
Co-authored-by: Marko Mikulicic <mkm@influxdata.com>