From 8189770c50165b62867327ad388f2c2ef237ab6f Mon Sep 17 00:00:00 2001 From: Aliaksandr Valialkin Date: Mon, 5 Dec 2022 15:15:00 -0800 Subject: [PATCH] all: add `-inmemoryDataFlushInterval` command-line flag for controlling the frequency of saving in-memory data to disk The main purpose of this command-line flag is to increase the lifetime of low-end flash storage with the limited number of write operations it can perform. Such flash storage is usually installed on Raspberry PI or similar appliances. For example, `-inmemoryDataFlushInterval=1h` reduces the frequency of disk write operations to up to once per hour if the ingested one-hour worth of data fits the limit for in-memory data. The in-memory data is searchable in the same way as the data stored on disk. VictoriaMetrics automatically flushes the in-memory data to disk on graceful shutdown via SIGINT signal. The in-memory data is lost on unclean shutdown (hardware power loss, OOM crash, SIGKILL). Updates https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3337 --- README.md | 77 +- app/victoria-metrics/main.go | 5 + app/vmstorage/main.go | 139 ++- docs/CHANGELOG.md | 28 + docs/README.md | 77 +- docs/Single-server-VictoriaMetrics.md | 77 +- lib/mergeset/inmemory_part.go | 34 + lib/mergeset/table.go | 925 +++++++++++----- lib/mergeset/table_test.go | 12 +- lib/storage/index_db_test.go | 4 +- lib/storage/inmemory_part.go | 34 + lib/storage/partition.go | 1290 ++++++++++++++--------- lib/storage/partition_search_test.go | 11 +- lib/storage/storage_test.go | 71 +- lib/storage/table.go | 7 +- lib/storage/table_search_test.go | 10 +- lib/storage/table_search_timing_test.go | 5 +- lib/storage/table_timing_test.go | 3 +- 18 files changed, 1833 insertions(+), 976 deletions(-) diff --git a/README.md b/README.md index 85c15a2e7..c19d003e3 100644 --- a/README.md +++ b/README.md @@ -1363,18 +1363,50 @@ It is recommended passing different `-promscrape.cluster.name` values to HA pair ## Storage -VictoriaMetrics stores time series data in [MergeTree](https://en.wikipedia.org/wiki/Log-structured_merge-tree)-like -data structures. On insert, VictoriaMetrics accumulates up to 1s of data and dumps it on disk to -`<-storageDataPath>/data/small/YYYY_MM/` subdirectory forming a `part` with the following -name pattern: `rowsCount_blocksCount_minTimestamp_maxTimestamp`. Each part consists of two "columns": -values and timestamps. These are sorted and compressed raw time series values. Additionally, part contains -index files for searching for specific series in the values and timestamps files. +VictoriaMetrics buffers the ingested data in memory for up to a second. Then the buffered data is written to in-memory `parts`, +which can be searched during queries. The in-memory `parts` are periodically persisted to disk, so they could survive unclean shutdown +such as out of memory crash, hardware power loss or `SIGKILL` signal. The interval for flushing the in-memory data to disk +can be configured with the `-inmemoryDataFlushInterval` command-line flag (note that too short flush interval may significantly increase disk IO). -`Parts` are periodically merged into the bigger parts. The resulting `part` is constructed -under `<-storageDataPath>/data/{small,big}/YYYY_MM/tmp` subdirectory. -When the resulting `part` is complete, it is atomically moved from the `tmp` -to its own subdirectory, while the source parts are atomically removed. The end result is that the source -parts are substituted by a single resulting bigger `part` in the `<-storageDataPath>/data/{small,big}/YYYY_MM/` directory. +In-memory parts are persisted to disk into `part` directories under the `<-storageDataPath>/data/small/YYYY_MM/` folder, +where `YYYY_MM` is the month partition for the stored data. For example, `2022_11` is the partition for `parts` +with [raw samples](https://docs.victoriametrics.com/keyConcepts.html#raw-samples) from `November 2022`. + +The `part` directory has the following name pattern: `rowsCount_blocksCount_minTimestamp_maxTimestamp`, where: + +- `rowsCount` - the number of [raw samples](https://docs.victoriametrics.com/keyConcepts.html#raw-samples) stored in the part +- `blocksCount` - the number of blocks stored in the part (see details about blocks below) +- `minTimestamp` and `maxTimestamp` - minimum and maximum timestamps across raw samples stored in the part + +Each `part` consists of `blocks` sorted by internal time series id (aka `TSID`). +Each `block` contains up to 8K [raw samples](https://docs.victoriametrics.com/keyConcepts.html#raw-samples), +which belong to a single [time series](https://docs.victoriametrics.com/keyConcepts.html#time-series). +Raw samples in each block are sorted by `timestamp`. Blocks for the same time series are sorted +by the `timestamp` of the first sample. Timestamps and values for all the blocks +are stored in [compressed form](https://faun.pub/victoriametrics-achieving-better-compression-for-time-series-data-than-gorilla-317bc1f95932) +in separate files under `part` directory - `timestamps.bin` and `values.bin`. + +The `part` directory also contains `index.bin` and `metaindex.bin` files - these files contain index +for fast block lookups, which belong to the given `TSID` and cover the given time range. + +`Parts` are periodically merged into bigger parts in background. The background merge provides the following benefits: + +* keeping the number of data files under control, so they don't exceed limits on open files +* improved data compression, since bigger parts are usually compressed better than smaller parts +* improved query speed, since queries over smaller number of parts are executed faster +* various background maintenance tasks such as [de-duplication](#deduplication), [downsampling](#downsampling) + and [freeing up disk space for the deleted time series](#how-to-delete-time-series) are performed during the merge + +Newly added `parts` either successfully appear in the storage or fail to appear. +The newly added `parts` are being created in a temporary directory under `<-storageDataPath>/data/{small,big}/YYYY_MM/tmp` folder. +When the newly added `part` is fully written and [fsynced](https://man7.org/linux/man-pages/man2/fsync.2.html) +to a temporary directory, then it is atomically moved to the storage directory. +Thanks to this alogrithm, storage never contains partially created parts, even if hardware power off +occurrs in the middle of writing the `part` to disk - such incompletely written `parts` +are automatically deleted on the next VictoriaMetrics start. + +The same applies to merge process — `parts` are either fully merged into a new `part` or fail to merge, +leaving the source `parts` untouched. VictoriaMetrics doesn't merge parts if their summary size exceeds free disk space. This prevents from potential out of disk space errors during merge. @@ -1383,24 +1415,10 @@ This increases overhead during data querying, since VictoriaMetrics needs to rea bigger number of parts per each request. That's why it is recommended to have at least 20% of free disk space under directory pointed by `-storageDataPath` command-line flag. -Information about merging process is available in [single-node VictoriaMetrics](https://grafana.com/dashboards/10229) -and [clustered VictoriaMetrics](https://grafana.com/grafana/dashboards/11176) Grafana dashboards. +Information about merging process is available in [the dashboard for single-node VictoriaMetrics](https://grafana.com/dashboards/10229) +and [the dashboard for VictoriaMetrics cluster](https://grafana.com/grafana/dashboards/11176). See more details in [monitoring docs](#monitoring). -The `merge` process improves compression rate and keeps number of `parts` on disk relatively low. -Benefits of doing the merge process are the following: - -* it improves query performance, since lower number of `parts` are inspected with each query -* it reduces the number of data files, since each `part` contains fixed number of files -* various background maintenance tasks such as [de-duplication](#deduplication), [downsampling](#downsampling) - and [freeing up disk space for the deleted time series](#how-to-delete-time-series) are performed during the merge. - -Newly added `parts` either appear in the storage or fail to appear. -Storage never contains partially created parts. The same applies to merge process — `parts` are either fully -merged into a new `part` or fail to merge. MergeTree doesn't contain partially merged `parts`. -`Part` contents in MergeTree never change. Parts are immutable. They may be only deleted after the merge -to a bigger `part` or when the `part` contents goes outside the configured `-retentionPeriod`. - See [this article](https://valyala.medium.com/how-victoriametrics-makes-instant-snapshots-for-multi-terabyte-time-series-data-e1f3fb0e0282) for more details. See also [how to work with snapshots](#how-to-work-with-snapshots). @@ -1723,9 +1741,10 @@ and [cardinality explorer docs](#cardinality-explorer). * VictoriaMetrics buffers incoming data in memory for up to a few seconds before flushing it to persistent storage. This may lead to the following "issues": - * Data becomes available for querying in a few seconds after inserting. It is possible to flush in-memory buffers to persistent storage + * Data becomes available for querying in a few seconds after inserting. It is possible to flush in-memory buffers to searchable parts by requesting `/internal/force_flush` http handler. This handler is mostly needed for testing and debugging purposes. * The last few seconds of inserted data may be lost on unclean shutdown (i.e. OOM, `kill -9` or hardware reset). + The `-inmemoryDataFlushInterval` command-line flag allows controlling the frequency of in-memory data flush to persistent storage. See [this article for technical details](https://valyala.medium.com/wal-usage-looks-broken-in-modern-time-series-databases-b62a627ab704). * If VictoriaMetrics works slowly and eats more than a CPU core per 100K ingested data points per second, @@ -2133,6 +2152,8 @@ Pass `-help` to VictoriaMetrics in order to see the list of supported command-li Uses '{measurement}' instead of '{measurement}{separator}{field_name}' for metic name if InfluxDB line contains only a single field -influxTrimTimestamp duration Trim timestamps for InfluxDB line protocol data to this duration. Minimum practical duration is 1ms. Higher duration (i.e. 1s) may be used for reducing disk space usage for timestamp data (default 1ms) + -inmemoryDataFlushInterval duration + The interval for guaranteed saving of in-memory data to disk. The saved data survives unclean shutdown such as OOM crash, hardware reset, SIGKILL, etc. Bigger intervals may help increasing lifetime of flash storage with limited write cycles (e.g. Raspberry PI). Smaller intervals increase disk IO load. Minimum supported value is 1s (default 5s) -insert.maxQueueDuration duration The maximum duration for waiting in the queue for insert requests due to -maxConcurrentInserts (default 1m0s) -logNewSeries diff --git a/app/victoria-metrics/main.go b/app/victoria-metrics/main.go index 16fc87405..770259cc8 100644 --- a/app/victoria-metrics/main.go +++ b/app/victoria-metrics/main.go @@ -29,6 +29,10 @@ var ( "equal to -dedup.minScrapeInterval > 0. See https://docs.victoriametrics.com/#deduplication and https://docs.victoriametrics.com/#downsampling") dryRun = flag.Bool("dryRun", false, "Whether to check only -promscrape.config and then exit. "+ "Unknown config entries aren't allowed in -promscrape.config by default. This can be changed with -promscrape.config.strictParse=false command-line flag") + inmemoryDataFlushInterval = flag.Duration("inmemoryDataFlushInterval", 5*time.Second, "The interval for guaranteed saving of in-memory data to disk. "+ + "The saved data survives unclean shutdown such as OOM crash, hardware reset, SIGKILL, etc. "+ + "Bigger intervals may help increasing lifetime of flash storage with limited write cycles (e.g. Raspberry PI). "+ + "Smaller intervals increase disk IO load. Minimum supported value is 1s") ) func main() { @@ -54,6 +58,7 @@ func main() { logger.Infof("starting VictoriaMetrics at %q...", *httpListenAddr) startTime := time.Now() storage.SetDedupInterval(*minScrapeInterval) + storage.SetDataFlushInterval(*inmemoryDataFlushInterval) vmstorage.Init(promql.ResetRollupResultCacheIfNeeded) vmselect.Init() vminsert.Init() diff --git a/app/vmstorage/main.go b/app/vmstorage/main.go index 1ec41a60a..2033386ad 100644 --- a/app/vmstorage/main.go +++ b/app/vmstorage/main.go @@ -100,7 +100,7 @@ func InitWithoutMetrics(resetCacheIfNeeded func(mrs []storage.MetricRow)) { storage.SetLogNewSeries(*logNewSeries) storage.SetFinalMergeDelay(*finalMergeDelay) storage.SetBigMergeWorkersCount(*bigMergeConcurrency) - storage.SetSmallMergeWorkersCount(*smallMergeConcurrency) + storage.SetMergeWorkersCount(*smallMergeConcurrency) storage.SetRetentionTimezoneOffset(*retentionTimezoneOffset) storage.SetFreeDiskSpaceLimit(minFreeDiskSpaceBytes.N) storage.SetTSIDCacheSize(cacheSizeStorageTSID.N) @@ -453,56 +453,80 @@ func registerStorageMetrics(strg *storage.Storage) { return 0 }) - metrics.NewGauge(`vm_active_merges{type="storage/big"}`, func() float64 { - return float64(tm().ActiveBigMerges) + metrics.NewGauge(`vm_active_merges{type="storage/inmemory"}`, func() float64 { + return float64(tm().ActiveInmemoryMerges) }) metrics.NewGauge(`vm_active_merges{type="storage/small"}`, func() float64 { return float64(tm().ActiveSmallMerges) }) - metrics.NewGauge(`vm_active_merges{type="indexdb"}`, func() float64 { - return float64(idbm().ActiveMerges) + metrics.NewGauge(`vm_active_merges{type="storage/big"}`, func() float64 { + return float64(tm().ActiveBigMerges) + }) + metrics.NewGauge(`vm_active_merges{type="indexdb/inmemory"}`, func() float64 { + return float64(idbm().ActiveInmemoryMerges) + }) + metrics.NewGauge(`vm_active_merges{type="indexdb/file"}`, func() float64 { + return float64(idbm().ActiveFileMerges) }) - metrics.NewGauge(`vm_merges_total{type="storage/big"}`, func() float64 { - return float64(tm().BigMergesCount) + metrics.NewGauge(`vm_merges_total{type="storage/inmemory"}`, func() float64 { + return float64(tm().InmemoryMergesCount) }) metrics.NewGauge(`vm_merges_total{type="storage/small"}`, func() float64 { return float64(tm().SmallMergesCount) }) - metrics.NewGauge(`vm_merges_total{type="indexdb"}`, func() float64 { - return float64(idbm().MergesCount) + metrics.NewGauge(`vm_merges_total{type="storage/big"}`, func() float64 { + return float64(tm().BigMergesCount) + }) + metrics.NewGauge(`vm_merges_total{type="indexdb/inmemory"}`, func() float64 { + return float64(idbm().InmemoryMergesCount) + }) + metrics.NewGauge(`vm_merges_total{type="indexdb/file"}`, func() float64 { + return float64(idbm().FileMergesCount) }) - metrics.NewGauge(`vm_rows_merged_total{type="storage/big"}`, func() float64 { - return float64(tm().BigRowsMerged) + metrics.NewGauge(`vm_rows_merged_total{type="storage/inmemory"}`, func() float64 { + return float64(tm().InmemoryRowsMerged) }) metrics.NewGauge(`vm_rows_merged_total{type="storage/small"}`, func() float64 { return float64(tm().SmallRowsMerged) }) - metrics.NewGauge(`vm_rows_merged_total{type="indexdb"}`, func() float64 { - return float64(idbm().ItemsMerged) + metrics.NewGauge(`vm_rows_merged_total{type="storage/big"}`, func() float64 { + return float64(tm().BigRowsMerged) + }) + metrics.NewGauge(`vm_rows_merged_total{type="indexdb/inmemory"}`, func() float64 { + return float64(idbm().InmemoryItemsMerged) + }) + metrics.NewGauge(`vm_rows_merged_total{type="indexdb/file"}`, func() float64 { + return float64(idbm().FileItemsMerged) }) - metrics.NewGauge(`vm_rows_deleted_total{type="storage/big"}`, func() float64 { - return float64(tm().BigRowsDeleted) + metrics.NewGauge(`vm_rows_deleted_total{type="storage/inmemory"}`, func() float64 { + return float64(tm().InmemoryRowsDeleted) }) metrics.NewGauge(`vm_rows_deleted_total{type="storage/small"}`, func() float64 { return float64(tm().SmallRowsDeleted) }) - - metrics.NewGauge(`vm_references{type="storage/big", name="parts"}`, func() float64 { - return float64(tm().BigPartsRefCount) + metrics.NewGauge(`vm_rows_deleted_total{type="storage/big"}`, func() float64 { + return float64(tm().BigRowsDeleted) }) - metrics.NewGauge(`vm_references{type="storage/small", name="parts"}`, func() float64 { + + metrics.NewGauge(`vm_part_references{type="storage/inmemory"}`, func() float64 { + return float64(tm().InmemoryPartsRefCount) + }) + metrics.NewGauge(`vm_part_references{type="storage/small"}`, func() float64 { return float64(tm().SmallPartsRefCount) }) - metrics.NewGauge(`vm_references{type="storage", name="partitions"}`, func() float64 { + metrics.NewGauge(`vm_part_references{type="storage/big"}`, func() float64 { + return float64(tm().BigPartsRefCount) + }) + metrics.NewGauge(`vm_partition_references{type="storage"}`, func() float64 { return float64(tm().PartitionsRefCount) }) - metrics.NewGauge(`vm_references{type="indexdb", name="objects"}`, func() float64 { + metrics.NewGauge(`vm_object_references{type="indexdb"}`, func() float64 { return float64(idbm().IndexDBRefCount) }) - metrics.NewGauge(`vm_references{type="indexdb", name="parts"}`, func() float64 { + metrics.NewGauge(`vm_part_references{type="indexdb"}`, func() float64 { return float64(idbm().PartsRefCount) }) @@ -531,11 +555,11 @@ func registerStorageMetrics(strg *storage.Storage) { return float64(idbm().CompositeFilterMissingConversions) }) - metrics.NewGauge(`vm_assisted_merges_total{type="storage/small"}`, func() float64 { - return float64(tm().SmallAssistedMerges) + metrics.NewGauge(`vm_assisted_merges_total{type="storage/inmemory"}`, func() float64 { + return float64(tm().InmemoryAssistedMerges) }) - metrics.NewGauge(`vm_assisted_merges_total{type="indexdb"}`, func() float64 { - return float64(idbm().AssistedMerges) + metrics.NewGauge(`vm_assisted_merges_total{type="indexdb/inmemory"}`, func() float64 { + return float64(idbm().AssistedInmemoryMerges) }) metrics.NewGauge(`vm_indexdb_items_added_total`, func() float64 { @@ -546,11 +570,8 @@ func registerStorageMetrics(strg *storage.Storage) { }) // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/686 - metrics.NewGauge(`vm_merge_need_free_disk_space{type="storage/small"}`, func() float64 { - return float64(tm().SmallMergeNeedFreeDiskSpace) - }) - metrics.NewGauge(`vm_merge_need_free_disk_space{type="storage/big"}`, func() float64 { - return float64(tm().BigMergeNeedFreeDiskSpace) + metrics.NewGauge(`vm_merge_need_free_disk_space`, func() float64 { + return float64(tm().MergeNeedFreeDiskSpace) }) metrics.NewGauge(`vm_pending_rows{type="storage"}`, func() float64 { @@ -560,34 +581,52 @@ func registerStorageMetrics(strg *storage.Storage) { return float64(idbm().PendingItems) }) - metrics.NewGauge(`vm_parts{type="storage/big"}`, func() float64 { - return float64(tm().BigPartsCount) + metrics.NewGauge(`vm_parts{type="storage/inmemory"}`, func() float64 { + return float64(tm().InmemoryPartsCount) }) metrics.NewGauge(`vm_parts{type="storage/small"}`, func() float64 { return float64(tm().SmallPartsCount) }) - metrics.NewGauge(`vm_parts{type="indexdb"}`, func() float64 { - return float64(idbm().PartsCount) + metrics.NewGauge(`vm_parts{type="storage/big"}`, func() float64 { + return float64(tm().BigPartsCount) + }) + metrics.NewGauge(`vm_parts{type="indexdb/inmemory"}`, func() float64 { + return float64(idbm().InmemoryPartsCount) + }) + metrics.NewGauge(`vm_parts{type="indexdb/file"}`, func() float64 { + return float64(idbm().FilePartsCount) }) - metrics.NewGauge(`vm_blocks{type="storage/big"}`, func() float64 { - return float64(tm().BigBlocksCount) + metrics.NewGauge(`vm_blocks{type="storage/inmemory"}`, func() float64 { + return float64(tm().InmemoryBlocksCount) }) metrics.NewGauge(`vm_blocks{type="storage/small"}`, func() float64 { return float64(tm().SmallBlocksCount) }) - metrics.NewGauge(`vm_blocks{type="indexdb"}`, func() float64 { - return float64(idbm().BlocksCount) + metrics.NewGauge(`vm_blocks{type="storage/big"}`, func() float64 { + return float64(tm().BigBlocksCount) + }) + metrics.NewGauge(`vm_blocks{type="indexdb/inmemory"}`, func() float64 { + return float64(idbm().InmemoryBlocksCount) + }) + metrics.NewGauge(`vm_blocks{type="indexdb/file"}`, func() float64 { + return float64(idbm().FileBlocksCount) }) - metrics.NewGauge(`vm_data_size_bytes{type="storage/big"}`, func() float64 { - return float64(tm().BigSizeBytes) + metrics.NewGauge(`vm_data_size_bytes{type="storage/inmemory"}`, func() float64 { + return float64(tm().InmemorySizeBytes) }) metrics.NewGauge(`vm_data_size_bytes{type="storage/small"}`, func() float64 { return float64(tm().SmallSizeBytes) }) - metrics.NewGauge(`vm_data_size_bytes{type="indexdb"}`, func() float64 { - return float64(idbm().SizeBytes) + metrics.NewGauge(`vm_data_size_bytes{type="storage/big"}`, func() float64 { + return float64(tm().BigSizeBytes) + }) + metrics.NewGauge(`vm_data_size_bytes{type="indexdb/inmemory"}`, func() float64 { + return float64(idbm().InmemorySizeBytes) + }) + metrics.NewGauge(`vm_data_size_bytes{type="indexdb/file"}`, func() float64 { + return float64(idbm().FileSizeBytes) }) metrics.NewGauge(`vm_rows_added_to_storage_total`, func() float64 { @@ -665,14 +704,20 @@ func registerStorageMetrics(strg *storage.Storage) { return float64(m().TimestampsBytesSaved) }) - metrics.NewGauge(`vm_rows{type="storage/big"}`, func() float64 { - return float64(tm().BigRowsCount) + metrics.NewGauge(`vm_rows{type="storage/inmemory"}`, func() float64 { + return float64(tm().InmemoryRowsCount) }) metrics.NewGauge(`vm_rows{type="storage/small"}`, func() float64 { return float64(tm().SmallRowsCount) }) - metrics.NewGauge(`vm_rows{type="indexdb"}`, func() float64 { - return float64(idbm().ItemsCount) + metrics.NewGauge(`vm_rows{type="storage/big"}`, func() float64 { + return float64(tm().BigRowsCount) + }) + metrics.NewGauge(`vm_rows{type="indexdb/inmemory"}`, func() float64 { + return float64(idbm().InmemoryItemsCount) + }) + metrics.NewGauge(`vm_rows{type="indexdb/file"}`, func() float64 { + return float64(idbm().FileItemsCount) }) metrics.NewGauge(`vm_date_range_search_calls_total`, func() float64 { diff --git a/docs/CHANGELOG.md b/docs/CHANGELOG.md index d1b5507a2..96694cc09 100644 --- a/docs/CHANGELOG.md +++ b/docs/CHANGELOG.md @@ -17,6 +17,34 @@ The following tip changes can be tested by building VictoriaMetrics components f **Update note 1:** this release drops support for direct upgrade from VictoriaMetrics versions prior [v1.28.0](https://github.com/VictoriaMetrics/VictoriaMetrics/releases/tag/v1.28.0). Please upgrade to `v1.84.0`, wait until `finished round 2 of background conversion` line is emitted to log by single-node VictoriaMetrics or by `vmstorage`, and then upgrade to newer releases. +**Update note 2:** this release splits `type="indexdb"` metrics into `type="indexdb/inmemory"` and `type="indexdb/file"` metrics. This may break old dashboards and alerting rules, which contain label filters on `{type="indexdb"}`. It is recommended upgrading to the latest available dashboards and alerting rules mentioned in [these docs](https://docs.victoriametrics.com/#monitoring). + +* FEATURE: add `-inmemoryDataFlushInterval` command-line flag, which can be used for controlling the frequency of in-memory data flush to disk. The data flush frequency can be reduced when VictoriaMetrics stores data to low-end flash device with limited number of write cycles (for example, on Raspberry PI). See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3337). +* FEATURE: expose additional metrics for `indexdb` and `storage` parts stored in memory and for `indexdb` parts stored in files (see [storage docs](https://docs.victoriametrics.com/#storage) for technical details): + * `vm_active_merges{type="storage/inmemory"}` - active merges for in-memory `storage` parts + * `vm_active_merges{type="indexdb/inmemory"}` - active merges for in-memory `indexdb` parts + * `vm_active_merges{type="indexdb/file"}` - active merges for file-based `indexdb` parts + * `vm_merges_total{type="storage/inmemory"}` - the total merges for in-memory `storage` parts + * `vm_merges_total{type="indexdb/inmemory"}` - the total merges for in-memory `indexdb` parts + * `vm_merges_total{type="indexdb/file"}` - the total merges for file-based `indexdb` parts + * `vm_rows_merged_total{type="storage/inmemory"}` - the total rows merged for in-memory `storage` parts + * `vm_rows_merged_total{type="indexdb/inmemory"}` - the total rows merged for in-memory `indexdb` parts + * `vm_rows_merged_total{type="indexdb/file"}` - the total rows merged for file-based `indexdb` parts + * `vm_rows_deleted_total{type="storage/inmemory"}` - the total rows deleted for in-memory `storage` parts + * `vm_assisted_merges_total{type="storage/inmemory"}` - the total number of assisted merges for in-memory `storage` parts + * `vm_assisted_merges_total{type="indexdb/inmemory"}` - the total number of assisted merges for in-memory `indexdb` parts + * `vm_parts{type="storage/inmemory"}` - the total number of in-memory `storage` parts + * `vm_parts{type="indexdb/inmemory"}` - the total number of in-memory `indexdb` parts + * `vm_parts{type="indexdb/file"}` - the total number of file-based `indexdb` parts + * `vm_blocks{type="storage/inmemory"}` - the total number of in-memory `storage` blocks + * `vm_blocks{type="indexdb/inmemory"}` - the total number of in-memory `indexdb` blocks + * `vm_blocks{type="indexdb/file"}` - the total number of file-based `indexdb` blocks + * `vm_data_size_bytes{type="storage/inmemory"}` - the total size of in-memory `storage` blocks + * `vm_data_size_bytes{type="indexdb/inmemory"}` - the total size of in-memory `indexdb` blocks + * `vm_data_size_bytes{type="indexdb/file"}` - the total size of file-based `indexdb` blocks + * `vm_rows{type="storage/inmemory"}` - the total number of in-memory `storage` rows + * `vm_rows{type="indexdb/inmemory"}` - the total number of in-memory `indexdb` rows + * `vm_rows{type="indexdb/file"}` - the total number of file-based `indexdb` rows * FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): improve [service discovery](https://docs.victoriametrics.com/sd_configs.html) performance when discovering big number of targets (10K and more). * FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): add `exported_` prefix to metric names exported by scrape targets if these metric names clash with [automatically generated metrics](https://docs.victoriametrics.com/vmagent.html#automatically-generated-metrics) such as `up`, `scrape_samples_scraped`, etc. This prevents from corruption of automatically generated metrics. See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3406). * FEATURE: [VictoriaMetrics cluster](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html): improve error message when the requested path cannot be properly parsed, so users could identify the issue and properly fix the path. Now the error message links to [url format docs](https://docs.victoriametrics.com/Cluster-VictoriaMetrics.html#url-format). See [this issue](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/3402). diff --git a/docs/README.md b/docs/README.md index dce732e89..1fe27c686 100644 --- a/docs/README.md +++ b/docs/README.md @@ -1364,18 +1364,50 @@ It is recommended passing different `-promscrape.cluster.name` values to HA pair ## Storage -VictoriaMetrics stores time series data in [MergeTree](https://en.wikipedia.org/wiki/Log-structured_merge-tree)-like -data structures. On insert, VictoriaMetrics accumulates up to 1s of data and dumps it on disk to -`<-storageDataPath>/data/small/YYYY_MM/` subdirectory forming a `part` with the following -name pattern: `rowsCount_blocksCount_minTimestamp_maxTimestamp`. Each part consists of two "columns": -values and timestamps. These are sorted and compressed raw time series values. Additionally, part contains -index files for searching for specific series in the values and timestamps files. +VictoriaMetrics buffers the ingested data in memory for up to a second. Then the buffered data is written to in-memory `parts`, +which can be searched during queries. The in-memory `parts` are periodically persisted to disk, so they could survive unclean shutdown +such as out of memory crash, hardware power loss or `SIGKILL` signal. The interval for flushing the in-memory data to disk +can be configured with the `-inmemoryDataFlushInterval` command-line flag (note that too short flush interval may significantly increase disk IO). -`Parts` are periodically merged into the bigger parts. The resulting `part` is constructed -under `<-storageDataPath>/data/{small,big}/YYYY_MM/tmp` subdirectory. -When the resulting `part` is complete, it is atomically moved from the `tmp` -to its own subdirectory, while the source parts are atomically removed. The end result is that the source -parts are substituted by a single resulting bigger `part` in the `<-storageDataPath>/data/{small,big}/YYYY_MM/` directory. +In-memory parts are persisted to disk into `part` directories under the `<-storageDataPath>/data/small/YYYY_MM/` folder, +where `YYYY_MM` is the month partition for the stored data. For example, `2022_11` is the partition for `parts` +with [raw samples](https://docs.victoriametrics.com/keyConcepts.html#raw-samples) from `November 2022`. + +The `part` directory has the following name pattern: `rowsCount_blocksCount_minTimestamp_maxTimestamp`, where: + +- `rowsCount` - the number of [raw samples](https://docs.victoriametrics.com/keyConcepts.html#raw-samples) stored in the part +- `blocksCount` - the number of blocks stored in the part (see details about blocks below) +- `minTimestamp` and `maxTimestamp` - minimum and maximum timestamps across raw samples stored in the part + +Each `part` consists of `blocks` sorted by internal time series id (aka `TSID`). +Each `block` contains up to 8K [raw samples](https://docs.victoriametrics.com/keyConcepts.html#raw-samples), +which belong to a single [time series](https://docs.victoriametrics.com/keyConcepts.html#time-series). +Raw samples in each block are sorted by `timestamp`. Blocks for the same time series are sorted +by the `timestamp` of the first sample. Timestamps and values for all the blocks +are stored in [compressed form](https://faun.pub/victoriametrics-achieving-better-compression-for-time-series-data-than-gorilla-317bc1f95932) +in separate files under `part` directory - `timestamps.bin` and `values.bin`. + +The `part` directory also contains `index.bin` and `metaindex.bin` files - these files contain index +for fast block lookups, which belong to the given `TSID` and cover the given time range. + +`Parts` are periodically merged into bigger parts in background. The background merge provides the following benefits: + +* keeping the number of data files under control, so they don't exceed limits on open files +* improved data compression, since bigger parts are usually compressed better than smaller parts +* improved query speed, since queries over smaller number of parts are executed faster +* various background maintenance tasks such as [de-duplication](#deduplication), [downsampling](#downsampling) + and [freeing up disk space for the deleted time series](#how-to-delete-time-series) are performed during the merge + +Newly added `parts` either successfully appear in the storage or fail to appear. +The newly added `parts` are being created in a temporary directory under `<-storageDataPath>/data/{small,big}/YYYY_MM/tmp` folder. +When the newly added `part` is fully written and [fsynced](https://man7.org/linux/man-pages/man2/fsync.2.html) +to a temporary directory, then it is atomically moved to the storage directory. +Thanks to this alogrithm, storage never contains partially created parts, even if hardware power off +occurrs in the middle of writing the `part` to disk - such incompletely written `parts` +are automatically deleted on the next VictoriaMetrics start. + +The same applies to merge process — `parts` are either fully merged into a new `part` or fail to merge, +leaving the source `parts` untouched. VictoriaMetrics doesn't merge parts if their summary size exceeds free disk space. This prevents from potential out of disk space errors during merge. @@ -1384,24 +1416,10 @@ This increases overhead during data querying, since VictoriaMetrics needs to rea bigger number of parts per each request. That's why it is recommended to have at least 20% of free disk space under directory pointed by `-storageDataPath` command-line flag. -Information about merging process is available in [single-node VictoriaMetrics](https://grafana.com/dashboards/10229) -and [clustered VictoriaMetrics](https://grafana.com/grafana/dashboards/11176) Grafana dashboards. +Information about merging process is available in [the dashboard for single-node VictoriaMetrics](https://grafana.com/dashboards/10229) +and [the dashboard for VictoriaMetrics cluster](https://grafana.com/grafana/dashboards/11176). See more details in [monitoring docs](#monitoring). -The `merge` process improves compression rate and keeps number of `parts` on disk relatively low. -Benefits of doing the merge process are the following: - -* it improves query performance, since lower number of `parts` are inspected with each query -* it reduces the number of data files, since each `part` contains fixed number of files -* various background maintenance tasks such as [de-duplication](#deduplication), [downsampling](#downsampling) - and [freeing up disk space for the deleted time series](#how-to-delete-time-series) are performed during the merge. - -Newly added `parts` either appear in the storage or fail to appear. -Storage never contains partially created parts. The same applies to merge process — `parts` are either fully -merged into a new `part` or fail to merge. MergeTree doesn't contain partially merged `parts`. -`Part` contents in MergeTree never change. Parts are immutable. They may be only deleted after the merge -to a bigger `part` or when the `part` contents goes outside the configured `-retentionPeriod`. - See [this article](https://valyala.medium.com/how-victoriametrics-makes-instant-snapshots-for-multi-terabyte-time-series-data-e1f3fb0e0282) for more details. See also [how to work with snapshots](#how-to-work-with-snapshots). @@ -1724,9 +1742,10 @@ and [cardinality explorer docs](#cardinality-explorer). * VictoriaMetrics buffers incoming data in memory for up to a few seconds before flushing it to persistent storage. This may lead to the following "issues": - * Data becomes available for querying in a few seconds after inserting. It is possible to flush in-memory buffers to persistent storage + * Data becomes available for querying in a few seconds after inserting. It is possible to flush in-memory buffers to searchable parts by requesting `/internal/force_flush` http handler. This handler is mostly needed for testing and debugging purposes. * The last few seconds of inserted data may be lost on unclean shutdown (i.e. OOM, `kill -9` or hardware reset). + The `-inmemoryDataFlushInterval` command-line flag allows controlling the frequency of in-memory data flush to persistent storage. See [this article for technical details](https://valyala.medium.com/wal-usage-looks-broken-in-modern-time-series-databases-b62a627ab704). * If VictoriaMetrics works slowly and eats more than a CPU core per 100K ingested data points per second, @@ -2134,6 +2153,8 @@ Pass `-help` to VictoriaMetrics in order to see the list of supported command-li Uses '{measurement}' instead of '{measurement}{separator}{field_name}' for metic name if InfluxDB line contains only a single field -influxTrimTimestamp duration Trim timestamps for InfluxDB line protocol data to this duration. Minimum practical duration is 1ms. Higher duration (i.e. 1s) may be used for reducing disk space usage for timestamp data (default 1ms) + -inmemoryDataFlushInterval duration + The interval for guaranteed saving of in-memory data to disk. The saved data survives unclean shutdown such as OOM crash, hardware reset, SIGKILL, etc. Bigger intervals may help increasing lifetime of flash storage with limited write cycles (e.g. Raspberry PI). Smaller intervals increase disk IO load. Minimum supported value is 1s (default 5s) -insert.maxQueueDuration duration The maximum duration for waiting in the queue for insert requests due to -maxConcurrentInserts (default 1m0s) -logNewSeries diff --git a/docs/Single-server-VictoriaMetrics.md b/docs/Single-server-VictoriaMetrics.md index 7a9908aab..ed764040b 100644 --- a/docs/Single-server-VictoriaMetrics.md +++ b/docs/Single-server-VictoriaMetrics.md @@ -1367,18 +1367,50 @@ It is recommended passing different `-promscrape.cluster.name` values to HA pair ## Storage -VictoriaMetrics stores time series data in [MergeTree](https://en.wikipedia.org/wiki/Log-structured_merge-tree)-like -data structures. On insert, VictoriaMetrics accumulates up to 1s of data and dumps it on disk to -`<-storageDataPath>/data/small/YYYY_MM/` subdirectory forming a `part` with the following -name pattern: `rowsCount_blocksCount_minTimestamp_maxTimestamp`. Each part consists of two "columns": -values and timestamps. These are sorted and compressed raw time series values. Additionally, part contains -index files for searching for specific series in the values and timestamps files. +VictoriaMetrics buffers the ingested data in memory for up to a second. Then the buffered data is written to in-memory `parts`, +which can be searched during queries. The in-memory `parts` are periodically persisted to disk, so they could survive unclean shutdown +such as out of memory crash, hardware power loss or `SIGKILL` signal. The interval for flushing the in-memory data to disk +can be configured with the `-inmemoryDataFlushInterval` command-line flag (note that too short flush interval may significantly increase disk IO). -`Parts` are periodically merged into the bigger parts. The resulting `part` is constructed -under `<-storageDataPath>/data/{small,big}/YYYY_MM/tmp` subdirectory. -When the resulting `part` is complete, it is atomically moved from the `tmp` -to its own subdirectory, while the source parts are atomically removed. The end result is that the source -parts are substituted by a single resulting bigger `part` in the `<-storageDataPath>/data/{small,big}/YYYY_MM/` directory. +In-memory parts are persisted to disk into `part` directories under the `<-storageDataPath>/data/small/YYYY_MM/` folder, +where `YYYY_MM` is the month partition for the stored data. For example, `2022_11` is the partition for `parts` +with [raw samples](https://docs.victoriametrics.com/keyConcepts.html#raw-samples) from `November 2022`. + +The `part` directory has the following name pattern: `rowsCount_blocksCount_minTimestamp_maxTimestamp`, where: + +- `rowsCount` - the number of [raw samples](https://docs.victoriametrics.com/keyConcepts.html#raw-samples) stored in the part +- `blocksCount` - the number of blocks stored in the part (see details about blocks below) +- `minTimestamp` and `maxTimestamp` - minimum and maximum timestamps across raw samples stored in the part + +Each `part` consists of `blocks` sorted by internal time series id (aka `TSID`). +Each `block` contains up to 8K [raw samples](https://docs.victoriametrics.com/keyConcepts.html#raw-samples), +which belong to a single [time series](https://docs.victoriametrics.com/keyConcepts.html#time-series). +Raw samples in each block are sorted by `timestamp`. Blocks for the same time series are sorted +by the `timestamp` of the first sample. Timestamps and values for all the blocks +are stored in [compressed form](https://faun.pub/victoriametrics-achieving-better-compression-for-time-series-data-than-gorilla-317bc1f95932) +in separate files under `part` directory - `timestamps.bin` and `values.bin`. + +The `part` directory also contains `index.bin` and `metaindex.bin` files - these files contain index +for fast block lookups, which belong to the given `TSID` and cover the given time range. + +`Parts` are periodically merged into bigger parts in background. The background merge provides the following benefits: + +* keeping the number of data files under control, so they don't exceed limits on open files +* improved data compression, since bigger parts are usually compressed better than smaller parts +* improved query speed, since queries over smaller number of parts are executed faster +* various background maintenance tasks such as [de-duplication](#deduplication), [downsampling](#downsampling) + and [freeing up disk space for the deleted time series](#how-to-delete-time-series) are performed during the merge + +Newly added `parts` either successfully appear in the storage or fail to appear. +The newly added `parts` are being created in a temporary directory under `<-storageDataPath>/data/{small,big}/YYYY_MM/tmp` folder. +When the newly added `part` is fully written and [fsynced](https://man7.org/linux/man-pages/man2/fsync.2.html) +to a temporary directory, then it is atomically moved to the storage directory. +Thanks to this alogrithm, storage never contains partially created parts, even if hardware power off +occurrs in the middle of writing the `part` to disk - such incompletely written `parts` +are automatically deleted on the next VictoriaMetrics start. + +The same applies to merge process — `parts` are either fully merged into a new `part` or fail to merge, +leaving the source `parts` untouched. VictoriaMetrics doesn't merge parts if their summary size exceeds free disk space. This prevents from potential out of disk space errors during merge. @@ -1387,24 +1419,10 @@ This increases overhead during data querying, since VictoriaMetrics needs to rea bigger number of parts per each request. That's why it is recommended to have at least 20% of free disk space under directory pointed by `-storageDataPath` command-line flag. -Information about merging process is available in [single-node VictoriaMetrics](https://grafana.com/dashboards/10229) -and [clustered VictoriaMetrics](https://grafana.com/grafana/dashboards/11176) Grafana dashboards. +Information about merging process is available in [the dashboard for single-node VictoriaMetrics](https://grafana.com/dashboards/10229) +and [the dashboard for VictoriaMetrics cluster](https://grafana.com/grafana/dashboards/11176). See more details in [monitoring docs](#monitoring). -The `merge` process improves compression rate and keeps number of `parts` on disk relatively low. -Benefits of doing the merge process are the following: - -* it improves query performance, since lower number of `parts` are inspected with each query -* it reduces the number of data files, since each `part` contains fixed number of files -* various background maintenance tasks such as [de-duplication](#deduplication), [downsampling](#downsampling) - and [freeing up disk space for the deleted time series](#how-to-delete-time-series) are performed during the merge. - -Newly added `parts` either appear in the storage or fail to appear. -Storage never contains partially created parts. The same applies to merge process — `parts` are either fully -merged into a new `part` or fail to merge. MergeTree doesn't contain partially merged `parts`. -`Part` contents in MergeTree never change. Parts are immutable. They may be only deleted after the merge -to a bigger `part` or when the `part` contents goes outside the configured `-retentionPeriod`. - See [this article](https://valyala.medium.com/how-victoriametrics-makes-instant-snapshots-for-multi-terabyte-time-series-data-e1f3fb0e0282) for more details. See also [how to work with snapshots](#how-to-work-with-snapshots). @@ -1727,9 +1745,10 @@ and [cardinality explorer docs](#cardinality-explorer). * VictoriaMetrics buffers incoming data in memory for up to a few seconds before flushing it to persistent storage. This may lead to the following "issues": - * Data becomes available for querying in a few seconds after inserting. It is possible to flush in-memory buffers to persistent storage + * Data becomes available for querying in a few seconds after inserting. It is possible to flush in-memory buffers to searchable parts by requesting `/internal/force_flush` http handler. This handler is mostly needed for testing and debugging purposes. * The last few seconds of inserted data may be lost on unclean shutdown (i.e. OOM, `kill -9` or hardware reset). + The `-inmemoryDataFlushInterval` command-line flag allows controlling the frequency of in-memory data flush to persistent storage. See [this article for technical details](https://valyala.medium.com/wal-usage-looks-broken-in-modern-time-series-databases-b62a627ab704). * If VictoriaMetrics works slowly and eats more than a CPU core per 100K ingested data points per second, @@ -2137,6 +2156,8 @@ Pass `-help` to VictoriaMetrics in order to see the list of supported command-li Uses '{measurement}' instead of '{measurement}{separator}{field_name}' for metic name if InfluxDB line contains only a single field -influxTrimTimestamp duration Trim timestamps for InfluxDB line protocol data to this duration. Minimum practical duration is 1ms. Higher duration (i.e. 1s) may be used for reducing disk space usage for timestamp data (default 1ms) + -inmemoryDataFlushInterval duration + The interval for guaranteed saving of in-memory data to disk. The saved data survives unclean shutdown such as OOM crash, hardware reset, SIGKILL, etc. Bigger intervals may help increasing lifetime of flash storage with limited write cycles (e.g. Raspberry PI). Smaller intervals increase disk IO load. Minimum supported value is 1s (default 5s) -insert.maxQueueDuration duration The maximum duration for waiting in the queue for insert requests due to -maxConcurrentInserts (default 1m0s) -logNewSeries diff --git a/lib/mergeset/inmemory_part.go b/lib/mergeset/inmemory_part.go index c3caca201..d8da08c97 100644 --- a/lib/mergeset/inmemory_part.go +++ b/lib/mergeset/inmemory_part.go @@ -1,8 +1,12 @@ package mergeset import ( + "fmt" + "path/filepath" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" "github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/fs" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" ) @@ -28,6 +32,36 @@ func (mp *inmemoryPart) Reset() { mp.lensData.Reset() } +// StoreToDisk stores mp to the given path on disk. +func (mp *inmemoryPart) StoreToDisk(path string) error { + if err := fs.MkdirAllIfNotExist(path); err != nil { + return fmt.Errorf("cannot create directory %q: %w", path, err) + } + metaindexPath := path + "/metaindex.bin" + if err := fs.WriteFileAndSync(metaindexPath, mp.metaindexData.B); err != nil { + return fmt.Errorf("cannot store metaindex: %w", err) + } + indexPath := path + "/index.bin" + if err := fs.WriteFileAndSync(indexPath, mp.indexData.B); err != nil { + return fmt.Errorf("cannot store index: %w", err) + } + itemsPath := path + "/items.bin" + if err := fs.WriteFileAndSync(itemsPath, mp.itemsData.B); err != nil { + return fmt.Errorf("cannot store items: %w", err) + } + lensPath := path + "/lens.bin" + if err := fs.WriteFileAndSync(lensPath, mp.lensData.B); err != nil { + return fmt.Errorf("cannot store lens: %w", err) + } + if err := mp.ph.WriteMetadata(path); err != nil { + return fmt.Errorf("cannot store metadata: %w", err) + } + // Sync parent directory in order to make sure the written files remain visible after hardware reset + parentDirPath := filepath.Dir(path) + fs.MustSyncPath(parentDirPath) + return nil +} + // Init initializes mp from ib. func (mp *inmemoryPart) Init(ib *inmemoryBlock) { mp.Reset() diff --git a/lib/mergeset/table.go b/lib/mergeset/table.go index d88cba7cb..ea18b78ad 100644 --- a/lib/mergeset/table.go +++ b/lib/mergeset/table.go @@ -22,10 +22,10 @@ import ( "github.com/VictoriaMetrics/VictoriaMetrics/lib/syncwg" ) -// maxParts is the maximum number of parts in the table. +// maxInmemoryParts is the maximum number of inmemory parts in the table. // // This number may be reached when the insertion pace outreaches merger pace. -const maxParts = 512 +const maxInmemoryParts = 64 // Default number of parts to merge at once. // @@ -46,6 +46,24 @@ const finalPartsToMerge = 2 // The required time shouldn't exceed a day. const maxPartSize = 400e9 +// The interval for flushing buffered data to parts, so it becomes visible to search. +const pendingItemsFlushInterval = time.Second + +// The interval for guaranteed flush of recently ingested data from memory to on-disk parts, +// so they survive process crash. +var dataFlushInterval = 5 * time.Second + +// SetDataFlushInterval sets the interval for guaranteed flush of recently ingested data from memory to disk. +// +// The data can be flushed from memory to disk more frequently if it doesn't fit the memory limit. +// +// This function must be called before initializing the indexdb. +func SetDataFlushInterval(d time.Duration) { + if d > pendingItemsFlushInterval { + dataFlushInterval = d + } +} + // maxItemsPerCachedPart is the maximum items per created part by the merge, // which must be cached in the OS page cache. // @@ -65,20 +83,23 @@ func maxItemsPerCachedPart() uint64 { return maxItems } -// The interval for flushing (converting) recent raw items into parts, -// so they become visible to search. -const rawItemsFlushInterval = time.Second - // Table represents mergeset table. type Table struct { // Atomically updated counters must go first in the struct, so they are properly // aligned to 8 bytes on 32-bit architectures. // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/212 - activeMerges uint64 - mergesCount uint64 - itemsMerged uint64 - assistedMerges uint64 + activeInmemoryMerges uint64 + activeFileMerges uint64 + + inmemoryMergesCount uint64 + fileMergesCount uint64 + + inmemoryItemsMerged uint64 + fileItemsMerged uint64 + + assistedInmemoryMerges uint64 + itemsAdded uint64 itemsAddedSizeBytes uint64 @@ -93,14 +114,20 @@ type Table struct { prepareBlock PrepareBlockCallback isReadOnly *uint32 - partsLock sync.Mutex - parts []*partWrapper - // rawItems contains recently added items that haven't been converted to parts yet. // // rawItems aren't used in search for performance reasons rawItems rawItemsShards + // partsLock protects inmemoryParts and fileParts. + partsLock sync.Mutex + + // inmemoryParts contains inmemory parts. + inmemoryParts []*partWrapper + + // fileParts contains file-backed parts. + fileParts []*partWrapper + snapshotLock sync.RWMutex flockF *os.File @@ -139,10 +166,13 @@ func (riss *rawItemsShards) init() { } func (riss *rawItemsShards) addItems(tb *Table, items [][]byte) { - n := atomic.AddUint32(&riss.shardIdx, 1) shards := riss.shards - idx := n % uint32(len(shards)) - shards[idx].addItems(tb, items) + shardsLen := uint32(len(shards)) + for len(items) > 0 { + n := atomic.AddUint32(&riss.shardIdx, 1) + idx := n % shardsLen + items = shards[idx].addItems(tb, items) + } } func (riss *rawItemsShards) Len() int { @@ -179,8 +209,9 @@ func (ris *rawItemsShard) Len() int { return n } -func (ris *rawItemsShard) addItems(tb *Table, items [][]byte) { - var blocksToFlush []*inmemoryBlock +func (ris *rawItemsShard) addItems(tb *Table, items [][]byte) [][]byte { + var ibsToFlush []*inmemoryBlock + var tailItems [][]byte ris.mu.Lock() ibs := ris.ibs @@ -190,10 +221,17 @@ func (ris *rawItemsShard) addItems(tb *Table, items [][]byte) { ris.ibs = ibs } ib := ibs[len(ibs)-1] - for _, item := range items { + for i, item := range items { if ib.Add(item) { continue } + if len(ibs) >= maxBlocksPerShard { + ibsToFlush = ibs + ibs = make([]*inmemoryBlock, 0, maxBlocksPerShard) + tailItems = items[i:] + atomic.StoreUint64(&ris.lastFlushTime, fasttime.UnixTimestamp()) + break + } ib = getInmemoryBlock() if ib.Add(item) { ibs = append(ibs, ib) @@ -203,17 +241,11 @@ func (ris *rawItemsShard) addItems(tb *Table, items [][]byte) { logger.Panicf("BUG: cannot insert too big item into an empty inmemoryBlock len(item)=%d; the caller should be responsible for avoiding too big items", len(item)) } ris.ibs = ibs - if len(ibs) >= maxBlocksPerShard { - blocksToFlush = append(blocksToFlush, ibs...) - for i := range ibs { - ibs[i] = nil - } - ris.ibs = ibs[:0] - atomic.StoreUint64(&ris.lastFlushTime, fasttime.UnixTimestamp()) - } ris.mu.Unlock() - tb.mergeRawItemsBlocks(blocksToFlush, false) + tb.flushBlocksToParts(ibsToFlush, false) + + return tailItems } type partWrapper struct { @@ -224,6 +256,9 @@ type partWrapper struct { refCount uint64 isInMerge bool + + // The deadline when the in-memory part must be flushed to disk. + flushToDiskDeadline time.Time } func (pw *partWrapper) incRef() { @@ -285,7 +320,7 @@ func OpenTable(path string, flushCallback func(), prepareBlock PrepareBlockCallb flushCallback: flushCallback, prepareBlock: prepareBlock, isReadOnly: isReadOnly, - parts: pws, + fileParts: pws, mergeIdx: uint64(time.Now().UnixNano()), flockF: flockF, stopCh: make(chan struct{}), @@ -296,7 +331,7 @@ func OpenTable(path string, flushCallback func(), prepareBlock PrepareBlockCallb var m TableMetrics tb.UpdateMetrics(&m) logger.Infof("table %q has been opened in %.3f seconds; partsCount: %d; blocksCount: %d, itemsCount: %d; sizeBytes: %d", - path, time.Since(startTime).Seconds(), m.PartsCount, m.BlocksCount, m.ItemsCount, m.SizeBytes) + path, time.Since(startTime).Seconds(), m.FilePartsCount, m.FileBlocksCount, m.FileItemsCount, m.FileSizeBytes) if flushCallback != nil { tb.flushCallbackWorkerWG.Add(1) @@ -323,8 +358,9 @@ func OpenTable(path string, flushCallback func(), prepareBlock PrepareBlockCallb } func (tb *Table) startBackgroundWorkers() { - tb.startPartMergers() - tb.startRawItemsFlusher() + tb.startMergeWorkers() + tb.startInmemoryPartsFlusher() + tb.startPendingItemsFlusher() } // MustClose closes the table. @@ -339,42 +375,26 @@ func (tb *Table) MustClose() { logger.Infof("flushing inmemory parts to files on %q...", tb.path) startTime = time.Now() - // Flush raw items the last time before exit. - tb.flushPendingItems(true) - - // Flush inmemory parts to disk. - var pws []*partWrapper - tb.partsLock.Lock() - for _, pw := range tb.parts { - if pw.mp == nil { - continue - } - if pw.isInMerge { - logger.Panicf("BUG: the inmemory part %s mustn't be in merge after stopping parts merger in %q", &pw.mp.ph, tb.path) - } - pw.isInMerge = true - pws = append(pws, pw) - } - tb.partsLock.Unlock() - - if err := tb.mergePartsOptimal(pws); err != nil { - logger.Panicf("FATAL: cannot flush inmemory parts to files in %q: %s", tb.path, err) - } - logger.Infof("%d inmemory parts have been flushed to files in %.3f seconds on %q", len(pws), time.Since(startTime).Seconds(), tb.path) + // Flush inmemory items the last time before exit. + tb.flushInmemoryItems() logger.Infof("waiting for flush callback worker to stop on %q...", tb.path) startTime = time.Now() tb.flushCallbackWorkerWG.Wait() logger.Infof("flush callback worker stopped in %.3f seconds on %q", time.Since(startTime).Seconds(), tb.path) - // Remove references to parts from the tb, so they may be eventually closed - // after all the searches are done. + // Remove references to parts from the tb, so they may be eventually closed after all the searches are done. tb.partsLock.Lock() - parts := tb.parts - tb.parts = nil + inmemoryParts := tb.inmemoryParts + fileParts := tb.fileParts + tb.inmemoryParts = nil + tb.fileParts = nil tb.partsLock.Unlock() - for _, pw := range parts { + for _, pw := range inmemoryParts { + pw.decRef() + } + for _, pw := range fileParts { pw.decRef() } @@ -391,20 +411,33 @@ func (tb *Table) Path() string { // TableMetrics contains essential metrics for the Table. type TableMetrics struct { - ActiveMerges uint64 - MergesCount uint64 - ItemsMerged uint64 - AssistedMerges uint64 + ActiveInmemoryMerges uint64 + ActiveFileMerges uint64 + + InmemoryMergesCount uint64 + FileMergesCount uint64 + + InmemoryItemsMerged uint64 + FileItemsMerged uint64 + + AssistedInmemoryMerges uint64 + ItemsAdded uint64 ItemsAddedSizeBytes uint64 PendingItems uint64 - PartsCount uint64 + InmemoryPartsCount uint64 + FilePartsCount uint64 - BlocksCount uint64 - ItemsCount uint64 - SizeBytes uint64 + InmemoryBlocksCount uint64 + FileBlocksCount uint64 + + InmemoryItemsCount uint64 + FileItemsCount uint64 + + InmemorySizeBytes uint64 + FileSizeBytes uint64 DataBlocksCacheSize uint64 DataBlocksCacheSizeBytes uint64 @@ -421,26 +454,46 @@ type TableMetrics struct { PartsRefCount uint64 } +// TotalItemsCount returns the total number of items in the table. +func (tm *TableMetrics) TotalItemsCount() uint64 { + return tm.InmemoryItemsCount + tm.FileItemsCount +} + // UpdateMetrics updates m with metrics from tb. func (tb *Table) UpdateMetrics(m *TableMetrics) { - m.ActiveMerges += atomic.LoadUint64(&tb.activeMerges) - m.MergesCount += atomic.LoadUint64(&tb.mergesCount) - m.ItemsMerged += atomic.LoadUint64(&tb.itemsMerged) - m.AssistedMerges += atomic.LoadUint64(&tb.assistedMerges) + m.ActiveInmemoryMerges += atomic.LoadUint64(&tb.activeInmemoryMerges) + m.ActiveFileMerges += atomic.LoadUint64(&tb.activeFileMerges) + + m.InmemoryMergesCount += atomic.LoadUint64(&tb.inmemoryMergesCount) + m.FileMergesCount += atomic.LoadUint64(&tb.fileMergesCount) + + m.InmemoryItemsMerged += atomic.LoadUint64(&tb.inmemoryItemsMerged) + m.FileItemsMerged += atomic.LoadUint64(&tb.fileItemsMerged) + + m.AssistedInmemoryMerges += atomic.LoadUint64(&tb.assistedInmemoryMerges) + m.ItemsAdded += atomic.LoadUint64(&tb.itemsAdded) m.ItemsAddedSizeBytes += atomic.LoadUint64(&tb.itemsAddedSizeBytes) m.PendingItems += uint64(tb.rawItems.Len()) tb.partsLock.Lock() - m.PartsCount += uint64(len(tb.parts)) - for _, pw := range tb.parts { + + m.InmemoryPartsCount += uint64(len(tb.inmemoryParts)) + for _, pw := range tb.inmemoryParts { p := pw.p + m.InmemoryBlocksCount += p.ph.blocksCount + m.InmemoryItemsCount += p.ph.itemsCount + m.InmemorySizeBytes += p.size + m.PartsRefCount += atomic.LoadUint64(&pw.refCount) + } - m.BlocksCount += p.ph.blocksCount - m.ItemsCount += p.ph.itemsCount - m.SizeBytes += p.size - + m.FilePartsCount += uint64(len(tb.fileParts)) + for _, pw := range tb.fileParts { + p := pw.p + m.FileBlocksCount += p.ph.blocksCount + m.FileItemsCount += p.ph.itemsCount + m.FileSizeBytes += p.size m.PartsRefCount += atomic.LoadUint64(&pw.refCount) } tb.partsLock.Unlock() @@ -477,10 +530,14 @@ func (tb *Table) AddItems(items [][]byte) { // The appended parts must be released with putParts. func (tb *Table) getParts(dst []*partWrapper) []*partWrapper { tb.partsLock.Lock() - for _, pw := range tb.parts { + for _, pw := range tb.inmemoryParts { pw.incRef() } - dst = append(dst, tb.parts...) + for _, pw := range tb.fileParts { + pw.incRef() + } + dst = append(dst, tb.inmemoryParts...) + dst = append(dst, tb.fileParts...) tb.partsLock.Unlock() return dst @@ -493,79 +550,142 @@ func (tb *Table) putParts(pws []*partWrapper) { } } -func (tb *Table) startRawItemsFlusher() { +func (tb *Table) mergePartsOptimal(pws []*partWrapper) error { + sortPartsForOptimalMerge(pws) + for len(pws) > 0 { + n := defaultPartsToMerge + if n > len(pws) { + n = len(pws) + } + pwsChunk := pws[:n] + pws = pws[n:] + err := tb.mergeParts(pwsChunk, nil, true) + if err == nil { + continue + } + tb.releasePartsToMerge(pws) + return fmt.Errorf("cannot optimally merge %d parts: %w", n, err) + } + return nil +} + +// DebugFlush flushes all the added items to the storage, so they become visible to search. +// +// This function is only for debugging and testing. +func (tb *Table) DebugFlush() { + tb.flushPendingItems(nil, true) + + // Wait for background flushers to finish. + tb.rawItemsPendingFlushesWG.Wait() +} + +func (tb *Table) startInmemoryPartsFlusher() { tb.wg.Add(1) go func() { - tb.rawItemsFlusher() + tb.inmemoryPartsFlusher() tb.wg.Done() }() } -func (tb *Table) rawItemsFlusher() { - ticker := time.NewTicker(rawItemsFlushInterval) +func (tb *Table) startPendingItemsFlusher() { + tb.wg.Add(1) + go func() { + tb.pendingItemsFlusher() + tb.wg.Done() + }() +} + +func (tb *Table) inmemoryPartsFlusher() { + ticker := time.NewTicker(dataFlushInterval) defer ticker.Stop() for { select { case <-tb.stopCh: return case <-ticker.C: - tb.flushPendingItems(false) + tb.flushInmemoryParts(false) } } } -func (tb *Table) mergePartsOptimal(pws []*partWrapper) error { - for len(pws) > defaultPartsToMerge { - pwsChunk := pws[:defaultPartsToMerge] - pws = pws[defaultPartsToMerge:] - if err := tb.mergeParts(pwsChunk, nil, false); err != nil { - tb.releasePartsToMerge(pws) - return fmt.Errorf("cannot merge %d parts: %w", defaultPartsToMerge, err) +func (tb *Table) pendingItemsFlusher() { + ticker := time.NewTicker(pendingItemsFlushInterval) + defer ticker.Stop() + var ibs []*inmemoryBlock + for { + select { + case <-tb.stopCh: + return + case <-ticker.C: + ibs = tb.flushPendingItems(ibs[:0], false) + for i := range ibs { + ibs[i] = nil + } } } - if len(pws) == 0 { - return nil +} + +func (tb *Table) flushPendingItems(dst []*inmemoryBlock, isFinal bool) []*inmemoryBlock { + return tb.rawItems.flush(tb, dst, isFinal) +} + +func (tb *Table) flushInmemoryItems() { + tb.rawItems.flush(tb, nil, true) + tb.flushInmemoryParts(true) +} + +func (tb *Table) flushInmemoryParts(isFinal bool) { + for { + currentTime := time.Now() + var pws []*partWrapper + + tb.partsLock.Lock() + for _, pw := range tb.inmemoryParts { + if !pw.isInMerge && (isFinal || pw.flushToDiskDeadline.Before(currentTime)) { + pw.isInMerge = true + pws = append(pws, pw) + } + } + tb.partsLock.Unlock() + + if err := tb.mergePartsOptimal(pws); err != nil { + logger.Panicf("FATAL: cannot merge in-memory parts: %s", err) + } + if !isFinal { + return + } + tb.partsLock.Lock() + n := len(tb.inmemoryParts) + tb.partsLock.Unlock() + if n == 0 { + // All the in-memory parts were flushed to disk. + return + } + // Some parts weren't flushed to disk because they were being merged. + // Sleep for a while and try flushing them again. + time.Sleep(10 * time.Millisecond) } - if err := tb.mergeParts(pws, nil, false); err != nil { - return fmt.Errorf("cannot merge %d parts: %w", len(pws), err) - } - return nil } -// DebugFlush flushes all the added items to the storage, -// so they become visible to search. -// -// This function is only for debugging and testing. -func (tb *Table) DebugFlush() { - tb.flushPendingItems(true) - - // Wait for background flushers to finish. - tb.rawItemsPendingFlushesWG.Wait() -} - -func (tb *Table) flushPendingItems(isFinal bool) { - tb.rawItems.flush(tb, isFinal) -} - -func (riss *rawItemsShards) flush(tb *Table, isFinal bool) { +func (riss *rawItemsShards) flush(tb *Table, dst []*inmemoryBlock, isFinal bool) []*inmemoryBlock { tb.rawItemsPendingFlushesWG.Add(1) defer tb.rawItemsPendingFlushesWG.Done() - var blocksToFlush []*inmemoryBlock for i := range riss.shards { - blocksToFlush = riss.shards[i].appendBlocksToFlush(blocksToFlush, tb, isFinal) + dst = riss.shards[i].appendBlocksToFlush(dst, tb, isFinal) } - tb.mergeRawItemsBlocks(blocksToFlush, isFinal) + tb.flushBlocksToParts(dst, isFinal) + return dst } func (ris *rawItemsShard) appendBlocksToFlush(dst []*inmemoryBlock, tb *Table, isFinal bool) []*inmemoryBlock { currentTime := fasttime.UnixTimestamp() - flushSeconds := int64(rawItemsFlushInterval.Seconds()) + flushSeconds := int64(pendingItemsFlushInterval.Seconds()) if flushSeconds <= 0 { flushSeconds = 1 } lastFlushTime := atomic.LoadUint64(&ris.lastFlushTime) - if !isFinal && currentTime <= lastFlushTime+uint64(flushSeconds) { + if !isFinal && currentTime < lastFlushTime+uint64(flushSeconds) { // Fast path - nothing to flush return dst } @@ -582,27 +702,29 @@ func (ris *rawItemsShard) appendBlocksToFlush(dst []*inmemoryBlock, tb *Table, i return dst } -func (tb *Table) mergeRawItemsBlocks(ibs []*inmemoryBlock, isFinal bool) { +func (tb *Table) flushBlocksToParts(ibs []*inmemoryBlock, isFinal bool) { if len(ibs) == 0 { return } - - pws := make([]*partWrapper, 0, (len(ibs)+defaultPartsToMerge-1)/defaultPartsToMerge) var pwsLock sync.Mutex - var wg sync.WaitGroup + pws := make([]*partWrapper, 0, (len(ibs)+defaultPartsToMerge-1)/defaultPartsToMerge) + wg := getWaitGroup() for len(ibs) > 0 { n := defaultPartsToMerge if n > len(ibs) { n = len(ibs) } wg.Add(1) - go func(ibsPart []*inmemoryBlock) { - defer wg.Done() - pw := tb.mergeInmemoryBlocks(ibsPart) + flushConcurrencyCh <- struct{}{} + go func(ibsChunk []*inmemoryBlock) { + defer func() { + <-flushConcurrencyCh + wg.Done() + }() + pw := tb.createInmemoryPart(ibsChunk) if pw == nil { return } - pw.isInMerge = true pwsLock.Lock() pws = append(pws, pw) pwsLock.Unlock() @@ -610,49 +732,78 @@ func (tb *Table) mergeRawItemsBlocks(ibs []*inmemoryBlock, isFinal bool) { ibs = ibs[n:] } wg.Wait() - if len(pws) > 0 { - if err := tb.mergeParts(pws, nil, true); err != nil { - logger.Panicf("FATAL: cannot merge raw parts: %s", err) - } - if tb.flushCallback != nil { - if isFinal { - tb.flushCallback() - } else { - atomic.CompareAndSwapUint32(&tb.needFlushCallbackCall, 0, 1) - } + putWaitGroup(wg) + + tb.partsLock.Lock() + tb.inmemoryParts = append(tb.inmemoryParts, pws...) + tb.partsLock.Unlock() + + flushConcurrencyCh <- struct{}{} + tb.assistedMergeForInmemoryParts() + <-flushConcurrencyCh + // There is no need in assited merge for file parts, + // since the bottleneck is possible only at inmemory parts. + + if tb.flushCallback != nil { + if isFinal { + tb.flushCallback() + } else { + atomic.CompareAndSwapUint32(&tb.needFlushCallbackCall, 0, 1) } } +} +var flushConcurrencyCh = make(chan struct{}, cgroup.AvailableCPUs()) + +func (tb *Table) assistedMergeForInmemoryParts() { for { tb.partsLock.Lock() - ok := len(tb.parts) <= maxParts + ok := getNotInMergePartsCount(tb.inmemoryParts) < maxInmemoryParts tb.partsLock.Unlock() if ok { return } - // The added part exceeds maxParts count. Assist with merging other parts. - // // Prioritize assisted merges over searches. storagepacelimiter.Search.Inc() - err := tb.mergeExistingParts(false) + err := tb.mergeInmemoryParts() storagepacelimiter.Search.Dec() if err == nil { - atomic.AddUint64(&tb.assistedMerges, 1) + atomic.AddUint64(&tb.assistedInmemoryMerges, 1) continue } - if errors.Is(err, errNothingToMerge) || errors.Is(err, errForciblyStopped) || errors.Is(err, errReadOnlyMode) { + if errors.Is(err, errNothingToMerge) || errors.Is(err, errForciblyStopped) { return } - logger.Panicf("FATAL: cannot merge small parts: %s", err) + logger.Panicf("FATAL: cannot assist with merging inmemory parts: %s", err) } } -func (tb *Table) mergeInmemoryBlocks(ibs []*inmemoryBlock) *partWrapper { - atomic.AddUint64(&tb.mergesCount, 1) - atomic.AddUint64(&tb.activeMerges, 1) - defer atomic.AddUint64(&tb.activeMerges, ^uint64(0)) +func getNotInMergePartsCount(pws []*partWrapper) int { + n := 0 + for _, pw := range pws { + if !pw.isInMerge { + n++ + } + } + return n +} +func getWaitGroup() *sync.WaitGroup { + v := wgPool.Get() + if v == nil { + return &sync.WaitGroup{} + } + return v.(*sync.WaitGroup) +} + +func putWaitGroup(wg *sync.WaitGroup) { + wgPool.Put(wg) +} + +var wgPool sync.Pool + +func (tb *Table) createInmemoryPart(ibs []*inmemoryBlock) *partWrapper { outItemsCount := uint64(0) for _, ib := range ibs { outItemsCount += uint64(ib.Len()) @@ -672,16 +823,14 @@ func (tb *Table) mergeInmemoryBlocks(ibs []*inmemoryBlock) *partWrapper { if len(bsrs) == 0 { return nil } + flushToDiskDeadline := time.Now().Add(dataFlushInterval) if len(bsrs) == 1 { // Nothing to merge. Just return a single inmemory part. + bsr := bsrs[0] mp := &inmemoryPart{} - mp.Init(&bsrs[0].Block) - p := mp.NewPart() - return &partWrapper{ - p: p, - mp: mp, - refCount: 1, - } + mp.Init(&bsr.Block) + putBlockStreamReader(bsr) + return newPartWrapperFromInmemoryPart(mp, flushToDiskDeadline) } // Prepare blockStreamWriter for destination part. @@ -693,7 +842,10 @@ func (tb *Table) mergeInmemoryBlocks(ibs []*inmemoryBlock) *partWrapper { // Merge parts. // The merge shouldn't be interrupted by stopCh, // since it may be final after stopCh is closed. - err := mergeBlockStreams(&mpDst.ph, bsw, bsrs, tb.prepareBlock, nil, &tb.itemsMerged) + atomic.AddUint64(&tb.activeInmemoryMerges, 1) + err := mergeBlockStreams(&mpDst.ph, bsw, bsrs, tb.prepareBlock, nil, &tb.inmemoryItemsMerged) + atomic.AddUint64(&tb.activeInmemoryMerges, ^uint64(0)) + atomic.AddUint64(&tb.inmemoryMergesCount, 1) if err != nil { logger.Panicf("FATAL: cannot merge inmemoryBlocks: %s", err) } @@ -701,33 +853,64 @@ func (tb *Table) mergeInmemoryBlocks(ibs []*inmemoryBlock) *partWrapper { for _, bsr := range bsrs { putBlockStreamReader(bsr) } + return newPartWrapperFromInmemoryPart(mpDst, flushToDiskDeadline) +} - p := mpDst.NewPart() +func newPartWrapperFromInmemoryPart(mp *inmemoryPart, flushToDiskDeadline time.Time) *partWrapper { + p := mp.NewPart() return &partWrapper{ - p: p, - mp: mpDst, - refCount: 1, + p: p, + mp: mp, + refCount: 1, + flushToDiskDeadline: flushToDiskDeadline, } } -func (tb *Table) startPartMergers() { - for i := 0; i < mergeWorkersCount; i++ { +func (tb *Table) startMergeWorkers() { + for i := 0; i < cap(mergeWorkersLimitCh); i++ { tb.wg.Add(1) go func() { - if err := tb.partMerger(); err != nil { - logger.Panicf("FATAL: unrecoverable error when merging parts in %q: %s", tb.path, err) - } + tb.mergeWorker() tb.wg.Done() }() } } +func getMaxInmemoryPartSize() uint64 { + // Allow up to 5% of memory for in-memory parts. + n := uint64(0.05 * float64(memory.Allowed()) / maxInmemoryParts) + if n < 1e6 { + n = 1e6 + } + return n +} + +func (tb *Table) getMaxFilePartSize() uint64 { + n := fs.MustGetFreeSpace(tb.path) + // Divide free space by the max number of concurrent merges. + maxOutBytes := n / uint64(cap(mergeWorkersLimitCh)) + if maxOutBytes > maxPartSize { + maxOutBytes = maxPartSize + } + return maxOutBytes +} + func (tb *Table) canBackgroundMerge() bool { return atomic.LoadUint32(tb.isReadOnly) == 0 } var errReadOnlyMode = fmt.Errorf("storage is in readonly mode") +func (tb *Table) mergeInmemoryParts() error { + maxOutBytes := tb.getMaxFilePartSize() + + tb.partsLock.Lock() + pws := getPartsToMerge(tb.inmemoryParts, maxOutBytes, false) + tb.partsLock.Unlock() + + return tb.mergeParts(pws, tb.stopCh, false) +} + func (tb *Table) mergeExistingParts(isFinal bool) error { if !tb.canBackgroundMerge() { // Do not perform background merge in read-only mode @@ -735,32 +918,32 @@ func (tb *Table) mergeExistingParts(isFinal bool) error { // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/2603 return errReadOnlyMode } - n := fs.MustGetFreeSpace(tb.path) - // Divide free space by the max number of concurrent merges. - maxOutBytes := n / uint64(mergeWorkersCount) - if maxOutBytes > maxPartSize { - maxOutBytes = maxPartSize - } + maxOutBytes := tb.getMaxFilePartSize() tb.partsLock.Lock() - pws := getPartsToMerge(tb.parts, maxOutBytes, isFinal) + dst := make([]*partWrapper, 0, len(tb.inmemoryParts)+len(tb.fileParts)) + dst = append(dst, tb.inmemoryParts...) + dst = append(dst, tb.fileParts...) + pws := getPartsToMerge(dst, maxOutBytes, isFinal) tb.partsLock.Unlock() - return tb.mergeParts(pws, tb.stopCh, false) + return tb.mergeParts(pws, tb.stopCh, isFinal) } const ( - minMergeSleepTime = time.Millisecond - maxMergeSleepTime = time.Second + minMergeSleepTime = 10 * time.Millisecond + maxMergeSleepTime = 10 * time.Second ) -func (tb *Table) partMerger() error { +func (tb *Table) mergeWorker() { sleepTime := minMergeSleepTime var lastMergeTime uint64 isFinal := false t := time.NewTimer(sleepTime) for { + mergeWorkersLimitCh <- struct{}{} err := tb.mergeExistingParts(isFinal) + <-mergeWorkersLimitCh if err == nil { // Try merging additional parts. sleepTime = minMergeSleepTime @@ -770,12 +953,13 @@ func (tb *Table) partMerger() error { } if errors.Is(err, errForciblyStopped) { // The merger has been stopped. - return nil + return } if !errors.Is(err, errNothingToMerge) && !errors.Is(err, errReadOnlyMode) { - return err + // Unexpected error. + logger.Panicf("FATAL: unrecoverable error when merging inmemory parts in %q: %s", tb.path, err) } - if fasttime.UnixTimestamp()-lastMergeTime > 30 { + if finalMergeDelaySeconds > 0 && fasttime.UnixTimestamp()-lastMergeTime > finalMergeDelaySeconds { // We have free time for merging into bigger parts. // This should improve select performance. lastMergeTime = fasttime.UnixTimestamp() @@ -790,13 +974,27 @@ func (tb *Table) partMerger() error { } select { case <-tb.stopCh: - return nil + return case <-t.C: t.Reset(sleepTime) } } } +// Disable final merge by default, since it may lead to high disk IO and CPU usage +// after some inactivity time. +var finalMergeDelaySeconds = uint64(0) + +// SetFinalMergeDelay sets the delay before doing final merge for Table without newly ingested data. +// +// This function may be called only before Table initialization. +func SetFinalMergeDelay(delay time.Duration) { + if delay <= 0 { + return + } + finalMergeDelaySeconds = uint64(delay.Seconds() + 1) +} + var errNothingToMerge = fmt.Errorf("nothing to merge") func (tb *Table) releasePartsToMerge(pws []*partWrapper) { @@ -810,150 +1008,315 @@ func (tb *Table) releasePartsToMerge(pws []*partWrapper) { tb.partsLock.Unlock() } -// mergeParts merges pws. +// mergeParts merges pws to a single resulting part. // // Merging is immediately stopped if stopCh is closed. // +// If isFinal is set, then the resulting part will be stored to disk. +// // All the parts inside pws must have isInMerge field set to true. -func (tb *Table) mergeParts(pws []*partWrapper, stopCh <-chan struct{}, isOuterParts bool) error { +func (tb *Table) mergeParts(pws []*partWrapper, stopCh <-chan struct{}, isFinal bool) error { if len(pws) == 0 { // Nothing to merge. return errNothingToMerge } defer tb.releasePartsToMerge(pws) - atomic.AddUint64(&tb.mergesCount, 1) - atomic.AddUint64(&tb.activeMerges, 1) - defer atomic.AddUint64(&tb.activeMerges, ^uint64(0)) - startTime := time.Now() - // Prepare blockStreamReaders for source parts. - bsrs := make([]*blockStreamReader, 0, len(pws)) - defer func() { + // Initialize destination paths. + dstPartType := getDstPartType(pws, isFinal) + tmpPartPath, mergeIdx := tb.getDstPartPaths(dstPartType) + + if isFinal && len(pws) == 1 && pws[0].mp != nil { + // Fast path: flush a single in-memory part to disk. + mp := pws[0].mp + if tmpPartPath == "" { + logger.Panicf("BUG: tmpPartPath must be non-empty") + } + if err := mp.StoreToDisk(tmpPartPath); err != nil { + return fmt.Errorf("cannot store in-memory part to %q: %w", tmpPartPath, err) + } + pwNew, err := tb.openCreatedPart(&mp.ph, pws, nil, tmpPartPath, mergeIdx) + if err != nil { + return fmt.Errorf("cannot atomically register the created part: %w", err) + } + tb.swapSrcWithDstParts(pws, pwNew, dstPartType) + return nil + } + + // Prepare BlockStreamReaders for source parts. + bsrs, err := openBlockStreamReaders(pws) + if err != nil { + return err + } + closeBlockStreamReaders := func() { for _, bsr := range bsrs { putBlockStreamReader(bsr) } - }() + bsrs = nil + } + + // Prepare BlockStreamWriter for destination part. + srcSize := uint64(0) + srcItemsCount := uint64(0) + srcBlocksCount := uint64(0) + for _, pw := range pws { + srcSize += pw.p.size + srcItemsCount += pw.p.ph.itemsCount + srcBlocksCount += pw.p.ph.blocksCount + } + compressLevel := getCompressLevel(srcItemsCount) + bsw := getBlockStreamWriter() + var mpNew *inmemoryPart + if dstPartType == partInmemory { + mpNew = &inmemoryPart{} + bsw.InitFromInmemoryPart(mpNew, compressLevel) + } else { + if tmpPartPath == "" { + logger.Panicf("BUG: tmpPartPath must be non-empty") + } + nocache := srcItemsCount > maxItemsPerCachedPart() + if err := bsw.InitFromFilePart(tmpPartPath, nocache, compressLevel); err != nil { + closeBlockStreamReaders() + return fmt.Errorf("cannot create destination part at %q: %w", tmpPartPath, err) + } + } + + // Merge source parts to destination part. + ph, err := tb.mergePartsInternal(tmpPartPath, bsw, bsrs, dstPartType, stopCh) + putBlockStreamWriter(bsw) + closeBlockStreamReaders() + if err != nil { + return fmt.Errorf("cannot merge %d parts: %w", len(pws), err) + } + if mpNew != nil { + // Update partHeader for destination inmemory part after the merge. + mpNew.ph = *ph + } + + // Atomically move the created part from tmpPartPath to its destination + // and swap the source parts with the newly created part. + pwNew, err := tb.openCreatedPart(ph, pws, mpNew, tmpPartPath, mergeIdx) + if err != nil { + return fmt.Errorf("cannot atomically register the created part: %w", err) + } + tb.swapSrcWithDstParts(pws, pwNew, dstPartType) + + d := time.Since(startTime) + if d <= 30*time.Second { + return nil + } + + // Log stats for long merges. + dstItemsCount := uint64(0) + dstBlocksCount := uint64(0) + dstSize := uint64(0) + dstPartPath := "" + if pwNew != nil { + pDst := pwNew.p + dstItemsCount = pDst.ph.itemsCount + dstBlocksCount = pDst.ph.blocksCount + dstSize = pDst.size + dstPartPath = pDst.path + } + durationSecs := d.Seconds() + itemsPerSec := int(float64(srcItemsCount) / durationSecs) + logger.Infof("merged (%d parts, %d items, %d blocks, %d bytes) into (1 part, %d items, %d blocks, %d bytes) in %.3f seconds at %d items/sec to %q", + len(pws), srcItemsCount, srcBlocksCount, srcSize, dstItemsCount, dstBlocksCount, dstSize, durationSecs, itemsPerSec, dstPartPath) + + return nil +} + +func getFlushToDiskDeadline(pws []*partWrapper) time.Time { + d := pws[0].flushToDiskDeadline + for _, pw := range pws[1:] { + if pw.flushToDiskDeadline.Before(d) { + d = pw.flushToDiskDeadline + } + } + return d +} + +type partType int + +var ( + partInmemory = partType(0) + partFile = partType(1) +) + +func getDstPartType(pws []*partWrapper, isFinal bool) partType { + dstPartSize := getPartsSize(pws) + if isFinal || dstPartSize > getMaxInmemoryPartSize() { + return partFile + } + if !areAllInmemoryParts(pws) { + // If at least a single source part is located in file, + // then the destination part must be in file for durability reasons. + return partFile + } + return partInmemory +} + +func (tb *Table) getDstPartPaths(dstPartType partType) (string, uint64) { + tmpPartPath := "" + mergeIdx := tb.nextMergeIdx() + switch dstPartType { + case partInmemory: + case partFile: + tmpPartPath = fmt.Sprintf("%s/tmp/%016X", tb.path, mergeIdx) + default: + logger.Panicf("BUG: unknown partType=%d", dstPartType) + } + return tmpPartPath, mergeIdx +} + +func openBlockStreamReaders(pws []*partWrapper) ([]*blockStreamReader, error) { + bsrs := make([]*blockStreamReader, 0, len(pws)) for _, pw := range pws { bsr := getBlockStreamReader() if pw.mp != nil { - if !isOuterParts { - logger.Panicf("BUG: inmemory part must be always outer") - } bsr.InitFromInmemoryPart(pw.mp) } else { if err := bsr.InitFromFilePart(pw.p.path); err != nil { - return fmt.Errorf("cannot open source part for merging: %w", err) + for _, bsr := range bsrs { + putBlockStreamReader(bsr) + } + return nil, fmt.Errorf("cannot open source part for merging: %w", err) } } bsrs = append(bsrs, bsr) } + return bsrs, nil +} - outItemsCount := uint64(0) - outBlocksCount := uint64(0) - for _, pw := range pws { - outItemsCount += pw.p.ph.itemsCount - outBlocksCount += pw.p.ph.blocksCount - } - nocache := true - if outItemsCount < maxItemsPerCachedPart() { - // Cache small (i.e. recent) output parts in OS file cache, - // since there is high chance they will be read soon. - nocache = false - } - - // Prepare blockStreamWriter for destination part. - mergeIdx := tb.nextMergeIdx() - tmpPartPath := fmt.Sprintf("%s/tmp/%016X", tb.path, mergeIdx) - bsw := getBlockStreamWriter() - compressLevel := getCompressLevel(outItemsCount) - if err := bsw.InitFromFilePart(tmpPartPath, nocache, compressLevel); err != nil { - return fmt.Errorf("cannot create destination part %q: %w", tmpPartPath, err) - } - - // Merge parts into a temporary location. +func (tb *Table) mergePartsInternal(tmpPartPath string, bsw *blockStreamWriter, bsrs []*blockStreamReader, dstPartType partType, stopCh <-chan struct{}) (*partHeader, error) { var ph partHeader - err := mergeBlockStreams(&ph, bsw, bsrs, tb.prepareBlock, stopCh, &tb.itemsMerged) - putBlockStreamWriter(bsw) + var itemsMerged *uint64 + var mergesCount *uint64 + var activeMerges *uint64 + switch dstPartType { + case partInmemory: + itemsMerged = &tb.inmemoryItemsMerged + mergesCount = &tb.inmemoryMergesCount + activeMerges = &tb.activeInmemoryMerges + case partFile: + itemsMerged = &tb.fileItemsMerged + mergesCount = &tb.fileMergesCount + activeMerges = &tb.activeFileMerges + default: + logger.Panicf("BUG: unknown partType=%d", dstPartType) + } + atomic.AddUint64(activeMerges, 1) + err := mergeBlockStreams(&ph, bsw, bsrs, tb.prepareBlock, stopCh, itemsMerged) + atomic.AddUint64(activeMerges, ^uint64(0)) + atomic.AddUint64(mergesCount, 1) if err != nil { - return fmt.Errorf("error when merging parts to %q: %w", tmpPartPath, err) + return nil, fmt.Errorf("cannot merge parts to %q: %w", tmpPartPath, err) } - if err := ph.WriteMetadata(tmpPartPath); err != nil { - return fmt.Errorf("cannot write metadata to destination part %q: %w", tmpPartPath, err) - } - - // Close bsrs (aka source parts). - for _, bsr := range bsrs { - putBlockStreamReader(bsr) - } - bsrs = nil - - // Create a transaction for atomic deleting old parts and moving - // new part to its destination place. - var bb bytesutil.ByteBuffer - for _, pw := range pws { - if pw.mp == nil { - fmt.Fprintf(&bb, "%s\n", pw.p.path) + if tmpPartPath != "" { + if err := ph.WriteMetadata(tmpPartPath); err != nil { + return nil, fmt.Errorf("cannot write metadata to destination part %q: %w", tmpPartPath, err) } } - dstPartPath := ph.Path(tb.path, mergeIdx) - fmt.Fprintf(&bb, "%s -> %s\n", tmpPartPath, dstPartPath) - txnPath := fmt.Sprintf("%s/txn/%016X", tb.path, mergeIdx) - if err := fs.WriteFileAtomically(txnPath, bb.B, false); err != nil { - return fmt.Errorf("cannot create transaction file %q: %w", txnPath, err) - } + return &ph, nil +} - // Run the created transaction. - if err := runTransaction(&tb.snapshotLock, tb.path, txnPath); err != nil { - return fmt.Errorf("cannot execute transaction %q: %w", txnPath, err) - } +func (tb *Table) openCreatedPart(ph *partHeader, pws []*partWrapper, mpNew *inmemoryPart, tmpPartPath string, mergeIdx uint64) (*partWrapper, error) { + dstPartPath := "" + if mpNew == nil || !areAllInmemoryParts(pws) { + // Either source or destination parts are located on disk. + // Create a transaction for atomic deleting of old parts and moving new part to its destination on disk. + var bb bytesutil.ByteBuffer + for _, pw := range pws { + if pw.mp == nil { + fmt.Fprintf(&bb, "%s\n", pw.p.path) + } + } + dstPartPath = ph.Path(tb.path, mergeIdx) + fmt.Fprintf(&bb, "%s -> %s\n", tmpPartPath, dstPartPath) + txnPath := fmt.Sprintf("%s/txn/%016X", tb.path, mergeIdx) + if err := fs.WriteFileAtomically(txnPath, bb.B, false); err != nil { + return nil, fmt.Errorf("cannot create transaction file %q: %w", txnPath, err) + } - // Open the merged part. - newP, err := openFilePart(dstPartPath) + // Run the created transaction. + if err := runTransaction(&tb.snapshotLock, tb.path, txnPath); err != nil { + return nil, fmt.Errorf("cannot execute transaction %q: %w", txnPath, err) + } + } + // Open the created part. + if mpNew != nil { + // Open the created part from memory. + flushToDiskDeadline := getFlushToDiskDeadline(pws) + pwNew := newPartWrapperFromInmemoryPart(mpNew, flushToDiskDeadline) + return pwNew, nil + } + // Open the created part from disk. + pNew, err := openFilePart(dstPartPath) if err != nil { - return fmt.Errorf("cannot open merged part %q: %w", dstPartPath, err) + return nil, fmt.Errorf("cannot open merged part %q: %w", dstPartPath, err) } - newPSize := newP.size - newPW := &partWrapper{ - p: newP, + pwNew := &partWrapper{ + p: pNew, refCount: 1, } + return pwNew, nil +} - // Atomically remove old parts and add new part. +func areAllInmemoryParts(pws []*partWrapper) bool { + for _, pw := range pws { + if pw.mp == nil { + return false + } + } + return true +} + +func (tb *Table) swapSrcWithDstParts(pws []*partWrapper, pwNew *partWrapper, dstPartType partType) { + // Atomically unregister old parts and add new part to tb. m := make(map[*partWrapper]bool, len(pws)) for _, pw := range pws { m[pw] = true } if len(m) != len(pws) { - logger.Panicf("BUG: %d duplicate parts found in the merge of %d parts", len(pws)-len(m), len(pws)) + logger.Panicf("BUG: %d duplicate parts found when merging %d parts", len(pws)-len(m), len(pws)) } - removedParts := 0 + removedInmemoryParts := 0 + removedFileParts := 0 + tb.partsLock.Lock() - tb.parts, removedParts = removeParts(tb.parts, m) - tb.parts = append(tb.parts, newPW) + tb.inmemoryParts, removedInmemoryParts = removeParts(tb.inmemoryParts, m) + tb.fileParts, removedFileParts = removeParts(tb.fileParts, m) + if pwNew != nil { + switch dstPartType { + case partInmemory: + tb.inmemoryParts = append(tb.inmemoryParts, pwNew) + case partFile: + tb.fileParts = append(tb.fileParts, pwNew) + default: + logger.Panicf("BUG: unknown partType=%d", dstPartType) + } + } tb.partsLock.Unlock() + + removedParts := removedInmemoryParts + removedFileParts if removedParts != len(m) { - if !isOuterParts { - logger.Panicf("BUG: unexpected number of parts removed; got %d; want %d", removedParts, len(m)) - } - if removedParts != 0 { - logger.Panicf("BUG: removed non-zero outer parts: %d", removedParts) - } + logger.Panicf("BUG: unexpected number of parts removed; got %d, want %d", removedParts, len(m)) } - // Remove partition references from old parts. + // Remove references from old parts. for _, pw := range pws { pw.decRef() } +} - d := time.Since(startTime) - if d > 30*time.Second { - logger.Infof("merged %d items across %d blocks in %.3f seconds at %d items/sec to %q; sizeBytes: %d", - outItemsCount, outBlocksCount, d.Seconds(), int(float64(outItemsCount)/d.Seconds()), dstPartPath, newPSize) +func getPartsSize(pws []*partWrapper) uint64 { + n := uint64(0) + for _, pw := range pws { + n += pw.p.size } - - return nil + return n } func getCompressLevel(itemsCount uint64) int { @@ -990,7 +1353,7 @@ func (tb *Table) nextMergeIdx() uint64 { return atomic.AddUint64(&tb.mergeIdx, 1) } -var mergeWorkersCount = cgroup.AvailableCPUs() +var mergeWorkersLimitCh = make(chan struct{}, cgroup.AvailableCPUs()) func openParts(path string) ([]*partWrapper, error) { // The path can be missing after restoring from backup, so create it if needed. @@ -1095,7 +1458,7 @@ func (tb *Table) CreateSnapshotAt(dstDir string) error { } // Flush inmemory items to disk. - tb.flushPendingItems(true) + tb.flushInmemoryItems() // The snapshot must be created under the lock in order to prevent from // concurrent modifications via runTransaction. @@ -1154,7 +1517,7 @@ func runTransactions(txnLock *sync.RWMutex, path string) error { if os.IsNotExist(err) { return nil } - return fmt.Errorf("cannot open %q: %w", txnDir, err) + return fmt.Errorf("cannot open transaction dir: %w", err) } defer fs.MustClose(d) @@ -1334,8 +1697,7 @@ func appendPartsToMerge(dst, src []*partWrapper, maxPartsToMerge int, maxOutByte } src = tmp - // Sort src parts by size. - sort.Slice(src, func(i, j int) bool { return src[i].p.size < src[j].p.size }) + sortPartsForOptimalMerge(src) maxSrcParts := maxPartsToMerge if maxSrcParts > len(src) { @@ -1386,17 +1748,24 @@ func appendPartsToMerge(dst, src []*partWrapper, maxPartsToMerge int, maxOutByte return append(dst, pws...) } +func sortPartsForOptimalMerge(pws []*partWrapper) { + // Sort src parts by size. + sort.Slice(pws, func(i, j int) bool { + return pws[i].p.size < pws[j].p.size + }) +} + func removeParts(pws []*partWrapper, partsToRemove map[*partWrapper]bool) ([]*partWrapper, int) { - removedParts := 0 dst := pws[:0] for _, pw := range pws { if !partsToRemove[pw] { dst = append(dst, pw) - continue } - removedParts++ } - return dst, removedParts + for i := len(dst); i < len(pws); i++ { + pws[i] = nil + } + return dst, len(pws) - len(dst) } func isSpecialDir(name string) bool { diff --git a/lib/mergeset/table_test.go b/lib/mergeset/table_test.go index 6a7968537..eff20bcb5 100644 --- a/lib/mergeset/table_test.go +++ b/lib/mergeset/table_test.go @@ -90,8 +90,8 @@ func TestTableAddItemsSerial(t *testing.T) { var m TableMetrics tb.UpdateMetrics(&m) - if m.ItemsCount != itemsCount { - t.Fatalf("unexpected itemsCount; got %d; want %v", m.ItemsCount, itemsCount) + if n := m.TotalItemsCount(); n != itemsCount { + t.Fatalf("unexpected itemsCount; got %d; want %v", n, itemsCount) } tb.MustClose() @@ -235,8 +235,8 @@ func TestTableAddItemsConcurrent(t *testing.T) { var m TableMetrics tb.UpdateMetrics(&m) - if m.ItemsCount != itemsCount { - t.Fatalf("unexpected itemsCount; got %d; want %v", m.ItemsCount, itemsCount) + if n := m.TotalItemsCount(); n != itemsCount { + t.Fatalf("unexpected itemsCount; got %d; want %v", n, itemsCount) } tb.MustClose() @@ -292,8 +292,8 @@ func testReopenTable(t *testing.T, path string, itemsCount int) { } var m TableMetrics tb.UpdateMetrics(&m) - if m.ItemsCount != uint64(itemsCount) { - t.Fatalf("unexpected itemsCount after re-opening; got %d; want %v", m.ItemsCount, itemsCount) + if n := m.TotalItemsCount(); n != uint64(itemsCount) { + t.Fatalf("unexpected itemsCount after re-opening; got %d; want %v", n, itemsCount) } tb.MustClose() } diff --git a/lib/storage/index_db_test.go b/lib/storage/index_db_test.go index 5268a24fb..4bf20312b 100644 --- a/lib/storage/index_db_test.go +++ b/lib/storage/index_db_test.go @@ -1480,8 +1480,8 @@ func TestIndexDBRepopulateAfterRotation(t *testing.T) { // verify the storage contains rows. var m Metrics s.UpdateMetrics(&m) - if m.TableMetrics.SmallRowsCount < uint64(metricRowsN) { - t.Fatalf("expecting at least %d rows in the table; got %d", metricRowsN, m.TableMetrics.SmallRowsCount) + if rowsCount := m.TableMetrics.TotalRowsCount(); rowsCount < uint64(metricRowsN) { + t.Fatalf("expecting at least %d rows in the table; got %d", metricRowsN, rowsCount) } // check new series were registered in indexDB diff --git a/lib/storage/inmemory_part.go b/lib/storage/inmemory_part.go index b0681c984..70f05c15f 100644 --- a/lib/storage/inmemory_part.go +++ b/lib/storage/inmemory_part.go @@ -1,9 +1,13 @@ package storage import ( + "fmt" + "path/filepath" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/bytesutil" "github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup" "github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/fs" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" ) @@ -31,6 +35,36 @@ func (mp *inmemoryPart) Reset() { mp.creationTime = 0 } +// StoreToDisk stores the mp to the given path on disk. +func (mp *inmemoryPart) StoreToDisk(path string) error { + if err := fs.MkdirAllIfNotExist(path); err != nil { + return fmt.Errorf("cannot create directory %q: %w", path, err) + } + timestampsPath := path + "/timestamps.bin" + if err := fs.WriteFileAndSync(timestampsPath, mp.timestampsData.B); err != nil { + return fmt.Errorf("cannot store timestamps: %w", err) + } + valuesPath := path + "/values.bin" + if err := fs.WriteFileAndSync(valuesPath, mp.valuesData.B); err != nil { + return fmt.Errorf("cannot store values: %w", err) + } + indexPath := path + "/index.bin" + if err := fs.WriteFileAndSync(indexPath, mp.indexData.B); err != nil { + return fmt.Errorf("cannot store index: %w", err) + } + metaindexPath := path + "/metaindex.bin" + if err := fs.WriteFileAndSync(metaindexPath, mp.metaindexData.B); err != nil { + return fmt.Errorf("cannot store metaindex: %w", err) + } + if err := mp.ph.writeMinDedupInterval(path); err != nil { + return fmt.Errorf("cannot store min dedup interval: %w", err) + } + // Sync parent directory in order to make sure the written files remain visible after hardware reset + parentDirPath := filepath.Dir(path) + fs.MustSyncPath(parentDirPath) + return nil +} + // InitFromRows initializes mp from the given rows. func (mp *inmemoryPart) InitFromRows(rows []rawRow) { if len(rows) == 0 { diff --git a/lib/storage/partition.go b/lib/storage/partition.go index 17e8aa8ba..714a50f8b 100644 --- a/lib/storage/partition.go +++ b/lib/storage/partition.go @@ -19,33 +19,19 @@ import ( "github.com/VictoriaMetrics/VictoriaMetrics/lib/fs" "github.com/VictoriaMetrics/VictoriaMetrics/lib/logger" "github.com/VictoriaMetrics/VictoriaMetrics/lib/memory" + "github.com/VictoriaMetrics/VictoriaMetrics/lib/mergeset" "github.com/VictoriaMetrics/VictoriaMetrics/lib/storagepacelimiter" "github.com/VictoriaMetrics/VictoriaMetrics/lib/syncwg" ) -func maxSmallPartSize() uint64 { - // Small parts are cached in the OS page cache, - // so limit their size by the remaining free RAM. - mem := memory.Remaining() - // It is expected no more than defaultPartsToMerge/2 parts exist - // in the OS page cache before they are merged into bigger part. - // Half of the remaining RAM must be left for lib/mergeset parts, - // so the maxItems is calculated using the below code: - maxSize := uint64(mem) / defaultPartsToMerge - if maxSize < 10e6 { - maxSize = 10e6 - } - return maxSize -} - // The maximum size of big part. // // This number limits the maximum time required for building big part. // This time shouldn't exceed a few days. const maxBigPartSize = 1e12 -// The maximum number of small parts in the partition. -const maxSmallPartsPerPartition = 256 +// The maximum number of inmemory parts in the partition. +const maxInmemoryPartsPerPartition = 32 // Default number of parts to merge at once. // @@ -65,6 +51,25 @@ const finalPartsToMerge = 3 // Higher number of shards reduces CPU contention and increases the max bandwidth on multi-core systems. var rawRowsShardsPerPartition = (cgroup.AvailableCPUs() + 1) / 2 +// The interval for flushing bufferred rows into parts, so they become visible to search. +const pendingRowsFlushInterval = time.Second + +// The interval for guaranteed flush of recently ingested data from memory to on-disk parts, +// so they survive process crash. +var dataFlushInterval = 5 * time.Second + +// SetDataFlushInterval sets the interval for guaranteed flush of recently ingested data from memory to disk. +// +// The data can be flushed from memory to disk more frequently if it doesn't fit the memory limit. +// +// This function must be called before initializing the storage. +func SetDataFlushInterval(d time.Duration) { + if d > pendingRowsFlushInterval { + dataFlushInterval = d + mergeset.SetDataFlushInterval(d) + } +} + // getMaxRawRowsPerShard returns the maximum number of rows that haven't been converted into parts yet. func getMaxRawRowsPerShard() int { maxRawRowsPerPartitionOnce.Do(func() { @@ -85,32 +90,30 @@ var ( maxRawRowsPerPartitionOnce sync.Once ) -// The interval for flushing (converting) recent raw rows into parts, -// so they become visible to search. -const rawRowsFlushInterval = time.Second - -// The interval for flushing inmemory parts to persistent storage, -// so they survive process crash. -const inmemoryPartsFlushInterval = 5 * time.Second - // partition represents a partition. type partition struct { // Put atomic counters to the top of struct, so they are aligned to 8 bytes on 32-bit arch. // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/212 - activeBigMerges uint64 - activeSmallMerges uint64 - bigMergesCount uint64 - smallMergesCount uint64 - bigRowsMerged uint64 - smallRowsMerged uint64 - bigRowsDeleted uint64 - smallRowsDeleted uint64 + activeInmemoryMerges uint64 + activeSmallMerges uint64 + activeBigMerges uint64 - smallAssistedMerges uint64 + inmemoryMergesCount uint64 + smallMergesCount uint64 + bigMergesCount uint64 - smallMergeNeedFreeDiskSpace uint64 - bigMergeNeedFreeDiskSpace uint64 + inmemoryRowsMerged uint64 + smallRowsMerged uint64 + bigRowsMerged uint64 + + inmemoryRowsDeleted uint64 + smallRowsDeleted uint64 + bigRowsDeleted uint64 + + inmemoryAssistedMerges uint64 + + mergeNeedFreeDiskSpace uint64 mergeIdx uint64 @@ -126,21 +129,24 @@ type partition struct { // The time range for the partition. Usually this is a whole month. tr TimeRange - // partsLock protects smallParts and bigParts. + // rawRows contains recently added rows that haven't been converted into parts yet. + // rawRows are periodically converted into inmemroyParts. + // rawRows aren't used in search for performance reasons. + rawRows rawRowsShards + + // partsLock protects inmemoryParts, smallParts and bigParts. partsLock sync.Mutex - // Contains all the inmemoryPart plus file-based parts - // with small number of items (up to maxRowsCountPerSmallPart). + // Contains inmemory parts with recently ingested data. + // It must be merged into either smallParts or bigParts to become visible to search. + inmemoryParts []*partWrapper + + // Contains file-based parts with small number of items. smallParts []*partWrapper // Contains file-based parts with big number of items. bigParts []*partWrapper - // rawRows contains recently added rows that haven't been converted into parts yet. - // - // rawRows aren't used in search for performance reasons. - rawRows rawRowsShards - snapshotLock sync.RWMutex stopCh chan struct{} @@ -164,6 +170,9 @@ type partWrapper struct { // Whether the part is in merge now. isInMerge bool + + // The deadline when in-memory part must be flushed to disk. + flushToDiskDeadline time.Time } func (pw *partWrapper) incRef() { @@ -213,8 +222,8 @@ func createPartition(timestamp int64, smallPartitionsPath, bigPartitionsPath str func (pt *partition) startBackgroundWorkers() { pt.startMergeWorkers() - pt.startRawRowsFlusher() pt.startInmemoryPartsFlusher() + pt.startPendingRowsFlusher() pt.startStalePartsRemover() } @@ -292,67 +301,83 @@ type partitionMetrics struct { IndexBlocksCacheRequests uint64 IndexBlocksCacheMisses uint64 - BigSizeBytes uint64 - SmallSizeBytes uint64 + InmemorySizeBytes uint64 + SmallSizeBytes uint64 + BigSizeBytes uint64 - BigRowsCount uint64 - SmallRowsCount uint64 + InmemoryRowsCount uint64 + SmallRowsCount uint64 + BigRowsCount uint64 - BigBlocksCount uint64 - SmallBlocksCount uint64 + InmemoryBlocksCount uint64 + SmallBlocksCount uint64 + BigBlocksCount uint64 - BigPartsCount uint64 - SmallPartsCount uint64 + InmemoryPartsCount uint64 + SmallPartsCount uint64 + BigPartsCount uint64 - ActiveBigMerges uint64 - ActiveSmallMerges uint64 + ActiveInmemoryMerges uint64 + ActiveSmallMerges uint64 + ActiveBigMerges uint64 - BigMergesCount uint64 - SmallMergesCount uint64 + InmemoryMergesCount uint64 + SmallMergesCount uint64 + BigMergesCount uint64 - BigRowsMerged uint64 - SmallRowsMerged uint64 + InmemoryRowsMerged uint64 + SmallRowsMerged uint64 + BigRowsMerged uint64 - BigRowsDeleted uint64 - SmallRowsDeleted uint64 + InmemoryRowsDeleted uint64 + SmallRowsDeleted uint64 + BigRowsDeleted uint64 - BigPartsRefCount uint64 - SmallPartsRefCount uint64 + InmemoryPartsRefCount uint64 + SmallPartsRefCount uint64 + BigPartsRefCount uint64 - SmallAssistedMerges uint64 + InmemoryAssistedMerges uint64 - SmallMergeNeedFreeDiskSpace uint64 - BigMergeNeedFreeDiskSpace uint64 + MergeNeedFreeDiskSpace uint64 +} + +// TotalRowsCount returns total number of rows in tm. +func (pm *partitionMetrics) TotalRowsCount() uint64 { + return pm.PendingRows + pm.InmemoryRowsCount + pm.SmallRowsCount + pm.BigRowsCount } // UpdateMetrics updates m with metrics from pt. func (pt *partition) UpdateMetrics(m *partitionMetrics) { - rawRowsLen := uint64(pt.rawRows.Len()) - m.PendingRows += rawRowsLen - m.SmallRowsCount += rawRowsLen + m.PendingRows += uint64(pt.rawRows.Len()) pt.partsLock.Lock() + for _, pw := range pt.inmemoryParts { + p := pw.p + m.InmemoryRowsCount += p.ph.RowsCount + m.InmemoryBlocksCount += p.ph.BlocksCount + m.InmemorySizeBytes += p.size + m.InmemoryPartsRefCount += atomic.LoadUint64(&pw.refCount) + } + for _, pw := range pt.smallParts { + p := pw.p + m.SmallRowsCount += p.ph.RowsCount + m.SmallBlocksCount += p.ph.BlocksCount + m.SmallSizeBytes += p.size + m.SmallPartsRefCount += atomic.LoadUint64(&pw.refCount) + } for _, pw := range pt.bigParts { p := pw.p - m.BigRowsCount += p.ph.RowsCount m.BigBlocksCount += p.ph.BlocksCount m.BigSizeBytes += p.size m.BigPartsRefCount += atomic.LoadUint64(&pw.refCount) } - for _, pw := range pt.smallParts { - p := pw.p - - m.SmallRowsCount += p.ph.RowsCount - m.SmallBlocksCount += p.ph.BlocksCount - m.SmallSizeBytes += p.size - m.SmallPartsRefCount += atomic.LoadUint64(&pw.refCount) - } - - m.BigPartsCount += uint64(len(pt.bigParts)) + m.InmemoryPartsCount += uint64(len(pt.inmemoryParts)) m.SmallPartsCount += uint64(len(pt.smallParts)) + m.BigPartsCount += uint64(len(pt.bigParts)) pt.partsLock.Unlock() @@ -362,22 +387,25 @@ func (pt *partition) UpdateMetrics(m *partitionMetrics) { m.IndexBlocksCacheRequests = ibCache.Requests() m.IndexBlocksCacheMisses = ibCache.Misses() - m.ActiveBigMerges += atomic.LoadUint64(&pt.activeBigMerges) + m.ActiveInmemoryMerges += atomic.LoadUint64(&pt.activeInmemoryMerges) m.ActiveSmallMerges += atomic.LoadUint64(&pt.activeSmallMerges) + m.ActiveBigMerges += atomic.LoadUint64(&pt.activeBigMerges) - m.BigMergesCount += atomic.LoadUint64(&pt.bigMergesCount) + m.InmemoryMergesCount += atomic.LoadUint64(&pt.inmemoryMergesCount) m.SmallMergesCount += atomic.LoadUint64(&pt.smallMergesCount) + m.BigMergesCount += atomic.LoadUint64(&pt.bigMergesCount) - m.BigRowsMerged += atomic.LoadUint64(&pt.bigRowsMerged) + m.InmemoryRowsMerged += atomic.LoadUint64(&pt.inmemoryRowsMerged) m.SmallRowsMerged += atomic.LoadUint64(&pt.smallRowsMerged) + m.BigRowsMerged += atomic.LoadUint64(&pt.bigRowsMerged) - m.BigRowsDeleted += atomic.LoadUint64(&pt.bigRowsDeleted) + m.InmemoryRowsDeleted += atomic.LoadUint64(&pt.inmemoryRowsDeleted) m.SmallRowsDeleted += atomic.LoadUint64(&pt.smallRowsDeleted) + m.BigRowsDeleted += atomic.LoadUint64(&pt.bigRowsDeleted) - m.SmallAssistedMerges += atomic.LoadUint64(&pt.smallAssistedMerges) + m.InmemoryAssistedMerges += atomic.LoadUint64(&pt.inmemoryAssistedMerges) - m.SmallMergeNeedFreeDiskSpace += atomic.LoadUint64(&pt.smallMergeNeedFreeDiskSpace) - m.BigMergeNeedFreeDiskSpace += atomic.LoadUint64(&pt.bigMergeNeedFreeDiskSpace) + m.MergeNeedFreeDiskSpace += atomic.LoadUint64(&pt.mergeNeedFreeDiskSpace) } // AddRows adds the given rows to the partition pt. @@ -415,11 +443,13 @@ func (rrss *rawRowsShards) init() { } func (rrss *rawRowsShards) addRows(pt *partition, rows []rawRow) { - n := atomic.AddUint32(&rrss.shardIdx, 1) shards := rrss.shards - idx := n % uint32(len(shards)) - shard := &shards[idx] - shard.addRows(pt, rows) + shardsLen := uint32(len(shards)) + for len(rows) > 0 { + n := atomic.AddUint32(&rrss.shardIdx, 1) + idx := n % shardsLen + rows = shards[idx].addRows(pt, rows) + } } func (rrss *rawRowsShards) Len() int { @@ -453,8 +483,8 @@ func (rrs *rawRowsShard) Len() int { return n } -func (rrs *rawRowsShard) addRows(pt *partition, rows []rawRow) { - var rowsToFlush []rawRow +func (rrs *rawRowsShard) addRows(pt *partition, rows []rawRow) []rawRow { + var rrb *rawRowsBlock rrs.mu.Lock() if cap(rrs.rows) == 0 { @@ -464,23 +494,25 @@ func (rrs *rawRowsShard) addRows(pt *partition, rows []rawRow) { rrs.rows = rrs.rows[:len(rrs.rows)+n] rows = rows[n:] if len(rows) > 0 { - // Slow path - rows did't fit rrs.rows capacity. - // Convert rrs.rows to rowsToFlush and convert it to a part, - // then try moving the remaining rows to rrs.rows. - rowsToFlush = rrs.rows - rrs.rows = newRawRowsBlock() - if len(rows) <= n { - rrs.rows = append(rrs.rows[:0], rows...) - } else { - // The slowest path - rows do not fit rrs.rows capacity. - // So append them directly to rowsToFlush. - rowsToFlush = append(rowsToFlush, rows...) - } + rrb = getRawRowsBlock() + rrb.rows, rrs.rows = rrs.rows, rrb.rows + n = copy(rrs.rows[:cap(rrs.rows)], rows) + rrs.rows = rrs.rows[:n] + rows = rows[n:] atomic.StoreUint64(&rrs.lastFlushTime, fasttime.UnixTimestamp()) } rrs.mu.Unlock() - pt.flushRowsToParts(rowsToFlush) + if rrb != nil { + pt.flushRowsToParts(rrb.rows) + putRawRowsBlock(rrb) + } + + return rows +} + +type rawRowsBlock struct { + rows []rawRow } func newRawRowsBlock() []rawRow { @@ -488,8 +520,30 @@ func newRawRowsBlock() []rawRow { return make([]rawRow, 0, n) } +func getRawRowsBlock() *rawRowsBlock { + v := rawRowsBlockPool.Get() + if v == nil { + return &rawRowsBlock{ + rows: newRawRowsBlock(), + } + } + return v.(*rawRowsBlock) +} + +func putRawRowsBlock(rrb *rawRowsBlock) { + rrb.rows = rrb.rows[:0] + rawRowsBlockPool.Put(rrb) +} + +var rawRowsBlockPool sync.Pool + func (pt *partition) flushRowsToParts(rows []rawRow) { + if len(rows) == 0 { + return + } maxRows := getMaxRawRowsPerShard() + var pwsLock sync.Mutex + pws := make([]*partWrapper, 0, (len(rows)+maxRows-1)/maxRows) wg := getWaitGroup() for len(rows) > 0 { n := maxRows @@ -497,14 +551,73 @@ func (pt *partition) flushRowsToParts(rows []rawRow) { n = len(rows) } wg.Add(1) - go func(rowsPart []rawRow) { - defer wg.Done() - pt.addRowsPart(rowsPart) + flushConcurrencyCh <- struct{}{} + go func(rowsChunk []rawRow) { + defer func() { + <-flushConcurrencyCh + wg.Done() + }() + pw := pt.createInmemoryPart(rowsChunk) + if pw == nil { + return + } + pwsLock.Lock() + pws = append(pws, pw) + pwsLock.Unlock() }(rows[:n]) rows = rows[n:] } wg.Wait() putWaitGroup(wg) + + pt.partsLock.Lock() + pt.inmemoryParts = append(pt.inmemoryParts, pws...) + pt.partsLock.Unlock() + + flushConcurrencyCh <- struct{}{} + pt.assistedMergeForInmemoryParts() + <-flushConcurrencyCh + // There is no need in assisted merges for small and big parts, + // since the bottleneck is possible only at inmemory parts. +} + +var flushConcurrencyCh = make(chan struct{}, cgroup.AvailableCPUs()) + +func (pt *partition) assistedMergeForInmemoryParts() { + for { + pt.partsLock.Lock() + ok := getNotInMergePartsCount(pt.inmemoryParts) < maxInmemoryPartsPerPartition + pt.partsLock.Unlock() + if ok { + return + } + + // There are too many unmerged inmemory parts. + // This usually means that the app cannot keep up with the data ingestion rate. + // Assist with mering inmemory parts. + // Prioritize assisted merges over searches. + storagepacelimiter.Search.Inc() + err := pt.mergeInmemoryParts() + storagepacelimiter.Search.Dec() + if err == nil { + atomic.AddUint64(&pt.inmemoryAssistedMerges, 1) + continue + } + if errors.Is(err, errNothingToMerge) || errors.Is(err, errForciblyStopped) { + return + } + logger.Panicf("FATAL: cannot merge inmemory parts: %s", err) + } +} + +func getNotInMergePartsCount(pws []*partWrapper) int { + n := 0 + for _, pw := range pws { + if !pw.isInMerge { + n++ + } + } + return n } func getWaitGroup() *sync.WaitGroup { @@ -521,11 +634,10 @@ func putWaitGroup(wg *sync.WaitGroup) { var wgPool sync.Pool -func (pt *partition) addRowsPart(rows []rawRow) { +func (pt *partition) createInmemoryPart(rows []rawRow) *partWrapper { if len(rows) == 0 { - return + return nil } - mp := getInmemoryPart() mp.InitFromRows(rows) @@ -542,40 +654,22 @@ func (pt *partition) addRowsPart(rows []rawRow) { logger.Panicf("BUG: the part %q cannot be added to partition %q because of too big MaxTimestamp; got %d; want at least %d", &mp.ph, pt.smallPartsPath, mp.ph.MaxTimestamp, pt.tr.MaxTimestamp) } + flushToDiskDeadline := time.Now().Add(dataFlushInterval) + return newPartWrapperFromInmemoryPart(mp, flushToDiskDeadline) +} +func newPartWrapperFromInmemoryPart(mp *inmemoryPart, flushToDiskDeadline time.Time) *partWrapper { p, err := mp.NewPart() if err != nil { logger.Panicf("BUG: cannot create part from %q: %s", &mp.ph, err) } - pw := &partWrapper{ - p: p, - mp: mp, - refCount: 1, + p: p, + mp: mp, + refCount: 1, + flushToDiskDeadline: flushToDiskDeadline, } - - pt.partsLock.Lock() - pt.smallParts = append(pt.smallParts, pw) - ok := len(pt.smallParts) <= maxSmallPartsPerPartition - pt.partsLock.Unlock() - if ok { - return - } - - // The added part exceeds available limit. Help merging parts. - // - // Prioritize assisted merges over searches. - storagepacelimiter.Search.Inc() - err = pt.mergeSmallParts(false) - storagepacelimiter.Search.Dec() - if err == nil { - atomic.AddUint64(&pt.smallAssistedMerges, 1) - return - } - if errors.Is(err, errNothingToMerge) || errors.Is(err, errForciblyStopped) || errors.Is(err, errReadOnlyMode) { - return - } - logger.Panicf("FATAL: cannot merge small parts: %s", err) + return pw } // HasTimestamp returns true if the pt contains the given timestamp. @@ -588,6 +682,10 @@ func (pt *partition) HasTimestamp(timestamp int64) bool { // The appended parts must be released with PutParts. func (pt *partition) GetParts(dst []*partWrapper) []*partWrapper { pt.partsLock.Lock() + for _, pw := range pt.inmemoryParts { + pw.incRef() + } + dst = append(dst, pt.inmemoryParts...) for _, pw := range pt.smallParts { pw.incRef() } @@ -625,93 +723,132 @@ func (pt *partition) MustClose() { logger.Infof("flushing inmemory parts to files on %q...", pt.smallPartsPath) startTime = time.Now() - // Flush raw rows the last time before exit. - pt.flushPendingRows(true) + // Flush inmemory rows the last time before exit. + pt.flushInmemoryRows() - // Flush inmemory parts to disk. - var pws []*partWrapper - pt.partsLock.Lock() - for _, pw := range pt.smallParts { - if pw.mp == nil { - continue - } - if pw.isInMerge { - logger.Panicf("BUG: the inmemory part %q mustn't be in merge after stopping small parts merger in the partition %q", &pw.mp.ph, pt.smallPartsPath) - } - pw.isInMerge = true - pws = append(pws, pw) - } - pt.partsLock.Unlock() - - if err := pt.mergePartsOptimal(pws, nil); err != nil { - logger.Panicf("FATAL: cannot flush %d inmemory parts to files on %q: %s", len(pws), pt.smallPartsPath, err) - } - logger.Infof("%d inmemory parts have been flushed to files in %.3f seconds on %q", len(pws), time.Since(startTime).Seconds(), pt.smallPartsPath) - - // Remove references to smallParts from the pt, so they may be eventually closed + // Remove references from inmemoryParts, smallParts and bigParts, so they may be eventually closed // after all the searches are done. pt.partsLock.Lock() + inmemoryParts := pt.inmemoryParts smallParts := pt.smallParts - pt.smallParts = nil - pt.partsLock.Unlock() - - for _, pw := range smallParts { - pw.decRef() - } - - // Remove references to bigParts from the pt, so they may be eventually closed - // after all the searches are done. - pt.partsLock.Lock() bigParts := pt.bigParts + pt.inmemoryParts = nil + pt.smallParts = nil pt.bigParts = nil pt.partsLock.Unlock() + for _, pw := range inmemoryParts { + pw.decRef() + } + for _, pw := range smallParts { + pw.decRef() + } for _, pw := range bigParts { pw.decRef() } } -func (pt *partition) startRawRowsFlusher() { +func (pt *partition) startInmemoryPartsFlusher() { pt.wg.Add(1) go func() { - pt.rawRowsFlusher() + pt.inmemoryPartsFlusher() pt.wg.Done() }() } -func (pt *partition) rawRowsFlusher() { - ticker := time.NewTicker(rawRowsFlushInterval) +func (pt *partition) startPendingRowsFlusher() { + pt.wg.Add(1) + go func() { + pt.pendingRowsFlusher() + pt.wg.Done() + }() +} + +func (pt *partition) inmemoryPartsFlusher() { + ticker := time.NewTicker(dataFlushInterval) defer ticker.Stop() for { select { case <-pt.stopCh: return case <-ticker.C: - pt.flushPendingRows(false) + pt.flushInmemoryParts(false) } } } -func (pt *partition) flushPendingRows(isFinal bool) { - pt.rawRows.flush(pt, isFinal) +func (pt *partition) pendingRowsFlusher() { + ticker := time.NewTicker(pendingRowsFlushInterval) + defer ticker.Stop() + var rows []rawRow + for { + select { + case <-pt.stopCh: + return + case <-ticker.C: + rows = pt.flushPendingRows(rows[:0], false) + } + } } -func (rrss *rawRowsShards) flush(pt *partition, isFinal bool) { - var rowsToFlush []rawRow - for i := range rrss.shards { - rowsToFlush = rrss.shards[i].appendRawRowsToFlush(rowsToFlush, pt, isFinal) +func (pt *partition) flushPendingRows(dst []rawRow, isFinal bool) []rawRow { + return pt.rawRows.flush(pt, dst, isFinal) +} + +func (pt *partition) flushInmemoryRows() { + pt.rawRows.flush(pt, nil, true) + pt.flushInmemoryParts(true) +} + +func (pt *partition) flushInmemoryParts(isFinal bool) { + for { + currentTime := time.Now() + var pws []*partWrapper + + pt.partsLock.Lock() + for _, pw := range pt.inmemoryParts { + if !pw.isInMerge && (isFinal || pw.flushToDiskDeadline.Before(currentTime)) { + pw.isInMerge = true + pws = append(pws, pw) + } + } + pt.partsLock.Unlock() + + if err := pt.mergePartsOptimal(pws, nil); err != nil { + logger.Panicf("FATAL: cannot merge in-memory parts: %s", err) + } + if !isFinal { + return + } + pt.partsLock.Lock() + n := len(pt.inmemoryParts) + pt.partsLock.Unlock() + if n == 0 { + // All the in-memory parts were flushed to disk. + return + } + // Some parts weren't flushed to disk because they were being merged. + // Sleep for a while and try flushing them again. + time.Sleep(10 * time.Millisecond) } - pt.flushRowsToParts(rowsToFlush) +} + +func (rrss *rawRowsShards) flush(pt *partition, dst []rawRow, isFinal bool) []rawRow { + for i := range rrss.shards { + dst = rrss.shards[i].appendRawRowsToFlush(dst, pt, isFinal) + } + pt.flushRowsToParts(dst) + return dst } func (rrs *rawRowsShard) appendRawRowsToFlush(dst []rawRow, pt *partition, isFinal bool) []rawRow { currentTime := fasttime.UnixTimestamp() - flushSeconds := int64(rawRowsFlushInterval.Seconds()) + flushSeconds := int64(pendingRowsFlushInterval.Seconds()) if flushSeconds <= 0 { flushSeconds = 1 } lastFlushTime := atomic.LoadUint64(&rrs.lastFlushTime) - if !isFinal && currentTime <= lastFlushTime+uint64(flushSeconds) { + if !isFinal && currentTime < lastFlushTime+uint64(flushSeconds) { // Fast path - nothing to flush return dst } @@ -724,112 +861,73 @@ func (rrs *rawRowsShard) appendRawRowsToFlush(dst []rawRow, pt *partition, isFin return dst } -func (pt *partition) startInmemoryPartsFlusher() { - pt.wg.Add(1) - go func() { - pt.inmemoryPartsFlusher() - pt.wg.Done() - }() -} - -func (pt *partition) inmemoryPartsFlusher() { - ticker := time.NewTicker(inmemoryPartsFlushInterval) - defer ticker.Stop() - var pwsBuf []*partWrapper - var err error - for { - select { - case <-pt.stopCh: - return - case <-ticker.C: - pwsBuf, err = pt.flushInmemoryParts(pwsBuf[:0], false) - if err != nil { - logger.Panicf("FATAL: cannot flush inmemory parts: %s", err) - } +func (pt *partition) mergePartsOptimal(pws []*partWrapper, stopCh <-chan struct{}) error { + sortPartsForOptimalMerge(pws) + for len(pws) > 0 { + n := defaultPartsToMerge + if n > len(pws) { + n = len(pws) } - } -} - -func (pt *partition) flushInmemoryParts(dstPws []*partWrapper, force bool) ([]*partWrapper, error) { - currentTime := fasttime.UnixTimestamp() - flushSeconds := int64(inmemoryPartsFlushInterval.Seconds()) - if flushSeconds <= 0 { - flushSeconds = 1 - } - - // Inmemory parts may present only in small parts. - pt.partsLock.Lock() - for _, pw := range pt.smallParts { - if pw.mp == nil || pw.isInMerge { + pwsChunk := pws[:n] + pws = pws[n:] + err := pt.mergeParts(pwsChunk, stopCh, true) + if err == nil { continue } - if force || currentTime-pw.mp.creationTime >= uint64(flushSeconds) { - pw.isInMerge = true - dstPws = append(dstPws, pw) + pt.releasePartsToMerge(pws) + if errors.Is(err, errForciblyStopped) { + return nil } - } - pt.partsLock.Unlock() - - if err := pt.mergePartsOptimal(dstPws, nil); err != nil { - return dstPws, fmt.Errorf("cannot merge %d inmemory parts: %w", len(dstPws), err) - } - return dstPws, nil -} - -func (pt *partition) mergePartsOptimal(pws []*partWrapper, stopCh <-chan struct{}) error { - for len(pws) > defaultPartsToMerge { - pwsChunk := pws[:defaultPartsToMerge] - pws = pws[defaultPartsToMerge:] - if err := pt.mergeParts(pwsChunk, stopCh); err != nil { - pt.releasePartsToMerge(pws) - return fmt.Errorf("cannot merge %d parts: %w", defaultPartsToMerge, err) - } - } - if len(pws) == 0 { - return nil - } - if err := pt.mergeParts(pws, stopCh); err != nil { - return fmt.Errorf("cannot merge %d parts: %w", len(pws), err) + return fmt.Errorf("cannot merge parts optimally: %w", err) } return nil } -// ForceMergeAllParts runs merge for all the parts in pt - small and big. +// ForceMergeAllParts runs merge for all the parts in pt. func (pt *partition) ForceMergeAllParts() error { - var pws []*partWrapper - pt.partsLock.Lock() - if !hasActiveMerges(pt.smallParts) && !hasActiveMerges(pt.bigParts) { - pws = appendAllPartsToMerge(pws, pt.smallParts) - pws = appendAllPartsToMerge(pws, pt.bigParts) - } - pt.partsLock.Unlock() - + pws := pt.getAllPartsForMerge() if len(pws) == 0 { // Nothing to merge. return nil } + for { + // Check whether there is enough disk space for merging pws. + newPartSize := getPartsSize(pws) + maxOutBytes := fs.MustGetFreeSpace(pt.bigPartsPath) + if newPartSize > maxOutBytes { + freeSpaceNeededBytes := newPartSize - maxOutBytes + forceMergeLogger.Warnf("cannot initiate force merge for the partition %s; additional space needed: %d bytes", pt.name, freeSpaceNeededBytes) + return nil + } - // Check whether there is enough disk space for merging pws. - newPartSize := getPartsSize(pws) - maxOutBytes := fs.MustGetFreeSpace(pt.bigPartsPath) - if newPartSize > maxOutBytes { - freeSpaceNeededBytes := newPartSize - maxOutBytes - forceMergeLogger.Warnf("cannot initiate force merge for the partition %s; additional space needed: %d bytes", pt.name, freeSpaceNeededBytes) - return nil + // If len(pws) == 1, then the merge must run anyway. + // This allows applying the configured retention, removing the deleted series + // and performing de-duplication if needed. + if err := pt.mergePartsOptimal(pws, pt.stopCh); err != nil { + return fmt.Errorf("cannot force merge %d parts from partition %q: %w", len(pws), pt.name, err) + } + pws = pt.getAllPartsForMerge() + if len(pws) <= 1 { + return nil + } } - - // If len(pws) == 1, then the merge must run anyway. - // This allows applying the configured retention, removing the deleted series - // and performing de-duplication if needed. - if err := pt.mergePartsOptimal(pws, pt.stopCh); err != nil { - return fmt.Errorf("cannot force merge %d parts from partition %q: %w", len(pws), pt.name, err) - } - return nil } var forceMergeLogger = logger.WithThrottler("forceMerge", time.Minute) -func appendAllPartsToMerge(dst, src []*partWrapper) []*partWrapper { +func (pt *partition) getAllPartsForMerge() []*partWrapper { + var pws []*partWrapper + pt.partsLock.Lock() + if !hasActiveMerges(pt.inmemoryParts) && !hasActiveMerges(pt.smallParts) && !hasActiveMerges(pt.bigParts) { + pws = appendAllPartsForMerge(pws, pt.inmemoryParts) + pws = appendAllPartsForMerge(pws, pt.smallParts) + pws = appendAllPartsForMerge(pws, pt.bigParts) + } + pt.partsLock.Unlock() + return pws +} + +func appendAllPartsForMerge(dst, src []*partWrapper) []*partWrapper { for _, pw := range src { if pw.isInMerge { logger.Panicf("BUG: part %q is already in merge", pw.p.path) @@ -849,10 +947,9 @@ func hasActiveMerges(pws []*partWrapper) bool { return false } -var ( - bigMergeWorkersCount = getDefaultMergeConcurrency(4) - smallMergeWorkersCount = getDefaultMergeConcurrency(16) -) +var mergeWorkersLimitCh = make(chan struct{}, getDefaultMergeConcurrency(16)) + +var bigMergeWorkersLimitCh = make(chan struct{}, getDefaultMergeConcurrency(4)) func getDefaultMergeConcurrency(max int) int { v := (cgroup.AvailableCPUs() + 1) / 2 @@ -870,47 +967,28 @@ func SetBigMergeWorkersCount(n int) { // Do nothing return } - bigMergeWorkersCount = n + bigMergeWorkersLimitCh = make(chan struct{}, n) } -// SetSmallMergeWorkersCount sets the maximum number of concurrent mergers for small blocks. +// SetMergeWorkersCount sets the maximum number of concurrent mergers for parts. // // The function must be called before opening or creating any storage. -func SetSmallMergeWorkersCount(n int) { +func SetMergeWorkersCount(n int) { if n <= 0 { // Do nothing return } - smallMergeWorkersCount = n + mergeWorkersLimitCh = make(chan struct{}, n) } func (pt *partition) startMergeWorkers() { - for i := 0; i < smallMergeWorkersCount; i++ { + for i := 0; i < cap(mergeWorkersLimitCh); i++ { pt.wg.Add(1) go func() { - pt.smallPartsMerger() + pt.mergeWorker() pt.wg.Done() }() } - for i := 0; i < bigMergeWorkersCount; i++ { - pt.wg.Add(1) - go func() { - pt.bigPartsMerger() - pt.wg.Done() - }() - } -} - -func (pt *partition) bigPartsMerger() { - if err := pt.partsMerger(pt.mergeBigParts); err != nil { - logger.Panicf("FATAL: unrecoverable error when merging big parts in the partition %q: %s", pt.bigPartsPath, err) - } -} - -func (pt *partition) smallPartsMerger() { - if err := pt.partsMerger(pt.mergeSmallParts); err != nil { - logger.Panicf("FATAL: unrecoverable error when merging small parts in the partition %q: %s", pt.smallPartsPath, err) - } } const ( @@ -918,13 +996,16 @@ const ( maxMergeSleepTime = 10 * time.Second ) -func (pt *partition) partsMerger(mergerFunc func(isFinal bool) error) error { +func (pt *partition) mergeWorker() { sleepTime := minMergeSleepTime var lastMergeTime uint64 isFinal := false t := time.NewTimer(sleepTime) for { - err := mergerFunc(isFinal) + // Limit the number of concurrent calls to mergeExistingParts, cine the number of merge + mergeWorkersLimitCh <- struct{}{} + err := pt.mergeExistingParts(isFinal) + <-mergeWorkersLimitCh if err == nil { // Try merging additional parts. sleepTime = minMergeSleepTime @@ -934,10 +1015,11 @@ func (pt *partition) partsMerger(mergerFunc func(isFinal bool) error) error { } if errors.Is(err, errForciblyStopped) { // The merger has been stopped. - return nil + return } if !errors.Is(err, errNothingToMerge) && !errors.Is(err, errReadOnlyMode) { - return err + // Unexpected error. + logger.Panicf("FATAL: unrecoverable error when merging parts in the partition (%q, %q): %s", pt.smallPartsPath, pt.bigPartsPath, err) } if finalMergeDelaySeconds > 0 && fasttime.UnixTimestamp()-lastMergeTime > finalMergeDelaySeconds { // We have free time for merging into bigger parts. @@ -954,7 +1036,7 @@ func (pt *partition) partsMerger(mergerFunc func(isFinal bool) error) error { } select { case <-pt.stopCh: - return nil + return case <-t.C: t.Reset(sleepTime) } @@ -973,6 +1055,40 @@ func SetFinalMergeDelay(delay time.Duration) { return } finalMergeDelaySeconds = uint64(delay.Seconds() + 1) + mergeset.SetFinalMergeDelay(delay) +} + +func getMaxInmemoryPartSize() uint64 { + // Allocate 10% of allowed memory for in-memory parts. + n := uint64(0.1 * float64(memory.Allowed()) / maxInmemoryPartsPerPartition) + if n < 1e6 { + n = 1e6 + } + return n +} + +func (pt *partition) getMaxSmallPartSize() uint64 { + // Small parts are cached in the OS page cache, + // so limit their size by the remaining free RAM. + mem := memory.Remaining() + // It is expected no more than defaultPartsToMerge/2 parts exist + // in the OS page cache before they are merged into bigger part. + // Half of the remaining RAM must be left for lib/mergeset parts, + // so the maxItems is calculated using the below code: + n := uint64(mem) / defaultPartsToMerge + if n < 10e6 { + n = 10e6 + } + // Make sure the output part fits available disk space for small parts. + sizeLimit := getMaxOutBytes(pt.smallPartsPath, cap(mergeWorkersLimitCh)) + if n > sizeLimit { + n = sizeLimit + } + return n +} + +func (pt *partition) getMaxBigPartSize() uint64 { + return getMaxOutBytes(pt.bigPartsPath, cap(bigMergeWorkersLimitCh)) } func getMaxOutBytes(path string, workersCount int) uint64 { @@ -994,56 +1110,35 @@ func (pt *partition) canBackgroundMerge() bool { var errReadOnlyMode = fmt.Errorf("storage is in readonly mode") -func (pt *partition) mergeBigParts(isFinal bool) error { - if !pt.canBackgroundMerge() { - // Do not perform merge in read-only mode, since this may result in disk space shortage. - // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/2603 - return errReadOnlyMode - } - maxOutBytes := getMaxOutBytes(pt.bigPartsPath, bigMergeWorkersCount) +func (pt *partition) mergeInmemoryParts() error { + maxOutBytes := pt.getMaxBigPartSize() pt.partsLock.Lock() - pws, needFreeSpace := getPartsToMerge(pt.bigParts, maxOutBytes, isFinal) + pws, needFreeSpace := getPartsToMerge(pt.inmemoryParts, maxOutBytes, false) pt.partsLock.Unlock() - atomicSetBool(&pt.bigMergeNeedFreeDiskSpace, needFreeSpace) - return pt.mergeParts(pws, pt.stopCh) + atomicSetBool(&pt.mergeNeedFreeDiskSpace, needFreeSpace) + return pt.mergeParts(pws, pt.stopCh, false) } -func (pt *partition) mergeSmallParts(isFinal bool) error { +func (pt *partition) mergeExistingParts(isFinal bool) error { if !pt.canBackgroundMerge() { // Do not perform merge in read-only mode, since this may result in disk space shortage. // See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/2603 return errReadOnlyMode } - // Try merging small parts to a big part at first. - maxBigPartOutBytes := getMaxOutBytes(pt.bigPartsPath, bigMergeWorkersCount) + maxOutBytes := pt.getMaxBigPartSize() + pt.partsLock.Lock() - pws, needFreeSpace := getPartsToMerge(pt.smallParts, maxBigPartOutBytes, isFinal) + dst := make([]*partWrapper, 0, len(pt.inmemoryParts)+len(pt.smallParts)+len(pt.bigParts)) + dst = append(dst, pt.inmemoryParts...) + dst = append(dst, pt.smallParts...) + dst = append(dst, pt.bigParts...) + pws, needFreeSpace := getPartsToMerge(dst, maxOutBytes, isFinal) pt.partsLock.Unlock() - atomicSetBool(&pt.bigMergeNeedFreeDiskSpace, needFreeSpace) - outSize := getPartsSize(pws) - if outSize > maxSmallPartSize() { - // Merge small parts to a big part. - return pt.mergeParts(pws, pt.stopCh) - } - - // Make sure that the output small part fits small parts storage. - maxSmallPartOutBytes := getMaxOutBytes(pt.smallPartsPath, smallMergeWorkersCount) - if outSize <= maxSmallPartOutBytes { - // Merge small parts to a small part. - return pt.mergeParts(pws, pt.stopCh) - } - - // The output small part doesn't fit small parts storage. Try merging small parts according to maxSmallPartOutBytes limit. - pt.releasePartsToMerge(pws) - pt.partsLock.Lock() - pws, needFreeSpace = getPartsToMerge(pt.smallParts, maxSmallPartOutBytes, isFinal) - pt.partsLock.Unlock() - atomicSetBool(&pt.smallMergeNeedFreeDiskSpace, needFreeSpace) - - return pt.mergeParts(pws, pt.stopCh) + atomicSetBool(&pt.mergeNeedFreeDiskSpace, needFreeSpace) + return pt.mergeParts(pws, pt.stopCh, isFinal) } func (pt *partition) releasePartsToMerge(pws []*partWrapper) { @@ -1105,12 +1200,14 @@ func getMinDedupInterval(pws []*partWrapper) int64 { return dMin } -// mergeParts merges pws. +// mergeParts merges pws to a single resulting part. // // Merging is immediately stopped if stopCh is closed. // +// if isFinal is set, then the resulting part will be saved to disk. +// // All the parts inside pws must have isInMerge field set to true. -func (pt *partition) mergeParts(pws []*partWrapper, stopCh <-chan struct{}) error { +func (pt *partition) mergeParts(pws []*partWrapper, stopCh <-chan struct{}, isFinal bool) error { if len(pws) == 0 { // Nothing to merge. return errNothingToMerge @@ -1119,164 +1216,332 @@ func (pt *partition) mergeParts(pws []*partWrapper, stopCh <-chan struct{}) erro startTime := time.Now() + // Initialize destination paths. + dstPartType := pt.getDstPartType(pws, isFinal) + ptPath, tmpPartPath, mergeIdx := pt.getDstPartPaths(dstPartType) + + if dstPartType == partBig { + bigMergeWorkersLimitCh <- struct{}{} + defer func() { + <-bigMergeWorkersLimitCh + }() + } + + if isFinal && len(pws) == 1 && pws[0].mp != nil { + // Fast path: flush a single in-memory part to disk. + mp := pws[0].mp + if tmpPartPath == "" { + logger.Panicf("BUG: tmpPartPath must be non-empty") + } + if err := mp.StoreToDisk(tmpPartPath); err != nil { + return fmt.Errorf("cannot store in-memory part to %q: %w", tmpPartPath, err) + } + pwNew, err := pt.openCreatedPart(&mp.ph, pws, nil, ptPath, tmpPartPath, mergeIdx) + if err != nil { + return fmt.Errorf("cannot atomically register the created part: %w", err) + } + pt.swapSrcWithDstParts(pws, pwNew, dstPartType) + return nil + } + // Prepare BlockStreamReaders for source parts. - bsrs := make([]*blockStreamReader, 0, len(pws)) - defer func() { + bsrs, err := openBlockStreamReaders(pws) + if err != nil { + return err + } + closeBlockStreamReaders := func() { for _, bsr := range bsrs { putBlockStreamReader(bsr) } - }() + bsrs = nil + } + + // Prepare BlockStreamWriter for destination part. + srcSize := uint64(0) + srcRowsCount := uint64(0) + srcBlocksCount := uint64(0) + for _, pw := range pws { + srcSize += pw.p.size + srcRowsCount += pw.p.ph.RowsCount + srcBlocksCount += pw.p.ph.BlocksCount + } + rowsPerBlock := float64(srcRowsCount) / float64(srcBlocksCount) + compressLevel := getCompressLevel(rowsPerBlock) + bsw := getBlockStreamWriter() + var mpNew *inmemoryPart + if dstPartType == partInmemory { + mpNew = getInmemoryPart() + bsw.InitFromInmemoryPart(mpNew, compressLevel) + } else { + if tmpPartPath == "" { + logger.Panicf("BUG: tmpPartPath must be non-empty") + } + nocache := dstPartType == partBig + if err := bsw.InitFromFilePart(tmpPartPath, nocache, compressLevel); err != nil { + closeBlockStreamReaders() + return fmt.Errorf("cannot create destination part at %q: %w", tmpPartPath, err) + } + } + + // Merge source parts to destination part. + ph, err := pt.mergePartsInternal(tmpPartPath, bsw, bsrs, dstPartType, stopCh) + putBlockStreamWriter(bsw) + closeBlockStreamReaders() + if err != nil { + return fmt.Errorf("cannot merge %d parts: %w", len(pws), err) + } + if mpNew != nil { + // Update partHeader for destination inmemory part after the merge. + mpNew.ph = *ph + } + + // Atomically move the created part from tmpPartPath to its destination + // and swap the source parts with the newly created part. + pwNew, err := pt.openCreatedPart(ph, pws, mpNew, ptPath, tmpPartPath, mergeIdx) + if err != nil { + return fmt.Errorf("cannot atomically register the created part: %w", err) + } + pt.swapSrcWithDstParts(pws, pwNew, dstPartType) + + d := time.Since(startTime) + if d <= 30*time.Second { + return nil + } + + // Log stats for long merges. + dstRowsCount := uint64(0) + dstBlocksCount := uint64(0) + dstSize := uint64(0) + dstPartPath := "" + if pwNew != nil { + pDst := pwNew.p + dstRowsCount = pDst.ph.RowsCount + dstBlocksCount = pDst.ph.BlocksCount + dstSize = pDst.size + dstPartPath = pDst.String() + } + durationSecs := d.Seconds() + rowsPerSec := int(float64(srcRowsCount) / durationSecs) + logger.Infof("merged (%d parts, %d rows, %d blocks, %d bytes) into (1 part, %d rows, %d blocks, %d bytes) in %.3f seconds at %d rows/sec to %q", + len(pws), srcRowsCount, srcBlocksCount, srcSize, dstRowsCount, dstBlocksCount, dstSize, durationSecs, rowsPerSec, dstPartPath) + + return nil +} + +func getFlushToDiskDeadline(pws []*partWrapper) time.Time { + d := pws[0].flushToDiskDeadline + for _, pw := range pws[1:] { + if pw.flushToDiskDeadline.Before(d) { + d = pw.flushToDiskDeadline + } + } + return d +} + +type partType int + +var ( + partInmemory = partType(0) + partSmall = partType(1) + partBig = partType(2) +) + +func (pt *partition) getDstPartType(pws []*partWrapper, isFinal bool) partType { + dstPartSize := getPartsSize(pws) + if dstPartSize > pt.getMaxSmallPartSize() { + return partBig + } + if isFinal || dstPartSize > getMaxInmemoryPartSize() { + return partSmall + } + if !areAllInmemoryParts(pws) { + // If at least a single source part is located in file, + // then the destination part must be in file for durability reasons. + return partSmall + } + return partInmemory +} + +func (pt *partition) getDstPartPaths(dstPartType partType) (string, string, uint64) { + ptPath := "" + switch dstPartType { + case partSmall: + ptPath = pt.smallPartsPath + case partBig: + ptPath = pt.bigPartsPath + case partInmemory: + ptPath = pt.smallPartsPath + default: + logger.Panicf("BUG: unknown partType=%d", dstPartType) + } + ptPath = filepath.Clean(ptPath) + mergeIdx := pt.nextMergeIdx() + tmpPartPath := "" + if dstPartType != partInmemory { + tmpPartPath = fmt.Sprintf("%s/tmp/%016X", ptPath, mergeIdx) + } + return ptPath, tmpPartPath, mergeIdx +} + +func openBlockStreamReaders(pws []*partWrapper) ([]*blockStreamReader, error) { + bsrs := make([]*blockStreamReader, 0, len(pws)) for _, pw := range pws { bsr := getBlockStreamReader() if pw.mp != nil { bsr.InitFromInmemoryPart(pw.mp) } else { if err := bsr.InitFromFilePart(pw.p.path); err != nil { - return fmt.Errorf("cannot open source part for merging: %w", err) + for _, bsr := range bsrs { + putBlockStreamReader(bsr) + } + return nil, fmt.Errorf("cannot open source part for merging: %w", err) } } bsrs = append(bsrs, bsr) } + return bsrs, nil +} - outSize := uint64(0) - outRowsCount := uint64(0) - outBlocksCount := uint64(0) - for _, pw := range pws { - outSize += pw.p.size - outRowsCount += pw.p.ph.RowsCount - outBlocksCount += pw.p.ph.BlocksCount - } - isBigPart := outSize > maxSmallPartSize() - nocache := isBigPart - - // Prepare BlockStreamWriter for destination part. - ptPath := pt.smallPartsPath - if isBigPart { - ptPath = pt.bigPartsPath - } - ptPath = filepath.Clean(ptPath) - mergeIdx := pt.nextMergeIdx() - tmpPartPath := fmt.Sprintf("%s/tmp/%016X", ptPath, mergeIdx) - bsw := getBlockStreamWriter() - rowsPerBlock := float64(outRowsCount) / float64(outBlocksCount) - compressLevel := getCompressLevel(rowsPerBlock) - if err := bsw.InitFromFilePart(tmpPartPath, nocache, compressLevel); err != nil { - return fmt.Errorf("cannot create destination part %q: %w", tmpPartPath, err) - } - - // Merge parts. +func (pt *partition) mergePartsInternal(tmpPartPath string, bsw *blockStreamWriter, bsrs []*blockStreamReader, dstPartType partType, stopCh <-chan struct{}) (*partHeader, error) { var ph partHeader - rowsMerged := &pt.smallRowsMerged - rowsDeleted := &pt.smallRowsDeleted - if isBigPart { + var rowsMerged *uint64 + var rowsDeleted *uint64 + var mergesCount *uint64 + var activeMerges *uint64 + switch dstPartType { + case partInmemory: + rowsMerged = &pt.inmemoryRowsMerged + rowsDeleted = &pt.inmemoryRowsDeleted + mergesCount = &pt.inmemoryMergesCount + activeMerges = &pt.activeInmemoryMerges + case partSmall: + rowsMerged = &pt.smallRowsMerged + rowsDeleted = &pt.smallRowsDeleted + mergesCount = &pt.smallMergesCount + activeMerges = &pt.activeSmallMerges + case partBig: rowsMerged = &pt.bigRowsMerged rowsDeleted = &pt.bigRowsDeleted - atomic.AddUint64(&pt.bigMergesCount, 1) - atomic.AddUint64(&pt.activeBigMerges, 1) - } else { - atomic.AddUint64(&pt.smallMergesCount, 1) - atomic.AddUint64(&pt.activeSmallMerges, 1) + mergesCount = &pt.bigMergesCount + activeMerges = &pt.activeBigMerges + default: + logger.Panicf("BUG: unknown partType=%d", dstPartType) } - retentionDeadline := timestampFromTime(startTime) - pt.s.retentionMsecs + retentionDeadline := timestampFromTime(time.Now()) - pt.s.retentionMsecs + atomic.AddUint64(activeMerges, 1) err := mergeBlockStreams(&ph, bsw, bsrs, stopCh, pt.s, retentionDeadline, rowsMerged, rowsDeleted) - if isBigPart { - atomic.AddUint64(&pt.activeBigMerges, ^uint64(0)) - } else { - atomic.AddUint64(&pt.activeSmallMerges, ^uint64(0)) - } - putBlockStreamWriter(bsw) + atomic.AddUint64(activeMerges, ^uint64(0)) + atomic.AddUint64(mergesCount, 1) if err != nil { - return fmt.Errorf("error when merging parts to %q: %w", tmpPartPath, err) + return nil, fmt.Errorf("cannot merge parts to %q: %w", tmpPartPath, err) } - - // Close bsrs. - for _, bsr := range bsrs { - putBlockStreamReader(bsr) + if tmpPartPath != "" { + ph.MinDedupInterval = GetDedupInterval() + if err := ph.writeMinDedupInterval(tmpPartPath); err != nil { + return nil, fmt.Errorf("cannot store min dedup interval: %w", err) + } } - bsrs = nil + return &ph, nil +} - ph.MinDedupInterval = GetDedupInterval() - if err := ph.writeMinDedupInterval(tmpPartPath); err != nil { - return fmt.Errorf("cannot store min dedup interval for part %q: %w", tmpPartPath, err) +func (pt *partition) openCreatedPart(ph *partHeader, pws []*partWrapper, mpNew *inmemoryPart, ptPath, tmpPartPath string, mergeIdx uint64) (*partWrapper, error) { + dstPartPath := "" + if mpNew == nil || !areAllInmemoryParts(pws) { + // Either source or destination parts are located on disk. + // Create a transaction for atomic deleting of old parts and moving new part to its destination on disk. + var bb bytesutil.ByteBuffer + for _, pw := range pws { + if pw.mp == nil { + fmt.Fprintf(&bb, "%s\n", pw.p.path) + } + } + if ph.RowsCount > 0 { + // The destination part may have no rows if they are deleted during the merge. + dstPartPath = ph.Path(ptPath, mergeIdx) + } + fmt.Fprintf(&bb, "%s -> %s\n", tmpPartPath, dstPartPath) + txnPath := fmt.Sprintf("%s/txn/%016X", ptPath, mergeIdx) + if err := fs.WriteFileAtomically(txnPath, bb.B, false); err != nil { + return nil, fmt.Errorf("cannot create transaction file %q: %w", txnPath, err) + } + + // Run the created transaction. + if err := runTransaction(&pt.snapshotLock, pt.smallPartsPath, pt.bigPartsPath, txnPath); err != nil { + return nil, fmt.Errorf("cannot execute transaction %q: %w", txnPath, err) + } } + // Open the created part. + if ph.RowsCount == 0 { + // The created part is empty. + return nil, nil + } + if mpNew != nil { + // Open the created part from memory. + flushToDiskDeadline := getFlushToDiskDeadline(pws) + pwNew := newPartWrapperFromInmemoryPart(mpNew, flushToDiskDeadline) + return pwNew, nil + } + // Open the created part from disk. + pNew, err := openFilePart(dstPartPath) + if err != nil { + return nil, fmt.Errorf("cannot open merged part %q: %w", dstPartPath, err) + } + pwNew := &partWrapper{ + p: pNew, + refCount: 1, + } + return pwNew, nil +} - // Create a transaction for atomic deleting old parts and moving - // new part to its destination place. - var bb bytesutil.ByteBuffer +func areAllInmemoryParts(pws []*partWrapper) bool { for _, pw := range pws { if pw.mp == nil { - fmt.Fprintf(&bb, "%s\n", pw.p.path) + return false } } - dstPartPath := "" - if ph.RowsCount > 0 { - // The destination part may have no rows if they are deleted - // during the merge due to deleted time series. - dstPartPath = ph.Path(ptPath, mergeIdx) - } - fmt.Fprintf(&bb, "%s -> %s\n", tmpPartPath, dstPartPath) - txnPath := fmt.Sprintf("%s/txn/%016X", ptPath, mergeIdx) - if err := fs.WriteFileAtomically(txnPath, bb.B, false); err != nil { - return fmt.Errorf("cannot create transaction file %q: %w", txnPath, err) - } + return true +} - // Run the created transaction. - if err := runTransaction(&pt.snapshotLock, pt.smallPartsPath, pt.bigPartsPath, txnPath); err != nil { - return fmt.Errorf("cannot execute transaction %q: %w", txnPath, err) - } - - var newPW *partWrapper - var newPSize uint64 - if len(dstPartPath) > 0 { - // Open the merged part if it is non-empty. - newP, err := openFilePart(dstPartPath) - if err != nil { - return fmt.Errorf("cannot open merged part %q: %w", dstPartPath, err) - } - newPSize = newP.size - newPW = &partWrapper{ - p: newP, - refCount: 1, - } - } - - // Atomically remove old parts and add new part. +func (pt *partition) swapSrcWithDstParts(pws []*partWrapper, pwNew *partWrapper, dstPartType partType) { + // Atomically unregister old parts and add new part to pt. m := make(map[*partWrapper]bool, len(pws)) for _, pw := range pws { m[pw] = true } if len(m) != len(pws) { - logger.Panicf("BUG: %d duplicate parts found in the merge of %d parts", len(pws)-len(m), len(pws)) + logger.Panicf("BUG: %d duplicate parts found when merging %d parts", len(pws)-len(m), len(pws)) } + removedInmemoryParts := 0 removedSmallParts := 0 removedBigParts := 0 + pt.partsLock.Lock() - pt.smallParts, removedSmallParts = removeParts(pt.smallParts, m, false) - pt.bigParts, removedBigParts = removeParts(pt.bigParts, m, true) - if newPW != nil { - if isBigPart { - pt.bigParts = append(pt.bigParts, newPW) - } else { - pt.smallParts = append(pt.smallParts, newPW) + pt.inmemoryParts, removedInmemoryParts = removeParts(pt.inmemoryParts, m) + pt.smallParts, removedSmallParts = removeParts(pt.smallParts, m) + pt.bigParts, removedBigParts = removeParts(pt.bigParts, m) + if pwNew != nil { + switch dstPartType { + case partInmemory: + pt.inmemoryParts = append(pt.inmemoryParts, pwNew) + case partSmall: + pt.smallParts = append(pt.smallParts, pwNew) + case partBig: + pt.bigParts = append(pt.bigParts, pwNew) + default: + logger.Panicf("BUG: unknown partType=%d", dstPartType) } } pt.partsLock.Unlock() - if removedSmallParts+removedBigParts != len(m) { - logger.Panicf("BUG: unexpected number of parts removed; got %d, want %d", removedSmallParts+removedBigParts, len(m)) + + removedParts := removedInmemoryParts + removedSmallParts + removedBigParts + if removedParts != len(m) { + logger.Panicf("BUG: unexpected number of parts removed; got %d, want %d", removedParts, len(m)) } // Remove partition references from old parts. for _, pw := range pws { pw.decRef() } - - d := time.Since(startTime) - if d > 30*time.Second { - logger.Infof("merged %d rows across %d blocks in %.3f seconds at %d rows/sec to %q; sizeBytes: %d", - outRowsCount, outBlocksCount, d.Seconds(), int(float64(outRowsCount)/d.Seconds()), dstPartPath, newPSize) - } - - return nil } func getCompressLevel(rowsPerBlock float64) int { @@ -1309,17 +1574,17 @@ func (pt *partition) nextMergeIdx() uint64 { return atomic.AddUint64(&pt.mergeIdx, 1) } -func removeParts(pws []*partWrapper, partsToRemove map[*partWrapper]bool, isBig bool) ([]*partWrapper, int) { - removedParts := 0 +func removeParts(pws []*partWrapper, partsToRemove map[*partWrapper]bool) ([]*partWrapper, int) { dst := pws[:0] for _, pw := range pws { if !partsToRemove[pw] { dst = append(dst, pw) - continue } - removedParts++ } - return dst, removedParts + for i := len(dst); i < len(pws); i++ { + pws[i] = nil + } + return dst, len(pws) - len(dst) } func (pt *partition) startStalePartsRemover() { @@ -1349,9 +1614,9 @@ func (pt *partition) removeStaleParts() { retentionDeadline := timestampFromTime(startTime) - pt.s.retentionMsecs pt.partsLock.Lock() - for _, pw := range pt.bigParts { + for _, pw := range pt.inmemoryParts { if !pw.isInMerge && pw.p.ph.MaxTimestamp < retentionDeadline { - atomic.AddUint64(&pt.bigRowsDeleted, pw.p.ph.RowsCount) + atomic.AddUint64(&pt.inmemoryRowsDeleted, pw.p.ph.RowsCount) m[pw] = true } } @@ -1361,28 +1626,38 @@ func (pt *partition) removeStaleParts() { m[pw] = true } } + for _, pw := range pt.bigParts { + if !pw.isInMerge && pw.p.ph.MaxTimestamp < retentionDeadline { + atomic.AddUint64(&pt.bigRowsDeleted, pw.p.ph.RowsCount) + m[pw] = true + } + } + removedInmemoryParts := 0 removedSmallParts := 0 removedBigParts := 0 if len(m) > 0 { - pt.smallParts, removedSmallParts = removeParts(pt.smallParts, m, false) - pt.bigParts, removedBigParts = removeParts(pt.bigParts, m, true) + pt.inmemoryParts, removedInmemoryParts = removeParts(pt.inmemoryParts, m) + pt.smallParts, removedSmallParts = removeParts(pt.smallParts, m) + pt.bigParts, removedBigParts = removeParts(pt.bigParts, m) } pt.partsLock.Unlock() - if removedSmallParts+removedBigParts != len(m) { - logger.Panicf("BUG: unexpected number of stale parts removed; got %d, want %d", removedSmallParts+removedBigParts, len(m)) + removedParts := removedInmemoryParts + removedSmallParts + removedBigParts + if removedParts != len(m) { + logger.Panicf("BUG: unexpected number of stale parts removed; got %d, want %d", removedParts, len(m)) } // Physically remove stale parts under snapshotLock in order to provide // consistent snapshots with table.CreateSnapshot(). pt.snapshotLock.RLock() for pw := range m { - logger.Infof("removing part %q, since its data is out of the configured retention (%d secs)", pw.p.path, pt.s.retentionMsecs/1000) - fs.MustRemoveDirAtomic(pw.p.path) + if pw.mp == nil { + logger.Infof("removing part %q, since its data is out of the configured retention (%d secs)", pw.p.path, pt.s.retentionMsecs/1000) + fs.MustRemoveDirAtomic(pw.p.path) + } } // There is no need in calling fs.MustSyncPath() on pt.smallPartsPath and pt.bigPartsPath, // since they should be automatically called inside fs.MustRemoveDirAtomic(). - pt.snapshotLock.RUnlock() // Remove partition references from removed parts. @@ -1458,16 +1733,7 @@ func appendPartsToMerge(dst, src []*partWrapper, maxPartsToMerge int, maxOutByte src = tmp needFreeSpace := skippedBigParts > 1 - // Sort src parts by size and backwards timestamp. - // This should improve adjanced points' locality in the merged parts. - sort.Slice(src, func(i, j int) bool { - a := src[i].p - b := src[j].p - if a.size == b.size { - return a.ph.MinTimestamp > b.ph.MinTimestamp - } - return a.size < b.size - }) + sortPartsForOptimalMerge(src) maxSrcParts := maxPartsToMerge if maxSrcParts > len(src) { @@ -1518,6 +1784,19 @@ func appendPartsToMerge(dst, src []*partWrapper, maxPartsToMerge int, maxOutByte return append(dst, pws...), needFreeSpace } +func sortPartsForOptimalMerge(pws []*partWrapper) { + // Sort src parts by size and backwards timestamp. + // This should improve adjanced points' locality in the merged parts. + sort.Slice(pws, func(i, j int) bool { + a := pws[i].p + b := pws[j].p + if a.size == b.size { + return a.ph.MinTimestamp > b.ph.MinTimestamp + } + return a.size < b.size + }) +} + func getPartsSize(pws []*partWrapper) uint64 { n := uint64(0) for _, pw := range pws { @@ -1534,7 +1813,7 @@ func openParts(pathPrefix1, pathPrefix2, path string) ([]*partWrapper, error) { fs.MustRemoveTemporaryDirs(path) d, err := os.Open(path) if err != nil { - return nil, fmt.Errorf("cannot open directory %q: %w", path, err) + return nil, fmt.Errorf("cannot open partition directory: %w", err) } defer fs.MustClose(d) @@ -1616,10 +1895,7 @@ func (pt *partition) CreateSnapshotAt(smallPath, bigPath string) error { startTime := time.Now() // Flush inmemory data to disk. - pt.flushPendingRows(true) - if _, err := pt.flushInmemoryParts(nil, true); err != nil { - return fmt.Errorf("cannot flush inmemory parts: %w", err) - } + pt.flushInmemoryRows() // The snapshot must be created under the lock in order to prevent from // concurrent modifications via runTransaction. @@ -1645,13 +1921,13 @@ func (pt *partition) createSnapshot(srcDir, dstDir string) error { d, err := os.Open(srcDir) if err != nil { - return fmt.Errorf("cannot open difrectory: %w", err) + return fmt.Errorf("cannot open partition difrectory: %w", err) } defer fs.MustClose(d) fis, err := d.Readdir(-1) if err != nil { - return fmt.Errorf("cannot read directory: %w", err) + return fmt.Errorf("cannot read partition directory: %w", err) } for _, fi := range fis { fn := fi.Name() @@ -1700,7 +1976,7 @@ func runTransactions(txnLock *sync.RWMutex, pathPrefix1, pathPrefix2, path strin if os.IsNotExist(err) { return nil } - return fmt.Errorf("cannot open %q: %w", txnDir, err) + return fmt.Errorf("cannot open transaction directory: %w", err) } defer fs.MustClose(d) @@ -1764,30 +2040,32 @@ func runTransaction(txnLock *sync.RWMutex, pathPrefix1, pathPrefix2, txnPath str // Move the new part to new directory. srcPath := mvPaths[0] dstPath := mvPaths[1] - srcPath, err = validatePath(pathPrefix1, pathPrefix2, srcPath) - if err != nil { - return fmt.Errorf("invalid source path to rename: %w", err) - } - if len(dstPath) > 0 { - // Move srcPath to dstPath. - dstPath, err = validatePath(pathPrefix1, pathPrefix2, dstPath) + if len(srcPath) > 0 { + srcPath, err = validatePath(pathPrefix1, pathPrefix2, srcPath) if err != nil { - return fmt.Errorf("invalid destination path to rename: %w", err) + return fmt.Errorf("invalid source path to rename: %w", err) } - if fs.IsPathExist(srcPath) { - if err := os.Rename(srcPath, dstPath); err != nil { - return fmt.Errorf("cannot rename %q to %q: %w", srcPath, dstPath, err) + if len(dstPath) > 0 { + // Move srcPath to dstPath. + dstPath, err = validatePath(pathPrefix1, pathPrefix2, dstPath) + if err != nil { + return fmt.Errorf("invalid destination path to rename: %w", err) } - } else if !fs.IsPathExist(dstPath) { - // Emit info message for the expected condition after unclean shutdown on NFS disk. - // The dstPath part may be missing because it could be already merged into bigger part - // while old source parts for the current txn weren't still deleted due to NFS locks. - logger.Infof("cannot find both source and destination paths: %q -> %q; this may be the case after unclean shutdown (OOM, `kill -9`, hard reset) on NFS disk", - srcPath, dstPath) + if fs.IsPathExist(srcPath) { + if err := os.Rename(srcPath, dstPath); err != nil { + return fmt.Errorf("cannot rename %q to %q: %w", srcPath, dstPath, err) + } + } else if !fs.IsPathExist(dstPath) { + // Emit info message for the expected condition after unclean shutdown on NFS disk. + // The dstPath part may be missing because it could be already merged into bigger part + // while old source parts for the current txn weren't still deleted due to NFS locks. + logger.Infof("cannot find both source and destination paths: %q -> %q; this may be the case after unclean shutdown "+ + "(OOM, `kill -9`, hard reset) on NFS disk", srcPath, dstPath) + } + } else { + // Just remove srcPath. + fs.MustRemoveDirAtomic(srcPath) } - } else { - // Just remove srcPath. - fs.MustRemoveDirAtomic(srcPath) } // Flush pathPrefix* directory metadata to the underying storage, diff --git a/lib/storage/partition_search_test.go b/lib/storage/partition_search_test.go index b0e39ef7d..afd5b7fce 100644 --- a/lib/storage/partition_search_test.go +++ b/lib/storage/partition_search_test.go @@ -181,11 +181,12 @@ func testPartitionSearchEx(t *testing.T, ptt int64, tr TimeRange, partsCount, ma t.Fatalf("cannot remove big parts directory: %s", err) } }() + var tmpRows []rawRow for _, rows := range rowss { pt.AddRows(rows) - // Flush just added rows to a separate partition. - pt.flushPendingRows(true) + // Flush just added rows to a separate partitions. + tmpRows = pt.flushPendingRows(tmpRows[:0], true) } testPartitionSearch(t, pt, tsids, tr, rbsExpected, -1) pt.MustClose() @@ -232,8 +233,7 @@ func testPartitionSearchSerial(pt *partition, tsids []TSID, tr TimeRange, rbsExp // due to the race with raw rows flusher. var m partitionMetrics pt.UpdateMetrics(&m) - rowsCount := m.BigRowsCount + m.SmallRowsCount - if rowsCount != uint64(rowsCountExpected) { + if rowsCount := m.TotalRowsCount(); rowsCount != uint64(rowsCountExpected) { return fmt.Errorf("unexpected rows count; got %d; want %d", rowsCount, rowsCountExpected) } } @@ -258,8 +258,7 @@ func testPartitionSearchSerial(pt *partition, tsids []TSID, tr TimeRange, rbsExp if rowsCountExpected >= 0 { var m partitionMetrics pt.UpdateMetrics(&m) - rowsCount := m.BigRowsCount + m.SmallRowsCount - if rowsCount != uint64(rowsCountExpected) { + if rowsCount := m.TotalRowsCount(); rowsCount != uint64(rowsCountExpected) { return fmt.Errorf("unexpected rows count after search; got %d; want %d", rowsCount, rowsCountExpected) } } diff --git a/lib/storage/storage_test.go b/lib/storage/storage_test.go index d4d330a7e..4dbc62700 100644 --- a/lib/storage/storage_test.go +++ b/lib/storage/storage_test.go @@ -454,7 +454,7 @@ func TestStorageOpenMultipleTimes(t *testing.T) { func TestStorageRandTimestamps(t *testing.T) { path := "TestStorageRandTimestamps" - retentionMsecs := int64(60 * msecsPerMonth) + retentionMsecs := int64(10 * msecsPerMonth) s, err := OpenStorage(path, retentionMsecs, 0, 0) if err != nil { t.Fatalf("cannot open storage: %s", err) @@ -462,10 +462,13 @@ func TestStorageRandTimestamps(t *testing.T) { t.Run("serial", func(t *testing.T) { for i := 0; i < 3; i++ { if err := testStorageRandTimestamps(s); err != nil { - t.Fatal(err) + t.Fatalf("error on iteration %d: %s", i, err) } s.MustClose() s, err = OpenStorage(path, retentionMsecs, 0, 0) + if err != nil { + t.Fatalf("cannot open storage on iteration %d: %s", i, err) + } } }) t.Run("concurrent", func(t *testing.T) { @@ -479,14 +482,15 @@ func TestStorageRandTimestamps(t *testing.T) { ch <- err }() } + tt := time.NewTimer(time.Second * 10) for i := 0; i < cap(ch); i++ { select { case err := <-ch: if err != nil { - t.Fatal(err) + t.Fatalf("error on iteration %d: %s", i, err) } - case <-time.After(time.Second * 10): - t.Fatal("timeout") + case <-tt.C: + t.Fatalf("timeout on iteration %d", i) } } }) @@ -497,9 +501,9 @@ func TestStorageRandTimestamps(t *testing.T) { } func testStorageRandTimestamps(s *Storage) error { - const rowsPerAdd = 1e3 - const addsCount = 2 - typ := reflect.TypeOf(int64(0)) + currentTime := timestampFromTime(time.Now()) + const rowsPerAdd = 5e3 + const addsCount = 3 rnd := rand.New(rand.NewSource(1)) for i := 0; i < addsCount; i++ { @@ -512,15 +516,8 @@ func testStorageRandTimestamps(s *Storage) error { for j := 0; j < rowsPerAdd; j++ { mn.MetricGroup = []byte(fmt.Sprintf("metric_%d", rand.Intn(100))) metricNameRaw := mn.marshalRaw(nil) - timestamp := int64(rnd.NormFloat64() * 1e12) - if j%2 == 0 { - ts, ok := quick.Value(typ, rnd) - if !ok { - return fmt.Errorf("cannot create random timestamp via quick.Value") - } - timestamp = ts.Interface().(int64) - } - value := rnd.NormFloat64() * 1e12 + timestamp := currentTime - int64((rnd.Float64()-0.2)*float64(2*s.retentionMsecs)) + value := rnd.NormFloat64() * 1e11 mr := MetricRow{ MetricNameRaw: metricNameRaw, @@ -540,8 +537,8 @@ func testStorageRandTimestamps(s *Storage) error { // Verify the storage contains rows. var m Metrics s.UpdateMetrics(&m) - if m.TableMetrics.SmallRowsCount == 0 { - return fmt.Errorf("expecting at least one row in the table") + if rowsCount := m.TableMetrics.TotalRowsCount(); rowsCount == 0 { + return fmt.Errorf("expecting at least one row in storage") } return nil } @@ -592,14 +589,15 @@ func TestStorageDeleteSeries(t *testing.T) { ch <- err }(i) } + tt := time.NewTimer(30 * time.Second) for i := 0; i < cap(ch); i++ { select { case err := <-ch: if err != nil { - t.Fatalf("unexpected error: %s", err) + t.Fatalf("unexpected error on iteration %d: %s", i, err) } - case <-time.After(30 * time.Second): - t.Fatalf("timeout") + case <-tt.C: + t.Fatalf("timeout on iteration %d", i) } } }) @@ -932,7 +930,8 @@ func testStorageRegisterMetricNames(s *Storage) error { func TestStorageAddRowsSerial(t *testing.T) { path := "TestStorageAddRowsSerial" - s, err := OpenStorage(path, 0, 1e5, 1e5) + retentionMsecs := int64(msecsPerMonth * 10) + s, err := OpenStorage(path, retentionMsecs, 1e5, 1e5) if err != nil { t.Fatalf("cannot open storage: %s", err) } @@ -947,7 +946,8 @@ func TestStorageAddRowsSerial(t *testing.T) { func TestStorageAddRowsConcurrent(t *testing.T) { path := "TestStorageAddRowsConcurrent" - s, err := OpenStorage(path, 0, 1e5, 1e5) + retentionMsecs := int64(msecsPerMonth * 10) + s, err := OpenStorage(path, retentionMsecs, 1e5, 1e5) if err != nil { t.Fatalf("cannot open storage: %s", err) } @@ -1000,8 +1000,10 @@ func testStorageAddRows(s *Storage) error { const rowsPerAdd = 1e3 const addsCount = 10 + maxTimestamp := timestampFromTime(time.Now()) + minTimestamp := maxTimestamp - s.retentionMsecs for i := 0; i < addsCount; i++ { - mrs := testGenerateMetricRows(rowsPerAdd, 0, 1e10) + mrs := testGenerateMetricRows(rowsPerAdd, minTimestamp, maxTimestamp) if err := s.AddRows(mrs, defaultPrecisionBits); err != nil { return fmt.Errorf("unexpected error when adding mrs: %w", err) } @@ -1011,8 +1013,8 @@ func testStorageAddRows(s *Storage) error { minRowsExpected := uint64(rowsPerAdd * addsCount) var m Metrics s.UpdateMetrics(&m) - if m.TableMetrics.SmallRowsCount < minRowsExpected { - return fmt.Errorf("expecting at least %d rows in the table; got %d", minRowsExpected, m.TableMetrics.SmallRowsCount) + if rowsCount := m.TableMetrics.TotalRowsCount(); rowsCount < minRowsExpected { + return fmt.Errorf("expecting at least %d rows in the table; got %d", minRowsExpected, rowsCount) } // Try creating a snapshot from the storage. @@ -1040,8 +1042,8 @@ func testStorageAddRows(s *Storage) error { // Verify the snapshot contains rows var m1 Metrics s1.UpdateMetrics(&m1) - if m1.TableMetrics.SmallRowsCount < minRowsExpected { - return fmt.Errorf("snapshot %q must contain at least %d rows; got %d", snapshotPath, minRowsExpected, m1.TableMetrics.SmallRowsCount) + if rowsCount := m1.TableMetrics.TotalRowsCount(); rowsCount < minRowsExpected { + return fmt.Errorf("snapshot %q must contain at least %d rows; got %d", snapshotPath, minRowsExpected, rowsCount) } // Verify that force merge for the snapshot leaves only a single part per partition. @@ -1155,22 +1157,25 @@ func testStorageAddMetrics(s *Storage, workerNum int) error { minRowsExpected := uint64(rowsCount) var m Metrics s.UpdateMetrics(&m) - if m.TableMetrics.SmallRowsCount < minRowsExpected { - return fmt.Errorf("expecting at least %d rows in the table; got %d", minRowsExpected, m.TableMetrics.SmallRowsCount) + if rowsCount := m.TableMetrics.TotalRowsCount(); rowsCount < minRowsExpected { + return fmt.Errorf("expecting at least %d rows in the table; got %d", minRowsExpected, rowsCount) } return nil } func TestStorageDeleteStaleSnapshots(t *testing.T) { path := "TestStorageDeleteStaleSnapshots" - s, err := OpenStorage(path, 0, 1e5, 1e5) + retentionMsecs := int64(msecsPerMonth * 10) + s, err := OpenStorage(path, retentionMsecs, 1e5, 1e5) if err != nil { t.Fatalf("cannot open storage: %s", err) } const rowsPerAdd = 1e3 const addsCount = 10 + maxTimestamp := timestampFromTime(time.Now()) + minTimestamp := maxTimestamp - s.retentionMsecs for i := 0; i < addsCount; i++ { - mrs := testGenerateMetricRows(rowsPerAdd, 0, 1e10) + mrs := testGenerateMetricRows(rowsPerAdd, minTimestamp, maxTimestamp) if err := s.AddRows(mrs, defaultPrecisionBits); err != nil { t.Fatalf("unexpected error when adding mrs: %s", err) } diff --git a/lib/storage/table.go b/lib/storage/table.go index e4b603044..5eca5e4a3 100644 --- a/lib/storage/table.go +++ b/lib/storage/table.go @@ -215,15 +215,16 @@ func (tb *table) MustClose() { } } -// flushPendingRows flushes all the pending rows, so they become visible to search. +// flushPendingRows flushes all the pending raw rows, so they become visible to search. // // This function is for debug purposes only. func (tb *table) flushPendingRows() { ptws := tb.GetPartitions(nil) defer tb.PutPartitions(ptws) + var rows []rawRow for _, ptw := range ptws { - ptw.pt.flushPendingRows(true) + rows = ptw.pt.flushPendingRows(rows[:0], true) } } @@ -524,7 +525,7 @@ func openPartitions(smallPartitionsPath, bigPartitionsPath string, s *Storage) ( func populatePartitionNames(partitionsPath string, ptNames map[string]bool) error { d, err := os.Open(partitionsPath) if err != nil { - return fmt.Errorf("cannot open directory with partitions %q: %w", partitionsPath, err) + return fmt.Errorf("cannot open directory with partitions: %w", err) } defer fs.MustClose(d) diff --git a/lib/storage/table_search_test.go b/lib/storage/table_search_test.go index 17e1f5b86..fb28939d6 100644 --- a/lib/storage/table_search_test.go +++ b/lib/storage/table_search_test.go @@ -35,7 +35,7 @@ func TestTableSearch(t *testing.T) { MinTimestamp: trData.MinTimestamp + 4e3, MaxTimestamp: trData.MaxTimestamp - 4e3, } - testTableSearchEx(t, trData, trSearch, 12, 100, 1, 10) + testTableSearchEx(t, trData, trSearch, 12, 20, 1, 10) }) t.Run("SingleTSID", func(t *testing.T) { @@ -51,7 +51,7 @@ func TestTableSearch(t *testing.T) { MinTimestamp: trData.MinTimestamp + 4e3, MaxTimestamp: trData.MaxTimestamp - 4e3, } - testTableSearchEx(t, trData, trSearch, 60, 20, 30, 20) + testTableSearchEx(t, trData, trSearch, 20, 10, 30, 20) }) t.Run("ManyTSIDs", func(t *testing.T) { @@ -244,8 +244,7 @@ func testTableSearchSerial(tb *table, tsids []TSID, tr TimeRange, rbsExpected [] // they may race with raw rows flusher. var m TableMetrics tb.UpdateMetrics(&m) - rowsCount := m.BigRowsCount + m.SmallRowsCount - if rowsCount != uint64(rowsCountExpected) { + if rowsCount := m.TotalRowsCount(); rowsCount != uint64(rowsCountExpected) { return fmt.Errorf("unexpected rows count in the table; got %d; want %d", rowsCount, rowsCountExpected) } } @@ -270,8 +269,7 @@ func testTableSearchSerial(tb *table, tsids []TSID, tr TimeRange, rbsExpected [] if rowsCountExpected >= 0 { var m TableMetrics tb.UpdateMetrics(&m) - rowsCount := m.BigRowsCount + m.SmallRowsCount - if rowsCount != uint64(rowsCountExpected) { + if rowsCount := m.TotalRowsCount(); rowsCount != uint64(rowsCountExpected) { return fmt.Errorf("unexpected rows count in the table; got %d; want %d", rowsCount, rowsCountExpected) } } diff --git a/lib/storage/table_search_timing_test.go b/lib/storage/table_search_timing_test.go index cf046c513..eb1a72b67 100644 --- a/lib/storage/table_search_timing_test.go +++ b/lib/storage/table_search_timing_test.go @@ -55,9 +55,8 @@ func openBenchTable(b *testing.B, startTimestamp int64, rowsPerInsert, rowsCount rowsCountExpected := insertsCount * uint64(rowsPerInsert) var m TableMetrics tb.UpdateMetrics(&m) - rowsCountActual := m.BigRowsCount + m.SmallRowsCount - if rowsCountActual != rowsCountExpected { - b.Fatalf("unexpected rows count in the table %q; got %d; want %d", path, rowsCountActual, rowsCountExpected) + if rowsCount := m.TotalRowsCount(); rowsCount != rowsCountExpected { + b.Fatalf("unexpected rows count in the table %q; got %d; want %d", path, rowsCount, rowsCountExpected) } return tb diff --git a/lib/storage/table_timing_test.go b/lib/storage/table_timing_test.go index 0f7ae00d1..11a3766fd 100644 --- a/lib/storage/table_timing_test.go +++ b/lib/storage/table_timing_test.go @@ -101,8 +101,7 @@ func benchmarkTableAddRows(b *testing.B, rowsPerInsert, tsidsCount int) { } var m TableMetrics tb.UpdateMetrics(&m) - rowsCount := m.BigRowsCount + m.SmallRowsCount - if rowsCount != uint64(rowsCountExpected) { + if rowsCount := m.TotalRowsCount(); rowsCount != uint64(rowsCountExpected) { b.Fatalf("unexpected rows count in the final table %q: got %d; want %d", tablePath, rowsCount, rowsCountExpected) } tb.MustClose()