* fix: make batch size back to 1024 to see if the OOM in the compactor go away
* fix: address review comments
* chore: Apply suggestions from code review
Co-authored-by: Marco Neumann <marco@crepererum.net>
* fix: import needed constant
Co-authored-by: Marco Neumann <marco@crepererum.net>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* refactor: increase batch size when reading parquet
This reduced our overhead when reading parquet files quite a lot.
In some internal benchmark, this reduces the size to perform a single
series aggregation of a rather large series with cold caches from 58s to
48s for cold caches. No real difference could be measured for warm
caches (~21ms for both).
This should also help the compactor since the record batches should be
larger.
* refactor: ensure that parquet row group size is in-sync
Ensure that we use the same row group size for reading and writing
parquet files. This is the same value as upstream currently uses as a
default, but let's make sure we don't diverge from that:
3032a521c9/parquet/src/file/properties.rs (L65)
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* 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>
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>
Prior to this change background tasks that we feed into `AdapterStream`
can panic but that would just end the stream without any user-visible
error (except for the panic message on stdout/stderr).
This was found while developing #4964. I have proposed another fix in #4966
but found that I actually developed an existing solution a 2nd time:
`watch_task`. But I also see a major issue with the existing API: one
can create `AdapterStream` with ordinary tokio tasks that are not
watched at all, leaving the burden to the implementor to check for that
(and actually we forgot that in `parquet_file`).
So this change takes a slightly different approach:
The `AdapterStream` does NOT accept ordinary join handles any longer but
requires that you pass a "watched task". The newly introduced
`WatchedTask` does the same as we did manually before: wrapping a future
into a tokio task, watch it and wrap the watcher into a task.
It is now way more difficult to do anything stupid (sure you can still
mix up the tasks and the channels, but we need at least some flexibility
here to allow for "split" and potential future fan-in/out constructs).
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
* 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>
* 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: column handling when reading parquet files
This improves/fixes/tests a few aspects when reading parquet files:
- fix usage of `Selection::Some(...)`. This was broken since #4912 but
apparently no test caught that.
- ensure that the order of `Selection::Some(...)` is preserved
- ensure that schema metadata is attached to output batches
- ignore parquet columns that we don't care about (i.e. do not select)
- allow parquet file to have a different column order than our internal
bookkeeping, this makes it way simpler to read parquet files w/o
scanning the metadata first
- extend the test coverage
Ref #4124.
* test: even more tests for parquet reader
* 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.
* fix: do NOT block in parquet file IO
I think for historical reason we were using blocking IO to read parquet
files. With the current streaming `SendableRecordStream` approach this
is technically NOT required anymore.
Now one might think that the sync-async dance that we did is kinda
harmless, but looking at our producition querier I think it is really
bad. The querier seems to be stuck but looking at `strace` and other
health signal it seems it is not entirely dead. Looking at GDB
backtraces it seems that nearly all threads are busy in
`download_and_scan_parquet`. Looking at the tokio docs
(<https://docs.rs/tokio/1.18.2/tokio/task/fn.spawn_blocking.html>)
for `spawn_blocking` (which is used to start the sync download) this
makes sense: tokio only starts replacement threads for the current
runtime thread (which calls `spawn_blocking`) if this does NOT exceed the
runtime thread limit. However we set the runtime thread limit to the
number of CPU cores available to IOx, so this is a limiting factor. This
means that there are only a few threads left to do actual work (I've
seen postgres data flowing back and forth for example) but tokio is not
able to use its full potential anymore. This is esp. bad because the
sync code in `download_and_scan_parquet` then uses `futures` `block_on`
functionality to call back into async code, so it waits for tokio
itself.
The change is rather simple: just use async task spawns.
* fix: use async IO to write stream to temp file
* fix: do not block tokio thread during parquet file reading
* refactor: ensure parquet IO tasks are cancelled if they are not needed anymore
There is no REAL way to cancel sync tasks, but at least we can try our
best.
* test: test that hits panic becasue of no column meta data
* chore: Apply suggestions from code review
* chore: run format after applying changes
* chore: Apply suggestions from code review
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* chore: run clippy
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* test: reproducer for 4695
* chore: some debug info
* test: test with many columns and rows
* chore: cleanup and add debug info
* chore: cleanup
* chore: cleanup
* chore: more debug info
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
Adds two integration tests covering validation of the embedded IOx
metadata within the Parquet file metadata, and validation of the derived
ParquetFileParams metadata used to populate the catalog.
Enable more lints on the parquet_file crate to keep it a little cleaner
- adds the following:
clippy::clone_on_ref_ptr,
unreachable_pub,
missing_docs,
clippy::todo,
clippy::dbg_macro
This commit includes fixes for any new lint failures.
Derive the ParquetFilePath from the IoxMetadata within the
ParquetStorage::read_filter() call.
This prevents the "put/get RecordBatches" abstraction from leaking out
the object store path generation concern - an implementation detail of
the ParquetStorage layer.
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
Construct a IoxParquetMetaData instance directly from the FileMetaData
instance returned by the ArrowWriter.
This change will allow us to avoid the inefficient impl currently in
use:
* Serialise batches into memory
* Wrap buffer in arrow cursor
* Read parquet metadata with arrow file reader
* Serialise schema with thrift
* Serialise each row group's metadata with thrift
* Construct our own FileMetaData instance
* Serialise FileMetaData with thrift
* zstd encode resulting thrift bytes
* Wrap in IoxParquetMetaData
Now we "only":
* Stream batches into opaque Write impl
* Serialise FileMetaData with thrift
* zstd encode resulting thrift bytes
* Wrap in IoxParquetMetaData
Then accessing any data within the IoxParquetMetaData (as before this
change) requires deserialising it first.
There are still a number of easy performance improvements to be had
w.r.t the metadata handling.
Implements a streaming RecordBatch to Parquet file serialiser.
This impl automatically discovers the schema of the RecordBatch stream,
and accepts &mut destination types (internalising the handle
cloning/etc) to simplify caller usage.
This encoder returns the resulting FileMetaData to allow callers to
inspect the resulting metadata without reading back the file.
Currently unused / not yet plumbed in.
* fix: ensure that query tokio background tasks are canceled
While I am not entirely sure if this explains some of the memory leaks I
am seeing in prod, not canceling the tasks correctly certainly makes
debugging way harder and also renders certain form of throttling (e.g.
max. concurrent queries) somewhat ineffective.
Note that parquet file downloads are currently NOT canceled because
tokios `spawn_blocking` cannot be canceled.
* refactor: `Vec` -> `Option`
* refactor: `spawn_blocking` creates a join handle, even though it is useless
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
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).
* 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>