diff --git a/src/cmd/tools/dtest/util/seed/generator.go b/src/cmd/tools/dtest/util/seed/generator.go index b6406a3311..b1523e2079 100644 --- a/src/cmd/tools/dtest/util/seed/generator.go +++ b/src/cmd/tools/dtest/util/seed/generator.go @@ -71,7 +71,7 @@ func (g *generator) Generate(namespace ident.ID, shard uint32) error { g.logger.Debug("created fake data") writer := generate.NewWriter(gOpts) - err := writer.Write(namespace, shardSet, data) + err := writer.WriteData(namespace, shardSet, data) if err != nil { return fmt.Errorf("unable to write data: %v", err) } diff --git a/src/dbnode/generated-source-files.mk b/src/dbnode/generated-source-files.mk index bb9d0f4835..ab32a61b4e 100644 --- a/src/dbnode/generated-source-files.mk +++ b/src/dbnode/generated-source-files.mk @@ -34,7 +34,8 @@ genny-map-all: \ genny-map-storage \ genny-map-storage-namespace-metadata \ genny-map-storage-repair \ - genny-map-storage-index-results + genny-map-storage-index-results \ + genny-map-storage-bootstrap-bootstrapper-commitlog # Map generation rule for client/receivedBlocksMap .PHONY: genny-map-client-received-blocks @@ -127,6 +128,19 @@ genny-map-storage-repair: install-m3x-repo value_type=ReplicaSeriesBlocksMetadata \ target_package=$(m3db_package)/src/dbnode/storage/repair +# Map generation rule for storage/bootstrap/bootstrapper/commitlog +.PHONY: genny-map-storage-bootstrap-bootstrapper-commitlog +genny-map-storage-bootstrap-bootstrapper-commitlog: install-m3x-repo + cd $(m3x_package_path) && make idhashmap-gen \ + pkg=commitlog \ + value_type=metadataAndEncodersByTime \ + target_package=$(m3db_package)/src/dbnode/storage/bootstrap/bootstrapper/commitlog \ + rename_constructor=newMetadataAndEncodersByTimeMap \ + rename_constructor_options=newMetadataAndEncodersByTimeMapOptions + # Rename both generated map and constructor files + mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go + mv -f $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/new_map_gen.go $(m3db_package_path)/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go + # Map generation rule for storage/index/ResultsMap .PHONY: genny-map-storage-index-results genny-map-storage-index-results: install-m3x-repo diff --git a/src/dbnode/integration/admin_session_fetch_blocks_test.go b/src/dbnode/integration/admin_session_fetch_blocks_test.go index 0514b2b9d6..67085f09a5 100644 --- a/src/dbnode/integration/admin_session_fetch_blocks_test.go +++ b/src/dbnode/integration/admin_session_fetch_blocks_test.go @@ -167,6 +167,7 @@ func testSetupToSeriesMaps( nsMetadata namespace.Metadata, metadatasByShard map[uint32][]block.ReplicaMetadata, ) map[xtime.UnixNano]generate.SeriesBlock { + blockSize := nsMetadata.Options().RetentionOptions().BlockSize() seriesMap := make(map[xtime.UnixNano]generate.SeriesBlock) resultOpts := newDefaulTestResultOptions(testSetup.storageOpts) consistencyLevel := testSetup.storageOpts.RepairOptions().RepairConsistencyLevel() @@ -201,12 +202,12 @@ func testSetupToSeriesMaps( ctx.Close() firstTs := datapoints[0].Timestamp - seriesMapList := seriesMap[xtime.ToUnixNano(firstTs)] + seriesMapList := seriesMap[xtime.ToUnixNano(firstTs.Truncate(blockSize))] seriesMapList = append(seriesMapList, generate.Series{ ID: id, Data: datapoints, }) - seriesMap[xtime.ToUnixNano(firstTs)] = seriesMapList + seriesMap[xtime.ToUnixNano(firstTs.Truncate(blockSize))] = seriesMapList } require.NoError(t, blocksIter.Err()) } diff --git a/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go b/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go index 5e8b5a9870..67e9129295 100644 --- a/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go +++ b/src/dbnode/integration/bootstrap_after_buffer_rotation_regression_test.go @@ -100,7 +100,7 @@ func TestBootstrapAfterBufferRotation(t *testing.T) { }, }, } - writeCommitLogData(t, setup, commitLogOpts, seriesMaps, ns1.ID()) + writeCommitLogData(t, setup, commitLogOpts, seriesMaps, ns1, false) // Setup bootstrappers - We order them such that the custom test bootstrapper runs first // which does not bootstrap any data, but simply waits until it is signaled, allowing us @@ -182,7 +182,7 @@ func TestBootstrapAfterBufferRotation(t *testing.T) { // Verify in-memory data match what we expect - both commitlog and memory write // should be present. expectedSeriesMaps := map[xtime.UnixNano]generate.SeriesBlock{ - xtime.ToUnixNano(commitlogWrite.Timestamp): generate.SeriesBlock{ + xtime.ToUnixNano(commitlogWrite.Timestamp.Truncate(blockSize)): generate.SeriesBlock{ generate.Series{ ID: testID, Data: []ts.Datapoint{commitlogWrite, memoryWrite}, diff --git a/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go b/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go index df9b85371d..c2f25dce28 100644 --- a/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go +++ b/src/dbnode/integration/bootstrap_before_buffer_rotation_no_tick_regression_test.go @@ -113,7 +113,7 @@ func TestBootstrapBeforeBufferRotationNoTick(t *testing.T) { }, }, } - writeCommitLogData(t, setup, commitLogOpts, seriesMaps, ns1.ID()) + writeCommitLogData(t, setup, commitLogOpts, seriesMaps, ns1, false) // Setup bootstrappers - We order them such that the custom test bootstrapper runs first // which does not bootstrap any data, but simply waits until it is signaled, allowing us @@ -202,7 +202,7 @@ func TestBootstrapBeforeBufferRotationNoTick(t *testing.T) { // Verify in-memory data match what we expect - commitlog write should not be lost expectedSeriesMaps := map[xtime.UnixNano]generate.SeriesBlock{ - xtime.ToUnixNano(commitlogWrite.Timestamp): generate.SeriesBlock{ + xtime.ToUnixNano(commitlogWrite.Timestamp.Truncate(blockSize)): generate.SeriesBlock{ generate.Series{ ID: testID, Data: []ts.Datapoint{commitlogWrite}, diff --git a/src/dbnode/integration/commitlog_bootstrap_helpers.go b/src/dbnode/integration/commitlog_bootstrap_helpers.go index c30a87661e..83067918a8 100644 --- a/src/dbnode/integration/commitlog_bootstrap_helpers.go +++ b/src/dbnode/integration/commitlog_bootstrap_helpers.go @@ -30,6 +30,7 @@ import ( "github.com/m3db/m3db/src/dbnode/integration/generate" "github.com/m3db/m3db/src/dbnode/persist/fs" "github.com/m3db/m3db/src/dbnode/persist/fs/commitlog" + "github.com/m3db/m3db/src/dbnode/storage/namespace" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" xtime "github.com/m3db/m3x/time" @@ -89,7 +90,7 @@ func generateSeriesMaps(numBlocks int, starts ...time.Time) generate.SeriesBlock blockConfig = append(blockConfig, generate.BlockConfig{ IDs: name, NumPoints: rand.Intn(100) + 1, - Start: start.Add(time.Duration(i) * time.Minute), + Start: start, }) } return generate.BlocksByStart(blockConfig) @@ -100,9 +101,10 @@ func writeCommitLogData( s *testSetup, opts commitlog.Options, data generate.SeriesBlocksByStart, - namespace ident.ID, + namespace namespace.Metadata, + genSnapshots bool, ) { - writeCommitLogDataBase(t, s, opts, data, namespace, nil) + writeCommitLogDataBase(t, s, opts, data, namespace, nil, nil) } func writeCommitLogDataSpecifiedTS( @@ -110,10 +112,22 @@ func writeCommitLogDataSpecifiedTS( s *testSetup, opts commitlog.Options, data generate.SeriesBlocksByStart, - namespace ident.ID, + namespace namespace.Metadata, ts time.Time, + genSnapshots bool, ) { - writeCommitLogDataBase(t, s, opts, data, namespace, &ts) + writeCommitLogDataBase(t, s, opts, data, namespace, &ts, nil) +} + +func writeCommitLogDataWithPredicate( + t *testing.T, + s *testSetup, + opts commitlog.Options, + data generate.SeriesBlocksByStart, + namespace namespace.Metadata, + pred generate.WriteDatapointPredicate, +) { + writeCommitLogDataBase(t, s, opts, data, namespace, nil, pred) } func writeCommitLogDataBase( @@ -121,9 +135,14 @@ func writeCommitLogDataBase( s *testSetup, opts commitlog.Options, data generate.SeriesBlocksByStart, - namespace ident.ID, + namespace namespace.Metadata, specifiedTS *time.Time, + pred generate.WriteDatapointPredicate, ) { + if pred == nil { + pred = generate.WriteAllPredicate + } + // ensure commit log is flushing frequently require.Equal(t, defaultIntegrationTestFlushInterval, opts.FlushInterval()) @@ -132,6 +151,7 @@ func writeCommitLogDataBase( shardSet = s.shardSet ) + // Write out commit log data for ts, blk := range data { if specifiedTS != nil { s.setNowFn(*specifiedTS) @@ -160,13 +180,15 @@ func writeCommitLogDataBase( series, ok := seriesLookup[point.ID.String()] require.True(t, ok) cId := commitlog.Series{ - Namespace: namespace, + Namespace: namespace.ID(), Shard: shardSet.Lookup(point.ID), ID: point.ID, Tags: series.tags, UniqueIndex: series.uniqueIndex, } - require.NoError(t, commitLog.Write(ctx, cId, point.Datapoint, xtime.Second, nil)) + if pred(point.Datapoint) { + require.NoError(t, commitLog.Write(ctx, cId, point.Datapoint, xtime.Second, nil)) + } } // ensure writes finished @@ -174,6 +196,22 @@ func writeCommitLogDataBase( } } +func writeSnapshotsWithPredicate( + t *testing.T, + s *testSetup, + opts commitlog.Options, + data generate.SeriesBlocksByStart, + namespace namespace.Metadata, + specifiedTS *time.Time, + pred generate.WriteDatapointPredicate, + snapshotInterval time.Duration, +) { + // Write out snapshots + err := writeTestSnapshotsToDiskWithPredicate( + namespace, s, data, pred, snapshotInterval) + require.NoError(t, err) +} + func mustInspectFilesystem(fsOpts fs.Options) fs.Inspection { inspection, err := fs.InspectFilesystem(fsOpts) if err != nil { diff --git a/src/dbnode/integration/commitlog_bootstrap_index_test.go b/src/dbnode/integration/commitlog_bootstrap_index_test.go index 93d7189269..01fdebc667 100644 --- a/src/dbnode/integration/commitlog_bootstrap_index_test.go +++ b/src/dbnode/integration/commitlog_bootstrap_index_test.go @@ -132,8 +132,8 @@ func TestCommitLogIndexBootstrap(t *testing.T) { }) log.Info("writing data") - writeCommitLogData(t, setup, commitLogOpts, seriesMaps, testNamespaces[0]) - writeCommitLogData(t, setup, commitLogOpts, seriesMaps, testNamespaces[1]) + writeCommitLogData(t, setup, commitLogOpts, seriesMaps, ns1, false) + writeCommitLogData(t, setup, commitLogOpts, seriesMaps, ns2, false) log.Info("finished writing data") // Setup bootstrapper after writing data so filesystem inspection can find it. diff --git a/src/dbnode/integration/commitlog_bootstrap_merge_test.go b/src/dbnode/integration/commitlog_bootstrap_merge_test.go index e3ea7f9f97..ee8fa7f96d 100644 --- a/src/dbnode/integration/commitlog_bootstrap_merge_test.go +++ b/src/dbnode/integration/commitlog_bootstrap_merge_test.go @@ -118,7 +118,7 @@ func TestCommitLogAndFSMergeBootstrap(t *testing.T) { t1Nano: seriesMaps[t1Nano], t2Nano: seriesMaps[t2Nano], } - writeCommitLogData(t, setup, commitLogOpts, commitlogSeriesMaps, ns1.ID()) + writeCommitLogData(t, setup, commitLogOpts, commitlogSeriesMaps, ns1, false) // commit log bootstrapper (must be after writing out commitlog files so inspection finds files) noOpAll := bootstrapper.NewNoOpAllBootstrapperProvider() diff --git a/src/dbnode/integration/commitlog_bootstrap_multi_ns_test.go b/src/dbnode/integration/commitlog_bootstrap_multi_ns_test.go index 91a3e29f3d..fab9338c0c 100644 --- a/src/dbnode/integration/commitlog_bootstrap_multi_ns_test.go +++ b/src/dbnode/integration/commitlog_bootstrap_multi_ns_test.go @@ -83,7 +83,7 @@ func TestCommitLogBootstrapMultipleNamespaces(t *testing.T) { setup.namespaceMetadataOrFail(testNamespaces[0]) log.Info("writing data - ns1") - writeCommitLogData(t, setup, commitLogOpts, ns1SeriesMap, testNamespaces[0]) + writeCommitLogData(t, setup, commitLogOpts, ns1SeriesMap, ns1, false) log.Info("written data - ns1") // Write test data for ns2 @@ -96,7 +96,7 @@ func TestCommitLogBootstrapMultipleNamespaces(t *testing.T) { }) setup.namespaceMetadataOrFail(testNamespaces[1]) log.Info("writing data - ns2") - writeCommitLogData(t, setup, commitLogOpts, ns2SeriesMap, testNamespaces[1]) + writeCommitLogData(t, setup, commitLogOpts, ns2SeriesMap, ns2, false) log.Info("written data - ns2") // Setup bootstrapper after writing data so filesystem inspection can find it diff --git a/src/dbnode/integration/commitlog_bootstrap_only_reads_required_files_test.go b/src/dbnode/integration/commitlog_bootstrap_only_reads_required_files_test.go index 52768c3251..6977860707 100644 --- a/src/dbnode/integration/commitlog_bootstrap_only_reads_required_files_test.go +++ b/src/dbnode/integration/commitlog_bootstrap_only_reads_required_files_test.go @@ -68,11 +68,11 @@ func TestCommitLogBootstrapOnlyReadsRequiredFiles(t *testing.T) { now := setup.getNowFn() seriesMaps := generateSeriesMaps(30, now.Add(-2*blockSize), now.Add(-blockSize)) log.Info("writing data") - writeCommitLogData(t, setup, commitLogOpts, seriesMaps, testNamespaces[0]) + writeCommitLogData(t, setup, commitLogOpts, seriesMaps, ns1, false) log.Info("finished writing data") // The datapoints in this generated data are within the retention period and - // would oridinarly be bootstrapped, however, we intentionally write them to a + // would ordinarily be bootstrapped, however, we intentionally write them to a // commitlog file that has a timestamp outside of the retention period. This // allows us to verify the commitlog bootstrapping logic will not waste time // reading commitlog files that are outside of the retention period. @@ -84,8 +84,9 @@ func TestCommitLogBootstrapOnlyReadsRequiredFiles(t *testing.T) { setup, commitLogOpts, seriesMapsExpiredCommitlog, - testNamespaces[0], + ns1, now.Add(-2*ropts.RetentionPeriod()), + false, ) log.Info("finished writing data to commitlog file with out of range timestamp") diff --git a/src/dbnode/integration/commitlog_bootstrap_test.go b/src/dbnode/integration/commitlog_bootstrap_test.go index e13524df45..9c996fbcc1 100644 --- a/src/dbnode/integration/commitlog_bootstrap_test.go +++ b/src/dbnode/integration/commitlog_bootstrap_test.go @@ -71,7 +71,7 @@ func TestCommitLogBootstrap(t *testing.T) { now := setup.getNowFn() seriesMaps := generateSeriesMaps(30, now.Add(-2*blockSize), now.Add(-blockSize)) log.Info("writing data") - writeCommitLogData(t, setup, commitLogOpts, seriesMaps, testNamespaces[0]) + writeCommitLogData(t, setup, commitLogOpts, seriesMaps, ns1, false) log.Info("finished writing data") // Setup bootstrapper after writing data so filesystem inspection can find it. @@ -106,7 +106,7 @@ func TestCommitLogBootstrap(t *testing.T) { verifySeriesMapsEqual(t, seriesMaps, observedSeriesMaps) // Verify in-memory data match what we expect - no writes should be present - // because we didn't issue any writes for this namespaces + // because we didn't issue any writes for this namespace. emptySeriesMaps := make(generate.SeriesBlocksByStart) metadatasByShard2 := testSetupMetadatas(t, setup, testNamespaces[1], now.Add(-2*blockSize), now) observedSeriesMaps2 := testSetupToSeriesMaps(t, setup, ns2, metadatasByShard2) diff --git a/src/dbnode/integration/commitlog_bootstrap_with_snapshots_test.go b/src/dbnode/integration/commitlog_bootstrap_with_snapshots_test.go new file mode 100644 index 0000000000..4c37a5bf8a --- /dev/null +++ b/src/dbnode/integration/commitlog_bootstrap_with_snapshots_test.go @@ -0,0 +1,150 @@ +// +build integration + +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package integration + +import ( + "testing" + "time" + + "github.com/m3db/m3db/src/dbnode/integration/generate" + "github.com/m3db/m3db/src/dbnode/retention" + "github.com/m3db/m3db/src/dbnode/storage/bootstrap" + "github.com/m3db/m3db/src/dbnode/storage/bootstrap/bootstrapper" + bcl "github.com/m3db/m3db/src/dbnode/storage/bootstrap/bootstrapper/commitlog" + "github.com/m3db/m3db/src/dbnode/storage/namespace" + "github.com/m3db/m3db/src/dbnode/ts" + + "github.com/stretchr/testify/require" +) + +func TestCommitLogBootstrapWithSnapshots(t *testing.T) { + if testing.Short() { + t.SkipNow() // Just skip if we're doing a short run + } + + // Test setup + var ( + ropts = retention.NewOptions().SetRetentionPeriod(12 * time.Hour) + blockSize = ropts.BlockSize() + ) + ns1, err := namespace.NewMetadata(testNamespaces[0], namespace.NewOptions().SetRetentionOptions(ropts)) + require.NoError(t, err) + ns2, err := namespace.NewMetadata(testNamespaces[1], namespace.NewOptions().SetRetentionOptions(ropts)) + require.NoError(t, err) + opts := newTestOptions(t). + SetCommitLogRetentionPeriod(ropts.RetentionPeriod()). + SetCommitLogBlockSize(blockSize). + SetNamespaces([]namespace.Metadata{ns1, ns2}) + + setup, err := newTestSetup(t, opts, nil) + require.NoError(t, err) + defer setup.close() + + commitLogOpts := setup.storageOpts.CommitLogOptions(). + SetFlushInterval(defaultIntegrationTestFlushInterval) + setup.storageOpts = setup.storageOpts.SetCommitLogOptions(commitLogOpts) + + log := setup.storageOpts.InstrumentOptions().Logger() + log.Info("commit log bootstrap test") + + // Write test data + log.Info("generating data") + var ( + now = setup.getNowFn().Truncate(blockSize) + seriesMaps = generateSeriesMaps(30, now.Add(-2*blockSize), now.Add(-blockSize)) + ) + log.Info("writing data") + + var ( + snapshotInterval = 10 * time.Second + numDatapointsNotInSnapshots = 0 + pred = func(dp ts.Datapoint) bool { + blockStart := dp.Timestamp.Truncate(blockSize) + if dp.Timestamp.Before(blockStart.Add(snapshotInterval)) { + return true + } + + numDatapointsNotInSnapshots++ + return false + } + ) + + writeSnapshotsWithPredicate( + t, setup, commitLogOpts, seriesMaps, ns1, nil, pred, snapshotInterval) + + numDatapointsNotInCommitLogs := 0 + writeCommitLogDataWithPredicate(t, setup, commitLogOpts, seriesMaps, ns1, func(dp ts.Datapoint) bool { + blockStart := dp.Timestamp.Truncate(blockSize) + if dp.Timestamp.Equal(blockStart.Add(snapshotInterval)) || dp.Timestamp.After(blockStart.Add(snapshotInterval)) { + return true + } + + numDatapointsNotInCommitLogs++ + return false + }) + + // Make sure we actually excluded some datapoints from the snapshot and commitlog files + require.True(t, numDatapointsNotInSnapshots > 0) + require.True(t, numDatapointsNotInCommitLogs > 0) + + log.Info("finished writing data") + + // Setup bootstrapper after writing data so filesystem inspection can find it. + noOpAll := bootstrapper.NewNoOpAllBootstrapperProvider() + bsOpts := newDefaulTestResultOptions(setup.storageOpts) + bclOpts := bcl.NewOptions(). + SetResultOptions(bsOpts). + SetCommitLogOptions(commitLogOpts) + fsOpts := setup.storageOpts.CommitLogOptions().FilesystemOptions() + bs, err := bcl.NewCommitLogBootstrapperProvider( + bclOpts, mustInspectFilesystem(fsOpts), noOpAll) + require.NoError(t, err) + process := bootstrap.NewProcessProvider( + bs, bootstrap.NewProcessOptions(), bsOpts) + setup.storageOpts = setup.storageOpts.SetBootstrapProcessProvider(process) + + setup.setNowFn(now) + // Start the server with filesystem bootstrapper + require.NoError(t, setup.startServer()) + log.Debug("server is now up") + + // Stop the server + defer func() { + require.NoError(t, setup.stopServer()) + log.Debug("server is now down") + }() + + // Verify in-memory data match what we expect - all writes from seriesMaps + // should be present + metadatasByShard := testSetupMetadatas(t, setup, testNamespaces[0], now.Add(-2*blockSize), now) + observedSeriesMaps := testSetupToSeriesMaps(t, setup, ns1, metadatasByShard) + verifySeriesMapsEqual(t, seriesMaps, observedSeriesMaps) + + // Verify in-memory data match what we expect - no writes should be present + // because we didn't issue any writes for this namespaces + emptySeriesMaps := make(generate.SeriesBlocksByStart) + metadatasByShard2 := testSetupMetadatas(t, setup, testNamespaces[1], now.Add(-2*blockSize), now) + observedSeriesMaps2 := testSetupToSeriesMaps(t, setup, ns2, metadatasByShard2) + verifySeriesMapsEqual(t, emptySeriesMaps, observedSeriesMaps2) + +} diff --git a/src/dbnode/integration/mixed_mode_read_write_test.go b/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go similarity index 97% rename from src/dbnode/integration/mixed_mode_read_write_test.go rename to src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go index 99d9e25755..9988aab454 100644 --- a/src/dbnode/integration/mixed_mode_read_write_test.go +++ b/src/dbnode/integration/fs_commitlog_mixed_mode_read_write_test.go @@ -43,7 +43,11 @@ import ( "github.com/stretchr/testify/require" ) -func TestMixedModeReadWrite(t *testing.T) { +func TestFsCommitLogMixedModeReadWrite(t *testing.T) { + testMixedModeReadWrite(t, false) +} + +func testMixedModeReadWrite(t *testing.T, snapshotEnabled bool) { if testing.Short() { t.SkipNow() // Just skip if we're doing a short run } @@ -55,7 +59,11 @@ func TestMixedModeReadWrite(t *testing.T) { ns1ROpts = retention.NewOptions().SetRetentionPeriod(3 * time.Hour).SetBlockSize(ns1BlockSize) nsID = testNamespaces[0] ) - ns1, err := namespace.NewMetadata(nsID, namespace.NewOptions().SetRetentionOptions(ns1ROpts)) + + ns1Opts := namespace.NewOptions(). + SetRetentionOptions(ns1ROpts). + SetSnapshotEnabled(snapshotEnabled) + ns1, err := namespace.NewMetadata(nsID, ns1Opts) require.NoError(t, err) opts := newTestOptions(t). SetCommitLogRetentionPeriod(commitLogRetetion). diff --git a/src/dbnode/integration/fs_commitlog_snapshots_mixed_mode_read_write_test.go b/src/dbnode/integration/fs_commitlog_snapshots_mixed_mode_read_write_test.go new file mode 100644 index 0000000000..eda20644dd --- /dev/null +++ b/src/dbnode/integration/fs_commitlog_snapshots_mixed_mode_read_write_test.go @@ -0,0 +1,29 @@ +// +build integration + +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package integration + +import "testing" + +func TestFsCommitLogSnapshotsMixedModeReadWrite(t *testing.T) { + testMixedModeReadWrite(t, true) +} diff --git a/src/dbnode/integration/generate/options.go b/src/dbnode/integration/generate/options.go index 2950e466f1..7f6f270ed9 100644 --- a/src/dbnode/integration/generate/options.go +++ b/src/dbnode/integration/generate/options.go @@ -47,6 +47,10 @@ const ( // defaultWriteEmptyShards is the writeEmptyShards value by default defaultWriteEmptyShards = true + + // defaultWriteSnapshot determines whether the writer writes snapshots instead + // of data files. + defaultWriteSnapshot = false ) var ( @@ -62,6 +66,7 @@ type options struct { newDirectoryMode os.FileMode writerBufferSize int writeEmptyShards bool + writeSnapshot bool encoderPool encoding.EncoderPool } @@ -82,6 +87,7 @@ func NewOptions() Options { newDirectoryMode: defaultNewDirectoryMode, writerBufferSize: defaultWriterBufferSize, writeEmptyShards: defaultWriteEmptyShards, + writeSnapshot: defaultWriteSnapshot, encoderPool: encoderPool, } } @@ -166,6 +172,16 @@ func (o *options) WriteEmptyShards() bool { return o.writeEmptyShards } +func (o *options) SetWriteSnapshot(value bool) Options { + opts := *o + opts.writeSnapshot = value + return &opts +} + +func (o *options) WriteSnapshot() bool { + return o.writeSnapshot +} + func (o *options) SetEncoderPool(value encoding.EncoderPool) Options { opts := *o opts.encoderPool = value diff --git a/src/dbnode/integration/generate/types.go b/src/dbnode/integration/generate/types.go index 55d050f58d..9827311777 100644 --- a/src/dbnode/integration/generate/types.go +++ b/src/dbnode/integration/generate/types.go @@ -64,8 +64,34 @@ type SeriesBlocksByStart map[xtime.UnixNano]SeriesBlock // Writer writes generated data to disk type Writer interface { - // Write writes the data - Write(ns ident.ID, shards sharding.ShardSet, data SeriesBlocksByStart) error + // WriteData writes the data as data files. + WriteData( + ns ident.ID, shards sharding.ShardSet, data SeriesBlocksByStart) error + + // WriteSnapshot writes the data as snapshot files. + WriteSnapshot( + ns ident.ID, + shards sharding.ShardSet, + data SeriesBlocksByStart, + snapshotInterval time.Duration, + ) error + + // WriteDataWithPredicate writes all data that passes the predicate test as data files. + WriteDataWithPredicate( + ns ident.ID, + shards sharding.ShardSet, + data SeriesBlocksByStart, + pred WriteDatapointPredicate, + ) error + + // WriteSnapshotWithPredicate writes all data that passes the predicate test as snapshot files. + WriteSnapshotWithPredicate( + ns ident.ID, + shards sharding.ShardSet, + data SeriesBlocksByStart, + pred WriteDatapointPredicate, + snapshotInterval time.Duration, + ) error } // Options represent the parameters needed for the Writer @@ -118,9 +144,18 @@ type Options interface { // WriteEmptyShards returns whether writes are done even for empty start periods WriteEmptyShards() bool + // SetWriteSnapshot sets whether writes are written as snapshot files + SetWriteSnapshot(bool) Options + + // WriteSnapshots returns whether writes are written as snapshot files + WriteSnapshot() bool + // SetEncoderPool sets the contextPool SetEncoderPool(value encoding.EncoderPool) Options // EncoderPool returns the contextPool EncoderPool() encoding.EncoderPool } + +// WriteDatapointPredicate returns a boolean indicating whether a datapoint should be written. +type WriteDatapointPredicate func(dp ts.Datapoint) bool diff --git a/src/dbnode/integration/generate/writer.go b/src/dbnode/integration/generate/writer.go index 4ffeccc64d..64e3cced26 100644 --- a/src/dbnode/integration/generate/writer.go +++ b/src/dbnode/integration/generate/writer.go @@ -21,13 +21,14 @@ package generate import ( - "fmt" "time" "github.com/m3db/m3db/src/dbnode/digest" "github.com/m3db/m3db/src/dbnode/encoding" + "github.com/m3db/m3db/src/dbnode/persist" "github.com/m3db/m3db/src/dbnode/persist/fs" "github.com/m3db/m3db/src/dbnode/sharding" + "github.com/m3db/m3db/src/dbnode/ts" "github.com/m3db/m3x/checked" "github.com/m3db/m3x/ident" xtime "github.com/m3db/m3x/time" @@ -37,6 +38,11 @@ type writer struct { opts Options } +// WriteAllPredicate writes all datapoints +func WriteAllPredicate(_ ts.Datapoint) bool { + return true +} + // NewWriter returns a new writer func NewWriter(opts Options) Writer { return &writer{ @@ -44,7 +50,53 @@ func NewWriter(opts Options) Writer { } } -func (w *writer) Write(namespace ident.ID, shardSet sharding.ShardSet, seriesMaps SeriesBlocksByStart) error { +func (w *writer) WriteData( + namespace ident.ID, + shardSet sharding.ShardSet, + seriesMaps SeriesBlocksByStart, +) error { + return w.WriteDataWithPredicate(namespace, shardSet, seriesMaps, WriteAllPredicate) +} + +func (w *writer) WriteSnapshot( + namespace ident.ID, + shardSet sharding.ShardSet, + seriesMaps SeriesBlocksByStart, + snapshotInterval time.Duration, +) error { + return w.WriteSnapshotWithPredicate( + namespace, shardSet, seriesMaps, WriteAllPredicate, snapshotInterval) +} + +func (w *writer) WriteDataWithPredicate( + namespace ident.ID, + shardSet sharding.ShardSet, + seriesMaps SeriesBlocksByStart, + pred WriteDatapointPredicate, +) error { + return w.writeWithPredicate( + namespace, shardSet, seriesMaps, pred, persist.FileSetFlushType, 0) +} + +func (w *writer) WriteSnapshotWithPredicate( + namespace ident.ID, + shardSet sharding.ShardSet, + seriesMaps SeriesBlocksByStart, + pred WriteDatapointPredicate, + snapshotInterval time.Duration, +) error { + return w.writeWithPredicate( + namespace, shardSet, seriesMaps, pred, persist.FileSetSnapshotType, snapshotInterval) +} + +func (w *writer) writeWithPredicate( + namespace ident.ID, + shardSet sharding.ShardSet, + seriesMaps SeriesBlocksByStart, + pred WriteDatapointPredicate, + fileSetType persist.FileSetType, + snapshotInterval time.Duration, +) error { var ( gOpts = w.opts blockSize = gOpts.BlockSize() @@ -70,7 +122,9 @@ func (w *writer) Write(namespace ident.ID, shardSet sharding.ShardSet, seriesMap } encoder := gOpts.EncoderPool().Get() for start, data := range seriesMaps { - err := writeToDisk(writer, shardSet, encoder, start.ToTime(), namespace, blockSize, data) + err := writeToDiskWithPredicate( + writer, shardSet, encoder, start.ToTime(), namespace, blockSize, + data, pred, fileSetType, snapshotInterval) if err != nil { return err } @@ -80,7 +134,9 @@ func (w *writer) Write(namespace ident.ID, shardSet sharding.ShardSet, seriesMap // Write remaining files even for empty start periods to avoid unfulfilled ranges if w.opts.WriteEmptyShards() { for start := range starts { - err := writeToDisk(writer, shardSet, encoder, start.ToTime(), namespace, blockSize, nil) + err := writeToDiskWithPredicate( + writer, shardSet, encoder, start.ToTime(), namespace, blockSize, + nil, pred, fileSetType, snapshotInterval) if err != nil { return err } @@ -90,7 +146,7 @@ func (w *writer) Write(namespace ident.ID, shardSet sharding.ShardSet, seriesMap return nil } -func writeToDisk( +func writeToDiskWithPredicate( writer fs.DataFileSetWriter, shardSet sharding.ShardSet, encoder encoding.Encoder, @@ -98,6 +154,9 @@ func writeToDisk( namespace ident.ID, blockSize time.Duration, seriesList SeriesBlock, + pred WriteDatapointPredicate, + fileSetType persist.FileSetType, + snapshotInterval time.Duration, ) error { seriesPerShard := make(map[uint32][]Series) for _, shard := range shardSet.AllIDs() { @@ -117,6 +176,10 @@ func writeToDisk( Shard: shard, BlockStart: start, }, + FileSetType: fileSetType, + Snapshot: fs.DataWriterSnapshotOptions{ + SnapshotTime: start.Add(snapshotInterval), + }, } if err := writer.Open(writerOpts); err != nil { return err @@ -124,13 +187,19 @@ func writeToDisk( for _, series := range seriesList { encoder.Reset(start, 0) for _, dp := range series.Data { + if !pred(dp) { + continue + } + if err := encoder.Encode(dp, xtime.Second, nil); err != nil { return err } } + stream := encoder.Stream() if stream == nil { - return fmt.Errorf("nil stream") + // None of the datapoints passed the predicate. + continue } segment, err := stream.Segment() if err != nil { diff --git a/src/dbnode/integration/integration.go b/src/dbnode/integration/integration.go index 7281dd3789..8a32c75fe6 100644 --- a/src/dbnode/integration/integration.go +++ b/src/dbnode/integration/integration.go @@ -45,7 +45,6 @@ import ( "github.com/m3db/m3x/instrument" xlog "github.com/m3db/m3x/log" xretry "github.com/m3db/m3x/retry" - xtime "github.com/m3db/m3x/time" "github.com/stretchr/testify/require" "github.com/uber-go/tally" @@ -299,11 +298,24 @@ func newDefaultBootstrappableTestSetups( func writeTestDataToDisk( metadata namespace.Metadata, setup *testSetup, - seriesMaps map[xtime.UnixNano]generate.SeriesBlock, + seriesMaps generate.SeriesBlocksByStart, ) error { ropts := metadata.Options().RetentionOptions() writer := generate.NewWriter(setup.generatorOptions(ropts)) - return writer.Write(metadata.ID(), setup.shardSet, seriesMaps) + return writer.WriteData(metadata.ID(), setup.shardSet, seriesMaps) +} + +func writeTestSnapshotsToDiskWithPredicate( + metadata namespace.Metadata, + setup *testSetup, + seriesMaps generate.SeriesBlocksByStart, + pred generate.WriteDatapointPredicate, + snapshotInterval time.Duration, +) error { + ropts := metadata.Options().RetentionOptions() + writer := generate.NewWriter(setup.generatorOptions(ropts)) + return writer.WriteSnapshotWithPredicate( + metadata.ID(), setup.shardSet, seriesMaps, pred, snapshotInterval) } func concatShards(a, b shard.Shards) shard.Shards { diff --git a/src/dbnode/persist/fs/commitlog/reader.go b/src/dbnode/persist/fs/commitlog/reader.go index 839257986b..c602a069a2 100644 --- a/src/dbnode/persist/fs/commitlog/reader.go +++ b/src/dbnode/persist/fs/commitlog/reader.go @@ -248,6 +248,7 @@ func (r *reader) readLoop() { if err == io.EOF { return } + r.decoderQueues[0] <- decoderArg{ bytes: data, err: err, diff --git a/src/dbnode/persist/fs/files.go b/src/dbnode/persist/fs/files.go index 06213390c1..e1ecb9daf4 100644 --- a/src/dbnode/persist/fs/files.go +++ b/src/dbnode/persist/fs/files.go @@ -21,6 +21,7 @@ package fs import ( + "bufio" "fmt" "os" "path" @@ -53,12 +54,42 @@ const ( indexFileSetComponentPosition = 2 ) +var ( + defaultBufioReaderSize = bufio.NewReader(nil).Size() +) + type fileOpener func(filePath string) (*os.File, error) // FileSetFile represents a set of FileSet files for a given block start type FileSetFile struct { ID FileSetFileIdentifier AbsoluteFilepaths []string + + CachedSnapshotTime time.Time + filePathPrefix string +} + +// SnapshotTime returns the SnapshotTime for the given FileSetFile. Value is meaningless +// if the the FileSetFile is a flush instead of a snapshot. +func (f *FileSetFile) SnapshotTime() (time.Time, error) { + if !f.CachedSnapshotTime.IsZero() { + // Return immediately if we've already cached it. + return f.CachedSnapshotTime, nil + } + + snapshotTime, err := SnapshotTime(f.filePathPrefix, f.ID) + if err != nil { + return time.Time{}, err + } + + // Cache for future use and return. + f.CachedSnapshotTime = snapshotTime + return f.CachedSnapshotTime, nil +} + +// IsZero returns whether the FileSetFile is a zero value. +func (f FileSetFile) IsZero() bool { + return len(f.AbsoluteFilepaths) == 0 } // HasCheckpointFile returns a bool indicating whether the given set of @@ -88,25 +119,33 @@ func (f FileSetFilesSlice) Filepaths() []string { } // LatestVolumeForBlock returns the latest (highest index) FileSetFile in the -// slice for a given block start, only applicable for index file set files. +// slice for a given block start, only applicable for index and snapshot file set files. func (f FileSetFilesSlice) LatestVolumeForBlock(blockStart time.Time) (FileSetFile, bool) { // Make sure we're already sorted f.sortByTimeAndVolumeIndexAscending() for i, curr := range f { if curr.ID.BlockStart.Equal(blockStart) { - isEnd := i == len(f)-1 - isHighestIdx := true - if !isEnd { - next := f[i+1] - if next.ID.BlockStart.Equal(blockStart) && next.ID.VolumeIndex > curr.ID.VolumeIndex { - isHighestIdx = false + var ( + bestSoFar FileSetFile + bestSoFarExists bool + ) + + for j := i; j < len(f); j++ { + curr = f[j] + + if !curr.ID.BlockStart.Equal(blockStart) { + break + } + + if curr.HasCheckpointFile() && curr.ID.VolumeIndex >= bestSoFar.ID.VolumeIndex { + bestSoFar = curr + bestSoFarExists = true } - } - if isEnd || isHighestIdx { - return curr, true } + + return bestSoFar, bestSoFarExists } } @@ -132,10 +171,11 @@ func (f FileSetFilesSlice) sortByTimeAndVolumeIndexAscending() { } // NewFileSetFile creates a new FileSet file -func NewFileSetFile(id FileSetFileIdentifier) FileSetFile { +func NewFileSetFile(id FileSetFileIdentifier, filePathPrefix string) FileSetFile { return FileSetFile{ ID: id, AbsoluteFilepaths: []string{}, + filePathPrefix: filePathPrefix, } } @@ -292,6 +332,63 @@ func timeAndIndexFromFileName(fname string, componentPosition int) (time.Time, i return t, int(index), nil } +// SnapshotTime returns the time at which the snapshot was taken. +func SnapshotTime( + filePathPrefix string, id FileSetFileIdentifier) (time.Time, error) { + decoder := msgpack.NewDecoder(nil) + return snapshotTime(filePathPrefix, id, decoder) +} + +func snapshotTime( + filePathPrefix string, id FileSetFileIdentifier, decoder *msgpack.Decoder) (time.Time, error) { + infoBytes, err := readSnapshotInfoFile(filePathPrefix, id, defaultBufioReaderSize) + if err != nil { + return time.Time{}, err + } + + decoder.Reset(msgpack.NewDecoderStream(infoBytes)) + info, err := decoder.DecodeIndexInfo() + return time.Unix(0, info.SnapshotTime), err +} + +func readSnapshotInfoFile(filePathPrefix string, id FileSetFileIdentifier, readerBufferSize int) ([]byte, error) { + var ( + shardDir = ShardSnapshotsDirPath(filePathPrefix, id.Namespace, id.Shard) + checkpointFilePath = filesetPathFromTimeAndIndex(shardDir, id.BlockStart, id.VolumeIndex, checkpointFileSuffix) + + digestFilePath = filesetPathFromTimeAndIndex(shardDir, id.BlockStart, id.VolumeIndex, digestFileSuffix) + infoFilePath = filesetPathFromTimeAndIndex(shardDir, id.BlockStart, id.VolumeIndex, infoFileSuffix) + ) + + checkpointFd, err := os.Open(checkpointFilePath) + if err != nil { + return nil, err + } + + // Read digest of digests from the checkpoint file + digestBuf := digest.NewBuffer() + expectedDigestOfDigest, err := digestBuf.ReadDigestFromFile(checkpointFd) + closeErr := checkpointFd.Close() + if err != nil { + return nil, err + } + if closeErr != nil { + return nil, closeErr + } + + // Read and validate the digest file + digestData, err := readAndValidate( + digestFilePath, readerBufferSize, expectedDigestOfDigest) + if err != nil { + return nil, err + } + + // Read and validate the info file + expectedInfoDigest := digest.ToBuffer(digestData).ReadDigest() + return readAndValidate( + infoFilePath, readerBufferSize, expectedInfoDigest) +} + type forEachInfoFileSelector struct { fileSetType persist.FileSetType contentType persist.FileSetContentType @@ -821,7 +918,7 @@ func filesetFiles(args filesetFilesSelector) (FileSetFilesSlice, error) { BlockStart: currentFileBlockStart, Shard: args.shard, VolumeIndex: volumeIndex, - }) + }, args.filePathPrefix) } else if !currentFileBlockStart.Equal(latestBlockStart) || latestVolumeIndex != volumeIndex { filesetFiles = append(filesetFiles, latestFileSetFile) latestFileSetFile = NewFileSetFile(FileSetFileIdentifier{ @@ -829,7 +926,7 @@ func filesetFiles(args filesetFilesSelector) (FileSetFilesSlice, error) { BlockStart: currentFileBlockStart, Shard: args.shard, VolumeIndex: volumeIndex, - }) + }, args.filePathPrefix) } latestBlockStart = currentFileBlockStart latestVolumeIndex = volumeIndex diff --git a/src/dbnode/persist/fs/files_test.go b/src/dbnode/persist/fs/files_test.go index 2fb24926d5..badb787cd6 100644 --- a/src/dbnode/persist/fs/files_test.go +++ b/src/dbnode/persist/fs/files_test.go @@ -26,6 +26,7 @@ import ( "io/ioutil" "os" "path" + "path/filepath" "sort" "strconv" "strings" @@ -430,18 +431,24 @@ func TestFileSetFilesNoFiles(t *testing.T) { } func TestSnapshotFiles(t *testing.T) { - shard := uint32(0) - dir := createDataSnapshotInfoFilesDir(t, testNs1ID, shard, 20) + var ( + shard = uint32(0) + dir = createTempDir(t) + filePathPrefix = filepath.Join(dir, "") + ) defer os.RemoveAll(dir) - files, err := SnapshotFiles(dir, testNs1ID, shard) - require.NoError(t, err) - require.Equal(t, 20, len(files)) - for i, snapshotFile := range files { - require.Equal(t, int64(i), snapshotFile.ID.BlockStart.UnixNano()) - } + // Write out snapshot file + writeOutTestSnapshot(t, filePathPrefix, shard, testWriterStart, 0) - require.Equal(t, 20, len(files.Filepaths())) + // Load snapshot files + snapshotFiles, err := SnapshotFiles(filePathPrefix, testNs1ID, shard) + require.NoError(t, err) + require.Equal(t, 1, len(snapshotFiles)) + snapshotTime, err := snapshotFiles[0].SnapshotTime() + require.NoError(t, err) + require.True(t, testWriterStart.Equal(snapshotTime)) + require.False(t, testWriterStart.IsZero()) } func TestSnapshotFilesNoFiles(t *testing.T) { @@ -463,15 +470,15 @@ func TestSnapshotFilesNoFiles(t *testing.T) { } func TestNextSnapshotFileSetVolumeIndex(t *testing.T) { - // Make empty directory - shard := uint32(0) - dir := createTempDir(t) - shardDir := ShardSnapshotsDirPath(dir, testNs1ID, shard) + var ( + shard = uint32(0) + dir = createTempDir(t) + shardDir = ShardSnapshotsDirPath(dir, testNs1ID, shard) + blockStart = time.Now().Truncate(time.Hour) + ) require.NoError(t, os.MkdirAll(shardDir, 0755)) defer os.RemoveAll(shardDir) - blockStart := time.Now().Truncate(time.Hour) - // Check increments properly curr := -1 for i := 0; i <= 10; i++ { @@ -481,9 +488,7 @@ func TestNextSnapshotFileSetVolumeIndex(t *testing.T) { require.Equal(t, curr+1, index) curr = index - p := filesetPathFromTimeAndIndex(shardDir, blockStart, index, "foo") - err = ioutil.WriteFile(p, []byte("bar"), defaultNewFileMode) - require.NoError(t, err) + writeOutTestSnapshot(t, dir, shard, blockStart, index) } } @@ -504,7 +509,7 @@ func TestNextIndexFileSetVolumeIndex(t *testing.T) { require.Equal(t, curr+1, index) curr = index - p := filesetPathFromTimeAndIndex(dataDir, blockStart, index, "foo") + p := filesetPathFromTimeAndIndex(dataDir, blockStart, index, checkpointFileSuffix) err = ioutil.WriteFile(p, []byte("bar"), defaultNewFileMode) require.NoError(t, err) } @@ -522,19 +527,19 @@ func TestMultipleForBlockStart(t *testing.T) { // Write out many files with the same blockStart, but different indices ts := time.Unix(0, 0) for i := 0; i < numSnapshots; i++ { + volume := i % numSnapshotsPerBlock // Periodically update the blockStart - if i%numSnapshotsPerBlock == 0 { + if volume == 0 { ts = time.Unix(0, int64(i)) } - filePath := filesetPathFromTimeAndIndex(shardDir, ts, - i%numSnapshotsPerBlock, infoFileSuffix) - createFile(t, filePath, nil) + + writeOutTestSnapshot(t, dir, shard, ts, volume) } files, err := SnapshotFiles(dir, testNs1ID, shard) require.NoError(t, err) require.Equal(t, 20, len(files)) - require.Equal(t, 20, len(files.Filepaths())) + require.Equal(t, 20*7, len(files.Filepaths())) // Make sure LatestForBlock works even if the input list is not sorted properly for i := range files { @@ -575,14 +580,15 @@ func TestShardSnapshotsDirPath(t *testing.T) { } func TestSnapshotFileSetExistsAt(t *testing.T) { - shard := uint32(0) - ts := time.Unix(0, 0) - dir := createTempDir(t) - shardPath := ShardSnapshotsDirPath(dir, testNs1ID, 0) + var ( + shard = uint32(0) + ts = time.Unix(0, 0) + dir = createTempDir(t) + shardPath = ShardSnapshotsDirPath(dir, testNs1ID, 0) + ) require.NoError(t, os.MkdirAll(shardPath, 0755)) - filePath := filesetPathFromTimeAndIndex(shardPath, ts, 0, checkpointFileSuffix) - createFile(t, filePath, []byte{}) + writeOutTestSnapshot(t, dir, shard, ts, 0) exists, err := SnapshotFileSetExistsAt(dir, testNs1ID, shard, ts) require.NoError(t, err) @@ -821,6 +827,27 @@ func TestIndexFileSetsBefore(t *testing.T) { } } +func TestSnapshotFileSnapshotTime(t *testing.T) { + var ( + dir = createTempDir(t) + filePathPrefix = filepath.Join(dir, "") + ) + defer os.RemoveAll(dir) + + // Write out snapshot file + writeOutTestSnapshot(t, filePathPrefix, 0, testWriterStart, 0) + + // Load snapshot files + snapshotFiles, err := SnapshotFiles(filePathPrefix, testNs1ID, 0) + require.NoError(t, err) + require.Equal(t, 1, len(snapshotFiles)) + + // Verify SnapshotTime() returns the expected time + snapshotTime, err := SnapshotTime(filePathPrefix, snapshotFiles[0].ID) + require.NoError(t, err) + require.Equal(t, true, testWriterStart.Equal(snapshotTime)) +} + func createTempFile(t *testing.T) *os.File { fd, err := ioutil.TempFile("", "testfile") require.NoError(t, err) @@ -848,10 +875,6 @@ func createDataFlushInfoFilesDir(t *testing.T, namespace ident.ID, shard uint32, return createDataInfoFiles(t, dataDirName, namespace, shard, iter, false) } -func createDataSnapshotInfoFilesDir(t *testing.T, namespace ident.ID, shard uint32, iter int) string { - return createDataInfoFiles(t, snapshotDirName, namespace, shard, iter, true) -} - func createDataCheckpointFilesDir(t *testing.T, namespace ident.ID, shard uint32, iter int) string { return createDataCheckpointFiles(t, dataDirName, namespace, shard, iter, false) } @@ -969,3 +992,22 @@ func validateCommitLogFiles(t *testing.T, slot, index, perSlot, resIdx int, dir x := (resIdx * perSlot) + index require.Equal(t, path.Join(dir, commitLogsDirName, fileName), files[x]) } + +func writeOutTestSnapshot( + t *testing.T, filePathPrefix string, + shard uint32, blockStart time.Time, volume int) { + var ( + entries = []testEntry{ + {"foo", nil, []byte{1, 2, 3}}, + {"bar", nil, []byte{4, 5, 6}}, + {"baz", nil, make([]byte, 65536)}, + {"cat", nil, make([]byte, 100000)}, + {"echo", nil, []byte{7, 8, 9}}, + } + w = newTestWriter(t, filePathPrefix) + ) + defer w.Close() + + writeTestDataWithVolume( + t, w, shard, blockStart, volume, entries, persist.FileSetSnapshotType) +} diff --git a/src/dbnode/persist/fs/read_write_test.go b/src/dbnode/persist/fs/read_write_test.go index c0a2093eb4..edc19dcca3 100644 --- a/src/dbnode/persist/fs/read_write_test.go +++ b/src/dbnode/persist/fs/read_write_test.go @@ -31,6 +31,7 @@ import ( "github.com/m3db/bloom" "github.com/m3db/m3db/src/dbnode/digest" + "github.com/m3db/m3db/src/dbnode/persist" "github.com/m3db/m3x/checked" "github.com/m3db/m3x/ident" xtime "github.com/m3db/m3x/time" @@ -77,15 +78,42 @@ func newTestWriter(t *testing.T, filePathPrefix string) DataFileSetWriter { return writer } -func writeTestData(t *testing.T, w DataFileSetWriter, shard uint32, timestamp time.Time, entries []testEntry) { +func writeTestData( + t *testing.T, + w DataFileSetWriter, + shard uint32, + timestamp time.Time, + entries []testEntry, + fileSetType persist.FileSetType, +) { + writeTestDataWithVolume( + t, w, shard, timestamp, 0, entries, fileSetType) +} + +func writeTestDataWithVolume( + t *testing.T, + w DataFileSetWriter, + shard uint32, + timestamp time.Time, + volume int, + entries []testEntry, + fileSetType persist.FileSetType, +) { writerOpts := DataWriterOpenOptions{ Identifier: FileSetFileIdentifier{ - Namespace: testNs1ID, - Shard: shard, - BlockStart: timestamp, + Namespace: testNs1ID, + Shard: shard, + BlockStart: timestamp, + VolumeIndex: volume, }, - BlockSize: testBlockSize, + BlockSize: testBlockSize, + FileSetType: fileSetType, + } + + if fileSetType == persist.FileSetSnapshotType { + writerOpts.Snapshot.SnapshotTime = timestamp } + err := w.Open(writerOpts) assert.NoError(t, err) @@ -219,7 +247,7 @@ func TestSimpleReadWrite(t *testing.T) { } w := newTestWriter(t, filePathPrefix) - writeTestData(t, w, 0, testWriterStart, entries) + writeTestData(t, w, 0, testWriterStart, entries, persist.FileSetFlushType) r := newTestReader(t, filePathPrefix) readTestData(t, r, 0, testWriterStart, entries) @@ -274,7 +302,7 @@ func TestReadWithReusedReader(t *testing.T) { } w := newTestWriter(t, filePathPrefix) - writeTestData(t, w, 0, testWriterStart, entries) + writeTestData(t, w, 0, testWriterStart, entries, persist.FileSetFlushType) r := newTestReader(t, filePathPrefix) readTestData(t, r, 0, testWriterStart, entries) @@ -299,7 +327,7 @@ func TestInfoReadWrite(t *testing.T) { } w := newTestWriter(t, filePathPrefix) - writeTestData(t, w, 0, testWriterStart, entries) + writeTestData(t, w, 0, testWriterStart, entries, persist.FileSetFlushType) readInfoFileResults := ReadInfoFiles(filePathPrefix, testNs1ID, 0, 16, nil) require.Equal(t, 1, len(readInfoFileResults)) @@ -330,7 +358,7 @@ func TestReusingReaderWriter(t *testing.T) { } w := newTestWriter(t, filePathPrefix) for i := range allEntries { - writeTestData(t, w, 0, testWriterStart.Add(time.Duration(i)*time.Hour), allEntries[i]) + writeTestData(t, w, 0, testWriterStart.Add(time.Duration(i)*time.Hour), allEntries[i], persist.FileSetFlushType) } r := newTestReader(t, filePathPrefix) @@ -385,7 +413,7 @@ func TestReusingWriterAfterWriteError(t *testing.T) { require.Equal(t, ErrCheckpointFileNotFound, r.Open(rOpenOpts)) // Now reuse the writer and validate the data are written as expected. - writeTestData(t, w, shard, testWriterStart, entries) + writeTestData(t, w, shard, testWriterStart, entries, persist.FileSetFlushType) readTestData(t, r, shard, testWriterStart, entries) } diff --git a/src/dbnode/persist/fs/reader_open_options_matcher.go b/src/dbnode/persist/fs/reader_open_options_matcher.go index c4623ab13a..a728ee61a5 100644 --- a/src/dbnode/persist/fs/reader_open_options_matcher.go +++ b/src/dbnode/persist/fs/reader_open_options_matcher.go @@ -48,6 +48,9 @@ func (m ReaderOpenOptionsMatcher) Matches(x interface{}) bool { if !m.ID.BlockStart.Equal(readerOpenOptions.Identifier.BlockStart) { return false } + if m.ID.VolumeIndex != readerOpenOptions.Identifier.VolumeIndex { + return false + } if m.FileSetType != readerOpenOptions.FileSetType { return false } @@ -57,7 +60,7 @@ func (m ReaderOpenOptionsMatcher) Matches(x interface{}) bool { func (m ReaderOpenOptionsMatcher) String() string { return fmt.Sprintf( - "namespace: %s, shard: %d, blockstart: %d, filesetType: %s", - m.ID.Namespace.String(), m.ID.Shard, m.ID.BlockStart.Unix(), m.FileSetType, + "namespace: %s, shard: %d, blockstart: %d, volumeIndex: %d, filesetType: %s", + m.ID.Namespace.String(), m.ID.Shard, m.ID.BlockStart.Unix(), m.ID.VolumeIndex, m.FileSetType, ) } diff --git a/src/dbnode/storage/block/block.go b/src/dbnode/storage/block/block.go index b9bc71e902..2308a94b95 100644 --- a/src/dbnode/storage/block/block.go +++ b/src/dbnode/storage/block/block.go @@ -413,21 +413,33 @@ func (b *dbBlock) resetRetrievableWithLock( b.wasRetrievedFromDisk = false } +func (b *dbBlock) Discard() ts.Segment { + return b.closeAndDiscard() +} + func (b *dbBlock) Close() { - b.Lock() - defer b.Unlock() + segment := b.closeAndDiscard() + segment.Finalize() +} +func (b *dbBlock) closeAndDiscard() ts.Segment { + b.Lock() if b.closed { - return + b.Unlock() + return ts.Segment{} } + segment := b.segment b.closed = true - b.segment.Finalize() b.resetMergeTargetWithLock() + b.Unlock() + if pool := b.opts.DatabaseBlockPool(); pool != nil { pool.Put(b) } + + return segment } func (b *dbBlock) resetMergeTargetWithLock() { diff --git a/src/dbnode/storage/block/block_mock.go b/src/dbnode/storage/block/block_mock.go index 2dd642679f..c57edfb671 100644 --- a/src/dbnode/storage/block/block_mock.go +++ b/src/dbnode/storage/block/block_mock.go @@ -439,6 +439,18 @@ func (mr *MockDatabaseBlockMockRecorder) ResetRetrievable(startTime, blockSize, return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetRetrievable", reflect.TypeOf((*MockDatabaseBlock)(nil).ResetRetrievable), startTime, blockSize, retriever, metadata) } +// Discard mocks base method +func (m *MockDatabaseBlock) Discard() ts.Segment { + ret := m.ctrl.Call(m, "Discard") + ret0, _ := ret[0].(ts.Segment) + return ret0 +} + +// Discard indicates an expected call of Discard +func (mr *MockDatabaseBlockMockRecorder) Discard() *gomock.Call { + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Discard", reflect.TypeOf((*MockDatabaseBlock)(nil).Discard)) +} + // Close mocks base method func (m *MockDatabaseBlock) Close() { m.ctrl.Call(m, "Close") diff --git a/src/dbnode/storage/block/types.go b/src/dbnode/storage/block/types.go index f20d829d70..148392bdaf 100644 --- a/src/dbnode/storage/block/types.go +++ b/src/dbnode/storage/block/types.go @@ -191,6 +191,9 @@ type DatabaseBlock interface { metadata RetrievableBlockMetadata, ) + // Discard closes the block, but returns the (unfinalized) segment. + Discard() ts.Segment + // Close closes the block. Close() diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go new file mode 100644 index 0000000000..340476d9f4 --- /dev/null +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_map_gen.go @@ -0,0 +1,259 @@ +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +package commitlog + +import ( + "github.com/m3db/m3x/ident" +) + +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// MapHash is the hash for a given map entry, this is public to support +// iterating over the map using a native Go for loop. +type MapHash uint64 + +// HashFn is the hash function to execute when hashing a key. +type HashFn func(ident.ID) MapHash + +// EqualsFn is the equals key function to execute when detecting equality of a key. +type EqualsFn func(ident.ID, ident.ID) bool + +// CopyFn is the copy key function to execute when copying the key. +type CopyFn func(ident.ID) ident.ID + +// FinalizeFn is the finalize key function to execute when finished with a key. +type FinalizeFn func(ident.ID) + +// Map uses the genny package to provide a generic hash map that can be specialized +// by running the following command from this root of the repository: +// ``` +// make hashmap-gen pkg=outpkg key_type=Type value_type=Type out_dir=/tmp +// ``` +// Or if you would like to use bytes or ident.ID as keys you can use the +// partially specialized maps to generate your own maps as well: +// ``` +// make byteshashmap-gen pkg=outpkg value_type=Type out_dir=/tmp +// make idhashmap-gen pkg=outpkg value_type=Type out_dir=/tmp +// ``` +// This will output to stdout the generated source file to use for your map. +// It uses linear probing by incrementing the number of the hash created when +// hashing the identifier if there is a collision. +// Map is a value type and not an interface to allow for less painful +// upgrades when adding/removing methods, it is not likely to need mocking so +// an interface would not be super useful either. +type Map struct { + mapOptions + + // lookup uses hash of the identifier for the key and the MapEntry value + // wraps the value type and the key (used to ensure lookup is correct + // when dealing with collisions), we use uint64 for the hash partially + // because lookups of maps with uint64 keys has a fast path for Go. + lookup map[MapHash]MapEntry +} + +// mapOptions is a set of options used when creating an identifier map, it is kept +// private so that implementers of the generated map can specify their own options +// that partially fulfill these options. +type mapOptions struct { + // hash is the hash function to execute when hashing a key. + hash HashFn + // equals is the equals key function to execute when detecting equality. + equals EqualsFn + // copy is the copy key function to execute when copying the key. + copy CopyFn + // finalize is the finalize key function to execute when finished with a + // key, this is optional to specify. + finalize FinalizeFn + // initialSize is the initial size for the map, use zero to use Go's std map + // initial size and consequently is optional to specify. + initialSize int +} + +// MapEntry is an entry in the map, this is public to support iterating +// over the map using a native Go for loop. +type MapEntry struct { + // key is used to check equality on lookups to resolve collisions + key mapKey + // value type stored + value metadataAndEncodersByTime +} + +type mapKey struct { + key ident.ID + finalize bool +} + +// Key returns the map entry key. +func (e MapEntry) Key() ident.ID { + return e.key.key +} + +// Value returns the map entry value. +func (e MapEntry) Value() metadataAndEncodersByTime { + return e.value +} + +// mapAlloc is a non-exported function so that when generating the source code +// for the map you can supply a public constructor that sets the correct +// hash, equals, copy, finalize options without users of the map needing to +// implement them themselves. +func mapAlloc(opts mapOptions) *Map { + m := &Map{mapOptions: opts} + m.Reallocate() + return m +} + +func (m *Map) newMapKey(k ident.ID, opts mapKeyOptions) mapKey { + key := mapKey{key: k, finalize: opts.finalizeKey} + if !opts.copyKey { + return key + } + + key.key = m.copy(k) + return key +} + +func (m *Map) removeMapKey(hash MapHash, key mapKey) { + delete(m.lookup, hash) + if key.finalize { + m.finalize(key.key) + } +} + +// Get returns a value in the map for an identifier if found. +func (m *Map) Get(k ident.ID) (metadataAndEncodersByTime, bool) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + return entry.value, true + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } + var empty metadataAndEncodersByTime + return empty, false +} + +// Set will set the value for an identifier. +func (m *Map) Set(k ident.ID, v metadataAndEncodersByTime) { + m.set(k, v, mapKeyOptions{ + copyKey: true, + finalizeKey: m.finalize != nil, + }) +} + +// SetUnsafeOptions is a set of options to use when setting a value with +// the SetUnsafe method. +type SetUnsafeOptions struct { + NoCopyKey bool + NoFinalizeKey bool +} + +// SetUnsafe will set the value for an identifier with unsafe options for how +// the map treats the key. +func (m *Map) SetUnsafe(k ident.ID, v metadataAndEncodersByTime, opts SetUnsafeOptions) { + m.set(k, v, mapKeyOptions{ + copyKey: !opts.NoCopyKey, + finalizeKey: !opts.NoFinalizeKey, + }) +} + +type mapKeyOptions struct { + copyKey bool + finalizeKey bool +} + +func (m *Map) set(k ident.ID, v metadataAndEncodersByTime, opts mapKeyOptions) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + m.lookup[hash] = MapEntry{ + key: entry.key, + value: v, + } + return + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } + + m.lookup[hash] = MapEntry{ + key: m.newMapKey(k, opts), + value: v, + } +} + +// Iter provides the underlying map to allow for using a native Go for loop +// to iterate the map, however callers should only ever read and not write +// the map. +func (m *Map) Iter() map[MapHash]MapEntry { + return m.lookup +} + +// Len returns the number of map entries in the map. +func (m *Map) Len() int { + return len(m.lookup) +} + +// Contains returns true if value exists for key, false otherwise, it is +// shorthand for a call to Get that doesn't return the value. +func (m *Map) Contains(k ident.ID) bool { + _, ok := m.Get(k) + return ok +} + +// Delete will remove a value set in the map for the specified key. +func (m *Map) Delete(k ident.ID) { + hash := m.hash(k) + for entry, ok := m.lookup[hash]; ok; entry, ok = m.lookup[hash] { + if m.equals(entry.key.key, k) { + m.removeMapKey(hash, entry.key) + return + } + // Linear probe to "next" to this entry (really a rehash) + hash++ + } +} + +// Reset will reset the map by simply deleting all keys to avoid +// allocating a new map. +func (m *Map) Reset() { + for hash, entry := range m.lookup { + m.removeMapKey(hash, entry.key) + } +} + +// Reallocate will avoid deleting all keys and reallocate a new +// map, this is useful if you believe you have a large map and +// will not need to grow back to a similar size. +func (m *Map) Reallocate() { + if m.initialSize > 0 { + m.lookup = make(map[MapHash]MapEntry, m.initialSize) + } else { + m.lookup = make(map[MapHash]MapEntry) + } +} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go new file mode 100644 index 0000000000..16db841b26 --- /dev/null +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/metadata_and_encoders_by_time_new_map_gen.go @@ -0,0 +1,76 @@ +// This file was automatically generated by genny. +// Any changes will be lost if this file is regenerated. +// see https://github.com/mauricelam/genny + +package commitlog + +import ( + "github.com/m3db/m3x/ident" + + "github.com/m3db/m3x/pool" + + "github.com/cespare/xxhash" +) + +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// MapOptions provides options used when created the map. +type MapOptions struct { + InitialSize int + KeyCopyPool pool.BytesPool +} + +// NewMap returns a new byte keyed map. +func NewMap(opts MapOptions) *Map { + var ( + copyFn CopyFn + finalizeFn FinalizeFn + ) + if pool := opts.KeyCopyPool; pool == nil { + copyFn = func(k ident.ID) ident.ID { + return ident.BytesID(append([]byte(nil), k.Bytes()...)) + } + } else { + copyFn = func(k ident.ID) ident.ID { + bytes := k.Bytes() + keyLen := len(bytes) + pooled := pool.Get(keyLen)[:keyLen] + copy(pooled, bytes) + return ident.BytesID(pooled) + } + finalizeFn = func(k ident.ID) { + if slice, ok := k.(ident.BytesID); ok { + pool.Put(slice) + } + } + } + return mapAlloc(mapOptions{ + hash: func(id ident.ID) MapHash { + return MapHash(xxhash.Sum64(id.Bytes())) + }, + equals: func(x, y ident.ID) bool { + return x.Equal(y) + }, + copy: copyFn, + finalize: finalizeFn, + initialSize: opts.InitialSize, + }) +} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go index 1b0e78b83d..8d69a733ac 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source.go @@ -23,10 +23,12 @@ package commitlog import ( "errors" "fmt" + "io" "sync" "time" "github.com/m3db/m3db/src/dbnode/encoding" + "github.com/m3db/m3db/src/dbnode/persist" "github.com/m3db/m3db/src/dbnode/persist/fs" "github.com/m3db/m3db/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3db/src/dbnode/storage/block" @@ -36,8 +38,11 @@ import ( "github.com/m3db/m3db/src/dbnode/storage/namespace" "github.com/m3db/m3db/src/dbnode/ts" "github.com/m3db/m3db/src/dbnode/x/xio" + "github.com/m3db/m3x/checked" "github.com/m3db/m3x/ident" + "github.com/m3db/m3x/instrument" xlog "github.com/m3db/m3x/log" + "github.com/m3db/m3x/pool" xsync "github.com/m3db/m3x/sync" xtime "github.com/m3db/m3x/time" ) @@ -49,13 +54,19 @@ var ( const encoderChanBufSize = 1000 type newIteratorFn func(opts commitlog.IteratorOpts) (commitlog.Iterator, error) +type snapshotFilesFn func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error) +type newReaderFn func(bytesPool pool.CheckedBytesPool, opts fs.Options) (fs.DataFileSetReader, error) type commitLogSource struct { - opts Options - inspection fs.Inspection - log xlog.Logger - newIteratorFn newIteratorFn - cachedShardDataByNS map[string]*cachedShardData + opts Options + log xlog.Logger + + // Filesystem inspection capture before node was started. + inspection fs.Inspection + + newIteratorFn newIteratorFn + snapshotFilesFn snapshotFilesFn + newReaderFn newReaderFn } type encoder struct { @@ -65,11 +76,18 @@ type encoder struct { func newCommitLogSource(opts Options, inspection fs.Inspection) bootstrap.Source { return &commitLogSource{ - opts: opts, - inspection: inspection, - log: opts.ResultOptions().InstrumentOptions().Logger(), - newIteratorFn: commitlog.NewIterator, - cachedShardDataByNS: map[string]*cachedShardData{}, + opts: opts, + log: opts. + ResultOptions(). + InstrumentOptions(). + Logger(). + WithFields(xlog.NewField("bootstrapper", "commitlog")), + + inspection: inspection, + + newIteratorFn: commitlog.NewIterator, + snapshotFilesFn: fs.SnapshotFiles, + newReaderFn: fs.NewReader, } } @@ -91,6 +109,71 @@ func (s *commitLogSource) AvailableData( return shardsTimeRanges } +// ReadData will read a combination of the available snapshot files and commit log files to +// restore as much unflushed data from disk as possible. The logic for performing this +// correctly is as follows: +// +// 1. For every shard/blockStart combination, find the most recently written and complete +// (has a checkpoint file) snapshot. +// 2. For every shard/blockStart combination, determine the most recent complete SnapshotTime. +// This value corresponds to the (local) moment in time right before the snapshotting process +// began. +// 3. Find the minimum SnapshotTime for all of the shards and block starts (call it t0), and +// replay (M3TSZ encode) all commit log entries whose system timestamps overlap the range +// [minimumSnapshotTimeAcrossShards, blockStart.Add(blockSize).Add(bufferPast)]. This logic +// has one exception which is in the case where there is no minimimum snapshot time across +// shards (the code treats this case as minimum snapshot time across shards == blockStart). +// In that case, we replay all commit log entries whose system timestamps overlap the range +// [blockStart.Add(-bufferFuture), blockStart.Add(blockSize).Add(bufferPast)]. +// 4. For each shard/blockStart combination, merge all of the M3TSZ encoders that we created from +// reading the commit log along with the data available in the corresponding snapshot file. +// +// Example #1: +// +// BlockSize: 2hr +// BufferPast: 10m +// BufferFuture: 5m +// CommitLogBlockSize: 10m +// BlockToBootstrap: 12PM->2PM +// SnapshotTime: 12:30PM +// +// W1 comes in at 11:57AM +// W2 comes in at 12:29PM +// W3 comes in at 12:31PM +// W4 comes in at 2:04PM +// +// 1) W1 captured by snapshot (hence why we don't need to worry about buffer future +// with regards to commit logs when a snapshot file is present.) +// 2) W2 captured by snapshot file +// 3) W3 not captured by snapshot file (present in commit log with start time 12:30PM) +// 4) W4 not captured by snapshot file (present in commit log with start time 2:00PM) +// +// Need to read all commit logs that contain writes with system times spanning from +// 12:30PM -> 2:10PM which will bootstrap all of the data points above. I.E: +// [minimumMostRecentSnapshotTimeAcrossShards, blockStart.Add(blockSize).Add(bufferPast)] +// +// Example #2: +// +// BlockSize: 2hr +// BufferPast: 10m +// BufferFuture: 5m +// CommitLogBlockSize: 10m +// BlockToBootstrap: 12PM->2PM +// SnapshotTime: 12:00PM (snapshot does not exist) +// +// W1 comes in at 11:57AM +// W2 comes in at 12:29PM +// W3 comes in at 12:31PM +// W4 comes in at 2:04PM +// +// 1) W1 only present in commit log with start time 11:50PM +// 2) W2 only present in commit log with start time 12:20PM +// 3) W3 only present in commit log with start time 12:30PM +// 4) W4 only present in commit log with start time 2:00PM +// +// Need to read all commit logs that contain writes with system times spanning from +// 11:55AM -> 2:10PM which will bootstrap all of the data points above. I.E: +// [blockStart.Add(-bufferFuture), blockStart.Add(blockSize).Add(bufferPast)] func (s *commitLogSource) ReadData( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, @@ -100,19 +183,64 @@ func (s *commitLogSource) ReadData( return result.NewDataBootstrapResult(), nil } - readCommitLogPredicate := newReadCommitLogPredicate( - ns, shardsTimeRanges, s.opts, s.inspection) + var ( + fsOpts = s.opts.CommitLogOptions().FilesystemOptions() + filePathPrefix = fsOpts.FilePathPrefix() + ) + + // Determine which snapshot files are available. + snapshotFilesByShard, err := s.snapshotFilesByShard( + ns.ID(), filePathPrefix, shardsTimeRanges) + if err != nil { + return nil, err + } + + var ( + bOpts = s.opts.ResultOptions() + blOpts = bOpts.DatabaseBlockOptions() + blockSize = ns.Options().RetentionOptions().BlockSize() + ) - // TODO(rartoul): When we implement caching data across namespaces, this will need - // to be commitlog.ReadAllSeriesPredicate() if CacheSeriesMetadata() is enabled - // because we'll need to read data for all namespaces, not just the one we're currently - // bootstrapping. - readSeriesPredicate := newReadSeriesPredicate(ns) - iterOpts := commitlog.IteratorOpts{ - CommitLogOptions: s.opts.CommitLogOptions(), - FileFilterPredicate: readCommitLogPredicate, - SeriesFilterPredicate: readSeriesPredicate, + // Determine the minimum number of commit logs files that we + // must read based on the available snapshot files. + readCommitLogPred, mostRecentCompleteSnapshotByBlockShard, err := s.newReadCommitLogPredBasedOnAvailableSnapshotFiles( + ns, shardsTimeRanges, snapshotFilesByShard) + if err != nil { + return nil, err } + + // Setup the commit log iterator. + var ( + nsID = ns.ID() + seriesSkipped int + datapointsSkipped int + datapointsRead int + + // TODO(rartoul): When we implement caching data across namespaces, this will need + // to be commitlog.ReadAllSeriesPredicate() if CacheSeriesMetadata() is enabled + // because we'll need to read data for all namespaces, not just the one we're currently + // bootstrapping. + readSeriesPredicate = func(id ident.ID, namespace ident.ID) bool { + shouldReadSeries := nsID.Equal(namespace) + if !shouldReadSeries { + seriesSkipped++ + } + return shouldReadSeries + } + + iterOpts = commitlog.IteratorOpts{ + CommitLogOptions: s.opts.CommitLogOptions(), + FileFilterPredicate: readCommitLogPred, + SeriesFilterPredicate: readSeriesPredicate, + } + ) + + defer func() { + s.log.Infof("seriesSkipped: %d", seriesSkipped) + s.log.Infof("datapointsSkipped: %d", datapointsSkipped) + s.log.Infof("datapointsRead: %d", datapointsRead) + }() + iter, err := s.newIteratorFn(iterOpts) if err != nil { return nil, fmt.Errorf("unable to create commit log iterator: %v", err) @@ -120,26 +248,17 @@ func (s *commitLogSource) ReadData( defer iter.Close() + // Setup the M3TSZ encoding pipeline var ( // +1 so we can use the shard number as an index throughout without constantly // remembering to subtract 1 to convert to zero-based indexing - numShards = s.findHighestShard(shardsTimeRanges) + 1 - numConc = s.opts.EncodingConcurrency() - bopts = s.opts.ResultOptions() - blopts = bopts.DatabaseBlockOptions() - blockSize = ns.Options().RetentionOptions().BlockSize() - encoderPool = bopts.DatabaseBlockOptions().EncoderPool() - workerErrs = make([]int, numConc) + numShards = s.findHighestShard(shardsTimeRanges) + 1 + numConc = s.opts.EncodingConcurrency() + encoderPool = blOpts.EncoderPool() + workerErrs = make([]int, numConc) + shardDataByShard = s.newShardDataByShard(shardsTimeRanges, numShards) ) - shardDataByShard := make([]shardData, numShards) - for shard := range shardsTimeRanges { - shardDataByShard[shard] = shardData{ - series: make(map[uint64]metadataAndEncodersByTime), - ranges: shardsTimeRanges[shard], - } - } - encoderChans := make([]chan encoderArg, numConc) for i := 0; i < numConc; i++ { encoderChans[i] = make(chan encoderArg, encoderChanBufSize) @@ -152,15 +271,19 @@ func (s *commitLogSource) ReadData( for workerNum, encoderChan := range encoderChans { wg.Add(1) go s.startM3TSZEncodingWorker( - ns, runOpts, workerNum, encoderChan, shardDataByShard, encoderPool, workerErrs, blopts, wg) + ns, runOpts, workerNum, encoderChan, shardDataByShard, encoderPool, workerErrs, blOpts, wg) } + // Read / M3TSZ encode all the datapoints in the commit log that we need to read. for iter.Next() { series, dp, unit, annotation := iter.Current() if !s.shouldEncodeForData(shardDataByShard, blockSize, series, dp.Timestamp) { + datapointsSkipped++ continue } + datapointsRead++ + // Distribute work such that each encoder goroutine is responsible for // approximately numShards / numConc shards. This also means that all // datapoints for a given shard/series will be processed in a serialized @@ -178,20 +301,516 @@ func (s *commitLogSource) ReadData( } } + if iterErr := iter.Err(); iterErr != nil { + return nil, iterErr + } + for _, encoderChan := range encoderChans { close(encoderChan) } - // Block until all data has been read and encoded by the worker goroutines + // Block until all required data from the commit log has been read and + // encoded by the worker goroutines wg.Wait() s.logEncodingOutcome(workerErrs, iter) - result := s.mergeShards(int(numShards), bopts, blockSize, blopts, encoderPool, shardDataByShard) - // After merging shards, its safe to cache the shardData (which involves some mutation). - if s.shouldCacheSeriesMetadata(runOpts, ns) { - s.cacheShardData(ns, shardDataByShard) + // Merge all the different encoders from the commit log that we created with + // the data that is available in the snapshot files. + mergeStart := time.Now() + s.log.Infof("starting merge...") + bootstrapResult, err := s.mergeAllShardsCommitLogEncodersAndSnapshots( + ns, + shardsTimeRanges, + snapshotFilesByShard, + mostRecentCompleteSnapshotByBlockShard, + int(numShards), + blockSize, + shardDataByShard, + ) + if err != nil { + return nil, err + } + s.log.Infof("done merging..., took: %s", time.Since(mergeStart).String()) + + return bootstrapResult, nil +} + +func (s *commitLogSource) snapshotFilesByShard( + nsID ident.ID, + filePathPrefix string, + shardsTimeRanges result.ShardTimeRanges, +) (map[uint32]fs.FileSetFilesSlice, error) { + snapshotFilesByShard := map[uint32]fs.FileSetFilesSlice{} + for shard := range shardsTimeRanges { + snapshotFiles, err := s.snapshotFilesFn(filePathPrefix, nsID, shard) + if err != nil { + return nil, err + } + snapshotFilesByShard[shard] = snapshotFiles + } + + return snapshotFilesByShard, nil +} + +func (s *commitLogSource) newShardDataByShard( + shardsTimeRanges result.ShardTimeRanges, + numShards uint32, +) []shardData { + shardDataByShard := make([]shardData, numShards) + for shard := range shardsTimeRanges { + shardDataByShard[shard] = shardData{ + series: NewMap(MapOptions{}), + ranges: shardsTimeRanges[shard], + } + } + + return shardDataByShard +} + +// mostRecentCompleteSnapshotByBlockShard returns a +// map[xtime.UnixNano]map[uint32]fs.FileSetFile with the contract that +// for each shard/block combination in shardsTimeRanges, an entry will +// exist in the map such that FileSetFile.CachedSnapshotTime is the +// actual cached snapshot time, or the blockStart. +func (s *commitLogSource) mostRecentCompleteSnapshotByBlockShard( + shardsTimeRanges result.ShardTimeRanges, + blockSize time.Duration, + snapshotFilesByShard map[uint32]fs.FileSetFilesSlice, + fsOpts fs.Options, +) map[xtime.UnixNano]map[uint32]fs.FileSetFile { + var ( + minBlock, maxBlock = shardsTimeRanges.MinMax() + mostRecentSnapshotsByBlockShard = map[xtime.UnixNano]map[uint32]fs.FileSetFile{} + ) + + for currBlockStart := minBlock.Truncate(blockSize); currBlockStart.Before(maxBlock); currBlockStart = currBlockStart.Add(blockSize) { + for shard := range shardsTimeRanges { + // Anonymous func for easier clean up using defer. + func() { + var ( + currBlockUnixNanos = xtime.ToUnixNano(currBlockStart) + mostRecentSnapshot fs.FileSetFile + ) + + defer func() { + existing := mostRecentSnapshotsByBlockShard[currBlockUnixNanos] + if existing == nil { + existing = map[uint32]fs.FileSetFile{} + } + + if mostRecentSnapshot.IsZero() { + // If we were unable to determine the most recent snapshot time for a given + // shard/blockStart combination, then just fall back to using the blockStart + // time as that will force us to read the entire commit log for that duration. + mostRecentSnapshot.CachedSnapshotTime = currBlockStart + } + existing[shard] = mostRecentSnapshot + mostRecentSnapshotsByBlockShard[currBlockUnixNanos] = existing + }() + + snapshotFiles, ok := snapshotFilesByShard[shard] + if !ok { + // If there are no snapshot files for this shard, then rely on + // the defer to fallback to using the block start time. + return + } + + mostRecentSnapshotVolume, ok := snapshotFiles.LatestVolumeForBlock(currBlockStart) + if !ok { + // If there are no complete snapshot files for this block, then rely on + // the defer to fallback to using the block start time. + return + } + + // Make sure we're able to read the snapshot time. This will also set the + // CachedSnapshotTime field so that we can rely upon it from here on out. + _, err := mostRecentSnapshotVolume.SnapshotTime() + if err != nil { + s.log. + WithFields( + xlog.NewField("namespace", mostRecentSnapshot.ID.Namespace), + xlog.NewField("blockStart", mostRecentSnapshot.ID.BlockStart), + xlog.NewField("shard", mostRecentSnapshot.ID.Shard), + xlog.NewField("index", mostRecentSnapshot.ID.VolumeIndex), + xlog.NewField("filepaths", mostRecentSnapshot.AbsoluteFilepaths), + ). + Error("error resolving snapshot time for snapshot file") + + // If we couldn't determine the snapshot time for the snapshot file, then rely + // on the defer to fallback to using the block start time. + return + } + + mostRecentSnapshot = mostRecentSnapshotVolume + }() + } + } + + return mostRecentSnapshotsByBlockShard +} + +func (s *commitLogSource) minimumMostRecentSnapshotTimeByBlock( + shardsTimeRanges result.ShardTimeRanges, + blockSize time.Duration, + mostRecentSnapshotByBlockShard map[xtime.UnixNano]map[uint32]fs.FileSetFile, +) map[xtime.UnixNano]time.Time { + minimumMostRecentSnapshotTimeByBlock := map[xtime.UnixNano]time.Time{} + for blockStart, mostRecentSnapshotsByShard := range mostRecentSnapshotByBlockShard { + + var minMostRecentSnapshot time.Time + for shard, mostRecentSnapshotForShard := range mostRecentSnapshotsByShard { + blockRange := xtime.Range{Start: blockStart.ToTime(), End: blockStart.ToTime().Add(blockSize)} + if !shardsTimeRanges[shard].Overlaps(blockRange) { + // In order for a minimum most recent snapshot to be valid, it needs to be for a block that + // we actually need to bootstrap for that shard. This check ensures that our algorithm doesn't + // do any extra work, even if we're bootstrapping different blocks for various shards. + continue + } + + if mostRecentSnapshotForShard.CachedSnapshotTime.Before(minMostRecentSnapshot) || minMostRecentSnapshot.IsZero() { + minMostRecentSnapshot = mostRecentSnapshotForShard.CachedSnapshotTime + } + } + + if minMostRecentSnapshot.IsZero() { + // If we didn't find a minimum most recent snapshot time for this blockStart, just use the + // blockStart as the minimum since we'll need to read the entire commit log in this case. + minMostRecentSnapshot = blockStart.ToTime() + } + + minimumMostRecentSnapshotTimeByBlock[blockStart] = minMostRecentSnapshot + } + + return minimumMostRecentSnapshotTimeByBlock +} + +func (s *commitLogSource) bootstrapShardSnapshots( + nsID ident.ID, + shard uint32, + metadataOnly bool, + shardTimeRanges xtime.Ranges, + blockSize time.Duration, + snapshotFiles fs.FileSetFilesSlice, + mostRecentCompleteSnapshotByBlockShard map[xtime.UnixNano]map[uint32]fs.FileSetFile, +) (result.ShardResult, error) { + var ( + shardResult result.ShardResult + allSeriesSoFar *result.Map + rangeIter = shardTimeRanges.Iter() + err error + ) + + for rangeIter.Next() { + + var ( + currRange = rangeIter.Value() + currRangeDuration = currRange.End.Sub(currRange.Start) + isMultipleOfBlockSize = currRangeDuration%blockSize == 0 + ) + + if !isMultipleOfBlockSize { + return nil, fmt.Errorf( + "received bootstrap range that is not multiple of blockSize, blockSize: %d, start: %s, end: %s", + blockSize, currRange.End.String(), currRange.Start.String(), + ) + } + + // Reset this after we bootstrap each block to make sure its up to date. + if shardResult != nil { + allSeriesSoFar = shardResult.AllSeries() + } + + for blockStart := currRange.Start.Truncate(blockSize); blockStart.Before(currRange.End); blockStart = blockStart.Add(blockSize) { + snapshotsForBlock := mostRecentCompleteSnapshotByBlockShard[xtime.ToUnixNano(blockStart)] + mostRecentCompleteSnapshotForShardBlock := snapshotsForBlock[shard] + + if mostRecentCompleteSnapshotForShardBlock.CachedSnapshotTime.Equal(blockStart) || + // Should never happen + mostRecentCompleteSnapshotForShardBlock.IsZero() { + // There is no snapshot file for this time, and even if there was, there would + // be no point in reading it. In this specific case its not an error scenario + // because the fact that snapshotTime == blockStart means we already accounted + // for the fact that this snapshot did not exist when we were deciding which + // commit logs to read. + s.log.Infof( + "no snapshots for shard: %d and blockStart: %s", + shard, blockStart.String()) + continue + } + + shardResult, err = s.bootstrapShardBlockSnapshot( + nsID, shard, blockStart, metadataOnly, shardResult, allSeriesSoFar, blockSize, + snapshotFiles, mostRecentCompleteSnapshotForShardBlock) + if err != nil { + return shardResult, err + } + } + } + + if shardResult == nil { + shardResult = result.NewShardResult(0, s.opts.ResultOptions()) + } + return shardResult, nil +} + +func (s *commitLogSource) bootstrapShardBlockSnapshot( + nsID ident.ID, + shard uint32, + blockStart time.Time, + metadataOnly bool, + shardResult result.ShardResult, + allSeriesSoFar *result.Map, + blockSize time.Duration, + snapshotFiles fs.FileSetFilesSlice, + mostRecentCompleteSnapshot fs.FileSetFile, +) (result.ShardResult, error) { + var ( + bOpts = s.opts.ResultOptions() + blOpts = bOpts.DatabaseBlockOptions() + blocksPool = blOpts.DatabaseBlockPool() + bytesPool = blOpts.BytesPool() + fsOpts = s.opts.CommitLogOptions().FilesystemOptions() + idPool = s.opts.CommitLogOptions().IdentifierPool() + ) + + // Bootstrap the snapshot file + reader, err := s.newReaderFn(bytesPool, fsOpts) + if err != nil { + return shardResult, err + } + + err = reader.Open(fs.DataReaderOpenOptions{ + Identifier: fs.FileSetFileIdentifier{ + Namespace: nsID, + BlockStart: blockStart, + Shard: shard, + VolumeIndex: mostRecentCompleteSnapshot.ID.VolumeIndex, + }, + FileSetType: persist.FileSetSnapshotType, + }) + if err != nil { + return shardResult, err + } + + s.log.Infof( + "reading snapshot for shard: %d and blockStart: %s and volume: %d", + shard, blockStart.String(), mostRecentCompleteSnapshot.ID.VolumeIndex) + for { + var ( + id ident.ID + tagsIter ident.TagIterator + data checked.Bytes + expectedChecksum uint32 + ) + if metadataOnly { + id, tagsIter, _, _, err = reader.ReadMetadata() + } else { + id, tagsIter, data, expectedChecksum, err = reader.Read() + } + if err != nil && err != io.EOF { + return shardResult, err + } + + if err == io.EOF { + break + } + + dbBlock := blocksPool.Get() + dbBlock.Reset(blockStart, blockSize, ts.NewSegment(data, nil, ts.FinalizeHead)) + + if !metadataOnly { + // Resetting the block will trigger a checksum calculation, so use that instead + // of calculating it twice. + checksum, err := dbBlock.Checksum() + if err != nil { + return shardResult, err + } + + if checksum != expectedChecksum { + return shardResult, fmt.Errorf("checksum for series: %s was %d but expected %d", id, checksum, expectedChecksum) + } + } + + var ( + tags ident.Tags + shouldDecodeTags = true + ) + if allSeriesSoFar != nil { + existing, ok := allSeriesSoFar.Get(id) + if ok { + // If we've already bootstrapped this series for a different block, we don't need + // another copy of the IDs and tags. + id.Finalize() + id = existing.ID + tags = existing.Tags + shouldDecodeTags = false + } + } + + if shouldDecodeTags { + // Only spend cycles decoding the tags if we've never seen them before. + if tagsIter.Remaining() > 0 { + tags, err = convert.TagsFromTagsIter(id, tagsIter, idPool) + if err != nil { + return shardResult, fmt.Errorf("unable to decode tags: %v", err) + } + } + } + // Always close even if we didn't use it. + tagsIter.Close() + + if shardResult == nil { + // Delay initialization so we can estimate size. + shardResult = result.NewShardResult(reader.Entries(), s.opts.ResultOptions()) + } + + // Mark the ID and Tags as no finalize to enable no-copy optimization later + // in the bootstrap process (when they're being loaded into the shard). Also, + // technically we'll be calling NoFinalize() repeatedly on the same IDs for + // different blocks since we're reusing them, but thats ok as it an idempotent + // operation and there is no concurrency here. + id.NoFinalize() + tags.NoFinalize() + shardResult.AddBlock(id, tags, dbBlock) + } + + return shardResult, nil +} + +func (s *commitLogSource) newReadCommitLogPredBasedOnAvailableSnapshotFiles( + ns namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, + snapshotFilesByShard map[uint32]fs.FileSetFilesSlice, +) ( + func(fileName string, fileStart time.Time, fileBlockSize time.Duration) bool, + map[xtime.UnixNano]map[uint32]fs.FileSetFile, + error, +) { + blockSize := ns.Options().RetentionOptions().BlockSize() + + // At this point we've bootstrapped all the snapshot files that we can, and we need to + // decide which commit logs to read. In order to do that, we'll need to figure out the + // minimum most recent snapshot time for each block, then we can use that information to + // decide how much of the commit log we need to read for each block that we're bootstrapping. + // To start, for each block that we're bootstrapping, we need to figure out the most recent + // snapshot that was taken for each shard. I.E we want to create a datastructure that looks + // like this: + // map[blockStart]map[shard]mostRecentSnapshotTime + mostRecentCompleteSnapshotByBlockShard := s.mostRecentCompleteSnapshotByBlockShard( + shardsTimeRanges, blockSize, snapshotFilesByShard, s.opts.CommitLogOptions().FilesystemOptions()) + for block, mostRecentByShard := range mostRecentCompleteSnapshotByBlockShard { + for shard, mostRecent := range mostRecentByShard { + + if mostRecent.CachedSnapshotTime.IsZero() { + // Should never happen. + return nil, nil, fmt.Errorf( + "%s shard: %d and block: %s had zero value for most recent snapshot time", + instrument.InvariantViolatedMetricName, shard, block.ToTime().String()) + } + + s.log.Infof( + "most recent snapshot for block: %s and shard: %d is %s", + block.ToTime().String(), shard, mostRecent.CachedSnapshotTime.String()) + } + } + + // Once we have the desired data structure, we next need to figure out the minimum most recent + // snapshot for that block across all shards. This will help us determine how much of the commit + // log we need to read. The new data structure we're trying to generate looks like: + // map[blockStart]minimumMostRecentSnapshotTime (across all shards) + // This structure is important because it tells us how much of the commit log we need to read for + // each block that we're trying to bootstrap (because the commit log is shared across all shards.) + minimumMostRecentSnapshotTimeByBlock := s.minimumMostRecentSnapshotTimeByBlock( + shardsTimeRanges, blockSize, mostRecentCompleteSnapshotByBlockShard) + for block, minSnapshotTime := range minimumMostRecentSnapshotTimeByBlock { + s.log.Infof( + "min snapshot time for block: %s is: %s", + block.ToTime().String(), minSnapshotTime.String()) + } + + // Now that we have the minimum most recent snapshot time for each block, we can use that data to + // decide how much of the commit log we need to read for each block that we're bootstrapping. We'll + // construct a new predicate based on the data structure we constructed earlier where the new + // predicate will check if there is any overlap between a commit log file and a temporary range + // we construct that begins with the minimum snapshot time and ends with the end of that block + bufferPast. + return s.newReadCommitLogPred(ns, minimumMostRecentSnapshotTimeByBlock), mostRecentCompleteSnapshotByBlockShard, nil +} + +func (s *commitLogSource) newReadCommitLogPred( + ns namespace.Metadata, + minimumMostRecentSnapshotTimeByBlock map[xtime.UnixNano]time.Time, +) func(fileName string, fileStart time.Time, fileBlockSize time.Duration) bool { + var ( + rOpts = ns.Options().RetentionOptions() + blockSize = rOpts.BlockSize() + bufferPast = rOpts.BufferPast() + bufferFuture = rOpts.BufferFuture() + rangesToCheck = []xtime.Range{} + commitlogFilesPresentBeforeStart = s.inspection.CommitLogFilesSet() + ) + + for blockStart, minimumMostRecentSnapshotTime := range minimumMostRecentSnapshotTimeByBlock { + // blockStart.Add(blockSize) represents the logical range that we're trying to bootstrap, but + // commitlog and snapshot timestamps are system timestamps so we need to create a system + // time range against which we can compare our commit log ranges. + // + // In this case, the snapshot will contain all datapoints for a given block that were received/written + // (system time) before the snapshot time, so we use that as the start of our range. + // + // The end of our range is the end of the block + the bufferPast window. This is because its + // still possible for writes for the block that we're trying to bootstrap to arrive up until + // blockStart.Add(blockSize).Add(bufferPast). + // + // Note that in the general case (snapshot files are present) we don't need to check bufferFuture + // at all, because the snapshot is guaranteed to have all writes that were written before the + // snapshot time, which includes any datapoints written during the bufferFuture range, by definition. + // However, if there is no snapshot (minimumMostRecentSnapshotTime.Equal(blockStart)), then we DO + // have to take bufferFuture into account because commit logs with system timestamps in the previous + // block may contain writes for the block that we're trying to bootstrap, and we can't rely upon the + // fact that they are already included in our (non-existent) snapshot. + if minimumMostRecentSnapshotTime.Equal(blockStart.ToTime()) { + minimumMostRecentSnapshotTime = minimumMostRecentSnapshotTime.Add(-bufferFuture) + } + rangesToCheck = append(rangesToCheck, xtime.Range{ + Start: minimumMostRecentSnapshotTime, + End: blockStart.ToTime().Add(blockSize).Add(bufferPast), + }) + } + + // TODO: We have to rely on the global minimum across shards to determine which commit log files + // we need to read, but we can still skip datapoints from the commitlog itself that belong to a shard + // that has a snapshot more recent than the global minimum. If we use an array for fast-access this could + // be a small win in terms of memory utilization. + return func(fileName string, fileStart time.Time, fileBlockSize time.Duration) bool { + _, ok := commitlogFilesPresentBeforeStart[fileName] + if !ok { + // If the file wasn't on disk before the node started then it only contains + // writes that are already in memory (and in-fact the file may be actively + // being written to.) + return false + } + + for _, rangeToCheck := range rangesToCheck { + commitLogEntryRange := xtime.Range{ + Start: fileStart, + End: fileStart.Add(fileBlockSize), + } + + if commitLogEntryRange.Overlaps(rangeToCheck) { + s.log. + Infof( + "opting to read commit log: %s with start: %s and duration: %s", + fileName, fileStart.String(), fileBlockSize.String()) + return true + } + } + + s.log. + Infof( + "opting to skip commit log: %s with start: %s and duration: %s", + fileName, fileStart.String(), fileBlockSize.String()) + return false } - return result, nil } func (s *commitLogSource) startM3TSZEncodingWorker( @@ -205,7 +824,6 @@ func (s *commitLogSource) startM3TSZEncodingWorker( blopts block.Options, wg *sync.WaitGroup, ) { - shouldCacheSeriesMetadata := s.shouldCacheSeriesMetadata(runOpts, ns) for arg := range ec { var ( series = arg.series @@ -215,30 +833,23 @@ func (s *commitLogSource) startM3TSZEncodingWorker( blockStart = arg.blockStart ) - unmergedShard := unmerged[series.Shard].series - unmergedSeries, ok := unmergedShard[series.UniqueIndex] + var ( + unmergedShard = unmerged[series.Shard].series + unmergedSeries, ok = unmergedShard.Get(series.ID) + ) if !ok { - if shouldCacheSeriesMetadata { - // If we're going to cache the IDs and Tags on the commitlog source, then - // we need to make sure that they won't get finalized by anything else in - // the code base. Specifically, since series.Tags is a struct (not a pointer - // to a struct), we need to call NoFinalize() on it as early in the code-path - // as possible so that the NoFinalize() state is propagated everywhere (since - // the struct will get copied repeatedly.) - // - // This is also the "ideal" spot to mark the IDs as NoFinalize(), because it - // only occurs once per series per run. So if we end up allocating the IDs/Tags - // multiple times during the bootstrap, we'll only mark the first appearance as - // NoFinalize, and all subsequent occurrences can be finalized per usual. - series.ID.NoFinalize() - series.Tags.NoFinalize() - } - unmergedSeries = metadataAndEncodersByTime{ id: series.ID, tags: series.Tags, - encoders: make(map[xtime.UnixNano]encoders)} - unmergedShard[series.UniqueIndex] = unmergedSeries + encoders: make(map[xtime.UnixNano][]encoder)} + // Have to use unsafe because we don't want to copy the IDs we put + // into this map because its lifecycle is much shorter than that of + // the IDs we're putting into it so copying would waste too much + // memory unnecessarily, and we don't want to finalize the IDs for the + // same reason. + unmergedShard.SetUnsafe( + series.ID, unmergedSeries, + SetUnsafeOptions{NoCopyKey: true, NoFinalizeKey: true}) } var ( @@ -336,44 +947,72 @@ func (s *commitLogSource) shouldIncludeInIndex( return rangesToBootstrap.Overlaps(indexBlockRange) } -func (s *commitLogSource) mergeShards( +func (s *commitLogSource) mergeAllShardsCommitLogEncodersAndSnapshots( + ns namespace.Metadata, + shardsTimeRanges result.ShardTimeRanges, + snapshotFiles map[uint32]fs.FileSetFilesSlice, + mostRecentCompleteSnapshotByBlockShard map[xtime.UnixNano]map[uint32]fs.FileSetFile, numShards int, - bopts result.Options, blockSize time.Duration, - blopts block.Options, - encoderPool encoding.EncoderPool, unmerged []shardData, -) result.DataBootstrapResult { +) (result.DataBootstrapResult, error) { var ( - shardErrs = make([]int, numShards) - shardEmptyErrs = make([]int, numShards) - bootstrapResult = result.NewDataBootstrapResult() - blocksPool = bopts.DatabaseBlockOptions().DatabaseBlockPool() - multiReaderIteratorPool = blopts.MultiReaderIteratorPool() + shardErrs = make([]int, numShards) + shardEmptyErrs = make([]int, numShards) + bootstrapResult = result.NewDataBootstrapResult() // Controls how many shards can be merged in parallel workerPool = xsync.NewWorkerPool(s.opts.MergeShardsConcurrency()) bootstrapResultLock sync.Mutex wg sync.WaitGroup ) - workerPool.Init() for shard, unmergedShard := range unmerged { if unmergedShard.series == nil { + // Not bootstrapping this shard continue } + + snapshotData, err := s.bootstrapShardSnapshots( + ns.ID(), + uint32(shard), + false, + shardsTimeRanges[uint32(shard)], + blockSize, + snapshotFiles[uint32(shard)], + mostRecentCompleteSnapshotByBlockShard, + ) + if err != nil { + bootstrapResultLock.Lock() + // Mark the shard time ranges as unfulfilled so a subsequent bootstrapper + // has the chance to fulfill it. + bootstrapResult.Add( + uint32(shard), + result.NewShardResult(0, s.opts.ResultOptions()), + shardsTimeRanges[uint32(shard)], + ) + bootstrapResultLock.Unlock() + return nil, err + } + + // Merge snapshot and commit log data wg.Add(1) shard, unmergedShard := shard, unmergedShard mergeShardFunc := func() { var shardResult result.ShardResult - shardResult, shardEmptyErrs[shard], shardErrs[shard] = s.mergeShard( - shard, unmergedShard, blocksPool, multiReaderIteratorPool, encoderPool, blockSize, blopts) + shardResult, shardEmptyErrs[shard], shardErrs[shard] = s.mergeShardCommitLogEncodersAndSnapshots( + shard, snapshotData, unmergedShard, blockSize) if shardResult != nil && shardResult.NumSeries() > 0 { // Prevent race conditions while updating bootstrapResult from multiple go-routines bootstrapResultLock.Lock() - // Shard is a slice index so conversion to uint32 is safe - bootstrapResult.Add(uint32(shard), shardResult, xtime.Ranges{}) + if shardEmptyErrs[shard] != 0 || shardErrs[shard] != 0 { + // If there were any errors, keep the data but mark the shard time ranges as + // unfulfilled so a subsequent bootstrapper has the chance to fulfill it. + bootstrapResult.Add(uint32(shard), shardResult, shardsTimeRanges[uint32(shard)]) + } else { + bootstrapResult.Add(uint32(shard), shardResult, xtime.Ranges{}) + } bootstrapResultLock.Unlock() } wg.Done() @@ -384,48 +1023,87 @@ func (s *commitLogSource) mergeShards( // Wait for all merge goroutines to complete wg.Wait() s.logMergeShardsOutcome(shardErrs, shardEmptyErrs) - return bootstrapResult + return bootstrapResult, nil } -func (s *commitLogSource) mergeShard( +func (s *commitLogSource) mergeShardCommitLogEncodersAndSnapshots( shard int, + snapshotData result.ShardResult, unmergedShard shardData, - blocksPool block.DatabaseBlockPool, - multiReaderIteratorPool encoding.MultiReaderIteratorPool, - encoderPool encoding.EncoderPool, blockSize time.Duration, - blopts block.Options, ) (result.ShardResult, int, int) { - var shardResult result.ShardResult - var numShardEmptyErrs int - var numErrs int + var ( + bOpts = s.opts.ResultOptions() + blOpts = bOpts.DatabaseBlockOptions() + blocksPool = blOpts.DatabaseBlockPool() + multiReaderIteratorPool = blOpts.MultiReaderIteratorPool() + segmentReaderPool = blOpts.SegmentReaderPool() + encoderPool = blOpts.EncoderPool() + ) - for _, unmergedBlocks := range unmergedShard.series { - seriesBlocks, numSeriesEmptyErrs, numSeriesErrs := s.mergeSeries( - unmergedBlocks, - blocksPool, - multiReaderIteratorPool, - encoderPool, - blockSize, - blopts, - ) + numSeries := 0 + if unmergedShard.series != nil { + numSeries = unmergedShard.series.Len() + } + + var ( + shardResult = result.NewShardResult(numSeries, s.opts.ResultOptions()) + numShardEmptyErrs int + numErrs int + ) - if seriesBlocks != nil && seriesBlocks.Len() > 0 { - if shardResult == nil { - shardResult = result.NewShardResult(len(unmergedShard.series), s.opts.ResultOptions()) + allSnapshotSeries := snapshotData.AllSeries() + + if unmergedShard.series != nil { + for _, unmergedBlocks := range unmergedShard.series.Iter() { + val := unmergedBlocks.Value() + snapshotSeriesData, _ := allSnapshotSeries.Get(val.id) + seriesBlocks, numSeriesEmptyErrs, numSeriesErrs := s.mergeSeries( + snapshotSeriesData, + val, + blocksPool, + multiReaderIteratorPool, + segmentReaderPool, + encoderPool, + blockSize, + blOpts, + ) + + if seriesBlocks != nil && seriesBlocks.Len() > 0 { + shardResult.AddSeries(val.id, val.tags, seriesBlocks) } - shardResult.AddSeries(unmergedBlocks.id, unmergedBlocks.tags, seriesBlocks) + + numShardEmptyErrs += numSeriesEmptyErrs + numErrs += numSeriesErrs + } + } + + allShardResultSeries := shardResult.AllSeries() + for _, val := range allSnapshotSeries.Iter() { + id := val.Key() + blocks := val.Value() + + if allShardResultSeries.Contains(id) { + // Already merged so we know the ID and tags from the snapshot + // won't be used and can be closed. We can't close the blocks + // though because we may have loaded some of the blocks into + // the shard result and we don't want to close them. + id.Finalize() + blocks.Tags.Finalize() + continue } - numShardEmptyErrs += numSeriesEmptyErrs - numErrs += numSeriesErrs + shardResult.AddSeries(id, blocks.Tags, blocks.Blocks) } return shardResult, numShardEmptyErrs, numErrs } func (s *commitLogSource) mergeSeries( - unmergedBlocks metadataAndEncodersByTime, blocksPool block.DatabaseBlockPool, + snapshotData result.DatabaseSeriesBlocks, + unmergedCommitlogBlocks metadataAndEncodersByTime, + blocksPool block.DatabaseBlockPool, multiReaderIteratorPool encoding.MultiReaderIteratorPool, + segmentReaderPool xio.SegmentReaderPool, encoderPool encoding.EncoderPool, blockSize time.Duration, blopts block.Options, @@ -434,30 +1112,33 @@ func (s *commitLogSource) mergeSeries( var numEmptyErrs int var numErrs int - for startNano, encoders := range unmergedBlocks.encoders { - start := startNano.ToTime() + for startNano, encoders := range unmergedCommitlogBlocks.encoders { + var ( + start = startNano.ToTime() + snapshotBlock block.DatabaseBlock + hasSnapshotBlock bool + ) - if len(encoders) == 0 { - numEmptyErrs++ - continue + if snapshotData.Blocks != nil { + snapshotBlock, hasSnapshotBlock = snapshotData.Blocks.BlockAt(start) } - if len(encoders) == 1 { - pooledBlock := blocksPool.Get() - pooledBlock.Reset(start, blockSize, encoders[0].enc.Discard()) - if seriesBlocks == nil { - seriesBlocks = block.NewDatabaseSeriesBlocks(len(unmergedBlocks.encoders)) - } - seriesBlocks.AddBlock(pooledBlock) + if !hasSnapshotBlock { + // Make sure snapshotBlock is nil if it does not exist. + snapshotBlock = nil + } + + // Closes encoders and snapshotBlock by calling Discard() on each. + readers, err := newIOReadersFromEncodersAndBlock( + segmentReaderPool, encoders, snapshotBlock) + if err != nil { + numErrs++ continue } - // Convert encoders to readers so we can use iteration helpers - readers := encoders.newReaders() iter := multiReaderIteratorPool.Get() iter.Reset(readers, time.Time{}, 0) - var err error enc := encoderPool.Get() enc.Reset(start, blopts.DatabaseBlockAllocSize()) for iter.Next() { @@ -479,8 +1160,12 @@ func (s *commitLogSource) mergeSeries( // Automatically returns iter to the pool iter.Close() - encoders.close() readers.close() + if hasSnapshotBlock { + // Block is already closed, but we need to remove from the Blocks + // to prevent a double free when we call Blocks.Close() later. + snapshotData.Blocks.RemoveBlockAt(start) + } if err != nil { continue @@ -489,10 +1174,31 @@ func (s *commitLogSource) mergeSeries( pooledBlock := blocksPool.Get() pooledBlock.Reset(start, blockSize, enc.Discard()) if seriesBlocks == nil { - seriesBlocks = block.NewDatabaseSeriesBlocks(len(unmergedBlocks.encoders)) + seriesBlocks = block.NewDatabaseSeriesBlocks(len(unmergedCommitlogBlocks.encoders)) } seriesBlocks.AddBlock(pooledBlock) } + + if snapshotData.Blocks != nil { + allSnapshotBlocks := snapshotData.Blocks.AllBlocks() + for startNano, snapshotBlock := range snapshotData.Blocks.AllBlocks() { + if seriesBlocks == nil { + seriesBlocks = block.NewDatabaseSeriesBlocks(len(allSnapshotBlocks)) + } + _, ok := seriesBlocks.BlockAt(startNano.ToTime()) + if ok { + // Should never happen because we would have called + // Blocks.RemoveBlockAt() above. + iOpts := s.opts.CommitLogOptions().InstrumentOptions() + invariantLogger := instrument.EmitInvariantViolationAndGetLogger(iOpts) + invariantLogger.Errorf( + "tried to merge block that should have been removed, blockStart: %d", startNano) + continue + } + + seriesBlocks.AddBlock(snapshotBlock) + } + } return seriesBlocks, numEmptyErrs, numErrs } @@ -537,87 +1243,6 @@ func (s *commitLogSource) logMergeShardsOutcome(shardErrs []int, shardEmptyErrs } } -// cacheShardData caches the shardData from a call to ReadData() on the source so that subsequent calls -// to ReadIndex() for the same time period don't have to read the commit log files again. -// -// In order for the subsequent call to ReadIndex() to avoid reading the same commit log files, we need -// to cache three pieces of information for every series: -// 1) The ID (so it can be indexed) -// 2) The tags (so they can be indexed) -// 3) The block starts for which the series had a datapoint (so that we know which index blocks -// / segments the series needs to be included in) -// -// In addition, for each shard we will need to store the ranges which we have already read commit log -// files for, so that the ReadIndex() call can easily filter commit log files down to those which -// have not already been read by a previous call to ReadData(). -// -// Its safe to cache the series IDs and Tags because we mark them both as NoFinalize() if the caching -// path is enabled. -func (s *commitLogSource) cacheShardData(ns namespace.Metadata, allShardData []shardData) { - nsString := ns.ID().String() - nsCache, ok := s.cachedShardDataByNS[nsString] - if !ok { - nsShardData := &cachedShardData{ - shardData: make([]shardData, len(allShardData)), - } - s.cachedShardDataByNS[nsString] = nsShardData - nsCache = nsShardData - } - - for shard, currShardData := range allShardData { - for _, seriesData := range currShardData.series { - for blockStart := range seriesData.encoders { - // Nil out any references to the encoders (which should be closed already anyways), - // so that they can be GC'd. - seriesData.encoders[blockStart] = nil - } - } - - for shard >= len(nsCache.shardData) { - // Extend the slice if necessary (could happen if different calls to - // ReadData() bootstrap different shards.) - nsCache.shardData = append(nsCache.shardData, shardData{}) - } - - nsCache.shardData[shard].ranges = nsCache.shardData[shard].ranges.AddRanges(currShardData.ranges) - - currSeries := currShardData.series - cachedSeries := nsCache.shardData[shard].series - - // If there are no existing series, just set what we have. - if len(cachedSeries) == 0 { - if currSeries != nil { - nsCache.shardData[shard].series = currSeries - } else { - nsCache.shardData[shard].series = make(map[uint64]metadataAndEncodersByTime) - } - continue - } - - // If there are existing series, then add any new series that we have, and merge block starts. - for uniqueIdx, seriesData := range currSeries { - // If its not already there, just add it - cachedSeriesData, ok := cachedSeries[uniqueIdx] - if ok { - } - if !ok { - cachedSeries[uniqueIdx] = seriesData - continue - } - - // If it is there, merge blockStart times - for blockStart := range seriesData.encoders { - // The existence of a key in the map is indicative of its presence in this case, - // so assigning nil is equivalent to adding an item to a set. This is counter-intuitive, - // but we do it so that we can re-use the existing datastructures that have already been - // allocated by the bootstrapping process, otherwise we'd have to perform millions of - // additional allocations. - cachedSeriesData.encoders[blockStart] = nil - } - } - } -} - func (s *commitLogSource) AvailableIndex( ns namespace.Metadata, shardsTimeRanges result.ShardTimeRanges, @@ -642,40 +1267,26 @@ func (s *commitLogSource) ReadIndex( } var ( - nsCache = s.cachedShardDataByNS[ns.ID().String()] - shardsTimeRangesToReadFromDisk = shardsTimeRanges.Copy() + fsOpts = s.opts.CommitLogOptions().FilesystemOptions() + filePathPrefix = fsOpts.FilePathPrefix() ) - if nsCache != nil { - cachedShardsTimeRanges := result.ShardTimeRanges{} - for shard, shardData := range nsCache.shardData { - cachedShardsTimeRanges[uint32(shard)] = shardData.ranges - } - shardsTimeRangesToReadFromDisk.Subtract(cachedShardsTimeRanges) - } - // Setup predicates for skipping files / series at iterator and reader level. - readCommitLogPredicate := newReadCommitLogPredicate( - ns, shardsTimeRangesToReadFromDisk, s.opts, s.inspection) - readSeriesPredicate := newReadSeriesPredicate(ns) - iterOpts := commitlog.IteratorOpts{ - CommitLogOptions: s.opts.CommitLogOptions(), - FileFilterPredicate: readCommitLogPredicate, - SeriesFilterPredicate: readSeriesPredicate, - } - - // Create the commitlog iterator - iter, err := s.newIteratorFn(iterOpts) + // Determine which snapshot files are available. + snapshotFilesByShard, err := s.snapshotFilesByShard( + ns.ID(), filePathPrefix, shardsTimeRanges) if err != nil { - return nil, fmt.Errorf("unable to create commit log iterator: %v", err) + return nil, err } - defer iter.Close() - highestShard := s.findHighestShard(shardsTimeRanges) - // +1 so we can use the shard number as an index throughout without constantly - // remembering to subtract 1 to convert to zero-based indexing. - numShards := highestShard + 1 - // Convert the map to a slice for faster lookups - bootstrapRangesByShard := make([]xtime.Ranges, numShards) + var ( + highestShard = s.findHighestShard(shardsTimeRanges) + // +1 so we can use the shard number as an index throughout without constantly + // remembering to subtract 1 to convert to zero-based indexing. + numShards = highestShard + 1 + // Convert the map to a slice for faster lookups + bootstrapRangesByShard = make([]xtime.Ranges, numShards) + ) + for shard, ranges := range shardsTimeRanges { bootstrapRangesByShard[shard] = ranges } @@ -686,10 +1297,55 @@ func (s *commitLogSource) ReadIndex( indexOptions = ns.Options().IndexOptions() indexBlockSize = indexOptions.BlockSize() resultOptions = s.opts.ResultOptions() + blockSize = ns.Options().RetentionOptions().BlockSize() + ) + + // Determine which commit log files we need to read based on which snapshot + // snapshot files are available. + readCommitLogPredicate, mostRecentCompleteSnapshotByBlockShard, err := s.newReadCommitLogPredBasedOnAvailableSnapshotFiles( + ns, shardsTimeRanges, snapshotFilesByShard) + if err != nil { + return nil, err + } + + var ( + readSeriesPredicate = newReadSeriesPredicate(ns) + iterOpts = commitlog.IteratorOpts{ + CommitLogOptions: s.opts.CommitLogOptions(), + FileFilterPredicate: readCommitLogPredicate, + SeriesFilterPredicate: readSeriesPredicate, + } ) - // Start by reading all the commit log files that we couldn't eliminate due to the - // cached metadata. + // Start by reading any available snapshot files. + for shard, tr := range shardsTimeRanges { + shardResult, err := s.bootstrapShardSnapshots( + ns.ID(), shard, true, tr, blockSize, snapshotFilesByShard[shard], + mostRecentCompleteSnapshotByBlockShard) + if err != nil { + return nil, err + } + + // Bootstrap any series we got from the snapshot files into the index. + for _, val := range shardResult.AllSeries().Iter() { + id := val.Key() + val := val.Value() + for block := range val.Blocks.AllBlocks() { + s.maybeAddToIndex( + id, val.Tags, shard, highestShard, block.ToTime(), bootstrapRangesByShard, + indexResults, indexOptions, indexBlockSize, resultOptions) + } + } + } + + // Next, read all of the data from the commit log files that wasn't covered + // by the snapshot files. + iter, err := s.newIteratorFn(iterOpts) + if err != nil { + return nil, fmt.Errorf("unable to create commit log iterator: %v", err) + } + defer iter.Close() + for iter.Next() { series, dp, _, _ := iter.Current() @@ -698,19 +1354,6 @@ func (s *commitLogSource) ReadIndex( indexResults, indexOptions, indexBlockSize, resultOptions) } - // Add in all the data that was cached by a previous run of ReadData() (if any). - if nsCache != nil { - for shard, shardData := range nsCache.shardData { - for _, series := range shardData.series { - for dataBlockStart := range series.encoders { - s.maybeAddToIndex( - series.id, series.tags, uint32(shard), highestShard, dataBlockStart.ToTime(), bootstrapRangesByShard, - indexResults, indexOptions, indexBlockSize, resultOptions) - } - } - } - } - // If all successful then we mark each index block as fulfilled for _, block := range indexResult.IndexResults() { blockRange := xtime.Range{ @@ -769,10 +1412,8 @@ func (s commitLogSource) maybeAddToIndex( return nil } - // We can use the NoClone variant here because the cached IDs/Tags are marked NoFinalize - // by the ReadData() path when it reads from the commitlog files, and the IDs/Tags read - // from the commit log files by the ReadIndex() method won't be finalized because this - // code path doesn't finalize them. + // We can use the NoClone variant here because the IDs/Tags read from the commit log files + // by the ReadIndex() method won't be finalized because this code path doesn't finalize them. d, err := convert.FromMetricNoClone(id, tags) if err != nil { return err @@ -782,50 +1423,6 @@ func (s commitLogSource) maybeAddToIndex( return err } -func (s commitLogSource) shouldCacheSeriesMetadata(runOpts bootstrap.RunOptions, nsMeta namespace.Metadata) bool { - return runOpts.CacheSeriesMetadata() && nsMeta.Options().IndexOptions().Enabled() -} - -func newReadCommitLogPredicate( - ns namespace.Metadata, - shardsTimeRanges result.ShardTimeRanges, - opts Options, - inspection fs.Inspection, -) commitlog.FileFilterPredicate { - // Minimum and maximum times for which we want to bootstrap - shardMin, shardMax := shardsTimeRanges.MinMax() - shardRange := xtime.Range{ - Start: shardMin, - End: shardMax, - } - - // How far into the past or future a commitlog might contain a write for a - // previous or future block - bufferPast := ns.Options().RetentionOptions().BufferPast() - bufferFuture := ns.Options().RetentionOptions().BufferFuture() - - // commitlogFilesPresentBeforeStart is a set of all the commitlog files that were - // on disk before the node started. - commitlogFilesPresentBeforeStart := inspection.CommitLogFilesSet() - - return func(name string, entryTime time.Time, entryDuration time.Duration) bool { - _, ok := commitlogFilesPresentBeforeStart[name] - if !ok { - // If the file wasn't on disk before the node started then it only contains - // writes that are already in memory (and in-fact the file may be actively - // being written to.) - return false - } - - // If there is any amount of overlap between the commitlog range and the - // shardRange then we need to read the commitlog file - return xtime.Range{ - Start: entryTime.Add(-bufferPast), - End: entryTime.Add(entryDuration).Add(bufferFuture), - }.Overlaps(shardRange) - } -} - func newReadSeriesPredicate(ns namespace.Metadata) commitlog.SeriesFilterPredicate { nsID := ns.ID() return func(id ident.ID, namespace ident.ID) bool { @@ -834,7 +1431,7 @@ func newReadSeriesPredicate(ns namespace.Metadata) commitlog.SeriesFilterPredica } type shardData struct { - series map[uint64]metadataAndEncodersByTime + series *Map ranges xtime.Ranges } @@ -843,7 +1440,7 @@ type metadataAndEncodersByTime struct { tags ident.Tags // int64 instead of time.Time because there is an optimized map access pattern // for i64's - encoders map[xtime.UnixNano]encoders + encoders map[xtime.UnixNano][]encoder } // encoderArg contains all the information a worker go-routine needs to encode @@ -856,22 +1453,33 @@ type encoderArg struct { blockStart time.Time } -type encoders []encoder - type ioReaders []xio.SegmentReader -func (e encoders) newReaders() ioReaders { - readers := make(ioReaders, len(e)) - for i := range e { - readers[i] = e[i].enc.Stream() +func newIOReadersFromEncodersAndBlock( + segmentReaderPool xio.SegmentReaderPool, + encoders []encoder, + dbBlock block.DatabaseBlock, +) (ioReaders, error) { + numReaders := len(encoders) + if dbBlock != nil { + numReaders++ } - return readers -} -func (e encoders) close() { - for i := range e { - e[i].enc.Close() + readers := make(ioReaders, 0, numReaders) + if dbBlock != nil { + blockSegment := dbBlock.Discard() + blockReader := segmentReaderPool.Get() + blockReader.Reset(blockSegment) + readers = append(readers, blockReader) } + + for _, encoder := range encoders { + segmentReader := segmentReaderPool.Get() + segmentReader.Reset(encoder.enc.Discard()) + readers = append(readers, segmentReader) + } + + return readers, nil } func (ir ioReaders) close() { @@ -879,7 +1487,3 @@ func (ir ioReaders) close() { r.(xio.SegmentReader).Finalize() } } - -type cachedShardData struct { - shardData []shardData -} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go index f480a1ac7d..1ca9328e3f 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_data_test.go @@ -28,8 +28,10 @@ import ( "testing" "time" + "github.com/m3db/m3db/src/dbnode/digest" "github.com/m3db/m3db/src/dbnode/encoding" "github.com/m3db/m3db/src/dbnode/encoding/m3tsz" + "github.com/m3db/m3db/src/dbnode/persist" "github.com/m3db/m3db/src/dbnode/persist/fs" "github.com/m3db/m3db/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3db/src/dbnode/storage/block" @@ -37,9 +39,12 @@ import ( "github.com/m3db/m3db/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3db/src/dbnode/storage/namespace" "github.com/m3db/m3db/src/dbnode/ts" + "github.com/m3db/m3x/checked" "github.com/m3db/m3x/ident" + "github.com/m3db/m3x/pool" xtime "github.com/m3db/m3x/time" + "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" ) @@ -126,14 +131,14 @@ func TestReadOrderedValues(t *testing.T) { blockSize := md.Options().RetentionOptions().BlockSize() now := time.Now() start := now.Truncate(blockSize).Add(-blockSize) - end := now + end := now.Truncate(blockSize) // Request a little after the start of data, because always reading full blocks // it should return the entire block beginning from "start" require.True(t, blockSize >= minCommitLogRetention) ranges := xtime.Ranges{} ranges = ranges.AddRange(xtime.Range{ - Start: start.Add(time.Minute), + Start: start, End: end, }) @@ -159,7 +164,8 @@ func TestReadOrderedValues(t *testing.T) { require.NotNil(t, res) require.Equal(t, 2, len(res.ShardResults())) require.Equal(t, 0, len(res.Unfulfilled())) - require.NoError(t, verifyShardResultsAreCorrect(values[:4], res.ShardResults(), opts)) + require.NoError(t, verifyShardResultsAreCorrect( + values[:4], blockSize, res.ShardResults(), opts)) } func TestReadUnorderedValues(t *testing.T) { @@ -170,14 +176,14 @@ func TestReadUnorderedValues(t *testing.T) { blockSize := md.Options().RetentionOptions().BlockSize() now := time.Now() start := now.Truncate(blockSize).Add(-blockSize) - end := now + end := now.Truncate(blockSize) // Request a little after the start of data, because always reading full blocks // it should return the entire block beginning from "start" require.True(t, blockSize >= minCommitLogRetention) ranges := xtime.Ranges{} ranges = ranges.AddRange(xtime.Range{ - Start: start.Add(time.Minute), + Start: start, End: end, }) @@ -200,7 +206,54 @@ func TestReadUnorderedValues(t *testing.T) { require.NotNil(t, res) require.Equal(t, 1, len(res.ShardResults())) require.Equal(t, 0, len(res.Unfulfilled())) - require.NoError(t, verifyShardResultsAreCorrect(values, res.ShardResults(), opts)) + require.NoError(t, verifyShardResultsAreCorrect( + values, blockSize, res.ShardResults(), opts)) +} + +// TestReadHandlesDifferentSeriesWithIdenticalUniqueIndex was added as a regression test to make +// sure that the commit log bootstrapper does not make any assumptions about series having a unique +// unique index because that only holds for the duration that an M3DB node is on, but commit log +// files can span multiple M3DB processes which means that unique indexes could be re-used for multiple +// different series. +func TestReadHandlesDifferentSeriesWithIdenticalUniqueIndex(t *testing.T) { + opts := testOptions() + md := testNsMetadata(t) + src := newCommitLogSource(opts, fs.Inspection{}).(*commitLogSource) + + blockSize := md.Options().RetentionOptions().BlockSize() + now := time.Now() + start := now.Truncate(blockSize).Add(-blockSize) + end := now.Truncate(blockSize) + + require.True(t, blockSize >= minCommitLogRetention) + ranges := xtime.Ranges{} + ranges = ranges.AddRange(xtime.Range{ + Start: start, + End: end, + }) + + // All series need to be in the same shard to exercise the regression. + foo := commitlog.Series{ + Namespace: testNamespaceID, Shard: 0, ID: ident.StringID("foo"), UniqueIndex: 0} + bar := commitlog.Series{ + Namespace: testNamespaceID, Shard: 0, ID: ident.StringID("bar"), UniqueIndex: 0} + + values := []testValue{ + {foo, start, 1.0, xtime.Second, nil}, + {bar, start, 2.0, xtime.Second, nil}, + } + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { + return newTestCommitLogIterator(values, nil), nil + } + + targetRanges := result.ShardTimeRanges{0: ranges, 1: ranges} + res, err := src.ReadData(md, targetRanges, testDefaultRunOpts) + require.NoError(t, err) + require.NotNil(t, res) + require.Equal(t, 1, len(res.ShardResults())) + require.Equal(t, 0, len(res.Unfulfilled())) + require.NoError(t, verifyShardResultsAreCorrect( + values, blockSize, res.ShardResults(), opts)) } func TestReadTrimsToRanges(t *testing.T) { @@ -211,14 +264,14 @@ func TestReadTrimsToRanges(t *testing.T) { blockSize := md.Options().RetentionOptions().BlockSize() now := time.Now() start := now.Truncate(blockSize).Add(-blockSize) - end := now + end := now.Truncate(blockSize) // Request a little after the start of data, because always reading full blocks // it should return the entire block beginning from "start" require.True(t, blockSize >= minCommitLogRetention) ranges := xtime.Ranges{} ranges = ranges.AddRange(xtime.Range{ - Start: start.Add(time.Minute), + Start: start, End: end, }) @@ -240,143 +293,116 @@ func TestReadTrimsToRanges(t *testing.T) { require.NotNil(t, res) require.Equal(t, 1, len(res.ShardResults())) require.Equal(t, 0, len(res.Unfulfilled())) - require.NoError(t, verifyShardResultsAreCorrect(values[1:3], res.ShardResults(), opts)) + require.NoError(t, verifyShardResultsAreCorrect( + values[1:3], blockSize, res.ShardResults(), opts)) } -type predCommitlogFile struct { - name string - start time.Time -} +func TestItMergesSnapshotsAndCommitLogs(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + var ( + opts = testOptions() + md = testNsMetadata(t) + src = newCommitLogSource(opts, fs.Inspection{}).(*commitLogSource) + blockSize = md.Options().RetentionOptions().BlockSize() + now = time.Now() + start = now.Truncate(blockSize).Add(-blockSize) + end = now.Truncate(blockSize) + ranges = xtime.Ranges{} + + foo = commitlog.Series{Namespace: testNamespaceID, Shard: 0, ID: ident.StringID("foo")} + commitLogValues = []testValue{ + {foo, start.Add(2 * time.Minute), 1.0, xtime.Nanosecond, nil}, + {foo, start.Add(3 * time.Minute), 2.0, xtime.Nanosecond, nil}, + {foo, start.Add(4 * time.Minute), 3.0, xtime.Nanosecond, nil}, + + // Should not be present + {foo, end.Truncate(blockSize).Add(blockSize).Add(time.Nanosecond), 4.0, xtime.Nanosecond, nil}, + } + ) -func TestNewReadCommitLogPredicate(t *testing.T) { - testFilename := "test-file" - testCommitlogFile := predCommitlogFile{ - name: testFilename, - start: time.Time{}, - } - testCommitlogFiles := []predCommitlogFile{testCommitlogFile} - testInspection := fs.Inspection{ - SortedCommitLogFiles: []string{testFilename}, - } + // Request a little after the start of data, because always reading full blocks it + // should return the entire block beginning from "start". + require.True(t, blockSize >= minCommitLogRetention) - testCases := []struct { - title string - commitlogFiles []predCommitlogFile - shardTimeRanges []xtime.Range - bufferPast time.Duration - bufferFuture time.Duration - blockSize time.Duration - inspection fs.Inspection - expectedPredicateResults []bool - }{ - { - title: "Test no overlap", - commitlogFiles: testCommitlogFiles, - shardTimeRanges: []xtime.Range{ - xtime.Range{ - Start: time.Time{}.Add(2 * time.Hour), - End: time.Time{}.Add(3 * time.Hour), - }, - }, - bufferPast: 5 * time.Minute, - bufferFuture: 10 * time.Minute, - blockSize: time.Hour, - inspection: testInspection, - expectedPredicateResults: []bool{false}, - }, - { - title: "Test overlap", - commitlogFiles: testCommitlogFiles, - shardTimeRanges: []xtime.Range{ - xtime.Range{ - Start: time.Time{}, - End: time.Time{}.Add(time.Hour), - }, - }, - bufferPast: 5 * time.Minute, - bufferFuture: 10 * time.Minute, - blockSize: time.Hour, - inspection: testInspection, - expectedPredicateResults: []bool{true}, - }, - { - title: "Test overlap bufferFuture", - commitlogFiles: testCommitlogFiles, - shardTimeRanges: []xtime.Range{ - xtime.Range{ - Start: time.Time{}.Add(1*time.Hour + 1*time.Minute), - End: time.Time{}.Add(2 * time.Hour), - }, - }, - bufferPast: 5 * time.Minute, - bufferFuture: 10 * time.Minute, - blockSize: time.Hour, - inspection: testInspection, - expectedPredicateResults: []bool{true}, - }, - { - title: "Test overlap bufferPast", - commitlogFiles: testCommitlogFiles, - shardTimeRanges: []xtime.Range{ - xtime.Range{ - Start: time.Time{}.Add(-1 * time.Hour), - End: time.Time{}.Add(-1 * time.Minute), - }, - }, - bufferPast: 5 * time.Minute, - bufferFuture: 10 * time.Minute, - blockSize: time.Hour, - inspection: testInspection, - expectedPredicateResults: []bool{true}, - }, - { - title: "Test file not in inspection", - commitlogFiles: testCommitlogFiles, - shardTimeRanges: []xtime.Range{ - xtime.Range{ - Start: time.Time{}, - End: time.Time{}.Add(time.Hour), + ranges = ranges.AddRange(xtime.Range{ + Start: start, + End: end, + }) + + src.newIteratorFn = func(_ commitlog.IteratorOpts) (commitlog.Iterator, error) { + return newTestCommitLogIterator(commitLogValues, nil), nil + } + src.snapshotFilesFn = func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error) { + return fs.FileSetFilesSlice{ + fs.FileSetFile{ + ID: fs.FileSetFileIdentifier{ + Namespace: namespace, + BlockStart: start, + Shard: shard, + VolumeIndex: 0, }, + AbsoluteFilepaths: []string{"checkpoint"}, + CachedSnapshotTime: start.Add(time.Minute), }, - bufferPast: 5 * time.Minute, - bufferFuture: 10 * time.Minute, - blockSize: time.Hour, - inspection: fs.Inspection{}, - expectedPredicateResults: []bool{false}, - }, + }, nil } - for _, tc := range testCases { - t.Run(tc.title, func(t *testing.T) { - // Setup opts with specified blocksize - opts := testOptions() - commitLogOptions := opts.CommitLogOptions().SetBlockSize(tc.blockSize) - opts = opts.SetCommitLogOptions(commitLogOptions) - - // Setup namespace with specified bufferPast / bufferFuture - nsOptions := namespace.NewOptions() - retentionOptions := nsOptions.RetentionOptions(). - SetBufferPast(tc.bufferPast). - SetBufferFuture(tc.bufferFuture) - nsOptions = nsOptions.SetRetentionOptions(retentionOptions) - ns, err := namespace.NewMetadata(testNamespaceID, nsOptions) - require.NoError(t, err) - - // Set up shardTimeRanges with specified ranges - shardTimeRanges := result.ShardTimeRanges{} - for i, xrange := range tc.shardTimeRanges { - ranges := xtime.NewRanges(xrange) - shardTimeRanges[uint32(i)] = ranges - } + mockReader := fs.NewMockDataFileSetReader(ctrl) + mockReader.EXPECT().Open(fs.ReaderOpenOptionsMatcher{ + ID: fs.FileSetFileIdentifier{ + Namespace: testNamespaceID, + BlockStart: start, + Shard: 0, + VolumeIndex: 0, + }, + FileSetType: persist.FileSetSnapshotType, + }).Return(nil).AnyTimes() + mockReader.EXPECT().Entries().Return(1).AnyTimes() - // Instantiate and test predicate - predicate := newReadCommitLogPredicate(ns, shardTimeRanges, opts, tc.inspection) - for i, cl := range tc.commitlogFiles { - predicateResult := predicate(cl.name, cl.start, tc.blockSize) - require.Equal(t, tc.expectedPredicateResults[i], predicateResult) - } - }) + snapshotValues := []testValue{ + {foo, start.Add(1 * time.Minute), 1.0, xtime.Nanosecond, nil}, } + + encoder := m3tsz.NewEncoder(snapshotValues[0].t, nil, true, nil) + for _, value := range snapshotValues { + dp := ts.Datapoint{ + Timestamp: value.t, + Value: value.v, + } + encoder.Encode(dp, value.u, value.a) + } + reader := encoder.Stream() + seg, err := reader.Segment() + require.NoError(t, err) + bytes := make([]byte, seg.Len()) + _, err = reader.Read(bytes) + require.NoError(t, err) + mockReader.EXPECT().Read().Return( + foo.ID, + ident.EmptyTagIterator, + checked.NewBytes(bytes, nil), + digest.Checksum(bytes), + nil, + ) + mockReader.EXPECT().Read().Return(nil, nil, nil, uint32(0), io.EOF) + + src.newReaderFn = func(bytesPool pool.CheckedBytesPool, opts fs.Options) (fs.DataFileSetReader, error) { + return mockReader, nil + } + + targetRanges := result.ShardTimeRanges{0: ranges} + res, err := src.ReadData(md, targetRanges, testDefaultRunOpts) + require.NoError(t, err) + require.NotNil(t, res) + require.Equal(t, 1, len(res.ShardResults())) + require.Equal(t, 0, len(res.Unfulfilled())) + expectedValues := append([]testValue{}, snapshotValues...) + expectedValues = append(expectedValues, commitLogValues[0:3]...) + + require.NoError(t, verifyShardResultsAreCorrect( + expectedValues, blockSize, res.ShardResults(), opts)) } type testValue struct { @@ -398,6 +424,7 @@ type seriesShardResult struct { func verifyShardResultsAreCorrect( values []testValue, + blockSize time.Duration, actual result.ShardResults, opts Options, ) error { @@ -410,7 +437,7 @@ func verifyShardResultsAreCorrect( "shard result is nil, but expected: %d values", len(values)) } // First create what result should be constructed for test values - expected, err := createExpectedShardResult(values, actual, opts) + expected, err := createExpectedShardResult(values, blockSize, actual, opts) if err != nil { return err } @@ -439,11 +466,11 @@ func verifyShardResultsAreCorrect( func createExpectedShardResult( values []testValue, + blockSize time.Duration, actual result.ShardResults, opts Options, ) (result.ShardResults, error) { bopts := opts.ResultOptions() - blockSize := opts.CommitLogOptions().BlockSize() blopts := bopts.DatabaseBlockOptions() expected := result.ShardResults{} @@ -556,8 +583,8 @@ func verifyShardResultsAreEqual(opts Options, shard uint32, actualResult, expect func verifyBlocksAreEqual(opts Options, expectedAllBlocks, actualAllBlocks map[xtime.UnixNano]block.DatabaseBlock) error { blopts := opts.ResultOptions().DatabaseBlockOptions() - for start, actualBlock := range actualAllBlocks { - expectedBlock, ok := expectedAllBlocks[start] + for start, expectedBlock := range expectedAllBlocks { + actualBlock, ok := actualAllBlocks[start] if !ok { return fmt.Errorf("Expected block for start time: %v", start) } diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go index f330930bf3..32e4ee781f 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_index_test.go @@ -111,7 +111,7 @@ func testBootstrapIndex(t *testing.T, bootstrapDataFirst bool) { {baz, start.Add(2 * dataBlockSize), 1.0, xtime.Second, nil}, {baz, start.Add(2 * dataBlockSize), 2.0, xtime.Second, nil}, {untagged, start.Add(2 * dataBlockSize), 1.0, xtime.Second, nil}, - {outOfRange, start.Add(-blockSize), 1.0, xtime.Second, nil}, + {outOfRange, start.Add(-dataBlockSize), 1.0, xtime.Second, nil}, {shardTooHigh, start.Add(dataBlockSize), 1.0, xtime.Second, nil}, {noShardBootstrapRange, start.Add(dataBlockSize), 1.0, xtime.Second, nil}, {someOtherNamespace, start.Add(dataBlockSize), 1.0, xtime.Second, nil}, diff --git a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go index 0c019c5ce2..c685baca97 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/commitlog/source_prop_test.go @@ -1,3 +1,5 @@ +// +build big +// // Copyright (c) 2017 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy @@ -25,15 +27,21 @@ import ( "io/ioutil" "os" "reflect" + "sort" "sync" "testing" "time" + "github.com/m3db/m3db/src/dbnode/digest" + "github.com/m3db/m3db/src/dbnode/encoding" + "github.com/m3db/m3db/src/dbnode/encoding/m3tsz" + "github.com/m3db/m3db/src/dbnode/persist" "github.com/m3db/m3db/src/dbnode/persist/fs" "github.com/m3db/m3db/src/dbnode/persist/fs/commitlog" "github.com/m3db/m3db/src/dbnode/storage/bootstrap/result" "github.com/m3db/m3db/src/dbnode/storage/namespace" "github.com/m3db/m3db/src/dbnode/ts" + "github.com/m3db/m3x/checked" "github.com/m3db/m3x/context" "github.com/m3db/m3x/ident" xtime "github.com/m3db/m3x/time" @@ -42,19 +50,48 @@ import ( "github.com/leanovate/gopter/gen" "github.com/leanovate/gopter/prop" "github.com/spaolacci/murmur3" + "github.com/stretchr/testify/require" ) const maxShards = 8192 const blockSize = 2 * time.Hour func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { - parameters := gopter.DefaultTestParameters() - parameters.Rng.Seed(123456789) - parameters.MinSuccessfulTests = 40 - props := gopter.NewProperties(parameters) + var ( + parameters = gopter.DefaultTestParameters() + seed = time.Now().UnixNano() + props = gopter.NewProperties(parameters) + reporter = gopter.NewFormatedReporter(true, 160, os.Stdout) + startTime = time.Now().Truncate(blockSize) + + nsOpts = namespace.NewOptions().SetIndexOptions( + namespace.NewOptions().IndexOptions().SetEnabled(true), + ) + ) + parameters.MinSuccessfulTests = 80 + parameters.Rng.Seed(seed) + nsMeta, err := namespace.NewMetadata(testNamespaceID, nsOpts) + require.NoError(t, err) props.Property("Commitlog bootstrapping properly bootstraps the entire commitlog", prop.ForAll( func(input propTestInput) (bool, error) { + if !input.commitLogExists { + // If there is no commit log then we need to make sure + // snapshot exists, regardless of what the prop test generated. + input.snapshotExists = true + } + + var ( + retentionOpts = nsOpts.RetentionOptions(). + SetBufferPast(input.bufferPast). + SetBufferFuture(input.bufferFuture) + nsOpts = nsOpts.SetRetentionOptions(retentionOpts) + ) + nsMeta, err := namespace.NewMetadata(testNamespaceID, nsOpts) + if err != nil { + return false, err + } + // Make sure we have a new directory for each test run dir, err := ioutil.TempDir("", "foo") if err != nil { @@ -64,34 +101,185 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { os.RemoveAll(dir) }() - // Configure the commitlog to use the test directory and set the blocksize - fsOpts := fs.NewOptions().SetFilePathPrefix(dir) - commitLogOpts := commitlog.NewOptions(). - SetBlockSize(2 * time.Hour). - SetFilesystemOptions(fsOpts) - bootstrapOpts := testOptions(). - SetCommitLogOptions(commitLogOpts) + var ( + // This is the earliest system time that we would be willing to write + // a datapoint for, so start with that and let the write themselves + // continue to increment the current time. + currentTime = startTime.Add(-input.bufferFuture) + lock = sync.RWMutex{} + + nowFn = func() time.Time { + lock.RLock() + curr := currentTime + lock.RUnlock() + return curr + } + ) - // Instantiate commitlog - log, err := commitlog.NewCommitLog(commitLogOpts) - if err != nil { - return false, err - } - err = log.Open() + commitLogBlockSize := 1 * time.Minute + require.True(t, commitLogBlockSize < blockSize) + + var ( + fsOpts = fs.NewOptions(). + SetFilePathPrefix(dir) + commitLogOpts = commitlog.NewOptions(). + SetBlockSize(blockSize). + SetFilesystemOptions(fsOpts). + SetBlockSize(commitLogBlockSize). + SetStrategy(commitlog.StrategyWriteBehind). + SetFlushInterval(time.Millisecond). + SetClockOptions(commitlog.NewOptions().ClockOptions().SetNowFn(nowFn)) + bootstrapOpts = testOptions().SetCommitLogOptions(commitLogOpts) + + start = input.currentTime.Truncate(blockSize) + ) + + writer, err := fs.NewWriter(fsOpts) if err != nil { return false, err } - // Write all the datapoints to the commitlog + orderedWritesBySeries := map[string][]generatedWrite{} for _, write := range input.writes { - err := log.Write(context.NewContext(), write.series, write.datapoint, write.unit, write.annotation) + id := write.series.ID + writesForSeries, ok := orderedWritesBySeries[id.String()] + if !ok { + writesForSeries = []generatedWrite{} + } + writesForSeries = append(writesForSeries, write) + orderedWritesBySeries[id.String()] = writesForSeries + } + + for _, writesForSeries := range orderedWritesBySeries { + sort.Slice(writesForSeries, func(i, j int) bool { + return writesForSeries[i].datapoint.Timestamp.Before(writesForSeries[j].datapoint.Timestamp) + }) + } + + if input.snapshotExists { + compressedWritesByShards := map[uint32]map[string][]byte{} + for seriesID, writesForSeries := range orderedWritesBySeries { + shard := hashIDToShard(ident.StringID(seriesID)) + encodersBySeries, ok := compressedWritesByShards[shard] + if !ok { + encodersBySeries = map[string][]byte{} + compressedWritesByShards[shard] = encodersBySeries + } + + encoder := m3tsz.NewEncoder(writesForSeries[0].datapoint.Timestamp, nil, true, encoding.NewOptions()) + for _, value := range writesForSeries { + // Only include datapoints that are before or during the snapshot time to ensure that we + // properly bootstrap from both snapshot files and commit logs and merge them together. + // Note that if the commit log does not exist we ignore the snapshot time because we need + // the snapshot to include all the data. + if !input.commitLogExists || + value.arrivedAt.Before(input.snapshotTime) || + value.arrivedAt.Equal(input.snapshotTime) { + err := encoder.Encode(value.datapoint, value.unit, value.annotation) + if err != nil { + return false, err + } + } + } + + reader := encoder.Stream() + if reader != nil { + seg, err := reader.Segment() + if err != nil { + return false, err + } + + bytes := make([]byte, seg.Len()) + _, err = reader.Read(bytes) + if err != nil { + return false, err + } + encodersBySeries[seriesID] = bytes + } + compressedWritesByShards[shard] = encodersBySeries + } + + for shard, seriesForShard := range compressedWritesByShards { + err = writer.Open(fs.DataWriterOpenOptions{ + Identifier: fs.FileSetFileIdentifier{ + Namespace: nsMeta.ID(), + BlockStart: start, + Shard: shard, + VolumeIndex: 0, + }, + BlockSize: blockSize, + FileSetType: persist.FileSetSnapshotType, + Snapshot: fs.DataWriterSnapshotOptions{ + SnapshotTime: input.snapshotTime, + }, + }) + + if err != nil { + return false, err + } + + for seriesID, data := range seriesForShard { + checkedBytes := checked.NewBytes(data, nil) + checkedBytes.IncRef() + tags := orderedWritesBySeries[seriesID][0].series.Tags + writer.Write(ident.StringID(seriesID), tags, checkedBytes, digest.Checksum(data)) + } + + err = writer.Close() + if err != nil { + return false, err + } + } + } + + if input.commitLogExists { + writesCh := make(chan struct{}, 5) + go func() { + for range writesCh { + lock.Lock() + currentTime = currentTime.Add(time.Millisecond) + lock.Unlock() + } + }() + + // Instantiate commitlog + log, err := commitlog.NewCommitLog(commitLogOpts) + if err != nil { + return false, err + } + err = log.Open() + if err != nil { + return false, err + } + + sort.Slice(input.writes, func(i, j int) bool { + return input.writes[i].arrivedAt.Before(input.writes[j].arrivedAt) + }) + + // Write all the datapoints to the commitlog + for _, write := range input.writes { + // Only write datapoints that are not in the snapshots. + if input.snapshotExists && + !write.arrivedAt.After(input.snapshotTime) { + continue + } + + lock.Lock() + currentTime = write.arrivedAt + lock.Unlock() + + err := log.Write(context.NewContext(), write.series, write.datapoint, write.unit, write.annotation) + if err != nil { + return false, err + } + writesCh <- struct{}{} + } + close(writesCh) + + err = log.Close() if err != nil { return false, err } - } - err = log.Close() - if err != nil { - return false, err } // Instantiate a commitlog source @@ -109,16 +297,6 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { } // Determine time range to bootstrap - nsOpts := namespace.NewOptions() - nsOpts = nsOpts.SetIndexOptions( - nsOpts.IndexOptions().SetEnabled(true), - ) - md, err := namespace.NewMetadata(testNamespaceID, nsOpts) - if err != nil { - return false, err - } - blockSize := md.Options().RetentionOptions().BlockSize() - start := input.currentTime.Truncate(blockSize) end := input.currentTime.Add(blockSize) ranges := xtime.Ranges{} ranges = ranges.AddRange(xtime.Range{ @@ -140,8 +318,8 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { } // Perform the bootstrap - runOpts := testDefaultRunOpts.SetCacheSeriesMetadata(input.shouldCacheSeriesMetadata) - dataResult, err := source.BootstrapData(md, shardTimeRanges, runOpts) + runOpts := testDefaultRunOpts + dataResult, err := source.BootstrapData(nsMeta, shardTimeRanges, runOpts) if err != nil { return false, err } @@ -152,38 +330,60 @@ func TestCommitLogSourcePropCorrectlyBootstrapsFromCommitlog(t *testing.T) { values = append(values, testValue{write.series, write.datapoint.Timestamp, write.datapoint.Value, write.unit, write.annotation}) } - err = verifyShardResultsAreCorrect(values, dataResult.ShardResults(), bootstrapOpts) + if !dataResult.Unfulfilled().IsEmpty() { + return false, fmt.Errorf( + "data result unfulfilled should be empty but was: %s", + dataResult.Unfulfilled().String(), + ) + } + err = verifyShardResultsAreCorrect(values, blockSize, dataResult.ShardResults(), bootstrapOpts) if err != nil { return false, err } - indexResult, err := source.BootstrapIndex(md, shardTimeRanges, testDefaultRunOpts) + indexResult, err := source.BootstrapIndex(nsMeta, shardTimeRanges, testDefaultRunOpts) if err != nil { return false, err } - indexBlockSize := md.Options().IndexOptions().BlockSize() + indexBlockSize := nsMeta.Options().IndexOptions().BlockSize() err = verifyIndexResultsAreCorrect( values, map[string]struct{}{}, indexResult.IndexResults(), indexBlockSize) if err != nil { return false, err } + if !indexResult.Unfulfilled().IsEmpty() { + return false, fmt.Errorf( + "index result unfulfilled should be empty but was: %s", + indexResult.Unfulfilled().String(), + ) + } + return true, nil }, - genPropTestInputs(testNamespaceID.String()), + genPropTestInputs(nsMeta, startTime), )) - props.TestingRun(t) + if !props.Run(reporter) { + t.Errorf("failed with initial seed: %d and startTime: %d", seed, startTime.UnixNano()) + } } type propTestInput struct { - currentTime time.Time - writes []generatedWrite - shouldCacheSeriesMetadata bool + currentTime time.Time + snapshotTime time.Time + snapshotExists bool + commitLogExists bool + bufferPast time.Duration + bufferFuture time.Duration + writes []generatedWrite } type generatedWrite struct { + // arrivedAt is used to simulate out-of-order writes which arrive somewhere + // between time.Now().Add(-bufferFuture) and time.Now().Add(bufferPast). + arrivedAt time.Time series commitlog.Series datapoint ts.Datapoint unit xtime.Unit @@ -194,39 +394,76 @@ func (w generatedWrite) String() string { return fmt.Sprintf("ID = %v, Datapoint = %+v", w.series.ID.String(), w.datapoint) } -func genPropTestInputs(ns string) gopter.Gen { +func genPropTestInputs(nsMeta namespace.Metadata, blockStart time.Time) gopter.Gen { curriedGenPropTestInput := func(input interface{}) gopter.Gen { - inputs := input.([]interface{}) - start := inputs[0].(time.Time) - numDatapoints := inputs[1].(int) - shouldCacheSeriesMetadata := inputs[2].(bool) - return genPropTestInput(start, numDatapoints, shouldCacheSeriesMetadata, ns) + var ( + inputs = input.([]interface{}) + snapshotTime = inputs[0].(time.Time) + snapshotExists = inputs[1].(bool) + commitLogExists = inputs[2].(bool) + bufferPast = time.Duration(inputs[3].(int64)) + bufferFuture = time.Duration(inputs[4].(int64)) + numDatapoints = inputs[5].(int) + ) + + return genPropTestInput( + blockStart, bufferPast, bufferFuture, + snapshotTime, snapshotExists, commitLogExists, + numDatapoints, nsMeta.ID().String()) } + return gopter.CombineGens( - // Runs iterations of the test starting 1000 hours in the past/future - gen.TimeRange(time.Now(), blockSize), - // Run iterations of the test with between 0 and 1000 datapoints - gen.IntRange(0, 1000), - // ShouldCacheSeriesMetadata + // Run iterations of the test with the snapshot time set at any point + // between the beginning and end of the block. + gen.TimeRange(blockStart, blockSize), + // SnapshotExists + gen.Bool(), + // CommitLogExists gen.Bool(), + // Run iterations with any bufferPast/bufferFuture between zero and + // the namespace blockSize (distinct from the commitLog blockSize). + gen.Int64Range(0, int64(blockSize)), + gen.Int64Range(0, int64(blockSize)), + // Run iterations of the test with between 0 and 100 datapoints + gen.IntRange(0, 100), ).FlatMap(curriedGenPropTestInput, reflect.TypeOf(propTestInput{})) } -func genPropTestInput(start time.Time, numDatapoints int, shouldCacheSeriesMetadata bool, ns string) gopter.Gen { - return gen.SliceOfN(numDatapoints, genWrite(start, ns)). +func genPropTestInput( + start time.Time, + bufferPast, + bufferFuture time.Duration, + snapshotTime time.Time, + snapshotExists bool, + commitLogExists bool, + numDatapoints int, + ns string, +) gopter.Gen { + return gen.SliceOfN(numDatapoints, genWrite(start, bufferPast, bufferFuture, ns)). Map(func(val interface{}) propTestInput { return propTestInput{ - currentTime: start, - writes: val.([]generatedWrite), - shouldCacheSeriesMetadata: shouldCacheSeriesMetadata, + currentTime: start, + bufferFuture: bufferFuture, + bufferPast: bufferPast, + snapshotTime: snapshotTime, + snapshotExists: snapshotExists, + commitLogExists: commitLogExists, + writes: val.([]generatedWrite), } }) } -func genWrite(start time.Time, ns string) gopter.Gen { +func genWrite(start time.Time, bufferPast, bufferFuture time.Duration, ns string) gopter.Gen { + latestDatapointTime := start.Truncate(blockSize).Add(blockSize).Sub(start) + return gopter.CombineGens( // Identifier gen.Identifier(), + // Only generate writes within the current block period + gen.TimeRange(start, latestDatapointTime), + // Boolean indicating whether we should move offset the datapoint by + // the maximum of eithe bufferPast or bufferFuture. + gen.Bool(), // Tag key/val gen.Identifier(), gen.Identifier(), @@ -234,18 +471,28 @@ func genWrite(start time.Time, ns string) gopter.Gen { // sometimes not include tags to ensure that the commitlog writer/readers can // handle both series that have tags and those that don't. gen.Bool(), - gen.TimeRange(start, 15*time.Minute), // M3TSZ is lossy, so we want to avoid very large numbers with high amounts of precision gen.Float64Range(-9999999, 99999999), ).Map(func(val []interface{}) generatedWrite { - id := val[0].(string) - tagKey := val[1].(string) - tagVal := val[2].(string) - includeTags := val[3].(bool) - t := val[4].(time.Time) - v := val[5].(float64) + var ( + id = val[0].(string) + t = val[1].(time.Time) + a = t + bufferPastOrFuture = val[2].(bool) + tagKey = val[3].(string) + tagVal = val[4].(string) + includeTags = val[5].(bool) + v = val[6].(float64) + ) + + if bufferPastOrFuture { + a = a.Add(-bufferFuture) + } else { + a = a.Add(bufferPast) + } return generatedWrite{ + arrivedAt: a, series: commitlog.Series{ ID: ident.StringID(id), Tags: seriesUniqueTags(id, tagKey, tagVal, includeTags), diff --git a/src/dbnode/storage/index/convert/convert.go b/src/dbnode/storage/index/convert/convert.go index 1920cbf55f..eafb4953c1 100644 --- a/src/dbnode/storage/index/convert/convert.go +++ b/src/dbnode/storage/index/convert/convert.go @@ -173,6 +173,55 @@ func FromMetricIterNoClone(id ident.ID, tags ident.TagIterator) (doc.Document, e }, nil } +// TagsFromTagsIter returns an ident.Tags from a TagIterator. It also tries +// to re-use bytes from the seriesID if they're also present in the tags +// instead of re-allocating them. This requires that the ident.Tags that is +// returned will have the same (or shorter) life time as the seriesID, +// otherwise the operation is unsafe. +func TagsFromTagsIter( + seriesID ident.ID, + iter ident.TagIterator, + idPool ident.Pool, +) (ident.Tags, error) { + var ( + seriesIDBytes = ident.BytesID(seriesID.Bytes()) + tags = idPool.Tags() + ) + + for iter.Next() { + curr := iter.Current() + + var ( + nameBytes, valueBytes = curr.Name.Bytes(), curr.Value.Bytes() + tag ident.Tag + idRef bool + ) + if idx := bytes.Index(seriesIDBytes, nameBytes); idx != -1 { + tag.Name = seriesIDBytes[idx : idx+len(nameBytes)] + idRef = true + } else { + tag.Name = idPool.Clone(curr.Name) + } + if idx := bytes.Index(seriesIDBytes, valueBytes); idx != -1 { + tag.Value = seriesIDBytes[idx : idx+len(valueBytes)] + idRef = true + } else { + tag.Value = idPool.Clone(curr.Value) + } + + if idRef { + tag.NoFinalize() // Taken ref, cannot finalize this + } + + tags.Append(tag) + } + + if err := iter.Err(); err != nil { + return ident.Tags{}, err + } + return tags, nil +} + // NB(prateek): we take an independent copy of the bytes underlying // any ids provided, as we need to maintain the lifecycle of the indexed // bytes separately from the rest of the storage subsystem. diff --git a/src/dbnode/storage/index/convert/convert_test.go b/src/dbnode/storage/index/convert/convert_test.go index 667ac410e3..21aa4e8535 100644 --- a/src/dbnode/storage/index/convert/convert_test.go +++ b/src/dbnode/storage/index/convert/convert_test.go @@ -28,6 +28,7 @@ import ( "github.com/m3db/m3x/pool" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) var ( @@ -127,6 +128,21 @@ func TestToMetricValid(t *testing.T) { ident.MustNewTagStringsIterator("bar", "baz", "some", "others")).Matches(tags)) } +func TestTagsFromTagsIter(t *testing.T) { + var ( + id = ident.StringID("foo") + expectedTags = ident.NewTags( + ident.StringTag("bar", "baz"), + ident.StringTag("foo", "m3"), + ) + tagsIter = ident.NewTagsIterator(expectedTags) + ) + + tags, err := convert.TagsFromTagsIter(id, tagsIter, testOpts.IdentPool) + require.NoError(t, err) + require.True(t, true, expectedTags.Equal(tags)) +} + func TestToMetricInvalidID(t *testing.T) { d := doc.Document{ Fields: []doc.Field{ diff --git a/src/dbnode/storage/series/series.go b/src/dbnode/storage/series/series.go index aa3ffcfe91..5030a32c87 100644 --- a/src/dbnode/storage/series/series.go +++ b/src/dbnode/storage/series/series.go @@ -660,7 +660,22 @@ func (s *dbSeries) Snapshot( return errSeriesNotBootstrapped } - stream, err := s.buffer.Snapshot(ctx, blockStart) + var ( + stream xio.SegmentReader + err error + ) + block, ok := s.blocks.BlockAt(blockStart) + if ok { + // First check if the data has already been rotated out of the buffer + // into an immutable block. If it has, there is no need to check the + // series buffer as the data can't be in both locations. + stream, err = block.Stream(ctx) + } else { + // If the data hasn't been rotated into an immutable block yet, + // then it may be in the series buffer (because its still mutable). + stream, err = s.buffer.Snapshot(ctx, blockStart) + } + if err != nil { return err } diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 8b6b6aaf3b..3f8db2a7e5 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -21,7 +21,6 @@ package storage import ( - "bytes" "container/list" "errors" "fmt" @@ -956,7 +955,7 @@ func (s *dbShard) tryRetrieveWritableSeries(id ident.ID) ( func (s *dbShard) newShardEntry( id ident.ID, - tags ident.TagIterator, + tagsIter ident.TagIterator, ) (*lookup.Entry, error) { series := s.seriesPool.Get() // NB(r): As documented in storage/series.DatabaseSeries the series IDs @@ -967,52 +966,27 @@ func (s *dbShard) newShardEntry( clonedID := ident.BytesID(append([]byte(nil), id.Bytes()...)) clonedID.NoFinalize() - var clonedTags ident.Tags - if tags.Remaining() > 0 { - // Inlining tag creation here so its obvious why we can safely index - // into clonedID below - clonedTags = s.identifierPool.Tags() - tags = tags.Duplicate() - - // Avoid finalizing the tags since series will let them be garbage collected - clonedTags.NoFinalize() - - for tags.Next() { - t := tags.Current() - - // NB(r): Optimization for workloads that embed the tags in the ID is to - // just take a ref to them directly, the cloned ID is frozen by casting to - // a BytesID in newShardEntry - var tag ident.Tag - - nameBytes := t.Name.Bytes() - if idx := bytes.Index(clonedID, nameBytes); idx != -1 { - tag.Name = clonedID[idx : idx+len(nameBytes)] - } else { - tag.Name = s.identifierPool.Clone(t.Name) - } - - valueBytes := t.Value.Bytes() - if idx := bytes.Index(clonedID, valueBytes); idx != -1 { - tag.Value = clonedID[idx : idx+len(valueBytes)] - } else { - tag.Value = s.identifierPool.Clone(t.Value) - } + var ( + tags ident.Tags + err error + ) - clonedTags.Append(tag) - } - err := tags.Err() - tags.Close() + dupTagsIter := tagsIter.Duplicate() + tagsIter = nil // Original tagsIter should not be closed + if dupTagsIter.Remaining() > 0 { + tags, err = convert.TagsFromTagsIter( + clonedID, dupTagsIter, s.identifierPool) + dupTagsIter.Close() if err != nil { return nil, err } - if err := convert.ValidateMetric(clonedID, clonedTags); err != nil { + if err := convert.ValidateMetric(clonedID, tags); err != nil { return nil, err } } - series.Reset(clonedID, clonedTags, s.seriesBlockRetriever, + series.Reset(clonedID, tags, s.seriesBlockRetriever, s.seriesOnRetrieveBlock, s, s.seriesOpts) uniqueIndex := s.increasingIndex.nextIndex() return lookup.NewEntry(series, uniqueIndex), nil diff --git a/src/dbnode/storage/shard_test.go b/src/dbnode/storage/shard_test.go index 4b180f03c0..c1adcfd482 100644 --- a/src/dbnode/storage/shard_test.go +++ b/src/dbnode/storage/shard_test.go @@ -1094,8 +1094,8 @@ func TestShardNewInvalidShardEntry(t *testing.T) { iter := ident.NewMockTagIterator(ctrl) gomock.InOrder( - iter.EXPECT().Remaining().Return(2), iter.EXPECT().Duplicate().Return(iter), + iter.EXPECT().Remaining().Return(2), iter.EXPECT().Next().Return(false), iter.EXPECT().Err().Return(fmt.Errorf("random err")), iter.EXPECT().Close(),