Skip to content

Commit ecfc114

Browse files
committed
base: clean up TempStorageConfig
`TempStorageConfig` is very confusing. It is very hard to follow the exact mechanics of when `InMemory` differs from `StoreSpec.InMemory`, how the paths differ, etc. In this change: - we replace the `StoreSpec` field with the `EncryptionOptions` and the `temp-dirs-record.txt` file path. - we clean up and clarify the initialization code. - we improve the documentation for the `--temp-dir` flag. Epic: none Release note: None
1 parent 37e906c commit ecfc114

24 files changed

+127
-117
lines changed

pkg/base/config.go

Lines changed: 32 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,9 @@ import (
1717
"github.com/cockroachdb/cockroach/pkg/roachpb"
1818
"github.com/cockroachdb/cockroach/pkg/security/username"
1919
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
20+
"github.com/cockroachdb/cockroach/pkg/storage/storageconfig"
2021
"github.com/cockroachdb/cockroach/pkg/util/envutil"
22+
"github.com/cockroachdb/cockroach/pkg/util/log"
2123
"github.com/cockroachdb/cockroach/pkg/util/mon"
2224
"github.com/cockroachdb/cockroach/pkg/util/retry"
2325
)
@@ -904,19 +906,25 @@ type TempStorageConfig struct {
904906
// InMemory specifies whether the temporary storage will remain
905907
// in-memory or occupy a temporary subdirectory on-disk.
906908
InMemory bool
907-
// Path is the filepath of the temporary subdirectory created for
908-
// the temp storage.
909+
// Path is the filepath of the temporary subdirectory created for the temp
910+
// storage. Empty if InMemory is true.
909911
Path string
910912
// Mon will be used by the temp storage to register all its capacity requests.
911913
// It can be used to limit the disk or memory that temp storage is allowed to
912914
// use. If InMemory is set, than this has to be a memory monitor; otherwise it
913915
// has to be a disk monitor.
914916
Mon *mon.BytesMonitor
915-
// Spec stores the StoreSpec this TempStorageConfig will use.
916-
Spec StoreSpec
917+
// Encryption is set if encryption is enabled. We use the same encryption
918+
// options as the store we chose for temp storage.
919+
Encryption *storageconfig.EncryptionOptions
917920
// Settings stores the cluster.Settings this TempStoreConfig will use. Must
918921
// not be nil.
919922
Settings *cluster.Settings
923+
// If set, TempDirsRecordPath is the path to a temp-dirs-record.txt file in
924+
// one of the stores (see server.TempDirsRecordFilename). Used when we create
925+
// a new temporary storage directory for a new shared-process tenant. Empty if
926+
// InMemory is false.
927+
TempDirsRecordPath string
920928
}
921929

922930
// ExternalIODirConfig describes various configuration options pertaining
@@ -948,32 +956,29 @@ type ExternalIODirConfig struct {
948956
EnableNonAdminImplicitAndArbitraryOutbound bool
949957
}
950958

951-
// TempStorageConfigFromEnv creates a TempStorageConfig.
952-
// If parentDir is not specified and the specified store is in-memory,
953-
// then the temp storage will also be in-memory.
954-
func TempStorageConfigFromEnv(
955-
ctx context.Context,
956-
st *cluster.Settings,
957-
useStore StoreSpec,
958-
parentDir string,
959-
maxSizeBytes int64,
960-
) TempStorageConfig {
961-
inMem := parentDir == "" && useStore.InMemory
962-
return newTempStorageConfig(ctx, st, inMem, useStore, maxSizeBytes)
963-
}
964-
965959
// InheritTempStorageConfig creates a new TempStorageConfig using the
966960
// configuration of the given TempStorageConfig. It assumes the given
967961
// TempStorageConfig has been fully initialized.
968962
func InheritTempStorageConfig(
969963
ctx context.Context, st *cluster.Settings, parentConfig TempStorageConfig,
970964
) TempStorageConfig {
971-
return newTempStorageConfig(ctx, st, parentConfig.InMemory, parentConfig.Spec, parentConfig.Mon.Limit())
965+
return NewTempStorageConfig(ctx, st, parentConfig.InMemory, parentConfig.Path, parentConfig.Encryption, parentConfig.Mon.Limit(), parentConfig.TempDirsRecordPath)
972966
}
973967

