github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/index/compaction/compactor.go (about)

     1  // Copyright (c) 2018 Uber Technologies, Inc.
     2  //
     3  // Permission is hereby granted, free of charge, to any person obtaining a copy
     4  // of this software and associated documentation files (the "Software"), to deal
     5  // in the Software without restriction, including without limitation the rights
     6  // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
     7  // copies of the Software, and to permit persons to whom the Software is
     8  // furnished to do so, subject to the following conditions:
     9  //
    10  // The above copyright notice and this permission notice shall be included in
    11  // all copies or substantial portions of the Software.
    12  //
    13  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
    14  // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
    15  // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
    16  // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
    17  // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
    18  // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
    19  // THE SOFTWARE.
    20  
    21  package compaction
    22  
    23  import (
    24  	"bytes"
    25  	"errors"
    26  	"io"
    27  	"sync"
    28  
    29  	"github.com/m3db/m3/src/m3ninx/doc"
    30  	"github.com/m3db/m3/src/m3ninx/index"
    31  	"github.com/m3db/m3/src/m3ninx/index/segment"
    32  	"github.com/m3db/m3/src/m3ninx/index/segment/builder"
    33  	"github.com/m3db/m3/src/m3ninx/index/segment/fst"
    34  	"github.com/m3db/m3/src/m3ninx/index/segment/fst/encoding/docs"
    35  	xerrors "github.com/m3db/m3/src/x/errors"
    36  	"github.com/m3db/m3/src/x/mmap"
    37  )
    38  
    39  var (
    40  	// ErrCompactorBuilderEmpty is returned when the compaction
    41  	// would result in an empty segment.
    42  	ErrCompactorBuilderEmpty = errors.New("builder has no documents")
    43  	errCompactorBuilderNil   = errors.New("builder is nil")
    44  	errCompactorClosed       = errors.New("compactor is closed")
    45  )
    46  
    47  // Compactor is a compactor.
    48  type Compactor struct {
    49  	sync.RWMutex
    50  
    51  	opts         CompactorOptions
    52  	writer       fst.Writer
    53  	metadataPool doc.MetadataArrayPool
    54  	docsMaxBatch int
    55  	fstOpts      fst.Options
    56  	builder      segment.SegmentsBuilder
    57  	buff         *bytes.Buffer
    58  	closed       bool
    59  }
    60  
    61  // CompactorOptions is a set of compactor options.
    62  type CompactorOptions struct {
    63  	// FSTWriterOptions if not nil are the options used to
    64  	// construct the FST writer.
    65  	FSTWriterOptions *fst.WriterOptions
    66  
    67  	// MmapDocsData when enabled will encode and mmmap the
    68  	// documents data, rather than keeping the original
    69  	// documents with references to substrings in the metric
    70  	// IDs (done for memory savings).
    71  	MmapDocsData bool
    72  }
    73  
    74  // NewCompactor returns a new compactor which reuses buffers
    75  // to avoid allocating intermediate buffers when compacting.
    76  func NewCompactor(
    77  	metadataPool doc.MetadataArrayPool,
    78  	docsMaxBatch int,
    79  	builderOpts builder.Options,
    80  	fstOpts fst.Options,
    81  	opts CompactorOptions,
    82  ) (*Compactor, error) {
    83  	var fstWriterOpts fst.WriterOptions
    84  	if v := opts.FSTWriterOptions; v != nil {
    85  		fstWriterOpts = *v
    86  	}
    87  	writer, err := fst.NewWriter(fstWriterOpts)
    88  	if err != nil {
    89  		return nil, err
    90  	}
    91  	return &Compactor{
    92  		opts:         opts,
    93  		writer:       writer,
    94  		metadataPool: metadataPool,
    95  		docsMaxBatch: docsMaxBatch,
    96  		builder:      builder.NewBuilderFromSegments(builderOpts),
    97  		fstOpts:      fstOpts,
    98  		buff:         bytes.NewBuffer(nil),
    99  	}, nil
   100  }
   101  
   102  // CompactResult is the result of a call to compact.
   103  type CompactResult struct {
   104  	Compacted        fst.Segment
   105  	SegmentMetadatas []segment.SegmentsBuilderSegmentMetadata
   106  }
   107  
   108  // Compact will take a set of segments and compact them into an immutable
   109  // FST segment, if there is a single mutable segment it can directly be
   110  // converted into an FST segment, otherwise an intermediary mutable segment
   111  // (reused by the compactor between runs) is used to combine all the segments
   112  // together first before compacting into an FST segment.
   113  // Note: This is not thread safe and only a single compaction may happen at a
   114  // time.
   115  func (c *Compactor) Compact(
   116  	segs []segment.Segment,
   117  	filter segment.DocumentsFilter,
   118  	reporterOptions mmap.ReporterOptions,
   119  ) (CompactResult, error) {
   120  	c.Lock()
   121  	defer c.Unlock()
   122  
   123  	if c.closed {
   124  		return CompactResult{}, errCompactorClosed
   125  	}
   126  
   127  	c.builder.Reset()
   128  	c.builder.SetFilter(filter)
   129  	if err := c.builder.AddSegments(segs); err != nil {
   130  		return CompactResult{}, err
   131  	}
   132  
   133  	metas, err := c.builder.SegmentMetadatas()
   134  	if err != nil {
   135  		return CompactResult{}, err
   136  	}
   137  
   138  	compacted, err := c.compactFromBuilderWithLock(c.builder, reporterOptions)
   139  	if err != nil {
   140  		return CompactResult{}, err
   141  	}
   142  
   143  	return CompactResult{
   144  		Compacted:        compacted,
   145  		SegmentMetadatas: metas,
   146  	}, nil
   147  }
   148  
   149  // CompactUsingBuilder compacts segments together using a provided segment builder.
   150  func (c *Compactor) CompactUsingBuilder(
   151  	builder segment.DocumentsBuilder,
   152  	segs []segment.Segment,
   153  	reporterOptions mmap.ReporterOptions,
   154  ) (fst.Segment, error) {
   155  	// NB(r): Ensure only single compaction happens at a time since the buffers are
   156  	// reused between runs.
   157  	c.Lock()
   158  	defer c.Unlock()
   159  
   160  	if c.closed {
   161  		return nil, errCompactorClosed
   162  	}
   163  
   164  	if builder == nil {
   165  		return nil, errCompactorBuilderNil
   166  	}
   167  
   168  	if len(segs) == 0 {
   169  		// No segments to compact, just compact from the builder
   170  		return c.compactFromBuilderWithLock(builder, reporterOptions)
   171  	}
   172  
   173  	// Need to combine segments first
   174  	batch := c.metadataPool.Get()
   175  	defer func() {
   176  		c.metadataPool.Put(batch)
   177  	}()
   178  
   179  	// flushBatch is declared to reuse the same code from the
   180  	// inner loop and the completion of the loop
   181  	flushBatch := func() error {
   182  		if len(batch) == 0 {
   183  			// Last flush might not have any docs enqueued
   184  			return nil
   185  		}
   186  
   187  		err := builder.InsertBatch(index.Batch{
   188  			Docs:                batch,
   189  			AllowPartialUpdates: true,
   190  		})
   191  		if err != nil && index.IsBatchPartialError(err) {
   192  			// If after filtering out duplicate ID errors
   193  			// there are no errors, then this was a successful
   194  			// insertion.
   195  			batchErr := err.(*index.BatchPartialError)
   196  			// NB(r): FilterDuplicateIDErrors returns nil
   197  			// if no errors remain after filtering duplicate ID
   198  			// errors, this case is covered in unit tests.
   199  			err = batchErr.FilterDuplicateIDErrors()
   200  		}
   201  		if err != nil {
   202  			return err
   203  		}
   204  
   205  		// Reset docs batch for reuse
   206  		var empty doc.Metadata
   207  		for i := range batch {
   208  			batch[i] = empty
   209  		}
   210  		batch = batch[:0]
   211  		return nil
   212  	}
   213  
   214  	for _, seg := range segs {
   215  		reader, err := seg.Reader()
   216  		if err != nil {
   217  			return nil, err
   218  		}
   219  
   220  		iter, err := reader.AllDocs()
   221  		if err != nil {
   222  			return nil, err
   223  		}
   224  
   225  		for iter.Next() {
   226  			batch = append(batch, iter.Current())
   227  			if len(batch) < c.docsMaxBatch {
   228  				continue
   229  			}
   230  			if err := flushBatch(); err != nil {
   231  				return nil, err
   232  			}
   233  		}
   234  
   235  		if err := iter.Err(); err != nil {
   236  			return nil, err
   237  		}
   238  		if err := iter.Close(); err != nil {
   239  			return nil, err
   240  		}
   241  		if err := reader.Close(); err != nil {
   242  			return nil, err
   243  		}
   244  	}
   245  
   246  	// Last flush in case some remaining docs that
   247  	// weren't written to the mutable segment
   248  	if err := flushBatch(); err != nil {
   249  		return nil, err
   250  	}
   251  
   252  	return c.compactFromBuilderWithLock(builder, reporterOptions)
   253  }
   254  
   255  func (c *Compactor) compactFromBuilderWithLock(
   256  	builder segment.Builder,
   257  	reporterOptions mmap.ReporterOptions,
   258  ) (fst.Segment, error) {
   259  	defer func() {
   260  		// Release resources regardless of result,
   261  		// otherwise old compacted segments are held onto
   262  		// strongly
   263  		builder.Reset()
   264  	}()
   265  
   266  	// Since this builder is likely reused between compaction
   267  	// runs, we need to copy the docs slice
   268  	allDocs := builder.Docs()
   269  	if len(allDocs) == 0 {
   270  		return nil, ErrCompactorBuilderEmpty
   271  	}
   272  
   273  	err := c.writer.Reset(builder)
   274  	if err != nil {
   275  		return nil, err
   276  	}
   277  
   278  	success := false
   279  	closers := new(closers)
   280  	fstData := fst.SegmentData{
   281  		Version: fst.Version{
   282  			Major: c.writer.MajorVersion(),
   283  			Minor: c.writer.MinorVersion(),
   284  		},
   285  		Metadata: append([]byte(nil), c.writer.Metadata()...),
   286  		Closer:   closers,
   287  	}
   288  
   289  	// Cleanup incase we run into issues
   290  	defer func() {
   291  		if !success {
   292  			closers.Close()
   293  		}
   294  	}()
   295  
   296  	if !c.opts.MmapDocsData {
   297  		// If retaining references to the original docs, simply take ownership
   298  		// of the documents and then reference them directly from the FST segment
   299  		// rather than encoding them and mmap'ing the encoded documents.
   300  		allDocsCopy := make([]doc.Metadata, len(allDocs))
   301  		copy(allDocsCopy, allDocs)
   302  		fstData.DocsReader = docs.NewSliceReader(allDocsCopy)
   303  	} else {
   304  		// Otherwise encode and reference the encoded bytes as mmap'd bytes.
   305  		c.buff.Reset()
   306  		if err := c.writer.WriteDocumentsData(c.buff); err != nil {
   307  			return nil, err
   308  		}
   309  
   310  		fstData.DocsData, err = c.mmapAndAppendCloser(c.buff.Bytes(), closers, reporterOptions)
   311  		if err != nil {
   312  			return nil, err
   313  		}
   314  
   315  		c.buff.Reset()
   316  		if err := c.writer.WriteDocumentsIndex(c.buff); err != nil {
   317  			return nil, err
   318  		}
   319  
   320  		fstData.DocsIdxData, err = c.mmapAndAppendCloser(c.buff.Bytes(), closers, reporterOptions)
   321  		if err != nil {
   322  			return nil, err
   323  		}
   324  	}
   325  
   326  	c.buff.Reset()
   327  	if err := c.writer.WritePostingsOffsets(c.buff); err != nil {
   328  		return nil, err
   329  	}
   330  
   331  	fstData.PostingsData, err = c.mmapAndAppendCloser(c.buff.Bytes(), closers, reporterOptions)
   332  	if err != nil {
   333  		return nil, err
   334  	}
   335  
   336  	c.buff.Reset()
   337  	if err := c.writer.WriteFSTTerms(c.buff); err != nil {
   338  		return nil, err
   339  	}
   340  
   341  	fstData.FSTTermsData, err = c.mmapAndAppendCloser(c.buff.Bytes(), closers, reporterOptions)
   342  	if err != nil {
   343  		return nil, err
   344  	}
   345  
   346  	c.buff.Reset()
   347  	if err := c.writer.WriteFSTFields(c.buff); err != nil {
   348  		return nil, err
   349  	}
   350  
   351  	fstData.FSTFieldsData, err = c.mmapAndAppendCloser(c.buff.Bytes(), closers, reporterOptions)
   352  	if err != nil {
   353  		return nil, err
   354  	}
   355  
   356  	compacted, err := fst.NewSegment(fstData, c.fstOpts)
   357  	if err != nil {
   358  		return nil, err
   359  	}
   360  
   361  	success = true
   362  
   363  	return compacted, nil
   364  }
   365  
   366  func (c *Compactor) mmapAndAppendCloser(
   367  	fromBytes []byte,
   368  	closers *closers,
   369  	reporterOptions mmap.ReporterOptions,
   370  ) (mmap.Descriptor, error) {
   371  	// Copy bytes to new mmap region to hide from the GC
   372  	mmapedResult, err := mmap.Bytes(int64(len(fromBytes)), mmap.Options{
   373  		Read:            true,
   374  		Write:           true,
   375  		ReporterOptions: reporterOptions,
   376  	})
   377  	if err != nil {
   378  		return mmap.Descriptor{}, err
   379  	}
   380  	copy(mmapedResult.Bytes, fromBytes)
   381  
   382  	closers.Append(closer(func() error {
   383  		return mmap.Munmap(mmapedResult)
   384  	}))
   385  
   386  	return mmapedResult, nil
   387  }
   388  
   389  // Close closes the compactor and frees buffered resources.
   390  func (c *Compactor) Close() error {
   391  	c.Lock()
   392  	defer c.Unlock()
   393  
   394  	if c.closed {
   395  		return errCompactorClosed
   396  	}
   397  
   398  	c.closed = true
   399  
   400  	c.writer = nil
   401  	c.metadataPool = nil
   402  	c.fstOpts = nil
   403  	c.builder = nil
   404  	c.buff = nil
   405  
   406  	return nil
   407  }
   408  
   409  var _ io.Closer = closer(nil)
   410  
   411  type closer func() error
   412  
   413  func (c closer) Close() error {
   414  	return c()
   415  }
   416  
   417  var _ io.Closer = &closers{}
   418  
   419  type closers struct {
   420  	closers []io.Closer
   421  }
   422  
   423  func (c *closers) Append(elem io.Closer) {
   424  	c.closers = append(c.closers, elem)
   425  }
   426  
   427  func (c *closers) Close() error {
   428  	multiErr := xerrors.NewMultiError()
   429  	for _, elem := range c.closers {
   430  		multiErr = multiErr.Add(elem.Close())
   431  	}
   432  	return multiErr.FinalError()
   433  }