2023-06-20 05:55:12 +00:00
package logstorage
import (
"os"
"path/filepath"
"sort"
2024-10-13 20:20:31 +00:00
"strings"
2023-06-20 05:55:12 +00:00
"sync"
"sync/atomic"
"time"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
2024-01-22 16:12:37 +00:00
"github.com/VictoriaMetrics/VictoriaMetrics/lib/timeutil"
2023-06-20 05:55:12 +00:00
)
// StorageStats represents stats for the storage. It may be obtained by calling Storage.UpdateStats().
type StorageStats struct {
// RowsDroppedTooBigTimestamp is the number of rows dropped during data ingestion because their timestamp is smaller than the minimum allowed
RowsDroppedTooBigTimestamp uint64
// RowsDroppedTooSmallTimestamp is the number of rows dropped during data ingestion because their timestamp is bigger than the maximum allowed
RowsDroppedTooSmallTimestamp uint64
// PartitionsCount is the number of partitions in the storage
PartitionsCount uint64
2023-09-29 09:55:38 +00:00
// IsReadOnly indicates whether the storage is read-only.
IsReadOnly bool
2023-10-02 14:26:02 +00:00
// PartitionStats contains partition stats.
PartitionStats
2023-06-20 05:55:12 +00:00
}
// Reset resets s.
func ( s * StorageStats ) Reset ( ) {
* s = StorageStats { }
}
// StorageConfig is the config for the Storage.
type StorageConfig struct {
// Retention is the retention for the ingested data.
//
// Older data is automatically deleted.
Retention time . Duration
2024-06-25 15:30:02 +00:00
// MaxDiskSpaceUsageBytes is an optional maximum disk space logs can use.
//
// The oldest per-day partitions are automatically dropped if the total disk space usage exceeds this limit.
MaxDiskSpaceUsageBytes int64
// FlushInterval is the interval for flushing the in-memory data to disk at the Storage.
2023-06-20 05:55:12 +00:00
FlushInterval time . Duration
// FutureRetention is the allowed retention from the current time to future for the ingested data.
//
// Log entries with timestamps bigger than now+FutureRetention are ignored.
FutureRetention time . Duration
2024-06-25 15:30:02 +00:00
// MinFreeDiskSpaceBytes is the minimum free disk space at storage path after which the storage stops accepting new data
// and enters read-only mode.
2023-10-02 14:26:02 +00:00
MinFreeDiskSpaceBytes int64
2023-06-20 05:55:12 +00:00
// LogNewStreams indicates whether to log newly created log streams.
//
// This can be useful for debugging of high cardinality issues.
2024-05-24 22:30:58 +00:00
// https://docs.victoriametrics.com/victorialogs/keyconcepts/#high-cardinality
2023-06-20 05:55:12 +00:00
LogNewStreams bool
// LogIngestedRows indicates whether to log the ingested log entries.
//
// This can be useful for debugging of data ingestion.
LogIngestedRows bool
}
// Storage is the storage for log entries.
type Storage struct {
2024-02-23 21:46:10 +00:00
rowsDroppedTooBigTimestamp atomic . Uint64
rowsDroppedTooSmallTimestamp atomic . Uint64
2023-06-20 05:55:12 +00:00
// path is the path to the Storage directory
path string
// retention is the retention for the stored data
//
// older data is automatically deleted
retention time . Duration
2024-06-25 15:30:02 +00:00
// maxDiskSpaceUsageBytes is an optional maximum disk space logs can use.
//
// The oldest per-day partitions are automatically dropped if the total disk space usage exceeds this limit.
maxDiskSpaceUsageBytes int64
2023-06-20 05:55:12 +00:00
// flushInterval is the interval for flushing in-memory data to disk
flushInterval time . Duration
// futureRetention is the maximum allowed interval to write data into the future
futureRetention time . Duration
2023-10-02 14:26:02 +00:00
// minFreeDiskSpaceBytes is the minimum free disk space at path after which the storage stops accepting new data
minFreeDiskSpaceBytes uint64
2023-06-20 05:55:12 +00:00
// logNewStreams instructs to log new streams if it is set to true
logNewStreams bool
// logIngestedRows instructs to log all the ingested log entries if it is set to true
logIngestedRows bool
// flockF is a file, which makes sure that the Storage is opened by a single process
flockF * os . File
// partitions is a list of partitions for the Storage.
//
// It must be accessed under partitionsLock.
2024-10-13 20:20:31 +00:00
//
// partitions are sorted by time.
2023-06-20 05:55:12 +00:00
partitions [ ] * partitionWrapper
// ptwHot is the "hot" partition, were the last rows were ingested.
//
// It must be accessed under partitionsLock.
ptwHot * partitionWrapper
// partitionsLock protects partitions and ptwHot.
partitionsLock sync . Mutex
// stopCh is closed when the Storage must be stopped.
stopCh chan struct { }
// wg is used for waiting for background workers at MustClose().
wg sync . WaitGroup
// streamIDCache caches (partition, streamIDs) seen during data ingestion.
//
// It reduces the load on persistent storage during data ingestion by skipping
// the check whether the given stream is already registered in the persistent storage.
2024-10-18 00:15:03 +00:00
streamIDCache * cache
2023-06-20 05:55:12 +00:00
2024-05-12 14:33:29 +00:00
// filterStreamCache caches streamIDs keyed by (partition, []TenanID, StreamFilter).
2023-06-20 05:55:12 +00:00
//
// It reduces the load on persistent storage during querying by _stream:{...} filter.
2024-10-18 00:15:03 +00:00
filterStreamCache * cache
2023-06-20 05:55:12 +00:00
}
type partitionWrapper struct {
// refCount is the number of active references to p.
// When it reaches zero, then the p is closed.
2024-02-23 21:04:38 +00:00
refCount atomic . Int32
2023-06-20 05:55:12 +00:00
// The flag, which is set when the partition must be deleted after refCount reaches zero.
2024-02-23 21:04:38 +00:00
mustDrop atomic . Bool
2023-06-20 05:55:12 +00:00
// day is the day for the partition in the unix timestamp divided by the number of seconds in the day.
day int64
// pt is the wrapped partition.
pt * partition
}
func newPartitionWrapper ( pt * partition , day int64 ) * partitionWrapper {
pw := & partitionWrapper {
day : day ,
pt : pt ,
}
pw . incRef ( )
return pw
}
func ( ptw * partitionWrapper ) incRef ( ) {
2024-02-23 21:04:38 +00:00
ptw . refCount . Add ( 1 )
2023-06-20 05:55:12 +00:00
}
func ( ptw * partitionWrapper ) decRef ( ) {
2024-02-23 21:04:38 +00:00
n := ptw . refCount . Add ( - 1 )
2023-06-20 05:55:12 +00:00
if n > 0 {
return
}
deletePath := ""
2024-02-23 21:04:38 +00:00
if ptw . mustDrop . Load ( ) {
2023-06-20 05:55:12 +00:00
deletePath = ptw . pt . path
}
// Close pw.pt, since nobody refers to it.
mustClosePartition ( ptw . pt )
ptw . pt = nil
// Delete partition if needed.
if deletePath != "" {
mustDeletePartition ( deletePath )
}
}
func ( ptw * partitionWrapper ) canAddAllRows ( lr * LogRows ) bool {
2024-09-06 14:17:04 +00:00
minTimestamp := ptw . day * nsecsPerDay
maxTimestamp := minTimestamp + nsecsPerDay - 1
2023-06-20 05:55:12 +00:00
for _ , ts := range lr . timestamps {
if ts < minTimestamp || ts > maxTimestamp {
return false
}
}
return true
}
// mustCreateStorage creates Storage at the given path.
func mustCreateStorage ( path string ) {
fs . MustMkdirFailIfExist ( path )
partitionsPath := filepath . Join ( path , partitionsDirname )
fs . MustMkdirFailIfExist ( partitionsPath )
}
// MustOpenStorage opens Storage at the given path.
//
// MustClose must be called on the returned Storage when it is no longer needed.
func MustOpenStorage ( path string , cfg * StorageConfig ) * Storage {
flushInterval := cfg . FlushInterval
if flushInterval < time . Second {
flushInterval = time . Second
}
retention := cfg . Retention
if retention < 24 * time . Hour {
retention = 24 * time . Hour
}
futureRetention := cfg . FutureRetention
if futureRetention < 24 * time . Hour {
futureRetention = 24 * time . Hour
}
2023-10-02 14:26:02 +00:00
var minFreeDiskSpaceBytes uint64
if cfg . MinFreeDiskSpaceBytes >= 0 {
minFreeDiskSpaceBytes = uint64 ( cfg . MinFreeDiskSpaceBytes )
}
2023-06-20 05:55:12 +00:00
if ! fs . IsPathExist ( path ) {
mustCreateStorage ( path )
}
flockF := fs . MustCreateFlockFile ( path )
// Load caches
2024-10-18 00:15:03 +00:00
streamIDCache := newCache ( )
filterStreamCache := newCache ( )
2023-06-20 05:55:12 +00:00
s := & Storage {
2024-06-25 15:30:02 +00:00
path : path ,
retention : retention ,
maxDiskSpaceUsageBytes : cfg . MaxDiskSpaceUsageBytes ,
flushInterval : flushInterval ,
futureRetention : futureRetention ,
minFreeDiskSpaceBytes : minFreeDiskSpaceBytes ,
logNewStreams : cfg . LogNewStreams ,
logIngestedRows : cfg . LogIngestedRows ,
flockF : flockF ,
stopCh : make ( chan struct { } ) ,
2023-06-20 05:55:12 +00:00
streamIDCache : streamIDCache ,
2024-05-12 14:33:29 +00:00
filterStreamCache : filterStreamCache ,
2023-06-20 05:55:12 +00:00
}
partitionsPath := filepath . Join ( path , partitionsDirname )
fs . MustMkdirIfNotExist ( partitionsPath )
des := fs . MustReadDir ( partitionsPath )
ptws := make ( [ ] * partitionWrapper , len ( des ) )
for i , de := range des {
fname := de . Name ( )
// Parse the day for the partition
t , err := time . Parse ( partitionNameFormat , fname )
if err != nil {
logger . Panicf ( "FATAL: cannot parse partition filename %q at %q; it must be in the form YYYYMMDD: %s" , fname , partitionsPath , err )
}
2024-09-06 14:17:04 +00:00
day := t . UTC ( ) . UnixNano ( ) / nsecsPerDay
2023-06-20 05:55:12 +00:00
partitionPath := filepath . Join ( partitionsPath , fname )
pt := mustOpenPartition ( s , partitionPath )
ptws [ i ] = newPartitionWrapper ( pt , day )
}
sort . Slice ( ptws , func ( i , j int ) bool {
return ptws [ i ] . day < ptws [ j ] . day
} )
// Delete partitions from the future if needed
maxAllowedDay := s . getMaxAllowedDay ( )
j := len ( ptws ) - 1
for j >= 0 {
ptw := ptws [ j ]
if ptw . day <= maxAllowedDay {
break
}
logger . Infof ( "the partition %s is scheduled to be deleted because it is outside the -futureRetention=%dd" , ptw . pt . path , durationToDays ( s . futureRetention ) )
2024-02-23 21:04:38 +00:00
ptw . mustDrop . Store ( true )
2023-06-20 05:55:12 +00:00
ptw . decRef ( )
j --
}
j ++
for i := j ; i < len ( ptws ) ; i ++ {
ptws [ i ] = nil
}
ptws = ptws [ : j ]
s . partitions = ptws
s . runRetentionWatcher ( )
2024-06-25 15:30:02 +00:00
s . runMaxDiskSpaceUsageWatcher ( )
2023-06-20 05:55:12 +00:00
return s
}
const partitionNameFormat = "20060102"
func ( s * Storage ) runRetentionWatcher ( ) {
s . wg . Add ( 1 )
go func ( ) {
s . watchRetention ( )
s . wg . Done ( )
} ( )
}
2024-06-25 15:30:02 +00:00
func ( s * Storage ) runMaxDiskSpaceUsageWatcher ( ) {
if s . maxDiskSpaceUsageBytes <= 0 {
return
}
s . wg . Add ( 1 )
go func ( ) {
s . watchMaxDiskSpaceUsage ( )
s . wg . Done ( )
} ( )
}
2023-06-20 05:55:12 +00:00
func ( s * Storage ) watchRetention ( ) {
2024-01-22 16:12:37 +00:00
d := timeutil . AddJitterToDuration ( time . Hour )
ticker := time . NewTicker ( d )
2023-06-20 05:55:12 +00:00
defer ticker . Stop ( )
for {
var ptwsToDelete [ ] * partitionWrapper
minAllowedDay := s . getMinAllowedDay ( )
s . partitionsLock . Lock ( )
// Delete outdated partitions.
// s.partitions are sorted by day, so the partitions, which can become outdated, are located at the beginning of the list
for _ , ptw := range s . partitions {
if ptw . day >= minAllowedDay {
break
}
ptwsToDelete = append ( ptwsToDelete , ptw )
2023-08-29 09:04:11 +00:00
if ptw == s . ptwHot {
s . ptwHot = nil
}
2023-06-20 05:55:12 +00:00
}
for i := range ptwsToDelete {
s . partitions [ i ] = nil
}
s . partitions = s . partitions [ len ( ptwsToDelete ) : ]
s . partitionsLock . Unlock ( )
for _ , ptw := range ptwsToDelete {
logger . Infof ( "the partition %s is scheduled to be deleted because it is outside the -retentionPeriod=%dd" , ptw . pt . path , durationToDays ( s . retention ) )
2024-02-23 21:04:38 +00:00
ptw . mustDrop . Store ( true )
2023-06-20 05:55:12 +00:00
ptw . decRef ( )
}
select {
case <- s . stopCh :
return
case <- ticker . C :
}
}
}
2024-06-25 15:30:02 +00:00
func ( s * Storage ) watchMaxDiskSpaceUsage ( ) {
d := timeutil . AddJitterToDuration ( 10 * time . Second )
ticker := time . NewTicker ( d )
defer ticker . Stop ( )
for {
s . partitionsLock . Lock ( )
var n uint64
ptws := s . partitions
var ptwsToDelete [ ] * partitionWrapper
for i := len ( ptws ) - 1 ; i >= 0 ; i -- {
ptw := ptws [ i ]
var ps PartitionStats
ptw . pt . updateStats ( & ps )
n += ps . IndexdbSizeBytes + ps . CompressedSmallPartSize + ps . CompressedBigPartSize
if n <= uint64 ( s . maxDiskSpaceUsageBytes ) {
continue
}
if i >= len ( ptws ) - 2 {
// Keep the last two per-day partitions, so logs could be queried for one day time range.
continue
}
// ptws are sorted by time, so just drop all the partitions until i, including i.
i ++
ptwsToDelete = ptws [ : i ]
s . partitions = ptws [ i : ]
// Remove reference to deleted partitions from s.ptwHot
for _ , ptw := range ptwsToDelete {
if ptw == s . ptwHot {
s . ptwHot = nil
break
}
}
break
}
s . partitionsLock . Unlock ( )
for i , ptw := range ptwsToDelete {
logger . Infof ( "the partition %s is scheduled to be deleted because the total size of partitions exceeds -retention.maxDiskSpaceUsageBytes=%d" ,
ptw . pt . path , s . maxDiskSpaceUsageBytes )
ptw . mustDrop . Store ( true )
ptw . decRef ( )
ptwsToDelete [ i ] = nil
}
select {
case <- s . stopCh :
return
case <- ticker . C :
}
}
}
2023-06-20 05:55:12 +00:00
func ( s * Storage ) getMinAllowedDay ( ) int64 {
2024-09-06 14:17:04 +00:00
return time . Now ( ) . UTC ( ) . Add ( - s . retention ) . UnixNano ( ) / nsecsPerDay
2023-06-20 05:55:12 +00:00
}
func ( s * Storage ) getMaxAllowedDay ( ) int64 {
2024-09-06 14:17:04 +00:00
return time . Now ( ) . UTC ( ) . Add ( s . futureRetention ) . UnixNano ( ) / nsecsPerDay
2023-06-20 05:55:12 +00:00
}
// MustClose closes s.
//
// It is expected that nobody uses the storage at the close time.
func ( s * Storage ) MustClose ( ) {
// Stop background workers
close ( s . stopCh )
s . wg . Wait ( )
// Close partitions
for _ , pw := range s . partitions {
pw . decRef ( )
2024-02-23 21:04:38 +00:00
if n := pw . refCount . Load ( ) ; n != 0 {
logger . Panicf ( "BUG: there are %d users of partition" , n )
2023-06-20 05:55:12 +00:00
}
}
s . partitions = nil
2023-08-29 09:01:19 +00:00
s . ptwHot = nil
2023-06-20 05:55:12 +00:00
2024-10-17 23:11:23 +00:00
// Stop caches
// Do not persist caches, since they may become out of sync with partitions
// if partitions are deleted, restored from backups or copied from other sources
// between VictoriaLogs restarts. This may result in various issues
// during data ingestion and querying.
2024-10-18 00:15:03 +00:00
s . streamIDCache . MustStop ( )
2023-06-20 05:55:12 +00:00
s . streamIDCache = nil
2024-10-18 00:15:03 +00:00
s . filterStreamCache . MustStop ( )
2024-05-12 14:33:29 +00:00
s . filterStreamCache = nil
2023-06-20 05:55:12 +00:00
// release lock file
fs . MustClose ( s . flockF )
s . flockF = nil
s . path = ""
}
2024-10-13 20:20:31 +00:00
// MustForceMerge force-merges parts in s partitions with names starting from the given partitionNamePrefix.
//
// Partitions are merged sequentially in order to reduce load on the system.
func ( s * Storage ) MustForceMerge ( partitionNamePrefix string ) {
var ptws [ ] * partitionWrapper
s . partitionsLock . Lock ( )
for _ , ptw := range s . partitions {
if strings . HasPrefix ( ptw . pt . name , partitionNamePrefix ) {
ptw . incRef ( )
ptws = append ( ptws , ptw )
}
}
s . partitionsLock . Unlock ( )
s . wg . Add ( 1 )
defer s . wg . Done ( )
for _ , ptw := range ptws {
logger . Infof ( "started force merge for partition %s" , ptw . pt . name )
startTime := time . Now ( )
ptw . pt . mustForceMerge ( )
ptw . decRef ( )
logger . Infof ( "finished force merge for partition %s in %.3fs" , ptw . pt . name , time . Since ( startTime ) . Seconds ( ) )
}
}
2023-10-02 14:26:02 +00:00
// MustAddRows adds lr to s.
//
// It is recommended checking whether the s is in read-only mode by calling IsReadOnly()
// before calling MustAddRows.
func ( s * Storage ) MustAddRows ( lr * LogRows ) {
2023-06-20 05:55:12 +00:00
// Fast path - try adding all the rows to the hot partition
s . partitionsLock . Lock ( )
ptwHot := s . ptwHot
if ptwHot != nil {
ptwHot . incRef ( )
}
s . partitionsLock . Unlock ( )
2023-08-29 09:01:19 +00:00
if ptwHot != nil {
2023-06-20 05:55:12 +00:00
if ptwHot . canAddAllRows ( lr ) {
ptwHot . pt . mustAddRows ( lr )
ptwHot . decRef ( )
2023-10-02 14:26:02 +00:00
return
2023-06-20 05:55:12 +00:00
}
ptwHot . decRef ( )
}
// Slow path - rows cannot be added to the hot partition, so split rows among available partitions
minAllowedDay := s . getMinAllowedDay ( )
maxAllowedDay := s . getMaxAllowedDay ( )
m := make ( map [ int64 ] * LogRows )
for i , ts := range lr . timestamps {
2024-09-06 14:17:04 +00:00
day := ts / nsecsPerDay
2023-06-20 05:55:12 +00:00
if day < minAllowedDay {
rf := RowFormatter ( lr . rows [ i ] )
tsf := TimeFormatter ( ts )
2024-09-06 14:17:04 +00:00
minAllowedTsf := TimeFormatter ( minAllowedDay * nsecsPerDay )
2023-06-20 05:55:12 +00:00
tooSmallTimestampLogger . Warnf ( "skipping log entry with too small timestamp=%s; it must be bigger than %s according " +
2024-05-24 22:30:58 +00:00
"to the configured -retentionPeriod=%dd. See https://docs.victoriametrics.com/victorialogs/#retention ; " +
2023-07-18 01:23:10 +00:00
"log entry: %s" , & tsf , & minAllowedTsf , durationToDays ( s . retention ) , & rf )
2024-02-23 21:46:10 +00:00
s . rowsDroppedTooSmallTimestamp . Add ( 1 )
2023-06-20 05:55:12 +00:00
continue
}
if day > maxAllowedDay {
rf := RowFormatter ( lr . rows [ i ] )
tsf := TimeFormatter ( ts )
2024-09-06 14:17:04 +00:00
maxAllowedTsf := TimeFormatter ( maxAllowedDay * nsecsPerDay )
2023-06-20 05:55:12 +00:00
tooBigTimestampLogger . Warnf ( "skipping log entry with too big timestamp=%s; it must be smaller than %s according " +
2024-05-24 22:30:58 +00:00
"to the configured -futureRetention=%dd; see https://docs.victoriametrics.com/victorialogs/#retention ; " +
2023-07-18 01:23:10 +00:00
"log entry: %s" , & tsf , & maxAllowedTsf , durationToDays ( s . futureRetention ) , & rf )
2024-02-23 21:46:10 +00:00
s . rowsDroppedTooBigTimestamp . Add ( 1 )
2023-06-20 05:55:12 +00:00
continue
}
lrPart := m [ day ]
if lrPart == nil {
2024-11-01 19:06:15 +00:00
lrPart = GetLogRows ( nil , nil , nil , "" )
2023-06-20 05:55:12 +00:00
m [ day ] = lrPart
}
lrPart . mustAddInternal ( lr . streamIDs [ i ] , ts , lr . rows [ i ] , lr . streamTagsCanonicals [ i ] )
}
for day , lrPart := range m {
ptw := s . getPartitionForDay ( day )
ptw . pt . mustAddRows ( lrPart )
ptw . decRef ( )
PutLogRows ( lrPart )
}
}
var tooSmallTimestampLogger = logger . WithThrottler ( "too_small_timestamp" , 5 * time . Second )
var tooBigTimestampLogger = logger . WithThrottler ( "too_big_timestamp" , 5 * time . Second )
// TimeFormatter implements fmt.Stringer for timestamp in nanoseconds
type TimeFormatter int64
// String returns human-readable representation for tf.
func ( tf * TimeFormatter ) String ( ) string {
ts := int64 ( * tf )
t := time . Unix ( 0 , ts ) . UTC ( )
2023-06-22 02:39:22 +00:00
return t . Format ( time . RFC3339Nano )
2023-06-20 05:55:12 +00:00
}
func ( s * Storage ) getPartitionForDay ( day int64 ) * partitionWrapper {
s . partitionsLock . Lock ( )
// Search for the partition using binary search
ptws := s . partitions
n := sort . Search ( len ( ptws ) , func ( i int ) bool {
return ptws [ i ] . day >= day
} )
var ptw * partitionWrapper
if n < len ( ptws ) {
ptw = ptws [ n ]
if ptw . day != day {
ptw = nil
}
}
if ptw == nil {
// Missing partition for the given day. Create it.
2024-09-06 14:17:04 +00:00
fname := time . Unix ( 0 , day * nsecsPerDay ) . UTC ( ) . Format ( partitionNameFormat )
2023-06-20 05:55:12 +00:00
partitionPath := filepath . Join ( s . path , partitionsDirname , fname )
mustCreatePartition ( partitionPath )
pt := mustOpenPartition ( s , partitionPath )
ptw = newPartitionWrapper ( pt , day )
if n == len ( ptws ) {
ptws = append ( ptws , ptw )
} else {
ptws = append ( ptws [ : n + 1 ] , ptws [ n : ] ... )
ptws [ n ] = ptw
}
s . partitions = ptws
}
s . ptwHot = ptw
ptw . incRef ( )
s . partitionsLock . Unlock ( )
return ptw
}
// UpdateStats updates ss for the given s.
func ( s * Storage ) UpdateStats ( ss * StorageStats ) {
2024-02-23 21:46:10 +00:00
ss . RowsDroppedTooBigTimestamp += s . rowsDroppedTooBigTimestamp . Load ( )
ss . RowsDroppedTooSmallTimestamp += s . rowsDroppedTooSmallTimestamp . Load ( )
2023-06-20 05:55:12 +00:00
s . partitionsLock . Lock ( )
ss . PartitionsCount += uint64 ( len ( s . partitions ) )
for _ , ptw := range s . partitions {
ptw . pt . updateStats ( & ss . PartitionStats )
}
s . partitionsLock . Unlock ( )
2023-10-02 14:26:02 +00:00
2023-09-29 09:55:38 +00:00
ss . IsReadOnly = s . IsReadOnly ( )
}
2023-10-02 14:26:02 +00:00
// IsReadOnly returns true if s is in read-only mode.
2023-09-29 09:55:38 +00:00
func ( s * Storage ) IsReadOnly ( ) bool {
2023-10-02 14:26:02 +00:00
available := fs . MustGetFreeSpace ( s . path )
2023-10-03 10:39:41 +00:00
return available < s . minFreeDiskSpaceBytes
2023-06-20 05:55:12 +00:00
}
func ( s * Storage ) debugFlush ( ) {
s . partitionsLock . Lock ( )
ptws := append ( [ ] * partitionWrapper { } , s . partitions ... )
for _ , ptw := range ptws {
ptw . incRef ( )
}
s . partitionsLock . Unlock ( )
for _ , ptw := range ptws {
ptw . pt . debugFlush ( )
ptw . decRef ( )
}
}
func durationToDays ( d time . Duration ) int64 {
return int64 ( d / ( time . Hour * 24 ) )
}