To avoid other tests' state bleeding into this one and this one's state
bleeding into other tests, now that it's testing some queries without
scoping by shard.
* feat: compactor ignores max file count for first file
chore: typo in comment in compactor
* feat: restore special first file in partition compaction logic; add limit
* fix: calculation in compaction max file count
chore: clippy
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>
* feat: reject writes that are outside the retention period
* feat: add retention validator into handler stack
* chore: Apply suggestions from code review
Co-authored-by: Dom <dom@itsallbroken.com>
* refactor: address review comments
* test: unit tests fot retention validation
* chore: address review comments
* test: more unit tests and integration tests
* refactor: make time inside retention period for emphemeral_mode test
* fix: 2 hours
Co-authored-by: Dom <dom@itsallbroken.com>
* 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>
* feat: deletion flagging in GC based on retention policy
* chore: typo in comment
* fix: only soft delete parquet files that aren't yet soft deleted
* fix: guard against flakiness in catalog test
* chore: some better tests for parquet file delete flagging
Co-authored-by: Nga Tran <nga-tran@live.com>
* refactor: make namespace folder for all namesapce's commands
* feat: WIP for add command to set retention period
* feat: more on updating retention period
* feat: grpc for update namespace retention period
* test: end to end test fpr namespace retention
* fix: lint proto
* chore: cleanup
* chore: kick CI run again
* fix: command hierachy
* chore: fix comments
The checks for whether a column already exists with a different type
were relying on ordering of the input matching the ordering of the
columns returned from inserting the columns in Postgres.
Rather than trying to match the new ordering that is required to avoid
Postgres deadlocks, switch from a Vec to a HashMap and look up the
column type from the name.
This also reduces some allocations that weren't really needed.
* 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>
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!
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.
* fix: always pick cold partitions in next cycle even if it has been partially compacted recently
* fix: comment
* fix: test output
* refactor: using var instead of literal
* fix: consider deleted L0s for recent writes
* chore: cleanup
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Adds a Partition::most_recent_n() method to the catalog interface,
returning the N most recent partitions for a given set of shards.
The most recently created partitions are likely to be currently "hot"
for writes, and are cheap to list.
* 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>
The default statement logging verbosity of the `sqlx` crate is INFO, which
is frankly surprising.
The reason we didn't bother with lowering this before is that the `sqlx` crate
emits logs using the `log` crate, and we're using the `tracing` crate for logging too.
We did bridge the two logging ecosystems with https://docs.rs/tracing-log/latest/tracing_log/
but until https://github.com/influxdata/influxdb_iox/pull/5680 the bridge wasn't really working
so we didn't notice the *very* verbose logs of sqlx sstatement logging (which log our whole SQL multiline statements as INFO logs...)
Adds a migration to add a column "persisted_sequence_number" that
defines the inclusive upper-bound on sequencer writes materialised and
uploaded to object store for the partition.
* feat: garbage collector now cleans up old parquet files
* chore: clarifying comment in GC
* chore: typos in GC
* chore: typos in GC
* fix: cmdline arg in GC test needs updating after refactor
* fix: use select! on shutdown rx in GC
* fix: recalc cutoff in GD each loop
* chore: add delete_old that returns IDs only, for GC
* chore: use duration in GC args instead of usize days
* chore: GC lister runs forever w/ sleep; tests updated accordingly
* docs: fix link in GC comments to automatic link
* chore: test for delete_old_ids_only; refactor mem impl thereof
* chore: make GC test less flakey
* chore: make GC test less flakey
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"
* feat: make compactors to select candidates based on the last n minutes to reduce workload for postgres catalog query
* refactor: remove 1-minute case per review comment
* 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>
* 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>
In the all-in-one command, only one write buffer partition is supported,
and it's specified using Kafka Partition ID 0:
```
// All-in-one mode only supports one write buffer partition.
let write_buffer_partition_range_start = 0;
let write_buffer_partition_range_end = 0;
```
When using all-in-one mode with an ephemeral, in-memory catalog,
`create_or_get_default_records` is what puts records into the catalog
that need to match the write buffer configuration.
* 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: 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
* 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
* 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>
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>
This PR is the first step where we add a new column sort_key_arr whose content we'll manually migrate from sort_key.
When we're done with this, we'll merge https://github.com/influxdata/influxdb_iox/pull/4801/ (whose migration script must be adapted slightly to rename the `sort_key_arr` column back to `sort_key`).
All this must be done while we shut down the ingesters and the compactors.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
This is useful for local instances that run against a prod system,
because port forwarding can lead to long connection delays.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Add lookup of partitions by table id to catalog.
Add API to catalog to return partitions by table id.
Add to client to return partitions by table id.
Add CLI to pull remote schema, partition, and parquet files into a local catalog and object store.
Add method to catalog to get parquet file by object store id.
Add gRPC service for object store to get a file from by its uuid.
Add the object store service to router2 with object store config.
Fix the ingester to track the max persisted sequence number per partition.
Ensure replay takes in data from unpersisted partitions.
Simplify the table persist info to not return a max persisted sequence number for the table as that information isn't needed.
After checking the postgres workload for the catalog in prod, this
missing index was noted as the cause of unexpectedly expensive plans for
simple queries.
* fix: create_or_get_multi for column in catalog now enforces limits
fix: create_or_get_multi for column in catalog now enforces limits
chore: reorder catalog column create fns to be next to each other
test: add failing test for multi col insert w/ limits
test: bend catalog mem impl to match postgres for tests
fix: postgres column insert many column type error checks
chore: clippy
* test: assert column counts in partial column insert test
* chore: add some sql comments to the monster multicolumn insert query; s/RIGHT/INNER/ join
* chore: adding comments to clarify partial failure behaviour of multi col insert
* test: add tests for create_or_get_many columns in catalog
* test: forgot how macros work for a moment
* test: service limit test handles partial update of cols
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Add indexes so compactor can find candidate partitions and specific partition files quickly.
Limit number of level 0 files returned for determining candidates. This should ensure that if comapction is very backed up, it will be able to work through the backlog without evaluating the entire world.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@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>
* 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
Lowercases the error messages in the big iox_catalog Error enum for
better composition of messages (no random capitalisation in
glued-together strings, which is common with wrapped errors).
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>
* feat: `TombstoneRepo::list_by_table`
* feat: `ParquetFileRepo::list_by_table_not_to_delete`
* refactor: `querier` w/o `db`
Get the `querier` to work w/o relying on `db`. A few notes:
- Testing is kinda shallow, we really need to get `query_tests` working
w/ `querier` (see #3934).
- We still run a sync loop for namespaces, tables and schemas. This will
be a replaced by "update namespace incl. tables and schemas on demand".
Note however that we cannot fetch single tables and schemas on demand
at the moment, because DataFusion doesn't implement async schema
inspection (only `scan` / "give me all the chunks" is async). I think
that's OK for now and we can address this later.
- There is NO cache for parquet files and tombstones at the moment. For
correctness, they need to be fetched in a single transaction (or we
need a kinda tricky sequence number / logical clock tracking) and I am
not sure yet how this makes sense when we have the ingester data wired
up and predicates pushed down to the catalog (see next point). So
let's measure first and then decide on a caching strategy for this.
- Predicates are currently NOT pushed down to the catalog. I'll need to
figure out how to extract time range from generic DataFusion
expressions to make that work (it's easier for InfluxRPC queries, but
they are not tested at the moment, see first point).
Sorry that this commit is kinda huge. I initially planned to only
migrate the chunks away from `db` and leave the tables and schemas for a
follow-up PR, but the DataFusion trait structure (chunks are bound to
their tables) makes this kinda pointless.
Closes#3974.
* docs: explain what we're doing
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* docs: mention tracking issues
* docs: explain what we're doing
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* feat: `TableRepo::get_by_namespace_and_name`
* refactor: rework `TableCache`
- dual cache that can also map table names to IDs
- deal w/ missing tables w/o panics
- set proper timeouts to missing data
For #3974.
* test: extend table cache tests
fix: refactor table & col limit enforcement in catalog into single SQL statement
fix: borked rebase
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
Configure the postgres catalog to close unused connections after 1
minute, rather than 500s to introduce a bit of fluidity to pool of
connection acquires.
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
Adds indexes to the JOINed fields to reduce execution cost, as the
TableRepo::get_table_persist_info() is currently by far the most
expensive catalog operation.
This includes a bit of a refactor in the locking structure of the buffer data. Locking at the partition collection and within the partition data was making things more complex than they needed to be. The partitions in the buffer are there only temporarily until they get persisted. Locking on the table simplifies things a bit and makes it more clear when the table state is being modified since it no longer has any interior mutability. Having access to separate partitions without the same lock isn't something we need because queries will hit all partitions and data is brought in sequentially, regardless of which partition it is hitting in a sequencer.
Fixes#3850
Uses the new ColumnRepo::create_or_get_many() catalog method to perform
a bulk upsert of (potentially) new columns to the catalog during schema
validation.
Removes openssl as a dependency, switching to rustls[1] as the TLS
implementation throughout.
It is important to note that this change brings with it a significant
behavioural difference - rustls does not currently support IP SANs in
certificates (instead only supporting fully-qualified names / DNS) and
this will manifest as a failure to connect to IP endpoints over TLS.
This might be a blocker that prevents us using rustls exclusively, but
there's noe asy way to know without trying it. Fortunately the rustls
project has received funding to work on IP SAN support[2].
[1]: https://github.com/rustls/rustls
[2]: https://www.abetterinternet.org/post/preparing-rustls-for-wider-adoption/
* feat: changes needed to apply tombstones correctly on the life-cycle ingest bacthes
* refactor: adjust the design after discussing with Paul
* feat: apply the coming tombstone on all data but persiting one
* chore: fmt
* fix: build on buffer tombstone
* test: delete & write tests for a parition and some cleanup
* feat: No need add processed tombstones for newly created parquet file in the ingester becasue all deletes before that parquet file is created were applied
* chore: cleanup
* feat: intitial implementation for preparing data to send back to the Querier
* feat: full implementation of prepare_data_to_querier
* fix: apply filters for the batches
* chore: Apply suggestions from code review
Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>
* chore: cleanup
* fix: typos in comments
* fix: typos in comments
* fix: typos in comments
* test: create different scenarios and test them
* chore: fix typos
* test: add tests with deletes
* chore: make pub pub(crate)
* chore: Apply suggestions from code review
Co-authored-by: Jake Goulding <jake.goulding@integer32.com>
* refactor: address review comments
* fix: keep batches in their arrival order
* refactor: not assign unecessary values to enum
* refactor: use bitflags enum
* fix: use bitflags correctly
* chore: Apply suggestions from code review
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* refactor: avoid using use at the end of the function
* chore: merge main to branch
* fix: fix downgrade versions
* refactor: address review comments
* chore: remove unnecessary comments
* refactor: Make the whole test_utils module test-only and bring paths into module scope
Co-authored-by: Paul Dix <paul@pauldix.net>
Co-authored-by: Carol (Nichols || Goulding) <193874+carols10cents@users.noreply.github.com>
Co-authored-by: Jake Goulding <jake.goulding@integer32.com>
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: Carol (Nichols || Goulding) <carol.nichols@gmail.com>
A quick change to perform the ColumnRepo::create_or_get() calls in
parallel (up to a maximum of 3 in-flight at any one time) in order to
mitigate the latency of the call and reduce the overall schema
validation call duration.
The in-flight limit is enforced to avoid starving the DB connection pool
of connections.
Wraps the postgres implementation of the catalog with a MetricDecorator.
This is slightly intrusive, with the metrics registry being pushed into
the PostgresCatalog type in order to decorate the impls returned when
calling the Catalog::repositories() and Catalog::start_transaction()
methods (rather than being a pure decorator) in order to use static
dispatch and let the compiler optimise away as much overhead as
possible.
Adds a MetricDecorator type that wraps a Catalog impl, delegating calls
to the underlying impl and recording per-method call latency broken down
by success/error state.
Fixes#3702. This pulls the min sequence tracking into the LifecycleManager. Because the number requires looking at all other partitions in memory, this was the most efficient place to put it. The manager updates the sequencer state after it calls persist. The number is meant to be a lower bound on the sequence number. Issue #3783 will add functionality for the ingester to ignore replayed data that has already been persisted.
* feat: changes needed to apply tombstones correctly on the life-cycle ingest bacthes
* refactor: adjust the design after discussing with Paul
* feat: apply the coming tombstone on all data but persiting one
* chore: fmt
* fix: build on buffer tombstone
* test: delete & write tests for a parition and some cleanup
* feat: No need add processed tombstones for newly created parquet file in the ingester becasue all deletes before that parquet file is created were applied
* chore: cleanup
Co-authored-by: Paul Dix <paul@pauldix.net>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
The _sql_migrations table cannot be created by the `catalog setup`
bootstrap command because it is created before the migrations run, one
of which creates the iox_catalog namespace the catalog operates in.
This change allows the _sql_migrations table to be created in the public
schema to start the migration process, with everything else living in
the iox_catalog schema.
If two callers call create_or_get() for a tombstone, providing the same
(table_id, sequencer_id, sequence_number) triplet but a different
predicate / timestamps the catalog MUST NOT silently continue.
As this is unexpected, this behaviour causes a panic.
If two callers call create_or_get() for a partition, providing the same
partition key & table ID, but different sequence numbers the catalog
MUST NOT continue silently.
As this is unexpected, this behaviour causes a panic.
This commit introduces test cases that ensure that for each of the
tombstone & partition repos:
* create_or_get() is idempotent
* create_or_get() does not silently drop conflicting writes
The former covers the expected use case: callers issuing potentially
multiple calls to create_or_get() with the same args, causing the
catalog impl to transparently turn the subsequent calls into NOPs.
The latter illustrates an issue where multiple calls to create_or_get()
with differing arguments is silently accepted by the catalog, causing
both callers to believe they have committed the (differing) details they
provided. This test is expected to pass but fails in this commit.
This commit changes the iox_catalog test harness so that each test is
run in an independent, randomly generated schema to avoid concurrent
tests interfering with each other.
Each test creates a randomly-named schema, grants permissions to the new
schema, runs the full migration stack against the new schema, and then
executes the code under test.
Allow the caller to set the Postgres schema a migration should be
applied to, rather than restricting the migration to a specific,
hard-coded schema.
BREAKING CHANGE: manually adds a new migration that precedes the
existing migration to ensure the iox_catalog schema exists before
applying the migration. You'll probably have to drop any existing
databases and migrate from scratch:
sqlx database drop; sqlx database create;
* chore: port sqlx-hotswap-pool over from conductor
Co-authored-by: Marko Mikulicic <mkm@influxdata.com>
* chore: workspace hack fixes
* fix: unique schema per test db connection
* fix: adjust search path in catalog pg tests to see if it fixes test schema issue
* fix: actually fixed sqlx hotswap pool test
Co-authored-by: Marko Mikulicic <mkm@influxdata.com>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* feat: allow catalog access w/o a transaction
Now the caller has the full control if they want to use a transaction or
not.
* fix: remove non-transaction-safe `create_many`
* fix: remove unnecessary transactions
* feat: projection pushdown for QueryableBatch
* chore: clean up and remove unwrap
* fix: Add Sync to a Snafu source to have the code compile
* chore: cleanup and add comments for tests
* refactor: Add tests for scanning non existing columns and fix related bugs
* chore: modify comment to trigger auto check in github work
* refactor: catalog Unit of Work (= transaction)
Setup an inteface to handle Units of Work within our catalog. Previously
both the Postgres and the in-mem backend used "mini-transactions on
demand". Now the caller has a clear way to establish boundaries and
gets read and write isolation. A single `Arc<dyn Catalog>` can create as
many `Box<dyn UnitOfWork>` as you like, but note that depending on the
backend you may not scale infinitely (postgres will likely impose
certain limits and the in-mem backend limits concurrency to 1 to keep
things simple).
* docs: improve wording
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
* refactor: rename Unit of Work to Transaction
* test: improve `test_txn_isolation`
* feat: clearify transaction drop semantics
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
* feat: add ProcessedTombstoneRepo
* feat: add function add_parquet_file_with_tombstones
* fix: remove unecessary use
* feat: handling transaction when adding parquet file and its processed tombstones
* feat: tests update catalog for parquet file and processed tombstones
* fix: make add parquet file & its processed tombstones fully transactional
* chore: cleanup
* test: add integration tests for new catalog update functions
* chore: remove catalog_update.rs
* chore: cleanup
* fix: assert the right values
* fix: create unique namespace
* fix: support non transaction create_many
* test: remove tests that do not work in a transaction
* fix: one more case with unique namespace
* chore: more verification around for better understanding why certain tests fail
* fix: compare difference rather than absolute becasue the DB already has data
* fix: fix the argument provided to SQL
* fix: return non-empty processed tombstones
* fix: insert the right parquet file
* chore: remove unsed file
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
This adds the scaffolding for the ingester server to consume data from Kafka. This ingests data in an in memory structure while creating records in the catalog for any partitions that don't yet exist.
I've removed catalog_update.rs in ingester for now. That was mostly a placeholder and will be going in a combination of handler.rs and data.rs on my next PR which will have some primitive lifecycle wired up.
There's one ugly bit here where the DML write is cloned because it's getting borrowed to output spans and metrics. I'll need to follow up with a refactor to make it so that the DML write's tables can be consumed without it gumming up the metrics stuff.
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>