Use a constructor to initialise a ParquetFileWithTombstone struct,
rather than making the fields pub.
This allows IDEs to "go to" places where this is constructed when
browsing the code, but also keeps the type closed for modification of
internals (SOLID).
* fix: let us not compact no-data
* fix: split time must be greater min_time, too
* fix: resolve merge conflict
* chore: increase size of a compactor job and level of concurrency
Co-authored-by: Dom <dom@itsallbroken.com>
* fix: let us not compact no-data
* fix: split time must be greater min_time, too
* fix: resolve merge conflict
Co-authored-by: Dom <dom@itsallbroken.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Assert consistent metadata when evaluating candidate parquet files for
compaction.
Asserts all files have the same:
* Sequencer ID
* Namespace ID
* Table ID
* Partition ID
* Sort key
Changes the compaction logic to always reference the same SortKey
instance, rather than repeatedly querying for it.
The Partition metadata is always read from the catalog as part of
compact_partition(), where it previously threw away all metadata except
the sort key, which was passed into compact(). Then compact() would
always re-query the catalog to look up just the sort key again, and mix
up the two instances during use - one passed into the fn, one freshly
queried within the fn.
Now the Partition metadata is resolved in compact_partition() as it was
previously, but the entire Partition reference is passed to compact(),
and this is consistently used do access the sort key. This also removes
a catalog query per compaction call.
* refactor: split compact_partition into two functions to handle concurrency better
* feat: limit number of files to compact
* test: add test for limit num files
* chore: fix cipply
* feat: split group if over max size
* fix: split the overlapped group to limit size or file num
* chore: reduce config values
* test: add tests and clearer comments for the split_overlapped_groups and test_limit_size_and_num_files
* chore: more comments
* chore: cleanup
Changes the compactor to consume both StreamSplitExec output partitions
concurrently.
Practically speaking this means both Parquet files will be generated
concurrently, and uploaded to object store concurrently.
This commit changes the Compactor::compact() method to stream the
RecordBatch instances directly to the parquet serialiser, before being
uploaded directly to object storage.
Removes the min/max timestamp fields from the IoxMetadata proto
structure embedded within a Parquet file's metadata.
These values are redundant as they already exist within the Parquet
column statistics, and precluded streaming serialisation as these
removed min/max values were needed before serialising the file.
Remove the redundant row_count from the IoxMetadata structure that is
serialised into the Parquet file.
The reasoning is twofold:
* The Parquet file's native metadata already contains a row count
* Needing to know the number of rows up-front precludes streaming
Implements an upload() method on the ParquetStorage type, consuming a
stream of RecordBatch, serialising the Parquet file, and uploading the
result to object storage. Returns the IOx-specific file metadata.
Currently while the upload() method accepts a stream of RecordBatch, the
actual resulting Parquet file is buffered in memory before uploading to
object store, due to lack of streaming upload functionality in the
ObjectStore abstraction - this isn't the end of the world, as the files
tend to be relatively small with our current usage.
This impl should be easily modified to be fully streaming once streaming
object store puts are implemented:
https://github.com/influxdata/object_store_rs/issues/9
Changes the code paths that interact with Parquet files in the object
store to reference the ParquetStorage directly (DRY refactor).
This change takes us from a dependency graph of:
┌─────────────────┐
│ │
▼ │
Parquet Consumer │
│ ┌──────────────┐
├────────▶│ParquetStorage│
▼ └──────────────┘
┌──────────────┐
│ ObjectStore │
└──────────────┘
│
┌────┴────┐
▼ ▼
File s3
System (etc)
to:
Parquet Consumer
│
▼
┌──────────────┐
│ParquetStorage│
└──────────────┘
│
▼
┌──────────────┐
│ ObjectStore │
└──────────────┘
│
┌────┴────┐
▼ ▼
File s3
System (etc)
With the ParquetStorage being solely responsible for managing
interactions with the object store when dealing with Parquet files.
Renames the Storage type so the context is clear in usage (i.e. fn
args), rather than having to rely on knowing the fully-qualified import
path to know what the type stores.
Removes two unused constructors for a ParquetChunk, and moves the bare
fn constructor that is actually used to be an associated method (a
conventional constructor).
* refactor: require `Resource`s to be convertible to `u64`
* refactor: require `Resource`s to have a unit name
* refactor: make LRU cache IDs static
* feat: add LRU cache metrics
* docs: improve type names in LRU doctest
* docs: epxlain `MeasuredT`
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* docs: explain `test_metrics`
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* ci: fix cargo deny
* chore: downgrade `socket2`, version 0.4.5 was yanked
* chore: rename `query` to `iox_query`
`query` is already taken on crates.io and yanked and I am getting tired
of working around that.
* feat: `SortKey::size`
* feat: `FunctionEstimator`
* feat: querier RAM pool
Let's put all the caches into a single RAM pool, so we can at least
somewhat control RAM usage. Note that this does NOT limit the peak
memory during query execution though, but should at least stop unlimited
cache growth. A follow-up PR will add metrics.
* refactor: improve some size calculations
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* chore: move noisy debug to trace and fix some comments
* chore: Apply suggestions from code review
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* chore: fix format
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
These were found by iterating over all of the dependencies of each
Cargo.toml, then grepping that crate for the dependency's name. If it
didn't show up, I attempted to remove it.
I left a few dependencies that this process flagged:
* generated_types
- `pbjson`,`serde`. Apparently used by the generated code.
* grpc-router-test-gen
- `prost`. Apparently used by the generated code.
* influxdb_iox
- `heappy`. Doesn't appear used, but is behind enough feature
flags that I don't care to reason about and it's already optional.
- `tikv_jemalloc_sys`. Appears to be setting a feature flag of an
indirect dependency.
* iox_gitops_adapter
- `k8s_openapi`. Appears to be setting a feature flag of an indirect
dependency.
* chore: Tool for automating arrow version update
* chore: Update datafusion and arrow/parquet/arrow-flight
* fix: update for changes in Arrow API
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* feat: use stored sort key to deduplicate data
* refactor: verify if one is a super sort key of the other
* test: unit tests for scan and deduplication plans
* fix: typo
* refactor: refactor and add comments
* feat: cache partition sort key to read during planning as needed
* test: tests for query plans with different overlap groups
* chore: cleanup
* chore: resolve merge conflicts
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* refactor: grouping overlaps is now use the same overlap function in both compactor and deduplication
* chore: commit missing file
* chore: address review comments
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* test: Failing test for finding overlapped groups
* test: Failing test for query overlap too :(
* fix: Group parquet files overlapped by time correctly
Inspired by https://towardsdatascience.com/overlapping-time-period-problem-b7f1719347db
Not sure what the real name for this algorithm is
* refactor: Group items without an intermediate hashmap needed
* chore: cleanup
Co-authored-by: NGA-TRAN <nga-tran@live.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* test: use Paul deadlock reproducer and add more debug log
* test: remove compare many output rows
* test: verify the test putput
* chore: cleanup
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* chore: add more compactor debug info
* chore: Apply suggestions from code review
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* chore: fix format
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
In parquet files written after compaction, use the catalog sort key but
filter it to only those columns that appear in the merged schema.
Panic if there are any columns in the merged schema's primary key that
aren't in the catalog sort key; that shouldn't happen.
Because we decided a panic was ok to do if the catalog doesn't have a
sort key for the partition, move the panic earlier to catch it before
doing other work.
* fix: not to add IOxReadFilterNode for no data of non-duplicated chunks if there is already scan node for overlapped/duplicated chunks
* refactor: address review comments
* chore: Apply suggestions from code review
Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>
Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* feat: compact small contiguous files of the same partition even if they do not overlap
* test: more tests
* chore: Apply suggestions from code review
Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>
* refactor: address review comments
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>
Add configuration options for compactor for the max size of level 0 files and split percentage.
Add metrics for compaction to track the number of candidates, compactions, and durations.
Add functions to separate identifying partitions to compact from running compaction.
Make compaction run in smaller chunks, specifically per partition.
Update compaction to automatically promote level 0 files that are non-overlapping without waiting some period of time.
Closes#4120
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
This allows us to remove the table name from the low-level chunk
representations (like `ParquetFile`, RUB, ...) since table names are
already tracked by the higher-level data structures (e.g. catalog,
catalog chunk) that manage the low-level chunk representations.
This is similar to #4167.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
The parquet chunk is always wrapped into some higher-level data
structure (e.g. a catalog chunk, a partition, ...) that knows exactly
"where" the chunk is located. There is no need for the parquet chunk to
back-reference container-level attributes. In the contrary:
double-bookkeeping makes the code more complex and costs additional
memory.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* fix: make sure the chunks are always sorted correctly
* fix: output
* chore: Apply suggestions from code review
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* refactor: make new function for new chunk id
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* feat: remove fully processed tombstones
* test: first few tests
* fix: delete SQL
* fix: test how IN (...) works in PG
* fix: test how IN (?) works in PG
* fix: test how IN (?) works in PG
* fix: dynamically add IN (?, ?, ...)
* fix: dynamically add IN (?, ?, ...) & its dynamic values
* fix: add argument directly in the SQL
* test: more tests for catalog read and update functions
* chore: move a subfunction to make it easier to read)
* test: first test for find_can_compact but disabled due to bug
* test: integration tests and a bug fix for find_and_compact
* chore: cleanup
* refactor: address review comments
* fix: put 2 delete processed tombstones and tombstones in a transaction
The compactor stub code would wait on nothing when the caller waited on
join()-ing the compactor handler, and this meant any caller who blocked
on join() would immediately return.
Set to_delete to the time the file was marked as deleted rather than
true.
Fixes#4059.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
The sort key is optional and currently only produced by `iox_tests`.
Writing it within the ingester/compactor is tracked by #3968. The sort
key is read by the querier (and this will be verified by the query tests
and is required to merge #4103).
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
This should never be done on its own so doesn't really need to be its
own method. We also don't do anything with the returned data, so no need
to allocate those vectors.
- this is what DataFusion is doing as well; it's also fast enough
because the number of chunks in a query is not THAT massive (it's not
like we are doing row-level dyn dispatching)
- it simplifies abstracting over different databases
- it allows us to drop our enum-based dispatching that we have for
`DbChunk` and that we would also need for the querier (e.g. depending
on if a chunk is backed by a parquet file or ingester data)
- it likely speeds up compile times because the `query` is no longer
contains massive amounts of generic code
For #3934.
This makes it way easier to dyn-type database implementations. The only
real change is that we make `QueryChunk::Error` opaque. Nobody is going
to inspect that anyways, it's just printed to the user.
This is a follow-up of #4053.
Ref #3934.
Changes all consumers of the object store to use the dynamically
dispatched DynObjectStore type, instead of using a hardcoded concrete
implementation type.
* feat: initial implementation of compact a given list of overlapped parquet files
* feat: Add QueryableParquetChunk and some refactoring
* feat: build queryable parquet chunks for parquet files with tombstones
* feat: second half the implementation for Compactor's compact. Tests will be next
* fix: comments for trait funnctions fof QueryChunkMeta
* test: add tests for compactor's compact function
* fix: typos
* refactor: address Jake's review comments
* refactor: address Andrew's comments and add one more test for files in different order in the vector
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* feat: main function for finding and compacting parquet files
* chore: Apply suggestions from code review
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* refactor: rename file and struct
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>