There was a very small window where it was possible to deadlock during
the close of the Store. When closing, the Store waited on its Waitgroup
under a `Lock`. Naturally, all other goroutines must have been in a
position to call `Done` on the `Waitgroup` before the `Wait` call in
`Close` would return.
For the goroutine running the `monitorShards` method it was possible
that it would be unable to do this. Specifically, if the `monitorShards`
goroutine was jumping into the `t.C` case as the `Close()` goroutine was
acquiring the `Lock` then then `monitorShards` goroutine would be unable
to acquire the `RLock`. Since it would also be unable to progress around
its loop to jump into the `s.closing` case, it would be unable to call
`Done` on the `WaitGroup` and we would have a deadlock.
This was identified during an AppVeyor CI run, though I was unable to
reproduce this locally.
Previously we used the EngineOptions to determine which shard index
type we were using. However, these options are set once at runtime
initialisation. Therefore if you're running with TSI enabled but then
accessing a legacy database with the inmem index, TagValues would not
have taken advantage of the inmem index.
This change ensures we always check the actual index of the shard(s).
This commit adds time support to SHOW TAG VALUES. Time can be used as
both a lower and upper boundary. However, there are some caveats.
For the `inmem` index, filtering by time will still return all results
because the index data is shared across shards.
For the `tsi1` index, filtering by time will only work down to the shard
lever. Specifically, when querying by time all shards within that time
range will be used to generate the results.
When a meta query does not include a time component then it can be
answered exclusively by the index. This should result in a much faster
query execution that if the TSM engine was engaged.
This commit rewrites the following queries such that they make use
of the index where no time component is present:
- SHOW MEASUREMENTS
- SHOW SERIES
- SHOW TAG KEYS
- SHOW FIELD KEYS
Update support in the `toml` package for parsing human-readble byte sizes.
Supported size suffixes are "k" or "K" for kibibytes, "m" or "M" for
mebibytes, and "g" or "G" for gibibytes. If a size suffix isn't specified
then bytes are assumed.
In the config, `cache-max-memory-size` and `cache-snapshot-memory-size` are
now typed as `toml.Size` and support the new syntax.
* Fprint* functions
* No nakedness
* clarify panic messages
* spacing between case statements
* remove break in favor of return
* remove goto in favor of for { continue }
* batch cursors return slices of timestamps and values to reduce call
overhead. Significantly improved iteration.
* added CreateCursor API to Shard, Engine
* moved build*Cursor to code gen
* array has already been sized correctly
* eliminates bounds checking for each element access
* reduces decoding of 30,000,000 points via storage API from
584ms to 540ms on average
Fixes#8989 and #8633.
Previously when issuing commands involving a regex check, walking
through the tags keys/values on a measurement, using the measurement's
index, would be racy.
This commit adds a new `TagKeyValue` type that abstracts away the
multi-layer map we were using as an inverted index from tag keys and
values to series ids. With this abstraction we can also make concurrent
access to this inverted index goroutine safe.
Finally, this commit fixes a very old bug in the index which will affect
any query using a regex. Previously we would always check _every_ tag
against a regex for a measurement, even when we had found a match.
* Introduces EXPLAIN ANALYZE command, which
produces a detailed tree of operations used to
execute the query.
introduce context.Context to APIs
metrics package
* create groups of named measurements
* safe for concurrent access
tracing package
EXPLAIN ANALYZE implementation for OSS
Serialize EXPLAIN ANALYZE traces from remote nodes
use context.Background for tests
group with other stdlib packages
additional documentation and remove unused API
use influxdb/pkg/testing/assert
remove testify reference
If multiple tombstones exists for a series that ended up causing the
full data to be deleted, the blocks were not removed from the offsets
in the index. This causes the TSMReader to report that a key exist
but does not have any data.
During a compaction, every key should have at least one value. Since
this invariant was broken, the compaction aborted early and ends up
dropping all series keys that are lexigraphically greater than where
the breakage occured. This would cause data to be dropped during the
compaction.
This fixes a potential bug where the BlockIterator would skip blocks
if the underlying TSMReader had deletes on it concurrently. This
could possibly occur due to changes in 91eb9de3 that now use the
existing TSMReaders from the FileStore instead of creating new ones
during compaction.
TSI did not check that the max select series limit during planning
the same way that inmem did. This means that the limit could be
set but the planning of a high cardinality query would still OOM
the server. This fixes that limit as well as makes the query interruptible
during planning.
There was a race on the WaitGroup where we could end up calling Add
while another goroutine was still waiting. The functions were confusing
so they have been simplified a bit since the compactions goroutines
have been reworked a lot already.
The scheduling logic ended up favoring more backlogged shards
too much and would starved active, less backed up shards. This
occurred because the scheduling kicks in once a second. When it
runs, it schedules as many compactions as it can. A backed up shard
would end up having more compactions to run during the loop an would
generally get to schedule them more frequently.
This now allows each shard to try and schedule one compaction at a time
which provides a more balanced approach. At some point, we'll probably
want to more directly balanc the each shards backlog vs letting it happen
somewhat randomly.
Some files seem to get orphan behind higher levels. This causes
the compactions to get blocked as the lowere level files will not
get picked up by their lower level planners. This allows the full
plan to identify them and pull them into their plans.
This check doesn't make sense for high cardinality data as the files
typically get big and sparse very quickly. This causes a lot of extra
disk space to be used which is taken up by large indexes and sparse
data.
One shard might be able to run a compaction, but could fail to
limits being hit. This loop would continue indefinitely as the
same task would continue to be rescheduled.
With higher cardinality or larger series keys, the files can roll
over early which causes them to take longer to be compacted by higher
levels. This causes larger disk usage and higher numbers of tsm files
at times.
This changes the compaction scheduling to better utilize the available
cores that are free. Previously, a level was planned in its own goroutine
and would kick off a number of compactions groups. The problem with this
model was that if there were 4 groups, and 3 completed quickly, the planning
would be blocked for that level until the last group finished. If the compactions
at the prior level are running more quickly, a large backlog could accumlate.
This now moves the planning to a single goroutine that plans each level in
succession and starts as many groups as it can. When one group finishes,
the planning will start the next group for the level.
The fysncs due to large writes when writing to TSM files and the
WAL can eventually cause large pauses. Since we already buffer
writes, using synchronous IO reduces fsync latency by ensuring
the individiual writes hit disk. This spreads out the latecncy
across multiple writes better.
This commit adds a basic TSI versioning scheme, by adding a Version field
to an index's MANIFEST file.
Existing TSI indexes will not have this field present in their MANIFEST
files, and thus will be deemed incomatible with the current version.
Users with existing TSI indexes will be able to remove them, and convert the
resulting inmem indexes to the current version of a TSI index using the
influx_inspect tooling.
With higher cardinalities, the encoder pools where become a bottleneck.
This changes the snapshot compactions ot checkout one encoder of each
type and re-use it while writing the snapshots as opposed to repeatedly
checking it out and in.
This perioically re-allocates the cache store to avoid memory
fragmentation and gradual slow down of the store after repeated
deletes and inserts into the map.
This instructs the kernel that it can release memory used by mmap'd
TSM files when they are not actively being used. It the mappings are
use, the kernel will fault the pages back in. On linux, this causes
RES memory to drop immediately when run.
A cold shard that suddenly receives a lot of writes could get a very
big cache that takes a long time to snapshot or causes the cache
max memory limit to be hit more quickly. This re-enables the compactions
if necessary during writes so we don't have to wait for the shard monitor
goroutine to re-enable them.
Compactions would create their own TSMReaders for simplicity. With
very high cardinality compactions, creating the reader and indirectIndex
can start to use a significant amount of memory.
This changes the compactions to use a reader that is already allocated
and managed by the FileStore.
These are already sorted during compaction, so switch to sorting lazily
to avoid the CPU and allocations. This would only occur when using if
using the writer directly.
The directIndex used by the TSMWriter maintained a map of series keys
to index entries. When the index is written to the TSM file, the keys
are sorted and then written out in order.
The reason for this is because directIndex used to be the only index
and it was optimized more for reading. The reading has been replaced
by the indirectIndex so the map of keys ends up wasting space.
During compactions, the series keys (and index entries) are already sorted
so this change uses the sorting to avoid the map and sort when writing the
index. This reduces allocations and CPU usage quite a bit for larger cardinality
TSM files.
This leaves the slower compactions that create full blocks to only
the full compaction. This helps reduce CPU usage and memory while shards
are hot, but increases disk usage (reduced compression) slightly.
Deleting high cardinality series could take a very long time, cause
write timeouts as well as dead lock the process. This fixes these
issue to by changing the approach for cleaning up the indexes and
reducing lock contention.
The prior approach delete each series and updated every index (inmem)
during the delete. This was very slow and cause the index to be locked
while it items in a slice were removed one by one. This has been changed
to mark series as deleted and then rebuild the index asynchronously which
speeds up the process.
There was also a dead lock that could occur when deleing the field set.
Deleting the field set held a write lock and the function it invoked under
the lock could try to take a read lock on the field set. This would then
deadlock. This approach was also very slow and caused time out for writes.
It now uses faster approach that checks for the existing of the measurment
in the cache and filestore which does not take write locks.
It prints the statistics of each iterator that will access the storage
engine. For each access of the storage engine, it will print the number
of shards that will potentially be accessed, the number of files that
may be accessed, the number of series that will be created, the number
of blocks, and the size of those blocks.
This is used quite a bit to determine which fields are needed in a
condition. When the condition gets large, the memory usage begins to
slow it down considerably and it doesn't take care of duplicates.
There are several places in the code where comma-ok map retrieval was
being used poorly. Some were benign, like checking existence before
issuing an unconditional delete with no cleanup. Others were potentially
far more serious: assuming that if 'ok' was true, then the resulting
pointer retrieved from the map would be non-nil. `nil` is a perfectly
valid value to store in a map of pointers, and the comma-ok syntax is
meant for when membership is distinct from having a non-zero value.
There was only one or two cases that I saw that being used correctly for
maps of pointers.
If a large compaction was running and was aborted. It could would leave
some tmp files around for files that it had fully written. The current
active file was cleaned up, but already completed ones would not. This
would occur when a TSM file needed to rollover due to size.
* <type>FinalizerIterator sets a runtime finalizer and calls Close
when garbage collected. This will ensure any associated cursors
are closed and the associated TSM files released
* `query.Iterators#Merge` call could return an error and the inputs
would not be closed, causing a cursor leak
The OnReplace func ends up trying to acquire locks on MeasurementFields. When
its called via snapshotting, this can deadlock because the snapshotting goroutine
also holds an RLock on the engine. If a delete measurement calls is run at the
right time, it will lock the MeasurementFields and try to acquire a lock on the engine
to disable compactions. This creates a deadlock.
To fix this, the OnReplace callback is moved to a function param to allow only Replace
calls as part of a compaction to invoke it as opposed to both snapshotting and compactions.
Fixes#8713
The ConditionExpr function is more accurate because it parses the
condition and ensures that time conditions are actually used correctly.
That means that attempting to combine conditions with OR will not result
in the query silently pretending it's an AND and nested conditions work
correctly so there is only one way to read the query.
It also extracts the non-time conditions into a separate condition so we
can stop attempting to parse around the time conditions in lower layers
of the storage engine. This change does not remove those hacks, but a
following commit should be able to sanitize the condition and remove
them.
This change provides a clear separation between the query engine
mechanics and the query language so that the language can be parsed and
dealt with separate from the query engine itself.
Previously pseudo iterators could be created for meta data such
as series, measurement, and tag data. These iterators were created
at a higher level and lacked a lot of the power of the query engine.
This commit moves system iterators down to the series level and
supports the following:
- _name
- _seriesKey
- _tagKey
- _tagValue
- _fieldKey
These can be used as normal fields such as:
SELECT _seriesKey FROM cpu
This will return all the series keys for `cpu`.
The tag cardinality checks were run for all inmem shards. Since inmem
shards share the same index, a lot of the work is redundant. Inmem shards
also need to sort their measurmenet and tag keys which can be CPU intensive
with many shards or higher cardinality.
This changes the monitoring to just check one shard in each database which
should lower CPU usage due to excessive sorting. The longer term solution
is to use TSI which would not have this check or required sorting.
If there were multiple shards, drop measurement could update the index
and remove the measurement before the other shards ran their deletes.
This causes the later shards to not see any series to delete.
The fix is to all deleteSeries to handle the index delete which already
accounts for removing the measurement when it is fully removed from the
index.
The partiallyRead func didn't account for the initial values and would
return true for blocks that had not been read at all. This causes a
slower path during compactions that forces a block to be decoded when
it could just be merged as is without decoded. This causes compactions
to consume more CPU and run slower at times.
This switches all the interfaces that take string series key to
take a []byte. This eliminates many small allocations where we
convert between to two repeatedly. Eventually, this change should
propogate futher up the stack.
The refs map was to increment the file references one time each.
It doesn't hurt to increment them multiple times though.
We also do not need to copy the files slice as we are accessing it
under a read lock so it can't be changed.
tsdb.Shard and tsdb.Shards both implement tsdb.ShardGroup and neither
were tested within the tsdb package itself. This adds tests for those
methods which are used by the query engine.
There was a change to speed up deleting and dropping measurements
that executed the deletes in parallel for all shards at once. #7015
When TSI was merged in #7618, the series keys passed into Shard.DeleteMeasurement
were removed and were expanded lower down. This causes memory to blow up
when a delete across many shards occurs as we now expand the set of series
keys N times instead of just once as before.
While running the deletes in parallel would be ideal, there have been a number
of optimizations in the delete path that make running deletes serially pretty
good. This change just limits the concurrency of the deletes which keeps memory
more stable.
When snapshots and compactions are disabled, the check to see if
the compaction should be aborted occurs in between writing to the
next TSM file. If a large compaction is running, it might take
a while for the file to be finished writing causing long delays.
This now interrupts compactions while iterating over the blocks to
write which allows them to abort immediately.
* introduced UnsignedValue type
* leveraged existing int64 compression algorithms (RLE, Simple 8B)
* tsm and WAL can read and write UnsignedValue
* compaction is aware of UnsignedValue
* unsigned support to model, cursors and write points
NOTE: there is no support to create unsigned points, as the line
protocol has not been modified.
The sortedSeriesIds slice was not getting reset to 0 which caused
the same series ids to exist in the slice more than once. Since
the size of the slice never matched the size of the seriesID map,
it kept appendending to the slice and sorting it which cause multiple
cursor to get created for the same series.
Fixes#8531
There was a race in the WAL writeToLog and scheduleSync which could
lead to a writing goroutine blocking indefinitely on its syncErr channel.
The issue was that the clearing of the syncCount happenend after the
wal was unlock. If a goroutine was able to lock, write and call scheduleSync
before the existing scheduleSync goroutine returns and ran the defer to
clear the syncCount, then a new scheduleSync goroutine would not get started.
This left the writing goroutine block with nothing to signal it.
While in this state, a RLock on the engine was held. If a Lock was requested
on the engine during this time, all future writes and queries would block waiting
on the blocked wal writer.
The fix is to move the atomic clearing of syncCount before the Lock is released.
The min key was not used in OverlapsKeyRange which caused it to return
false when it should be true. This causes a bug where deletes would not
write tombstones for files that actually contained the data it was supposed
to delete.
The in-memory index can get out of sync when deletes and writes
to the same measurement are running concurrently. The index is
updated independently from data on disk and it's possible for the
index to unassign a shard when data still exists on disk. What happens
is that there are TSM files on disk, but the index does not know that
the series that exist in those files still are in the shard. Restarting
the server reloads the index and the data is visible again. From and
end user perspective, this can look like more data is deleted than should
have been or that deleted data re-appears after a restart or writes to the
shard occur again.
There isn't an easy way to resolve this since the index and storage
are not transactional resources and we cannot atomically commit or
rollback changes to both at once.
As a workaround, after new TSM files are installed, we refresh the
index with series keys that exist in the new tsm files as well as
any lingering data still in the cache. There is a small window of time
when the index may be missing series, but it will re-appear after the refresh
completes.
This adds a v3 format that is a gzip compressed version of the v2
format. It reduces the size of tombstone files substantially without
having to support a more feature rich file format for tombstones.
The monitor goroutine calls enable compactions every 10s to spin down
(or start up) goroutines for cold shards. This frequent Lock may be
causing lock contention for writes and queries which get blocked trying
to acquire an RLock.
The go RWMutex says that new RLock calls will block if there is a
pending Lock call that is blocked. Switching the common path to use
an RLock should avoid the Lock and reduce lock contention for writes
and queries.
Currently two write locks in `inmem` are obtained and then
manually unlocked at function exit points. However, we have
reports that the `inmem` index is hanging on a write lock and
cannot track the issue down to anything else besides a lock
that could have been left unlocked because of a panic.
This commit changes the two locks to always defer their unlocks
to prevent these hangs.
This fixes the case where log files are compacted out of order
and cause non-contiguous sets of index files to be compacted.
Previously, the compaction planner would fetch a list of index files
for each level and compact them in order starting with the oldest
ones. This can be a problem for level 1 because level 0 (log files)
are compacted individually and in some cases a log file can finish
compacting before older log files are finished compacting. This
causes there to be a gap in the list of level 1 files that is
ignored when fetching a list of index files.
Now, the planner reads the list of index files starting from the
oldest but stops once it hits a log file. This prevents that gap
from being ignored.
This check was previously in a different section of code which
was lost during a refactor to the new compaction strategy. The
compaction planning now makes a check to ensure at least two
files are available for compaction in a level.
WriteBlock was missing the check for the max series keys which allowed
series keys to be written that were larger than the 2 bytes allocated
to store their length. When this occurred, the TSM can fail to load.
The defer was never executed because the planning happens in a
long running goroutine that loops. The plans need to be released
immediately after applying them.