github.com/m3db/m3@v1.5.0/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go (about)

     1  // +build big
     2  //
     3  // Copyright (c) 2017 Uber Technologies, Inc.
     4  //
     5  // Permission is hereby granted, free of charge, to any person obtaining a copy
     6  // of this software and associated documentation files (the "Software"), to deal
     7  // in the Software without restriction, including without limitation the rights
     8  // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
     9  // copies of the Software, and to permit persons to whom the Software is
    10  // furnished to do so, subject to the following conditions:
    11  //
    12  // The above copyright notice and this permission notice shall be included in
    13  // all copies or substantial portions of the Software.
    14  //
    15  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
    16  // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
    17  // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
    18  // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
    19  // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
    20  // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
    21  // THE SOFTWARE.
    22  
    23  package commitlog
    24  
    25  import (
    26  	"errors"
    27  	"fmt"
    28  	"io"
    29  	"io/ioutil"
    30  	"os"
    31  	"reflect"
    32  	"sort"
    33  	"sync"
    34  	"testing"
    35  	"time"
    36  
    37  	"github.com/m3db/m3/src/cluster/shard"
    38  	"github.com/m3db/m3/src/dbnode/digest"
    39  	"github.com/m3db/m3/src/dbnode/encoding"
    40  	"github.com/m3db/m3/src/dbnode/encoding/m3tsz"
    41  	"github.com/m3db/m3/src/dbnode/namespace"
    42  	"github.com/m3db/m3/src/dbnode/persist"
    43  	"github.com/m3db/m3/src/dbnode/persist/fs"
    44  	"github.com/m3db/m3/src/dbnode/persist/fs/commitlog"
    45  	"github.com/m3db/m3/src/dbnode/storage/bootstrap"
    46  	"github.com/m3db/m3/src/dbnode/storage/bootstrap/result"
    47  	"github.com/m3db/m3/src/dbnode/topology"
    48  	tu "github.com/m3db/m3/src/dbnode/topology/testutil"
    49  	"github.com/m3db/m3/src/dbnode/ts"
    50  	"github.com/m3db/m3/src/dbnode/x/xio"
    51  	"github.com/m3db/m3/src/x/checked"
    52  	"github.com/m3db/m3/src/x/context"
    53  	"github.com/m3db/m3/src/x/ident"
    54  	xtime "github.com/m3db/m3/src/x/time"
    55  
    56  	"github.com/leanovate/gopter"
    57  	"github.com/leanovate/gopter/gen"
    58  	"github.com/leanovate/gopter/prop"
    59  	murmur3 "github.com/m3db/stackmurmur3/v2"
    60  	"github.com/stretchr/testify/require"
    61  )
    62  
    63  const maxShards = 1024
    64  const blockSize = 2 * time.Hour
    65  
    66  var (
    67  	testFsOpts        = fs.NewOptions()
    68  	testCommitlogOpts = commitlog.NewOptions()
    69  )
    70  
    71  func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) {
    72  	var (
    73  		parameters = gopter.DefaultTestParameters()
    74  		seed       = time.Now().UnixNano()
    75  		props      = gopter.NewProperties(parameters)
    76  		reporter   = gopter.NewFormatedReporter(true, 160, os.Stdout)
    77  		startTime  = time.Now().Truncate(blockSize)
    78  
    79  		nsOpts = namespace.NewOptions().SetIndexOptions(
    80  			namespace.NewOptions().IndexOptions().SetEnabled(true),
    81  		)
    82  	)
    83  	parameters.MinSuccessfulTests = 40
    84  	parameters.Rng.Seed(seed)
    85  	nsMeta, err := namespace.NewMetadata(testNamespaceID, nsOpts)
    86  	require.NoError(t, err)
    87  
    88  	props.Property("Commitlog bootstrapping properly bootstraps the entire commitlog", prop.ForAll(
    89  		func(input propTestInput) (bool, error) {
    90  			if !input.commitLogExists {
    91  				// If there is no commit log then we need to make sure
    92  				// snapshot exists, regardless of what the prop test generated.
    93  				input.snapshotExists = true
    94  			}
    95  
    96  			var (
    97  				retentionOpts = nsOpts.RetentionOptions().
    98  						SetBufferPast(input.bufferPast).
    99  						SetBufferFuture(input.bufferFuture)
   100  				nsOpts = nsOpts.SetRetentionOptions(retentionOpts)
   101  			)
   102  			nsMeta, err := namespace.NewMetadata(testNamespaceID, nsOpts)
   103  			if err != nil {
   104  				return false, err
   105  			}
   106  
   107  			// Make sure we have a new directory for each test run
   108  			dir, err := ioutil.TempDir("", "foo")
   109  			if err != nil {
   110  				return false, err
   111  			}
   112  			defer func() {
   113  				os.RemoveAll(dir)
   114  			}()
   115  
   116  			var (
   117  				// This is the earliest system time that we would be willing to write
   118  				// a datapoint for, so start with that and let the write themselves
   119  				// continue to increment the current time.
   120  				currentTime = startTime.Add(-input.bufferFuture)
   121  				lock        = sync.RWMutex{}
   122  
   123  				nowFn = func() time.Time {
   124  					lock.RLock()
   125  					curr := currentTime
   126  					lock.RUnlock()
   127  					return curr
   128  				}
   129  			)
   130  
   131  			var (
   132  				fsOpts = testFsOpts.
   133  					SetFilePathPrefix(dir)
   134  				commitLogOpts = testCommitlogOpts.
   135  						SetBlockSize(blockSize).
   136  						SetFilesystemOptions(fsOpts).
   137  						SetStrategy(commitlog.StrategyWriteBehind).
   138  						SetFlushInterval(time.Millisecond).
   139  						SetClockOptions(testCommitlogOpts.ClockOptions().SetNowFn(nowFn))
   140  				bootstrapOpts = testDefaultOpts.SetCommitLogOptions(commitLogOpts).
   141  						SetReturnUnfulfilledForCorruptCommitLogFiles(true)
   142  
   143  				start        = xtime.ToUnixNano(input.currentTime.Truncate(blockSize))
   144  				snapshotTime = xtime.ToUnixNano(input.snapshotTime)
   145  			)
   146  
   147  			writer, err := fs.NewWriter(fsOpts)
   148  			if err != nil {
   149  				return false, err
   150  			}
   151  
   152  			orderedWritesBySeries := map[string][]generatedWrite{}
   153  			for _, write := range input.writes {
   154  				id := write.series.ID
   155  				writesForSeries, ok := orderedWritesBySeries[id.String()]
   156  				if !ok {
   157  					writesForSeries = []generatedWrite{}
   158  				}
   159  				writesForSeries = append(writesForSeries, write)
   160  				orderedWritesBySeries[id.String()] = writesForSeries
   161  			}
   162  
   163  			for _, writesForSeries := range orderedWritesBySeries {
   164  				writesForSeries := writesForSeries
   165  				sort.Slice(writesForSeries, func(i, j int) bool {
   166  					return writesForSeries[i].datapoint.TimestampNanos.
   167  						Before(writesForSeries[j].datapoint.TimestampNanos)
   168  				})
   169  			}
   170  
   171  			if input.snapshotExists {
   172  				compressedWritesByShards := map[uint32]map[string][]byte{}
   173  				for seriesID, writesForSeries := range orderedWritesBySeries {
   174  					shard := hashIDToShard(ident.StringID(seriesID))
   175  					encodersBySeries, ok := compressedWritesByShards[shard]
   176  					if !ok {
   177  						encodersBySeries = map[string][]byte{}
   178  						compressedWritesByShards[shard] = encodersBySeries
   179  					}
   180  
   181  					encoder := m3tsz.NewEncoder(writesForSeries[0].datapoint.TimestampNanos,
   182  						nil, true, encoding.NewOptions())
   183  					for _, value := range writesForSeries {
   184  						// Only include datapoints that are before or during the snapshot time to ensure that we
   185  						// properly bootstrap from both snapshot files and commit logs and merge them together.
   186  						// Note that if the commit log does not exist we ignore the snapshot time because we need
   187  						// the snapshot to include all the data.
   188  						if !input.commitLogExists ||
   189  							value.arrivedAt.Before(input.snapshotTime) ||
   190  							value.arrivedAt.Equal(input.snapshotTime) {
   191  							err := encoder.Encode(value.datapoint, value.unit, value.annotation)
   192  							if err != nil {
   193  								return false, err
   194  							}
   195  						}
   196  					}
   197  
   198  					ctx := context.NewBackground()
   199  					reader, ok := encoder.Stream(ctx)
   200  					if ok {
   201  						bytes, err := xio.ToBytes(reader)
   202  						if !errors.Is(err, io.EOF) {
   203  							return false, err
   204  						}
   205  						encodersBySeries[seriesID] = bytes
   206  					}
   207  					ctx.Close()
   208  
   209  					compressedWritesByShards[shard] = encodersBySeries
   210  				}
   211  
   212  				for shard, seriesForShard := range compressedWritesByShards {
   213  					err = writer.Open(fs.DataWriterOpenOptions{
   214  						Identifier: fs.FileSetFileIdentifier{
   215  							Namespace:   nsMeta.ID(),
   216  							BlockStart:  start,
   217  							Shard:       shard,
   218  							VolumeIndex: 0,
   219  						},
   220  						BlockSize:   blockSize,
   221  						FileSetType: persist.FileSetSnapshotType,
   222  						Snapshot: fs.DataWriterSnapshotOptions{
   223  							SnapshotTime: snapshotTime,
   224  						},
   225  					})
   226  
   227  					if err != nil {
   228  						return false, err
   229  					}
   230  
   231  					for seriesID, data := range seriesForShard {
   232  						checkedBytes := checked.NewBytes(data, nil)
   233  						checkedBytes.IncRef()
   234  						tags := orderedWritesBySeries[seriesID][0].tags
   235  						metadata := persist.NewMetadataFromIDAndTags(ident.StringID(seriesID), tags,
   236  							persist.MetadataOptions{})
   237  						writer.Write(metadata, checkedBytes, digest.Checksum(data))
   238  					}
   239  
   240  					err = writer.Close()
   241  					if err != nil {
   242  						return false, err
   243  					}
   244  				}
   245  			}
   246  
   247  			if input.commitLogExists {
   248  				writesCh := make(chan struct{}, 5)
   249  				go func() {
   250  					for range writesCh {
   251  						lock.Lock()
   252  						currentTime = currentTime.Add(time.Millisecond)
   253  						lock.Unlock()
   254  					}
   255  				}()
   256  
   257  				// Instantiate commitlog
   258  				log, err := commitlog.NewCommitLog(commitLogOpts)
   259  				if err != nil {
   260  					return false, err
   261  				}
   262  				err = log.Open()
   263  				if err != nil {
   264  					return false, err
   265  				}
   266  
   267  				sort.Slice(input.writes, func(i, j int) bool {
   268  					return input.writes[i].arrivedAt.Before(input.writes[j].arrivedAt)
   269  				})
   270  
   271  				// Write all the datapoints to the commitlog
   272  				for _, write := range input.writes {
   273  					// Only write datapoints that are not in the snapshots.
   274  					if input.snapshotExists &&
   275  						!write.arrivedAt.After(input.snapshotTime) {
   276  						continue
   277  					}
   278  
   279  					lock.Lock()
   280  					currentTime = write.arrivedAt
   281  					lock.Unlock()
   282  
   283  					err := log.Write(context.NewBackground(), write.series, write.datapoint, write.unit, write.annotation)
   284  					if err != nil {
   285  						return false, err
   286  					}
   287  
   288  					writesCh <- struct{}{}
   289  				}
   290  				close(writesCh)
   291  
   292  				err = log.Close()
   293  				if err != nil {
   294  					return false, err
   295  				}
   296  
   297  				if input.includeCorruptedCommitlogFile {
   298  					// Write out an additional commit log file with a corrupt info header to
   299  					// make sure that the commitlog source skips it in the single node scenario.
   300  					commitLogFiles, corruptFiles, err := commitlog.Files(commitLogOpts)
   301  					if err != nil {
   302  						return false, err
   303  					}
   304  					if len(corruptFiles) > 0 {
   305  						return false, fmt.Errorf("found corrupt commit log files: %v", corruptFiles)
   306  					}
   307  
   308  					if len(commitLogFiles) > 0 {
   309  						nextCommitLogFile, _, err := commitlog.NextFile(commitLogOpts)
   310  						if err != nil {
   311  							return false, err
   312  						}
   313  
   314  						err = ioutil.WriteFile(nextCommitLogFile, []byte("corruption"), 0644)
   315  						if err != nil {
   316  							return false, err
   317  						}
   318  					}
   319  				}
   320  			}
   321  
   322  			// Instantiate a commitlog source
   323  			inspection, err := fs.InspectFilesystem(fsOpts)
   324  			if err != nil {
   325  				return false, err
   326  			}
   327  			provider, err := NewCommitLogBootstrapperProvider(bootstrapOpts, inspection, nil)
   328  			if err != nil {
   329  				return false, err
   330  			}
   331  			source, err := provider.Provide()
   332  			if err != nil {
   333  				return false, err
   334  			}
   335  
   336  			// Determine time range to bootstrap
   337  			end := xtime.ToUnixNano(input.currentTime.Add(blockSize))
   338  			ranges := xtime.NewRanges(xtime.Range{Start: start, End: end})
   339  
   340  			// Determine which shards we need to bootstrap (based on the randomly
   341  			// generated data)
   342  			var (
   343  				allShardsMap   = map[uint32]bool{}
   344  				allShardsSlice = []uint32{}
   345  			)
   346  			for _, write := range input.writes {
   347  				shard := write.series.Shard
   348  				if _, ok := allShardsMap[shard]; !ok {
   349  					allShardsSlice = append(allShardsSlice, shard)
   350  				}
   351  				allShardsMap[shard] = true
   352  			}
   353  
   354  			// Assign the previously-determined bootstrap range to each known shard
   355  			shardTimeRanges := result.NewShardTimeRanges()
   356  			for shard := range allShardsMap {
   357  				shardTimeRanges.Set(shard, ranges)
   358  			}
   359  
   360  			// Perform the bootstrap
   361  			var initialTopoState *topology.StateSnapshot
   362  			if input.multiNodeCluster {
   363  				initialTopoState = tu.NewStateSnapshot(2, tu.HostShardStates{
   364  					tu.SelfID:   tu.Shards(allShardsSlice, shard.Available),
   365  					"not-self1": tu.Shards(allShardsSlice, shard.Available),
   366  					"not-self2": tu.Shards(allShardsSlice, shard.Available),
   367  				})
   368  			} else {
   369  				initialTopoState = tu.NewStateSnapshot(1, tu.HostShardStates{
   370  					tu.SelfID: tu.Shards(allShardsSlice, shard.Available),
   371  				})
   372  			}
   373  
   374  			runOpts := testDefaultRunOpts.SetInitialTopologyState(initialTopoState)
   375  			tester := bootstrap.BuildNamespacesTesterWithFilesystemOptions(t, runOpts, shardTimeRanges, fsOpts, nsMeta)
   376  
   377  			ctx := context.NewBackground()
   378  			defer ctx.Close()
   379  
   380  			bootstrapResults, err := source.Bootstrap(ctx, tester.Namespaces, tester.Cache)
   381  			if err != nil {
   382  				return false, err
   383  			}
   384  
   385  			// Create testValues for each datapoint for comparison
   386  			values := testValues{}
   387  			for _, write := range input.writes {
   388  				values = append(values, testValue{
   389  					write.series, write.datapoint.TimestampNanos,
   390  					write.datapoint.Value, write.unit, write.annotation})
   391  			}
   392  
   393  			commitLogFiles, corruptFiles, err := commitlog.Files(commitLogOpts)
   394  			if err != nil {
   395  				return false, err
   396  			}
   397  			if len(corruptFiles) > 0 && !input.includeCorruptedCommitlogFile {
   398  				return false, fmt.Errorf("found corrupt commit log files: %v", corruptFiles)
   399  			}
   400  
   401  			var (
   402  				commitLogFilesExist = len(commitLogFiles) > 0
   403  				// In the multi-node setup we want to return unfulfilled if there are any corrupt files, but
   404  				// we always want to return fulfilled in the single node setup. In addition, the source will not
   405  				// return unfulfilled in the presence of corrupt files if the range we request it to bootstrap
   406  				// is empty so we need to handle that case too.
   407  				shouldReturnUnfulfilled = input.multiNodeCluster &&
   408  					input.includeCorruptedCommitlogFile &&
   409  					commitLogFilesExist &&
   410  					!shardTimeRanges.IsEmpty()
   411  			)
   412  
   413  			nsResult, found := bootstrapResults.Results.Get(nsMeta.ID())
   414  			if !found {
   415  				return false, fmt.Errorf("could not find id: %s", nsMeta.ID().String())
   416  			}
   417  
   418  			dataResult := nsResult.DataResult
   419  			if shouldReturnUnfulfilled {
   420  				if dataResult.Unfulfilled().IsEmpty() {
   421  					return false, fmt.Errorf(
   422  						"data result unfulfilled should not be empty in multi node cluster but was")
   423  				}
   424  			} else {
   425  				if !dataResult.Unfulfilled().IsEmpty() {
   426  					return false, fmt.Errorf(
   427  						"data result unfulfilled in single node cluster should be empty but was: %s",
   428  						dataResult.Unfulfilled().String())
   429  				}
   430  			}
   431  
   432  			written, err := tester.EnsureDumpAllForNamespace(nsMeta)
   433  			if err != nil {
   434  				return false, err
   435  			}
   436  
   437  			indexResult := nsResult.IndexResult
   438  			if shouldReturnUnfulfilled {
   439  				if indexResult.Unfulfilled().IsEmpty() {
   440  					return false, fmt.Errorf(
   441  						"index result unfulfilled should not be empty in multi node cluster but was")
   442  				}
   443  			} else {
   444  				if !indexResult.Unfulfilled().IsEmpty() {
   445  					return false, fmt.Errorf(
   446  						"index result unfulfilled in single node cluster should be empty but was: %s",
   447  						indexResult.Unfulfilled().String())
   448  				}
   449  			}
   450  
   451  			err = verifyValuesAreCorrect(values, written)
   452  			if err != nil {
   453  				return false, err
   454  			}
   455  
   456  			return true, nil
   457  		},
   458  		genPropTestInputs(t, nsMeta, startTime),
   459  	))
   460  
   461  	if !props.Run(reporter) {
   462  		t.Errorf("failed with initial seed: %d and startTime: %d", seed, startTime.UnixNano())
   463  	}
   464  }
   465  
   466  type propTestInput struct {
   467  	currentTime                   time.Time
   468  	snapshotTime                  time.Time
   469  	snapshotExists                bool
   470  	commitLogExists               bool
   471  	bufferPast                    time.Duration
   472  	bufferFuture                  time.Duration
   473  	writes                        []generatedWrite
   474  	includeCorruptedCommitlogFile bool
   475  	multiNodeCluster              bool
   476  }
   477  
   478  type generatedWrite struct {
   479  	// arrivedAt is used to simulate out-of-order writes which arrive somewhere
   480  	// between time.Now().Add(-bufferFuture) and time.Now().Add(bufferPast).
   481  	arrivedAt  time.Time
   482  	series     ts.Series
   483  	tags       ident.Tags
   484  	datapoint  ts.Datapoint
   485  	unit       xtime.Unit
   486  	annotation ts.Annotation
   487  }
   488  
   489  func (w generatedWrite) String() string {
   490  	return fmt.Sprintf("ID = %v, Datapoint = %+v", w.series.ID.String(), w.datapoint)
   491  }
   492  
   493  func genPropTestInputs(t *testing.T, nsMeta namespace.Metadata, blockStart time.Time) gopter.Gen {
   494  	curriedGenPropTestInput := func(input interface{}) gopter.Gen {
   495  		var (
   496  			inputs                        = input.([]interface{})
   497  			snapshotTime                  = inputs[0].(time.Time)
   498  			snapshotExists                = inputs[1].(bool)
   499  			commitLogExists               = inputs[2].(bool)
   500  			bufferPast                    = time.Duration(inputs[3].(int64))
   501  			bufferFuture                  = time.Duration(inputs[4].(int64))
   502  			numDatapoints                 = inputs[5].(int)
   503  			includeCorruptedCommitlogFile = inputs[6].(bool)
   504  			multiNodeCluster              = inputs[7].(bool)
   505  		)
   506  
   507  		return genPropTestInput(
   508  			t,
   509  			blockStart, bufferPast, bufferFuture,
   510  			snapshotTime, snapshotExists, commitLogExists,
   511  			numDatapoints, nsMeta.ID().String(), includeCorruptedCommitlogFile, multiNodeCluster)
   512  	}
   513  
   514  	return gopter.CombineGens(
   515  		// Run iterations of the test with the snapshot time set at any point
   516  		// between the beginning and end of the block.
   517  		gen.TimeRange(blockStart, blockSize),
   518  		// SnapshotExists
   519  		gen.Bool(),
   520  		// CommitLogExists
   521  		gen.Bool(),
   522  		// Run iterations with any bufferPast/bufferFuture between zero and
   523  		// the namespace blockSize (distinct from the commitLog blockSize).
   524  		gen.Int64Range(0, int64(blockSize)),
   525  		gen.Int64Range(0, int64(blockSize)),
   526  		// Run iterations of the test with between 0 and 100 datapoints
   527  		gen.IntRange(0, 100),
   528  		// Whether the test should generate an additional corrupt commitlog file
   529  		// to ensure the commit log bootstrapper skips it correctly.
   530  		gen.Bool(),
   531  		// Whether the test should simulate the InitialTopologyState to mimic a
   532  		// multi node cluster or not.
   533  		gen.Bool(),
   534  	).FlatMap(curriedGenPropTestInput, reflect.TypeOf(propTestInput{}))
   535  }
   536  
   537  func genPropTestInput(
   538  	t *testing.T,
   539  	start time.Time,
   540  	bufferPast,
   541  	bufferFuture time.Duration,
   542  	snapshotTime time.Time,
   543  	snapshotExists bool,
   544  	commitLogExists bool,
   545  	numDatapoints int,
   546  	ns string,
   547  	includeCorruptedCommitlogFile bool,
   548  	multiNodeCluster bool,
   549  ) gopter.Gen {
   550  	return gen.SliceOfN(numDatapoints, genWrite(t, start, bufferPast, bufferFuture, ns)).
   551  		Map(func(val []generatedWrite) propTestInput {
   552  			return propTestInput{
   553  				currentTime:                   start,
   554  				bufferFuture:                  bufferFuture,
   555  				bufferPast:                    bufferPast,
   556  				snapshotTime:                  snapshotTime,
   557  				snapshotExists:                snapshotExists,
   558  				commitLogExists:               commitLogExists,
   559  				writes:                        val,
   560  				includeCorruptedCommitlogFile: includeCorruptedCommitlogFile,
   561  				multiNodeCluster:              multiNodeCluster,
   562  			}
   563  		})
   564  }
   565  
   566  func genWrite(t *testing.T, start time.Time, bufferPast, bufferFuture time.Duration, ns string) gopter.Gen {
   567  	latestDatapointTime := start.Truncate(blockSize).Add(blockSize).Sub(start)
   568  
   569  	return gopter.CombineGens(
   570  		// Identifier
   571  		gen.Identifier(),
   572  		// Only generate writes within the current block period
   573  		gen.TimeRange(start, latestDatapointTime),
   574  		// Boolean indicating whether we should move offset the datapoint by
   575  		// the maximum of eithe bufferPast or bufferFuture.
   576  		gen.Bool(),
   577  		// Tag key/val
   578  		gen.Identifier(),
   579  		gen.Identifier(),
   580  		// Boolean indicating whether or not to include tags for this series. We want to
   581  		// sometimes not include tags to ensure that the commitlog writer/readers can
   582  		// handle both series that have tags and those that don't.
   583  		gen.Bool(),
   584  		// M3TSZ is lossy, so we want to avoid very large numbers with high amounts of precision
   585  		gen.Float64Range(-9999999, 99999999),
   586  	).Map(func(val []interface{}) generatedWrite {
   587  		var (
   588  			id                 = val[0].(string)
   589  			tm                 = val[1].(time.Time)
   590  			a                  = tm
   591  			bufferPastOrFuture = val[2].(bool)
   592  			tagKey             = val[3].(string)
   593  			tagVal             = val[4].(string)
   594  			includeTags        = val[5].(bool)
   595  			v                  = val[6].(float64)
   596  
   597  			tagEncoderPool = testCommitlogOpts.FilesystemOptions().TagEncoderPool()
   598  			tagSliceIter   = ident.NewTagsIterator(ident.Tags{})
   599  		)
   600  
   601  		if bufferPastOrFuture {
   602  			a = a.Add(-bufferFuture)
   603  		} else {
   604  			a = a.Add(bufferPast)
   605  		}
   606  
   607  		tags := seriesUniqueTags(id, tagKey, tagVal, includeTags)
   608  		tagSliceIter.Reset(tags)
   609  
   610  		tagEncoder := tagEncoderPool.Get()
   611  		err := tagEncoder.Encode(tagSliceIter)
   612  		require.NoError(t, err)
   613  
   614  		encodedTagsChecked, ok := tagEncoder.Data()
   615  		require.True(t, ok)
   616  
   617  		return generatedWrite{
   618  			arrivedAt: a,
   619  			series: ts.Series{
   620  				ID:          ident.StringID(id),
   621  				Namespace:   ident.StringID(ns),
   622  				Shard:       hashIDToShard(ident.StringID(id)),
   623  				UniqueIndex: seriesUniqueIndex(id),
   624  				EncodedTags: ts.EncodedTags(encodedTagsChecked.Bytes()),
   625  			},
   626  			tags: tags,
   627  			datapoint: ts.Datapoint{
   628  				TimestampNanos: xtime.ToUnixNano(tm),
   629  				Value:          v,
   630  			},
   631  			unit: xtime.Nanosecond,
   632  		}
   633  	})
   634  }
   635  
   636  type globalSeriesRegistry struct {
   637  	sync.Mutex
   638  
   639  	idx      uint64
   640  	idToIdx  map[string]uint64
   641  	idToTags map[string]ident.Tags
   642  }
   643  
   644  var seriesRegistry = globalSeriesRegistry{
   645  	idToIdx:  make(map[string]uint64),
   646  	idToTags: make(map[string]ident.Tags),
   647  }
   648  
   649  // seriesUniqueIndex ensures that each string series ID maps to exactly one UniqueIndex
   650  func seriesUniqueIndex(series string) uint64 {
   651  	seriesRegistry.Lock()
   652  	defer seriesRegistry.Unlock()
   653  
   654  	idx, ok := seriesRegistry.idToIdx[series]
   655  	if ok {
   656  		return idx
   657  	}
   658  
   659  	idx = seriesRegistry.idx
   660  	seriesRegistry.idx++
   661  	seriesRegistry.idToIdx[series] = idx
   662  	return idx
   663  }
   664  
   665  // seriesUniqueTag ensures that each string series ID ALWAYS maps to the same set of tags
   666  func seriesUniqueTags(seriesID, proposedTagKey, proposedTagVal string, includeTags bool) ident.Tags {
   667  	seriesRegistry.Lock()
   668  	defer seriesRegistry.Unlock()
   669  
   670  	tags, ok := seriesRegistry.idToTags[seriesID]
   671  	if ok {
   672  		return tags
   673  	}
   674  
   675  	if includeTags {
   676  		tags = ident.NewTags(ident.StringTag(proposedTagKey, proposedTagVal))
   677  	}
   678  	seriesRegistry.idToTags[seriesID] = tags
   679  	return tags
   680  }
   681  
   682  // hashIDToShard generates a HashFn based on murmur32
   683  func hashIDToShard(id ident.ID) uint32 {
   684  	return murmur3.Sum32(id.Bytes()) % uint32(maxShards)
   685  }