* 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`
And only allow setting this when no record batch or line protocol is
specified so that there isn't a way to create a parquet file with data
that has a mismatched row count.
* 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: initial implementation for selecting compaction candidates
* feat: 2 catalog functions to choose the most thorughput partitions to compact and the selecting candidate function itself
* test: tests for the new 2 queries
* feat: more tests and metrics for chooing compaction candidates
* chore: Apply self suggestions from self review
* chore: cleanup
* chore: fix doc comment
* chore: Apply suggestions from code review
Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>
* refactor: address review comments
* fix: get the right time provider for the tests
* refactor: remove the left over compaction_
* fix: typos
* fix: make the param name and env name consistent
* refactor: make relevant iSomething to uSomething
* fix: typo
Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>
Fixes#5118.
Given a partition ID, look up the non-deleted Parquet files for that
partition. Separate them into level 0 and level 1, and sort the level 0
files by max sequence number.
This is not called anywhere yet.
`Executor` is only used as a performance boundary, not as a correctness
or data boundary so let's try to re-use it. This also simplifies
profiling of tests since we don't end up with hundreds (or even
thousands) of threads.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* 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>
* 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
* feat: split times of compacting results based on the max file size
* feat: cosider max file size while computing split time
* test: tests for comput_split_time
* feat: first step to teach the function split_the_steam to know how to split data into n streams using n-1 input PhysicalExprs
* feat: make StreamSplitNode support a list of expression
* docs: explain how StreamSplitNode works
* feat: Teach compute_split_time to split a time range into many contiguous ranges and split compacted result into multiple non-overlapped files based on the config comapction_max_size_bytes
* chore: cleanup
* chore: clean up doc
* chore: address review comments
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
* 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
* 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
* 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>
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.
* 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.
* 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: improve `IngesterData` public interface
* feat: impl `Debug` for `Test{Namespace,Sequencer}`
* refactor: trait interface for `LifecyleHandle`
This is required to mock the lifecycle for query tests.
* refactor: trait for partitioner
* refactor: document and improve `MockIngesterConnection`
* refactor: split `OldOneMeasurementFourChunksWithDuplicates` for `EXPLAIN` queries
* fix: mark "IngsterPartition" chunks as unsorted
* fix: "group by" queries may require sorted comparison
* refactor: re-export a few more types from querier
* fix: ensure that test parquet files are de-duped
* test: chunks in ingester stage
* docs: explain test code
* feat: impl `Debug` for `TestCatalog`
* feat: add sequencer ID and correct partition key to `IngesterPartition`
- simplifies debugging (parquet chunks and ingester chunks now use the
same partition key naming)
- the sequencer ID is required to correctly reason about tombstones (to
be implemented in a later PR)
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>
* 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 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>
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>
When created in the catalog, parquet files should always have compaction
level 0. Updating the compaction level should always happen in the
compactor.
Only the catalog should need to know about the initial compaction level
value.
This has the advantages of:
- Not needing to create fake parquet file IDs or fake deleted_at
values that aren't used by create before insertion
- Not needing too many arguments for create
- Naming the arguments so it's easier to see what value is what
argument, especially in tests
- Easier to reuse arguments or parts of arguments by using copies of
params, which makes it easier to see differences, especially in tests