At the moment it takes way to long to half-open and close circuits ones
they were opened.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* feat: create namespace API call in router
Co-authored-by: Nga Tran <nga-tran@live.com>
* chore: treat retention as ns except in CLI
* fix: overflow in nanosecond calc
* fix: retention test after changing it from hours to ns
* chore: comment clarification in cli; better response type for error in ns API
* fix: correct some rebase mistakes
* chore: merge namespace create & create_with_retention; renamed ns create test helper fn & const
* fix: ns autocreation test was wrong after rebase
* fix: mem catalog has default 1hr retention, accidently removed in rebase
* chore: remove mem catalogs default 1hr retention; make it settable in sets & router
Co-authored-by: Luke Bond <luke.n.bond@gmail.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* chore: Update datafusion pin + api code
* chore: Run cargo hakari tasks
* refactor: combine_sort_key is more idomatic and add rationale comments
* refactor: satisfy borrow checker and updated comments
* fix: Add test case for combine_sort_key
* fix: Apply suggestions from code review
Co-authored-by: Marco Neumann <marco@crepererum.net>
* fix: Add back test for deeply nested expression
* fix: Update output ordering
Co-authored-by: CircleCI[bot] <circleci@influxdata.com>
Co-authored-by: Marco Neumann <marco@crepererum.net>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.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>
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>
* feat: Add object_store handler to querier
* test: end to end test for get-table from querier
* fix: doc links
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* test: check server exit status on `TestServer` drop
* fix: handle recursing limit in querier<>ingester comm
Fixes#5974.
* docs: improve
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
* test: simplify
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* 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>
We basically assume everywhere that a column falls into one of the three
known categories (time, tag, field), so lets encode this in our type
system instead of defining "unknown" as "undefined behavior, may or may
not crash".
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Use the proper top-level DataFusion context and register the object
store there.
Note that we still hide the `ParquetExec` behind an opaque record batch
stream. Fixing that is next on my list.
Helps with #5897.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* feat: rework cache refresh logic
Instead of issuing a single refresh when a GET request for a cached key
comes in, start a background job (using some efficient logic to not
overload tokio) per key that refreshes the key using some exponential
backoff. The timer is reset a new GET request comes in. This has the
following advantages:
- our backoff logic decorrelates the requests
- the longer a key was not used, the less often it will be updated
All test (esp. integration tests) as adjusted accordingly, mostly to
account for the fact that no extra GET is required to start the refresh
timer.
Closes#5720.
* docs: improve
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
* refactor: simplify rng overwrite
Co-authored-by: Andrew Lamb <alamb@influxdata.com>
* feat: query only from parquet
* Revert "feat: query only from parquet"
This reverts commit 5ce3c3449c0b9c90154c8c6ece4a40a9c083b7ba.
* Revert "revert: disable read buffer usage in querier (#5579) (#5603)"
This reverts commit df5ef875b4.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
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!
* feat: initial step to identify where the projection should be provided
* feat: start getting columns of all expressions
* chore: format
* test: test for the table_chunk_stream
* fix: fix a compile error. Thanks @alamb
* test: full tests for table_chunk_stream
* chore: cleanup
* fix: do not cut any columns in case all fields are needed
* test: add one more test case of reading all columns
* refactor: move code that identify columbs ot push down to a function. Add the use of field_columns
* chore: cleanup
* refactor: make sream_from_batch support empty batches
* chore: cleanup
* chore: fix clippy after auto merge
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
- 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.
* feat: send only needed projection columns from querier to ingester in case of normal SQL queries
* refactor: push column index down until we need to convert them strings
* fix: make the test deterministic
* test: test for the projection pushdown
* test: add asserts for the proj pushdown test
* test: implement projection pushdown for partitions of MockIngesterConnection
* chore: cleanup
* chore: address review comments
* chore: Apply suggestions from code review
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* refactor: address review comments
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* 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>
* refactor: arc the cached table
* refactor: use cheaper hash keys for projected schemas
Instead of using the column names to address projected schemas, let's
use the column IDs.
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>
* refactor: retry querier->ingester requests
Esp. for InfluxRPC requests that scan multiple tables, it may be that
one ingester requests fails. We shall retry that request instead of
failing the entire query.
* refactor: improve docs
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* fix: less foo
* docs: remove outdated TODO
* test: assert that panic happened
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* refactor: improve consistent access under "remove if"
With all the concurrency introduced in #5668, we should be a bit more
careful with our "remove if" handling, esp. if a removal is triggered
while a load is running concurrently. This change introduces as
`remove_if_and_get` helper that ensures this and the querier over to use
it. The parquet file and tombstone caches required a bit of a larger
change because there the invalidation and the actual GET were kinda
separate. We had this separation for the other caches as well at some
point and decided that this easily leads to API misuse, so I took this
opportunity to "fix" the parquet file and tombstone cache as well.
* docs: improve
* feat: split "pruned" metric into "early" and "late"
* docs: improve
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* docs: explain `PruningMetrics`
* test: try to test pruning
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Create chunks in querier concurrently after we've pre-filtered them.
Chunk creation still may require a bit of cached information (e.g. the
partition sort key) and we can easily fetch these concurrently instead
of in order.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
This should lower catalog load and eliminate a few costly cache misses.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
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: read querier parquet files from cache
* refactor: only use parquet files in querier (no RB)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* 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"
Remove our own hand-rolled logic and let DataFusion read the parquet
files.
As a bonus, this now supports predicate pushdown to the deserialization
step, so we can use parquets as in in-mem buffer.
Note that this currently uses some "nested" DataFusion hack due to the
way the `QueryChunk` interface works. Midterm I'll change the interface
so that the `ParquetExec` nodes are directly visible to DataFusion
instead of some opaque `SendableRecordBatchStream`.
* refactor: do not override parquet file size in querier
This is going to be an issue when we actually rely on the size for
reading, see #5531.
* refactor: use selected file size mocking in compactor
Do not blindly override parquet file sizes for all subsystems.
This is going to be an issue when we actually rely on the size for
reading, see #5531.
* refactor: remove ability to override file sizes in catalog
Blindly overriding data for all subsystems is dangerous, because some
parts of our stack actually rely on the actual file size. See #5531.
* docs: explain `size_overrides`
The API user may still use a `Box<dyn ...>` if they want, but they
technically don't have to.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
The API user still CAN use dynamic dispatch but doesn't have to. This
also simplifies the generics a bit.
This is similar to #5520.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
This removes some `Box<dyn ...>` indirection when the user doesn't want
it (you still can, but don't have to) and makes the whole type handling
easier to understand.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
1. Cache converted schema instead of catalog schema. This safes a buch
of memcopies during conversion.
2. Simplify creation of new chunks, we now only need a `CachedTable`
instead of a namespace and a table schema.
In an artificial benchmark, this removed around 10ms from the query
(although that was prior to #5467 which moved schema conversion one
level up). Still I think it is the cleaner cache design.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* refactor: use a single timestamp in policy backend
Prior to this PR we had at least 1 `TimeProvider::now` calls per GET
request (for caches that only used LRU) and up to 3 calls (caches with
LRU + refresh + TTL). Let's instead use a single timestamp that is
created by the policy backend itself (instead of the policies). This has
the following consequences:
- **efficiency:** `SystemProvider::now` is not free, even though under Linux
this doesn't result in a syscall, it uses the stdlib time system which
also checks for monotonicity
- **consistency:** All changes for a single trigger (e.g. a
GET cache call) now use a single timestamp instead of slightly
increasing ones. I argue this is the better semantic, simpler to
understand and better to debug.
For some (slightly artificial) local performance experiment, this shaves
off around 2ms per single-table SQL query. However I expect that there might
be more degenerated cases (e.g. multi-table SQL queries or some
InfluxRPC requests that hit multiple tables).
The majority of this patch is moving the `TimeProvider` from the
policies into the policy backend.
* docs: explain `now` parameter
* fix: hoist repeated computation out of chunk creation
We have hundreds of chunks per table, so it is beneficial to only
do common work once.
* chore: remove TableCache as it is no longer used
* fix: prune chunks both before and after metadata fetch
Fetching the metadata for all the chunks in a table is expensive,
especially when we have a narrow time range query that only
needs a few chunks.
* chore: fix clippy
* fix: fix up some last tests
* fix: review comments
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
This doesn't really need to be fallible but forces propagation of a ton
of error handling - no shards is always a sign of something being very
wrong, and can be caught in the caller if it's for some reason an
acceptable state / can be recovered from.
* refactor: allow `ChangeRequest` to carry a lifetime
Let's not restrict our change functions to `'static` because this would
require us to clone loads of data to achieve predicate-based
`remove_if`.
* refactor: convert `remove_if` feature to policy framework
Decided to drop the "shared" functionality. We only use the small
`remove_if` bit which is way easier to reason about.
For #5320.
* refactor: address review comments
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* refactor: port TTL backend to policy framework
Note that this is "just" a port, it does NOT change how TTL works. This
will be done in #5318.
Helps with #5320.
* fix: ensure inner backend is empty
* test: add some smoke test
We already prune all chunks in the query-access layer. There's no need
to do that another time (which is actually the first time) in
`QuerierTable::chunks`. The time savings we get from feeding less chunks
into the state reconciling should be negligible. On the pro-side however
we get a more streamlined data flow and actually correct chunk pruning
metrics. Also see #5336.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
- emit a warning if we cannot even attempt to prune chunks due to an
error. This is always either a missing feature or a bug (even though
it does not impact correctness but _only_ performance). Also see
https://github.com/influxdata/conductor/issues/1107
- change metrics to clearly differentiate between "could not prune" and
"not pruned"
- add new "not pruned" observer hook (this was missing for some reason,
the "pruned" hook existed though)
* refactor: make could-not-prune reason a static string
* refactor: introduce `QuerierTableArgs`
* feat: chunk pruning metrics
Closes#4974.
* refactor: address review comments
* refactor: use static typing for not-pruned reason
* refactor: pass chunk to not-pruned observer and use it for some metrics
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* feat: make querier RAM pool split a proper feature
- use propre pool names
- expose sizing via CLI/env
Closes https://github.com/influxdata/conductor/issues/1102.
* refactor: improve naming and docs
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Quick&Dirty implementation of a RAM-pool split to see if this has any
effect. I expect the querier performance to improve due to this because
large read buffers can no longer evict precious metadata.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
This is what DataFusion uses by default and I don't see a reason why we
should use such small batch sizes.
The affect is probably only visible in certain filter-aggregate queries
that don't focus on a single series (because there we likely end up with
1 or 2 batches only, esp. after #5250) for coarse-grained filters, esp.
when the filter key is not the first sort key.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.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>
* feat: cache tracing
Add tracing to the metrics cache wrapper. The extra arguments for GET
and PEEK make this quite simple, because the wrapper can just extend the
inner args with the trace information.
We currently terminate the span in `querier::cache` (i.e. only pass in
`None`, so no tracing will occur) to keep this PR rather small. This
will be changed in subsequent PRs.
For #5129.
* fix: typo
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
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>
This will be used to pass spans down to `CacheWithMetrics` (or a new
wrapper specific to tracing) and will help with #5129.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
This adds tracing of querire->ingester request up to the point where we
perform the network request, i.e. the trace will only appear on the
querier side. We may extend this at some point to carry the tracing
information to the ingester as well.
Ref #5129.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
I forgot to address a TODO in #5091. Extends to test to actually check
the chunk stage and removes the function for manual force-loads.
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.
This is not relevant at the moment for prod since other layers
prevent/filter queries for non-existing namespaces.
However this messes up the flux integration tests, see
https://github.com/influxdata/conductor/issues/997
So let's disable this specific cache case until #4617 is implemented
which may be used by the flux tests.
Fixes https://github.com/influxdata/conductor/issues/997
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>
* chore: Update datafusion pin
* fix: Update for api
* fix: Explicitly set coalsce batch size
* fix: Update batch size as well
* fix: update tests for new explain plan, and improved coercion
Currently the querier fetches RB in a serial manner, which is probably
not good since each cache miss takes between 10ms and 250ms.
Let's try to fetch 2 in parallel and if that works well, make this a
proper config.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* fix: `Tombstone::size` must include serialized predicate
* fix: `CachedPartition::size` must include `Arc` heap allocation
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* refactor: remove `DecodedParquetFile` from `iox_tests`
* refactor: remove `DecodedParquetFile` from querier
Also pull out all the chunk schema and sort key handling into a function
so that RB chunks and parquet chunks mostly use the same code path.
* refactor: remove `DecodedParquetFile`
* refactor: remove `ParquetFileWithMetadata` usage
* fix: test data consistency
* refactor: simplify sort key calculation
* refactor: use schema from catalog instead from file
* refactor: do not request parquet file MD in compactor
* test: ensure that `QueryableParquetChunk` works correctly
* refactor: avoid feeding sort key from struct into same struct
* feat: allow namespace schema query by ID
* refactor: do not use binary parquet file MD in compactor tests
* refactor: do not use in-parquet IOx metadata
* refactor: reduce number of catalog queries
* feat: conversion from `ParquetFile` to `ParquetFilePath`
* refactor: slim down parquet chunk
- ensure it works without binary parquet metadata
- timestamp range is no longer optional (ensured by the NG type system)
- remove table summary: this is only needed for SOME API users. The
compactor can perfectly work without statistics since has the timestamp
range which is sufficient for the current overlap check (we don't use
any other primary key stats at the moment). The querier currently does
NOT use parquet chunks (was replaced by read buffer) but if it will
again in some future it will likely need to find a way to fetch and
cache the statistics.
- the schema is now provided by the API user since it can be
reconstructed using the NG catalog only (and "wrong" column orders are
tolerated as of #4921)
Ref #4124
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* fix: use proper sort key in tests
* feat: do not rely on encoded parquet metadata for RB chunks
Ref #4124.
* refactor: allocate less strings
* refactor: use upstream PK calculation
* fix: cache expiration w/o a good reason
* refactor: make namespace cache safer to use
* refactor: make partition cache safer to use
Queries internals are not meant to be used by other crates. Only a
handful selected interfaces should be used by IOxD and the query tests.
The compactor only used a very small subset just to read parquet files
back into memory. It shall rather use the official `parquet_file`
interface instead.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* refactor: `TestPartition::update_sort_key` should return an `Arc`
The whole test framework is built around `Arc`s, so let's fix this
consistency issue.
* fix: actually calculate correct column set in test framework
* feat: check expected parquet file schema
While working on the querier I made some mistakes regarding schemas and
such a check would have greatly improved the debugging experience.
* feat: namespace cache expiration
* fix: improve parquet schema check
* fix: remove clone
The low-level chunk storage shouldn't care about the table name (this is
also true for parquet chunks btw). In fact, the table name is already
only a partial information since it misses the namespace.
If we need a table name, then the high-level chunk/data management is
responsible for that.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* 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(querier): split ingester partitions into chunks
With the new wire protocol the ingester can now transmit multiple
snapshots per partition with different schemas. This changes the querier
to reflect this and and splits uses the individual snapshots as chunks
for the query engine instead of a single partition.
The schema handling was changed so that instead of a table-wide schema
enforcement, we now use the snapshot-specific projections. This means we
do not need to create all-NULL columns any longer because the batches
within the chunks now always have the correct schema.
* refactor: "disassembler" -> "decoder"
* fix: make ChunkOrder u64 data type to accept min sequence number 0
* fix: make ChunkOrder i64 to match with sequence number type
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
The metrics and logs introduced in #4806 will be emitted once for all
ingesters instead of per request. The accumulated view makes it pretty
hard to judge the actual request-response timings and the number of
requests.
Instead we now measure the data per request.
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
* feat: extend flight client to accept multiple (changing) schemas
See #4849.
Originally I intended not to use Flight at all for the new
ingester<>querier protocol. However since flight also deals with
dictionary batches and multiple batches and the gRPC protocol that I
would write would look very similar, I will use Flight with a bit more
flexible message types.
The rough idea for the protocol is the following stream:
- for each partition:
1. "none" message with partition metadata
2. for each chunk (can have different schemas under certain
circumstances):
1. "schema" message (resets dictionary state)
2. (optional) dictionary batch messages
3. one or more "record batch" message
The nice thing about it is that the same arrow client works also for the
existing client<>querier protocol since there we just send:
1. "schema" message (no app metadata)
2. (optional) dictionary batch messages
3. zero, one or more "record batch" message (no app metadata)
* refactor: separate high- and low-level flight client
It is very unlikely that a user will use the high-level batch-producing
functionality and the low-level stuff within the same session. So let's
split this into to clients (high-level uses the low-level one
internally) to avoid confusion.
Also add documentation on our protocol handling.
* refactor: enumerate all variants in match statement to better catch errors in the future
* feat: Log time spent requesting ingester partitions
Fixes#4558.
* feat: Record a metric for the duration queriers wait on ingesters
* fix: Use DurationHistogram instead of U64 Histogram
* test: Add a test for the ingester ms metric
* feat: Add back the logging to provide both logging and metrics for ingester duration
* refactor: Use sample_count method on metrics
* feat: Record ingester duration separately for success or failure
* fix: Create a separate test for the ingester metrics
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* 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.
To roughly gauge how much data we re-load into cached (i.e. data that
was already loaded but was later evicted due to LRU pressure or TTL
eviction) this change introduces a new metric that estimates if a cache
entry that is requested from the loader was already seen before (using a
probabilistic filter).
* 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>