974-
func newTempStorageConfig(
975-
ctx context.Context, st *cluster.Settings, inMemory bool, useStore StoreSpec, maxSizeBytes int64,
968+
// NewTempStorageConfig creates a new TempStorageConfig.
969+
// The path should be empty iff inMemory is true.
970+
func NewTempStorageConfig(
971+
ctx context.Context,
972+
st *cluster.Settings,
973+
inMemory bool,
974+
path string,
975+
encryption *storageconfig.EncryptionOptions,
976+
maxSizeBytes int64,
977+
tempDirsRecordPath string,
976978
) TempStorageConfig {
979+
if inMemory != (path == "") {
980+
log.Dev.Fatalf(ctx, "inMemory (%t) must be true iff path is empty (%q)", inMemory, path)
981+
}
977982
var monitorName mon.Name
978983
if inMemory {
979984
monitorName = mon.MakeName("in-mem temp storage")
@@ -988,9 +993,11 @@ func newTempStorageConfig(
988993
})
989994
monitor.Start(ctx, nil /* pool */, mon.NewStandaloneBudget(maxSizeBytes))
990995
return TempStorageConfig{
991-
InMemory: inMemory,
992-
Mon: monitor,
993-
Spec: useStore,
994-
Settings: st,
996+
InMemory: inMemory,
997+
Path: path,
998+
Mon: monitor,
999+
Encryption: encryption,
1000+
Settings: st,
1001+
TempDirsRecordPath: tempDirsRecordPath,
9951002
}
9961003
}

pkg/base/test_server_args.go

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -625,7 +625,6 @@ func DefaultTestTempStorageConfigWithSize(
625625
return TempStorageConfig{
626626
InMemory: true,
627627
Mon: monitor,
628-
Spec: DefaultTestStoreSpec,
629628
Settings: st,
630629
}
631630
}

pkg/cli/cliflags/flags.go

Lines changed: 10 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1197,10 +1197,14 @@ Verbose output.`,
11971197
TempDir = FlagInfo{
11981198
Name: "temp-dir",
11991199
Description: `
1200-
The parent directory path where a temporary subdirectory will be created to be used for temporary files.
1201-
This path must exist or the node will not start.
1202-
The temporary subdirectory is used primarily as working memory for distributed computations
1203-
and CSV importing.
1200+
The parent directory path where a temporary subdirectory will be created to be
1201+
used for temporary files. This path must exist or the node will not start.
1202+
1203+
The encryption from one of the stores is used with the temporary directory
1204+
(specifically, the first store that is on-disk and encrypted).
1205+
1206+
The temporary subdirectory is used primarily as working memory for distributed
1207+
computations and CSV importing.
12041208
For example, the following will generate an arbitrary, temporary subdirectory
12051209
"/mnt/ssd01/temp/cockroach-temp<NUMBER>":
12061210
<PRE>
@@ -1209,7 +1213,8 @@ For example, the following will generate an arbitrary, temporary subdirectory
12091213
12101214
</PRE>
12111215
If this flag is unspecified, the temporary subdirectory will be located under
1212-
the root of the first store.`,
1216+
the root of one of the stores (with preference for on-disk, encrypted stores).
1217+
`,
12131218
}
12141219

12151220
ExternalIODir = FlagInfo{

pkg/cli/start.go

Lines changed: 33 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -264,40 +264,44 @@ func initTempStorageConfig(
264264
}
265265
}
266266

267-
// Initialize a base.TempStorageConfig based on first store's spec and
268-
// cli flags.
269-
tempStorageConfig := base.TempStorageConfigFromEnv(
270-
ctx,
271-
st,
272-
useStore,
273-
startCtx.tempDir,
274-
tempStorageMaxSizeBytes,
275-
)
267+
// If all stores are in-memory and no temp dir was specified, the temp
268+
// store will also be in memory. This is a testing scenario.
269+
if startCtx.tempDir == "" && useStore.InMemory {
270+
return base.NewTempStorageConfig(
271+
ctx,
272+
st,
273+
true, // inMem
274+
"", // path
275+
useStore.EncryptionOptions,
276+
tempStorageMaxSizeBytes,
277+
"", // tempDirsRecordPath
278+
), nil
279+
}
276280

