mirror of https://github.com/grafana/loki
You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
523 lines
14 KiB
523 lines
14 KiB
package compactor
|
|
|
|
import (
|
|
"context"
|
|
"errors"
|
|
"fmt"
|
|
"os"
|
|
"path/filepath"
|
|
"strings"
|
|
"time"
|
|
|
|
util_log "github.com/cortexproject/cortex/pkg/util/log"
|
|
"github.com/go-kit/log"
|
|
"github.com/go-kit/log/level"
|
|
"go.etcd.io/bbolt"
|
|
|
|
chunk_util "github.com/grafana/loki/pkg/storage/chunk/util"
|
|
"github.com/grafana/loki/pkg/storage/stores/shipper/compactor/retention"
|
|
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
|
|
shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util"
|
|
util_math "github.com/grafana/loki/pkg/util/math"
|
|
)
|
|
|
|
const (
|
|
uploaderName = "compactor"
|
|
|
|
readDBsParallelism = 50
|
|
batchSize = 1000
|
|
|
|
// we want to recreate compactedDB when the chances of it changing due to compaction or deletion of data are low.
|
|
// this is to avoid recreation of the DB too often which would be too costly in a large cluster.
|
|
recreateCompactedDBOlderThan = 12 * time.Hour
|
|
dropFreePagesTxMaxSize = 100 * 1024 * 1024 // 100MB
|
|
recreatedCompactedDBSuffix = ".r.gz"
|
|
)
|
|
|
|
var bucketName = []byte("index")
|
|
|
|
type indexEntry struct {
|
|
k, v []byte
|
|
}
|
|
|
|
type table struct {
|
|
name string
|
|
workingDirectory string
|
|
indexStorageClient storage.Client
|
|
applyRetention bool
|
|
tableMarker retention.TableMarker
|
|
|
|
sourceFiles []storage.IndexFile
|
|
compactedDB *bbolt.DB
|
|
compactedDBRecreated bool
|
|
uploadCompactedDB bool
|
|
removeSourceFiles bool
|
|
logger log.Logger
|
|
|
|
ctx context.Context
|
|
quit chan struct{}
|
|
}
|
|
|
|
func newTable(ctx context.Context, workingDirectory string, indexStorageClient storage.Client, applyRetention bool, tableMarker retention.TableMarker) (*table, error) {
|
|
err := chunk_util.EnsureDirectory(workingDirectory)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
table := table{
|
|
ctx: ctx,
|
|
name: filepath.Base(workingDirectory),
|
|
workingDirectory: workingDirectory,
|
|
indexStorageClient: indexStorageClient,
|
|
quit: make(chan struct{}),
|
|
applyRetention: applyRetention,
|
|
tableMarker: tableMarker,
|
|
}
|
|
table.logger = log.With(util_log.Logger, "table-name", table.name)
|
|
|
|
return &table, nil
|
|
}
|
|
|
|
func (t *table) compact(tableHasExpiredStreams bool) error {
|
|
indexFiles, err := t.indexStorageClient.ListFiles(t.ctx, t.name)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
if len(indexFiles) == 0 {
|
|
level.Info(t.logger).Log("msg", "no index files found")
|
|
return nil
|
|
}
|
|
|
|
t.sourceFiles = indexFiles
|
|
level.Info(t.logger).Log("msg", "listed files", "count", len(indexFiles))
|
|
|
|
defer func() {
|
|
err := t.cleanup()
|
|
if err != nil {
|
|
level.Error(t.logger).Log("msg", "failed to cleanup table")
|
|
}
|
|
}()
|
|
|
|
applyRetention := t.applyRetention && tableHasExpiredStreams
|
|
|
|
if len(indexFiles) > 1 {
|
|
if err := t.compactFiles(indexFiles); err != nil {
|
|
return err
|
|
}
|
|
|
|
// we have compacted the files to a single file so let use upload the compacted db and remove the source files.
|
|
t.uploadCompactedDB = true
|
|
t.removeSourceFiles = true
|
|
} else if !applyRetention && !t.mustRecreateCompactedDB() {
|
|
return nil
|
|
} else {
|
|
// download the db for applying retention or recreating the compacted db
|
|
downloadAt := filepath.Join(t.workingDirectory, indexFiles[0].Name)
|
|
err = shipper_util.GetFileFromStorage(t.ctx, t.indexStorageClient, t.name, indexFiles[0].Name, downloadAt, false)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
t.compactedDB, err = openBoltdbFileWithNoSync(downloadAt)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
if applyRetention {
|
|
empty, modified, err := t.tableMarker.MarkForDelete(t.ctx, t.name, t.compactedDB)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
if empty {
|
|
// we have deleted all the data so we can remove the source files without uploading the compacted db
|
|
t.removeSourceFiles = true
|
|
t.uploadCompactedDB = false
|
|
} else if modified {
|
|
// we have modified the compacted db so we need to upload the compacted db and remove the source file(s)
|
|
t.uploadCompactedDB = true
|
|
t.removeSourceFiles = true
|
|
}
|
|
}
|
|
|
|
// file was not modified so see if we must recreate the compacted db to optimize storage usage
|
|
if !t.uploadCompactedDB && !t.removeSourceFiles && t.mustRecreateCompactedDB() {
|
|
err := t.recreateCompactedDB()
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
// we have recreated the compacted db so we need to upload the compacted db and remove the source file
|
|
t.uploadCompactedDB = true
|
|
t.removeSourceFiles = true
|
|
t.compactedDBRecreated = true
|
|
}
|
|
|
|
return t.done()
|
|
}
|
|
|
|
// done takes care of uploading the files and cleaning up the working directory based on the value in uploadCompactedDB and removeSourceFiles
|
|
func (t *table) done() error {
|
|
if t.uploadCompactedDB {
|
|
err := t.upload()
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
if t.removeSourceFiles {
|
|
err := t.removeSourceFilesFromStorage()
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
// mustRecreateCompactedDB returns true if the compacted db should be recreated
|
|
func (t *table) mustRecreateCompactedDB() bool {
|
|
if len(t.sourceFiles) != 1 {
|
|
// do not recreate if there are multiple source files
|
|
return false
|
|
} else if time.Since(t.sourceFiles[0].ModifiedAt) < recreateCompactedDBOlderThan {
|
|
// do not recreate if the source file is younger than the threshold
|
|
return false
|
|
}
|
|
|
|
// recreate the compacted db only if we have not recreated it before
|
|
return !strings.HasSuffix(t.sourceFiles[0].Name, recreatedCompactedDBSuffix)
|
|
}
|
|
|
|
// recreateCompactedDB just copies the old db to the new one using bbolt.Compact for following reasons:
|
|
// 1. When files are deleted, boltdb leaves free pages in the file. The only way to drop those free pages is to re-create the file.
|
|
// See https://github.com/boltdb/bolt/issues/308 for more details.
|
|
// 2. boltdb by default fills only about 50% of the page in the file. See https://github.com/etcd-io/bbolt/blob/master/bucket.go#L26.
|
|
// This setting is optimal for unordered writes.
|
|
// bbolt.Compact fills the whole page by setting FillPercent to 1 which works well here since while copying the data, it receives the index entries in order.
|
|
// The storage space goes down from anywhere between 25% to 50% as per my(Sandeep) tests.
|
|
func (t *table) recreateCompactedDB() error {
|
|
destDB, err := openBoltdbFileWithNoSync(filepath.Join(t.workingDirectory, fmt.Sprint(time.Now().Unix())))
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
level.Info(t.logger).Log("msg", "recreating compacted db")
|
|
|
|
err = bbolt.Compact(destDB, t.compactedDB, dropFreePagesTxMaxSize)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
sourceSize := int64(0)
|
|
destSize := int64(0)
|
|
|
|
if err := t.compactedDB.View(func(tx *bbolt.Tx) error {
|
|
sourceSize = tx.Size()
|
|
return nil
|
|
}); err != nil {
|
|
return err
|
|
}
|
|
|
|
if err := destDB.View(func(tx *bbolt.Tx) error {
|
|
destSize = tx.Size()
|
|
return nil
|
|
}); err != nil {
|
|
return err
|
|
}
|
|
|
|
level.Info(t.logger).Log("msg", "recreated compacted db", "src_size_bytes", sourceSize, "dest_size_bytes", destSize)
|
|
|
|
err = t.compactedDB.Close()
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
t.compactedDB = destDB
|
|
return nil
|
|
}
|
|
|
|
func (t *table) compactFiles(files []storage.IndexFile) error {
|
|
var err error
|
|
level.Info(t.logger).Log("msg", "starting compaction of dbs")
|
|
|
|
compactedDBName := filepath.Join(t.workingDirectory, fmt.Sprint(time.Now().Unix()))
|
|
seedFileIdx := findSeedFileIdx(files)
|
|
|
|
level.Info(t.logger).Log("msg", fmt.Sprintf("using %s as seed file", files[seedFileIdx].Name))
|
|
|
|
err = shipper_util.GetFileFromStorage(t.ctx, t.indexStorageClient, t.name, files[seedFileIdx].Name, compactedDBName, false)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
t.compactedDB, err = openBoltdbFileWithNoSync(compactedDBName)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
errChan := make(chan error)
|
|
readFileChan := make(chan string)
|
|
n := util_math.Min(len(files), readDBsParallelism)
|
|
|
|
// read files in parallel
|
|
for i := 0; i < n; i++ {
|
|
go func() {
|
|
var err error
|
|
defer func() {
|
|
errChan <- err
|
|
}()
|
|
|
|
for {
|
|
select {
|
|
case fileName, ok := <-readFileChan:
|
|
if !ok {
|
|
return
|
|
}
|
|
|
|
downloadAt := filepath.Join(t.workingDirectory, fileName)
|
|
|
|
err = shipper_util.GetFileFromStorage(t.ctx, t.indexStorageClient, t.name, fileName, downloadAt, false)
|
|
if err != nil {
|
|
return
|
|
}
|
|
|
|
err = t.readFile(downloadAt)
|
|
if err != nil {
|
|
level.Error(t.logger).Log("msg", fmt.Sprintf("error reading file %s", fileName), "err", err)
|
|
return
|
|
}
|
|
case <-t.quit:
|
|
return
|
|
case <-t.ctx.Done():
|
|
return
|
|
}
|
|
}
|
|
}()
|
|
}
|
|
|
|
// send all files to readFileChan
|
|
go func() {
|
|
for i, file := range files {
|
|
// skip seed file
|
|
if i == seedFileIdx {
|
|
continue
|
|
}
|
|
select {
|
|
case readFileChan <- file.Name:
|
|
case <-t.quit:
|
|
break
|
|
case <-t.ctx.Done():
|
|
break
|
|
}
|
|
}
|
|
|
|
level.Debug(t.logger).Log("msg", "closing readFileChan")
|
|
|
|
close(readFileChan)
|
|
}()
|
|
|
|
var firstErr error
|
|
|
|
// read all the errors
|
|
for i := 0; i < n; i++ {
|
|
err := <-errChan
|
|
if err != nil && firstErr == nil {
|
|
firstErr = err
|
|
close(t.quit)
|
|
}
|
|
}
|
|
|
|
if firstErr != nil {
|
|
return firstErr
|
|
}
|
|
|
|
// check whether we stopped compaction due to context being cancelled.
|
|
select {
|
|
case <-t.ctx.Done():
|
|
return nil
|
|
default:
|
|
}
|
|
|
|
level.Info(t.logger).Log("msg", "finished compacting the dbs")
|
|
return nil
|
|
}
|
|
|
|
func (t *table) cleanup() error {
|
|
if t.compactedDB != nil {
|
|
err := t.compactedDB.Close()
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
return os.RemoveAll(t.workingDirectory)
|
|
}
|
|
|
|
// writeBatch writes a batch to compactedDB
|
|
func (t *table) writeBatch(batch []indexEntry) error {
|
|
return t.compactedDB.Batch(func(tx *bbolt.Tx) error {
|
|
b, err := tx.CreateBucketIfNotExists(bucketName)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
for _, w := range batch {
|
|
err = b.Put(w.k, w.v)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
return nil
|
|
})
|
|
}
|
|
|
|
// readFile reads a boltdb file from a path and writes the index in batched mode to compactedDB
|
|
func (t *table) readFile(path string) error {
|
|
level.Debug(t.logger).Log("msg", "reading file for compaction", "path", path)
|
|
|
|
db, err := openBoltdbFileWithNoSync(path)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
defer func() {
|
|
if err := db.Close(); err != nil {
|
|
level.Error(t.logger).Log("msg", "failed to close db", "path", path, "err", err)
|
|
}
|
|
|
|
if err = os.Remove(path); err != nil {
|
|
level.Error(t.logger).Log("msg", "failed to remove file", "path", path, "err", err)
|
|
}
|
|
}()
|
|
|
|
writeBatch := make([]indexEntry, 0, batchSize)
|
|
|
|
return db.View(func(tx *bbolt.Tx) error {
|
|
b := tx.Bucket(bucketName)
|
|
if b == nil {
|
|
return errors.New("bucket not found")
|
|
}
|
|
|
|
err := b.ForEach(func(k, v []byte) error {
|
|
ie := indexEntry{
|
|
k: make([]byte, len(k)),
|
|
v: make([]byte, len(v)),
|
|
}
|
|
|
|
// make a copy since k, v are only valid for the life of the transaction.
|
|
// See: https://godoc.org/github.com/boltdb/bolt#Cursor.Seek
|
|
copy(ie.k, k)
|
|
copy(ie.v, v)
|
|
|
|
writeBatch = append(writeBatch, ie)
|
|
|
|
if len(writeBatch) == cap(writeBatch) {
|
|
// batch is full, write the batch and create a new one.
|
|
err := t.writeBatch(writeBatch)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
// todo(cyriltovena) we should just re-slice to avoid allocations
|
|
writeBatch = make([]indexEntry, 0, batchSize)
|
|
}
|
|
|
|
return nil
|
|
})
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
// write the remaining batch which might have been left unwritten due to it not being full yet.
|
|
return t.writeBatch(writeBatch)
|
|
})
|
|
}
|
|
|
|
// upload uploads the compacted db in compressed format.
|
|
func (t *table) upload() error {
|
|
compactedDBPath := t.compactedDB.Path()
|
|
|
|
// close the compactedDB to make sure all the writes are processed.
|
|
err := t.compactedDB.Close()
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
t.compactedDB = nil
|
|
|
|
// compress the compactedDB.
|
|
compressedDBPath := fmt.Sprintf("%s.gz", compactedDBPath)
|
|
err = shipper_util.CompressFile(compactedDBPath, compressedDBPath, false)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
// open the file for reading.
|
|
compressedDB, err := os.Open(compressedDBPath)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
defer func() {
|
|
if err := compressedDB.Close(); err != nil {
|
|
level.Error(t.logger).Log("msg", "failed to close file", "path", compactedDBPath, "err", err)
|
|
}
|
|
|
|
if err := os.Remove(compressedDBPath); err != nil {
|
|
level.Error(t.logger).Log("msg", "failed to remove file", "path", compressedDBPath, "err", err)
|
|
}
|
|
}()
|
|
|
|
fileNameFormat := "%s.gz"
|
|
if t.compactedDBRecreated {
|
|
fileNameFormat = "%s" + recreatedCompactedDBSuffix
|
|
}
|
|
fileName := fmt.Sprintf(fileNameFormat, shipper_util.BuildIndexFileName(t.name, uploaderName, fmt.Sprint(time.Now().Unix())))
|
|
level.Info(t.logger).Log("msg", "uploading the compacted file", "fileName", fileName)
|
|
|
|
return t.indexStorageClient.PutFile(t.ctx, t.name, fileName, compressedDB)
|
|
}
|
|
|
|
// removeSourceFilesFromStorage deletes source db files from storage.
|
|
func (t *table) removeSourceFilesFromStorage() error {
|
|
level.Info(t.logger).Log("msg", "removing source db files from storage", "count", len(t.sourceFiles))
|
|
|
|
for _, file := range t.sourceFiles {
|
|
err := t.indexStorageClient.DeleteFile(t.ctx, t.name, file.Name)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
// openBoltdbFileWithNoSync opens a boltdb file and configures it to not sync the file to disk.
|
|
// Compaction process is idempotent and we do not retain the files so there is no need to sync them to disk.
|
|
func openBoltdbFileWithNoSync(path string) (*bbolt.DB, error) {
|
|
boltdb, err := shipper_util.SafeOpenBoltdbFile(path)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// no need to enforce write to disk, we'll upload and delete the file anyway.
|
|
boltdb.NoSync = true
|
|
|
|
return boltdb, nil
|
|
}
|
|
|
|
// findSeedFileIdx returns index of file to use as seed which would then get index from all the files written to.
|
|
// It tries to find previously compacted file(which has uploaderName) which would be the biggest file.
|
|
// In a large cluster, using previously compacted file as seed would significantly reduce compaction time.
|
|
// If it can't find a previously compacted file, it would just use the first file from the list of files.
|
|
func findSeedFileIdx(files []storage.IndexFile) int {
|
|
for i, file := range files {
|
|
if strings.HasPrefix(file.Name, uploaderName) {
|
|
return i
|
|
}
|
|
}
|
|
|
|
return 0
|
|
}
|
|
|