Commit Graph

49315 Commits (054ac7e8a305defd6765bba7cd9489ec803bdf0f)

Author SHA1 Message Date
Paul Dix 054ac7e8a3
feat: add host_id to PersistedSnapshot (#25335) 2024-09-16 09:49:11 -04:00
Paul Dix 341b8d7aff
feat: add watch to writebuffer for persisted snapshots (#25330) 2024-09-13 16:58:18 -04:00
Trevor Hilton ed2050f448
test: split e2e test harness up for pro (#25322)
This makes some changes to the TestServer E2E framework, which is used
for running integration tests in the influxdb3 crate. These changes are
meant so that we can more easily split the code for pro.
2024-09-13 11:36:59 -04:00
Paul Dix f8b6cfac5b
refactor: Rename level 0 to gen1 to match compaction wording (#25317) 2024-09-12 15:57:30 -04:00
praveen-influx dd8f324728
chore: update core dependency revision (#25306)
- version changed from 1d5011bde4c343890bb58aa77415b20cb900a4a8 to
  1eaa4ed5ea147bc24db98d9686e457c124dfd5b7
2024-09-12 14:54:46 +01:00
Trevor Hilton 68ea7fc428
feat: partition buffer chunks from the table buffer (#25304) 2024-09-10 14:22:59 -04:00
Trevor Hilton ad2ca83d72
chore: sync to latest core (#25284) 2024-09-06 13:49:38 -04:00
Michael Gattozzi fb9d7d02f3
fix: test failure due to global static (#25287)
This commit changes the write_buffer tests to acquire a lock so that
in tests where we need to have access to NEXT_FILE_ID that it won't
be overwritten since rust tests run as one process and share the
same statics.

While this isn't a problem for Edge as a singular process it is for
our tests. It's a bit unfortunate, but this solution is the easiest
and the locks are not held for long so there's no real big impact
on running these tests.

Closes #25286
2024-09-05 13:53:38 -04:00
Trevor Hilton 4e664d3da5
chore: updates for pro (#25285)
This applies some needed updates downstream in Pro. Namely,
* visibility changes that allow types to be used in the pro buffer
* allow parsing a WAL file sequence number from a file path
* remove duplicates when adding parquet files to a persisted files list
2024-09-04 16:02:07 -04:00
dependabot[bot] cb76f7a63c
chore(deps): bump quinn-proto from 0.11.6 to 0.11.8 (#25280)
Bumps [quinn-proto](https://github.com/quinn-rs/quinn) from 0.11.6 to 0.11.8.
- [Release notes](https://github.com/quinn-rs/quinn/releases)
- [Commits](https://github.com/quinn-rs/quinn/compare/quinn-proto-0.11.6...quinn-proto-0.11.8)

---
updated-dependencies:
- dependency-name: quinn-proto
  dependency-type: indirect
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
2024-09-04 13:04:13 -04:00
Trevor Hilton cd23be6e5c
test: repro for dropped wal files during snapshot (#25276)
* test: repro for dropped wal files during snapshot

This commit provides a reproducer for an issue in the snapshotting process
whereby WAL files are removed for writes that have not been persisted yet.

* fix: do not snapshot most recent WAL period

This addresses #25277

Snapshots that are triggered when the number of WAL periods in the
tracker grows to be >= 3x the snapshot size will not include the most
recent wal period, and doing so was removing WAL files containing data
that was not yet persisted.

* docs: add doc comment to reproducer test

* fix: broken parquet_files system table test

* fix: broken snapshot_tracker test

* fix: broken write_buffer test

* refactor: remove redundant helper function

* test: add another snapshot test to write_buffer

* test: future writes do not get dropped on restart
2024-09-03 20:21:33 -04:00
Trevor Hilton 3f7b0e655c
refactor: move catalog and last cache initialization out of write buffer (#25273)
* refactor: add catalog as dep to influxdb3

* refactor: move catalog and last cache initialization out of write buffer

The Write buffer used to handle initialization of the catalog and last
n value cache. This commit moves that logic out, so that both can be
initialized independently, and injected into the write buffer. This is to
enable downstream changes that will need to make sharing the catalog and
last cache possible.
2024-08-27 16:41:40 -04:00
Trevor Hilton 734fd2d19a
chore: remove check for VERSION_HASH in build.rs (#25271) 2024-08-26 12:20:38 -04:00
Trevor Hilton e0e0075766
refactor: use more `dyn Trait` in write buffer (#25264)
* refactor: use dyn traits in WriteBufferImpl

This changes the WriteBufferImpl to use a dyn TimeProvider instead of
a generic in its type signature.

The Server type now uses a dyn WriteBuffer instead of using a generic
in its type signature, and the ServerBuilder was updated to accommodate
this accordingly.

These chages were to make downstream code changes more seamless.

* refactor: make some items pub

This makes functions on the QueryableBuffer and LastCache pub so that they
can be used downstream.
2024-08-23 14:21:20 -04:00
Trevor Hilton cbb7bc5901
refactor: remove Persister trait in favour of concrete impl (#25260)
The Persister trait was only implemented by a single type, because the
underlying ObjectStore interface has several ways of being mocked, we
mock that instead of the Persister interface.

This commit removes the Persister trait, and moves its interface/impl
directly on a single Persister type in the persister module of the
influxdb3_write crate.

deny.toml had some incorrect field names in license.exceptions, those
were fixed from 'crate' to 'name'.
2024-08-22 10:41:33 -04:00
Michael Gattozzi 0fec72d243
feat: Add u64 id field to ParquetFiles (#25258)
* feat: Add u64 id field to ParquetFiles

This commit does a few things:
1. It adds a u64 id field to ParquetFile
2. It gets this from an AtomicU64 so they can always use the most up to
   date one across threads
3. The last available file id is persisted as part of our snapshot
   process
4. The snapshot when loaded will set the AtomicU64 to the proper value

With this current work on the FileIndex in our Pro version will be able
to utilize these ids while doing compaction and we can refer to them
with a unique u64.

Closes #25250
2024-08-21 14:14:33 -04:00
Trevor Hilton 3b174a2f98
feat: snapshots track their own sequence number (#25255) 2024-08-20 15:55:47 -07:00
Trevor Hilton d0720a4fe4
chore: core sync and add reqwest json feature (#25251)
* chore: core sync and add reqwest json feature

* chore: update to latest commit sha on core branch
2024-08-19 14:40:04 -04:00
Trevor Hilton 1cc5af438d
chore: ignore sqlx rustsec advisory (#25252) 2024-08-19 11:15:43 -04:00
Paul Dix d9cb3a58c5
feat: Catalog apply_catalog_batch only updates if new (#25236)
* feat: Catalog apply_catalog_batch only updates if new

This updates the Catalog so that when applying a catalog batch it only updates the inner catalog and bumps the sequence number and updated tracker if there are new updates in the batch. Also does validation that the catalog batch schema is compatible with any existing.

Closes #25205

* feat: only persist catalog when updated (#25238)
2024-08-12 10:21:34 -04:00
Paul Dix 8bcc7522d0
feat: Add last cache create/delete to WAL (#25233)
* feat: Add last cache create/delete to WAL

This moves the LastCacheDefinition into the WAL so that it can be serialized there. This ended up being a pretty large refactor to get the last cache creation to work through the WAL.

I think I also stumbled on a bug where the last cache wasn't getting initialized from the catalog on reboot so that it wouldn't actually end up caching values. The refactored last cache persistence test in write_buffer/mod.rs surfaced this.

Finally, I also had to update the WAL so that it would persist if there were only catalog updates and no writes.

Fixes #25203

* fix: typos
2024-08-09 05:46:35 -07:00
Trevor Hilton 2082135077
fix: main (#25231) 2024-08-08 11:56:09 -04:00
Paul Dix 05ab730ae6
refactor: Make Level0Duration part of WAL (#25228)
* refactor: Make Level0Duration part of WAL

I noticed this during some testing and cleanup with other PRs. The WAL had its own level_0_duration and the write buffer had a different one, which would cause some weird problems if they weren't the same. This refactors Level0Duration to be in the WAL and fixes up the tests.

As an added bonus, this surfaced a bug where multiple L0 blocks getting persisted in the same snapshot wasn't supported. So now snapshot details can have many files per table.

* fix: have persisted files always return in descending data time order

* fix: sort record batches for test verification
2024-08-08 09:47:21 -04:00
Trevor Hilton 4067c91be0
fix: un-pub QueryableBuffer and fix compile errors (#25230) 2024-08-08 09:39:12 -04:00
Trevor Hilton 7474c0b3b4
feat: add `system.parquet_files` table (#25225)
This extends the system tables available with a new `parquet_files` table
which will list the parquet files associated with a given table in a
database.

Queries to system.parquet_files must provide a table_name predicate to
specify the table name of interest.

The files are accessed through the QueryableBuffer.

In addition, a test was added to check success and failure modes of the
new system table query.

Finally, the Persister trait had its associated error type removed. This
was somewhat of a consequence of how I initially implemented this change,
but I felt cleaned the code up a bit, so I kept it in the commit.
2024-08-08 08:46:26 -04:00
Trevor Hilton b0beab5b0c
feat: use host identifier prefix in object store paths (#25224)
This enforces the use of a host identifier prefix in all object store
paths (currently, for parquet files, catalog files, and snapshot files).

The persister retains the host identifier prefix, and uses it when
constructing paths.

The WalObjectStore also holds the host identifier prefix, so that it can
use it when saving and loading WAL files.

The influxdb3 binary requires a new argument 'host-id' to be passed that
is used to specify the prefix.
2024-08-07 16:23:36 -04:00
Paul Dix 43877beb15
fix: query bugs with buffer (#25213)
* fix: query bugs with buffer

This fixes three different bugs with the buffer. First was that aggregations would fail because projection was pushed down to the in-buffer data that de-duplication needs to be called on. The test in influxdb3/tests/server/query.rs catches that.

I also added a test in write_buffer/mod.rs to ensure that data is correctly queryable when combining with different states: only data in buffer, only data in parquet files, and data across both. This showed two bugs, one where the parquet data was being doubled up (parquet chunks were being created in write buffer mod and in queryable buffer. The second was that the timestamp min max on table buffer would panic if the buffer was empty.

* refactor: PR feedback

* fix: fix wal replay and buffer snapshot

Fixes two problems uncovered by adding to the write_buffer/mod.rs test. Ensures we can replay wal data and that snapshots work properly with replayed data.

* fix: run cargo update to fix audit
2024-08-07 16:00:17 -04:00
Michael Gattozzi 29d3a28a9c
fix: make ParquetChunk fields and mod chunk pub (#25219)
* fix: make ParquetChunk fields and mod chunk pub

This doesn't affect anything in the OSS version, but these changes are
needed for Pro as part of our compactor work.

* fix: cargo deny failure
2024-08-06 15:07:14 -04:00
Paul Dix 6aa6d924c6
fix: wal skip persist and notify if empty buffer (#25211)
* fix: wal skip persist and notify if empty buffer

This fixes the WAL so that it will skip persisting a file and notifying the file notifier if the wal buffer is empty.

* fix: fix last cache persist test
2024-08-05 18:08:11 -04:00
Paul Dix 2b8fc7b44e
refactor: Move Catalog into influxdb3_catalog crate (#25210)
* refactor: Move Catalog into influxdb3_catalog crate

This moves the catalog and its serialization logic into its own crate. This is a precursor to recording more catalog modifications into the WAL.

Fixes #25204

* fix: cargo update

* fix: add version = 2 to deny.toml

* fix: update deny.toml

* fix: add CCO to deny.toml
2024-08-02 16:04:12 -04:00
Paul Dix 3265960010
refactor: implement new wal and refactor write buffer (#25196)
* feat: refactor WAL and WriteBuffer

There is a ton going on here, but here are the high level things. This implements a new WAL, which is backed entirely by object store. It then updates the WriteBuffer to be able to work with how the new WAL works, which also required an update to how the Catalog is modified and persisted.

The concept of Segments has been removed. Previously there was a separate WAL per segment of time. Instead, there is now a single WAL that all writes and updates flow into. Data within the write buffer is organized by Chunk(s) within tables, which is based on the timestamp of the row data. These are known as the Level0 files, which will be persisted as Parquet into object store. The default chunk duration for level 0 files is 10 minutes.

The WAL is written as single files that get created at the configured WAL flush interval (1s by default). After a certain number of files have been created, the server will attempt to snapshot the WAL (default is to snapshot the first 600 files of the WAL after we have 900 total, i.e. snapshot 10 minutes of WAL data).

The design goal with this is to persist 10 minute chunks of data that are no longer receiving writes, while clearing out old WAL files. This works if data getting written in around "now" with no more than 5 minutes of delay. If we continue to have delayed writes, a snapshot of all data will be forced in order to clear out the WAL and free up memory in the buffer.

Overall, this structure of a single wal, with flushes and snapshots and chunks in the queryable buffer led to a simpler setup for the write buffer overall. I was able to clear out quite a bit of code related to the old segment organization.

Fixes #25142 and fixes #25173

* refactor: address PR feedback

* refactor: wal to replay and background flush on new

* chore: remove stray println
2024-08-01 15:04:15 -04:00
Trevor Hilton f472d9d241
chore: update object_store to 0.10.2 (#25195) 2024-07-26 10:14:19 -04:00
Trevor Hilton 8c1a1418b2
test: add a test to check last cache init from catalog (#25192) 2024-07-26 09:58:02 -04:00
Michael Gattozzi 05a8a7da43
chore: Upgrade to rustc 1.80 (#25193)
This commit updates us to rustc 1.80. There are three significant changes
here:

1. LazyLock and LazyCell have been stabilized meaning we can replace our
   usage of Lazy from the once_cell crate with the std lib versions
2. Lints were added to handle unknown cfg directives. `tokio_unstable`
   is affected by this and while we do have the flags in our
   .cargo/config.toml Cargo still output a lint for it so we supress
   that warning now in our Cargo.toml for the workspace
3. clippy now throws a new warning about priority levels for lints. It's
   quite frankly a thing that doesn't make sense to me and should be
   something cargo fixes, but here we are.

Besides that it was a painless upgrade and now we're on the latest and
greatest.
2024-07-25 11:38:18 -04:00
Trevor Hilton e25e811d2b
docs: `PROFILING.md` (#25075)
Part of #25067

Changes in this PR:

Addition of a PROFILING.md file, which briefly outlines how to build the influxdb3 binary in preparation for profiling and explains usage of macOS's Instruments tool
Addition of a quick-bench profile, which extends the already existing quick-release profile with debuginfo turned on
2024-07-24 11:01:36 -04:00
Trevor Hilton 10dd22b6de
fix: last cache catalog configuration tracks explicit vs. non-explicit value columns (#25185)
* fix: catalog support for last caches that accept new fields

Last cache definitions in the catalog were augmented to either store an
explicit set of column names (including time), or to accept new fields.

This will allow these caches to be loaded properly on server restart such
that all non-key columns are cached.

* refactor: use tagged serialization for last cache values def

This also updated the client code to accept the new structure in
influxdb3_client.

* test: add e2e tests to catch regressions in influxdb3_client

* chore: cargo update for audit
2024-07-24 11:00:40 -04:00
Trevor Hilton dfecf570e6
feat: support `!=`, `IN`, and `NOT IN` predicates in last cache queries (#25175)
Part of #25174

This PR adds support for three more predicate types when querying the last cache: !=, IN, and NOT IN. Previously only = was supported.

Existing tests were extended to check that these predicate types work as expected, both in the last_cache module and in the influxdb3_server crate. The latter was important to ensure that the new predicate logic works in the context of actual query parsing/execution.
2024-07-23 14:17:09 -04:00
Trevor Hilton 7a7db7d529
feat: connect `LastCacheProvider` with catalog at runtime (#25170)
Closes #25169

This PR ensures the last cache configuration is persisted to the catalog when last caches are created, and are removed from the catalog when they are deleted. The last cache is initialized on server start fro the catalog.

A new trait was added to the write buffer: LastCacheManager, which provides the methods to create and delete last caches (and which is invoked from the HTTP API). Both create/delete methods will update the catalog, but also force persistence of the catalog to object store, vs. waiting for the WAL flush interval / segment persistence process to do it. This should ensure that the catalog is up-to-date with respect to the last cache configuration, in the event that the server is stopped before segment persistence.

A test was added to check this behaviour in influxdb3_write/src/write_buffer/mod.rs.
2024-07-23 12:41:42 -04:00
Trevor Hilton 9b9699da60
feat: CLI to manage last caches (#25168)
* feat: new last-cache CLI

This adds two new CLIs:

influxdb3 last-cache create
influxdb3 last-cache delete

These utilize the new underlying APIs/client methods for the last-n-value
cache feature.

* refactor: switch around the token CLI to new convention

* docs: re-word CLI docs
2024-07-17 11:33:58 -04:00
Trevor Hilton 6c8a3e4e34
feat: add methods to create and delete last caches to `influxdb3_client` (#25167)
* feat: add create last cache method to client

* feat: add delete last cache method to client

* docs: add doc comment to client method for last cache create

* test: create and delete last cache client methods
2024-07-17 09:34:36 -04:00
Trevor Hilton 7752d03a79
feat: `last_caches` system table (#25166)
Added a new system table, system.last_caches, to enable queries that display information about last caches in a database.

You can query the table like so:

SELECT * FROM system.last_caches

Since queries are scoped to a database, this will only show last caches configured for the database being queried.

Results look like so:

+-------+----------------+----------------+---------------+-------+-----+
| table | name           | key_columns    | value_columns | count | ttl |
+-------+----------------+----------------+---------------+-------+-----+
| mem   | mem_last_cache | [host, region] | [time, usage] | 1     | 60  |
+-------+----------------+----------------+---------------+-------+-----+

An end-to-end test was added to verify queries to the system.last_caches table.
2024-07-17 09:14:51 -04:00
Trevor Hilton e8d9b02818
feat: `DELETE` last cache API (#25162)
Adds an API for deleting last caches.
- The API allows parameters to be passed in either the request URI query string, or in the body as JSON
- Some additional error modes were handled, specifically, for better HTTP status code responses, e.g., invalid content type is now a 415, URL query string parsing errors are now 400
- An end-to-end test was added to check behaviour of the API
2024-07-16 10:57:48 -04:00
Trevor Hilton 56488592db
feat: API to create last caches (#25147)
Closes #25096

- Adds a new HTTP API that allows the creation of a last cache, see the issue for details
- An E2E test was added to check success/failure behaviour of the API
- Adds the mime crate, for parsing request MIME types, but this is only used in the code I added - we may adopt it in other APIs / parts of the HTTP server in future PRs
2024-07-16 10:32:26 -04:00
Trevor Hilton 0279461738
feat: hook up last cache to query executor using DataFusion traits (#25143)
* feat: impl datafusion traits on last cache

Created a new module for the DataFusion table function implementations.
The TableProvider impl for LastCache was moved there, and new code that
implements the TableFunctionImpl trait to make the last cache queryable
was also written.

The LastCacheProvider and LastCache were augmented to make this work:
- The provider stores an Arc<LastCache> instead of a LastCache
- The LastCache uses interior mutability via an RwLock, to make the above
  possible.

* feat: register last_cache UDTF on query context

* refactor: make server accept listener instead of socket addr

The server used to accept a socket address and bind it directly, returning
error if the bind fails.

This commit changes that so the ServerBuilder accepts a TcpListener. The
behaviour is essentially the same, but this allows us to bind the address
from tests when instantiating the server, so we can easily assign unused
ports.

Tests in the influxdb3_server were updated to exploit this in order to
use port 0 auto assignment and stop flaky test failures.

A new, failing, test was also added to that module for the last cache.

* refactor: naive implementation of last cache key columns

Committing here as the last cache is in a working state, but it is naively
implemented as it just stores all key columns again (still with the hierarchy)

* refactor: make the last cache work with the query executor

* chore: fix my own feedback and appease clippy

* refactor: remove lower lock in last cache

* chore: cargo update

* refactor: rename function

* fix: broken doc comment
2024-07-16 10:10:47 -04:00
Trevor Hilton 0b8fbf456c
refactor: improvements to last cache implementation (#25133)
* refactor: make cache creation more idempotent

Last Cache creation is more idempotent, if a cache is created, and then
an attempt to create it again with the same parameters is used, it will
not result in an error.

* refactor: only store a single buffer of Instants

The last cache column buffers were storing an instant next to each
buffered value, which is unnecessary and not space efficient. This makes
it so the LastCacheStore holds a single buffer of Instants and manages
TTLs using that.

* refactor: clean up derelict cache members on eviction
2024-07-10 13:45:24 -04:00
Trevor Hilton 8fd50cefe1
chore: sync latest core (#25138)
* chore: sync latest core

* chore: clippy
2024-07-10 12:25:09 -04:00
wayne cd6734a7c4
chore: remove unused dependencies ioxd_common and test_helpers_end_to_end (#25134)
Co-authored-by: Trevor Hilton <thilton@influxdata.com>
2024-07-10 09:17:54 -06:00
Trevor Hilton 2609b590c9
feat: support addition of newly written columns to last cache (#25125)
* feat: support last caches that can add new fields

* feat: support new values in last cache

Support the addition of new fields to the last cache, for caches that do
not have a specified set of value columns.

A test was added along with the changes.

* chore: clippy

* docs: add comments throughout new last cache code

* fix: last cache schema merging when new fields added

* refactor: use outer schema for RecordBatch production

Enabling addition of new fields to a last cache made the insertion order
guarantee of the IndexMap break down. It could not be relied upon anymore
so this commit removes reference to that fact, despite still using the
IndexMap type, and strips out the schema from the inner LastCacheStore
type of the LastCache.

Now, the outer LastCache schema is relied on for producing RecordBatches,
which requires a lookup to the inner LastCacheStore's HashMap for each
field in the schema. This may not be as convenient as iterating over the
map as before, but trying to manage the disparate schema, and maintaining
the map ordering was making the code too complicated. This seems like a
reasonable compromise for now, until we see the need to optimize.

The IndexMap is still used for its fast iteration and lookup
characteristics.

The test that checks for new field ordering behaviour was modified to be
correct.

* refactor: use has set instead of scanning entire row on each push

Some renaming of variables was done to clarify meaning as well.
2024-07-09 16:35:27 -04:00
Trevor Hilton 53e5c5f5c5
feat: last cache implementation (#25109)
* feat: base for last cache implementation

Each last cache holds a ring buffer for each column in an index map, which
preserves the insertion order for faster record batch production.

The ring buffer uses a custom type to handle the different supported
data types that we can have in the system.

* feat: implement last cache provider

LastCacheProvider is the API used to create last caches and write
table batches to them. It uses a two-layer RwLock/HashMap: the first for
the database, and the second layer for the table within the database.

This allows for table-level locks when writing in buffered data, and only
gets a database-level lock when creating a cache (and in future, when
removing them as well).

* test: APIs on write buffer and test for last cache

Added basic APIs on the write buffer to access the last cache and then a
test to the last_cache module to see that it works with a simple example

* docs: add some doc comments to last_cache

* chore: clippy

* chore: one small comment on IndexMap

* chore: clean up some stale comments

* refactor: part of PR feedback

Addressed three parts of PR feedback:

1. Remove double-lock on cache map
2. Re-order the get when writing to the cache to be outside the loop
3. Move the time check into the cache itself

* refactor: nest cache by key columns

This refactors the last cache to use a nested caching structure, where
the key columns for a given cache are used to create a hierarchy of
nested maps, terminating in the actual store for the values in the cache.

Access to the cache is done via a set of predicates which can optionally
specify the key column values at any level in the cache hierarchy to only
gather record batches from children of that node in the cache.

Some todos:
- Need to handle the TTL
- Need to move the TableProvider impl up to the LastCache type

* refactor: TableProvider impl to LastCache

This re-writes the datafusion TableProvider implementation on the correct
type, i.e., the LastCache, and adds conversion from the filter Expr's to
the Predicate type for the cache.

* feat: support TTL in last cache

Last caches will have expired entries walked when writes come in.

* refactor: add panic when unexpected predicate used

* refactor: small naming convention change

* refactor: include keys in query results and no null keys

Changed key columns so that they do not accept null values, i.e., rows
that are pushed that are missing key column values will be ignored.

When producing record batches for a cache, if not all key columns are
used in the predicate, then this change makes it so that the non-predicate
key columns are produced as columns in the outputted record batches.

A test with a few cases showing this was added.

* fix: last cache key column query output

Ensure key columns in the last cache that are not included in the
predicate are emitted in the RecordBatches as a column.

Cleaned up and added comments to the new test.

* chore: clippy and some un-needed code

* fix: clean up some logic errors in last_cache

* test: add tests for non default cache size and TTL

Added two tests, as per commit title. Also moved the eviction process
to a separate function so that it was not being done on every write to
the cache, which could be expensive, and this ensures that entries are
evicted regardless of whether writes are coming in or not.

* test: add invalid predicate test cases to last_cache

* test: last_cache with field key columns

* test: last_cache uses series key for default keys

* test: last_cache uses tag set as default keys

* docs: add doc comments to last_cache

* fix: logic error in last cache creation

CacheAlreadyExists errors were only being based on the database and
table names, and not including the cache names, which was not
correct.

* docs: add some comments to last cache create fn

* feat: support null values in last cache

This also adds explicit support for series key columns to distinguish
them from normal tags in terms of nullability

A test was added to check nulls work

* fix: reset last cache last time when ttl evicts all data
2024-07-09 15:22:04 -04:00
damageboy 338019ea12
fix: restore windows build to working state (#25131) 2024-07-09 11:11:09 -04:00