This also switches the remaining iterators to be lazy so they can return
errors properly. They needed to be converted to lazy initialization
anyway, which has the side effect of making it much easier for us to
propagate the underlying error during initialization.
Updated the Emitter to return errors when it cannot read properly from
the iterators.
When a GROUP BY or multiple sources are used, the top level limit
iterator requires reading the entire iterator stream so it can find all
of the tag groups it needs to return. For large data series, this ends
up with the limit iterator discarding a lot of output.
This change adds a new lower level limit iterator on each series itself
so that there are fewer data points that have to be thrown away by the
top level iterator.
Fixes#5553.
Now it is possible to compare tags and fields and it is also now
possible to compare tags and tags. Previously, it was only possible to
compare fields with fields and tags with a string or a regex.
Fixes#3371.
This commit makes a number of performance improvements to
reduce allocations during query execution. Several objects
and buffers are now reused across the components to avoid
allocations.
Previously a simple `count(value)` query across 1M points
would require 26,000+ allocations. After the changes in
this commit that number has been reduced to 88.
Send nil values from the tsm1 cursor at the end of the cursor. After the
cursor reached tsm1, the `nextAt()` call would always return the default
value rather than a nil value.
Descending also didn't work correctly because the seeking functionality
for tsm1 iterators would always act like they were ascending instead of
descending when choosing which value to select. This resulted in very
strange output from the emitter since it couldn't figure out if it was
ascending or descending.
Fixes#6206.
Both Shard and Engine had the same reference to the measurementField map,
but they each protected it with their own locks. This causes a race when
write and queries are occurring because writes can add new fields to the
map while queries are reading from it.
The fix moves the ownership to the Engine and provides protected accessors
to that Shard now users. For the most parts, the access on shard were old
dead code.
Fixing the measurementFields map race created a new race on the internal
fields map. This is now unexported and protected via MeasurementFields
exported funcs.
Fixes#6188
Writing a key that exceeds the max key length could cause a panic
when reading a tsm file because the 2 bytes used for the key length
would not be enough to represent the actual key length.
The writer will now return an error if when trying to write a key
that is too large.
After adding type-switches to the tsm1 packages, the custom
implementation found in the conversion tool broke. This change uses
tsm1.NewValue() instead of a custom implementation.
This change also ensures that the tsm1.Value interface can only be
implemented internally to allow for the optimized type-switch based
encoding
When loading many shards concurrently they block trying to
acquire a write lock in the sync pool adding a new source of
contention. Since this code flow always needs to allocate a
buffer it's not really buying us much.
This commit adds a buffer for stats to be updated without
requiring a mutex lock/unlock on every point. The tradeoff
is that stats are not exactly precise. This works for our
use case because stats are only periodically checked.
If an OR was used, merging filters between different expressions would
not work correctly. If one of the sides had a set of series ids with a
condition and the other side had no series ids associated with the
expression, all of the series from the side with a condition would have
the condition ignored. Instead of defaulting a non-existant series
filter to true, it should just be false and the evaluation of the one
side that does exist should take care of determining if the series id
should be included or not. The AND condition used false correctly so did
not have to be changed.
If a tag did not exist and `!=` or `!~` were used, it would return false
even though the neither a field or a tag equaled those values. This has
now been modified to correctly return the correct series ids and the
correct condition.
Also fixed a panic that would occur when a tag caused a field access to
become unnecessary. The filter using the field access still got created
and used even though it was unnecessary, resulting in an attempted
access to a non-initialized map.
Fixes#5152 and a bunch of other miscellaneous issues.
After reading the initial buffer, ORDER BY desc would read the next
block into the buffer and only read the first element. It's because the
code that was copied from the ascending cursor wasn't modified correctly
to set the position to the last element in the buffer.
The buffer size has also been lowered from 1000 to 10 to match with the
ascending cursor for performance with limit queries.
Fixes#6055.
This commit adds an `IteratorStats` that holds aggregate
iterator processing information. A method is also added to
`Iterator` to return the stats:
Stats() influxql.IteratorStats
The remote iterators will also emit their stats in the point
stream upon first connection, on a given interval, and then
finally once the last point has been sent.
The TSM writer uses a bufio.Writer that needs to be flushed before
it's closed. If the flush fails for some reason, the error is not
handled by the defer and the compactor continues on as if all is good.
This can create files with truncated indexes or zero-length TSM files.
Fixes#5889
Use of the iterator is spread out into both `IteratorCreators` and
inside of the iterators themselves. Part of the interrupt must be
handled inside of the engine so it stops trying to emit points when an
interrupt is found and another part of the interrupt has to happen when
combining the iterators so it doesn't just start reading the next shard.
A deadlock occurs under write load if a backup is run in between the
time when a snapshot compactions has snapshotted the cache and successfully
written it to disk. The issus is that the second snapshot call will block
on the commit lock while it is holding the engine write lock. This causes
all writes to block as well as prevents the currently runnign snapshot
compaction from completing because it needs to acquire a read-lock.
This PR removes the commit lock and just returns an error if a snapshot is
in progress to all any locks being held to be released. The caller can determine
whether to retry or giveup.
Slices of tsm1.Value interfaces are only ever used with all the same
types, and the previous code would switch on the type returned from a
call to Value(), which allocated and returned an interface{} object for
the underlying value.
This change instead type-switches on the tsm1.Value object itself,
allowing it direct access to the underlying value field, eliminating the
unecessary allocations.
... by extracting the db/rp from the given path.
Now that the code has "standardized" on extracting db/rp this way, the
ShardLocation struct is no longer necessary and thus has been removed.
We're back on the previous style of passing the path and walPath to
NewShard.
The current go compiler at the tip of the go master (1d5001af) has a modified implementation of
testing.quick.Check that now generates nil slices as test data. (See: https://gophers.slack.com/archives/general/p14567053570110). The existing tests expect round tripping to work in this case
but it does not. So, in these cases we change the expectation to reflect actual behaviour.
This needs to be checked for reasonableness.
Previously, the for loop at the end of the method assumed that all entries
had been deduplicated, including the entry discovered in the snapshot.
However, this wasn't actually true. With this change, we make it true.
Signed-off-by: Jon Seymour <jon@wildducktheories.com>
Consider the write sequence: 6,1,snapshot,7,2.
The hot cache gets deduplicated, so is 2,7.
Now consider the test if 1 >= 2, this is false, so needSort is not set to true.
The problem is the implicit assumption that the snapshot is always sorted
by the time that merged() runs, but this may not be true.
Signed-off-by: Jon Seymour <jon@wildducktheories.com>
Previously, we needed a write lock on the cache because it was the
only lock we had available to guard updates to entry.values and
entry.needSort.
However, now we have a entry-scoped lock for this purpose, we don't
need the cache write lock for this purpose. Since merged() doesn't
modify the .store or the c.snapshot.sort, there is no need for
a write lock on the cache to protect the cache.
So, we don't need to escalate here - we simply rely on the entry lock
to protect the entries we are iterating over.
Signed-off-by: Jon Seymour <jon@wildducktheories.com>