277281
// Set temp directory to first store's path if the temp storage is not
278282
// in memory.
279-
tempDir := startCtx.tempDir
280-
if tempDir == "" && !tempStorageConfig.InMemory {
281-
tempDir = useStore.Path
283+
parentDir := startCtx.tempDir
284+
if parentDir == "" {
285+
parentDir = useStore.Path
282286
}
283287

284-
tmpPath, unlockDirFn, err := fs.CreateTempDir(tempDir, server.TempDirPrefix)
288+
tmpPath, unlockDirFn, err := fs.CreateTempDir(parentDir, server.TempDirPrefix)
285289
if err != nil {
286290
return base.TempStorageConfig{}, errors.Wrap(err, "could not create temporary directory for temp storage")
287291
}
288-
tempStorageConfig.Path = tmpPath
289292

293+
recordPath := ""
290294
if useStore.InMemory {
291295
stopper.AddCloser(stop.CloserFn(func() {
292296
unlockDirFn()
293297
// Remove the temp directory directly since there is no record file.
294-
if err := os.RemoveAll(tempStorageConfig.Path); err != nil {
298+
if err := os.RemoveAll(tmpPath); err != nil {
295299
log.Dev.Errorf(ctx, "could not remove temporary store directory: %v", err.Error())
296300
}
297301
}))
298302
} else {
299-
recordPath := filepath.Join(useStore.Path, server.TempDirsRecordFilename)
300-
if err := fs.RecordTempDir(recordPath, tempStorageConfig.Path); err != nil {
303+
recordPath = filepath.Join(useStore.Path, server.TempDirsRecordFilename)
304+
if err := fs.RecordTempDir(recordPath, tmpPath); err != nil {
301305
return base.TempStorageConfig{}, errors.Wrapf(
302306
err,
303307
"could not record temporary directory path to record file: %s",
@@ -312,7 +316,18 @@ func initTempStorageConfig(
312316
}
313317
}))
314318
}
315-
return tempStorageConfig, nil
319+
320+
// Initialize a base.TempStorageConfig based on first store's spec and
321+
// cli flags.
322+
return base.NewTempStorageConfig(
323+
ctx,
324+
st,
325+
false, // inMem
326+
tmpPath,
327+
useStore.EncryptionOptions,
328+
tempStorageMaxSizeBytes,
329+
recordPath,
330+
), nil
316331
}
317332

318333
type newServerFn func(ctx context.Context, serverCfg server.Config, stopper *stop.Stopper) (serverctl.ServerStartupInterface, error)

pkg/kv/kvclient/kvstreamer/results_buffer_test.go

Lines changed: 1 addition & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -35,12 +35,7 @@ func TestInOrderResultsBuffer(t *testing.T) {
3535
ctx := context.Background()
3636
rng, _ := randutil.NewTestRand()
3737
st := cluster.MakeTestingClusterSettings()
38-
tempEngine, _, err := storage.NewTempEngine(
39-
ctx,
40-
base.DefaultTestTempStorageConfig(st),
41-
base.DefaultTestStoreSpec,
42-
nil, /* statsCollector */
43-
)
38+
tempEngine, _, err := storage.NewTempEngine(ctx, base.DefaultTestTempStorageConfig(st), nil /* statsCollector */)
4439
require.NoError(t, err)
4540
defer tempEngine.Close()
4641
memMonitor := mon.NewMonitor(mon.Options{

pkg/server/server_controller_new_server.go

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -329,17 +329,19 @@ func makeSharedProcessTenantServerConfig(
329329

330330
tempStorageCfg := base.InheritTempStorageConfig(ctx, st, kvServerCfg.SQLConfig.TempStorageConfig)
331331
if !tempStorageCfg.InMemory {
332-
useStore := tempStorageCfg.Spec
333-
// TODO(knz): Make tempDir configurable.
334-
tempDir := useStore.Path
335-
var unlockDirFn func()
336-
if tempStorageCfg.Path, unlockDirFn, err = fs.CreateTempDir(tempDir, TempDirPrefix); err != nil {
332+
// We create another temp directory alongside the existing one.
333+
parentDir := filepath.Dir(tempStorageCfg.Path)
334+
if parentDir == "" {
335+
return BaseConfig{}, SQLConfig{}, errors.Newf("invalid temp storage config path %q", tempStorageCfg.Path)
336+
}
337+
tmpDir, unlockDirFn, err := fs.CreateTempDir(parentDir, TempDirPrefix)
338+
if err != nil {
337339
return BaseConfig{}, SQLConfig{}, errors.Wrap(err, "could not create temporary directory for temp storage")
338340
}
339341
stopper.AddCloser(stop.CloserFn(unlockDirFn))
340-
if useStore.Path != "" {
341-
recordPath := filepath.Join(useStore.Path, TempDirsRecordFilename)
342-
if err := fs.RecordTempDir(recordPath, tempStorageCfg.Path); err != nil {
342+
tempStorageCfg.Path = tmpDir
343+
if tempStorageCfg.TempDirsRecordPath != "" {
344+
if err := fs.RecordTempDir(tempStorageCfg.TempDirsRecordPath, tempStorageCfg.Path); err != nil {
343345
return BaseConfig{}, SQLConfig{}, errors.Wrap(err, "could not record temp dir")
344346
}
345347
}

pkg/server/server_sql.go

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -725,8 +725,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
725725

726726
// Set up the DistSQL temp engine.
727727

728-
useStoreSpec := cfg.TempStorageConfig.Spec
729-
tempEngine, tempFS, err := storage.NewTempEngine(ctx, cfg.TempStorageConfig, useStoreSpec, cfg.DiskWriteStats)
728+
tempEngine, tempFS, err := storage.NewTempEngine(ctx, cfg.TempStorageConfig, cfg.DiskWriteStats)
730729
if err != nil {
731730
return nil, errors.Wrap(err, "creating temp storage")
732731
}

pkg/sql/colflow/draining_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ func TestDrainingAfterRemoteError(t *testing.T) {
4343
diskMonitor.Start(ctx, nil /* pool */, mon.NewStandaloneBudget(1))
4444

4545
// Set up a two node cluster.
46-
tempStorageConfig := base.TempStorageConfig{InMemory: true, Mon: diskMonitor, Settings: st, Spec: base.DefaultTestStoreSpec}
46+
tempStorageConfig := base.TempStorageConfig{InMemory: true, Mon: diskMonitor, Settings: st}
4747
args := base.TestClusterArgs{
4848
ServerArgs: base.TestServerArgs{
4949
Settings: st,

pkg/sql/conn_executor_internal_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -280,7 +280,7 @@ func startConnExecutor(
280280
nodeID := base.TestingIDContainer
281281
distSQLMetrics := execinfra.MakeDistSQLMetrics(time.Hour /* histogramWindow */)
282282
gw := gossip.MakeOptionalGossip(nil)
283-
tempEngine, tempFS, err := storage.NewTempEngine(ctx, base.DefaultTestTempStorageConfig(st), base.DefaultTestStoreSpec, nil /* statsCollector */)
283+
tempEngine, tempFS, err := storage.NewTempEngine(ctx, base.DefaultTestTempStorageConfig(st), nil /* statsCollector */)
284284
if err != nil {
285285
return nil, nil, nil, nil, nil, err
286286
}

pkg/sql/distsql/columnar_utils_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,7 @@ func verifyColOperator(t *testing.T, args verifyColOperatorArgs) error {
8282

8383
ctx := context.Background()
8484
st := cluster.MakeTestingClusterSettings()
85-
tempEngine, tempFS, err := storage.NewTempEngine(ctx, base.DefaultTestTempStorageConfig(st), base.DefaultTestStoreSpec, nil /* statsCollector */)
85+
tempEngine, tempFS, err := storage.NewTempEngine(ctx, base.DefaultTestTempStorageConfig(st), nil /* statsCollector */)
8686
if err != nil {
8787
return err
8888
}

0 commit comments

Comments
 (0)